From ee8c09667e955cbb67e12e2cfa3f6ad7570d2360 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 20 Sep 2023 12:53:58 +0000 Subject: [PATCH 001/195] refactor backend --- source/libs/stream/src/streamMeta.c | 135 ++++++++++++++++++++-------- 1 file changed, 100 insertions(+), 35 deletions(-) diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 652ef7cde7..5c64619ca9 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -103,6 +103,76 @@ int32_t metaRefMgtAdd(int64_t vgId, int64_t* rid) { return 0; } +typedef struct { + int64_t chkpId; + char* path; + char* taskId; + + SArray* pChkpSave; + SArray* pChkpInUse; + int8_t chkpCap; + void* backend; + +} StreamMetaTaskState; + +int32_t streamMetaOpenTdb(SStreamMeta* pMeta) { + if (tdbOpen(pMeta->path, 16 * 1024, 1, &pMeta->db, 0) < 0) { + return -1; + // goto _err; + } + + if (tdbTbOpen("task.db", STREAM_TASK_KEY_LEN, -1, NULL, pMeta->db, &pMeta->pTaskDb, 0) < 0) { + return -1; + } + + if (tdbTbOpen("checkpoint.db", sizeof(int32_t), -1, NULL, pMeta->db, &pMeta->pCheckpointDb, 0) < 0) { + return -1; + } + return 0; +} + +// +// impl later +// +enum STREAM_STATE_VER { + STREAM_STATA_NO_COMPATIBLE, + STREAM_STATA_COMPATIBLE, + STREAM_STATA_NEED_CONVERT, +}; + +int32_t streamMetaCheckStateVer(SStreamMeta* pMeta) { + TBC* pCur = NULL; + if (tdbTbcOpen(pMeta->pTaskDb, &pCur, NULL) < 0) { + // no task info, no stream + return STREAM_STATA_COMPATIBLE; + } + + void* pKey = NULL; + int32_t kLen = 0; + void* pVal = NULL; + int32_t vLen = 0; + SDecoder decoder; + + tdbTbcMoveToFirst(pCur); + while (tdbTbcNext(pCur, &pKey, &kLen, &pVal, &vLen) == 0) { + if (pVal == NULL || vLen == 0) { + break; + } + SCheckpointInfo info; + tDecoderInit(&decoder, (uint8_t*)pVal, vLen); + if (tDecodeStreamTaskChkInfo(&decoder, &info) < 0) { + continue; + } + tDecoderClear(&decoder); + } + tdbFree(pKey); + tdbFree(pVal); + tdbTbcClose(pCur); + + return STREAM_STATA_NEED_CONVERT; +} + +int32_t streamMetaDoStateDataConvert(SStreamMeta* pMeta) { return 0; } SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandFunc, int32_t vgId, int64_t stage) { int32_t code = -1; SStreamMeta* pMeta = taosMemoryCalloc(1, sizeof(SStreamMeta)); @@ -118,15 +188,11 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF sprintf(tpath, "%s%s%s", path, TD_DIRSEP, "stream"); pMeta->path = tpath; - if (tdbOpen(pMeta->path, 16 * 1024, 1, &pMeta->db, 0) < 0) { + if (streamMetaOpenTdb(pMeta) < 0) { goto _err; } - if (tdbTbOpen("task.db", STREAM_TASK_KEY_LEN, -1, NULL, pMeta->db, &pMeta->pTaskDb, 0) < 0) { - goto _err; - } - - if (tdbTbOpen("checkpoint.db", sizeof(int32_t), -1, NULL, pMeta->db, &pMeta->pCheckpointDb, 0) < 0) { + if (streamMetaDoStateDataConvert(pMeta) < 0) { goto _err; } @@ -134,8 +200,7 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF goto _err; } - _hash_fn_t fp = taosGetDefaultHashFunction(TSDB_DATA_TYPE_VARCHAR); - pMeta->pTasks = taosHashInit(64, fp, true, HASH_NO_LOCK); + pMeta->pTasks = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_VARCHAR), true, HASH_NO_LOCK); if (pMeta->pTasks == NULL) { goto _err; } @@ -153,41 +218,43 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF pMeta->expandFunc = expandFunc; pMeta->stage = stage; - // send heartbeat every 5sec. - pMeta->rid = taosAddRef(streamMetaId, pMeta); int64_t* pRid = taosMemoryMalloc(sizeof(int64_t)); + pMeta->rid = taosAddRef(streamMetaId, pMeta); *pRid = pMeta->rid; metaRefMgtAdd(pMeta->vgId, pRid); + // send heartbeat every 5sec. + // TODO: start hb later pMeta->hbInfo.hbTmr = taosTmrStart(metaHbToMnode, META_HB_CHECK_INTERVAL, pRid, streamEnv.timer); pMeta->hbInfo.tickCounter = 0; pMeta->hbInfo.stopFlag = 0; - pMeta->pTaskBackendUnique = - taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), false, HASH_ENTRY_LOCK); - pMeta->chkpSaved = taosArrayInit(4, sizeof(int64_t)); - pMeta->chkpInUse = taosArrayInit(4, sizeof(int64_t)); - pMeta->chkpCap = 8; - taosInitRWLatch(&pMeta->chkpDirLock); + // start backend + // taosInitRWLatch(&pMeta->chkpDirLock); - pMeta->chkpId = streamGetLatestCheckpointId(pMeta); - pMeta->streamBackend = streamBackendInit(pMeta->path, pMeta->chkpId); - while (pMeta->streamBackend == NULL) { - taosMsleep(2 * 1000); - pMeta->streamBackend = streamBackendInit(pMeta->path, pMeta->chkpId); - if (pMeta->streamBackend == NULL) { - qError("vgId:%d failed to init stream backend", pMeta->vgId); - qInfo("vgId:%d retry to init stream backend", pMeta->vgId); - } - } - pMeta->streamBackendRid = taosAddRef(streamBackendId, pMeta->streamBackend); + // pMeta->pTaskBackendUnique = + // taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), false, HASH_ENTRY_LOCK); + // pMeta->chkpSaved = taosArrayInit(4, sizeof(int64_t)); + // pMeta->chkpInUse = taosArrayInit(4, sizeof(int64_t)); + // pMeta->chkpCap = 8; - code = streamBackendLoadCheckpointInfo(pMeta); + // pMeta->chkpId = streamGetLatestCheckpointId(pMeta); + // pMeta->streamBackend = streamBackendInit(pMeta->path, pMeta->chkpId); + // while (pMeta->streamBackend == NULL) { + // qError("vgId:%d failed to init stream backend", pMeta->vgId); + // taosMsleep(2 * 1000); + // qInfo("vgId:%d retry to init stream backend", pMeta->vgId); + // pMeta->streamBackend = streamBackendInit(pMeta->path, pMeta->chkpId); + // if (pMeta->streamBackend == NULL) { + // } + // } + // pMeta->streamBackendRid = taosAddRef(streamBackendId, pMeta->streamBackend); + + // code = streamBackendLoadCheckpointInfo(pMeta); + // taosThreadMutexInit(&pMeta->backendMutex, NULL); taosInitRWLatch(&pMeta->lock); - taosThreadMutexInit(&pMeta->backendMutex, NULL); - pMeta->pauseTaskNum = 0; qInfo("vgId:%d open stream meta successfully, latest checkpoint:%" PRId64 ", stage:%" PRId64, vgId, pMeta->chkpId, @@ -241,13 +308,11 @@ int32_t streamMetaReopen(SStreamMeta* pMeta, int64_t chkpId) { pMeta->streamBackend = streamBackendInit(pMeta->path, pMeta->chkpId); while (pMeta->streamBackend == NULL) { + qError("vgId:%d failed to init stream backend", pMeta->vgId); taosMsleep(2 * 1000); + + qInfo("vgId:%d retry to init stream backend", pMeta->vgId); pMeta->streamBackend = streamBackendInit(pMeta->path, pMeta->chkpId); - if (pMeta->streamBackend == NULL) { - qError("vgId:%d failed to init stream backend", pMeta->vgId); - qInfo("vgId:%d retry to init stream backend", pMeta->vgId); - // return -1; - } } pMeta->streamBackendRid = taosAddRef(streamBackendId, pMeta->streamBackend); streamBackendLoadCheckpointInfo(pMeta); From 470849982e26eac46db1b7d714b60cb9589f244b Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 21 Sep 2023 09:41:30 +0000 Subject: [PATCH 002/195] add cvt func --- include/libs/stream/tstream.h | 32 ++-- source/libs/stream/inc/streamBackendRocksdb.h | 24 +++ source/libs/stream/src/streamBackendRocksdb.c | 178 +++++++++++++++++- source/libs/stream/src/streamMeta.c | 63 +++++-- source/libs/stream/src/streamTask.c | 5 +- 5 files changed, 270 insertions(+), 32 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 3eb624f932..29dc054d78 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -31,7 +31,10 @@ extern "C" { typedef struct SStreamTask SStreamTask; -#define SSTREAM_TASK_VER 2 +#define SSTREAM_TASK_VER 3 +#define SSTREAM_TASK_INCOMPATIBLE_VER 1 +#define SSTREAM_TASK_NEED_CONVERT_VER 2 + enum { STREAM_STATUS__NORMAL = 0, STREAM_STATUS__STOP, @@ -117,7 +120,7 @@ typedef struct { typedef struct { int8_t type; int64_t ver; - SArray* submits; // SArray + SArray* submits; // SArray } SStreamMergedSubmit; typedef struct { @@ -257,8 +260,9 @@ typedef struct SStreamTaskId { typedef struct SCheckpointInfo { int64_t checkpointId; - int64_t checkpointVer; // latest checkpointId version - int64_t nextProcessVer; // current offset in WAL, not serialize it + int64_t checkpointVer; // latest checkpointId version + int64_t nextProcessVer; // current offset in WAL, not serialize it + int64_t msgVer; } SCheckpointInfo; typedef struct SStreamStatus { @@ -283,8 +287,8 @@ typedef struct SSTaskBasicInfo { int32_t selfChildId; int32_t totalLevel; int8_t taskLevel; - int8_t fillHistory; // is fill history task or not - int64_t triggerParam; // in msec + int8_t fillHistory; // is fill history task or not + int64_t triggerParam; // in msec } SSTaskBasicInfo; typedef struct SDispatchMsgInfo { @@ -301,13 +305,13 @@ typedef struct STaskOutputInfo { } STaskOutputInfo; typedef struct STaskInputInfo { - int8_t status; + int8_t status; SStreamQueue* queue; } STaskInputInfo; typedef struct STaskSchedInfo { - int8_t status; - void* pTimer; + int8_t status; + void* pTimer; } STaskSchedInfo; typedef struct SSinkTaskRecorder { @@ -325,10 +329,10 @@ typedef struct { } STaskTimestamp; typedef struct STokenBucket { - int32_t capacity; // total capacity - int64_t fillTimestamp;// fill timestamp - int32_t numOfToken; // total available tokens - int32_t rate; // number of token per second + int32_t capacity; // total capacity + int64_t fillTimestamp; // fill timestamp + int32_t numOfToken; // total available tokens + int32_t rate; // number of token per second } STokenBucket; struct SStreamTask { @@ -650,7 +654,7 @@ bool streamTaskShouldStop(const SStreamStatus* pStatus); bool streamTaskShouldPause(const SStreamStatus* pStatus); bool streamTaskIsIdle(const SStreamTask* pTask); -void initRpcMsg(SRpcMsg* pMsg, int32_t msgType, void* pCont, int32_t contLen); +void initRpcMsg(SRpcMsg* pMsg, int32_t msgType, void* pCont, int32_t contLen); char* createStreamTaskIdStr(int64_t streamId, int32_t taskId); diff --git a/source/libs/stream/inc/streamBackendRocksdb.h b/source/libs/stream/inc/streamBackendRocksdb.h index 39854d1824..5bcc29287d 100644 --- a/source/libs/stream/inc/streamBackendRocksdb.h +++ b/source/libs/stream/inc/streamBackendRocksdb.h @@ -42,8 +42,31 @@ typedef struct { TdThreadMutex cfMutex; SHashObj* cfInst; int64_t defaultCfInit; + } SBackendWrapper; +typedef struct { + void* tableOpt; +} RocksdbCfParam; + +typedef struct { + rocksdb_t* db; + rocksdb_writeoptions_t* writeOpts; + rocksdb_readoptions_t* readOpts; + rocksdb_options_t* dbOpt; + rocksdb_env_t* env; + rocksdb_cache_t* cache; + + rocksdb_column_family_handle_t** pCf; + rocksdb_comparator_t** pCompares; + rocksdb_options_t** pCfOpts; + RocksdbCfParam* pCfParams; + + rocksdb_compactionfilterfactory_t* filterFactory; + TdThreadMutex cfMutex; + +} STaskBackendWrapper; + void* streamBackendInit(const char* path, int64_t chkpId); void streamBackendCleanup(void* arg); void streamBackendHandleCleanup(void* arg); @@ -51,6 +74,7 @@ int32_t streamBackendLoadCheckpointInfo(void* pMeta); int32_t streamBackendDoCheckpoint(void* pMeta, uint64_t checkpointId); SListNode* streamBackendAddCompare(void* backend, void* arg); void streamBackendDelCompare(void* backend, void* arg); +int32_t streamStateConvertDataFormat(char* path, char* key, void* cfInst); int streamStateOpenBackend(void* backend, SStreamState* pState); void streamStateCloseBackend(SStreamState* pState, bool remove); diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 19f4ebbeea..7feb029e81 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -45,9 +45,6 @@ typedef struct SCompactFilteFactory { void* status; } SCompactFilteFactory; -typedef struct { - void* tableOpt; -} RocksdbCfParam; typedef struct { rocksdb_t* db; rocksdb_column_family_handle_t** pHandle; @@ -1455,6 +1452,181 @@ void destroyRocksdbCfInst(RocksdbCfInst* inst) { taosMemoryFree(inst); } +STaskBackendWrapper* streamStateOpenTaskBackend(char* path, char* key) { + int32_t code = 0; + + char* taskPath = taosMemoryCalloc(1, strlen(path) + 128); + sprintf(taskPath, "%s%s%s%s%s", path, TD_DIRSEP, "state", TD_DIRSEP, key); + + STaskBackendWrapper* pTaskBackend = taosMemoryCalloc(1, sizeof(SBackendWrapper)); + rocksdb_env_t* env = rocksdb_create_default_env(); + + rocksdb_cache_t* cache = rocksdb_cache_create_lru(256); + rocksdb_options_t* opts = rocksdb_options_create(); + // rocksdb_options_set_env(opts, env); + rocksdb_options_set_create_if_missing(opts, 1); + rocksdb_options_set_create_missing_column_families(opts, 1); + // rocksdb_options_set_max_total_wal_size(opts, dbMemLimit); + rocksdb_options_set_recycle_log_file_num(opts, 6); + rocksdb_options_set_max_write_buffer_number(opts, 3); + rocksdb_options_set_info_log_level(opts, 1); + // rocksdb_options_set_db_write_buffer_size(opts, dbMemLimit); + // rocksdb_options_set_write_buffer_size(opts, dbMemLimit / 2); + rocksdb_options_set_atomic_flush(opts, 1); + + pTaskBackend->env = env; + pTaskBackend->dbOpt = opts; + pTaskBackend->cache = cache; + pTaskBackend->filterFactory = rocksdb_compactionfilterfactory_create( + NULL, destroyCompactFilteFactory, compactFilteFactoryCreateFilter, compactFilteFactoryName); + rocksdb_options_set_compaction_filter_factory(pTaskBackend->dbOpt, pTaskBackend->filterFactory); + + char* err = NULL; + size_t nCf = 0; + + char** cfs = rocksdb_list_column_families(opts, taskPath, &nCf, &err); + if (nCf == 0 || nCf == 1 || err != NULL) { + taosMemoryFreeClear(err); + pTaskBackend->db = rocksdb_open(opts, taskPath, &err); + if (err != NULL) { + qError("failed to open rocksdb, path:%s, reason:%s", taskPath, err); + taosMemoryFreeClear(err); + } + } else { + ASSERT(0); + } + if (cfs != NULL) { + rocksdb_list_column_families_destroy(cfs, nCf); + } + qDebug("succ to init stream backend at %s, backend:%p", taskPath, pTaskBackend); + + nCf = sizeof(ginitDict) / sizeof(ginitDict[0]); + pTaskBackend->pCf = taosMemoryCalloc(nCf, sizeof(rocksdb_column_family_handle_t*)); + pTaskBackend->pCfParams = taosMemoryCalloc(nCf, sizeof(RocksdbCfParam)); + pTaskBackend->pCfOpts = taosMemoryCalloc(nCf, sizeof(rocksdb_options_t*)); + pTaskBackend->pCompares = taosMemoryCalloc(nCf, sizeof(rocksdb_comparator_t*)); + + for (int i = 0; i < nCf; i++) { + rocksdb_options_t* opt = rocksdb_options_create_copy(pTaskBackend->dbOpt); + rocksdb_block_based_table_options_t* tableOpt = rocksdb_block_based_options_create(); + rocksdb_block_based_options_set_block_cache(tableOpt, pTaskBackend->cache); + rocksdb_block_based_options_set_partition_filters(tableOpt, 1); + + rocksdb_filterpolicy_t* filter = rocksdb_filterpolicy_create_bloom(15); + rocksdb_block_based_options_set_filter_policy(tableOpt, filter); + + rocksdb_options_set_block_based_table_factory((rocksdb_options_t*)opt, tableOpt); + + SCfInit* cfPara = &ginitDict[i]; + + rocksdb_comparator_t* compare = + rocksdb_comparator_create(NULL, cfPara->detroyFunc, cfPara->cmpFunc, cfPara->cmpName); + rocksdb_options_set_comparator((rocksdb_options_t*)opt, compare); + + pTaskBackend->pCompares[i] = compare; + pTaskBackend->pCfOpts[i] = opt; + pTaskBackend->pCfParams[i].tableOpt = tableOpt; + } + + return pTaskBackend; +} + +int8_t getCfIdx(const char* key) { + int idx = -1; + size_t len = strlen(key); + for (int i = 0; i < sizeof(ginitDict) / sizeof(ginitDict[0]); i++) { + if (len == ginitDict[i].len && strncmp(key, ginitDict[i].key, strlen(key)) == 0) { + idx = i; + break; + } + } + return idx; +} +rocksdb_column_family_handle_t* taskBackendOpenCf(STaskBackendWrapper* pBackend, const char* key) { + char* err = NULL; + int8_t idx = getCfIdx(key); + if (idx == -1) return NULL; + + rocksdb_column_family_handle_t* cf = + rocksdb_create_column_family(pBackend->db, pBackend->pCfOpts[idx], ginitDict[idx].key, &err); + if (err != NULL) { + qError("failed to open cf, key:%s, reason: %s", key, err); + taosMemoryFree(err); + return NULL; + } + return cf; +} +int32_t copyData(RocksdbCfInst* pSrc, STaskBackendWrapper* pDst, int8_t i) { + int32_t WRITE_BATCH = 1024; + char* err = NULL; + int code = 0; + + rocksdb_readoptions_t* pRdOpt = rocksdb_readoptions_create(); + + rocksdb_writebatch_t* wb = rocksdb_writebatch_create(); + rocksdb_iterator_t* pIter = rocksdb_create_iterator_cf(pSrc->db, pRdOpt, pSrc->pHandle[i]); + rocksdb_iter_seek_to_first(pIter); + while (rocksdb_iter_valid(pIter)) { + if (rocksdb_writebatch_count(wb) >= WRITE_BATCH) { + rocksdb_write(pDst->db, pDst->writeOpts, wb, &err); + if (err != NULL) { + code = -1; + goto _EXIT; + } + rocksdb_writebatch_clear(wb); + } + + size_t klen = 0, vlen = 0; + char* key = (char*)rocksdb_iter_key(pIter, &klen); + char* val = (char*)rocksdb_iter_value(pIter, &vlen); + + rocksdb_writebatch_put_cf(wb, pDst->pCf[i], key, klen, val, vlen); + rocksdb_iter_next(pIter); + } + + if (rocksdb_writebatch_count(wb) > 0) { + rocksdb_write(pDst->db, pDst->writeOpts, wb, &err); + if (err != NULL) { + code = -1; + goto _EXIT; + } + } + +_EXIT: + rocksdb_iter_destroy(pIter); + rocksdb_readoptions_destroy(pRdOpt); + taosMemoryFree(err); + + return code; +} + +int32_t streamStateConvertDataFormat(char* path, char* key, void* cfInst) { + int nCf = sizeof(ginitDict) / sizeof(ginitDict[0]); + + int32_t code = 0; + RocksdbCfInst* CfInst = cfInst; + + STaskBackendWrapper* pTaskBackend = streamStateOpenTaskBackend(path, key); + RocksdbCfInst* pSrcBackend = cfInst; + + for (int i = 0; i < nCf; i++) { + rocksdb_column_family_handle_t* pSrcCf = pSrcBackend->pHandle[i]; + if (pSrcCf == NULL) continue; + + rocksdb_column_family_handle_t* pDstCf = taskBackendOpenCf(pTaskBackend, ginitDict[i].key); + if (pDstCf == NULL) { + return -1; + } + + code = copyData(pSrcBackend, pTaskBackend, i); + if (code != 0) { + return -1; + } + + pTaskBackend->pCf[i] = pDstCf; + } + return 0; +} int32_t streamStateOpenBackendCf(void* backend, char* name, char** cfs, int32_t nCf) { SBackendWrapper* handle = backend; char* err = NULL; diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 5c64619ca9..f47961d026 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -140,39 +140,78 @@ enum STREAM_STATE_VER { STREAM_STATA_NEED_CONVERT, }; -int32_t streamMetaCheckStateVer(SStreamMeta* pMeta) { - TBC* pCur = NULL; +int32_t streamMetaCheckStateCompatible(SStreamMeta* pMeta) { + int8_t ret = STREAM_STATA_COMPATIBLE; + TBC* pCur = NULL; + if (tdbTbcOpen(pMeta->pTaskDb, &pCur, NULL) < 0) { // no task info, no stream - return STREAM_STATA_COMPATIBLE; + return ret; } - - void* pKey = NULL; - int32_t kLen = 0; - void* pVal = NULL; - int32_t vLen = 0; - SDecoder decoder; + void* pKey = NULL; + int32_t kLen = 0; + void* pVal = NULL; + int32_t vLen = 0; tdbTbcMoveToFirst(pCur); while (tdbTbcNext(pCur, &pKey, &kLen, &pVal, &vLen) == 0) { if (pVal == NULL || vLen == 0) { break; } + SDecoder decoder; SCheckpointInfo info; tDecoderInit(&decoder, (uint8_t*)pVal, vLen); if (tDecodeStreamTaskChkInfo(&decoder, &info) < 0) { continue; } + if (info.msgVer <= SSTREAM_TASK_INCOMPATIBLE_VER) { + ret = STREAM_STATA_NO_COMPATIBLE; + } else if (info.msgVer == SSTREAM_TASK_NEED_CONVERT_VER) { + ret = STREAM_STATA_NEED_CONVERT; + } else if (info.msgVer == SSTREAM_TASK_VER) { + ret = STREAM_STATA_COMPATIBLE; + } tDecoderClear(&decoder); + break; } tdbFree(pKey); tdbFree(pVal); tdbTbcClose(pCur); - - return STREAM_STATA_NEED_CONVERT; + return ret; } -int32_t streamMetaDoStateDataConvert(SStreamMeta* pMeta) { return 0; } +int32_t streamMetaDoStateDataConvertImpl(SStreamMeta* pMeta) { + int64_t chkpId = streamGetLatestCheckpointId(pMeta); + SBackendWrapper* pBackend = streamBackendInit(pMeta->path, chkpId); + + void* pIter = taosHashIterate(pBackend->cfInst, NULL); + while (pIter) { + size_t len = 0; + void* key = taosHashGetKey(pIter, &len); + + pIter = taosHashIterate(pBackend->cfInst, pIter); + } + + // streamBackendCleanup(); + + return 0; +} +int32_t streamMetaDoStateDataConvert(SStreamMeta* pMeta) { + int8_t compatible = streamMetaCheckStateCompatible(pMeta); + if (compatible == STREAM_STATA_COMPATIBLE) { + return 0; + } else if (compatible == STREAM_STATA_NO_COMPATIBLE) { + qError( + "stream read incompatible data, rm %s/vnode/vnode*/tq/stream if taosd cannot start, and rebuild stream " + "manually", + tsDataDir); + return -1; + } else if (compatible == STREAM_STATA_NEED_CONVERT) { + qError("stream state need covert backend format"); + return streamMetaDoStateDataConvertImpl(pMeta); + } + return 0; +} SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandFunc, int32_t vgId, int64_t stage) { int32_t code = -1; SStreamMeta* pMeta = taosMemoryCalloc(1, sizeof(SStreamMeta)); diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 663deca171..837f787dd2 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -226,9 +226,8 @@ int32_t tDecodeStreamTaskChkInfo(SDecoder* pDecoder, SCheckpointInfo* pChkpInfo) SEpSet epSet; if (tStartDecode(pDecoder) < 0) return -1; - if (tDecodeI64(pDecoder, &ver) < 0) return -1; - - if (ver != SSTREAM_TASK_VER) return -1; + if (tDecodeI64(pDecoder, &pChkpInfo->msgVer) < 0) return -1; + // if (ver != SSTREAM_TASK_VER) return -1; if (tDecodeI64(pDecoder, &skip64) < 0) return -1; if (tDecodeI32(pDecoder, &skip32) < 0) return -1; From 19e52c364e8ecfc496686ea743fd69a2dc512709 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Fri, 22 Sep 2023 08:26:03 +0000 Subject: [PATCH 003/195] add cvt state --- source/libs/stream/inc/streamBackendRocksdb.h | 23 +-- source/libs/stream/src/streamBackendRocksdb.c | 133 ++++++++++++------ source/libs/stream/src/streamMeta.c | 20 ++- 3 files changed, 117 insertions(+), 59 deletions(-) diff --git a/source/libs/stream/inc/streamBackendRocksdb.h b/source/libs/stream/inc/streamBackendRocksdb.h index 5bcc29287d..3e07f2532e 100644 --- a/source/libs/stream/inc/streamBackendRocksdb.h +++ b/source/libs/stream/inc/streamBackendRocksdb.h @@ -51,8 +51,8 @@ typedef struct { typedef struct { rocksdb_t* db; - rocksdb_writeoptions_t* writeOpts; - rocksdb_readoptions_t* readOpts; + rocksdb_writeoptions_t* writeOpt; + rocksdb_readoptions_t* readOpt; rocksdb_options_t* dbOpt; rocksdb_env_t* env; rocksdb_cache_t* cache; @@ -63,18 +63,19 @@ typedef struct { RocksdbCfParam* pCfParams; rocksdb_compactionfilterfactory_t* filterFactory; - TdThreadMutex cfMutex; + TdThreadMutex mutex; } STaskBackendWrapper; -void* streamBackendInit(const char* path, int64_t chkpId); -void streamBackendCleanup(void* arg); -void streamBackendHandleCleanup(void* arg); -int32_t streamBackendLoadCheckpointInfo(void* pMeta); -int32_t streamBackendDoCheckpoint(void* pMeta, uint64_t checkpointId); -SListNode* streamBackendAddCompare(void* backend, void* arg); -void streamBackendDelCompare(void* backend, void* arg); -int32_t streamStateConvertDataFormat(char* path, char* key, void* cfInst); +void* streamBackendInit(const char* path, int64_t chkpId); +void streamBackendCleanup(void* arg); +void streamBackendHandleCleanup(void* arg); +int32_t streamBackendLoadCheckpointInfo(void* pMeta); +int32_t streamBackendDoCheckpoint(void* pMeta, uint64_t checkpointId); +SListNode* streamBackendAddCompare(void* backend, void* arg); +void streamBackendDelCompare(void* backend, void* arg); +int32_t streamStateConvertDataFormat(char* path, char* key, void* cfInst); +STaskBackendWrapper* streamStateOpenTaskBackend(char* path, char* key); int streamStateOpenBackend(void* backend, SStreamState* pState); void streamStateCloseBackend(SStreamState* pState, bool remove); diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 7feb029e81..ff53ec6c51 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1457,6 +1457,15 @@ STaskBackendWrapper* streamStateOpenTaskBackend(char* path, char* key) { char* taskPath = taosMemoryCalloc(1, strlen(path) + 128); sprintf(taskPath, "%s%s%s%s%s", path, TD_DIRSEP, "state", TD_DIRSEP, key); + if (!taosDirExist(taskPath)) { + code = taosMkDir(taskPath); + if (code != 0) { + qError("failed to create dir: %s, reason:%s", taskPath, tstrerror(code)); + taosMemoryFree(taskPath); + return NULL; + } + } + char* err = NULL; STaskBackendWrapper* pTaskBackend = taosMemoryCalloc(1, sizeof(SBackendWrapper)); rocksdb_env_t* env = rocksdb_create_default_env(); @@ -1474,33 +1483,16 @@ STaskBackendWrapper* streamStateOpenTaskBackend(char* path, char* key) { // rocksdb_options_set_write_buffer_size(opts, dbMemLimit / 2); rocksdb_options_set_atomic_flush(opts, 1); - pTaskBackend->env = env; pTaskBackend->dbOpt = opts; + pTaskBackend->env = env; pTaskBackend->cache = cache; pTaskBackend->filterFactory = rocksdb_compactionfilterfactory_create( NULL, destroyCompactFilteFactory, compactFilteFactoryCreateFilter, compactFilteFactoryName); rocksdb_options_set_compaction_filter_factory(pTaskBackend->dbOpt, pTaskBackend->filterFactory); + pTaskBackend->readOpt = rocksdb_readoptions_create(); + pTaskBackend->writeOpt = rocksdb_writeoptions_create(); - char* err = NULL; - size_t nCf = 0; - - char** cfs = rocksdb_list_column_families(opts, taskPath, &nCf, &err); - if (nCf == 0 || nCf == 1 || err != NULL) { - taosMemoryFreeClear(err); - pTaskBackend->db = rocksdb_open(opts, taskPath, &err); - if (err != NULL) { - qError("failed to open rocksdb, path:%s, reason:%s", taskPath, err); - taosMemoryFreeClear(err); - } - } else { - ASSERT(0); - } - if (cfs != NULL) { - rocksdb_list_column_families_destroy(cfs, nCf); - } - qDebug("succ to init stream backend at %s, backend:%p", taskPath, pTaskBackend); - - nCf = sizeof(ginitDict) / sizeof(ginitDict[0]); + size_t nCf = sizeof(ginitDict) / sizeof(ginitDict[0]); pTaskBackend->pCf = taosMemoryCalloc(nCf, sizeof(rocksdb_column_family_handle_t*)); pTaskBackend->pCfParams = taosMemoryCalloc(nCf, sizeof(RocksdbCfParam)); pTaskBackend->pCfOpts = taosMemoryCalloc(nCf, sizeof(rocksdb_options_t*)); @@ -1528,8 +1520,65 @@ STaskBackendWrapper* streamStateOpenTaskBackend(char* path, char* key) { pTaskBackend->pCfParams[i].tableOpt = tableOpt; } + char** cfs = rocksdb_list_column_families(opts, taskPath, &nCf, &err); + if (nCf == 0 || nCf == 1 || err != NULL) { + taosMemoryFreeClear(err); + pTaskBackend->db = rocksdb_open(opts, taskPath, &err); + if (err != NULL) { + qError("failed to open rocksdb, path:%s, reason:%s", taskPath, err); + taosMemoryFreeClear(err); + } + } else { + } + if (cfs != NULL) { + rocksdb_list_column_families_destroy(cfs, nCf); + } + + taosThreadMutexInit(&pTaskBackend->mutex, NULL); + taosMemoryFree(taskPath); + + qDebug("succ to init stream backend at %s, backend:%p", taskPath, pTaskBackend); return pTaskBackend; } +void streamMeteCloseTaskBackend(STaskBackendWrapper* wrapper) { + if (wrapper == NULL) return; + + rocksdb_flushoptions_t* flushOpt = rocksdb_flushoptions_create(); + rocksdb_flushoptions_set_wait(flushOpt, 1); + + int8_t nCf = sizeof(ginitDict) / sizeof(ginitDict[0]); + char* err = NULL; + for (int i = 0; i < nCf; i++) { + if (wrapper->pCf[i] != NULL) rocksdb_flush_cf(wrapper->db, flushOpt, wrapper->pCf[i], &err); + if (err != NULL) { + qError("failed to flush cf:%s, reason:%s", ginitDict[i].key, err); + taosMemoryFreeClear(err); + } + rocksdb_flushoptions_destroy(flushOpt); + } + for (int i = 0; i < nCf; i++) { + if (wrapper->pCf[i] != NULL) { + rocksdb_column_family_handle_destroy(wrapper->pCf[i]); + } + } + rocksdb_options_destroy(wrapper->dbOpt); + rocksdb_readoptions_destroy(wrapper->readOpt); + rocksdb_writeoptions_destroy(wrapper->writeOpt); + rocksdb_env_destroy(wrapper->env); + rocksdb_cache_destroy(wrapper->cache); + + taosMemoryFree(wrapper->pCf); + taosMemoryFree(wrapper->pCompares); + taosMemoryFree(wrapper->pCfOpts); + taosMemoryFree(wrapper->pCfParams); + + taosThreadMutexDestroy(&wrapper->mutex); + + rocksdb_close(wrapper->db); + taosMemoryFree(wrapper); + + return; +} int8_t getCfIdx(const char* key) { int idx = -1; @@ -1542,21 +1591,25 @@ int8_t getCfIdx(const char* key) { } return idx; } -rocksdb_column_family_handle_t* taskBackendOpenCf(STaskBackendWrapper* pBackend, const char* key) { - char* err = NULL; - int8_t idx = getCfIdx(key); - if (idx == -1) return NULL; +int32_t taskBackendOpenCf(STaskBackendWrapper* pBackend, const char* key) { + int32_t code = 0; + char* err = NULL; + int8_t idx = getCfIdx(key); + if (idx == -1) return -1; rocksdb_column_family_handle_t* cf = rocksdb_create_column_family(pBackend->db, pBackend->pCfOpts[idx], ginitDict[idx].key, &err); if (err != NULL) { qError("failed to open cf, key:%s, reason: %s", key, err); taosMemoryFree(err); - return NULL; + code = -1; + return code; } - return cf; + + pBackend->pCf[idx] = cf; + return code; } -int32_t copyData(RocksdbCfInst* pSrc, STaskBackendWrapper* pDst, int8_t i) { +int32_t copyDataAt(RocksdbCfInst* pSrc, STaskBackendWrapper* pDst, int8_t i) { int32_t WRITE_BATCH = 1024; char* err = NULL; int code = 0; @@ -1568,7 +1621,7 @@ int32_t copyData(RocksdbCfInst* pSrc, STaskBackendWrapper* pDst, int8_t i) { rocksdb_iter_seek_to_first(pIter); while (rocksdb_iter_valid(pIter)) { if (rocksdb_writebatch_count(wb) >= WRITE_BATCH) { - rocksdb_write(pDst->db, pDst->writeOpts, wb, &err); + rocksdb_write(pDst->db, pDst->writeOpt, wb, &err); if (err != NULL) { code = -1; goto _EXIT; @@ -1585,7 +1638,7 @@ int32_t copyData(RocksdbCfInst* pSrc, STaskBackendWrapper* pDst, int8_t i) { } if (rocksdb_writebatch_count(wb) > 0) { - rocksdb_write(pDst->db, pDst->writeOpts, wb, &err); + rocksdb_write(pDst->db, pDst->writeOpt, wb, &err); if (err != NULL) { code = -1; goto _EXIT; @@ -1613,19 +1666,17 @@ int32_t streamStateConvertDataFormat(char* path, char* key, void* cfInst) { rocksdb_column_family_handle_t* pSrcCf = pSrcBackend->pHandle[i]; if (pSrcCf == NULL) continue; - rocksdb_column_family_handle_t* pDstCf = taskBackendOpenCf(pTaskBackend, ginitDict[i].key); - if (pDstCf == NULL) { - return -1; - } + code = taskBackendOpenCf(pTaskBackend, ginitDict[i].key); + if (code != 0) goto _EXIT; - code = copyData(pSrcBackend, pTaskBackend, i); - if (code != 0) { - return -1; - } - - pTaskBackend->pCf[i] = pDstCf; + code = copyDataAt(pSrcBackend, pTaskBackend, i); + if (code != 0) goto _EXIT; } - return 0; + +_EXIT: + streamMeteCloseTaskBackend(pTaskBackend); + + return code; } int32_t streamStateOpenBackendCf(void* backend, char* name, char** cfs, int32_t nCf) { SBackendWrapper* handle = backend; diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index f47961d026..d1f59d0a8f 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -181,17 +181,20 @@ int32_t streamMetaCheckStateCompatible(SStreamMeta* pMeta) { } int32_t streamMetaDoStateDataConvertImpl(SStreamMeta* pMeta) { + int32_t code = 0; int64_t chkpId = streamGetLatestCheckpointId(pMeta); SBackendWrapper* pBackend = streamBackendInit(pMeta->path, chkpId); void* pIter = taosHashIterate(pBackend->cfInst, NULL); while (pIter) { - size_t len = 0; - void* key = taosHashGetKey(pIter, &len); - + void* key = taosHashGetKey(pIter, NULL); + code = streamStateConvertDataFormat(pMeta->path, key, *(void**)pIter); + if (code != 0) { + // continue + } + pIter = taosHashIterate(pBackend->cfInst, pIter); } - // streamBackendCleanup(); return 0; @@ -200,18 +203,21 @@ int32_t streamMetaDoStateDataConvert(SStreamMeta* pMeta) { int8_t compatible = streamMetaCheckStateCompatible(pMeta); if (compatible == STREAM_STATA_COMPATIBLE) { return 0; + } else if (compatible == STREAM_STATA_NEED_CONVERT) { + qInfo("stream state need covert backend format"); + + return streamMetaDoStateDataConvertImpl(pMeta); } else if (compatible == STREAM_STATA_NO_COMPATIBLE) { qError( "stream read incompatible data, rm %s/vnode/vnode*/tq/stream if taosd cannot start, and rebuild stream " "manually", tsDataDir); + return -1; - } else if (compatible == STREAM_STATA_NEED_CONVERT) { - qError("stream state need covert backend format"); - return streamMetaDoStateDataConvertImpl(pMeta); } return 0; } + SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandFunc, int32_t vgId, int64_t stage) { int32_t code = -1; SStreamMeta* pMeta = taosMemoryCalloc(1, sizeof(SStreamMeta)); From 173604ee1afcbdca4fc32124a84418c499a4d8d9 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Fri, 22 Sep 2023 08:32:14 +0000 Subject: [PATCH 004/195] add cvt state --- source/libs/stream/src/streamBackendRocksdb.c | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index ff53ec6c51..5df33949ef 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1653,14 +1653,13 @@ _EXIT: return code; } -int32_t streamStateConvertDataFormat(char* path, char* key, void* cfInst) { +int32_t streamStateConvertDataFormat(char* path, char* key, void* pCfInst) { int nCf = sizeof(ginitDict) / sizeof(ginitDict[0]); - int32_t code = 0; - RocksdbCfInst* CfInst = cfInst; + int32_t code = 0; STaskBackendWrapper* pTaskBackend = streamStateOpenTaskBackend(path, key); - RocksdbCfInst* pSrcBackend = cfInst; + RocksdbCfInst* pSrcBackend = pCfInst; for (int i = 0; i < nCf; i++) { rocksdb_column_family_handle_t* pSrcCf = pSrcBackend->pHandle[i]; From 970b1cb84065ed9bb240d97267b0617b749f496d Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Fri, 22 Sep 2023 08:38:41 +0000 Subject: [PATCH 005/195] add cvt state --- source/libs/stream/src/streamMeta.c | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index d1f59d0a8f..aa29ea8159 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -180,7 +180,7 @@ int32_t streamMetaCheckStateCompatible(SStreamMeta* pMeta) { return ret; } -int32_t streamMetaDoStateDataConvertImpl(SStreamMeta* pMeta) { +int32_t streamMetaDoStateBackendConvertImpl(SStreamMeta* pMeta) { int32_t code = 0; int64_t chkpId = streamGetLatestCheckpointId(pMeta); SBackendWrapper* pBackend = streamBackendInit(pMeta->path, chkpId); @@ -190,23 +190,25 @@ int32_t streamMetaDoStateDataConvertImpl(SStreamMeta* pMeta) { void* key = taosHashGetKey(pIter, NULL); code = streamStateConvertDataFormat(pMeta->path, key, *(void**)pIter); if (code != 0) { - // continue + qError("failed to cvt data"); + goto _EXIT; } pIter = taosHashIterate(pBackend->cfInst, pIter); } - // streamBackendCleanup(); - return 0; +_EXIT: + streamBackendCleanup((void*)pBackend); + return code; } -int32_t streamMetaDoStateDataConvert(SStreamMeta* pMeta) { +int32_t streamMetaMayDoStateBackendConvert(SStreamMeta* pMeta) { int8_t compatible = streamMetaCheckStateCompatible(pMeta); if (compatible == STREAM_STATA_COMPATIBLE) { return 0; } else if (compatible == STREAM_STATA_NEED_CONVERT) { qInfo("stream state need covert backend format"); - return streamMetaDoStateDataConvertImpl(pMeta); + return streamMetaDoStateBackendConvertImpl(pMeta); } else if (compatible == STREAM_STATA_NO_COMPATIBLE) { qError( "stream read incompatible data, rm %s/vnode/vnode*/tq/stream if taosd cannot start, and rebuild stream " @@ -237,7 +239,7 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF goto _err; } - if (streamMetaDoStateDataConvert(pMeta) < 0) { + if (streamMetaMayDoStateBackendConvert(pMeta) < 0) { goto _err; } From 11da631bb5291e157176e9a7a8be7ef6ec073671 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Fri, 22 Sep 2023 09:08:42 +0000 Subject: [PATCH 006/195] add cvt state --- include/libs/stream/tstream.h | 6 +++++- source/dnode/vnode/src/tq/tq.c | 32 +++++++++++++++-------------- source/libs/stream/src/streamMeta.c | 16 +++++++++++++-- source/libs/stream/src/streamTask.c | 2 ++ 4 files changed, 38 insertions(+), 18 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 29dc054d78..e09ffb416c 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -31,7 +31,7 @@ extern "C" { typedef struct SStreamTask SStreamTask; -#define SSTREAM_TASK_VER 3 +#define SSTREAM_TASK_VER 3 #define SSTREAM_TASK_INCOMPATIBLE_VER 1 #define SSTREAM_TASK_NEED_CONVERT_VER 2 @@ -382,6 +382,7 @@ struct SStreamTask { int32_t transferStateAlignCnt; struct SStreamMeta* pMeta; SSHashObj* pNameMap; + void* pBackend; char reserve[256]; }; @@ -422,6 +423,8 @@ typedef struct SStreamMeta { int32_t chkpCap; SRWLatch chkpDirLock; int32_t pauseTaskNum; + + // SHashObj* pTaskBackend; } SStreamMeta; int32_t tEncodeStreamEpInfo(SEncoder* pEncoder, const SStreamChildEpInfo* pInfo); @@ -722,6 +725,7 @@ int32_t streamMetaReopen(SStreamMeta* pMeta, int64_t chkpId); int32_t streamMetaCommit(SStreamMeta* pMeta); int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta); void streamMetaNotifyClose(SStreamMeta* pMeta); +void* streamMetaGetBackendByTaskKey(SStreamMeta* pMeta, char* key); // checkpoint int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSourceReq* pReq); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 3aeb679eb7..e136b9b265 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -582,9 +582,9 @@ int32_t tqProcessDeleteSubReq(STQ* pTq, int64_t sversion, char* msg, int32_t msg taosWLockLatch(&pTq->lock); bool exec = tqIsHandleExec(pHandle); - if(exec){ + if (exec) { tqInfo("vgId:%d, topic:%s, subscription is executing, delete wait for 10ms and retry, pHandle:%p", vgId, - pHandle->subKey, pHandle); + pHandle->subKey, pHandle); taosWUnLockLatch(&pTq->lock); taosMsleep(10); continue; @@ -699,12 +699,12 @@ int32_t tqProcessSubscribeReq(STQ* pTq, int64_t sversion, char* msg, int32_t msg } ret = tqMetaSaveHandle(pTq, req.subKey, &handle); } else { - while(1){ + while (1) { taosWLockLatch(&pTq->lock); bool exec = tqIsHandleExec(pHandle); - if(exec){ - tqInfo("vgId:%d, topic:%s, subscription is executing, sub wait for 10ms and retry, pHandle:%p", pTq->pVnode->config.vgId, - pHandle->subKey, pHandle); + if (exec) { + tqInfo("vgId:%d, topic:%s, subscription is executing, sub wait for 10ms and retry, pHandle:%p", + pTq->pVnode->config.vgId, pHandle->subKey, pHandle); taosWUnLockLatch(&pTq->lock); taosMsleep(10); continue; @@ -713,7 +713,7 @@ int32_t tqProcessSubscribeReq(STQ* pTq, int64_t sversion, char* msg, int32_t msg tqInfo("vgId:%d no switch consumer:0x%" PRIx64 " remains, because redo wal log", req.vgId, req.newConsumerId); } else { tqInfo("vgId:%d switch consumer from Id:0x%" PRIx64 " to Id:0x%" PRIx64, req.vgId, pHandle->consumerId, - req.newConsumerId); + req.newConsumerId); atomic_store_64(&pHandle->consumerId, req.newConsumerId); atomic_store_32(&pHandle->epoch, 0); tqUnregisterPushHandle(pTq, pHandle); @@ -736,6 +736,9 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { tqDebug("s-task:0x%x start to expand task", pTask->id.taskId); int32_t code = streamTaskInit(pTask, pTq->pStreamMeta, &pTq->pVnode->msgCb, ver); + + pTask->pBackend = streamMetaGetBackendByTaskKey(pTq->pStreamMeta, (char*)pTask->id.idStr); + if (code != TSDB_CODE_SUCCESS) { return code; } @@ -859,8 +862,8 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { pChkInfo->checkpointId, pChkInfo->checkpointVer, pChkInfo->nextProcessVer); } - tqInfo("vgId:%d expand stream task, s-task:%s, checkpointId:%" PRId64 " checkpointVer:%" PRId64 " nextProcessVer:%" PRId64 - " child id:%d, level:%d, status:%s fill-history:%d, trigger:%" PRId64 " ms", + tqInfo("vgId:%d expand stream task, s-task:%s, checkpointId:%" PRId64 " checkpointVer:%" PRId64 + " nextProcessVer:%" PRId64 " child id:%d, level:%d, status:%s fill-history:%d, trigger:%" PRId64 " ms", vgId, pTask->id.idStr, pChkInfo->checkpointId, pChkInfo->checkpointVer, pChkInfo->nextProcessVer, pTask->info.selfChildId, pTask->info.taskLevel, streamGetTaskStatusStr(pTask->status.taskStatus), pTask->info.fillHistory, pTask->info.triggerParam); @@ -1004,7 +1007,7 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms bool restored = pTq->pVnode->restored; if (p != NULL && restored) { p->tsInfo.init = taosGetTimestampMs(); - tqDebug("s-task:%s set the init ts:%"PRId64, p->id.idStr, p->tsInfo.init); + tqDebug("s-task:%s set the init ts:%" PRId64, p->id.idStr, p->tsInfo.init); streamTaskCheckDownstream(p); } else if (!restored) { @@ -1443,7 +1446,7 @@ int32_t tqProcessTaskResumeImpl(STQ* pTq, SStreamTask* pTask, int64_t sversion, return 0; } - int8_t status = pTask->status.taskStatus; + int8_t status = pTask->status.taskStatus; if (status == TASK_STATUS__NORMAL || status == TASK_STATUS__SCAN_HISTORY || status == TASK_STATUS__CK) { // no lock needs to secure the access of the version if (igUntreated && level == TASK_LEVEL__SOURCE && !pTask->info.fillHistory) { @@ -1622,12 +1625,13 @@ int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg) { // downstream not ready, current the stream tasks are not all ready. Ignore this checkpoint req. if (pTask->status.downstreamReady != 1) { qError("s-task:%s not ready for checkpoint, since downstream not ready, ignore this checkpoint:%" PRId64 - ", set it failure", pTask->id.idStr, req.checkpointId); + ", set it failure", + pTask->id.idStr, req.checkpointId); streamMetaReleaseTask(pMeta, pTask); SRpcMsg rsp = {0}; buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, 0); - tmsgSendRsp(&rsp); // error occurs + tmsgSendRsp(&rsp); // error occurs return TSDB_CODE_SUCCESS; } @@ -1784,7 +1788,6 @@ _end: tDecoderClear(&decoder); if (allStopped) { - if (!pTq->pVnode->restored) { tqDebug("vgId:%d vnode restore not completed, not restart the tasks", vgId); } else { @@ -1816,4 +1819,3 @@ _end: return rsp.code; } - diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index aa29ea8159..e516678941 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -220,6 +220,17 @@ int32_t streamMetaMayDoStateBackendConvert(SStreamMeta* pMeta) { return 0; } +void* streamMetaGetBackendByTaskKey(SStreamMeta* pMeta, char* key) { + void** ppBackend = taosHashGet(pMeta->pTaskBackendUnique, key, strlen(key)); + if (ppBackend != NULL && *ppBackend != NULL) { + // add ref later + return *ppBackend; + } + void* pBackend = streamStateOpenTaskBackend(pMeta->path, key); + taosHashPut(pMeta->pTaskBackendUnique, key, strlen(key), &pBackend, sizeof(void*)); + + return pBackend; +} SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandFunc, int32_t vgId, int64_t stage) { int32_t code = -1; SStreamMeta* pMeta = taosMemoryCalloc(1, sizeof(SStreamMeta)); @@ -277,11 +288,12 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF pMeta->hbInfo.tickCounter = 0; pMeta->hbInfo.stopFlag = 0; + pMeta->pTaskBackendUnique = + taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), false, HASH_ENTRY_LOCK); + // start backend // taosInitRWLatch(&pMeta->chkpDirLock); - // pMeta->pTaskBackendUnique = - // taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), false, HASH_ENTRY_LOCK); // pMeta->chkpSaved = taosArrayInit(4, sizeof(int64_t)); // pMeta->chkpInUse = taosArrayInit(4, sizeof(int64_t)); // pMeta->chkpCap = 8; diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 837f787dd2..9a76f661f8 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -388,6 +388,8 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i taosThreadMutexInit(&pTask->lock, NULL); streamTaskOpenAllUpstreamInput(pTask); + //pTask->pBackend = streamStateOpenTaskBackend(pMeta->path, (char*)pTask->id.idStr); + return TSDB_CODE_SUCCESS; } From 8890f5c950e4133b6d1169c300cae1c71499e479 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Fri, 22 Sep 2023 12:51:29 +0000 Subject: [PATCH 007/195] add cvt state --- source/libs/stream/src/streamMeta.c | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index e516678941..2a76f178a7 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -140,7 +140,7 @@ enum STREAM_STATE_VER { STREAM_STATA_NEED_CONVERT, }; -int32_t streamMetaCheckStateCompatible(SStreamMeta* pMeta) { +int32_t streamMetaCheckBackendCompatible(SStreamMeta* pMeta) { int8_t ret = STREAM_STATA_COMPATIBLE; TBC* pCur = NULL; @@ -180,7 +180,7 @@ int32_t streamMetaCheckStateCompatible(SStreamMeta* pMeta) { return ret; } -int32_t streamMetaDoStateBackendConvertImpl(SStreamMeta* pMeta) { +int32_t streamMetaConvertBackendFormat(SStreamMeta* pMeta) { int32_t code = 0; int64_t chkpId = streamGetLatestCheckpointId(pMeta); SBackendWrapper* pBackend = streamBackendInit(pMeta->path, chkpId); @@ -201,14 +201,14 @@ _EXIT: streamBackendCleanup((void*)pBackend); return code; } -int32_t streamMetaMayDoStateBackendConvert(SStreamMeta* pMeta) { - int8_t compatible = streamMetaCheckStateCompatible(pMeta); +int32_t streamMetaMayConvertBackendFormat(SStreamMeta* pMeta) { + int8_t compatible = streamMetaCheckBackendCompatible(pMeta); if (compatible == STREAM_STATA_COMPATIBLE) { return 0; } else if (compatible == STREAM_STATA_NEED_CONVERT) { qInfo("stream state need covert backend format"); - return streamMetaDoStateBackendConvertImpl(pMeta); + return streamMetaConvertBackendFormat(pMeta); } else if (compatible == STREAM_STATA_NO_COMPATIBLE) { qError( "stream read incompatible data, rm %s/vnode/vnode*/tq/stream if taosd cannot start, and rebuild stream " @@ -250,7 +250,7 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF goto _err; } - if (streamMetaMayDoStateBackendConvert(pMeta) < 0) { + if (streamMetaMayConvertBackendFormat(pMeta) < 0) { goto _err; } From ed172a939cc08075887713b0662b9dccfb18e008 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Mon, 25 Sep 2023 18:09:35 +0800 Subject: [PATCH 008/195] rename cfg default name --- source/libs/stream/src/streamBackendRocksdb.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 5df33949ef..fcdb945aae 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1540,7 +1540,7 @@ STaskBackendWrapper* streamStateOpenTaskBackend(char* path, char* key) { qDebug("succ to init stream backend at %s, backend:%p", taskPath, pTaskBackend); return pTaskBackend; } -void streamMeteCloseTaskBackend(STaskBackendWrapper* wrapper) { +void streamStateCloseTaskBackend(STaskBackendWrapper* wrapper) { if (wrapper == NULL) return; rocksdb_flushoptions_t* flushOpt = rocksdb_flushoptions_create(); @@ -1673,7 +1673,7 @@ int32_t streamStateConvertDataFormat(char* path, char* key, void* pCfInst) { } _EXIT: - streamMeteCloseTaskBackend(pTaskBackend); + streamStateCloseTaskBackend(pTaskBackend); return code; } From d4f87378fd491895cfe2ede6bb01dc853e51f7d4 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 27 Sep 2023 14:44:09 +0800 Subject: [PATCH 009/195] avoid first tag index conflict --- source/dnode/snode/src/snode.c | 52 ++++++++++++---------- source/libs/stream/src/streamMeta.c | 4 +- source/libs/stream/src/streamState.c | 64 ++++++++++++++-------------- 3 files changed, 64 insertions(+), 56 deletions(-) diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index 2b1885fb0e..2dba51c150 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -50,7 +50,7 @@ void sndEnqueueStreamDispatch(SSnode *pSnode, SRpcMsg *pMsg) { FAIL: if (pMsg->info.handle == NULL) return; - SRpcMsg rsp = { .code = code, .info = pMsg->info}; + SRpcMsg rsp = {.code = code, .info = pMsg->info}; tmsgSendRsp(&rsp); rpcFreeCont(pMsg->pCont); taosFreeQitem(pMsg); @@ -62,6 +62,8 @@ int32_t sndExpandTask(SSnode *pSnode, SStreamTask *pTask, int64_t ver) { if (code != TSDB_CODE_SUCCESS) { return code; } + pTask->pBackend = NULL; + ASSERT(0); streamTaskOpenAllUpstreamInput(pTask); @@ -73,8 +75,11 @@ int32_t sndExpandTask(SSnode *pSnode, SStreamTask *pTask, int64_t ver) { qDebug("s-task:%s state:%p", pTask->id.idStr, pTask->pState); } - int32_t numOfChildEp = taosArrayGetSize(pTask->pUpstreamInfoList); - SReadHandle handle = { .vnode = NULL, .numOfVgroups = numOfChildEp, .pStateBackend = pTask->pState, .fillHistory = pTask->info.fillHistory }; + int32_t numOfChildEp = taosArrayGetSize(pTask->pUpstreamInfoList); + SReadHandle handle = {.vnode = NULL, + .numOfVgroups = numOfChildEp, + .pStateBackend = pTask->pState, + .fillHistory = pTask->info.fillHistory}; initStreamStateAPI(&handle.api); pTask->exec.pExecutor = qCreateStreamExecTaskInfo(pTask->exec.qmsg, &handle, 0, pTask->id.taskId); @@ -83,23 +88,23 @@ int32_t sndExpandTask(SSnode *pSnode, SStreamTask *pTask, int64_t ver) { streamTaskResetUpstreamStageInfo(pTask); streamSetupScheduleTrigger(pTask); - SCheckpointInfo* pChkInfo = &pTask->chkInfo; + SCheckpointInfo *pChkInfo = &pTask->chkInfo; // checkpoint ver is the kept version, handled data should be the next version. if (pTask->chkInfo.checkpointId != 0) { pTask->chkInfo.nextProcessVer = pTask->chkInfo.checkpointVer + 1; - qInfo("s-task:%s restore from the checkpointId:%" PRId64 " ver:%" PRId64 " nextProcessVer:%" PRId64, pTask->id.idStr, - pChkInfo->checkpointId, pChkInfo->checkpointVer, pChkInfo->nextProcessVer); + qInfo("s-task:%s restore from the checkpointId:%" PRId64 " ver:%" PRId64 " nextProcessVer:%" PRId64, + pTask->id.idStr, pChkInfo->checkpointId, pChkInfo->checkpointVer, pChkInfo->nextProcessVer); } else { if (pTask->chkInfo.nextProcessVer == -1) { pTask->chkInfo.nextProcessVer = 0; } } - qInfo("snode:%d expand stream task, s-task:%s, checkpointId:%" PRId64 " checkpointVer:%" PRId64 " nextProcessVer:%" PRId64 - " child id:%d, level:%d, status:%s fill-history:%d, trigger:%" PRId64 " ms", + qInfo("snode:%d expand stream task, s-task:%s, checkpointId:%" PRId64 " checkpointVer:%" PRId64 + " nextProcessVer:%" PRId64 " child id:%d, level:%d, status:%s fill-history:%d, trigger:%" PRId64 " ms", SNODE_HANDLE, pTask->id.idStr, pChkInfo->checkpointId, pChkInfo->checkpointVer, pChkInfo->nextProcessVer, - pTask->info.selfChildId, pTask->info.taskLevel, streamGetTaskStatusStr(pTask->status.taskStatus), - pTask->info.fillHistory, pTask->info.triggerParam); + pTask->info.selfChildId, pTask->info.taskLevel, streamGetTaskStatusStr(pTask->status.taskStatus), + pTask->info.fillHistory, pTask->info.triggerParam); return 0; } @@ -179,8 +184,8 @@ int32_t sndProcessTaskDeployReq(SSnode *pSnode, char *msg, int32_t msgLen) { int32_t numOfTasks = streamMetaGetNumOfTasks(pSnode->pMeta); taosWUnLockLatch(&pSnode->pMeta->lock); - qDebug("snode:%d s-task:%s is deployed on snode and add into meta, status:%s, numOfTasks:%d", SNODE_HANDLE, pTask->id.idStr, - streamGetTaskStatusStr(pTask->status.taskStatus), numOfTasks); + qDebug("snode:%d s-task:%s is deployed on snode and add into meta, status:%s, numOfTasks:%d", SNODE_HANDLE, + pTask->id.idStr, streamGetTaskStatusStr(pTask->status.taskStatus), numOfTasks); streamTaskCheckDownstream(pTask); return 0; @@ -190,7 +195,7 @@ int32_t sndProcessTaskDropReq(SSnode *pSnode, char *msg, int32_t msgLen) { SVDropStreamTaskReq *pReq = (SVDropStreamTaskReq *)msg; qDebug("snode:%d receive msg to drop stream task:0x%x", pSnode->pMeta->vgId, pReq->taskId); - SStreamTask* pTask = streamMetaAcquireTask(pSnode->pMeta, pReq->streamId, pReq->taskId); + SStreamTask *pTask = streamMetaAcquireTask(pSnode->pMeta, pReq->streamId, pReq->taskId); if (pTask == NULL) { qError("vgId:%d failed to acquire s-task:0x%x when dropping it", pSnode->pMeta->vgId, pReq->taskId); return 0; @@ -355,12 +360,13 @@ int32_t sndProcessStreamTaskCheckReq(SSnode *pSnode, SRpcMsg *pMsg) { rsp.status = streamTaskCheckStatus(pTask, req.upstreamTaskId, req.upstreamNodeId, req.stage); streamMetaReleaseTask(pSnode->pMeta, pTask); - const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus); + const char *pStatus = streamGetTaskStatusStr(pTask->status.taskStatus); qDebug("s-task:%s status:%s, recv task check req(reqId:0x%" PRIx64 ") task:0x%x (vgId:%d), ready:%d", - pTask->id.idStr, pStatus, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status); + pTask->id.idStr, pStatus, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status); } else { rsp.status = 0; - qDebug("recv task check(taskId:0x%x not built yet) req(reqId:0x%" PRIx64 ") from task:0x%x (vgId:%d), rsp status %d", + qDebug("recv task check(taskId:0x%x not built yet) req(reqId:0x%" PRIx64 + ") from task:0x%x (vgId:%d), rsp status %d", taskId, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status); } @@ -388,15 +394,15 @@ int32_t sndProcessStreamTaskCheckReq(SSnode *pSnode, SRpcMsg *pMsg) { return 0; } -int32_t sndProcessStreamTaskCheckRsp(SSnode* pSnode, SRpcMsg* pMsg) { - char* pReq = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); +int32_t sndProcessStreamTaskCheckRsp(SSnode *pSnode, SRpcMsg *pMsg) { + char *pReq = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); int32_t len = pMsg->contLen - sizeof(SMsgHead); int32_t code; SStreamTaskCheckRsp rsp; SDecoder decoder; - tDecoderInit(&decoder, (uint8_t*)pReq, len); + tDecoderInit(&decoder, (uint8_t *)pReq, len); code = tDecodeStreamTaskCheckRsp(&decoder, &rsp); if (code < 0) { @@ -405,13 +411,13 @@ int32_t sndProcessStreamTaskCheckRsp(SSnode* pSnode, SRpcMsg* pMsg) { } tDecoderClear(&decoder); - qDebug("tq task:0x%x (vgId:%d) recv check rsp(reqId:0x%" PRIx64 ") from 0x%x (vgId:%d) status %d", - rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.reqId, rsp.downstreamTaskId, rsp.downstreamNodeId, rsp.status); + qDebug("tq task:0x%x (vgId:%d) recv check rsp(reqId:0x%" PRIx64 ") from 0x%x (vgId:%d) status %d", rsp.upstreamTaskId, + rsp.upstreamNodeId, rsp.reqId, rsp.downstreamTaskId, rsp.downstreamNodeId, rsp.status); - SStreamTask* pTask = streamMetaAcquireTask(pSnode->pMeta, rsp.streamId, rsp.upstreamTaskId); + SStreamTask *pTask = streamMetaAcquireTask(pSnode->pMeta, rsp.streamId, rsp.upstreamTaskId); if (pTask == NULL) { qError("tq failed to locate the stream task:0x%x (vgId:%d), it may have been destroyed", rsp.upstreamTaskId, - pSnode->pMeta->vgId); + pSnode->pMeta->vgId); return -1; } diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 2a76f178a7..99b3b9cca1 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -221,14 +221,16 @@ int32_t streamMetaMayConvertBackendFormat(SStreamMeta* pMeta) { } void* streamMetaGetBackendByTaskKey(SStreamMeta* pMeta, char* key) { + taosThreadMutexLock(&pMeta->backendMutex); void** ppBackend = taosHashGet(pMeta->pTaskBackendUnique, key, strlen(key)); if (ppBackend != NULL && *ppBackend != NULL) { // add ref later + taosThreadMutexUnlock(&pMeta->backendMutex); return *ppBackend; } void* pBackend = streamStateOpenTaskBackend(pMeta->path, key); taosHashPut(pMeta->pTaskBackendUnique, key, strlen(key), &pBackend, sizeof(void*)); - + taosThreadMutexLock(&pMeta->backendMutex); return pBackend; } SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandFunc, int32_t vgId, int64_t stage) { diff --git a/source/libs/stream/src/streamState.c b/source/libs/stream/src/streamState.c index 44c7b4f2e0..f51181b0b4 100644 --- a/source/libs/stream/src/streamState.c +++ b/source/libs/stream/src/streamState.c @@ -106,43 +106,43 @@ SStreamState* streamStateOpen(char* path, void* pTask, bool specPath, int32_t sz } SStreamTask* pStreamTask = pTask; - char statePath[1024]; - if (!specPath) { - sprintf(statePath, "%s%s%d", path, TD_DIRSEP, pStreamTask->id.taskId); - } else { - memset(statePath, 0, 1024); - tstrncpy(statePath, path, 1024); - } + // char statePath[1024]; + // if (!specPath) { + // sprintf(statePath, "%s%s%d", path, TD_DIRSEP, pStreamTask->id.taskId); + // } else { + // memset(statePath, 0, 1024); + // tstrncpy(statePath, path, 1024); + // } pState->taskId = pStreamTask->id.taskId; pState->streamId = pStreamTask->id.streamId; sprintf(pState->pTdbState->idstr, "0x%" PRIx64 "-%d", pState->streamId, pState->taskId); #ifdef USE_ROCKSDB - SStreamMeta* pMeta = pStreamTask->pMeta; - pState->streamBackendRid = pMeta->streamBackendRid; - // taosWLockLatch(&pMeta->lock); - taosThreadMutexLock(&pMeta->backendMutex); - void* uniqueId = - taosHashGet(pMeta->pTaskBackendUnique, pState->pTdbState->idstr, strlen(pState->pTdbState->idstr) + 1); - if (uniqueId == NULL) { - int code = streamStateOpenBackend(pMeta->streamBackend, pState); - if (code == -1) { - taosThreadMutexUnlock(&pMeta->backendMutex); - taosMemoryFree(pState); - return NULL; - } - taosHashPut(pMeta->pTaskBackendUnique, pState->pTdbState->idstr, strlen(pState->pTdbState->idstr) + 1, - &pState->pTdbState->backendCfWrapperId, sizeof(pState->pTdbState->backendCfWrapperId)); - } else { - int64_t id = *(int64_t*)uniqueId; - pState->pTdbState->backendCfWrapperId = id; - pState->pTdbState->pBackendCfWrapper = taosAcquireRef(streamBackendCfWrapperId, id); - // already exist stream task for - qInfo("already exist stream-state for %s", pState->pTdbState->idstr); - // taosAcquireRef(streamBackendId, pState->streamBackendRid); - } - taosThreadMutexUnlock(&pMeta->backendMutex); + // SStreamMeta* pMeta = pStreamTask->pMeta; + // pState->streamBackendRid = pMeta->streamBackendRid; + // taosWLockLatch(&pMeta->lock); + // taosThreadMutexLock(&pMeta->backendMutex); + // void* uniqueId = + // taosHashGet(pMeta->pTaskBackendUnique, pState->pTdbState->idstr, strlen(pState->pTdbState->idstr) + 1); + // if (uniqueId == NULL) { + // int code = streamStateOpenBackend(pMeta->streamBackend, pState); + // if (code == -1) { + // taosThreadMutexUnlock(&pMeta->backendMutex); + // taosMemoryFree(pState); + // return NULL; + // } + // taosHashPut(pMeta->pTaskBackendUnique, pState->pTdbState->idstr, strlen(pState->pTdbState->idstr) + 1, + // &pState->pTdbState->backendCfWrapperId, sizeof(pState->pTdbState->backendCfWrapperId)); + // } else { + // int64_t id = *(int64_t*)uniqueId; + // pState->pTdbState->backendCfWrapperId = id; + // pState->pTdbState->pBackendCfWrapper = taosAcquireRef(streamBackendCfWrapperId, id); + // // already exist stream task for + // qInfo("already exist stream-state for %s", pState->pTdbState->idstr); + // // taosAcquireRef(streamBackendId, pState->streamBackendRid); + // } + // taosThreadMutexUnlock(&pMeta->backendMutex); pState->pTdbState->pOwner = pTask; pState->pFileState = NULL; @@ -1125,7 +1125,7 @@ int32_t streamStateGetParName(SStreamState* pState, int64_t groupId, void** pVal void streamStateDestroy(SStreamState* pState, bool remove) { #ifdef USE_ROCKSDB streamFileStateDestroy(pState->pFileState); - streamStateDestroy_rocksdb(pState, remove); + // streamStateDestroy_rocksdb(pState, remove); tSimpleHashCleanup(pState->parNameMap); // do nothong #endif From 54e3ac2c1ea26c274c22ad8531ee1b958e235e98 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 28 Sep 2023 15:46:12 +0800 Subject: [PATCH 010/195] refact task backend --- include/libs/stream/tstream.h | 1 + source/dnode/vnode/src/tq/tq.c | 6 +- source/libs/stream/inc/streamBackendRocksdb.h | 6 +- source/libs/stream/src/streamBackendRocksdb.c | 178 ++++++++++++------ source/libs/stream/src/streamMeta.c | 5 +- source/libs/stream/src/streamState.c | 10 +- source/libs/stream/src/streamTask.c | 2 +- 7 files changed, 136 insertions(+), 72 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index e09ffb416c..553e8f8217 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -383,6 +383,7 @@ struct SStreamTask { struct SStreamMeta* pMeta; SSHashObj* pNameMap; void* pBackend; + int64_t backendRefId; char reserve[256]; }; diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index e136b9b265..12b89dbd9c 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -736,12 +736,10 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { tqDebug("s-task:0x%x start to expand task", pTask->id.taskId); int32_t code = streamTaskInit(pTask, pTq->pStreamMeta, &pTq->pVnode->msgCb, ver); + if (code != TSDB_CODE_SUCCESS) return code; pTask->pBackend = streamMetaGetBackendByTaskKey(pTq->pStreamMeta, (char*)pTask->id.idStr); - - if (code != TSDB_CODE_SUCCESS) { - return code; - } + if (pTask->pBackend == NULL) return -1; streamTaskOpenAllUpstreamInput(pTask); diff --git a/source/libs/stream/inc/streamBackendRocksdb.h b/source/libs/stream/inc/streamBackendRocksdb.h index 3e07f2532e..a5533ef2f2 100644 --- a/source/libs/stream/inc/streamBackendRocksdb.h +++ b/source/libs/stream/inc/streamBackendRocksdb.h @@ -65,6 +65,8 @@ typedef struct { rocksdb_compactionfilterfactory_t* filterFactory; TdThreadMutex mutex; + int64_t refId; + } STaskBackendWrapper; void* streamBackendInit(const char* path, int64_t chkpId); @@ -75,7 +77,9 @@ int32_t streamBackendDoCheckpoint(void* pMeta, uint64_t checkpointI SListNode* streamBackendAddCompare(void* backend, void* arg); void streamBackendDelCompare(void* backend, void* arg); int32_t streamStateConvertDataFormat(char* path, char* key, void* cfInst); -STaskBackendWrapper* streamStateOpenTaskBackend(char* path, char* key); +STaskBackendWrapper* taskBackendOpen(char* path, char* key); + +void taskBackendAddRef(void* pTaskBackend); int streamStateOpenBackend(void* backend, SStreamState* pState); void streamStateCloseBackend(SStreamState* pState, bool remove); diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index fcdb945aae..eed65d3f3a 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -629,7 +629,7 @@ void streamBackendHandleCleanup(void* arg) { taosThreadRwlockDestroy(&wrapper->rwLock); wrapper->rocksdb = NULL; - taosReleaseRef(streamBackendId, wrapper->backendId); + // taosReleaseRef(streamBackendId, wrapper->backendId); qDebug("end to do-close backendwrapper %p, %s", wrapper, wrapper->idstr); taosMemoryFree(wrapper); @@ -776,7 +776,10 @@ int32_t chkpGetAllDbCfHandle(SStreamMeta* pMeta, rocksdb_column_family_handle_t* int64_t id = *(int64_t*)pIter; SBackendCfWrapper* wrapper = taosAcquireRef(streamBackendCfWrapperId, id); - if (wrapper == NULL) continue; + if (wrapper == NULL) { + pIter = taosHashIterate(pMeta->pTaskBackendUnique, pIter); + continue; + } taosThreadRwlockRdlock(&wrapper->rwLock); for (int i = 0; i < sizeof(ginitDict) / sizeof(ginitDict[0]); i++) { @@ -788,7 +791,6 @@ int32_t chkpGetAllDbCfHandle(SStreamMeta* pMeta, rocksdb_column_family_handle_t* taosThreadRwlockUnlock(&wrapper->rwLock); taosArrayPush(refs, &id); - pIter = taosHashIterate(pMeta->pTaskBackendUnique, pIter); } int32_t nCf = taosArrayGetSize(pHandle); @@ -1452,35 +1454,72 @@ void destroyRocksdbCfInst(RocksdbCfInst* inst) { taosMemoryFree(inst); } -STaskBackendWrapper* streamStateOpenTaskBackend(char* path, char* key) { - int32_t code = 0; - - char* taskPath = taosMemoryCalloc(1, strlen(path) + 128); - sprintf(taskPath, "%s%s%s%s%s", path, TD_DIRSEP, "state", TD_DIRSEP, key); - if (!taosDirExist(taskPath)) { - code = taosMkDir(taskPath); - if (code != 0) { - qError("failed to create dir: %s, reason:%s", taskPath, tstrerror(code)); - taosMemoryFree(taskPath); - return NULL; +int32_t getCfIdx(const char* cfName) { + int idx = -1; + size_t len = strlen(cfName); + for (int i = 0; i < sizeof(ginitDict) / sizeof(ginitDict[0]); i++) { + if (len == ginitDict[i].len && strncmp(cfName, ginitDict[i].key, strlen(cfName)) == 0) { + idx = i; + break; } } - char* err = NULL; + return idx; +} +int32_t taskBackendOpenCfs(STaskBackendWrapper* pTask, char* path, char** pCfNames, int32_t nCf) { + int32_t code = -1; + char* err = NULL; - STaskBackendWrapper* pTaskBackend = taosMemoryCalloc(1, sizeof(SBackendWrapper)); - rocksdb_env_t* env = rocksdb_create_default_env(); + rocksdb_options_t** cfOpts = taosMemoryCalloc(nCf, sizeof(rocksdb_options_t*)); + rocksdb_column_family_handle_t** cfHandle = taosMemoryCalloc(nCf, sizeof(rocksdb_column_family_handle_t*)); + + for (int i = 0; i < nCf; i++) { + int32_t idx = getCfIdx(pCfNames[i]); + cfOpts[i] = pTask->pCfOpts[idx]; + } + + rocksdb_t* db = rocksdb_open_column_families(pTask->dbOpt, path, nCf, (const char* const*)pCfNames, + (const rocksdb_options_t* const*)cfOpts, cfHandle, &err); + + if (err != NULL) { + qError("failed to open cf path: %s", err); + taosMemoryFree(err); + goto _EXIT; + } + + for (int i = 0; i < nCf; i++) { + int32_t idx = getCfIdx(pCfNames[i]); + pTask->pCf[idx] = cfHandle[i]; + } + + pTask->db = db; + code = 0; + +_EXIT: + taosMemoryFree(cfOpts); + taosMemoryFree(cfHandle); + return code; +} +void taskBackendAddRef(void* pTaskBackend) { + STaskBackendWrapper* pBackend = pTaskBackend; + taosAcquireRef(streamBackendCfWrapperId, pBackend->refId); + return; +} +void taskBackendDestroy(STaskBackendWrapper* wrapper); + +void taskBackendInitOpt(STaskBackendWrapper* pTaskBackend) { + rocksdb_env_t* env = rocksdb_create_default_env(); rocksdb_cache_t* cache = rocksdb_cache_create_lru(256); rocksdb_options_t* opts = rocksdb_options_create(); - // rocksdb_options_set_env(opts, env); + rocksdb_options_set_env(opts, env); rocksdb_options_set_create_if_missing(opts, 1); rocksdb_options_set_create_missing_column_families(opts, 1); // rocksdb_options_set_max_total_wal_size(opts, dbMemLimit); rocksdb_options_set_recycle_log_file_num(opts, 6); rocksdb_options_set_max_write_buffer_number(opts, 3); rocksdb_options_set_info_log_level(opts, 1); - // rocksdb_options_set_db_write_buffer_size(opts, dbMemLimit); - // rocksdb_options_set_write_buffer_size(opts, dbMemLimit / 2); + rocksdb_options_set_db_write_buffer_size(opts, 64 << 20); + rocksdb_options_set_write_buffer_size(opts, 32 << 20); rocksdb_options_set_atomic_flush(opts, 1); pTaskBackend->dbOpt = opts; @@ -1519,46 +1558,86 @@ STaskBackendWrapper* streamStateOpenTaskBackend(char* path, char* key) { pTaskBackend->pCfOpts[i] = opt; pTaskBackend->pCfParams[i].tableOpt = tableOpt; } + return; +} +int32_t taskBackendBuildFullPath(char* path, char* key, char** fullPath) { + int32_t code = 0; + char* taskPath = taosMemoryCalloc(1, strlen(path) + 128); + sprintf(taskPath, "%s%s%s%s%s", path, TD_DIRSEP, "state", TD_DIRSEP, key); + if (!taosDirExist(taskPath)) { + code = taosMkDir(taskPath); + if (code != 0) { + qError("failed to create dir: %s, reason:%s", taskPath, tstrerror(code)); + taosMemoryFree(taskPath); + return code; + } + } + *fullPath = taskPath; + return 0; +} +STaskBackendWrapper* taskBackendOpen(char* path, char* key) { + char* taskPath = NULL; + char* err = NULL; - char** cfs = rocksdb_list_column_families(opts, taskPath, &nCf, &err); + int32_t code = taskBackendBuildFullPath(path, key, &taskPath); + if (code != 0) return NULL; + + size_t nCf = 0; + + STaskBackendWrapper* pTaskBackend = taosMemoryCalloc(1, sizeof(STaskBackendWrapper)); + taskBackendInitOpt(pTaskBackend); + + char** cfs = rocksdb_list_column_families(pTaskBackend->dbOpt, taskPath, &nCf, &err); if (nCf == 0 || nCf == 1 || err != NULL) { taosMemoryFreeClear(err); - pTaskBackend->db = rocksdb_open(opts, taskPath, &err); + pTaskBackend->db = rocksdb_open(pTaskBackend->dbOpt, taskPath, &err); if (err != NULL) { qError("failed to open rocksdb, path:%s, reason:%s", taskPath, err); taosMemoryFreeClear(err); + code = -1; + goto _EXIT; } } else { + code = taskBackendOpenCfs(pTaskBackend, taskPath, cfs, nCf); + if (code != 0) goto _EXIT; } - if (cfs != NULL) { - rocksdb_list_column_families_destroy(cfs, nCf); - } + if (cfs != NULL) rocksdb_list_column_families_destroy(cfs, nCf); taosThreadMutexInit(&pTaskBackend->mutex, NULL); taosMemoryFree(taskPath); qDebug("succ to init stream backend at %s, backend:%p", taskPath, pTaskBackend); + + pTaskBackend->refId = taosAddRef(streamBackendCfWrapperId, pTaskBackend); + return pTaskBackend; +_EXIT: + + taskBackendDestroy(pTaskBackend); + return NULL; } -void streamStateCloseTaskBackend(STaskBackendWrapper* wrapper) { +void taskBackendDestroy(STaskBackendWrapper* wrapper) { if (wrapper == NULL) return; - rocksdb_flushoptions_t* flushOpt = rocksdb_flushoptions_create(); - rocksdb_flushoptions_set_wait(flushOpt, 1); + if (wrapper->db && wrapper->pCf) { + rocksdb_flushoptions_t* flushOpt = rocksdb_flushoptions_create(); + rocksdb_flushoptions_set_wait(flushOpt, 1); - int8_t nCf = sizeof(ginitDict) / sizeof(ginitDict[0]); - char* err = NULL; - for (int i = 0; i < nCf; i++) { - if (wrapper->pCf[i] != NULL) rocksdb_flush_cf(wrapper->db, flushOpt, wrapper->pCf[i], &err); - if (err != NULL) { - qError("failed to flush cf:%s, reason:%s", ginitDict[i].key, err); - taosMemoryFreeClear(err); + int8_t nCf = sizeof(ginitDict) / sizeof(ginitDict[0]); + char* err = NULL; + for (int i = 0; i < nCf; i++) { + if (wrapper->pCf[i] != NULL) rocksdb_flush_cf(wrapper->db, flushOpt, wrapper->pCf[i], &err); + if (err != NULL) { + qError("failed to flush cf:%s, reason:%s", ginitDict[i].key, err); + taosMemoryFreeClear(err); + } } rocksdb_flushoptions_destroy(flushOpt); - } - for (int i = 0; i < nCf; i++) { - if (wrapper->pCf[i] != NULL) { - rocksdb_column_family_handle_destroy(wrapper->pCf[i]); + + for (int i = 0; i < nCf; i++) { + if (wrapper->pCf[i] != NULL) { + rocksdb_column_family_handle_destroy(wrapper->pCf[i]); + } } } rocksdb_options_destroy(wrapper->dbOpt); @@ -1574,24 +1653,13 @@ void streamStateCloseTaskBackend(STaskBackendWrapper* wrapper) { taosThreadMutexDestroy(&wrapper->mutex); - rocksdb_close(wrapper->db); + if (wrapper->db) rocksdb_close(wrapper->db); taosMemoryFree(wrapper); return; } -int8_t getCfIdx(const char* key) { - int idx = -1; - size_t len = strlen(key); - for (int i = 0; i < sizeof(ginitDict) / sizeof(ginitDict[0]); i++) { - if (len == ginitDict[i].len && strncmp(key, ginitDict[i].key, strlen(key)) == 0) { - idx = i; - break; - } - } - return idx; -} -int32_t taskBackendOpenCf(STaskBackendWrapper* pBackend, const char* key) { +int32_t taskBackendOpenCfByKey(STaskBackendWrapper* pBackend, const char* key) { int32_t code = 0; char* err = NULL; int8_t idx = getCfIdx(key); @@ -1658,14 +1726,14 @@ int32_t streamStateConvertDataFormat(char* path, char* key, void* pCfInst) { int32_t code = 0; - STaskBackendWrapper* pTaskBackend = streamStateOpenTaskBackend(path, key); + STaskBackendWrapper* pTaskBackend = taskBackendOpen(path, key); RocksdbCfInst* pSrcBackend = pCfInst; for (int i = 0; i < nCf; i++) { rocksdb_column_family_handle_t* pSrcCf = pSrcBackend->pHandle[i]; if (pSrcCf == NULL) continue; - code = taskBackendOpenCf(pTaskBackend, ginitDict[i].key); + code = taskBackendOpenCfByKey(pTaskBackend, ginitDict[i].key); if (code != 0) goto _EXIT; code = copyDataAt(pSrcBackend, pTaskBackend, i); @@ -1673,7 +1741,7 @@ int32_t streamStateConvertDataFormat(char* path, char* key, void* pCfInst) { } _EXIT: - streamStateCloseTaskBackend(pTaskBackend); + taskBackendDestroy(pTaskBackend); return code; } diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 99b3b9cca1..2da4b2f370 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -224,11 +224,12 @@ void* streamMetaGetBackendByTaskKey(SStreamMeta* pMeta, char* key) { taosThreadMutexLock(&pMeta->backendMutex); void** ppBackend = taosHashGet(pMeta->pTaskBackendUnique, key, strlen(key)); if (ppBackend != NULL && *ppBackend != NULL) { - // add ref later + taskBackendAddRef(*ppBackend); taosThreadMutexUnlock(&pMeta->backendMutex); return *ppBackend; } - void* pBackend = streamStateOpenTaskBackend(pMeta->path, key); + void* pBackend = taskBackendOpen(pMeta->path, key); + taosHashPut(pMeta->pTaskBackendUnique, key, strlen(key), &pBackend, sizeof(void*)); taosThreadMutexLock(&pMeta->backendMutex); return pBackend; diff --git a/source/libs/stream/src/streamState.c b/source/libs/stream/src/streamState.c index f51181b0b4..d4ac1838d0 100644 --- a/source/libs/stream/src/streamState.c +++ b/source/libs/stream/src/streamState.c @@ -106,20 +106,12 @@ SStreamState* streamStateOpen(char* path, void* pTask, bool specPath, int32_t sz } SStreamTask* pStreamTask = pTask; - // char statePath[1024]; - // if (!specPath) { - // sprintf(statePath, "%s%s%d", path, TD_DIRSEP, pStreamTask->id.taskId); - // } else { - // memset(statePath, 0, 1024); - // tstrncpy(statePath, path, 1024); - // } - pState->taskId = pStreamTask->id.taskId; pState->streamId = pStreamTask->id.streamId; sprintf(pState->pTdbState->idstr, "0x%" PRIx64 "-%d", pState->streamId, pState->taskId); #ifdef USE_ROCKSDB - // SStreamMeta* pMeta = pStreamTask->pMeta; + SStreamMeta* pMeta = pStreamTask->pMeta; // pState->streamBackendRid = pMeta->streamBackendRid; // taosWLockLatch(&pMeta->lock); // taosThreadMutexLock(&pMeta->backendMutex); diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 9a76f661f8..e01f87788f 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -388,7 +388,7 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i taosThreadMutexInit(&pTask->lock, NULL); streamTaskOpenAllUpstreamInput(pTask); - //pTask->pBackend = streamStateOpenTaskBackend(pMeta->path, (char*)pTask->id.idStr); + // pTask->pBackend = taskBackendOpen(pMeta->path, (char*)pTask->id.idStr); return TSDB_CODE_SUCCESS; } From bb265887e6545e4179e0053919f1b3b6b0d24f5b Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 28 Sep 2023 17:07:19 +0800 Subject: [PATCH 011/195] refact task backend --- source/dnode/vnode/src/tq/tq.c | 1 - source/libs/stream/src/streamBackendRocksdb.c | 2 +- source/libs/stream/src/streamMeta.c | 5 ++++- 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 12b89dbd9c..e95114fcfb 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -990,7 +990,6 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms if (code < 0) { tqError("vgId:%d failed to add s-task:0x%x, total:%d, code:%s", vgId, taskId, numOfTasks, tstrerror(code)); - tFreeStreamTask(pTask); return -1; } diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index eed65d3f3a..6384d85eb6 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1565,7 +1565,7 @@ int32_t taskBackendBuildFullPath(char* path, char* key, char** fullPath) { char* taskPath = taosMemoryCalloc(1, strlen(path) + 128); sprintf(taskPath, "%s%s%s%s%s", path, TD_DIRSEP, "state", TD_DIRSEP, key); if (!taosDirExist(taskPath)) { - code = taosMkDir(taskPath); + code = taosMulMkDir(taskPath); if (code != 0) { qError("failed to create dir: %s, reason:%s", taskPath, tstrerror(code)); taosMemoryFree(taskPath); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 2da4b2f370..ad3ff82ec3 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -229,9 +229,12 @@ void* streamMetaGetBackendByTaskKey(SStreamMeta* pMeta, char* key) { return *ppBackend; } void* pBackend = taskBackendOpen(pMeta->path, key); + if (pBackend == NULL) { + taosThreadMutexUnlock(&pMeta->backendMutex); + return NULL; + } taosHashPut(pMeta->pTaskBackendUnique, key, strlen(key), &pBackend, sizeof(void*)); - taosThreadMutexLock(&pMeta->backendMutex); return pBackend; } SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandFunc, int32_t vgId, int64_t stage) { From c5bb37a5cbc1b50403603c6938f1cd6e52ced7c4 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 28 Sep 2023 17:30:49 +0800 Subject: [PATCH 012/195] refact task backend --- source/libs/stream/src/streamBackendRocksdb.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 6384d85eb6..76b9b7474c 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1604,9 +1604,9 @@ STaskBackendWrapper* taskBackendOpen(char* path, char* key) { if (cfs != NULL) rocksdb_list_column_families_destroy(cfs, nCf); taosThreadMutexInit(&pTaskBackend->mutex, NULL); - taosMemoryFree(taskPath); qDebug("succ to init stream backend at %s, backend:%p", taskPath, pTaskBackend); + taosMemoryFree(taskPath); pTaskBackend->refId = taosAddRef(streamBackendCfWrapperId, pTaskBackend); From 4f66dc538fdb476c2fbf482d2801a038abf7ee57 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 28 Sep 2023 11:52:18 +0000 Subject: [PATCH 013/195] add cvt state --- source/libs/stream/inc/streamBackendRocksdb.h | 4 +- source/libs/stream/src/streamBackendRocksdb.c | 253 +++++++++--------- 2 files changed, 127 insertions(+), 130 deletions(-) diff --git a/source/libs/stream/inc/streamBackendRocksdb.h b/source/libs/stream/inc/streamBackendRocksdb.h index a5533ef2f2..2c5eeb1fbe 100644 --- a/source/libs/stream/inc/streamBackendRocksdb.h +++ b/source/libs/stream/inc/streamBackendRocksdb.h @@ -64,8 +64,8 @@ typedef struct { rocksdb_compactionfilterfactory_t* filterFactory; TdThreadMutex mutex; - - int64_t refId; + char* idstr; + int64_t refId; } STaskBackendWrapper; diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 76b9b7474c..80b385d072 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -2000,26 +2000,25 @@ int streamStateGetCfIdx(SStreamState* pState, const char* funcName) { } } if (pState != NULL && idx != -1) { - SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; + STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; rocksdb_column_family_handle_t* cf = NULL; - taosThreadRwlockWrlock(&wrapper->rwLock); - cf = wrapper->pHandle[idx]; - if (cf == NULL) { - char buf[128] = {0}; - GEN_COLUMN_FAMILY_NAME(buf, wrapper->idstr, ginitDict[idx].key); - char* err = NULL; - cf = rocksdb_create_column_family(wrapper->rocksdb, wrapper->cfOpts[idx], buf, &err); + taosThreadMutexLock(&wrapper->mutex); + + cf = wrapper->pCf[idx]; + if (cf == NULL) { + char* err = NULL; + cf = rocksdb_create_column_family(wrapper->db, wrapper->pCfOpts[idx], ginitDict[idx].key, &err); if (err != NULL) { idx = -1; qError("failed to to open cf, %p %s_%s, reason:%s", pState, wrapper->idstr, funcName, err); taosMemoryFree(err); } else { qDebug("succ to to open cf, %p %s_%s", pState, wrapper->idstr, funcName); - wrapper->pHandle[idx] = cf; + wrapper->pCf[idx] = cf; } } - taosThreadRwlockUnlock(&wrapper->rwLock); + taosThreadMutexUnlock(&wrapper->mutex); } return idx; @@ -2040,15 +2039,14 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfKe *readOpt = rocksdb_readoptions_create(); - SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; + STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; if (snapshot != NULL) { - *snapshot = (rocksdb_snapshot_t*)rocksdb_create_snapshot(wrapper->rocksdb); + *snapshot = (rocksdb_snapshot_t*)rocksdb_create_snapshot(wrapper->db); rocksdb_readoptions_set_snapshot(*readOpt, *snapshot); rocksdb_readoptions_set_fill_cache(*readOpt, 0); } - return rocksdb_create_iterator_cf(wrapper->rocksdb, *readOpt, - ((rocksdb_column_family_handle_t**)wrapper->pHandle)[idx]); + return rocksdb_create_iterator_cf(wrapper->db, *readOpt, ((rocksdb_column_family_handle_t**)wrapper->pCf)[idx]); } #define STREAM_STATE_PUT_ROCKSDB(pState, funcname, key, value, vLen) \ @@ -2062,13 +2060,13 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfKe code = -1; \ break; \ } \ - SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; \ - char toString[128] = {0}; \ + STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; \ + char toString[128] = {0}; \ if (qDebugFlag & DEBUG_TRACE) ginitDict[i].toStrFunc((void*)key, toString); \ int32_t klen = ginitDict[i].enFunc((void*)key, buf); \ - rocksdb_column_family_handle_t* pHandle = ((rocksdb_column_family_handle_t**)wrapper->pHandle)[ginitDict[i].idx]; \ - rocksdb_t* db = wrapper->rocksdb; \ - rocksdb_writeoptions_t* opts = wrapper->writeOpts; \ + rocksdb_column_family_handle_t* pHandle = ((rocksdb_column_family_handle_t**)wrapper->pCf)[ginitDict[i].idx]; \ + rocksdb_t* db = wrapper->db; \ + rocksdb_writeoptions_t* opts = wrapper->writeOpt; \ char* ttlV = NULL; \ int32_t ttlVLen = ginitDict[i].enValueFunc((char*)value, vLen, 0, &ttlV); \ rocksdb_put_cf(db, opts, pHandle, (const char*)buf, klen, (const char*)ttlV, (size_t)ttlVLen, &err); \ @@ -2082,75 +2080,75 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfKe taosMemoryFree(ttlV); \ } while (0); -#define STREAM_STATE_GET_ROCKSDB(pState, funcname, key, pVal, vLen) \ - do { \ - code = 0; \ - char buf[128] = {0}; \ - char* err = NULL; \ - int i = streamStateGetCfIdx(pState, funcname); \ - if (i < 0) { \ - qWarn("streamState failed to get cf name: %s", funcname); \ - code = -1; \ - break; \ - } \ - SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; \ - char toString[128] = {0}; \ - if (qDebugFlag & DEBUG_TRACE) ginitDict[i].toStrFunc((void*)key, toString); \ - int32_t klen = ginitDict[i].enFunc((void*)key, buf); \ - rocksdb_column_family_handle_t* pHandle = ((rocksdb_column_family_handle_t**)wrapper->pHandle)[ginitDict[i].idx]; \ - rocksdb_t* db = wrapper->rocksdb; \ - rocksdb_readoptions_t* opts = wrapper->readOpts; \ - size_t len = 0; \ - char* val = rocksdb_get_cf(db, opts, pHandle, (const char*)buf, klen, (size_t*)&len, &err); \ - if (val == NULL || len == 0) { \ - if (err == NULL) { \ - qTrace("streamState str: %s failed to read from %s_%s, err: not exist", toString, wrapper->idstr, funcname); \ - } else { \ - qError("streamState str: %s failed to read from %s_%s, err: %s", toString, wrapper->idstr, funcname, err); \ - taosMemoryFreeClear(err); \ - } \ - code = -1; \ - } else { \ - char* p = NULL; \ - int32_t tlen = ginitDict[i].deValueFunc(val, len, NULL, (char**)pVal); \ - if (tlen <= 0) { \ - qError("streamState str: %s failed to read from %s_%s, err: already ttl ", toString, wrapper->idstr, \ - funcname); \ - code = -1; \ - } else { \ - qTrace("streamState str: %s succ to read from %s_%s, valLen:%d", toString, wrapper->idstr, funcname, tlen); \ - } \ - taosMemoryFree(val); \ - if (vLen != NULL) *vLen = tlen; \ - } \ +#define STREAM_STATE_GET_ROCKSDB(pState, funcname, key, pVal, vLen) \ + do { \ + code = 0; \ + char buf[128] = {0}; \ + char* err = NULL; \ + int i = streamStateGetCfIdx(pState, funcname); \ + if (i < 0) { \ + qWarn("streamState failed to get cf name: %s", funcname); \ + code = -1; \ + break; \ + } \ + STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; \ + char toString[128] = {0}; \ + if (qDebugFlag & DEBUG_TRACE) ginitDict[i].toStrFunc((void*)key, toString); \ + int32_t klen = ginitDict[i].enFunc((void*)key, buf); \ + rocksdb_column_family_handle_t* pHandle = ((rocksdb_column_family_handle_t**)wrapper->pCf)[ginitDict[i].idx]; \ + rocksdb_t* db = wrapper->db; \ + rocksdb_readoptions_t* opts = wrapper->readOpt; \ + size_t len = 0; \ + char* val = rocksdb_get_cf(db, opts, pHandle, (const char*)buf, klen, (size_t*)&len, &err); \ + if (val == NULL || len == 0) { \ + if (err == NULL) { \ + qTrace("streamState str: %s failed to read from %s_%s, err: not exist", toString, wrapper->idstr, funcname); \ + } else { \ + qError("streamState str: %s failed to read from %s_%s, err: %s", toString, wrapper->idstr, funcname, err); \ + taosMemoryFreeClear(err); \ + } \ + code = -1; \ + } else { \ + char* p = NULL; \ + int32_t tlen = ginitDict[i].deValueFunc(val, len, NULL, (char**)pVal); \ + if (tlen <= 0) { \ + qError("streamState str: %s failed to read from %s_%s, err: already ttl ", toString, wrapper->idstr, \ + funcname); \ + code = -1; \ + } else { \ + qTrace("streamState str: %s succ to read from %s_%s, valLen:%d", toString, wrapper->idstr, funcname, tlen); \ + } \ + taosMemoryFree(val); \ + if (vLen != NULL) *vLen = tlen; \ + } \ } while (0); -#define STREAM_STATE_DEL_ROCKSDB(pState, funcname, key) \ - do { \ - code = 0; \ - char buf[128] = {0}; \ - char* err = NULL; \ - int i = streamStateGetCfIdx(pState, funcname); \ - if (i < 0) { \ - qWarn("streamState failed to get cf name: %s_%s", pState->pTdbState->idstr, funcname); \ - code = -1; \ - break; \ - } \ - SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; \ - char toString[128] = {0}; \ - if (qDebugFlag & DEBUG_TRACE) ginitDict[i].toStrFunc((void*)key, toString); \ - int32_t klen = ginitDict[i].enFunc((void*)key, buf); \ - rocksdb_column_family_handle_t* pHandle = ((rocksdb_column_family_handle_t**)wrapper->pHandle)[ginitDict[i].idx]; \ - rocksdb_t* db = wrapper->rocksdb; \ - rocksdb_writeoptions_t* opts = wrapper->writeOpts; \ - rocksdb_delete_cf(db, opts, pHandle, (const char*)buf, klen, &err); \ - if (err != NULL) { \ - qError("streamState str: %s failed to del from %s_%s, err: %s", toString, wrapper->idstr, funcname, err); \ - taosMemoryFree(err); \ - code = -1; \ - } else { \ - qTrace("streamState str: %s succ to del from %s_%s", toString, wrapper->idstr, funcname); \ - } \ +#define STREAM_STATE_DEL_ROCKSDB(pState, funcname, key) \ + do { \ + code = 0; \ + char buf[128] = {0}; \ + char* err = NULL; \ + int i = streamStateGetCfIdx(pState, funcname); \ + if (i < 0) { \ + qWarn("streamState failed to get cf name: %s_%s", pState->pTdbState->idstr, funcname); \ + code = -1; \ + break; \ + } \ + STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; \ + char toString[128] = {0}; \ + if (qDebugFlag & DEBUG_TRACE) ginitDict[i].toStrFunc((void*)key, toString); \ + int32_t klen = ginitDict[i].enFunc((void*)key, buf); \ + rocksdb_column_family_handle_t* pHandle = ((rocksdb_column_family_handle_t**)wrapper->pCf)[ginitDict[i].idx]; \ + rocksdb_t* db = wrapper->db; \ + rocksdb_writeoptions_t* opts = wrapper->writeOpt; \ + rocksdb_delete_cf(db, opts, pHandle, (const char*)buf, klen, &err); \ + if (err != NULL) { \ + qError("streamState str: %s failed to del from %s_%s, err: %s", toString, wrapper->idstr, funcname, err); \ + taosMemoryFree(err); \ + code = -1; \ + } else { \ + qTrace("streamState str: %s succ to del from %s_%s", toString, wrapper->idstr, funcname); \ + } \ } while (0); // state cf @@ -2176,7 +2174,7 @@ int32_t streamStateDel_rocksdb(SStreamState* pState, const SWinKey* key) { int32_t streamStateClear_rocksdb(SStreamState* pState) { qDebug("streamStateClear_rocksdb"); - SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; + STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; char sKeyStr[128] = {0}; char eKeyStr[128] = {0}; @@ -2186,10 +2184,9 @@ int32_t streamStateClear_rocksdb(SStreamState* pState) { int sLen = stateKeyEncode(&sKey, sKeyStr); int eLen = stateKeyEncode(&eKey, eKeyStr); - if (wrapper->pHandle[1] != NULL) { + if (wrapper->pCf[1] != NULL) { char* err = NULL; - rocksdb_delete_range_cf(wrapper->rocksdb, wrapper->writeOpts, wrapper->pHandle[1], sKeyStr, sLen, eKeyStr, eLen, - &err); + rocksdb_delete_range_cf(wrapper->db, wrapper->writeOpt, wrapper->pCf[1], sKeyStr, sLen, eKeyStr, eLen, &err); if (err != NULL) { char toStringStart[128] = {0}; char toStringEnd[128] = {0}; @@ -2199,7 +2196,7 @@ int32_t streamStateClear_rocksdb(SStreamState* pState) { qWarn("failed to delete range cf(state) start: %s, end:%s, reason:%s", toStringStart, toStringEnd, err); taosMemoryFree(err); } else { - rocksdb_compact_range_cf(wrapper->rocksdb, wrapper->pHandle[1], sKeyStr, sLen, eKeyStr, eLen); + rocksdb_compact_range_cf(wrapper->db, wrapper->pCf[1], sKeyStr, sLen, eKeyStr, eLen); } } @@ -2300,9 +2297,9 @@ SStreamStateCur* streamStateSeekKeyNext_rocksdb(SStreamState* pState, const SWin if (pCur == NULL) { return NULL; } - SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; + STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; pCur->number = pState->number; - pCur->db = wrapper->rocksdb; + pCur->db = wrapper->db; pCur->iter = streamStateIterCreate(pState, "state", (rocksdb_snapshot_t**)&pCur->snapshot, (rocksdb_readoptions_t**)&pCur->readOpt); @@ -2356,7 +2353,7 @@ SStreamStateCur* streamStateSeekToLast_rocksdb(SStreamState* pState, const SWinK if (pCur == NULL) return NULL; pCur->number = pState->number; - pCur->db = ((SBackendCfWrapper*)pState->pTdbState->pBackendCfWrapper)->rocksdb; + pCur->db = ((STaskBackendWrapper*)pState->pTdbState->pOwner->pBackend)->db; pCur->iter = streamStateIterCreate(pState, "state", (rocksdb_snapshot_t**)&pCur->snapshot, (rocksdb_readoptions_t**)&pCur->readOpt); @@ -2467,13 +2464,13 @@ int32_t streamStateSessionDel_rocksdb(SStreamState* pState, const SSessionKey* k SStreamStateCur* streamStateSessionSeekKeyCurrentPrev_rocksdb(SStreamState* pState, const SSessionKey* key) { qDebug("streamStateSessionSeekKeyCurrentPrev_rocksdb"); - SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) { return NULL; } pCur->number = pState->number; - pCur->db = wrapper->rocksdb; + pCur->db = wrapper->db; pCur->iter = streamStateIterCreate(pState, "sess", (rocksdb_snapshot_t**)&pCur->snapshot, (rocksdb_readoptions_t**)&pCur->readOpt); @@ -2507,12 +2504,12 @@ SStreamStateCur* streamStateSessionSeekKeyCurrentPrev_rocksdb(SStreamState* pSta } SStreamStateCur* streamStateSessionSeekKeyCurrentNext_rocksdb(SStreamState* pState, SSessionKey* key) { qDebug("streamStateSessionSeekKeyCurrentNext_rocksdb"); - SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) { return NULL; } - pCur->db = wrapper->rocksdb; + pCur->db = wrapper->db; pCur->iter = streamStateIterCreate(pState, "sess", (rocksdb_snapshot_t**)&pCur->snapshot, (rocksdb_readoptions_t**)&pCur->readOpt); pCur->number = pState->number; @@ -2545,12 +2542,12 @@ SStreamStateCur* streamStateSessionSeekKeyCurrentNext_rocksdb(SStreamState* pSta SStreamStateCur* streamStateSessionSeekKeyNext_rocksdb(SStreamState* pState, const SSessionKey* key) { qDebug("streamStateSessionSeekKeyNext_rocksdb"); - SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) { return NULL; } - pCur->db = wrapper->rocksdb; + pCur->db = wrapper->db; pCur->iter = streamStateIterCreate(pState, "sess", (rocksdb_snapshot_t**)&pCur->snapshot, (rocksdb_readoptions_t**)&pCur->readOpt); pCur->number = pState->number; @@ -2647,12 +2644,12 @@ int32_t streamStateFillDel_rocksdb(SStreamState* pState, const SWinKey* key) { SStreamStateCur* streamStateFillGetCur_rocksdb(SStreamState* pState, const SWinKey* key) { qDebug("streamStateFillGetCur_rocksdb"); - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); - SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; + SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; if (pCur == NULL) return NULL; - pCur->db = wrapper->rocksdb; + pCur->db = wrapper->db; pCur->iter = streamStateIterCreate(pState, "fill", (rocksdb_snapshot_t**)&pCur->snapshot, (rocksdb_readoptions_t**)&pCur->readOpt); pCur->number = pState->number; @@ -2707,13 +2704,13 @@ int32_t streamStateFillGetKVByCur_rocksdb(SStreamStateCur* pCur, SWinKey* pKey, SStreamStateCur* streamStateFillSeekKeyNext_rocksdb(SStreamState* pState, const SWinKey* key) { qDebug("streamStateFillSeekKeyNext_rocksdb"); - SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (!pCur) { return NULL; } - pCur->db = wrapper->rocksdb; + pCur->db = wrapper->db; pCur->iter = streamStateIterCreate(pState, "fill", (rocksdb_snapshot_t**)&pCur->snapshot, (rocksdb_readoptions_t**)&pCur->readOpt); pCur->number = pState->number; @@ -2745,13 +2742,13 @@ SStreamStateCur* streamStateFillSeekKeyNext_rocksdb(SStreamState* pState, const } SStreamStateCur* streamStateFillSeekKeyPrev_rocksdb(SStreamState* pState, const SWinKey* key) { qDebug("streamStateFillSeekKeyPrev_rocksdb"); - SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) { return NULL; } - pCur->db = wrapper->rocksdb; + pCur->db = wrapper->db; pCur->iter = streamStateIterCreate(pState, "fill", (rocksdb_snapshot_t**)&pCur->snapshot, (rocksdb_readoptions_t**)&pCur->readOpt); pCur->number = pState->number; @@ -2783,12 +2780,12 @@ SStreamStateCur* streamStateFillSeekKeyPrev_rocksdb(SStreamState* pState, const } int32_t streamStateSessionGetKeyByRange_rocksdb(SStreamState* pState, const SSessionKey* key, SSessionKey* curKey) { qDebug("streamStateSessionGetKeyByRange_rocksdb"); - SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) { return -1; } - pCur->db = wrapper->rocksdb; + pCur->db = wrapper->db; pCur->iter = streamStateIterCreate(pState, "sess", (rocksdb_snapshot_t**)&pCur->snapshot, (rocksdb_readoptions_t**)&pCur->readOpt); pCur->number = pState->number; @@ -3015,7 +3012,7 @@ int32_t streamDefaultIterGet_rocksdb(SStreamState* pState, const void* start, co int code = 0; char* err = NULL; - SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; + STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; rocksdb_snapshot_t* snapshot = NULL; rocksdb_readoptions_t* readopts = NULL; rocksdb_iterator_t* pIter = streamStateIterCreate(pState, "default", &snapshot, &readopts); @@ -3048,16 +3045,16 @@ int32_t streamDefaultIterGet_rocksdb(SStreamState* pState, const void* start, co } rocksdb_iter_next(pIter); } - rocksdb_release_snapshot(wrapper->rocksdb, snapshot); + rocksdb_release_snapshot(wrapper->db, snapshot); rocksdb_readoptions_destroy(readopts); rocksdb_iter_destroy(pIter); return code; } void* streamDefaultIterCreate_rocksdb(SStreamState* pState) { - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); - SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; + SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; - pCur->db = wrapper->rocksdb; + pCur->db = wrapper->db; pCur->iter = streamStateIterCreate(pState, "default", (rocksdb_snapshot_t**)&pCur->snapshot, (rocksdb_readoptions_t**)&pCur->readOpt); pCur->number = pState->number; @@ -3106,7 +3103,7 @@ void streamStateClearBatch(void* pBatch) { rocksdb_writebatch_clear((rocksdb_ void streamStateDestroyBatch(void* pBatch) { rocksdb_writebatch_destroy((rocksdb_writebatch_t*)pBatch); } int32_t streamStatePutBatch(SStreamState* pState, const char* cfKeyName, rocksdb_writebatch_t* pBatch, void* key, void* val, int32_t vlen, int64_t ttl) { - SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; + STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; int i = streamStateGetCfIdx(pState, cfKeyName); if (i < 0) { @@ -3120,7 +3117,7 @@ int32_t streamStatePutBatch(SStreamState* pState, const char* cfKeyName, rocksdb char* ttlV = NULL; int32_t ttlVLen = ginitDict[i].enValueFunc(val, vlen, ttl, &ttlV); - rocksdb_column_family_handle_t* pCf = wrapper->pHandle[ginitDict[i].idx]; + rocksdb_column_family_handle_t* pCf = wrapper->pCf[ginitDict[i].idx]; rocksdb_writebatch_put_cf((rocksdb_writebatch_t*)pBatch, pCf, buf, (size_t)klen, ttlV, (size_t)ttlVLen); taosMemoryFree(ttlV); @@ -3139,8 +3136,8 @@ int32_t streamStatePutBatchOptimize(SStreamState* pState, int32_t cfIdx, rocksdb char* ttlV = tmpBuf; int32_t ttlVLen = ginitDict[cfIdx].enValueFunc(val, vlen, ttl, &ttlV); - SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; - rocksdb_column_family_handle_t* pCf = wrapper->pHandle[ginitDict[cfIdx].idx]; + STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + rocksdb_column_family_handle_t* pCf = wrapper->pCf[ginitDict[cfIdx].idx]; rocksdb_writebatch_put_cf((rocksdb_writebatch_t*)pBatch, pCf, buf, (size_t)klen, ttlV, (size_t)ttlVLen); if (tmpBuf == NULL) { @@ -3155,15 +3152,15 @@ int32_t streamStatePutBatchOptimize(SStreamState* pState, int32_t cfIdx, rocksdb return 0; } int32_t streamStatePutBatch_rocksdb(SStreamState* pState, void* pBatch) { - char* err = NULL; - SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; - rocksdb_write(wrapper->rocksdb, wrapper->writeOpts, (rocksdb_writebatch_t*)pBatch, &err); + char* err = NULL; + STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + rocksdb_write(wrapper->db, wrapper->writeOpt, (rocksdb_writebatch_t*)pBatch, &err); if (err != NULL) { qError("streamState failed to write batch, err:%s", err); taosMemoryFree(err); return -1; } else { - qDebug("write batch to backend:%p", wrapper->pBackend); + qDebug("write batch to backend:%p", wrapper->db); } return 0; } From eac16b9d1b12e4cda30ad9da0f98c4ac066f4278 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 28 Sep 2023 11:59:53 +0000 Subject: [PATCH 014/195] add cvt state --- source/libs/stream/src/streamBackendRocksdb.c | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 80b385d072..ecf93c577e 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -2374,12 +2374,12 @@ SStreamStateCur* streamStateSeekToLast_rocksdb(SStreamState* pState, const SWinK SStreamStateCur* streamStateGetCur_rocksdb(SStreamState* pState, const SWinKey* key) { qDebug("streamStateGetCur_rocksdb"); - SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; + STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) return NULL; - pCur->db = wrapper->rocksdb; + pCur->db = wrapper->db; pCur->iter = streamStateIterCreate(pState, "state", (rocksdb_snapshot_t**)&pCur->snapshot, (rocksdb_readoptions_t**)&pCur->readOpt); pCur->number = pState->number; @@ -2469,6 +2469,7 @@ SStreamStateCur* streamStateSessionSeekKeyCurrentPrev_rocksdb(SStreamState* pSta if (pCur == NULL) { return NULL; } + pCur->number = pState->number; pCur->db = wrapper->db; pCur->iter = streamStateIterCreate(pState, "sess", (rocksdb_snapshot_t**)&pCur->snapshot, From b8e265e502ed6357acaaa59f674835bd3041e29c Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Sat, 30 Sep 2023 09:05:28 +0000 Subject: [PATCH 015/195] add cvt state --- source/libs/stream/src/streamBackendRocksdb.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index ecf93c577e..0d34970cb3 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -2065,8 +2065,8 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfKe if (qDebugFlag & DEBUG_TRACE) ginitDict[i].toStrFunc((void*)key, toString); \ int32_t klen = ginitDict[i].enFunc((void*)key, buf); \ rocksdb_column_family_handle_t* pHandle = ((rocksdb_column_family_handle_t**)wrapper->pCf)[ginitDict[i].idx]; \ - rocksdb_t* db = wrapper->db; \ rocksdb_writeoptions_t* opts = wrapper->writeOpt; \ + rocksdb_t* db = wrapper->db; \ char* ttlV = NULL; \ int32_t ttlVLen = ginitDict[i].enValueFunc((char*)value, vLen, 0, &ttlV); \ rocksdb_put_cf(db, opts, pHandle, (const char*)buf, klen, (const char*)ttlV, (size_t)ttlVLen, &err); \ From fa702212382d2a3edb0abd9f8b1597191b5f0826 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Sat, 7 Oct 2023 12:17:55 +0800 Subject: [PATCH 016/195] refact task backend --- include/libs/stream/tstream.h | 2 +- source/dnode/vnode/src/tq/tq.c | 2 +- source/libs/stream/inc/streamBackendRocksdb.h | 23 +- source/libs/stream/inc/streamInt.h | 12 +- source/libs/stream/src/streamBackendRocksdb.c | 381 +++++++++--------- source/libs/stream/src/streamMeta.c | 7 +- source/libs/stream/src/streamTask.c | 6 +- 7 files changed, 230 insertions(+), 203 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 553e8f8217..9bc4a91f1c 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -726,7 +726,7 @@ int32_t streamMetaReopen(SStreamMeta* pMeta, int64_t chkpId); int32_t streamMetaCommit(SStreamMeta* pMeta); int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta); void streamMetaNotifyClose(SStreamMeta* pMeta); -void* streamMetaGetBackendByTaskKey(SStreamMeta* pMeta, char* key); +void* streamMetaGetBackendByTaskKey(SStreamMeta* pMeta, char* key, int64_t* ref); // checkpoint int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSourceReq* pReq); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index e95114fcfb..7cfe776378 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -738,7 +738,7 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { int32_t code = streamTaskInit(pTask, pTq->pStreamMeta, &pTq->pVnode->msgCb, ver); if (code != TSDB_CODE_SUCCESS) return code; - pTask->pBackend = streamMetaGetBackendByTaskKey(pTq->pStreamMeta, (char*)pTask->id.idStr); + pTask->pBackend = streamMetaGetBackendByTaskKey(pTq->pStreamMeta, (char*)pTask->id.idStr, &pTask->backendRefId); if (pTask->pBackend == NULL) return -1; streamTaskOpenAllUpstreamInput(pTask); diff --git a/source/libs/stream/inc/streamBackendRocksdb.h b/source/libs/stream/inc/streamBackendRocksdb.h index 2c5eeb1fbe..f5c303b809 100644 --- a/source/libs/stream/inc/streamBackendRocksdb.h +++ b/source/libs/stream/inc/streamBackendRocksdb.h @@ -69,17 +69,20 @@ typedef struct { } STaskBackendWrapper; -void* streamBackendInit(const char* path, int64_t chkpId); -void streamBackendCleanup(void* arg); -void streamBackendHandleCleanup(void* arg); -int32_t streamBackendLoadCheckpointInfo(void* pMeta); -int32_t streamBackendDoCheckpoint(void* pMeta, uint64_t checkpointId); -SListNode* streamBackendAddCompare(void* backend, void* arg); -void streamBackendDelCompare(void* backend, void* arg); -int32_t streamStateConvertDataFormat(char* path, char* key, void* cfInst); -STaskBackendWrapper* taskBackendOpen(char* path, char* key); +void* streamBackendInit(const char* path, int64_t chkpId); +void streamBackendCleanup(void* arg); +void streamBackendHandleCleanup(void* arg); +int32_t streamBackendLoadCheckpointInfo(void* pMeta); +int32_t streamBackendDoCheckpoint(void* pMeta, uint64_t checkpointId); +SListNode* streamBackendAddCompare(void* backend, void* arg); +void streamBackendDelCompare(void* backend, void* arg); +int32_t streamStateConvertDataFormat(char* path, char* key, void* cfInst); -void taskBackendAddRef(void* pTaskBackend); +STaskBackendWrapper* taskBackendOpen(char* path, char* key); +void taskBackendDestroy(void* pBackend); + +void* taskBackendAddRef(void* pTaskBackend); +void taskBackendRemoveRef(void* pTaskBackend); int streamStateOpenBackend(void* backend, SStreamState* pState); void streamStateCloseBackend(SStreamState* pState, bool remove); diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index bbb7595e5a..54aefc8962 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -18,9 +18,9 @@ #include "executor.h" #include "query.h" -#include "tstream.h" #include "streamBackendRocksdb.h" #include "trpc.h" +#include "tstream.h" #ifdef __cplusplus extern "C" { @@ -41,8 +41,9 @@ typedef struct { } SStreamContinueExecInfo; extern SStreamGlobalEnv streamEnv; -extern int32_t streamBackendId; -extern int32_t streamBackendCfWrapperId; +extern int32_t streamBackendId; +extern int32_t streamBackendCfWrapperId; +extern int32_t taskBackendWrapperId; const char* streamGetBlockTypeStr(int32_t type); void streamRetryDispatchStreamBlock(SStreamTask* pTask, int64_t waitDuration); @@ -68,10 +69,11 @@ int32_t streamTaskSendCheckpointReadyMsg(SStreamTask* pTask); int32_t streamTaskSendCheckpointSourceRsp(SStreamTask* pTask); int32_t streamTaskGetNumOfDownstream(const SStreamTask* pTask); -int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInput, int32_t* numOfBlocks); +int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInput, int32_t* numOfBlocks); SStreamQueueItem* streamMergeQueueItem(SStreamQueueItem* dst, SStreamQueueItem* pElem); -int32_t streamTaskBuildScanhistoryRspMsg(SStreamTask* pTask, SStreamScanHistoryFinishReq* pReq, void** pBuffer, int32_t* pLen); +int32_t streamTaskBuildScanhistoryRspMsg(SStreamTask* pTask, SStreamScanHistoryFinishReq* pReq, void** pBuffer, + int32_t* pLen); int32_t streamAddEndScanHistoryMsg(SStreamTask* pTask, SRpcHandleInfo* pRpcInfo, SStreamScanHistoryFinishReq* pReq); int32_t streamNotifyUpstreamContinue(SStreamTask* pTask); int32_t streamTaskFillHistoryFinished(SStreamTask* pTask); diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 0d34970cb3..5589932fb8 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -770,39 +770,40 @@ int32_t streamBackendLoadCheckpointInfo(void* arg) { } int32_t chkpGetAllDbCfHandle(SStreamMeta* pMeta, rocksdb_column_family_handle_t*** ppHandle, SArray* refs) { - SArray* pHandle = taosArrayInit(16, POINTER_BYTES); - void* pIter = taosHashIterate(pMeta->pTaskBackendUnique, NULL); - while (pIter) { - int64_t id = *(int64_t*)pIter; + return 0; + // SArray* pHandle = taosArrayInit(16, POINTER_BYTES); + // void* pIter = taosHashIterate(pMeta->pTaskBackendUnique, NULL); + // while (pIter) { + // int64_t id = *(int64_t*)pIter; - SBackendCfWrapper* wrapper = taosAcquireRef(streamBackendCfWrapperId, id); - if (wrapper == NULL) { - pIter = taosHashIterate(pMeta->pTaskBackendUnique, pIter); - continue; - } + // SBackendCfWrapper* wrapper = taosAcquireRef(streamBackendCfWrapperId, id); + // if (wrapper == NULL) { + // pIter = taosHashIterate(pMeta->pTaskBackendUnique, pIter); + // continue; + // } - taosThreadRwlockRdlock(&wrapper->rwLock); - for (int i = 0; i < sizeof(ginitDict) / sizeof(ginitDict[0]); i++) { - if (wrapper->pHandle[i]) { - rocksdb_column_family_handle_t* p = wrapper->pHandle[i]; - taosArrayPush(pHandle, &p); - } - } - taosThreadRwlockUnlock(&wrapper->rwLock); + // taosThreadRwlockRdlock(&wrapper->rwLock); + // for (int i = 0; i < sizeof(ginitDict) / sizeof(ginitDict[0]); i++) { + // if (wrapper->pHandle[i]) { + // rocksdb_column_family_handle_t* p = wrapper->pHandle[i]; + // taosArrayPush(pHandle, &p); + // } + // } + // taosThreadRwlockUnlock(&wrapper->rwLock); - taosArrayPush(refs, &id); - } + // taosArrayPush(refs, &id); + // } - int32_t nCf = taosArrayGetSize(pHandle); + // int32_t nCf = taosArrayGetSize(pHandle); - rocksdb_column_family_handle_t** ppCf = taosMemoryCalloc(nCf, sizeof(rocksdb_column_family_handle_t*)); - for (int i = 0; i < nCf; i++) { - ppCf[i] = taosArrayGetP(pHandle, i); - } - taosArrayDestroy(pHandle); + // rocksdb_column_family_handle_t** ppCf = taosMemoryCalloc(nCf, sizeof(rocksdb_column_family_handle_t*)); + // for (int i = 0; i < nCf; i++) { + // ppCf[i] = taosArrayGetP(pHandle, i); + // } + // taosArrayDestroy(pHandle); - *ppHandle = ppCf; - return nCf; + // *ppHandle = ppCf; + // return nCf; } int32_t chkpDoDbCheckpoint(rocksdb_t* db, char* path) { int32_t code = -1; @@ -826,172 +827,176 @@ _ERROR: return code; } int32_t chkpPreFlushDb(rocksdb_t* db, rocksdb_column_family_handle_t** cf, int32_t nCf) { - int code = 0; - char* err = NULL; + return 0; + // int code = 0; + // char* err = NULL; - rocksdb_flushoptions_t* flushOpt = rocksdb_flushoptions_create(); - rocksdb_flushoptions_set_wait(flushOpt, 1); + // rocksdb_flushoptions_t* flushOpt = rocksdb_flushoptions_create(); + // rocksdb_flushoptions_set_wait(flushOpt, 1); - rocksdb_flush_cfs(db, flushOpt, cf, nCf, &err); - if (err != NULL) { - qError("failed to flush db before streamBackend clean up, reason:%s", err); - taosMemoryFree(err); - code = -1; - } - rocksdb_flushoptions_destroy(flushOpt); - return code; + // rocksdb_flush_cfs(db, flushOpt, cf, nCf, &err); + // if (err != NULL) { + // qError("failed to flush db before streamBackend clean up, reason:%s", err); + // taosMemoryFree(err); + // code = -1; + // } + // rocksdb_flushoptions_destroy(flushOpt); + // return code; } int32_t chkpPreCheckDir(char* path, int64_t chkpId, char** chkpDir, char** chkpIdDir) { - int32_t code = 0; - char* pChkpDir = taosMemoryCalloc(1, 256); - char* pChkpIdDir = taosMemoryCalloc(1, 256); - - sprintf(pChkpDir, "%s%s%s", path, TD_DIRSEP, "checkpoints"); - code = taosMulModeMkDir(pChkpDir, 0755, true); - if (code != 0) { - qError("failed to prepare checkpoint dir, path:%s, reason:%s", path, tstrerror(code)); - taosMemoryFree(pChkpDir); - taosMemoryFree(pChkpIdDir); - code = -1; - return code; - } - - sprintf(pChkpIdDir, "%s%scheckpoint%" PRId64, pChkpDir, TD_DIRSEP, chkpId); - if (taosIsDir(pChkpIdDir)) { - qInfo("stream rm exist checkpoint%s", pChkpIdDir); - taosRemoveFile(pChkpIdDir); - } - *chkpDir = pChkpDir; - *chkpIdDir = pChkpIdDir; - return 0; + // int32_t code = 0; + // char* pChkpDir = taosMemoryCalloc(1, 256); + // char* pChkpIdDir = taosMemoryCalloc(1, 256); + + // sprintf(pChkpDir, "%s%s%s", path, TD_DIRSEP, "checkpoints"); + // code = taosMulModeMkDir(pChkpDir, 0755, true); + // if (code != 0) { + // qError("failed to prepare checkpoint dir, path:%s, reason:%s", path, tstrerror(code)); + // taosMemoryFree(pChkpDir); + // taosMemoryFree(pChkpIdDir); + // code = -1; + // return code; + // } + + // sprintf(pChkpIdDir, "%s%scheckpoint%" PRId64, pChkpDir, TD_DIRSEP, chkpId); + // if (taosIsDir(pChkpIdDir)) { + // qInfo("stream rm exist checkpoint%s", pChkpIdDir); + // taosRemoveFile(pChkpIdDir); + // } + // *chkpDir = pChkpDir; + // *chkpIdDir = pChkpIdDir; + + // return 0; } int32_t streamBackendTriggerChkp(void* arg, char* dst) { - SStreamMeta* pMeta = arg; - int64_t backendRid = pMeta->streamBackendRid; - int32_t code = -1; + return 0; + // SStreamMeta* pMeta = arg; + // int64_t backendRid = pMeta->streamBackendRid; + // int32_t code = -1; - SArray* refs = taosArrayInit(16, sizeof(int64_t)); - rocksdb_column_family_handle_t** ppCf = NULL; + // SArray* refs = taosArrayInit(16, sizeof(int64_t)); + // rocksdb_column_family_handle_t** ppCf = NULL; - int64_t st = taosGetTimestampMs(); - SBackendWrapper* pHandle = taosAcquireRef(streamBackendId, backendRid); + // int64_t st = taosGetTimestampMs(); + // SBackendWrapper* pHandle = taosAcquireRef(streamBackendId, backendRid); - if (pHandle == NULL || pHandle->db == NULL) { - goto _ERROR; - } - int32_t nCf = chkpGetAllDbCfHandle(pMeta, &ppCf, refs); - qDebug("stream backend:%p start to do checkpoint at:%s, cf num: %d ", pHandle, dst, nCf); + // if (pHandle == NULL || pHandle->db == NULL) { + // goto _ERROR; + // } + // int32_t nCf = chkpGetAllDbCfHandle(pMeta, &ppCf, refs); + // qDebug("stream backend:%p start to do checkpoint at:%s, cf num: %d ", pHandle, dst, nCf); - code = chkpPreFlushDb(pHandle->db, ppCf, nCf); - if (code == 0) { - code = chkpDoDbCheckpoint(pHandle->db, dst); - if (code != 0) { - qError("stream backend:%p failed to do checkpoint at:%s", pHandle, dst); - } else { - qDebug("stream backend:%p end to do checkpoint at:%s, time cost:%" PRId64 "ms", pHandle, dst, - taosGetTimestampMs() - st); - } - } else { - qError("stream backend:%p failed to flush db at:%s", pHandle, dst); - } + // code = chkpPreFlushDb(pHandle->db, ppCf, nCf); + // if (code == 0) { + // code = chkpDoDbCheckpoint(pHandle->db, dst); + // if (code != 0) { + // qError("stream backend:%p failed to do checkpoint at:%s", pHandle, dst); + // } else { + // qDebug("stream backend:%p end to do checkpoint at:%s, time cost:%" PRId64 "ms", pHandle, dst, + // taosGetTimestampMs() - st); + // } + // } else { + // qError("stream backend:%p failed to flush db at:%s", pHandle, dst); + // } - // release all ref to cfWrapper; - for (int i = 0; i < taosArrayGetSize(refs); i++) { - int64_t id = *(int64_t*)taosArrayGet(refs, i); - taosReleaseRef(streamBackendCfWrapperId, id); - } + // // release all ref to cfWrapper; + // // for (int i = 0; i < taosArrayGetSize(refs); i++) { + // // int64_t id = *(int64_t*)taosArrayGet(refs, i); + // // taosReleaseRef(streamBackendCfWrapperId, id); + // // } -_ERROR: - taosReleaseRef(streamBackendId, backendRid); - taosArrayDestroy(refs); - return code; + // _ERROR: + // // taosReleaseRef(streamBackendId, backendRid); + // // taosArrayDestroy(refs); + // return code; } int32_t streamBackendAddInUseChkp(void* arg, int64_t chkpId) { - if (arg == NULL) return 0; + // if (arg == NULL) return 0; - SStreamMeta* pMeta = arg; - taosWLockLatch(&pMeta->chkpDirLock); - taosArrayPush(pMeta->chkpInUse, &chkpId); - taosWUnLockLatch(&pMeta->chkpDirLock); + // SStreamMeta* pMeta = arg; + // taosWLockLatch(&pMeta->chkpDirLock); + // taosArrayPush(pMeta->chkpInUse, &chkpId); + // taosWUnLockLatch(&pMeta->chkpDirLock); return 0; } int32_t streamBackendDelInUseChkp(void* arg, int64_t chkpId) { - if (arg == NULL) return 0; - - SStreamMeta* pMeta = arg; - taosWLockLatch(&pMeta->chkpDirLock); - if (taosArrayGetSize(pMeta->chkpInUse) > 0) { - int64_t id = *(int64_t*)taosArrayGet(pMeta->chkpInUse, 0); - if (id == chkpId) { - taosArrayPopFrontBatch(pMeta->chkpInUse, 1); - } - } - taosWUnLockLatch(&pMeta->chkpDirLock); return 0; + // if (arg == NULL) return 0; + + // SStreamMeta* pMeta = arg; + // taosWLockLatch(&pMeta->chkpDirLock); + // if (taosArrayGetSize(pMeta->chkpInUse) > 0) { + // int64_t id = *(int64_t*)taosArrayGet(pMeta->chkpInUse, 0); + // if (id == chkpId) { + // taosArrayPopFrontBatch(pMeta->chkpInUse, 1); + // } + // } + // taosWUnLockLatch(&pMeta->chkpDirLock); } int32_t streamBackendDoCheckpoint(void* arg, uint64_t checkpointId) { - SStreamMeta* pMeta = arg; - int64_t backendRid = pMeta->streamBackendRid; - int64_t st = taosGetTimestampMs(); - int32_t code = -1; + return 0; + // SStreamMeta* pMeta = arg; + // int64_t backendRid = pMeta->streamBackendRid; + // int64_t st = taosGetTimestampMs(); + // int32_t code = -1; - SArray* refs = taosArrayInit(16, sizeof(int64_t)); + // SArray* refs = taosArrayInit(16, sizeof(int64_t)); - rocksdb_column_family_handle_t** ppCf = NULL; + // rocksdb_column_family_handle_t** ppCf = NULL; - char* pChkpDir = NULL; - char* pChkpIdDir = NULL; - if (chkpPreCheckDir(pMeta->path, checkpointId, &pChkpDir, &pChkpIdDir) != 0) { - taosArrayDestroy(refs); - return code; - } + // char* pChkpDir = NULL; + // char* pChkpIdDir = NULL; + // if (chkpPreCheckDir(pMeta->path, checkpointId, &pChkpDir, &pChkpIdDir) != 0) { + // taosArrayDestroy(refs); + // return code; + // } - SBackendWrapper* pHandle = taosAcquireRef(streamBackendId, backendRid); - if (pHandle == NULL || pHandle->db == NULL) { - goto _ERROR; - } + // SBackendWrapper* pHandle = taosAcquireRef(streamBackendId, backendRid); + // if (pHandle == NULL || pHandle->db == NULL) { + // goto _ERROR; + // } - // Get all cf and acquire cfWrappter - int32_t nCf = chkpGetAllDbCfHandle(pMeta, &ppCf, refs); - qDebug("stream backend:%p start to do checkpoint at:%s, cf num: %d ", pHandle, pChkpIdDir, nCf); + // // Get all cf and acquire cfWrappter + // int32_t nCf = chkpGetAllDbCfHandle(pMeta, &ppCf, refs); + // qDebug("stream backend:%p start to do checkpoint at:%s, cf num: %d ", pHandle, pChkpIdDir, nCf); - code = chkpPreFlushDb(pHandle->db, ppCf, nCf); - if (code == 0) { - code = chkpDoDbCheckpoint(pHandle->db, pChkpIdDir); - if (code != 0) { - qError("stream backend:%p failed to do checkpoint at:%s", pHandle, pChkpIdDir); - } else { - qDebug("stream backend:%p end to do checkpoint at:%s, time cost:%" PRId64 "ms", pHandle, pChkpIdDir, - taosGetTimestampMs() - st); - } - } else { - qError("stream backend:%p failed to flush db at:%s", pHandle, pChkpIdDir); - } - // release all ref to cfWrapper; - for (int i = 0; i < taosArrayGetSize(refs); i++) { - int64_t id = *(int64_t*)taosArrayGet(refs, i); - taosReleaseRef(streamBackendCfWrapperId, id); - } - if (code == 0) { - taosWLockLatch(&pMeta->chkpDirLock); - taosArrayPush(pMeta->chkpSaved, &checkpointId); - taosWUnLockLatch(&pMeta->chkpDirLock); + // code = chkpPreFlushDb(pHandle->db, ppCf, nCf); + // if (code == 0) { + // code = chkpDoDbCheckpoint(pHandle->db, pChkpIdDir); + // if (code != 0) { + // qError("stream backend:%p failed to do checkpoint at:%s", pHandle, pChkpIdDir); + // } else { + // qDebug("stream backend:%p end to do checkpoint at:%s, time cost:%" PRId64 "ms", pHandle, pChkpIdDir, + // taosGetTimestampMs() - st); + // } + // } else { + // qError("stream backend:%p failed to flush db at:%s", pHandle, pChkpIdDir); + // } + // // release all ref to cfWrapper; + // for (int i = 0; i < taosArrayGetSize(refs); i++) { + // int64_t id = *(int64_t*)taosArrayGet(refs, i); + // taosReleaseRef(streamBackendCfWrapperId, id); + // } + // if (code == 0) { + // taosWLockLatch(&pMeta->chkpDirLock); + // taosArrayPush(pMeta->chkpSaved, &checkpointId); + // taosWUnLockLatch(&pMeta->chkpDirLock); - // delete obsolte checkpoint - delObsoleteCheckpoint(arg, pChkpDir); - pMeta->chkpId = checkpointId; - } + // // delete obsolte checkpoint + // delObsoleteCheckpoint(arg, pChkpDir); + // pMeta->chkpId = checkpointId; + // } -_ERROR: - taosReleaseRef(streamBackendId, backendRid); - taosArrayDestroy(refs); - taosMemoryFree(ppCf); - taosMemoryFree(pChkpDir); - taosMemoryFree(pChkpIdDir); - return code; + // _ERROR: + // taosReleaseRef(streamBackendId, backendRid); + // taosArrayDestroy(refs); + // taosMemoryFree(ppCf); + // taosMemoryFree(pChkpDir); + // taosMemoryFree(pChkpIdDir); + // return code; } SListNode* streamBackendAddCompare(void* backend, void* arg) { @@ -1499,12 +1504,15 @@ _EXIT: taosMemoryFree(cfHandle); return code; } -void taskBackendAddRef(void* pTaskBackend) { +void* taskBackendAddRef(void* pTaskBackend) { STaskBackendWrapper* pBackend = pTaskBackend; - taosAcquireRef(streamBackendCfWrapperId, pBackend->refId); - return; + return taosAcquireRef(taskBackendWrapperId, pBackend->refId); } -void taskBackendDestroy(STaskBackendWrapper* wrapper); +void taskBackendRemoveRef(void* pTaskBackend) { + // STaskBackendWrapper* pBackend = pTaskBackend; + // taosReleaseRef(taskBackendWrapperId, pBackend->refId); +} +// void taskBackendDestroy(STaskBackendWrapper* wrapper); void taskBackendInitOpt(STaskBackendWrapper* pTaskBackend) { rocksdb_env_t* env = rocksdb_create_default_env(); @@ -1579,44 +1587,49 @@ STaskBackendWrapper* taskBackendOpen(char* path, char* key) { char* taskPath = NULL; char* err = NULL; - int32_t code = taskBackendBuildFullPath(path, key, &taskPath); - if (code != 0) return NULL; - + char** cfNames = NULL; size_t nCf = 0; + if (0 != taskBackendBuildFullPath(path, key, &taskPath)) return NULL; + STaskBackendWrapper* pTaskBackend = taosMemoryCalloc(1, sizeof(STaskBackendWrapper)); taskBackendInitOpt(pTaskBackend); - char** cfs = rocksdb_list_column_families(pTaskBackend->dbOpt, taskPath, &nCf, &err); - if (nCf == 0 || nCf == 1 || err != NULL) { - taosMemoryFreeClear(err); - pTaskBackend->db = rocksdb_open(pTaskBackend->dbOpt, taskPath, &err); - if (err != NULL) { - qError("failed to open rocksdb, path:%s, reason:%s", taskPath, err); - taosMemoryFreeClear(err); - code = -1; - goto _EXIT; - } - } else { - code = taskBackendOpenCfs(pTaskBackend, taskPath, cfs, nCf); - if (code != 0) goto _EXIT; + cfNames = rocksdb_list_column_families(pTaskBackend->dbOpt, taskPath, &nCf, &err); + if (nCf == 0) { + pTaskBackend->db = rocksdb_open(pTaskBackend->pCfOpts[0], taskPath, &err); + rocksdb_close(pTaskBackend->db); + + if (cfNames != NULL) rocksdb_list_column_families_destroy(cfNames, nCf); + taosMemoryFree(err); + } + + cfNames = rocksdb_list_column_families(pTaskBackend->dbOpt, taskPath, &nCf, &err); + ASSERT(err != NULL); + + if (0 != taskBackendOpenCfs(pTaskBackend, taskPath, cfNames, nCf)) { + goto _EXIT; } - if (cfs != NULL) rocksdb_list_column_families_destroy(cfs, nCf); taosThreadMutexInit(&pTaskBackend->mutex, NULL); qDebug("succ to init stream backend at %s, backend:%p", taskPath, pTaskBackend); taosMemoryFree(taskPath); - pTaskBackend->refId = taosAddRef(streamBackendCfWrapperId, pTaskBackend); + if (cfNames != NULL) rocksdb_list_column_families_destroy(cfNames, nCf); return pTaskBackend; _EXIT: taskBackendDestroy(pTaskBackend); + + if (err != NULL) taosMemoryFree(err); + if (cfNames != NULL) rocksdb_list_column_families_destroy(cfNames, nCf); return NULL; } -void taskBackendDestroy(STaskBackendWrapper* wrapper) { +void taskBackendDestroy(void* pBackend) { + STaskBackendWrapper* wrapper = pBackend; + if (wrapper == NULL) return; if (wrapper->db && wrapper->pCf) { diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index ad3ff82ec3..1892f58b9e 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -30,6 +30,7 @@ static TdThreadOnce streamMetaModuleInit = PTHREAD_ONCE_INIT; int32_t streamBackendId = 0; int32_t streamBackendCfWrapperId = 0; int32_t streamMetaId = 0; +int32_t taskBackendWrapperId = 0; static int64_t streamGetLatestCheckpointId(SStreamMeta* pMeta); static void metaHbToMnode(void* param, void* tmrId); @@ -52,6 +53,7 @@ int32_t metaRefMgtAdd(int64_t vgId, int64_t* rid); static void streamMetaEnvInit() { streamBackendId = taosOpenRef(64, streamBackendCleanup); streamBackendCfWrapperId = taosOpenRef(64, streamBackendHandleCleanup); + taskBackendWrapperId = taosOpenRef(64, taskBackendDestroy); streamMetaId = taosOpenRef(64, streamMetaCloseImpl); @@ -220,19 +222,22 @@ int32_t streamMetaMayConvertBackendFormat(SStreamMeta* pMeta) { return 0; } -void* streamMetaGetBackendByTaskKey(SStreamMeta* pMeta, char* key) { +void* streamMetaGetBackendByTaskKey(SStreamMeta* pMeta, char* key, int64_t* ref) { taosThreadMutexLock(&pMeta->backendMutex); void** ppBackend = taosHashGet(pMeta->pTaskBackendUnique, key, strlen(key)); if (ppBackend != NULL && *ppBackend != NULL) { taskBackendAddRef(*ppBackend); + *ref = ((STaskBackendWrapper*)*ppBackend)->refId; taosThreadMutexUnlock(&pMeta->backendMutex); return *ppBackend; } + void* pBackend = taskBackendOpen(pMeta->path, key); if (pBackend == NULL) { taosThreadMutexUnlock(&pMeta->backendMutex); return NULL; } + *ref = taosAddRef(taskBackendWrapperId, pBackend); taosHashPut(pMeta->pTaskBackendUnique, key, strlen(key), &pBackend, sizeof(void*)); return pBackend; diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index e01f87788f..152b426628 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -354,7 +354,11 @@ void tFreeStreamTask(SStreamTask* pTask) { taosArrayDestroyEx(pTask->pUpstreamInfoList, freeUpstreamItem); pTask->pUpstreamInfoList = NULL; } - + if (pTask->pBackend) { + taskBackendRemoveRef(pTask->pBackend); + + pTask->pBackend = NULL; + } taosThreadMutexDestroy(&pTask->lock); taosMemoryFree(pTask); From 82e024288904227e6c43800c3770ae475c860e18 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Sat, 7 Oct 2023 13:43:52 +0800 Subject: [PATCH 017/195] refact task backend --- source/libs/stream/src/streamBackendRocksdb.c | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 5589932fb8..49dada00d2 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1618,9 +1618,10 @@ STaskBackendWrapper* taskBackendOpen(char* path, char* key) { if (cfNames != NULL) rocksdb_list_column_families_destroy(cfNames, nCf); + pTaskBackend->idstr = taosStrdup(key); return pTaskBackend; -_EXIT: +_EXIT: taskBackendDestroy(pTaskBackend); if (err != NULL) taosMemoryFree(err); @@ -1667,6 +1668,8 @@ void taskBackendDestroy(void* pBackend) { taosThreadMutexDestroy(&wrapper->mutex); if (wrapper->db) rocksdb_close(wrapper->db); + + taosMemoryFree(wrapper->idstr); taosMemoryFree(wrapper); return; @@ -2024,10 +2027,10 @@ int streamStateGetCfIdx(SStreamState* pState, const char* funcName) { cf = rocksdb_create_column_family(wrapper->db, wrapper->pCfOpts[idx], ginitDict[idx].key, &err); if (err != NULL) { idx = -1; - qError("failed to to open cf, %p %s_%s, reason:%s", pState, wrapper->idstr, funcName, err); + qError("failed to open cf, %p %s_%s, reason:%s", pState, wrapper->idstr, funcName, err); taosMemoryFree(err); } else { - qDebug("succ to to open cf, %p %s_%s", pState, wrapper->idstr, funcName); + qDebug("succ to open cf, %p %s_%s", pState, wrapper->idstr, funcName); wrapper->pCf[idx] = cf; } } From 66315fe4058332b1805221d1378cb88133719f0d Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Sat, 7 Oct 2023 13:53:44 +0800 Subject: [PATCH 018/195] refact task backend --- source/libs/stream/src/streamBackendRocksdb.c | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 49dada00d2..9e1a7ceeec 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1590,7 +1590,9 @@ STaskBackendWrapper* taskBackendOpen(char* path, char* key) { char** cfNames = NULL; size_t nCf = 0; - if (0 != taskBackendBuildFullPath(path, key, &taskPath)) return NULL; + if (taskBackendBuildFullPath(path, key, &taskPath) != 0) { + return NULL; + } STaskBackendWrapper* pTaskBackend = taosMemoryCalloc(1, sizeof(STaskBackendWrapper)); taskBackendInitOpt(pTaskBackend); @@ -1607,18 +1609,19 @@ STaskBackendWrapper* taskBackendOpen(char* path, char* key) { cfNames = rocksdb_list_column_families(pTaskBackend->dbOpt, taskPath, &nCf, &err); ASSERT(err != NULL); - if (0 != taskBackendOpenCfs(pTaskBackend, taskPath, cfNames, nCf)) { + if (taskBackendOpenCfs(pTaskBackend, taskPath, cfNames, nCf) != 0) { goto _EXIT; } - taosThreadMutexInit(&pTaskBackend->mutex, NULL); + if (cfNames != NULL) { + rocksdb_list_column_families_destroy(cfNames, nCf); + } qDebug("succ to init stream backend at %s, backend:%p", taskPath, pTaskBackend); taosMemoryFree(taskPath); - if (cfNames != NULL) rocksdb_list_column_families_destroy(cfNames, nCf); - pTaskBackend->idstr = taosStrdup(key); + taosThreadMutexInit(&pTaskBackend->mutex, NULL); return pTaskBackend; _EXIT: @@ -1628,6 +1631,7 @@ _EXIT: if (cfNames != NULL) rocksdb_list_column_families_destroy(cfNames, nCf); return NULL; } + void taskBackendDestroy(void* pBackend) { STaskBackendWrapper* wrapper = pBackend; From c146327da843b4a4eb8ac361f2ef2865e2607838 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Sat, 7 Oct 2023 16:48:36 +0800 Subject: [PATCH 019/195] refact task backend --- include/libs/stream/tstream.h | 2 +- source/libs/stream/inc/streamBackendRocksdb.h | 7 + source/libs/stream/src/streamBackendRocksdb.c | 257 ++++++++++++++---- 3 files changed, 213 insertions(+), 53 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 9bc4a91f1c..a9e593eaad 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -419,9 +419,9 @@ typedef struct SStreamMeta { int64_t rid; int64_t chkpId; + int32_t chkpCap; SArray* chkpSaved; SArray* chkpInUse; - int32_t chkpCap; SRWLatch chkpDirLock; int32_t pauseTaskNum; diff --git a/source/libs/stream/inc/streamBackendRocksdb.h b/source/libs/stream/inc/streamBackendRocksdb.h index f5c303b809..b79979df67 100644 --- a/source/libs/stream/inc/streamBackendRocksdb.h +++ b/source/libs/stream/inc/streamBackendRocksdb.h @@ -66,6 +66,13 @@ typedef struct { TdThreadMutex mutex; char* idstr; int64_t refId; + char* path; + + int64_t chkpId; + SArray* chkpSaved; + SArray* chkpInUse; + int32_t chkpCap; + TdThreadRwlock chkpDirLock; } STaskBackendWrapper; diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 9e1a7ceeec..4edb166982 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -709,6 +709,60 @@ int32_t delObsoleteCheckpoint(void* arg, const char* path) { taosArrayDestroy(chkpDel); return 0; } +/* + * checkpointSave |--cp1--|--cp2--|--cp3--|--cp4--|--cp5--| + * chkpInUse: |--cp2--|--cp4--| + * chkpInUse is doing translation, cannot del until + * replication is finished + */ +int32_t chkpDelObsolete(void* arg, char* path) { + STaskBackendWrapper* pBackend = arg; + taosThreadRwlockWrlock(&pBackend->chkpDirLock); + + SArray* chkpDel = taosArrayInit(8, sizeof(int64_t)); + SArray* chkpDup = taosArrayInit(8, sizeof(int64_t)); + + int64_t firsId = 0; + if (taosArrayGetSize(pBackend->chkpInUse) >= 1) { + firsId = *(int64_t*)taosArrayGet(pBackend->chkpInUse, 0); + + for (int i = 0; i < taosArrayGetSize(pBackend->chkpSaved); i++) { + int64_t id = *(int64_t*)taosArrayGet(pBackend->chkpSaved, i); + if (id >= firsId) { + taosArrayPush(chkpDup, &id); + } else { + taosArrayPush(chkpDel, &id); + } + } + } else { + int32_t sz = taosArrayGetSize(pBackend->chkpSaved); + int32_t dsz = sz - pBackend->chkpCap; // del size + + for (int i = 0; i < dsz; i++) { + int64_t id = *(int64_t*)taosArrayGet(pBackend->chkpSaved, i); + taosArrayPush(chkpDel, &id); + } + for (int i = dsz < 0 ? 0 : dsz; i < sz; i++) { + int64_t id = *(int64_t*)taosArrayGet(pBackend->chkpSaved, i); + taosArrayPush(chkpDup, &id); + } + } + taosArrayDestroy(pBackend->chkpSaved); + pBackend->chkpSaved = chkpDup; + + taosThreadRwlockUnlock(&pBackend->chkpDirLock); + + for (int i = 0; i < taosArrayGetSize(chkpDel); i++) { + int64_t id = *(int64_t*)taosArrayGet(chkpDel, i); + char tbuf[256] = {0}; + sprintf(tbuf, "%s%scheckpoint%" PRId64 "", path, TD_DIRSEP, id); + if (taosIsDir(tbuf)) { + taosRemoveDir(tbuf); + } + } + taosArrayDestroy(chkpDel); + return 0; +} static int32_t compareCheckpoint(const void* a, const void* b) { int64_t x = *(int64_t*)a; @@ -805,6 +859,27 @@ int32_t chkpGetAllDbCfHandle(SStreamMeta* pMeta, rocksdb_column_family_handle_t* // *ppHandle = ppCf; // return nCf; } + +int32_t chkpGetAllDbCfHandle2(STaskBackendWrapper* pBackend, rocksdb_column_family_handle_t*** ppHandle) { + SArray* pHandle = taosArrayInit(8, POINTER_BYTES); + for (int i = 0; i < sizeof(ginitDict) / sizeof(ginitDict[0]); i++) { + if (pBackend->pCf[i]) { + rocksdb_column_family_handle_t* p = pBackend->pCf[i]; + taosArrayPush(pHandle, &p); + } + } + int32_t nCf = taosArrayGetSize(pHandle); + + rocksdb_column_family_handle_t** ppCf = taosMemoryCalloc(nCf, sizeof(rocksdb_column_family_handle_t*)); + for (int i = 0; i < nCf; i++) { + ppCf[i] = taosArrayGetP(pHandle, i); + } + taosArrayDestroy(pHandle); + + *ppHandle = ppCf; + return nCf; +} + int32_t chkpDoDbCheckpoint(rocksdb_t* db, char* path) { int32_t code = -1; char* err = NULL; @@ -827,47 +902,45 @@ _ERROR: return code; } int32_t chkpPreFlushDb(rocksdb_t* db, rocksdb_column_family_handle_t** cf, int32_t nCf) { - return 0; - // int code = 0; - // char* err = NULL; + int code = 0; + char* err = NULL; - // rocksdb_flushoptions_t* flushOpt = rocksdb_flushoptions_create(); - // rocksdb_flushoptions_set_wait(flushOpt, 1); + rocksdb_flushoptions_t* flushOpt = rocksdb_flushoptions_create(); + rocksdb_flushoptions_set_wait(flushOpt, 1); - // rocksdb_flush_cfs(db, flushOpt, cf, nCf, &err); - // if (err != NULL) { - // qError("failed to flush db before streamBackend clean up, reason:%s", err); - // taosMemoryFree(err); - // code = -1; - // } - // rocksdb_flushoptions_destroy(flushOpt); - // return code; + rocksdb_flush_cfs(db, flushOpt, cf, nCf, &err); + if (err != NULL) { + qError("failed to flush db before streamBackend clean up, reason:%s", err); + taosMemoryFree(err); + code = -1; + } + rocksdb_flushoptions_destroy(flushOpt); + return code; } int32_t chkpPreCheckDir(char* path, int64_t chkpId, char** chkpDir, char** chkpIdDir) { + int32_t code = 0; + char* pChkpDir = taosMemoryCalloc(1, 256); + char* pChkpIdDir = taosMemoryCalloc(1, 256); + + sprintf(pChkpDir, "%s%s%s", path, TD_DIRSEP, "checkpoints"); + code = taosMulModeMkDir(pChkpDir, 0755, true); + if (code != 0) { + qError("failed to prepare checkpoint dir, path:%s, reason:%s", path, tstrerror(code)); + taosMemoryFree(pChkpDir); + taosMemoryFree(pChkpIdDir); + code = -1; + return code; + } + + sprintf(pChkpIdDir, "%s%scheckpoint%" PRId64, pChkpDir, TD_DIRSEP, chkpId); + if (taosIsDir(pChkpIdDir)) { + qInfo("stream rm exist checkpoint%s", pChkpIdDir); + taosRemoveFile(pChkpIdDir); + } + *chkpDir = pChkpDir; + *chkpIdDir = pChkpIdDir; + return 0; - // int32_t code = 0; - // char* pChkpDir = taosMemoryCalloc(1, 256); - // char* pChkpIdDir = taosMemoryCalloc(1, 256); - - // sprintf(pChkpDir, "%s%s%s", path, TD_DIRSEP, "checkpoints"); - // code = taosMulModeMkDir(pChkpDir, 0755, true); - // if (code != 0) { - // qError("failed to prepare checkpoint dir, path:%s, reason:%s", path, tstrerror(code)); - // taosMemoryFree(pChkpDir); - // taosMemoryFree(pChkpIdDir); - // code = -1; - // return code; - // } - - // sprintf(pChkpIdDir, "%s%scheckpoint%" PRId64, pChkpDir, TD_DIRSEP, chkpId); - // if (taosIsDir(pChkpIdDir)) { - // qInfo("stream rm exist checkpoint%s", pChkpIdDir); - // taosRemoveFile(pChkpIdDir); - // } - // *chkpDir = pChkpDir; - // *chkpIdDir = pChkpIdDir; - - // return 0; } int32_t streamBackendTriggerChkp(void* arg, char* dst) { @@ -936,6 +1009,49 @@ int32_t streamBackendDelInUseChkp(void* arg, int64_t chkpId) { // taosWUnLockLatch(&pMeta->chkpDirLock); } +int32_t taskBackendDoCheckpoint(void* arg, uint64_t chkpId) { + STaskBackendWrapper* pBackend = arg; + int64_t st = taosGetTimestampMs(); + int32_t code = -1; + int64_t refId = pBackend->refId; + + if (taosAcquireRef(taskBackendWrapperId, refId) == NULL) { + return -1; + } + + char* pChkpDir = NULL; + char* pChkpIdDir = NULL; + if (chkpPreCheckDir(pBackend->path, chkpId, &pChkpDir, &pChkpIdDir) != 0) { + goto _EXIT; + } + // Get all cf and acquire cfWrappter + rocksdb_column_family_handle_t** ppCf = NULL; + + int32_t nCf = chkpGetAllDbCfHandle2(pBackend, &ppCf); + qDebug("stream backend:%p start to do checkpoint at:%s, cf num: %d ", pBackend, pChkpIdDir, nCf); + + if ((code = chkpPreFlushDb(pBackend->db, ppCf, nCf)) == 0) { + if ((code = chkpDoDbCheckpoint(pBackend->db, pChkpIdDir)) != 0) { + qError("stream backend:%p failed to do checkpoint at:%s", pBackend, pChkpIdDir); + } else { + qDebug("stream backend:%p end to do checkpoint at:%s, time cost:%" PRId64 "ms", pBackend, pChkpIdDir, + taosGetTimestampMs() - st); + } + } else { + qError("stream backend:%p failed to flush db at:%s", pBackend, pChkpIdDir); + } + + code = chkpDelObsolete(pBackend, pChkpDir); + taosReleaseRef(taskBackendWrapperId, refId); + + return code; + +_EXIT: + taosMemoryFree(pChkpDir); + taosMemoryFree(pChkpIdDir); + taosReleaseRef(taskBackendWrapperId, refId); + return -1; +} int32_t streamBackendDoCheckpoint(void* arg, uint64_t checkpointId) { return 0; // SStreamMeta* pMeta = arg; @@ -1514,7 +1630,7 @@ void taskBackendRemoveRef(void* pTaskBackend) { } // void taskBackendDestroy(STaskBackendWrapper* wrapper); -void taskBackendInitOpt(STaskBackendWrapper* pTaskBackend) { +void taskBackendInitDBOpt(STaskBackendWrapper* pTaskBackend) { rocksdb_env_t* env = rocksdb_create_default_env(); rocksdb_cache_t* cache = rocksdb_cache_create_lru(256); @@ -1568,10 +1684,26 @@ void taskBackendInitOpt(STaskBackendWrapper* pTaskBackend) { } return; } -int32_t taskBackendBuildFullPath(char* path, char* key, char** fullPath) { +void taskBackendInitChkpOpt(STaskBackendWrapper* pBackend) { + pBackend->chkpId = -1; + pBackend->chkpCap = 4; + pBackend->chkpSaved = taosArrayInit(4, sizeof(int64_t)); + pBackend->chkpInUse = taosArrayInit(4, sizeof(int64_t)); + + taosThreadRwlockInit(&pBackend->chkpDirLock, NULL); +} + +void taskBackendDestroyChkpOpt(STaskBackendWrapper* pBackend) { + taosArrayDestroy(pBackend->chkpSaved); + taosArrayDestroy(pBackend->chkpInUse); + taosThreadRwlockDestroy(&pBackend->chkpDirLock); +} + +int32_t taskBackendBuildFullPath(char* path, char* key, char** dbFullPath, char** taskFullPath) { int32_t code = 0; - char* taskPath = taosMemoryCalloc(1, strlen(path) + 128); - sprintf(taskPath, "%s%s%s%s%s", path, TD_DIRSEP, "state", TD_DIRSEP, key); + + char* taskPath = taosMemoryCalloc(1, strlen(path) + 128); + sprintf(taskPath, "%s%s%s", path, TD_DIRSEP, key); if (!taosDirExist(taskPath)) { code = taosMulMkDir(taskPath); if (code != 0) { @@ -1580,36 +1712,53 @@ int32_t taskBackendBuildFullPath(char* path, char* key, char** fullPath) { return code; } } - *fullPath = taskPath; + + char* dbPath = taosMemoryCalloc(1, strlen(taskPath) + 128); + sprintf(dbPath, "%s%s%s", taskPath, TD_DIRSEP, "state"); + if (!taosDirExist(dbPath)) { + code = taosMulMkDir(dbPath); + if (code != 0) { + qError("failed to create dir: %s, reason:%s", dbPath, tstrerror(code)); + taosMemoryFree(taskPath); + taosMemoryFree(dbPath); + return code; + } + } + + *dbFullPath = dbPath; + *taskFullPath = taskPath; return 0; } STaskBackendWrapper* taskBackendOpen(char* path, char* key) { - char* taskPath = NULL; - char* err = NULL; - + char* taskPath = NULL; + char* err = NULL; + char* dbPath = NULL; char** cfNames = NULL; size_t nCf = 0; - if (taskBackendBuildFullPath(path, key, &taskPath) != 0) { + if (taskBackendBuildFullPath(path, key, &dbPath, &taskPath) != 0) { return NULL; } STaskBackendWrapper* pTaskBackend = taosMemoryCalloc(1, sizeof(STaskBackendWrapper)); - taskBackendInitOpt(pTaskBackend); - cfNames = rocksdb_list_column_families(pTaskBackend->dbOpt, taskPath, &nCf, &err); + pTaskBackend->path = taskPath; + taskBackendInitChkpOpt(pTaskBackend); + taskBackendInitDBOpt(pTaskBackend); + + cfNames = rocksdb_list_column_families(pTaskBackend->dbOpt, dbPath, &nCf, &err); if (nCf == 0) { - pTaskBackend->db = rocksdb_open(pTaskBackend->pCfOpts[0], taskPath, &err); + pTaskBackend->db = rocksdb_open(pTaskBackend->pCfOpts[0], dbPath, &err); rocksdb_close(pTaskBackend->db); if (cfNames != NULL) rocksdb_list_column_families_destroy(cfNames, nCf); taosMemoryFree(err); } - cfNames = rocksdb_list_column_families(pTaskBackend->dbOpt, taskPath, &nCf, &err); + cfNames = rocksdb_list_column_families(pTaskBackend->dbOpt, dbPath, &nCf, &err); ASSERT(err != NULL); - if (taskBackendOpenCfs(pTaskBackend, taskPath, cfNames, nCf) != 0) { + if (taskBackendOpenCfs(pTaskBackend, dbPath, cfNames, nCf) != 0) { goto _EXIT; } @@ -1617,11 +1766,12 @@ STaskBackendWrapper* taskBackendOpen(char* path, char* key) { rocksdb_list_column_families_destroy(cfNames, nCf); } - qDebug("succ to init stream backend at %s, backend:%p", taskPath, pTaskBackend); - taosMemoryFree(taskPath); + qDebug("succ to init stream backend at %s, backend:%p", dbPath, pTaskBackend); + taosMemoryFree(dbPath); pTaskBackend->idstr = taosStrdup(key); taosThreadMutexInit(&pTaskBackend->mutex, NULL); + return pTaskBackend; _EXIT: @@ -1673,7 +1823,10 @@ void taskBackendDestroy(void* pBackend) { if (wrapper->db) rocksdb_close(wrapper->db); + taskBackendDestroyChkpOpt(pBackend); + taosMemoryFree(wrapper->idstr); + taosMemoryFree(wrapper->path); taosMemoryFree(wrapper); return; From 3ebc7eef5f32193398b5afc51137a9d2a52b488a Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Sat, 7 Oct 2023 17:03:12 +0800 Subject: [PATCH 020/195] refact task backend --- source/libs/stream/src/streamBackendRocksdb.c | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 4edb166982..07249862fb 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -715,9 +715,10 @@ int32_t delObsoleteCheckpoint(void* arg, const char* path) { * chkpInUse is doing translation, cannot del until * replication is finished */ -int32_t chkpDelObsolete(void* arg, char* path) { +int32_t chkpMayDelObsolete(void* arg, int64_t chkpId, char* path) { STaskBackendWrapper* pBackend = arg; taosThreadRwlockWrlock(&pBackend->chkpDirLock); + taosArrayPush(pBackend->chkpSaved, &chkpId); SArray* chkpDel = taosArrayInit(8, sizeof(int64_t)); SArray* chkpDup = taosArrayInit(8, sizeof(int64_t)); @@ -1041,9 +1042,9 @@ int32_t taskBackendDoCheckpoint(void* arg, uint64_t chkpId) { qError("stream backend:%p failed to flush db at:%s", pBackend, pChkpIdDir); } - code = chkpDelObsolete(pBackend, pChkpDir); - taosReleaseRef(taskBackendWrapperId, refId); + code = chkpMayDelObsolete(pBackend, chkpId, pChkpDir); + taosReleaseRef(taskBackendWrapperId, refId); return code; _EXIT: From e41da13d5bdde89f84299281d8c9dfe6f7022e4a Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Sat, 7 Oct 2023 17:07:26 +0800 Subject: [PATCH 021/195] refact task backend --- source/libs/stream/src/streamBackendRocksdb.c | 63 +------------------ source/libs/stream/src/streamCheckpoint.c | 24 +++---- 2 files changed, 14 insertions(+), 73 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 07249862fb..97d6f0634a 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1053,68 +1053,7 @@ _EXIT: taosReleaseRef(taskBackendWrapperId, refId); return -1; } -int32_t streamBackendDoCheckpoint(void* arg, uint64_t checkpointId) { - return 0; - // SStreamMeta* pMeta = arg; - // int64_t backendRid = pMeta->streamBackendRid; - // int64_t st = taosGetTimestampMs(); - // int32_t code = -1; - - // SArray* refs = taosArrayInit(16, sizeof(int64_t)); - - // rocksdb_column_family_handle_t** ppCf = NULL; - - // char* pChkpDir = NULL; - // char* pChkpIdDir = NULL; - // if (chkpPreCheckDir(pMeta->path, checkpointId, &pChkpDir, &pChkpIdDir) != 0) { - // taosArrayDestroy(refs); - // return code; - // } - - // SBackendWrapper* pHandle = taosAcquireRef(streamBackendId, backendRid); - // if (pHandle == NULL || pHandle->db == NULL) { - // goto _ERROR; - // } - - // // Get all cf and acquire cfWrappter - // int32_t nCf = chkpGetAllDbCfHandle(pMeta, &ppCf, refs); - // qDebug("stream backend:%p start to do checkpoint at:%s, cf num: %d ", pHandle, pChkpIdDir, nCf); - - // code = chkpPreFlushDb(pHandle->db, ppCf, nCf); - // if (code == 0) { - // code = chkpDoDbCheckpoint(pHandle->db, pChkpIdDir); - // if (code != 0) { - // qError("stream backend:%p failed to do checkpoint at:%s", pHandle, pChkpIdDir); - // } else { - // qDebug("stream backend:%p end to do checkpoint at:%s, time cost:%" PRId64 "ms", pHandle, pChkpIdDir, - // taosGetTimestampMs() - st); - // } - // } else { - // qError("stream backend:%p failed to flush db at:%s", pHandle, pChkpIdDir); - // } - // // release all ref to cfWrapper; - // for (int i = 0; i < taosArrayGetSize(refs); i++) { - // int64_t id = *(int64_t*)taosArrayGet(refs, i); - // taosReleaseRef(streamBackendCfWrapperId, id); - // } - // if (code == 0) { - // taosWLockLatch(&pMeta->chkpDirLock); - // taosArrayPush(pMeta->chkpSaved, &checkpointId); - // taosWUnLockLatch(&pMeta->chkpDirLock); - - // // delete obsolte checkpoint - // delObsoleteCheckpoint(arg, pChkpDir); - // pMeta->chkpId = checkpointId; - // } - - // _ERROR: - // taosReleaseRef(streamBackendId, backendRid); - // taosArrayDestroy(refs); - // taosMemoryFree(ppCf); - // taosMemoryFree(pChkpDir); - // taosMemoryFree(pChkpIdDir); - // return code; -} +int32_t streamBackendDoCheckpoint(void* arg, uint64_t chkpId) { return taskBackendDoCheckpoint(arg, chkpId); } SListNode* streamBackendAddCompare(void* backend, void* arg) { SBackendWrapper* pHandle = (SBackendWrapper*)backend; diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index cc93d25fd5..63e3b94561 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -120,7 +120,7 @@ static int32_t appendCheckpointIntoInputQ(SStreamTask* pTask, int32_t checkpoint pBlock->info.rows = 1; pBlock->info.childId = pTask->info.selfChildId; - pChkpoint->blocks = taosArrayInit(4, sizeof(SSDataBlock));//pBlock; + pChkpoint->blocks = taosArrayInit(4, sizeof(SSDataBlock)); // pBlock; taosArrayPush(pChkpoint->blocks, pBlock); taosMemoryFree(pBlock); @@ -166,10 +166,10 @@ static int32_t continueDispatchCheckpointBlock(SStreamDataBlock* pBlock, SStream int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBlock) { SSDataBlock* pDataBlock = taosArrayGet(pBlock->blocks, 0); - int64_t checkpointId = pDataBlock->info.version; + int64_t checkpointId = pDataBlock->info.version; const char* id = pTask->id.idStr; - int32_t code = TSDB_CODE_SUCCESS; + int32_t code = TSDB_CODE_SUCCESS; // set the task status pTask->checkpointingId = checkpointId; @@ -177,7 +177,7 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc // set task status pTask->status.taskStatus = TASK_STATUS__CK; - { // todo: remove this when the pipeline checkpoint generating is used. + { // todo: remove this when the pipeline checkpoint generating is used. SStreamMeta* pMeta = pTask->pMeta; taosWLockLatch(&pMeta->lock); @@ -189,10 +189,11 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc taosWUnLockLatch(&pMeta->lock); } - //todo fix race condition: set the status and append checkpoint block + // todo fix race condition: set the status and append checkpoint block int32_t taskLevel = pTask->info.taskLevel; if (taskLevel == TASK_LEVEL__SOURCE) { - if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH || pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { + if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH || + pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { qDebug("s-task:%s set childIdx:%d, and add checkpoint block into outputQ", id, pTask->info.selfChildId); continueDispatchCheckpointBlock(pBlock, pTask); } else { // only one task exists, no need to dispatch downstream info @@ -223,7 +224,8 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc } else { qDebug( "s-task:%s process checkpoint block, all %d upstreams sent checkpoint msgs, dispatch checkpoint msg " - "downstream", id, num); + "downstream", + id, num); // set the needed checked downstream tasks, only when all downstream tasks do checkpoint complete, this task // can start local checkpoint procedure @@ -288,11 +290,11 @@ int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, int64_t checkpointId) { // save the task streamMetaSaveTask(pMeta, p); - streamTaskOpenAllUpstreamInput(p); // open inputQ for all upstream tasks + streamTaskOpenAllUpstreamInput(p); // open inputQ for all upstream tasks qDebug("vgId:%d s-task:%s level:%d commit task status after checkpoint completed, checkpointId:%" PRId64 ", Ver(saved):%" PRId64 " currentVer:%" PRId64 ", status to be normal, prev:%s", - pMeta->vgId, p->id.idStr, p->info.taskLevel, checkpointId, p->chkInfo.checkpointVer, p->chkInfo.nextProcessVer, - streamGetTaskStatusStr(prev)); + pMeta->vgId, p->id.idStr, p->info.taskLevel, checkpointId, p->chkInfo.checkpointVer, + p->chkInfo.nextProcessVer, streamGetTaskStatusStr(prev)); } if (streamMetaCommit(pMeta) < 0) { @@ -320,7 +322,7 @@ int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { qDebug("s-task:%s is ready for checkpoint", pTask->id.idStr); pMeta->totalTasks = 0; - streamBackendDoCheckpoint(pMeta, pTask->checkpointingId); + streamBackendDoCheckpoint(pTask->pBackend, pTask->checkpointingId); streamSaveAllTaskStatus(pMeta, pTask->checkpointingId); qDebug("vgId:%d vnode wide checkpoint completed, save all tasks status, checkpointId:%" PRId64, pMeta->vgId, pTask->checkpointingId); From bdc4afec8323e251e3606f189c2efdd19d81f476 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Sat, 7 Oct 2023 17:17:30 +0800 Subject: [PATCH 022/195] refact task backend --- source/libs/stream/src/streamBackendRocksdb.c | 28 +++++++++---------- source/libs/stream/src/streamMeta.c | 2 ++ 2 files changed, 16 insertions(+), 14 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 97d6f0634a..539844a4e8 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1639,50 +1639,50 @@ void taskBackendDestroyChkpOpt(STaskBackendWrapper* pBackend) { taosThreadRwlockDestroy(&pBackend->chkpDirLock); } -int32_t taskBackendBuildFullPath(char* path, char* key, char** dbFullPath, char** taskFullPath) { +int32_t taskBackendBuildFullPath(char* path, char* key, char** dbFullPath, char** stateFullPath) { int32_t code = 0; - char* taskPath = taosMemoryCalloc(1, strlen(path) + 128); - sprintf(taskPath, "%s%s%s", path, TD_DIRSEP, key); - if (!taosDirExist(taskPath)) { - code = taosMulMkDir(taskPath); + char* statePath = taosMemoryCalloc(1, strlen(path) + 128); + sprintf(statePath, "%s%s%s", path, TD_DIRSEP, key); + if (!taosDirExist(statePath)) { + code = taosMulMkDir(statePath); if (code != 0) { - qError("failed to create dir: %s, reason:%s", taskPath, tstrerror(code)); - taosMemoryFree(taskPath); + qError("failed to create dir: %s, reason:%s", statePath, tstrerror(code)); + taosMemoryFree(statePath); return code; } } - char* dbPath = taosMemoryCalloc(1, strlen(taskPath) + 128); - sprintf(dbPath, "%s%s%s", taskPath, TD_DIRSEP, "state"); + char* dbPath = taosMemoryCalloc(1, strlen(statePath) + 128); + sprintf(dbPath, "%s%s%s", statePath, TD_DIRSEP, "state"); if (!taosDirExist(dbPath)) { code = taosMulMkDir(dbPath); if (code != 0) { qError("failed to create dir: %s, reason:%s", dbPath, tstrerror(code)); - taosMemoryFree(taskPath); + taosMemoryFree(statePath); taosMemoryFree(dbPath); return code; } } *dbFullPath = dbPath; - *taskFullPath = taskPath; + *stateFullPath = statePath; return 0; } STaskBackendWrapper* taskBackendOpen(char* path, char* key) { - char* taskPath = NULL; + char* statePath = NULL; char* err = NULL; char* dbPath = NULL; char** cfNames = NULL; size_t nCf = 0; - if (taskBackendBuildFullPath(path, key, &dbPath, &taskPath) != 0) { + if (taskBackendBuildFullPath(path, key, &dbPath, &statePath) != 0) { return NULL; } STaskBackendWrapper* pTaskBackend = taosMemoryCalloc(1, sizeof(STaskBackendWrapper)); - pTaskBackend->path = taskPath; + pTaskBackend->path = statePath; taskBackendInitChkpOpt(pTaskBackend); taskBackendInitDBOpt(pTaskBackend); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 1892f58b9e..8cce4dc2f9 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -237,9 +237,11 @@ void* streamMetaGetBackendByTaskKey(SStreamMeta* pMeta, char* key, int64_t* ref) taosThreadMutexUnlock(&pMeta->backendMutex); return NULL; } + *ref = taosAddRef(taskBackendWrapperId, pBackend); taosHashPut(pMeta->pTaskBackendUnique, key, strlen(key), &pBackend, sizeof(void*)); + taosThreadMutexUnlock(&pMeta->backendMutex); return pBackend; } SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandFunc, int32_t vgId, int64_t stage) { From 98099ebbae54151c24fad76912d3ec7f50806673 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Sat, 7 Oct 2023 18:41:27 +0800 Subject: [PATCH 023/195] fix mem leak --- source/libs/stream/src/streamBackendRocksdb.c | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 539844a4e8..19dfe3cf8d 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -870,6 +870,10 @@ int32_t chkpGetAllDbCfHandle2(STaskBackendWrapper* pBackend, rocksdb_column_fami } } int32_t nCf = taosArrayGetSize(pHandle); + if (nCf == 0) { + taosArrayDestroy(pHandle); + return nCf; + } rocksdb_column_family_handle_t** ppCf = taosMemoryCalloc(nCf, sizeof(rocksdb_column_family_handle_t*)); for (int i = 0; i < nCf; i++) { @@ -1031,14 +1035,14 @@ int32_t taskBackendDoCheckpoint(void* arg, uint64_t chkpId) { int32_t nCf = chkpGetAllDbCfHandle2(pBackend, &ppCf); qDebug("stream backend:%p start to do checkpoint at:%s, cf num: %d ", pBackend, pChkpIdDir, nCf); - if ((code = chkpPreFlushDb(pBackend->db, ppCf, nCf)) == 0) { + if ((code = chkpPreFlushDb(pBackend->db, ppCf, nCf)) == 0 && nCf != 0) { if ((code = chkpDoDbCheckpoint(pBackend->db, pChkpIdDir)) != 0) { qError("stream backend:%p failed to do checkpoint at:%s", pBackend, pChkpIdDir); } else { qDebug("stream backend:%p end to do checkpoint at:%s, time cost:%" PRId64 "ms", pBackend, pChkpIdDir, taosGetTimestampMs() - st); } - } else { + } else if (nCf != 0) { qError("stream backend:%p failed to flush db at:%s", pBackend, pChkpIdDir); } From 2fa91341a7c4a0d79cbae3248db25319335f9583 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Sun, 8 Oct 2023 10:35:49 +0800 Subject: [PATCH 024/195] refact task backend --- source/libs/stream/src/streamBackendRocksdb.c | 24 +++++++++++-------- 1 file changed, 14 insertions(+), 10 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 19dfe3cf8d..775eaebf89 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1685,22 +1685,27 @@ STaskBackendWrapper* taskBackendOpen(char* path, char* key) { } STaskBackendWrapper* pTaskBackend = taosMemoryCalloc(1, sizeof(STaskBackendWrapper)); - + pTaskBackend->idstr = taosStrdup(key); pTaskBackend->path = statePath; + taosThreadMutexInit(&pTaskBackend->mutex, NULL); taskBackendInitChkpOpt(pTaskBackend); taskBackendInitDBOpt(pTaskBackend); + statePath = NULL; cfNames = rocksdb_list_column_families(pTaskBackend->dbOpt, dbPath, &nCf, &err); if (nCf == 0) { + // pre create db pTaskBackend->db = rocksdb_open(pTaskBackend->pCfOpts[0], dbPath, &err); rocksdb_close(pTaskBackend->db); - if (cfNames != NULL) rocksdb_list_column_families_destroy(cfNames, nCf); + if (cfNames != NULL) { + rocksdb_list_column_families_destroy(cfNames, nCf); + } taosMemoryFree(err); - } - cfNames = rocksdb_list_column_families(pTaskBackend->dbOpt, dbPath, &nCf, &err); - ASSERT(err != NULL); + cfNames = rocksdb_list_column_families(pTaskBackend->dbOpt, dbPath, &nCf, &err); + ASSERT(err != NULL); + } if (taskBackendOpenCfs(pTaskBackend, dbPath, cfNames, nCf) != 0) { goto _EXIT; @@ -1713,16 +1718,15 @@ STaskBackendWrapper* taskBackendOpen(char* path, char* key) { qDebug("succ to init stream backend at %s, backend:%p", dbPath, pTaskBackend); taosMemoryFree(dbPath); - pTaskBackend->idstr = taosStrdup(key); - taosThreadMutexInit(&pTaskBackend->mutex, NULL); - return pTaskBackend; _EXIT: taskBackendDestroy(pTaskBackend); - if (err != NULL) taosMemoryFree(err); - if (cfNames != NULL) rocksdb_list_column_families_destroy(cfNames, nCf); + if (err) taosMemoryFree(err); + if (cfNames) rocksdb_list_column_families_destroy(cfNames, nCf); + if (dbPath) taosMemoryFree(dbPath); + if (statePath) taosMemoryFree(statePath); return NULL; } From f49a2cbeb058c92fd6c5eb4a579aab9277e0f932 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Sun, 8 Oct 2023 18:14:17 +0800 Subject: [PATCH 025/195] refact task backend --- source/dnode/vnode/src/tq/tqStreamStateSnap.c | 6 ++---- source/dnode/vnode/src/vnd/vnodeSnapshot.c | 1 + source/libs/stream/src/streamBackendRocksdb.c | 5 +++-- source/libs/stream/src/streamMeta.c | 2 +- source/libs/stream/src/streamSnapshot.c | 2 +- 5 files changed, 8 insertions(+), 8 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqStreamStateSnap.c b/source/dnode/vnode/src/tq/tqStreamStateSnap.c index 4a1b3961cd..53f9cc3c92 100644 --- a/source/dnode/vnode/src/tq/tqStreamStateSnap.c +++ b/source/dnode/vnode/src/tq/tqStreamStateSnap.c @@ -168,10 +168,8 @@ int32_t streamStateSnapWriterClose(SStreamStateWriter* pWriter, int8_t rollback) } int32_t streamStateRebuildFromSnap(SStreamStateWriter* pWriter, int64_t chkpId) { tqDebug("vgId:%d, vnode %s start to rebuild stream-state", TD_VID(pWriter->pTq->pVnode), STREAM_STATE_TRANSFER); - int32_t code = streamMetaReopen(pWriter->pTq->pStreamMeta, chkpId); - if (code == 0) { - code = streamStateLoadTasks(pWriter); - } + // int32_t code = streamMetaReopen(pWriter->pTq->pStreamMeta, chkpId); + int32_t code = streamStateLoadTasks(pWriter); tqDebug("vgId:%d, vnode %s succ to rebuild stream-state", TD_VID(pWriter->pTq->pVnode), STREAM_STATE_TRANSFER); taosMemoryFree(pWriter); return code; diff --git a/source/dnode/vnode/src/vnd/vnodeSnapshot.c b/source/dnode/vnode/src/vnd/vnodeSnapshot.c index f19068ea88..35c8a4102e 100644 --- a/source/dnode/vnode/src/vnd/vnodeSnapshot.c +++ b/source/dnode/vnode/src/vnd/vnodeSnapshot.c @@ -256,6 +256,7 @@ int32_t vnodeSnapRead(SVSnapReader *pReader, uint8_t **ppData, uint32_t *nData) if (code) { pReader->streamStateDone = 1; pReader->pStreamStateReader = NULL; + code = 0; goto _err; } } diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 775eaebf89..ba05d9968b 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -922,7 +922,8 @@ int32_t chkpPreFlushDb(rocksdb_t* db, rocksdb_column_family_handle_t** cf, int32 rocksdb_flushoptions_destroy(flushOpt); return code; } -int32_t chkpPreCheckDir(char* path, int64_t chkpId, char** chkpDir, char** chkpIdDir) { + +int32_t chkpPreBuildDir(char* path, int64_t chkpId, char** chkpDir, char** chkpIdDir) { int32_t code = 0; char* pChkpDir = taosMemoryCalloc(1, 256); char* pChkpIdDir = taosMemoryCalloc(1, 256); @@ -1026,7 +1027,7 @@ int32_t taskBackendDoCheckpoint(void* arg, uint64_t chkpId) { char* pChkpDir = NULL; char* pChkpIdDir = NULL; - if (chkpPreCheckDir(pBackend->path, chkpId, &pChkpDir, &pChkpIdDir) != 0) { + if (chkpPreBuildDir(pBackend->path, chkpId, &pChkpDir, &pChkpIdDir) != 0) { goto _EXIT; } // Get all cf and acquire cfWrappter diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 8cce4dc2f9..72399865e6 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -806,7 +806,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { continue; } - // do duplicate task check. + // do duplicate task checke int64_t keys[2] = {pTask->id.streamId, pTask->id.taskId}; void* p = taosHashGet(pMeta->pTasks, keys, sizeof(keys)); if (p == NULL) { diff --git a/source/libs/stream/src/streamSnapshot.c b/source/libs/stream/src/streamSnapshot.c index 8a4500dd86..42d77fc4c0 100644 --- a/source/libs/stream/src/streamSnapshot.c +++ b/source/libs/stream/src/streamSnapshot.c @@ -270,7 +270,7 @@ void streamSnapHandleDestroy(SStreamSnapHandle* handle) { if (handle->checkpointId == 0) { // del tmp dir - if (taosIsDir(pFile->path)) { + if (pFile && taosIsDir(pFile->path)) { taosRemoveDir(pFile->path); } } else { From bffa6387fe09d623fb2a4a8664858afcdf472634 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Sun, 8 Oct 2023 18:16:45 +0800 Subject: [PATCH 026/195] refact task backend --- source/dnode/vnode/src/tq/tq.c | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 7cfe776378..3546df7e91 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1792,12 +1792,12 @@ _end: taosWLockLatch(&pMeta->lock); terrno = 0; - int32_t code = streamMetaReopen(pMeta, 0); - if (code != 0) { - tqError("vgId:%d failed to reopen stream meta", vgId); - taosWUnLockLatch(&pMeta->lock); - return -1; - } + // int32_t code = streamMetaReopen(pMeta, 0); + // if (code != 0) { + // tqError("vgId:%d failed to reopen stream meta", vgId); + // taosWUnLockLatch(&pMeta->lock); + // return -1; + // } if (streamMetaLoadAllTasks(pTq->pStreamMeta) < 0) { tqError("vgId:%d failed to load stream tasks", vgId); From 0b92a6b1eaf0013469a0d99b1cd465ec70c49394 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Mon, 9 Oct 2023 11:16:16 +0800 Subject: [PATCH 027/195] fix mem leak --- source/libs/stream/src/streamBackendRocksdb.c | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index ba05d9968b..f936b569ae 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -717,7 +717,9 @@ int32_t delObsoleteCheckpoint(void* arg, const char* path) { */ int32_t chkpMayDelObsolete(void* arg, int64_t chkpId, char* path) { STaskBackendWrapper* pBackend = arg; + taosThreadRwlockWrlock(&pBackend->chkpDirLock); + taosArrayPush(pBackend->chkpSaved, &chkpId); SArray* chkpDel = taosArrayInit(8, sizeof(int64_t)); @@ -1028,6 +1030,7 @@ int32_t taskBackendDoCheckpoint(void* arg, uint64_t chkpId) { char* pChkpDir = NULL; char* pChkpIdDir = NULL; if (chkpPreBuildDir(pBackend->path, chkpId, &pChkpDir, &pChkpIdDir) != 0) { + code = -1; goto _EXIT; } // Get all cf and acquire cfWrappter @@ -1036,27 +1039,24 @@ int32_t taskBackendDoCheckpoint(void* arg, uint64_t chkpId) { int32_t nCf = chkpGetAllDbCfHandle2(pBackend, &ppCf); qDebug("stream backend:%p start to do checkpoint at:%s, cf num: %d ", pBackend, pChkpIdDir, nCf); - if ((code = chkpPreFlushDb(pBackend->db, ppCf, nCf)) == 0 && nCf != 0) { + if ((code = chkpPreFlushDb(pBackend->db, ppCf, nCf)) == 0) { if ((code = chkpDoDbCheckpoint(pBackend->db, pChkpIdDir)) != 0) { qError("stream backend:%p failed to do checkpoint at:%s", pBackend, pChkpIdDir); } else { qDebug("stream backend:%p end to do checkpoint at:%s, time cost:%" PRId64 "ms", pBackend, pChkpIdDir, taosGetTimestampMs() - st); } - } else if (nCf != 0) { + } else { qError("stream backend:%p failed to flush db at:%s", pBackend, pChkpIdDir); } code = chkpMayDelObsolete(pBackend, chkpId, pChkpDir); - taosReleaseRef(taskBackendWrapperId, refId); - return code; - _EXIT: taosMemoryFree(pChkpDir); taosMemoryFree(pChkpIdDir); taosReleaseRef(taskBackendWrapperId, refId); - return -1; + return code; } int32_t streamBackendDoCheckpoint(void* arg, uint64_t chkpId) { return taskBackendDoCheckpoint(arg, chkpId); } From addb1107d264228308b150e276f70072c0f0631b Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Mon, 9 Oct 2023 15:11:59 +0800 Subject: [PATCH 028/195] fix mem leak --- source/dnode/vnode/src/vnd/vnodeSnapshot.c | 1 - 1 file changed, 1 deletion(-) diff --git a/source/dnode/vnode/src/vnd/vnodeSnapshot.c b/source/dnode/vnode/src/vnd/vnodeSnapshot.c index 35c8a4102e..f19068ea88 100644 --- a/source/dnode/vnode/src/vnd/vnodeSnapshot.c +++ b/source/dnode/vnode/src/vnd/vnodeSnapshot.c @@ -256,7 +256,6 @@ int32_t vnodeSnapRead(SVSnapReader *pReader, uint8_t **ppData, uint32_t *nData) if (code) { pReader->streamStateDone = 1; pReader->pStreamStateReader = NULL; - code = 0; goto _err; } } From 2aa88dfe9b9bbe06a7ad595b486abba8de310242 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Mon, 9 Oct 2023 15:53:42 +0800 Subject: [PATCH 029/195] fix mem leak --- source/dnode/vnode/src/tq/tqStreamStateSnap.c | 1 + source/libs/stream/inc/streamBackendRocksdb.h | 2 + source/libs/stream/src/streamBackendRocksdb.c | 9 ++- source/libs/stream/src/streamSnapshot.c | 77 +++++++++++-------- 4 files changed, 56 insertions(+), 33 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqStreamStateSnap.c b/source/dnode/vnode/src/tq/tqStreamStateSnap.c index 53f9cc3c92..38cd1e1b76 100644 --- a/source/dnode/vnode/src/tq/tqStreamStateSnap.c +++ b/source/dnode/vnode/src/tq/tqStreamStateSnap.c @@ -104,6 +104,7 @@ int32_t streamStateSnapRead(SStreamStateReader* pReader, uint8_t** ppData) { pHdr->type = SNAP_DATA_STREAM_STATE_BACKEND; pHdr->size = len; memcpy(pHdr->data, rowData, len); + taosMemoryFree(rowData); tqDebug("vgId:%d, vnode stream-state snapshot read data success", TD_VID(pReader->pTq->pVnode)); return code; diff --git a/source/libs/stream/inc/streamBackendRocksdb.h b/source/libs/stream/inc/streamBackendRocksdb.h index b79979df67..c8d5797dad 100644 --- a/source/libs/stream/inc/streamBackendRocksdb.h +++ b/source/libs/stream/inc/streamBackendRocksdb.h @@ -181,5 +181,7 @@ int32_t streamBackendTriggerChkp(void* pMeta, char* dst); int32_t streamBackendAddInUseChkp(void* arg, int64_t chkpId); int32_t streamBackendDelInUseChkp(void* arg, int64_t chkpId); +int32_t taskBackendBuildSnap(void* arg, int64_t chkpId); + // int32_t streamDefaultIter_rocksdb(SStreamState* pState, const void* start, const void* end, SArray* result); #endif \ No newline at end of file diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index f936b569ae..5e48a345b1 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -950,7 +950,14 @@ int32_t chkpPreBuildDir(char* path, int64_t chkpId, char** chkpDir, char** chkpI return 0; } - +int32_t taskBackendBuildSnap(void* arg, int64_t chkpId) { + SStreamMeta* pMeta = arg; + void* pIter = taosHashIterate(pMeta->pTaskBackendUnique, NULL); + while (pIter) { + pIter = taosHashIterate(pMeta->pTaskBackendUnique, pIter); + } + return 0; +} int32_t streamBackendTriggerChkp(void* arg, char* dst) { return 0; // SStreamMeta* pMeta = arg; diff --git a/source/libs/stream/src/streamSnapshot.c b/source/libs/stream/src/streamSnapshot.c index 42d77fc4c0..88790f2511 100644 --- a/source/libs/stream/src/streamSnapshot.c +++ b/source/libs/stream/src/streamSnapshot.c @@ -108,44 +108,57 @@ TdFilePtr streamOpenFile(char* path, char* name, int32_t opt) { return taosOpenFile(fullname, opt); } -int32_t streamSnapHandleInit(SStreamSnapHandle* pHandle, char* path, int64_t chkpId, void* pMeta) { - // impl later - int len = strlen(path); - char* tdir = taosMemoryCalloc(1, len + 256); - memcpy(tdir, path, len); +int32_t streamStateSnapBuild(void* arg, char* path, int64_t chkpId) { + return taskBackendBuildSnap(arg, chkpId); + // int32_t code = 0; + // int8_t validChkp = 0; - int32_t code = 0; + // int len = strlen(path); + // char* tpath = taosMemoryCalloc(1, len + 256); + // memcpy(tpath, path, len); - int8_t validChkp = 0; - if (chkpId != 0) { - sprintf(tdir, "%s%s%s%s%s%scheckpoint%" PRId64 "", path, TD_DIRSEP, "stream", TD_DIRSEP, "checkpoints", TD_DIRSEP, - chkpId); - if (taosIsDir(tdir)) { - validChkp = 1; - qInfo("%s start to read snap %s", STREAM_STATE_TRANSFER, tdir); - streamBackendAddInUseChkp(pMeta, chkpId); - } else { - qWarn("%s failed to read from %s, reason: dir not exist,retry to default state dir", STREAM_STATE_TRANSFER, tdir); - } - } + // SStreamMeta *pMeta = arg; + // if (chkpId != 0) { + // sprintf(tpath, "%s%s%s%s%s%scheckpoint%" PRId64 "", path, TD_DIRSEP, "stream", TD_DIRSEP, "checkpoints", + // TD_DIRSEP, + // chkpId); + // if (taosIsDir(tpath)) { + // validChkp = 1; + // qInfo("%s start to read snap %s", STREAM_STATE_TRANSFER, tpath); + // streamBackendAddInUseChkp(pMeta, chkpId); + // } else { + // qWarn("%s failed to read from %s, reason: dir not exist,retry to default state dir", STREAM_STATE_TRANSFER, + // tpath); + // } + // } // no checkpoint specified or not exists invalid checkpoint, do checkpoint at default path and translate it - if (validChkp == 0) { - sprintf(tdir, "%s%s%s%s%s", path, TD_DIRSEP, "stream", TD_DIRSEP, "state"); - char* chkpdir = taosMemoryCalloc(1, len + 256); - sprintf(chkpdir, "%s%s%s", tdir, TD_DIRSEP, "tmp"); - taosMemoryFree(tdir); + // if (validChkp == 0) { + // sprintf(tpath, "%s%s%s%s%s", path, TD_DIRSEP, "stream", TD_DIRSEP, "state"); + // char* chkpdir = taosMemoryCalloc(1, len + 256); + // sprintf(chkpdir, "%s%s%s", tpath, TD_DIRSEP, "tmp"); + // taosMemoryFree(tpath); - tdir = chkpdir; - qInfo("%s start to trigger checkpoint on %s", STREAM_STATE_TRANSFER, tdir); + // tpath = chkpdir; + // qInfo("%s start to trigger checkpoint on %s", STREAM_STATE_TRANSFER, tpath); - code = streamBackendTriggerChkp(pMeta, tdir); - if (code != 0) { - qError("%s failed to trigger chekckpoint at %s", STREAM_STATE_TRANSFER, tdir); - taosMemoryFree(tdir); - return code; - } - chkpId = 0; + // code = streamBackendTriggerChkp(arg, tpath); + // if (code != 0) { + // qError("%s failed to trigger chekckpoint at %s", STREAM_STATE_TRANSFER, tpath); + // taosMemoryFree(tpath); + // return code; + // } + // chkpId = 0; + // } + + //*dstPath = tpath; +} +int32_t streamSnapHandleInit(SStreamSnapHandle* pHandle, char* path, int64_t chkpId, void* pMeta) { + // impl later + char* tdir = NULL; + int32_t code = streamStateSnapBuild(pMeta, path, chkpId); + if (code != 0) { + return -1; } qInfo("%s start to read dir: %s", STREAM_STATE_TRANSFER, tdir); From 28421cd35e37a0423fe69ce2cd147fe83ad4d72b Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Tue, 10 Oct 2023 20:51:41 +0800 Subject: [PATCH 030/195] refactor backend --- source/libs/stream/inc/streamBackendRocksdb.h | 3 +- source/libs/stream/inc/streamInt.h | 5 + source/libs/stream/src/streamBackendRocksdb.c | 19 +- source/libs/stream/src/streamSnapshot.c | 339 ++++++++++-------- 4 files changed, 204 insertions(+), 162 deletions(-) diff --git a/source/libs/stream/inc/streamBackendRocksdb.h b/source/libs/stream/inc/streamBackendRocksdb.h index c8d5797dad..9aa616e190 100644 --- a/source/libs/stream/inc/streamBackendRocksdb.h +++ b/source/libs/stream/inc/streamBackendRocksdb.h @@ -80,13 +80,14 @@ void* streamBackendInit(const char* path, int64_t chkpId); void streamBackendCleanup(void* arg); void streamBackendHandleCleanup(void* arg); int32_t streamBackendLoadCheckpointInfo(void* pMeta); -int32_t streamBackendDoCheckpoint(void* pMeta, uint64_t checkpointId); +int32_t streamBackendDoCheckpoint(void* pMeta, int64_t checkpointId); SListNode* streamBackendAddCompare(void* backend, void* arg); void streamBackendDelCompare(void* backend, void* arg); int32_t streamStateConvertDataFormat(char* path, char* key, void* cfInst); STaskBackendWrapper* taskBackendOpen(char* path, char* key); void taskBackendDestroy(void* pBackend); +int32_t taskBackendDoCheckpoint(void* arg, int64_t chkpId); void* taskBackendAddRef(void* pTaskBackend); void taskBackendRemoveRef(void* pTaskBackend); diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index 54aefc8962..f200c714ab 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -40,6 +40,11 @@ typedef struct { SRpcMsg msg; } SStreamContinueExecInfo; +typedef struct { + int64_t streamId; + int64_t taskId; + int64_t chkpId; +} SStreamTaskSnap; extern SStreamGlobalEnv streamEnv; extern int32_t streamBackendId; extern int32_t streamBackendCfWrapperId; diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 5e48a345b1..917bc0f796 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -953,7 +953,15 @@ int32_t chkpPreBuildDir(char* path, int64_t chkpId, char** chkpDir, char** chkpI int32_t taskBackendBuildSnap(void* arg, int64_t chkpId) { SStreamMeta* pMeta = arg; void* pIter = taosHashIterate(pMeta->pTaskBackendUnique, NULL); + int32_t code = 0; + while (pIter) { + STaskBackendWrapper* pBackend = *(STaskBackendWrapper**)pIter; + taskBackendAddRef(pBackend); + + code = taskBackendDoCheckpoint((STaskBackendWrapper*)pBackend, chkpId); + + taskBackendRemoveRef(pBackend); pIter = taosHashIterate(pMeta->pTaskBackendUnique, pIter); } return 0; @@ -1024,7 +1032,10 @@ int32_t streamBackendDelInUseChkp(void* arg, int64_t chkpId) { // taosWUnLockLatch(&pMeta->chkpDirLock); } -int32_t taskBackendDoCheckpoint(void* arg, uint64_t chkpId) { +/* + 0 +*/ +int32_t taskBackendDoCheckpoint(void* arg, int64_t chkpId) { STaskBackendWrapper* pBackend = arg; int64_t st = taosGetTimestampMs(); int32_t code = -1; @@ -1065,7 +1076,7 @@ _EXIT: taosReleaseRef(taskBackendWrapperId, refId); return code; } -int32_t streamBackendDoCheckpoint(void* arg, uint64_t chkpId) { return taskBackendDoCheckpoint(arg, chkpId); } +int32_t streamBackendDoCheckpoint(void* arg, int64_t chkpId) { return taskBackendDoCheckpoint(arg, chkpId); } SListNode* streamBackendAddCompare(void* backend, void* arg) { SBackendWrapper* pHandle = (SBackendWrapper*)backend; @@ -1577,8 +1588,8 @@ void* taskBackendAddRef(void* pTaskBackend) { return taosAcquireRef(taskBackendWrapperId, pBackend->refId); } void taskBackendRemoveRef(void* pTaskBackend) { - // STaskBackendWrapper* pBackend = pTaskBackend; - // taosReleaseRef(taskBackendWrapperId, pBackend->refId); + STaskBackendWrapper* pBackend = pTaskBackend; + taosReleaseRef(taskBackendWrapperId, pBackend->refId); } // void taskBackendDestroy(STaskBackendWrapper* wrapper); diff --git a/source/libs/stream/src/streamSnapshot.c b/source/libs/stream/src/streamSnapshot.c index 88790f2511..a4a507fc13 100644 --- a/source/libs/stream/src/streamSnapshot.c +++ b/source/libs/stream/src/streamSnapshot.c @@ -17,6 +17,7 @@ #include "query.h" #include "rocksdb/c.h" #include "streamBackendRocksdb.h" +#include "streamInt.h" #include "tcommon.h" enum SBackendFileType { @@ -39,7 +40,26 @@ typedef struct SBackendFile { SArray* pSst; char* pCheckpointMeta; char* path; + } SBanckendFile; + +typedef struct SBackendSnapFiles2 { + char* pCurrent; + char* pMainfest; + char* pOptions; + SArray* pSst; + char* pCheckpointMeta; + char* path; + + int64_t checkpointId; + int64_t seraial; + int64_t offset; + TdFilePtr fd; + int8_t filetype; + SArray* pFileList; + int32_t currFileIdx; + +} SBackendSnapFile2; struct SStreamSnapHandle { void* handle; SBanckendFile* pBackendFile; @@ -50,6 +70,9 @@ struct SStreamSnapHandle { int8_t filetype; SArray* pFileList; int32_t currFileIdx; + + SArray* pBackendSnapSet; + int32_t currIdx; }; struct SStreamSnapBlockHdr { int8_t type; @@ -108,202 +131,204 @@ TdFilePtr streamOpenFile(char* path, char* name, int32_t opt) { return taosOpenFile(fullname, opt); } -int32_t streamStateSnapBuild(void* arg, char* path, int64_t chkpId) { - return taskBackendBuildSnap(arg, chkpId); - // int32_t code = 0; - // int8_t validChkp = 0; +int32_t streamBackendGetSnapInfo(void* arg, char* path, int64_t chkpId) { return taskBackendBuildSnap(arg, chkpId); } - // int len = strlen(path); - // char* tpath = taosMemoryCalloc(1, len + 256); - // memcpy(tpath, path, len); +void snapFileDebugInfo(SBackendSnapFile2* pSnapFile) { + char* buf = taosMemoryCalloc(1, 512); + sprintf(buf, "[current: %s,", pSnapFile->pCurrent); + sprintf(buf + strlen(buf), "MANIFEST: %s,", pSnapFile->pMainfest); + sprintf(buf + strlen(buf), "options: %s,", pSnapFile->pOptions); - // SStreamMeta *pMeta = arg; - // if (chkpId != 0) { - // sprintf(tpath, "%s%s%s%s%s%scheckpoint%" PRId64 "", path, TD_DIRSEP, "stream", TD_DIRSEP, "checkpoints", - // TD_DIRSEP, - // chkpId); - // if (taosIsDir(tpath)) { - // validChkp = 1; - // qInfo("%s start to read snap %s", STREAM_STATE_TRANSFER, tpath); - // streamBackendAddInUseChkp(pMeta, chkpId); - // } else { - // qWarn("%s failed to read from %s, reason: dir not exist,retry to default state dir", STREAM_STATE_TRANSFER, - // tpath); - // } - // } + for (int i = 0; i < taosArrayGetSize(pSnapFile->pSst); i++) { + char* name = taosArrayGetP(pSnapFile->pSst, i); + sprintf(buf + strlen(buf), "%s,", name); + } + sprintf(buf + strlen(buf) - 1, "]"); - // no checkpoint specified or not exists invalid checkpoint, do checkpoint at default path and translate it - // if (validChkp == 0) { - // sprintf(tpath, "%s%s%s%s%s", path, TD_DIRSEP, "stream", TD_DIRSEP, "state"); - // char* chkpdir = taosMemoryCalloc(1, len + 256); - // sprintf(chkpdir, "%s%s%s", tpath, TD_DIRSEP, "tmp"); - // taosMemoryFree(tpath); - - // tpath = chkpdir; - // qInfo("%s start to trigger checkpoint on %s", STREAM_STATE_TRANSFER, tpath); - - // code = streamBackendTriggerChkp(arg, tpath); - // if (code != 0) { - // qError("%s failed to trigger chekckpoint at %s", STREAM_STATE_TRANSFER, tpath); - // taosMemoryFree(tpath); - // return code; - // } - // chkpId = 0; - // } - - //*dstPath = tpath; + qInfo("%s get file list: %s", STREAM_STATE_TRANSFER, buf); + taosMemoryFree(buf); } -int32_t streamSnapHandleInit(SStreamSnapHandle* pHandle, char* path, int64_t chkpId, void* pMeta) { - // impl later - char* tdir = NULL; - int32_t code = streamStateSnapBuild(pMeta, path, chkpId); - if (code != 0) { + +int32_t snapFileCvtMeta(SBackendSnapFile2* pSnapFile) { + SBackendFileItem item; + // current + item.name = pSnapFile->pCurrent; + item.type = ROCKSDB_CURRENT_TYPE; + streamGetFileSize(pSnapFile->path, item.name, &item.size); + taosArrayPush(pSnapFile->pFileList, &item); + + // mainfest + item.name = pSnapFile->pMainfest; + item.type = ROCKSDB_MAINFEST_TYPE; + streamGetFileSize(pSnapFile->path, item.name, &item.size); + taosArrayPush(pSnapFile->pFileList, &item); + + // options + item.name = pSnapFile->pOptions; + item.type = ROCKSDB_OPTIONS_TYPE; + streamGetFileSize(pSnapFile->path, item.name, &item.size); + taosArrayPush(pSnapFile->pFileList, &item); + // sst + for (int i = 0; i < taosArrayGetSize(pSnapFile->pSst); i++) { + char* sst = taosArrayGetP(pSnapFile->pSst, i); + item.name = sst; + item.type = ROCKSDB_SST_TYPE; + streamGetFileSize(pSnapFile->path, item.name, &item.size); + taosArrayPush(pSnapFile->pFileList, &item); + } + // meta + item.name = pSnapFile->pCheckpointMeta; + item.type = ROCKSDB_CHECKPOINT_META_TYPE; + if (streamGetFileSize(pSnapFile->path, item.name, &item.size) == 0) { + taosArrayPush(pSnapFile->pFileList, &item); + } + return 0; +} +int32_t snapFileReadMeta(SBackendSnapFile2* pSnapFile) { + TdDirPtr pDir = taosOpenDir(pSnapFile->path); + if (NULL == pDir) { + qError("%s failed to open %s", STREAM_STATE_TRANSFER, pSnapFile->path); return -1; } - qInfo("%s start to read dir: %s", STREAM_STATE_TRANSFER, tdir); - - TdDirPtr pDir = taosOpenDir(tdir); - if (NULL == pDir) { - qError("%s failed to open %s", STREAM_STATE_TRANSFER, tdir); - goto _err; - } - - SBanckendFile* pFile = taosMemoryCalloc(1, sizeof(SBanckendFile)); - pHandle->pBackendFile = pFile; - pHandle->checkpointId = chkpId; - pHandle->seraial = 0; - - pFile->path = tdir; - pFile->pSst = taosArrayInit(16, sizeof(void*)); - TdDirEntryPtr pDirEntry; while ((pDirEntry = taosReadDir(pDir)) != NULL) { char* name = taosGetDirEntryName(pDirEntry); if (strlen(name) >= strlen(ROCKSDB_CURRENT) && 0 == strncmp(name, ROCKSDB_CURRENT, strlen(ROCKSDB_CURRENT))) { - pFile->pCurrent = taosStrdup(name); + pSnapFile->pCurrent = taosStrdup(name); continue; } if (strlen(name) >= strlen(ROCKSDB_MAINFEST) && 0 == strncmp(name, ROCKSDB_MAINFEST, strlen(ROCKSDB_MAINFEST))) { - pFile->pMainfest = taosStrdup(name); + pSnapFile->pMainfest = taosStrdup(name); continue; } if (strlen(name) >= strlen(ROCKSDB_OPTIONS) && 0 == strncmp(name, ROCKSDB_OPTIONS, strlen(ROCKSDB_OPTIONS))) { - pFile->pOptions = taosStrdup(name); + pSnapFile->pOptions = taosStrdup(name); continue; } if (strlen(name) >= strlen(ROCKSDB_CHECKPOINT_META) && 0 == strncmp(name, ROCKSDB_CHECKPOINT_META, strlen(ROCKSDB_CHECKPOINT_META))) { - pFile->pCheckpointMeta = taosStrdup(name); + pSnapFile->pCheckpointMeta = taosStrdup(name); continue; } if (strlen(name) >= strlen(ROCKSDB_SST) && 0 == strncmp(name + strlen(name) - strlen(ROCKSDB_SST), ROCKSDB_SST, strlen(ROCKSDB_SST))) { char* sst = taosStrdup(name); - taosArrayPush(pFile->pSst, &sst); + taosArrayPush(pSnapFile->pSst, &sst); } } - { - char* buf = taosMemoryCalloc(1, 512); - sprintf(buf, "[current: %s,", pFile->pCurrent); - sprintf(buf + strlen(buf), "MANIFEST: %s,", pFile->pMainfest); - sprintf(buf + strlen(buf), "options: %s,", pFile->pOptions); - - for (int i = 0; i < taosArrayGetSize(pFile->pSst); i++) { - char* name = taosArrayGetP(pFile->pSst, i); - sprintf(buf + strlen(buf), "%s,", name); - } - sprintf(buf + strlen(buf) - 1, "]"); - - qInfo("%s get file list: %s", STREAM_STATE_TRANSFER, buf); - taosMemoryFree(buf); - } - taosCloseDir(&pDir); - - if (pFile->pCurrent == NULL) { - qError("%s failed to open %s, reason: no valid file", STREAM_STATE_TRANSFER, tdir); - code = -1; - tdir = NULL; - goto _err; - } - SArray* list = taosArrayInit(64, sizeof(SBackendFileItem)); - - SBackendFileItem item; - // current - item.name = pFile->pCurrent; - item.type = ROCKSDB_CURRENT_TYPE; - streamGetFileSize(pFile->path, item.name, &item.size); - taosArrayPush(list, &item); - - // mainfest - item.name = pFile->pMainfest; - item.type = ROCKSDB_MAINFEST_TYPE; - streamGetFileSize(pFile->path, item.name, &item.size); - taosArrayPush(list, &item); - - // options - item.name = pFile->pOptions; - item.type = ROCKSDB_OPTIONS_TYPE; - streamGetFileSize(pFile->path, item.name, &item.size); - taosArrayPush(list, &item); - // sst - for (int i = 0; i < taosArrayGetSize(pFile->pSst); i++) { - char* sst = taosArrayGetP(pFile->pSst, i); - item.name = sst; - item.type = ROCKSDB_SST_TYPE; - streamGetFileSize(pFile->path, item.name, &item.size); - taosArrayPush(list, &item); - } - // meta - item.name = pFile->pCheckpointMeta; - item.type = ROCKSDB_CHECKPOINT_META_TYPE; - if (streamGetFileSize(pFile->path, item.name, &item.size) == 0) { - taosArrayPush(list, &item); - } - - pHandle->pBackendFile = pFile; - - pHandle->currFileIdx = 0; - pHandle->pFileList = list; - pHandle->seraial = 0; - pHandle->offset = 0; - pHandle->handle = pMeta; return 0; +} +int32_t streamBackendSnapInitFile(char* path, SStreamTaskSnap* pSnap, SBackendSnapFile2* pSnapFile) { + // SBanckendFile* pFile = taosMemoryCalloc(1, sizeof(SBanckendFile)); + int32_t code = -1; + + char* snapPath = taosMemoryCalloc(1, strlen(path) + 256); + sprintf(snapPath, "%s%s%" PRId64 "_%" PRId64 "%s%s%s%s%scheckpoint%" PRId64 "", path, TD_DIRSEP, pSnap->streamId, + pSnap->taskId, TD_DIRSEP, "state", TD_DIRSEP, "checkpoints", TD_DIRSEP, pSnap->chkpId); + if (taosIsDir(snapPath)) { + goto _ERROR; + } + + pSnapFile->pSst = taosArrayInit(16, sizeof(void*)); + pSnapFile->pFileList = taosArrayInit(64, sizeof(SBackendFileItem)); + pSnapFile->path = snapPath; + if ((code = snapFileReadMeta(pSnapFile)) != 0) { + goto _ERROR; + } + if ((code = snapFileCvtMeta(pSnapFile)) != 0) { + goto _ERROR; + } + + snapFileDebugInfo(pSnapFile); + + code = 0; + +_ERROR: + taosMemoryFree(snapPath); + return code; +} +void snapFileDestroy(SBackendSnapFile2* pSnap) { + taosMemoryFree(pSnap->pCheckpointMeta); + taosMemoryFree(pSnap->pCurrent); + taosMemoryFree(pSnap->pMainfest); + taosMemoryFree(pSnap->pOptions); + taosMemoryFree(pSnap->path); + for (int i = 0; i < taosArrayGetSize(pSnap->pSst); i++) { + char* sst = taosArrayGetP(pSnap->pSst, i); + taosMemoryFree(sst); + } + taosArrayDestroy(pSnap->pFileList); + taosArrayDestroy(pSnap->pSst); + taosCloseFile(&pSnap->fd); + + return; +} +int32_t streamSnapHandleInit(SStreamSnapHandle* pHandle, char* path, int64_t chkpId, void* pMeta) { + // impl later + + SArray* pSnapSet = NULL; + int32_t code = streamBackendGetSnapInfo(pMeta, path, chkpId); + if (code != 0) { + return -1; + } + + SArray* pBdSnapSet = taosArrayInit(8, sizeof(SBackendSnapFile2)); + + for (int i = 0; i < taosArrayGetSize(pSnapSet); i++) { + SStreamTaskSnap* pSnap = taosArrayGet(pSnapSet, i); + + SBackendSnapFile2 snapFile = {0}; + code = streamBackendSnapInitFile(path, pSnap, &snapFile); + ASSERT(code == 0); + taosArrayPush(pBdSnapSet, &snapFile); + } + + pHandle->pBackendSnapSet = pBdSnapSet; + pHandle->currIdx = 0; + return 0; + _err: streamSnapHandleDestroy(pHandle); - taosMemoryFreeClear(tdir); code = -1; return code; } void streamSnapHandleDestroy(SStreamSnapHandle* handle) { - SBanckendFile* pFile = handle->pBackendFile; + // SBanckendFile* pFile = handle->pBackendFile; + if (handle->pBackendSnapSet) { + for (int i = 0; i < taosArrayGetSize(handle->pBackendSnapSet); i++) { + SBackendSnapFile2* pSnapFile = taosArrayGet(handle->pBackendSnapSet, i); + snapFileDestroy(pSnapFile); + } + taosArrayDestroy(handle->pBackendSnapSet); + } - if (handle->checkpointId == 0) { - // del tmp dir - if (pFile && taosIsDir(pFile->path)) { - taosRemoveDir(pFile->path); - } - } else { - streamBackendDelInUseChkp(handle->handle, handle->checkpointId); - } - if (pFile) { - taosMemoryFree(pFile->pCheckpointMeta); - taosMemoryFree(pFile->pCurrent); - taosMemoryFree(pFile->pMainfest); - taosMemoryFree(pFile->pOptions); - taosMemoryFree(pFile->path); - for (int i = 0; i < taosArrayGetSize(pFile->pSst); i++) { - char* sst = taosArrayGetP(pFile->pSst, i); - taosMemoryFree(sst); - } - taosArrayDestroy(pFile->pSst); - taosMemoryFree(pFile); - } - taosArrayDestroy(handle->pFileList); - taosCloseFile(&handle->fd); + // if (handle->checkpointId == 0) { + // // del tmp dir + // if (pFile && taosIsDir(pFile->path)) { + // taosRemoveDir(pFile->path); + // } + // } else { + // streamBackendDelInUseChkp(handle->handle, handle->checkpointId); + // } + // if (pFile) { + // taosMemoryFree(pFile->pCheckpointMeta); + // taosMemoryFree(pFile->pCurrent); + // taosMemoryFree(pFile->pMainfest); + // taosMemoryFree(pFile->pOptions); + // taosMemoryFree(pFile->path); + // for (int i = 0; i < taosArrayGetSize(pFile->pSst); i++) { + // char* sst = taosArrayGetP(pFile->pSst, i); + // taosMemoryFree(sst); + // } + // taosArrayDestroy(pFile->pSst); + // taosMemoryFree(pFile); + // } + // taosArrayDestroy(handle->pFileList); + // taosCloseFile(&handle->fd); return; } From 4cff121a38ccffbd4f38645d9bf395713155bbf8 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 11 Oct 2023 14:46:41 +0800 Subject: [PATCH 031/195] refactor backend --- source/libs/stream/src/streamSnapshot.c | 98 +++++++++++++++---------- 1 file changed, 58 insertions(+), 40 deletions(-) diff --git a/source/libs/stream/src/streamSnapshot.c b/source/libs/stream/src/streamSnapshot.c index a4a507fc13..64ff7b7569 100644 --- a/source/libs/stream/src/streamSnapshot.c +++ b/source/libs/stream/src/streamSnapshot.c @@ -51,13 +51,14 @@ typedef struct SBackendSnapFiles2 { char* pCheckpointMeta; char* path; - int64_t checkpointId; - int64_t seraial; - int64_t offset; - TdFilePtr fd; - int8_t filetype; - SArray* pFileList; - int32_t currFileIdx; + int64_t checkpointId; + int64_t seraial; + int64_t offset; + TdFilePtr fd; + int8_t filetype; + SArray* pFileList; + int32_t currFileIdx; + SStreamTaskSnap snapInfo; } SBackendSnapFile2; struct SStreamSnapHandle { @@ -78,10 +79,13 @@ struct SStreamSnapBlockHdr { int8_t type; int8_t flag; int64_t index; - char name[128]; - int64_t totalSize; - int64_t size; - uint8_t data[]; + // int64_t streamId; + // int64_t taskId; + SStreamTaskSnap snapInfo; + char name[128]; + int64_t totalSize; + int64_t size; + uint8_t data[]; }; struct SStreamSnapReader { void* pMeta; @@ -234,6 +238,7 @@ int32_t streamBackendSnapInitFile(char* path, SStreamTaskSnap* pSnap, SBackendSn pSnapFile->pSst = taosArrayInit(16, sizeof(void*)); pSnapFile->pFileList = taosArrayInit(64, sizeof(SBackendFileItem)); pSnapFile->path = snapPath; + pSnapFile->snapInfo = *pSnap; if ((code = snapFileReadMeta(pSnapFile)) != 0) { goto _ERROR; } @@ -355,31 +360,43 @@ int32_t streamSnapReaderClose(SStreamSnapReader* pReader) { taosMemoryFree(pReader); return 0; } + int32_t streamSnapRead(SStreamSnapReader* pReader, uint8_t** ppData, int64_t* size) { // impl later int32_t code = 0; SStreamSnapHandle* pHandle = &pReader->handle; - SBanckendFile* pFile = pHandle->pBackendFile; + int32_t idx = pHandle->currIdx; + SBackendSnapFile2* pSnapFile = taosArrayGet(pHandle->pBackendSnapSet, idx); + SBackendFileItem* item = NULL; - SBackendFileItem* item = taosArrayGet(pHandle->pFileList, pHandle->currFileIdx); +_NEXT: + + if (pSnapFile->fd == NULL) { + if (pSnapFile->currFileIdx >= taosArrayGetSize(pSnapFile->pFileList)) { + if (pHandle->currIdx + 1 < taosArrayGetSize(pHandle->pBackendSnapSet)) { + pHandle->currIdx += 1; + + pSnapFile = taosArrayGet(pHandle->pBackendSnapSet, pHandle->currIdx); + goto _NEXT; + } else { + *ppData = NULL; + *size = 0; + return 0; + } - if (pHandle->fd == NULL) { - if (pHandle->currFileIdx >= taosArrayGetSize(pHandle->pFileList)) { - // finish - *ppData = NULL; - *size = 0; - return 0; } else { - pHandle->fd = streamOpenFile(pFile->path, item->name, TD_FILE_READ); + item = taosArrayGet(pSnapFile->pFileList, pSnapFile->currFileIdx); + pSnapFile->fd = streamOpenFile(pSnapFile->path, item->name, TD_FILE_READ); qDebug("%s open file %s, current offset:%" PRId64 ", size:% " PRId64 ", file no.%d", STREAM_STATE_TRANSFER, - item->name, (int64_t)pHandle->offset, item->size, pHandle->currFileIdx); + item->name, (int64_t)pSnapFile->offset, item->size, pSnapFile->currFileIdx); } } qDebug("%s start to read file %s, current offset:%" PRId64 ", size:%" PRId64 ", file no.%d", STREAM_STATE_TRANSFER, - item->name, (int64_t)pHandle->offset, item->size, pHandle->currFileIdx); + item->name, (int64_t)pSnapFile->offset, item->size, pSnapFile->currFileIdx); + uint8_t* buf = taosMemoryCalloc(1, sizeof(SStreamSnapBlockHdr) + kBlockSize); - int64_t nread = taosPReadFile(pHandle->fd, buf + sizeof(SStreamSnapBlockHdr), kBlockSize, pHandle->offset); + int64_t nread = taosPReadFile(pHandle->fd, buf + sizeof(SStreamSnapBlockHdr), kBlockSize, pSnapFile->offset); if (nread == -1) { code = TAOS_SYSTEM_ERROR(terrno); qError("%s snap failed to read snap, file name:%s, type:%d,reason:%s", STREAM_STATE_TRANSFER, item->name, @@ -388,43 +405,44 @@ int32_t streamSnapRead(SStreamSnapReader* pReader, uint8_t** ppData, int64_t* si } else if (nread > 0 && nread <= kBlockSize) { // left bytes less than kBlockSize qDebug("%s read file %s, current offset:%" PRId64 ",size:% " PRId64 ", file no.%d", STREAM_STATE_TRANSFER, - item->name, (int64_t)pHandle->offset, item->size, pHandle->currFileIdx); - pHandle->offset += nread; - if (pHandle->offset >= item->size || nread < kBlockSize) { - taosCloseFile(&pHandle->fd); - pHandle->offset = 0; - pHandle->currFileIdx += 1; + item->name, (int64_t)pSnapFile->offset, item->size, pSnapFile->currFileIdx); + pSnapFile->offset += nread; + if (pSnapFile->offset >= item->size || nread < kBlockSize) { + taosCloseFile(&pSnapFile->fd); + pSnapFile->offset = 0; + pSnapFile->currFileIdx += 1; } } else { qDebug("%s no data read, close file no.%d, move to next file, open and read", STREAM_STATE_TRANSFER, - pHandle->currFileIdx); - taosCloseFile(&pHandle->fd); - pHandle->offset = 0; - pHandle->currFileIdx += 1; + pSnapFile->currFileIdx); + taosCloseFile(&pSnapFile->fd); + pSnapFile->offset = 0; + pSnapFile->currFileIdx += 1; - if (pHandle->currFileIdx >= taosArrayGetSize(pHandle->pFileList)) { + if (pSnapFile->currFileIdx >= taosArrayGetSize(pSnapFile->pFileList)) { // finish *ppData = NULL; *size = 0; return 0; } - item = taosArrayGet(pHandle->pFileList, pHandle->currFileIdx); - pHandle->fd = streamOpenFile(pFile->path, item->name, TD_FILE_READ); + item = taosArrayGet(pSnapFile->pFileList, pSnapFile->currFileIdx); + pSnapFile->fd = streamOpenFile(pSnapFile->path, item->name, TD_FILE_READ); - nread = taosPReadFile(pHandle->fd, buf + sizeof(SStreamSnapBlockHdr), kBlockSize, pHandle->offset); - pHandle->offset += nread; + nread = taosPReadFile(pSnapFile->fd, buf + sizeof(SStreamSnapBlockHdr), kBlockSize, pSnapFile->offset); + pSnapFile->offset += nread; qDebug("%s open file and read file %s, current offset:%" PRId64 ", size:% " PRId64 ", file no.%d", - STREAM_STATE_TRANSFER, item->name, (int64_t)pHandle->offset, item->size, pHandle->currFileIdx); + STREAM_STATE_TRANSFER, item->name, (int64_t)pSnapFile->offset, item->size, pSnapFile->currFileIdx); } SStreamSnapBlockHdr* pHdr = (SStreamSnapBlockHdr*)buf; pHdr->size = nread; pHdr->type = item->type; pHdr->totalSize = item->size; + pHdr->snapInfo = pSnapFile->snapInfo; memcpy(pHdr->name, item->name, strlen(item->name)); - pHandle->seraial += nread; + pSnapFile->seraial += nread; *ppData = buf; *size = sizeof(SStreamSnapBlockHdr) + nread; From 0baef335d70fdcd53b86b9600ad19ebd45b5f629 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 11 Oct 2023 16:03:04 +0800 Subject: [PATCH 032/195] refactor backend --- source/libs/stream/src/streamSnapshot.c | 125 +++++++++++++++++------- 1 file changed, 89 insertions(+), 36 deletions(-) diff --git a/source/libs/stream/src/streamSnapshot.c b/source/libs/stream/src/streamSnapshot.c index 64ff7b7569..a4363f46b1 100644 --- a/source/libs/stream/src/streamSnapshot.c +++ b/source/libs/stream/src/streamSnapshot.c @@ -59,6 +59,7 @@ typedef struct SBackendSnapFiles2 { SArray* pFileList; int32_t currFileIdx; SStreamTaskSnap snapInfo; + int8_t inited; } SBackendSnapFile2; struct SStreamSnapHandle { @@ -71,6 +72,7 @@ struct SStreamSnapHandle { int8_t filetype; SArray* pFileList; int32_t currFileIdx; + char* metaPath; SArray* pBackendSnapSet; int32_t currIdx; @@ -310,6 +312,7 @@ void streamSnapHandleDestroy(SStreamSnapHandle* handle) { } taosArrayDestroy(handle->pBackendSnapSet); } + taosMemoryFree(handle->metaPath); // if (handle->checkpointId == 0) { // // del tmp dir @@ -455,76 +458,126 @@ int32_t streamSnapWriterOpen(void* pMeta, int64_t sver, int64_t ever, char* path if (pWriter == NULL) { return TSDB_CODE_OUT_OF_MEMORY; } + SBackendSnapFile2 snapFile = {0}; + SStreamSnapHandle* pHandle = &pWriter->handle; + pHandle->pBackendSnapSet = taosArrayInit(8, sizeof(SBackendSnapFile2)); - SBanckendFile* pFile = taosMemoryCalloc(1, sizeof(SBanckendFile)); - pFile->path = taosStrdup(path); - SArray* list = taosArrayInit(64, sizeof(SBackendFileItem)); + taosArrayPush(pHandle->pBackendSnapSet, &snapFile); + pHandle->currIdx = 0; + pHandle->metaPath = taosStrdup(path); - SBackendFileItem item; - item.name = taosStrdup((char*)ROCKSDB_CURRENT); - item.type = ROCKSDB_CURRENT_TYPE; - taosArrayPush(list, &item); + // SBanckendFile* pFile = taosMemoryCalloc(1, sizeof(SBanckendFile)); + // pFile->path = taosStrdup(path); + // SArray* list = taosArrayInit(64, sizeof(SBackendFileItem)); - pHandle->pBackendFile = pFile; + // SBackendFileItem item; + // item.name = taosStrdup((char*)ROCKSDB_CURRENT); + // item.type = ROCKSDB_CURRENT_TYPE; + // taosArrayPush(list, &item); - pHandle->pFileList = list; - pHandle->currFileIdx = 0; - pHandle->offset = 0; + // pHandle->pBackendFile = pFile; + + // pHandle->pFileList = list; + // pHandle->currFileIdx = 0; + // pHandle->offset = 0; *ppWriter = pWriter; return 0; } -int32_t streamSnapWrite(SStreamSnapWriter* pWriter, uint8_t* pData, uint32_t nData) { - int32_t code = 0; +int32_t snapInfoEqual(SStreamTaskSnap* a, SStreamTaskSnap* b) { + if (a->streamId != b->streamId || a->taskId != b->taskId || a->chkpId != b->chkpId) { + return 0; + } + return 1; +} +int32_t streamSnapWriteImpl(SStreamSnapWriter* pWriter, uint8_t* pData, uint32_t nData, + SBackendSnapFile2* pBackendFile) { + int code = -1; SStreamSnapBlockHdr* pHdr = (SStreamSnapBlockHdr*)pData; SStreamSnapHandle* pHandle = &pWriter->handle; - SBanckendFile* pFile = pHandle->pBackendFile; - SBackendFileItem* pItem = taosArrayGet(pHandle->pFileList, pHandle->currFileIdx); + SStreamTaskSnap snapInfo = pHdr->snapInfo; - if (pHandle->fd == NULL) { - pHandle->fd = streamOpenFile(pFile->path, pItem->name, TD_FILE_CREATE | TD_FILE_WRITE | TD_FILE_APPEND); - if (pHandle->fd == NULL) { + SStreamTaskSnap* pSnapInfo = &pBackendFile->snapInfo; + + SBackendFileItem* pItem = taosArrayGet(pBackendFile->pFileList, pBackendFile->currFileIdx); + + if (pBackendFile->fd == 0) { + pBackendFile->fd = streamOpenFile(pHandle->metaPath, pItem->name, TD_FILE_CREATE | TD_FILE_WRITE | TD_FILE_APPEND); + if (pBackendFile->fd == NULL) { code = TAOS_SYSTEM_ERROR(terrno); - qError("%s failed to open file name:%s%s%s, reason:%s", STREAM_STATE_TRANSFER, pFile->path, TD_DIRSEP, pHdr->name, - tstrerror(code)); + qError("%s failed to open file name:%s%s%s, reason:%s", STREAM_STATE_TRANSFER, pHandle->metaPath, TD_DIRSEP, + pHdr->name, tstrerror(code)); } } - if (strlen(pHdr->name) == strlen(pItem->name) && strcmp(pHdr->name, pItem->name) == 0) { - int64_t bytes = taosPWriteFile(pHandle->fd, pHdr->data, pHdr->size, pHandle->offset); + int64_t bytes = taosPWriteFile(pBackendFile->fd, pHdr->data, pHdr->size, pBackendFile->offset); if (bytes != pHdr->size) { code = TAOS_SYSTEM_ERROR(terrno); qError("%s failed to write snap, file name:%s, reason:%s", STREAM_STATE_TRANSFER, pHdr->name, tstrerror(code)); return code; } - pHandle->offset += bytes; + pBackendFile->offset += bytes; } else { - taosCloseFile(&pHandle->fd); - pHandle->offset = 0; - pHandle->currFileIdx += 1; + taosCloseFile(&pBackendFile->fd); + pBackendFile->offset = 0; + pBackendFile->currFileIdx += 1; SBackendFileItem item; item.name = taosStrdup(pHdr->name); item.type = pHdr->type; - taosArrayPush(pHandle->pFileList, &item); + taosArrayPush(pBackendFile->pFileList, &item); - SBackendFileItem* pItem = taosArrayGet(pHandle->pFileList, pHandle->currFileIdx); - pHandle->fd = streamOpenFile(pFile->path, pItem->name, TD_FILE_CREATE | TD_FILE_WRITE | TD_FILE_APPEND); - if (pHandle->fd == NULL) { + SBackendFileItem* pItem = taosArrayGet(pBackendFile->pFileList, pBackendFile->currFileIdx); + pBackendFile->fd = streamOpenFile(pHandle->metaPath, pItem->name, TD_FILE_CREATE | TD_FILE_WRITE | TD_FILE_APPEND); + if (pBackendFile->fd == NULL) { code = TAOS_SYSTEM_ERROR(terrno); - qError("%s failed to open file name:%s%s%s, reason:%s", STREAM_STATE_TRANSFER, pFile->path, TD_DIRSEP, pHdr->name, - tstrerror(code)); + qError("%s failed to open file name:%s%s%s, reason:%s", STREAM_STATE_TRANSFER, pBackendFile->path, TD_DIRSEP, + pHdr->name, tstrerror(code)); } - taosPWriteFile(pHandle->fd, pHdr->data, pHdr->size, pHandle->offset); - pHandle->offset += pHdr->size; + taosPWriteFile(pBackendFile->fd, pHdr->data, pHdr->size, pBackendFile->offset); + pBackendFile->offset += pHdr->size; } + code = 0; +_EXIT: + return code; +} +int32_t streamSnapWrite(SStreamSnapWriter* pWriter, uint8_t* pData, uint32_t nData) { + int32_t code = 0; - // impl later - return 0; + SStreamSnapBlockHdr* pHdr = (SStreamSnapBlockHdr*)pData; + SStreamSnapHandle* pHandle = &pWriter->handle; + SStreamTaskSnap snapInfo = pHdr->snapInfo; + + SBackendSnapFile2* pBackendFile = taosArrayGet(pHandle->pBackendSnapSet, pHandle->currIdx); + if (pBackendFile->inited == 0) { + pBackendFile->snapInfo = snapInfo; + pBackendFile->pFileList = taosArrayInit(64, sizeof(SBackendFileItem)); + pBackendFile->currFileIdx = 0; + pBackendFile->offset = 0; + + SBackendFileItem item; + item.name = taosStrdup((char*)ROCKSDB_CURRENT); + item.type = ROCKSDB_CURRENT_TYPE; + taosArrayPush(pBackendFile->pFileList, &item); + + pBackendFile->inited = 1; + return streamSnapWriteImpl(pWriter, pData, nData, pBackendFile); + } else { + if (snapInfoEqual(&snapInfo, &pBackendFile->snapInfo)) { + return streamSnapWriteImpl(pWriter, pData, nData, pBackendFile); + } else { + SBackendSnapFile2 snapFile = {0}; + taosArrayPush(pHandle->pBackendSnapSet, &snapFile); + pHandle->currIdx += 1; + + return streamSnapWrite(pWriter, pData, nData); + } + } + return code; } int32_t streamSnapWriterClose(SStreamSnapWriter* pWriter, int8_t rollback) { SStreamSnapHandle* handle = &pWriter->handle; From 722d72659b5c2b38680f5045ffc6054935c18a7b Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 11 Oct 2023 16:10:16 +0800 Subject: [PATCH 033/195] refactor backend --- source/libs/stream/src/streamSnapshot.c | 58 ++++++++++++------------- 1 file changed, 28 insertions(+), 30 deletions(-) diff --git a/source/libs/stream/src/streamSnapshot.c b/source/libs/stream/src/streamSnapshot.c index a4363f46b1..e7bfca8252 100644 --- a/source/libs/stream/src/streamSnapshot.c +++ b/source/libs/stream/src/streamSnapshot.c @@ -140,19 +140,21 @@ TdFilePtr streamOpenFile(char* path, char* name, int32_t opt) { int32_t streamBackendGetSnapInfo(void* arg, char* path, int64_t chkpId) { return taskBackendBuildSnap(arg, chkpId); } void snapFileDebugInfo(SBackendSnapFile2* pSnapFile) { - char* buf = taosMemoryCalloc(1, 512); - sprintf(buf, "[current: %s,", pSnapFile->pCurrent); - sprintf(buf + strlen(buf), "MANIFEST: %s,", pSnapFile->pMainfest); - sprintf(buf + strlen(buf), "options: %s,", pSnapFile->pOptions); + if (qDebugFlag & DEBUG_DEBUG) { + char* buf = taosMemoryCalloc(1, 512); + sprintf(buf, "[current: %s,", pSnapFile->pCurrent); + sprintf(buf + strlen(buf), "MANIFEST: %s,", pSnapFile->pMainfest); + sprintf(buf + strlen(buf), "options: %s,", pSnapFile->pOptions); - for (int i = 0; i < taosArrayGetSize(pSnapFile->pSst); i++) { - char* name = taosArrayGetP(pSnapFile->pSst, i); - sprintf(buf + strlen(buf), "%s,", name); + for (int i = 0; i < taosArrayGetSize(pSnapFile->pSst); i++) { + char* name = taosArrayGetP(pSnapFile->pSst, i); + sprintf(buf + strlen(buf), "%s,", name); + } + sprintf(buf + strlen(buf) - 1, "]"); + + qInfo("%s get file list: %s", STREAM_STATE_TRANSFER, buf); + taosMemoryFree(buf); } - sprintf(buf + strlen(buf) - 1, "]"); - - qInfo("%s get file list: %s", STREAM_STATE_TRANSFER, buf); - taosMemoryFree(buf); } int32_t snapFileCvtMeta(SBackendSnapFile2* pSnapFile) { @@ -308,6 +310,7 @@ void streamSnapHandleDestroy(SStreamSnapHandle* handle) { if (handle->pBackendSnapSet) { for (int i = 0; i < taosArrayGetSize(handle->pBackendSnapSet); i++) { SBackendSnapFile2* pSnapFile = taosArrayGet(handle->pBackendSnapSet, i); + snapFileDebugInfo(pSnapFile); snapFileDestroy(pSnapFile); } taosArrayDestroy(handle->pBackendSnapSet); @@ -581,25 +584,20 @@ int32_t streamSnapWrite(SStreamSnapWriter* pWriter, uint8_t* pData, uint32_t nDa } int32_t streamSnapWriterClose(SStreamSnapWriter* pWriter, int8_t rollback) { SStreamSnapHandle* handle = &pWriter->handle; - if (qDebugFlag & DEBUG_DEBUG) { - char* buf = (char*)taosMemoryMalloc(1024); - int n = sprintf(buf, "["); - for (int i = 0; i < taosArrayGetSize(handle->pFileList); i++) { - SBackendFileItem* item = taosArrayGet(handle->pFileList, i); - if (i != taosArrayGetSize(handle->pFileList) - 1) { - n += sprintf(buf + n, "%s %" PRId64 ",", item->name, item->size); - } else { - n += sprintf(buf + n, "%s %" PRId64 "]", item->name, item->size); - } - } - qDebug("%s snap get file list, %s", STREAM_STATE_TRANSFER, buf); - taosMemoryFree(buf); - } - - for (int i = 0; i < taosArrayGetSize(handle->pFileList); i++) { - SBackendFileItem* item = taosArrayGet(handle->pFileList, i); - taosMemoryFree(item->name); - } + // if (qDebugFlag & DEBUG_DEBUG) { + // char* buf = (char*)taosMemoryMalloc(1024); + // int n = sprintf(buf, "["); + // for (int i = 0; i < taosArrayGetSize(handle->pFileList); i++) { + // SBackendFileItem* item = taosArrayGet(handle->pFileList, i); + // if (i != taosArrayGetSize(handle->pFileList) - 1) { + // n += sprintf(buf + n, "%s %" PRId64 ",", item->name, item->size); + // } else { + // n += sprintf(buf + n, "%s %" PRId64 "]", item->name, item->size); + // } + // } + // qDebug("%s snap get file list, %s", STREAM_STATE_TRANSFER, buf); + // taosMemoryFree(buf); + // } streamSnapHandleDestroy(handle); taosMemoryFree(pWriter); From 4f471625122ee1c2abd84e36a7491a2a434c5ef2 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 11 Oct 2023 16:36:43 +0800 Subject: [PATCH 034/195] refactor backend --- source/libs/stream/src/streamSnapshot.c | 27 +++++++------------------ 1 file changed, 7 insertions(+), 20 deletions(-) diff --git a/source/libs/stream/src/streamSnapshot.c b/source/libs/stream/src/streamSnapshot.c index e7bfca8252..25c228e97d 100644 --- a/source/libs/stream/src/streamSnapshot.c +++ b/source/libs/stream/src/streamSnapshot.c @@ -145,14 +145,16 @@ void snapFileDebugInfo(SBackendSnapFile2* pSnapFile) { sprintf(buf, "[current: %s,", pSnapFile->pCurrent); sprintf(buf + strlen(buf), "MANIFEST: %s,", pSnapFile->pMainfest); sprintf(buf + strlen(buf), "options: %s,", pSnapFile->pOptions); - - for (int i = 0; i < taosArrayGetSize(pSnapFile->pSst); i++) { - char* name = taosArrayGetP(pSnapFile->pSst, i); - sprintf(buf + strlen(buf), "%s,", name); + if (pSnapFile->pSst) { + for (int i = 0; i < taosArrayGetSize(pSnapFile->pSst); i++) { + char* name = taosArrayGetP(pSnapFile->pSst, i); + sprintf(buf + strlen(buf), "%s,", name); + } } sprintf(buf + strlen(buf) - 1, "]"); - qInfo("%s get file list: %s", STREAM_STATE_TRANSFER, buf); + qInfo("%s %" PRId64 "-%" PRId64 " get file list: %s", STREAM_STATE_TRANSFER, pSnapFile->snapInfo.streamId, + pSnapFile->snapInfo.taskId, buf); taosMemoryFree(buf); } } @@ -584,21 +586,6 @@ int32_t streamSnapWrite(SStreamSnapWriter* pWriter, uint8_t* pData, uint32_t nDa } int32_t streamSnapWriterClose(SStreamSnapWriter* pWriter, int8_t rollback) { SStreamSnapHandle* handle = &pWriter->handle; - // if (qDebugFlag & DEBUG_DEBUG) { - // char* buf = (char*)taosMemoryMalloc(1024); - // int n = sprintf(buf, "["); - // for (int i = 0; i < taosArrayGetSize(handle->pFileList); i++) { - // SBackendFileItem* item = taosArrayGet(handle->pFileList, i); - // if (i != taosArrayGetSize(handle->pFileList) - 1) { - // n += sprintf(buf + n, "%s %" PRId64 ",", item->name, item->size); - // } else { - // n += sprintf(buf + n, "%s %" PRId64 "]", item->name, item->size); - // } - // } - // qDebug("%s snap get file list, %s", STREAM_STATE_TRANSFER, buf); - // taosMemoryFree(buf); - // } - streamSnapHandleDestroy(handle); taosMemoryFree(pWriter); From 031506aef8b2088b5c03902ee7e40c718fd88a3b Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 11 Oct 2023 19:46:35 +0800 Subject: [PATCH 035/195] refactor backend --- include/libs/stream/tstream.h | 1 + source/dnode/vnode/src/tq/tqStreamStateSnap.c | 5 +++-- source/libs/stream/src/streamMeta.c | 16 ++++++++++++++++ 3 files changed, 20 insertions(+), 2 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index a9e593eaad..859d7cea54 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -725,6 +725,7 @@ void streamMetaReleaseTask(SStreamMeta* pMeta, SStreamTask* pTask); int32_t streamMetaReopen(SStreamMeta* pMeta, int64_t chkpId); int32_t streamMetaCommit(SStreamMeta* pMeta); int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta); +int32_t streamMetaReloadAllTasks(SStreamMeta* pMeta); void streamMetaNotifyClose(SStreamMeta* pMeta); void* streamMetaGetBackendByTaskKey(SStreamMeta* pMeta, char* key, int64_t* ref); diff --git a/source/dnode/vnode/src/tq/tqStreamStateSnap.c b/source/dnode/vnode/src/tq/tqStreamStateSnap.c index 38cd1e1b76..b2835f2198 100644 --- a/source/dnode/vnode/src/tq/tqStreamStateSnap.c +++ b/source/dnode/vnode/src/tq/tqStreamStateSnap.c @@ -169,14 +169,15 @@ int32_t streamStateSnapWriterClose(SStreamStateWriter* pWriter, int8_t rollback) } int32_t streamStateRebuildFromSnap(SStreamStateWriter* pWriter, int64_t chkpId) { tqDebug("vgId:%d, vnode %s start to rebuild stream-state", TD_VID(pWriter->pTq->pVnode), STREAM_STATE_TRANSFER); - // int32_t code = streamMetaReopen(pWriter->pTq->pStreamMeta, chkpId); int32_t code = streamStateLoadTasks(pWriter); tqDebug("vgId:%d, vnode %s succ to rebuild stream-state", TD_VID(pWriter->pTq->pVnode), STREAM_STATE_TRANSFER); taosMemoryFree(pWriter); return code; } -int32_t streamStateLoadTasks(SStreamStateWriter* pWriter) { return streamMetaLoadAllTasks(pWriter->pTq->pStreamMeta); } +int32_t streamStateLoadTasks(SStreamStateWriter* pWriter) { + return streamMetaReloadAllTasks(pWriter->pTq->pStreamMeta); +} int32_t streamStateSnapWrite(SStreamStateWriter* pWriter, uint8_t* pData, uint32_t nData) { tqDebug("vgId:%d, vnode %s snapshot write data", TD_VID(pWriter->pTq->pVnode), STREAM_STATE_TRANSFER); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 72399865e6..83fe0b03c0 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -756,6 +756,22 @@ static void doClear(void* pKey, void* pVal, TBC* pCur, SArray* pRecycleList) { taosArrayDestroy(pRecycleList); } +int32_t streamMetaReloadAllTasks(SStreamMeta* pMeta) { + if (pMeta == NULL) return 0; + + void* pIter = taosHashIterate(pMeta->pTaskBackendUnique, NULL); + while (pIter) { + STaskBackendWrapper* taskBackend = *(STaskBackendWrapper**)pIter; + if (taskBackend != NULL) { + taskBackendRemoveRef(taskBackend); + } + pIter = taosHashIterate(pMeta->pTaskBackendUnique, pIter); + } + taosHashClear(pMeta->pTaskBackendUnique); + + + return 0; +} int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { TBC* pCur = NULL; From 83b52a4c1d953e0873d5bb64b36cfd9e8141c9fd Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 11 Oct 2023 20:02:40 +0800 Subject: [PATCH 036/195] refactor backend --- source/libs/stream/src/streamMeta.c | 21 ++++++++++----------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 83fe0b03c0..c398254548 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -759,18 +759,17 @@ static void doClear(void* pKey, void* pVal, TBC* pCur, SArray* pRecycleList) { int32_t streamMetaReloadAllTasks(SStreamMeta* pMeta) { if (pMeta == NULL) return 0; - void* pIter = taosHashIterate(pMeta->pTaskBackendUnique, NULL); - while (pIter) { - STaskBackendWrapper* taskBackend = *(STaskBackendWrapper**)pIter; - if (taskBackend != NULL) { - taskBackendRemoveRef(taskBackend); - } - pIter = taosHashIterate(pMeta->pTaskBackendUnique, pIter); - } - taosHashClear(pMeta->pTaskBackendUnique); + // void* pIter = taosHashIterate(pMeta->pTaskBackendUnique, NULL); + // while (pIter) { + // STaskBackendWrapper* taskBackend = *(STaskBackendWrapper**)pIter; + // if (taskBackend != NULL) { + // taskBackendRemoveRef(taskBackend); + // } + // pIter = taosHashIterate(pMeta->pTaskBackendUnique, pIter); + // } + // taosHashClear(pMeta->pTaskBackendUnique); - - return 0; + return streamMetaLoadAllTasks(pMeta); } int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { TBC* pCur = NULL; From f264ffdcd645dac5f3de780d8501362af0f0cd1d Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 11 Oct 2023 20:12:07 +0800 Subject: [PATCH 037/195] refactor backend --- include/libs/stream/tstream.h | 2 +- source/libs/stream/inc/streamBackendRocksdb.h | 12 +- source/libs/stream/src/streamBackendRocksdb.c | 190 +++++++++--------- source/libs/stream/src/streamMeta.c | 10 +- source/libs/stream/src/streamTask.c | 4 +- 5 files changed, 109 insertions(+), 109 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 859d7cea54..260609e697 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -425,7 +425,7 @@ typedef struct SStreamMeta { SRWLatch chkpDirLock; int32_t pauseTaskNum; - // SHashObj* pTaskBackend; + // SHashObj* pTaskDb; } SStreamMeta; int32_t tEncodeStreamEpInfo(SEncoder* pEncoder, const SStreamChildEpInfo* pInfo); diff --git a/source/libs/stream/inc/streamBackendRocksdb.h b/source/libs/stream/inc/streamBackendRocksdb.h index 9aa616e190..cea948e2f1 100644 --- a/source/libs/stream/inc/streamBackendRocksdb.h +++ b/source/libs/stream/inc/streamBackendRocksdb.h @@ -74,7 +74,7 @@ typedef struct { int32_t chkpCap; TdThreadRwlock chkpDirLock; -} STaskBackendWrapper; +} STaskDbWrapper; void* streamBackendInit(const char* path, int64_t chkpId); void streamBackendCleanup(void* arg); @@ -85,12 +85,12 @@ SListNode* streamBackendAddCompare(void* backend, void* arg); void streamBackendDelCompare(void* backend, void* arg); int32_t streamStateConvertDataFormat(char* path, char* key, void* cfInst); -STaskBackendWrapper* taskBackendOpen(char* path, char* key); -void taskBackendDestroy(void* pBackend); -int32_t taskBackendDoCheckpoint(void* arg, int64_t chkpId); +STaskDbWrapper* taskBackendOpen(char* path, char* key); +void taskDbDestroy(void* pBackend); +int32_t taskBackendDoCheckpoint(void* arg, int64_t chkpId); -void* taskBackendAddRef(void* pTaskBackend); -void taskBackendRemoveRef(void* pTaskBackend); +void* taskDbAddRef(void* pTaskDb); +void taskDbRemoveRef(void* pTaskDb); int streamStateOpenBackend(void* backend, SStreamState* pState); void streamStateCloseBackend(SStreamState* pState, bool remove); diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 917bc0f796..d5450711a2 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -716,7 +716,7 @@ int32_t delObsoleteCheckpoint(void* arg, const char* path) { * replication is finished */ int32_t chkpMayDelObsolete(void* arg, int64_t chkpId, char* path) { - STaskBackendWrapper* pBackend = arg; + STaskDbWrapper* pBackend = arg; taosThreadRwlockWrlock(&pBackend->chkpDirLock); @@ -863,7 +863,7 @@ int32_t chkpGetAllDbCfHandle(SStreamMeta* pMeta, rocksdb_column_family_handle_t* // return nCf; } -int32_t chkpGetAllDbCfHandle2(STaskBackendWrapper* pBackend, rocksdb_column_family_handle_t*** ppHandle) { +int32_t chkpGetAllDbCfHandle2(STaskDbWrapper* pBackend, rocksdb_column_family_handle_t*** ppHandle) { SArray* pHandle = taosArrayInit(8, POINTER_BYTES); for (int i = 0; i < sizeof(ginitDict) / sizeof(ginitDict[0]); i++) { if (pBackend->pCf[i]) { @@ -956,12 +956,12 @@ int32_t taskBackendBuildSnap(void* arg, int64_t chkpId) { int32_t code = 0; while (pIter) { - STaskBackendWrapper* pBackend = *(STaskBackendWrapper**)pIter; - taskBackendAddRef(pBackend); + STaskDbWrapper* pBackend = *(STaskDbWrapper**)pIter; + taskDbAddRef(pBackend); - code = taskBackendDoCheckpoint((STaskBackendWrapper*)pBackend, chkpId); + code = taskBackendDoCheckpoint((STaskDbWrapper*)pBackend, chkpId); - taskBackendRemoveRef(pBackend); + taskDbRemoveRef(pBackend); pIter = taosHashIterate(pMeta->pTaskBackendUnique, pIter); } return 0; @@ -1036,10 +1036,10 @@ int32_t streamBackendDelInUseChkp(void* arg, int64_t chkpId) { 0 */ int32_t taskBackendDoCheckpoint(void* arg, int64_t chkpId) { - STaskBackendWrapper* pBackend = arg; - int64_t st = taosGetTimestampMs(); - int32_t code = -1; - int64_t refId = pBackend->refId; + STaskDbWrapper* pBackend = arg; + int64_t st = taosGetTimestampMs(); + int32_t code = -1; + int64_t refId = pBackend->refId; if (taosAcquireRef(taskBackendWrapperId, refId) == NULL) { return -1; @@ -1549,7 +1549,7 @@ int32_t getCfIdx(const char* cfName) { } return idx; } -int32_t taskBackendOpenCfs(STaskBackendWrapper* pTask, char* path, char** pCfNames, int32_t nCf) { +int32_t taskDbOpenCfs(STaskDbWrapper* pTask, char* path, char** pCfNames, int32_t nCf) { int32_t code = -1; char* err = NULL; @@ -1583,17 +1583,17 @@ _EXIT: taosMemoryFree(cfHandle); return code; } -void* taskBackendAddRef(void* pTaskBackend) { - STaskBackendWrapper* pBackend = pTaskBackend; +void* taskDbAddRef(void* pTaskDb) { + STaskDbWrapper* pBackend = pTaskDb; return taosAcquireRef(taskBackendWrapperId, pBackend->refId); } -void taskBackendRemoveRef(void* pTaskBackend) { - STaskBackendWrapper* pBackend = pTaskBackend; +void taskDbRemoveRef(void* pTaskDb) { + STaskDbWrapper* pBackend = pTaskDb; taosReleaseRef(taskBackendWrapperId, pBackend->refId); } -// void taskBackendDestroy(STaskBackendWrapper* wrapper); +// void taskDbDestroy(STaskDbWrapper* wrapper); -void taskBackendInitDBOpt(STaskBackendWrapper* pTaskBackend) { +void taskDbInitDBOpt(STaskDbWrapper* pTaskDb) { rocksdb_env_t* env = rocksdb_create_default_env(); rocksdb_cache_t* cache = rocksdb_cache_create_lru(256); @@ -1609,25 +1609,25 @@ void taskBackendInitDBOpt(STaskBackendWrapper* pTaskBackend) { rocksdb_options_set_write_buffer_size(opts, 32 << 20); rocksdb_options_set_atomic_flush(opts, 1); - pTaskBackend->dbOpt = opts; - pTaskBackend->env = env; - pTaskBackend->cache = cache; - pTaskBackend->filterFactory = rocksdb_compactionfilterfactory_create( + pTaskDb->dbOpt = opts; + pTaskDb->env = env; + pTaskDb->cache = cache; + pTaskDb->filterFactory = rocksdb_compactionfilterfactory_create( NULL, destroyCompactFilteFactory, compactFilteFactoryCreateFilter, compactFilteFactoryName); - rocksdb_options_set_compaction_filter_factory(pTaskBackend->dbOpt, pTaskBackend->filterFactory); - pTaskBackend->readOpt = rocksdb_readoptions_create(); - pTaskBackend->writeOpt = rocksdb_writeoptions_create(); + rocksdb_options_set_compaction_filter_factory(pTaskDb->dbOpt, pTaskDb->filterFactory); + pTaskDb->readOpt = rocksdb_readoptions_create(); + pTaskDb->writeOpt = rocksdb_writeoptions_create(); size_t nCf = sizeof(ginitDict) / sizeof(ginitDict[0]); - pTaskBackend->pCf = taosMemoryCalloc(nCf, sizeof(rocksdb_column_family_handle_t*)); - pTaskBackend->pCfParams = taosMemoryCalloc(nCf, sizeof(RocksdbCfParam)); - pTaskBackend->pCfOpts = taosMemoryCalloc(nCf, sizeof(rocksdb_options_t*)); - pTaskBackend->pCompares = taosMemoryCalloc(nCf, sizeof(rocksdb_comparator_t*)); + pTaskDb->pCf = taosMemoryCalloc(nCf, sizeof(rocksdb_column_family_handle_t*)); + pTaskDb->pCfParams = taosMemoryCalloc(nCf, sizeof(RocksdbCfParam)); + pTaskDb->pCfOpts = taosMemoryCalloc(nCf, sizeof(rocksdb_options_t*)); + pTaskDb->pCompares = taosMemoryCalloc(nCf, sizeof(rocksdb_comparator_t*)); for (int i = 0; i < nCf; i++) { - rocksdb_options_t* opt = rocksdb_options_create_copy(pTaskBackend->dbOpt); + rocksdb_options_t* opt = rocksdb_options_create_copy(pTaskDb->dbOpt); rocksdb_block_based_table_options_t* tableOpt = rocksdb_block_based_options_create(); - rocksdb_block_based_options_set_block_cache(tableOpt, pTaskBackend->cache); + rocksdb_block_based_options_set_block_cache(tableOpt, pTaskDb->cache); rocksdb_block_based_options_set_partition_filters(tableOpt, 1); rocksdb_filterpolicy_t* filter = rocksdb_filterpolicy_create_bloom(15); @@ -1641,13 +1641,13 @@ void taskBackendInitDBOpt(STaskBackendWrapper* pTaskBackend) { rocksdb_comparator_create(NULL, cfPara->detroyFunc, cfPara->cmpFunc, cfPara->cmpName); rocksdb_options_set_comparator((rocksdb_options_t*)opt, compare); - pTaskBackend->pCompares[i] = compare; - pTaskBackend->pCfOpts[i] = opt; - pTaskBackend->pCfParams[i].tableOpt = tableOpt; + pTaskDb->pCompares[i] = compare; + pTaskDb->pCfOpts[i] = opt; + pTaskDb->pCfParams[i].tableOpt = tableOpt; } return; } -void taskBackendInitChkpOpt(STaskBackendWrapper* pBackend) { +void taskDbInitChkpOpt(STaskDbWrapper* pBackend) { pBackend->chkpId = -1; pBackend->chkpCap = 4; pBackend->chkpSaved = taosArrayInit(4, sizeof(int64_t)); @@ -1656,13 +1656,13 @@ void taskBackendInitChkpOpt(STaskBackendWrapper* pBackend) { taosThreadRwlockInit(&pBackend->chkpDirLock, NULL); } -void taskBackendDestroyChkpOpt(STaskBackendWrapper* pBackend) { +void taskDbDestroyChkpOpt(STaskDbWrapper* pBackend) { taosArrayDestroy(pBackend->chkpSaved); taosArrayDestroy(pBackend->chkpInUse); taosThreadRwlockDestroy(&pBackend->chkpDirLock); } -int32_t taskBackendBuildFullPath(char* path, char* key, char** dbFullPath, char** stateFullPath) { +int32_t taskDbBuildFullPath(char* path, char* key, char** dbFullPath, char** stateFullPath) { int32_t code = 0; char* statePath = taosMemoryCalloc(1, strlen(path) + 128); @@ -1692,41 +1692,41 @@ int32_t taskBackendBuildFullPath(char* path, char* key, char** dbFullPath, char* *stateFullPath = statePath; return 0; } -STaskBackendWrapper* taskBackendOpen(char* path, char* key) { +STaskDbWrapper* taskBackendOpen(char* path, char* key) { char* statePath = NULL; char* err = NULL; char* dbPath = NULL; char** cfNames = NULL; size_t nCf = 0; - if (taskBackendBuildFullPath(path, key, &dbPath, &statePath) != 0) { + if (taskDbBuildFullPath(path, key, &dbPath, &statePath) != 0) { return NULL; } - STaskBackendWrapper* pTaskBackend = taosMemoryCalloc(1, sizeof(STaskBackendWrapper)); - pTaskBackend->idstr = taosStrdup(key); - pTaskBackend->path = statePath; - taosThreadMutexInit(&pTaskBackend->mutex, NULL); - taskBackendInitChkpOpt(pTaskBackend); - taskBackendInitDBOpt(pTaskBackend); + STaskDbWrapper* pTaskDb = taosMemoryCalloc(1, sizeof(STaskDbWrapper)); + pTaskDb->idstr = taosStrdup(key); + pTaskDb->path = statePath; + taosThreadMutexInit(&pTaskDb->mutex, NULL); + taskDbInitChkpOpt(pTaskDb); + taskDbInitDBOpt(pTaskDb); statePath = NULL; - cfNames = rocksdb_list_column_families(pTaskBackend->dbOpt, dbPath, &nCf, &err); + cfNames = rocksdb_list_column_families(pTaskDb->dbOpt, dbPath, &nCf, &err); if (nCf == 0) { // pre create db - pTaskBackend->db = rocksdb_open(pTaskBackend->pCfOpts[0], dbPath, &err); - rocksdb_close(pTaskBackend->db); + pTaskDb->db = rocksdb_open(pTaskDb->pCfOpts[0], dbPath, &err); + rocksdb_close(pTaskDb->db); if (cfNames != NULL) { rocksdb_list_column_families_destroy(cfNames, nCf); } taosMemoryFree(err); - cfNames = rocksdb_list_column_families(pTaskBackend->dbOpt, dbPath, &nCf, &err); + cfNames = rocksdb_list_column_families(pTaskDb->dbOpt, dbPath, &nCf, &err); ASSERT(err != NULL); } - if (taskBackendOpenCfs(pTaskBackend, dbPath, cfNames, nCf) != 0) { + if (taskDbOpenCfs(pTaskDb, dbPath, cfNames, nCf) != 0) { goto _EXIT; } @@ -1734,13 +1734,13 @@ STaskBackendWrapper* taskBackendOpen(char* path, char* key) { rocksdb_list_column_families_destroy(cfNames, nCf); } - qDebug("succ to init stream backend at %s, backend:%p", dbPath, pTaskBackend); + qDebug("succ to init stream backend at %s, backend:%p", dbPath, pTaskDb); taosMemoryFree(dbPath); - return pTaskBackend; + return pTaskDb; _EXIT: - taskBackendDestroy(pTaskBackend); + taskDbDestroy(pTaskDb); if (err) taosMemoryFree(err); if (cfNames) rocksdb_list_column_families_destroy(cfNames, nCf); @@ -1749,8 +1749,8 @@ _EXIT: return NULL; } -void taskBackendDestroy(void* pBackend) { - STaskBackendWrapper* wrapper = pBackend; +void taskDbDestroy(void* pBackend) { + STaskDbWrapper* wrapper = pBackend; if (wrapper == NULL) return; @@ -1790,7 +1790,7 @@ void taskBackendDestroy(void* pBackend) { if (wrapper->db) rocksdb_close(wrapper->db); - taskBackendDestroyChkpOpt(pBackend); + taskDbDestroyChkpOpt(pBackend); taosMemoryFree(wrapper->idstr); taosMemoryFree(wrapper->path); @@ -1799,7 +1799,7 @@ void taskBackendDestroy(void* pBackend) { return; } -int32_t taskBackendOpenCfByKey(STaskBackendWrapper* pBackend, const char* key) { +int32_t taskDbOpenCfByKey(STaskDbWrapper* pBackend, const char* key) { int32_t code = 0; char* err = NULL; int8_t idx = getCfIdx(key); @@ -1817,7 +1817,7 @@ int32_t taskBackendOpenCfByKey(STaskBackendWrapper* pBackend, const char* key) { pBackend->pCf[idx] = cf; return code; } -int32_t copyDataAt(RocksdbCfInst* pSrc, STaskBackendWrapper* pDst, int8_t i) { +int32_t copyDataAt(RocksdbCfInst* pSrc, STaskDbWrapper* pDst, int8_t i) { int32_t WRITE_BATCH = 1024; char* err = NULL; int code = 0; @@ -1866,22 +1866,22 @@ int32_t streamStateConvertDataFormat(char* path, char* key, void* pCfInst) { int32_t code = 0; - STaskBackendWrapper* pTaskBackend = taskBackendOpen(path, key); - RocksdbCfInst* pSrcBackend = pCfInst; + STaskDbWrapper* pTaskDb = taskBackendOpen(path, key); + RocksdbCfInst* pSrcBackend = pCfInst; for (int i = 0; i < nCf; i++) { rocksdb_column_family_handle_t* pSrcCf = pSrcBackend->pHandle[i]; if (pSrcCf == NULL) continue; - code = taskBackendOpenCfByKey(pTaskBackend, ginitDict[i].key); + code = taskDbOpenCfByKey(pTaskDb, ginitDict[i].key); if (code != 0) goto _EXIT; - code = copyDataAt(pSrcBackend, pTaskBackend, i); + code = copyDataAt(pSrcBackend, pTaskDb, i); if (code != 0) goto _EXIT; } _EXIT: - taskBackendDestroy(pTaskBackend); + taskDbDestroy(pTaskDb); return code; } @@ -2140,7 +2140,7 @@ int streamStateGetCfIdx(SStreamState* pState, const char* funcName) { } } if (pState != NULL && idx != -1) { - STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; rocksdb_column_family_handle_t* cf = NULL; taosThreadMutexLock(&wrapper->mutex); @@ -2179,7 +2179,7 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfKe *readOpt = rocksdb_readoptions_create(); - STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; if (snapshot != NULL) { *snapshot = (rocksdb_snapshot_t*)rocksdb_create_snapshot(wrapper->db); rocksdb_readoptions_set_snapshot(*readOpt, *snapshot); @@ -2200,8 +2200,8 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfKe code = -1; \ break; \ } \ - STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; \ - char toString[128] = {0}; \ + STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; \ + char toString[128] = {0}; \ if (qDebugFlag & DEBUG_TRACE) ginitDict[i].toStrFunc((void*)key, toString); \ int32_t klen = ginitDict[i].enFunc((void*)key, buf); \ rocksdb_column_family_handle_t* pHandle = ((rocksdb_column_family_handle_t**)wrapper->pCf)[ginitDict[i].idx]; \ @@ -2231,8 +2231,8 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfKe code = -1; \ break; \ } \ - STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; \ - char toString[128] = {0}; \ + STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; \ + char toString[128] = {0}; \ if (qDebugFlag & DEBUG_TRACE) ginitDict[i].toStrFunc((void*)key, toString); \ int32_t klen = ginitDict[i].enFunc((void*)key, buf); \ rocksdb_column_family_handle_t* pHandle = ((rocksdb_column_family_handle_t**)wrapper->pCf)[ginitDict[i].idx]; \ @@ -2274,8 +2274,8 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfKe code = -1; \ break; \ } \ - STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; \ - char toString[128] = {0}; \ + STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; \ + char toString[128] = {0}; \ if (qDebugFlag & DEBUG_TRACE) ginitDict[i].toStrFunc((void*)key, toString); \ int32_t klen = ginitDict[i].enFunc((void*)key, buf); \ rocksdb_column_family_handle_t* pHandle = ((rocksdb_column_family_handle_t**)wrapper->pCf)[ginitDict[i].idx]; \ @@ -2314,7 +2314,7 @@ int32_t streamStateDel_rocksdb(SStreamState* pState, const SWinKey* key) { int32_t streamStateClear_rocksdb(SStreamState* pState) { qDebug("streamStateClear_rocksdb"); - STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; char sKeyStr[128] = {0}; char eKeyStr[128] = {0}; @@ -2437,7 +2437,7 @@ SStreamStateCur* streamStateSeekKeyNext_rocksdb(SStreamState* pState, const SWin if (pCur == NULL) { return NULL; } - STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; pCur->number = pState->number; pCur->db = wrapper->db; pCur->iter = streamStateIterCreate(pState, "state", (rocksdb_snapshot_t**)&pCur->snapshot, @@ -2493,7 +2493,7 @@ SStreamStateCur* streamStateSeekToLast_rocksdb(SStreamState* pState, const SWinK if (pCur == NULL) return NULL; pCur->number = pState->number; - pCur->db = ((STaskBackendWrapper*)pState->pTdbState->pOwner->pBackend)->db; + pCur->db = ((STaskDbWrapper*)pState->pTdbState->pOwner->pBackend)->db; pCur->iter = streamStateIterCreate(pState, "state", (rocksdb_snapshot_t**)&pCur->snapshot, (rocksdb_readoptions_t**)&pCur->readOpt); @@ -2514,7 +2514,7 @@ SStreamStateCur* streamStateSeekToLast_rocksdb(SStreamState* pState, const SWinK SStreamStateCur* streamStateGetCur_rocksdb(SStreamState* pState, const SWinKey* key) { qDebug("streamStateGetCur_rocksdb"); - STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) return NULL; @@ -2604,8 +2604,8 @@ int32_t streamStateSessionDel_rocksdb(SStreamState* pState, const SSessionKey* k SStreamStateCur* streamStateSessionSeekKeyCurrentPrev_rocksdb(SStreamState* pState, const SSessionKey* key) { qDebug("streamStateSessionSeekKeyCurrentPrev_rocksdb"); - STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) { return NULL; } @@ -2645,8 +2645,8 @@ SStreamStateCur* streamStateSessionSeekKeyCurrentPrev_rocksdb(SStreamState* pSta } SStreamStateCur* streamStateSessionSeekKeyCurrentNext_rocksdb(SStreamState* pState, SSessionKey* key) { qDebug("streamStateSessionSeekKeyCurrentNext_rocksdb"); - STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) { return NULL; } @@ -2683,8 +2683,8 @@ SStreamStateCur* streamStateSessionSeekKeyCurrentNext_rocksdb(SStreamState* pSta SStreamStateCur* streamStateSessionSeekKeyNext_rocksdb(SStreamState* pState, const SSessionKey* key) { qDebug("streamStateSessionSeekKeyNext_rocksdb"); - STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) { return NULL; } @@ -2785,8 +2785,8 @@ int32_t streamStateFillDel_rocksdb(SStreamState* pState, const SWinKey* key) { SStreamStateCur* streamStateFillGetCur_rocksdb(SStreamState* pState, const SWinKey* key) { qDebug("streamStateFillGetCur_rocksdb"); - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); - STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; if (pCur == NULL) return NULL; @@ -2845,8 +2845,8 @@ int32_t streamStateFillGetKVByCur_rocksdb(SStreamStateCur* pCur, SWinKey* pKey, SStreamStateCur* streamStateFillSeekKeyNext_rocksdb(SStreamState* pState, const SWinKey* key) { qDebug("streamStateFillSeekKeyNext_rocksdb"); - STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (!pCur) { return NULL; } @@ -2883,8 +2883,8 @@ SStreamStateCur* streamStateFillSeekKeyNext_rocksdb(SStreamState* pState, const } SStreamStateCur* streamStateFillSeekKeyPrev_rocksdb(SStreamState* pState, const SWinKey* key) { qDebug("streamStateFillSeekKeyPrev_rocksdb"); - STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) { return NULL; } @@ -2921,8 +2921,8 @@ SStreamStateCur* streamStateFillSeekKeyPrev_rocksdb(SStreamState* pState, const } int32_t streamStateSessionGetKeyByRange_rocksdb(SStreamState* pState, const SSessionKey* key, SSessionKey* curKey) { qDebug("streamStateSessionGetKeyByRange_rocksdb"); - STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) { return -1; } @@ -3153,7 +3153,7 @@ int32_t streamDefaultIterGet_rocksdb(SStreamState* pState, const void* start, co int code = 0; char* err = NULL; - STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; rocksdb_snapshot_t* snapshot = NULL; rocksdb_readoptions_t* readopts = NULL; rocksdb_iterator_t* pIter = streamStateIterCreate(pState, "default", &snapshot, &readopts); @@ -3192,8 +3192,8 @@ int32_t streamDefaultIterGet_rocksdb(SStreamState* pState, const void* start, co return code; } void* streamDefaultIterCreate_rocksdb(SStreamState* pState) { - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); - STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; pCur->db = wrapper->db; pCur->iter = streamStateIterCreate(pState, "default", (rocksdb_snapshot_t**)&pCur->snapshot, @@ -3244,7 +3244,7 @@ void streamStateClearBatch(void* pBatch) { rocksdb_writebatch_clear((rocksdb_ void streamStateDestroyBatch(void* pBatch) { rocksdb_writebatch_destroy((rocksdb_writebatch_t*)pBatch); } int32_t streamStatePutBatch(SStreamState* pState, const char* cfKeyName, rocksdb_writebatch_t* pBatch, void* key, void* val, int32_t vlen, int64_t ttl) { - STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; int i = streamStateGetCfIdx(pState, cfKeyName); if (i < 0) { @@ -3277,7 +3277,7 @@ int32_t streamStatePutBatchOptimize(SStreamState* pState, int32_t cfIdx, rocksdb char* ttlV = tmpBuf; int32_t ttlVLen = ginitDict[cfIdx].enValueFunc(val, vlen, ttl, &ttlV); - STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; rocksdb_column_family_handle_t* pCf = wrapper->pCf[ginitDict[cfIdx].idx]; rocksdb_writebatch_put_cf((rocksdb_writebatch_t*)pBatch, pCf, buf, (size_t)klen, ttlV, (size_t)ttlVLen); @@ -3293,8 +3293,8 @@ int32_t streamStatePutBatchOptimize(SStreamState* pState, int32_t cfIdx, rocksdb return 0; } int32_t streamStatePutBatch_rocksdb(SStreamState* pState, void* pBatch) { - char* err = NULL; - STaskBackendWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + char* err = NULL; + STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; rocksdb_write(wrapper->db, wrapper->writeOpt, (rocksdb_writebatch_t*)pBatch, &err); if (err != NULL) { qError("streamState failed to write batch, err:%s", err); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index c398254548..8d855d3513 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -53,7 +53,7 @@ int32_t metaRefMgtAdd(int64_t vgId, int64_t* rid); static void streamMetaEnvInit() { streamBackendId = taosOpenRef(64, streamBackendCleanup); streamBackendCfWrapperId = taosOpenRef(64, streamBackendHandleCleanup); - taskBackendWrapperId = taosOpenRef(64, taskBackendDestroy); + taskBackendWrapperId = taosOpenRef(64, taskDbDestroy); streamMetaId = taosOpenRef(64, streamMetaCloseImpl); @@ -226,8 +226,8 @@ void* streamMetaGetBackendByTaskKey(SStreamMeta* pMeta, char* key, int64_t* ref) taosThreadMutexLock(&pMeta->backendMutex); void** ppBackend = taosHashGet(pMeta->pTaskBackendUnique, key, strlen(key)); if (ppBackend != NULL && *ppBackend != NULL) { - taskBackendAddRef(*ppBackend); - *ref = ((STaskBackendWrapper*)*ppBackend)->refId; + taskDbAddRef(*ppBackend); + *ref = ((STaskDbWrapper*)*ppBackend)->refId; taosThreadMutexUnlock(&pMeta->backendMutex); return *ppBackend; } @@ -761,9 +761,9 @@ int32_t streamMetaReloadAllTasks(SStreamMeta* pMeta) { // void* pIter = taosHashIterate(pMeta->pTaskBackendUnique, NULL); // while (pIter) { - // STaskBackendWrapper* taskBackend = *(STaskBackendWrapper**)pIter; + // STaskDbWrapper* taskBackend = *(STaskDbWrapper**)pIter; // if (taskBackend != NULL) { - // taskBackendRemoveRef(taskBackend); + // taskDbRemoveRef(taskBackend); // } // pIter = taosHashIterate(pMeta->pTaskBackendUnique, pIter); // } diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 152b426628..c0f9bc65ff 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -355,8 +355,8 @@ void tFreeStreamTask(SStreamTask* pTask) { pTask->pUpstreamInfoList = NULL; } if (pTask->pBackend) { - taskBackendRemoveRef(pTask->pBackend); - + taskDbRemoveRef(pTask->pBackend); + pTask->pBackend = NULL; } taosThreadMutexDestroy(&pTask->lock); From 4c94fbb44f223eba2c5781b703fb9b4a56923475 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 11 Oct 2023 20:26:19 +0800 Subject: [PATCH 038/195] refactor backend --- source/libs/stream/inc/streamBackendRocksdb.h | 6 +- source/libs/stream/inc/streamInt.h | 2 +- source/libs/stream/src/streamBackendRocksdb.c | 68 +++++++++---------- source/libs/stream/src/streamMeta.c | 8 +-- source/libs/stream/src/streamSnapshot.c | 2 +- source/libs/stream/src/streamTask.c | 2 +- 6 files changed, 44 insertions(+), 44 deletions(-) diff --git a/source/libs/stream/inc/streamBackendRocksdb.h b/source/libs/stream/inc/streamBackendRocksdb.h index cea948e2f1..0f97bda4c5 100644 --- a/source/libs/stream/inc/streamBackendRocksdb.h +++ b/source/libs/stream/inc/streamBackendRocksdb.h @@ -85,9 +85,9 @@ SListNode* streamBackendAddCompare(void* backend, void* arg); void streamBackendDelCompare(void* backend, void* arg); int32_t streamStateConvertDataFormat(char* path, char* key, void* cfInst); -STaskDbWrapper* taskBackendOpen(char* path, char* key); +STaskDbWrapper* taskDbOpen(char* path, char* key); void taskDbDestroy(void* pBackend); -int32_t taskBackendDoCheckpoint(void* arg, int64_t chkpId); +int32_t taskDbDoCheckpoint(void* arg, int64_t chkpId); void* taskDbAddRef(void* pTaskDb); void taskDbRemoveRef(void* pTaskDb); @@ -182,7 +182,7 @@ int32_t streamBackendTriggerChkp(void* pMeta, char* dst); int32_t streamBackendAddInUseChkp(void* arg, int64_t chkpId); int32_t streamBackendDelInUseChkp(void* arg, int64_t chkpId); -int32_t taskBackendBuildSnap(void* arg, int64_t chkpId); +int32_t taskDbBuildSnap(void* arg, int64_t chkpId); // int32_t streamDefaultIter_rocksdb(SStreamState* pState, const void* start, const void* end, SArray* result); #endif \ No newline at end of file diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index f200c714ab..a5e7531f7e 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -48,7 +48,7 @@ typedef struct { extern SStreamGlobalEnv streamEnv; extern int32_t streamBackendId; extern int32_t streamBackendCfWrapperId; -extern int32_t taskBackendWrapperId; +extern int32_t taskDbWrapperId; const char* streamGetBlockTypeStr(int32_t type); void streamRetryDispatchStreamBlock(SStreamTask* pTask, int64_t waitDuration); diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index d5450711a2..78f67d06d5 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -950,18 +950,18 @@ int32_t chkpPreBuildDir(char* path, int64_t chkpId, char** chkpDir, char** chkpI return 0; } -int32_t taskBackendBuildSnap(void* arg, int64_t chkpId) { +int32_t taskDbBuildSnap(void* arg, int64_t chkpId) { SStreamMeta* pMeta = arg; void* pIter = taosHashIterate(pMeta->pTaskBackendUnique, NULL); int32_t code = 0; while (pIter) { - STaskDbWrapper* pBackend = *(STaskDbWrapper**)pIter; - taskDbAddRef(pBackend); + STaskDbWrapper* pTaskDb = *(STaskDbWrapper**)pIter; + taskDbAddRef(pTaskDb); - code = taskBackendDoCheckpoint((STaskDbWrapper*)pBackend, chkpId); + code = taskDbDoCheckpoint((STaskDbWrapper*)pTaskDb, chkpId); - taskDbRemoveRef(pBackend); + taskDbRemoveRef(pTaskDb); pIter = taosHashIterate(pMeta->pTaskBackendUnique, pIter); } return 0; @@ -1035,48 +1035,48 @@ int32_t streamBackendDelInUseChkp(void* arg, int64_t chkpId) { /* 0 */ -int32_t taskBackendDoCheckpoint(void* arg, int64_t chkpId) { - STaskDbWrapper* pBackend = arg; +int32_t taskDbDoCheckpoint(void* arg, int64_t chkpId) { + STaskDbWrapper* pTaskDb = arg; int64_t st = taosGetTimestampMs(); int32_t code = -1; - int64_t refId = pBackend->refId; + int64_t refId = pTaskDb->refId; - if (taosAcquireRef(taskBackendWrapperId, refId) == NULL) { + if (taosAcquireRef(taskDbWrapperId, refId) == NULL) { return -1; } char* pChkpDir = NULL; char* pChkpIdDir = NULL; - if (chkpPreBuildDir(pBackend->path, chkpId, &pChkpDir, &pChkpIdDir) != 0) { + if (chkpPreBuildDir(pTaskDb->path, chkpId, &pChkpDir, &pChkpIdDir) != 0) { code = -1; goto _EXIT; } // Get all cf and acquire cfWrappter rocksdb_column_family_handle_t** ppCf = NULL; - int32_t nCf = chkpGetAllDbCfHandle2(pBackend, &ppCf); - qDebug("stream backend:%p start to do checkpoint at:%s, cf num: %d ", pBackend, pChkpIdDir, nCf); + int32_t nCf = chkpGetAllDbCfHandle2(pTaskDb, &ppCf); + qDebug("stream backend:%p start to do checkpoint at:%s, cf num: %d ", pTaskDb, pChkpIdDir, nCf); - if ((code = chkpPreFlushDb(pBackend->db, ppCf, nCf)) == 0) { - if ((code = chkpDoDbCheckpoint(pBackend->db, pChkpIdDir)) != 0) { - qError("stream backend:%p failed to do checkpoint at:%s", pBackend, pChkpIdDir); + if ((code = chkpPreFlushDb(pTaskDb->db, ppCf, nCf)) == 0) { + if ((code = chkpDoDbCheckpoint(pTaskDb->db, pChkpIdDir)) != 0) { + qError("stream backend:%p failed to do checkpoint at:%s", pTaskDb, pChkpIdDir); } else { - qDebug("stream backend:%p end to do checkpoint at:%s, time cost:%" PRId64 "ms", pBackend, pChkpIdDir, + qDebug("stream backend:%p end to do checkpoint at:%s, time cost:%" PRId64 "ms", pTaskDb, pChkpIdDir, taosGetTimestampMs() - st); } } else { - qError("stream backend:%p failed to flush db at:%s", pBackend, pChkpIdDir); + qError("stream backend:%p failed to flush db at:%s", pTaskDb, pChkpIdDir); } - code = chkpMayDelObsolete(pBackend, chkpId, pChkpDir); + code = chkpMayDelObsolete(pTaskDb, chkpId, pChkpDir); _EXIT: taosMemoryFree(pChkpDir); taosMemoryFree(pChkpIdDir); - taosReleaseRef(taskBackendWrapperId, refId); + taosReleaseRef(taskDbWrapperId, refId); return code; } -int32_t streamBackendDoCheckpoint(void* arg, int64_t chkpId) { return taskBackendDoCheckpoint(arg, chkpId); } +int32_t streamBackendDoCheckpoint(void* arg, int64_t chkpId) { return taskDbDoCheckpoint(arg, chkpId); } SListNode* streamBackendAddCompare(void* backend, void* arg) { SBackendWrapper* pHandle = (SBackendWrapper*)backend; @@ -1585,11 +1585,11 @@ _EXIT: } void* taskDbAddRef(void* pTaskDb) { STaskDbWrapper* pBackend = pTaskDb; - return taosAcquireRef(taskBackendWrapperId, pBackend->refId); + return taosAcquireRef(taskDbWrapperId, pBackend->refId); } void taskDbRemoveRef(void* pTaskDb) { STaskDbWrapper* pBackend = pTaskDb; - taosReleaseRef(taskBackendWrapperId, pBackend->refId); + taosReleaseRef(taskDbWrapperId, pBackend->refId); } // void taskDbDestroy(STaskDbWrapper* wrapper); @@ -1647,19 +1647,19 @@ void taskDbInitDBOpt(STaskDbWrapper* pTaskDb) { } return; } -void taskDbInitChkpOpt(STaskDbWrapper* pBackend) { - pBackend->chkpId = -1; - pBackend->chkpCap = 4; - pBackend->chkpSaved = taosArrayInit(4, sizeof(int64_t)); - pBackend->chkpInUse = taosArrayInit(4, sizeof(int64_t)); +void taskDbInitChkpOpt(STaskDbWrapper* pTaskDb) { + pTaskDb->chkpId = -1; + pTaskDb->chkpCap = 4; + pTaskDb->chkpSaved = taosArrayInit(4, sizeof(int64_t)); + pTaskDb->chkpInUse = taosArrayInit(4, sizeof(int64_t)); - taosThreadRwlockInit(&pBackend->chkpDirLock, NULL); + taosThreadRwlockInit(&pTaskDb->chkpDirLock, NULL); } -void taskDbDestroyChkpOpt(STaskDbWrapper* pBackend) { - taosArrayDestroy(pBackend->chkpSaved); - taosArrayDestroy(pBackend->chkpInUse); - taosThreadRwlockDestroy(&pBackend->chkpDirLock); +void taskDbDestroyChkpOpt(STaskDbWrapper* pTaskDb) { + taosArrayDestroy(pTaskDb->chkpSaved); + taosArrayDestroy(pTaskDb->chkpInUse); + taosThreadRwlockDestroy(&pTaskDb->chkpDirLock); } int32_t taskDbBuildFullPath(char* path, char* key, char** dbFullPath, char** stateFullPath) { @@ -1692,7 +1692,7 @@ int32_t taskDbBuildFullPath(char* path, char* key, char** dbFullPath, char** sta *stateFullPath = statePath; return 0; } -STaskDbWrapper* taskBackendOpen(char* path, char* key) { +STaskDbWrapper* taskDbOpen(char* path, char* key) { char* statePath = NULL; char* err = NULL; char* dbPath = NULL; @@ -1866,7 +1866,7 @@ int32_t streamStateConvertDataFormat(char* path, char* key, void* pCfInst) { int32_t code = 0; - STaskDbWrapper* pTaskDb = taskBackendOpen(path, key); + STaskDbWrapper* pTaskDb = taskDbOpen(path, key); RocksdbCfInst* pSrcBackend = pCfInst; for (int i = 0; i < nCf; i++) { diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 8d855d3513..f4c8021ac6 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -30,7 +30,7 @@ static TdThreadOnce streamMetaModuleInit = PTHREAD_ONCE_INIT; int32_t streamBackendId = 0; int32_t streamBackendCfWrapperId = 0; int32_t streamMetaId = 0; -int32_t taskBackendWrapperId = 0; +int32_t taskDbWrapperId = 0; static int64_t streamGetLatestCheckpointId(SStreamMeta* pMeta); static void metaHbToMnode(void* param, void* tmrId); @@ -53,7 +53,7 @@ int32_t metaRefMgtAdd(int64_t vgId, int64_t* rid); static void streamMetaEnvInit() { streamBackendId = taosOpenRef(64, streamBackendCleanup); streamBackendCfWrapperId = taosOpenRef(64, streamBackendHandleCleanup); - taskBackendWrapperId = taosOpenRef(64, taskDbDestroy); + taskDbWrapperId = taosOpenRef(64, taskDbDestroy); streamMetaId = taosOpenRef(64, streamMetaCloseImpl); @@ -232,13 +232,13 @@ void* streamMetaGetBackendByTaskKey(SStreamMeta* pMeta, char* key, int64_t* ref) return *ppBackend; } - void* pBackend = taskBackendOpen(pMeta->path, key); + void* pBackend = taskDbOpen(pMeta->path, key); if (pBackend == NULL) { taosThreadMutexUnlock(&pMeta->backendMutex); return NULL; } - *ref = taosAddRef(taskBackendWrapperId, pBackend); + *ref = taosAddRef(taskDbWrapperId, pBackend); taosHashPut(pMeta->pTaskBackendUnique, key, strlen(key), &pBackend, sizeof(void*)); taosThreadMutexUnlock(&pMeta->backendMutex); diff --git a/source/libs/stream/src/streamSnapshot.c b/source/libs/stream/src/streamSnapshot.c index 25c228e97d..d33f018a2c 100644 --- a/source/libs/stream/src/streamSnapshot.c +++ b/source/libs/stream/src/streamSnapshot.c @@ -137,7 +137,7 @@ TdFilePtr streamOpenFile(char* path, char* name, int32_t opt) { return taosOpenFile(fullname, opt); } -int32_t streamBackendGetSnapInfo(void* arg, char* path, int64_t chkpId) { return taskBackendBuildSnap(arg, chkpId); } +int32_t streamBackendGetSnapInfo(void* arg, char* path, int64_t chkpId) { return taskDbBuildSnap(arg, chkpId); } void snapFileDebugInfo(SBackendSnapFile2* pSnapFile) { if (qDebugFlag & DEBUG_DEBUG) { diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index c0f9bc65ff..5b4b0c309d 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -392,7 +392,7 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i taosThreadMutexInit(&pTask->lock, NULL); streamTaskOpenAllUpstreamInput(pTask); - // pTask->pBackend = taskBackendOpen(pMeta->path, (char*)pTask->id.idStr); + // pTask->pBackend = taskDbOpen(pMeta->path, (char*)pTask->id.idStr); return TSDB_CODE_SUCCESS; } From bfd205b3f7f7980bed63297449647f1fba7198e0 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 12 Oct 2023 11:31:39 +0800 Subject: [PATCH 039/195] refactor backend --- source/libs/stream/src/streamSnapshot.c | 118 +++++++++--------------- 1 file changed, 45 insertions(+), 73 deletions(-) diff --git a/source/libs/stream/src/streamSnapshot.c b/source/libs/stream/src/streamSnapshot.c index d33f018a2c..16b0ae0406 100644 --- a/source/libs/stream/src/streamSnapshot.c +++ b/source/libs/stream/src/streamSnapshot.c @@ -32,6 +32,7 @@ typedef struct SBackendFileItem { char* name; int8_t type; int64_t size; + int8_t ref; } SBackendFileItem; typedef struct SBackendFile { char* pCurrent; @@ -74,7 +75,7 @@ struct SStreamSnapHandle { int32_t currFileIdx; char* metaPath; - SArray* pBackendSnapSet; + SArray* pDbSnapSet; int32_t currIdx; }; struct SStreamSnapBlockHdr { @@ -142,9 +143,11 @@ int32_t streamBackendGetSnapInfo(void* arg, char* path, int64_t chkpId) { return void snapFileDebugInfo(SBackendSnapFile2* pSnapFile) { if (qDebugFlag & DEBUG_DEBUG) { char* buf = taosMemoryCalloc(1, 512); - sprintf(buf, "[current: %s,", pSnapFile->pCurrent); - sprintf(buf + strlen(buf), "MANIFEST: %s,", pSnapFile->pMainfest); - sprintf(buf + strlen(buf), "options: %s,", pSnapFile->pOptions); + sprintf(buf + strlen(buf), "["); + + if (pSnapFile->pCurrent) sprintf(buf, "current: %s,", pSnapFile->pCurrent); + if (pSnapFile->pMainfest) sprintf(buf + strlen(buf), "MANIFEST: %s,", pSnapFile->pMainfest); + if (pSnapFile->pOptions) sprintf(buf + strlen(buf), "options: %s,", pSnapFile->pOptions); if (pSnapFile->pSst) { for (int i = 0; i < taosArrayGetSize(pSnapFile->pSst); i++) { char* name = taosArrayGetP(pSnapFile->pSst, i); @@ -160,7 +163,8 @@ void snapFileDebugInfo(SBackendSnapFile2* pSnapFile) { } int32_t snapFileCvtMeta(SBackendSnapFile2* pSnapFile) { - SBackendFileItem item; + SBackendFileItem item = {0}; + item.ref = 1; // current item.name = pSnapFile->pCurrent; item.type = ROCKSDB_CURRENT_TYPE; @@ -270,6 +274,13 @@ void snapFileDestroy(SBackendSnapFile2* pSnap) { char* sst = taosArrayGetP(pSnap->pSst, i); taosMemoryFree(sst); } + // unite read/write snap file + for (int i = 0; i < taosArrayGetSize(pSnap->pFileList); i++) { + SBackendFileItem* pItem = taosArrayGet(pSnap->pFileList, i); + if (pItem->ref == 0) { + taosMemoryFree(pItem->name); + } + } taosArrayDestroy(pSnap->pFileList); taosArrayDestroy(pSnap->pSst); taosCloseFile(&pSnap->fd); @@ -285,7 +296,7 @@ int32_t streamSnapHandleInit(SStreamSnapHandle* pHandle, char* path, int64_t chk return -1; } - SArray* pBdSnapSet = taosArrayInit(8, sizeof(SBackendSnapFile2)); + SArray* pDbSnapSet = taosArrayInit(8, sizeof(SBackendSnapFile2)); for (int i = 0; i < taosArrayGetSize(pSnapSet); i++) { SStreamTaskSnap* pSnap = taosArrayGet(pSnapSet, i); @@ -293,10 +304,10 @@ int32_t streamSnapHandleInit(SStreamSnapHandle* pHandle, char* path, int64_t chk SBackendSnapFile2 snapFile = {0}; code = streamBackendSnapInitFile(path, pSnap, &snapFile); ASSERT(code == 0); - taosArrayPush(pBdSnapSet, &snapFile); + taosArrayPush(pDbSnapSet, &snapFile); } - pHandle->pBackendSnapSet = pBdSnapSet; + pHandle->pDbSnapSet = pDbSnapSet; pHandle->currIdx = 0; return 0; @@ -308,40 +319,15 @@ _err: } void streamSnapHandleDestroy(SStreamSnapHandle* handle) { - // SBanckendFile* pFile = handle->pBackendFile; - if (handle->pBackendSnapSet) { - for (int i = 0; i < taosArrayGetSize(handle->pBackendSnapSet); i++) { - SBackendSnapFile2* pSnapFile = taosArrayGet(handle->pBackendSnapSet, i); + if (handle->pDbSnapSet) { + for (int i = 0; i < taosArrayGetSize(handle->pDbSnapSet); i++) { + SBackendSnapFile2* pSnapFile = taosArrayGet(handle->pDbSnapSet, i); snapFileDebugInfo(pSnapFile); snapFileDestroy(pSnapFile); } - taosArrayDestroy(handle->pBackendSnapSet); + taosArrayDestroy(handle->pDbSnapSet); } taosMemoryFree(handle->metaPath); - - // if (handle->checkpointId == 0) { - // // del tmp dir - // if (pFile && taosIsDir(pFile->path)) { - // taosRemoveDir(pFile->path); - // } - // } else { - // streamBackendDelInUseChkp(handle->handle, handle->checkpointId); - // } - // if (pFile) { - // taosMemoryFree(pFile->pCheckpointMeta); - // taosMemoryFree(pFile->pCurrent); - // taosMemoryFree(pFile->pMainfest); - // taosMemoryFree(pFile->pOptions); - // taosMemoryFree(pFile->path); - // for (int i = 0; i < taosArrayGetSize(pFile->pSst); i++) { - // char* sst = taosArrayGetP(pFile->pSst, i); - // taosMemoryFree(sst); - // } - // taosArrayDestroy(pFile->pSst); - // taosMemoryFree(pFile); - // } - // taosArrayDestroy(handle->pFileList); - // taosCloseFile(&handle->fd); return; } @@ -374,17 +360,17 @@ int32_t streamSnapRead(SStreamSnapReader* pReader, uint8_t** ppData, int64_t* si int32_t code = 0; SStreamSnapHandle* pHandle = &pReader->handle; int32_t idx = pHandle->currIdx; - SBackendSnapFile2* pSnapFile = taosArrayGet(pHandle->pBackendSnapSet, idx); + SBackendSnapFile2* pSnapFile = taosArrayGet(pHandle->pDbSnapSet, idx); SBackendFileItem* item = NULL; _NEXT: if (pSnapFile->fd == NULL) { if (pSnapFile->currFileIdx >= taosArrayGetSize(pSnapFile->pFileList)) { - if (pHandle->currIdx + 1 < taosArrayGetSize(pHandle->pBackendSnapSet)) { + if (pHandle->currIdx + 1 < taosArrayGetSize(pHandle->pDbSnapSet)) { pHandle->currIdx += 1; - pSnapFile = taosArrayGet(pHandle->pBackendSnapSet, pHandle->currIdx); + pSnapFile = taosArrayGet(pHandle->pDbSnapSet, pHandle->currIdx); goto _NEXT; } else { *ppData = NULL; @@ -466,27 +452,12 @@ int32_t streamSnapWriterOpen(void* pMeta, int64_t sver, int64_t ever, char* path SBackendSnapFile2 snapFile = {0}; SStreamSnapHandle* pHandle = &pWriter->handle; - pHandle->pBackendSnapSet = taosArrayInit(8, sizeof(SBackendSnapFile2)); + pHandle->pDbSnapSet = taosArrayInit(8, sizeof(SBackendSnapFile2)); - taosArrayPush(pHandle->pBackendSnapSet, &snapFile); + taosArrayPush(pHandle->pDbSnapSet, &snapFile); pHandle->currIdx = 0; pHandle->metaPath = taosStrdup(path); - // SBanckendFile* pFile = taosMemoryCalloc(1, sizeof(SBanckendFile)); - // pFile->path = taosStrdup(path); - // SArray* list = taosArrayInit(64, sizeof(SBackendFileItem)); - - // SBackendFileItem item; - // item.name = taosStrdup((char*)ROCKSDB_CURRENT); - // item.type = ROCKSDB_CURRENT_TYPE; - // taosArrayPush(list, &item); - - // pHandle->pBackendFile = pFile; - - // pHandle->pFileList = list; - // pHandle->currFileIdx = 0; - // pHandle->offset = 0; - *ppWriter = pWriter; return 0; } @@ -530,9 +501,10 @@ int32_t streamSnapWriteImpl(SStreamSnapWriter* pWriter, uint8_t* pData, uint32_t pBackendFile->offset = 0; pBackendFile->currFileIdx += 1; - SBackendFileItem item; + SBackendFileItem item = {0}; item.name = taosStrdup(pHdr->name); item.type = pHdr->type; + taosArrayPush(pBackendFile->pFileList, &item); SBackendFileItem* pItem = taosArrayGet(pBackendFile->pFileList, pBackendFile->currFileIdx); @@ -557,26 +529,26 @@ int32_t streamSnapWrite(SStreamSnapWriter* pWriter, uint8_t* pData, uint32_t nDa SStreamSnapHandle* pHandle = &pWriter->handle; SStreamTaskSnap snapInfo = pHdr->snapInfo; - SBackendSnapFile2* pBackendFile = taosArrayGet(pHandle->pBackendSnapSet, pHandle->currIdx); - if (pBackendFile->inited == 0) { - pBackendFile->snapInfo = snapInfo; - pBackendFile->pFileList = taosArrayInit(64, sizeof(SBackendFileItem)); - pBackendFile->currFileIdx = 0; - pBackendFile->offset = 0; + SBackendSnapFile2* pDbSnapFile = taosArrayGet(pHandle->pDbSnapSet, pHandle->currIdx); + if (pDbSnapFile->inited == 0) { + pDbSnapFile->snapInfo = snapInfo; + pDbSnapFile->pFileList = taosArrayInit(64, sizeof(SBackendFileItem)); + pDbSnapFile->currFileIdx = 0; + pDbSnapFile->offset = 0; - SBackendFileItem item; + SBackendFileItem item = {0}; item.name = taosStrdup((char*)ROCKSDB_CURRENT); item.type = ROCKSDB_CURRENT_TYPE; - taosArrayPush(pBackendFile->pFileList, &item); + taosArrayPush(pDbSnapFile->pFileList, &item); - pBackendFile->inited = 1; - return streamSnapWriteImpl(pWriter, pData, nData, pBackendFile); + pDbSnapFile->inited = 1; + return streamSnapWriteImpl(pWriter, pData, nData, pDbSnapFile); } else { - if (snapInfoEqual(&snapInfo, &pBackendFile->snapInfo)) { - return streamSnapWriteImpl(pWriter, pData, nData, pBackendFile); + if (snapInfoEqual(&snapInfo, &pDbSnapFile->snapInfo)) { + return streamSnapWriteImpl(pWriter, pData, nData, pDbSnapFile); } else { SBackendSnapFile2 snapFile = {0}; - taosArrayPush(pHandle->pBackendSnapSet, &snapFile); + taosArrayPush(pHandle->pDbSnapSet, &snapFile); pHandle->currIdx += 1; return streamSnapWrite(pWriter, pData, nData); @@ -585,8 +557,8 @@ int32_t streamSnapWrite(SStreamSnapWriter* pWriter, uint8_t* pData, uint32_t nDa return code; } int32_t streamSnapWriterClose(SStreamSnapWriter* pWriter, int8_t rollback) { - SStreamSnapHandle* handle = &pWriter->handle; - streamSnapHandleDestroy(handle); + if (pWriter == NULL) return 0; + streamSnapHandleDestroy(&pWriter->handle); taosMemoryFree(pWriter); return 0; From f704d4b82eb5fed7e739642d31e1ab29a76574d3 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 12 Oct 2023 11:45:18 +0800 Subject: [PATCH 040/195] refactor backend --- source/libs/stream/src/streamSnapshot.c | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/source/libs/stream/src/streamSnapshot.c b/source/libs/stream/src/streamSnapshot.c index 16b0ae0406..1717a284bc 100644 --- a/source/libs/stream/src/streamSnapshot.c +++ b/source/libs/stream/src/streamSnapshot.c @@ -449,14 +449,14 @@ int32_t streamSnapWriterOpen(void* pMeta, int64_t sver, int64_t ever, char* path if (pWriter == NULL) { return TSDB_CODE_OUT_OF_MEMORY; } - SBackendSnapFile2 snapFile = {0}; SStreamSnapHandle* pHandle = &pWriter->handle; - pHandle->pDbSnapSet = taosArrayInit(8, sizeof(SBackendSnapFile2)); - - taosArrayPush(pHandle->pDbSnapSet, &snapFile); pHandle->currIdx = 0; pHandle->metaPath = taosStrdup(path); + pHandle->pDbSnapSet = taosArrayInit(8, sizeof(SBackendSnapFile2)); + + SBackendSnapFile2 snapFile = {0}; + taosArrayPush(pHandle->pDbSnapSet, &snapFile); *ppWriter = pWriter; return 0; From 861d26e356297994f01d5cc51b873d1d19fa8fad Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 12 Oct 2023 15:24:50 +0800 Subject: [PATCH 041/195] refactor backend --- source/libs/stream/src/streamMeta.c | 14 ++------ source/libs/stream/src/streamSnapshot.c | 44 ++++++++++++++----------- 2 files changed, 27 insertions(+), 31 deletions(-) diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index f4c8021ac6..9d76951adb 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -203,7 +203,7 @@ _EXIT: streamBackendCleanup((void*)pBackend); return code; } -int32_t streamMetaMayConvertBackendFormat(SStreamMeta* pMeta) { +int32_t streamMetaMayCvtDbFormat(SStreamMeta* pMeta) { int8_t compatible = streamMetaCheckBackendCompatible(pMeta); if (compatible == STREAM_STATA_COMPATIBLE) { return 0; @@ -263,7 +263,7 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF goto _err; } - if (streamMetaMayConvertBackendFormat(pMeta) < 0) { + if (streamMetaMayCvtDbFormat(pMeta) < 0) { goto _err; } @@ -759,16 +759,6 @@ static void doClear(void* pKey, void* pVal, TBC* pCur, SArray* pRecycleList) { int32_t streamMetaReloadAllTasks(SStreamMeta* pMeta) { if (pMeta == NULL) return 0; - // void* pIter = taosHashIterate(pMeta->pTaskBackendUnique, NULL); - // while (pIter) { - // STaskDbWrapper* taskBackend = *(STaskDbWrapper**)pIter; - // if (taskBackend != NULL) { - // taskDbRemoveRef(taskBackend); - // } - // pIter = taosHashIterate(pMeta->pTaskBackendUnique, pIter); - // } - // taosHashClear(pMeta->pTaskBackendUnique); - return streamMetaLoadAllTasks(pMeta); } int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { diff --git a/source/libs/stream/src/streamSnapshot.c b/source/libs/stream/src/streamSnapshot.c index 1717a284bc..112589613f 100644 --- a/source/libs/stream/src/streamSnapshot.c +++ b/source/libs/stream/src/streamSnapshot.c @@ -134,6 +134,7 @@ int32_t streamGetFileSize(char* path, char* name, int64_t* sz) { TdFilePtr streamOpenFile(char* path, char* name, int32_t opt) { char fullname[256] = {0}; + STREAM_ROCKSDB_BUILD_FULLNAME(path, name, fullname); return taosOpenFile(fullname, opt); } @@ -469,54 +470,53 @@ int32_t snapInfoEqual(SStreamTaskSnap* a, SStreamTaskSnap* b) { return 1; } -int32_t streamSnapWriteImpl(SStreamSnapWriter* pWriter, uint8_t* pData, uint32_t nData, - SBackendSnapFile2* pBackendFile) { +int32_t streamSnapWriteImpl(SStreamSnapWriter* pWriter, uint8_t* pData, uint32_t nData, SBackendSnapFile2* pSnapFile) { int code = -1; SStreamSnapBlockHdr* pHdr = (SStreamSnapBlockHdr*)pData; SStreamSnapHandle* pHandle = &pWriter->handle; SStreamTaskSnap snapInfo = pHdr->snapInfo; - SStreamTaskSnap* pSnapInfo = &pBackendFile->snapInfo; + SStreamTaskSnap* pSnapInfo = &pSnapFile->snapInfo; - SBackendFileItem* pItem = taosArrayGet(pBackendFile->pFileList, pBackendFile->currFileIdx); + SBackendFileItem* pItem = taosArrayGet(pSnapFile->pFileList, pSnapFile->currFileIdx); - if (pBackendFile->fd == 0) { - pBackendFile->fd = streamOpenFile(pHandle->metaPath, pItem->name, TD_FILE_CREATE | TD_FILE_WRITE | TD_FILE_APPEND); - if (pBackendFile->fd == NULL) { + if (pSnapFile->fd == 0) { + pSnapFile->fd = streamOpenFile(pSnapFile->path, pItem->name, TD_FILE_CREATE | TD_FILE_WRITE | TD_FILE_APPEND); + if (pSnapFile->fd == NULL) { code = TAOS_SYSTEM_ERROR(terrno); qError("%s failed to open file name:%s%s%s, reason:%s", STREAM_STATE_TRANSFER, pHandle->metaPath, TD_DIRSEP, pHdr->name, tstrerror(code)); } } if (strlen(pHdr->name) == strlen(pItem->name) && strcmp(pHdr->name, pItem->name) == 0) { - int64_t bytes = taosPWriteFile(pBackendFile->fd, pHdr->data, pHdr->size, pBackendFile->offset); + int64_t bytes = taosPWriteFile(pSnapFile->fd, pHdr->data, pHdr->size, pSnapFile->offset); if (bytes != pHdr->size) { code = TAOS_SYSTEM_ERROR(terrno); qError("%s failed to write snap, file name:%s, reason:%s", STREAM_STATE_TRANSFER, pHdr->name, tstrerror(code)); return code; } - pBackendFile->offset += bytes; + pSnapFile->offset += bytes; } else { - taosCloseFile(&pBackendFile->fd); - pBackendFile->offset = 0; - pBackendFile->currFileIdx += 1; + taosCloseFile(&pSnapFile->fd); + pSnapFile->offset = 0; + pSnapFile->currFileIdx += 1; SBackendFileItem item = {0}; item.name = taosStrdup(pHdr->name); item.type = pHdr->type; - taosArrayPush(pBackendFile->pFileList, &item); + taosArrayPush(pSnapFile->pFileList, &item); - SBackendFileItem* pItem = taosArrayGet(pBackendFile->pFileList, pBackendFile->currFileIdx); - pBackendFile->fd = streamOpenFile(pHandle->metaPath, pItem->name, TD_FILE_CREATE | TD_FILE_WRITE | TD_FILE_APPEND); - if (pBackendFile->fd == NULL) { + SBackendFileItem* pItem = taosArrayGet(pSnapFile->pFileList, pSnapFile->currFileIdx); + pSnapFile->fd = streamOpenFile(pSnapFile->path, pItem->name, TD_FILE_CREATE | TD_FILE_WRITE | TD_FILE_APPEND); + if (pSnapFile->fd == NULL) { code = TAOS_SYSTEM_ERROR(terrno); - qError("%s failed to open file name:%s%s%s, reason:%s", STREAM_STATE_TRANSFER, pBackendFile->path, TD_DIRSEP, + qError("%s failed to open file name:%s%s%s, reason:%s", STREAM_STATE_TRANSFER, pSnapFile->path, TD_DIRSEP, pHdr->name, tstrerror(code)); } - taosPWriteFile(pBackendFile->fd, pHdr->data, pHdr->size, pBackendFile->offset); - pBackendFile->offset += pHdr->size; + taosPWriteFile(pSnapFile->fd, pHdr->data, pHdr->size, pSnapFile->offset); + pSnapFile->offset += pHdr->size; } code = 0; _EXIT: @@ -531,6 +531,11 @@ int32_t streamSnapWrite(SStreamSnapWriter* pWriter, uint8_t* pData, uint32_t nDa SBackendSnapFile2* pDbSnapFile = taosArrayGet(pHandle->pDbSnapSet, pHandle->currIdx); if (pDbSnapFile->inited == 0) { + char* path = taosMemoryCalloc(1, strlen(pHandle->metaPath) + 256); + sprintf(path, "%s%s%" PRId64 "_%" PRId64 "%s%s%s%s%scheckpoint%" PRId64 "", path, TD_DIRSEP, snapInfo.streamId, + snapInfo.taskId, TD_DIRSEP, "state", TD_DIRSEP, "checkpoints", TD_DIRSEP, snapInfo.chkpId); + + pDbSnapFile->path = path; pDbSnapFile->snapInfo = snapInfo; pDbSnapFile->pFileList = taosArrayInit(64, sizeof(SBackendFileItem)); pDbSnapFile->currFileIdx = 0; @@ -539,6 +544,7 @@ int32_t streamSnapWrite(SStreamSnapWriter* pWriter, uint8_t* pData, uint32_t nDa SBackendFileItem item = {0}; item.name = taosStrdup((char*)ROCKSDB_CURRENT); item.type = ROCKSDB_CURRENT_TYPE; + taosArrayPush(pDbSnapFile->pFileList, &item); pDbSnapFile->inited = 1; From a56d831a144a25b540b224d6ee635b2085fd79ee Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 12 Oct 2023 20:16:41 +0800 Subject: [PATCH 042/195] refactor backend --- include/libs/stream/tstream.h | 4 +- source/dnode/vnode/src/tq/tq.c | 5 +- source/libs/stream/inc/streamBackendRocksdb.h | 8 +- source/libs/stream/src/streamBackendRocksdb.c | 84 ++++++++++++++++--- source/libs/stream/src/streamMeta.c | 21 +++-- source/libs/stream/src/streamSnapshot.c | 38 +++++---- source/libs/stream/src/streamState.c | 4 +- source/libs/stream/src/streamTask.c | 2 - 8 files changed, 120 insertions(+), 46 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 260609e697..1d4e87e2bd 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -410,7 +410,7 @@ typedef struct SStreamMeta { int32_t walScanCounter; void* streamBackend; int64_t streamBackendRid; - SHashObj* pTaskBackendUnique; + SHashObj* pTaskDbUnique; TdThreadMutex backendMutex; SMetaHbInfo hbInfo; int32_t closedTask; @@ -727,7 +727,7 @@ int32_t streamMetaCommit(SStreamMeta* pMeta); int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta); int32_t streamMetaReloadAllTasks(SStreamMeta* pMeta); void streamMetaNotifyClose(SStreamMeta* pMeta); -void* streamMetaGetBackendByTaskKey(SStreamMeta* pMeta, char* key, int64_t* ref); +void* streamMetaGetBackendByTaskKey(SStreamMeta* pMeta, char* key, int64_t chkpId, int64_t* ref); // checkpoint int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSourceReq* pReq); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 3546df7e91..fbd8a3ee05 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -738,7 +738,10 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { int32_t code = streamTaskInit(pTask, pTq->pStreamMeta, &pTq->pVnode->msgCb, ver); if (code != TSDB_CODE_SUCCESS) return code; - pTask->pBackend = streamMetaGetBackendByTaskKey(pTq->pStreamMeta, (char*)pTask->id.idStr, &pTask->backendRefId); + pTask->pBackend = streamMetaGetBackendByTaskKey(pTq->pStreamMeta, (char*)pTask->id.idStr, pTask->checkpointingId, + &pTask->backendRefId); + + // taskDbUpdateChkpId(pTask->pBackend, pTask->checkpointingId); if (pTask->pBackend == NULL) return -1; streamTaskOpenAllUpstreamInput(pTask); diff --git a/source/libs/stream/inc/streamBackendRocksdb.h b/source/libs/stream/inc/streamBackendRocksdb.h index 0f97bda4c5..2403bca7f6 100644 --- a/source/libs/stream/inc/streamBackendRocksdb.h +++ b/source/libs/stream/inc/streamBackendRocksdb.h @@ -83,12 +83,14 @@ int32_t streamBackendLoadCheckpointInfo(void* pMeta); int32_t streamBackendDoCheckpoint(void* pMeta, int64_t checkpointId); SListNode* streamBackendAddCompare(void* backend, void* arg); void streamBackendDelCompare(void* backend, void* arg); -int32_t streamStateConvertDataFormat(char* path, char* key, void* cfInst); +int32_t streamStateCvtDataFormat(char* path, char* key, void* cfInst); -STaskDbWrapper* taskDbOpen(char* path, char* key); +STaskDbWrapper* taskDbOpen(char* path, char* key, int64_t chkpId); void taskDbDestroy(void* pBackend); int32_t taskDbDoCheckpoint(void* arg, int64_t chkpId); +void taskDbUpdateChkpId(void* pTaskDb, int64_t chkpId); + void* taskDbAddRef(void* pTaskDb); void taskDbRemoveRef(void* pTaskDb); @@ -182,7 +184,7 @@ int32_t streamBackendTriggerChkp(void* pMeta, char* dst); int32_t streamBackendAddInUseChkp(void* arg, int64_t chkpId); int32_t streamBackendDelInUseChkp(void* arg, int64_t chkpId); -int32_t taskDbBuildSnap(void* arg, int64_t chkpId); +int32_t taskDbBuildSnap(void* arg); // int32_t streamDefaultIter_rocksdb(SStreamState* pState, const void* start, const void* end, SArray* result); #endif \ No newline at end of file diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 78f67d06d5..39c423b7fb 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -450,6 +450,60 @@ int32_t rebuildDirFromCheckpoint(const char* path, int64_t chkpId, char** dst) { return 0; } +int32_t rebuildDirFromChkp2(const char* path, char* key, int64_t chkpId, char** dbPrefixPath, char** dbPath) { + // impl later + int32_t code = 0; + + /*param@1: checkpointId dir + param@2: state + copy pChkpIdDir's file to state dir + opt to set hard link to previous file + */ + + char* prefixPath = taosMemoryCalloc(1, strlen(path) + 128); + sprintf(prefixPath, "%s%s%s", path, TD_DIRSEP, key); + + if (!taosIsDir(prefixPath)) { + code = taosMkDir(prefixPath); + ASSERT(code == 0); + } + + char* defaultPath = taosMemoryCalloc(1, strlen(path) + 256); + sprintf(defaultPath, "%s%s%s", prefixPath, TD_DIRSEP, "defaultPath"); + if (!taosIsDir(defaultPath)) { + taosMulMkDir(defaultPath); + } + + if (chkpId != 0) { + char* chkpPath = taosMemoryCalloc(1, strlen(path) + 256); + sprintf(chkpPath, "%s%s%s%s%" PRId64 "", prefixPath, TD_DIRSEP, "checkpoints", TD_DIRSEP, "checkpoint", chkpId); + if (taosIsDir(chkpPath) && isValidCheckpoint(chkpPath)) { + if (taosIsDir(defaultPath)) { + // remove dir if exists + // taosRenameFile(const char *oldName, const char *newName) + taosRemoveDir(defaultPath); + } + taosMkDir(defaultPath); + code = copyFiles(chkpPath, defaultPath); + if (code != 0) { + qError("failed to restart stream backend from %s, reason: %s", chkpPath, tstrerror(TAOS_SYSTEM_ERROR(errno))); + } else { + qInfo("start to restart stream backend at checkpoint path: %s", chkpPath); + } + + } else { + qError("failed to start stream backend at %s, reason: %s, restart from default defaultPath dir:%s", chkpPath, + tstrerror(TAOS_SYSTEM_ERROR(errno)), defaultPath); + taosMkDir(defaultPath); + } + taosMemoryFree(chkpPath); + } + + *dbPath = defaultPath; + *dbPrefixPath = prefixPath; + + return 0; +} void* streamBackendInit(const char* streamPath, int64_t chkpId) { char* backendPath = NULL; @@ -829,13 +883,13 @@ int32_t streamBackendLoadCheckpointInfo(void* arg) { int32_t chkpGetAllDbCfHandle(SStreamMeta* pMeta, rocksdb_column_family_handle_t*** ppHandle, SArray* refs) { return 0; // SArray* pHandle = taosArrayInit(16, POINTER_BYTES); - // void* pIter = taosHashIterate(pMeta->pTaskBackendUnique, NULL); + // void* pIter = taosHashIterate(pMeta->pTaskDbUnique, NULL); // while (pIter) { // int64_t id = *(int64_t*)pIter; // SBackendCfWrapper* wrapper = taosAcquireRef(streamBackendCfWrapperId, id); // if (wrapper == NULL) { - // pIter = taosHashIterate(pMeta->pTaskBackendUnique, pIter); + // pIter = taosHashIterate(pMeta->pTaskDbUnique, pIter); // continue; // } @@ -940,7 +994,7 @@ int32_t chkpPreBuildDir(char* path, int64_t chkpId, char** chkpDir, char** chkpI return code; } - sprintf(pChkpIdDir, "%s%scheckpoint%" PRId64, pChkpDir, TD_DIRSEP, chkpId); + sprintf(pChkpIdDir, "%s%s%s%" PRId64, pChkpDir, TD_DIRSEP, "checkpoint", chkpId); if (taosIsDir(pChkpIdDir)) { qInfo("stream rm exist checkpoint%s", pChkpIdDir); taosRemoveFile(pChkpIdDir); @@ -950,19 +1004,19 @@ int32_t chkpPreBuildDir(char* path, int64_t chkpId, char** chkpDir, char** chkpI return 0; } -int32_t taskDbBuildSnap(void* arg, int64_t chkpId) { +int32_t taskDbBuildSnap(void* arg) { SStreamMeta* pMeta = arg; - void* pIter = taosHashIterate(pMeta->pTaskBackendUnique, NULL); + void* pIter = taosHashIterate(pMeta->pTaskDbUnique, NULL); int32_t code = 0; while (pIter) { STaskDbWrapper* pTaskDb = *(STaskDbWrapper**)pIter; taskDbAddRef(pTaskDb); - code = taskDbDoCheckpoint((STaskDbWrapper*)pTaskDb, chkpId); + code = taskDbDoCheckpoint(pTaskDb, pTaskDb->chkpId); taskDbRemoveRef(pTaskDb); - pIter = taosHashIterate(pMeta->pTaskBackendUnique, pIter); + pIter = taosHashIterate(pMeta->pTaskDbUnique, pIter); } return 0; } @@ -1070,6 +1124,8 @@ int32_t taskDbDoCheckpoint(void* arg, int64_t chkpId) { code = chkpMayDelObsolete(pTaskDb, chkpId, pChkpDir); + pTaskDb->chkpId = chkpId; + _EXIT: taosMemoryFree(pChkpDir); taosMemoryFree(pChkpIdDir); @@ -1692,14 +1748,20 @@ int32_t taskDbBuildFullPath(char* path, char* key, char** dbFullPath, char** sta *stateFullPath = statePath; return 0; } -STaskDbWrapper* taskDbOpen(char* path, char* key) { +void taskDbUpdateChkpId(void* pTaskDb, int64_t chkpId) { + STaskDbWrapper* p = pTaskDb; + taosThreadMutexLock(&p->mutex); + p->chkpId = chkpId; + taosThreadMutexUnlock(&p->mutex); +} +STaskDbWrapper* taskDbOpen(char* path, char* key, int64_t chkpId) { char* statePath = NULL; char* err = NULL; char* dbPath = NULL; char** cfNames = NULL; size_t nCf = 0; - if (taskDbBuildFullPath(path, key, &dbPath, &statePath) != 0) { + if (rebuildDirFromChkp2(path, key, chkpId, &statePath, &dbPath) != 0) { return NULL; } @@ -1861,12 +1923,12 @@ _EXIT: return code; } -int32_t streamStateConvertDataFormat(char* path, char* key, void* pCfInst) { +int32_t streamStateCvtDataFormat(char* path, char* key, void* pCfInst) { int nCf = sizeof(ginitDict) / sizeof(ginitDict[0]); int32_t code = 0; - STaskDbWrapper* pTaskDb = taskDbOpen(path, key); + STaskDbWrapper* pTaskDb = taskDbOpen(path, key, 0); RocksdbCfInst* pSrcBackend = pCfInst; for (int i = 0; i < nCf; i++) { diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 9d76951adb..016f446fe9 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -182,7 +182,7 @@ int32_t streamMetaCheckBackendCompatible(SStreamMeta* pMeta) { return ret; } -int32_t streamMetaConvertBackendFormat(SStreamMeta* pMeta) { +int32_t streamMetaCvtDbFormat(SStreamMeta* pMeta) { int32_t code = 0; int64_t chkpId = streamGetLatestCheckpointId(pMeta); SBackendWrapper* pBackend = streamBackendInit(pMeta->path, chkpId); @@ -190,7 +190,7 @@ int32_t streamMetaConvertBackendFormat(SStreamMeta* pMeta) { void* pIter = taosHashIterate(pBackend->cfInst, NULL); while (pIter) { void* key = taosHashGetKey(pIter, NULL); - code = streamStateConvertDataFormat(pMeta->path, key, *(void**)pIter); + code = streamStateCvtDataFormat(pMeta->path, key, *(void**)pIter); if (code != 0) { qError("failed to cvt data"); goto _EXIT; @@ -210,7 +210,7 @@ int32_t streamMetaMayCvtDbFormat(SStreamMeta* pMeta) { } else if (compatible == STREAM_STATA_NEED_CONVERT) { qInfo("stream state need covert backend format"); - return streamMetaConvertBackendFormat(pMeta); + return streamMetaCvtDbFormat(pMeta); } else if (compatible == STREAM_STATA_NO_COMPATIBLE) { qError( "stream read incompatible data, rm %s/vnode/vnode*/tq/stream if taosd cannot start, and rebuild stream " @@ -222,9 +222,9 @@ int32_t streamMetaMayCvtDbFormat(SStreamMeta* pMeta) { return 0; } -void* streamMetaGetBackendByTaskKey(SStreamMeta* pMeta, char* key, int64_t* ref) { +void* streamMetaGetBackendByTaskKey(SStreamMeta* pMeta, char* key, int64_t chkpId, int64_t* ref) { taosThreadMutexLock(&pMeta->backendMutex); - void** ppBackend = taosHashGet(pMeta->pTaskBackendUnique, key, strlen(key)); + void** ppBackend = taosHashGet(pMeta->pTaskDbUnique, key, strlen(key)); if (ppBackend != NULL && *ppBackend != NULL) { taskDbAddRef(*ppBackend); *ref = ((STaskDbWrapper*)*ppBackend)->refId; @@ -232,7 +232,7 @@ void* streamMetaGetBackendByTaskKey(SStreamMeta* pMeta, char* key, int64_t* ref) return *ppBackend; } - void* pBackend = taskDbOpen(pMeta->path, key); + void* pBackend = taskDbOpen(pMeta->path, key, chkpId); if (pBackend == NULL) { taosThreadMutexUnlock(&pMeta->backendMutex); return NULL; @@ -240,7 +240,7 @@ void* streamMetaGetBackendByTaskKey(SStreamMeta* pMeta, char* key, int64_t* ref) *ref = taosAddRef(taskDbWrapperId, pBackend); - taosHashPut(pMeta->pTaskBackendUnique, key, strlen(key), &pBackend, sizeof(void*)); + taosHashPut(pMeta->pTaskDbUnique, key, strlen(key), &pBackend, sizeof(void*)); taosThreadMutexUnlock(&pMeta->backendMutex); return pBackend; } @@ -301,8 +301,7 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF pMeta->hbInfo.tickCounter = 0; pMeta->hbInfo.stopFlag = 0; - pMeta->pTaskBackendUnique = - taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), false, HASH_ENTRY_LOCK); + pMeta->pTaskDbUnique = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), false, HASH_ENTRY_LOCK); // start backend // taosInitRWLatch(&pMeta->chkpDirLock); @@ -411,7 +410,7 @@ void streamMetaClear(SStreamMeta* pMeta) { taosRemoveRef(streamBackendId, pMeta->streamBackendRid); taosHashClear(pMeta->pTasks); - taosHashClear(pMeta->pTaskBackendUnique); + taosHashClear(pMeta->pTaskDbUnique); taosArrayClear(pMeta->pTaskList); taosArrayClear(pMeta->chkpSaved); @@ -452,7 +451,7 @@ void streamMetaCloseImpl(void* arg) { taosArrayDestroy(pMeta->chkpInUse); taosHashCleanup(pMeta->pTasks); - taosHashCleanup(pMeta->pTaskBackendUnique); + taosHashCleanup(pMeta->pTaskDbUnique); taosMemoryFree(pMeta->path); taosThreadMutexDestroy(&pMeta->backendMutex); diff --git a/source/libs/stream/src/streamSnapshot.c b/source/libs/stream/src/streamSnapshot.c index 112589613f..676effe778 100644 --- a/source/libs/stream/src/streamSnapshot.c +++ b/source/libs/stream/src/streamSnapshot.c @@ -110,7 +110,7 @@ const char* ROCKSDB_CURRENT = "CURRENT"; const char* ROCKSDB_CHECKPOINT_META = "CHECKPOINT"; static int64_t kBlockSize = 64 * 1024; -int32_t streamSnapHandleInit(SStreamSnapHandle* handle, char* path, int64_t chkpId, void* pMeta); +int32_t streamSnapHandleInit(SStreamSnapHandle* handle, char* path, void* pMeta); void streamSnapHandleDestroy(SStreamSnapHandle* handle); // static void streamBuildFname(char* path, char* file, char* fullname) @@ -139,7 +139,7 @@ TdFilePtr streamOpenFile(char* path, char* name, int32_t opt) { return taosOpenFile(fullname, opt); } -int32_t streamBackendGetSnapInfo(void* arg, char* path, int64_t chkpId) { return taskDbBuildSnap(arg, chkpId); } +int32_t streamTaskDbGetSnapInfo(void* arg, char* path) { return taskDbBuildSnap(arg); } void snapFileDebugInfo(SBackendSnapFile2* pSnapFile) { if (qDebugFlag & DEBUG_DEBUG) { @@ -235,20 +235,23 @@ int32_t snapFileReadMeta(SBackendSnapFile2* pSnapFile) { taosCloseDir(&pDir); return 0; } -int32_t streamBackendSnapInitFile(char* path, SStreamTaskSnap* pSnap, SBackendSnapFile2* pSnapFile) { +int32_t streamBackendSnapInitFile(char* metaPath, SStreamTaskSnap* pSnap, SBackendSnapFile2* pSnapFile) { // SBanckendFile* pFile = taosMemoryCalloc(1, sizeof(SBanckendFile)); int32_t code = -1; - char* snapPath = taosMemoryCalloc(1, strlen(path) + 256); - sprintf(snapPath, "%s%s%" PRId64 "_%" PRId64 "%s%s%s%s%scheckpoint%" PRId64 "", path, TD_DIRSEP, pSnap->streamId, - pSnap->taskId, TD_DIRSEP, "state", TD_DIRSEP, "checkpoints", TD_DIRSEP, pSnap->chkpId); - if (taosIsDir(snapPath)) { + char* path = taosMemoryCalloc(1, strlen(metaPath) + 256); + char idstr[64] = {0}; + sprintf(idstr, "0x%" PRIx64 "-0x%x", pSnap->streamId, (int32_t)(pSnap->taskId)); + + sprintf(path, "%s%s%s%s%s%s%s%" PRId64 "", metaPath, TD_DIRSEP, idstr, TD_DIRSEP, "checkpoints", TD_DIRSEP, + "checkpoint", pSnap->chkpId); + if (taosIsDir(path)) { goto _ERROR; } pSnapFile->pSst = taosArrayInit(16, sizeof(void*)); pSnapFile->pFileList = taosArrayInit(64, sizeof(SBackendFileItem)); - pSnapFile->path = snapPath; + pSnapFile->path = path; pSnapFile->snapInfo = *pSnap; if ((code = snapFileReadMeta(pSnapFile)) != 0) { goto _ERROR; @@ -262,7 +265,7 @@ int32_t streamBackendSnapInitFile(char* path, SStreamTaskSnap* pSnap, SBackendSn code = 0; _ERROR: - taosMemoryFree(snapPath); + taosMemoryFree(path); return code; } void snapFileDestroy(SBackendSnapFile2* pSnap) { @@ -288,11 +291,11 @@ void snapFileDestroy(SBackendSnapFile2* pSnap) { return; } -int32_t streamSnapHandleInit(SStreamSnapHandle* pHandle, char* path, int64_t chkpId, void* pMeta) { +int32_t streamSnapHandleInit(SStreamSnapHandle* pHandle, char* path, void* pMeta) { // impl later SArray* pSnapSet = NULL; - int32_t code = streamBackendGetSnapInfo(pMeta, path, chkpId); + int32_t code = streamTaskDbGetSnapInfo(pMeta, path); if (code != 0) { return -1; } @@ -339,7 +342,7 @@ int32_t streamSnapReaderOpen(void* pMeta, int64_t sver, int64_t chkpId, char* pa return TSDB_CODE_OUT_OF_MEMORY; } - if (streamSnapHandleInit(&pReader->handle, (char*)path, chkpId, pMeta) < 0) { + if (streamSnapHandleInit(&pReader->handle, (char*)path, pMeta) < 0) { taosMemoryFree(pReader); return -1; } @@ -531,9 +534,16 @@ int32_t streamSnapWrite(SStreamSnapWriter* pWriter, uint8_t* pData, uint32_t nDa SBackendSnapFile2* pDbSnapFile = taosArrayGet(pHandle->pDbSnapSet, pHandle->currIdx); if (pDbSnapFile->inited == 0) { + char idstr[64] = {0}; + sprintf(idstr, "0x%" PRIx64 "-0x%x", snapInfo.streamId, (int32_t)(snapInfo.taskId)); + char* path = taosMemoryCalloc(1, strlen(pHandle->metaPath) + 256); - sprintf(path, "%s%s%" PRId64 "_%" PRId64 "%s%s%s%s%scheckpoint%" PRId64 "", path, TD_DIRSEP, snapInfo.streamId, - snapInfo.taskId, TD_DIRSEP, "state", TD_DIRSEP, "checkpoints", TD_DIRSEP, snapInfo.chkpId); + sprintf(path, "%s%s%s%s%s%s%s%" PRId64 "", path, TD_DIRSEP, idstr, TD_DIRSEP, "checkpoints", TD_DIRSEP, + "checkpoint", snapInfo.chkpId); + if (!taosIsDir(path)) { + code = taosMulMkDir(path); + ASSERT(code == 0); + } pDbSnapFile->path = path; pDbSnapFile->snapInfo = snapInfo; diff --git a/source/libs/stream/src/streamState.c b/source/libs/stream/src/streamState.c index d4ac1838d0..b8158c774e 100644 --- a/source/libs/stream/src/streamState.c +++ b/source/libs/stream/src/streamState.c @@ -116,7 +116,7 @@ SStreamState* streamStateOpen(char* path, void* pTask, bool specPath, int32_t sz // taosWLockLatch(&pMeta->lock); // taosThreadMutexLock(&pMeta->backendMutex); // void* uniqueId = - // taosHashGet(pMeta->pTaskBackendUnique, pState->pTdbState->idstr, strlen(pState->pTdbState->idstr) + 1); + // taosHashGet(pMeta->pTaskDbUnique, pState->pTdbState->idstr, strlen(pState->pTdbState->idstr) + 1); // if (uniqueId == NULL) { // int code = streamStateOpenBackend(pMeta->streamBackend, pState); // if (code == -1) { @@ -124,7 +124,7 @@ SStreamState* streamStateOpen(char* path, void* pTask, bool specPath, int32_t sz // taosMemoryFree(pState); // return NULL; // } - // taosHashPut(pMeta->pTaskBackendUnique, pState->pTdbState->idstr, strlen(pState->pTdbState->idstr) + 1, + // taosHashPut(pMeta->pTaskDbUnique, pState->pTdbState->idstr, strlen(pState->pTdbState->idstr) + 1, // &pState->pTdbState->backendCfWrapperId, sizeof(pState->pTdbState->backendCfWrapperId)); // } else { // int64_t id = *(int64_t*)uniqueId; diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 5b4b0c309d..8da515a650 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -392,8 +392,6 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i taosThreadMutexInit(&pTask->lock, NULL); streamTaskOpenAllUpstreamInput(pTask); - // pTask->pBackend = taskDbOpen(pMeta->path, (char*)pTask->id.idStr); - return TSDB_CODE_SUCCESS; } From 0ba7e2278a9531b02640e8ac533f2d62c9a399c2 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 12 Oct 2023 20:21:11 +0800 Subject: [PATCH 043/195] refactor backend --- source/libs/stream/src/streamBackendRocksdb.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 39c423b7fb..d3e8a4e428 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -476,7 +476,7 @@ int32_t rebuildDirFromChkp2(const char* path, char* key, int64_t chkpId, char** if (chkpId != 0) { char* chkpPath = taosMemoryCalloc(1, strlen(path) + 256); - sprintf(chkpPath, "%s%s%s%s%" PRId64 "", prefixPath, TD_DIRSEP, "checkpoints", TD_DIRSEP, "checkpoint", chkpId); + sprintf(chkpPath, "%s%s%s%s%s%" PRId64 "", prefixPath, TD_DIRSEP, "checkpoints", TD_DIRSEP, "checkpoint", chkpId); if (taosIsDir(chkpPath) && isValidCheckpoint(chkpPath)) { if (taosIsDir(defaultPath)) { // remove dir if exists From 8c1a9f7c585f05387d191f98ae1b01b7357cf76f Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 12 Oct 2023 20:28:40 +0800 Subject: [PATCH 044/195] refactor backend --- source/libs/stream/src/streamBackendRocksdb.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index d3e8a4e428..274b298445 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -469,7 +469,7 @@ int32_t rebuildDirFromChkp2(const char* path, char* key, int64_t chkpId, char** } char* defaultPath = taosMemoryCalloc(1, strlen(path) + 256); - sprintf(defaultPath, "%s%s%s", prefixPath, TD_DIRSEP, "defaultPath"); + sprintf(defaultPath, "%s%s%s", prefixPath, TD_DIRSEP, "state"); if (!taosIsDir(defaultPath)) { taosMulMkDir(defaultPath); } From d6ff2820060b157b94bf47c1674a6a4cc59db964 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 12 Oct 2023 20:42:36 +0800 Subject: [PATCH 045/195] refactor backend --- source/libs/stream/src/streamBackendRocksdb.c | 1 + 1 file changed, 1 insertion(+) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 274b298445..48e9d506d6 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1768,6 +1768,7 @@ STaskDbWrapper* taskDbOpen(char* path, char* key, int64_t chkpId) { STaskDbWrapper* pTaskDb = taosMemoryCalloc(1, sizeof(STaskDbWrapper)); pTaskDb->idstr = taosStrdup(key); pTaskDb->path = statePath; + taosThreadMutexInit(&pTaskDb->mutex, NULL); taskDbInitChkpOpt(pTaskDb); taskDbInitDBOpt(pTaskDb); From 8392c99d36435173532dd61a9df7ffa2c374b078 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Fri, 13 Oct 2023 14:54:00 +0800 Subject: [PATCH 046/195] refactor backend --- source/dnode/vnode/src/tq/tq.c | 28 ++++++++++++++-------------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index fbd8a3ee05..778a01719f 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -748,14 +748,14 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { SStreamTask* pStateTask = pTask; - SStreamTask task = {0}; - if (pTask->info.fillHistory) { - task.id = pTask->streamTaskId; - task.pMeta = pTask->pMeta; - pStateTask = &task; - } + // SStreamTask task = {0}; + // if (pTask->info.fillHistory) { + // task.id = pTask->streamTaskId; + // task.pMeta = pTask->pMeta; + // pStateTask = &task; + // } - pTask->pState = streamStateOpen(pTq->pStreamMeta->path, pStateTask, false, -1, -1); + pTask->pState = streamStateOpen(pTq->pStreamMeta->path, pTask, false, -1, -1); if (pTask->pState == NULL) { tqError("s-task:%s (vgId:%d) failed to open state for task", pTask->id.idStr, vgId); return -1; @@ -782,14 +782,14 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { qSetTaskId(pTask->exec.pExecutor, pTask->id.taskId, pTask->id.streamId); } else if (pTask->info.taskLevel == TASK_LEVEL__AGG) { SStreamTask* pSateTask = pTask; - SStreamTask task = {0}; - if (pTask->info.fillHistory) { - task.id = pTask->streamTaskId; - task.pMeta = pTask->pMeta; - pSateTask = &task; - } + // SStreamTask task = {0}; + // if (pTask->info.fillHistory) { + // task.id = pTask->streamTaskId; + // task.pMeta = pTask->pMeta; + // pSateTask = &task; + // } - pTask->pState = streamStateOpen(pTq->pStreamMeta->path, pSateTask, false, -1, -1); + pTask->pState = streamStateOpen(pTq->pStreamMeta->path, pTask, false, -1, -1); if (pTask->pState == NULL) { tqError("s-task:%s (vgId:%d) failed to open state for task", pTask->id.idStr, vgId); return -1; From b6005d618351e9301a387aff6d9f836b529f5d51 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Fri, 13 Oct 2023 16:07:34 +0800 Subject: [PATCH 047/195] refactor backend --- .../executor/src/streamtimewindowoperator.c | 42 ++++++++++--------- 1 file changed, 23 insertions(+), 19 deletions(-) diff --git a/source/libs/executor/src/streamtimewindowoperator.c b/source/libs/executor/src/streamtimewindowoperator.c index 1c909cb47d..cf558be1be 100644 --- a/source/libs/executor/src/streamtimewindowoperator.c +++ b/source/libs/executor/src/streamtimewindowoperator.c @@ -1081,10 +1081,10 @@ void doStreamIntervalDecodeOpState(void* buf, int32_t len, SOperatorInfo* pOpera int32_t mapSize = 0; buf = taosDecodeFixedI32(buf, &mapSize); for (int32_t i = 0; i < mapSize; i++) { - SWinKey key = {0}; + SWinKey key = {0}; buf = decodeSWinKey(buf, &key); SRowBuffPos* pPos = NULL; - int32_t resSize = pInfo->aggSup.resultRowSize; + int32_t resSize = pInfo->aggSup.resultRowSize; pInfo->stateStore.streamStateAddIfNotExist(pInfo->pState, &key, (void**)&pPos, &resSize); tSimpleHashPut(pInfo->aggSup.pResultRowHashTable, &key, sizeof(SWinKey), &pPos, POINTER_BYTES); } @@ -1388,10 +1388,12 @@ void streamIntervalReloadState(SOperatorInfo* pOperator) { void* pBuf = NULL; int32_t code = pInfo->stateStore.streamStateGetInfo(pInfo->pState, STREAM_INTERVAL_OP_STATE_NAME, strlen(STREAM_INTERVAL_OP_STATE_NAME), &pBuf, &size); - TSKEY ts = *(TSKEY*)pBuf; - taosMemoryFree(pBuf); - pInfo->twAggSup.maxTs = TMAX(pInfo->twAggSup.maxTs, ts); - pInfo->stateStore.streamStateReloadInfo(pInfo->pState, ts); + if (code == 0) { + TSKEY ts = *(TSKEY*)pBuf; + taosMemoryFree(pBuf); + pInfo->twAggSup.maxTs = TMAX(pInfo->twAggSup.maxTs, ts); + pInfo->stateStore.streamStateReloadInfo(pInfo->pState, ts); + } } SOperatorInfo* downstream = pOperator->pDownstream[0]; if (downstream->fpSet.reloadStreamStateFn) { @@ -2527,16 +2529,16 @@ void resetWinRange(STimeWindow* winRange) { void streamSessionSemiReloadState(SOperatorInfo* pOperator) { SStreamSessionAggOperatorInfo* pInfo = pOperator->info; - SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; + SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; resetWinRange(&pAggSup->winRange); SResultWindowInfo winInfo = {0}; - int32_t size = 0; - void* pBuf = NULL; - int32_t code = pAggSup->stateStore.streamStateGetInfo(pAggSup->pState, STREAM_SESSION_OP_STATE_NAME, - strlen(STREAM_SESSION_OP_STATE_NAME), &pBuf, &size); - int32_t num = size / sizeof(SSessionKey); - SSessionKey* pSeKeyBuf = (SSessionKey*) pBuf; + int32_t size = 0; + void* pBuf = NULL; + int32_t code = pAggSup->stateStore.streamStateGetInfo(pAggSup->pState, STREAM_SESSION_OP_STATE_NAME, + strlen(STREAM_SESSION_OP_STATE_NAME), &pBuf, &size); + int32_t num = size / sizeof(SSessionKey); + SSessionKey* pSeKeyBuf = (SSessionKey*)pBuf; ASSERT(size == num * sizeof(SSessionKey)); for (int32_t i = 0; i < num; i++) { SResultWindowInfo winInfo = {0}; @@ -2671,7 +2673,7 @@ SOperatorInfo* createStreamSessionAggOperatorInfo(SOperatorInfo* downstream, SPh pInfo->pCheckpointRes = createSpecialDataBlock(STREAM_CHECKPOINT); pOperator->operatorType = QUERY_NODE_PHYSICAL_PLAN_STREAM_SESSION; - // for stream + // for stream void* buff = NULL; int32_t len = 0; int32_t res = @@ -2792,7 +2794,7 @@ static SSDataBlock* doStreamSessionSemiAgg(SOperatorInfo* pOperator) { tSimpleHashCleanup(pInfo->pStUpdated); pInfo->pStUpdated = NULL; - if(pInfo->isHistoryOp) { + if (pInfo->isHistoryOp) { getMaxTsWins(pInfo->pUpdated, pInfo->historyWins); } @@ -2826,8 +2828,9 @@ SOperatorInfo* createStreamFinalSessionAggOperatorInfo(SOperatorInfo* downstream pOperator->operatorType = pPhyNode->type; if (pPhyNode->type != QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_SESSION) { - pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, doStreamSessionSemiAgg, NULL, - destroyStreamSessionAggOperatorInfo, optrDefaultBufFn, NULL, optrDefaultGetNextExtFn, NULL); + pOperator->fpSet = + createOperatorFpSet(optrDummyOpenFn, doStreamSessionSemiAgg, NULL, destroyStreamSessionAggOperatorInfo, + optrDefaultBufFn, NULL, optrDefaultGetNextExtFn, NULL); setOperatorStreamStateFn(pOperator, streamSessionReleaseState, streamSessionSemiReloadState); } setOperatorInfo(pOperator, getStreamOpName(pOperator->operatorType), pPhyNode->type, false, OP_NOT_OPENED, pInfo, @@ -3738,8 +3741,9 @@ SOperatorInfo* createStreamIntervalOperatorInfo(SOperatorInfo* downstream, SPhys setOperatorInfo(pOperator, "StreamIntervalOperator", QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL, true, OP_NOT_OPENED, pInfo, pTaskInfo); - pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, doStreamIntervalAgg, NULL, - destroyStreamFinalIntervalOperatorInfo, optrDefaultBufFn, NULL, optrDefaultGetNextExtFn, NULL); + pOperator->fpSet = + createOperatorFpSet(optrDummyOpenFn, doStreamIntervalAgg, NULL, destroyStreamFinalIntervalOperatorInfo, + optrDefaultBufFn, NULL, optrDefaultGetNextExtFn, NULL); setOperatorStreamStateFn(pOperator, streamIntervalReleaseState, streamIntervalReloadState); pInfo->stateStore = pTaskInfo->storageAPI.stateStore; From 77542cc461f1b78da131b5d8f9f6bf2af190f9df Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Tue, 17 Oct 2023 11:23:39 +0800 Subject: [PATCH 048/195] fix stream snap deadlock --- source/libs/stream/inc/streamBackendRocksdb.h | 2 +- source/libs/stream/src/streamMeta.c | 7 +++---- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/source/libs/stream/inc/streamBackendRocksdb.h b/source/libs/stream/inc/streamBackendRocksdb.h index 2403bca7f6..637fbf1017 100644 --- a/source/libs/stream/inc/streamBackendRocksdb.h +++ b/source/libs/stream/inc/streamBackendRocksdb.h @@ -65,8 +65,8 @@ typedef struct { rocksdb_compactionfilterfactory_t* filterFactory; TdThreadMutex mutex; char* idstr; - int64_t refId; char* path; + int64_t refId; int64_t chkpId; SArray* chkpSaved; diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 016f446fe9..656b115bcf 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -237,8 +237,9 @@ void* streamMetaGetBackendByTaskKey(SStreamMeta* pMeta, char* key, int64_t chkpI taosThreadMutexUnlock(&pMeta->backendMutex); return NULL; } - - *ref = taosAddRef(taskDbWrapperId, pBackend); + int64_t tref = taosAddRef(taskDbWrapperId, pBackend); + *ref = tref; + ((STaskDbWrapper*)pBackend)->refId = tref; taosHashPut(pMeta->pTaskDbUnique, key, strlen(key), &pBackend, sizeof(void*)); taosThreadMutexUnlock(&pMeta->backendMutex); @@ -824,8 +825,6 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { taosArrayPush(pMeta->pTaskList, &pTask->id); } else { - tdbFree(pKey); - tdbFree(pVal); taosMemoryFree(pTask); continue; } From 2004c1a34681a67d433ccc9624da031fbf52540f Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Tue, 17 Oct 2023 18:19:52 +0800 Subject: [PATCH 049/195] fix transfer crash --- include/libs/stream/tstream.h | 2 +- source/dnode/vnode/src/tq/tq.c | 8 ++---- source/dnode/vnode/src/tq/tqStreamTaskSnap.c | 3 -- source/libs/stream/inc/streamBackendRocksdb.h | 7 +++-- source/libs/stream/inc/streamInt.h | 1 + source/libs/stream/src/streamBackendRocksdb.c | 9 +++++- source/libs/stream/src/streamMeta.c | 28 +++++++++++++------ source/libs/stream/src/streamSnapshot.c | 27 ++++++++++-------- source/libs/stream/src/streamTask.c | 5 ++++ 9 files changed, 58 insertions(+), 32 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 1d4e87e2bd..04866cb3c4 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -727,7 +727,7 @@ int32_t streamMetaCommit(SStreamMeta* pMeta); int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta); int32_t streamMetaReloadAllTasks(SStreamMeta* pMeta); void streamMetaNotifyClose(SStreamMeta* pMeta); -void* streamMetaGetBackendByTaskKey(SStreamMeta* pMeta, char* key, int64_t chkpId, int64_t* ref); +int32_t streamTaskSetDb(SStreamMeta* pMeta, void* pTask); // checkpoint int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSourceReq* pReq); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 778a01719f..6f1675f528 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -738,11 +738,9 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { int32_t code = streamTaskInit(pTask, pTq->pStreamMeta, &pTq->pVnode->msgCb, ver); if (code != TSDB_CODE_SUCCESS) return code; - pTask->pBackend = streamMetaGetBackendByTaskKey(pTq->pStreamMeta, (char*)pTask->id.idStr, pTask->checkpointingId, - &pTask->backendRefId); - - // taskDbUpdateChkpId(pTask->pBackend, pTask->checkpointingId); - if (pTask->pBackend == NULL) return -1; + // code = streamTaskSetDb(pTq->pStreamMeta, pTask); + // taskDbUpdateChkpId(pTask->pBackend, pTask->checkpointingId); + // if (pTask->pBackend == NULL) return -1; streamTaskOpenAllUpstreamInput(pTask); diff --git a/source/dnode/vnode/src/tq/tqStreamTaskSnap.c b/source/dnode/vnode/src/tq/tqStreamTaskSnap.c index 7b3f1aac6d..566affa50d 100644 --- a/source/dnode/vnode/src/tq/tqStreamTaskSnap.c +++ b/source/dnode/vnode/src/tq/tqStreamTaskSnap.c @@ -242,11 +242,8 @@ int32_t streamTaskSnapWrite(SStreamTaskWriter* pWriter, uint8_t* pData, uint32_t goto _err; } tDecoderClear(&decoder); - // tdbTbInsert(TTB *pTb, const void *pKey, int keyLen, const void *pVal, int valLen, TXN *pTxn) - taosWLockLatch(&pTq->pStreamMeta->lock); int64_t key[2] = {task.streamId, task.taskId}; - taosWLockLatch(&pTq->pStreamMeta->lock); if (tdbTbUpsert(pTq->pStreamMeta->pTaskDb, key, sizeof(int64_t) << 1, (uint8_t*)pData + sizeof(SSnapDataHdr), nData - sizeof(SSnapDataHdr), pTq->pStreamMeta->txn) < 0) { diff --git a/source/libs/stream/inc/streamBackendRocksdb.h b/source/libs/stream/inc/streamBackendRocksdb.h index 637fbf1017..c108acce3e 100644 --- a/source/libs/stream/inc/streamBackendRocksdb.h +++ b/source/libs/stream/inc/streamBackendRocksdb.h @@ -66,8 +66,11 @@ typedef struct { TdThreadMutex mutex; char* idstr; char* path; - int64_t refId; + int64_t refId; + void* pTask; + int64_t streamId; + int64_t taskId; int64_t chkpId; SArray* chkpSaved; SArray* chkpInUse; @@ -184,7 +187,7 @@ int32_t streamBackendTriggerChkp(void* pMeta, char* dst); int32_t streamBackendAddInUseChkp(void* arg, int64_t chkpId); int32_t streamBackendDelInUseChkp(void* arg, int64_t chkpId); -int32_t taskDbBuildSnap(void* arg); +int32_t taskDbBuildSnap(void* arg, SArray* pSnap); // int32_t streamDefaultIter_rocksdb(SStreamState* pState, const void* start, const void* end, SArray* result); #endif \ No newline at end of file diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index a5e7531f7e..801b1d6211 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -44,6 +44,7 @@ typedef struct { int64_t streamId; int64_t taskId; int64_t chkpId; + char* dbPrefixPath; } SStreamTaskSnap; extern SStreamGlobalEnv streamEnv; extern int32_t streamBackendId; diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 48e9d506d6..8aa6878b10 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1004,7 +1004,7 @@ int32_t chkpPreBuildDir(char* path, int64_t chkpId, char** chkpDir, char** chkpI return 0; } -int32_t taskDbBuildSnap(void* arg) { +int32_t taskDbBuildSnap(void* arg, SArray* pSnap) { SStreamMeta* pMeta = arg; void* pIter = taosHashIterate(pMeta->pTaskDbUnique, NULL); int32_t code = 0; @@ -1017,6 +1017,13 @@ int32_t taskDbBuildSnap(void* arg) { taskDbRemoveRef(pTaskDb); pIter = taosHashIterate(pMeta->pTaskDbUnique, pIter); + + SStreamTask* pTask = pTaskDb->pTask; + SStreamTaskSnap snap = {.streamId = pTask->id.streamId, + .taskId = pTask->id.taskId, + .chkpId = pTaskDb->chkpId, + .dbPrefixPath = taosStrdup(pTaskDb->path)}; + taosArrayPush(pSnap, &snap); } return 0; } diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 656b115bcf..3ba360a4ae 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -222,28 +222,40 @@ int32_t streamMetaMayCvtDbFormat(SStreamMeta* pMeta) { return 0; } -void* streamMetaGetBackendByTaskKey(SStreamMeta* pMeta, char* key, int64_t chkpId, int64_t* ref) { +int32_t streamTaskSetDb(SStreamMeta* pMeta, void* arg) { + SStreamTask* pTask = arg; + + char* key = (char*)pTask->id.idStr; + int64_t chkpId = pTask->checkpointingId; + taosThreadMutexLock(&pMeta->backendMutex); void** ppBackend = taosHashGet(pMeta->pTaskDbUnique, key, strlen(key)); if (ppBackend != NULL && *ppBackend != NULL) { taskDbAddRef(*ppBackend); - *ref = ((STaskDbWrapper*)*ppBackend)->refId; + + STaskDbWrapper* pBackend = *ppBackend; + + pTask->backendRefId = pBackend->refId; + pTask->pBackend = pBackend; taosThreadMutexUnlock(&pMeta->backendMutex); - return *ppBackend; + return 0; } - void* pBackend = taskDbOpen(pMeta->path, key, chkpId); + STaskDbWrapper* pBackend = taskDbOpen(pMeta->path, key, chkpId); if (pBackend == NULL) { taosThreadMutexUnlock(&pMeta->backendMutex); - return NULL; + return -1; } + int64_t tref = taosAddRef(taskDbWrapperId, pBackend); - *ref = tref; - ((STaskDbWrapper*)pBackend)->refId = tref; + pTask->backendRefId = tref; + pTask->pBackend = pBackend; + pBackend->refId = tref; + pBackend->pTask = pTask; taosHashPut(pMeta->pTaskDbUnique, key, strlen(key), &pBackend, sizeof(void*)); taosThreadMutexUnlock(&pMeta->backendMutex); - return pBackend; + return 0; } SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandFunc, int32_t vgId, int64_t stage) { int32_t code = -1; diff --git a/source/libs/stream/src/streamSnapshot.c b/source/libs/stream/src/streamSnapshot.c index 676effe778..88e47e127b 100644 --- a/source/libs/stream/src/streamSnapshot.c +++ b/source/libs/stream/src/streamSnapshot.c @@ -139,7 +139,7 @@ TdFilePtr streamOpenFile(char* path, char* name, int32_t opt) { return taosOpenFile(fullname, opt); } -int32_t streamTaskDbGetSnapInfo(void* arg, char* path) { return taskDbBuildSnap(arg); } +int32_t streamTaskDbGetSnapInfo(void* arg, char* path, SArray* pSnap) { return taskDbBuildSnap(arg, pSnap); } void snapFileDebugInfo(SBackendSnapFile2* pSnapFile) { if (qDebugFlag & DEBUG_DEBUG) { @@ -236,16 +236,13 @@ int32_t snapFileReadMeta(SBackendSnapFile2* pSnapFile) { return 0; } int32_t streamBackendSnapInitFile(char* metaPath, SStreamTaskSnap* pSnap, SBackendSnapFile2* pSnapFile) { - // SBanckendFile* pFile = taosMemoryCalloc(1, sizeof(SBanckendFile)); int32_t code = -1; - char* path = taosMemoryCalloc(1, strlen(metaPath) + 256); - char idstr[64] = {0}; - sprintf(idstr, "0x%" PRIx64 "-0x%x", pSnap->streamId, (int32_t)(pSnap->taskId)); - - sprintf(path, "%s%s%s%s%s%s%s%" PRId64 "", metaPath, TD_DIRSEP, idstr, TD_DIRSEP, "checkpoints", TD_DIRSEP, - "checkpoint", pSnap->chkpId); - if (taosIsDir(path)) { + char* path = taosMemoryCalloc(1, strlen(pSnap->dbPrefixPath) + 256); + // char idstr[64] = {0}; + sprintf(path, "%s%s%s%s%s%" PRId64 "", pSnap->dbPrefixPath, TD_DIRSEP, "checkpoints", TD_DIRSEP, "checkpoint", + pSnap->chkpId); + if (!taosIsDir(path)) { goto _ERROR; } @@ -261,7 +258,7 @@ int32_t streamBackendSnapInitFile(char* metaPath, SStreamTaskSnap* pSnap, SBacke } snapFileDebugInfo(pSnapFile); - + path = NULL; code = 0; _ERROR: @@ -294,8 +291,8 @@ void snapFileDestroy(SBackendSnapFile2* pSnap) { int32_t streamSnapHandleInit(SStreamSnapHandle* pHandle, char* path, void* pMeta) { // impl later - SArray* pSnapSet = NULL; - int32_t code = streamTaskDbGetSnapInfo(pMeta, path); + SArray* pSnapSet = taosArrayInit(4, sizeof(SStreamTaskSnap)); + int32_t code = streamTaskDbGetSnapInfo(pMeta, path, pSnapSet); if (code != 0) { return -1; } @@ -310,6 +307,11 @@ int32_t streamSnapHandleInit(SStreamSnapHandle* pHandle, char* path, void* pMeta ASSERT(code == 0); taosArrayPush(pDbSnapSet, &snapFile); } + for (int i = 0; i < taosArrayGetSize(pSnapSet); i++) { + SStreamTaskSnap* pSnap = taosArrayGet(pSnapSet, i); + taosMemoryFree(pSnap->dbPrefixPath); + } + taosArrayDestroy(pSnapSet); pHandle->pDbSnapSet = pDbSnapSet; pHandle->currIdx = 0; @@ -389,6 +391,7 @@ _NEXT: item->name, (int64_t)pSnapFile->offset, item->size, pSnapFile->currFileIdx); } } + item = taosArrayGet(pSnapFile->pFileList, pSnapFile->currFileIdx); qDebug("%s start to read file %s, current offset:%" PRId64 ", size:%" PRId64 ", file no.%d", STREAM_STATE_TRANSFER, item->name, (int64_t)pSnapFile->offset, item->size, pSnapFile->currFileIdx); diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 8da515a650..7295323ec1 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -356,6 +356,7 @@ void tFreeStreamTask(SStreamTask* pTask) { } if (pTask->pBackend) { taskDbRemoveRef(pTask->pBackend); + pTask->pBackend = NULL; } @@ -390,6 +391,10 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i streamTaskInitTokenBucket(&pTask->tokenBucket, 150, 100); taosThreadMutexInit(&pTask->lock, NULL); + + if (streamTaskSetDb(pMeta, pTask) != 0) { + return -1; + } streamTaskOpenAllUpstreamInput(pTask); return TSDB_CODE_SUCCESS; From d67fdb5e46f75bbda6699338ecaed08c09dec74d Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Tue, 17 Oct 2023 21:08:31 +0800 Subject: [PATCH 050/195] fix transfer crash --- source/dnode/vnode/src/tq/tqStreamStateSnap.c | 2 +- source/libs/stream/src/streamSnapshot.c | 22 ++++++++++++++----- 2 files changed, 17 insertions(+), 7 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqStreamStateSnap.c b/source/dnode/vnode/src/tq/tqStreamStateSnap.c index b2835f2198..b7c440dfa5 100644 --- a/source/dnode/vnode/src/tq/tqStreamStateSnap.c +++ b/source/dnode/vnode/src/tq/tqStreamStateSnap.c @@ -139,7 +139,7 @@ int32_t streamStateSnapWriterOpen(STQ* pTq, int64_t sver, int64_t ever, SStreamS pWriter->sver = sver; pWriter->ever = ever; - sprintf(tdir, "%s%s%s%s%s", pTq->path, TD_DIRSEP, VNODE_TQ_STREAM, TD_DIRSEP, "received"); + sprintf(tdir, "%s%s%s", pTq->path, TD_DIRSEP, VNODE_TQ_STREAM); taosMkDir(tdir); SStreamSnapWriter* pSnapWriter = NULL; diff --git a/source/libs/stream/src/streamSnapshot.c b/source/libs/stream/src/streamSnapshot.c index 88e47e127b..70c3031eb2 100644 --- a/source/libs/stream/src/streamSnapshot.c +++ b/source/libs/stream/src/streamSnapshot.c @@ -393,8 +393,10 @@ _NEXT: } item = taosArrayGet(pSnapFile->pFileList, pSnapFile->currFileIdx); - qDebug("%s start to read file %s, current offset:%" PRId64 ", size:%" PRId64 ", file no.%d", STREAM_STATE_TRANSFER, - item->name, (int64_t)pSnapFile->offset, item->size, pSnapFile->currFileIdx); + qDebug("%s start to read file %s, current offset:%" PRId64 ", size:%" PRId64 + ", file no.%d, total set:%d, current set idx: %d", + STREAM_STATE_TRANSFER, item->name, (int64_t)pSnapFile->offset, item->size, pSnapFile->currFileIdx, + (int)taosArrayGetSize(pHandle->pDbSnapSet), pHandle->currIdx); uint8_t* buf = taosMemoryCalloc(1, sizeof(SStreamSnapBlockHdr) + kBlockSize); int64_t nread = taosPReadFile(pHandle->fd, buf + sizeof(SStreamSnapBlockHdr), kBlockSize, pSnapFile->offset); @@ -422,9 +424,16 @@ _NEXT: if (pSnapFile->currFileIdx >= taosArrayGetSize(pSnapFile->pFileList)) { // finish - *ppData = NULL; - *size = 0; - return 0; + if (pHandle->currIdx + 1 < taosArrayGetSize(pHandle->pDbSnapSet)) { + // skip to next snap set + pHandle->currIdx += 1; + pSnapFile = taosArrayGet(pHandle->pDbSnapSet, pHandle->currIdx); + goto _NEXT; + } else { + *ppData = NULL; + *size = 0; + return 0; + } } item = taosArrayGet(pSnapFile->pFileList, pSnapFile->currFileIdx); pSnapFile->fd = streamOpenFile(pSnapFile->path, item->name, TD_FILE_READ); @@ -541,10 +550,11 @@ int32_t streamSnapWrite(SStreamSnapWriter* pWriter, uint8_t* pData, uint32_t nDa sprintf(idstr, "0x%" PRIx64 "-0x%x", snapInfo.streamId, (int32_t)(snapInfo.taskId)); char* path = taosMemoryCalloc(1, strlen(pHandle->metaPath) + 256); - sprintf(path, "%s%s%s%s%s%s%s%" PRId64 "", path, TD_DIRSEP, idstr, TD_DIRSEP, "checkpoints", TD_DIRSEP, + sprintf(path, "%s%s%s%s%s%s%s%" PRId64 "", pHandle->metaPath, TD_DIRSEP, idstr, TD_DIRSEP, "checkpoints", TD_DIRSEP, "checkpoint", snapInfo.chkpId); if (!taosIsDir(path)) { code = taosMulMkDir(path); + qInfo("%s mkdir %s", STREAM_STATE_TRANSFER, path); ASSERT(code == 0); } From e49b9b2276bb7b64b961d85015e70974daeaf2c5 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 18 Oct 2023 11:34:11 +0800 Subject: [PATCH 051/195] fix transfer crash --- source/dnode/vnode/src/tq/tqStreamStateSnap.c | 10 +++---- source/libs/stream/src/streamBackendRocksdb.c | 26 +++++++++++++++++-- source/libs/stream/src/streamSnapshot.c | 11 +++++++- 3 files changed, 39 insertions(+), 8 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqStreamStateSnap.c b/source/dnode/vnode/src/tq/tqStreamStateSnap.c index b7c440dfa5..885d12a6aa 100644 --- a/source/dnode/vnode/src/tq/tqStreamStateSnap.c +++ b/source/dnode/vnode/src/tq/tqStreamStateSnap.c @@ -167,6 +167,11 @@ int32_t streamStateSnapWriterClose(SStreamStateWriter* pWriter, int8_t rollback) return code; } + +int32_t streamStateSnapWrite(SStreamStateWriter* pWriter, uint8_t* pData, uint32_t nData) { + tqDebug("vgId:%d, vnode %s snapshot write data", TD_VID(pWriter->pTq->pVnode), STREAM_STATE_TRANSFER); + return streamSnapWrite(pWriter->pWriterImpl, pData + sizeof(SSnapDataHdr), nData - sizeof(SSnapDataHdr)); +} int32_t streamStateRebuildFromSnap(SStreamStateWriter* pWriter, int64_t chkpId) { tqDebug("vgId:%d, vnode %s start to rebuild stream-state", TD_VID(pWriter->pTq->pVnode), STREAM_STATE_TRANSFER); int32_t code = streamStateLoadTasks(pWriter); @@ -178,8 +183,3 @@ int32_t streamStateRebuildFromSnap(SStreamStateWriter* pWriter, int64_t chkpId) int32_t streamStateLoadTasks(SStreamStateWriter* pWriter) { return streamMetaReloadAllTasks(pWriter->pTq->pStreamMeta); } - -int32_t streamStateSnapWrite(SStreamStateWriter* pWriter, uint8_t* pData, uint32_t nData) { - tqDebug("vgId:%d, vnode %s snapshot write data", TD_VID(pWriter->pTq->pVnode), STREAM_STATE_TRANSFER); - return streamSnapWrite(pWriter->pWriterImpl, pData + sizeof(SSnapDataHdr), nData - sizeof(SSnapDataHdr)); -} diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 8aa6878b10..d1ddd5c81a 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -492,8 +492,30 @@ int32_t rebuildDirFromChkp2(const char* path, char* key, int64_t chkpId, char** } } else { - qError("failed to start stream backend at %s, reason: %s, restart from default defaultPath dir:%s", chkpPath, - tstrerror(TAOS_SYSTEM_ERROR(errno)), defaultPath); + qInfo("failed to start stream backend at %s, reason: %s, restart from default defaultPath dir:%s", chkpPath, + tstrerror(TAOS_SYSTEM_ERROR(errno)), defaultPath); + taosMkDir(defaultPath); + } + taosMemoryFree(chkpPath); + } else { + char* chkpPath = taosMemoryCalloc(1, strlen(path) + 256); + sprintf(chkpPath, "%s%s%s%s%s%" PRId64 "", prefixPath, TD_DIRSEP, "checkpoints", TD_DIRSEP, "checkpoint", + (int64_t)-1); + qInfo("no chkp id specified, try to restart from received chkp id -1, dir: %s", chkpPath); + if (taosIsDir(chkpPath) && isValidCheckpoint(chkpPath)) { + if (taosIsDir(defaultPath)) { + taosRemoveDir(defaultPath); + } + taosMkDir(defaultPath); + code = copyFiles(chkpPath, defaultPath); + if (code != 0) { + qError("failed to restart stream backend from %s, reason: %s", chkpPath, tstrerror(TAOS_SYSTEM_ERROR(errno))); + } else { + qInfo("start to restart stream backend at checkpoint path: %s", chkpPath); + } + } else { + qInfo("failed to start stream backend at %s, reason: %s, restart from default defaultPath dir:%s", chkpPath, + tstrerror(TAOS_SYSTEM_ERROR(errno)), defaultPath); taosMkDir(defaultPath); } taosMemoryFree(chkpPath); diff --git a/source/libs/stream/src/streamSnapshot.c b/source/libs/stream/src/streamSnapshot.c index 70c3031eb2..3993fe09a1 100644 --- a/source/libs/stream/src/streamSnapshot.c +++ b/source/libs/stream/src/streamSnapshot.c @@ -400,7 +400,13 @@ _NEXT: uint8_t* buf = taosMemoryCalloc(1, sizeof(SStreamSnapBlockHdr) + kBlockSize); int64_t nread = taosPReadFile(pHandle->fd, buf + sizeof(SStreamSnapBlockHdr), kBlockSize, pSnapFile->offset); - if (nread == -1) { + + qInfo("%s read impl %d, file name: %s", STREAM_STATE_TRANSFER, (int)nread, item->name); + if (nread == 0) { + code = TAOS_SYSTEM_ERROR(errno); + qError("%s snap failed to read snap, file name:%s, type:%d,reason:%s", STREAM_STATE_TRANSFER, item->name, + item->type, tstrerror(code)); + } else if (nread == -1) { code = TAOS_SYSTEM_ERROR(terrno); qError("%s snap failed to read snap, file name:%s, type:%d,reason:%s", STREAM_STATE_TRANSFER, item->name, item->type, tstrerror(code)); @@ -509,6 +515,8 @@ int32_t streamSnapWriteImpl(SStreamSnapWriter* pWriter, uint8_t* pData, uint32_t code = TAOS_SYSTEM_ERROR(terrno); qError("%s failed to write snap, file name:%s, reason:%s", STREAM_STATE_TRANSFER, pHdr->name, tstrerror(code)); return code; + } else { + qInfo("succ to write data %s", pItem->name); } pSnapFile->offset += bytes; } else { @@ -531,6 +539,7 @@ int32_t streamSnapWriteImpl(SStreamSnapWriter* pWriter, uint8_t* pData, uint32_t } taosPWriteFile(pSnapFile->fd, pHdr->data, pHdr->size, pSnapFile->offset); + qInfo("succ to write data %s", pItem->name); pSnapFile->offset += pHdr->size; } code = 0; From 1b755dd4e79c5a1355f667ee7fba1cdc24671b0d Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 18 Oct 2023 14:22:33 +0800 Subject: [PATCH 052/195] fix transfer crash --- source/libs/stream/src/streamSnapshot.c | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/source/libs/stream/src/streamSnapshot.c b/source/libs/stream/src/streamSnapshot.c index 3993fe09a1..f192474a65 100644 --- a/source/libs/stream/src/streamSnapshot.c +++ b/source/libs/stream/src/streamSnapshot.c @@ -136,6 +136,7 @@ TdFilePtr streamOpenFile(char* path, char* name, int32_t opt) { char fullname[256] = {0}; STREAM_ROCKSDB_BUILD_FULLNAME(path, name, fullname); + qInfo("stream open file %s", fullname); return taosOpenFile(fullname, opt); } @@ -399,14 +400,9 @@ _NEXT: (int)taosArrayGetSize(pHandle->pDbSnapSet), pHandle->currIdx); uint8_t* buf = taosMemoryCalloc(1, sizeof(SStreamSnapBlockHdr) + kBlockSize); - int64_t nread = taosPReadFile(pHandle->fd, buf + sizeof(SStreamSnapBlockHdr), kBlockSize, pSnapFile->offset); + int64_t nread = taosPReadFile(pSnapFile->fd, buf + sizeof(SStreamSnapBlockHdr), kBlockSize, pSnapFile->offset); - qInfo("%s read impl %d, file name: %s", STREAM_STATE_TRANSFER, (int)nread, item->name); - if (nread == 0) { - code = TAOS_SYSTEM_ERROR(errno); - qError("%s snap failed to read snap, file name:%s, type:%d,reason:%s", STREAM_STATE_TRANSFER, item->name, - item->type, tstrerror(code)); - } else if (nread == -1) { + if (nread == -1) { code = TAOS_SYSTEM_ERROR(terrno); qError("%s snap failed to read snap, file name:%s, type:%d,reason:%s", STREAM_STATE_TRANSFER, item->name, item->type, tstrerror(code)); From d6840ae07fda355fa764fa166378bf0de9844b6d Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 18 Oct 2023 14:22:52 +0800 Subject: [PATCH 053/195] fix transfer crash --- source/libs/stream/src/streamSnapshot.c | 1 - 1 file changed, 1 deletion(-) diff --git a/source/libs/stream/src/streamSnapshot.c b/source/libs/stream/src/streamSnapshot.c index f192474a65..5a6fdc314d 100644 --- a/source/libs/stream/src/streamSnapshot.c +++ b/source/libs/stream/src/streamSnapshot.c @@ -136,7 +136,6 @@ TdFilePtr streamOpenFile(char* path, char* name, int32_t opt) { char fullname[256] = {0}; STREAM_ROCKSDB_BUILD_FULLNAME(path, name, fullname); - qInfo("stream open file %s", fullname); return taosOpenFile(fullname, opt); } From c6ba2ca20568ec5f943660bcb7c271b091f98645 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 18 Oct 2023 14:49:12 +0800 Subject: [PATCH 054/195] fix transfer crash --- source/libs/stream/src/streamBackendRocksdb.c | 3 +++ 1 file changed, 3 insertions(+) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index d1ddd5c81a..6b0275f44e 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -433,6 +433,7 @@ int32_t rebuildDirFromCheckpoint(const char* path, int64_t chkpId, char** dst) { } taosMkDir(state); code = copyFiles(chkp, state); + qInfo("copy snap file from %s to %s", chkp, state); if (code != 0) { qError("failed to restart stream backend from %s, reason: %s", chkp, tstrerror(TAOS_SYSTEM_ERROR(errno))); } else { @@ -508,6 +509,7 @@ int32_t rebuildDirFromChkp2(const char* path, char* key, int64_t chkpId, char** } taosMkDir(defaultPath); code = copyFiles(chkpPath, defaultPath); + qInfo("copy snap file from %s to %s", chkpPath, defaultPath); if (code != 0) { qError("failed to restart stream backend from %s, reason: %s", chkpPath, tstrerror(TAOS_SYSTEM_ERROR(errno))); } else { @@ -1805,6 +1807,7 @@ STaskDbWrapper* taskDbOpen(char* path, char* key, int64_t chkpId) { cfNames = rocksdb_list_column_families(pTaskDb->dbOpt, dbPath, &nCf, &err); if (nCf == 0) { + qInfo("newly create db, need to restart"); // pre create db pTaskDb->db = rocksdb_open(pTaskDb->pCfOpts[0], dbPath, &err); rocksdb_close(pTaskDb->db); From 67dc52d2d624107e24d97e6657dc8e050124ea2e Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 19 Oct 2023 15:19:52 +0800 Subject: [PATCH 055/195] refactor stream checkpoint --- source/dnode/mnode/impl/src/mndStream.c | 225 ++++++++++-------------- 1 file changed, 97 insertions(+), 128 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 049b4e737a..fab3a0f414 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -886,7 +886,7 @@ static int32_t mndProcessCreateStreamReq(SRpcMsg *pReq) { SName name = {0}; tNameFromString(&name, createStreamReq.name, T_NAME_ACCT | T_NAME_DB); - //reuse this function for stream + // reuse this function for stream auditRecord(pReq, pMnode->clusterId, "createStream", name.dbname, "", detail); @@ -961,107 +961,102 @@ static int32_t mndBuildStreamCheckpointSourceReq2(void **pBuf, int32_t *pLen, in return 0; } -// static int32_t mndProcessStreamCheckpointTrans(SMnode *pMnode, SStreamObj *pStream, int64_t checkpointId) { -// int64_t timestampMs = taosGetTimestampMs(); -// if (timestampMs - pStream->checkpointFreq < tsStreamCheckpointTickInterval * 1000) { -// return -1; -// } +static int32_t mndProcessStreamCheckpointTrans(SMnode *pMnode, SStreamObj *pStream, int64_t checkpointId) { + int32_t code = -1; + int64_t timestampMs = taosGetTimestampMs(); + if (timestampMs - pStream->checkpointFreq < tsStreamCheckpointTickInterval * 1000) { + return -1; + } -// STrans *pTrans = mndTransCreate(pMnode, TRN_POLICY_RETRY, TRN_CONFLICT_DB_INSIDE, NULL, "stream-checkpoint"); -// if (pTrans == NULL) return -1; -// mndTransSetDbName(pTrans, pStream->sourceDb, pStream->targetDb); -// if (mndTrancCheckConflict(pMnode, pTrans) != 0) { -// mError("failed to checkpoint of stream name%s, checkpointId: %" PRId64 ", reason:%s", pStream->name, -// checkpointId, -// tstrerror(TSDB_CODE_MND_TRANS_CONFLICT)); -// mndTransDrop(pTrans); -// return -1; -// } -// mDebug("start to trigger checkpoint for stream:%s, checkpoint: %" PRId64 "", pStream->name, checkpointId); -// atomic_store_64(&pStream->currentTick, 1); -// taosWLockLatch(&pStream->lock); -// // 1. redo action: broadcast checkpoint source msg for all source vg -// int32_t totLevel = taosArrayGetSize(pStream->tasks); -// for (int32_t i = 0; i < totLevel; i++) { -// SArray *pLevel = taosArrayGetP(pStream->tasks, i); -// SStreamTask *pTask = taosArrayGetP(pLevel, 0); -// if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { -// int32_t sz = taosArrayGetSize(pLevel); -// for (int32_t j = 0; j < sz; j++) { -// SStreamTask *pTask = taosArrayGetP(pLevel, j); -// /*A(pTask->info.nodeId > 0);*/ -// SVgObj *pVgObj = mndAcquireVgroup(pMnode, pTask->info.nodeId); -// if (pVgObj == NULL) { -// taosWUnLockLatch(&pStream->lock); -// mndTransDrop(pTrans); -// return -1; -// } + STrans *pTrans = mndTransCreate(pMnode, TRN_POLICY_RETRY, TRN_CONFLICT_DB_INSIDE, NULL, "stream-checkpoint"); + if (pTrans == NULL) return -1; -// void *buf; -// int32_t tlen; -// if (mndBuildStreamCheckpointSourceReq2(&buf, &tlen, pTask->info.nodeId, checkpointId, pTask->id.streamId, -// pTask->id.taskId) < 0) { -// mndReleaseVgroup(pMnode, pVgObj); -// taosWUnLockLatch(&pStream->lock); -// mndTransDrop(pTrans); -// return -1; -// } + mndTransSetDbName(pTrans, pStream->sourceDb, pStream->targetDb); + if (mndTrancCheckConflict(pMnode, pTrans) != 0) { + mError("failed to checkpoint of stream name%s, checkpointId: %" PRId64 ", reason:%s", pStream->name, checkpointId, + tstrerror(TSDB_CODE_MND_TRANS_CONFLICT)); + goto _ERR; + } -// STransAction action = {0}; -// action.epSet = mndGetVgroupEpset(pMnode, pVgObj); -// action.pCont = buf; -// action.contLen = tlen; -// action.msgType = TDMT_VND_STREAM_CHECK_POINT_SOURCE; + mDebug("start to trigger checkpoint for stream:%s, checkpoint: %" PRId64 "", pStream->name, checkpointId); + taosWLockLatch(&pStream->lock); + pStream->currentTick = 1; + // 1. redo action: broadcast checkpoint source msg for all source vg -// mndReleaseVgroup(pMnode, pVgObj); + int32_t totLevel = taosArrayGetSize(pStream->tasks); + for (int32_t i = 0; i < totLevel; i++) { + SArray *pLevel = taosArrayGetP(pStream->tasks, i); + SStreamTask *pTask = taosArrayGetP(pLevel, 0); + if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { + int32_t sz = taosArrayGetSize(pLevel); + for (int32_t j = 0; j < sz; j++) { + SStreamTask *pTask = taosArrayGetP(pLevel, j); + /*A(pTask->info.nodeId > 0);*/ + SVgObj *pVgObj = mndAcquireVgroup(pMnode, pTask->info.nodeId); + if (pVgObj == NULL) { + taosWUnLockLatch(&pStream->lock); + goto _ERR; + } -// if (mndTransAppendRedoAction(pTrans, &action) != 0) { -// taosMemoryFree(buf); -// taosWUnLockLatch(&pStream->lock); -// mndReleaseStream(pMnode, pStream); -// mndTransDrop(pTrans); -// return -1; -// } -// } -// } -// } -// // 2. reset tick -// pStream->checkpointFreq = checkpointId; -// pStream->checkpointId = checkpointId; -// pStream->checkpointFreq = taosGetTimestampMs(); -// atomic_store_64(&pStream->currentTick, 0); -// // 3. commit log: stream checkpoint info -// pStream->version = pStream->version + 1; -// taosWUnLockLatch(&pStream->lock); + void *buf; + int32_t tlen; + if (mndBuildStreamCheckpointSourceReq2(&buf, &tlen, pTask->info.nodeId, checkpointId, pTask->id.streamId, + pTask->id.taskId) < 0) { + mndReleaseVgroup(pMnode, pVgObj); + taosWUnLockLatch(&pStream->lock); + goto _ERR; + } -// // // code condtion + STransAction action = {0}; + action.epSet = mndGetVgroupEpset(pMnode, pVgObj); + action.pCont = buf; + action.contLen = tlen; + action.msgType = TDMT_VND_STREAM_CHECK_POINT_SOURCE; -// SSdbRaw *pCommitRaw = mndStreamActionEncode(pStream); -// if (pCommitRaw == NULL) { -// mError("failed to prepare trans rebalance since %s", terrstr()); -// goto _ERR; -// } -// if (mndTransAppendCommitlog(pTrans, pCommitRaw) != 0) { -// sdbFreeRaw(pCommitRaw); -// mError("failed to prepare trans rebalance since %s", terrstr()); -// goto _ERR; -// } -// if (sdbSetRawStatus(pCommitRaw, SDB_STATUS_READY) != 0) { -// sdbFreeRaw(pCommitRaw); -// mError("failed to prepare trans rebalance since %s", terrstr()); -// goto _ERR; -// } + mndReleaseVgroup(pMnode, pVgObj); -// if (mndTransPrepare(pMnode, pTrans) != 0) { -// mError("failed to prepare trans rebalance since %s", terrstr()); -// goto _ERR; -// } -// mndTransDrop(pTrans); -// return 0; -// _ERR: -// mndTransDrop(pTrans); -// return -1; -// } + if (mndTransAppendRedoAction(pTrans, &action) != 0) { + taosMemoryFree(buf); + taosWUnLockLatch(&pStream->lock); + goto _ERR; + } + } + } + } + // 2. reset tick + pStream->checkpointFreq = checkpointId; + pStream->checkpointId = checkpointId; + pStream->checkpointFreq = taosGetTimestampMs(); + pStream->currentTick = 0; + // 3. commit log: stream checkpoint info + pStream->version = pStream->version + 1; + taosWUnLockLatch(&pStream->lock); + + SSdbRaw *pCommitRaw = mndStreamActionEncode(pStream); + if (pCommitRaw == NULL) { + mError("failed to prepare trans rebalance since %s", terrstr()); + goto _ERR; + } + if (mndTransAppendCommitlog(pTrans, pCommitRaw) != 0) { + sdbFreeRaw(pCommitRaw); + mError("failed to prepare trans rebalance since %s", terrstr()); + goto _ERR; + } + if (sdbSetRawStatus(pCommitRaw, SDB_STATUS_READY) != 0) { + sdbFreeRaw(pCommitRaw); + mError("failed to prepare trans rebalance since %s", terrstr()); + goto _ERR; + } + + if (mndTransPrepare(pMnode, pTrans) != 0) { + mError("failed to prepare trans rebalance since %s", terrstr()); + goto _ERR; + } + code = 0; +_ERR: + mndTransDrop(pTrans); + return code; +} static int32_t mndAddStreamCheckpointToTrans(STrans *pTrans, SStreamObj *pStream, SMnode *pMnode, int64_t checkpointId) { @@ -1111,8 +1106,8 @@ static int32_t mndAddStreamCheckpointToTrans(STrans *pTrans, SStreamObj *pStream pStream->checkpointId = checkpointId; pStream->checkpointFreq = taosGetTimestampMs(); - atomic_store_64(&pStream->currentTick, 0); - // 3. commit log: stream checkpoint info + pStream->currentTick = 0; + // 3. commit log: stream checkpoint info pStream->version = pStream->version + 1; taosWUnLockLatch(&pStream->lock); @@ -1212,43 +1207,17 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { SMStreamDoCheckpointMsg *pMsg = (SMStreamDoCheckpointMsg *)pReq->pCont; int64_t checkpointId = pMsg->checkpointId; - - STrans *pTrans = mndTransCreate(pMnode, TRN_POLICY_RETRY, TRN_CONFLICT_DB, NULL, MND_STREAM_CHECKPOINT_NAME); - if (pTrans == NULL) { - mError("failed to trigger checkpoint, reason: %s", tstrerror(TSDB_CODE_OUT_OF_MEMORY)); - return -1; - } - mDebug("start to trigger checkpoint, checkpointId: %" PRId64 "", checkpointId); - - const char *pDb = mndGetStreamDB(pMnode); - mndTransSetDbName(pTrans, pDb, "checkpoint"); - taosMemoryFree((void *)pDb); - - if (mndTransCheckConflict(pMnode, pTrans) != 0) { - mError("failed to trigger checkpoint, checkpointId: %" PRId64 ", reason:%s", checkpointId, - tstrerror(TSDB_CODE_MND_TRANS_CONFLICT)); - mndTransDrop(pTrans); - return -1; - } - while (1) { pIter = sdbFetch(pSdb, SDB_STREAM, pIter, (void **)&pStream); if (pIter == NULL) break; - code = mndAddStreamCheckpointToTrans(pTrans, pStream, pMnode, checkpointId); + code = mndProcessStreamCheckpointTrans(pMnode, pStream, checkpointId); sdbRelease(pSdb, pStream); if (code == -1) { break; } } - if (code == 0) { - if (mndTransPrepare(pMnode, pTrans) != 0) { - mError("failed to prepre trans rebalance since %s", terrstr()); - } - } - - mndTransDrop(pTrans); return code; } @@ -1324,7 +1293,7 @@ static int32_t mndProcessDropStreamReq(SRpcMsg *pReq) { SName name = {0}; tNameFromString(&name, dropReq.name, T_NAME_ACCT | T_NAME_DB); - //reuse this function for stream + // reuse this function for stream auditRecord(pReq, pMnode->clusterId, "dropStream", name.dbname, "", detail); @@ -2018,7 +1987,7 @@ static int32_t createStreamUpdateTrans(SMnode *pMnode, SStreamObj *pStream, SVgr static bool isNodeEpsetChanged(const SEpSet *pPrevEpset, const SEpSet *pCurrent) { const SEp *pEp = GET_ACTIVE_EP(pPrevEpset); - const SEp* p = GET_ACTIVE_EP(pCurrent); + const SEp *p = GET_ACTIVE_EP(pCurrent); if (pEp->port == p->port && strncmp(pEp->fqdn, p->fqdn, TSDB_FQDN_LEN) == 0) { return false; @@ -2287,7 +2256,7 @@ static void keepStreamTasksInBuf(SStreamObj *pStream, SStreamVnodeRevertIndex *p // todo: this process should be executed by the write queue worker of the mnode int32_t mndProcessStreamHb(SRpcMsg *pReq) { - SMnode *pMnode = pReq->info.node; + SMnode *pMnode = pReq->info.node; SStreamHbMsg req = {0}; int32_t code = TSDB_CODE_SUCCESS; @@ -2312,8 +2281,8 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { for (int32_t i = 0; i < req.numOfTasks; ++i) { STaskStatusEntry *p = taosArrayGet(req.pTaskStatus, i); - int64_t k[2] = {p->streamId, p->taskId}; - int32_t *index = taosHashGet(execNodeList.pTaskMap, &k, sizeof(k)); + int64_t k[2] = {p->streamId, p->taskId}; + int32_t *index = taosHashGet(execNodeList.pTaskMap, &k, sizeof(k)); if (index == NULL) { continue; } From 1c3c58062d6e8df7372117be13b466a4f9de267a Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Fri, 20 Oct 2023 15:40:47 +0800 Subject: [PATCH 056/195] Merge branch 'enh/new3.0' into enh/refactorBackend --- source/libs/stream/src/streamBackendRocksdb.c | 1 + 1 file changed, 1 insertion(+) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 79976a66c4..104289aed3 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1064,6 +1064,7 @@ int32_t taskDbBuildSnap(void* arg, SArray* pSnap) { .dbPrefixPath = taosStrdup(pTaskDb->path)}; taosArrayPush(pSnap, &snap); } + return code; } int32_t streamBackendAddInUseChkp(void* arg, int64_t chkpId) { // if (arg == NULL) return 0; From 9d210ec95716f6964db60bb138e310d01c12ca7c Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Mon, 23 Oct 2023 14:06:25 +0800 Subject: [PATCH 057/195] refactor stream backend --- source/dnode/mnode/impl/src/mndStream.c | 91 ++++++++++++------------- 1 file changed, 45 insertions(+), 46 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 3929e897e6..283d3fde38 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1119,36 +1119,30 @@ static const char *mndGetStreamDB(SMnode *pMnode) { return p; } -static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { - SMnode *pMnode = pReq->info.node; - SSdb *pSdb = pMnode->pSdb; - void *pIter = NULL; - SStreamObj *pStream = NULL; - int32_t code = 0; +static int32_t mndCheckNodeStatus(SMnode *pMnode) { + bool ready = true; + // check if the node update happens or not + int64_t ts = taosGetTimestampSec(); - { // check if the node update happens or not - int64_t ts = taosGetTimestampSec(); - - if (execNodeList.pNodeEntryList == NULL || (taosArrayGetSize(execNodeList.pNodeEntryList) == 0)) { - if (execNodeList.pNodeEntryList != NULL) { - execNodeList.pNodeEntryList = taosArrayDestroy(execNodeList.pNodeEntryList); - } - - execNodeList.pNodeEntryList = extractNodeListFromStream(pMnode); + if (execNodeList.pNodeEntryList == NULL || (taosArrayGetSize(execNodeList.pNodeEntryList) == 0)) { + if (execNodeList.pNodeEntryList != NULL) { + execNodeList.pNodeEntryList = taosArrayDestroy(execNodeList.pNodeEntryList); } - if (taosArrayGetSize(execNodeList.pNodeEntryList) == 0) { - mDebug("stream task node change checking done, no vgroups exist, do nothing"); - execNodeList.ts = ts; - return 0; - } + execNodeList.pNodeEntryList = extractNodeListFromStream(pMnode); + } - for (int32_t i = 0; i < taosArrayGetSize(execNodeList.pNodeEntryList); ++i) { - SNodeEntry *pNodeEntry = taosArrayGet(execNodeList.pNodeEntryList, i); - if (pNodeEntry->stageUpdated) { - mDebug("stream task not ready due to node update detected, checkpoint not issued"); - return 0; - } + if (taosArrayGetSize(execNodeList.pNodeEntryList) == 0) { + mDebug("stream task node change checking done, no vgroups exist, do nothing"); + execNodeList.ts = ts; + return -1; + } + + for (int32_t i = 0; i < taosArrayGetSize(execNodeList.pNodeEntryList); ++i) { + SNodeEntry *pNodeEntry = taosArrayGet(execNodeList.pNodeEntryList, i); + if (pNodeEntry->stageUpdated) { + mDebug("stream task not ready due to node update detected, checkpoint not issued"); + return -1; } SArray *pNodeSnapshot = mndTakeVgroupSnapshot(pMnode); @@ -1161,34 +1155,39 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { if (nodeUpdated) { mDebug("stream task not ready due to node update, checkpoint not issued"); - return 0; + return -1; } } - { // check if all tasks are in TASK_STATUS__NORMAL status - bool ready = true; + // check if all tasks are in TASK_STATUS__NORMAL status - taosThreadMutexLock(&execNodeList.lock); - for (int32_t i = 0; i < taosArrayGetSize(execNodeList.pTaskList); ++i) { - STaskId *p = taosArrayGet(execNodeList.pTaskList, i); - STaskStatusEntry *pEntry = taosHashGet(execNodeList.pTaskMap, p, sizeof(*p)); - if (pEntry == NULL) { - continue; - } - - if (pEntry->status != TASK_STATUS__NORMAL) { - mDebug("s-task:0x%" PRIx64 "-0x%x (nodeId:%d) status:%s not ready, checkpoint msg not issued", - pEntry->id.streamId, (int32_t)pEntry->id.taskId, 0, streamGetTaskStatusStr(pEntry->status)); - ready = false; - break; - } + taosThreadMutexLock(&execNodeList.lock); + for (int32_t i = 0; i < taosArrayGetSize(execNodeList.pTaskList); ++i) { + STaskId *p = taosArrayGet(execNodeList.pTaskList, i); + STaskStatusEntry *pEntry = taosHashGet(execNodeList.pTaskMap, p, sizeof(*p)); + if (pEntry == NULL) { + continue; } - taosThreadMutexUnlock(&execNodeList.lock); - if (!ready) { - return 0; + if (pEntry->status != TASK_STATUS__NORMAL) { + mDebug("s-task:0x%" PRIx64 "-0x%x (nodeId:%d) status:%s not ready, checkpoint msg not issued", + pEntry->id.streamId, (int32_t)pEntry->id.taskId, 0, streamGetTaskStatusStr(pEntry->status)); + ready = false; + break; } } + taosThreadMutexUnlock(&execNodeList.lock); + return ready == true ? 0 : -1; +} +static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { + SMnode *pMnode = pReq->info.node; + SSdb *pSdb = pMnode->pSdb; + void *pIter = NULL; + SStreamObj *pStream = NULL; + int32_t code = 0; + if ((code = mndCheckNodeStatus(pMnode)) != 0) { + return code; + } SMStreamDoCheckpointMsg *pMsg = (SMStreamDoCheckpointMsg *)pReq->pCont; int64_t checkpointId = pMsg->checkpointId; From ca1ffd584d2f5cf473754b65bb64af49e9d119d9 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Mon, 23 Oct 2023 16:45:17 +0800 Subject: [PATCH 058/195] refactor stream backend --- source/dnode/mnode/impl/src/mndStream.c | 50 ++++++++++++------------- 1 file changed, 25 insertions(+), 25 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 283d3fde38..87a85c4086 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -2450,32 +2450,34 @@ int32_t createStreamResetStatusTrans(SMnode *pMnode, SStreamObj *pStream) { return TSDB_CODE_ACTION_IN_PROGRESS; } +int32_t mndFindTransByName(SMnode *pMnode, char *key, int32_t *transId) { + SSdb *pSdb = pMnode->pSdb; + STrans *pTrans = NULL; + void *pIter = NULL; + bool found = false; + while (1) { + pIter = sdbFetch(pSdb, SDB_TRANS, pIter, (void **)&pTrans); + if (pIter == NULL) { + break; + } + + if (strncmp(pTrans->opername, key, tListLen(pTrans->opername) - 1) == 0) { + *transId = pTrans->id; + found = true; + sdbRelease(pSdb, pTrans); + sdbCancelFetch(pSdb, pIter); + break; + } + + sdbRelease(pSdb, pTrans); + } + return found ? 0 : -1; +} int32_t mndResetFromCheckpoint(SMnode *pMnode) { // find the checkpoint trans id int32_t transId = 0; - - { - SSdb *pSdb = pMnode->pSdb; - STrans *pTrans = NULL; - void *pIter = NULL; - while (1) { - pIter = sdbFetch(pSdb, SDB_TRANS, pIter, (void **)&pTrans); - if (pIter == NULL) { - break; - } - - if (strncmp(pTrans->opername, MND_STREAM_CHECKPOINT_NAME, tListLen(pTrans->opername) - 1) == 0) { - transId = pTrans->id; - sdbRelease(pSdb, pTrans); - sdbCancelFetch(pSdb, pIter); - break; - } - - sdbRelease(pSdb, pTrans); - } - } - - if (transId == 0) { + int32_t code = mndFindTransByName(pMnode, MND_STREAM_CHECKPOINT_NAME, &transId); + if (code == -1) { mError("failed to find the checkpoint trans, reset not executed"); return TSDB_CODE_SUCCESS; } @@ -2492,7 +2494,6 @@ int32_t mndResetFromCheckpoint(SMnode *pMnode) { if (pIter == NULL) { break; } - mDebug("stream:%s (0x%" PRIx64 ") reset checkpoint procedure, create reset trans", pStream->name, pStream->uid); int32_t code = createStreamResetStatusTrans(pMnode, pStream); if (code != TSDB_CODE_SUCCESS) { @@ -2500,7 +2501,6 @@ int32_t mndResetFromCheckpoint(SMnode *pMnode) { return code; } } - return 0; } From 986a502019f1073a425e709db0a7d1ec17cf753b Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Mon, 23 Oct 2023 18:22:58 +0800 Subject: [PATCH 059/195] refactor stream backend --- source/dnode/mnode/impl/src/mndStream.c | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 87a85c4086..b1bcc401f8 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -2411,6 +2411,7 @@ int32_t createStreamResetStatusTrans(SMnode *pMnode, SStreamObj *pStream) { terrno = TSDB_CODE_OUT_OF_MEMORY; mError("failed to malloc in reset stream, size:%" PRIzu ", code:%s", sizeof(SVResetStreamTaskReq), tstrerror(TSDB_CODE_OUT_OF_MEMORY)); + taosWUnLockLatch(&pStream->lock); return terrno; } @@ -2495,7 +2496,7 @@ int32_t mndResetFromCheckpoint(SMnode *pMnode) { break; } mDebug("stream:%s (0x%" PRIx64 ") reset checkpoint procedure, create reset trans", pStream->name, pStream->uid); - int32_t code = createStreamResetStatusTrans(pMnode, pStream); + code = createStreamResetStatusTrans(pMnode, pStream); if (code != TSDB_CODE_SUCCESS) { sdbCancelFetch(pSdb, pIter); return code; From d46cf878eb48241eacb77a2c073f26a04007ed2f Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Tue, 24 Oct 2023 11:53:52 +0800 Subject: [PATCH 060/195] refactor stream backend --- source/dnode/mnode/impl/src/mndStream.c | 13 +++++++----- source/libs/stream/src/streamBackendRocksdb.c | 9 ++++++--- source/libs/stream/src/streamState.c | 20 ++++++++++++++++--- 3 files changed, 31 insertions(+), 11 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index b1bcc401f8..eb813c3d86 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1120,10 +1120,11 @@ static const char *mndGetStreamDB(SMnode *pMnode) { } static int32_t mndCheckNodeStatus(SMnode *pMnode) { - bool ready = true; + bool ready = false; // check if the node update happens or not int64_t ts = taosGetTimestampSec(); + taosThreadMutexLock(&execNodeList.lock); if (execNodeList.pNodeEntryList == NULL || (taosArrayGetSize(execNodeList.pNodeEntryList) == 0)) { if (execNodeList.pNodeEntryList != NULL) { execNodeList.pNodeEntryList = taosArrayDestroy(execNodeList.pNodeEntryList); @@ -1135,14 +1136,14 @@ static int32_t mndCheckNodeStatus(SMnode *pMnode) { if (taosArrayGetSize(execNodeList.pNodeEntryList) == 0) { mDebug("stream task node change checking done, no vgroups exist, do nothing"); execNodeList.ts = ts; - return -1; + goto _EXIT; } for (int32_t i = 0; i < taosArrayGetSize(execNodeList.pNodeEntryList); ++i) { SNodeEntry *pNodeEntry = taosArrayGet(execNodeList.pNodeEntryList, i); if (pNodeEntry->stageUpdated) { mDebug("stream task not ready due to node update detected, checkpoint not issued"); - return -1; + goto _EXIT; } SArray *pNodeSnapshot = mndTakeVgroupSnapshot(pMnode); @@ -1155,13 +1156,12 @@ static int32_t mndCheckNodeStatus(SMnode *pMnode) { if (nodeUpdated) { mDebug("stream task not ready due to node update, checkpoint not issued"); - return -1; + goto _EXIT; } } // check if all tasks are in TASK_STATUS__NORMAL status - taosThreadMutexLock(&execNodeList.lock); for (int32_t i = 0; i < taosArrayGetSize(execNodeList.pTaskList); ++i) { STaskId *p = taosArrayGet(execNodeList.pTaskList, i); STaskStatusEntry *pEntry = taosHashGet(execNodeList.pTaskMap, p, sizeof(*p)); @@ -1176,6 +1176,9 @@ static int32_t mndCheckNodeStatus(SMnode *pMnode) { break; } } + ready = true; +_EXIT: + taosThreadMutexUnlock(&execNodeList.lock); return ready == true ? 0 : -1; } diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 104289aed3..4add9d2912 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -2637,6 +2637,9 @@ int32_t streamStateFuncDel_rocksdb(SStreamState* pState, const STupleKey* key) { int32_t streamStateSessionPut_rocksdb(SStreamState* pState, const SSessionKey* key, const void* value, int32_t vLen) { int code = 0; SStateSessionKey sKey = {.key = *key, .opNum = pState->number}; + if (value == NULL || vLen == 0) { + stError("streamStateSessionPut_rocksdb val: %p, len: %d", value, vLen); + } STREAM_STATE_PUT_ROCKSDB(pState, "sess", &sKey, value, vLen); return code; } @@ -2685,11 +2688,11 @@ SStreamStateCur* streamStateSessionSeekToLast_rocksdb(SStreamState* pState) { if (code != 0) { return NULL; } + STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; - SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; - SStreamStateCur* pCur = createStreamStateCursor(); + SStreamStateCur* pCur = createStreamStateCursor(); pCur->number = pState->number; - pCur->db = wrapper->rocksdb; + pCur->db = wrapper->db; pCur->iter = streamStateIterCreate(pState, "sess", (rocksdb_snapshot_t**)&pCur->snapshot, (rocksdb_readoptions_t**)&pCur->readOpt); diff --git a/source/libs/stream/src/streamState.c b/source/libs/stream/src/streamState.c index 8c44d3eeb2..218d906504 100644 --- a/source/libs/stream/src/streamState.c +++ b/source/libs/stream/src/streamState.c @@ -706,9 +706,23 @@ void streamStateFreeVal(void* val) { int32_t streamStateSessionPut(SStreamState* pState, const SSessionKey* key, void* value, int32_t vLen) { #ifdef USE_ROCKSDB - qDebug("===stream===save skey:%" PRId64 ", ekey:%" PRId64 ", groupId:%" PRIu64, key->win.skey, key->win.ekey, - key->groupId); - return streamStateSessionPut_rocksdb(pState, key, value, vLen); + int32_t code = TSDB_CODE_SUCCESS; + SRowBuffPos* pos = (SRowBuffPos*)value; + if (pos->needFree) { + if (isFlushedState(pState->pFileState, key->win.ekey, 0)) { + if (!pos->pRowBuff) { + return code; + } + code = streamStateSessionPut_rocksdb(pState, key, pos->pRowBuff, vLen); + streamStateReleaseBuf(pState, pos, true); + putFreeBuff(pState->pFileState, pos); + stDebug("===stream===save skey:%" PRId64 ", ekey:%" PRId64 ", groupId:%" PRIu64 ".code:%d", key->win.skey, + key->win.ekey, key->groupId, code); + } else { + code = putSessionWinResultBuff(pState->pFileState, value); + } + } + return code; #else SStateSessionKey sKey = {.key = *key, .opNum = pState->number}; return tdbTbUpsert(pState->pTdbState->pSessionStateDb, &sKey, sizeof(SStateSessionKey), value, vLen, From b3c54a0f06afe0f9e7e5bb8b6dd043d6c54e4517 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Tue, 24 Oct 2023 16:04:43 +0800 Subject: [PATCH 061/195] refactor stream backend --- source/libs/stream/src/streamState.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamState.c b/source/libs/stream/src/streamState.c index 218d906504..a368707cb1 100644 --- a/source/libs/stream/src/streamState.c +++ b/source/libs/stream/src/streamState.c @@ -757,7 +757,7 @@ int32_t streamStateSessionDel(SStreamState* pState, const SSessionKey* key) { #ifdef USE_ROCKSDB qDebug("===stream===delete skey:%" PRId64 ", ekey:%" PRId64 ", groupId:%" PRIu64, key->win.skey, key->win.ekey, key->groupId); - return streamStateSessionDel_rocksdb(pState, key); + return deleteRowBuff(pState->pFileState, key, sizeof(SSessionKey)); #else SStateSessionKey sKey = {.key = *key, .opNum = pState->number}; return tdbTbDelete(pState->pTdbState->pSessionStateDb, &sKey, sizeof(SStateSessionKey), pState->pTdbState->txn); From d43b3b4a32b42757f0b16d62d37e485b725c430e Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Tue, 24 Oct 2023 16:17:54 +0800 Subject: [PATCH 062/195] refactor stream backend --- source/libs/stream/src/streamState.c | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/source/libs/stream/src/streamState.c b/source/libs/stream/src/streamState.c index a368707cb1..32672d7b22 100644 --- a/source/libs/stream/src/streamState.c +++ b/source/libs/stream/src/streamState.c @@ -687,8 +687,7 @@ void streamStateResetCur(SStreamStateCur* pCur) { } void streamStateFreeCur(SStreamStateCur* pCur) { - if (!pCur || pCur->buffIndex >= 0) { - taosMemoryFree(pCur); + if (!pCur) { return; } qDebug("streamStateFreeCur"); From db0f6258fe8ba2fdbad27a15861fcd7c69221f5e Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 25 Oct 2023 19:38:22 +0800 Subject: [PATCH 063/195] fix stream case error --- include/libs/stream/tstream.h | 4 ++-- source/dnode/vnode/src/sma/smaRollup.c | 10 ++++++---- source/dnode/vnode/src/tq/tq.c | 20 ++++++++----------- source/libs/stream/src/streamMeta.c | 4 ++-- source/libs/stream/src/streamState.c | 27 +++----------------------- source/libs/stream/src/streamTask.c | 11 ++++++++--- 6 files changed, 29 insertions(+), 47 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 713a457d52..dc24106575 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -774,7 +774,7 @@ int32_t streamMetaCommit(SStreamMeta* pMeta); int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta); int32_t streamMetaReloadAllTasks(SStreamMeta* pMeta); void streamMetaNotifyClose(SStreamMeta* pMeta); -int32_t streamTaskSetDb(SStreamMeta* pMeta, void* pTask); +int32_t streamTaskSetDb(SStreamMeta* pMeta, void* pTask, char *key); void streamMetaStartHb(SStreamMeta* pMeta); void streamMetaInitForSnode(SStreamMeta* pMeta); @@ -794,4 +794,4 @@ int32_t buildCheckpointSourceRsp(SStreamCheckpointSourceReq* pReq, SRpcHandleInf } #endif -#endif /* ifndef _STREAM_H_ */ +#endif /* ifndef _STREAM_H_ */ \ No newline at end of file diff --git a/source/dnode/vnode/src/sma/smaRollup.c b/source/dnode/vnode/src/sma/smaRollup.c index 8da2fff5a6..fe458fdba3 100644 --- a/source/dnode/vnode/src/sma/smaRollup.c +++ b/source/dnode/vnode/src/sma/smaRollup.c @@ -256,9 +256,11 @@ static int32_t tdSetRSmaInfoItemParams(SSma *pSma, SRSmaParam *param, SRSmaStat taosMemoryFree(s); } - SStreamTask task = {.id.taskId = 0, .id.streamId = 0}; // TODO: assign value - task.pMeta = pVnode->pTq->pStreamMeta; - pStreamState = streamStateOpen(taskInfDir, &task, true, -1, -1); + //SStreamTask task = {.id.taskId = 0, .id.streamId = 0}; // TODO: assign value + SStreamTask *pTask = taosMemoryCalloc(1, sizeof(SStreamTask)); + pTask->pMeta = pVnode->pTq->pStreamMeta; + + pStreamState = streamStateOpen(taskInfDir, pTask, true, -1, -1); if (!pStreamState) { terrno = TSDB_CODE_RSMA_STREAM_STATE_OPEN; return TSDB_CODE_FAILED; @@ -1429,4 +1431,4 @@ _exit: smaError("vgId:%d, %s failed at line %d since %s", TD_VID(pVnode), __func__, lino, tstrerror(code)); } return code; -} +} \ No newline at end of file diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index c18a58716c..d028989865 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -749,14 +749,12 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t nextProcessVer) { if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { SStreamTask* pStateTask = pTask; - // SStreamTask task = {0}; - // if (pTask->info.fillHistory) { - // task.id.streamId = pTask->streamTaskId.streamId; - // task.id.taskId = pTask->streamTaskId.taskId; - // task.pMeta = pTask->pMeta; - // pStateTask = &task; - // } + // if (pTask->info.fillHistory) { + // pTask->id.streamId = pTask->streamTaskId.streamId; + // pTask->id.taskId = pTask->streamTaskId.taskId; + // } + pTask->pState = streamStateOpen(pTq->pStreamMeta->path, pTask, false, -1, -1); if (pTask->pState == NULL) { tqError("s-task:%s (vgId:%d) failed to open state for task", pTask->id.idStr, vgId); @@ -786,10 +784,8 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t nextProcessVer) { SStreamTask* pSateTask = pTask; // SStreamTask task = {0}; // if (pTask->info.fillHistory) { - // task.id.streamId = pTask->streamTaskId.streamId; - // task.id.taskId = pTask->streamTaskId.taskId; - // task.pMeta = pTask->pMeta; - // pSateTask = &task; + // pTask->id.streamId = pTask->streamTaskId.streamId; + // pTask->id.taskId = pTask->streamTaskId.taskId; // } pTask->pState = streamStateOpen(pTq->pStreamMeta->path, pTask, false, -1, -1); @@ -1984,4 +1980,4 @@ int32_t tqProcessTaskResetReq(STQ* pTq, SRpcMsg* pMsg) { streamMetaReleaseTask(pMeta, pTask); return TSDB_CODE_SUCCESS; -} +} \ No newline at end of file diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 9ae4c2e043..7c616cba5b 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -226,10 +226,10 @@ int32_t streamMetaMayCvtDbFormat(SStreamMeta* pMeta) { return 0; } -int32_t streamTaskSetDb(SStreamMeta* pMeta, void* arg) { +int32_t streamTaskSetDb(SStreamMeta* pMeta, void* arg, char *key) { SStreamTask* pTask = arg; - char* key = (char*)pTask->id.idStr; + //char* key = (char*)pTask->id.idStr; int64_t chkpId = pTask->checkpointingId; taosThreadMutexLock(&pMeta->backendMutex); diff --git a/source/libs/stream/src/streamState.c b/source/libs/stream/src/streamState.c index 32672d7b22..a6809b11dd 100644 --- a/source/libs/stream/src/streamState.c +++ b/source/libs/stream/src/streamState.c @@ -110,31 +110,10 @@ SStreamState* streamStateOpen(char* path, void* pTask, bool specPath, int32_t sz pState->streamId = pStreamTask->id.streamId; sprintf(pState->pTdbState->idstr, "0x%" PRIx64 "-%d", pState->streamId, pState->taskId); + streamTaskSetDb(pStreamTask->pMeta, pTask, pState->pTdbState->idstr); + #ifdef USE_ROCKSDB SStreamMeta* pMeta = pStreamTask->pMeta; - // pState->streamBackendRid = pMeta->streamBackendRid; - // taosWLockLatch(&pMeta->lock); - // taosThreadMutexLock(&pMeta->backendMutex); - // void* uniqueId = - // taosHashGet(pMeta->pTaskDbUnique, pState->pTdbState->idstr, strlen(pState->pTdbState->idstr) + 1); - // if (uniqueId == NULL) { - // int code = streamStateOpenBackend(pMeta->streamBackend, pState); - // if (code == -1) { - // taosThreadMutexUnlock(&pMeta->backendMutex); - // taosMemoryFree(pState); - // return NULL; - // } - // taosHashPut(pMeta->pTaskDbUnique, pState->pTdbState->idstr, strlen(pState->pTdbState->idstr) + 1, - // &pState->pTdbState->backendCfWrapperId, sizeof(pState->pTdbState->backendCfWrapperId)); - // } else { - // int64_t id = *(int64_t*)uniqueId; - // pState->pTdbState->backendCfWrapperId = id; - // pState->pTdbState->pBackendCfWrapper = taosAcquireRef(streamBackendCfWrapperId, id); - // // already exist stream task for - // qInfo("already exist stream-state for %s", pState->pTdbState->idstr); - // // taosAcquireRef(streamBackendId, pState->streamBackendRid); - // } - // taosThreadMutexUnlock(&pMeta->backendMutex); pState->pTdbState->pOwner = pTask; pState->pFileState = NULL; @@ -1219,4 +1198,4 @@ char* streamStateIntervalDump(SStreamState* pState) { streamStateFreeCur(pCur); return dumpBuf; } -#endif +#endif \ No newline at end of file diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 15f3ba4a94..6d7a03bc6c 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -443,9 +443,14 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i } taosThreadMutexInit(&pTask->lock, &attr); - if (streamTaskSetDb(pMeta, pTask) != 0) { - return -1; - } + // if (pTask->info.fillHistory == 1) { + // // + // } else { + + // } + // if (streamTaskSetDb(pMeta, pTask) != 0) { + // return -1; + // } streamTaskOpenAllUpstreamInput(pTask); return TSDB_CODE_SUCCESS; From 87db93e72ec3de889493ba85138989602829b26e Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 26 Oct 2023 11:36:29 +0800 Subject: [PATCH 064/195] fix stream case error --- source/libs/stream/src/streamMeta.c | 1 - 1 file changed, 1 deletion(-) diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 7c616cba5b..a2d3001968 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -229,7 +229,6 @@ int32_t streamMetaMayCvtDbFormat(SStreamMeta* pMeta) { int32_t streamTaskSetDb(SStreamMeta* pMeta, void* arg, char *key) { SStreamTask* pTask = arg; - //char* key = (char*)pTask->id.idStr; int64_t chkpId = pTask->checkpointingId; taosThreadMutexLock(&pMeta->backendMutex); From 4c03d372ef1886af9f0b68d0900fa2e8a1cdbdb6 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 26 Oct 2023 14:19:34 +0800 Subject: [PATCH 065/195] fix stream case error --- source/dnode/vnode/src/tq/tq.c | 34 ++++++++++++++++++++++++++-------- 1 file changed, 26 insertions(+), 8 deletions(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index d028989865..b1e5058ea9 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -750,10 +750,15 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t nextProcessVer) { if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { SStreamTask* pStateTask = pTask; - // if (pTask->info.fillHistory) { - // pTask->id.streamId = pTask->streamTaskId.streamId; - // pTask->id.taskId = pTask->streamTaskId.taskId; - // } + STaskId taskId = {.streamId = 0, .taskId = 0}; + if (pTask->info.fillHistory) { + + taskId.streamId = pTask->id.streamId; + taskId.taskId = pTask->id.taskId; + + pTask->id.streamId = pTask->streamTaskId.streamId; + pTask->id.taskId = pTask->streamTaskId.taskId; + } pTask->pState = streamStateOpen(pTq->pStreamMeta->path, pTask, false, -1, -1); if (pTask->pState == NULL) { @@ -762,6 +767,10 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t nextProcessVer) { } else { tqDebug("s-task:%s state:%p", pTask->id.idStr, pTask->pState); } + if (pTask->info.fillHistory) { + pTask->id.streamId = taskId.streamId; + pTask->id.taskId = taskId.taskId; + } SReadHandle handle = { .checkpointId = pTask->chkInfo.checkpointId, @@ -783,10 +792,14 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t nextProcessVer) { } else if (pTask->info.taskLevel == TASK_LEVEL__AGG) { SStreamTask* pSateTask = pTask; // SStreamTask task = {0}; - // if (pTask->info.fillHistory) { - // pTask->id.streamId = pTask->streamTaskId.streamId; - // pTask->id.taskId = pTask->streamTaskId.taskId; - // } + + STaskId taskId = {.streamId = 0, .taskId = 0}; + if (pTask->info.fillHistory) { + taskId.streamId = pTask->id.streamId; + taskId.taskId = pTask->id.taskId; + pTask->id.streamId = pTask->streamTaskId.streamId; + pTask->id.taskId = pTask->streamTaskId.taskId; + } pTask->pState = streamStateOpen(pTq->pStreamMeta->path, pTask, false, -1, -1); if (pTask->pState == NULL) { @@ -796,6 +809,11 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t nextProcessVer) { tqDebug("s-task:%s state:%p", pTask->id.idStr, pTask->pState); } + if (pTask->info.fillHistory) { + pTask->id.streamId = taskId.streamId; + pTask->id.taskId = taskId.taskId; + } + int32_t numOfVgroups = (int32_t)taosArrayGetSize(pTask->upstreamInfo.pList); SReadHandle handle = { .checkpointId = pTask->chkInfo.checkpointId, From 7e1e68d8f5b679f871999194c03b104d18c687da Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 26 Oct 2023 18:27:09 +0800 Subject: [PATCH 066/195] fix stream case error --- source/libs/stream/src/streamBackendRocksdb.c | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 4add9d2912..f9b77b4e9a 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -2287,7 +2287,7 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfKe taosMemoryFree(err); \ code = -1; \ } else { \ - qTrace("streamState str:%s succ to write to %s, rowValLen:%d, ttlValLen:%d", toString, funcname, vLen, ttlVLen); \ + qTrace("streamState str:%s succ to write to %s, rowValLen:%d, ttlValLen:%d, %p", toString, funcname, vLen, ttlVLen, wrapper); \ } \ taosMemoryFree(ttlV); \ } while (0); @@ -2328,7 +2328,7 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfKe funcname); \ code = -1; \ } else { \ - qTrace("streamState str: %s succ to read from %s_%s, valLen:%d", toString, wrapper->idstr, funcname, tlen); \ + qTrace("streamState str: %s succ to read from %s_%s, valLen:%d, %p", toString, wrapper->idstr, funcname, tlen, wrapper); \ } \ taosMemoryFree(val); \ if (vLen != NULL) *vLen = tlen; \ @@ -2725,7 +2725,7 @@ SStreamStateCur* streamStateSessionSeekKeyCurrentPrev_rocksdb(SStreamState* pSta stDebug("streamStateSessionSeekKeyCurrentPrev_rocksdb"); STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + SStreamStateCur* pCur = createStreamStateCursor(); if (pCur == NULL) { return NULL; } @@ -2766,7 +2766,7 @@ SStreamStateCur* streamStateSessionSeekKeyCurrentPrev_rocksdb(SStreamState* pSta SStreamStateCur* streamStateSessionSeekKeyCurrentNext_rocksdb(SStreamState* pState, SSessionKey* key) { qDebug("streamStateSessionSeekKeyCurrentNext_rocksdb"); STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + SStreamStateCur* pCur = createStreamStateCursor(); if (pCur == NULL) { return NULL; } @@ -2804,7 +2804,7 @@ SStreamStateCur* streamStateSessionSeekKeyCurrentNext_rocksdb(SStreamState* pSta SStreamStateCur* streamStateSessionSeekKeyNext_rocksdb(SStreamState* pState, const SSessionKey* key) { qDebug("streamStateSessionSeekKeyNext_rocksdb"); STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + SStreamStateCur* pCur = createStreamStateCursor(); if (pCur == NULL) { return NULL; } @@ -2907,7 +2907,7 @@ int32_t streamStateFillDel_rocksdb(SStreamState* pState, const SWinKey* key) { SStreamStateCur* streamStateFillGetCur_rocksdb(SStreamState* pState, const SWinKey* key) { qDebug("streamStateFillGetCur_rocksdb"); - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + SStreamStateCur* pCur = createStreamStateCursor(); STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; if (pCur == NULL) return NULL; @@ -2968,7 +2968,7 @@ int32_t streamStateFillGetKVByCur_rocksdb(SStreamStateCur* pCur, SWinKey* pKey, SStreamStateCur* streamStateFillSeekKeyNext_rocksdb(SStreamState* pState, const SWinKey* key) { qDebug("streamStateFillSeekKeyNext_rocksdb"); STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + SStreamStateCur* pCur = createStreamStateCursor(); if (!pCur) { return NULL; } @@ -3006,7 +3006,7 @@ SStreamStateCur* streamStateFillSeekKeyNext_rocksdb(SStreamState* pState, const SStreamStateCur* streamStateFillSeekKeyPrev_rocksdb(SStreamState* pState, const SWinKey* key) { qDebug("streamStateFillSeekKeyPrev_rocksdb"); STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + SStreamStateCur* pCur = createStreamStateCursor(); if (pCur == NULL) { return NULL; } @@ -3044,7 +3044,7 @@ SStreamStateCur* streamStateFillSeekKeyPrev_rocksdb(SStreamState* pState, const int32_t streamStateSessionGetKeyByRange_rocksdb(SStreamState* pState, const SSessionKey* key, SSessionKey* curKey) { stDebug("streamStateSessionGetKeyByRange_rocksdb"); STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + SStreamStateCur* pCur = createStreamStateCursor(); if (pCur == NULL) { return -1; } @@ -3314,7 +3314,7 @@ int32_t streamDefaultIterGet_rocksdb(SStreamState* pState, const void* start, co return code; } void* streamDefaultIterCreate_rocksdb(SStreamState* pState) { - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + SStreamStateCur* pCur = createStreamStateCursor(); STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; pCur->db = wrapper->db; @@ -3437,4 +3437,4 @@ uint32_t nextPow2(uint32_t x) { x = x | (x >> 8); x = x | (x >> 16); return x + 1; -} +} \ No newline at end of file From ee6b620597fea11be432af0deb3feb84332db718 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Mon, 30 Oct 2023 17:29:32 +0800 Subject: [PATCH 067/195] fix mem leak --- source/libs/stream/src/streamBackendRocksdb.c | 16 +++++++++++++++- source/libs/stream/src/streamTask.c | 2 ++ 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index f9b77b4e9a..35b7b0c233 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1819,6 +1819,9 @@ _EXIT: void taskDbDestroy(void* pBackend) { STaskDbWrapper* wrapper = pBackend; + qDebug("succ to destroy stream backend:%p", wrapper); + + int8_t nCf = sizeof(ginitDict) / sizeof(ginitDict[0]); if (wrapper == NULL) return; @@ -1826,7 +1829,6 @@ void taskDbDestroy(void* pBackend) { rocksdb_flushoptions_t* flushOpt = rocksdb_flushoptions_create(); rocksdb_flushoptions_set_wait(flushOpt, 1); - int8_t nCf = sizeof(ginitDict) / sizeof(ginitDict[0]); char* err = NULL; for (int i = 0; i < nCf; i++) { if (wrapper->pCf[i] != NULL) rocksdb_flush_cf(wrapper->db, flushOpt, wrapper->pCf[i], &err); @@ -1849,7 +1851,19 @@ void taskDbDestroy(void* pBackend) { rocksdb_env_destroy(wrapper->env); rocksdb_cache_destroy(wrapper->cache); + taosMemoryFree(wrapper->pCf); + + for (int i = 0; i < nCf; i++) { + rocksdb_options_t *opt = wrapper->pCfOpts[i]; + rocksdb_comparator_t *compare = wrapper->pCompares[i]; + rocksdb_block_based_table_options_t *tblOpt = wrapper->pCfParams[i].tableOpt; + + rocksdb_options_destroy(opt); + rocksdb_comparator_destroy(compare); + rocksdb_block_based_options_destroy(tblOpt); + + } taosMemoryFree(wrapper->pCompares); taosMemoryFree(wrapper->pCfOpts); taosMemoryFree(wrapper->pCfParams); diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 6d7a03bc6c..d4b9b39fa9 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -368,6 +368,8 @@ void tFreeStreamTask(SStreamTask* pTask) { if (pTask->pState) { stDebug("s-task:0x%x start to free task state", taskId); streamStateClose(pTask->pState, status == TASK_STATUS__DROPPING); + taskDbRemoveRef(pTask->pBackend); + } if (pTask->id.idStr != NULL) { From 7fad855247e7fd125116a996dd8c7263e1d4ed51 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Mon, 30 Oct 2023 18:16:12 +0800 Subject: [PATCH 068/195] fix mem leak --- source/libs/stream/src/streamState.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamState.c b/source/libs/stream/src/streamState.c index a6809b11dd..16363e1bdd 100644 --- a/source/libs/stream/src/streamState.c +++ b/source/libs/stream/src/streamState.c @@ -108,7 +108,7 @@ SStreamState* streamStateOpen(char* path, void* pTask, bool specPath, int32_t sz SStreamTask* pStreamTask = pTask; pState->taskId = pStreamTask->id.taskId; pState->streamId = pStreamTask->id.streamId; - sprintf(pState->pTdbState->idstr, "0x%" PRIx64 "-%d", pState->streamId, pState->taskId); + sprintf(pState->pTdbState->idstr, "0x%" PRIx64 "-0x%x", pState->streamId, pState->taskId); streamTaskSetDb(pStreamTask->pMeta, pTask, pState->pTdbState->idstr); From bb8252df16f5dc74de8167a9abae946b12acaecc Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Tue, 31 Oct 2023 20:59:24 +0800 Subject: [PATCH 069/195] fix mem leak --- include/libs/stream/streamState.h | 3 ++- source/dnode/vnode/src/sma/smaRollup.c | 4 ++++ source/libs/stream/src/streamState.c | 6 ++++++ 3 files changed, 12 insertions(+), 1 deletion(-) diff --git a/include/libs/stream/streamState.h b/include/libs/stream/streamState.h index b7f100733b..17b766906c 100644 --- a/include/libs/stream/streamState.h +++ b/include/libs/stream/streamState.h @@ -35,6 +35,7 @@ int32_t streamStateBegin(SStreamState* pState); int32_t streamStateCommit(SStreamState* pState); void streamStateDestroy(SStreamState* pState, bool remove); int32_t streamStateDeleteCheckPoint(SStreamState* pState, TSKEY mark); +int32_t streamStateDelTaskDb(SStreamState* pState); int32_t streamStateFuncPut(SStreamState* pState, const SWinKey* key, const void* value, int32_t vLen); int32_t streamStateFuncGet(SStreamState* pState, const SWinKey* key, void** ppVal, int32_t* pVLen); @@ -130,4 +131,4 @@ char* streamStateIntervalDump(SStreamState* pState); } #endif -#endif /* ifndef _STREAM_STATE_H_ */ +#endif /* ifndef _STREAM_STATE_H_ */ \ No newline at end of file diff --git a/source/dnode/vnode/src/sma/smaRollup.c b/source/dnode/vnode/src/sma/smaRollup.c index fe458fdba3..5485ab88e3 100644 --- a/source/dnode/vnode/src/sma/smaRollup.c +++ b/source/dnode/vnode/src/sma/smaRollup.c @@ -89,7 +89,11 @@ void *tdFreeRSmaInfo(SSma *pSma, SRSmaInfo *pInfo, bool isDeepFree) { } if (isDeepFree && pItem->pStreamState) { + //SStreamTask *pTask = pItem->pStreamState->pTdb + streamStateDelTaskDb(pItem->pStreamState); streamStateClose(pItem->pStreamState, false); + + } if (isDeepFree && pInfo->taskInfo[i]) { diff --git a/source/libs/stream/src/streamState.c b/source/libs/stream/src/streamState.c index 16363e1bdd..e874f42b7a 100644 --- a/source/libs/stream/src/streamState.c +++ b/source/libs/stream/src/streamState.c @@ -209,6 +209,12 @@ _err: #endif } +int32_t streamStateDelTaskDb(SStreamState* pState) { + SStreamTask* pTask = pState->pTdbState->pOwner; + taskDbRemoveRef(pTask->pBackend); + taosMemoryFree(pTask); + return 0; +} void streamStateClose(SStreamState* pState, bool remove) { SStreamTask* pTask = pState->pTdbState->pOwner; #ifdef USE_ROCKSDB From 6cab686e82a9a46d29828e34f61456bfc0fa59e3 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 1 Nov 2023 09:37:54 +0800 Subject: [PATCH 070/195] fix mem leak --- source/dnode/snode/src/snode.c | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index ad28237bf4..85d25a4b4b 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -60,7 +60,6 @@ int32_t sndExpandTask(SSnode *pSnode, SStreamTask *pTask, int64_t nextProcessVer return code; } pTask->pBackend = NULL; - ASSERT(0); streamTaskOpenAllUpstreamInput(pTask); @@ -448,4 +447,4 @@ int32_t sndProcessStreamMsg(SSnode *pSnode, SRpcMsg *pMsg) { ASSERT(0); } return 0; -} +} \ No newline at end of file From 1e96edcdd763af19e9f281ce303bd4b92f9a04fb Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 1 Nov 2023 17:30:46 +0800 Subject: [PATCH 071/195] Merge branch 'enh/new3.0' into enh/refactorBackend --- source/dnode/mnode/impl/src/mndStream.c | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index b6a15ae368..f8b976d90e 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -2532,15 +2532,24 @@ int32_t doKillActiveCheckpointTrans(SMnode *pMnode) { } sdbRelease(pSdb, pTrans); - } - return 0; + + if (transId == 0) { + mError("failed to find the checkpoint trans, reset not executed"); + return TSDB_CODE_SUCCESS; + } + + pTrans = mndAcquireTrans(pMnode, transId); + mInfo("kill checkpoint trans:%d", transId); + + mndKillTrans(pMnode, pTrans); + mndReleaseTrans(pMnode, pTrans); + return TSDB_CODE_SUCCESS; } int32_t mndResetFromCheckpoint(SMnode* pMnode) { doKillActiveCheckpointTrans(pMnode); - int32_t code = 0; // set all tasks status to be normal, refactor later to be stream level, instead of vnode level. SSdb *pSdb = pMnode->pSdb; SStreamObj *pStream = NULL; @@ -2553,7 +2562,7 @@ int32_t mndResetFromCheckpoint(SMnode* pMnode) { // todo this transaction should exist be only one mDebug("stream:%s (0x%" PRIx64 ") reset checkpoint procedure, create reset trans", pStream->name, pStream->uid); - code = createStreamResetStatusTrans(pMnode, pStream); + int32_t code = createStreamResetStatusTrans(pMnode, pStream); if (code != TSDB_CODE_SUCCESS) { sdbCancelFetch(pSdb, pIter); return code; From b9114922705895e6b3c3128d5a5c7d3201ecee1b Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 2 Nov 2023 09:46:36 +0800 Subject: [PATCH 072/195] Merge branch 'enh/new3.0' into enh/refactorBackend --- source/libs/stream/src/streamMeta.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 41c9ca2361..a5bb9e59d8 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -484,7 +484,7 @@ void streamMetaCloseImpl(void* arg) { taosHashCleanup(pMeta->pTaskDbUnique); taosHashCleanup(pMeta->pUpdateTaskSet); //taosHashCleanup(pMeta->pTaskBackendUnique); - //taosHashCleanup(pMeta->updateInfo.pTasks); + taosHashCleanup(pMeta->updateInfo.pTasks); taosHashCleanup(pMeta->startInfo.pReadyTaskSet); taosMemoryFree(pMeta->pHbInfo); From 7e7a39f077527d5c95bdc016a815fc344fc8417b Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 2 Nov 2023 10:15:34 +0800 Subject: [PATCH 073/195] Merge branch 'enh/new3.0' into enh/refactorBackend --- source/libs/stream/src/streamMeta.c | 20 ++++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index a5bb9e59d8..d12661c494 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -65,6 +65,7 @@ static void streamMetaEnvInit() { } void streamMetaInit() { taosThreadOnce(&streamMetaModuleInit, streamMetaEnvInit); } + void streamMetaCleanup() { taosCloseRef(streamBackendId); taosCloseRef(streamBackendCfWrapperId); @@ -319,15 +320,7 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF pMeta->expandFunc = expandFunc; pMeta->stage = stage; - int64_t* pRid = taosMemoryMalloc(sizeof(int64_t)); - pMeta->rid = taosAddRef(streamMetaId, pMeta); - *pRid = pMeta->rid; - metaRefMgtAdd(pMeta->vgId, pRid); - - pMeta->pHbInfo->hbTmr = taosTmrStart(metaHbToMnode, META_HB_CHECK_INTERVAL, pRid, streamEnv.timer); - pMeta->pHbInfo->tickCounter = 0; - pMeta->pHbInfo->stopFlag = 0; pMeta->pTaskDbUnique = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), false, HASH_ENTRY_LOCK); @@ -347,6 +340,17 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF pMeta->numOfStreamTasks = 0; stInfo("vgId:%d open stream meta successfully, latest checkpoint:%" PRId64 ", stage:%" PRId64, vgId, pMeta->chkpId, stage); + + pMeta->rid = taosAddRef(streamMetaId, pMeta); + + int64_t* pRid = taosMemoryMalloc(sizeof(int64_t)); + memcpy(pRid, &pMeta->rid, sizeof(pMeta->rid)); + metaRefMgtAdd(pMeta->vgId, pRid); + + pMeta->pHbInfo->hbTmr = taosTmrStart(metaHbToMnode, META_HB_CHECK_INTERVAL, pRid, streamEnv.timer); + pMeta->pHbInfo->tickCounter = 0; + pMeta->pHbInfo->stopFlag = 0; + return pMeta; _err: From 0a73832fd2ba492ed16610de6480c731eae3d7a8 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 2 Nov 2023 20:39:55 +0800 Subject: [PATCH 074/195] add checkpoint delta --- source/libs/stream/src/streamBackendRocksdb.c | 268 ++++++++++++------ 1 file changed, 188 insertions(+), 80 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 7271247ef2..bcfeb77ff9 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -20,6 +20,28 @@ #include "tcommon.h" #include "tref.h" +typedef struct SDbChkp { + int8_t init; + char* pCurrent; + char* pManifest; + SArray* pSST; + int64_t preCkptId; + int64_t curChkpId; + char* path; + + char* buf; + int32_t len; + + // ping-pong buf + SHashObj* pSstTbl[2]; + int8_t idx; + + SArray* pAdd; + SArray* pDel; + int8_t update; + + TdThreadRwlock rwLock; +} SDbChkp; typedef struct { int8_t init; char* pCurrent; @@ -39,6 +61,10 @@ typedef struct { SArray* pAdd; SArray* pDel; int8_t update; + + SHashObj *pDbChkpTbl; + + TdThreadRwlock rwLock; } SBackendManager; typedef struct SCompactFilteFactory { @@ -145,42 +171,23 @@ void destroyFunc(void* arg); int32_t encodeValueFunc(void* value, int32_t vlen, int64_t ttl, char** dest); int32_t decodeValueFunc(void* value, int32_t vlen, int64_t* ttl, char** dest); -SBackendManager* bkdMgtCreate(char* path) { - SBackendManager* p = taosMemoryCalloc(1, sizeof(SBackendManager)); - p->curChkpId = 0; - p->preCkptId = 0; - p->pSST = taosArrayInit(64, sizeof(void*)); - p->path = taosStrdup(path); - p->len = strlen(path) + 128; - p->buf = taosMemoryCalloc(1, p->len); - p->idx = 0; - p->pSstTbl[0] = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), false, HASH_ENTRY_LOCK); - p->pSstTbl[1] = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), false, HASH_ENTRY_LOCK); +void dbChkpDestroy(SDbChkp* pChkp) { + taosMemoryFree(pChkp->buf); + taosMemoryFree(pChkp->path); - p->pAdd = taosArrayInit(64, sizeof(void*)); - p->pDel = taosArrayInit(64, sizeof(void*)); - p->update = 0; - return p; + taosArrayDestroyP(pChkp->pSST, taosMemoryFree); + taosArrayDestroyP(pChkp->pAdd, taosMemoryFree); + taosArrayDestroyP(pChkp->pDel, taosMemoryFree); + + taosHashCleanup(pChkp->pSstTbl[0]); + taosHashCleanup(pChkp->pSstTbl[1]); + + taosMemoryFree(pChkp->pCurrent); + taosMemoryFree(pChkp->pManifest); + } -void bkdMgtDestroy(SBackendManager* bm) { - if (bm == NULL) return; - taosMemoryFree(bm->buf); - taosMemoryFree(bm->path); - - taosArrayDestroyP(bm->pSST, taosMemoryFree); - taosArrayDestroyP(bm->pAdd, taosMemoryFree); - taosArrayDestroyP(bm->pDel, taosMemoryFree); - - taosHashCleanup(bm->pSstTbl[0]); - taosHashCleanup(bm->pSstTbl[1]); - - taosMemoryFree(bm->pCurrent); - taosMemoryFree(bm->pManifest); - - taosMemoryFree(bm); -} int32_t compareHashTableImpl(SHashObj* p1, SHashObj* p2, SArray* diff) { int32_t code = 0; @@ -204,7 +211,12 @@ int32_t compareHashTable(SHashObj* p1, SHashObj* p2, SArray* add, SArray* del) { return code; } -int32_t bkdMgtGetDelta(SBackendManager* bm, int64_t chkpId, SArray* list) { + +int32_t dbChkpGetDelta(SDbChkp* p, int64_t chkpId, SArray *list) { + taosThreadRwlockWrlock(&p->rwLock); + + p->preCkptId = p->curChkpId; + p->curChkpId = chkpId; const char* pCurrent = "CURRENT"; int32_t currLen = strlen(pCurrent); @@ -214,81 +226,112 @@ int32_t bkdMgtGetDelta(SBackendManager* bm, int64_t chkpId, SArray* list) { const char* pSST = ".sst"; int32_t sstLen = strlen(pSST); - memset(bm->buf, 0, bm->len); - sprintf(bm->buf, "%s%scheckpoint%" PRId64 "", bm->path, TD_DIRSEP, chkpId); + memset(p->buf, 0, p->len); + sprintf(p->buf, "%s%scheckpoint%" PRId64 "", p->path, TD_DIRSEP, chkpId); - taosArrayClearP(bm->pAdd, taosMemoryFree); - taosArrayClearP(bm->pDel, taosMemoryFree); + taosArrayClearP(p->pAdd, taosMemoryFree); + taosArrayClearP(p->pDel, taosMemoryFree); - TdDirPtr pDir = taosOpenDir(bm->buf); + TdDirPtr pDir = taosOpenDir(p->buf); TdDirEntryPtr de = NULL; int8_t dummy = 0; while ((de = taosReadDir(pDir)) != NULL) { char* name = taosGetDirEntryName(de); if (strcmp(name, ".") == 0 || strcmp(name, "..") == 0) continue; if (strlen(name) == currLen && strcmp(name, pCurrent) == 0) { - taosMemoryFreeClear(bm->pCurrent); - bm->pCurrent = taosStrdup(name); - taosHashPut(bm->pSstTbl[1 - bm->idx], name, strlen(name), &dummy, sizeof(dummy)); + taosMemoryFreeClear(p->pCurrent); + p->pCurrent = taosStrdup(name); + taosHashPut(p->pSstTbl[1 - p->idx], name, strlen(name), &dummy, sizeof(dummy)); continue; } if (strlen(name) >= maniLen && strncmp(name, pManifest, maniLen) == 0) { - taosMemoryFreeClear(bm->pManifest); - bm->pManifest = taosStrdup(name); - taosHashPut(bm->pSstTbl[1 - bm->idx], name, strlen(name), &dummy, sizeof(dummy)); + taosMemoryFreeClear(p->pManifest); + p->pManifest = taosStrdup(name); + taosHashPut(p->pSstTbl[1 - p->idx], name, strlen(name), &dummy, sizeof(dummy)); continue; } if (strlen(name) >= sstLen && strncmp(name + strlen(name) - 4, pSST, sstLen) == 0) { // char* p = taosStrdup(name); - taosHashPut(bm->pSstTbl[1 - bm->idx], name, strlen(name), &dummy, sizeof(dummy)); + taosHashPut(p->pSstTbl[1 - p->idx], name, strlen(name), &dummy, sizeof(dummy)); continue; } } - if (bm->init == 0) { - bm->preCkptId = -1; - bm->curChkpId = chkpId; - bm->init = 1; + if (p->init == 0) { + p->preCkptId = -1; + p->curChkpId = chkpId; + p->init = 1; - void* pIter = taosHashIterate(bm->pSstTbl[1 - bm->idx], NULL); + void* pIter = taosHashIterate(p->pSstTbl[1 - p->idx], NULL); while (pIter) { size_t len; char* name = taosHashGetKey(pIter, &len); if (name != NULL && len != 0) { - taosArrayPush(bm->pAdd, &name); + taosArrayPush(p->pAdd, &name); } - pIter = taosHashIterate(bm->pSstTbl[1 - bm->idx], pIter); + pIter = taosHashIterate(p->pSstTbl[1 - p->idx], pIter); } - if (taosArrayGetSize(bm->pAdd) > 0) bm->update = 1; + if (taosArrayGetSize(p->pAdd) > 0) p->update = 1; } else { - int32_t code = compareHashTable(bm->pSstTbl[bm->idx], bm->pSstTbl[1 - bm->idx], bm->pAdd, bm->pDel); + int32_t code = compareHashTable(p->pSstTbl[p->idx], p->pSstTbl[1 - p->idx], p->pAdd, p->pDel); if (code != 0) { // dead code - taosArrayClearP(bm->pAdd, taosMemoryFree); - taosArrayClearP(bm->pDel, taosMemoryFree); - taosHashClear(bm->pSstTbl[1 - bm->idx]); - bm->update = 0; + taosArrayClearP(p->pAdd, taosMemoryFree); + taosArrayClearP(p->pDel, taosMemoryFree); + taosHashClear(p->pSstTbl[1 - p->idx]); + p->update = 0; taosCloseDir(&pDir); return code; } - bm->preCkptId = bm->curChkpId; - bm->curChkpId = chkpId; - if (taosArrayGetSize(bm->pAdd) == 0 && taosArrayGetSize(bm->pDel) == 0) { - bm->update = 0; + p->preCkptId = p->curChkpId; + p->curChkpId = chkpId; + if (taosArrayGetSize(p->pAdd) == 0 && taosArrayGetSize(p->pDel) == 0) { + p->update = 0; } } - taosHashClear(bm->pSstTbl[bm->idx]); - bm->idx = 1 - bm->idx; + taosHashClear(p->pSstTbl[p->idx]); + p->idx = 1 - p->idx; taosCloseDir(&pDir); + taosThreadRwlockUnlock(&p->rwLock); return 0; + +} +SDbChkp* dbChktCreate(char* path, int64_t initChkpId) { + SDbChkp *p = taosMemoryCalloc(1, sizeof(SDbChkp)); + p->curChkpId = initChkpId; + p->preCkptId = -1; + p->pSST = taosArrayInit(64, sizeof(void*)); + p->path = taosStrdup(path); + p->len = strlen(path) + 128; + p->buf = taosMemoryCalloc(1, p->len); + + p->idx = 0; + p->pSstTbl[0] = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), false, HASH_ENTRY_LOCK); + p->pSstTbl[1] = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), false, HASH_ENTRY_LOCK); + + p->pAdd = taosArrayInit(64, sizeof(void*)); + p->pDel = taosArrayInit(64, sizeof(void*)); + p->update = 0; + taosThreadRwlockInit(&p->rwLock, NULL); + + SArray *list = NULL; + int32_t code = dbChkpGetDelta(p, initChkpId, list); + + return p; } -int32_t bkdMgtDumpTo(SBackendManager* bm, char* dname) { +int32_t dbChkpInit(SDbChkp* p) { + if (p == NULL) return 0; + return 0; +} +int32_t dbChkpDumpTo(SDbChkp* p, char* dname) { + + taosThreadRwlockRdlock(&p->rwLock); int32_t code = 0; - int32_t len = bm->len + 128; + int32_t len = p->len + 128; char* srcBuf = taosMemoryCalloc(1, len); char* dstBuf = taosMemoryCalloc(1, len); @@ -296,8 +339,8 @@ int32_t bkdMgtDumpTo(SBackendManager* bm, char* dname) { char* srcDir = taosMemoryCalloc(1, len); char* dstDir = taosMemoryCalloc(1, len); - sprintf(srcDir, "%s%s%s%" PRId64 "", bm->path, TD_DIRSEP, "checkpoint", bm->curChkpId); - sprintf(dstDir, "%s%s%s", bm->path, TD_DIRSEP, dname); + sprintf(srcDir, "%s%s%s%" PRId64 "", p->path, TD_DIRSEP, "checkpoint", p->curChkpId); + sprintf(dstDir, "%s%s%s", p->path, TD_DIRSEP, dname); if (!taosDirExist(srcDir)) { stError("failed to dump srcDir %s, reason: not exist such dir", srcDir); @@ -314,30 +357,30 @@ int32_t bkdMgtDumpTo(SBackendManager* bm, char* dname) { // clear current file memset(dstBuf, 0, len); - sprintf(dstBuf, "%s%s%s", dstDir, TD_DIRSEP, bm->pCurrent); + sprintf(dstBuf, "%s%s%s", dstDir, TD_DIRSEP, p->pCurrent); taosRemoveFile(dstBuf); memset(dstBuf, 0, len); - sprintf(dstBuf, "%s%s%s", dstDir, TD_DIRSEP, bm->pManifest); + sprintf(dstBuf, "%s%s%s", dstDir, TD_DIRSEP, p->pManifest); taosRemoveFile(dstBuf); // add file to $name dir - for (int i = 0; i < taosArrayGetSize(bm->pAdd); i++) { + for (int i = 0; i < taosArrayGetSize(p->pAdd); i++) { memset(dstBuf, 0, len); memset(srcBuf, 0, len); - char* filename = taosArrayGetP(bm->pAdd, i); + char* filename = taosArrayGetP(p->pAdd, i); sprintf(srcBuf, "%s%s%s", srcDir, TD_DIRSEP, filename); sprintf(dstBuf, "%s%s%s", dstDir, TD_DIRSEP, filename); taosCopyFile(srcBuf, dstBuf); } // del file in $name - for (int i = 0; i < taosArrayGetSize(bm->pDel); i++) { + for (int i = 0; i < taosArrayGetSize(p->pDel); i++) { memset(dstBuf, 0, len); memset(srcBuf, 0, len); - char* filename = taosArrayGetP(bm->pDel, i); + char* filename = taosArrayGetP(p->pDel, i); sprintf(dstBuf, "%s%s%s", dstDir, TD_DIRSEP, filename); taosRemoveFile(dstBuf); } @@ -345,27 +388,92 @@ int32_t bkdMgtDumpTo(SBackendManager* bm, char* dname) { // copy current file to dst dir memset(srcBuf, 0, len); memset(dstBuf, 0, len); - sprintf(srcBuf, "%s%s%s", srcDir, TD_DIRSEP, bm->pCurrent); - sprintf(dstBuf, "%s%s%s", dstDir, TD_DIRSEP, bm->pCurrent); + sprintf(srcBuf, "%s%s%s", srcDir, TD_DIRSEP, p->pCurrent); + sprintf(dstBuf, "%s%s%s", dstDir, TD_DIRSEP, p->pCurrent); taosCopyFile(srcBuf, dstBuf); // copy manifest file to dst dir memset(srcBuf, 0, len); memset(dstBuf, 0, len); - sprintf(srcBuf, "%s%s%s", srcDir, TD_DIRSEP, bm->pManifest); - sprintf(dstBuf, "%s%s%s", dstDir, TD_DIRSEP, bm->pManifest); + sprintf(srcBuf, "%s%s%s", srcDir, TD_DIRSEP, p->pManifest); + sprintf(dstBuf, "%s%s%s", dstDir, TD_DIRSEP, p->pManifest); taosCopyFile(srcBuf, dstBuf); // clear delta data buf - taosArrayClearP(bm->pAdd, taosMemoryFree); - taosArrayClearP(bm->pDel, taosMemoryFree); + taosArrayClearP(p->pAdd, taosMemoryFree); + taosArrayClearP(p->pDel, taosMemoryFree); _ERROR: + taosThreadRwlockUnlock(&p->rwLock); taosMemoryFree(srcBuf); taosMemoryFree(dstBuf); taosMemoryFree(srcDir); taosMemoryFree(dstDir); return code; + +} +SBackendManager* bkdMgtCreate(char* path) { + SBackendManager* p = taosMemoryCalloc(1, sizeof(SBackendManager)); + p->pDbChkpTbl = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), true, HASH_ENTRY_LOCK); + taosThreadRwlockInit(&p->rwLock, NULL); + return p; +} + +void bkdMgtDestroy(SBackendManager* bm) { + if (bm == NULL) return; + void *pIter = taosHashIterate(bm->pDbChkpTbl, NULL); + while (pIter) { + SDbChkp *pChkp = *(SDbChkp **)(pIter); + dbChkpDestroy(pChkp); + + pIter = taosHashIterate(bm->pDbChkpTbl, pIter); + } + + taosThreadRwlockDestroy(&bm->rwLock); + + taosMemoryFree(bm); +} +int32_t bkdMgtGetDelta(SBackendManager* bm, char *taskId, int64_t chkpId, SArray* list) { + int32_t code = 0; + taosThreadRwlockWrlock(&bm->rwLock); + + SDbChkp *pChkp = taosHashGet(bm->pDbChkpTbl, taskId, strlen(taskId)); + code = dbChkpGetDelta(pChkp, chkpId, list); + + taosThreadRwlockUnlock(&bm->rwLock); + return code ; +} + +int32_t bkdMgtAddChkp(SBackendManager *bm, char *task, char *path) { + int32_t code = -1; + + taosThreadRwlockWrlock(&bm->rwLock); + SDbChkp **pp = taosHashGet(bm->pDbChkpTbl, task, strlen(task)); + if (pp == NULL) { + SDbChkp *p = dbChktCreate(path, 0); + if (p != NULL) { + taosHashPut(bm->pDbChkpTbl, task, strlen(task), &p, sizeof(void *)); + code = 0; + } + } else { + stError("task chkp already exists"); + } + + taosThreadRwlockUnlock(&bm->rwLock); + + return code; +} + +int32_t bkdMgtDumpTo(SBackendManager* bm, char *taskId, char* dname) { + int32_t code = 0; + taosThreadRwlockRdlock(&bm->rwLock); + + SDbChkp *p = taosHashGet(bm->pDbChkpTbl, taskId, strlen(taskId)); + code = dbChkpDumpTo(p, dname); + + taosThreadRwlockUnlock(&bm->rwLock); + return code; + } SCfInit ginitDict[] = { From 069dd8de84c2a1759b7a01e0fea5647ca99aa25f Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Fri, 3 Nov 2023 09:28:30 +0800 Subject: [PATCH 075/195] Merge branch 'enh/new3.0' into enh/refactorBackend --- source/libs/stream/src/streamBackendRocksdb.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index bcfeb77ff9..589627c41a 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -39,7 +39,7 @@ typedef struct SDbChkp { SArray* pAdd; SArray* pDel; int8_t update; - + TdThreadRwlock rwLock; } SDbChkp; typedef struct { From d82d51cf8f646892a2760da8c2353a611e43dd48 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Fri, 3 Nov 2023 10:40:00 +0800 Subject: [PATCH 076/195] Merge branch 'enh/new3.0' into enh/refactorBackend --- source/libs/stream/src/streamBackendRocksdb.c | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 589627c41a..664802bd8a 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -258,9 +258,6 @@ int32_t dbChkpGetDelta(SDbChkp* p, int64_t chkpId, SArray *list) { } } if (p->init == 0) { - p->preCkptId = -1; - p->curChkpId = chkpId; - p->init = 1; void* pIter = taosHashIterate(p->pSstTbl[1 - p->idx], NULL); while (pIter) { @@ -272,6 +269,10 @@ int32_t dbChkpGetDelta(SDbChkp* p, int64_t chkpId, SArray *list) { pIter = taosHashIterate(p->pSstTbl[1 - p->idx], pIter); } if (taosArrayGetSize(p->pAdd) > 0) p->update = 1; + + p->init = 1; + p->preCkptId = -1; + p->curChkpId = chkpId; } else { int32_t code = compareHashTable(p->pSstTbl[p->idx], p->pSstTbl[1 - p->idx], p->pAdd, p->pDel); if (code != 0) { @@ -284,12 +285,15 @@ int32_t dbChkpGetDelta(SDbChkp* p, int64_t chkpId, SArray *list) { return code; } - p->preCkptId = p->curChkpId; - p->curChkpId = chkpId; if (taosArrayGetSize(p->pAdd) == 0 && taosArrayGetSize(p->pDel) == 0) { p->update = 0; } + + p->preCkptId = p->curChkpId; + p->curChkpId = chkpId; } + + taosHashClear(p->pSstTbl[p->idx]); p->idx = 1 - p->idx; From feeead50c5343359244ad63fab5818f805db0c50 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Mon, 6 Nov 2023 13:26:38 +0800 Subject: [PATCH 077/195] Merge branch 'enh/new3.0' into enh/refactorBackend --- source/libs/stream/src/streamBackendRocksdb.c | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 664802bd8a..797329d52b 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1691,7 +1691,7 @@ const char* compactFilteName(void* arg) { return "stream_filte"; } rocksdb_compactionfilter_t* compactFilteFactoryCreateFilter(void* arg, rocksdb_compactionfiltercontext_t* ctx) { SCompactFilteFactory* state = arg; rocksdb_compactionfilter_t* filter = - rocksdb_compactionfilter_create(NULL, destroyCompactFilte, compactFilte, compactFilteName); + rocksdb_compactionfilter_create(state, destroyCompactFilte, compactFilte, compactFilteName); return filter; } @@ -1773,7 +1773,7 @@ void taskDbRemoveRef(void* pTaskDb) { } // void taskDbDestroy(STaskDbWrapper* wrapper); -void taskDbInitDBOpt(STaskDbWrapper* pTaskDb) { +void taskDbInitOpt(STaskDbWrapper* pTaskDb) { rocksdb_env_t* env = rocksdb_create_default_env(); rocksdb_cache_t* cache = rocksdb_cache_create_lru(256); @@ -1895,7 +1895,7 @@ STaskDbWrapper* taskDbOpen(char* path, char* key, int64_t chkpId) { taosThreadMutexInit(&pTaskDb->mutex, NULL); taskDbInitChkpOpt(pTaskDb); - taskDbInitDBOpt(pTaskDb); + taskDbInitOpt(pTaskDb); statePath = NULL; cfNames = rocksdb_list_column_families(pTaskDb->dbOpt, dbPath, &nCf, &err); From ba52769867133e60984e745cbb8e287717a9ff0a Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Mon, 6 Nov 2023 17:56:09 +0800 Subject: [PATCH 078/195] Merge branch 'enh/new3.0' into enh/refactorBackend --- source/libs/stream/src/streamBackendRocksdb.c | 123 +++++++++++++++++- 1 file changed, 120 insertions(+), 3 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 797329d52b..efb9bb599c 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -93,10 +93,18 @@ void destroyRocksdbCfInst(RocksdbCfInst* inst); void destroyCompactFilteFactory(void* arg); void destroyCompactFilte(void* arg); const char* compactFilteFactoryName(void* arg); +const char* compactFilteFactoryNameSess(void* arg); +const char* compactFilteFactoryNameState(void* arg); +const char* compactFilteFactoryNameFunc(void* arg); +const char* compactFilteFactoryNameFill(void* arg); const char* compactFilteName(void* arg); unsigned char compactFilte(void* arg, int level, const char* key, size_t klen, const char* val, size_t vlen, char** newval, size_t* newvlen, unsigned char* value_changed); rocksdb_compactionfilter_t* compactFilteFactoryCreateFilter(void* arg, rocksdb_compactionfiltercontext_t* ctx); +rocksdb_compactionfilter_t* compactFilteFactoryCreateFilterSess(void* arg, rocksdb_compactionfiltercontext_t* ctx); +rocksdb_compactionfilter_t* compactFilteFactoryCreateFilterState(void* arg, rocksdb_compactionfiltercontext_t* ctx); +rocksdb_compactionfilter_t* compactFilteFactoryCreateFilterFunc(void* arg, rocksdb_compactionfiltercontext_t* ctx); +rocksdb_compactionfilter_t* compactFilteFactoryCreateFilterFill(void* arg, rocksdb_compactionfiltercontext_t* ctx); const char* cfName[] = {"default", "state", "fill", "sess", "func", "parname", "partag"}; @@ -303,7 +311,7 @@ int32_t dbChkpGetDelta(SDbChkp* p, int64_t chkpId, SArray *list) { return 0; } -SDbChkp* dbChktCreate(char* path, int64_t initChkpId) { +SDbChkp* dbChkpCreate(char* path, int64_t initChkpId) { SDbChkp *p = taosMemoryCalloc(1, sizeof(SDbChkp)); p->curChkpId = initChkpId; p->preCkptId = -1; @@ -454,7 +462,7 @@ int32_t bkdMgtAddChkp(SBackendManager *bm, char *task, char *path) { taosThreadRwlockWrlock(&bm->rwLock); SDbChkp **pp = taosHashGet(bm->pDbChkpTbl, task, strlen(task)); if (pp == NULL) { - SDbChkp *p = dbChktCreate(path, 0); + SDbChkp *p = dbChkpCreate(path, 0); if (p != NULL) { taosHashPut(bm->pDbChkpTbl, task, strlen(task), &p, sizeof(void *)); code = 0; @@ -497,6 +505,40 @@ SCfInit ginitDict[] = { encodeValueFunc, decodeValueFunc}, }; + +typedef rocksdb_compactionfilter_t* (*CreateFactoryFunc)(void* arg, rocksdb_compactionfiltercontext_t* ctx); +typedef const char* (*FactoryNameFunc)(void* arg); +typedef void(*DestroyFactoryFunc)(void *arg); + +typedef struct { + void *funcName; + + DestroyFactoryFunc destroy; + CreateFactoryFunc create; + FactoryNameFunc factoryName; +} SCfFilterFactory; + +SCfFilterFactory ginitFilterDict[] = { + {"default", destroyCompactFilteFactory,compactFilteFactoryCreateFilter,compactFilteFactoryName}, + {"state", destroyCompactFilteFactory,compactFilteFactoryCreateFilterState,compactFilteFactoryNameState}, + {"fill", destroyCompactFilteFactory,compactFilteFactoryCreateFilterFill,compactFilteFactoryNameFill}, + {"sess", destroyCompactFilteFactory,compactFilteFactoryCreateFilterSess,compactFilteFactoryNameSess}, + {"func", destroyCompactFilteFactory,compactFilteFactoryCreateFilterFunc,compactFilteFactoryNameFunc}, + {"parname", destroyCompactFilteFactory,compactFilteFactoryCreateFilter,compactFilteFactoryName}, + {"partag", destroyCompactFilteFactory,compactFilteFactoryCreateFilter,compactFilteFactoryName}, +}; + + + +// pTaskDb->filterFactory = rocksdb_compactionfilterfactory_create( +// NULL, destroyCompactFilteFactory, compactFilteFactoryCreateFilter, compactFilteFactoryName); +// rocksdb_options_set_compaction_filter_factory(pTaskDb->dbOpt, pTaskDb->filterFactory); +void dbSetFilterFactory(rocksdb_options_t* opt, int32_t i, void *arg) { + rocksdb_compactionfilterfactory_t *filterFactory = rocksdb_compactionfilterfactory_create(arg,destroyCompactFilteFactory, ginitFilterDict[i].create, ginitFilterDict[i].funcName); + rocksdb_options_set_compaction_filter_factory(opt, filterFactory); + +} + bool isValidCheckpoint(const char* dir) { return true; } int32_t copyFiles(const char* src, const char* dst) { @@ -1680,6 +1722,22 @@ const char* compactFilteFactoryName(void* arg) { SCompactFilteFactory* state = arg; return "stream_compact_filter"; } +const char* compactFilteFactoryNameSess(void* arg) { + SCompactFilteFactory* state = arg; + return "stream_compact_filter_sess"; +} +const char* compactFilteFactoryNameState(void* arg) { + SCompactFilteFactory* state = arg; + return "stream_compact_filter_state"; +} +const char* compactFilteFactoryNameFill(void* arg) { + SCompactFilteFactory* state = arg; + return "stream_compact_filter_fill"; +} +const char* compactFilteFactoryNameFunc(void* arg) { + SCompactFilteFactory* state = arg; + return "stream_compact_filter_func"; +} void destroyCompactFilte(void* arg) { (void)arg; } unsigned char compactFilte(void* arg, int level, const char* key, size_t klen, const char* val, size_t vlen, @@ -1688,12 +1746,69 @@ unsigned char compactFilte(void* arg, int level, const char* key, size_t klen, c } const char* compactFilteName(void* arg) { return "stream_filte"; } +unsigned char compactFilteSess(void* arg, int level, const char* key, size_t klen, const char* val, size_t vlen, + char** newval, size_t* newvlen, unsigned char* value_changed) { + // not impl yet + return 0; + //return streamStateValueIsStale((char*)val) ? 1 : 0; +} +const char* compactFilteNameSess(void* arg) { return "stream_filte_sess"; } + +unsigned char compactFilteState(void* arg, int level, const char* key, size_t klen, const char* val, size_t vlen, + char** newval, size_t* newvlen, unsigned char* value_changed) { + // not impl yet + return 0; + //return streamStateValueIsStale((char*)val) ? 1 : 0; +} +const char* compactFilteNameState(void* arg) { return "stream_filte_state"; } + +unsigned char compactFilteFill(void* arg, int level, const char* key, size_t klen, const char* val, size_t vlen, + char** newval, size_t* newvlen, unsigned char* value_changed) { + // not impl yet + return 0; + //return streamStateValueIsStale((char*)val) ? 1 : 0; +} +const char* compactFilteNameFill(void* arg) { return "stream_filte_fill"; } + +unsigned char compactFilteFunc(void* arg, int level, const char* key, size_t klen, const char* val, size_t vlen, + char** newval, size_t* newvlen, unsigned char* value_changed) { + // not impl yet + return 0; + //return streamStateValueIsStale((char*)val) ? 1 : 0; +} +const char* compactFilteNameFunc(void* arg) { return "stream_filte_func"; } + + rocksdb_compactionfilter_t* compactFilteFactoryCreateFilter(void* arg, rocksdb_compactionfiltercontext_t* ctx) { SCompactFilteFactory* state = arg; rocksdb_compactionfilter_t* filter = rocksdb_compactionfilter_create(state, destroyCompactFilte, compactFilte, compactFilteName); return filter; } +rocksdb_compactionfilter_t* compactFilteFactoryCreateFilterSess(void* arg, rocksdb_compactionfiltercontext_t* ctx) { + SCompactFilteFactory* state = arg; + rocksdb_compactionfilter_t* filter = + rocksdb_compactionfilter_create(state, destroyCompactFilte, compactFilteSess, compactFilteNameSess); + return filter; +} +rocksdb_compactionfilter_t* compactFilteFactoryCreateFilterState(void* arg, rocksdb_compactionfiltercontext_t* ctx) { + SCompactFilteFactory* state = arg; + rocksdb_compactionfilter_t* filter = + rocksdb_compactionfilter_create(state, destroyCompactFilte, compactFilteState, compactFilteNameState); + return filter; +} +rocksdb_compactionfilter_t* compactFilteFactoryCreateFilterFill(void* arg, rocksdb_compactionfiltercontext_t* ctx) { + SCompactFilteFactory* state = arg; + rocksdb_compactionfilter_t* filter = + rocksdb_compactionfilter_create(state, destroyCompactFilte, compactFilteFill, compactFilteNameFill); + return filter; +} +rocksdb_compactionfilter_t* compactFilteFactoryCreateFilterFunc(void* arg, rocksdb_compactionfiltercontext_t* ctx) { + SCompactFilteFactory* state = arg; + rocksdb_compactionfilter_t* filter = + rocksdb_compactionfilter_create(state, destroyCompactFilte, compactFilteFunc, compactFilteNameFunc); + return filter; +} void destroyRocksdbCfInst(RocksdbCfInst* inst) { int cfLen = sizeof(ginitDict) / sizeof(ginitDict[0]); @@ -1820,10 +1935,12 @@ void taskDbInitOpt(STaskDbWrapper* pTaskDb) { rocksdb_comparator_t* compare = rocksdb_comparator_create(NULL, cfPara->detroyFunc, cfPara->cmpFunc, cfPara->cmpName); rocksdb_options_set_comparator((rocksdb_options_t*)opt, compare); - + pTaskDb->pCompares[i] = compare; pTaskDb->pCfOpts[i] = opt; pTaskDb->pCfParams[i].tableOpt = tableOpt; + // set filter factory + dbSetFilterFactory(opt, i, NULL); } return; } From aee1b700a6864d4182efc01d9e185afe2959f6c5 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Mon, 6 Nov 2023 18:08:03 +0800 Subject: [PATCH 079/195] Merge branch 'enh/new3.0' into enh/refactorBackend --- source/libs/stream/src/streamBackendRocksdb.c | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index efb9bb599c..60fd32f1a5 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -530,9 +530,6 @@ SCfFilterFactory ginitFilterDict[] = { -// pTaskDb->filterFactory = rocksdb_compactionfilterfactory_create( -// NULL, destroyCompactFilteFactory, compactFilteFactoryCreateFilter, compactFilteFactoryName); -// rocksdb_options_set_compaction_filter_factory(pTaskDb->dbOpt, pTaskDb->filterFactory); void dbSetFilterFactory(rocksdb_options_t* opt, int32_t i, void *arg) { rocksdb_compactionfilterfactory_t *filterFactory = rocksdb_compactionfilterfactory_create(arg,destroyCompactFilteFactory, ginitFilterDict[i].create, ginitFilterDict[i].funcName); rocksdb_options_set_compaction_filter_factory(opt, filterFactory); @@ -1939,6 +1936,7 @@ void taskDbInitOpt(STaskDbWrapper* pTaskDb) { pTaskDb->pCompares[i] = compare; pTaskDb->pCfOpts[i] = opt; pTaskDb->pCfParams[i].tableOpt = tableOpt; + // set filter factory dbSetFilterFactory(opt, i, NULL); } From 9ca1151b6768d41c0e6973b5969aaa32e6924697 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Mon, 6 Nov 2023 19:47:47 +0800 Subject: [PATCH 080/195] refactor checkpoint --- source/libs/stream/src/streamBackendRocksdb.c | 158 +++++++++--------- 1 file changed, 80 insertions(+), 78 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 60fd32f1a5..e674f2b78c 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -20,6 +20,8 @@ #include "tcommon.h" #include "tref.h" +#define GEN_COLUMN_FAMILY_NAME(name, idstr, SUFFIX) sprintf(name, "%s_%s", idstr, (SUFFIX)); + typedef struct SDbChkp { int8_t init; char* pCurrent; @@ -106,7 +108,6 @@ rocksdb_compactionfilter_t* compactFilteFactoryCreateFilterState(void* arg, rock rocksdb_compactionfilter_t* compactFilteFactoryCreateFilterFunc(void* arg, rocksdb_compactionfiltercontext_t* ctx); rocksdb_compactionfilter_t* compactFilteFactoryCreateFilterFill(void* arg, rocksdb_compactionfiltercontext_t* ctx); -const char* cfName[] = {"default", "state", "fill", "sess", "func", "parname", "partag"}; typedef int (*EncodeFunc)(void* key, char* buf); typedef int (*DecodeFunc)(void* key, char* buf); @@ -131,7 +132,18 @@ typedef struct { } SCfInit; -#define GEN_COLUMN_FAMILY_NAME(name, idstr, SUFFIX) sprintf(name, "%s_%s", idstr, (SUFFIX)); +typedef rocksdb_compactionfilter_t* (*CreateFactoryFunc)(void* arg, rocksdb_compactionfiltercontext_t* ctx); +typedef const char* (*FactoryNameFunc)(void* arg); +typedef void(*DestroyFactoryFunc)(void *arg); + +void dbSetFilterFactory(rocksdb_options_t* opt, int32_t i, void *arg); +typedef struct { + void *funcName; + DestroyFactoryFunc destroy; + CreateFactoryFunc create; + FactoryNameFunc factoryName; +} SCfFilterFactory; + const char* compareDefaultName(void* name); const char* compareStateName(void* name); const char* compareWinKeyName(void* name); @@ -180,21 +192,36 @@ int32_t encodeValueFunc(void* value, int32_t vlen, int64_t ttl, char** dest); int32_t decodeValueFunc(void* value, int32_t vlen, int64_t* ttl, char** dest); -void dbChkpDestroy(SDbChkp* pChkp) { - taosMemoryFree(pChkp->buf); - taosMemoryFree(pChkp->path); +static bool streamStateIterSeekAndValid(rocksdb_iterator_t* iter, char* buf, size_t len); - taosArrayDestroyP(pChkp->pSST, taosMemoryFree); - taosArrayDestroyP(pChkp->pAdd, taosMemoryFree); - taosArrayDestroyP(pChkp->pDel, taosMemoryFree); +SCfInit ginitDict[] = { + {"default", 7, 0, defaultKeyComp, defaultKeyEncode, defaultKeyDecode, defaultKeyToString, compareDefaultName, + destroyFunc, encodeValueFunc, decodeValueFunc}, + {"state", 5, 1, stateKeyDBComp, stateKeyEncode, stateKeyDecode, stateKeyToString, compareStateName, destroyFunc, + encodeValueFunc, decodeValueFunc}, + {"fill", 4, 2, winKeyDBComp, winKeyEncode, winKeyDecode, winKeyToString, compareWinKeyName, destroyFunc, + encodeValueFunc, decodeValueFunc}, + {"sess", 4, 3, stateSessionKeyDBComp, stateSessionKeyEncode, stateSessionKeyDecode, stateSessionKeyToString, + compareSessionKeyName, destroyFunc, encodeValueFunc, decodeValueFunc}, + {"func", 4, 4, tupleKeyDBComp, tupleKeyEncode, tupleKeyDecode, tupleKeyToString, compareFuncKeyName, destroyFunc, + encodeValueFunc, decodeValueFunc}, + {"parname", 7, 5, parKeyDBComp, parKeyEncode, parKeyDecode, parKeyToString, compareParKeyName, destroyFunc, + encodeValueFunc, decodeValueFunc}, + {"partag", 6, 6, parKeyDBComp, parKeyEncode, parKeyDecode, parKeyToString, comparePartagKeyName, destroyFunc, + encodeValueFunc, decodeValueFunc}, +}; - taosHashCleanup(pChkp->pSstTbl[0]); - taosHashCleanup(pChkp->pSstTbl[1]); +SCfFilterFactory ginitFilterDict[] = { + {"default", destroyCompactFilteFactory,compactFilteFactoryCreateFilter,compactFilteFactoryName}, + {"state", destroyCompactFilteFactory,compactFilteFactoryCreateFilterState,compactFilteFactoryNameState}, + {"fill", destroyCompactFilteFactory,compactFilteFactoryCreateFilterFill,compactFilteFactoryNameFill}, + {"sess", destroyCompactFilteFactory,compactFilteFactoryCreateFilterSess,compactFilteFactoryNameSess}, + {"func", destroyCompactFilteFactory,compactFilteFactoryCreateFilterFunc,compactFilteFactoryNameFunc}, + {"parname", destroyCompactFilteFactory,compactFilteFactoryCreateFilter,compactFilteFactoryName}, + {"partag", destroyCompactFilteFactory,compactFilteFactoryCreateFilter,compactFilteFactoryName}, +}; - taosMemoryFree(pChkp->pCurrent); - taosMemoryFree(pChkp->pManifest); - -} +const char* cfName[] = {"default", "state", "fill", "sess", "func", "parname", "partag"}; int32_t compareHashTableImpl(SHashObj* p1, SHashObj* p2, SArray* diff) { @@ -335,6 +362,22 @@ SDbChkp* dbChkpCreate(char* path, int64_t initChkpId) { return p; } +void dbChkpDestroy(SDbChkp* pChkp) { + taosMemoryFree(pChkp->buf); + taosMemoryFree(pChkp->path); + + taosArrayDestroyP(pChkp->pSST, taosMemoryFree); + taosArrayDestroyP(pChkp->pAdd, taosMemoryFree); + taosArrayDestroyP(pChkp->pDel, taosMemoryFree); + + taosHashCleanup(pChkp->pSstTbl[0]); + taosHashCleanup(pChkp->pSstTbl[1]); + + taosMemoryFree(pChkp->pCurrent); + taosMemoryFree(pChkp->pManifest); + +} + int32_t dbChkpInit(SDbChkp* p) { if (p == NULL) return 0; return 0; @@ -488,48 +531,6 @@ int32_t bkdMgtDumpTo(SBackendManager* bm, char *taskId, char* dname) { } -SCfInit ginitDict[] = { - {"default", 7, 0, defaultKeyComp, defaultKeyEncode, defaultKeyDecode, defaultKeyToString, compareDefaultName, - destroyFunc, encodeValueFunc, decodeValueFunc}, - {"state", 5, 1, stateKeyDBComp, stateKeyEncode, stateKeyDecode, stateKeyToString, compareStateName, destroyFunc, - encodeValueFunc, decodeValueFunc}, - {"fill", 4, 2, winKeyDBComp, winKeyEncode, winKeyDecode, winKeyToString, compareWinKeyName, destroyFunc, - encodeValueFunc, decodeValueFunc}, - {"sess", 4, 3, stateSessionKeyDBComp, stateSessionKeyEncode, stateSessionKeyDecode, stateSessionKeyToString, - compareSessionKeyName, destroyFunc, encodeValueFunc, decodeValueFunc}, - {"func", 4, 4, tupleKeyDBComp, tupleKeyEncode, tupleKeyDecode, tupleKeyToString, compareFuncKeyName, destroyFunc, - encodeValueFunc, decodeValueFunc}, - {"parname", 7, 5, parKeyDBComp, parKeyEncode, parKeyDecode, parKeyToString, compareParKeyName, destroyFunc, - encodeValueFunc, decodeValueFunc}, - {"partag", 6, 6, parKeyDBComp, parKeyEncode, parKeyDecode, parKeyToString, comparePartagKeyName, destroyFunc, - encodeValueFunc, decodeValueFunc}, -}; - - -typedef rocksdb_compactionfilter_t* (*CreateFactoryFunc)(void* arg, rocksdb_compactionfiltercontext_t* ctx); -typedef const char* (*FactoryNameFunc)(void* arg); -typedef void(*DestroyFactoryFunc)(void *arg); - -typedef struct { - void *funcName; - - DestroyFactoryFunc destroy; - CreateFactoryFunc create; - FactoryNameFunc factoryName; -} SCfFilterFactory; - -SCfFilterFactory ginitFilterDict[] = { - {"default", destroyCompactFilteFactory,compactFilteFactoryCreateFilter,compactFilteFactoryName}, - {"state", destroyCompactFilteFactory,compactFilteFactoryCreateFilterState,compactFilteFactoryNameState}, - {"fill", destroyCompactFilteFactory,compactFilteFactoryCreateFilterFill,compactFilteFactoryNameFill}, - {"sess", destroyCompactFilteFactory,compactFilteFactoryCreateFilterSess,compactFilteFactoryNameSess}, - {"func", destroyCompactFilteFactory,compactFilteFactoryCreateFilterFunc,compactFilteFactoryNameFunc}, - {"parname", destroyCompactFilteFactory,compactFilteFactoryCreateFilter,compactFilteFactoryName}, - {"partag", destroyCompactFilteFactory,compactFilteFactoryCreateFilter,compactFilteFactoryName}, -}; - - - void dbSetFilterFactory(rocksdb_options_t* opt, int32_t i, void *arg) { rocksdb_compactionfilterfactory_t *filterFactory = rocksdb_compactionfilterfactory_create(arg,destroyCompactFilteFactory, ginitFilterDict[i].create, ginitFilterDict[i].funcName); rocksdb_options_set_compaction_filter_factory(opt, filterFactory); @@ -1309,7 +1310,28 @@ void streamBackendDelCompare(void* backend, void* arg) { } } void streamStateDestroy_rocksdb(SStreamState* pState, bool remove) { streamStateCloseBackend(pState, remove); } -static bool streamStateIterSeekAndValid(rocksdb_iterator_t* iter, char* buf, size_t len); +void destroyRocksdbCfInst(RocksdbCfInst* inst) { + int cfLen = sizeof(ginitDict) / sizeof(ginitDict[0]); + if (inst->pHandle) { + for (int i = 0; i < cfLen; i++) { + if (inst->pHandle[i]) rocksdb_column_family_handle_destroy((inst->pHandle)[i]); + } + taosMemoryFree(inst->pHandle); + } + + if (inst->cfOpt) { + for (int i = 0; i < cfLen; i++) { + rocksdb_options_destroy(inst->cfOpt[i]); + rocksdb_block_based_options_destroy(((RocksdbCfParam*)inst->param)[i].tableOpt); + } + taosMemoryFreeClear(inst->cfOpt); + taosMemoryFreeClear(inst->param); + } + if (inst->wOpt) rocksdb_writeoptions_destroy(inst->wOpt); + if (inst->rOpt) rocksdb_readoptions_destroy(inst->rOpt); + + taosMemoryFree(inst); +} // |key|-----value------| // |key|ttl|len|userData| @@ -1736,6 +1758,7 @@ const char* compactFilteFactoryNameFunc(void* arg) { return "stream_compact_filter_func"; } + void destroyCompactFilte(void* arg) { (void)arg; } unsigned char compactFilte(void* arg, int level, const char* key, size_t klen, const char* val, size_t vlen, char** newval, size_t* newvlen, unsigned char* value_changed) { @@ -1807,28 +1830,7 @@ rocksdb_compactionfilter_t* compactFilteFactoryCreateFilterFunc(void* arg, rocks return filter; } -void destroyRocksdbCfInst(RocksdbCfInst* inst) { - int cfLen = sizeof(ginitDict) / sizeof(ginitDict[0]); - if (inst->pHandle) { - for (int i = 0; i < cfLen; i++) { - if (inst->pHandle[i]) rocksdb_column_family_handle_destroy((inst->pHandle)[i]); - } - taosMemoryFree(inst->pHandle); - } - if (inst->cfOpt) { - for (int i = 0; i < cfLen; i++) { - rocksdb_options_destroy(inst->cfOpt[i]); - rocksdb_block_based_options_destroy(((RocksdbCfParam*)inst->param)[i].tableOpt); - } - taosMemoryFreeClear(inst->cfOpt); - taosMemoryFreeClear(inst->param); - } - if (inst->wOpt) rocksdb_writeoptions_destroy(inst->wOpt); - if (inst->rOpt) rocksdb_readoptions_destroy(inst->rOpt); - - taosMemoryFree(inst); -} int32_t getCfIdx(const char* cfName) { int idx = -1; From 819be65db0a06d1fe6200bfcf3b2a6aabf1dd883 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Mon, 6 Nov 2023 19:59:11 +0800 Subject: [PATCH 081/195] refactor checkpoint --- source/libs/stream/src/streamBackendRocksdb.c | 1 - 1 file changed, 1 deletion(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index e674f2b78c..5e00a39417 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -534,7 +534,6 @@ int32_t bkdMgtDumpTo(SBackendManager* bm, char *taskId, char* dname) { void dbSetFilterFactory(rocksdb_options_t* opt, int32_t i, void *arg) { rocksdb_compactionfilterfactory_t *filterFactory = rocksdb_compactionfilterfactory_create(arg,destroyCompactFilteFactory, ginitFilterDict[i].create, ginitFilterDict[i].funcName); rocksdb_options_set_compaction_filter_factory(opt, filterFactory); - } bool isValidCheckpoint(const char* dir) { return true; } From c9bf4f65071102ab682ea9a7bfb93d1766a62015 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Mon, 6 Nov 2023 20:19:11 +0800 Subject: [PATCH 082/195] refactor checkpoint --- source/libs/stream/src/streamBackendRocksdb.c | 93 ++++++++++--------- 1 file changed, 47 insertions(+), 46 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 5e00a39417..c3ab5ed3ee 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -91,6 +91,7 @@ uint32_t nextPow2(uint32_t x); int32_t streamStateOpenBackendCf(void* backend, char* name, char** cfs, int32_t nCf); void destroyRocksdbCfInst(RocksdbCfInst* inst); +int32_t getCfIdx(const char* cfName); void destroyCompactFilteFactory(void* arg); void destroyCompactFilte(void* arg); @@ -117,6 +118,10 @@ typedef int (*BackendCmpFunc)(void* state, const char* aBuf, size_t aLen, const typedef void (*DestroyFunc)(void* state); typedef int32_t (*EncodeValueFunc)(void* value, int32_t vlen, int64_t ttl, char** dest); typedef int32_t (*DecodeValueFunc)(void* value, int32_t vlen, int64_t* ttl, char** dest); + +typedef rocksdb_compactionfilter_t* (*CreateFactoryFunc)(void* arg, rocksdb_compactionfiltercontext_t* ctx); +typedef const char* (*FactoryNameFunc)(void* arg); +typedef void(*DestroyFactoryFunc)(void *arg); typedef struct { const char* key; int32_t len; @@ -130,11 +135,12 @@ typedef struct { EncodeValueFunc enValueFunc; DecodeValueFunc deValueFunc; + CreateFactoryFunc createFilter; + DestroyFactoryFunc destroyFilter; + FactoryNameFunc funcName; + } SCfInit; -typedef rocksdb_compactionfilter_t* (*CreateFactoryFunc)(void* arg, rocksdb_compactionfiltercontext_t* ctx); -typedef const char* (*FactoryNameFunc)(void* arg); -typedef void(*DestroyFactoryFunc)(void *arg); void dbSetFilterFactory(rocksdb_options_t* opt, int32_t i, void *arg); typedef struct { @@ -193,36 +199,45 @@ int32_t decodeValueFunc(void* value, int32_t vlen, int64_t* ttl, char** dest); static bool streamStateIterSeekAndValid(rocksdb_iterator_t* iter, char* buf, size_t len); +static rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfName, + rocksdb_snapshot_t** snapshot, rocksdb_readoptions_t** readOpt); SCfInit ginitDict[] = { {"default", 7, 0, defaultKeyComp, defaultKeyEncode, defaultKeyDecode, defaultKeyToString, compareDefaultName, - destroyFunc, encodeValueFunc, decodeValueFunc}, - {"state", 5, 1, stateKeyDBComp, stateKeyEncode, stateKeyDecode, stateKeyToString, compareStateName, destroyFunc, - encodeValueFunc, decodeValueFunc}, - {"fill", 4, 2, winKeyDBComp, winKeyEncode, winKeyDecode, winKeyToString, compareWinKeyName, destroyFunc, - encodeValueFunc, decodeValueFunc}, - {"sess", 4, 3, stateSessionKeyDBComp, stateSessionKeyEncode, stateSessionKeyDecode, stateSessionKeyToString, - compareSessionKeyName, destroyFunc, encodeValueFunc, decodeValueFunc}, - {"func", 4, 4, tupleKeyDBComp, tupleKeyEncode, tupleKeyDecode, tupleKeyToString, compareFuncKeyName, destroyFunc, - encodeValueFunc, decodeValueFunc}, - {"parname", 7, 5, parKeyDBComp, parKeyEncode, parKeyDecode, parKeyToString, compareParKeyName, destroyFunc, - encodeValueFunc, decodeValueFunc}, - {"partag", 6, 6, parKeyDBComp, parKeyEncode, parKeyDecode, parKeyToString, comparePartagKeyName, destroyFunc, - encodeValueFunc, decodeValueFunc}, -}; + destroyFunc, encodeValueFunc, decodeValueFunc, compactFilteFactoryCreateFilter, destroyCompactFilteFactory, compactFilteFactoryName}, -SCfFilterFactory ginitFilterDict[] = { - {"default", destroyCompactFilteFactory,compactFilteFactoryCreateFilter,compactFilteFactoryName}, - {"state", destroyCompactFilteFactory,compactFilteFactoryCreateFilterState,compactFilteFactoryNameState}, - {"fill", destroyCompactFilteFactory,compactFilteFactoryCreateFilterFill,compactFilteFactoryNameFill}, - {"sess", destroyCompactFilteFactory,compactFilteFactoryCreateFilterSess,compactFilteFactoryNameSess}, - {"func", destroyCompactFilteFactory,compactFilteFactoryCreateFilterFunc,compactFilteFactoryNameFunc}, - {"parname", destroyCompactFilteFactory,compactFilteFactoryCreateFilter,compactFilteFactoryName}, - {"partag", destroyCompactFilteFactory,compactFilteFactoryCreateFilter,compactFilteFactoryName}, + {"state", 5, 1, stateKeyDBComp, stateKeyEncode, stateKeyDecode, stateKeyToString, compareStateName, destroyFunc, + encodeValueFunc, decodeValueFunc,compactFilteFactoryCreateFilterState, destroyCompactFilteFactory, compactFilteFactoryNameState}, + + {"fill", 4, 2, winKeyDBComp, winKeyEncode, winKeyDecode, winKeyToString, compareWinKeyName, destroyFunc, + encodeValueFunc, decodeValueFunc,compactFilteFactoryCreateFilterFill, destroyCompactFilteFactory,compactFilteFactoryNameFill}, + + {"sess", 4, 3, stateSessionKeyDBComp, stateSessionKeyEncode, stateSessionKeyDecode, stateSessionKeyToString, + compareSessionKeyName, destroyFunc, encodeValueFunc, decodeValueFunc, compactFilteFactoryCreateFilterSess, destroyCompactFilteFactory,compactFilteFactoryNameSess}, + + {"func", 4, 4, tupleKeyDBComp, tupleKeyEncode, tupleKeyDecode, tupleKeyToString, compareFuncKeyName, destroyFunc, + encodeValueFunc, decodeValueFunc, compactFilteFactoryCreateFilterFunc, destroyCompactFilteFactory, compactFilteFactoryNameFunc}, + + {"parname", 7, 5, parKeyDBComp, parKeyEncode, parKeyDecode, parKeyToString, compareParKeyName, destroyFunc, + encodeValueFunc, decodeValueFunc,compactFilteFactoryCreateFilter, destroyCompactFilteFactory, compactFilteFactoryName}, + + {"partag", 6, 6, parKeyDBComp, parKeyEncode, parKeyDecode, parKeyToString, comparePartagKeyName, destroyFunc, + encodeValueFunc, decodeValueFunc, compactFilteFactoryCreateFilter, destroyCompactFilteFactory, compactFilteFactoryName}, }; const char* cfName[] = {"default", "state", "fill", "sess", "func", "parname", "partag"}; +int32_t getCfIdx(const char* cfName) { + int idx = -1; + size_t len = strlen(cfName); + for (int i = 0; i < sizeof(ginitDict) / sizeof(ginitDict[0]); i++) { + if (len == ginitDict[i].len && strncmp(cfName, ginitDict[i].key, strlen(cfName)) == 0) { + idx = i; + break; + } + } + return idx; +} int32_t compareHashTableImpl(SHashObj* p1, SHashObj* p2, SArray* diff) { int32_t code = 0; @@ -383,7 +398,6 @@ int32_t dbChkpInit(SDbChkp* p) { return 0; } int32_t dbChkpDumpTo(SDbChkp* p, char* dname) { - taosThreadRwlockRdlock(&p->rwLock); int32_t code = 0; int32_t len = p->len + 128; @@ -532,7 +546,7 @@ int32_t bkdMgtDumpTo(SBackendManager* bm, char *taskId, char* dname) { } void dbSetFilterFactory(rocksdb_options_t* opt, int32_t i, void *arg) { - rocksdb_compactionfilterfactory_t *filterFactory = rocksdb_compactionfilterfactory_create(arg,destroyCompactFilteFactory, ginitFilterDict[i].create, ginitFilterDict[i].funcName); + rocksdb_compactionfilterfactory_t *filterFactory = rocksdb_compactionfilterfactory_create(arg,ginitDict[i].destroyFilter, ginitDict[i].createFilter, ginitDict[i].funcName); rocksdb_options_set_compaction_filter_factory(opt, filterFactory); } @@ -1011,7 +1025,7 @@ int32_t chkpMayDelObsolete(void* arg, int64_t chkpId, char* path) { return 0; } -static int32_t compareCheckpoint(const void* a, const void* b) { +static int32_t chkpIdComp(const void* a, const void* b) { int64_t x = *(int64_t*)a; int64_t y = *(int64_t*)b; return x < y ? -1 : 1; @@ -1056,7 +1070,7 @@ int32_t streamBackendLoadCheckpointInfo(void* arg) { continue; } } - taosArraySort(suffix, compareCheckpoint); + taosArraySort(suffix, chkpIdComp); // free previous chkpSaved taosArrayClear(pMeta->chkpSaved); for (int i = 0; i < taosArrayGetSize(suffix); i++) { @@ -1335,8 +1349,6 @@ void destroyRocksdbCfInst(RocksdbCfInst* inst) { // |key|-----value------| // |key|ttl|len|userData| -static rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfName, - rocksdb_snapshot_t** snapshot, rocksdb_readoptions_t** readOpt); int defaultKeyComp(void* state, const char* aBuf, size_t aLen, const char* bBuf, size_t bLen) { int len = aLen < bLen ? aLen : bLen; @@ -1764,6 +1776,10 @@ unsigned char compactFilte(void* arg, int level, const char* key, size_t klen, c return streamStateValueIsStale((char*)val) ? 1 : 0; } const char* compactFilteName(void* arg) { return "stream_filte"; } +const char* compactFilteNameSess(void* arg) { return "stream_filte_sess"; } +const char* compactFilteNameState(void* arg) { return "stream_filte_state"; } +const char* compactFilteNameFill(void* arg) { return "stream_filte_fill"; } +const char* compactFilteNameFunc(void* arg) { return "stream_filte_func"; } unsigned char compactFilteSess(void* arg, int level, const char* key, size_t klen, const char* val, size_t vlen, char** newval, size_t* newvlen, unsigned char* value_changed) { @@ -1771,7 +1787,6 @@ unsigned char compactFilteSess(void* arg, int level, const char* key, size_t kle return 0; //return streamStateValueIsStale((char*)val) ? 1 : 0; } -const char* compactFilteNameSess(void* arg) { return "stream_filte_sess"; } unsigned char compactFilteState(void* arg, int level, const char* key, size_t klen, const char* val, size_t vlen, char** newval, size_t* newvlen, unsigned char* value_changed) { @@ -1779,7 +1794,6 @@ unsigned char compactFilteState(void* arg, int level, const char* key, size_t kl return 0; //return streamStateValueIsStale((char*)val) ? 1 : 0; } -const char* compactFilteNameState(void* arg) { return "stream_filte_state"; } unsigned char compactFilteFill(void* arg, int level, const char* key, size_t klen, const char* val, size_t vlen, char** newval, size_t* newvlen, unsigned char* value_changed) { @@ -1787,7 +1801,6 @@ unsigned char compactFilteFill(void* arg, int level, const char* key, size_t kle return 0; //return streamStateValueIsStale((char*)val) ? 1 : 0; } -const char* compactFilteNameFill(void* arg) { return "stream_filte_fill"; } unsigned char compactFilteFunc(void* arg, int level, const char* key, size_t klen, const char* val, size_t vlen, char** newval, size_t* newvlen, unsigned char* value_changed) { @@ -1795,7 +1808,6 @@ unsigned char compactFilteFunc(void* arg, int level, const char* key, size_t kle return 0; //return streamStateValueIsStale((char*)val) ? 1 : 0; } -const char* compactFilteNameFunc(void* arg) { return "stream_filte_func"; } rocksdb_compactionfilter_t* compactFilteFactoryCreateFilter(void* arg, rocksdb_compactionfiltercontext_t* ctx) { @@ -1831,17 +1843,6 @@ rocksdb_compactionfilter_t* compactFilteFactoryCreateFilterFunc(void* arg, rocks -int32_t getCfIdx(const char* cfName) { - int idx = -1; - size_t len = strlen(cfName); - for (int i = 0; i < sizeof(ginitDict) / sizeof(ginitDict[0]); i++) { - if (len == ginitDict[i].len && strncmp(cfName, ginitDict[i].key, strlen(cfName)) == 0) { - idx = i; - break; - } - } - return idx; -} int32_t taskDbOpenCfs(STaskDbWrapper* pTask, char* path, char** pCfNames, int32_t nCf) { int32_t code = -1; char* err = NULL; From 021859361f11f59309f2cd62f6854a22c9482320 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Tue, 7 Nov 2023 09:22:57 +0800 Subject: [PATCH 083/195] refactor checkpoint --- source/libs/stream/src/streamBackendRocksdb.c | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index c3ab5ed3ee..82de4fb933 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -100,7 +100,15 @@ const char* compactFilteFactoryNameSess(void* arg); const char* compactFilteFactoryNameState(void* arg); const char* compactFilteFactoryNameFunc(void* arg); const char* compactFilteFactoryNameFill(void* arg); -const char* compactFilteName(void* arg); + +const char* compactFilteName(void* arg); +const char* compactFilteNameSess(void* arg); +const char* compactFilteNameState(void* arg); +const char* compactFilteNameFill(void* arg); +const char* compactFilteNameFunc(void* arg); + + + unsigned char compactFilte(void* arg, int level, const char* key, size_t klen, const char* val, size_t vlen, char** newval, size_t* newvlen, unsigned char* value_changed); rocksdb_compactionfilter_t* compactFilteFactoryCreateFilter(void* arg, rocksdb_compactionfiltercontext_t* ctx); From 95cfc5eb4625fc378914b0ab14cc2456e76f2740 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Tue, 7 Nov 2023 09:55:55 +0800 Subject: [PATCH 084/195] refactor checkpoint --- source/libs/stream/src/streamBackendRocksdb.c | 272 +++++++++--------- 1 file changed, 132 insertions(+), 140 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 82de4fb933..c6dafc4e41 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -41,7 +41,7 @@ typedef struct SDbChkp { SArray* pAdd; SArray* pDel; int8_t update; - + TdThreadRwlock rwLock; } SDbChkp; typedef struct { @@ -64,7 +64,7 @@ typedef struct { SArray* pDel; int8_t update; - SHashObj *pDbChkpTbl; + SHashObj* pDbChkpTbl; TdThreadRwlock rwLock; } SBackendManager; @@ -90,16 +90,16 @@ uint32_t nextPow2(uint32_t x); int32_t streamStateOpenBackendCf(void* backend, char* name, char** cfs, int32_t nCf); -void destroyRocksdbCfInst(RocksdbCfInst* inst); +void destroyRocksdbCfInst(RocksdbCfInst* inst); int32_t getCfIdx(const char* cfName); -void destroyCompactFilteFactory(void* arg); -void destroyCompactFilte(void* arg); -const char* compactFilteFactoryName(void* arg); -const char* compactFilteFactoryNameSess(void* arg); -const char* compactFilteFactoryNameState(void* arg); -const char* compactFilteFactoryNameFunc(void* arg); -const char* compactFilteFactoryNameFill(void* arg); +void destroyCompactFilteFactory(void* arg); +void destroyCompactFilte(void* arg); +const char* compactFilteFactoryName(void* arg); +const char* compactFilteFactoryNameSess(void* arg); +const char* compactFilteFactoryNameState(void* arg); +const char* compactFilteFactoryNameFunc(void* arg); +const char* compactFilteFactoryNameFill(void* arg); const char* compactFilteName(void* arg); const char* compactFilteNameSess(void* arg); @@ -107,8 +107,6 @@ const char* compactFilteNameState(void* arg); const char* compactFilteNameFill(void* arg); const char* compactFilteNameFunc(void* arg); - - unsigned char compactFilte(void* arg, int level, const char* key, size_t klen, const char* val, size_t vlen, char** newval, size_t* newvlen, unsigned char* value_changed); rocksdb_compactionfilter_t* compactFilteFactoryCreateFilter(void* arg, rocksdb_compactionfiltercontext_t* ctx); @@ -117,7 +115,6 @@ rocksdb_compactionfilter_t* compactFilteFactoryCreateFilterState(void* arg, rock rocksdb_compactionfilter_t* compactFilteFactoryCreateFilterFunc(void* arg, rocksdb_compactionfiltercontext_t* ctx); rocksdb_compactionfilter_t* compactFilteFactoryCreateFilterFill(void* arg, rocksdb_compactionfiltercontext_t* ctx); - typedef int (*EncodeFunc)(void* key, char* buf); typedef int (*DecodeFunc)(void* key, char* buf); typedef int (*ToStringFunc)(void* key, char* buf); @@ -129,7 +126,7 @@ typedef int32_t (*DecodeValueFunc)(void* value, int32_t vlen, int64_t* ttl, char typedef rocksdb_compactionfilter_t* (*CreateFactoryFunc)(void* arg, rocksdb_compactionfiltercontext_t* ctx); typedef const char* (*FactoryNameFunc)(void* arg); -typedef void(*DestroyFactoryFunc)(void *arg); +typedef void (*DestroyFactoryFunc)(void* arg); typedef struct { const char* key; int32_t len; @@ -143,19 +140,18 @@ typedef struct { EncodeValueFunc enValueFunc; DecodeValueFunc deValueFunc; - CreateFactoryFunc createFilter; + CreateFactoryFunc createFilter; DestroyFactoryFunc destroyFilter; FactoryNameFunc funcName; } SCfInit; - -void dbSetFilterFactory(rocksdb_options_t* opt, int32_t i, void *arg); +void dbSetFilterFactory(rocksdb_options_t* opt, int32_t i, void* arg); typedef struct { - void *funcName; + void* funcName; DestroyFactoryFunc destroy; - CreateFactoryFunc create; - FactoryNameFunc factoryName; + CreateFactoryFunc create; + FactoryNameFunc factoryName; } SCfFilterFactory; const char* compareDefaultName(void* name); @@ -205,32 +201,38 @@ void destroyFunc(void* arg); int32_t encodeValueFunc(void* value, int32_t vlen, int64_t ttl, char** dest); int32_t decodeValueFunc(void* value, int32_t vlen, int64_t* ttl, char** dest); - -static bool streamStateIterSeekAndValid(rocksdb_iterator_t* iter, char* buf, size_t len); +static bool streamStateIterSeekAndValid(rocksdb_iterator_t* iter, char* buf, size_t len); static rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfName, rocksdb_snapshot_t** snapshot, rocksdb_readoptions_t** readOpt); SCfInit ginitDict[] = { {"default", 7, 0, defaultKeyComp, defaultKeyEncode, defaultKeyDecode, defaultKeyToString, compareDefaultName, - destroyFunc, encodeValueFunc, decodeValueFunc, compactFilteFactoryCreateFilter, destroyCompactFilteFactory, compactFilteFactoryName}, + destroyFunc, encodeValueFunc, decodeValueFunc, compactFilteFactoryCreateFilter, destroyCompactFilteFactory, + compactFilteFactoryName}, {"state", 5, 1, stateKeyDBComp, stateKeyEncode, stateKeyDecode, stateKeyToString, compareStateName, destroyFunc, - encodeValueFunc, decodeValueFunc,compactFilteFactoryCreateFilterState, destroyCompactFilteFactory, compactFilteFactoryNameState}, + encodeValueFunc, decodeValueFunc, compactFilteFactoryCreateFilterState, destroyCompactFilteFactory, + compactFilteFactoryNameState}, {"fill", 4, 2, winKeyDBComp, winKeyEncode, winKeyDecode, winKeyToString, compareWinKeyName, destroyFunc, - encodeValueFunc, decodeValueFunc,compactFilteFactoryCreateFilterFill, destroyCompactFilteFactory,compactFilteFactoryNameFill}, + encodeValueFunc, decodeValueFunc, compactFilteFactoryCreateFilterFill, destroyCompactFilteFactory, + compactFilteFactoryNameFill}, {"sess", 4, 3, stateSessionKeyDBComp, stateSessionKeyEncode, stateSessionKeyDecode, stateSessionKeyToString, - compareSessionKeyName, destroyFunc, encodeValueFunc, decodeValueFunc, compactFilteFactoryCreateFilterSess, destroyCompactFilteFactory,compactFilteFactoryNameSess}, + compareSessionKeyName, destroyFunc, encodeValueFunc, decodeValueFunc, compactFilteFactoryCreateFilterSess, + destroyCompactFilteFactory, compactFilteFactoryNameSess}, {"func", 4, 4, tupleKeyDBComp, tupleKeyEncode, tupleKeyDecode, tupleKeyToString, compareFuncKeyName, destroyFunc, - encodeValueFunc, decodeValueFunc, compactFilteFactoryCreateFilterFunc, destroyCompactFilteFactory, compactFilteFactoryNameFunc}, + encodeValueFunc, decodeValueFunc, compactFilteFactoryCreateFilterFunc, destroyCompactFilteFactory, + compactFilteFactoryNameFunc}, {"parname", 7, 5, parKeyDBComp, parKeyEncode, parKeyDecode, parKeyToString, compareParKeyName, destroyFunc, - encodeValueFunc, decodeValueFunc,compactFilteFactoryCreateFilter, destroyCompactFilteFactory, compactFilteFactoryName}, + encodeValueFunc, decodeValueFunc, compactFilteFactoryCreateFilter, destroyCompactFilteFactory, + compactFilteFactoryName}, {"partag", 6, 6, parKeyDBComp, parKeyEncode, parKeyDecode, parKeyToString, comparePartagKeyName, destroyFunc, - encodeValueFunc, decodeValueFunc, compactFilteFactoryCreateFilter, destroyCompactFilteFactory, compactFilteFactoryName}, + encodeValueFunc, decodeValueFunc, compactFilteFactoryCreateFilter, destroyCompactFilteFactory, + compactFilteFactoryName}, }; const char* cfName[] = {"default", "state", "fill", "sess", "func", "parname", "partag"}; @@ -270,7 +272,7 @@ int32_t compareHashTable(SHashObj* p1, SHashObj* p2, SArray* add, SArray* del) { return code; } -int32_t dbChkpGetDelta(SDbChkp* p, int64_t chkpId, SArray *list) { +int32_t dbChkpGetDelta(SDbChkp* p, int64_t chkpId, SArray* list) { taosThreadRwlockWrlock(&p->rwLock); p->preCkptId = p->curChkpId; @@ -316,7 +318,6 @@ int32_t dbChkpGetDelta(SDbChkp* p, int64_t chkpId, SArray *list) { } } if (p->init == 0) { - void* pIter = taosHashIterate(p->pSstTbl[1 - p->idx], NULL); while (pIter) { size_t len; @@ -351,7 +352,6 @@ int32_t dbChkpGetDelta(SDbChkp* p, int64_t chkpId, SArray *list) { p->curChkpId = chkpId; } - taosHashClear(p->pSstTbl[p->idx]); p->idx = 1 - p->idx; @@ -359,10 +359,9 @@ int32_t dbChkpGetDelta(SDbChkp* p, int64_t chkpId, SArray *list) { taosThreadRwlockUnlock(&p->rwLock); return 0; - } SDbChkp* dbChkpCreate(char* path, int64_t initChkpId) { - SDbChkp *p = taosMemoryCalloc(1, sizeof(SDbChkp)); + SDbChkp* p = taosMemoryCalloc(1, sizeof(SDbChkp)); p->curChkpId = initChkpId; p->preCkptId = -1; p->pSST = taosArrayInit(64, sizeof(void*)); @@ -379,7 +378,7 @@ SDbChkp* dbChkpCreate(char* path, int64_t initChkpId) { p->update = 0; taosThreadRwlockInit(&p->rwLock, NULL); - SArray *list = NULL; + SArray* list = NULL; int32_t code = dbChkpGetDelta(p, initChkpId, list); return p; @@ -398,11 +397,10 @@ void dbChkpDestroy(SDbChkp* pChkp) { taosMemoryFree(pChkp->pCurrent); taosMemoryFree(pChkp->pManifest); - } int32_t dbChkpInit(SDbChkp* p) { - if (p == NULL) return 0; + if (p == NULL) return 0; return 0; } int32_t dbChkpDumpTo(SDbChkp* p, char* dname) { @@ -487,7 +485,6 @@ _ERROR: taosMemoryFree(srcDir); taosMemoryFree(dstDir); return code; - } SBackendManager* bkdMgtCreate(char* path) { SBackendManager* p = taosMemoryCalloc(1, sizeof(SBackendManager)); @@ -498,9 +495,9 @@ SBackendManager* bkdMgtCreate(char* path) { void bkdMgtDestroy(SBackendManager* bm) { if (bm == NULL) return; - void *pIter = taosHashIterate(bm->pDbChkpTbl, NULL); + void* pIter = taosHashIterate(bm->pDbChkpTbl, NULL); while (pIter) { - SDbChkp *pChkp = *(SDbChkp **)(pIter); + SDbChkp* pChkp = *(SDbChkp**)(pIter); dbChkpDestroy(pChkp); pIter = taosHashIterate(bm->pDbChkpTbl, pIter); @@ -510,52 +507,52 @@ void bkdMgtDestroy(SBackendManager* bm) { taosMemoryFree(bm); } -int32_t bkdMgtGetDelta(SBackendManager* bm, char *taskId, int64_t chkpId, SArray* list) { +int32_t bkdMgtGetDelta(SBackendManager* bm, char* taskId, int64_t chkpId, SArray* list) { int32_t code = 0; taosThreadRwlockWrlock(&bm->rwLock); - SDbChkp *pChkp = taosHashGet(bm->pDbChkpTbl, taskId, strlen(taskId)); + SDbChkp* pChkp = taosHashGet(bm->pDbChkpTbl, taskId, strlen(taskId)); code = dbChkpGetDelta(pChkp, chkpId, list); taosThreadRwlockUnlock(&bm->rwLock); - return code ; + return code; } -int32_t bkdMgtAddChkp(SBackendManager *bm, char *task, char *path) { +int32_t bkdMgtAddChkp(SBackendManager* bm, char* task, char* path) { int32_t code = -1; taosThreadRwlockWrlock(&bm->rwLock); - SDbChkp **pp = taosHashGet(bm->pDbChkpTbl, task, strlen(task)); + SDbChkp** pp = taosHashGet(bm->pDbChkpTbl, task, strlen(task)); if (pp == NULL) { - SDbChkp *p = dbChkpCreate(path, 0); + SDbChkp* p = dbChkpCreate(path, 0); if (p != NULL) { - taosHashPut(bm->pDbChkpTbl, task, strlen(task), &p, sizeof(void *)); + taosHashPut(bm->pDbChkpTbl, task, strlen(task), &p, sizeof(void*)); code = 0; } } else { stError("task chkp already exists"); } - + taosThreadRwlockUnlock(&bm->rwLock); - - return code; + + return code; } -int32_t bkdMgtDumpTo(SBackendManager* bm, char *taskId, char* dname) { +int32_t bkdMgtDumpTo(SBackendManager* bm, char* taskId, char* dname) { int32_t code = 0; taosThreadRwlockRdlock(&bm->rwLock); - SDbChkp *p = taosHashGet(bm->pDbChkpTbl, taskId, strlen(taskId)); - code = dbChkpDumpTo(p, dname); + SDbChkp* p = taosHashGet(bm->pDbChkpTbl, taskId, strlen(taskId)); + code = dbChkpDumpTo(p, dname); taosThreadRwlockUnlock(&bm->rwLock); return code; - } -void dbSetFilterFactory(rocksdb_options_t* opt, int32_t i, void *arg) { - rocksdb_compactionfilterfactory_t *filterFactory = rocksdb_compactionfilterfactory_create(arg,ginitDict[i].destroyFilter, ginitDict[i].createFilter, ginitDict[i].funcName); - rocksdb_options_set_compaction_filter_factory(opt, filterFactory); +void dbSetFilterFactory(rocksdb_options_t* opt, int32_t i, void* arg) { + rocksdb_compactionfilterfactory_t* filterFactory = rocksdb_compactionfilterfactory_create( + arg, ginitDict[i].destroyFilter, ginitDict[i].createFilter, ginitDict[i].funcName); + rocksdb_options_set_compaction_filter_factory(opt, filterFactory); } bool isValidCheckpoint(const char* dir) { return true; } @@ -1330,7 +1327,7 @@ void streamBackendDelCompare(void* backend, void* arg) { taosMemoryFree(node); } } -void streamStateDestroy_rocksdb(SStreamState* pState, bool remove) { streamStateCloseBackend(pState, remove); } +void streamStateDestroy_rocksdb(SStreamState* pState, bool remove) { streamStateCloseBackend(pState, remove); } void destroyRocksdbCfInst(RocksdbCfInst* inst) { int cfLen = sizeof(ginitDict) / sizeof(ginitDict[0]); if (inst->pHandle) { @@ -1357,7 +1354,6 @@ void destroyRocksdbCfInst(RocksdbCfInst* inst) { // |key|-----value------| // |key|ttl|len|userData| - int defaultKeyComp(void* state, const char* aBuf, size_t aLen, const char* bBuf, size_t bLen) { int len = aLen < bLen ? aLen : bLen; int ret = memcmp(aBuf, bBuf, len); @@ -1777,7 +1773,6 @@ const char* compactFilteFactoryNameFunc(void* arg) { return "stream_compact_filter_func"; } - void destroyCompactFilte(void* arg) { (void)arg; } unsigned char compactFilte(void* arg, int level, const char* key, size_t klen, const char* val, size_t vlen, char** newval, size_t* newvlen, unsigned char* value_changed) { @@ -1790,34 +1785,33 @@ const char* compactFilteNameFill(void* arg) { return "stream_filte_fill"; } const char* compactFilteNameFunc(void* arg) { return "stream_filte_func"; } unsigned char compactFilteSess(void* arg, int level, const char* key, size_t klen, const char* val, size_t vlen, - char** newval, size_t* newvlen, unsigned char* value_changed) { + char** newval, size_t* newvlen, unsigned char* value_changed) { // not impl yet return 0; - //return streamStateValueIsStale((char*)val) ? 1 : 0; + // return streamStateValueIsStale((char*)val) ? 1 : 0; } unsigned char compactFilteState(void* arg, int level, const char* key, size_t klen, const char* val, size_t vlen, - char** newval, size_t* newvlen, unsigned char* value_changed) { + char** newval, size_t* newvlen, unsigned char* value_changed) { // not impl yet return 0; - //return streamStateValueIsStale((char*)val) ? 1 : 0; + // return streamStateValueIsStale((char*)val) ? 1 : 0; } unsigned char compactFilteFill(void* arg, int level, const char* key, size_t klen, const char* val, size_t vlen, - char** newval, size_t* newvlen, unsigned char* value_changed) { + char** newval, size_t* newvlen, unsigned char* value_changed) { // not impl yet return 0; - //return streamStateValueIsStale((char*)val) ? 1 : 0; + // return streamStateValueIsStale((char*)val) ? 1 : 0; } unsigned char compactFilteFunc(void* arg, int level, const char* key, size_t klen, const char* val, size_t vlen, - char** newval, size_t* newvlen, unsigned char* value_changed) { + char** newval, size_t* newvlen, unsigned char* value_changed) { // not impl yet return 0; - //return streamStateValueIsStale((char*)val) ? 1 : 0; + // return streamStateValueIsStale((char*)val) ? 1 : 0; } - rocksdb_compactionfilter_t* compactFilteFactoryCreateFilter(void* arg, rocksdb_compactionfiltercontext_t* ctx) { SCompactFilteFactory* state = arg; rocksdb_compactionfilter_t* filter = @@ -1849,8 +1843,6 @@ rocksdb_compactionfilter_t* compactFilteFactoryCreateFilterFunc(void* arg, rocks return filter; } - - int32_t taskDbOpenCfs(STaskDbWrapper* pTask, char* path, char** pCfNames, int32_t nCf) { int32_t code = -1; char* err = NULL; @@ -1942,7 +1934,7 @@ void taskDbInitOpt(STaskDbWrapper* pTaskDb) { rocksdb_comparator_t* compare = rocksdb_comparator_create(NULL, cfPara->detroyFunc, cfPara->cmpFunc, cfPara->cmpName); rocksdb_options_set_comparator((rocksdb_options_t*)opt, compare); - + pTaskDb->pCompares[i] = compare; pTaskDb->pCfOpts[i] = opt; pTaskDb->pCfParams[i].tableOpt = tableOpt; @@ -2065,7 +2057,7 @@ _EXIT: void taskDbDestroy(void* pBackend) { STaskDbWrapper* wrapper = pBackend; qDebug("succ to destroy stream backend:%p", wrapper); - + int8_t nCf = sizeof(ginitDict) / sizeof(ginitDict[0]); if (wrapper == NULL) return; @@ -2074,7 +2066,7 @@ void taskDbDestroy(void* pBackend) { rocksdb_flushoptions_t* flushOpt = rocksdb_flushoptions_create(); rocksdb_flushoptions_set_wait(flushOpt, 1); - char* err = NULL; + char* err = NULL; for (int i = 0; i < nCf; i++) { if (wrapper->pCf[i] != NULL) rocksdb_flush_cf(wrapper->db, flushOpt, wrapper->pCf[i], &err); if (err != NULL) { @@ -2096,18 +2088,16 @@ void taskDbDestroy(void* pBackend) { rocksdb_env_destroy(wrapper->env); rocksdb_cache_destroy(wrapper->cache); - taosMemoryFree(wrapper->pCf); for (int i = 0; i < nCf; i++) { - rocksdb_options_t *opt = wrapper->pCfOpts[i]; - rocksdb_comparator_t *compare = wrapper->pCompares[i]; - rocksdb_block_based_table_options_t *tblOpt = wrapper->pCfParams[i].tableOpt; + rocksdb_options_t* opt = wrapper->pCfOpts[i]; + rocksdb_comparator_t* compare = wrapper->pCompares[i]; + rocksdb_block_based_table_options_t* tblOpt = wrapper->pCfParams[i].tableOpt; rocksdb_options_destroy(opt); rocksdb_comparator_destroy(compare); rocksdb_block_based_options_destroy(tblOpt); - } taosMemoryFree(wrapper->pCompares); taosMemoryFree(wrapper->pCfOpts); @@ -2527,7 +2517,39 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfKe return rocksdb_create_iterator_cf(wrapper->db, *readOpt, ((rocksdb_column_family_handle_t**)wrapper->pCf)[idx]); } -#define STREAM_STATE_PUT_ROCKSDB(pState, funcname, key, value, vLen) \ +#define STREAM_STATE_PUT_ROCKSDB(pState, funcname, key, value, vLen) \ + do { \ + code = 0; \ + char buf[128] = {0}; \ + char* err = NULL; \ + int i = streamStateGetCfIdx(pState, funcname); \ + if (i < 0) { \ + qWarn("streamState failed to get cf name: %s", funcname); \ + code = -1; \ + break; \ + } \ + STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; \ + char toString[128] = {0}; \ + if (qDebugFlag & DEBUG_TRACE) ginitDict[i].toStrFunc((void*)key, toString); \ + int32_t klen = ginitDict[i].enFunc((void*)key, buf); \ + rocksdb_column_family_handle_t* pHandle = ((rocksdb_column_family_handle_t**)wrapper->pCf)[ginitDict[i].idx]; \ + rocksdb_writeoptions_t* opts = wrapper->writeOpt; \ + rocksdb_t* db = wrapper->db; \ + char* ttlV = NULL; \ + int32_t ttlVLen = ginitDict[i].enValueFunc((char*)value, vLen, 0, &ttlV); \ + rocksdb_put_cf(db, opts, pHandle, (const char*)buf, klen, (const char*)ttlV, (size_t)ttlVLen, &err); \ + if (err != NULL) { \ + stError("streamState str: %s failed to write to %s, err: %s", toString, funcname, err); \ + taosMemoryFree(err); \ + code = -1; \ + } else { \ + qTrace("streamState str:%s succ to write to %s, rowValLen:%d, ttlValLen:%d, %p", toString, funcname, vLen, \ + ttlVLen, wrapper); \ + } \ + taosMemoryFree(ttlV); \ + } while (0); + +#define STREAM_STATE_GET_ROCKSDB(pState, funcname, key, pVal, vLen) \ do { \ code = 0; \ char buf[128] = {0}; \ @@ -2543,62 +2565,32 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfKe if (qDebugFlag & DEBUG_TRACE) ginitDict[i].toStrFunc((void*)key, toString); \ int32_t klen = ginitDict[i].enFunc((void*)key, buf); \ rocksdb_column_family_handle_t* pHandle = ((rocksdb_column_family_handle_t**)wrapper->pCf)[ginitDict[i].idx]; \ - rocksdb_writeoptions_t* opts = wrapper->writeOpt; \ rocksdb_t* db = wrapper->db; \ - char* ttlV = NULL; \ - int32_t ttlVLen = ginitDict[i].enValueFunc((char*)value, vLen, 0, &ttlV); \ - rocksdb_put_cf(db, opts, pHandle, (const char*)buf, klen, (const char*)ttlV, (size_t)ttlVLen, &err); \ - if (err != NULL) { \ - stError("streamState str: %s failed to write to %s, err: %s", toString, funcname, err); \ - taosMemoryFree(err); \ + rocksdb_readoptions_t* opts = wrapper->readOpt; \ + size_t len = 0; \ + char* val = rocksdb_get_cf(db, opts, pHandle, (const char*)buf, klen, (size_t*)&len, &err); \ + if (val == NULL || len == 0) { \ + if (err == NULL) { \ + qTrace("streamState str: %s failed to read from %s_%s, err: not exist", toString, wrapper->idstr, funcname); \ + } else { \ + stError("streamState str: %s failed to read from %s_%s, err: %s", toString, wrapper->idstr, funcname, err); \ + taosMemoryFreeClear(err); \ + } \ code = -1; \ } else { \ - qTrace("streamState str:%s succ to write to %s, rowValLen:%d, ttlValLen:%d, %p", toString, funcname, vLen, ttlVLen, wrapper); \ + char* p = NULL; \ + int32_t tlen = ginitDict[i].deValueFunc(val, len, NULL, (char**)pVal); \ + if (tlen <= 0) { \ + stError("streamState str: %s failed to read from %s_%s, err: already ttl ", toString, wrapper->idstr, \ + funcname); \ + code = -1; \ + } else { \ + qTrace("streamState str: %s succ to read from %s_%s, valLen:%d, %p", toString, wrapper->idstr, funcname, tlen, \ + wrapper); \ + } \ + taosMemoryFree(val); \ + if (vLen != NULL) *vLen = tlen; \ } \ - taosMemoryFree(ttlV); \ - } while (0); - -#define STREAM_STATE_GET_ROCKSDB(pState, funcname, key, pVal, vLen) \ - do { \ - code = 0; \ - char buf[128] = {0}; \ - char* err = NULL; \ - int i = streamStateGetCfIdx(pState, funcname); \ - if (i < 0) { \ - qWarn("streamState failed to get cf name: %s", funcname); \ - code = -1; \ - break; \ - } \ - STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; \ - char toString[128] = {0}; \ - if (qDebugFlag & DEBUG_TRACE) ginitDict[i].toStrFunc((void*)key, toString); \ - int32_t klen = ginitDict[i].enFunc((void*)key, buf); \ - rocksdb_column_family_handle_t* pHandle = ((rocksdb_column_family_handle_t**)wrapper->pCf)[ginitDict[i].idx]; \ - rocksdb_t* db = wrapper->db; \ - rocksdb_readoptions_t* opts = wrapper->readOpt; \ - size_t len = 0; \ - char* val = rocksdb_get_cf(db, opts, pHandle, (const char*)buf, klen, (size_t*)&len, &err); \ - if (val == NULL || len == 0) { \ - if (err == NULL) { \ - qTrace("streamState str: %s failed to read from %s_%s, err: not exist", toString, wrapper->idstr, funcname); \ - } else { \ - stError("streamState str: %s failed to read from %s_%s, err: %s", toString, wrapper->idstr, funcname, err); \ - taosMemoryFreeClear(err); \ - } \ - code = -1; \ - } else { \ - char* p = NULL; \ - int32_t tlen = ginitDict[i].deValueFunc(val, len, NULL, (char**)pVal); \ - if (tlen <= 0) { \ - stError("streamState str: %s failed to read from %s_%s, err: already ttl ", toString, wrapper->idstr, \ - funcname); \ - code = -1; \ - } else { \ - qTrace("streamState str: %s succ to read from %s_%s, valLen:%d, %p", toString, wrapper->idstr, funcname, tlen, wrapper); \ - } \ - taosMemoryFree(val); \ - if (vLen != NULL) *vLen = tlen; \ - } \ } while (0); #define STREAM_STATE_DEL_ROCKSDB(pState, funcname, key) \ @@ -2986,7 +2978,7 @@ SStreamStateCur* streamStateSessionSeekKeyCurrentPrev_rocksdb(SStreamState* pSta stDebug("streamStateSessionSeekKeyCurrentPrev_rocksdb"); STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; - SStreamStateCur* pCur = createStreamStateCursor(); + SStreamStateCur* pCur = createStreamStateCursor(); if (pCur == NULL) { return NULL; } @@ -3027,7 +3019,7 @@ SStreamStateCur* streamStateSessionSeekKeyCurrentPrev_rocksdb(SStreamState* pSta SStreamStateCur* streamStateSessionSeekKeyCurrentNext_rocksdb(SStreamState* pState, SSessionKey* key) { qDebug("streamStateSessionSeekKeyCurrentNext_rocksdb"); STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; - SStreamStateCur* pCur = createStreamStateCursor(); + SStreamStateCur* pCur = createStreamStateCursor(); if (pCur == NULL) { return NULL; } @@ -3065,7 +3057,7 @@ SStreamStateCur* streamStateSessionSeekKeyCurrentNext_rocksdb(SStreamState* pSta SStreamStateCur* streamStateSessionSeekKeyNext_rocksdb(SStreamState* pState, const SSessionKey* key) { qDebug("streamStateSessionSeekKeyNext_rocksdb"); STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; - SStreamStateCur* pCur = createStreamStateCursor(); + SStreamStateCur* pCur = createStreamStateCursor(); if (pCur == NULL) { return NULL; } @@ -3168,7 +3160,7 @@ int32_t streamStateFillDel_rocksdb(SStreamState* pState, const SWinKey* key) { SStreamStateCur* streamStateFillGetCur_rocksdb(SStreamState* pState, const SWinKey* key) { qDebug("streamStateFillGetCur_rocksdb"); - SStreamStateCur* pCur = createStreamStateCursor(); + SStreamStateCur* pCur = createStreamStateCursor(); STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; if (pCur == NULL) return NULL; @@ -3229,7 +3221,7 @@ int32_t streamStateFillGetKVByCur_rocksdb(SStreamStateCur* pCur, SWinKey* pKey, SStreamStateCur* streamStateFillSeekKeyNext_rocksdb(SStreamState* pState, const SWinKey* key) { qDebug("streamStateFillSeekKeyNext_rocksdb"); STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; - SStreamStateCur* pCur = createStreamStateCursor(); + SStreamStateCur* pCur = createStreamStateCursor(); if (!pCur) { return NULL; } @@ -3267,7 +3259,7 @@ SStreamStateCur* streamStateFillSeekKeyNext_rocksdb(SStreamState* pState, const SStreamStateCur* streamStateFillSeekKeyPrev_rocksdb(SStreamState* pState, const SWinKey* key) { qDebug("streamStateFillSeekKeyPrev_rocksdb"); STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; - SStreamStateCur* pCur = createStreamStateCursor(); + SStreamStateCur* pCur = createStreamStateCursor(); if (pCur == NULL) { return NULL; } @@ -3305,7 +3297,7 @@ SStreamStateCur* streamStateFillSeekKeyPrev_rocksdb(SStreamState* pState, const int32_t streamStateSessionGetKeyByRange_rocksdb(SStreamState* pState, const SSessionKey* key, SSessionKey* curKey) { stDebug("streamStateSessionGetKeyByRange_rocksdb"); STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; - SStreamStateCur* pCur = createStreamStateCursor(); + SStreamStateCur* pCur = createStreamStateCursor(); if (pCur == NULL) { return -1; } @@ -3575,7 +3567,7 @@ int32_t streamDefaultIterGet_rocksdb(SStreamState* pState, const void* start, co return code; } void* streamDefaultIterCreate_rocksdb(SStreamState* pState) { - SStreamStateCur* pCur = createStreamStateCursor(); + SStreamStateCur* pCur = createStreamStateCursor(); STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; pCur->db = wrapper->db; From e5449bf161ec320b6f7893b7eb9c05082374e433 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Tue, 7 Nov 2023 11:49:56 +0800 Subject: [PATCH 085/195] refactor backend --- source/libs/stream/src/streamBackendRocksdb.c | 132 ++++++++---------- 1 file changed, 62 insertions(+), 70 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index c6dafc4e41..5edba5e180 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -22,6 +22,7 @@ #define GEN_COLUMN_FAMILY_NAME(name, idstr, SUFFIX) sprintf(name, "%s_%s", idstr, (SUFFIX)); +int32_t copyFiles(const char* src, const char* dst); typedef struct SDbChkp { int8_t init; char* pCurrent; @@ -67,7 +68,7 @@ typedef struct { SHashObj* pDbChkpTbl; TdThreadRwlock rwLock; -} SBackendManager; +} SBkdMgt; typedef struct SCompactFilteFactory { void* status; @@ -146,7 +147,6 @@ typedef struct { } SCfInit; -void dbSetFilterFactory(rocksdb_options_t* opt, int32_t i, void* arg); typedef struct { void* funcName; DestroyFactoryFunc destroy; @@ -486,14 +486,14 @@ _ERROR: taosMemoryFree(dstDir); return code; } -SBackendManager* bkdMgtCreate(char* path) { - SBackendManager* p = taosMemoryCalloc(1, sizeof(SBackendManager)); +SBkdMgt* bkdMgtCreate(char* path) { + SBkdMgt* p = taosMemoryCalloc(1, sizeof(SBkdMgt)); p->pDbChkpTbl = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), true, HASH_ENTRY_LOCK); taosThreadRwlockInit(&p->rwLock, NULL); return p; } -void bkdMgtDestroy(SBackendManager* bm) { +void bkdMgtDestroy(SBkdMgt* bm) { if (bm == NULL) return; void* pIter = taosHashIterate(bm->pDbChkpTbl, NULL); while (pIter) { @@ -507,7 +507,7 @@ void bkdMgtDestroy(SBackendManager* bm) { taosMemoryFree(bm); } -int32_t bkdMgtGetDelta(SBackendManager* bm, char* taskId, int64_t chkpId, SArray* list) { +int32_t bkdMgtGetDelta(SBkdMgt* bm, char* taskId, int64_t chkpId, SArray* list) { int32_t code = 0; taosThreadRwlockWrlock(&bm->rwLock); @@ -518,7 +518,7 @@ int32_t bkdMgtGetDelta(SBackendManager* bm, char* taskId, int64_t chkpId, SArray return code; } -int32_t bkdMgtAddChkp(SBackendManager* bm, char* task, char* path) { +int32_t bkdMgtAddChkp(SBkdMgt* bm, char* task, char* path) { int32_t code = -1; taosThreadRwlockWrlock(&bm->rwLock); @@ -538,7 +538,7 @@ int32_t bkdMgtAddChkp(SBackendManager* bm, char* task, char* path) { return code; } -int32_t bkdMgtDumpTo(SBackendManager* bm, char* taskId, char* dname) { +int32_t bkdMgtDumpTo(SBkdMgt* bm, char* taskId, char* dname) { int32_t code = 0; taosThreadRwlockRdlock(&bm->rwLock); @@ -549,53 +549,8 @@ int32_t bkdMgtDumpTo(SBackendManager* bm, char* taskId, char* dname) { return code; } -void dbSetFilterFactory(rocksdb_options_t* opt, int32_t i, void* arg) { - rocksdb_compactionfilterfactory_t* filterFactory = rocksdb_compactionfilterfactory_create( - arg, ginitDict[i].destroyFilter, ginitDict[i].createFilter, ginitDict[i].funcName); - rocksdb_options_set_compaction_filter_factory(opt, filterFactory); -} - bool isValidCheckpoint(const char* dir) { return true; } -int32_t copyFiles(const char* src, const char* dst) { - int32_t code = 0; - // opt later, just hard link - int32_t sLen = strlen(src); - int32_t dLen = strlen(dst); - char* srcName = taosMemoryCalloc(1, sLen + 64); - char* dstName = taosMemoryCalloc(1, dLen + 64); - - TdDirPtr pDir = taosOpenDir(src); - if (pDir == NULL) { - taosMemoryFree(srcName); - taosMemoryFree(dstName); - return -1; - } - - TdDirEntryPtr de = NULL; - while ((de = taosReadDir(pDir)) != NULL) { - char* name = taosGetDirEntryName(de); - if (strcmp(name, ".") == 0 || strcmp(name, "..") == 0) continue; - - sprintf(srcName, "%s%s%s", src, TD_DIRSEP, name); - sprintf(dstName, "%s%s%s", dst, TD_DIRSEP, name); - if (!taosDirEntryIsDir(de)) { - code = taosCopyFile(srcName, dstName); - if (code == -1) { - goto _err; - } - } - - memset(srcName, 0, sLen + 64); - memset(dstName, 0, dLen + 64); - } - -_err: - taosMemoryFreeClear(srcName); - taosMemoryFreeClear(dstName); - taosCloseDir(&pDir); - return code >= 0 ? 0 : -1; -} int32_t rebuildDirFromCheckpoint(const char* path, int64_t chkpId, char** dst) { // impl later int32_t code = 0; @@ -1754,23 +1709,23 @@ void destroyCompactFilteFactory(void* arg) { } const char* compactFilteFactoryName(void* arg) { SCompactFilteFactory* state = arg; - return "stream_compact_filter"; + return "stream_compact_factory_filter"; } const char* compactFilteFactoryNameSess(void* arg) { SCompactFilteFactory* state = arg; - return "stream_compact_filter_sess"; + return "stream_compact_factory_filter_sess"; } const char* compactFilteFactoryNameState(void* arg) { SCompactFilteFactory* state = arg; - return "stream_compact_filter_state"; + return "stream_compact_factory_filter_state"; } const char* compactFilteFactoryNameFill(void* arg) { SCompactFilteFactory* state = arg; - return "stream_compact_filter_fill"; + return "stream_compact_factory_filter_fill"; } const char* compactFilteFactoryNameFunc(void* arg) { SCompactFilteFactory* state = arg; - return "stream_compact_filter_func"; + return "stream_compact_factory_filter_func"; } void destroyCompactFilte(void* arg) { (void)arg; } @@ -1788,21 +1743,18 @@ unsigned char compactFilteSess(void* arg, int level, const char* key, size_t kle char** newval, size_t* newvlen, unsigned char* value_changed) { // not impl yet return 0; - // return streamStateValueIsStale((char*)val) ? 1 : 0; } unsigned char compactFilteState(void* arg, int level, const char* key, size_t klen, const char* val, size_t vlen, char** newval, size_t* newvlen, unsigned char* value_changed) { // not impl yet return 0; - // return streamStateValueIsStale((char*)val) ? 1 : 0; } unsigned char compactFilteFill(void* arg, int level, const char* key, size_t klen, const char* val, size_t vlen, char** newval, size_t* newvlen, unsigned char* value_changed) { // not impl yet return 0; - // return streamStateValueIsStale((char*)val) ? 1 : 0; } unsigned char compactFilteFunc(void* arg, int level, const char* key, size_t klen, const char* val, size_t vlen, @@ -1935,12 +1887,13 @@ void taskDbInitOpt(STaskDbWrapper* pTaskDb) { rocksdb_comparator_create(NULL, cfPara->detroyFunc, cfPara->cmpFunc, cfPara->cmpName); rocksdb_options_set_comparator((rocksdb_options_t*)opt, compare); + rocksdb_compactionfilterfactory_t* filterFactory = + rocksdb_compactionfilterfactory_create(NULL, cfPara->destroyFilter, cfPara->createFilter, cfPara->funcName); + rocksdb_options_set_compaction_filter_factory(opt, filterFactory); + pTaskDb->pCompares[i] = compare; pTaskDb->pCfOpts[i] = opt; pTaskDb->pCfParams[i].tableOpt = tableOpt; - - // set filter factory - dbSetFilterFactory(opt, i, NULL); } return; } @@ -2054,8 +2007,8 @@ _EXIT: return NULL; } -void taskDbDestroy(void* pBackend) { - STaskDbWrapper* wrapper = pBackend; +void taskDbDestroy(void* pDb) { + STaskDbWrapper* wrapper = pDb; qDebug("succ to destroy stream backend:%p", wrapper); int8_t nCf = sizeof(ginitDict) / sizeof(ginitDict[0]); @@ -2107,7 +2060,7 @@ void taskDbDestroy(void* pBackend) { if (wrapper->db) rocksdb_close(wrapper->db); - taskDbDestroyChkpOpt(pBackend); + taskDbDestroyChkpOpt(wrapper); taosMemoryFree(wrapper->idstr); taosMemoryFree(wrapper->path); @@ -2116,14 +2069,14 @@ void taskDbDestroy(void* pBackend) { return; } -int32_t taskDbOpenCfByKey(STaskDbWrapper* pBackend, const char* key) { +int32_t taskDbOpenCfByKey(STaskDbWrapper* pDb, const char* key) { int32_t code = 0; char* err = NULL; int8_t idx = getCfIdx(key); if (idx == -1) return -1; rocksdb_column_family_handle_t* cf = - rocksdb_create_column_family(pBackend->db, pBackend->pCfOpts[idx], ginitDict[idx].key, &err); + rocksdb_create_column_family(pDb->db, pDb->pCfOpts[idx], ginitDict[idx].key, &err); if (err != NULL) { stError("failed to open cf, key:%s, reason: %s", key, err); taosMemoryFree(err); @@ -2131,7 +2084,7 @@ int32_t taskDbOpenCfByKey(STaskDbWrapper* pBackend, const char* key) { return code; } - pBackend->pCf[idx] = cf; + pDb->pCf[idx] = cf; return code; } int32_t copyDataAt(RocksdbCfInst* pSrc, STaskDbWrapper* pDst, int8_t i) { @@ -3690,4 +3643,43 @@ uint32_t nextPow2(uint32_t x) { x = x | (x >> 8); x = x | (x >> 16); return x + 1; +} +int32_t copyFiles(const char* src, const char* dst) { + int32_t code = 0; + // opt later, just hard link + int32_t sLen = strlen(src); + int32_t dLen = strlen(dst); + char* srcName = taosMemoryCalloc(1, sLen + 64); + char* dstName = taosMemoryCalloc(1, dLen + 64); + + TdDirPtr pDir = taosOpenDir(src); + if (pDir == NULL) { + taosMemoryFree(srcName); + taosMemoryFree(dstName); + return -1; + } + + TdDirEntryPtr de = NULL; + while ((de = taosReadDir(pDir)) != NULL) { + char* name = taosGetDirEntryName(de); + if (strcmp(name, ".") == 0 || strcmp(name, "..") == 0) continue; + + sprintf(srcName, "%s%s%s", src, TD_DIRSEP, name); + sprintf(dstName, "%s%s%s", dst, TD_DIRSEP, name); + if (!taosDirEntryIsDir(de)) { + code = taosCopyFile(srcName, dstName); + if (code == -1) { + goto _err; + } + } + + memset(srcName, 0, sLen + 64); + memset(dstName, 0, dLen + 64); + } + +_err: + taosMemoryFreeClear(srcName); + taosMemoryFreeClear(dstName); + taosCloseDir(&pDir); + return code >= 0 ? 0 : -1; } \ No newline at end of file From e015134d6da2f3f138cde983051ed51344317941 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Tue, 7 Nov 2023 14:46:52 +0800 Subject: [PATCH 086/195] refactor backend --- source/libs/stream/src/streamBackendRocksdb.c | 23 ++++++------------- 1 file changed, 7 insertions(+), 16 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 5edba5e180..cca1057662 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -20,9 +20,6 @@ #include "tcommon.h" #include "tref.h" -#define GEN_COLUMN_FAMILY_NAME(name, idstr, SUFFIX) sprintf(name, "%s_%s", idstr, (SUFFIX)); - -int32_t copyFiles(const char* src, const char* dst); typedef struct SDbChkp { int8_t init; char* pCurrent; @@ -87,8 +84,6 @@ typedef struct { rocksdb_comparator_t** pCompares; } RocksdbCfInst; -uint32_t nextPow2(uint32_t x); - int32_t streamStateOpenBackendCf(void* backend, char* name, char** cfs, int32_t nCf); void destroyRocksdbCfInst(RocksdbCfInst* inst); @@ -147,13 +142,6 @@ typedef struct { } SCfInit; -typedef struct { - void* funcName; - DestroyFactoryFunc destroy; - CreateFactoryFunc create; - FactoryNameFunc factoryName; -} SCfFilterFactory; - const char* compareDefaultName(void* name); const char* compareStateName(void* name); const char* compareWinKeyName(void* name); @@ -205,6 +193,10 @@ static bool streamStateIterSeekAndValid(rocksdb_iterator_t* iter, static rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfName, rocksdb_snapshot_t** snapshot, rocksdb_readoptions_t** readOpt); +#define GEN_COLUMN_FAMILY_NAME(name, idstr, SUFFIX) sprintf(name, "%s_%s", idstr, (SUFFIX)); +int32_t copyFiles(const char* src, const char* dst); +uint32_t nextPow2(uint32_t x); + SCfInit ginitDict[] = { {"default", 7, 0, defaultKeyComp, defaultKeyEncode, defaultKeyDecode, defaultKeyToString, compareDefaultName, destroyFunc, encodeValueFunc, decodeValueFunc, compactFilteFactoryCreateFilter, destroyCompactFilteFactory, @@ -1709,7 +1701,7 @@ void destroyCompactFilteFactory(void* arg) { } const char* compactFilteFactoryName(void* arg) { SCompactFilteFactory* state = arg; - return "stream_compact_factory_filter"; + return "stream_compact_factory_filter_default"; } const char* compactFilteFactoryNameSess(void* arg) { SCompactFilteFactory* state = arg; @@ -1733,7 +1725,7 @@ unsigned char compactFilte(void* arg, int level, const char* key, size_t klen, c char** newval, size_t* newvlen, unsigned char* value_changed) { return streamStateValueIsStale((char*)val) ? 1 : 0; } -const char* compactFilteName(void* arg) { return "stream_filte"; } +const char* compactFilteName(void* arg) { return "stream_filte_default"; } const char* compactFilteNameSess(void* arg) { return "stream_filte_sess"; } const char* compactFilteNameState(void* arg) { return "stream_filte_state"; } const char* compactFilteNameFill(void* arg) { return "stream_filte_fill"; } @@ -1837,7 +1829,6 @@ void taskDbRemoveRef(void* pTaskDb) { STaskDbWrapper* pBackend = pTaskDb; taosReleaseRef(taskDbWrapperId, pBackend->refId); } -// void taskDbDestroy(STaskDbWrapper* wrapper); void taskDbInitOpt(STaskDbWrapper* pTaskDb) { rocksdb_env_t* env = rocksdb_create_default_env(); @@ -1899,7 +1890,7 @@ void taskDbInitOpt(STaskDbWrapper* pTaskDb) { } void taskDbInitChkpOpt(STaskDbWrapper* pTaskDb) { pTaskDb->chkpId = -1; - pTaskDb->chkpCap = 4; + pTaskDb->chkpCap = 2; pTaskDb->chkpSaved = taosArrayInit(4, sizeof(int64_t)); pTaskDb->chkpInUse = taosArrayInit(4, sizeof(int64_t)); From 042522ac7af0889712a34ce2917e5a19e4a53ce0 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Tue, 7 Nov 2023 15:18:24 +0800 Subject: [PATCH 087/195] refactor backend --- source/libs/stream/src/streamBackendRocksdb.c | 151 +++++++++--------- 1 file changed, 75 insertions(+), 76 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index cca1057662..b8f363004a 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -111,34 +111,34 @@ rocksdb_compactionfilter_t* compactFilteFactoryCreateFilterState(void* arg, rock rocksdb_compactionfilter_t* compactFilteFactoryCreateFilterFunc(void* arg, rocksdb_compactionfiltercontext_t* ctx); rocksdb_compactionfilter_t* compactFilteFactoryCreateFilterFill(void* arg, rocksdb_compactionfiltercontext_t* ctx); -typedef int (*EncodeFunc)(void* key, char* buf); -typedef int (*DecodeFunc)(void* key, char* buf); -typedef int (*ToStringFunc)(void* key, char* buf); -typedef const char* (*CompareName)(void* statue); -typedef int (*BackendCmpFunc)(void* state, const char* aBuf, size_t aLen, const char* bBuf, size_t bLen); -typedef void (*DestroyFunc)(void* state); -typedef int32_t (*EncodeValueFunc)(void* value, int32_t vlen, int64_t ttl, char** dest); -typedef int32_t (*DecodeValueFunc)(void* value, int32_t vlen, int64_t* ttl, char** dest); +typedef int (*__db_key_encode_fn_t)(void* key, char* buf); +typedef int (*__db_key_decode_fn_t)(void* key, char* buf); +typedef int (*__db_key_tostr_fn_t)(void* key, char* buf); +typedef const char* (*__db_key_cmpname_fn_t)(void* statue); +typedef int (*__db_key_cmp_fn_t)(void* state, const char* aBuf, size_t aLen, const char* bBuf, size_t bLen); +typedef void (*__db_destroy_cmp_fn_t)(void* state); +typedef int32_t (*__db_value_encode_fn_t)(void* value, int32_t vlen, int64_t ttl, char** dest); +typedef int32_t (*__db_value_decode_fn_t)(void* value, int32_t vlen, int64_t* ttl, char** dest); -typedef rocksdb_compactionfilter_t* (*CreateFactoryFunc)(void* arg, rocksdb_compactionfiltercontext_t* ctx); -typedef const char* (*FactoryNameFunc)(void* arg); -typedef void (*DestroyFactoryFunc)(void* arg); +typedef rocksdb_compactionfilter_t* (*__db_factory_create_fn_t)(void* arg, rocksdb_compactionfiltercontext_t* ctx); +typedef const char* (*__db_factory_name_fn_t)(void* arg); +typedef void (*__db_factory_destroy_fn_t)(void* arg); typedef struct { - const char* key; - int32_t len; - int idx; - BackendCmpFunc cmpFunc; - EncodeFunc enFunc; - DecodeFunc deFunc; - ToStringFunc toStrFunc; - CompareName cmpName; - DestroyFunc detroyFunc; - EncodeValueFunc enValueFunc; - DecodeValueFunc deValueFunc; + const char* key; + int32_t len; + int idx; + __db_key_cmp_fn_t cmpKey; + __db_key_encode_fn_t enFunc; + __db_key_decode_fn_t deFunc; + __db_key_tostr_fn_t toStrFunc; + __db_key_cmpname_fn_t cmpName; + __db_destroy_cmp_fn_t destroyCmp; + __db_value_encode_fn_t enValueFunc; + __db_value_decode_fn_t deValueFunc; - CreateFactoryFunc createFilter; - DestroyFactoryFunc destroyFilter; - FactoryNameFunc funcName; + __db_factory_create_fn_t createFilter; + __db_factory_destroy_fn_t destroyFilter; + __db_factory_name_fn_t funcName; } SCfInit; @@ -180,14 +180,15 @@ int parKeyEncode(void* k, char* buf); int parKeyDecode(void* k, char* buf); int parKeyToString(void* k, char* buf); -int stremaValueEncode(void* k, char* buf); -int streamValueDecode(void* k, char* buf); -int32_t streamValueToString(void* k, char* buf); -int32_t streaValueIsStale(void* k, int64_t ts); -void destroyFunc(void* arg); +// int stremaValueEncode(void* k, char* buf); +// int streamValueDecode(void* k, char* buf); -int32_t encodeValueFunc(void* value, int32_t vlen, int64_t ttl, char** dest); -int32_t decodeValueFunc(void* value, int32_t vlen, int64_t* ttl, char** dest); +int32_t valueEncode(void* value, int32_t vlen, int64_t ttl, char** dest); +int32_t valueDecode(void* value, int32_t vlen, int64_t* ttl, char** dest); +int32_t valueToString(void* k, char* buf); +int32_t valueIsStale(void* k, int64_t ts); + +void destroyCompare(void* arg); static bool streamStateIterSeekAndValid(rocksdb_iterator_t* iter, char* buf, size_t len); static rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfName, @@ -199,32 +200,30 @@ uint32_t nextPow2(uint32_t x); SCfInit ginitDict[] = { {"default", 7, 0, defaultKeyComp, defaultKeyEncode, defaultKeyDecode, defaultKeyToString, compareDefaultName, - destroyFunc, encodeValueFunc, decodeValueFunc, compactFilteFactoryCreateFilter, destroyCompactFilteFactory, + destroyCompare, valueEncode, valueDecode, compactFilteFactoryCreateFilter, destroyCompactFilteFactory, compactFilteFactoryName}, - {"state", 5, 1, stateKeyDBComp, stateKeyEncode, stateKeyDecode, stateKeyToString, compareStateName, destroyFunc, - encodeValueFunc, decodeValueFunc, compactFilteFactoryCreateFilterState, destroyCompactFilteFactory, + {"state", 5, 1, stateKeyDBComp, stateKeyEncode, stateKeyDecode, stateKeyToString, compareStateName, destroyCompare, + valueEncode, valueDecode, compactFilteFactoryCreateFilterState, destroyCompactFilteFactory, compactFilteFactoryNameState}, - {"fill", 4, 2, winKeyDBComp, winKeyEncode, winKeyDecode, winKeyToString, compareWinKeyName, destroyFunc, - encodeValueFunc, decodeValueFunc, compactFilteFactoryCreateFilterFill, destroyCompactFilteFactory, + {"fill", 4, 2, winKeyDBComp, winKeyEncode, winKeyDecode, winKeyToString, compareWinKeyName, destroyCompare, + valueEncode, valueDecode, compactFilteFactoryCreateFilterFill, destroyCompactFilteFactory, compactFilteFactoryNameFill}, {"sess", 4, 3, stateSessionKeyDBComp, stateSessionKeyEncode, stateSessionKeyDecode, stateSessionKeyToString, - compareSessionKeyName, destroyFunc, encodeValueFunc, decodeValueFunc, compactFilteFactoryCreateFilterSess, + compareSessionKeyName, destroyCompare, valueEncode, valueDecode, compactFilteFactoryCreateFilterSess, destroyCompactFilteFactory, compactFilteFactoryNameSess}, - {"func", 4, 4, tupleKeyDBComp, tupleKeyEncode, tupleKeyDecode, tupleKeyToString, compareFuncKeyName, destroyFunc, - encodeValueFunc, decodeValueFunc, compactFilteFactoryCreateFilterFunc, destroyCompactFilteFactory, + {"func", 4, 4, tupleKeyDBComp, tupleKeyEncode, tupleKeyDecode, tupleKeyToString, compareFuncKeyName, destroyCompare, + valueEncode, valueDecode, compactFilteFactoryCreateFilterFunc, destroyCompactFilteFactory, compactFilteFactoryNameFunc}, - {"parname", 7, 5, parKeyDBComp, parKeyEncode, parKeyDecode, parKeyToString, compareParKeyName, destroyFunc, - encodeValueFunc, decodeValueFunc, compactFilteFactoryCreateFilter, destroyCompactFilteFactory, - compactFilteFactoryName}, + {"parname", 7, 5, parKeyDBComp, parKeyEncode, parKeyDecode, parKeyToString, compareParKeyName, destroyCompare, + valueEncode, valueDecode, compactFilteFactoryCreateFilter, destroyCompactFilteFactory, compactFilteFactoryName}, - {"partag", 6, 6, parKeyDBComp, parKeyEncode, parKeyDecode, parKeyToString, comparePartagKeyName, destroyFunc, - encodeValueFunc, decodeValueFunc, compactFilteFactoryCreateFilter, destroyCompactFilteFactory, - compactFilteFactoryName}, + {"partag", 6, 6, parKeyDBComp, parKeyEncode, parKeyDecode, parKeyToString, comparePartagKeyName, destroyCompare, + valueEncode, valueDecode, compactFilteFactoryCreateFilter, destroyCompactFilteFactory, compactFilteFactoryName}, }; const char* cfName[] = {"default", "state", "fill", "sess", "func", "parname", "partag"}; @@ -1583,23 +1582,23 @@ int parKeyToString(void* k, char* buf) { n = sprintf(buf + n, "[groupId:%" PRIi64 "]", *key); return n; } -int stremaValueEncode(void* k, char* buf) { - int len = 0; - SStreamValue* key = k; - len += taosEncodeFixedI64((void**)&buf, key->unixTimestamp); - len += taosEncodeFixedI32((void**)&buf, key->len); - len += taosEncodeBinary((void**)&buf, key->data, key->len); - return len; -} -int streamValueDecode(void* k, char* buf) { - SStreamValue* key = k; - char* p = buf; - p = taosDecodeFixedI64(p, &key->unixTimestamp); - p = taosDecodeFixedI32(p, &key->len); - p = taosDecodeBinary(p, (void**)&key->data, key->len); - return p - buf; -} -int32_t streamValueToString(void* k, char* buf) { +// int stremaValueEncode(void* k, char* buf) { +// int len = 0; +// SStreamValue* key = k; +// len += taosEncodeFixedI64((void**)&buf, key->unixTimestamp); +// len += taosEncodeFixedI32((void**)&buf, key->len); +// len += taosEncodeBinary((void**)&buf, key->data, key->len); +// return len; +// } +// int streamValueDecode(void* k, char* buf) { +// SStreamValue* key = k; +// char* p = buf; +// p = taosDecodeFixedI64(p, &key->unixTimestamp); +// p = taosDecodeFixedI32(p, &key->len); +// p = taosDecodeBinary(p, (void**)&key->data, key->len); +// return p - buf; +// } +int32_t valueToString(void* k, char* buf) { SStreamValue* key = k; int n = 0; n += sprintf(buf + n, "[unixTimestamp:%" PRIi64 ",", key->unixTimestamp); @@ -1609,7 +1608,7 @@ int32_t streamValueToString(void* k, char* buf) { } /*1: stale, 0: no stale*/ -int32_t streaValueIsStale(void* k, int64_t ts) { +int32_t valueIsStale(void* k, int64_t ts) { SStreamValue* key = k; if (key->unixTimestamp < ts) { return 1; @@ -1617,12 +1616,12 @@ int32_t streaValueIsStale(void* k, int64_t ts) { return 0; } -void destroyFunc(void* arg) { +void destroyCompare(void* arg) { (void)arg; return; } -int32_t encodeValueFunc(void* value, int32_t vlen, int64_t ttl, char** dest) { +int32_t valueEncode(void* value, int32_t vlen, int64_t ttl, char** dest) { SStreamValue key = {.unixTimestamp = ttl, .len = vlen, .data = (char*)(value)}; int32_t len = 0; if (*dest == NULL) { @@ -1644,7 +1643,7 @@ int32_t encodeValueFunc(void* value, int32_t vlen, int64_t ttl, char** dest) { * ret >= 0 : found valid value * ret < 0 : error or timeout */ -int32_t decodeValueFunc(void* value, int32_t vlen, int64_t* ttl, char** dest) { +int32_t valueDecode(void* value, int32_t vlen, int64_t* ttl, char** dest) { SStreamValue key = {0}; char* p = value; if (streamStateValueIsStale(p)) { @@ -1875,7 +1874,7 @@ void taskDbInitOpt(STaskDbWrapper* pTaskDb) { SCfInit* cfPara = &ginitDict[i]; rocksdb_comparator_t* compare = - rocksdb_comparator_create(NULL, cfPara->detroyFunc, cfPara->cmpFunc, cfPara->cmpName); + rocksdb_comparator_create(NULL, cfPara->destroyCmp, cfPara->cmpKey, cfPara->cmpName); rocksdb_options_set_comparator((rocksdb_options_t*)opt, compare); rocksdb_compactionfilterfactory_t* filterFactory = @@ -2182,7 +2181,7 @@ int32_t streamStateOpenBackendCf(void* backend, char* name, char** cfs, int32_t SCfInit* cfPara = &ginitDict[idx]; rocksdb_comparator_t* compare = - rocksdb_comparator_create(NULL, cfPara->detroyFunc, cfPara->cmpFunc, cfPara->cmpName); + rocksdb_comparator_create(NULL, cfPara->destroyCmp, cfPara->cmpKey, cfPara->cmpName); rocksdb_options_set_comparator((rocksdb_options_t*)cfOpts[i], compare); pCompare[i] = compare; } @@ -2266,7 +2265,7 @@ int32_t streamStateOpenBackendCf(void* backend, char* name, char** cfs, int32_t SCfInit* cfPara = &ginitDict[i]; rocksdb_comparator_t* compare = - rocksdb_comparator_create(NULL, cfPara->detroyFunc, cfPara->cmpFunc, cfPara->cmpName); + rocksdb_comparator_create(NULL, cfPara->destroyCmp, cfPara->cmpKey, cfPara->cmpName); rocksdb_options_set_comparator((rocksdb_options_t*)opt, compare); inst->pCompares[i] = compare; @@ -2346,7 +2345,7 @@ int streamStateOpenBackend(void* backend, SStreamState* pState) { for (int i = 0; i < cfLen; i++) { SCfInit* cf = &ginitDict[i]; - rocksdb_comparator_t* compare = rocksdb_comparator_create(NULL, cf->detroyFunc, cf->cmpFunc, cf->cmpName); + rocksdb_comparator_t* compare = rocksdb_comparator_create(NULL, cf->destroyCmp, cf->cmpKey, cf->cmpName); rocksdb_options_set_comparator((rocksdb_options_t*)cfOpt[i], compare); pCompare[i] = compare; } @@ -2684,7 +2683,7 @@ int32_t streamStateGetKVByCur_rocksdb(SStreamStateCur* pCur, SWinKey* pKey, cons if (pVLen != NULL) { size_t vlen = 0; const char* valStr = rocksdb_iter_value(pCur->iter, &vlen); - *pVLen = decodeValueFunc((void*)valStr, vlen, NULL, (char**)pVal); + *pVLen = valueDecode((void*)valStr, vlen, NULL, (char**)pVal); } *pKey = pKtmp->key; @@ -3058,7 +3057,7 @@ int32_t streamStateSessionGetKVByCur_rocksdb(SStreamStateCur* pCur, SSessionKey* SStateSessionKey* pKTmp = &ktmp; const char* vval = rocksdb_iter_value(pCur->iter, (size_t*)&vLen); char* val = NULL; - int32_t len = decodeValueFunc((void*)vval, vLen, NULL, &val); + int32_t len = valueDecode((void*)vval, vLen, NULL, &val); if (len < 0) { taosMemoryFree(val); return -1; @@ -3152,7 +3151,7 @@ int32_t streamStateFillGetKVByCur_rocksdb(SStreamStateCur* pCur, SWinKey* pKey, winKeyDecode(&winKey, keyStr); const char* valStr = rocksdb_iter_value(pCur->iter, &vlen); - int32_t len = decodeValueFunc((void*)valStr, vlen, NULL, (char**)pVal); + int32_t len = valueDecode((void*)valStr, vlen, NULL, (char**)pVal); if (len < 0) { return -1; } @@ -3486,7 +3485,7 @@ int32_t streamDefaultIterGet_rocksdb(SStreamState* pState, const void* start, co int32_t vlen = 0; const char* vval = rocksdb_iter_value(pIter, (size_t*)&vlen); char* val = NULL; - int32_t len = decodeValueFunc((void*)vval, vlen, NULL, NULL); + int32_t len = valueDecode((void*)vval, vlen, NULL, NULL); if (len < 0) { rocksdb_iter_next(pIter); continue; @@ -3542,7 +3541,7 @@ char* streamDefaultIterVal_rocksdb(void* iter, int32_t* len) { int32_t vlen = 0; const char* val = rocksdb_iter_value(pCur->iter, (size_t*)&vlen); - *len = decodeValueFunc((void*)val, vlen, NULL, &ret); + *len = valueDecode((void*)val, vlen, NULL, &ret); if (*len < 0) { taosMemoryFree(ret); return NULL; From 8f7ad2bd900bd154fc6f38cc48b34d4fb188dbe7 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Tue, 7 Nov 2023 15:20:58 +0800 Subject: [PATCH 088/195] refactor backend --- source/libs/stream/src/streamBackendRocksdb.c | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index b8f363004a..e579fdc07d 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -116,7 +116,7 @@ typedef int (*__db_key_decode_fn_t)(void* key, char* buf); typedef int (*__db_key_tostr_fn_t)(void* key, char* buf); typedef const char* (*__db_key_cmpname_fn_t)(void* statue); typedef int (*__db_key_cmp_fn_t)(void* state, const char* aBuf, size_t aLen, const char* bBuf, size_t bLen); -typedef void (*__db_destroy_cmp_fn_t)(void* state); +typedef void (*__db_key_cmp_destroy_fn_t)(void* state); typedef int32_t (*__db_value_encode_fn_t)(void* value, int32_t vlen, int64_t ttl, char** dest); typedef int32_t (*__db_value_decode_fn_t)(void* value, int32_t vlen, int64_t* ttl, char** dest); @@ -124,17 +124,17 @@ typedef rocksdb_compactionfilter_t* (*__db_factory_create_fn_t)(void* arg, rocks typedef const char* (*__db_factory_name_fn_t)(void* arg); typedef void (*__db_factory_destroy_fn_t)(void* arg); typedef struct { - const char* key; - int32_t len; - int idx; - __db_key_cmp_fn_t cmpKey; - __db_key_encode_fn_t enFunc; - __db_key_decode_fn_t deFunc; - __db_key_tostr_fn_t toStrFunc; - __db_key_cmpname_fn_t cmpName; - __db_destroy_cmp_fn_t destroyCmp; - __db_value_encode_fn_t enValueFunc; - __db_value_decode_fn_t deValueFunc; + const char* key; + int32_t len; + int idx; + __db_key_cmp_fn_t cmpKey; + __db_key_encode_fn_t enFunc; + __db_key_decode_fn_t deFunc; + __db_key_tostr_fn_t toStrFunc; + __db_key_cmpname_fn_t cmpName; + __db_key_cmp_destroy_fn_t destroyCmp; + __db_value_encode_fn_t enValueFunc; + __db_value_decode_fn_t deValueFunc; __db_factory_create_fn_t createFilter; __db_factory_destroy_fn_t destroyFilter; From 1ee82f5e2c9c81782d822ea64fcce8a98482601e Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Tue, 7 Nov 2023 15:50:42 +0800 Subject: [PATCH 089/195] refactor backend --- source/libs/stream/src/streamBackendRocksdb.c | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index e579fdc07d..300251885d 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -127,7 +127,7 @@ typedef struct { const char* key; int32_t len; int idx; - __db_key_cmp_fn_t cmpKey; + __db_key_cmp_fn_t cmpFunc; __db_key_encode_fn_t enFunc; __db_key_decode_fn_t deFunc; __db_key_tostr_fn_t toStrFunc; @@ -1874,7 +1874,7 @@ void taskDbInitOpt(STaskDbWrapper* pTaskDb) { SCfInit* cfPara = &ginitDict[i]; rocksdb_comparator_t* compare = - rocksdb_comparator_create(NULL, cfPara->destroyCmp, cfPara->cmpKey, cfPara->cmpName); + rocksdb_comparator_create(NULL, cfPara->destroyCmp, cfPara->cmpFunc, cfPara->cmpName); rocksdb_options_set_comparator((rocksdb_options_t*)opt, compare); rocksdb_compactionfilterfactory_t* filterFactory = @@ -2181,7 +2181,7 @@ int32_t streamStateOpenBackendCf(void* backend, char* name, char** cfs, int32_t SCfInit* cfPara = &ginitDict[idx]; rocksdb_comparator_t* compare = - rocksdb_comparator_create(NULL, cfPara->destroyCmp, cfPara->cmpKey, cfPara->cmpName); + rocksdb_comparator_create(NULL, cfPara->destroyCmp, cfPara->cmpFunc, cfPara->cmpName); rocksdb_options_set_comparator((rocksdb_options_t*)cfOpts[i], compare); pCompare[i] = compare; } @@ -2265,7 +2265,7 @@ int32_t streamStateOpenBackendCf(void* backend, char* name, char** cfs, int32_t SCfInit* cfPara = &ginitDict[i]; rocksdb_comparator_t* compare = - rocksdb_comparator_create(NULL, cfPara->destroyCmp, cfPara->cmpKey, cfPara->cmpName); + rocksdb_comparator_create(NULL, cfPara->destroyCmp, cfPara->cmpFunc, cfPara->cmpName); rocksdb_options_set_comparator((rocksdb_options_t*)opt, compare); inst->pCompares[i] = compare; @@ -2345,7 +2345,7 @@ int streamStateOpenBackend(void* backend, SStreamState* pState) { for (int i = 0; i < cfLen; i++) { SCfInit* cf = &ginitDict[i]; - rocksdb_comparator_t* compare = rocksdb_comparator_create(NULL, cf->destroyCmp, cf->cmpKey, cf->cmpName); + rocksdb_comparator_t* compare = rocksdb_comparator_create(NULL, cf->destroyCmp, cf->cmpFunc, cf->cmpName); rocksdb_options_set_comparator((rocksdb_options_t*)cfOpt[i], compare); pCompare[i] = compare; } From 4f53ffd76f59edae62c211e25aa9e1007e1d4833 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Tue, 7 Nov 2023 16:01:57 +0800 Subject: [PATCH 090/195] refactor backend --- source/libs/stream/src/streamBackendRocksdb.c | 630 +++++++++--------- 1 file changed, 306 insertions(+), 324 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 300251885d..074d23cdbf 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -127,7 +127,7 @@ typedef struct { const char* key; int32_t len; int idx; - __db_key_cmp_fn_t cmpFunc; + __db_key_cmp_fn_t cmpKey; __db_key_encode_fn_t enFunc; __db_key_decode_fn_t deFunc; __db_key_tostr_fn_t toStrFunc; @@ -180,9 +180,6 @@ int parKeyEncode(void* k, char* buf); int parKeyDecode(void* k, char* buf); int parKeyToString(void* k, char* buf); -// int stremaValueEncode(void* k, char* buf); -// int streamValueDecode(void* k, char* buf); - int32_t valueEncode(void* value, int32_t vlen, int64_t ttl, char** dest); int32_t valueDecode(void* value, int32_t vlen, int64_t* ttl, char** dest); int32_t valueToString(void* k, char* buf); @@ -240,306 +237,6 @@ int32_t getCfIdx(const char* cfName) { return idx; } -int32_t compareHashTableImpl(SHashObj* p1, SHashObj* p2, SArray* diff) { - int32_t code = 0; - size_t len = 0; - void* pIter = taosHashIterate(p2, NULL); - while (pIter) { - char* name = taosHashGetKey(pIter, &len); - if (!taosHashGet(p1, name, len)) { - char* p = taosStrdup(name); - taosArrayPush(diff, &p); - } - pIter = taosHashIterate(p2, pIter); - } - return code; -} -int32_t compareHashTable(SHashObj* p1, SHashObj* p2, SArray* add, SArray* del) { - int32_t code = 0; - - code = compareHashTableImpl(p1, p2, add); - code = compareHashTableImpl(p2, p1, del); - - return code; -} - -int32_t dbChkpGetDelta(SDbChkp* p, int64_t chkpId, SArray* list) { - taosThreadRwlockWrlock(&p->rwLock); - - p->preCkptId = p->curChkpId; - p->curChkpId = chkpId; - const char* pCurrent = "CURRENT"; - int32_t currLen = strlen(pCurrent); - - const char* pManifest = "MANIFEST-"; - int32_t maniLen = strlen(pManifest); - - const char* pSST = ".sst"; - int32_t sstLen = strlen(pSST); - - memset(p->buf, 0, p->len); - sprintf(p->buf, "%s%scheckpoint%" PRId64 "", p->path, TD_DIRSEP, chkpId); - - taosArrayClearP(p->pAdd, taosMemoryFree); - taosArrayClearP(p->pDel, taosMemoryFree); - - TdDirPtr pDir = taosOpenDir(p->buf); - TdDirEntryPtr de = NULL; - int8_t dummy = 0; - while ((de = taosReadDir(pDir)) != NULL) { - char* name = taosGetDirEntryName(de); - if (strcmp(name, ".") == 0 || strcmp(name, "..") == 0) continue; - if (strlen(name) == currLen && strcmp(name, pCurrent) == 0) { - taosMemoryFreeClear(p->pCurrent); - p->pCurrent = taosStrdup(name); - taosHashPut(p->pSstTbl[1 - p->idx], name, strlen(name), &dummy, sizeof(dummy)); - continue; - } - - if (strlen(name) >= maniLen && strncmp(name, pManifest, maniLen) == 0) { - taosMemoryFreeClear(p->pManifest); - p->pManifest = taosStrdup(name); - taosHashPut(p->pSstTbl[1 - p->idx], name, strlen(name), &dummy, sizeof(dummy)); - continue; - } - if (strlen(name) >= sstLen && strncmp(name + strlen(name) - 4, pSST, sstLen) == 0) { - // char* p = taosStrdup(name); - taosHashPut(p->pSstTbl[1 - p->idx], name, strlen(name), &dummy, sizeof(dummy)); - continue; - } - } - if (p->init == 0) { - void* pIter = taosHashIterate(p->pSstTbl[1 - p->idx], NULL); - while (pIter) { - size_t len; - char* name = taosHashGetKey(pIter, &len); - if (name != NULL && len != 0) { - taosArrayPush(p->pAdd, &name); - } - pIter = taosHashIterate(p->pSstTbl[1 - p->idx], pIter); - } - if (taosArrayGetSize(p->pAdd) > 0) p->update = 1; - - p->init = 1; - p->preCkptId = -1; - p->curChkpId = chkpId; - } else { - int32_t code = compareHashTable(p->pSstTbl[p->idx], p->pSstTbl[1 - p->idx], p->pAdd, p->pDel); - if (code != 0) { - // dead code - taosArrayClearP(p->pAdd, taosMemoryFree); - taosArrayClearP(p->pDel, taosMemoryFree); - taosHashClear(p->pSstTbl[1 - p->idx]); - p->update = 0; - taosCloseDir(&pDir); - return code; - } - - if (taosArrayGetSize(p->pAdd) == 0 && taosArrayGetSize(p->pDel) == 0) { - p->update = 0; - } - - p->preCkptId = p->curChkpId; - p->curChkpId = chkpId; - } - - taosHashClear(p->pSstTbl[p->idx]); - p->idx = 1 - p->idx; - - taosCloseDir(&pDir); - taosThreadRwlockUnlock(&p->rwLock); - - return 0; -} -SDbChkp* dbChkpCreate(char* path, int64_t initChkpId) { - SDbChkp* p = taosMemoryCalloc(1, sizeof(SDbChkp)); - p->curChkpId = initChkpId; - p->preCkptId = -1; - p->pSST = taosArrayInit(64, sizeof(void*)); - p->path = taosStrdup(path); - p->len = strlen(path) + 128; - p->buf = taosMemoryCalloc(1, p->len); - - p->idx = 0; - p->pSstTbl[0] = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), false, HASH_ENTRY_LOCK); - p->pSstTbl[1] = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), false, HASH_ENTRY_LOCK); - - p->pAdd = taosArrayInit(64, sizeof(void*)); - p->pDel = taosArrayInit(64, sizeof(void*)); - p->update = 0; - taosThreadRwlockInit(&p->rwLock, NULL); - - SArray* list = NULL; - int32_t code = dbChkpGetDelta(p, initChkpId, list); - - return p; -} - -void dbChkpDestroy(SDbChkp* pChkp) { - taosMemoryFree(pChkp->buf); - taosMemoryFree(pChkp->path); - - taosArrayDestroyP(pChkp->pSST, taosMemoryFree); - taosArrayDestroyP(pChkp->pAdd, taosMemoryFree); - taosArrayDestroyP(pChkp->pDel, taosMemoryFree); - - taosHashCleanup(pChkp->pSstTbl[0]); - taosHashCleanup(pChkp->pSstTbl[1]); - - taosMemoryFree(pChkp->pCurrent); - taosMemoryFree(pChkp->pManifest); -} - -int32_t dbChkpInit(SDbChkp* p) { - if (p == NULL) return 0; - return 0; -} -int32_t dbChkpDumpTo(SDbChkp* p, char* dname) { - taosThreadRwlockRdlock(&p->rwLock); - int32_t code = 0; - int32_t len = p->len + 128; - - char* srcBuf = taosMemoryCalloc(1, len); - char* dstBuf = taosMemoryCalloc(1, len); - - char* srcDir = taosMemoryCalloc(1, len); - char* dstDir = taosMemoryCalloc(1, len); - - sprintf(srcDir, "%s%s%s%" PRId64 "", p->path, TD_DIRSEP, "checkpoint", p->curChkpId); - sprintf(dstDir, "%s%s%s", p->path, TD_DIRSEP, dname); - - if (!taosDirExist(srcDir)) { - stError("failed to dump srcDir %s, reason: not exist such dir", srcDir); - code = -1; - goto _ERROR; - } - - code = taosMkDir(dstDir); - if (code != 0) { - terrno = TAOS_SYSTEM_ERROR(errno); - stError("failed to mkdir srcDir %s, reason: %s", dstDir, terrstr()); - goto _ERROR; - } - - // clear current file - memset(dstBuf, 0, len); - sprintf(dstBuf, "%s%s%s", dstDir, TD_DIRSEP, p->pCurrent); - taosRemoveFile(dstBuf); - - memset(dstBuf, 0, len); - sprintf(dstBuf, "%s%s%s", dstDir, TD_DIRSEP, p->pManifest); - taosRemoveFile(dstBuf); - - // add file to $name dir - for (int i = 0; i < taosArrayGetSize(p->pAdd); i++) { - memset(dstBuf, 0, len); - memset(srcBuf, 0, len); - - char* filename = taosArrayGetP(p->pAdd, i); - sprintf(srcBuf, "%s%s%s", srcDir, TD_DIRSEP, filename); - sprintf(dstBuf, "%s%s%s", dstDir, TD_DIRSEP, filename); - - taosCopyFile(srcBuf, dstBuf); - } - // del file in $name - for (int i = 0; i < taosArrayGetSize(p->pDel); i++) { - memset(dstBuf, 0, len); - memset(srcBuf, 0, len); - - char* filename = taosArrayGetP(p->pDel, i); - sprintf(dstBuf, "%s%s%s", dstDir, TD_DIRSEP, filename); - taosRemoveFile(dstBuf); - } - - // copy current file to dst dir - memset(srcBuf, 0, len); - memset(dstBuf, 0, len); - sprintf(srcBuf, "%s%s%s", srcDir, TD_DIRSEP, p->pCurrent); - sprintf(dstBuf, "%s%s%s", dstDir, TD_DIRSEP, p->pCurrent); - taosCopyFile(srcBuf, dstBuf); - - // copy manifest file to dst dir - memset(srcBuf, 0, len); - memset(dstBuf, 0, len); - sprintf(srcBuf, "%s%s%s", srcDir, TD_DIRSEP, p->pManifest); - sprintf(dstBuf, "%s%s%s", dstDir, TD_DIRSEP, p->pManifest); - taosCopyFile(srcBuf, dstBuf); - - // clear delta data buf - taosArrayClearP(p->pAdd, taosMemoryFree); - taosArrayClearP(p->pDel, taosMemoryFree); - -_ERROR: - taosThreadRwlockUnlock(&p->rwLock); - taosMemoryFree(srcBuf); - taosMemoryFree(dstBuf); - taosMemoryFree(srcDir); - taosMemoryFree(dstDir); - return code; -} -SBkdMgt* bkdMgtCreate(char* path) { - SBkdMgt* p = taosMemoryCalloc(1, sizeof(SBkdMgt)); - p->pDbChkpTbl = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), true, HASH_ENTRY_LOCK); - taosThreadRwlockInit(&p->rwLock, NULL); - return p; -} - -void bkdMgtDestroy(SBkdMgt* bm) { - if (bm == NULL) return; - void* pIter = taosHashIterate(bm->pDbChkpTbl, NULL); - while (pIter) { - SDbChkp* pChkp = *(SDbChkp**)(pIter); - dbChkpDestroy(pChkp); - - pIter = taosHashIterate(bm->pDbChkpTbl, pIter); - } - - taosThreadRwlockDestroy(&bm->rwLock); - - taosMemoryFree(bm); -} -int32_t bkdMgtGetDelta(SBkdMgt* bm, char* taskId, int64_t chkpId, SArray* list) { - int32_t code = 0; - taosThreadRwlockWrlock(&bm->rwLock); - - SDbChkp* pChkp = taosHashGet(bm->pDbChkpTbl, taskId, strlen(taskId)); - code = dbChkpGetDelta(pChkp, chkpId, list); - - taosThreadRwlockUnlock(&bm->rwLock); - return code; -} - -int32_t bkdMgtAddChkp(SBkdMgt* bm, char* task, char* path) { - int32_t code = -1; - - taosThreadRwlockWrlock(&bm->rwLock); - SDbChkp** pp = taosHashGet(bm->pDbChkpTbl, task, strlen(task)); - if (pp == NULL) { - SDbChkp* p = dbChkpCreate(path, 0); - if (p != NULL) { - taosHashPut(bm->pDbChkpTbl, task, strlen(task), &p, sizeof(void*)); - code = 0; - } - } else { - stError("task chkp already exists"); - } - - taosThreadRwlockUnlock(&bm->rwLock); - - return code; -} - -int32_t bkdMgtDumpTo(SBkdMgt* bm, char* taskId, char* dname) { - int32_t code = 0; - taosThreadRwlockRdlock(&bm->rwLock); - - SDbChkp* p = taosHashGet(bm->pDbChkpTbl, taskId, strlen(taskId)); - code = dbChkpDumpTo(p, dname); - - taosThreadRwlockUnlock(&bm->rwLock); - return code; -} - bool isValidCheckpoint(const char* dir) { return true; } int32_t rebuildDirFromCheckpoint(const char* path, int64_t chkpId, char** dst) { @@ -1582,22 +1279,6 @@ int parKeyToString(void* k, char* buf) { n = sprintf(buf + n, "[groupId:%" PRIi64 "]", *key); return n; } -// int stremaValueEncode(void* k, char* buf) { -// int len = 0; -// SStreamValue* key = k; -// len += taosEncodeFixedI64((void**)&buf, key->unixTimestamp); -// len += taosEncodeFixedI32((void**)&buf, key->len); -// len += taosEncodeBinary((void**)&buf, key->data, key->len); -// return len; -// } -// int streamValueDecode(void* k, char* buf) { -// SStreamValue* key = k; -// char* p = buf; -// p = taosDecodeFixedI64(p, &key->unixTimestamp); -// p = taosDecodeFixedI32(p, &key->len); -// p = taosDecodeBinary(p, (void**)&key->data, key->len); -// return p - buf; -// } int32_t valueToString(void* k, char* buf) { SStreamValue* key = k; int n = 0; @@ -1874,7 +1555,7 @@ void taskDbInitOpt(STaskDbWrapper* pTaskDb) { SCfInit* cfPara = &ginitDict[i]; rocksdb_comparator_t* compare = - rocksdb_comparator_create(NULL, cfPara->destroyCmp, cfPara->cmpFunc, cfPara->cmpName); + rocksdb_comparator_create(NULL, cfPara->destroyCmp, cfPara->cmpKey, cfPara->cmpName); rocksdb_options_set_comparator((rocksdb_options_t*)opt, compare); rocksdb_compactionfilterfactory_t* filterFactory = @@ -2181,7 +1862,7 @@ int32_t streamStateOpenBackendCf(void* backend, char* name, char** cfs, int32_t SCfInit* cfPara = &ginitDict[idx]; rocksdb_comparator_t* compare = - rocksdb_comparator_create(NULL, cfPara->destroyCmp, cfPara->cmpFunc, cfPara->cmpName); + rocksdb_comparator_create(NULL, cfPara->destroyCmp, cfPara->cmpKey, cfPara->cmpName); rocksdb_options_set_comparator((rocksdb_options_t*)cfOpts[i], compare); pCompare[i] = compare; } @@ -2265,7 +1946,7 @@ int32_t streamStateOpenBackendCf(void* backend, char* name, char** cfs, int32_t SCfInit* cfPara = &ginitDict[i]; rocksdb_comparator_t* compare = - rocksdb_comparator_create(NULL, cfPara->destroyCmp, cfPara->cmpFunc, cfPara->cmpName); + rocksdb_comparator_create(NULL, cfPara->destroyCmp, cfPara->cmpKey, cfPara->cmpName); rocksdb_options_set_comparator((rocksdb_options_t*)opt, compare); inst->pCompares[i] = compare; @@ -2345,7 +2026,7 @@ int streamStateOpenBackend(void* backend, SStreamState* pState) { for (int i = 0; i < cfLen; i++) { SCfInit* cf = &ginitDict[i]; - rocksdb_comparator_t* compare = rocksdb_comparator_create(NULL, cf->destroyCmp, cf->cmpFunc, cf->cmpName); + rocksdb_comparator_t* compare = rocksdb_comparator_create(NULL, cf->destroyCmp, cf->cmpKey, cf->cmpName); rocksdb_options_set_comparator((rocksdb_options_t*)cfOpt[i], compare); pCompare[i] = compare; } @@ -3672,4 +3353,305 @@ _err: taosMemoryFreeClear(dstName); taosCloseDir(&pDir); return code >= 0 ? 0 : -1; +} + +int32_t compareHashTableImpl(SHashObj* p1, SHashObj* p2, SArray* diff) { + int32_t code = 0; + size_t len = 0; + void* pIter = taosHashIterate(p2, NULL); + while (pIter) { + char* name = taosHashGetKey(pIter, &len); + if (!taosHashGet(p1, name, len)) { + char* p = taosStrdup(name); + taosArrayPush(diff, &p); + } + pIter = taosHashIterate(p2, pIter); + } + return code; +} +int32_t compareHashTable(SHashObj* p1, SHashObj* p2, SArray* add, SArray* del) { + int32_t code = 0; + + code = compareHashTableImpl(p1, p2, add); + code = compareHashTableImpl(p2, p1, del); + + return code; +} + +int32_t dbChkpGetDelta(SDbChkp* p, int64_t chkpId, SArray* list) { + taosThreadRwlockWrlock(&p->rwLock); + + p->preCkptId = p->curChkpId; + p->curChkpId = chkpId; + const char* pCurrent = "CURRENT"; + int32_t currLen = strlen(pCurrent); + + const char* pManifest = "MANIFEST-"; + int32_t maniLen = strlen(pManifest); + + const char* pSST = ".sst"; + int32_t sstLen = strlen(pSST); + + memset(p->buf, 0, p->len); + sprintf(p->buf, "%s%scheckpoint%" PRId64 "", p->path, TD_DIRSEP, chkpId); + + taosArrayClearP(p->pAdd, taosMemoryFree); + taosArrayClearP(p->pDel, taosMemoryFree); + + TdDirPtr pDir = taosOpenDir(p->buf); + TdDirEntryPtr de = NULL; + int8_t dummy = 0; + while ((de = taosReadDir(pDir)) != NULL) { + char* name = taosGetDirEntryName(de); + if (strcmp(name, ".") == 0 || strcmp(name, "..") == 0) continue; + if (strlen(name) == currLen && strcmp(name, pCurrent) == 0) { + taosMemoryFreeClear(p->pCurrent); + p->pCurrent = taosStrdup(name); + taosHashPut(p->pSstTbl[1 - p->idx], name, strlen(name), &dummy, sizeof(dummy)); + continue; + } + + if (strlen(name) >= maniLen && strncmp(name, pManifest, maniLen) == 0) { + taosMemoryFreeClear(p->pManifest); + p->pManifest = taosStrdup(name); + taosHashPut(p->pSstTbl[1 - p->idx], name, strlen(name), &dummy, sizeof(dummy)); + continue; + } + if (strlen(name) >= sstLen && strncmp(name + strlen(name) - 4, pSST, sstLen) == 0) { + // char* p = taosStrdup(name); + taosHashPut(p->pSstTbl[1 - p->idx], name, strlen(name), &dummy, sizeof(dummy)); + continue; + } + } + if (p->init == 0) { + void* pIter = taosHashIterate(p->pSstTbl[1 - p->idx], NULL); + while (pIter) { + size_t len; + char* name = taosHashGetKey(pIter, &len); + if (name != NULL && len != 0) { + taosArrayPush(p->pAdd, &name); + } + pIter = taosHashIterate(p->pSstTbl[1 - p->idx], pIter); + } + if (taosArrayGetSize(p->pAdd) > 0) p->update = 1; + + p->init = 1; + p->preCkptId = -1; + p->curChkpId = chkpId; + } else { + int32_t code = compareHashTable(p->pSstTbl[p->idx], p->pSstTbl[1 - p->idx], p->pAdd, p->pDel); + if (code != 0) { + // dead code + taosArrayClearP(p->pAdd, taosMemoryFree); + taosArrayClearP(p->pDel, taosMemoryFree); + taosHashClear(p->pSstTbl[1 - p->idx]); + p->update = 0; + taosCloseDir(&pDir); + return code; + } + + if (taosArrayGetSize(p->pAdd) == 0 && taosArrayGetSize(p->pDel) == 0) { + p->update = 0; + } + + p->preCkptId = p->curChkpId; + p->curChkpId = chkpId; + } + + taosHashClear(p->pSstTbl[p->idx]); + p->idx = 1 - p->idx; + + taosCloseDir(&pDir); + taosThreadRwlockUnlock(&p->rwLock); + + return 0; +} + +SDbChkp* dbChkpCreate(char* path, int64_t initChkpId) { + SDbChkp* p = taosMemoryCalloc(1, sizeof(SDbChkp)); + p->curChkpId = initChkpId; + p->preCkptId = -1; + p->pSST = taosArrayInit(64, sizeof(void*)); + p->path = taosStrdup(path); + p->len = strlen(path) + 128; + p->buf = taosMemoryCalloc(1, p->len); + + p->idx = 0; + p->pSstTbl[0] = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), false, HASH_ENTRY_LOCK); + p->pSstTbl[1] = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), false, HASH_ENTRY_LOCK); + + p->pAdd = taosArrayInit(64, sizeof(void*)); + p->pDel = taosArrayInit(64, sizeof(void*)); + p->update = 0; + taosThreadRwlockInit(&p->rwLock, NULL); + + SArray* list = NULL; + int32_t code = dbChkpGetDelta(p, initChkpId, list); + + return p; +} + +void dbChkpDestroy(SDbChkp* pChkp) { + taosMemoryFree(pChkp->buf); + taosMemoryFree(pChkp->path); + + taosArrayDestroyP(pChkp->pSST, taosMemoryFree); + taosArrayDestroyP(pChkp->pAdd, taosMemoryFree); + taosArrayDestroyP(pChkp->pDel, taosMemoryFree); + + taosHashCleanup(pChkp->pSstTbl[0]); + taosHashCleanup(pChkp->pSstTbl[1]); + + taosMemoryFree(pChkp->pCurrent); + taosMemoryFree(pChkp->pManifest); +} + +int32_t dbChkpInit(SDbChkp* p) { + if (p == NULL) return 0; + return 0; +} +int32_t dbChkpDumpTo(SDbChkp* p, char* dname) { + taosThreadRwlockRdlock(&p->rwLock); + int32_t code = 0; + int32_t len = p->len + 128; + + char* srcBuf = taosMemoryCalloc(1, len); + char* dstBuf = taosMemoryCalloc(1, len); + + char* srcDir = taosMemoryCalloc(1, len); + char* dstDir = taosMemoryCalloc(1, len); + + sprintf(srcDir, "%s%s%s%" PRId64 "", p->path, TD_DIRSEP, "checkpoint", p->curChkpId); + sprintf(dstDir, "%s%s%s", p->path, TD_DIRSEP, dname); + + if (!taosDirExist(srcDir)) { + stError("failed to dump srcDir %s, reason: not exist such dir", srcDir); + code = -1; + goto _ERROR; + } + + code = taosMkDir(dstDir); + if (code != 0) { + terrno = TAOS_SYSTEM_ERROR(errno); + stError("failed to mkdir srcDir %s, reason: %s", dstDir, terrstr()); + goto _ERROR; + } + + // clear current file + memset(dstBuf, 0, len); + sprintf(dstBuf, "%s%s%s", dstDir, TD_DIRSEP, p->pCurrent); + taosRemoveFile(dstBuf); + + memset(dstBuf, 0, len); + sprintf(dstBuf, "%s%s%s", dstDir, TD_DIRSEP, p->pManifest); + taosRemoveFile(dstBuf); + + // add file to $name dir + for (int i = 0; i < taosArrayGetSize(p->pAdd); i++) { + memset(dstBuf, 0, len); + memset(srcBuf, 0, len); + + char* filename = taosArrayGetP(p->pAdd, i); + sprintf(srcBuf, "%s%s%s", srcDir, TD_DIRSEP, filename); + sprintf(dstBuf, "%s%s%s", dstDir, TD_DIRSEP, filename); + + taosCopyFile(srcBuf, dstBuf); + } + // del file in $name + for (int i = 0; i < taosArrayGetSize(p->pDel); i++) { + memset(dstBuf, 0, len); + memset(srcBuf, 0, len); + + char* filename = taosArrayGetP(p->pDel, i); + sprintf(dstBuf, "%s%s%s", dstDir, TD_DIRSEP, filename); + taosRemoveFile(dstBuf); + } + + // copy current file to dst dir + memset(srcBuf, 0, len); + memset(dstBuf, 0, len); + sprintf(srcBuf, "%s%s%s", srcDir, TD_DIRSEP, p->pCurrent); + sprintf(dstBuf, "%s%s%s", dstDir, TD_DIRSEP, p->pCurrent); + taosCopyFile(srcBuf, dstBuf); + + // copy manifest file to dst dir + memset(srcBuf, 0, len); + memset(dstBuf, 0, len); + sprintf(srcBuf, "%s%s%s", srcDir, TD_DIRSEP, p->pManifest); + sprintf(dstBuf, "%s%s%s", dstDir, TD_DIRSEP, p->pManifest); + taosCopyFile(srcBuf, dstBuf); + + // clear delta data buf + taosArrayClearP(p->pAdd, taosMemoryFree); + taosArrayClearP(p->pDel, taosMemoryFree); + +_ERROR: + taosThreadRwlockUnlock(&p->rwLock); + taosMemoryFree(srcBuf); + taosMemoryFree(dstBuf); + taosMemoryFree(srcDir); + taosMemoryFree(dstDir); + return code; +} +SBkdMgt* bkdMgtCreate(char* path) { + SBkdMgt* p = taosMemoryCalloc(1, sizeof(SBkdMgt)); + p->pDbChkpTbl = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), true, HASH_ENTRY_LOCK); + taosThreadRwlockInit(&p->rwLock, NULL); + return p; +} + +void bkdMgtDestroy(SBkdMgt* bm) { + if (bm == NULL) return; + void* pIter = taosHashIterate(bm->pDbChkpTbl, NULL); + while (pIter) { + SDbChkp* pChkp = *(SDbChkp**)(pIter); + dbChkpDestroy(pChkp); + + pIter = taosHashIterate(bm->pDbChkpTbl, pIter); + } + + taosThreadRwlockDestroy(&bm->rwLock); + + taosMemoryFree(bm); +} +int32_t bkdMgtGetDelta(SBkdMgt* bm, char* taskId, int64_t chkpId, SArray* list) { + int32_t code = 0; + taosThreadRwlockWrlock(&bm->rwLock); + + SDbChkp* pChkp = taosHashGet(bm->pDbChkpTbl, taskId, strlen(taskId)); + code = dbChkpGetDelta(pChkp, chkpId, list); + + taosThreadRwlockUnlock(&bm->rwLock); + return code; +} + +int32_t bkdMgtAddChkp(SBkdMgt* bm, char* task, char* path) { + int32_t code = -1; + + taosThreadRwlockWrlock(&bm->rwLock); + SDbChkp** pp = taosHashGet(bm->pDbChkpTbl, task, strlen(task)); + if (pp == NULL) { + SDbChkp* p = dbChkpCreate(path, 0); + if (p != NULL) { + taosHashPut(bm->pDbChkpTbl, task, strlen(task), &p, sizeof(void*)); + code = 0; + } + } else { + stError("task chkp already exists"); + } + + taosThreadRwlockUnlock(&bm->rwLock); + + return code; +} + +int32_t bkdMgtDumpTo(SBkdMgt* bm, char* taskId, char* dname) { + int32_t code = 0; + taosThreadRwlockRdlock(&bm->rwLock); + + SDbChkp* p = taosHashGet(bm->pDbChkpTbl, taskId, strlen(taskId)); + code = dbChkpDumpTo(p, dname); + + taosThreadRwlockUnlock(&bm->rwLock); + return code; } \ No newline at end of file From 0b4946eefbc9e232d083d72c096c290dcc8ef7de Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Tue, 7 Nov 2023 17:25:04 +0800 Subject: [PATCH 091/195] merge 3.0 --- source/common/src/tglobal.c | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index c7d2091383..7339c70487 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -675,10 +675,10 @@ static int32_t taosAddServerCfg(SConfig *pCfg) { if (cfgAddString(pCfg, "telemetryServer", tsTelemServer, CFG_SCOPE_BOTH, CFG_DYN_BOTH) != 0) return -1; if (cfgAddInt32(pCfg, "telemetryPort", tsTelemPort, 1, 65056, CFG_SCOPE_BOTH, CFG_DYN_NONE) != 0) return -1; - if (cfgAddString(pCfg, "snodeIp", tsSnodeIp, CFG_SCOPE_SERVER) != 0) return -1; - if (cfgAddString(pCfg, "checkpointBackupDir", tsCheckpointBackupDir, CFG_SCOPE_SERVER) != 0) return -1; + if (cfgAddString(pCfg, "snodeIp", tsSnodeIp, CFG_SCOPE_SERVER, CFG_DYN_NONE) != 0) return -1; + if (cfgAddString(pCfg, "checkpointBackupDir", tsCheckpointBackupDir, CFG_SCOPE_SERVER, CFG_DYN_NONE) != 0) return -1; - if (cfgAddInt32(pCfg, "tmqMaxTopicNum", tmqMaxTopicNum, 1, 10000, CFG_SCOPE_SERVER) != 0) return -1; + if (cfgAddInt32(pCfg, "tmqMaxTopicNum", tmqMaxTopicNum, 1, 10000, CFG_SCOPE_SERVER, CFG_DYN_NONE) != 0) return -1; if (cfgAddInt32(pCfg, "transPullupInterval", tsTransPullupInterval, 1, 10000, CFG_SCOPE_SERVER, CFG_DYN_ENT_SERVER) != 0) From 594da2d6031182299a0d0f6470d0423a8ea9d722 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 8 Nov 2023 10:04:46 +0800 Subject: [PATCH 092/195] merge 3.0 --- source/libs/stream/inc/streamBackendRocksdb.h | 60 ++++++++++++++++++- source/libs/stream/src/streamBackendRocksdb.c | 51 +--------------- 2 files changed, 60 insertions(+), 51 deletions(-) diff --git a/source/libs/stream/inc/streamBackendRocksdb.h b/source/libs/stream/inc/streamBackendRocksdb.h index 122f803ab6..c9cb5ab64c 100644 --- a/source/libs/stream/inc/streamBackendRocksdb.h +++ b/source/libs/stream/inc/streamBackendRocksdb.h @@ -66,7 +66,7 @@ typedef struct { TdThreadMutex mutex; char* idstr; char* path; - int64_t refId; + int64_t refId; void* pTask; int64_t streamId; @@ -79,6 +79,53 @@ typedef struct { } STaskDbWrapper; +typedef struct SDbChkp { + int8_t init; + char* pCurrent; + char* pManifest; + SArray* pSST; + int64_t preCkptId; + int64_t curChkpId; + char* path; + + char* buf; + int32_t len; + + // ping-pong buf + SHashObj* pSstTbl[2]; + int8_t idx; + + SArray* pAdd; + SArray* pDel; + int8_t update; + + TdThreadRwlock rwLock; +} SDbChkp; +typedef struct { + int8_t init; + char* pCurrent; + char* pManifest; + SArray* pSST; + int64_t preCkptId; + int64_t curChkpId; + char* path; + + char* buf; + int32_t len; + + // ping-pong buf + SHashObj* pSstTbl[2]; + int8_t idx; + + SArray* pAdd; + SArray* pDel; + int8_t update; + + SHashObj* pDbChkpTbl; + + TdThreadRwlock rwLock; +} SBkdMgt; + void* streamBackendInit(const char* path, int64_t chkpId); void streamBackendCleanup(void* arg); void streamBackendHandleCleanup(void* arg); @@ -194,4 +241,15 @@ int32_t streamBackendDelInUseChkp(void* arg, int64_t chkpId); int32_t taskDbBuildSnap(void* arg, SArray* pSnap); // int32_t streamDefaultIter_rocksdb(SStreamState* pState, const void* start, const void* end, SArray* result); + +STaskDbWrapper* taskDbOpen(char* path, char* key, int64_t chkpId); +void taskDbDestroy(void* pDb); + +int32_t taskDbDoCheckpoint(void* arg, int64_t chkpId); + +SBkdMgt* bkdMgtCreate(char* path); +int32_t bkdMgtAddChkp(SBkdMgt* bm, char* task, char* path); +int32_t bkdMgtGetDelta(SBkdMgt* bm, char* taskId, int64_t chkpId, SArray* list); +int32_t bkdMgtDumpTo(SBkdMgt* bm, char* taskId, char* dname); +void bkdMgtDestroy(SBkdMgt* bm); #endif \ No newline at end of file diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 074d23cdbf..a06ec8c9c6 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -20,53 +20,6 @@ #include "tcommon.h" #include "tref.h" -typedef struct SDbChkp { - int8_t init; - char* pCurrent; - char* pManifest; - SArray* pSST; - int64_t preCkptId; - int64_t curChkpId; - char* path; - - char* buf; - int32_t len; - - // ping-pong buf - SHashObj* pSstTbl[2]; - int8_t idx; - - SArray* pAdd; - SArray* pDel; - int8_t update; - - TdThreadRwlock rwLock; -} SDbChkp; -typedef struct { - int8_t init; - char* pCurrent; - char* pManifest; - SArray* pSST; - int64_t preCkptId; - int64_t curChkpId; - char* path; - - char* buf; - int32_t len; - - // ping-pong buf - SHashObj* pSstTbl[2]; - int8_t idx; - - SArray* pAdd; - SArray* pDel; - int8_t update; - - SHashObj* pDbChkpTbl; - - TdThreadRwlock rwLock; -} SBkdMgt; - typedef struct SCompactFilteFactory { void* status; } SCompactFilteFactory; @@ -223,8 +176,6 @@ SCfInit ginitDict[] = { valueEncode, valueDecode, compactFilteFactoryCreateFilter, destroyCompactFilteFactory, compactFilteFactoryName}, }; -const char* cfName[] = {"default", "state", "fill", "sess", "func", "parname", "partag"}; - int32_t getCfIdx(const char* cfName) { int idx = -1; size_t len = strlen(cfName); @@ -868,7 +819,6 @@ int32_t taskDbBuildSnap(void* arg, SArray* pSnap) { code = taskDbDoCheckpoint(pTaskDb, pTaskDb->chkpId); taskDbRemoveRef(pTaskDb); - pIter = taosHashIterate(pMeta->pTaskDbUnique, pIter); SStreamTask* pTask = pTaskDb->pTask; SStreamTaskSnap snap = {.streamId = pTask->id.streamId, @@ -876,6 +826,7 @@ int32_t taskDbBuildSnap(void* arg, SArray* pSnap) { .chkpId = pTaskDb->chkpId, .dbPrefixPath = taosStrdup(pTaskDb->path)}; taosArrayPush(pSnap, &snap); + pIter = taosHashIterate(pMeta->pTaskDbUnique, pIter); } return code; } From 17bd6badec1063bd78fddd7b82c603fec4e93ff6 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 8 Nov 2023 10:47:54 +0800 Subject: [PATCH 093/195] refactor(stream): do checkpoint for each task. --- source/dnode/vnode/src/tq/tq.c | 7 +- source/libs/stream/inc/streamInt.h | 2 +- source/libs/stream/src/streamCheckpoint.c | 120 +++++++++++----------- 3 files changed, 64 insertions(+), 65 deletions(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 1f0f6a3e92..4d5ddec233 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1702,13 +1702,12 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) return TSDB_CODE_SUCCESS; } - // todo handle failure to reset from checkpoint procedure - // downstream not ready, current the stream tasks are not all ready. Ignore this checkpoint req. + // Downstream not ready, current the stream tasks are not all ready. Ignore this checkpoint req. if (pTask->status.downstreamReady != 1) { pTask->chkInfo.failedId = req.checkpointId; // record the latest failed checkpoint id pTask->checkpointingId = req.checkpointId; - qError("s-task:%s not ready for checkpoint, since downstream not ready, ignore this checkpoint:%" PRId64 + tqError("s-task:%s not ready for checkpoint, since downstream not ready, ignore this checkpoint:%" PRId64 ", set it failure", pTask->id.idStr, req.checkpointId); streamMetaReleaseTask(pMeta, pTask); @@ -1735,6 +1734,8 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) tmsgSendRsp(&rsp); // error occurs return TSDB_CODE_SUCCESS; } + + // todo: already in checkpoint status, return error streamProcessCheckpointSourceReq(pTask, &req); taosThreadMutexUnlock(&pTask->lock); diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index feaacb7969..095461bd92 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -112,7 +112,7 @@ int32_t streamBroadcastToChildren(SStreamTask* pTask, const SSDataBlock* pBlock) int32_t tEncodeStreamRetrieveReq(SEncoder* pEncoder, const SStreamRetrieveReq* pReq); -int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, int64_t checkpointId); +int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, SStreamTask* p, int64_t checkpointId); int32_t streamTaskBuildCheckpoint(SStreamTask* pTask); int32_t streamSendCheckMsg(SStreamTask* pTask, const SStreamTaskCheckReq* pReq, int32_t nodeId, SEpSet* pEpSet); diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 3a52e68e93..bf2c89bea4 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -146,7 +146,6 @@ int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSo pTask->execInfo.checkpoint += 1; // 2. Put the checkpoint block into inputQ, to make sure all blocks with less version have been handled by this task - // already. int32_t code = appendCheckpointIntoInputQ(pTask, STREAM_INPUT__CHECKPOINT_TRIGGER); return code; } @@ -169,9 +168,8 @@ static int32_t continueDispatchCheckpointBlock(SStreamDataBlock* pBlock, SStream int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBlock) { SSDataBlock* pDataBlock = taosArrayGet(pBlock->blocks, 0); int64_t checkpointId = pDataBlock->info.version; - - const char* id = pTask->id.idStr; - int32_t code = TSDB_CODE_SUCCESS; + const char* id = pTask->id.idStr; + int32_t code = TSDB_CODE_SUCCESS; // set task status if (streamTaskGetStatus(pTask, NULL) != TASK_STATUS__CK) { @@ -197,15 +195,15 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc // todo fix race condition: set the status and append checkpoint block int32_t taskLevel = pTask->info.taskLevel; if (taskLevel == TASK_LEVEL__SOURCE) { - if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH || - pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { + int8_t type = pTask->outputInfo.type; + if (type == TASK_OUTPUT__FIXED_DISPATCH || type == TASK_OUTPUT__SHUFFLE_DISPATCH) { stDebug("s-task:%s set childIdx:%d, and add checkpoint-trigger block into outputQ", id, pTask->info.selfChildId); continueDispatchCheckpointBlock(pBlock, pTask); - } else { // only one task exists, no need to dispatch downstream info - atomic_add_fetch_32(&pTask->checkpointNotReadyTasks, 1); - streamProcessCheckpointReadyMsg(pTask); - streamFreeQitem((SStreamQueueItem*)pBlock); - } + } else { // only one task exists, no need to dispatch downstream info + atomic_add_fetch_32(&pTask->checkpointNotReadyTasks, 1); + streamProcessCheckpointReadyMsg(pTask); + streamFreeQitem((SStreamQueueItem*)pBlock); + } } else if (taskLevel == TASK_LEVEL__SINK || taskLevel == TASK_LEVEL__AGG) { ASSERT(taosArrayGetSize(pTask->upstreamInfo.pList) > 0); if (pTask->chkInfo.startTs == 0) { @@ -231,11 +229,9 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc id, num); streamFreeQitem((SStreamQueueItem*)pBlock); streamTaskBuildCheckpoint(pTask); - } else { - stDebug( - "s-task:%s process checkpoint block, all %d upstreams sent checkpoint msgs, dispatch checkpoint msg " - "downstream", - id, num); + } else { // source & agg tasks need to forward the checkpoint msg downwards + stDebug("s-task:%s process checkpoint block, all %d upstreams sent checkpoint msgs, continue forwards msg", id, + num); // set the needed checked downstream tasks, only when all downstream tasks do checkpoint complete, this task // can start local checkpoint procedure @@ -282,48 +278,47 @@ void streamTaskClearCheckInfo(SStreamTask* pTask) { streamTaskOpenAllUpstreamInput(pTask); // open inputQ for all upstream tasks } -int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, int64_t checkpointId) { +int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, SStreamTask* p, int64_t checkpointId) { int32_t vgId = pMeta->vgId; int32_t code = 0; streamMetaWLock(pMeta); - for (int32_t i = 0; i < taosArrayGetSize(pMeta->pTaskList); ++i) { - STaskId* pId = taosArrayGet(pMeta->pTaskList, i); - SStreamTask** ppTask = taosHashGet(pMeta->pTasksMap, pId, sizeof(*pId)); - if (ppTask == NULL) { - continue; - } + // for (int32_t i = 0; i < taosArrayGetSize(pMeta->pTaskList); ++i) { + // STaskId* pId = taosArrayGet(pMeta->pTaskList, i); + // SStreamTask** ppTask = taosHashGet(pMeta->pTasksMap, pId, sizeof(*pId)); + // if (ppTask == NULL) { + // continue; + // } - SStreamTask* p = *ppTask; - if (p->info.fillHistory == 1) { - continue; - } - - ASSERT(p->chkInfo.checkpointId < p->checkpointingId && p->checkpointingId == checkpointId); - - p->chkInfo.checkpointId = p->checkpointingId; - streamTaskClearCheckInfo(p); - - char* str = NULL; - streamTaskGetStatus(p, &str); - - code = streamTaskHandleEvent(p->status.pSM, TASK_EVENT_CHECKPOINT_DONE); - if (code != TSDB_CODE_SUCCESS) { - stDebug("s-task:%s vgId:%d save task status failed, since handle event failed", p->id.idStr, vgId); - streamMetaWUnLock(pMeta); - return -1; - } else { // save the task - streamMetaSaveTask(pMeta, p); - } - - stDebug( - "vgId:%d s-task:%s level:%d open upstream inputQ, commit task status after checkpoint completed, " - "checkpointId:%" PRId64 ", Ver(saved):%" PRId64 " currentVer:%" PRId64 ", status to be normal, prev:%s", - pMeta->vgId, p->id.idStr, p->info.taskLevel, checkpointId, p->chkInfo.checkpointVer, p->chkInfo.nextProcessVer, - str); + // SStreamTask* p = *ppTask; + if (p->info.fillHistory == 1) { + // continue; } + ASSERT(p->chkInfo.checkpointId < p->checkpointingId && p->checkpointingId == checkpointId); + + p->chkInfo.checkpointId = p->checkpointingId; + streamTaskClearCheckInfo(p); + + char* str = NULL; + streamTaskGetStatus(p, &str); + + code = streamTaskHandleEvent(p->status.pSM, TASK_EVENT_CHECKPOINT_DONE); + if (code != TSDB_CODE_SUCCESS) { + stDebug("s-task:%s vgId:%d save task status failed, since handle event failed", p->id.idStr, vgId); + streamMetaWUnLock(pMeta); + return -1; + } else { // save the task + streamMetaSaveTask(pMeta, p); + } + + stDebug( + "vgId:%d s-task:%s level:%d open upstream inputQ, commit task status after checkpoint completed, " + "checkpointId:%" PRId64 ", Ver(saved):%" PRId64 " currentVer:%" PRId64 ", status to be normal, prev:%s", + pMeta->vgId, p->id.idStr, p->info.taskLevel, checkpointId, p->chkInfo.checkpointVer, p->chkInfo.nextProcessVer, + str); + code = streamMetaCommit(pMeta); if (code < 0) { stError("vgId:%d failed to commit stream meta after do checkpoint, checkpointId:%" PRId64 ", since %s", pMeta->vgId, @@ -341,24 +336,24 @@ int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { // check for all tasks, and do generate the vnode-wide checkpoint data. SStreamMeta* pMeta = pTask->pMeta; - int32_t remain = atomic_sub_fetch_32(&pMeta->chkptNotReadyTasks, 1); - ASSERT(remain >= 0); +// int32_t remain = atomic_sub_fetch_32(&pMeta->chkptNotReadyTasks, 1); +// ASSERT(remain >= 0); double el = (taosGetTimestampMs() - pTask->chkInfo.startTs) / 1000.0; - if (remain == 0) { // all tasks are ready +// if (remain == 0) { // all tasks are ready stDebug("s-task:%s all downstreams are ready, ready for do checkpoint", pTask->id.idStr); - streamBackendDoCheckpoint(pMeta, pTask->checkpointingId); - streamSaveAllTaskStatus(pMeta, pTask->checkpointingId); + streamBackendDoCheckpoint(pTask->pBackend, pTask->checkpointingId); + streamSaveAllTaskStatus(pMeta, pTask, pTask->checkpointingId); stInfo( "vgId:%d vnode wide checkpoint completed, save all tasks status, last:%s, level:%d elapsed time:%.2f Sec " "checkpointId:%" PRId64, pMeta->vgId, pTask->id.idStr, pTask->info.taskLevel, el, pTask->checkpointingId); - } else { - stInfo( - "vgId:%d vnode wide tasks not reach checkpoint ready status, ready s-task:%s, level:%d elapsed time:%.2f Sec " - "not ready:%d/%d", - pMeta->vgId, pTask->id.idStr, pTask->info.taskLevel, el, remain, pMeta->numOfStreamTasks); - } +// } else { +// stInfo( +// "vgId:%d vnode wide tasks not reach checkpoint ready status, ready s-task:%s, level:%d elapsed time:%.2f Sec " +// "not ready:%d/%d", +// pMeta->vgId, pTask->id.idStr, pTask->info.taskLevel, el, remain, pMeta->numOfStreamTasks); +// } // send check point response to upstream task if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { @@ -368,6 +363,9 @@ int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { } if (code != TSDB_CODE_SUCCESS) { + // record the failure checkpoint id + pTask->chkInfo.failedId = pTask->checkpointingId; + // todo: let's retry send rsp to upstream/mnode stError("s-task:%s failed to send checkpoint rsp to upstream, checkpointId:%" PRId64 ", code:%s", pTask->id.idStr, pTask->checkpointingId, tstrerror(code)); From 7b0891981ecfcbc1a16843a6e92a9c0dbabe9474 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 8 Nov 2023 14:14:01 +0800 Subject: [PATCH 094/195] merge 3.0 --- include/libs/stream/tstream.h | 105 ++++++++++++++-------------- source/libs/stream/src/streamMeta.c | 19 +++-- 2 files changed, 62 insertions(+), 62 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index e4bca14a9e..b41fc75782 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -39,16 +39,16 @@ extern "C" { #define TASK_DOWNSTREAM_NOT_LEADER 0x2 #define TASK_SELF_NEW_STAGE 0x3 -#define TASK_DOWNSTREAM_READY 0x0 -#define TASK_DOWNSTREAM_NOT_READY 0x1 -#define TASK_DOWNSTREAM_NOT_LEADER 0x2 -#define TASK_UPSTREAM_NEW_STAGE 0x3 +#define TASK_DOWNSTREAM_READY 0x0 +#define TASK_DOWNSTREAM_NOT_READY 0x1 +#define TASK_DOWNSTREAM_NOT_LEADER 0x2 +#define TASK_UPSTREAM_NEW_STAGE 0x3 -#define NODE_ROLE_UNINIT 0x1 -#define NODE_ROLE_LEADER 0x2 -#define NODE_ROLE_FOLLOWER 0x3 +#define NODE_ROLE_UNINIT 0x1 +#define NODE_ROLE_LEADER 0x2 +#define NODE_ROLE_FOLLOWER 0x3 -#define HAS_RELATED_FILLHISTORY_TASK(_t) ((_t)->hTaskInfo.id.taskId != 0) +#define HAS_RELATED_FILLHISTORY_TASK(_t) ((_t)->hTaskInfo.id.taskId != 0) #define CLEAR_RELATED_FILLHISTORY_TASK(_t) \ do { \ (_t)->hTaskInfo.id.taskId = 0; \ @@ -378,7 +378,7 @@ typedef struct SHistoryTaskInfo { int32_t tickCount; int32_t retryTimes; int32_t waitInterval; - int64_t haltVer; // offset in wal when halt the stream task + int64_t haltVer; // offset in wal when halt the stream task } SHistoryTaskInfo; typedef struct STaskOutputInfo { @@ -442,7 +442,7 @@ typedef struct STaskStartInfo { int64_t startTs; int64_t readyTs; int32_t startAllTasksFlag; - SHashObj* pReadyTaskSet; // tasks that are all ready for running stream processing + SHashObj* pReadyTaskSet; // tasks that are all ready for running stream processing int32_t elapsedTime; } STaskStartInfo; @@ -453,32 +453,32 @@ typedef struct STaskUpdateInfo { // meta typedef struct SStreamMeta { - char* path; - TDB* db; - TTB* pTaskDb; - TTB* pCheckpointDb; - SHashObj* pTasksMap; - SArray* pTaskList; // SArray - void* ahandle; - TXN* txn; - FTaskExpand* expandFunc; - int32_t vgId; - int64_t stage; - int32_t role; - STaskStartInfo startInfo; - SRWLatch lock; - int32_t walScanCounter; - void* streamBackend; - int64_t streamBackendRid; - SHashObj* pTaskDbUnique; - TdThreadMutex backendMutex; - SMetaHbInfo* pHbInfo; + char* path; + TDB* db; + TTB* pTaskDb; + TTB* pCheckpointDb; + SHashObj* pTasksMap; + SArray* pTaskList; // SArray + void* ahandle; + TXN* txn; + FTaskExpand* expandFunc; + int32_t vgId; + int64_t stage; + int32_t role; + STaskStartInfo startInfo; + SRWLatch lock; + int32_t walScanCounter; + void* streamBackend; + int64_t streamBackendRid; + SHashObj* pTaskDbUnique; + TdThreadMutex backendMutex; + SMetaHbInfo* pHbInfo; STaskUpdateInfo updateInfo; - SHashObj* pUpdateTaskSet; - int32_t numOfStreamTasks; // this value should be increased when a new task is added into the meta - int32_t numOfPausedTasks; - int32_t chkptNotReadyTasks; - int64_t rid; + SHashObj* pUpdateTaskSet; + int32_t numOfStreamTasks; // this value should be increased when a new task is added into the meta + int32_t numOfPausedTasks; + int32_t chkptNotReadyTasks; + int64_t rid; int64_t chkpId; int32_t chkpCap; @@ -486,6 +486,7 @@ typedef struct SStreamMeta { SArray* chkpInUse; SRWLatch chkpDirLock; + void* qHandle; int32_t pauseTaskNum; } SStreamMeta; @@ -658,8 +659,8 @@ typedef struct STaskStatusEntry { typedef struct SStreamHbMsg { int32_t vgId; int32_t numOfTasks; - SArray* pTaskStatus; // SArray - SArray* pUpdateNodes; // SArray, needs update the epsets in stream tasks for those nodes. + SArray* pTaskStatus; // SArray + SArray* pUpdateNodes; // SArray, needs update the epsets in stream tasks for those nodes. } SStreamHbMsg; int32_t tEncodeStreamHbMsg(SEncoder* pEncoder, const SStreamHbMsg* pRsp); @@ -683,7 +684,7 @@ typedef struct SNodeUpdateInfo { } SNodeUpdateInfo; typedef struct SStreamTaskNodeUpdateMsg { - int32_t transId; // to identify the msg + int32_t transId; // to identify the msg int64_t streamId; int32_t taskId; SArray* pNodeList; // SArray @@ -740,7 +741,7 @@ const char* streamTaskGetStatusStr(ETaskStatus status); void streamTaskResetStatus(SStreamTask* pTask); void streamTaskSetStatusReady(SStreamTask* pTask); -void initRpcMsg(SRpcMsg* pMsg, int32_t msgType, void* pCont, int32_t contLen); +void initRpcMsg(SRpcMsg* pMsg, int32_t msgType, void* pCont, int32_t contLen); // recover and fill history void streamTaskCheckDownstream(SStreamTask* pTask); @@ -771,18 +772,18 @@ bool streamHistoryTaskSetVerRangeStep2(SStreamTask* pTask, int64_t latestVer) int32_t streamQueueGetNumOfItems(const SStreamQueue* pQueue); // common -int32_t streamRestoreParam(SStreamTask* pTask); -void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta); -void streamTaskResume(SStreamTask* pTask); -void streamTaskEnablePause(SStreamTask* pTask); -int32_t streamTaskSetUpstreamInfo(SStreamTask* pTask, const SStreamTask* pUpstreamTask); -void streamTaskUpdateUpstreamInfo(SStreamTask* pTask, int32_t nodeId, const SEpSet* pEpSet); -void streamTaskUpdateDownstreamInfo(SStreamTask* pTask, int32_t nodeId, const SEpSet* pEpSet); -void streamTaskSetFixedDownstreamInfo(SStreamTask* pTask, const SStreamTask* pDownstreamTask); -int32_t streamTaskReleaseState(SStreamTask* pTask); -int32_t streamTaskReloadState(SStreamTask* pTask); -void streamTaskCloseUpstreamInput(SStreamTask* pTask, int32_t taskId); -void streamTaskOpenAllUpstreamInput(SStreamTask* pTask); +int32_t streamRestoreParam(SStreamTask* pTask); +void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta); +void streamTaskResume(SStreamTask* pTask); +void streamTaskEnablePause(SStreamTask* pTask); +int32_t streamTaskSetUpstreamInfo(SStreamTask* pTask, const SStreamTask* pUpstreamTask); +void streamTaskUpdateUpstreamInfo(SStreamTask* pTask, int32_t nodeId, const SEpSet* pEpSet); +void streamTaskUpdateDownstreamInfo(SStreamTask* pTask, int32_t nodeId, const SEpSet* pEpSet); +void streamTaskSetFixedDownstreamInfo(SStreamTask* pTask, const SStreamTask* pDownstreamTask); +int32_t streamTaskReleaseState(SStreamTask* pTask); +int32_t streamTaskReloadState(SStreamTask* pTask); +void streamTaskCloseUpstreamInput(SStreamTask* pTask, int32_t taskId); +void streamTaskOpenAllUpstreamInput(SStreamTask* pTask); void streamTaskStatusInit(STaskStatusEntry* pEntry, const SStreamTask* pTask); void streamTaskStatusCopy(STaskStatusEntry* pDst, const STaskStatusEntry* pSrc); @@ -815,7 +816,7 @@ int32_t streamMetaCommit(SStreamMeta* pMeta); int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta); int32_t streamMetaReloadAllTasks(SStreamMeta* pMeta); void streamMetaNotifyClose(SStreamMeta* pMeta); -int32_t streamTaskSetDb(SStreamMeta* pMeta, void* pTask, char *key); +int32_t streamTaskSetDb(SStreamMeta* pMeta, void* pTask, char* key); void streamMetaStartHb(SStreamMeta* pMeta); void streamMetaInitForSnode(SStreamMeta* pMeta); bool streamMetaTaskInTimer(SStreamMeta* pMeta); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index b6c973f0d0..5e5166cc34 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -18,6 +18,7 @@ #include "streamInt.h" #include "tmisce.h" #include "tref.h" +#include "tsched.h" #include "tstream.h" #include "ttimer.h" #include "wal.h" @@ -227,7 +228,7 @@ int32_t streamMetaMayCvtDbFormat(SStreamMeta* pMeta) { return 0; } -int32_t streamTaskSetDb(SStreamMeta* pMeta, void* arg, char *key) { +int32_t streamTaskSetDb(SStreamMeta* pMeta, void* arg, char* key) { SStreamTask* pTask = arg; int64_t chkpId = pTask->checkpointingId; @@ -320,8 +321,6 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF pMeta->expandFunc = expandFunc; pMeta->stage = stage; - - pMeta->pTaskDbUnique = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), false, HASH_ENTRY_LOCK); // pMeta->chkpId = streamGetLatestCheckpointId(pMeta); @@ -350,10 +349,10 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF pMeta->pHbInfo->hbTmr = taosTmrStart(metaHbToMnode, META_HB_CHECK_INTERVAL, pRid, streamEnv.timer); pMeta->pHbInfo->tickCounter = 0; pMeta->pHbInfo->stopFlag = 0; - + pMeta->qHandle = taosInitScheduler(32, 1, "stream-chkp", NULL); return pMeta; - _err: +_err: taosMemoryFree(pMeta->path); if (pMeta->pTasksMap) taosHashCleanup(pMeta->pTasksMap); if (pMeta->pTaskList) taosArrayDestroy(pMeta->pTaskList); @@ -483,7 +482,7 @@ void streamMetaCloseImpl(void* arg) { taosHashCleanup(pMeta->pTasksMap); taosHashCleanup(pMeta->pTaskDbUnique); taosHashCleanup(pMeta->pUpdateTaskSet); - //taosHashCleanup(pMeta->pTaskBackendUnique); + // taosHashCleanup(pMeta->pTaskBackendUnique); taosHashCleanup(pMeta->updateInfo.pTasks); taosHashCleanup(pMeta->startInfo.pReadyTaskSet); @@ -1144,7 +1143,9 @@ void metaHbToMnode(void* param, void* tmrId) { } tEncoderClear(&encoder); - SRpcMsg msg = {.info.noResp = 1,}; + SRpcMsg msg = { + .info.noResp = 1, + }; initRpcMsg(&msg, TDMT_MND_STREAM_HEARTBEAT, buf, tlen); pMeta->pHbInfo->hbCount += 1; @@ -1156,7 +1157,7 @@ void metaHbToMnode(void* param, void* tmrId) { stDebug("vgId:%d no tasks and no mnd epset, not send stream hb to mnode", pMeta->vgId); } - _end: +_end: clearHbMsg(&hbMsg, pIdList); taosTmrReset(metaHbToMnode, META_HB_CHECK_INTERVAL, param, streamEnv.timer, &pMeta->pHbInfo->hbTmr); taosReleaseRef(streamMetaId, rid); @@ -1251,7 +1252,6 @@ void streamMetaRLock(SStreamMeta* pMeta) { void streamMetaRUnLock(SStreamMeta* pMeta) { stTrace("vgId:%d meta-runlock", pMeta->vgId); taosRUnLockLatch(&pMeta->lock); - } void streamMetaWLock(SStreamMeta* pMeta) { stTrace("vgId:%d meta-wlock", pMeta->vgId); @@ -1261,4 +1261,3 @@ void streamMetaWUnLock(SStreamMeta* pMeta) { stTrace("vgId:%d meta-wunlock", pMeta->vgId); taosWUnLockLatch(&pMeta->lock); } - From 99dbbe61d7b3b35521823f8b4f2608118bc6c5be Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 8 Nov 2023 15:20:47 +0800 Subject: [PATCH 095/195] refactor(stream): do some internal refactor. --- include/libs/stream/tstream.h | 7 +- source/dnode/vnode/src/tq/tq.c | 6 +- source/libs/stream/inc/streamInt.h | 2 +- source/libs/stream/src/streamCheckpoint.c | 98 ++++++++--------------- source/libs/stream/src/streamMeta.c | 9 +-- 5 files changed, 44 insertions(+), 78 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index e4bca14a9e..d46c107712 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -294,6 +294,9 @@ typedef struct SCheckpointInfo { int64_t checkpointVer; // latest checkpointId version int64_t nextProcessVer; // current offset in WAL, not serialize it int64_t failedId; // record the latest failed checkpoint id + int64_t checkpointingId; + int32_t downstreamAlignNum; + int32_t checkpointNotReadyTasks; int64_t msgVer; } SCheckpointInfo; @@ -427,9 +430,6 @@ struct SStreamTask { int64_t checkReqId; SArray* checkReqIds; // shuffle int32_t refCnt; - int64_t checkpointingId; - int32_t checkpointAlignCnt; - int32_t checkpointNotReadyTasks; int32_t transferStateAlignCnt; struct SStreamMeta* pMeta; SSHashObj* pNameMap; @@ -477,7 +477,6 @@ typedef struct SStreamMeta { SHashObj* pUpdateTaskSet; int32_t numOfStreamTasks; // this value should be increased when a new task is added into the meta int32_t numOfPausedTasks; - int32_t chkptNotReadyTasks; int64_t rid; int64_t chkpId; diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 4d5ddec233..2d831b9dd1 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1705,7 +1705,7 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) // Downstream not ready, current the stream tasks are not all ready. Ignore this checkpoint req. if (pTask->status.downstreamReady != 1) { pTask->chkInfo.failedId = req.checkpointId; // record the latest failed checkpoint id - pTask->checkpointingId = req.checkpointId; + pTask->chkInfo.checkpointingId = req.checkpointId; tqError("s-task:%s not ready for checkpoint, since downstream not ready, ignore this checkpoint:%" PRId64 ", set it failure", @@ -1744,10 +1744,6 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) // set the initial value for generating check point // set the mgmt epset info according to the checkout source msg from mnode, todo update mgmt epset if needed - if (pMeta->chkptNotReadyTasks == 0) { - pMeta->chkptNotReadyTasks = pMeta->numOfStreamTasks; - } - total = pMeta->numOfStreamTasks; streamMetaWUnLock(pMeta); diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index 095461bd92..7e2e1d6553 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -112,7 +112,7 @@ int32_t streamBroadcastToChildren(SStreamTask* pTask, const SSDataBlock* pBlock) int32_t tEncodeStreamRetrieveReq(SEncoder* pEncoder, const SStreamRetrieveReq* pReq); -int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, SStreamTask* p, int64_t checkpointId); +int32_t streamSaveTaskCheckpointInfo(SStreamTask* p, int64_t checkpointId); int32_t streamTaskBuildCheckpoint(SStreamTask* pTask); int32_t streamSendCheckMsg(SStreamTask* pTask, const SStreamTaskCheckReq* pReq, int32_t nodeId, SEpSet* pEpSet); diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index bf2c89bea4..4f6778a286 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -94,12 +94,12 @@ int32_t tDecodeStreamCheckpointReadyMsg(SDecoder* pDecoder, SStreamCheckpointRea static int32_t streamAlignCheckpoint(SStreamTask* pTask) { int32_t num = taosArrayGetSize(pTask->upstreamInfo.pList); - int64_t old = atomic_val_compare_exchange_32(&pTask->checkpointAlignCnt, 0, num); + int64_t old = atomic_val_compare_exchange_32(&pTask->chkInfo.downstreamAlignNum, 0, num); if (old == 0) { stDebug("s-task:%s set initial align upstream num:%d", pTask->id.idStr, num); } - return atomic_sub_fetch_32(&pTask->checkpointAlignCnt, 1); + return atomic_sub_fetch_32(&pTask->chkInfo.downstreamAlignNum, 1); } static int32_t appendCheckpointIntoInputQ(SStreamTask* pTask, int32_t checkpointType) { @@ -117,7 +117,7 @@ static int32_t appendCheckpointIntoInputQ(SStreamTask* pTask, int32_t checkpoint } pBlock->info.type = STREAM_CHECKPOINT; - pBlock->info.version = pTask->checkpointingId; + pBlock->info.version = pTask->chkInfo.checkpointingId; pBlock->info.rows = 1; pBlock->info.childId = pTask->info.selfChildId; @@ -140,8 +140,8 @@ int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSo // 1. set task status to be prepared for check point, no data are allowed to put into inputQ. streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_GEN_CHECKPOINT); - pTask->checkpointingId = pReq->checkpointId; - pTask->checkpointNotReadyTasks = streamTaskGetNumOfDownstream(pTask); + pTask->chkInfo.checkpointingId = pReq->checkpointId; + pTask->chkInfo.checkpointNotReadyTasks = streamTaskGetNumOfDownstream(pTask); pTask->chkInfo.startTs = taosGetTimestampMs(); pTask->execInfo.checkpoint += 1; @@ -173,7 +173,7 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc // set task status if (streamTaskGetStatus(pTask, NULL) != TASK_STATUS__CK) { - pTask->checkpointingId = checkpointId; + pTask->chkInfo.checkpointingId = checkpointId; code = streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_GEN_CHECKPOINT); if (code != TSDB_CODE_SUCCESS) { stError("s-task:%s handle checkpoint-trigger block failed, code:%s", id, tstrerror(code)); @@ -181,17 +181,6 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc } } - { // todo: remove this when the pipeline checkpoint generating is used. - SStreamMeta* pMeta = pTask->pMeta; - streamMetaWLock(pMeta); - - if (pMeta->chkptNotReadyTasks == 0) { - pMeta->chkptNotReadyTasks = pMeta->numOfStreamTasks; - } - - streamMetaWUnLock(pMeta); - } - // todo fix race condition: set the status and append checkpoint block int32_t taskLevel = pTask->info.taskLevel; if (taskLevel == TASK_LEVEL__SOURCE) { @@ -200,7 +189,7 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc stDebug("s-task:%s set childIdx:%d, and add checkpoint-trigger block into outputQ", id, pTask->info.selfChildId); continueDispatchCheckpointBlock(pBlock, pTask); } else { // only one task exists, no need to dispatch downstream info - atomic_add_fetch_32(&pTask->checkpointNotReadyTasks, 1); + atomic_add_fetch_32(&pTask->chkInfo.checkpointNotReadyTasks, 1); streamProcessCheckpointReadyMsg(pTask); streamFreeQitem((SStreamQueueItem*)pBlock); } @@ -235,7 +224,7 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc // set the needed checked downstream tasks, only when all downstream tasks do checkpoint complete, this task // can start local checkpoint procedure - pTask->checkpointNotReadyTasks = streamTaskGetNumOfDownstream(pTask); + pTask->chkInfo.checkpointNotReadyTasks = streamTaskGetNumOfDownstream(pTask); // Put the checkpoint block into inputQ, to make sure all blocks with less version have been handled by this task // already. And then, dispatch check point msg to all downstream tasks @@ -254,7 +243,7 @@ int32_t streamProcessCheckpointReadyMsg(SStreamTask* pTask) { ASSERT(pTask->info.taskLevel == TASK_LEVEL__SOURCE || pTask->info.taskLevel == TASK_LEVEL__AGG); // only when all downstream tasks are send checkpoint rsp, we can start the checkpoint procedure for the agg task - int32_t notReady = atomic_sub_fetch_32(&pTask->checkpointNotReadyTasks, 1); + int32_t notReady = atomic_sub_fetch_32(&pTask->chkInfo.checkpointNotReadyTasks, 1); ASSERT(notReady >= 0); if (notReady == 0) { @@ -270,35 +259,27 @@ int32_t streamProcessCheckpointReadyMsg(SStreamTask* pTask) { } void streamTaskClearCheckInfo(SStreamTask* pTask) { - pTask->checkpointingId = 0; // clear the checkpoint id + pTask->chkInfo.checkpointingId = 0; // clear the checkpoint id pTask->chkInfo.failedId = 0; pTask->chkInfo.startTs = 0; // clear the recorded start time - pTask->checkpointNotReadyTasks = 0; - pTask->checkpointAlignCnt = 0; + pTask->chkInfo.downstreamAlignNum = 0; + pTask->chkInfo.checkpointNotReadyTasks = 0; streamTaskOpenAllUpstreamInput(pTask); // open inputQ for all upstream tasks } -int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, SStreamTask* p, int64_t checkpointId) { - int32_t vgId = pMeta->vgId; - int32_t code = 0; +int32_t streamSaveTaskCheckpointInfo(SStreamTask* p, int64_t checkpointId) { + SStreamMeta* pMeta = p->pMeta; + int32_t vgId = pMeta->vgId; + int32_t code = 0; - streamMetaWLock(pMeta); - - // for (int32_t i = 0; i < taosArrayGetSize(pMeta->pTaskList); ++i) { - // STaskId* pId = taosArrayGet(pMeta->pTaskList, i); - // SStreamTask** ppTask = taosHashGet(pMeta->pTasksMap, pId, sizeof(*pId)); - // if (ppTask == NULL) { - // continue; - // } - - // SStreamTask* p = *ppTask; if (p->info.fillHistory == 1) { - // continue; + return code; } - ASSERT(p->chkInfo.checkpointId < p->checkpointingId && p->checkpointingId == checkpointId); + streamMetaWLock(pMeta); + ASSERT(p->chkInfo.checkpointId < p->chkInfo.checkpointingId && p->chkInfo.checkpointingId == checkpointId); - p->chkInfo.checkpointId = p->checkpointingId; + p->chkInfo.checkpointId = p->chkInfo.checkpointingId; streamTaskClearCheckInfo(p); char* str = NULL; @@ -316,8 +297,7 @@ int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, SStreamTask* p, int64_t chec stDebug( "vgId:%d s-task:%s level:%d open upstream inputQ, commit task status after checkpoint completed, " "checkpointId:%" PRId64 ", Ver(saved):%" PRId64 " currentVer:%" PRId64 ", status to be normal, prev:%s", - pMeta->vgId, p->id.idStr, p->info.taskLevel, checkpointId, p->chkInfo.checkpointVer, p->chkInfo.nextProcessVer, - str); + vgId, p->id.idStr, p->info.taskLevel, checkpointId, p->chkInfo.checkpointVer, p->chkInfo.nextProcessVer, str); code = streamMetaCommit(pMeta); if (code < 0) { @@ -332,28 +312,21 @@ int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, SStreamTask* p, int64_t chec } int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { - int32_t code = 0; + int32_t code = TSDB_CODE_SUCCESS; // check for all tasks, and do generate the vnode-wide checkpoint data. - SStreamMeta* pMeta = pTask->pMeta; -// int32_t remain = atomic_sub_fetch_32(&pMeta->chkptNotReadyTasks, 1); -// ASSERT(remain >= 0); + int64_t checkpointStartTs = pTask->chkInfo.startTs; - double el = (taosGetTimestampMs() - pTask->chkInfo.startTs) / 1000.0; -// if (remain == 0) { // all tasks are ready - stDebug("s-task:%s all downstreams are ready, ready for do checkpoint", pTask->id.idStr); - streamBackendDoCheckpoint(pTask->pBackend, pTask->checkpointingId); - streamSaveAllTaskStatus(pMeta, pTask, pTask->checkpointingId); - stInfo( - "vgId:%d vnode wide checkpoint completed, save all tasks status, last:%s, level:%d elapsed time:%.2f Sec " - "checkpointId:%" PRId64, - pMeta->vgId, pTask->id.idStr, pTask->info.taskLevel, el, pTask->checkpointingId); -// } else { -// stInfo( -// "vgId:%d vnode wide tasks not reach checkpoint ready status, ready s-task:%s, level:%d elapsed time:%.2f Sec " -// "not ready:%d/%d", -// pMeta->vgId, pTask->id.idStr, pTask->info.taskLevel, el, remain, pMeta->numOfStreamTasks); -// } + // sink task do not need to save the status, and generated the checkpoint + if (pTask->info.taskLevel != TASK_LEVEL__SINK) { + stDebug("s-task:%s level:%d start gen checkpoint", pTask->id.idStr, pTask->info.taskLevel); + streamBackendDoCheckpoint(pTask->pBackend, pTask->chkInfo.checkpointingId); + streamSaveTaskCheckpointInfo(pTask, pTask->chkInfo.checkpointingId); + } + + double el = (taosGetTimestampMs() - checkpointStartTs) / 1000.0; + stInfo("s-task:%s vgId:%d checkpointId:%" PRId64 " save all tasks status, level:%d elapsed time:%.2f Sec ", + pTask->id.idStr, pTask->pMeta->vgId, pTask->chkInfo.checkpointingId, pTask->info.taskLevel, el); // send check point response to upstream task if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { @@ -364,17 +337,16 @@ int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { if (code != TSDB_CODE_SUCCESS) { // record the failure checkpoint id - pTask->chkInfo.failedId = pTask->checkpointingId; + pTask->chkInfo.failedId = pTask->chkInfo.checkpointingId; // todo: let's retry send rsp to upstream/mnode stError("s-task:%s failed to send checkpoint rsp to upstream, checkpointId:%" PRId64 ", code:%s", pTask->id.idStr, - pTask->checkpointingId, tstrerror(code)); + pTask->chkInfo.checkpointingId, tstrerror(code)); } return code; } - //static int64_t kBlockSize = 64 * 1024; //static int sendCheckpointToS3(char* id, SArray* fileList){ // code = s3PutObjectFromFile2(from->fname, object_name); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index b6c973f0d0..1a814eaa84 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -230,7 +230,7 @@ int32_t streamMetaMayCvtDbFormat(SStreamMeta* pMeta) { int32_t streamTaskSetDb(SStreamMeta* pMeta, void* arg, char *key) { SStreamTask* pTask = arg; - int64_t chkpId = pTask->checkpointingId; + int64_t chkpId = pTask->chkInfo.checkpointingId; taosThreadMutexLock(&pMeta->backendMutex); void** ppBackend = taosHashGet(pMeta->pTaskDbUnique, key, strlen(key)); @@ -442,7 +442,6 @@ void streamMetaClear(SStreamMeta* pMeta) { taosArrayClear(pMeta->chkpInUse); pMeta->numOfStreamTasks = 0; pMeta->numOfPausedTasks = 0; - pMeta->chkptNotReadyTasks = 0; streamMetaResetStartInfo(&pMeta->startInfo); } @@ -1078,9 +1077,9 @@ void metaHbToMnode(void* param, void* tmrId) { entry.sinkDataSize = SIZE_IN_MiB((*pTask)->execInfo.sink.dataSize); } - if ((*pTask)->checkpointingId != 0) { - entry.checkpointFailed = ((*pTask)->chkInfo.failedId >= (*pTask)->checkpointingId); - entry.activeCheckpointId = (*pTask)->checkpointingId; + if ((*pTask)->chkInfo.checkpointingId != 0) { + entry.checkpointFailed = ((*pTask)->chkInfo.failedId >= (*pTask)->chkInfo.checkpointingId); + entry.activeCheckpointId = (*pTask)->chkInfo.checkpointingId; } if ((*pTask)->exec.pWalReader != NULL) { From f949333e8b73ade4a0e58816bf74130be6204502 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 8 Nov 2023 16:57:55 +0800 Subject: [PATCH 096/195] refactor:do some internal refactor. --- source/libs/stream/src/streamCheckpoint.c | 112 ++++++++++++++-------- 1 file changed, 74 insertions(+), 38 deletions(-) diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 4f6778a286..e95c90fcaa 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -270,80 +270,116 @@ void streamTaskClearCheckInfo(SStreamTask* pTask) { int32_t streamSaveTaskCheckpointInfo(SStreamTask* p, int64_t checkpointId) { SStreamMeta* pMeta = p->pMeta; int32_t vgId = pMeta->vgId; + const char* id = p->id.idStr; int32_t code = 0; if (p->info.fillHistory == 1) { return code; } - streamMetaWLock(pMeta); - ASSERT(p->chkInfo.checkpointId < p->chkInfo.checkpointingId && p->chkInfo.checkpointingId == checkpointId); + taosThreadMutexLock(&p->lock); + ASSERT(p->chkInfo.checkpointId < p->chkInfo.checkpointingId && p->chkInfo.checkpointingId == checkpointId); p->chkInfo.checkpointId = p->chkInfo.checkpointingId; + streamTaskClearCheckInfo(p); char* str = NULL; streamTaskGetStatus(p, &str); code = streamTaskHandleEvent(p->status.pSM, TASK_EVENT_CHECKPOINT_DONE); + taosThreadMutexUnlock(&p->lock); + if (code != TSDB_CODE_SUCCESS) { - stDebug("s-task:%s vgId:%d save task status failed, since handle event failed", p->id.idStr, vgId); - streamMetaWUnLock(pMeta); + stDebug("s-task:%s vgId:%d handle event:checkpoint-done failed", id, vgId); return -1; - } else { // save the task - streamMetaSaveTask(pMeta, p); } - stDebug( - "vgId:%d s-task:%s level:%d open upstream inputQ, commit task status after checkpoint completed, " - "checkpointId:%" PRId64 ", Ver(saved):%" PRId64 " currentVer:%" PRId64 ", status to be normal, prev:%s", - vgId, p->id.idStr, p->info.taskLevel, checkpointId, p->chkInfo.checkpointVer, p->chkInfo.nextProcessVer, str); + stDebug("vgId:%d s-task:%s level:%d open upstream inputQ, save status after checkpoint, checkpointId:%" PRId64 + ", Ver(saved):%" PRId64 " currentVer:%" PRId64 ", status: normal, prev:%s", + vgId, id, p->info.taskLevel, checkpointId, p->chkInfo.checkpointVer, p->chkInfo.nextProcessVer, str); - code = streamMetaCommit(pMeta); - if (code < 0) { - stError("vgId:%d failed to commit stream meta after do checkpoint, checkpointId:%" PRId64 ", since %s", pMeta->vgId, - checkpointId, terrstr()); - } else { - stInfo("vgId:%d commit stream meta after do checkpoint, checkpointId:%" PRId64 " DONE", pMeta->vgId, checkpointId); + // save the task if not sink task + if (p->info.taskLevel != TASK_LEVEL__SINK) { + streamMetaWLock(pMeta); + + code = streamMetaSaveTask(pMeta, p); + if (code != TSDB_CODE_SUCCESS) { + streamMetaWUnLock(pMeta); + stError("s-task:%s vgId:%d failed to save task info after do checkpoint, checkpointId:%" PRId64 ", since %s", id, + vgId, checkpointId, terrstr()); + return code; + } + + code = streamMetaCommit(pMeta); + if (code != TSDB_CODE_SUCCESS) { + stError("s-task:%s vgId:%d failed to commit stream meta after do checkpoint, checkpointId:%" PRId64 ", since %s", + id, vgId, checkpointId, terrstr()); + } + + streamMetaWUnLock(pMeta); } - - streamMetaWUnLock(pMeta); return code; } +void streamTaskSetFailedId(SStreamTask* pTask) { + pTask->chkInfo.failedId = pTask->chkInfo.checkpointingId; + pTask->chkInfo.checkpointId = pTask->chkInfo.checkpointingId; +} + int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { int32_t code = TSDB_CODE_SUCCESS; - - // check for all tasks, and do generate the vnode-wide checkpoint data. - int64_t checkpointStartTs = pTask->chkInfo.startTs; + int64_t startTs = pTask->chkInfo.startTs; + int64_t ckId = pTask->chkInfo.checkpointingId; // sink task do not need to save the status, and generated the checkpoint if (pTask->info.taskLevel != TASK_LEVEL__SINK) { stDebug("s-task:%s level:%d start gen checkpoint", pTask->id.idStr, pTask->info.taskLevel); - streamBackendDoCheckpoint(pTask->pBackend, pTask->chkInfo.checkpointingId); - streamSaveTaskCheckpointInfo(pTask, pTask->chkInfo.checkpointingId); + code = streamBackendDoCheckpoint(pTask->pBackend, ckId); + if (code != TSDB_CODE_SUCCESS) { + stError("s-task:%s gen checkpoint:%" PRId64 " failed, code:%s", pTask->id.idStr, ckId, tstrerror(terrno)); + } } - double el = (taosGetTimestampMs() - checkpointStartTs) / 1000.0; - stInfo("s-task:%s vgId:%d checkpointId:%" PRId64 " save all tasks status, level:%d elapsed time:%.2f Sec ", - pTask->id.idStr, pTask->pMeta->vgId, pTask->chkInfo.checkpointingId, pTask->info.taskLevel, el); - // send check point response to upstream task - if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { - code = streamTaskSendCheckpointSourceRsp(pTask); - } else { - code = streamTaskSendCheckpointReadyMsg(pTask); + if (code == TSDB_CODE_SUCCESS) { + if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { + code = streamTaskSendCheckpointSourceRsp(pTask); + } else { + code = streamTaskSendCheckpointReadyMsg(pTask); + } + + if (code != TSDB_CODE_SUCCESS) { + // todo: let's retry send rsp to upstream/mnode + stError("s-task:%s failed to send checkpoint rsp to upstream, checkpointId:%" PRId64 ", code:%s", pTask->id.idStr, + ckId, tstrerror(code)); + } } - if (code != TSDB_CODE_SUCCESS) { - // record the failure checkpoint id - pTask->chkInfo.failedId = pTask->chkInfo.checkpointingId; - - // todo: let's retry send rsp to upstream/mnode - stError("s-task:%s failed to send checkpoint rsp to upstream, checkpointId:%" PRId64 ", code:%s", pTask->id.idStr, - pTask->chkInfo.checkpointingId, tstrerror(code)); + // clear the checkpoint info, and commit the newest checkpoint info if all works are done successfully + if (code == TSDB_CODE_SUCCESS) { + code = streamSaveTaskCheckpointInfo(pTask, ckId); + if (code != TSDB_CODE_SUCCESS) { + stError("s-task:%s commit taskInfo failed, checkpoint:%" PRId64 " failed, code:%s", pTask->id.idStr, ckId, + tstrerror(terrno)); + } } + if (code != TSDB_CODE_SUCCESS) { // clear the checkpoint info if failed + taosThreadMutexLock(&pTask->lock); + streamTaskClearCheckInfo(pTask); + code = streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_CHECKPOINT_DONE); + taosThreadMutexUnlock(&pTask->lock); + + streamTaskSetFailedId(pTask); + stDebug("s-task:%s clear checkpoint flag since gen checkpoint failed, checkpointId:%"PRId64, pTask->id.idStr, ckId); + } + + double el = (taosGetTimestampMs() - startTs) / 1000.0; + stInfo("s-task:%s vgId:%d level:%d, checkpointId:%" PRId64 " ver:%" PRId64 " elapsed time:%.2f Sec, %s ", + pTask->id.idStr, pTask->pMeta->vgId, pTask->info.taskLevel, ckId, pTask->chkInfo.checkpointVer, el, + (code == TSDB_CODE_SUCCESS) ? "succ" : "failed"); + return code; } From 76fcc49c6d247de7f9410e7413ea271387585d0f Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 8 Nov 2023 17:10:54 +0800 Subject: [PATCH 097/195] fix stream transfer error --- source/libs/stream/inc/streamInt.h | 5 +- source/libs/stream/src/streamCheckpoint.c | 90 +++++++++++------------ source/libs/stream/src/streamMeta.c | 17 +++++ 3 files changed, 66 insertions(+), 46 deletions(-) diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index 095461bd92..418c4c2848 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -152,7 +152,7 @@ void streamFreeQitem(SStreamQueueItem* data); int32_t streamQueueGetItemSize(const SStreamQueue* pQueue); //#define CHECKPOINT_PATH_LEN 128 -//typedef struct SChekpointDataHeader{ +// typedef struct SChekpointDataHeader{ // int64_t size; // char name[CHECKPOINT_PATH_LEN]; // char id[CHECKPOINT_PATH_LEN]; @@ -162,6 +162,9 @@ int uploadCheckpoint(char* id, char* path); int downloadCheckpoint(char* id, char* path); int deleteCheckpoint(char* id); +typedef int32_t (*__stream_async_exec_fn_t)(void* param); + +int32_t streamMetaAsyncExec(SStreamMeta* pMeta, __stream_async_exec_fn_t fn, void* param, int32_t* code); #ifdef __cplusplus } #endif diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index bf2c89bea4..3f1be03e75 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -13,8 +13,8 @@ * along with this program. If not, see . */ -#include "streamInt.h" #include "rsync.h" +#include "streamInt.h" int32_t tEncodeStreamCheckpointSourceReq(SEncoder* pEncoder, const SStreamCheckpointSourceReq* pReq) { if (tStartEncode(pEncoder) < 0) return -1; @@ -199,11 +199,11 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc if (type == TASK_OUTPUT__FIXED_DISPATCH || type == TASK_OUTPUT__SHUFFLE_DISPATCH) { stDebug("s-task:%s set childIdx:%d, and add checkpoint-trigger block into outputQ", id, pTask->info.selfChildId); continueDispatchCheckpointBlock(pBlock, pTask); - } else { // only one task exists, no need to dispatch downstream info - atomic_add_fetch_32(&pTask->checkpointNotReadyTasks, 1); - streamProcessCheckpointReadyMsg(pTask); - streamFreeQitem((SStreamQueueItem*)pBlock); - } + } else { // only one task exists, no need to dispatch downstream info + atomic_add_fetch_32(&pTask->checkpointNotReadyTasks, 1); + streamProcessCheckpointReadyMsg(pTask); + streamFreeQitem((SStreamQueueItem*)pBlock); + } } else if (taskLevel == TASK_LEVEL__SINK || taskLevel == TASK_LEVEL__AGG) { ASSERT(taosArrayGetSize(pTask->upstreamInfo.pList) > 0); if (pTask->chkInfo.startTs == 0) { @@ -229,7 +229,7 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc id, num); streamFreeQitem((SStreamQueueItem*)pBlock); streamTaskBuildCheckpoint(pTask); - } else { // source & agg tasks need to forward the checkpoint msg downwards + } else { // source & agg tasks need to forward the checkpoint msg downwards stDebug("s-task:%s process checkpoint block, all %d upstreams sent checkpoint msgs, continue forwards msg", id, num); @@ -331,29 +331,29 @@ int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, SStreamTask* p, int64_t chec return code; } -int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { - int32_t code = 0; - +int32_t streamTaskBuildCheckpointImpl(void* arg) { + int32_t code = 0; + SStreamTask* pTask = arg; // check for all tasks, and do generate the vnode-wide checkpoint data. SStreamMeta* pMeta = pTask->pMeta; -// int32_t remain = atomic_sub_fetch_32(&pMeta->chkptNotReadyTasks, 1); -// ASSERT(remain >= 0); + // int32_t remain = atomic_sub_fetch_32(&pMeta->chkptNotReadyTasks, 1); + // ASSERT(remain >= 0); double el = (taosGetTimestampMs() - pTask->chkInfo.startTs) / 1000.0; -// if (remain == 0) { // all tasks are ready - stDebug("s-task:%s all downstreams are ready, ready for do checkpoint", pTask->id.idStr); - streamBackendDoCheckpoint(pTask->pBackend, pTask->checkpointingId); - streamSaveAllTaskStatus(pMeta, pTask, pTask->checkpointingId); - stInfo( - "vgId:%d vnode wide checkpoint completed, save all tasks status, last:%s, level:%d elapsed time:%.2f Sec " - "checkpointId:%" PRId64, - pMeta->vgId, pTask->id.idStr, pTask->info.taskLevel, el, pTask->checkpointingId); -// } else { -// stInfo( -// "vgId:%d vnode wide tasks not reach checkpoint ready status, ready s-task:%s, level:%d elapsed time:%.2f Sec " -// "not ready:%d/%d", -// pMeta->vgId, pTask->id.idStr, pTask->info.taskLevel, el, remain, pMeta->numOfStreamTasks); -// } + // if (remain == 0) { // all tasks are ready + stDebug("s-task:%s all downstreams are ready, ready for do checkpoint", pTask->id.idStr); + streamBackendDoCheckpoint(pTask->pBackend, pTask->checkpointingId); + streamSaveAllTaskStatus(pMeta, pTask, pTask->checkpointingId); + stInfo( + "vgId:%d vnode wide checkpoint completed, save all tasks status, last:%s, level:%d elapsed time:%.2f Sec " + "checkpointId:%" PRId64, + pMeta->vgId, pTask->id.idStr, pTask->info.taskLevel, el, pTask->checkpointingId); + // } else { + // stInfo( + // "vgId:%d vnode wide tasks not reach checkpoint ready status, ready s-task:%s, level:%d elapsed time:%.2f Sec + // " "not ready:%d/%d", pMeta->vgId, pTask->id.idStr, pTask->info.taskLevel, el, remain, + // pMeta->numOfStreamTasks); + // } // send check point response to upstream task if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { @@ -373,14 +373,17 @@ int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { return code; } +int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { + int32_t code = 0; + return streamMetaAsyncExec(pTask->pMeta, streamTaskBuildCheckpointImpl, pTask, NULL); +} - -//static int64_t kBlockSize = 64 * 1024; -//static int sendCheckpointToS3(char* id, SArray* fileList){ +// static int64_t kBlockSize = 64 * 1024; +// static int sendCheckpointToS3(char* id, SArray* fileList){ // code = s3PutObjectFromFile2(from->fname, object_name); // return 0; //} -//static int sendCheckpointToSnode(char* id, SArray* fileList){ +// static int sendCheckpointToSnode(char* id, SArray* fileList){ // if(strlen(id) >= CHECKPOINT_PATH_LEN){ // tqError("uploadCheckpoint id name too long, name:%s", id); // return -1; @@ -466,41 +469,38 @@ int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { //} -int uploadCheckpoint(char* id, char* path){ - if(id == NULL || path == NULL || strlen(id) == 0 || strlen(path) == 0 || strlen(path) >= PATH_MAX){ +int uploadCheckpoint(char* id, char* path) { + if (id == NULL || path == NULL || strlen(id) == 0 || strlen(path) == 0 || strlen(path) >= PATH_MAX) { stError("uploadCheckpoint parameters invalid"); return -1; } - if(strlen(tsSnodeIp) != 0){ + if (strlen(tsSnodeIp) != 0) { uploadRsync(id, path); -// }else if(tsS3StreamEnabled){ - + // }else if(tsS3StreamEnabled){ } return 0; } -int downloadCheckpoint(char* id, char* path){ - if(id == NULL || path == NULL || strlen(id) == 0 || strlen(path) == 0 || strlen(path) >= PATH_MAX){ +int downloadCheckpoint(char* id, char* path) { + if (id == NULL || path == NULL || strlen(id) == 0 || strlen(path) == 0 || strlen(path) >= PATH_MAX) { stError("downloadCheckpoint parameters invalid"); return -1; } - if(strlen(tsSnodeIp) != 0){ + if (strlen(tsSnodeIp) != 0) { downloadRsync(id, path); -// }else if(tsS3StreamEnabled){ - + // }else if(tsS3StreamEnabled){ } return 0; } -int deleteCheckpoint(char* id){ - if(id == NULL || strlen(id) == 0){ +int deleteCheckpoint(char* id) { + if (id == NULL || strlen(id) == 0) { stError("deleteCheckpoint parameters invalid"); return -1; } - if(strlen(tsSnodeIp) != 0){ + if (strlen(tsSnodeIp) != 0) { deleteRsync(id); -// }else if(tsS3StreamEnabled){ - + // }else if(tsS3StreamEnabled){ } return 0; } diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 5e5166cc34..4b344478a1 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -490,6 +490,7 @@ void streamMetaCloseImpl(void* arg) { taosMemoryFree(pMeta->path); taosThreadMutexDestroy(&pMeta->backendMutex); + taosCleanUpScheduler(pMeta->qHandle); pMeta->role = NODE_ROLE_UNINIT; taosMemoryFree(pMeta); stDebug("end to close stream meta"); @@ -1261,3 +1262,19 @@ void streamMetaWUnLock(SStreamMeta* pMeta) { stTrace("vgId:%d meta-wunlock", pMeta->vgId); taosWUnLockLatch(&pMeta->lock); } +static void execHelper(struct SSchedMsg* pSchedMsg) { + __async_exec_fn_t execFn = (__async_exec_fn_t)pSchedMsg->ahandle; + int32_t code = execFn(pSchedMsg->thandle); + if (code != 0 && pSchedMsg->msg != NULL) { + *(int32_t*)pSchedMsg->msg = code; + } +} + +int32_t streamMetaAsyncExec(SStreamMeta* pMeta, __stream_async_exec_fn_t fn, void* param, int32_t* code) { + SSchedMsg schedMsg = {0}; + schedMsg.fp = execHelper; + schedMsg.ahandle = fn; + schedMsg.thandle = param; + schedMsg.msg = code; + return taosScheduleTask(pMeta->qHandle, &schedMsg); +} From 2de5c4a97d1c3f98a8eae0045a9d12a463714691 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 8 Nov 2023 17:11:11 +0800 Subject: [PATCH 098/195] add bg thread --- source/libs/stream/src/streamCheckpoint.c | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 3f1be03e75..d208fabe99 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -331,9 +331,8 @@ int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, SStreamTask* p, int64_t chec return code; } -int32_t streamTaskBuildCheckpointImpl(void* arg) { - int32_t code = 0; - SStreamTask* pTask = arg; +int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { + int32_t code = 0; // check for all tasks, and do generate the vnode-wide checkpoint data. SStreamMeta* pMeta = pTask->pMeta; // int32_t remain = atomic_sub_fetch_32(&pMeta->chkptNotReadyTasks, 1); @@ -373,10 +372,6 @@ int32_t streamTaskBuildCheckpointImpl(void* arg) { return code; } -int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { - int32_t code = 0; - return streamMetaAsyncExec(pTask->pMeta, streamTaskBuildCheckpointImpl, pTask, NULL); -} // static int64_t kBlockSize = 64 * 1024; // static int sendCheckpointToS3(char* id, SArray* fileList){ From 08e344849804e8978582c36c2e8e6b9eaf26c825 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 8 Nov 2023 19:54:57 +0800 Subject: [PATCH 099/195] add seq --- source/dnode/mnode/impl/inc/mndDef.h | 8 +- source/dnode/mnode/impl/inc/mndStream.h | 5 + source/dnode/mnode/impl/src/mndStream.c | 209 +++++++++++++----------- source/dnode/mnode/sdb/inc/sdb.h | 37 +++-- 4 files changed, 144 insertions(+), 115 deletions(-) diff --git a/source/dnode/mnode/impl/inc/mndDef.h b/source/dnode/mnode/impl/inc/mndDef.h index efa99db74b..1e4bae17f6 100644 --- a/source/dnode/mnode/impl/inc/mndDef.h +++ b/source/dnode/mnode/impl/inc/mndDef.h @@ -699,6 +699,11 @@ typedef struct { } SStreamObj; +typedef struct SStreamSeq { + char name[24]; + uint64_t seq; + SRWLatch lock; +} SStreamSeq; int32_t tEncodeSStreamObj(SEncoder* pEncoder, const SStreamObj* pObj); int32_t tDecodeSStreamObj(SDecoder* pDecoder, SStreamObj* pObj, int32_t sver); void tFreeStreamObj(SStreamObj* pObj); @@ -730,14 +735,13 @@ typedef struct { int8_t type; int32_t numOfCols; SSchema* pSchema; - SRWLatch lock; + SRWLatch lock; } SViewObj; int32_t tEncodeSViewObj(SEncoder* pEncoder, const SViewObj* pObj); int32_t tDecodeSViewObj(SDecoder* pDecoder, SViewObj* pObj, int32_t sver); void tFreeSViewObj(SViewObj* pObj); - #ifdef __cplusplus } #endif diff --git a/source/dnode/mnode/impl/inc/mndStream.h b/source/dnode/mnode/impl/inc/mndStream.h index 19fd2a3fd4..6813bb5497 100644 --- a/source/dnode/mnode/impl/inc/mndStream.h +++ b/source/dnode/mnode/impl/inc/mndStream.h @@ -34,6 +34,11 @@ SSdbRow *mndStreamActionDecode(SSdbRaw *pRaw); int32_t mndDropStreamByDb(SMnode *pMnode, STrans *pTrans, SDbObj *pDb); int32_t mndPersistStream(SMnode *pMnode, STrans *pTrans, SStreamObj *pStream); +SSdbRaw * mndStreamSeqActionEncode(SStreamObj *pStream); +SSdbRow * mndStreamSeqActionDecode(SSdbRaw *pRaw); +static int32_t mndStreamSeqActionInsert(SSdb *pSdb, SStreamSeq *pStream); +static int32_t mndStreamSeqActionDelete(SSdb *pSdb, SStreamSeq *pStream); +static int32_t mndStreamSeqActionUpdate(SSdb *pSdb, SStreamSeq *pOldStream, SStreamSeq *pNewStream); // for sma // TODO refactor int32_t mndDropStreamTasks(SMnode *pMnode, STrans *pTrans, SStreamObj *pStream); diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 7e0667ddbf..fcdf024a59 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -43,17 +43,17 @@ typedef struct SNodeEntry { } SNodeEntry; typedef struct SStreamExecInfo { - SArray *pNodeEntryList; + SArray * pNodeEntryList; int64_t ts; // snapshot ts int64_t activeCheckpoint; // active check point id - SHashObj *pTaskMap; - SArray *pTaskList; + SHashObj * pTaskMap; + SArray * pTaskList; TdThreadMutex lock; } SStreamExecInfo; typedef struct SVgroupChangeInfo { SHashObj *pDBMap; - SArray *pUpdateNodeList; // SArray + SArray * pUpdateNodeList; // SArray } SVgroupChangeInfo; static int32_t mndNodeCheckSentinel = 0; @@ -78,7 +78,7 @@ static int32_t mndBuildStreamCheckpointSourceReq2(void **pBuf, int32_t *pLen, in static int32_t mndProcessNodeCheck(SRpcMsg *pReq); static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg); static SArray *extractNodeListFromStream(SMnode *pMnode); -static SArray *mndTakeVgroupSnapshot(SMnode *pMnode, bool* allReady); +static SArray *mndTakeVgroupSnapshot(SMnode *pMnode, bool *allReady); static SVgroupChangeInfo mndFindChangedNodeInfo(SMnode *pMnode, const SArray *pPrevNodeList, const SArray *pNodeList); @@ -91,7 +91,7 @@ static void removeStreamTasksInBuf(SStreamObj *pStream, SStreamExecInfo *pExe static void keepStreamTasksInBuf(SStreamObj *pStream, SStreamExecInfo *pExecNode); static int32_t removeExpirednodeEntryAndTask(SArray *pNodeSnapshot); static int32_t doKillActiveCheckpointTrans(SMnode *pMnode); -static int32_t setNodeEpsetExpiredFlag(const SArray* pNodeList); +static int32_t setNodeEpsetExpiredFlag(const SArray *pNodeList); int32_t mndInitStream(SMnode *pMnode) { SSdbTable table = { @@ -103,6 +103,15 @@ int32_t mndInitStream(SMnode *pMnode) { .updateFp = (SdbUpdateFp)mndStreamActionUpdate, .deleteFp = (SdbDeleteFp)mndStreamActionDelete, }; + SSdbTable tableSeq = { + .sdbType = SDB_STREAM_SEQ, + .keyType = SDB_KEY_BINARY, + .encodeFp = (SdbEncodeFp)mndStreamSeqActionEncode, + .decodeFp = (SdbDecodeFp)mndStreamSeqActionDecode, + .insertFp = (SdbInsertFp)mndStreamSeqActionInsert, + .updateFp = (SdbUpdateFp)mndStreamSeqActionUpdate, + .deleteFp = (SdbDeleteFp)mndStreamSeqActionDelete, + }; mndSetMsgHandle(pMnode, TDMT_MND_CREATE_STREAM, mndProcessCreateStreamReq); mndSetMsgHandle(pMnode, TDMT_MND_DROP_STREAM, mndProcessDropStreamReq); @@ -135,7 +144,13 @@ int32_t mndInitStream(SMnode *pMnode) { execInfo.pTaskMap = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_VARCHAR), true, HASH_NO_LOCK); execInfo.pTaskList = taosArrayInit(4, sizeof(STaskId)); - return sdbSetTable(pMnode->pSdb, table); + if (sdbSetTable(pMnode->pSdb, table) != 0) { + return -1; + } + if (sdbSetTable(pMnode->pSdb, tableSeq) != 0) { + return -1; + } + return 0; } void mndCleanupStream(SMnode *pMnode) { @@ -193,9 +208,9 @@ STREAM_ENCODE_OVER: SSdbRow *mndStreamActionDecode(SSdbRaw *pRaw) { terrno = TSDB_CODE_OUT_OF_MEMORY; - SSdbRow *pRow = NULL; + SSdbRow * pRow = NULL; SStreamObj *pStream = NULL; - void *buf = NULL; + void * buf = NULL; int8_t sver = 0; if (sdbGetRawSoftVer(pRaw, &sver) != 0) { @@ -272,7 +287,7 @@ static int32_t mndStreamActionUpdate(SSdb *pSdb, SStreamObj *pOldStream, SStream } SStreamObj *mndAcquireStream(SMnode *pMnode, char *streamName) { - SSdb *pSdb = pMnode->pSdb; + SSdb * pSdb = pMnode->pSdb; SStreamObj *pStream = sdbAcquire(pSdb, SDB_STREAM, streamName); if (pStream == NULL && terrno == TSDB_CODE_SDB_OBJ_NOT_THERE) { terrno = TSDB_CODE_MND_STREAM_NOT_EXIST; @@ -300,6 +315,12 @@ static void mndShowStreamStatus(char *dst, SStreamObj *pStream) { } } +SSdbRaw * mndStreamSeqActionEncode(SStreamObj *pStream) { return NULL; } +SSdbRow * mndStreamSeqActionDecode(SSdbRaw *pRaw) { return NULL; } +static int32_t mndStreamSeqActionInsert(SSdb *pSdb, SStreamSeq *pStream) { return 0; } +static int32_t mndStreamSeqActionDelete(SSdb *pSdb, SStreamSeq *pStream) { return 0; } +static int32_t mndStreamSeqActionUpdate(SSdb *pSdb, SStreamSeq *pOldStream, SStreamSeq *pNewStream) { return 0; } + static void mndShowStreamTrigger(char *dst, SStreamObj *pStream) { int8_t trigger = pStream->conf.trigger; if (trigger == STREAM_TRIGGER_AT_ONCE) { @@ -325,7 +346,7 @@ static int32_t mndStreamGetPlanString(const char *ast, int8_t triggerType, int64 return TSDB_CODE_SUCCESS; } - SNode *pAst = NULL; + SNode * pAst = NULL; int32_t code = nodesStringToNode(ast, &pAst); SQueryPlan *pPlan = NULL; @@ -350,7 +371,7 @@ static int32_t mndStreamGetPlanString(const char *ast, int8_t triggerType, int64 } static int32_t mndBuildStreamObjFromCreateReq(SMnode *pMnode, SStreamObj *pObj, SCMCreateStreamReq *pCreate) { - SNode *pAst = NULL; + SNode * pAst = NULL; SQueryPlan *pPlan = NULL; mInfo("stream:%s to create", pCreate->name); @@ -589,7 +610,7 @@ int32_t mndPersistDropStreamLog(SMnode *pMnode, STrans *pTrans, SStreamObj *pStr static int32_t mndCreateStbForStream(SMnode *pMnode, STrans *pTrans, const SStreamObj *pStream, const char *user) { SStbObj *pStb = NULL; - SDbObj *pDb = NULL; + SDbObj * pDb = NULL; SMCreateStbReq createReq = {0}; tstrncpy(createReq.name, pStream->targetSTbName, TSDB_TABLE_FNAME_LEN); @@ -715,10 +736,10 @@ int32_t mndDropStreamTasks(SMnode *pMnode, STrans *pTrans, SStreamObj *pStream) } static int32_t mndProcessCreateStreamReq(SRpcMsg *pReq) { - SMnode *pMnode = pReq->info.node; + SMnode * pMnode = pReq->info.node; int32_t code = -1; - SStreamObj *pStream = NULL; - SDbObj *pDb = NULL; + SStreamObj * pStream = NULL; + SDbObj * pDb = NULL; SCMCreateStreamReq createStreamReq = {0}; SStreamObj streamObj = {0}; @@ -761,7 +782,7 @@ static int32_t mndProcessCreateStreamReq(SRpcMsg *pReq) { int32_t numOfStream = 0; SStreamObj *pStream = NULL; - void *pIter = NULL; + void * pIter = NULL; while (1) { pIter = sdbFetch(pMnode->pSdb, SDB_STREAM, pIter, (void **)&pStream); @@ -858,12 +879,12 @@ static int32_t mndProcessCreateStreamReq(SRpcMsg *pReq) { SName name = {0}; tNameFromString(&name, createStreamReq.name, T_NAME_ACCT | T_NAME_DB | T_NAME_TABLE); - //reuse this function for stream + // reuse this function for stream // TODO if (createStreamReq.sql != NULL) { - auditRecord(pReq, pMnode->clusterId, "createStream", name.dbname, name.tname, - createStreamReq.sql, strlen(createStreamReq.sql)); + auditRecord(pReq, pMnode->clusterId, "createStream", name.dbname, name.tname, createStreamReq.sql, + strlen(createStreamReq.sql)); } _OVER: if (code != 0 && code != TSDB_CODE_ACTION_IN_PROGRESS) { @@ -879,7 +900,7 @@ _OVER: static int32_t mndProcessStreamCheckpointTmr(SRpcMsg *pReq) { SMnode *pMnode = pReq->info.node; - SSdb *pSdb = pMnode->pSdb; + SSdb * pSdb = pMnode->pSdb; if (sdbGetSize(pSdb, SDB_STREAM) <= 0) { return 0; } @@ -919,7 +940,7 @@ static int32_t mndBuildStreamCheckpointSourceReq2(void **pBuf, int32_t *pLen, in return -1; } - void *abuf = POINTER_SHIFT(buf, sizeof(SMsgHead)); + void * abuf = POINTER_SHIFT(buf, sizeof(SMsgHead)); SEncoder encoder; tEncoderInit(&encoder, abuf, tlen); tEncodeStreamCheckpointSourceReq(&encoder, &req); @@ -960,7 +981,7 @@ static int32_t mndProcessStreamCheckpointTrans(SMnode *pMnode, SStreamObj *pStre int32_t totLevel = taosArrayGetSize(pStream->tasks); for (int32_t i = 0; i < totLevel; i++) { - SArray *pLevel = taosArrayGetP(pStream->tasks, i); + SArray * pLevel = taosArrayGetP(pStream->tasks, i); SStreamTask *pTask = taosArrayGetP(pLevel, 0); if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { int32_t sz = taosArrayGetSize(pLevel); @@ -973,7 +994,7 @@ static int32_t mndProcessStreamCheckpointTrans(SMnode *pMnode, SStreamObj *pStre goto _ERR; } - void *buf; + void * buf; int32_t tlen; if (mndBuildStreamCheckpointSourceReq2(&buf, &tlen, pTask->info.nodeId, checkpointId, pTask->id.streamId, pTask->id.taskId) < 0) { @@ -1038,7 +1059,7 @@ static int32_t mndAddStreamCheckpointToTrans(STrans *pTrans, SStreamObj *pStream int32_t totLevel = taosArrayGetSize(pStream->tasks); for (int32_t i = 0; i < totLevel; i++) { - SArray *pLevel = taosArrayGetP(pStream->tasks, i); + SArray * pLevel = taosArrayGetP(pStream->tasks, i); SStreamTask *pTask = taosArrayGetP(pLevel, 0); if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { @@ -1055,7 +1076,7 @@ static int32_t mndAddStreamCheckpointToTrans(STrans *pTrans, SStreamObj *pStream return -1; } - void *buf; + void * buf; int32_t tlen; if (mndBuildStreamCheckpointSourceReq2(&buf, &tlen, pTask->info.nodeId, chkptId, pTask->id.streamId, pTask->id.taskId) < 0) { @@ -1066,7 +1087,8 @@ static int32_t mndAddStreamCheckpointToTrans(STrans *pTrans, SStreamObj *pStream STransAction action = {0}; SEpSet epset = mndGetVgroupEpset(pMnode, pVgObj); - initTransAction(&action, buf, tlen, TDMT_VND_STREAM_CHECK_POINT_SOURCE, &epset, TSDB_CODE_SYN_PROPOSE_NOT_READY); + initTransAction(&action, buf, tlen, TDMT_VND_STREAM_CHECK_POINT_SOURCE, &epset, + TSDB_CODE_SYN_PROPOSE_NOT_READY); mndReleaseVgroup(pMnode, pVgObj); if (mndTransAppendRedoAction(pTrans, &action) != 0) { @@ -1106,9 +1128,9 @@ static int32_t mndAddStreamCheckpointToTrans(STrans *pTrans, SStreamObj *pStream } static const char *mndGetStreamDB(SMnode *pMnode) { - SSdb *pSdb = pMnode->pSdb; + SSdb * pSdb = pMnode->pSdb; SStreamObj *pStream = NULL; - void *pIter = NULL; + void * pIter = NULL; pIter = sdbFetch(pSdb, SDB_STREAM, pIter, (void **)&pStream); if (pIter == NULL) { @@ -1147,11 +1169,10 @@ static int32_t mndCheckNodeStatus(SMnode *pMnode) { mDebug("stream task not ready due to node update detected, checkpoint not issued"); goto _EXIT; } - bool allReady = true; + bool allReady = true; SArray *pNodeSnapshot = mndTakeVgroupSnapshot(pMnode, &allReady); if (!allReady) { - mWarn("not all vnodes are ready, ignore the checkpoint") - taosArrayDestroy(pNodeSnapshot); + mWarn("not all vnodes are ready, ignore the checkpoint") taosArrayDestroy(pNodeSnapshot); return 0; } @@ -1170,7 +1191,7 @@ static int32_t mndCheckNodeStatus(SMnode *pMnode) { // check if all tasks are in TASK_STATUS__NORMAL status for (int32_t i = 0; i < taosArrayGetSize(execInfo.pTaskList); ++i) { - STaskId *p = taosArrayGet(execInfo.pTaskList, i); + STaskId * p = taosArrayGet(execInfo.pTaskList, i); STaskStatusEntry *pEntry = taosHashGet(execInfo.pTaskMap, p, sizeof(*p)); if (pEntry == NULL) { continue; @@ -1190,9 +1211,9 @@ _EXIT: return ready == true ? 0 : -1; } static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { - SMnode *pMnode = pReq->info.node; - SSdb *pSdb = pMnode->pSdb; - void *pIter = NULL; + SMnode * pMnode = pReq->info.node; + SSdb * pSdb = pMnode->pSdb; + void * pIter = NULL; SStreamObj *pStream = NULL; int32_t code = 0; if ((code = mndCheckNodeStatus(pMnode)) != 0) { @@ -1215,7 +1236,7 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { } static int32_t mndProcessDropStreamReq(SRpcMsg *pReq) { - SMnode *pMnode = pReq->info.node; + SMnode * pMnode = pReq->info.node; SStreamObj *pStream = NULL; SMDropStreamReq dropReq = {0}; @@ -1343,7 +1364,7 @@ int32_t mndDropStreamByDb(SMnode *pMnode, STrans *pTrans, SDbObj *pDb) { } int32_t mndGetNumOfStreams(SMnode *pMnode, char *dbName, int32_t *pNumOfStreams) { - SSdb *pSdb = pMnode->pSdb; + SSdb * pSdb = pMnode->pSdb; SDbObj *pDb = mndAcquireDb(pMnode, dbName); if (pDb == NULL) { terrno = TSDB_CODE_MND_DB_NOT_SELECTED; @@ -1351,7 +1372,7 @@ int32_t mndGetNumOfStreams(SMnode *pMnode, char *dbName, int32_t *pNumOfStreams) } int32_t numOfStreams = 0; - void *pIter = NULL; + void * pIter = NULL; while (1) { SStreamObj *pStream = NULL; pIter = sdbFetch(pSdb, SDB_STREAM, pIter, (void **)&pStream); @@ -1370,8 +1391,8 @@ int32_t mndGetNumOfStreams(SMnode *pMnode, char *dbName, int32_t *pNumOfStreams) } static int32_t mndRetrieveStream(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock *pBlock, int32_t rows) { - SMnode *pMnode = pReq->info.node; - SSdb *pSdb = pMnode->pSdb; + SMnode * pMnode = pReq->info.node; + SSdb * pSdb = pMnode->pSdb; int32_t numOfRows = 0; SStreamObj *pStream = NULL; @@ -1447,8 +1468,8 @@ static void mndCancelGetNextStream(SMnode *pMnode, void *pIter) { } static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock *pBlock, int32_t rowsCapacity) { - SMnode *pMnode = pReq->info.node; - SSdb *pSdb = pMnode->pSdb; + SMnode * pMnode = pReq->info.node; + SSdb * pSdb = pMnode->pSdb; int32_t numOfRows = 0; SStreamObj *pStream = NULL; @@ -1537,13 +1558,13 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock // status char status[20 + VARSTR_HEADER_SIZE] = {0}; - STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; - STaskStatusEntry* pe = taosHashGet(execInfo.pTaskMap, &id, sizeof(id)); + STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; + STaskStatusEntry *pe = taosHashGet(execInfo.pTaskMap, &id, sizeof(id)); if (pe == NULL) { continue; } - const char* pStatus = streamTaskGetStatusStr(pe->status); + const char *pStatus = streamTaskGetStatusStr(pe->status); STR_TO_VARSTR(status, pStatus); // status @@ -1664,7 +1685,7 @@ static int32_t mndPersistStreamLog(STrans *pTrans, const SStreamObj *pStream, in } static int32_t mndProcessPauseStreamReq(SRpcMsg *pReq) { - SMnode *pMnode = pReq->info.node; + SMnode * pMnode = pReq->info.node; SStreamObj *pStream = NULL; SMPauseStreamReq pauseReq = {0}; @@ -1780,7 +1801,7 @@ int32_t mndResumeAllStreamTasks(STrans *pTrans, SStreamObj *pStream, int8_t igUn } static int32_t mndProcessResumeStreamReq(SRpcMsg *pReq) { - SMnode *pMnode = pReq->info.node; + SMnode * pMnode = pReq->info.node; SStreamObj *pStream = NULL; SMResumeStreamReq pauseReq = {0}; @@ -1865,7 +1886,7 @@ static void initNodeUpdateMsg(SStreamTaskNodeUpdateMsg *pMsg, const SVgroupChang } static int32_t doBuildStreamTaskUpdateMsg(void **pBuf, int32_t *pLen, SVgroupChangeInfo *pInfo, int32_t nodeId, - SStreamTaskId* pId, int32_t transId) { + SStreamTaskId *pId, int32_t transId) { SStreamTaskNodeUpdateMsg req = {0}; initNodeUpdateMsg(&req, pInfo, pId, transId); @@ -1888,7 +1909,7 @@ static int32_t doBuildStreamTaskUpdateMsg(void **pBuf, int32_t *pLen, SVgroupCha return -1; } - void *abuf = POINTER_SHIFT(buf, sizeof(SMsgHead)); + void * abuf = POINTER_SHIFT(buf, sizeof(SMsgHead)); SEncoder encoder; tEncoderInit(&encoder, abuf, tlen); tEncodeStreamTaskUpdateMsg(&encoder, &req); @@ -1955,7 +1976,7 @@ static int32_t createStreamUpdateTrans(SStreamObj *pStream, SVgroupChangeInfo *p for (int32_t k = 0; k < numOfTasks; ++k) { SStreamTask *pTask = taosArrayGetP(pLevel, k); - void *pBuf = NULL; + void * pBuf = NULL; int32_t len = 0; streamTaskUpdateEpsetInfo(pTask, pInfo->pUpdateNodeList); doBuildStreamTaskUpdateMsg(&pBuf, &len, pInfo, pTask->info.nodeId, &pTask->id, pTrans->id); @@ -1987,8 +2008,8 @@ static bool isNodeEpsetChanged(const SEpSet *pPrevEpset, const SEpSet *pCurrent) // 1. increase the replica does not affect the stream process. // 2. decreasing the replica may affect the stream task execution in the way that there is one or more running stream // tasks on the will be removed replica. -// 3. vgroup redistribution is an combination operation of first increase replica and then decrease replica. So we will -// handle it as mentioned in 1 & 2 items. +// 3. vgroup redistribution is an combination operation of first increase replica and then decrease replica. So we +// will handle it as mentioned in 1 & 2 items. static SVgroupChangeInfo mndFindChangedNodeInfo(SMnode *pMnode, const SArray *pPrevNodeList, const SArray *pNodeList) { SVgroupChangeInfo info = { .pUpdateNodeList = taosArrayInit(4, sizeof(SNodeUpdateInfo)), @@ -2030,9 +2051,9 @@ static SVgroupChangeInfo mndFindChangedNodeInfo(SMnode *pMnode, const SArray *pP return info; } -static SArray *mndTakeVgroupSnapshot(SMnode *pMnode, bool* allReady) { - SSdb *pSdb = pMnode->pSdb; - void *pIter = NULL; +static SArray *mndTakeVgroupSnapshot(SMnode *pMnode, bool *allReady) { + SSdb * pSdb = pMnode->pSdb; + void * pIter = NULL; SVgObj *pVgroup = NULL; *allReady = true; @@ -2079,8 +2100,8 @@ static int32_t mndProcessVgroupChange(SMnode *pMnode, SVgroupChangeInfo *pChange // check all streams that involved this vnode should update the epset info SStreamObj *pStream = NULL; - void *pIter = NULL; - STrans *pTrans = NULL; + void * pIter = NULL; + STrans * pTrans = NULL; while (1) { pIter = sdbFetch(pSdb, SDB_STREAM, pIter, (void **)&pStream); @@ -2141,9 +2162,9 @@ static int32_t mndProcessVgroupChange(SMnode *pMnode, SVgroupChangeInfo *pChange } static SArray *extractNodeListFromStream(SMnode *pMnode) { - SSdb *pSdb = pMnode->pSdb; + SSdb * pSdb = pMnode->pSdb; SStreamObj *pStream = NULL; - void *pIter = NULL; + void * pIter = NULL; SHashObj *pHash = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), false, HASH_NO_LOCK); while (1) { @@ -2190,9 +2211,9 @@ static SArray *extractNodeListFromStream(SMnode *pMnode) { } static void doExtractTasksFromStream(SMnode *pMnode) { - SSdb *pSdb = pMnode->pSdb; + SSdb * pSdb = pMnode->pSdb; SStreamObj *pStream = NULL; - void *pIter = NULL; + void * pIter = NULL; while (1) { pIter = sdbFetch(pSdb, SDB_STREAM, pIter, (void **)&pStream); @@ -2240,12 +2261,12 @@ static bool taskNodeExists(SArray *pList, int32_t nodeId) { } int32_t removeExpirednodeEntryAndTask(SArray *pNodeSnapshot) { - SArray* pRemovedTasks = taosArrayInit(4, sizeof(STaskId)); + SArray *pRemovedTasks = taosArrayInit(4, sizeof(STaskId)); int32_t numOfTask = taosArrayGetSize(execInfo.pTaskList); - for(int32_t i = 0; i < numOfTask; ++i) { - STaskId* pId = taosArrayGet(execInfo.pTaskList, i); - STaskStatusEntry* pEntry = taosHashGet(execInfo.pTaskMap, pId, sizeof(*pId)); + for (int32_t i = 0; i < numOfTask; ++i) { + STaskId * pId = taosArrayGet(execInfo.pTaskList, i); + STaskStatusEntry *pEntry = taosHashGet(execInfo.pTaskMap, pId, sizeof(*pId)); bool existed = taskNodeExists(pNodeSnapshot, pEntry->nodeId); if (!existed) { @@ -2253,18 +2274,18 @@ int32_t removeExpirednodeEntryAndTask(SArray *pNodeSnapshot) { } } - for(int32_t i = 0; i < taosArrayGetSize(pRemovedTasks); ++i) { - STaskId* pId = taosArrayGet(pRemovedTasks, i); + for (int32_t i = 0; i < taosArrayGetSize(pRemovedTasks); ++i) { + STaskId *pId = taosArrayGet(pRemovedTasks, i); doRemoveTasks(&execInfo, pId); } mDebug("remove invalid stream tasks:%d, remain:%d", (int32_t)taosArrayGetSize(pRemovedTasks), - (int32_t) taosArrayGetSize(execInfo.pTaskList)); + (int32_t)taosArrayGetSize(execInfo.pTaskList)); int32_t size = taosArrayGetSize(pNodeSnapshot); - SArray* pValidNodeEntryList = taosArrayInit(4, sizeof(SNodeEntry)); - for(int32_t i = 0; i < taosArrayGetSize(execInfo.pNodeEntryList); ++i) { - SNodeEntry* p = taosArrayGet(execInfo.pNodeEntryList, i); + SArray *pValidNodeEntryList = taosArrayInit(4, sizeof(SNodeEntry)); + for (int32_t i = 0; i < taosArrayGetSize(execInfo.pNodeEntryList); ++i) { + SNodeEntry *p = taosArrayGet(execInfo.pNodeEntryList, i); for (int32_t j = 0; j < size; ++j) { SNodeEntry *pEntry = taosArrayGet(pNodeSnapshot, j); @@ -2278,7 +2299,7 @@ int32_t removeExpirednodeEntryAndTask(SArray *pNodeSnapshot) { execInfo.pNodeEntryList = taosArrayDestroy(execInfo.pNodeEntryList); execInfo.pNodeEntryList = pValidNodeEntryList; - mDebug("remain %d valid node entries", (int32_t) taosArrayGetSize(pValidNodeEntryList)); + mDebug("remain %d valid node entries", (int32_t)taosArrayGetSize(pValidNodeEntryList)); taosArrayDestroy(pRemovedTasks); return 0; } @@ -2310,7 +2331,7 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { return 0; } - bool allVnodeReady = true; + bool allVnodeReady = true; SArray *pNodeSnapshot = mndTakeVgroupSnapshot(pMnode, &allVnodeReady); if (!allVnodeReady) { taosArrayDestroy(pNodeSnapshot); @@ -2324,7 +2345,6 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { SVgroupChangeInfo changeInfo = mndFindChangedNodeInfo(pMnode, execInfo.pNodeEntryList, pNodeSnapshot); if (taosArrayGetSize(changeInfo.pUpdateNodeList) > 0) { - // kill current active checkpoint transaction, since the transaction is vnode wide. doKillActiveCheckpointTrans(pMnode); code = mndProcessVgroupChange(pMnode, &changeInfo); @@ -2359,7 +2379,7 @@ typedef struct SMStreamNodeCheckMsg { static int32_t mndProcessNodeCheck(SRpcMsg *pReq) { SMnode *pMnode = pReq->info.node; - SSdb *pSdb = pMnode->pSdb; + SSdb * pSdb = pMnode->pSdb; if (sdbGetSize(pSdb, SDB_STREAM) <= 0) { return 0; } @@ -2383,7 +2403,7 @@ void keepStreamTasksInBuf(SStreamObj *pStream, SStreamExecInfo *pExecNode) { SStreamTask *pTask = taosArrayGetP(pLevel, j); STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; - void *p = taosHashGet(pExecNode->pTaskMap, &id, sizeof(id)); + void * p = taosHashGet(pExecNode->pTaskMap, &id, sizeof(id)); if (p == NULL) { STaskStatusEntry entry = {0}; streamTaskStatusInit(&entry, pTask); @@ -2397,7 +2417,7 @@ void keepStreamTasksInBuf(SStreamObj *pStream, SStreamExecInfo *pExecNode) { } } -void removeStreamTasksInBuf(SStreamObj* pStream, SStreamExecInfo * pExecNode) { +void removeStreamTasksInBuf(SStreamObj *pStream, SStreamExecInfo *pExecNode) { int32_t level = taosArrayGetSize(pStream->tasks); for (int32_t i = 0; i < level; i++) { SArray *pLevel = taosArrayGetP(pStream->tasks, i); @@ -2407,7 +2427,7 @@ void removeStreamTasksInBuf(SStreamObj* pStream, SStreamExecInfo * pExecNode) { SStreamTask *pTask = taosArrayGetP(pLevel, j); STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; - void *p = taosHashGet(pExecNode->pTaskMap, &id, sizeof(id)); + void * p = taosHashGet(pExecNode->pTaskMap, &id, sizeof(id)); if (p != NULL) { taosHashRemove(pExecNode->pTaskMap, &id, sizeof(id)); @@ -2513,9 +2533,9 @@ int32_t createStreamResetStatusTrans(SMnode *pMnode, SStreamObj *pStream) { int32_t doKillActiveCheckpointTrans(SMnode *pMnode) { int32_t transId = 0; - SSdb *pSdb = pMnode->pSdb; + SSdb * pSdb = pMnode->pSdb; STrans *pTrans = NULL; - void *pIter = NULL; + void * pIter = NULL; while (1) { pIter = sdbFetch(pSdb, SDB_TRANS, pIter, (void **)&pTrans); @@ -2546,13 +2566,13 @@ int32_t doKillActiveCheckpointTrans(SMnode *pMnode) { return TSDB_CODE_SUCCESS; } -int32_t mndResetFromCheckpoint(SMnode* pMnode) { +int32_t mndResetFromCheckpoint(SMnode *pMnode) { doKillActiveCheckpointTrans(pMnode); // set all tasks status to be normal, refactor later to be stream level, instead of vnode level. - SSdb *pSdb = pMnode->pSdb; + SSdb * pSdb = pMnode->pSdb; SStreamObj *pStream = NULL; - void *pIter = NULL; + void * pIter = NULL; while (1) { pIter = sdbFetch(pSdb, SDB_STREAM, pIter, (void **)&pStream); if (pIter == NULL) { @@ -2570,15 +2590,15 @@ int32_t mndResetFromCheckpoint(SMnode* pMnode) { return 0; } -int32_t setNodeEpsetExpiredFlag(const SArray* pNodeList) { +int32_t setNodeEpsetExpiredFlag(const SArray *pNodeList) { int32_t num = taosArrayGetSize(pNodeList); for (int k = 0; k < num; ++k) { - int32_t* pVgId = taosArrayGet(pNodeList, k); + int32_t *pVgId = taosArrayGet(pNodeList, k); int32_t numOfNodes = taosArrayGetSize(execInfo.pNodeEntryList); for (int i = 0; i < numOfNodes; ++i) { - SNodeEntry* pNodeEntry = taosArrayGet(execInfo.pNodeEntryList, i); + SNodeEntry *pNodeEntry = taosArrayGet(execInfo.pNodeEntryList, i); if (pNodeEntry->nodeId == *pVgId) { mInfo("vgId:%d expired in stream task, needs update nodeEp", *pVgId); @@ -2591,12 +2611,11 @@ int32_t setNodeEpsetExpiredFlag(const SArray* pNodeList) { return TSDB_CODE_SUCCESS; } -static void updateStageInfo(STaskStatusEntry* pTaskEntry, int32_t stage) { +static void updateStageInfo(STaskStatusEntry *pTaskEntry, int32_t stage) { int32_t numOfNodes = taosArrayGetSize(execInfo.pNodeEntryList); - for(int32_t j = 0; j < numOfNodes; ++j) { - SNodeEntry* pNodeEntry = taosArrayGet(execInfo.pNodeEntryList, j); + for (int32_t j = 0; j < numOfNodes; ++j) { + SNodeEntry *pNodeEntry = taosArrayGet(execInfo.pNodeEntryList, j); if (pNodeEntry->nodeId == pTaskEntry->nodeId) { - mInfo("vgId:%d stage updated from %d to %d, nodeUpdate trigger by s-task:0x%" PRIx64, pTaskEntry->nodeId, pTaskEntry->stage, stage, pTaskEntry->id.taskId); @@ -2608,7 +2627,7 @@ static void updateStageInfo(STaskStatusEntry* pTaskEntry, int32_t stage) { } int32_t mndProcessStreamHb(SRpcMsg *pReq) { - SMnode *pMnode = pReq->info.node; + SMnode * pMnode = pReq->info.node; SStreamHbMsg req = {0}; bool checkpointFailed = false; @@ -2661,15 +2680,15 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { pTaskEntry->status = p->status; if (p->status != TASK_STATUS__READY) { - mDebug("received s-task:0x%"PRIx64" not in ready status:%s", p->id.taskId, streamTaskGetStatusStr(p->status)); + mDebug("received s-task:0x%" PRIx64 " not in ready status:%s", p->id.taskId, streamTaskGetStatusStr(p->status)); } } // current checkpoint is failed, rollback from the checkpoint trans // kill the checkpoint trans and then set all tasks status to be normal if (checkpointFailed && activeCheckpointId != 0) { - bool allReady = true; - SArray* p = mndTakeVgroupSnapshot(pMnode, &allReady); + bool allReady = true; + SArray *p = mndTakeVgroupSnapshot(pMnode, &allReady); taosArrayDestroy(p); if (allReady) { diff --git a/source/dnode/mnode/sdb/inc/sdb.h b/source/dnode/mnode/sdb/inc/sdb.h index ddde645fae..0a20dcfd09 100644 --- a/source/dnode/mnode/sdb/inc/sdb.h +++ b/source/dnode/mnode/sdb/inc/sdb.h @@ -149,7 +149,8 @@ typedef enum { SDB_FUNC = 20, SDB_IDX = 21, SDB_VIEW = 22, - SDB_MAX = 23 + SDB_STREAM_SEQ = 23, + SDB_MAX = 24 } ESdbType; typedef struct SSdbRaw { @@ -169,11 +170,11 @@ typedef struct SSdbRow { } SSdbRow; typedef struct SSdb { - SMnode *pMnode; - SWal *pWal; + SMnode * pMnode; + SWal * pWal; int64_t sync; - char *currDir; - char *tmpDir; + char * currDir; + char * tmpDir; int64_t commitIndex; int64_t commitTerm; int64_t commitConfig; @@ -183,7 +184,7 @@ typedef struct SSdb { int64_t tableVer[SDB_MAX]; int64_t maxId[SDB_MAX]; EKeyType keyTypes[SDB_MAX]; - SHashObj *hashObjs[SDB_MAX]; + SHashObj * hashObjs[SDB_MAX]; TdThreadRwlock locks[SDB_MAX]; SdbInsertFp insertFps[SDB_MAX]; SdbUpdateFp updateFps[SDB_MAX]; @@ -198,25 +199,25 @@ typedef struct SSdb { typedef struct SSdbIter { TdFilePtr file; int64_t total; - char *name; + char * name; } SSdbIter; typedef struct { - ESdbType sdbType; - EKeyType keyType; - SdbDeployFp deployFp; - SdbEncodeFp encodeFp; - SdbDecodeFp decodeFp; - SdbInsertFp insertFp; - SdbUpdateFp updateFp; - SdbDeleteFp deleteFp; + ESdbType sdbType; + EKeyType keyType; + SdbDeployFp deployFp; + SdbEncodeFp encodeFp; + SdbDecodeFp decodeFp; + SdbInsertFp insertFp; + SdbUpdateFp updateFp; + SdbDeleteFp deleteFp; SdbValidateFp validateFp; } SSdbTable; typedef struct SSdbOpt { const char *path; - SMnode *pMnode; - SWal *pWal; + SMnode * pMnode; + SWal * pWal; int64_t sync; } SSdbOpt; @@ -393,7 +394,7 @@ int32_t sdbGetRawSoftVer(SSdbRaw *pRaw, int8_t *sver); int32_t sdbGetRawTotalSize(SSdbRaw *pRaw); SSdbRow *sdbAllocRow(int32_t objSize); -void *sdbGetRowObj(SSdbRow *pRow); +void * sdbGetRowObj(SSdbRow *pRow); void sdbFreeRow(SSdb *pSdb, SSdbRow *pRow, bool callFunc); int32_t sdbStartRead(SSdb *pSdb, SSdbIter **ppIter, int64_t *index, int64_t *term, int64_t *config); From 0f98f4daa58201d2bf5344c6a71530d6d19ba48c Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 8 Nov 2023 20:29:37 +0800 Subject: [PATCH 100/195] change check out seq --- source/dnode/mnode/impl/src/mndStream.c | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index fcdf024a59..b3fd81f450 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1219,14 +1219,11 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { if ((code = mndCheckNodeStatus(pMnode)) != 0) { return code; } - SMStreamDoCheckpointMsg *pMsg = (SMStreamDoCheckpointMsg *)pReq->pCont; - int64_t checkpointId = pMsg->checkpointId; while (1) { pIter = sdbFetch(pSdb, SDB_STREAM, pIter, (void **)&pStream); if (pIter == NULL) break; - - code = mndProcessStreamCheckpointTrans(pMnode, pStream, checkpointId); + code = mndProcessStreamCheckpointTrans(pMnode, pStream, pStream->checkpointId + 1); sdbRelease(pSdb, pStream); if (code == -1) { break; From 06f196b71d0e022a1b4217857ee5e1811d096c3d Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 9 Nov 2023 09:08:41 +0800 Subject: [PATCH 101/195] change check out seq --- source/dnode/mnode/impl/src/mndStream.c | 1 - 1 file changed, 1 deletion(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index b3fd81f450..42f883aa2d 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1020,7 +1020,6 @@ static int32_t mndProcessStreamCheckpointTrans(SMnode *pMnode, SStreamObj *pStre } } // 2. reset tick - pStream->checkpointFreq = checkpointId; pStream->checkpointId = checkpointId; pStream->checkpointFreq = taosGetTimestampMs(); pStream->currentTick = 0; From dd2252598ed402b1bda417182e40c9c31e6f72ca Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 9 Nov 2023 17:39:34 +0800 Subject: [PATCH 102/195] upload by rsync --- source/libs/stream/inc/streamBackendRocksdb.h | 3 ++ source/libs/stream/src/streamBackendRocksdb.c | 38 ++++++++++++++++ source/libs/stream/src/streamCheckpoint.c | 43 +++++++++++++++++++ 3 files changed, 84 insertions(+) diff --git a/source/libs/stream/inc/streamBackendRocksdb.h b/source/libs/stream/inc/streamBackendRocksdb.h index c9cb5ab64c..9d84e76a29 100644 --- a/source/libs/stream/inc/streamBackendRocksdb.h +++ b/source/libs/stream/inc/streamBackendRocksdb.h @@ -17,6 +17,7 @@ #define _STREAM_BACKEDN_ROCKSDB_H_ #include "rocksdb/c.h" +//#include "streamInt.h" #include "streamState.h" #include "tcoding.h" #include "tcommon.h" @@ -252,4 +253,6 @@ int32_t bkdMgtAddChkp(SBkdMgt* bm, char* task, char* path); int32_t bkdMgtGetDelta(SBkdMgt* bm, char* taskId, int64_t chkpId, SArray* list); int32_t bkdMgtDumpTo(SBkdMgt* bm, char* taskId, char* dname); void bkdMgtDestroy(SBkdMgt* bm); + +int32_t taskDbGenChkpUploadPath(void* arg, int64_t chkpId, int8_t type, char** pathkj); #endif \ No newline at end of file diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index a06ec8c9c6..55dd938835 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1691,6 +1691,44 @@ void taskDbDestroy(void* pDb) { return; } +int32_t taskDbGenChkpUplaodPath__rsync(STaskDbWrapper* pDb, int64_t chkpId, char** path) { + int64_t st = taosGetTimestampMs(); + int32_t code = -1; + int64_t refId = pDb->refId; + + if (taosAcquireRef(taskDbWrapperId, refId) == NULL) { + return -1; + } + char* pChkpDir = NULL; + char* pChkpIdDir = NULL; + if (chkpPreBuildDir(pDb->path, chkpId, &pChkpDir, &pChkpIdDir) != 0) { + code = -1; + } + + if (taosIsDir(pChkpIdDir) && isValidCheckpoint(pChkpIdDir)) { + code = 0; + *path = pChkpIdDir; + pChkpIdDir = NULL; + } + + taosMemoryFree(pChkpDir); + taosMemoryFree(pChkpIdDir); + taosReleaseRef(taskDbWrapperId, refId); + + return code; +} +int32_t taskDbGenChkpUploadPath(void* arg, int64_t chkpId, int8_t type, char** path) { + STaskDbWrapper* pDb = arg; + UPLOAD_TYPE utype = type; + + if (utype == UPLOAD_RSYNC) { + return taskDbGenChkpUplaodPath__rsync(pDb, chkpId, path); + } else if (utype == UPLOAD_S3) { + return 0; + } + return -1; +} + int32_t taskDbOpenCfByKey(STaskDbWrapper* pDb, const char* key) { int32_t code = 0; char* err = NULL; diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 34943454ff..51f2a18504 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -15,8 +15,16 @@ #include "cos.h" #include "rsync.h" +#include "streamBackendRocksdb.h" #include "streamInt.h" +typedef struct { + UPLOAD_TYPE type; + char* taskId; + int64_t chkpId; + + SStreamTask* pTask; +} SAsyncUploadArg; int32_t tEncodeStreamCheckpointSourceReq(SEncoder* pEncoder, const SStreamCheckpointSourceReq* pReq) { if (tStartEncode(pEncoder) < 0) return -1; if (tEncodeI64(pEncoder, pReq->streamId) < 0) return -1; @@ -328,6 +336,36 @@ void streamTaskSetFailedId(SStreamTask* pTask) { pTask->chkInfo.checkpointId = pTask->chkInfo.checkpointingId; } +int32_t doUploadChkp(void* param) { + SAsyncUploadArg* arg = param; + char* path = NULL; + int32_t code = 0; + if ((code = taskDbGenChkpUploadPath(arg->pTask->pBackend, arg->chkpId, (int8_t)(arg->type), &path)) != 0) { + stError("s-task:%s faile to gen upload checkpoint:%" PRId64 "", arg->pTask->id.idStr, arg->chkpId); + } + if (code == 0 && uploadCheckpoint(arg->taskId, path) != 0) { + stError("s-task:%s faile to upload checkpoint:%" PRId64 "", arg->pTask->id.idStr, arg->chkpId); + } + + taosMemoryFree(path); + taosMemoryFree(arg->taskId); + taosMemoryFree(arg); + return 0; +} +int32_t streamTaskUploadChkp(SStreamTask* pTask, int64_t chkpId, char* taskId) { + // async upload + UPLOAD_TYPE type = getUploadType(); + if (type == UPLOAD_DISABLE) { + return 0; + } + SAsyncUploadArg* arg = taosMemoryCalloc(1, sizeof(SAsyncUploadArg)); + arg->type = type; + arg->taskId = taosStrdup(taskId); + arg->chkpId = chkpId; + arg->pTask = pTask; + + return streamMetaAsyncExec(pTask->pMeta, doUploadChkp, arg, NULL); +} int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { int32_t code = TSDB_CODE_SUCCESS; int64_t startTs = pTask->chkInfo.startTs; @@ -363,6 +401,11 @@ int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { if (code != TSDB_CODE_SUCCESS) { stError("s-task:%s commit taskInfo failed, checkpoint:%" PRId64 " failed, code:%s", pTask->id.idStr, ckId, tstrerror(terrno)); + } else { + code = streamTaskUploadChkp(pTask, ckId, (char*)pTask->id.idStr); + if (code != 0) { + stError("s-task:%s failed to upload checkpoint:%" PRId64 " failed", pTask->id.idStr, ckId); + } } } From 5f9b922a042be4ab65e3db2dc9673a117b251997 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 9 Nov 2023 20:15:08 +0800 Subject: [PATCH 103/195] support s3 --- include/libs/stream/tstream.h | 2 + source/libs/stream/inc/streamBackendRocksdb.h | 4 +- source/libs/stream/src/streamBackendRocksdb.c | 42 ++++++++++++++++--- source/libs/stream/src/streamCheckpoint.c | 6 ++- source/libs/stream/src/streamMeta.c | 2 + 5 files changed, 46 insertions(+), 10 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index e3c52c78b0..00f01e4266 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -513,6 +513,8 @@ typedef struct SStreamMeta { void* qHandle; int32_t pauseTaskNum; + + void* bkdChkptMgt; } SStreamMeta; int32_t tEncodeStreamEpInfo(SEncoder* pEncoder, const SStreamChildEpInfo* pInfo); diff --git a/source/libs/stream/inc/streamBackendRocksdb.h b/source/libs/stream/inc/streamBackendRocksdb.h index 9d84e76a29..cd620ed5b7 100644 --- a/source/libs/stream/inc/streamBackendRocksdb.h +++ b/source/libs/stream/inc/streamBackendRocksdb.h @@ -250,9 +250,9 @@ int32_t taskDbDoCheckpoint(void* arg, int64_t chkpId); SBkdMgt* bkdMgtCreate(char* path); int32_t bkdMgtAddChkp(SBkdMgt* bm, char* task, char* path); -int32_t bkdMgtGetDelta(SBkdMgt* bm, char* taskId, int64_t chkpId, SArray* list); +int32_t bkdMgtGetDelta(SBkdMgt* bm, char* taskId, int64_t chkpId, SArray* list, char* name); int32_t bkdMgtDumpTo(SBkdMgt* bm, char* taskId, char* dname); void bkdMgtDestroy(SBkdMgt* bm); -int32_t taskDbGenChkpUploadPath(void* arg, int64_t chkpId, int8_t type, char** pathkj); +int32_t taskDbGenChkpUploadPath(void* arg, void* bkdMgt, int64_t chkpId, int8_t type, char** pathkj); #endif \ No newline at end of file diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 55dd938835..253c18467e 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1717,14 +1717,30 @@ int32_t taskDbGenChkpUplaodPath__rsync(STaskDbWrapper* pDb, int64_t chkpId, char return code; } -int32_t taskDbGenChkpUploadPath(void* arg, int64_t chkpId, int8_t type, char** path) { + +int32_t taskDbGenChkpUplaodPath__s3(STaskDbWrapper* pDb, void* bkdChkpMgt, int64_t chkpId, char** path) { + SBkdMgt* p = (SBkdMgt*)bkdChkpMgt; + + char* temp = taosMemoryCalloc(1, strlen(pDb->path)); + sprintf(temp, "%s%s%s", pDb->path, TD_DIRSEP, "tmp"); + + if (!taosDirExist(temp)) { + taosMkDir(temp); + } + bkdMgtGetDelta(p, pDb->idstr, chkpId, NULL, temp); + + *path = temp; + + return 0; +} +int32_t taskDbGenChkpUploadPath(void* arg, void* mgt, int64_t chkpId, int8_t type, char** path) { STaskDbWrapper* pDb = arg; UPLOAD_TYPE utype = type; if (utype == UPLOAD_RSYNC) { - return taskDbGenChkpUplaodPath__rsync(pDb, chkpId, path); + return taskDbGenChkpUplaodPath__rsync(pDb,chkpId, path); } else if (utype == UPLOAD_S3) { - return 0; + return taskDbGenChkpUplaodPath__s3(pDb,mgt, chkpId, path); } return -1; } @@ -3603,14 +3619,28 @@ void bkdMgtDestroy(SBkdMgt* bm) { taosMemoryFree(bm); } -int32_t bkdMgtGetDelta(SBkdMgt* bm, char* taskId, int64_t chkpId, SArray* list) { +int32_t bkdMgtGetDelta(SBkdMgt* bm, char* taskId, int64_t chkpId, SArray* list, char* dname) { int32_t code = 0; - taosThreadRwlockWrlock(&bm->rwLock); + taosThreadRwlockWrlock(&bm->rwLock); SDbChkp* pChkp = taosHashGet(bm->pDbChkpTbl, taskId, strlen(taskId)); - code = dbChkpGetDelta(pChkp, chkpId, list); + + if (pChkp == NULL) { + char* taskPath = taosMemoryCalloc(1, strlen(bm->path) + 64); + sprintf(taskPath, "%s%s%s", bm->path, TD_DIRSEP, taskId); + + SDbChkp* p = dbChkpCreate(taskPath, chkpId); + taosHashPut(bm->pDbChkpTbl, taskId, strlen(taskId), &p, sizeof(void*)); + + taosMemoryFree(taskPath); + } taosThreadRwlockUnlock(&bm->rwLock); + + code = dbChkpGetDelta(pChkp, chkpId, list); + + code = dbChkpDumpTo(pChkp, dname); + return code; } diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 51f2a18504..03696e1122 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -340,8 +340,10 @@ int32_t doUploadChkp(void* param) { SAsyncUploadArg* arg = param; char* path = NULL; int32_t code = 0; - if ((code = taskDbGenChkpUploadPath(arg->pTask->pBackend, arg->chkpId, (int8_t)(arg->type), &path)) != 0) { - stError("s-task:%s faile to gen upload checkpoint:%" PRId64 "", arg->pTask->id.idStr, arg->chkpId); + + if ((code = taskDbGenChkpUploadPath(arg->pTask->pBackend, arg->pTask->pMeta->bkdChkptMgt, arg->chkpId, + (int8_t)(arg->type), &path)) != 0) { + stError("s-task:%s failed to gen upload checkpoint:%" PRId64 "", arg->pTask->id.idStr, arg->chkpId); } if (code == 0 && uploadCheckpoint(arg->taskId, path) != 0) { stError("s-task:%s faile to upload checkpoint:%" PRId64 "", arg->pTask->id.idStr, arg->chkpId); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index c5b98784a6..102023a728 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -356,6 +356,8 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF pMeta->pHbInfo->stopFlag = 0; pMeta->qHandle = taosInitScheduler(32, 1, "stream-chkp", NULL); + pMeta->bkdChkptMgt = bkdMgtCreate(tpath); + return pMeta; _err: From 4a99bdc2b01eb88be26fea2963c7ccce98dea73d Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 9 Nov 2023 20:27:30 +0800 Subject: [PATCH 104/195] support s3 --- source/libs/stream/src/streamBackendRocksdb.c | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 253c18467e..4f5076b7ff 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1738,9 +1738,9 @@ int32_t taskDbGenChkpUploadPath(void* arg, void* mgt, int64_t chkpId, int8_t typ UPLOAD_TYPE utype = type; if (utype == UPLOAD_RSYNC) { - return taskDbGenChkpUplaodPath__rsync(pDb,chkpId, path); + return taskDbGenChkpUplaodPath__rsync(pDb, chkpId, path); } else if (utype == UPLOAD_S3) { - return taskDbGenChkpUplaodPath__s3(pDb,mgt, chkpId, path); + return taskDbGenChkpUplaodPath__s3(pDb, mgt, chkpId, path); } return -1; } @@ -3633,14 +3633,15 @@ int32_t bkdMgtGetDelta(SBkdMgt* bm, char* taskId, int64_t chkpId, SArray* list, taosHashPut(bm->pDbChkpTbl, taskId, strlen(taskId), &p, sizeof(void*)); taosMemoryFree(taskPath); + pChkp = p; + + code = dbChkpDumpTo(pChkp, dname); + taosThreadRwlockUnlock(&bm->rwLock); + return code; } - taosThreadRwlockUnlock(&bm->rwLock); - code = dbChkpGetDelta(pChkp, chkpId, list); - - code = dbChkpDumpTo(pChkp, dname); - + taosThreadRwlockUnlock(&bm->rwLock); return code; } From 0669037de0f4b805939c5ad0cae04d06412ff51c Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 9 Nov 2023 20:37:08 +0800 Subject: [PATCH 105/195] fix mem leak --- source/libs/stream/src/streamMeta.c | 2 ++ 1 file changed, 2 insertions(+) diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 102023a728..41f0d8dcfe 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -499,6 +499,8 @@ void streamMetaCloseImpl(void* arg) { taosThreadMutexDestroy(&pMeta->backendMutex); taosCleanUpScheduler(pMeta->qHandle); + bkdMgtDestroy(pMeta->bkdChkptMgt); + pMeta->role = NODE_ROLE_UNINIT; taosMemoryFree(pMeta); stDebug("end to close stream meta"); From 54fe42d3725a03d2009f9291d9378d4cff380ab9 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Fri, 10 Nov 2023 09:24:40 +0800 Subject: [PATCH 106/195] Merge branch '3.0' into enh/refactorBackend --- source/libs/stream/src/streamBackendRocksdb.c | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 4f5076b7ff..1a29a82c11 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1724,7 +1724,8 @@ int32_t taskDbGenChkpUplaodPath__s3(STaskDbWrapper* pDb, void* bkdChkpMgt, int64 char* temp = taosMemoryCalloc(1, strlen(pDb->path)); sprintf(temp, "%s%s%s", pDb->path, TD_DIRSEP, "tmp"); - if (!taosDirExist(temp)) { + if (taosDirExist(temp)) { + taosRemoveDir(temp); taosMkDir(temp); } bkdMgtGetDelta(p, pDb->idstr, chkpId, NULL, temp); From ad119ea4c0a3690e2b260e17d3e41ddbb800e119 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Fri, 10 Nov 2023 12:07:33 +0800 Subject: [PATCH 107/195] Merge branch '3.0' into enh/refactorBackend --- source/libs/stream/src/streamBackendRocksdb.c | 23 ++++++++++++------- 1 file changed, 15 insertions(+), 8 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 1a29a82c11..519019892f 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1691,7 +1691,7 @@ void taskDbDestroy(void* pDb) { return; } -int32_t taskDbGenChkpUplaodPath__rsync(STaskDbWrapper* pDb, int64_t chkpId, char** path) { +int32_t taskDbGenChkpUploadData__rsync(STaskDbWrapper* pDb, int64_t chkpId, char** path) { int64_t st = taosGetTimestampMs(); int32_t code = -1; int64_t refId = pDb->refId; @@ -1718,7 +1718,7 @@ int32_t taskDbGenChkpUplaodPath__rsync(STaskDbWrapper* pDb, int64_t chkpId, char return code; } -int32_t taskDbGenChkpUplaodPath__s3(STaskDbWrapper* pDb, void* bkdChkpMgt, int64_t chkpId, char** path) { +int32_t taskDbGenChkpUploadData__s3(STaskDbWrapper* pDb, void* bkdChkpMgt, int64_t chkpId, char** path) { SBkdMgt* p = (SBkdMgt*)bkdChkpMgt; char* temp = taosMemoryCalloc(1, strlen(pDb->path)); @@ -1727,6 +1727,8 @@ int32_t taskDbGenChkpUplaodPath__s3(STaskDbWrapper* pDb, void* bkdChkpMgt, int64 if (taosDirExist(temp)) { taosRemoveDir(temp); taosMkDir(temp); + } else { + taosMkDir(temp); } bkdMgtGetDelta(p, pDb->idstr, chkpId, NULL, temp); @@ -1739,9 +1741,9 @@ int32_t taskDbGenChkpUploadPath(void* arg, void* mgt, int64_t chkpId, int8_t typ UPLOAD_TYPE utype = type; if (utype == UPLOAD_RSYNC) { - return taskDbGenChkpUplaodPath__rsync(pDb, chkpId, path); + return taskDbGenChkpUploadData__rsync(pDb, chkpId, path); } else if (utype == UPLOAD_S3) { - return taskDbGenChkpUplaodPath__s3(pDb, mgt, chkpId, path); + return taskDbGenChkpUploadData__s3(pDb, mgt, chkpId, path); } return -1; } @@ -3399,7 +3401,7 @@ int32_t dbChkpGetDelta(SDbChkp* p, int64_t chkpId, SArray* list) { int32_t sstLen = strlen(pSST); memset(p->buf, 0, p->len); - sprintf(p->buf, "%s%scheckpoint%" PRId64 "", p->path, TD_DIRSEP, chkpId); + sprintf(p->buf, "%s%s%s%scheckpoint%" PRId64 "", p->path, TD_DIRSEP, "checkpoints", TD_DIRSEP, chkpId); taosArrayClearP(p->pAdd, taosMemoryFree); taosArrayClearP(p->pDel, taosMemoryFree); @@ -3527,8 +3529,8 @@ int32_t dbChkpDumpTo(SDbChkp* p, char* dname) { char* srcDir = taosMemoryCalloc(1, len); char* dstDir = taosMemoryCalloc(1, len); - sprintf(srcDir, "%s%s%s%" PRId64 "", p->path, TD_DIRSEP, "checkpoint", p->curChkpId); - sprintf(dstDir, "%s%s%s", p->path, TD_DIRSEP, dname); + sprintf(srcDir, "%s%s%s%s%s%" PRId64 "", p->path, TD_DIRSEP, "checkpoints", TD_DIRSEP, "checkpoint", p->curChkpId); + sprintf(dstDir, "%s", dname); if (!taosDirExist(srcDir)) { stError("failed to dump srcDir %s, reason: not exist such dir", srcDir); @@ -3602,6 +3604,7 @@ _ERROR: SBkdMgt* bkdMgtCreate(char* path) { SBkdMgt* p = taosMemoryCalloc(1, sizeof(SBkdMgt)); p->pDbChkpTbl = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), true, HASH_ENTRY_LOCK); + p->path = taosStrdup(path); taosThreadRwlockInit(&p->rwLock, NULL); return p; } @@ -3624,7 +3627,8 @@ int32_t bkdMgtGetDelta(SBkdMgt* bm, char* taskId, int64_t chkpId, SArray* list, int32_t code = 0; taosThreadRwlockWrlock(&bm->rwLock); - SDbChkp* pChkp = taosHashGet(bm->pDbChkpTbl, taskId, strlen(taskId)); + SDbChkp** ppChkp = taosHashGet(bm->pDbChkpTbl, taskId, strlen(taskId)); + SDbChkp* pChkp = NULL; if (pChkp == NULL) { char* taskPath = taosMemoryCalloc(1, strlen(bm->path) + 64); @@ -3639,9 +3643,12 @@ int32_t bkdMgtGetDelta(SBkdMgt* bm, char* taskId, int64_t chkpId, SArray* list, code = dbChkpDumpTo(pChkp, dname); taosThreadRwlockUnlock(&bm->rwLock); return code; + } else { + pChkp = *ppChkp; } code = dbChkpGetDelta(pChkp, chkpId, list); + code = dbChkpDumpTo(pChkp, dname); taosThreadRwlockUnlock(&bm->rwLock); return code; } From a3fdb6dd59a98d63b3761e09b01c3d615302a6a9 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Fri, 10 Nov 2023 16:06:08 +0800 Subject: [PATCH 108/195] Merge branch '3.0' into enh/refactorBackend --- source/common/src/cos.c | 89 ++++++++++--------- source/libs/stream/inc/streamBackendRocksdb.h | 2 +- source/libs/stream/src/streamBackendRocksdb.c | 55 +++++++++--- source/libs/stream/src/streamCheckpoint.c | 4 +- 4 files changed, 90 insertions(+), 60 deletions(-) diff --git a/source/common/src/cos.c b/source/common/src/cos.c index 0b6b0db885..c0b44a5df0 100644 --- a/source/common/src/cos.c +++ b/source/common/src/cos.c @@ -70,7 +70,7 @@ static void s3PrintError(const char *func, S3Status status, char error_details[] if (status < S3StatusErrorAccessDenied) { uError("%s: %s", __func__, S3_get_status_name(status)); } else { - uError("%s: %s, %s", __func__, S3_get_status_name(status), error_details); + uError("%s: %s, %s, %d", __func__, S3_get_status_name(status), error_details, status); } } @@ -85,7 +85,7 @@ typedef struct { char err_msg[128]; S3Status status; uint64_t content_length; - char *buf; + char * buf; int64_t buf_pos; } TS3SizeCBD; @@ -271,7 +271,7 @@ typedef struct MultipartPartData { S3Status status; put_object_callback_data put_object_data; int seq; - UploadManager *manager; + UploadManager * manager; } MultipartPartData; static int putObjectDataCallback(int bufferSize, char *buffer, void *callbackData) { @@ -316,7 +316,7 @@ S3Status MultipartResponseProperiesCallback(const S3ResponseProperties *properti MultipartPartData *data = (MultipartPartData *)callbackData; int seq = data->seq; - const char *etag = properties->eTag; + const char * etag = properties->eTag; data->manager->etags[seq - 1] = strdup(etag); data->manager->next_etags_pos = seq; return S3StatusOK; @@ -449,10 +449,10 @@ int32_t s3PutObjectFromFile2(const char *file, const char *object) { int32_t code = 0; const char *key = object; // const char *uploadId = 0; - const char *filename = 0; + const char * filename = 0; uint64_t contentLength = 0; - const char *cacheControl = 0, *contentType = 0, *md5 = 0; - const char *contentDispositionFilename = 0, *contentEncoding = 0; + const char * cacheControl = 0, *contentType = 0, *md5 = 0; + const char * contentDispositionFilename = 0, *contentEncoding = 0; int64_t expires = -1; S3CannedAcl cannedAcl = S3CannedAclPrivate; int metaPropertiesCount = 0; @@ -466,6 +466,7 @@ int32_t s3PutObjectFromFile2(const char *file, const char *object) { data.gb = 0; data.noStatus = noStatus; + uError("ERROR: %s stat file %s: ", __func__, file); if (taosStatFile(file, &contentLength, NULL, NULL) < 0) { uError("ERROR: %s Failed to stat file %s: ", __func__, file); code = TAOS_SYSTEM_ERROR(errno); @@ -637,7 +638,7 @@ typedef struct list_bucket_callback_data { char nextMarker[1024]; int keyCount; int allDetails; - SArray *objectArray; + SArray * objectArray; } list_bucket_callback_data; static S3Status listBucketCallback(int isTruncated, const char *nextMarker, int contentsCount, @@ -682,11 +683,11 @@ static void s3FreeObjectKey(void *pItem) { static SArray *getListByPrefix(const char *prefix) { S3BucketContext bucketContext = {0, tsS3BucketName, protocolG, uriStyleG, tsS3AccessKeyId, tsS3AccessKeySecret, - 0, awsRegionG}; + 0, awsRegionG}; S3ListBucketHandler listBucketHandler = {{&responsePropertiesCallbackNull, &responseCompleteCallback}, &listBucketCallback}; - const char *marker = 0, *delimiter = 0; + const char * marker = 0, *delimiter = 0; int maxkeys = 0, allDetails = 0; list_bucket_callback_data data; data.objectArray = taosArrayInit(32, sizeof(void *)); @@ -725,7 +726,7 @@ static SArray *getListByPrefix(const char *prefix) { void s3DeleteObjects(const char *object_name[], int nobject) { S3BucketContext bucketContext = {0, tsS3BucketName, protocolG, uriStyleG, tsS3AccessKeyId, tsS3AccessKeySecret, - 0, awsRegionG}; + 0, awsRegionG}; S3ResponseHandler responseHandler = {0, &responseCompleteCallback}; for (int i = 0; i < nobject; ++i) { @@ -776,7 +777,7 @@ int32_t s3GetObjectBlock(const char *object_name, int64_t offset, int64_t size, const char *ifMatch = 0, *ifNotMatch = 0; S3BucketContext bucketContext = {0, tsS3BucketName, protocolG, uriStyleG, tsS3AccessKeyId, tsS3AccessKeySecret, - 0, awsRegionG}; + 0, awsRegionG}; S3GetConditions getConditions = {ifModifiedSince, ifNotModifiedSince, ifMatch, ifNotMatch}; S3GetObjectHandler getObjectHandler = {{&responsePropertiesCallback, &responseCompleteCallback}, &getObjectDataCallback}; @@ -814,7 +815,7 @@ int32_t s3GetObjectToFile(const char *object_name, char *fileName) { const char *ifMatch = 0, *ifNotMatch = 0; S3BucketContext bucketContext = {0, tsS3BucketName, protocolG, uriStyleG, tsS3AccessKeyId, tsS3AccessKeySecret, - 0, awsRegionG}; + 0, awsRegionG}; S3GetConditions getConditions = {ifModifiedSince, ifNotModifiedSince, ifMatch, ifNotMatch}; S3GetObjectHandler getObjectHandler = {{&responsePropertiesCallbackNull, &responseCompleteCallback}, &getObjectCallback}; @@ -845,7 +846,7 @@ int32_t s3GetObjectsByPrefix(const char *prefix, const char *path) { if (objectArray == NULL) return -1; for (size_t i = 0; i < taosArrayGetSize(objectArray); i++) { - char *object = taosArrayGetP(objectArray, i); + char * object = taosArrayGetP(objectArray, i); const char *tmp = strchr(object, '/'); tmp = (tmp == NULL) ? object : tmp + 1; char fileName[PATH_MAX] = {0}; @@ -934,12 +935,12 @@ static void s3InitRequestOptions(cos_request_options_t *options, int is_cname) { int32_t s3PutObjectFromFile(const char *file_str, const char *object_str) { int32_t code = 0; - cos_pool_t *p = NULL; + cos_pool_t * p = NULL; int is_cname = 0; - cos_status_t *s = NULL; + cos_status_t * s = NULL; cos_request_options_t *options = NULL; cos_string_t bucket, object, file; - cos_table_t *resp_headers; + cos_table_t * resp_headers; // int traffic_limit = 0; cos_pool_create(&p, NULL); @@ -970,14 +971,14 @@ int32_t s3PutObjectFromFile(const char *file_str, const char *object_str) { int32_t s3PutObjectFromFile2(const char *file_str, const char *object_str) { int32_t code = 0; - cos_pool_t *p = NULL; + cos_pool_t * p = NULL; int is_cname = 0; - cos_status_t *s = NULL; - cos_request_options_t *options = NULL; + cos_status_t * s = NULL; + cos_request_options_t * options = NULL; cos_string_t bucket, object, file; - cos_table_t *resp_headers; + cos_table_t * resp_headers; int traffic_limit = 0; - cos_table_t *headers = NULL; + cos_table_t * headers = NULL; cos_resumable_clt_params_t *clt_params = NULL; cos_pool_create(&p, NULL); @@ -1010,11 +1011,11 @@ int32_t s3PutObjectFromFile2(const char *file_str, const char *object_str) { } void s3DeleteObjectsByPrefix(const char *prefix_str) { - cos_pool_t *p = NULL; + cos_pool_t * p = NULL; cos_request_options_t *options = NULL; int is_cname = 0; cos_string_t bucket; - cos_status_t *s = NULL; + cos_status_t * s = NULL; cos_string_t prefix; cos_pool_create(&p, NULL); @@ -1029,10 +1030,10 @@ void s3DeleteObjectsByPrefix(const char *prefix_str) { } void s3DeleteObjects(const char *object_name[], int nobject) { - cos_pool_t *p = NULL; + cos_pool_t * p = NULL; int is_cname = 0; cos_string_t bucket; - cos_table_t *resp_headers = NULL; + cos_table_t * resp_headers = NULL; cos_request_options_t *options = NULL; cos_list_t object_list; cos_list_t deleted_object_list; @@ -1066,14 +1067,14 @@ void s3DeleteObjects(const char *object_name[], int nobject) { bool s3Exists(const char *object_name) { bool ret = false; - cos_pool_t *p = NULL; + cos_pool_t * p = NULL; int is_cname = 0; - cos_status_t *s = NULL; - cos_request_options_t *options = NULL; + cos_status_t * s = NULL; + cos_request_options_t * options = NULL; cos_string_t bucket; cos_string_t object; - cos_table_t *resp_headers; - cos_table_t *headers = NULL; + cos_table_t * resp_headers; + cos_table_t * headers = NULL; cos_object_exist_status_e object_exist; cos_pool_create(&p, NULL); @@ -1100,15 +1101,15 @@ bool s3Exists(const char *object_name) { bool s3Get(const char *object_name, const char *path) { bool ret = false; - cos_pool_t *p = NULL; + cos_pool_t * p = NULL; int is_cname = 0; - cos_status_t *s = NULL; + cos_status_t * s = NULL; cos_request_options_t *options = NULL; cos_string_t bucket; cos_string_t object; cos_string_t file; - cos_table_t *resp_headers = NULL; - cos_table_t *headers = NULL; + cos_table_t * resp_headers = NULL; + cos_table_t * headers = NULL; int traffic_limit = 0; //创建内存池 @@ -1144,15 +1145,15 @@ bool s3Get(const char *object_name, const char *path) { int32_t s3GetObjectBlock(const char *object_name, int64_t offset, int64_t block_size, bool check, uint8_t **ppBlock) { (void)check; int32_t code = 0; - cos_pool_t *p = NULL; + cos_pool_t * p = NULL; int is_cname = 0; - cos_status_t *s = NULL; + cos_status_t * s = NULL; cos_request_options_t *options = NULL; cos_string_t bucket; cos_string_t object; - cos_table_t *resp_headers; - cos_table_t *headers = NULL; - cos_buf_t *content = NULL; + cos_table_t * resp_headers; + cos_table_t * headers = NULL; + cos_buf_t * content = NULL; // cos_string_t file; // int traffic_limit = 0; char range_buf[64]; @@ -1246,7 +1247,7 @@ void s3EvictCache(const char *path, long object_size) { terrno = TAOS_SYSTEM_ERROR(errno); vError("failed to open %s since %s", dir_name, terrstr()); } - SArray *evict_files = taosArrayInit(16, sizeof(SEvictFile)); + SArray * evict_files = taosArrayInit(16, sizeof(SEvictFile)); tdbDirEntryPtr pDirEntry; while ((pDirEntry = taosReadDir(pDir)) != NULL) { char *name = taosGetDirEntryName(pDirEntry); @@ -1288,13 +1289,13 @@ void s3EvictCache(const char *path, long object_size) { long s3Size(const char *object_name) { long size = 0; - cos_pool_t *p = NULL; + cos_pool_t * p = NULL; int is_cname = 0; - cos_status_t *s = NULL; + cos_status_t * s = NULL; cos_request_options_t *options = NULL; cos_string_t bucket; cos_string_t object; - cos_table_t *resp_headers = NULL; + cos_table_t * resp_headers = NULL; //创建内存池 cos_pool_create(&p, NULL); diff --git a/source/libs/stream/inc/streamBackendRocksdb.h b/source/libs/stream/inc/streamBackendRocksdb.h index cd620ed5b7..bce51a9d2f 100644 --- a/source/libs/stream/inc/streamBackendRocksdb.h +++ b/source/libs/stream/inc/streamBackendRocksdb.h @@ -254,5 +254,5 @@ int32_t bkdMgtGetDelta(SBkdMgt* bm, char* taskId, int64_t chkpId, SArray* list, int32_t bkdMgtDumpTo(SBkdMgt* bm, char* taskId, char* dname); void bkdMgtDestroy(SBkdMgt* bm); -int32_t taskDbGenChkpUploadPath(void* arg, void* bkdMgt, int64_t chkpId, int8_t type, char** pathkj); +int32_t taskDbGenChkpUploadData(void* arg, void* bkdMgt, int64_t chkpId, int8_t type, char** pathkj); #endif \ No newline at end of file diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 519019892f..604f5f7c2a 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1736,7 +1736,7 @@ int32_t taskDbGenChkpUploadData__s3(STaskDbWrapper* pDb, void* bkdChkpMgt, int64 return 0; } -int32_t taskDbGenChkpUploadPath(void* arg, void* mgt, int64_t chkpId, int8_t type, char** path) { +int32_t taskDbGenChkpUploadData(void* arg, void* mgt, int64_t chkpId, int8_t type, char** path) { STaskDbWrapper* pDb = arg; UPLOAD_TYPE utype = type; @@ -3363,13 +3363,27 @@ _err: return code >= 0 ? 0 : -1; } +int32_t isBkdDataMeta(char* name) { + const char* pCurrent = "CURRENT"; + int32_t currLen = strlen(pCurrent); + + const char* pManifest = "MANIFEST-"; + int32_t maniLen = strlen(pManifest); + + if (strlen(name) >= maniLen && strncmp(name, pManifest, maniLen) == 0) { + return 1; + } else if (strlen(name) == currLen && strcmp(name, pCurrent) == 0) { + return 1; + } + return 0; +} int32_t compareHashTableImpl(SHashObj* p1, SHashObj* p2, SArray* diff) { int32_t code = 0; size_t len = 0; void* pIter = taosHashIterate(p2, NULL); while (pIter) { char* name = taosHashGetKey(pIter, &len); - if (!taosHashGet(p1, name, len)) { + if (!isBkdDataMeta(name) && !taosHashGet(p1, name, len)) { char* p = taosStrdup(name); taosArrayPush(diff, &p); } @@ -3431,13 +3445,22 @@ int32_t dbChkpGetDelta(SDbChkp* p, int64_t chkpId, SArray* list) { continue; } } + + void* pIter = taosHashIterate(p->pSstTbl[1 - p->idx], NULL); + while (pIter) { + char *name = taosHashGetKey(pIter, NULL); + stError("curr file list: %s", name); + pIter = taosHashIterate(p->pSstTbl[1 - p->idx], pIter); + } + if (p->init == 0) { void* pIter = taosHashIterate(p->pSstTbl[1 - p->idx], NULL); while (pIter) { size_t len; char* name = taosHashGetKey(pIter, &len); - if (name != NULL && len != 0) { - taosArrayPush(p->pAdd, &name); + if (name != NULL && !isBkdDataMeta(name)) { + char* fname = taosStrdup(name); + taosArrayPush(p->pAdd, &fname); } pIter = taosHashIterate(p->pSstTbl[1 - p->idx], pIter); } @@ -3538,12 +3561,12 @@ int32_t dbChkpDumpTo(SDbChkp* p, char* dname) { goto _ERROR; } - code = taosMkDir(dstDir); - if (code != 0) { - terrno = TAOS_SYSTEM_ERROR(errno); - stError("failed to mkdir srcDir %s, reason: %s", dstDir, terrstr()); - goto _ERROR; - } + // code = taosMkDir(dstDir); + // if (code != 0) { + // terrno = TAOS_SYSTEM_ERROR(errno); + // stError("failed to mkdir srcDir %s, reason: %s", dstDir, terrstr()); + // goto _ERROR; + // } // clear current file memset(dstBuf, 0, len); @@ -3563,7 +3586,9 @@ int32_t dbChkpDumpTo(SDbChkp* p, char* dname) { sprintf(srcBuf, "%s%s%s", srcDir, TD_DIRSEP, filename); sprintf(dstBuf, "%s%s%s", dstDir, TD_DIRSEP, filename); - taosCopyFile(srcBuf, dstBuf); + if (taosCopyFile(srcBuf, dstBuf) < 0) { + stError("failed to copy file from %s to %s", srcBuf, dstBuf); + } } // del file in $name for (int i = 0; i < taosArrayGetSize(p->pDel); i++) { @@ -3580,14 +3605,18 @@ int32_t dbChkpDumpTo(SDbChkp* p, char* dname) { memset(dstBuf, 0, len); sprintf(srcBuf, "%s%s%s", srcDir, TD_DIRSEP, p->pCurrent); sprintf(dstBuf, "%s%s%s", dstDir, TD_DIRSEP, p->pCurrent); - taosCopyFile(srcBuf, dstBuf); + if (taosCopyFile(srcBuf, dstBuf) < 0) { + stError("failed to copy file from %s to %s", srcBuf, dstBuf); + } // copy manifest file to dst dir memset(srcBuf, 0, len); memset(dstBuf, 0, len); sprintf(srcBuf, "%s%s%s", srcDir, TD_DIRSEP, p->pManifest); sprintf(dstBuf, "%s%s%s", dstDir, TD_DIRSEP, p->pManifest); - taosCopyFile(srcBuf, dstBuf); + if (taosCopyFile(srcBuf, dstBuf) < 0) { + stError("failed to copy file from %s to %s", srcBuf, dstBuf); + } // clear delta data buf taosArrayClearP(p->pAdd, taosMemoryFree); diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 03696e1122..854cdc12b1 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -341,12 +341,12 @@ int32_t doUploadChkp(void* param) { char* path = NULL; int32_t code = 0; - if ((code = taskDbGenChkpUploadPath(arg->pTask->pBackend, arg->pTask->pMeta->bkdChkptMgt, arg->chkpId, + if ((code = taskDbGenChkpUploadData(arg->pTask->pBackend, arg->pTask->pMeta->bkdChkptMgt, arg->chkpId, (int8_t)(arg->type), &path)) != 0) { stError("s-task:%s failed to gen upload checkpoint:%" PRId64 "", arg->pTask->id.idStr, arg->chkpId); } if (code == 0 && uploadCheckpoint(arg->taskId, path) != 0) { - stError("s-task:%s faile to upload checkpoint:%" PRId64 "", arg->pTask->id.idStr, arg->chkpId); + stError("s-task:%s failed to upload checkpoint:%" PRId64, arg->pTask->id.idStr, arg->chkpId); } taosMemoryFree(path); From 1e8d46c405a06614103e4e2f6f746a8a2872a158 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Fri, 10 Nov 2023 18:37:25 +0800 Subject: [PATCH 109/195] Merge branch '3.0' into enh/refactorBackend --- source/libs/stream/src/streamBackendRocksdb.c | 37 +++++++++++-------- source/libs/stream/src/streamCheckpoint.c | 4 +- 2 files changed, 24 insertions(+), 17 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 604f5f7c2a..5c834a6e95 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1722,7 +1722,7 @@ int32_t taskDbGenChkpUploadData__s3(STaskDbWrapper* pDb, void* bkdChkpMgt, int64 SBkdMgt* p = (SBkdMgt*)bkdChkpMgt; char* temp = taosMemoryCalloc(1, strlen(pDb->path)); - sprintf(temp, "%s%s%s", pDb->path, TD_DIRSEP, "tmp"); + sprintf(temp, "%s%s%s%" PRId64 "", pDb->path, TD_DIRSEP, "tmp", chkpId); if (taosDirExist(temp)) { taosRemoveDir(temp); @@ -3363,16 +3363,16 @@ _err: return code >= 0 ? 0 : -1; } -int32_t isBkdDataMeta(char* name) { +int32_t isBkdDataMeta(char* name, int32_t len) { const char* pCurrent = "CURRENT"; int32_t currLen = strlen(pCurrent); const char* pManifest = "MANIFEST-"; int32_t maniLen = strlen(pManifest); - if (strlen(name) >= maniLen && strncmp(name, pManifest, maniLen) == 0) { + if (len >= maniLen && strncmp(name, pManifest, maniLen) == 0) { return 1; - } else if (strlen(name) == currLen && strcmp(name, pCurrent) == 0) { + } else if (len == currLen && strcmp(name, pCurrent) == 0) { return 1; } return 0; @@ -3383,9 +3383,10 @@ int32_t compareHashTableImpl(SHashObj* p1, SHashObj* p2, SArray* diff) { void* pIter = taosHashIterate(p2, NULL); while (pIter) { char* name = taosHashGetKey(pIter, &len); - if (!isBkdDataMeta(name) && !taosHashGet(p1, name, len)) { - char* p = taosStrdup(name); - taosArrayPush(diff, &p); + if (!isBkdDataMeta(name, len) && !taosHashGet(p1, name, len)) { + char* fname = taosMemoryCalloc(1, len + 1); + strncpy(fname, name, len); + taosArrayPush(diff, &fname); } pIter = taosHashIterate(p2, pIter); } @@ -3429,18 +3430,17 @@ int32_t dbChkpGetDelta(SDbChkp* p, int64_t chkpId, SArray* list) { if (strlen(name) == currLen && strcmp(name, pCurrent) == 0) { taosMemoryFreeClear(p->pCurrent); p->pCurrent = taosStrdup(name); - taosHashPut(p->pSstTbl[1 - p->idx], name, strlen(name), &dummy, sizeof(dummy)); + // taosHashPut(p->pSstTbl[1 - p->idx], name, strlen(name), &dummy, sizeof(dummy)); continue; } if (strlen(name) >= maniLen && strncmp(name, pManifest, maniLen) == 0) { taosMemoryFreeClear(p->pManifest); p->pManifest = taosStrdup(name); - taosHashPut(p->pSstTbl[1 - p->idx], name, strlen(name), &dummy, sizeof(dummy)); + // taosHashPut(p->pSstTbl[1 - p->idx], name, strlen(name), &dummy, sizeof(dummy)); continue; } if (strlen(name) >= sstLen && strncmp(name + strlen(name) - 4, pSST, sstLen) == 0) { - // char* p = taosStrdup(name); taosHashPut(p->pSstTbl[1 - p->idx], name, strlen(name), &dummy, sizeof(dummy)); continue; } @@ -3448,18 +3448,23 @@ int32_t dbChkpGetDelta(SDbChkp* p, int64_t chkpId, SArray* list) { void* pIter = taosHashIterate(p->pSstTbl[1 - p->idx], NULL); while (pIter) { - char *name = taosHashGetKey(pIter, NULL); - stError("curr file list: %s", name); + size_t len = 0; + char* name = taosHashGetKey(pIter, &len); + + char* buf = taosMemoryCalloc(1, len + 1); + strncpy(buf, name, len); + stError("curr file list: %s", buf); pIter = taosHashIterate(p->pSstTbl[1 - p->idx], pIter); } if (p->init == 0) { void* pIter = taosHashIterate(p->pSstTbl[1 - p->idx], NULL); while (pIter) { - size_t len; + size_t len = 0; char* name = taosHashGetKey(pIter, &len); - if (name != NULL && !isBkdDataMeta(name)) { - char* fname = taosStrdup(name); + if (name != NULL && !isBkdDataMeta(name, len)) { + char* fname = taosMemoryCalloc(1, len + 1); + strncpy(fname, name, len); taosArrayPush(p->pAdd, &fname); } pIter = taosHashIterate(p->pSstTbl[1 - p->idx], pIter); @@ -3579,8 +3584,8 @@ int32_t dbChkpDumpTo(SDbChkp* p, char* dname) { // add file to $name dir for (int i = 0; i < taosArrayGetSize(p->pAdd); i++) { - memset(dstBuf, 0, len); memset(srcBuf, 0, len); + memset(dstBuf, 0, len); char* filename = taosArrayGetP(p->pAdd, i); sprintf(srcBuf, "%s%s%s", srcDir, TD_DIRSEP, filename); diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 854cdc12b1..f99713a431 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -348,7 +348,7 @@ int32_t doUploadChkp(void* param) { if (code == 0 && uploadCheckpoint(arg->taskId, path) != 0) { stError("s-task:%s failed to upload checkpoint:%" PRId64, arg->pTask->id.idStr, arg->chkpId); } - + taosRemoveDir(path); taosMemoryFree(path); taosMemoryFree(arg->taskId); taosMemoryFree(arg); @@ -435,6 +435,7 @@ static int uploadCheckpointToS3(char* id, char* path) { if (pDir == NULL) return -1; TdDirEntryPtr de = NULL; + s3Init(); while ((de = taosReadDir(pDir)) != NULL) { char* name = taosGetDirEntryName(de); if (strcmp(name, ".") == 0 || strcmp(name, "..") == 0 || taosDirEntryIsDir(de)) continue; @@ -454,6 +455,7 @@ static int uploadCheckpointToS3(char* id, char* path) { return -1; } stDebug("[s3] upload checkpoint:%s", filename); + break; } taosCloseDir(&pDir); From cf3e01e9426b2e29415aafa75b61522c914f6aa9 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Mon, 13 Nov 2023 10:34:44 +0800 Subject: [PATCH 110/195] Merge branch '3.0' into enh/refactorBackend --- source/common/src/tglobal.c | 4 ++-- source/libs/stream/src/streamBackendRocksdb.c | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index ab6ff53fbc..58e5e4e79e 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -247,7 +247,7 @@ int32_t tsTtlBatchDropNum = 10000; // number of tables dropped per batch // internal int32_t tsTransPullupInterval = 2; int32_t tsMqRebalanceInterval = 2; -int32_t tsStreamCheckpointInterval = 60; +int32_t tsStreamCheckpointInterval = 30; float tsSinkDataRate = 2.0; int32_t tsStreamNodeCheckInterval = 30; int32_t tsTtlUnit = 86400; @@ -720,7 +720,7 @@ static int32_t taosAddServerCfg(SConfig *pCfg) { if (cfgAddBool(pCfg, "disableStream", tsDisableStream, CFG_SCOPE_SERVER, CFG_DYN_ENT_SERVER) != 0) return -1; if (cfgAddInt64(pCfg, "streamBufferSize", tsStreamBufferSize, 0, INT64_MAX, CFG_SCOPE_SERVER, CFG_DYN_NONE) != 0) return -1; - if (cfgAddInt64(pCfg, "checkpointInterval", tsStreamCheckpointInterval, 60, 1200, CFG_SCOPE_SERVER, + if (cfgAddInt64(pCfg, "checkpointInterval", tsStreamCheckpointInterval, 20, 1200, CFG_SCOPE_SERVER, CFG_DYN_ENT_SERVER) != 0) return -1; if (cfgAddFloat(pCfg, "streamSinkDataRate", tsSinkDataRate, 0.1, 5, CFG_SCOPE_SERVER, CFG_DYN_NONE) != 0) return -1; diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 5c834a6e95..72a7c718bd 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1521,7 +1521,7 @@ void taskDbInitOpt(STaskDbWrapper* pTaskDb) { } void taskDbInitChkpOpt(STaskDbWrapper* pTaskDb) { pTaskDb->chkpId = -1; - pTaskDb->chkpCap = 2; + pTaskDb->chkpCap = 4; pTaskDb->chkpSaved = taosArrayInit(4, sizeof(int64_t)); pTaskDb->chkpInUse = taosArrayInit(4, sizeof(int64_t)); From 1786f7077d3ed8bb7bf5c0e3fc2cbfb8783668bc Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Mon, 13 Nov 2023 14:35:23 +0800 Subject: [PATCH 111/195] Merge branch '3.0' into enh/refactorBackend --- source/libs/stream/src/streamBackendRocksdb.c | 2 ++ source/libs/stream/src/streamMeta.c | 2 ++ 2 files changed, 4 insertions(+) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index b65e26be3b..3740d8fe56 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -3654,6 +3654,8 @@ void bkdMgtDestroy(SBkdMgt* bm) { } taosThreadRwlockDestroy(&bm->rwLock); + taosMemoryFree(bm->path); + taosHashCleanup(bm->pDbChkpTbl); taosMemoryFree(bm); } diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index d144d5a58e..8fb10ddc01 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -500,6 +500,8 @@ void streamMetaCloseImpl(void* arg) { taosThreadMutexDestroy(&pMeta->backendMutex); taosCleanUpScheduler(pMeta->qHandle); + taosMemoryFree(pMeta->qHandle); + bkdMgtDestroy(pMeta->bkdChkptMgt); pMeta->role = NODE_ROLE_UNINIT; From 4104840ceaa909015131ef1ed7e224e77e42c931 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Mon, 13 Nov 2023 18:07:38 +0800 Subject: [PATCH 112/195] Merge branch '3.0' into enh/refactorBackend --- source/common/src/cos.c | 2 +- source/common/src/tglobal.c | 4 +- source/libs/stream/inc/streamBackendRocksdb.h | 1 + source/libs/stream/src/streamBackendRocksdb.c | 57 ++++++++++++------- source/libs/stream/src/streamCheckpoint.c | 2 +- 5 files changed, 43 insertions(+), 23 deletions(-) diff --git a/source/common/src/cos.c b/source/common/src/cos.c index 509bd31512..09b2b8e4db 100644 --- a/source/common/src/cos.c +++ b/source/common/src/cos.c @@ -467,7 +467,7 @@ int32_t s3PutObjectFromFile2(const char *file, const char *object) { // data.infileFD = NULL; // data.noStatus = noStatus; - uError("ERROR: %s stat file %s: ", __func__, file); + // uError("ERROR: %s stat file %s: ", __func__, file); if (taosStatFile(file, &contentLength, NULL, NULL) < 0) { uError("ERROR: %s Failed to stat file %s: ", __func__, file); code = TAOS_SYSTEM_ERROR(errno); diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index a1566c23a5..8fe93ae201 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -247,7 +247,7 @@ int32_t tsTtlBatchDropNum = 10000; // number of tables dropped per batch // internal int32_t tsTransPullupInterval = 2; int32_t tsMqRebalanceInterval = 2; -int32_t tsStreamCheckpointInterval = 30; +int32_t tsStreamCheckpointInterval = 10; float tsSinkDataRate = 2.0; int32_t tsStreamNodeCheckInterval = 15; int32_t tsTtlUnit = 86400; @@ -721,7 +721,7 @@ static int32_t taosAddServerCfg(SConfig *pCfg) { if (cfgAddBool(pCfg, "disableStream", tsDisableStream, CFG_SCOPE_SERVER, CFG_DYN_ENT_SERVER) != 0) return -1; if (cfgAddInt64(pCfg, "streamBufferSize", tsStreamBufferSize, 0, INT64_MAX, CFG_SCOPE_SERVER, CFG_DYN_NONE) != 0) return -1; - if (cfgAddInt64(pCfg, "checkpointInterval", tsStreamCheckpointInterval, 20, 1200, CFG_SCOPE_SERVER, + if (cfgAddInt64(pCfg, "checkpointInterval", tsStreamCheckpointInterval, 1, 1200, CFG_SCOPE_SERVER, CFG_DYN_ENT_SERVER) != 0) return -1; if (cfgAddFloat(pCfg, "streamSinkDataRate", tsSinkDataRate, 0.1, 5, CFG_SCOPE_SERVER, CFG_DYN_NONE) != 0) return -1; diff --git a/source/libs/stream/inc/streamBackendRocksdb.h b/source/libs/stream/inc/streamBackendRocksdb.h index a101b6a09d..92ec1899db 100644 --- a/source/libs/stream/inc/streamBackendRocksdb.h +++ b/source/libs/stream/inc/streamBackendRocksdb.h @@ -77,6 +77,7 @@ typedef struct { SArray* chkpInUse; int32_t chkpCap; TdThreadRwlock chkpDirLock; + int64_t dataWritten; } STaskDbWrapper; diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 3740d8fe56..0d18b6a900 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -891,6 +891,7 @@ int32_t taskDbDoCheckpoint(void* arg, int64_t chkpId) { } code = chkpMayDelObsolete(pTaskDb, chkpId, pChkpDir); + pTaskDb->dataWritten = 0; pTaskDb->chkpId = chkpId; @@ -2161,7 +2162,8 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfKe break; \ } \ STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; \ - char toString[128] = {0}; \ + wrapper->dataWritten += 1; \ + char toString[128] = {0}; \ if (qDebugFlag & DEBUG_TRACE) ginitDict[i].toStrFunc((void*)key, toString); \ int32_t klen = ginitDict[i].enFunc((void*)key, buf); \ rocksdb_column_family_handle_t* pHandle = ((rocksdb_column_family_handle_t**)wrapper->pCf)[ginitDict[i].idx]; \ @@ -2237,7 +2239,8 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfKe break; \ } \ STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; \ - char toString[128] = {0}; \ + wrapper->dataWritten += 1; \ + char toString[128] = {0}; \ if (qDebugFlag & DEBUG_TRACE) ginitDict[i].toStrFunc((void*)key, toString); \ int32_t klen = ginitDict[i].enFunc((void*)key, buf); \ rocksdb_column_family_handle_t* pHandle = ((rocksdb_column_family_handle_t**)wrapper->pCf)[ginitDict[i].idx]; \ @@ -2277,6 +2280,7 @@ int32_t streamStateClear_rocksdb(SStreamState* pState) { stDebug("streamStateClear_rocksdb"); STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + wrapper->dataWritten += 1; char sKeyStr[128] = {0}; char eKeyStr[128] = {0}; @@ -3253,6 +3257,7 @@ void streamStateDestroyBatch(void* pBatch) { rocksdb_writebatch_destroy((rock int32_t streamStatePutBatch(SStreamState* pState, const char* cfKeyName, rocksdb_writebatch_t* pBatch, void* key, void* val, int32_t vlen, int64_t ttl) { STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + wrapper->dataWritten += 1; int i = streamStateGetCfIdx(pState, cfKeyName); if (i < 0) { @@ -3285,7 +3290,9 @@ int32_t streamStatePutBatchOptimize(SStreamState* pState, int32_t cfIdx, rocksdb char* ttlV = tmpBuf; int32_t ttlVLen = ginitDict[cfIdx].enValueFunc(val, vlen, ttl, &ttlV); - STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + wrapper->dataWritten += 1; + rocksdb_column_family_handle_t* pCf = wrapper->pCf[ginitDict[cfIdx].idx]; rocksdb_writebatch_put_cf((rocksdb_writebatch_t*)pBatch, pCf, buf, (size_t)klen, ttlV, (size_t)ttlVLen); @@ -3303,6 +3310,7 @@ int32_t streamStatePutBatchOptimize(SStreamState* pState, int32_t cfIdx, rocksdb int32_t streamStatePutBatch_rocksdb(SStreamState* pState, void* pBatch) { char* err = NULL; STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + wrapper->dataWritten += 1; rocksdb_write(wrapper->db, wrapper->writeOpt, (rocksdb_writebatch_t*)pBatch, &err); if (err != NULL) { stError("streamState failed to write batch, err:%s", err); @@ -3401,6 +3409,26 @@ int32_t compareHashTable(SHashObj* p1, SHashObj* p2, SArray* add, SArray* del) { return code; } +void hashTableToDebug(SHashObj* pTbl) { + size_t sz = taosHashGetSize(pTbl); + int32_t total = 0; + char* buf = taosMemoryCalloc(1, sz * 16); + void* pIter = taosHashIterate(pTbl, NULL); + while (pIter) { + size_t len = 0; + char* name = taosHashGetKey(pIter, &len); + char* tname = taosMemoryCalloc(1, len + 1); + memcpy(tname, name, len); + total += sprintf(buf + total, "%s,", tname); + + pIter = taosHashIterate(pTbl, pIter); + taosMemoryFree(tname); + } + buf[total - 1] = 0; + + stTrace("curr file list:[%s]", buf); + taosMemoryFree(buf); +} int32_t dbChkpGetDelta(SDbChkp* p, int64_t chkpId, SArray* list) { taosThreadRwlockWrlock(&p->rwLock); @@ -3420,6 +3448,7 @@ int32_t dbChkpGetDelta(SDbChkp* p, int64_t chkpId, SArray* list) { taosArrayClearP(p->pAdd, taosMemoryFree); taosArrayClearP(p->pDel, taosMemoryFree); + taosHashClear(p->pSstTbl[1 - p->idx]); TdDirPtr pDir = taosOpenDir(p->buf); TdDirEntryPtr de = NULL; @@ -3445,17 +3474,13 @@ int32_t dbChkpGetDelta(SDbChkp* p, int64_t chkpId, SArray* list) { continue; } } + taosCloseDir(&pDir); - void* pIter = taosHashIterate(p->pSstTbl[1 - p->idx], NULL); - while (pIter) { - size_t len = 0; - char* name = taosHashGetKey(pIter, &len); + stTrace("chkp get file list: 1-1"); + hashTableToDebug(p->pSstTbl[1 - p->idx]); - char* buf = taosMemoryCalloc(1, len + 1); - strncpy(buf, name, len); - stError("curr file list: %s", buf); - pIter = taosHashIterate(p->pSstTbl[1 - p->idx], pIter); - } + stTrace("chkp get file list: 1-2"); + hashTableToDebug(p->pSstTbl[p->idx]); if (p->init == 0) { void* pIter = taosHashIterate(p->pSstTbl[1 - p->idx], NULL); @@ -3482,7 +3507,6 @@ int32_t dbChkpGetDelta(SDbChkp* p, int64_t chkpId, SArray* list) { taosArrayClearP(p->pDel, taosMemoryFree); taosHashClear(p->pSstTbl[1 - p->idx]); p->update = 0; - taosCloseDir(&pDir); return code; } @@ -3493,11 +3517,8 @@ int32_t dbChkpGetDelta(SDbChkp* p, int64_t chkpId, SArray* list) { p->preCkptId = p->curChkpId; p->curChkpId = chkpId; } - - taosHashClear(p->pSstTbl[p->idx]); p->idx = 1 - p->idx; - taosCloseDir(&pDir); taosThreadRwlockUnlock(&p->rwLock); return 0; @@ -3664,7 +3685,7 @@ int32_t bkdMgtGetDelta(SBkdMgt* bm, char* taskId, int64_t chkpId, SArray* list, taosThreadRwlockWrlock(&bm->rwLock); SDbChkp** ppChkp = taosHashGet(bm->pDbChkpTbl, taskId, strlen(taskId)); - SDbChkp* pChkp = NULL; + SDbChkp* pChkp = ppChkp != NULL ? *ppChkp : NULL; if (pChkp == NULL) { char* taskPath = taosMemoryCalloc(1, strlen(bm->path) + 64); @@ -3679,8 +3700,6 @@ int32_t bkdMgtGetDelta(SBkdMgt* bm, char* taskId, int64_t chkpId, SArray* list, code = dbChkpDumpTo(pChkp, dname); taosThreadRwlockUnlock(&bm->rwLock); return code; - } else { - pChkp = *ppChkp; } code = dbChkpGetDelta(pChkp, chkpId, list); diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 946656c7d0..f7ac9e61bc 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -457,7 +457,7 @@ static int uploadCheckpointToS3(char* id, char* path) { return -1; } stDebug("[s3] upload checkpoint:%s", filename); - break; + // break; } taosCloseDir(&pDir); From 2867a2febb43768d923b99d86f620119385b1231 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Mon, 13 Nov 2023 18:07:56 +0800 Subject: [PATCH 113/195] Merge branch '3.0' into enh/refactorBackend --- cmake/cmake.options | 1 + 1 file changed, 1 insertion(+) diff --git a/cmake/cmake.options b/cmake/cmake.options index d34c34dd89..e12de3e4e6 100644 --- a/cmake/cmake.options +++ b/cmake/cmake.options @@ -151,6 +151,7 @@ IF(${BUILD_S3}) IF(${BUILD_WITH_S3}) +add_definitions(-DUSE_S3) option(BUILD_WITH_COS "If build with cos" OFF) ELSE () From 68b6cc1d9d421891aa6cab6693f4632ab7fbc13f Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Mon, 13 Nov 2023 19:07:58 +0800 Subject: [PATCH 114/195] Merge branch '3.0' into enh/refactorBackend --- source/libs/stream/src/streamBackendRocksdb.c | 54 ++++++++++++++----- 1 file changed, 42 insertions(+), 12 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 0d18b6a900..5c59621ca9 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -3409,25 +3409,58 @@ int32_t compareHashTable(SHashObj* p1, SHashObj* p2, SArray* add, SArray* del) { return code; } -void hashTableToDebug(SHashObj* pTbl) { +void hashTableToDebug(SHashObj* pTbl, char** buf) { size_t sz = taosHashGetSize(pTbl); int32_t total = 0; - char* buf = taosMemoryCalloc(1, sz * 16); + char* p = taosMemoryCalloc(1, sz * 16); void* pIter = taosHashIterate(pTbl, NULL); while (pIter) { size_t len = 0; char* name = taosHashGetKey(pIter, &len); char* tname = taosMemoryCalloc(1, len + 1); memcpy(tname, name, len); - total += sprintf(buf + total, "%s,", tname); + total += sprintf(p + total, "%s,", tname); pIter = taosHashIterate(pTbl, pIter); taosMemoryFree(tname); } - buf[total - 1] = 0; + p[total - 1] = 0; + *buf = p; +} +void strArrayDebugInfo(SArray* pArr, char** buf) { + int32_t sz = taosArrayGetSize(pArr); + if (sz <= 0) return; - stTrace("curr file list:[%s]", buf); - taosMemoryFree(buf); + char* p = (char*)taosMemoryCalloc(1, 64 + sz * 64); + int32_t total = 0; + + for (int i = 0; i < sz; i++) { + char* name = taosArrayGetP(pArr, i); + total += sprintf(p + total, "%s,", name); + } + p[total - 1] = 0; + + *buf = p; +} +void dbChkpDebugInfo(SDbChkp* pDb) { + // stTrace("chkp get file list: curr"); + char* p[4] = {NULL}; + + hashTableToDebug(pDb->pSstTbl[pDb->idx], &p[0]); + stTrace("chkp previous file: [%s]", p[0]); + + hashTableToDebug(pDb->pSstTbl[1 - pDb->idx], &p[1]); + stTrace("chkp curr file: [%s]", p[1]); + + strArrayDebugInfo(pDb->pAdd, &p[2]); + stTrace("chkp newly addded file: [%s]", p[2]); + + strArrayDebugInfo(pDb->pDel, &p[3]); + stTrace("chkp newly deleted file: [%s]", p[3]); + + for (int i = 0; i < 4; i++) { + taosMemoryFree(p[i]); + } } int32_t dbChkpGetDelta(SDbChkp* p, int64_t chkpId, SArray* list) { taosThreadRwlockWrlock(&p->rwLock); @@ -3476,12 +3509,6 @@ int32_t dbChkpGetDelta(SDbChkp* p, int64_t chkpId, SArray* list) { } taosCloseDir(&pDir); - stTrace("chkp get file list: 1-1"); - hashTableToDebug(p->pSstTbl[1 - p->idx]); - - stTrace("chkp get file list: 1-2"); - hashTableToDebug(p->pSstTbl[p->idx]); - if (p->init == 0) { void* pIter = taosHashIterate(p->pSstTbl[1 - p->idx], NULL); while (pIter) { @@ -3517,6 +3544,9 @@ int32_t dbChkpGetDelta(SDbChkp* p, int64_t chkpId, SArray* list) { p->preCkptId = p->curChkpId; p->curChkpId = chkpId; } + + dbChkpDebugInfo(p); + p->idx = 1 - p->idx; taosThreadRwlockUnlock(&p->rwLock); From 82ad690eb4b9b57f9893fee98ed604f5713355f9 Mon Sep 17 00:00:00 2001 From: Bob Liu Date: Mon, 13 Nov 2023 20:20:45 +0800 Subject: [PATCH 115/195] rewrite nodesNodeName --- include/common/tmsg.h | 3 +- include/libs/nodes/nodes.h | 1 + source/libs/nodes/src/nodesCodeFuncs.c | 567 +++++++++---------------- source/libs/nodes/src/nodesUtilFuncs.c | 1 + 4 files changed, 200 insertions(+), 372 deletions(-) diff --git a/include/common/tmsg.h b/include/common/tmsg.h index 86d34502c6..4d4744999f 100644 --- a/include/common/tmsg.h +++ b/include/common/tmsg.h @@ -414,7 +414,8 @@ typedef enum ENodeType { QUERY_NODE_PHYSICAL_PLAN_STREAM_EVENT, QUERY_NODE_PHYSICAL_PLAN_HASH_JOIN, QUERY_NODE_PHYSICAL_PLAN_GROUP_CACHE, - QUERY_NODE_PHYSICAL_PLAN_DYN_QUERY_CTRL + QUERY_NODE_PHYSICAL_PLAN_DYN_QUERY_CTRL, + QUERY_NODE_END } ENodeType; typedef struct { diff --git a/include/libs/nodes/nodes.h b/include/libs/nodes/nodes.h index 7fbdbfb211..21d33249b8 100644 --- a/include/libs/nodes/nodes.h +++ b/include/libs/nodes/nodes.h @@ -101,6 +101,7 @@ typedef struct SNodeList { typedef struct SNodeAllocator SNodeAllocator; +void initNodeName(); int32_t nodesInitAllocatorSet(); void nodesDestroyAllocatorSet(); int32_t nodesCreateAllocator(int64_t queryId, int32_t chunkSize, int64_t* pAllocatorId); diff --git a/source/libs/nodes/src/nodesCodeFuncs.c b/source/libs/nodes/src/nodesCodeFuncs.c index f3087dd5d4..582ecf6753 100644 --- a/source/libs/nodes/src/nodesCodeFuncs.c +++ b/source/libs/nodes/src/nodesCodeFuncs.c @@ -27,378 +27,203 @@ static int32_t jsonToNode(const SJson* pJson, void* pObj); static int32_t jsonToNodeObject(const SJson* pJson, const char* pName, SNode** pNode); static int32_t makeNodeByJson(const SJson* pJson, SNode** pNode); + +typedef const char* taoscstr; +taoscstr nodeNames[QUERY_NODE_END] = {}; + +#define TAOS_DEFINE_NODE_NAME(type, msg) nodeNames[type] = msg; + +void initNodeName() { + TAOS_DEFINE_NODE_NAME(QUERY_NODE_COLUMN, "Column") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_VALUE, "Value") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_OPERATOR, "Operator") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_CONDITION, "LogicCondition") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_FUNCTION, "Function") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_REAL_TABLE, "RealTable") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_TEMP_TABLE, "TempTable") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_JOIN_TABLE, "JoinTable") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_GROUPING_SET, "GroupingSet") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_ORDER_BY_EXPR, "OrderByExpr") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_LIMIT, "Limit") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_STATE_WINDOW, "StateWindow") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SESSION_WINDOW, "SessionWinow") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_INTERVAL_WINDOW, "IntervalWindow") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_NODE_LIST, "NodeList") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_FILL, "Fill") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_RAW_EXPR, "RawExpr") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_TARGET, "Target") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_DATABLOCK_DESC, "DataBlockDesc") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SLOT_DESC, "SlotDesc") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_COLUMN_DEF, "ColumnDef") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_DOWNSTREAM_SOURCE, "DownstreamSource") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_DATABASE_OPTIONS, "DatabaseOptions") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_TABLE_OPTIONS, "TableOptions") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_INDEX_OPTIONS, "IndexOptions") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_EXPLAIN_OPTIONS, "ExplainOptions") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_STREAM_OPTIONS, "StreamOptions") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_LEFT_VALUE, "LeftValue") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_WHEN_THEN, "WhenThen") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_CASE_WHEN, "CaseWhen") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_EVENT_WINDOW, "EventWindow") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SET_OPERATOR, "SetOperator") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SELECT_STMT, "SelectStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_VNODE_MODIFY_STMT, "VnodeModifStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_CREATE_DATABASE_STMT, "CreateDatabaseStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_DROP_DATABASE_STMT, "DropDatabaseStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_ALTER_DATABASE_STMT, "AlterDatabaseStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_FLUSH_DATABASE_STMT, "FlushDatabaseStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_TRIM_DATABASE_STMT, "TrimDatabaseStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_CREATE_TABLE_STMT, "CreateTableStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_CREATE_SUBTABLE_CLAUSE, "CreateSubtableClause") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_CREATE_MULTI_TABLES_STMT, "CreateMultiTableStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_DROP_TABLE_CLAUSE, "DropTableClause") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_DROP_TABLE_STMT, "DropTableStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_DROP_SUPER_TABLE_STMT, "DropSuperTableStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_ALTER_TABLE_STMT, "AlterTableStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_ALTER_SUPER_TABLE_STMT, "AlterSuperTableStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_CREATE_USER_STMT, "CreateUserStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_ALTER_USER_STMT, "AlterUserStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_DROP_USER_STMT, "DropUserStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_USE_DATABASE_STMT, "UseDatabaseStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_CREATE_DNODE_STMT, "CreateDnodeStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_DROP_DNODE_STMT, "DropDnodeStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_ALTER_DNODE_STMT, "AlterDnodeStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_CREATE_INDEX_STMT, "CreateIndexStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_DROP_INDEX_STMT, "DropIndexStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_CREATE_QNODE_STMT, "CreateQnodeStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_DROP_QNODE_STMT, "DropQnodeStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_CREATE_SNODE_STMT, "CreateSnodeStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_DROP_SNODE_STMT, "DropSnodeStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_CREATE_MNODE_STMT, "CreateMnodeStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_DROP_MNODE_STMT, "DropMnodeStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_CREATE_TOPIC_STMT, "CreateTopicStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_DROP_TOPIC_STMT, "DropTopicStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_DROP_CGROUP_STMT, "DropConsumerGroupStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_ALTER_LOCAL_STMT, "AlterLocalStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_EXPLAIN_STMT, "ExplainStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_DESCRIBE_STMT, "DescribeStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_COMPACT_DATABASE_STMT, "CompactDatabaseStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_CREATE_STREAM_STMT, "CreateStreamStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_DROP_STREAM_STMT, "DropStreamStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PAUSE_STREAM_STMT, "PauseStreamStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_RESUME_STREAM_STMT, "ResumeStreamStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_BALANCE_VGROUP_STMT, "BalanceVgroupStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_BALANCE_VGROUP_LEADER_STMT, "BalanceVgroupLeaderStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_MERGE_VGROUP_STMT, "MergeVgroupStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_DB_ALIVE_STMT, "ShowDbAliveStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_CLUSTER_ALIVE_STMT, "ShowClusterAliveStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_REDISTRIBUTE_VGROUP_STMT, "RedistributeVgroupStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SPLIT_VGROUP_STMT, "SplitVgroupStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_GRANT_STMT, "GrantStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_REVOKE_STMT, "RevokeStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_DNODES_STMT, "ShowDnodesStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_MNODES_STMT, "ShowMnodesStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_MODULES_STMT, "ShowModulesStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_QNODES_STMT, "ShowQnodesStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_SNODES_STMT, "ShowSnodesStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_BNODES_STMT, "ShowBnodesStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_CLUSTER_STMT, "ShowClusterStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_DATABASES_STMT, "ShowDatabaseStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_FUNCTIONS_STMT, "ShowFunctionsStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_INDEXES_STMT, "ShowIndexesStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_STABLES_STMT, "ShowStablesStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_STREAMS_STMT, "ShowStreamsStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_TABLES_STMT, "ShowTablesStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_TAGS_STMT, "ShowTagsStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_USERS_STMT, "ShowUsersStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_LICENCES_STMT, "ShowGrantsStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_VGROUPS_STMT, "ShowVgroupsStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_TOPICS_STMT, "ShowTopicsStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_CONSUMERS_STMT, "ShowConsumersStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_QUERIES_STMT, "ShowQueriesStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_VARIABLES_STMT, "ShowVariablesStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_DNODE_VARIABLES_STMT, "ShowDnodeVariablesStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_TRANSACTIONS_STMT, "ShowTransactionsStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_SUBSCRIPTIONS_STMT, "ShowSubscriptionsStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_VNODES_STMT, "ShowVnodeStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_USER_PRIVILEGES_STMT, "ShowUserPrivilegesStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_CREATE_DATABASE_STMT, "ShowCreateDatabasesStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_CREATE_TABLE_STMT, "ShowCreateTablesStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_CREATE_STABLE_STMT, "ShowCreateStablesStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_CREATE_VIEW_STMT, "ShowCreateViewStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_TABLE_DISTRIBUTED_STMT, "ShowTableDistributedStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_LOCAL_VARIABLES_STMT, "ShowLocalVariablesStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_TABLE_TAGS_STMT, "ShowTableTagsStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_DELETE_STMT, "DeleteStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_INSERT_STMT, "InsertStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_RESTORE_DNODE_STMT, "RestoreDnodeStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_RESTORE_QNODE_STMT, "RestoreQnodeStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_RESTORE_MNODE_STMT, "RestoreMnodeStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_RESTORE_VNODE_STMT, "RestoreVnodeStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_CREATE_VIEW_STMT, "CreateViewStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_DROP_VIEW_STMT, "DropViewStmt") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN_SCAN, "LogicScan") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN_JOIN, "LogicJoin") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN_AGG, "LogicAgg") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN_PROJECT, "LogicProject") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN_VNODE_MODIFY, "LogicVnodeModify") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN_EXCHANGE, "LogicExchange") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN_MERGE, "LogicMerge") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN_WINDOW, "LogicWindow") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN_FILL, "LogicFill") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN_SORT, "LogicSort") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN_PARTITION, "LogicPartition") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN_INDEF_ROWS_FUNC, "LogicIndefRowsFunc") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN_INTERP_FUNC, "LogicInterpFunc") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN_GROUP_CACHE, "LogicGroupCache") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN_DYN_QUERY_CTRL, "LogicDynamicQueryCtrl") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_SUBPLAN, "LogicSubplan") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN, "LogicPlan") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_TAG_SCAN, "PhysiTagScan") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_TABLE_SCAN, "PhysiTableScan") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_TABLE_SEQ_SCAN, "PhysiTableSeqScan") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_TABLE_MERGE_SCAN, "PhysiTableMergeScan") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN, "PhysiSreamScan") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_SYSTABLE_SCAN, "PhysiSystemTableScan") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_BLOCK_DIST_SCAN, "PhysiBlockDistScan") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_LAST_ROW_SCAN, "PhysiLastRowScan") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_TABLE_COUNT_SCAN, "PhysiTableCountScan") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_MERGE_EVENT, "PhysiMergeEventWindow") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_STREAM_EVENT, "PhysiStreamEventWindow") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_PROJECT, "PhysiProject") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_MERGE_JOIN, "PhysiMergeJoin") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_HASH_JOIN, "PhysiHashJoin") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_HASH_AGG, "PhysiAgg") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_EXCHANGE, "PhysiExchange") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_MERGE, "PhysiMerge") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_SORT, "PhysiSort") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_GROUP_SORT, "PhysiGroupSort") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_HASH_INTERVAL, "PhysiHashInterval") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_MERGE_ALIGNED_INTERVAL, "PhysiMergeAlignedInterval") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL, "PhysiStreamInterval") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL, "PhysiStreamFinalInterval") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL, "PhysiStreamSemiInterval") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_FILL, "PhysiFill") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_STREAM_FILL, "PhysiFill") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_MERGE_SESSION, "PhysiSessionWindow") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_STREAM_SESSION, "PhysiStreamSessionWindow") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_SESSION, "PhysiStreamSemiSessionWindow") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_SESSION, "PhysiStreamFinalSessionWindow") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_MERGE_STATE, "PhysiStateWindow") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_STREAM_STATE, "PhysiStreamStateWindow") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_PARTITION, "PhysiPartition") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_STREAM_PARTITION, "PhysiStreamPartition") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_INDEF_ROWS_FUNC, "PhysiIndefRowsFunc") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_INTERP_FUNC, "PhysiInterpFunc") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_DISPATCH, "PhysiDispatch") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_INSERT, "PhysiInsert") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_QUERY_INSERT, "PhysiQueryInsert") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_DELETE, "PhysiDelete") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_GROUP_CACHE, "PhysiGroupCache") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_DYN_QUERY_CTRL, "PhysiDynamicQueryCtrl") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_SUBPLAN, "PhysiSubplan") + TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN, "PhysiPlan") +} + const char* nodesNodeName(ENodeType type) { - switch (type) { - case QUERY_NODE_COLUMN: - return "Column"; - case QUERY_NODE_VALUE: - return "Value"; - case QUERY_NODE_OPERATOR: - return "Operator"; - case QUERY_NODE_LOGIC_CONDITION: - return "LogicCondition"; - case QUERY_NODE_FUNCTION: - return "Function"; - case QUERY_NODE_REAL_TABLE: - return "RealTable"; - case QUERY_NODE_TEMP_TABLE: - return "TempTable"; - case QUERY_NODE_JOIN_TABLE: - return "JoinTable"; - case QUERY_NODE_GROUPING_SET: - return "GroupingSet"; - case QUERY_NODE_ORDER_BY_EXPR: - return "OrderByExpr"; - case QUERY_NODE_LIMIT: - return "Limit"; - case QUERY_NODE_STATE_WINDOW: - return "StateWindow"; - case QUERY_NODE_SESSION_WINDOW: - return "SessionWinow"; - case QUERY_NODE_INTERVAL_WINDOW: - return "IntervalWindow"; - case QUERY_NODE_NODE_LIST: - return "NodeList"; - case QUERY_NODE_FILL: - return "Fill"; - case QUERY_NODE_RAW_EXPR: - return "RawExpr"; - case QUERY_NODE_TARGET: - return "Target"; - case QUERY_NODE_DATABLOCK_DESC: - return "DataBlockDesc"; - case QUERY_NODE_SLOT_DESC: - return "SlotDesc"; - case QUERY_NODE_COLUMN_DEF: - return "ColumnDef"; - case QUERY_NODE_DOWNSTREAM_SOURCE: - return "DownstreamSource"; - case QUERY_NODE_DATABASE_OPTIONS: - return "DatabaseOptions"; - case QUERY_NODE_TABLE_OPTIONS: - return "TableOptions"; - case QUERY_NODE_INDEX_OPTIONS: - return "IndexOptions"; - case QUERY_NODE_EXPLAIN_OPTIONS: - return "ExplainOptions"; - case QUERY_NODE_STREAM_OPTIONS: - return "StreamOptions"; - case QUERY_NODE_LEFT_VALUE: - return "LeftValue"; - case QUERY_NODE_WHEN_THEN: - return "WhenThen"; - case QUERY_NODE_CASE_WHEN: - return "CaseWhen"; - case QUERY_NODE_EVENT_WINDOW: - return "EventWindow"; - case QUERY_NODE_SET_OPERATOR: - return "SetOperator"; - case QUERY_NODE_SELECT_STMT: - return "SelectStmt"; - case QUERY_NODE_VNODE_MODIFY_STMT: - return "VnodeModifStmt"; - case QUERY_NODE_CREATE_DATABASE_STMT: - return "CreateDatabaseStmt"; - case QUERY_NODE_DROP_DATABASE_STMT: - return "DropDatabaseStmt"; - case QUERY_NODE_ALTER_DATABASE_STMT: - return "AlterDatabaseStmt"; - case QUERY_NODE_FLUSH_DATABASE_STMT: - return "FlushDatabaseStmt"; - case QUERY_NODE_TRIM_DATABASE_STMT: - return "TrimDatabaseStmt"; - case QUERY_NODE_CREATE_TABLE_STMT: - return "CreateTableStmt"; - case QUERY_NODE_CREATE_SUBTABLE_CLAUSE: - return "CreateSubtableClause"; - case QUERY_NODE_CREATE_MULTI_TABLES_STMT: - return "CreateMultiTableStmt"; - case QUERY_NODE_DROP_TABLE_CLAUSE: - return "DropTableClause"; - case QUERY_NODE_DROP_TABLE_STMT: - return "DropTableStmt"; - case QUERY_NODE_DROP_SUPER_TABLE_STMT: - return "DropSuperTableStmt"; - case QUERY_NODE_ALTER_TABLE_STMT: - return "AlterTableStmt"; - case QUERY_NODE_ALTER_SUPER_TABLE_STMT: - return "AlterSuperTableStmt"; - case QUERY_NODE_CREATE_USER_STMT: - return "CreateUserStmt"; - case QUERY_NODE_ALTER_USER_STMT: - return "AlterUserStmt"; - case QUERY_NODE_DROP_USER_STMT: - return "DropUserStmt"; - case QUERY_NODE_USE_DATABASE_STMT: - return "UseDatabaseStmt"; - case QUERY_NODE_CREATE_DNODE_STMT: - return "CreateDnodeStmt"; - case QUERY_NODE_DROP_DNODE_STMT: - return "DropDnodeStmt"; - case QUERY_NODE_ALTER_DNODE_STMT: - return "AlterDnodeStmt"; - case QUERY_NODE_CREATE_INDEX_STMT: - return "CreateIndexStmt"; - case QUERY_NODE_DROP_INDEX_STMT: - return "DropIndexStmt"; - case QUERY_NODE_CREATE_QNODE_STMT: - return "CreateQnodeStmt"; - case QUERY_NODE_DROP_QNODE_STMT: - return "DropQnodeStmt"; - case QUERY_NODE_CREATE_SNODE_STMT: - return "CreateSnodeStmt"; - case QUERY_NODE_DROP_SNODE_STMT: - return "DropSnodeStmt"; - case QUERY_NODE_CREATE_MNODE_STMT: - return "CreateMnodeStmt"; - case QUERY_NODE_DROP_MNODE_STMT: - return "DropMnodeStmt"; - case QUERY_NODE_CREATE_TOPIC_STMT: - return "CreateTopicStmt"; - case QUERY_NODE_DROP_TOPIC_STMT: - return "DropTopicStmt"; - case QUERY_NODE_DROP_CGROUP_STMT: - return "DropConsumerGroupStmt"; - case QUERY_NODE_ALTER_LOCAL_STMT: - return "AlterLocalStmt"; - case QUERY_NODE_EXPLAIN_STMT: - return "ExplainStmt"; - case QUERY_NODE_DESCRIBE_STMT: - return "DescribeStmt"; - case QUERY_NODE_COMPACT_DATABASE_STMT: - return "CompactDatabaseStmt"; - case QUERY_NODE_CREATE_STREAM_STMT: - return "CreateStreamStmt"; - case QUERY_NODE_DROP_STREAM_STMT: - return "DropStreamStmt"; - case QUERY_NODE_PAUSE_STREAM_STMT: - return "PauseStreamStmt"; - case QUERY_NODE_RESUME_STREAM_STMT: - return "ResumeStreamStmt"; - case QUERY_NODE_BALANCE_VGROUP_STMT: - return "BalanceVgroupStmt"; - case QUERY_NODE_BALANCE_VGROUP_LEADER_STMT: - return "BalanceVgroupLeaderStmt"; - case QUERY_NODE_MERGE_VGROUP_STMT: - return "MergeVgroupStmt"; - case QUERY_NODE_SHOW_DB_ALIVE_STMT: - return "ShowDbAliveStmt"; - case QUERY_NODE_SHOW_CLUSTER_ALIVE_STMT: - return "ShowClusterAliveStmt"; - case QUERY_NODE_REDISTRIBUTE_VGROUP_STMT: - return "RedistributeVgroupStmt"; - case QUERY_NODE_SPLIT_VGROUP_STMT: - return "SplitVgroupStmt"; - case QUERY_NODE_GRANT_STMT: - return "GrantStmt"; - case QUERY_NODE_REVOKE_STMT: - return "RevokeStmt"; - case QUERY_NODE_SHOW_DNODES_STMT: - return "ShowDnodesStmt"; - case QUERY_NODE_SHOW_MNODES_STMT: - return "ShowMnodesStmt"; - case QUERY_NODE_SHOW_MODULES_STMT: - return "ShowModulesStmt"; - case QUERY_NODE_SHOW_QNODES_STMT: - return "ShowQnodesStmt"; - case QUERY_NODE_SHOW_SNODES_STMT: - return "ShowSnodesStmt"; - case QUERY_NODE_SHOW_BNODES_STMT: - return "ShowBnodesStmt"; - case QUERY_NODE_SHOW_CLUSTER_STMT: - return "ShowClusterStmt"; - case QUERY_NODE_SHOW_DATABASES_STMT: - return "ShowDatabaseStmt"; - case QUERY_NODE_SHOW_FUNCTIONS_STMT: - return "ShowFunctionsStmt"; - case QUERY_NODE_SHOW_INDEXES_STMT: - return "ShowIndexesStmt"; - case QUERY_NODE_SHOW_STABLES_STMT: - return "ShowStablesStmt"; - case QUERY_NODE_SHOW_STREAMS_STMT: - return "ShowStreamsStmt"; - case QUERY_NODE_SHOW_TABLES_STMT: - return "ShowTablesStmt"; - case QUERY_NODE_SHOW_TAGS_STMT: - return "ShowTagsStmt"; - case QUERY_NODE_SHOW_USERS_STMT: - return "ShowUsersStmt"; - case QUERY_NODE_SHOW_LICENCES_STMT: - return "ShowGrantsStmt"; - case QUERY_NODE_SHOW_VGROUPS_STMT: - return "ShowVgroupsStmt"; - case QUERY_NODE_SHOW_TOPICS_STMT: - return "ShowTopicsStmt"; - case QUERY_NODE_SHOW_CONSUMERS_STMT: - return "ShowConsumersStmt"; - case QUERY_NODE_SHOW_QUERIES_STMT: - return "ShowQueriesStmt"; - case QUERY_NODE_SHOW_VARIABLES_STMT: - return "ShowVariablesStmt"; - case QUERY_NODE_SHOW_DNODE_VARIABLES_STMT: - return "ShowDnodeVariablesStmt"; - case QUERY_NODE_SHOW_TRANSACTIONS_STMT: - return "ShowTransactionsStmt"; - case QUERY_NODE_SHOW_SUBSCRIPTIONS_STMT: - return "ShowSubscriptionsStmt"; - case QUERY_NODE_SHOW_VNODES_STMT: - return "ShowVnodeStmt"; - case QUERY_NODE_SHOW_USER_PRIVILEGES_STMT: - return "ShowUserPrivilegesStmt"; - case QUERY_NODE_SHOW_CREATE_DATABASE_STMT: - return "ShowCreateDatabasesStmt"; - case QUERY_NODE_SHOW_CREATE_TABLE_STMT: - return "ShowCreateTablesStmt"; - case QUERY_NODE_SHOW_CREATE_STABLE_STMT: - return "ShowCreateStablesStmt"; - case QUERY_NODE_SHOW_CREATE_VIEW_STMT: - return "ShowCreateViewStmt"; - case QUERY_NODE_SHOW_TABLE_DISTRIBUTED_STMT: - return "ShowTableDistributedStmt"; - case QUERY_NODE_SHOW_LOCAL_VARIABLES_STMT: - return "ShowLocalVariablesStmt"; - case QUERY_NODE_SHOW_TABLE_TAGS_STMT: - return "ShowTableTagsStmt"; - case QUERY_NODE_DELETE_STMT: - return "DeleteStmt"; - case QUERY_NODE_INSERT_STMT: - return "InsertStmt"; - case QUERY_NODE_RESTORE_DNODE_STMT: - return "RestoreDnodeStmt"; - case QUERY_NODE_RESTORE_QNODE_STMT: - return "RestoreQnodeStmt"; - case QUERY_NODE_RESTORE_MNODE_STMT: - return "RestoreMnodeStmt"; - case QUERY_NODE_RESTORE_VNODE_STMT: - return "RestoreVnodeStmt"; - case QUERY_NODE_CREATE_VIEW_STMT: - return "CreateViewStmt"; - case QUERY_NODE_DROP_VIEW_STMT: - return "DropViewStmt"; - case QUERY_NODE_LOGIC_PLAN_SCAN: - return "LogicScan"; - case QUERY_NODE_LOGIC_PLAN_JOIN: - return "LogicJoin"; - case QUERY_NODE_LOGIC_PLAN_AGG: - return "LogicAgg"; - case QUERY_NODE_LOGIC_PLAN_PROJECT: - return "LogicProject"; - case QUERY_NODE_LOGIC_PLAN_VNODE_MODIFY: - return "LogicVnodeModify"; - case QUERY_NODE_LOGIC_PLAN_EXCHANGE: - return "LogicExchange"; - case QUERY_NODE_LOGIC_PLAN_MERGE: - return "LogicMerge"; - case QUERY_NODE_LOGIC_PLAN_WINDOW: - return "LogicWindow"; - case QUERY_NODE_LOGIC_PLAN_FILL: - return "LogicFill"; - case QUERY_NODE_LOGIC_PLAN_SORT: - return "LogicSort"; - case QUERY_NODE_LOGIC_PLAN_PARTITION: - return "LogicPartition"; - case QUERY_NODE_LOGIC_PLAN_INDEF_ROWS_FUNC: - return "LogicIndefRowsFunc"; - case QUERY_NODE_LOGIC_PLAN_INTERP_FUNC: - return "LogicInterpFunc"; - case QUERY_NODE_LOGIC_PLAN_GROUP_CACHE: - return "LogicGroupCache"; - case QUERY_NODE_LOGIC_PLAN_DYN_QUERY_CTRL: - return "LogicDynamicQueryCtrl"; - case QUERY_NODE_LOGIC_SUBPLAN: - return "LogicSubplan"; - case QUERY_NODE_LOGIC_PLAN: - return "LogicPlan"; - case QUERY_NODE_PHYSICAL_PLAN_TAG_SCAN: - return "PhysiTagScan"; - case QUERY_NODE_PHYSICAL_PLAN_TABLE_SCAN: - return "PhysiTableScan"; - case QUERY_NODE_PHYSICAL_PLAN_TABLE_SEQ_SCAN: - return "PhysiTableSeqScan"; - case QUERY_NODE_PHYSICAL_PLAN_TABLE_MERGE_SCAN: - return "PhysiTableMergeScan"; - case QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN: - return "PhysiSreamScan"; - case QUERY_NODE_PHYSICAL_PLAN_SYSTABLE_SCAN: - return "PhysiSystemTableScan"; - case QUERY_NODE_PHYSICAL_PLAN_BLOCK_DIST_SCAN: - return "PhysiBlockDistScan"; - case QUERY_NODE_PHYSICAL_PLAN_LAST_ROW_SCAN: - return "PhysiLastRowScan"; - case QUERY_NODE_PHYSICAL_PLAN_TABLE_COUNT_SCAN: - return "PhysiTableCountScan"; - case QUERY_NODE_PHYSICAL_PLAN_MERGE_EVENT: - return "PhysiMergeEventWindow"; - case QUERY_NODE_PHYSICAL_PLAN_STREAM_EVENT: - return "PhysiStreamEventWindow"; - case QUERY_NODE_PHYSICAL_PLAN_PROJECT: - return "PhysiProject"; - case QUERY_NODE_PHYSICAL_PLAN_MERGE_JOIN: - return "PhysiMergeJoin"; - case QUERY_NODE_PHYSICAL_PLAN_HASH_JOIN: - return "PhysiHashJoin"; - case QUERY_NODE_PHYSICAL_PLAN_HASH_AGG: - return "PhysiAgg"; - case QUERY_NODE_PHYSICAL_PLAN_EXCHANGE: - return "PhysiExchange"; - case QUERY_NODE_PHYSICAL_PLAN_MERGE: - return "PhysiMerge"; - case QUERY_NODE_PHYSICAL_PLAN_SORT: - return "PhysiSort"; - case QUERY_NODE_PHYSICAL_PLAN_GROUP_SORT: - return "PhysiGroupSort"; - case QUERY_NODE_PHYSICAL_PLAN_HASH_INTERVAL: - return "PhysiHashInterval"; - case QUERY_NODE_PHYSICAL_PLAN_MERGE_ALIGNED_INTERVAL: - return "PhysiMergeAlignedInterval"; - case QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL: - return "PhysiStreamInterval"; - case QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL: - return "PhysiStreamFinalInterval"; - case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL: - return "PhysiStreamSemiInterval"; - case QUERY_NODE_PHYSICAL_PLAN_FILL: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_FILL: - return "PhysiFill"; - case QUERY_NODE_PHYSICAL_PLAN_MERGE_SESSION: - return "PhysiSessionWindow"; - case QUERY_NODE_PHYSICAL_PLAN_STREAM_SESSION: - return "PhysiStreamSessionWindow"; - case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_SESSION: - return "PhysiStreamSemiSessionWindow"; - case QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_SESSION: - return "PhysiStreamFinalSessionWindow"; - case QUERY_NODE_PHYSICAL_PLAN_MERGE_STATE: - return "PhysiStateWindow"; - case QUERY_NODE_PHYSICAL_PLAN_STREAM_STATE: - return "PhysiStreamStateWindow"; - case QUERY_NODE_PHYSICAL_PLAN_PARTITION: - return "PhysiPartition"; - case QUERY_NODE_PHYSICAL_PLAN_STREAM_PARTITION: - return "PhysiStreamPartition"; - case QUERY_NODE_PHYSICAL_PLAN_INDEF_ROWS_FUNC: - return "PhysiIndefRowsFunc"; - case QUERY_NODE_PHYSICAL_PLAN_INTERP_FUNC: - return "PhysiInterpFunc"; - case QUERY_NODE_PHYSICAL_PLAN_DISPATCH: - return "PhysiDispatch"; - case QUERY_NODE_PHYSICAL_PLAN_INSERT: - return "PhysiInsert"; - case QUERY_NODE_PHYSICAL_PLAN_QUERY_INSERT: - return "PhysiQueryInsert"; - case QUERY_NODE_PHYSICAL_PLAN_DELETE: - return "PhysiDelete"; - case QUERY_NODE_PHYSICAL_PLAN_GROUP_CACHE: - return "PhysiGroupCache"; - case QUERY_NODE_PHYSICAL_PLAN_DYN_QUERY_CTRL: - return "PhysiDynamicQueryCtrl"; - case QUERY_NODE_PHYSICAL_SUBPLAN: - return "PhysiSubplan"; - case QUERY_NODE_PHYSICAL_PLAN: - return "PhysiPlan"; - default: - break; - } + if (nodeNames[type]) + return nodeNames[type]; + nodesWarn("nodesNodeName unknown node = %d", type); return "UnknownNode"; } diff --git a/source/libs/nodes/src/nodesUtilFuncs.c b/source/libs/nodes/src/nodesUtilFuncs.c index d167d81c82..c8de8bdfac 100644 --- a/source/libs/nodes/src/nodesUtilFuncs.c +++ b/source/libs/nodes/src/nodesUtilFuncs.c @@ -131,6 +131,7 @@ static void destroyNodeAllocator(void* p) { } int32_t nodesInitAllocatorSet() { + initNodeName(); if (g_allocatorReqRefPool >= 0) { nodesWarn("nodes already initialized"); return TSDB_CODE_SUCCESS; From 4476e46a5fc3663e0562b528e2602ffaa9a50973 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Tue, 14 Nov 2023 17:43:13 +0800 Subject: [PATCH 116/195] rm dup data --- include/common/cos.h | 1 + source/libs/stream/inc/streamBackendRocksdb.h | 4 +- source/libs/stream/inc/streamInt.h | 1 + source/libs/stream/src/streamBackendRocksdb.c | 84 ++++++++++--------- source/libs/stream/src/streamCheckpoint.c | 83 +++++++++++++++++- 5 files changed, 129 insertions(+), 44 deletions(-) diff --git a/include/common/cos.h b/include/common/cos.h index 21b645f604..c6b159c1da 100644 --- a/include/common/cos.h +++ b/include/common/cos.h @@ -43,6 +43,7 @@ int32_t s3GetObjectBlock(const char *object_name, int64_t offset, int64_t size, int32_t s3GetObjectsByPrefix(const char *prefix, const char *path); void s3EvictCache(const char *path, long object_size); long s3Size(const char *object_name); +int32_t s3GetObjectToFile(const char *object_name, char *fileName); #ifdef __cplusplus } diff --git a/source/libs/stream/inc/streamBackendRocksdb.h b/source/libs/stream/inc/streamBackendRocksdb.h index 92ec1899db..47404f311f 100644 --- a/source/libs/stream/inc/streamBackendRocksdb.h +++ b/source/libs/stream/inc/streamBackendRocksdb.h @@ -77,7 +77,7 @@ typedef struct { SArray* chkpInUse; int32_t chkpCap; TdThreadRwlock chkpDirLock; - int64_t dataWritten; + int64_t dataWritten; } STaskDbWrapper; @@ -255,5 +255,5 @@ int32_t bkdMgtGetDelta(SBkdMgt* bm, char* taskId, int64_t chkpId, SArray* list, int32_t bkdMgtDumpTo(SBkdMgt* bm, char* taskId, char* dname); void bkdMgtDestroy(SBkdMgt* bm); -int32_t taskDbGenChkpUploadData(void* arg, void* bkdMgt, int64_t chkpId, int8_t type, char** pathkj); +int32_t taskDbGenChkpUploadData(void* arg, void* bkdMgt, int64_t chkpId, int8_t type, char** path, SArray* list); #endif \ No newline at end of file diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index bbe7bcf65c..c299d0cfe1 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -163,6 +163,7 @@ int uploadCheckpoint(char* id, char* path); int downloadCheckpoint(char* id, char* path); int deleteCheckpoint(char* id); int deleteCheckpointFile(char* id, char* name); +int downloadCheckpointByName(char* id, char* fname, char* dstName); int32_t onNormalTaskReady(SStreamTask* pTask); int32_t onScanhistoryTaskReady(SStreamTask* pTask); diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 5c59621ca9..b156bce5dd 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1719,7 +1719,8 @@ int32_t taskDbGenChkpUploadData__rsync(STaskDbWrapper* pDb, int64_t chkpId, char return code; } -int32_t taskDbGenChkpUploadData__s3(STaskDbWrapper* pDb, void* bkdChkpMgt, int64_t chkpId, char** path) { +int32_t taskDbGenChkpUploadData__s3(STaskDbWrapper* pDb, void* bkdChkpMgt, int64_t chkpId, char** path, SArray* list) { + int32_t code = 0; SBkdMgt* p = (SBkdMgt*)bkdChkpMgt; char* temp = taosMemoryCalloc(1, strlen(pDb->path)); @@ -1731,20 +1732,20 @@ int32_t taskDbGenChkpUploadData__s3(STaskDbWrapper* pDb, void* bkdChkpMgt, int64 } else { taosMkDir(temp); } - bkdMgtGetDelta(p, pDb->idstr, chkpId, NULL, temp); + code = bkdMgtGetDelta(p, pDb->idstr, chkpId, list, temp); *path = temp; - return 0; + return code; } -int32_t taskDbGenChkpUploadData(void* arg, void* mgt, int64_t chkpId, int8_t type, char** path) { +int32_t taskDbGenChkpUploadData(void* arg, void* mgt, int64_t chkpId, int8_t type, char** path, SArray* list) { STaskDbWrapper* pDb = arg; UPLOAD_TYPE utype = type; if (utype == UPLOAD_RSYNC) { return taskDbGenChkpUploadData__rsync(pDb, chkpId, path); } else if (utype == UPLOAD_S3) { - return taskDbGenChkpUploadData__s3(pDb, mgt, chkpId, path); + return taskDbGenChkpUploadData__s3(pDb, mgt, chkpId, path, list); } return -1; } @@ -3559,7 +3560,7 @@ SDbChkp* dbChkpCreate(char* path, int64_t initChkpId) { p->curChkpId = initChkpId; p->preCkptId = -1; p->pSST = taosArrayInit(64, sizeof(void*)); - p->path = taosStrdup(path); + p->path = path; p->len = strlen(path) + 128; p->buf = taosMemoryCalloc(1, p->len); @@ -3597,9 +3598,9 @@ int32_t dbChkpInit(SDbChkp* p) { if (p == NULL) return 0; return 0; } -int32_t dbChkpDumpTo(SDbChkp* p, char* dname) { +int32_t dbChkpDumpTo(SDbChkp* p, char* dname, SArray* list) { taosThreadRwlockRdlock(&p->rwLock); - int32_t code = 0; + int32_t code = -1; int32_t len = p->len + 128; char* srcBuf = taosMemoryCalloc(1, len); @@ -3613,26 +3614,9 @@ int32_t dbChkpDumpTo(SDbChkp* p, char* dname) { if (!taosDirExist(srcDir)) { stError("failed to dump srcDir %s, reason: not exist such dir", srcDir); - code = -1; goto _ERROR; } - // code = taosMkDir(dstDir); - // if (code != 0) { - // terrno = TAOS_SYSTEM_ERROR(errno); - // stError("failed to mkdir srcDir %s, reason: %s", dstDir, terrstr()); - // goto _ERROR; - // } - - // clear current file - memset(dstBuf, 0, len); - sprintf(dstBuf, "%s%s%s", dstDir, TD_DIRSEP, p->pCurrent); - taosRemoveFile(dstBuf); - - memset(dstBuf, 0, len); - sprintf(dstBuf, "%s%s%s", dstDir, TD_DIRSEP, p->pManifest); - taosRemoveFile(dstBuf); - // add file to $name dir for (int i = 0; i < taosArrayGetSize(p->pAdd); i++) { memset(srcBuf, 0, len); @@ -3644,39 +3628,59 @@ int32_t dbChkpDumpTo(SDbChkp* p, char* dname) { if (taosCopyFile(srcBuf, dstBuf) < 0) { stError("failed to copy file from %s to %s", srcBuf, dstBuf); + goto _ERROR; } } // del file in $name for (int i = 0; i < taosArrayGetSize(p->pDel); i++) { - memset(dstBuf, 0, len); - memset(srcBuf, 0, len); - char* filename = taosArrayGetP(p->pDel, i); - sprintf(dstBuf, "%s%s%s", dstDir, TD_DIRSEP, filename); - taosRemoveFile(dstBuf); + char* p = taosStrdup(filename); + taosArrayPush(list, &p); } // copy current file to dst dir memset(srcBuf, 0, len); memset(dstBuf, 0, len); sprintf(srcBuf, "%s%s%s", srcDir, TD_DIRSEP, p->pCurrent); - sprintf(dstBuf, "%s%s%s", dstDir, TD_DIRSEP, p->pCurrent); + sprintf(dstBuf, "%s%s%s_%" PRId64 "", dstDir, TD_DIRSEP, p->pCurrent, p->curChkpId); if (taosCopyFile(srcBuf, dstBuf) < 0) { stError("failed to copy file from %s to %s", srcBuf, dstBuf); + goto _ERROR; } // copy manifest file to dst dir memset(srcBuf, 0, len); memset(dstBuf, 0, len); sprintf(srcBuf, "%s%s%s", srcDir, TD_DIRSEP, p->pManifest); - sprintf(dstBuf, "%s%s%s", dstDir, TD_DIRSEP, p->pManifest); + sprintf(dstBuf, "%s%s%s_%" PRId64 "", dstDir, TD_DIRSEP, p->pManifest, p->curChkpId); if (taosCopyFile(srcBuf, dstBuf) < 0) { stError("failed to copy file from %s to %s", srcBuf, dstBuf); + goto _ERROR; } + static char* chkpMeta = "META"; + memset(dstBuf, 0, len); + sprintf(dstDir, "%s%s%s", dstDir, TD_DIRSEP, chkpMeta); + + TdFilePtr pFile = taosOpenFile(dstDir, TD_FILE_CREATE | TD_FILE_WRITE | TD_FILE_TRUNC); + if (pFile == NULL) { + stError("chkp failed to create meta file: %s", dstDir); + goto _ERROR; + } + char content[128] = {0}; + snprintf(content, sizeof(content), "%s_%" PRId64 "\n%s_%" PRId64 "", p->pCurrent, p->curChkpId, p->pManifest, + p->curChkpId); + if (taosWriteFile(pFile, content, strlen(content)) <= 0) { + stError("chkp failed to write meta file: %s", dstDir); + taosCloseFile(&pFile); + goto _ERROR; + } + taosCloseFile(&pFile); + // clear delta data buf taosArrayClearP(p->pAdd, taosMemoryFree); taosArrayClearP(p->pDel, taosMemoryFree); + code = 0; _ERROR: taosThreadRwlockUnlock(&p->rwLock); @@ -3718,22 +3722,22 @@ int32_t bkdMgtGetDelta(SBkdMgt* bm, char* taskId, int64_t chkpId, SArray* list, SDbChkp* pChkp = ppChkp != NULL ? *ppChkp : NULL; if (pChkp == NULL) { - char* taskPath = taosMemoryCalloc(1, strlen(bm->path) + 64); - sprintf(taskPath, "%s%s%s", bm->path, TD_DIRSEP, taskId); + char* path = taosMemoryCalloc(1, strlen(bm->path) + 64); + sprintf(path, "%s%s%s", bm->path, TD_DIRSEP, taskId); - SDbChkp* p = dbChkpCreate(taskPath, chkpId); + SDbChkp* p = dbChkpCreate(path, chkpId); taosHashPut(bm->pDbChkpTbl, taskId, strlen(taskId), &p, sizeof(void*)); - taosMemoryFree(taskPath); pChkp = p; - code = dbChkpDumpTo(pChkp, dname); + code = dbChkpDumpTo(pChkp, dname, list); taosThreadRwlockUnlock(&bm->rwLock); return code; } - code = dbChkpGetDelta(pChkp, chkpId, list); - code = dbChkpDumpTo(pChkp, dname); + code = dbChkpGetDelta(pChkp, chkpId, NULL); + code = dbChkpDumpTo(pChkp, dname, list); + taosThreadRwlockUnlock(&bm->rwLock); return code; } @@ -3763,7 +3767,7 @@ int32_t bkdMgtDumpTo(SBkdMgt* bm, char* taskId, char* dname) { taosThreadRwlockRdlock(&bm->rwLock); SDbChkp* p = taosHashGet(bm->pDbChkpTbl, taskId, strlen(taskId)); - code = dbChkpDumpTo(p, dname); + code = dbChkpDumpTo(p, dname, NULL); taosThreadRwlockUnlock(&bm->rwLock); return code; diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index f7ac9e61bc..b61256371b 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -338,18 +338,72 @@ void streamTaskSetFailedId(SStreamTask* pTask) { pTask->chkInfo.checkpointId = pTask->chkInfo.checkpointingId; } +int32_t getChkpMeta(char* id, char* path, SArray* list) { + char* file = taosMemoryCalloc(1, strlen(path) + 32); + sprintf(file, "%s%s%s", path, TD_DIRSEP, "META_TMP"); + int32_t code = downloadCheckpointByName(id, "META", file); + if (code != 0) { + stDebug("chkp failed to download meta file:%s", file); + taosMemoryFree(file); + return code; + } + TdFilePtr pFile = taosOpenFile(file, TD_FILE_READ); + char buf[128] = {0}; + if (taosReadFile(pFile, buf, sizeof(buf)) <= 0) { + stError("chkp failed to read meta file:%s", file); + code = -1; + } else { + int32_t len = strlen(buf); + for (int i = 0; i < len; i++) { + if (buf[i] == '\n') { + char* item = taosMemoryCalloc(1, i + 1); + memcpy(item, buf, i); + taosArrayPush(list, &item); + + item = taosMemoryCalloc(1, len - i); + memcpy(item, buf + i + 1, len - i - 1); + taosArrayPush(list, &item); + } + } + } + taosCloseFile(&pFile); + taosRemoveFile(file); + taosMemoryFree(file); + return code; +} int32_t doUploadChkp(void* param) { SAsyncUploadArg* arg = param; char* path = NULL; int32_t code = 0; + SArray* list = taosArrayInit(4, sizeof(void*)); if ((code = taskDbGenChkpUploadData(arg->pTask->pBackend, arg->pTask->pMeta->bkdChkptMgt, arg->chkpId, - (int8_t)(arg->type), &path)) != 0) { + (int8_t)(arg->type), &path, list)) != 0) { stError("s-task:%s failed to gen upload checkpoint:%" PRId64 "", arg->pTask->id.idStr, arg->chkpId); } - if (code == 0 && uploadCheckpoint(arg->taskId, path) != 0) { + + code = getChkpMeta(arg->taskId, path, list); + if (code != 0) { + code = 0; + } + + if (code == 0 && (code = uploadCheckpoint(arg->taskId, path)) != 0) { stError("s-task:%s failed to upload checkpoint:%" PRId64, arg->pTask->id.idStr, arg->chkpId); } + + if (code == 0) { + for (int i = 0; i < taosArrayGetSize(list); i++) { + char* p = taosArrayGetP(list, i); + code = deleteCheckpointFile(arg->taskId, p); + stDebug("try to del file: %s", p); + if (code != 0) { + break; + } + } + } + + taosArrayDestroyP(list, taosMemoryFree); + taosRemoveDir(path); taosMemoryFree(path); taosMemoryFree(arg->taskId); @@ -464,6 +518,17 @@ static int uploadCheckpointToS3(char* id, char* path) { return 0; } +static int downloadCheckpointByNameS3(char* id, char* fname, char* dstName) { + int code = 0; + char* buf = taosMemoryCalloc(1, strlen(id) + strlen(dstName) + 4); + sprintf(buf, "%s/%s", id, fname); + if (s3GetObjectToFile(buf, dstName) != 0) { + code = -1; + } + taosMemoryFree(buf); + return code; +} + UPLOAD_TYPE getUploadType() { if (strlen(tsSnodeAddress) != 0) { return UPLOAD_RSYNC; @@ -487,6 +552,20 @@ int uploadCheckpoint(char* id, char* path) { return 0; } +// fileName: CURRENT +int downloadCheckpointByName(char* id, char* fname, char* dstName) { + if (id == NULL || fname == NULL || strlen(id) == 0 || strlen(fname) == 0 || strlen(fname) >= PATH_MAX) { + stError("uploadCheckpointByName parameters invalid"); + return -1; + } + if (strlen(tsSnodeAddress) != 0) { + return 0; + } else if (tsS3StreamEnabled) { + return downloadCheckpointByNameS3(id, fname, dstName); + } + return 0; +} + int downloadCheckpoint(char* id, char* path) { if (id == NULL || path == NULL || strlen(id) == 0 || strlen(path) == 0 || strlen(path) >= PATH_MAX) { stError("downloadCheckpoint parameters invalid"); From aa2c09ef5b88c346fb0c387611181acc321ca91f Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Tue, 14 Nov 2023 18:09:00 +0800 Subject: [PATCH 117/195] rm dup data --- source/libs/stream/src/streamCheckpoint.c | 1 + 1 file changed, 1 insertion(+) diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index b61256371b..74c391f386 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -377,6 +377,7 @@ int32_t doUploadChkp(void* param) { int32_t code = 0; SArray* list = taosArrayInit(4, sizeof(void*)); + if ((code = taskDbGenChkpUploadData(arg->pTask->pBackend, arg->pTask->pMeta->bkdChkptMgt, arg->chkpId, (int8_t)(arg->type), &path, list)) != 0) { stError("s-task:%s failed to gen upload checkpoint:%" PRId64 "", arg->pTask->id.idStr, arg->chkpId); From 20e1bd9cff0f66de278ede485b39a92298998859 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Tue, 14 Nov 2023 20:46:56 +0800 Subject: [PATCH 118/195] fix invalid read/write --- source/libs/stream/src/streamBackendRocksdb.c | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index b156bce5dd..376fb7be85 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1723,7 +1723,7 @@ int32_t taskDbGenChkpUploadData__s3(STaskDbWrapper* pDb, void* bkdChkpMgt, int64 int32_t code = 0; SBkdMgt* p = (SBkdMgt*)bkdChkpMgt; - char* temp = taosMemoryCalloc(1, strlen(pDb->path)); + char* temp = taosMemoryCalloc(1, strlen(pDb->path) + 32); sprintf(temp, "%s%s%s%" PRId64 "", pDb->path, TD_DIRSEP, "tmp", chkpId); if (taosDirExist(temp)) { @@ -3413,7 +3413,7 @@ int32_t compareHashTable(SHashObj* p1, SHashObj* p2, SArray* add, SArray* del) { void hashTableToDebug(SHashObj* pTbl, char** buf) { size_t sz = taosHashGetSize(pTbl); int32_t total = 0; - char* p = taosMemoryCalloc(1, sz * 16); + char* p = taosMemoryCalloc(1, sz * 16 + 4); void* pIter = taosHashIterate(pTbl, NULL); while (pIter) { size_t len = 0; @@ -3425,7 +3425,9 @@ void hashTableToDebug(SHashObj* pTbl, char** buf) { pIter = taosHashIterate(pTbl, pIter); taosMemoryFree(tname); } - p[total - 1] = 0; + if (total > 0) { + p[total - 1] = 0; + } *buf = p; } void strArrayDebugInfo(SArray* pArr, char** buf) { From b74105727d3e1caf76aabb72fba78801ffea7f16 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Tue, 14 Nov 2023 20:58:35 +0800 Subject: [PATCH 119/195] fix mem leak --- source/libs/stream/src/streamBackendRocksdb.c | 2 ++ 1 file changed, 2 insertions(+) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 376fb7be85..4e71eff431 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -899,6 +899,7 @@ _EXIT: taosMemoryFree(pChkpDir); taosMemoryFree(pChkpIdDir); taosReleaseRef(taskDbWrapperId, refId); + taosMemoryFree(ppCf); return code; } int32_t streamBackendDoCheckpoint(void* arg, int64_t chkpId) { return taskDbDoCheckpoint(arg, chkpId); } @@ -3594,6 +3595,7 @@ void dbChkpDestroy(SDbChkp* pChkp) { taosMemoryFree(pChkp->pCurrent); taosMemoryFree(pChkp->pManifest); + taosMemoryFree(pChkp); } int32_t dbChkpInit(SDbChkp* p) { From 77261068e0a41c204107685030eb6279e1765122 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 16 Nov 2023 11:06:55 +0800 Subject: [PATCH 120/195] fix stream backend convert --- include/libs/stream/tstream.h | 2 +- source/libs/stream/src/streamBackendRocksdb.c | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 337b760cf4..4979a576a7 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -59,7 +59,7 @@ typedef struct SStreamTask SStreamTask; typedef struct SStreamQueue SStreamQueue; typedef struct SStreamTaskSM SStreamTaskSM; -#define SSTREAM_TASK_VER 3 +#define SSTREAM_TASK_VER 2 #define SSTREAM_TASK_INCOMPATIBLE_VER 1 #define SSTREAM_TASK_NEED_CONVERT_VER 2 diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 4e71eff431..f9bd5476e7 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1759,7 +1759,7 @@ int32_t taskDbOpenCfByKey(STaskDbWrapper* pDb, const char* key) { rocksdb_column_family_handle_t* cf = rocksdb_create_column_family(pDb->db, pDb->pCfOpts[idx], ginitDict[idx].key, &err); - if (err != NULL) { + if (idx != 0 && err != NULL) { stError("failed to open cf, key:%s, reason: %s", key, err); taosMemoryFree(err); code = -1; From cee33ffb95a860a5e21fabe3c922b72ced545ca2 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 16 Nov 2023 11:22:18 +0800 Subject: [PATCH 121/195] fix stream backend convert --- source/libs/stream/src/streamBackendRocksdb.c | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index f9bd5476e7..c3196708ee 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1755,11 +1755,14 @@ int32_t taskDbOpenCfByKey(STaskDbWrapper* pDb, const char* key) { int32_t code = 0; char* err = NULL; int8_t idx = getCfIdx(key); + if (idx == -1) return -1; + if (pDb->pCf[idx] != NULL) return code; + rocksdb_column_family_handle_t* cf = rocksdb_create_column_family(pDb->db, pDb->pCfOpts[idx], ginitDict[idx].key, &err); - if (idx != 0 && err != NULL) { + if (err != NULL) { stError("failed to open cf, key:%s, reason: %s", key, err); taosMemoryFree(err); code = -1; From f5ef2e727fd0a0aff10d957bb3b3d0d91321649e Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 16 Nov 2023 14:41:04 +0800 Subject: [PATCH 122/195] fix stream backend convert --- source/libs/stream/inc/streamBackendRocksdb.h | 1 + source/libs/stream/src/streamBackendRocksdb.c | 10 ++++++++++ source/libs/stream/src/streamMeta.c | 17 +++++++++++++++-- 3 files changed, 26 insertions(+), 2 deletions(-) diff --git a/source/libs/stream/inc/streamBackendRocksdb.h b/source/libs/stream/inc/streamBackendRocksdb.h index 47404f311f..e6554e2fdf 100644 --- a/source/libs/stream/inc/streamBackendRocksdb.h +++ b/source/libs/stream/inc/streamBackendRocksdb.h @@ -128,6 +128,7 @@ typedef struct { TdThreadRwlock rwLock; } SBkdMgt; +bool streamBackendDataIsExist(const char* path, int64_t chkpId, int32_t vgId); void* streamBackendInit(const char* path, int64_t chkpId, int32_t vgId); void streamBackendCleanup(void* arg); void streamBackendHandleCleanup(void* arg); diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index c3196708ee..d0e343ae78 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -308,6 +308,16 @@ int32_t rebuildDirFromChkp2(const char* path, char* key, int64_t chkpId, char** return 0; } +bool streamBackendDataIsExist(const char* path, int64_t chkpId, int32_t vgId) { + bool exist = true; + char* state = taosMemoryCalloc(1, strlen(path) + 32); + sprintf(state, "%s%s%s", path, TD_DIRSEP, "state"); + if (!taosDirExist(state)) { + exist = false; + } + taosMemoryFree(state); + return exist; +} void* streamBackendInit(const char* streamPath, int64_t chkpId, int32_t vgId) { char* backendPath = NULL; int32_t code = rebuildDirFromCheckpoint(streamPath, chkpId, &backendPath); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 6b1ef6a7a0..afc7512cf8 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -188,8 +188,13 @@ int32_t streamMetaCheckBackendCompatible(SStreamMeta* pMeta) { } int32_t streamMetaCvtDbFormat(SStreamMeta* pMeta) { - int32_t code = 0; - int64_t chkpId = streamMetaGetLatestCheckpointId(pMeta); + int32_t code = 0; + int64_t chkpId = streamMetaGetLatestCheckpointId(pMeta); + + bool exist = streamBackendDataIsExist(pMeta->path, chkpId, pMeta->vgId); + if (exist == false) { + return code; + } SBackendWrapper* pBackend = streamBackendInit(pMeta->path, chkpId, pMeta->vgId); void* pIter = taosHashIterate(pBackend->cfInst, NULL); @@ -206,6 +211,14 @@ int32_t streamMetaCvtDbFormat(SStreamMeta* pMeta) { _EXIT: streamBackendCleanup((void*)pBackend); + + if (code == 0) { + char* state = taosMemoryCalloc(1, strlen(pMeta->path) + 32); + sprintf(state, "%s%s%s", pMeta->path, TD_DIRSEP, "state"); + taosRemoveDir(state); + taosMemoryFree(state); + } + return code; } int32_t streamMetaMayCvtDbFormat(SStreamMeta* pMeta) { From 8023f2bebdd20fd4ebdff7b030dcf5cddfc55240 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 16 Nov 2023 15:30:01 +0800 Subject: [PATCH 123/195] fix stream backend convert --- source/dnode/mnode/impl/src/mndStream.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index f394cb980d..b7e659057b 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1245,7 +1245,7 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { while (1) { pIter = sdbFetch(pSdb, SDB_STREAM, pIter, (void **)&pStream); if (pIter == NULL) break; - code = mndProcessStreamCheckpointTrans(pMnode, pStream, pStream->checkpointId + 1); + code = mndProcessStreamCheckpointTrans(pMnode, pStream, pMsg->checkpointId); sdbRelease(pSdb, pStream); if (code == -1) { break; From 7905f22aae7c787aa9538d657eea32c485e27022 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 16 Nov 2023 17:35:17 +0800 Subject: [PATCH 124/195] fix stream backend convert --- source/dnode/mnode/impl/src/mndStream.c | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index b7e659057b..89e9c9f075 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -202,7 +202,8 @@ STREAM_ENCODE_OVER: return NULL; } - mTrace("stream:%s, encode to raw:%p, row:%p", pStream->name, pRaw, pStream); + mTrace("stream:%s, encode to raw:%p, row:%p, checkpoint:%" PRId64 "", pStream->name, pRaw, pStream, + pStream->checkpointId); return pRaw; } @@ -255,7 +256,8 @@ STREAM_DECODE_OVER: return NULL; } - mTrace("stream:%s, decode from raw:%p, row:%p", pStream->name, pRaw, pStream); + mTrace("stream:%s, decode from raw:%p, row:%p, checkpoint:%" PRId64 "", pStream->name, pRaw, pStream, + pStream->checkpointId); return pRow; } @@ -908,8 +910,11 @@ int64_t mndStreamGenChkpId(SMnode *pMnode) { if (pIter == NULL) break; maxChkpId = TMAX(maxChkpId, pStream->checkpointId); + mError("stream %p checkpoint %" PRId64 "", pStream, pStream->checkpointId); sdbRelease(pSdb, pStream); } + + mError("generated checkpoint %" PRId64 "", maxChkpId + 1); return maxChkpId + 1; } From 52d7a26c102ca699f5d9b8b6bd3c4e8f9e171ab3 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 16 Nov 2023 21:22:47 +0800 Subject: [PATCH 125/195] fix stream backend convert --- source/libs/stream/src/streamBackendRocksdb.c | 93 ++++++++++++++++++- 1 file changed, 92 insertions(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index d0e343ae78..72f8167f49 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -230,6 +230,94 @@ int32_t rebuildDirFromCheckpoint(const char* path, int64_t chkpId, char** dst) { return 0; } +int32_t readMetaData(char* path, SArray* list) { + char buf[128] = {0}; + char* metaPath = taosMemoryCalloc(1, strlen(path)); + sprintf(metaPath, "%s%s%s", path, TD_DIRSEP, "META"); + + TdFilePtr pFile = taosOpenFile(path, TD_FILE_READ); + + if (taosReadFile(pFile, buf, sizeof(buf)) <= 0) { + taosMemoryFree(metaPath); + taosCloseFile(&pFile); + return -1; + } + int32_t len = strlen(buf); + for (int i = 0; i < len; i++) { + if (buf[i] == '\n') { + char* item = taosMemoryCalloc(1, i + 1); + memcpy(item, buf, i); + taosArrayPush(list, &item); + + item = taosMemoryCalloc(1, len - i); + memcpy(item, buf + i + 1, len - i - 1); + taosArrayPush(list, &item); + } + } + + taosCloseFile(&pFile); + taosMemoryFree(metaPath); + return 0; +} +int32_t validAndCvtMeta(char* path, SArray* list, int64_t chkpId) { + int32_t complete = 1; + int32_t len = strlen(path) + 32; + char* src = taosMemoryCalloc(1, len); + char* dst = taosMemoryCalloc(1, len); + + for (int i = 0; i < taosArrayGetSize(list); i++) { + char* p = taosArrayGetP(list, i); + sprintf(src, "%s%s%s", path, TD_DIRSEP, p); + if (taosStatFile(src, NULL, NULL, NULL) != 0) { + complete = 0; + break; + } + + char temp[64] = {0}; + for (int j = 0; j < strlen(p); j++) { + if (p[j] == '_') { + memcpy(temp, p, j); + } + if (taosStr2int64(p + j + 1) != chkpId) { + complete = 0; + break; + } + } + + sprintf(dst, "%s%s%s", path, TD_DIRSEP, temp); + taosRenameFile(src, dst); + + memset(src, 0, len); + memset(dst, 0, len); + } + + taosMemoryFree(src); + taosMemoryFree(dst); + return complete == 1 ? 0 : -1; +} + +int32_t rebuildFromRemote(char* key, char* chkpPath, int64_t chkpId, char* defaultPath) { + int32_t code = downloadCheckpoint(key, chkpPath); + if (code != 0) { + return -1; + } + + SArray* list = taosArrayInit(2, sizeof(void*)); + code = readMetaData(chkpPath, list); + if (code == 0) { + code = validAndCvtMeta(chkpPath, list, chkpId); + } + taosArrayDestroyP(list, taosMemoryFree); + + if (code == 0) { + if (taosIsDir(defaultPath)) { + taosRemoveDir(defaultPath); + } + taosMkDir(defaultPath); + code = copyFiles(chkpPath, defaultPath); + } + return code; +} int32_t rebuildDirFromChkp2(const char* path, char* key, int64_t chkpId, char** dbPrefixPath, char** dbPath) { // impl later int32_t code = 0; @@ -271,11 +359,14 @@ int32_t rebuildDirFromChkp2(const char* path, char* key, int64_t chkpId, char** stInfo("start to restart stream backend at checkpoint path: %s", chkpPath); } + } else if (!taosIsDir(chkpPath)) { + code = rebuildFromRemote(key, chkpPath, chkpId, defaultPath); } else { stInfo("failed to start stream backend at %s, reason: %s, restart from default defaultPath dir:%s", chkpPath, tstrerror(TAOS_SYSTEM_ERROR(errno)), defaultPath); taosMkDir(defaultPath); } + taosMemoryFree(chkpPath); } else { char* chkpPath = taosMemoryCalloc(1, strlen(path) + 256); @@ -309,7 +400,7 @@ int32_t rebuildDirFromChkp2(const char* path, char* key, int64_t chkpId, char** } bool streamBackendDataIsExist(const char* path, int64_t chkpId, int32_t vgId) { - bool exist = true; + bool exist = true; char* state = taosMemoryCalloc(1, strlen(path) + 32); sprintf(state, "%s%s%s", path, TD_DIRSEP, "state"); if (!taosDirExist(state)) { From b6994413d0f55d7e5580b54a5d7c183150645d2d Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Fri, 17 Nov 2023 10:13:39 +0800 Subject: [PATCH 126/195] fix stream backend convert --- source/libs/stream/src/streamBackendRocksdb.c | 91 +++++++++---------- 1 file changed, 41 insertions(+), 50 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 72f8167f49..c80b8163a4 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -296,10 +296,10 @@ int32_t validAndCvtMeta(char* path, SArray* list, int64_t chkpId) { return complete == 1 ? 0 : -1; } -int32_t rebuildFromRemote(char* key, char* chkpPath, int64_t chkpId, char* defaultPath) { +int32_t rebuildFromRemoteChkp(char* key, char* chkpPath, int64_t chkpId, char* defaultPath) { int32_t code = downloadCheckpoint(key, chkpPath); if (code != 0) { - return -1; + return code; } SArray* list = taosArrayInit(2, sizeof(void*)); @@ -318,16 +318,35 @@ int32_t rebuildFromRemote(char* key, char* chkpPath, int64_t chkpId, char* defau } return code; } + +int32_t rebuildFromLocalChkp(char* key, char* chkpPath, int64_t chkpId, char* defaultPath) { + int32_t code = 0; + if (taosIsDir(chkpPath) && isValidCheckpoint(chkpPath)) { + if (taosIsDir(defaultPath)) { + taosRemoveDir(defaultPath); + } + taosMkDir(defaultPath); + code = copyFiles(chkpPath, defaultPath); + if (code != 0) { + stError("failed to restart stream backend from %s, reason: %s", chkpPath, tstrerror(TAOS_SYSTEM_ERROR(errno))); + } else { + stInfo("start to restart stream backend at checkpoint path: %s", chkpPath); + } + + return code; + } + return -1; +} + +int32_t rebuildFromlocalDefault(char* key, char* chkpPath, int64_t chkpId, char* defaultPath) { + int32_t code = 0; + return code; +} + int32_t rebuildDirFromChkp2(const char* path, char* key, int64_t chkpId, char** dbPrefixPath, char** dbPath) { // impl later int32_t code = 0; - /*param@1: checkpointId dir - param@2: state - copy pChkpIdDir's file to state dir - opt to set hard link to previous file - */ - char* prefixPath = taosMemoryCalloc(1, strlen(path) + 128); sprintf(prefixPath, "%s%s%s", path, TD_DIRSEP, key); @@ -342,61 +361,33 @@ int32_t rebuildDirFromChkp2(const char* path, char* key, int64_t chkpId, char** taosMulMkDir(defaultPath); } + char* chkpPath = taosMemoryCalloc(1, strlen(path) + 256); if (chkpId != 0) { - char* chkpPath = taosMemoryCalloc(1, strlen(path) + 256); sprintf(chkpPath, "%s%s%s%s%s%" PRId64 "", prefixPath, TD_DIRSEP, "checkpoints", TD_DIRSEP, "checkpoint", chkpId); - if (taosIsDir(chkpPath) && isValidCheckpoint(chkpPath)) { - if (taosIsDir(defaultPath)) { - // remove dir if exists - // taosRenameFile(const char *oldName, const char *newName) - taosRemoveDir(defaultPath); - } - taosMkDir(defaultPath); - code = copyFiles(chkpPath, defaultPath); - if (code != 0) { - stError("failed to restart stream backend from %s, reason: %s", chkpPath, tstrerror(TAOS_SYSTEM_ERROR(errno))); - } else { - stInfo("start to restart stream backend at checkpoint path: %s", chkpPath); - } - - } else if (!taosIsDir(chkpPath)) { - code = rebuildFromRemote(key, chkpPath, chkpId, defaultPath); - } else { - stInfo("failed to start stream backend at %s, reason: %s, restart from default defaultPath dir:%s", chkpPath, - tstrerror(TAOS_SYSTEM_ERROR(errno)), defaultPath); - taosMkDir(defaultPath); + code = rebuildFromLocalChkp(key, chkpPath, chkpId, defaultPath); + if (code != 0) { + code = rebuildFromRemoteChkp(key, chkpPath, chkpId, defaultPath); } - taosMemoryFree(chkpPath); + if (code != 0) { + stInfo("failed to start stream backend at %s, reason: %s, restart from default defaultPath dir:%s", chkpPath, + tstrerror(TAOS_SYSTEM_ERROR(errno)), defaultPath); + code = taosMkDir(defaultPath); + } } else { - char* chkpPath = taosMemoryCalloc(1, strlen(path) + 256); sprintf(chkpPath, "%s%s%s%s%s%" PRId64 "", prefixPath, TD_DIRSEP, "checkpoints", TD_DIRSEP, "checkpoint", (int64_t)-1); - stInfo("no chkp id specified, try to restart from received chkp id -1, dir: %s", chkpPath); - if (taosIsDir(chkpPath) && isValidCheckpoint(chkpPath)) { - if (taosIsDir(defaultPath)) { - taosRemoveDir(defaultPath); - } - taosMkDir(defaultPath); - code = copyFiles(chkpPath, defaultPath); - stInfo("copy snap file from %s to %s", chkpPath, defaultPath); - if (code != 0) { - stError("failed to restart stream backend from %s, reason: %s", chkpPath, tstrerror(TAOS_SYSTEM_ERROR(errno))); - } else { - stInfo("start to restart stream backend at checkpoint path: %s", chkpPath); - } - } else { - stInfo("failed to start stream backend at %s, reason: %s, restart from default defaultPath dir:%s", chkpPath, - tstrerror(TAOS_SYSTEM_ERROR(errno)), defaultPath); - taosMkDir(defaultPath); + + code = rebuildFromLocalChkp(key, chkpPath, -1, defaultPath); + if (code != 0) { + code = taosMkDir(defaultPath); } - taosMemoryFree(chkpPath); } *dbPath = defaultPath; *dbPrefixPath = prefixPath; - return 0; + return code; } bool streamBackendDataIsExist(const char* path, int64_t chkpId, int32_t vgId) { From e455c4799d98406f2b0120c7703bb443d262ed1d Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Fri, 17 Nov 2023 10:33:43 +0800 Subject: [PATCH 127/195] fix stream backend convert --- source/libs/stream/src/streamBackendRocksdb.c | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index c80b8163a4..22f5b93416 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -230,13 +230,13 @@ int32_t rebuildDirFromCheckpoint(const char* path, int64_t chkpId, char** dst) { return 0; } -int32_t readMetaData(char* path, SArray* list) { - char buf[128] = {0}; +int32_t remoteChkp_readMetaData(char* path, SArray* list) { char* metaPath = taosMemoryCalloc(1, strlen(path)); sprintf(metaPath, "%s%s%s", path, TD_DIRSEP, "META"); TdFilePtr pFile = taosOpenFile(path, TD_FILE_READ); + char buf[128] = {0}; if (taosReadFile(pFile, buf, sizeof(buf)) <= 0) { taosMemoryFree(metaPath); taosCloseFile(&pFile); @@ -259,7 +259,7 @@ int32_t readMetaData(char* path, SArray* list) { taosMemoryFree(metaPath); return 0; } -int32_t validAndCvtMeta(char* path, SArray* list, int64_t chkpId) { +int32_t remoteChkp_validAndCvtMeta(char* path, SArray* list, int64_t chkpId) { int32_t complete = 1; int32_t len = strlen(path) + 32; char* src = taosMemoryCalloc(1, len); @@ -283,6 +283,7 @@ int32_t validAndCvtMeta(char* path, SArray* list, int64_t chkpId) { break; } } + if (complete == 0) break; sprintf(dst, "%s%s%s", path, TD_DIRSEP, temp); taosRenameFile(src, dst); @@ -293,6 +294,7 @@ int32_t validAndCvtMeta(char* path, SArray* list, int64_t chkpId) { taosMemoryFree(src); taosMemoryFree(dst); + return complete == 1 ? 0 : -1; } @@ -303,9 +305,9 @@ int32_t rebuildFromRemoteChkp(char* key, char* chkpPath, int64_t chkpId, char* d } SArray* list = taosArrayInit(2, sizeof(void*)); - code = readMetaData(chkpPath, list); + code = remoteChkp_readMetaData(chkpPath, list); if (code == 0) { - code = validAndCvtMeta(chkpPath, list, chkpId); + code = remoteChkp_validAndCvtMeta(chkpPath, list, chkpId); } taosArrayDestroyP(list, taosMemoryFree); From c7c1bdef6e1b86fcb8d2333abce86cfb81b83655 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Fri, 17 Nov 2023 11:44:21 +0800 Subject: [PATCH 128/195] fix stream backend convert --- source/libs/stream/src/streamBackendRocksdb.c | 33 ++++++++++++++----- 1 file changed, 24 insertions(+), 9 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 22f5b93416..60738b96bf 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -259,38 +259,53 @@ int32_t remoteChkp_readMetaData(char* path, SArray* list) { taosMemoryFree(metaPath); return 0; } +int32_t remoteChkp_validMetaFile(char* name, char* prename, int64_t chkpId) { + int8_t valid = 0; + for (int i = 0; i < strlen(name); i++) { + if (name[i] == '_') { + memcpy(prename, name, i); + if (taosStr2int64(name + i + 1) != chkpId) { + break; + } else { + valid = 1; + } + } + } + return valid; +} int32_t remoteChkp_validAndCvtMeta(char* path, SArray* list, int64_t chkpId) { int32_t complete = 1; int32_t len = strlen(path) + 32; char* src = taosMemoryCalloc(1, len); char* dst = taosMemoryCalloc(1, len); + int8_t count = 0; for (int i = 0; i < taosArrayGetSize(list); i++) { char* p = taosArrayGetP(list, i); sprintf(src, "%s%s%s", path, TD_DIRSEP, p); + + // check file exist if (taosStatFile(src, NULL, NULL, NULL) != 0) { complete = 0; break; } + // check file name char temp[64] = {0}; - for (int j = 0; j < strlen(p); j++) { - if (p[j] == '_') { - memcpy(temp, p, j); - } - if (taosStr2int64(p + j + 1) != chkpId) { - complete = 0; - break; - } + if (remoteChkp_validMetaFile(p, temp, chkpId)) { + count++; } - if (complete == 0) break; + // rename file sprintf(dst, "%s%s%s", path, TD_DIRSEP, temp); taosRenameFile(src, dst); memset(src, 0, len); memset(dst, 0, len); } + if (count != taosArrayGetSize(list)) { + complete = 0; + } taosMemoryFree(src); taosMemoryFree(dst); From 1dbdb32dd7cb048a4ae399dc4e50d10beecf9f91 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Fri, 17 Nov 2023 11:56:22 +0800 Subject: [PATCH 129/195] fix stream backend convert --- source/libs/stream/src/streamBackendRocksdb.c | 33 +++++++++++++++---- 1 file changed, 27 insertions(+), 6 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 60738b96bf..a01fabd3a1 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -319,6 +319,11 @@ int32_t rebuildFromRemoteChkp(char* key, char* chkpPath, int64_t chkpId, char* d return code; } + int32_t len = strlen(defaultPath) + 32; + char* tmp = taosMemoryCalloc(1, len); + sprintf(tmp, "%s%s%s", defaultPath, TD_DIRSEP, "_tmp"); + if (taosIsDir(tmp)) taosRemoveDir(tmp); + SArray* list = taosArrayInit(2, sizeof(void*)); code = remoteChkp_readMetaData(chkpPath, list); if (code == 0) { @@ -328,19 +333,31 @@ int32_t rebuildFromRemoteChkp(char* key, char* chkpPath, int64_t chkpId, char* d if (code == 0) { if (taosIsDir(defaultPath)) { - taosRemoveDir(defaultPath); + taosRenameFile(defaultPath, tmp); } taosMkDir(defaultPath); code = copyFiles(chkpPath, defaultPath); } + + if (code != 0) { + if (taosIsDir(defaultPath)) taosRemoveDir(defaultPath); + if (taosIsDir(tmp)) taosRenameFile(tmp, defaultPath); + } return code; } int32_t rebuildFromLocalChkp(char* key, char* chkpPath, int64_t chkpId, char* defaultPath) { - int32_t code = 0; + int32_t code = -1; + int32_t len = strlen(defaultPath) + 32; + char* tmp = taosMemoryCalloc(1, len); + sprintf(tmp, "%s%s%s", defaultPath, TD_DIRSEP, "_tmp"); + if (taosIsDir(chkpPath) && isValidCheckpoint(chkpPath)) { + if (taosIsDir(tmp)) { + taosRemoveDir(tmp); + } if (taosIsDir(defaultPath)) { - taosRemoveDir(defaultPath); + taosRenameFile(defaultPath, tmp); } taosMkDir(defaultPath); code = copyFiles(chkpPath, defaultPath); @@ -349,10 +366,14 @@ int32_t rebuildFromLocalChkp(char* key, char* chkpPath, int64_t chkpId, char* de } else { stInfo("start to restart stream backend at checkpoint path: %s", chkpPath); } - - return code; } - return -1; + if (code != 0) { + if (taosIsDir(defaultPath)) taosRemoveDir(defaultPath); + if (taosIsDir(tmp)) taosRenameFile(tmp, defaultPath); + } + + taosMemoryFree(tmp); + return code; } int32_t rebuildFromlocalDefault(char* key, char* chkpPath, int64_t chkpId, char* defaultPath) { From ccc74ef558cae8e91b27bf0bbee0bc6ad4557c4b Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Fri, 17 Nov 2023 15:39:03 +0800 Subject: [PATCH 130/195] fix stream backend convert --- source/libs/stream/src/streamBackendRocksdb.c | 20 +++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index a01fabd3a1..c91bb2a393 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -321,8 +321,9 @@ int32_t rebuildFromRemoteChkp(char* key, char* chkpPath, int64_t chkpId, char* d int32_t len = strlen(defaultPath) + 32; char* tmp = taosMemoryCalloc(1, len); - sprintf(tmp, "%s%s%s", defaultPath, TD_DIRSEP, "_tmp"); + sprintf(tmp, "%s%s", defaultPath, "_tmp"); if (taosIsDir(tmp)) taosRemoveDir(tmp); + if (taosIsDir(defaultPath)) taosRenameFile(defaultPath, tmp); SArray* list = taosArrayInit(2, sizeof(void*)); code = remoteChkp_readMetaData(chkpPath, list); @@ -332,9 +333,6 @@ int32_t rebuildFromRemoteChkp(char* key, char* chkpPath, int64_t chkpId, char* d taosArrayDestroyP(list, taosMemoryFree); if (code == 0) { - if (taosIsDir(defaultPath)) { - taosRenameFile(defaultPath, tmp); - } taosMkDir(defaultPath); code = copyFiles(chkpPath, defaultPath); } @@ -342,7 +340,11 @@ int32_t rebuildFromRemoteChkp(char* key, char* chkpPath, int64_t chkpId, char* d if (code != 0) { if (taosIsDir(defaultPath)) taosRemoveDir(defaultPath); if (taosIsDir(tmp)) taosRenameFile(tmp, defaultPath); + } else { + taosRemoveDir(tmp); } + + taosMemoryFree(tmp); return code; } @@ -350,15 +352,15 @@ int32_t rebuildFromLocalChkp(char* key, char* chkpPath, int64_t chkpId, char* de int32_t code = -1; int32_t len = strlen(defaultPath) + 32; char* tmp = taosMemoryCalloc(1, len); - sprintf(tmp, "%s%s%s", defaultPath, TD_DIRSEP, "_tmp"); + sprintf(tmp, "%s%s", defaultPath, "_tmp"); + + if (taosIsDir(tmp)) taosRemoveDir(tmp); + if (taosIsDir(defaultPath)) taosRenameFile(defaultPath, tmp); if (taosIsDir(chkpPath) && isValidCheckpoint(chkpPath)) { if (taosIsDir(tmp)) { taosRemoveDir(tmp); } - if (taosIsDir(defaultPath)) { - taosRenameFile(defaultPath, tmp); - } taosMkDir(defaultPath); code = copyFiles(chkpPath, defaultPath); if (code != 0) { @@ -370,6 +372,8 @@ int32_t rebuildFromLocalChkp(char* key, char* chkpPath, int64_t chkpId, char* de if (code != 0) { if (taosIsDir(defaultPath)) taosRemoveDir(defaultPath); if (taosIsDir(tmp)) taosRenameFile(tmp, defaultPath); + } else { + taosRemoveDir(tmp); } taosMemoryFree(tmp); From 286413abbb5c1bf84d127114cb3b0aebf034a243 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Fri, 17 Nov 2023 16:14:51 +0800 Subject: [PATCH 131/195] fix stream backend convert --- source/libs/stream/src/streamBackendRocksdb.c | 1 + source/libs/stream/src/streamMeta.c | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index c91bb2a393..d2aab7459b 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -425,6 +425,7 @@ int32_t rebuildDirFromChkp2(const char* path, char* key, int64_t chkpId, char** code = taosMkDir(defaultPath); } } + taosMemoryFree(chkpPath); *dbPath = defaultPath; *dbPrefixPath = prefixPath; diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index afc7512cf8..138759c179 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -243,7 +243,7 @@ int32_t streamMetaMayCvtDbFormat(SStreamMeta* pMeta) { int32_t streamTaskSetDb(SStreamMeta* pMeta, void* arg, char* key) { SStreamTask* pTask = arg; - int64_t chkpId = pTask->chkInfo.checkpointingId; + int64_t chkpId = pTask->chkInfo.checkpointId; taosThreadMutexLock(&pMeta->backendMutex); void** ppBackend = taosHashGet(pMeta->pTaskDbUnique, key, strlen(key)); From e7830bfdfdce50134a6b6e6e82c37351775d1e2f Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Fri, 17 Nov 2023 16:50:45 +0800 Subject: [PATCH 132/195] fix stream backend convert --- source/libs/stream/src/streamBackendRocksdb.c | 48 +++++++++++-------- 1 file changed, 29 insertions(+), 19 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index d2aab7459b..4c86beb44d 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -39,8 +39,9 @@ typedef struct { int32_t streamStateOpenBackendCf(void* backend, char* name, char** cfs, int32_t nCf); -void destroyRocksdbCfInst(RocksdbCfInst* inst); -int32_t getCfIdx(const char* cfName); +void destroyRocksdbCfInst(RocksdbCfInst* inst); +int32_t getCfIdx(const char* cfName); +STaskDbWrapper* taskDbOpenImpl(char* key, char* statePath, char* dbPath); void destroyCompactFilteFactory(void* arg); void destroyCompactFilte(void* arg); @@ -188,7 +189,14 @@ int32_t getCfIdx(const char* cfName) { return idx; } -bool isValidCheckpoint(const char* dir) { return true; } +bool isValidCheckpoint(const char* dir) { + STaskDbWrapper* pDb = taskDbOpenImpl(NULL, NULL, (char*)dir); + if (pDb == NULL) { + return true; + } + taskDbDestroy(pDb); + return true; +} int32_t rebuildDirFromCheckpoint(const char* path, int64_t chkpId, char** dst) { // impl later @@ -1707,25 +1715,19 @@ void taskDbUpdateChkpId(void* pTaskDb, int64_t chkpId) { p->chkpId = chkpId; taosThreadMutexUnlock(&p->mutex); } -STaskDbWrapper* taskDbOpen(char* path, char* key, int64_t chkpId) { - char* statePath = NULL; + +STaskDbWrapper* taskDbOpenImpl(char* key, char* statePath, char* dbPath) { char* err = NULL; - char* dbPath = NULL; char** cfNames = NULL; size_t nCf = 0; - if (rebuildDirFromChkp2(path, key, chkpId, &statePath, &dbPath) != 0) { - return NULL; - } - STaskDbWrapper* pTaskDb = taosMemoryCalloc(1, sizeof(STaskDbWrapper)); - pTaskDb->idstr = taosStrdup(key); - pTaskDb->path = statePath; + pTaskDb->idstr = key ? taosStrdup(key) : NULL; + pTaskDb->path = statePath ? taosStrdup(statePath) : NULL; taosThreadMutexInit(&pTaskDb->mutex, NULL); taskDbInitChkpOpt(pTaskDb); taskDbInitOpt(pTaskDb); - statePath = NULL; cfNames = rocksdb_list_column_families(pTaskDb->dbOpt, dbPath, &nCf, &err); if (nCf == 0) { @@ -1752,19 +1754,27 @@ STaskDbWrapper* taskDbOpen(char* path, char* key, int64_t chkpId) { } qDebug("succ to init stream backend at %s, backend:%p", dbPath, pTaskDb); - taosMemoryFree(dbPath); - - return pTaskDb; _EXIT: - taskDbDestroy(pTaskDb); + taskDbDestroy(pTaskDb); if (err) taosMemoryFree(err); if (cfNames) rocksdb_list_column_families_destroy(cfNames, nCf); - if (dbPath) taosMemoryFree(dbPath); - if (statePath) taosMemoryFree(statePath); return NULL; } +STaskDbWrapper* taskDbOpen(char* path, char* key, int64_t chkpId) { + char* statePath = NULL; + char* dbPath = NULL; + + if (rebuildDirFromChkp2(path, key, chkpId, &statePath, &dbPath) != 0) { + return NULL; + } + // taosMemoryFree(statePath); + + STaskDbWrapper* pTaskDb = taskDbOpenImpl(key, statePath, dbPath); + taosMemoryFree(dbPath); + return pTaskDb; +} void taskDbDestroy(void* pDb) { STaskDbWrapper* wrapper = pDb; From e453962271946fc458cad84b0e00b27cdcb91890 Mon Sep 17 00:00:00 2001 From: Bob Liu Date: Tue, 14 Nov 2023 15:21:26 +0800 Subject: [PATCH 133/195] rewrite nodes function --- include/libs/nodes/nodes.h | 3 +- source/dnode/mgmt/node_mgmt/src/dmEnv.c | 2 + source/libs/nodes/src/nodesCodeFuncs.c | 3092 +++++++++++++++++------ source/libs/nodes/src/nodesUtilFuncs.c | 1241 +-------- 4 files changed, 2262 insertions(+), 2076 deletions(-) diff --git a/include/libs/nodes/nodes.h b/include/libs/nodes/nodes.h index 21d33249b8..136926a856 100644 --- a/include/libs/nodes/nodes.h +++ b/include/libs/nodes/nodes.h @@ -101,7 +101,7 @@ typedef struct SNodeList { typedef struct SNodeAllocator SNodeAllocator; -void initNodeName(); +void nodesInit(); int32_t nodesInitAllocatorSet(); void nodesDestroyAllocatorSet(); int32_t nodesCreateAllocator(int64_t queryId, int32_t chunkSize, int64_t* pAllocatorId); @@ -111,6 +111,7 @@ int64_t nodesMakeAllocatorWeakRef(int64_t allocatorId); int64_t nodesReleaseAllocatorWeakRef(int64_t allocatorId); void nodesDestroyAllocator(int64_t allocatorId); +int32_t getNodeSize(ENodeType type); SNode* nodesMakeNode(ENodeType type); void nodesDestroyNode(SNode* pNode); void nodesFree(void* p); diff --git a/source/dnode/mgmt/node_mgmt/src/dmEnv.c b/source/dnode/mgmt/node_mgmt/src/dmEnv.c index 6f13abcebc..c57043b8bb 100644 --- a/source/dnode/mgmt/node_mgmt/src/dmEnv.c +++ b/source/dnode/mgmt/node_mgmt/src/dmEnv.c @@ -17,6 +17,7 @@ #include "dmMgmt.h" #include "audit.h" #include "libs/function/tudf.h" +#include "nodes.h" #define DM_INIT_AUDIT() \ do { \ @@ -155,6 +156,7 @@ extern void s3End(); int32_t dmInit() { dInfo("start to init dnode env"); + nodesInit(); if (dmDiskInit() != 0) return -1; if (!dmCheckDataDirVersion()) return -1; if (!dmCheckDiskSpace()) return -1; diff --git a/source/libs/nodes/src/nodesCodeFuncs.c b/source/libs/nodes/src/nodesCodeFuncs.c index 582ecf6753..01d808c7d8 100644 --- a/source/libs/nodes/src/nodesCodeFuncs.c +++ b/source/libs/nodes/src/nodesCodeFuncs.c @@ -28,204 +28,67 @@ static int32_t jsonToNodeObject(const SJson* pJson, const char* pName, SNode** p static int32_t makeNodeByJson(const SJson* pJson, SNode** pNode); -typedef const char* taoscstr; -taoscstr nodeNames[QUERY_NODE_END] = {}; +typedef int32_t (*FExecNodeToJson)(const void* pObj, SJson* pJson); +typedef int32_t (*FExecJsonToNode)(const SJson* pJson, void* pObj); +typedef void (*FExecDestoryNode)(SNode* pNode); -#define TAOS_DEFINE_NODE_NAME(type, msg) nodeNames[type] = msg; +/** + * @brief Node operation to binding function set + */ +typedef struct SBuiltinNodeDefinition { + const char* name; + int32_t nodeSize; + FExecNodeToJson toJsonFunc; + FExecJsonToNode toNodeFunc; + FExecDestoryNode destoryFunc; +} SBuiltinNodeDefinition; -void initNodeName() { - TAOS_DEFINE_NODE_NAME(QUERY_NODE_COLUMN, "Column") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_VALUE, "Value") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_OPERATOR, "Operator") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_CONDITION, "LogicCondition") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_FUNCTION, "Function") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_REAL_TABLE, "RealTable") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_TEMP_TABLE, "TempTable") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_JOIN_TABLE, "JoinTable") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_GROUPING_SET, "GroupingSet") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_ORDER_BY_EXPR, "OrderByExpr") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_LIMIT, "Limit") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_STATE_WINDOW, "StateWindow") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SESSION_WINDOW, "SessionWinow") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_INTERVAL_WINDOW, "IntervalWindow") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_NODE_LIST, "NodeList") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_FILL, "Fill") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_RAW_EXPR, "RawExpr") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_TARGET, "Target") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_DATABLOCK_DESC, "DataBlockDesc") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SLOT_DESC, "SlotDesc") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_COLUMN_DEF, "ColumnDef") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_DOWNSTREAM_SOURCE, "DownstreamSource") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_DATABASE_OPTIONS, "DatabaseOptions") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_TABLE_OPTIONS, "TableOptions") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_INDEX_OPTIONS, "IndexOptions") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_EXPLAIN_OPTIONS, "ExplainOptions") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_STREAM_OPTIONS, "StreamOptions") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_LEFT_VALUE, "LeftValue") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_WHEN_THEN, "WhenThen") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_CASE_WHEN, "CaseWhen") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_EVENT_WINDOW, "EventWindow") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SET_OPERATOR, "SetOperator") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SELECT_STMT, "SelectStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_VNODE_MODIFY_STMT, "VnodeModifStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_CREATE_DATABASE_STMT, "CreateDatabaseStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_DROP_DATABASE_STMT, "DropDatabaseStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_ALTER_DATABASE_STMT, "AlterDatabaseStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_FLUSH_DATABASE_STMT, "FlushDatabaseStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_TRIM_DATABASE_STMT, "TrimDatabaseStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_CREATE_TABLE_STMT, "CreateTableStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_CREATE_SUBTABLE_CLAUSE, "CreateSubtableClause") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_CREATE_MULTI_TABLES_STMT, "CreateMultiTableStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_DROP_TABLE_CLAUSE, "DropTableClause") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_DROP_TABLE_STMT, "DropTableStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_DROP_SUPER_TABLE_STMT, "DropSuperTableStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_ALTER_TABLE_STMT, "AlterTableStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_ALTER_SUPER_TABLE_STMT, "AlterSuperTableStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_CREATE_USER_STMT, "CreateUserStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_ALTER_USER_STMT, "AlterUserStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_DROP_USER_STMT, "DropUserStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_USE_DATABASE_STMT, "UseDatabaseStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_CREATE_DNODE_STMT, "CreateDnodeStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_DROP_DNODE_STMT, "DropDnodeStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_ALTER_DNODE_STMT, "AlterDnodeStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_CREATE_INDEX_STMT, "CreateIndexStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_DROP_INDEX_STMT, "DropIndexStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_CREATE_QNODE_STMT, "CreateQnodeStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_DROP_QNODE_STMT, "DropQnodeStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_CREATE_SNODE_STMT, "CreateSnodeStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_DROP_SNODE_STMT, "DropSnodeStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_CREATE_MNODE_STMT, "CreateMnodeStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_DROP_MNODE_STMT, "DropMnodeStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_CREATE_TOPIC_STMT, "CreateTopicStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_DROP_TOPIC_STMT, "DropTopicStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_DROP_CGROUP_STMT, "DropConsumerGroupStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_ALTER_LOCAL_STMT, "AlterLocalStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_EXPLAIN_STMT, "ExplainStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_DESCRIBE_STMT, "DescribeStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_COMPACT_DATABASE_STMT, "CompactDatabaseStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_CREATE_STREAM_STMT, "CreateStreamStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_DROP_STREAM_STMT, "DropStreamStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PAUSE_STREAM_STMT, "PauseStreamStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_RESUME_STREAM_STMT, "ResumeStreamStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_BALANCE_VGROUP_STMT, "BalanceVgroupStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_BALANCE_VGROUP_LEADER_STMT, "BalanceVgroupLeaderStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_MERGE_VGROUP_STMT, "MergeVgroupStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_DB_ALIVE_STMT, "ShowDbAliveStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_CLUSTER_ALIVE_STMT, "ShowClusterAliveStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_REDISTRIBUTE_VGROUP_STMT, "RedistributeVgroupStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SPLIT_VGROUP_STMT, "SplitVgroupStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_GRANT_STMT, "GrantStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_REVOKE_STMT, "RevokeStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_DNODES_STMT, "ShowDnodesStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_MNODES_STMT, "ShowMnodesStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_MODULES_STMT, "ShowModulesStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_QNODES_STMT, "ShowQnodesStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_SNODES_STMT, "ShowSnodesStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_BNODES_STMT, "ShowBnodesStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_CLUSTER_STMT, "ShowClusterStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_DATABASES_STMT, "ShowDatabaseStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_FUNCTIONS_STMT, "ShowFunctionsStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_INDEXES_STMT, "ShowIndexesStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_STABLES_STMT, "ShowStablesStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_STREAMS_STMT, "ShowStreamsStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_TABLES_STMT, "ShowTablesStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_TAGS_STMT, "ShowTagsStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_USERS_STMT, "ShowUsersStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_LICENCES_STMT, "ShowGrantsStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_VGROUPS_STMT, "ShowVgroupsStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_TOPICS_STMT, "ShowTopicsStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_CONSUMERS_STMT, "ShowConsumersStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_QUERIES_STMT, "ShowQueriesStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_VARIABLES_STMT, "ShowVariablesStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_DNODE_VARIABLES_STMT, "ShowDnodeVariablesStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_TRANSACTIONS_STMT, "ShowTransactionsStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_SUBSCRIPTIONS_STMT, "ShowSubscriptionsStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_VNODES_STMT, "ShowVnodeStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_USER_PRIVILEGES_STMT, "ShowUserPrivilegesStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_CREATE_DATABASE_STMT, "ShowCreateDatabasesStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_CREATE_TABLE_STMT, "ShowCreateTablesStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_CREATE_STABLE_STMT, "ShowCreateStablesStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_CREATE_VIEW_STMT, "ShowCreateViewStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_TABLE_DISTRIBUTED_STMT, "ShowTableDistributedStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_LOCAL_VARIABLES_STMT, "ShowLocalVariablesStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_SHOW_TABLE_TAGS_STMT, "ShowTableTagsStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_DELETE_STMT, "DeleteStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_INSERT_STMT, "InsertStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_RESTORE_DNODE_STMT, "RestoreDnodeStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_RESTORE_QNODE_STMT, "RestoreQnodeStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_RESTORE_MNODE_STMT, "RestoreMnodeStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_RESTORE_VNODE_STMT, "RestoreVnodeStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_CREATE_VIEW_STMT, "CreateViewStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_DROP_VIEW_STMT, "DropViewStmt") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN_SCAN, "LogicScan") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN_JOIN, "LogicJoin") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN_AGG, "LogicAgg") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN_PROJECT, "LogicProject") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN_VNODE_MODIFY, "LogicVnodeModify") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN_EXCHANGE, "LogicExchange") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN_MERGE, "LogicMerge") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN_WINDOW, "LogicWindow") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN_FILL, "LogicFill") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN_SORT, "LogicSort") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN_PARTITION, "LogicPartition") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN_INDEF_ROWS_FUNC, "LogicIndefRowsFunc") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN_INTERP_FUNC, "LogicInterpFunc") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN_GROUP_CACHE, "LogicGroupCache") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN_DYN_QUERY_CTRL, "LogicDynamicQueryCtrl") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_SUBPLAN, "LogicSubplan") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_LOGIC_PLAN, "LogicPlan") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_TAG_SCAN, "PhysiTagScan") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_TABLE_SCAN, "PhysiTableScan") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_TABLE_SEQ_SCAN, "PhysiTableSeqScan") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_TABLE_MERGE_SCAN, "PhysiTableMergeScan") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN, "PhysiSreamScan") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_SYSTABLE_SCAN, "PhysiSystemTableScan") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_BLOCK_DIST_SCAN, "PhysiBlockDistScan") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_LAST_ROW_SCAN, "PhysiLastRowScan") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_TABLE_COUNT_SCAN, "PhysiTableCountScan") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_MERGE_EVENT, "PhysiMergeEventWindow") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_STREAM_EVENT, "PhysiStreamEventWindow") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_PROJECT, "PhysiProject") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_MERGE_JOIN, "PhysiMergeJoin") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_HASH_JOIN, "PhysiHashJoin") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_HASH_AGG, "PhysiAgg") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_EXCHANGE, "PhysiExchange") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_MERGE, "PhysiMerge") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_SORT, "PhysiSort") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_GROUP_SORT, "PhysiGroupSort") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_HASH_INTERVAL, "PhysiHashInterval") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_MERGE_ALIGNED_INTERVAL, "PhysiMergeAlignedInterval") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL, "PhysiStreamInterval") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL, "PhysiStreamFinalInterval") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL, "PhysiStreamSemiInterval") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_FILL, "PhysiFill") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_STREAM_FILL, "PhysiFill") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_MERGE_SESSION, "PhysiSessionWindow") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_STREAM_SESSION, "PhysiStreamSessionWindow") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_SESSION, "PhysiStreamSemiSessionWindow") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_SESSION, "PhysiStreamFinalSessionWindow") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_MERGE_STATE, "PhysiStateWindow") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_STREAM_STATE, "PhysiStreamStateWindow") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_PARTITION, "PhysiPartition") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_STREAM_PARTITION, "PhysiStreamPartition") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_INDEF_ROWS_FUNC, "PhysiIndefRowsFunc") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_INTERP_FUNC, "PhysiInterpFunc") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_DISPATCH, "PhysiDispatch") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_INSERT, "PhysiInsert") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_QUERY_INSERT, "PhysiQueryInsert") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_DELETE, "PhysiDelete") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_GROUP_CACHE, "PhysiGroupCache") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN_DYN_QUERY_CTRL, "PhysiDynamicQueryCtrl") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_SUBPLAN, "PhysiSubplan") - TAOS_DEFINE_NODE_NAME(QUERY_NODE_PHYSICAL_PLAN, "PhysiPlan") +SBuiltinNodeDefinition funcNodes[QUERY_NODE_END] = {}; + +static TdThreadOnce functionNodeInit = PTHREAD_ONCE_INIT; +static int32_t initNodeCode = -1; + +static void setFunc(const char* name, int32_t type, int32_t nodeSize, FExecNodeToJson toJsonFunc, + FExecJsonToNode toNodeFunc, FExecDestoryNode destoryFunc) { + funcNodes[type].name = name; + funcNodes[type].nodeSize = nodeSize; + funcNodes[type].toJsonFunc = toJsonFunc; + funcNodes[type].toNodeFunc = toNodeFunc; + funcNodes[type].destoryFunc = destoryFunc; +} + +static void doInitNodeFuncArray(); + +void nodesInit() { + taosThreadOnce(&functionNodeInit, doInitNodeFuncArray); +} + +bool funcArrayCheck(int32_t type) { + if (type < 0 || QUERY_NODE_END < (type+1)) { + nodesError("funcArrayCheck unknown type = %d", type); + return false; + } + if (initNodeCode != 0) { + nodesInit(); + } + if (!funcNodes[type].name) { + return false; + } + return true; +} + +int32_t getNodeSize(ENodeType type) { + if (!funcArrayCheck(type)) { + return 0; + } + return funcNodes[type].nodeSize; } const char* nodesNodeName(ENodeType type) { - if (nodeNames[type]) - return nodeNames[type]; - - nodesWarn("nodesNodeName unknown node = %d", type); - return "UnknownNode"; + if (!funcArrayCheck(type)) { + return NULL; + } + return funcNodes[type].name; } static int32_t nodeListToJson(SJson* pJson, const char* pName, const SNodeList* pList) { @@ -6628,657 +6491,31 @@ static int32_t jsonToInsertStmt(const SJson* pJson, void* pObj) { return code; } -static int32_t specificNodeToJson(const void* pObj, SJson* pJson) { - switch (nodeType(pObj)) { - case QUERY_NODE_COLUMN: - return columnNodeToJson(pObj, pJson); - case QUERY_NODE_VALUE: - return valueNodeToJson(pObj, pJson); - case QUERY_NODE_OPERATOR: - return operatorNodeToJson(pObj, pJson); - case QUERY_NODE_LOGIC_CONDITION: - return logicConditionNodeToJson(pObj, pJson); - case QUERY_NODE_FUNCTION: - return functionNodeToJson(pObj, pJson); - case QUERY_NODE_REAL_TABLE: - return realTableNodeToJson(pObj, pJson); - case QUERY_NODE_TEMP_TABLE: - return tempTableNodeToJson(pObj, pJson); - case QUERY_NODE_JOIN_TABLE: - return joinTableNodeToJson(pObj, pJson); - case QUERY_NODE_GROUPING_SET: - return groupingSetNodeToJson(pObj, pJson); - case QUERY_NODE_ORDER_BY_EXPR: - return orderByExprNodeToJson(pObj, pJson); - case QUERY_NODE_LIMIT: - return limitNodeToJson(pObj, pJson); - case QUERY_NODE_STATE_WINDOW: - return stateWindowNodeToJson(pObj, pJson); - case QUERY_NODE_SESSION_WINDOW: - return sessionWindowNodeToJson(pObj, pJson); - case QUERY_NODE_INTERVAL_WINDOW: - return intervalWindowNodeToJson(pObj, pJson); - case QUERY_NODE_NODE_LIST: - return nodeListNodeToJson(pObj, pJson); - case QUERY_NODE_FILL: - return fillNodeToJson(pObj, pJson); - case QUERY_NODE_RAW_EXPR: - break; - case QUERY_NODE_TARGET: - return targetNodeToJson(pObj, pJson); - case QUERY_NODE_DATABLOCK_DESC: - return dataBlockDescNodeToJson(pObj, pJson); - case QUERY_NODE_SLOT_DESC: - return slotDescNodeToJson(pObj, pJson); - case QUERY_NODE_COLUMN_DEF: - return columnDefNodeToJson(pObj, pJson); - case QUERY_NODE_DOWNSTREAM_SOURCE: - return downstreamSourceNodeToJson(pObj, pJson); - case QUERY_NODE_DATABASE_OPTIONS: - return databaseOptionsToJson(pObj, pJson); - case QUERY_NODE_TABLE_OPTIONS: - return tableOptionsToJson(pObj, pJson); - case QUERY_NODE_INDEX_OPTIONS: - return indexOptionsToJson(pObj, pJson); - case QUERY_NODE_EXPLAIN_OPTIONS: - return explainOptionsToJson(pObj, pJson); - case QUERY_NODE_STREAM_OPTIONS: - return streamOptionsToJson(pObj, pJson); - case QUERY_NODE_LEFT_VALUE: - return TSDB_CODE_SUCCESS; // SLeftValueNode has no fields to serialize. - case QUERY_NODE_WHEN_THEN: - return whenThenNodeToJson(pObj, pJson); - case QUERY_NODE_CASE_WHEN: - return caseWhenNodeToJson(pObj, pJson); - case QUERY_NODE_EVENT_WINDOW: - return eventWindowNodeToJson(pObj, pJson); - case QUERY_NODE_SET_OPERATOR: - return setOperatorToJson(pObj, pJson); - case QUERY_NODE_SELECT_STMT: - return selectStmtToJson(pObj, pJson); - case QUERY_NODE_VNODE_MODIFY_STMT: - return vnodeModifyStmtToJson(pObj, pJson); - case QUERY_NODE_CREATE_DATABASE_STMT: - return createDatabaseStmtToJson(pObj, pJson); - case QUERY_NODE_ALTER_DATABASE_STMT: - return alterDatabaseStmtToJson(pObj, pJson); - case QUERY_NODE_TRIM_DATABASE_STMT: - return trimDatabaseStmtToJson(pObj, pJson); - case QUERY_NODE_CREATE_TABLE_STMT: - return createTableStmtToJson(pObj, pJson); - case QUERY_NODE_CREATE_SUBTABLE_CLAUSE: - return createSubTableClauseToJson(pObj, pJson); - case QUERY_NODE_CREATE_MULTI_TABLES_STMT: - return createMultiTablesStmtToJson(pObj, pJson); - case QUERY_NODE_DROP_TABLE_CLAUSE: - return dropTableClauseToJson(pObj, pJson); - case QUERY_NODE_DROP_TABLE_STMT: - return dropTableStmtToJson(pObj, pJson); - case QUERY_NODE_DROP_SUPER_TABLE_STMT: - return dropStableStmtToJson(pObj, pJson); - case QUERY_NODE_ALTER_TABLE_STMT: - return alterTableStmtToJson(pObj, pJson); - case QUERY_NODE_ALTER_SUPER_TABLE_STMT: - return alterStableStmtToJson(pObj, pJson); - case QUERY_NODE_CREATE_USER_STMT: - return createUserStmtToJson(pObj, pJson); - case QUERY_NODE_ALTER_USER_STMT: - return alterUserStmtToJson(pObj, pJson); - case QUERY_NODE_DROP_USER_STMT: - return dropUserStmtToJson(pObj, pJson); - case QUERY_NODE_USE_DATABASE_STMT: - return useDatabaseStmtToJson(pObj, pJson); - case QUERY_NODE_CREATE_DNODE_STMT: - return createDnodeStmtToJson(pObj, pJson); - case QUERY_NODE_DROP_DNODE_STMT: - return dropDnodeStmtToJson(pObj, pJson); - case QUERY_NODE_ALTER_DNODE_STMT: - return alterDnodeStmtToJson(pObj, pJson); - case QUERY_NODE_CREATE_INDEX_STMT: - return createIndexStmtToJson(pObj, pJson); - case QUERY_NODE_DROP_INDEX_STMT: - return dropIndexStmtToJson(pObj, pJson); - case QUERY_NODE_CREATE_QNODE_STMT: - return createQnodeStmtToJson(pObj, pJson); - case QUERY_NODE_DROP_QNODE_STMT: - return dropQnodeStmtToJson(pObj, pJson); - case QUERY_NODE_CREATE_SNODE_STMT: - return createSnodeStmtToJson(pObj, pJson); - case QUERY_NODE_DROP_SNODE_STMT: - return dropSnodeStmtToJson(pObj, pJson); - case QUERY_NODE_CREATE_MNODE_STMT: - return createMnodeStmtToJson(pObj, pJson); - case QUERY_NODE_DROP_MNODE_STMT: - return dropMnodeStmtToJson(pObj, pJson); - case QUERY_NODE_CREATE_TOPIC_STMT: - return createTopicStmtToJson(pObj, pJson); - case QUERY_NODE_DROP_TOPIC_STMT: - return dropTopicStmtToJson(pObj, pJson); - case QUERY_NODE_DROP_CGROUP_STMT: - return dropConsumerGroupStmtToJson(pObj, pJson); - case QUERY_NODE_ALTER_LOCAL_STMT: - return alterLocalStmtToJson(pObj, pJson); - case QUERY_NODE_EXPLAIN_STMT: - return explainStmtToJson(pObj, pJson); - case QUERY_NODE_DESCRIBE_STMT: - return describeStmtToJson(pObj, pJson); - case QUERY_NODE_COMPACT_DATABASE_STMT: - return compactDatabaseStmtToJson(pObj, pJson); - case QUERY_NODE_CREATE_STREAM_STMT: - return createStreamStmtToJson(pObj, pJson); - case QUERY_NODE_DROP_STREAM_STMT: - return dropStreamStmtToJson(pObj, pJson); - case QUERY_NODE_BALANCE_VGROUP_STMT: - return TSDB_CODE_SUCCESS; // SBalanceVgroupStmt has no fields to serialize. - case QUERY_NODE_BALANCE_VGROUP_LEADER_STMT: - return TSDB_CODE_SUCCESS; // SBalanceVgroupLeaderStmt has no fields to serialize. - case QUERY_NODE_MERGE_VGROUP_STMT: - return mergeVgroupStmtToJson(pObj, pJson); - case QUERY_NODE_REDISTRIBUTE_VGROUP_STMT: - return redistributeVgroupStmtToJson(pObj, pJson); - case QUERY_NODE_SPLIT_VGROUP_STMT: - return splitVgroupStmtToJson(pObj, pJson); - case QUERY_NODE_GRANT_STMT: - return grantStmtToJson(pObj, pJson); - case QUERY_NODE_REVOKE_STMT: - return revokeStmtToJson(pObj, pJson); - case QUERY_NODE_SHOW_DNODES_STMT: - return showDnodesStmtToJson(pObj, pJson); - case QUERY_NODE_SHOW_MNODES_STMT: - return showMnodesStmtToJson(pObj, pJson); - case QUERY_NODE_SHOW_QNODES_STMT: - return showQnodesStmtToJson(pObj, pJson); - case QUERY_NODE_SHOW_CLUSTER_STMT: - return showClusterStmtToJson(pObj, pJson); - case QUERY_NODE_SHOW_DATABASES_STMT: - return showDatabasesStmtToJson(pObj, pJson); - case QUERY_NODE_SHOW_FUNCTIONS_STMT: - return showFunctionsStmtToJson(pObj, pJson); - case QUERY_NODE_SHOW_INDEXES_STMT: - return showIndexesStmtToJson(pObj, pJson); - case QUERY_NODE_SHOW_STABLES_STMT: - return showStablesStmtToJson(pObj, pJson); - case QUERY_NODE_SHOW_STREAMS_STMT: - return showStreamsStmtToJson(pObj, pJson); - case QUERY_NODE_SHOW_TABLES_STMT: - return showTablesStmtToJson(pObj, pJson); - case QUERY_NODE_SHOW_TAGS_STMT: - return showTagsStmtToJson(pObj, pJson); - case QUERY_NODE_SHOW_USERS_STMT: - return showUsersStmtToJson(pObj, pJson); - case QUERY_NODE_SHOW_VGROUPS_STMT: - return showVgroupsStmtToJson(pObj, pJson); - case QUERY_NODE_SHOW_CONSUMERS_STMT: - return showConsumersStmtToJson(pObj, pJson); - case QUERY_NODE_SHOW_VARIABLES_STMT: - return showVariablesStmtToJson(pObj, pJson); - case QUERY_NODE_SHOW_DNODE_VARIABLES_STMT: - return showDnodeVariablesStmtToJson(pObj, pJson); - case QUERY_NODE_SHOW_TRANSACTIONS_STMT: - return showTransactionsStmtToJson(pObj, pJson); - case QUERY_NODE_SHOW_SUBSCRIPTIONS_STMT: - return showSubscriptionsStmtToJson(pObj, pJson); - case QUERY_NODE_SHOW_VNODES_STMT: - return showVnodesStmtToJson(pObj, pJson); - case QUERY_NODE_SHOW_USER_PRIVILEGES_STMT: - return showUserPrivilegesStmtToJson(pObj, pJson); - case QUERY_NODE_SHOW_CREATE_DATABASE_STMT: - return showCreateDatabaseStmtToJson(pObj, pJson); - case QUERY_NODE_SHOW_CREATE_TABLE_STMT: - return showCreateTableStmtToJson(pObj, pJson); - case QUERY_NODE_SHOW_CREATE_STABLE_STMT: - return showCreateStableStmtToJson(pObj, pJson); - case QUERY_NODE_SHOW_CREATE_VIEW_STMT: - return showCreateViewStmtToJson(pObj, pJson); - case QUERY_NODE_SHOW_TABLE_DISTRIBUTED_STMT: - return showTableDistributedStmtToJson(pObj, pJson); - case QUERY_NODE_SHOW_LOCAL_VARIABLES_STMT: - return showLocalVariablesStmtToJson(pObj, pJson); - case QUERY_NODE_SHOW_TABLE_TAGS_STMT: - return showTableTagsStmtToJson(pObj, pJson); - case QUERY_NODE_DELETE_STMT: - return deleteStmtToJson(pObj, pJson); - case QUERY_NODE_INSERT_STMT: - return insertStmtToJson(pObj, pJson); - case QUERY_NODE_LOGIC_PLAN_SCAN: - return logicScanNodeToJson(pObj, pJson); - case QUERY_NODE_LOGIC_PLAN_JOIN: - return logicJoinNodeToJson(pObj, pJson); - case QUERY_NODE_LOGIC_PLAN_AGG: - return logicAggNodeToJson(pObj, pJson); - case QUERY_NODE_LOGIC_PLAN_PROJECT: - return logicProjectNodeToJson(pObj, pJson); - case QUERY_NODE_LOGIC_PLAN_VNODE_MODIFY: - return logicVnodeModifyNodeToJson(pObj, pJson); - case QUERY_NODE_LOGIC_PLAN_EXCHANGE: - return logicExchangeNodeToJson(pObj, pJson); - case QUERY_NODE_LOGIC_PLAN_MERGE: - return logicMergeNodeToJson(pObj, pJson); - case QUERY_NODE_LOGIC_PLAN_WINDOW: - return logicWindowNodeToJson(pObj, pJson); - case QUERY_NODE_LOGIC_PLAN_FILL: - return logicFillNodeToJson(pObj, pJson); - case QUERY_NODE_LOGIC_PLAN_SORT: - return logicSortNodeToJson(pObj, pJson); - case QUERY_NODE_LOGIC_PLAN_PARTITION: - return logicPartitionNodeToJson(pObj, pJson); - case QUERY_NODE_LOGIC_PLAN_INDEF_ROWS_FUNC: - return logicIndefRowsFuncNodeToJson(pObj, pJson); - case QUERY_NODE_LOGIC_PLAN_INTERP_FUNC: - return logicInterpFuncNodeToJson(pObj, pJson); - case QUERY_NODE_LOGIC_PLAN_GROUP_CACHE: - return logicGroupCacheNodeToJson(pObj, pJson); - case QUERY_NODE_LOGIC_PLAN_DYN_QUERY_CTRL: - return logicDynQueryCtrlNodeToJson(pObj, pJson); - case QUERY_NODE_LOGIC_SUBPLAN: - return logicSubplanToJson(pObj, pJson); - case QUERY_NODE_LOGIC_PLAN: - return logicPlanToJson(pObj, pJson); - case QUERY_NODE_PHYSICAL_PLAN_TAG_SCAN: - return physiTagScanNodeToJson(pObj, pJson); - case QUERY_NODE_PHYSICAL_PLAN_BLOCK_DIST_SCAN: - return physiScanNodeToJson(pObj, pJson); - case QUERY_NODE_PHYSICAL_PLAN_LAST_ROW_SCAN: - case QUERY_NODE_PHYSICAL_PLAN_TABLE_COUNT_SCAN: - return physiLastRowScanNodeToJson(pObj, pJson); - case QUERY_NODE_PHYSICAL_PLAN_TABLE_SCAN: - case QUERY_NODE_PHYSICAL_PLAN_TABLE_MERGE_SCAN: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN: - return physiTableScanNodeToJson(pObj, pJson); - case QUERY_NODE_PHYSICAL_PLAN_SYSTABLE_SCAN: - return physiSysTableScanNodeToJson(pObj, pJson); - case QUERY_NODE_PHYSICAL_PLAN_PROJECT: - return physiProjectNodeToJson(pObj, pJson); - case QUERY_NODE_PHYSICAL_PLAN_MERGE_JOIN: - return physiMergeJoinNodeToJson(pObj, pJson); - case QUERY_NODE_PHYSICAL_PLAN_HASH_JOIN: - return physiHashJoinNodeToJson(pObj, pJson); - case QUERY_NODE_PHYSICAL_PLAN_HASH_AGG: - return physiAggNodeToJson(pObj, pJson); - case QUERY_NODE_PHYSICAL_PLAN_EXCHANGE: - return physiExchangeNodeToJson(pObj, pJson); - case QUERY_NODE_PHYSICAL_PLAN_MERGE: - return physiMergeNodeToJson(pObj, pJson); - case QUERY_NODE_PHYSICAL_PLAN_SORT: - case QUERY_NODE_PHYSICAL_PLAN_GROUP_SORT: - return physiSortNodeToJson(pObj, pJson); - case QUERY_NODE_PHYSICAL_PLAN_HASH_INTERVAL: - case QUERY_NODE_PHYSICAL_PLAN_MERGE_ALIGNED_INTERVAL: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL: - return physiIntervalNodeToJson(pObj, pJson); - case QUERY_NODE_PHYSICAL_PLAN_FILL: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_FILL: - return physiFillNodeToJson(pObj, pJson); - case QUERY_NODE_PHYSICAL_PLAN_MERGE_SESSION: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_SESSION: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_SESSION: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_SESSION: - return physiSessionWindowNodeToJson(pObj, pJson); - case QUERY_NODE_PHYSICAL_PLAN_MERGE_STATE: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_STATE: - return physiStateWindowNodeToJson(pObj, pJson); - case QUERY_NODE_PHYSICAL_PLAN_MERGE_EVENT: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_EVENT: - return physiEventWindowNodeToJson(pObj, pJson); - case QUERY_NODE_PHYSICAL_PLAN_PARTITION: - return physiPartitionNodeToJson(pObj, pJson); - case QUERY_NODE_PHYSICAL_PLAN_STREAM_PARTITION: - return physiStreamPartitionNodeToJson(pObj, pJson); - case QUERY_NODE_PHYSICAL_PLAN_INDEF_ROWS_FUNC: - return physiIndefRowsFuncNodeToJson(pObj, pJson); - case QUERY_NODE_PHYSICAL_PLAN_INTERP_FUNC: - return physiInterpFuncNodeToJson(pObj, pJson); - case QUERY_NODE_PHYSICAL_PLAN_DISPATCH: - return physiDispatchNodeToJson(pObj, pJson); - case QUERY_NODE_PHYSICAL_PLAN_INSERT: - break; - case QUERY_NODE_PHYSICAL_PLAN_QUERY_INSERT: - return physiQueryInsertNodeToJson(pObj, pJson); - case QUERY_NODE_PHYSICAL_PLAN_DELETE: - return physiDeleteNodeToJson(pObj, pJson); - case QUERY_NODE_PHYSICAL_PLAN_GROUP_CACHE: - return physiGroupCacheNodeToJson(pObj, pJson); - case QUERY_NODE_PHYSICAL_PLAN_DYN_QUERY_CTRL: - return physiDynQueryCtrlNodeToJson(pObj, pJson); - case QUERY_NODE_PHYSICAL_SUBPLAN: - return subplanToJson(pObj, pJson); - case QUERY_NODE_PHYSICAL_PLAN: - return planToJson(pObj, pJson); - default: - break; +int32_t specificNodeToJson(const void* pObj, SJson* pJson) { + ENodeType type = nodeType(pObj); + if (!funcArrayCheck(type)) { + return TSDB_CODE_SUCCESS; } - nodesWarn("specificNodeToJson unknown node = %s", nodesNodeName(nodeType(pObj))); + + if (funcNodes[type].toJsonFunc) { + return funcNodes[type].toJsonFunc(pObj, pJson); + } + + nodesWarn("specificNodeToJson unknown node type = %d", type); return TSDB_CODE_SUCCESS; } -static int32_t jsonToSpecificNode(const SJson* pJson, void* pObj) { - switch (nodeType(pObj)) { - case QUERY_NODE_COLUMN: - return jsonToColumnNode(pJson, pObj); - case QUERY_NODE_VALUE: - return jsonToValueNode(pJson, pObj); - case QUERY_NODE_OPERATOR: - return jsonToOperatorNode(pJson, pObj); - case QUERY_NODE_LOGIC_CONDITION: - return jsonToLogicConditionNode(pJson, pObj); - case QUERY_NODE_FUNCTION: - return jsonToFunctionNode(pJson, pObj); - case QUERY_NODE_REAL_TABLE: - return jsonToRealTableNode(pJson, pObj); - case QUERY_NODE_TEMP_TABLE: - return jsonToTempTableNode(pJson, pObj); - case QUERY_NODE_JOIN_TABLE: - return jsonToJoinTableNode(pJson, pObj); - case QUERY_NODE_GROUPING_SET: - return jsonToGroupingSetNode(pJson, pObj); - case QUERY_NODE_ORDER_BY_EXPR: - return jsonToOrderByExprNode(pJson, pObj); - case QUERY_NODE_LIMIT: - return jsonToLimitNode(pJson, pObj); - case QUERY_NODE_STATE_WINDOW: - return jsonToStateWindowNode(pJson, pObj); - case QUERY_NODE_SESSION_WINDOW: - return jsonToSessionWindowNode(pJson, pObj); - case QUERY_NODE_INTERVAL_WINDOW: - return jsonToIntervalWindowNode(pJson, pObj); - case QUERY_NODE_NODE_LIST: - return jsonToNodeListNode(pJson, pObj); - case QUERY_NODE_FILL: - return jsonToFillNode(pJson, pObj); - case QUERY_NODE_TARGET: - return jsonToTargetNode(pJson, pObj); - case QUERY_NODE_DATABLOCK_DESC: - return jsonToDataBlockDescNode(pJson, pObj); - case QUERY_NODE_SLOT_DESC: - return jsonToSlotDescNode(pJson, pObj); - case QUERY_NODE_COLUMN_DEF: - return jsonToColumnDefNode(pJson, pObj); - case QUERY_NODE_DOWNSTREAM_SOURCE: - return jsonToDownstreamSourceNode(pJson, pObj); - case QUERY_NODE_DATABASE_OPTIONS: - return jsonToDatabaseOptions(pJson, pObj); - case QUERY_NODE_TABLE_OPTIONS: - return jsonToTableOptions(pJson, pObj); - case QUERY_NODE_INDEX_OPTIONS: - return jsonToIndexOptions(pJson, pObj); - case QUERY_NODE_EXPLAIN_OPTIONS: - return jsonToExplainOptions(pJson, pObj); - case QUERY_NODE_STREAM_OPTIONS: - return jsonToStreamOptions(pJson, pObj); - case QUERY_NODE_LEFT_VALUE: - return TSDB_CODE_SUCCESS; // SLeftValueNode has no fields to deserialize. - case QUERY_NODE_WHEN_THEN: - return jsonToWhenThenNode(pJson, pObj); - case QUERY_NODE_CASE_WHEN: - return jsonToCaseWhenNode(pJson, pObj); - case QUERY_NODE_EVENT_WINDOW: - return jsonToEventWindowNode(pJson, pObj); - case QUERY_NODE_SET_OPERATOR: - return jsonToSetOperator(pJson, pObj); - case QUERY_NODE_SELECT_STMT: - return jsonToSelectStmt(pJson, pObj); - case QUERY_NODE_VNODE_MODIFY_STMT: - return jsonToVnodeModifyStmt(pJson, pObj); - case QUERY_NODE_CREATE_DATABASE_STMT: - return jsonToCreateDatabaseStmt(pJson, pObj); - case QUERY_NODE_ALTER_DATABASE_STMT: - return jsonToAlterDatabaseStmt(pJson, pObj); - case QUERY_NODE_TRIM_DATABASE_STMT: - return jsonToTrimDatabaseStmt(pJson, pObj); - case QUERY_NODE_CREATE_TABLE_STMT: - return jsonToCreateTableStmt(pJson, pObj); - case QUERY_NODE_CREATE_SUBTABLE_CLAUSE: - return jsonToCreateSubTableClause(pJson, pObj); - case QUERY_NODE_CREATE_MULTI_TABLES_STMT: - return jsonToCreateMultiTablesStmt(pJson, pObj); - case QUERY_NODE_DROP_TABLE_CLAUSE: - return jsonToDropTableClause(pJson, pObj); - case QUERY_NODE_DROP_TABLE_STMT: - return jsonToDropTableStmt(pJson, pObj); - case QUERY_NODE_DROP_SUPER_TABLE_STMT: - return jsonToDropStableStmt(pJson, pObj); - case QUERY_NODE_ALTER_TABLE_STMT: - return jsonToAlterTableStmt(pJson, pObj); - case QUERY_NODE_ALTER_SUPER_TABLE_STMT: - return jsonToAlterStableStmt(pJson, pObj); - case QUERY_NODE_CREATE_USER_STMT: - return jsonToCreateUserStmt(pJson, pObj); - case QUERY_NODE_ALTER_USER_STMT: - return jsonToAlterUserStmt(pJson, pObj); - case QUERY_NODE_DROP_USER_STMT: - return jsonToDropUserStmt(pJson, pObj); - case QUERY_NODE_USE_DATABASE_STMT: - return jsonToUseDatabaseStmt(pJson, pObj); - case QUERY_NODE_CREATE_DNODE_STMT: - return jsonToCreateDnodeStmt(pJson, pObj); - case QUERY_NODE_DROP_DNODE_STMT: - return jsonToDropDnodeStmt(pJson, pObj); - case QUERY_NODE_ALTER_DNODE_STMT: - return jsonToAlterDnodeStmt(pJson, pObj); - case QUERY_NODE_CREATE_INDEX_STMT: - return jsonToCreateIndexStmt(pJson, pObj); - case QUERY_NODE_DROP_INDEX_STMT: - return jsonToDropIndexStmt(pJson, pObj); - case QUERY_NODE_CREATE_QNODE_STMT: - return jsonToCreateQnodeStmt(pJson, pObj); - case QUERY_NODE_DROP_QNODE_STMT: - return jsonToDropQnodeStmt(pJson, pObj); - case QUERY_NODE_CREATE_SNODE_STMT: - return jsonToCreateSnodeStmt(pJson, pObj); - case QUERY_NODE_DROP_SNODE_STMT: - return jsonToDropSnodeStmt(pJson, pObj); - case QUERY_NODE_CREATE_MNODE_STMT: - return jsonToCreateMnodeStmt(pJson, pObj); - case QUERY_NODE_DROP_MNODE_STMT: - return jsonToDropMnodeStmt(pJson, pObj); - case QUERY_NODE_CREATE_TOPIC_STMT: - return jsonToCreateTopicStmt(pJson, pObj); - case QUERY_NODE_DROP_TOPIC_STMT: - return jsonToDropTopicStmt(pJson, pObj); - case QUERY_NODE_DROP_CGROUP_STMT: - return jsonToDropConsumerGroupStmt(pJson, pObj); - case QUERY_NODE_ALTER_LOCAL_STMT: - return jsonToAlterLocalStmt(pJson, pObj); - case QUERY_NODE_EXPLAIN_STMT: - return jsonToExplainStmt(pJson, pObj); - case QUERY_NODE_DESCRIBE_STMT: - return jsonToDescribeStmt(pJson, pObj); - case QUERY_NODE_COMPACT_DATABASE_STMT: - return jsonToCompactDatabaseStmt(pJson, pObj); - case QUERY_NODE_CREATE_STREAM_STMT: - return jsonToCreateStreamStmt(pJson, pObj); - case QUERY_NODE_DROP_STREAM_STMT: - return jsonToDropStreamStmt(pJson, pObj); - case QUERY_NODE_BALANCE_VGROUP_STMT: - return TSDB_CODE_SUCCESS; // SBalanceVgroupStmt has no fields to deserialize. - case QUERY_NODE_BALANCE_VGROUP_LEADER_STMT: - return TSDB_CODE_SUCCESS; // SBalanceVgroupLeaderStmt has no fields to deserialize. - case QUERY_NODE_MERGE_VGROUP_STMT: - return jsonToMergeVgroupStmt(pJson, pObj); - case QUERY_NODE_REDISTRIBUTE_VGROUP_STMT: - return jsonToRedistributeVgroupStmt(pJson, pObj); - case QUERY_NODE_SPLIT_VGROUP_STMT: - return jsonToSplitVgroupStmt(pJson, pObj); - case QUERY_NODE_GRANT_STMT: - return jsonToGrantStmt(pJson, pObj); - case QUERY_NODE_REVOKE_STMT: - return jsonToRevokeStmt(pJson, pObj); - case QUERY_NODE_SHOW_DNODES_STMT: - return jsonToShowDnodesStmt(pJson, pObj); - case QUERY_NODE_SHOW_MNODES_STMT: - return jsonToShowMnodesStmt(pJson, pObj); - case QUERY_NODE_SHOW_QNODES_STMT: - return jsonToShowQnodesStmt(pJson, pObj); - case QUERY_NODE_SHOW_CLUSTER_STMT: - return jsonToShowClusterStmt(pJson, pObj); - case QUERY_NODE_SHOW_DATABASES_STMT: - return jsonToShowDatabasesStmt(pJson, pObj); - case QUERY_NODE_SHOW_FUNCTIONS_STMT: - return jsonToShowFunctionsStmt(pJson, pObj); - case QUERY_NODE_SHOW_INDEXES_STMT: - return jsonToShowIndexesStmt(pJson, pObj); - case QUERY_NODE_SHOW_STABLES_STMT: - return jsonToShowStablesStmt(pJson, pObj); - case QUERY_NODE_SHOW_STREAMS_STMT: - return jsonToShowStreamsStmt(pJson, pObj); - case QUERY_NODE_SHOW_TABLES_STMT: - return jsonToShowTablesStmt(pJson, pObj); - case QUERY_NODE_SHOW_TAGS_STMT: - return jsonToShowTagsStmt(pJson, pObj); - case QUERY_NODE_SHOW_USERS_STMT: - return jsonToShowUsersStmt(pJson, pObj); - case QUERY_NODE_SHOW_VGROUPS_STMT: - return jsonToShowVgroupsStmt(pJson, pObj); - case QUERY_NODE_SHOW_CONSUMERS_STMT: - return jsonToShowConsumersStmt(pJson, pObj); - case QUERY_NODE_SHOW_VARIABLES_STMT: - return jsonToShowVariablesStmt(pJson, pObj); - case QUERY_NODE_SHOW_DNODE_VARIABLES_STMT: - return jsonToShowDnodeVariablesStmt(pJson, pObj); - case QUERY_NODE_SHOW_TRANSACTIONS_STMT: - return jsonToShowTransactionsStmt(pJson, pObj); - case QUERY_NODE_SHOW_SUBSCRIPTIONS_STMT: - return jsonToShowSubscriptionsStmt(pJson, pObj); - case QUERY_NODE_SHOW_VNODES_STMT: - return jsonToShowVnodesStmt(pJson, pObj); - case QUERY_NODE_SHOW_USER_PRIVILEGES_STMT: - return jsonToShowUserPrivilegesStmt(pJson, pObj); - case QUERY_NODE_SHOW_CREATE_DATABASE_STMT: - return jsonToShowCreateDatabaseStmt(pJson, pObj); - case QUERY_NODE_SHOW_CREATE_TABLE_STMT: - return jsonToShowCreateTableStmt(pJson, pObj); - case QUERY_NODE_SHOW_CREATE_STABLE_STMT: - return jsonToShowCreateStableStmt(pJson, pObj); - case QUERY_NODE_SHOW_CREATE_VIEW_STMT: - return jsonToShowCreateViewStmt(pJson, pObj); - case QUERY_NODE_SHOW_TABLE_DISTRIBUTED_STMT: - return jsonToShowTableDistributedStmt(pJson, pObj); - case QUERY_NODE_SHOW_LOCAL_VARIABLES_STMT: - return jsonToShowLocalVariablesStmt(pJson, pObj); - case QUERY_NODE_SHOW_TABLE_TAGS_STMT: - return jsonToShowTableTagsStmt(pJson, pObj); - case QUERY_NODE_DELETE_STMT: - return jsonToDeleteStmt(pJson, pObj); - case QUERY_NODE_INSERT_STMT: - return jsonToInsertStmt(pJson, pObj); - case QUERY_NODE_RESTORE_DNODE_STMT: - return jsonToRestoreDnodeStmt(pJson, pObj); - case QUERY_NODE_RESTORE_QNODE_STMT: - return jsonToRestoreQnodeStmt(pJson, pObj); - case QUERY_NODE_RESTORE_MNODE_STMT: - return jsonToRestoreMnodeStmt(pJson, pObj); - case QUERY_NODE_RESTORE_VNODE_STMT: - return jsonToRestoreVnodeStmt(pJson, pObj); - case QUERY_NODE_LOGIC_PLAN_SCAN: - return jsonToLogicScanNode(pJson, pObj); - case QUERY_NODE_LOGIC_PLAN_JOIN: - return jsonToLogicJoinNode(pJson, pObj); - case QUERY_NODE_LOGIC_PLAN_AGG: - return jsonToLogicAggNode(pJson, pObj); - case QUERY_NODE_LOGIC_PLAN_PROJECT: - return jsonToLogicProjectNode(pJson, pObj); - case QUERY_NODE_LOGIC_PLAN_VNODE_MODIFY: - return jsonToLogicVnodeModifyNode(pJson, pObj); - case QUERY_NODE_LOGIC_PLAN_EXCHANGE: - return jsonToLogicExchangeNode(pJson, pObj); - case QUERY_NODE_LOGIC_PLAN_MERGE: - return jsonToLogicMergeNode(pJson, pObj); - case QUERY_NODE_LOGIC_PLAN_WINDOW: - return jsonToLogicWindowNode(pJson, pObj); - case QUERY_NODE_LOGIC_PLAN_FILL: - return jsonToLogicFillNode(pJson, pObj); - case QUERY_NODE_LOGIC_PLAN_SORT: - return jsonToLogicSortNode(pJson, pObj); - case QUERY_NODE_LOGIC_PLAN_PARTITION: - return jsonToLogicPartitionNode(pJson, pObj); - case QUERY_NODE_LOGIC_PLAN_INDEF_ROWS_FUNC: - return jsonToLogicIndefRowsFuncNode(pJson, pObj); - case QUERY_NODE_LOGIC_PLAN_INTERP_FUNC: - return jsonToLogicInterpFuncNode(pJson, pObj); - case QUERY_NODE_LOGIC_PLAN_GROUP_CACHE: - return jsonToLogicGroupCacheNode(pJson, pObj); - case QUERY_NODE_LOGIC_PLAN_DYN_QUERY_CTRL: - return jsonToLogicDynQueryCtrlNode(pJson, pObj); - case QUERY_NODE_LOGIC_SUBPLAN: - return jsonToLogicSubplan(pJson, pObj); - case QUERY_NODE_LOGIC_PLAN: - return jsonToLogicPlan(pJson, pObj); - case QUERY_NODE_PHYSICAL_PLAN_TAG_SCAN: - return jsonToPhysiTagScanNode(pJson, pObj); - case QUERY_NODE_PHYSICAL_PLAN_BLOCK_DIST_SCAN: - case QUERY_NODE_PHYSICAL_PLAN_TABLE_COUNT_SCAN: - return jsonToPhysiScanNode(pJson, pObj); - case QUERY_NODE_PHYSICAL_PLAN_LAST_ROW_SCAN: - return jsonToPhysiLastRowScanNode(pJson, pObj); - case QUERY_NODE_PHYSICAL_PLAN_TABLE_SCAN: - case QUERY_NODE_PHYSICAL_PLAN_TABLE_MERGE_SCAN: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN: - return jsonToPhysiTableScanNode(pJson, pObj); - case QUERY_NODE_PHYSICAL_PLAN_SYSTABLE_SCAN: - return jsonToPhysiSysTableScanNode(pJson, pObj); - case QUERY_NODE_PHYSICAL_PLAN_PROJECT: - return jsonToPhysiProjectNode(pJson, pObj); - case QUERY_NODE_PHYSICAL_PLAN_MERGE_JOIN: - return jsonToPhysiMergeJoinNode(pJson, pObj); - case QUERY_NODE_PHYSICAL_PLAN_HASH_JOIN: - return jsonToPhysiHashJoinNode(pJson, pObj); - case QUERY_NODE_PHYSICAL_PLAN_HASH_AGG: - return jsonToPhysiAggNode(pJson, pObj); - case QUERY_NODE_PHYSICAL_PLAN_EXCHANGE: - return jsonToPhysiExchangeNode(pJson, pObj); - case QUERY_NODE_PHYSICAL_PLAN_MERGE: - return jsonToPhysiMergeNode(pJson, pObj); - case QUERY_NODE_PHYSICAL_PLAN_SORT: - case QUERY_NODE_PHYSICAL_PLAN_GROUP_SORT: - return jsonToPhysiSortNode(pJson, pObj); - case QUERY_NODE_PHYSICAL_PLAN_HASH_INTERVAL: - case QUERY_NODE_PHYSICAL_PLAN_MERGE_ALIGNED_INTERVAL: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL: - return jsonToPhysiIntervalNode(pJson, pObj); - case QUERY_NODE_PHYSICAL_PLAN_FILL: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_FILL: - return jsonToPhysiFillNode(pJson, pObj); - case QUERY_NODE_PHYSICAL_PLAN_MERGE_SESSION: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_SESSION: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_SESSION: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_SESSION: - return jsonToPhysiSessionWindowNode(pJson, pObj); - case QUERY_NODE_PHYSICAL_PLAN_MERGE_STATE: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_STATE: - return jsonToPhysiStateWindowNode(pJson, pObj); - case QUERY_NODE_PHYSICAL_PLAN_MERGE_EVENT: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_EVENT: - return jsonToPhysiEventWindowNode(pJson, pObj); - case QUERY_NODE_PHYSICAL_PLAN_PARTITION: - return jsonToPhysiPartitionNode(pJson, pObj); - case QUERY_NODE_PHYSICAL_PLAN_STREAM_PARTITION: - return jsonToPhysiStreamPartitionNode(pJson, pObj); - case QUERY_NODE_PHYSICAL_PLAN_INDEF_ROWS_FUNC: - return jsonToPhysiIndefRowsFuncNode(pJson, pObj); - case QUERY_NODE_PHYSICAL_PLAN_INTERP_FUNC: - return jsonToPhysiInterpFuncNode(pJson, pObj); - case QUERY_NODE_PHYSICAL_PLAN_DISPATCH: - return jsonToPhysiDispatchNode(pJson, pObj); - case QUERY_NODE_PHYSICAL_PLAN_QUERY_INSERT: - return jsonToPhysiQueryInsertNode(pJson, pObj); - case QUERY_NODE_PHYSICAL_PLAN_DELETE: - return jsonToPhysiDeleteNode(pJson, pObj); - case QUERY_NODE_PHYSICAL_PLAN_GROUP_CACHE: - return jsonToPhysiGroupCacheNode(pJson, pObj); - case QUERY_NODE_PHYSICAL_PLAN_DYN_QUERY_CTRL: - return jsonToPhysiDynQueryCtrlNode(pJson, pObj); - case QUERY_NODE_PHYSICAL_SUBPLAN: - return jsonToSubplan(pJson, pObj); - case QUERY_NODE_PHYSICAL_PLAN: - return jsonToPlan(pJson, pObj); - default: - break; +int32_t jsonToSpecificNode(const SJson* pJson, void* pObj) { + ENodeType type = nodeType(pObj); + if (!funcArrayCheck(type)) { + return TSDB_CODE_SUCCESS; } - nodesWarn("jsonToSpecificNode unknown node = %s", nodesNodeName(nodeType(pObj))); + + if (funcNodes[type].toNodeFunc) { + return funcNodes[type].toNodeFunc(pJson, pObj); + } + + nodesWarn("jsonToSpecificNode unknown node type = %d", type); return TSDB_CODE_SUCCESS; } @@ -7436,3 +6673,2180 @@ int32_t nodesStringToList(const char* pStr, SNodeList** pList) { } return TSDB_CODE_SUCCESS; } + +static int32_t emptyNodeToJson(const void* pObj, SJson* pJson) { + return TSDB_CODE_SUCCESS; +} + +static int32_t emptyJsonToNode(const SJson* pJson, void* pObj) { + return TSDB_CODE_SUCCESS; +} + +static void destroyVgDataBlockArray(SArray* pArray) { + size_t size = taosArrayGetSize(pArray); + for (size_t i = 0; i < size; ++i) { + SVgDataBlocks* pVg = taosArrayGetP(pArray, i); + taosMemoryFreeClear(pVg->pData); + taosMemoryFreeClear(pVg); + } + taosArrayDestroy(pArray); +} + +static void destroyLogicNode(SLogicNode* pNode) { + nodesDestroyList(pNode->pTargets); + nodesDestroyNode(pNode->pConditions); + nodesDestroyList(pNode->pChildren); + nodesDestroyNode(pNode->pLimit); + nodesDestroyNode(pNode->pSlimit); + nodesDestroyList(pNode->pHint); +} + +void destroyPhysiNode(SNode* pInput) { + SPhysiNode* pNode = (SPhysiNode*)pInput; + nodesDestroyList(pNode->pChildren); + nodesDestroyNode(pNode->pConditions); + nodesDestroyNode((SNode*)pNode->pOutputDataBlockDesc); + nodesDestroyNode(pNode->pLimit); + nodesDestroyNode(pNode->pSlimit); +} + +void destroyExprNode(SNode* pNode) { + SExprNode* pExpr = (SExprNode*)pNode; + taosArrayDestroy(pExpr->pAssociation); +} + +void destroyDataInSmaIndex(void* pIndex) { + taosMemoryFree(((STableIndexInfo*)pIndex)->expr); +} + +void destoryXNode(SNode* pNode) {} + +void destroyColumnNode(SNode* pNode) { + destroyExprNode(pNode); +} + +void destroyValueNode(SNode* pNode) { + SValueNode* pValue = (SValueNode*)pNode; + destroyExprNode(pNode); + taosMemoryFreeClear(pValue->literal); + if (IS_VAR_DATA_TYPE(pValue->node.resType.type)) { + taosMemoryFreeClear(pValue->datum.p); + } +} + +void destroyOperatorNode(SNode* pNode) { + SOperatorNode* pOp = (SOperatorNode*)pNode; + destroyExprNode(pNode); + nodesDestroyNode(pOp->pLeft); + nodesDestroyNode(pOp->pRight); +} + +void destoryLogicConditionNode(SNode* pNode) { + destroyExprNode(pNode); + nodesDestroyList(((SLogicConditionNode*)pNode)->pParameterList); +} + +void destoryFunctionNode(SNode* pNode) { + destroyExprNode(pNode); + nodesDestroyList(((SFunctionNode*)pNode)->pParameterList); +} + +void destoryRealTableNode(SNode* pNode) { + SRealTableNode* pReal = (SRealTableNode*)pNode; + taosMemoryFreeClear(pReal->pMeta); + taosMemoryFreeClear(pReal->pVgroupList); + taosArrayDestroyEx(pReal->pSmaIndexes, destroyDataInSmaIndex); +} + +void destoryTempTableNode(SNode* pNode) { nodesDestroyNode(((STempTableNode*)pNode)->pSubquery); } + +void destoryJoinTableNode(SNode* pNode) { + SJoinTableNode* pJoin = (SJoinTableNode*)pNode; + nodesDestroyNode(pJoin->pLeft); + nodesDestroyNode(pJoin->pRight); + nodesDestroyNode(pJoin->pOnCond); +} + +void destoryGroupingSetNode(SNode* pNode) { nodesDestroyList(((SGroupingSetNode*)pNode)->pParameterList); } + +void destoryOrderByExprNode(SNode* pNode) { nodesDestroyNode(((SOrderByExprNode*)pNode)->pExpr); } + +void destoryStateWindowNode(SNode* pNode) { + SStateWindowNode* pState = (SStateWindowNode*)pNode; + nodesDestroyNode(pState->pCol); + nodesDestroyNode(pState->pExpr); +} + +void destorySessionWindowNode(SNode* pNode) { + SSessionWindowNode* pSession = (SSessionWindowNode*)pNode; + nodesDestroyNode((SNode*)pSession->pCol); + nodesDestroyNode((SNode*)pSession->pGap); +} + +void destoryIntervalWindowNode(SNode* pNode) { + SIntervalWindowNode* pJoin = (SIntervalWindowNode*)pNode; + nodesDestroyNode(pJoin->pCol); + nodesDestroyNode(pJoin->pInterval); + nodesDestroyNode(pJoin->pOffset); + nodesDestroyNode(pJoin->pSliding); + nodesDestroyNode(pJoin->pFill); +} + +void destoryNodeListNode(SNode* pNode) { nodesDestroyList(((SNodeListNode*)pNode)->pNodeList); } + +void destoryFillNode(SNode* pNode) { + SFillNode* pFill = (SFillNode*)pNode; + nodesDestroyNode(pFill->pValues); + nodesDestroyNode(pFill->pWStartTs); +} + +void destoryRawExprNode(SNode* pNode) { nodesDestroyNode(((SRawExprNode*)pNode)->pNode); } + +void destoryTargetNode(SNode* pNode) { nodesDestroyNode(((STargetNode*)pNode)->pExpr); } + +void destoryDataBlockDescNode(SNode* pNode) { nodesDestroyList(((SDataBlockDescNode*)pNode)->pSlots); } + +void destoryDatabaseOptions(SNode* pNode) { + SDatabaseOptions* pOptions = (SDatabaseOptions*)pNode; + nodesDestroyNode((SNode*)pOptions->pDaysPerFile); + nodesDestroyList(pOptions->pKeep); + nodesDestroyList(pOptions->pRetentions); +} + +void destoryTableOptions(SNode* pNode) { + STableOptions* pOptions = (STableOptions*)pNode; + nodesDestroyList(pOptions->pMaxDelay); + nodesDestroyList(pOptions->pWatermark); + nodesDestroyList(pOptions->pRollupFuncs); + nodesDestroyList(pOptions->pSma); + nodesDestroyList(pOptions->pDeleteMark); +} + +void destoryIndexOptions(SNode* pNode) { + SIndexOptions* pOptions = (SIndexOptions*)pNode; + nodesDestroyList(pOptions->pFuncs); + nodesDestroyNode(pOptions->pInterval); + nodesDestroyNode(pOptions->pOffset); + nodesDestroyNode(pOptions->pSliding); + nodesDestroyNode(pOptions->pStreamOptions); +} + +void destoryStreamOptions(SNode* pNode) { + SStreamOptions* pOptions = (SStreamOptions*)pNode; + nodesDestroyNode(pOptions->pDelay); + nodesDestroyNode(pOptions->pWatermark); + nodesDestroyNode(pOptions->pDeleteMark); +} + +void destoryWhenThenNode(SNode* pNode) { + SWhenThenNode* pWhenThen = (SWhenThenNode*)pNode; + nodesDestroyNode(pWhenThen->pWhen); + nodesDestroyNode(pWhenThen->pThen); +} + +void destoryCaseWhenNode(SNode* pNode) { + SCaseWhenNode* pCaseWhen = (SCaseWhenNode*)pNode; + nodesDestroyNode(pCaseWhen->pCase); + nodesDestroyNode(pCaseWhen->pElse); + nodesDestroyList(pCaseWhen->pWhenThenList); +} + +void destoryEventWindowNode(SNode* pNode) { + SEventWindowNode* pEvent = (SEventWindowNode*)pNode; + nodesDestroyNode(pEvent->pCol); + nodesDestroyNode(pEvent->pStartCond); + nodesDestroyNode(pEvent->pEndCond); +} + +void destoryHintNode(SNode* pNode) { + SHintNode* pHint = (SHintNode*)pNode; + taosMemoryFree(pHint->value); +} + +void destoryViewNode(SNode* pNode) { + SViewNode* pView = (SViewNode*)pNode; + taosMemoryFreeClear(pView->pMeta); + taosMemoryFreeClear(pView->pVgroupList); + taosArrayDestroyEx(pView->pSmaIndexes, destroyDataInSmaIndex); +} + +void destorySetOperator(SNode* pNode) { + SSetOperator* pStmt = (SSetOperator*)pNode; + nodesDestroyList(pStmt->pProjectionList); + nodesDestroyNode(pStmt->pLeft); + nodesDestroyNode(pStmt->pRight); + nodesDestroyList(pStmt->pOrderByList); + nodesDestroyNode(pStmt->pLimit); +} + +void destorySelectStmt(SNode* pNode) { + SSelectStmt* pStmt = (SSelectStmt*)pNode; + nodesDestroyList(pStmt->pProjectionList); + nodesDestroyNode(pStmt->pFromTable); + nodesDestroyNode(pStmt->pWhere); + nodesDestroyList(pStmt->pPartitionByList); + nodesDestroyList(pStmt->pTags); + nodesDestroyNode(pStmt->pSubtable); + nodesDestroyNode(pStmt->pWindow); + nodesDestroyList(pStmt->pGroupByList); + nodesDestroyNode(pStmt->pHaving); + nodesDestroyNode(pStmt->pRange); + nodesDestroyNode(pStmt->pEvery); + nodesDestroyNode(pStmt->pFill); + nodesDestroyList(pStmt->pOrderByList); + nodesDestroyNode((SNode*)pStmt->pLimit); + nodesDestroyNode((SNode*)pStmt->pSlimit); + nodesDestroyList(pStmt->pHint); +} + +void destoryVnodeModifyOpStmt(SNode* pNode) { + SVnodeModifyOpStmt* pStmt = (SVnodeModifyOpStmt*)pNode; + destroyVgDataBlockArray(pStmt->pDataBlocks); + taosMemoryFreeClear(pStmt->pTableMeta); + nodesDestroyNode(pStmt->pTagCond); + taosArrayDestroy(pStmt->pTableTag); + taosHashCleanup(pStmt->pVgroupsHashObj); + taosHashCleanup(pStmt->pSubTableHashObj); + taosHashCleanup(pStmt->pTableNameHashObj); + taosHashCleanup(pStmt->pDbFNameHashObj); + if (pStmt->freeHashFunc) { + pStmt->freeHashFunc(pStmt->pTableBlockHashObj); + } + if (pStmt->freeArrayFunc) { + pStmt->freeArrayFunc(pStmt->pVgDataBlocks); + } + tdDestroySVCreateTbReq(pStmt->pCreateTblReq); + taosMemoryFreeClear(pStmt->pCreateTblReq); + if (pStmt->freeStbRowsCxtFunc) { + pStmt->freeStbRowsCxtFunc(pStmt->pStbRowsCxt); + } + taosMemoryFreeClear(pStmt->pStbRowsCxt); + taosCloseFile(&pStmt->fp); +} + +void destoryCreateDatabaseStmt(SNode* pNode) { nodesDestroyNode((SNode*)((SCreateDatabaseStmt*)pNode)->pOptions); } + +void destoryAlterDatabaseStmt(SNode* pNode) { nodesDestroyNode((SNode*)((SAlterDatabaseStmt*)pNode)->pOptions); } + +void destoryCreateTableStmt(SNode* pNode) { + SCreateTableStmt* pStmt = (SCreateTableStmt*)pNode; + nodesDestroyList(pStmt->pCols); + nodesDestroyList(pStmt->pTags); + nodesDestroyNode((SNode*)pStmt->pOptions); +} + +void destoryCreateSubTableClause(SNode* pNode) { + SCreateSubTableClause* pStmt = (SCreateSubTableClause*)pNode; + nodesDestroyList(pStmt->pSpecificTags); + nodesDestroyList(pStmt->pValsOfTags); + nodesDestroyNode((SNode*)pStmt->pOptions); +} + +void destoryCreateMultiTablesStmt(SNode* pNode) { + nodesDestroyList(((SCreateMultiTablesStmt*)pNode)->pSubTables); +} + +void destoryDropTableStmt(SNode* pNode) { + nodesDestroyList(((SDropTableStmt*)pNode)->pTables); +} + +void destoryAlterTableStmt(SNode* pNode) { + SAlterTableStmt* pStmt = (SAlterTableStmt*)pNode; + nodesDestroyNode((SNode*)pStmt->pOptions); + nodesDestroyNode((SNode*)pStmt->pVal); +} + +void destoryCreateUserStmt(SNode* pNode) { + SCreateUserStmt* pStmt = (SCreateUserStmt*)pNode; + taosMemoryFree(pStmt->pIpRanges); + nodesDestroyList(pStmt->pNodeListIpRanges); +} + +void destoryAlterUserStmt(SNode* pNode) { + SAlterUserStmt* pStmt = (SAlterUserStmt*)pNode; + taosMemoryFree(pStmt->pIpRanges); + nodesDestroyList(pStmt->pNodeListIpRanges); +} + +void destoryCreateIndexStmt(SNode* pNode) { + SCreateIndexStmt* pStmt = (SCreateIndexStmt*)pNode; + nodesDestroyNode((SNode*)pStmt->pOptions); + nodesDestroyList(pStmt->pCols); + if (pStmt->pReq) { + tFreeSMCreateSmaReq(pStmt->pReq); + taosMemoryFreeClear(pStmt->pReq); + } +} + +void destoryCreateTopicStmt(SNode* pNode) { + nodesDestroyNode(((SCreateTopicStmt*)pNode)->pQuery); + nodesDestroyNode(((SCreateTopicStmt*)pNode)->pWhere); +} + +void destoryExplainStmt(SNode* pNode) { + SExplainStmt* pStmt = (SExplainStmt*)pNode; + nodesDestroyNode((SNode*)pStmt->pOptions); + nodesDestroyNode(pStmt->pQuery); +} + +void destoryDescribeStmt(SNode* pNode) { + taosMemoryFree(((SDescribeStmt*)pNode)->pMeta); +} + +void destoryCompactDatabaseStmt(SNode* pNode) { + SCompactDatabaseStmt* pStmt = (SCompactDatabaseStmt*)pNode; + nodesDestroyNode(pStmt->pStart); + nodesDestroyNode(pStmt->pEnd); +} + +void destoryCreateStreamStmt(SNode* pNode) { + SCreateStreamStmt* pStmt = (SCreateStreamStmt*)pNode; + nodesDestroyNode((SNode*)pStmt->pOptions); + nodesDestroyNode(pStmt->pQuery); + nodesDestroyList(pStmt->pTags); + nodesDestroyNode(pStmt->pSubtable); + tFreeSCMCreateStreamReq(pStmt->pReq); + taosMemoryFreeClear(pStmt->pReq); +} + +void destoryRedistributeVgroupStmt(SNode* pNode) { + nodesDestroyList(((SRedistributeVgroupStmt*)pNode)->pDnodes); +} + +void destoryGrantStmt(SNode* pNode) { + nodesDestroyNode(((SGrantStmt*)pNode)->pTagCond); +} + +void destoryRevokeStmt(SNode* pNode) { + nodesDestroyNode(((SRevokeStmt*)pNode)->pTagCond); +} + +void destoryShowStmt(SNode* pNode) { + SShowStmt* pStmt = (SShowStmt*)pNode; + nodesDestroyNode(pStmt->pDbName); + nodesDestroyNode(pStmt->pTbName); +} + +void destoryShowTableTagsStmt(SNode* pNode) { + SShowTableTagsStmt* pStmt = (SShowTableTagsStmt*)pNode; + nodesDestroyNode(pStmt->pDbName); + nodesDestroyNode(pStmt->pTbName); + nodesDestroyList(pStmt->pTags); +} + +void destoryShowDnodeVariablesStmt(SNode* pNode) { + nodesDestroyNode(((SShowDnodeVariablesStmt*)pNode)->pDnodeId); + nodesDestroyNode(((SShowDnodeVariablesStmt*)pNode)->pLikePattern); +} + +void destoryShowCreateDatabaseStmt(SNode* pNode) { + taosMemoryFreeClear(((SShowCreateDatabaseStmt*)pNode)->pCfg); +} + +void destoryShowCreateTableStmt(SNode* pNode) { + STableCfg* pCfg = (STableCfg*)(((SShowCreateTableStmt*)pNode)->pTableCfg); + taosMemoryFreeClear(pCfg); + if (NULL == pCfg) { + return; + } + taosArrayDestroy(pCfg->pFuncs); + taosMemoryFree(pCfg->pComment); + taosMemoryFree(pCfg->pSchemas); + taosMemoryFree(pCfg->pTags); + taosMemoryFree(pCfg); +} + +void destoryDeleteStmt(SNode* pNode) { + SDeleteStmt* pStmt = (SDeleteStmt*)pNode; + nodesDestroyNode(pStmt->pFromTable); + nodesDestroyNode(pStmt->pWhere); + nodesDestroyNode(pStmt->pCountFunc); + nodesDestroyNode(pStmt->pFirstFunc); + nodesDestroyNode(pStmt->pLastFunc); + nodesDestroyNode(pStmt->pTagCond); +} + +void destoryInsertStmt(SNode* pNode) { + SInsertStmt* pStmt = (SInsertStmt*)pNode; + nodesDestroyNode(pStmt->pTable); + nodesDestroyList(pStmt->pCols); + nodesDestroyNode(pStmt->pQuery); +} + +void destoryQueryNode(SNode* pNode) { + SQuery* pQuery = (SQuery*)pNode; + nodesDestroyNode(pQuery->pPrevRoot); + nodesDestroyNode(pQuery->pRoot); + nodesDestroyNode(pQuery->pPostRoot); + taosMemoryFreeClear(pQuery->pResSchema); + if (NULL != pQuery->pCmdMsg) { + taosMemoryFreeClear(pQuery->pCmdMsg->pMsg); + taosMemoryFreeClear(pQuery->pCmdMsg); + } + taosArrayDestroy(pQuery->pDbList); + taosArrayDestroy(pQuery->pTableList); + taosArrayDestroy(pQuery->pTargetTableList); + taosArrayDestroy(pQuery->pPlaceholderValues); + nodesDestroyNode(pQuery->pPrepareRoot); +} + +void destoryCreateViewStmt(SNode* pNode) { + SCreateViewStmt* pStmt = (SCreateViewStmt*)pNode; + taosMemoryFree(pStmt->pQuerySql); + tFreeSCMCreateViewReq(&pStmt->createReq); + nodesDestroyNode(pStmt->pQuery); +} + +void destoryScanLogicNode(SNode* pNode) { + SScanLogicNode* pLogicNode = (SScanLogicNode*)pNode; + destroyLogicNode((SLogicNode*)pLogicNode); + nodesDestroyList(pLogicNode->pScanCols); + nodesDestroyList(pLogicNode->pScanPseudoCols); + taosMemoryFreeClear(pLogicNode->pVgroupList); + nodesDestroyList(pLogicNode->pDynamicScanFuncs); + nodesDestroyNode(pLogicNode->pTagCond); + nodesDestroyNode(pLogicNode->pTagIndexCond); + taosArrayDestroyEx(pLogicNode->pSmaIndexes, destroyDataInSmaIndex); + nodesDestroyList(pLogicNode->pGroupTags); + nodesDestroyList(pLogicNode->pTags); + nodesDestroyNode(pLogicNode->pSubtable); +} + +void destoryJoinLogicNode(SNode* pNode) { + SJoinLogicNode* pLogicNode = (SJoinLogicNode*)pNode; + destroyLogicNode((SLogicNode*)pLogicNode); + nodesDestroyNode(pLogicNode->pPrimKeyEqCond); + nodesDestroyNode(pLogicNode->pOtherOnCond); + nodesDestroyNode(pLogicNode->pColEqCond); +} + +void destoryAggLogicNode(SNode* pNode) { + SAggLogicNode* pLogicNode = (SAggLogicNode*)pNode; + destroyLogicNode((SLogicNode*)pLogicNode); + nodesDestroyList(pLogicNode->pAggFuncs); + nodesDestroyList(pLogicNode->pGroupKeys); +} + +void destoryProjectLogicNode(SNode* pNode) { + SProjectLogicNode* pLogicNode = (SProjectLogicNode*)pNode; + destroyLogicNode((SLogicNode*)pLogicNode); + nodesDestroyList(pLogicNode->pProjections); +} + +void destoryVnodeModifyLogicNode(SNode* pNode) { + SVnodeModifyLogicNode* pLogicNode = (SVnodeModifyLogicNode*)pNode; + destroyLogicNode((SLogicNode*)pLogicNode); + destroyVgDataBlockArray(pLogicNode->pDataBlocks); + // pVgDataBlocks is weak reference + nodesDestroyNode(pLogicNode->pAffectedRows); + nodesDestroyNode(pLogicNode->pStartTs); + nodesDestroyNode(pLogicNode->pEndTs); + taosMemoryFreeClear(pLogicNode->pVgroupList); + nodesDestroyList(pLogicNode->pInsertCols); +} + +void destoryExchangeLogicNode(SNode* pNode) { + destroyLogicNode((SLogicNode*)pNode); +} + +void destoryMergeLogicNode(SNode* pNode) { + SMergeLogicNode* pLogicNode = (SMergeLogicNode*)pNode; + destroyLogicNode((SLogicNode*)pLogicNode); + nodesDestroyList(pLogicNode->pMergeKeys); + nodesDestroyList(pLogicNode->pInputs); +} + +void destoryWindowLogicNode(SNode* pNode) { + SWindowLogicNode* pLogicNode = (SWindowLogicNode*)pNode; + destroyLogicNode((SLogicNode*)pLogicNode); + nodesDestroyList(pLogicNode->pFuncs); + nodesDestroyNode(pLogicNode->pTspk); + nodesDestroyNode(pLogicNode->pTsEnd); + nodesDestroyNode(pLogicNode->pStateExpr); + nodesDestroyNode(pLogicNode->pStartCond); + nodesDestroyNode(pLogicNode->pEndCond); +} + +void destoryFillLogicNode(SNode* pNode) { + SFillLogicNode* pLogicNode = (SFillLogicNode*)pNode; + destroyLogicNode((SLogicNode*)pLogicNode); + nodesDestroyNode(pLogicNode->pWStartTs); + nodesDestroyNode(pLogicNode->pValues); + nodesDestroyList(pLogicNode->pFillExprs); + nodesDestroyList(pLogicNode->pNotFillExprs); +} + +void destorySortLogicNode(SNode* pNode) { + SSortLogicNode* pLogicNode = (SSortLogicNode*)pNode; + destroyLogicNode((SLogicNode*)pLogicNode); + nodesDestroyList(pLogicNode->pSortKeys); +} + +void destoryPartitionLogicNode(SNode* pNode) { + SPartitionLogicNode* pLogicNode = (SPartitionLogicNode*)pNode; + destroyLogicNode((SLogicNode*)pLogicNode); + nodesDestroyList(pLogicNode->pPartitionKeys); + nodesDestroyList(pLogicNode->pTags); + nodesDestroyNode(pLogicNode->pSubtable); + nodesDestroyList(pLogicNode->pAggFuncs); +} + +void destoryIndefRowsFuncLogicNode(SNode* pNode) { + SIndefRowsFuncLogicNode* pLogicNode = (SIndefRowsFuncLogicNode*)pNode; + destroyLogicNode((SLogicNode*)pLogicNode); + nodesDestroyList(pLogicNode->pFuncs); +} + +void destoryInterpFuncLogicNode(SNode* pNode) { + SInterpFuncLogicNode* pLogicNode = (SInterpFuncLogicNode*)pNode; + destroyLogicNode((SLogicNode*)pLogicNode); + nodesDestroyList(pLogicNode->pFuncs); + nodesDestroyNode(pLogicNode->pFillValues); + nodesDestroyNode(pLogicNode->pTimeSeries); +} + +void destoryGroupCacheLogicNode(SNode* pNode) { + SGroupCacheLogicNode* pLogicNode = (SGroupCacheLogicNode*)pNode; + destroyLogicNode((SLogicNode*)pLogicNode); + nodesDestroyList(pLogicNode->pGroupCols); +} + +void destoryDynQueryCtrlLogicNode(SNode* pNode) { + destroyLogicNode((SLogicNode*)pNode); +} + +void destoryLogicSubplan(SNode* pNode) { + SLogicSubplan* pSubplan = (SLogicSubplan*)pNode; + nodesDestroyList(pSubplan->pChildren); + nodesDestroyNode((SNode*)pSubplan->pNode); + nodesClearList(pSubplan->pParents); + taosMemoryFreeClear(pSubplan->pVgroupList); +} + +void destoryQueryLogicPlan(SNode* pNode) { + nodesDestroyList(((SQueryLogicPlan*)pNode)->pTopSubplans); +} + +void destroyScanPhysiNode(SNode* pInput) { + SScanPhysiNode* pNode = (SScanPhysiNode*)pInput; + destroyPhysiNode(pInput); + nodesDestroyList(pNode->pScanCols); + nodesDestroyList(pNode->pScanPseudoCols); +} + +void destoryLastRowScanPhysiNode(SNode* pNode) { + SLastRowScanPhysiNode* pPhyNode = (SLastRowScanPhysiNode*)pNode; + destroyScanPhysiNode(pNode); + nodesDestroyList(pPhyNode->pGroupTags); + nodesDestroyList(pPhyNode->pTargets); +} + +void destoryTableScanPhysiNode(SNode* pNode) { + STableScanPhysiNode* pPhyNode = (STableScanPhysiNode*)pNode; + destroyScanPhysiNode(pNode); + nodesDestroyList(pPhyNode->pDynamicScanFuncs); + nodesDestroyList(pPhyNode->pGroupTags); + nodesDestroyList(pPhyNode->pTags); + nodesDestroyNode(pPhyNode->pSubtable); +} + +void destoryProjectPhysiNode(SNode* pNode) { + SProjectPhysiNode* pPhyNode = (SProjectPhysiNode*)pNode; + destroyPhysiNode(pNode); + nodesDestroyList(pPhyNode->pProjections); +} + +void destorySortMergeJoinPhysiNode(SNode* pNode) { + SSortMergeJoinPhysiNode* pPhyNode = (SSortMergeJoinPhysiNode*)pNode; + destroyPhysiNode(pNode); + nodesDestroyNode(pPhyNode->pPrimKeyCond); + nodesDestroyNode(pPhyNode->pOtherOnCond); + nodesDestroyList(pPhyNode->pTargets); + nodesDestroyNode(pPhyNode->pColEqCond); +} + +void destoryHashJoinPhysiNode(SNode* pNode) { + SHashJoinPhysiNode* pPhyNode = (SHashJoinPhysiNode*)pNode; + destroyPhysiNode(pNode); + nodesDestroyList(pPhyNode->pOnLeft); + nodesDestroyList(pPhyNode->pOnRight); + nodesDestroyNode(pPhyNode->pFilterConditions); + nodesDestroyList(pPhyNode->pTargets); + + nodesDestroyNode(pPhyNode->pPrimKeyCond); + nodesDestroyNode(pPhyNode->pColEqCond); + nodesDestroyNode(pPhyNode->pTagEqCond); +} + +void destoryAggPhysiNode(SNode* pNode) { + SAggPhysiNode* pPhyNode = (SAggPhysiNode*)pNode; + destroyPhysiNode(pNode); + nodesDestroyList(pPhyNode->pExprs); + nodesDestroyList(pPhyNode->pAggFuncs); + nodesDestroyList(pPhyNode->pGroupKeys); +} + +void destoryExchangePhysiNode(SNode* pNode) { + SExchangePhysiNode* pPhyNode = (SExchangePhysiNode*)pNode; + destroyPhysiNode(pNode); + nodesDestroyList(pPhyNode->pSrcEndPoints); +} + +void destoryMergePhysiNode(SNode* pNode) { + SMergePhysiNode* pPhyNode = (SMergePhysiNode*)pNode; + destroyPhysiNode(pNode); + nodesDestroyList(pPhyNode->pMergeKeys); + nodesDestroyList(pPhyNode->pTargets); +} + +void destorySortPhysiNode(SNode* pNode) { + SSortPhysiNode* pPhyNode = (SSortPhysiNode*)pNode; + destroyPhysiNode(pNode); + nodesDestroyList(pPhyNode->pExprs); + nodesDestroyList(pPhyNode->pSortKeys); + nodesDestroyList(pPhyNode->pTargets); +} + +void destroyWindowPhysiNode(SNode* pInput) { + SWindowPhysiNode* pNode = (SWindowPhysiNode*)pInput; + destroyPhysiNode(pInput); + nodesDestroyList(pNode->pExprs); + nodesDestroyList(pNode->pFuncs); + nodesDestroyNode(pNode->pTspk); + nodesDestroyNode(pNode->pTsEnd); +} + +void destoryFillPhysiNode(SNode* pNode) { + SFillPhysiNode* pPhyNode = (SFillPhysiNode*)pNode; + destroyPhysiNode(pNode); + nodesDestroyList(pPhyNode->pFillExprs); + nodesDestroyList(pPhyNode->pNotFillExprs); + nodesDestroyNode(pPhyNode->pWStartTs); + nodesDestroyNode(pPhyNode->pValues); +} + +void destoryStateWindowPhysiNode(SNode* pNode) { + SStateWinodwPhysiNode* pPhyNode = (SStateWinodwPhysiNode*)pNode; + destroyWindowPhysiNode(pNode); + nodesDestroyNode(pPhyNode->pStateKey); +} + +void destoryEventWindowPhysiNode(SNode* pNode) { + SEventWinodwPhysiNode* pPhyNode = (SEventWinodwPhysiNode*)pNode; + destroyWindowPhysiNode(pNode); + nodesDestroyNode(pPhyNode->pStartCond); + nodesDestroyNode(pPhyNode->pEndCond); +} + +void destroyPartitionPhysiNode(SNode* pNode) { + SPartitionPhysiNode* pPartitionNode = (SPartitionPhysiNode*)pNode; + destroyPhysiNode(pNode); + nodesDestroyList(pPartitionNode->pExprs); + nodesDestroyList(pPartitionNode->pPartitionKeys); + nodesDestroyList(pPartitionNode->pTargets); +} + +void destoryStreamPartitionPhysiNode(SNode* pNode) { + SStreamPartitionPhysiNode* pPhyNode = (SStreamPartitionPhysiNode*)pNode; + destroyPartitionPhysiNode(pNode); + nodesDestroyList(pPhyNode->pTags); + nodesDestroyNode(pPhyNode->pSubtable); +} + +void destoryIndefRowsFuncPhysiNode(SNode* pNode) { + SIndefRowsFuncPhysiNode* pPhyNode = (SIndefRowsFuncPhysiNode*)pNode; + destroyPhysiNode(pNode); + nodesDestroyList(pPhyNode->pExprs); + nodesDestroyList(pPhyNode->pFuncs); +} + +void destoryInterpFuncPhysiNode(SNode* pNode) { + SInterpFuncPhysiNode* pPhyNode = (SInterpFuncPhysiNode*)pNode; + destroyPhysiNode(pNode); + nodesDestroyList(pPhyNode->pExprs); + nodesDestroyList(pPhyNode->pFuncs); + nodesDestroyNode(pPhyNode->pFillValues); + nodesDestroyNode(pPhyNode->pTimeSeries); +} + +void destroyDataSinkNode(SNode* pNode) { + SDataSinkNode* pDataNode = (SDataSinkNode*)pNode; + nodesDestroyNode((SNode*)pDataNode->pInputDataBlockDesc); +} + +void destoryDataInserterNode(SNode* pNode) { + SDataInserterNode* pSink = (SDataInserterNode*)pNode; + destroyDataSinkNode(pNode); + taosMemoryFreeClear(pSink->pData); +} + +void destoryQueryInserterNode(SNode* pNode) { + SQueryInserterNode* pSink = (SQueryInserterNode*)pNode; + destroyDataSinkNode(pNode); + nodesDestroyList(pSink->pCols); +} + +void destoryDataDeleterNode(SNode* pNode) { + SDataDeleterNode* pSink = (SDataDeleterNode*)pNode; + destroyDataSinkNode(pNode); + nodesDestroyNode(pSink->pAffectedRows); + nodesDestroyNode(pSink->pStartTs); + nodesDestroyNode(pSink->pEndTs); +} + +void destoryGroupCachePhysiNode(SNode* pNode) { + SGroupCachePhysiNode* pPhyNode = (SGroupCachePhysiNode*)pNode; + destroyPhysiNode(pNode); + nodesDestroyList(pPhyNode->pGroupCols); +} + +void destoryDynQueryCtrlPhysiNode(SNode* pNode) { + destroyPhysiNode(pNode); +} + +void destorySubplanNode(SNode* pNode) { + SSubplan* pSubplan = (SSubplan*)pNode; + nodesClearList(pSubplan->pChildren); + nodesDestroyNode((SNode*)pSubplan->pNode); + nodesDestroyNode((SNode*)pSubplan->pDataSink); + nodesDestroyNode((SNode*)pSubplan->pTagCond); + nodesDestroyNode((SNode*)pSubplan->pTagIndexCond); + nodesClearList(pSubplan->pParents); +} + +void destoryPlanNode(SNode* pNode) { + nodesDestroyList(((SQueryPlan*)pNode)->pSubplans); +} + +void nodesDestroyNode(SNode* pNode) { + if (NULL == pNode) { + return; + } + + int32_t index = nodeType(pNode); + if (!funcArrayCheck(index)) { + return; + } + if (funcNodes[index].destoryFunc) { + funcNodes[index].destoryFunc(pNode); + nodesFree(pNode); + return; + } + nodesError("nodesDestroyNode unknown node type = %d", nodeType(pNode)); + nodesFree(pNode); + return; +} + +// clang-format off +static void doInitNodeFuncArray() { + setFunc("Column", + QUERY_NODE_COLUMN, + sizeof(SColumnNode), + columnNodeToJson, + jsonToColumnNode, + destroyColumnNode + ); + setFunc("Value", + QUERY_NODE_VALUE, + sizeof(SValueNode), + valueNodeToJson, + jsonToValueNode, + destroyValueNode + ); + setFunc("Operator", + QUERY_NODE_OPERATOR, + sizeof(SOperatorNode), + operatorNodeToJson, + jsonToOperatorNode, + destroyOperatorNode + ); + setFunc("LogicCondition", + QUERY_NODE_LOGIC_CONDITION, + sizeof(SLogicConditionNode), + logicConditionNodeToJson, + jsonToLogicConditionNode, + destoryLogicConditionNode + ); + setFunc("Function", + QUERY_NODE_FUNCTION, + sizeof(SFunctionNode), + functionNodeToJson, + jsonToFunctionNode, + destoryFunctionNode + ); + setFunc("RealTable", + QUERY_NODE_REAL_TABLE, + sizeof(SRealTableNode), + realTableNodeToJson, + jsonToRealTableNode, + destoryRealTableNode + ); + setFunc("TempTable", + QUERY_NODE_TEMP_TABLE, + sizeof(STempTableNode), + tempTableNodeToJson, + jsonToTempTableNode, + destoryTempTableNode + ); + setFunc("JoinTable", + QUERY_NODE_JOIN_TABLE, + sizeof(SJoinTableNode), + joinTableNodeToJson, + jsonToJoinTableNode, + destoryJoinTableNode + ); + setFunc("GroupingSet", + QUERY_NODE_GROUPING_SET, + sizeof(SGroupingSetNode), + groupingSetNodeToJson, + jsonToGroupingSetNode, + destoryGroupingSetNode + ); + setFunc("OrderByExpr", + QUERY_NODE_ORDER_BY_EXPR, + sizeof(SOrderByExprNode), + orderByExprNodeToJson, + jsonToOrderByExprNode, + destoryOrderByExprNode + ); + setFunc("Limit", + QUERY_NODE_LIMIT, + sizeof(SLimitNode), + limitNodeToJson, + jsonToLimitNode, + destoryXNode + ); + setFunc("StateWindow", + QUERY_NODE_STATE_WINDOW, + sizeof(SStateWindowNode), + stateWindowNodeToJson, + jsonToStateWindowNode, + destoryStateWindowNode + ); + setFunc("SessionWinow", + QUERY_NODE_SESSION_WINDOW, + sizeof(SSessionWindowNode), + sessionWindowNodeToJson, + jsonToSessionWindowNode, + destorySessionWindowNode + ); + setFunc("IntervalWindow", + QUERY_NODE_INTERVAL_WINDOW, + sizeof(SIntervalWindowNode), + intervalWindowNodeToJson, + jsonToIntervalWindowNode, + destoryIntervalWindowNode + ); + setFunc("NodeList", + QUERY_NODE_NODE_LIST, + sizeof(SNodeListNode), + nodeListNodeToJson, + jsonToNodeListNode, + destoryNodeListNode + ); + setFunc("Fill", + QUERY_NODE_FILL, + sizeof(SFillNode), + fillNodeToJson, + jsonToFillNode, + destoryFillNode + ); + setFunc("RawExpr", + QUERY_NODE_RAW_EXPR, + sizeof(SRawExprNode), + emptyNodeToJson, + emptyJsonToNode, + destoryRawExprNode + ); + setFunc("Target", + QUERY_NODE_TARGET, + sizeof(STargetNode), + targetNodeToJson, + jsonToTargetNode, + destoryTargetNode + ); + setFunc("DataBlockDesc", + QUERY_NODE_DATABLOCK_DESC, + sizeof(SDataBlockDescNode), + dataBlockDescNodeToJson, + jsonToDataBlockDescNode, + destoryDataBlockDescNode + ); + setFunc("SlotDesc", + QUERY_NODE_SLOT_DESC, + sizeof(SSlotDescNode), + slotDescNodeToJson, + jsonToSlotDescNode, + destoryXNode + ); + setFunc("ColumnDef", + QUERY_NODE_COLUMN_DEF, + sizeof(SColumnDefNode), + columnDefNodeToJson, + jsonToColumnDefNode, + destoryXNode + ); + setFunc("DownstreamSource", + QUERY_NODE_DOWNSTREAM_SOURCE, + sizeof(SDownstreamSourceNode), + downstreamSourceNodeToJson, + jsonToDownstreamSourceNode, + destoryXNode + ); + setFunc("DatabaseOptions", + QUERY_NODE_DATABASE_OPTIONS, + sizeof(SDatabaseOptions), + databaseOptionsToJson, + jsonToDatabaseOptions, + destoryDatabaseOptions + ); + setFunc("TableOptions", + QUERY_NODE_TABLE_OPTIONS, + sizeof(STableOptions), + tableOptionsToJson, + jsonToTableOptions, + destoryTableOptions + ); + setFunc("IndexOptions", + QUERY_NODE_INDEX_OPTIONS, + sizeof(SIndexOptions), + indexOptionsToJson, + jsonToIndexOptions, + destoryIndexOptions + ); + setFunc("ExplainOptions", + QUERY_NODE_EXPLAIN_OPTIONS, + sizeof(SExplainOptions), + explainOptionsToJson, + jsonToExplainOptions, + destoryXNode + ); + setFunc("StreamOptions", + QUERY_NODE_STREAM_OPTIONS, + sizeof(SStreamOptions), + streamOptionsToJson, + jsonToStreamOptions, + destoryStreamOptions + ); + setFunc("LeftValue", + QUERY_NODE_LEFT_VALUE, + sizeof(SLeftValueNode), + emptyNodeToJson, + emptyJsonToNode, + destoryXNode + ); + setFunc("ColumnRef", + QUERY_NODE_COLUMN_REF, + sizeof(SColumnDefNode), + emptyNodeToJson, + emptyJsonToNode, + destoryXNode + ); + setFunc("WhenThen", + QUERY_NODE_WHEN_THEN, + sizeof(SWhenThenNode), + whenThenNodeToJson, + jsonToWhenThenNode, + destoryWhenThenNode + ); + setFunc("CaseWhen", + QUERY_NODE_CASE_WHEN, + sizeof(SCaseWhenNode), + caseWhenNodeToJson, + jsonToCaseWhenNode, + destoryCaseWhenNode + ); + setFunc("EventWindow", + QUERY_NODE_EVENT_WINDOW, + sizeof(SEventWindowNode), + eventWindowNodeToJson, + jsonToEventWindowNode, + destoryEventWindowNode + ); + setFunc("HintNode", + QUERY_NODE_HINT, + sizeof(SHintNode), + emptyNodeToJson, + emptyJsonToNode, + destoryHintNode + ); + setFunc("ViewNode", + QUERY_NODE_VIEW, + sizeof(SViewNode), + emptyNodeToJson, + emptyJsonToNode, + destoryViewNode + ); + setFunc("SetOperator", + QUERY_NODE_SET_OPERATOR, + sizeof(SSetOperator), + setOperatorToJson, + jsonToSetOperator, + destorySetOperator + ); + setFunc("SelectStmt", + QUERY_NODE_SELECT_STMT, + sizeof(SSelectStmt), + selectStmtToJson, + jsonToSelectStmt, + destorySelectStmt + ); + setFunc("VnodeModifyStmt", + QUERY_NODE_VNODE_MODIFY_STMT, + sizeof(SVnodeModifyOpStmt), + vnodeModifyStmtToJson, + jsonToVnodeModifyStmt, + destoryVnodeModifyOpStmt + ); + setFunc("CreateDatabaseStmt", + QUERY_NODE_CREATE_DATABASE_STMT, + sizeof(SCreateDatabaseStmt), + createDatabaseStmtToJson, + jsonToCreateDatabaseStmt, + destoryCreateDatabaseStmt + ); + setFunc("DropDatabaseStmt", + QUERY_NODE_DROP_DATABASE_STMT, + sizeof(SDropDatabaseStmt), + emptyNodeToJson, + emptyJsonToNode, + destoryXNode + ); + setFunc("AlterDatabaseStmt", + QUERY_NODE_ALTER_DATABASE_STMT, + sizeof(SAlterDatabaseStmt), + alterDatabaseStmtToJson, + jsonToAlterDatabaseStmt, + destoryAlterDatabaseStmt + ); + setFunc("FlushDatabaseStmt", + QUERY_NODE_FLUSH_DATABASE_STMT, + sizeof(SFlushDatabaseStmt), + emptyNodeToJson, + emptyJsonToNode, + destoryXNode + ); + setFunc("TrimDatabaseStmt", + QUERY_NODE_TRIM_DATABASE_STMT, + sizeof(STrimDatabaseStmt), + trimDatabaseStmtToJson, + jsonToTrimDatabaseStmt, + destoryXNode + ); + setFunc("CreateTableStmt", + QUERY_NODE_CREATE_TABLE_STMT, + sizeof(SCreateTableStmt), + createTableStmtToJson, + jsonToCreateTableStmt, + destoryCreateTableStmt + ); + setFunc("CreateSubtableClause", + QUERY_NODE_CREATE_SUBTABLE_CLAUSE, + sizeof(SCreateSubTableClause), + createSubTableClauseToJson, + jsonToCreateSubTableClause, + destoryCreateSubTableClause + ); + setFunc("CreateMultiTableStmt", + QUERY_NODE_CREATE_MULTI_TABLES_STMT, + sizeof(SCreateMultiTablesStmt), + createMultiTablesStmtToJson, + jsonToCreateMultiTablesStmt, + destoryCreateMultiTablesStmt + ); + setFunc("DropTableClause", + QUERY_NODE_DROP_TABLE_CLAUSE, + sizeof(SDropTableClause), + dropTableClauseToJson, + jsonToDropTableClause, + destoryXNode + ); + setFunc("DropTableStmt", + QUERY_NODE_DROP_TABLE_STMT, + sizeof(SDropTableStmt), + dropTableStmtToJson, + jsonToDropTableStmt, + destoryDropTableStmt + ); + setFunc("DropSuperTableStmt", + QUERY_NODE_DROP_SUPER_TABLE_STMT, + sizeof(SDropSuperTableStmt), + dropStableStmtToJson, + jsonToDropStableStmt, + destoryXNode + ); + setFunc("AlterTableStmt", + QUERY_NODE_ALTER_TABLE_STMT, + sizeof(SAlterTableStmt), + alterTableStmtToJson, + jsonToAlterTableStmt, + destoryAlterTableStmt + ); + setFunc("AlterSuperTableStmt", + QUERY_NODE_ALTER_SUPER_TABLE_STMT, + sizeof(SAlterTableStmt), + alterStableStmtToJson, + jsonToAlterStableStmt, + destoryAlterTableStmt + ); + setFunc("CreateUserStmt", + QUERY_NODE_CREATE_USER_STMT, + sizeof(SCreateUserStmt), + createUserStmtToJson, + jsonToCreateUserStmt, + destoryCreateUserStmt + ); + setFunc("AlterUserStmt", + QUERY_NODE_ALTER_USER_STMT, + sizeof(SAlterUserStmt), + alterUserStmtToJson, + jsonToAlterUserStmt, + destoryAlterUserStmt + ); + setFunc("DropUserStmt", + QUERY_NODE_DROP_USER_STMT, + sizeof(SDropUserStmt), + dropUserStmtToJson, + jsonToDropUserStmt, + destoryXNode + ); + setFunc("UseDatabaseStmt", + QUERY_NODE_USE_DATABASE_STMT, + sizeof(SUseDatabaseStmt), + useDatabaseStmtToJson, + jsonToUseDatabaseStmt, + destoryXNode + ); + setFunc("CreateDnodeStmt", + QUERY_NODE_CREATE_DNODE_STMT, + sizeof(SCreateDnodeStmt), + createDnodeStmtToJson, + jsonToCreateDnodeStmt, + destoryXNode + ); + setFunc("DropDnodeStmt", + QUERY_NODE_DROP_DNODE_STMT, + sizeof(SDropDnodeStmt), + dropDnodeStmtToJson, + jsonToDropDnodeStmt, + destoryXNode + ); + setFunc("AlterDnodeStmt", + QUERY_NODE_ALTER_DNODE_STMT, + sizeof(SAlterDnodeStmt), + alterDnodeStmtToJson, + jsonToAlterDnodeStmt, + destoryXNode + ); + setFunc("CreateIndexStmt", + QUERY_NODE_CREATE_INDEX_STMT, + sizeof(SCreateIndexStmt), + createIndexStmtToJson, + jsonToCreateIndexStmt, + destoryCreateIndexStmt + ); + setFunc("DropIndexStmt", + QUERY_NODE_DROP_INDEX_STMT, + sizeof(SDropIndexStmt), + dropIndexStmtToJson, + jsonToDropIndexStmt, + destoryXNode + ); + setFunc("CreateQnodeStmt", + QUERY_NODE_CREATE_QNODE_STMT, + sizeof(SCreateComponentNodeStmt), + createQnodeStmtToJson, + jsonToCreateQnodeStmt, + destoryXNode + ); + setFunc("DropQnodeStmt", + QUERY_NODE_DROP_QNODE_STMT, + sizeof(SDropComponentNodeStmt), + dropQnodeStmtToJson, + jsonToDropQnodeStmt, + destoryXNode + ); + setFunc("CreateBnodeStmt", + QUERY_NODE_CREATE_BNODE_STMT, + sizeof(SCreateComponentNodeStmt), + emptyNodeToJson, + emptyJsonToNode, + destoryXNode + ); + setFunc("DropBnodeStmt", + QUERY_NODE_DROP_BNODE_STMT, + sizeof(SDropComponentNodeStmt), + emptyNodeToJson, + emptyJsonToNode, + destoryXNode + ); + setFunc("CreateSnodeStmt", + QUERY_NODE_CREATE_SNODE_STMT, + sizeof(SCreateComponentNodeStmt), + createSnodeStmtToJson, + jsonToCreateSnodeStmt, + destoryXNode + ); + setFunc("DropSnodeStmt", + QUERY_NODE_DROP_SNODE_STMT, + sizeof(SDropComponentNodeStmt), + dropSnodeStmtToJson, + jsonToDropSnodeStmt, + destoryXNode + ); + setFunc("CreateMnodeStmt", + QUERY_NODE_CREATE_MNODE_STMT, + sizeof(SCreateComponentNodeStmt), + createMnodeStmtToJson, + jsonToCreateMnodeStmt, + destoryXNode + ); + setFunc("DropMnodeStmt", + QUERY_NODE_DROP_MNODE_STMT, + sizeof(SDropComponentNodeStmt), + dropMnodeStmtToJson, + jsonToDropMnodeStmt, + destoryXNode + ); + setFunc("CreateTopicStmt", + QUERY_NODE_CREATE_TOPIC_STMT, + sizeof(SCreateTopicStmt), + createTopicStmtToJson, + jsonToCreateTopicStmt, + destoryCreateTopicStmt + ); + setFunc("DropTopicStmt", + QUERY_NODE_DROP_TOPIC_STMT, + sizeof(SDropTopicStmt), + dropTopicStmtToJson, + jsonToDropTopicStmt, + destoryXNode + ); + setFunc("DropConsumerGroupStmt", + QUERY_NODE_DROP_CGROUP_STMT, + sizeof(SDropCGroupStmt), + dropConsumerGroupStmtToJson, + jsonToDropConsumerGroupStmt, + destoryXNode + ); + setFunc("AlterLocalStmt", + QUERY_NODE_ALTER_LOCAL_STMT, + sizeof(SAlterLocalStmt), + alterLocalStmtToJson, + jsonToAlterLocalStmt, + destoryXNode + ); + setFunc("ExplainStmt", + QUERY_NODE_EXPLAIN_STMT, + sizeof(SExplainStmt), + explainStmtToJson, + jsonToExplainStmt, + destoryExplainStmt + ); + setFunc("DescribeStmt", + QUERY_NODE_DESCRIBE_STMT, + sizeof(SDescribeStmt), + describeStmtToJson, + jsonToDescribeStmt, + destoryDescribeStmt + ); + setFunc("QueryCacheStmt", + QUERY_NODE_RESET_QUERY_CACHE_STMT, + sizeof(SNode), + emptyNodeToJson, + emptyJsonToNode, + destoryXNode + ); + setFunc("CompactDatabaseStmt", + QUERY_NODE_COMPACT_DATABASE_STMT, + sizeof(SCompactDatabaseStmt), + compactDatabaseStmtToJson, + jsonToCompactDatabaseStmt, + destoryCompactDatabaseStmt + ); + setFunc("CreateFunctionStmt", + QUERY_NODE_CREATE_FUNCTION_STMT, + sizeof(SCreateFunctionStmt), + emptyNodeToJson, + emptyJsonToNode, + destoryXNode + ); + setFunc("DropFunctionStmt", + QUERY_NODE_DROP_FUNCTION_STMT, + sizeof(SDropFunctionStmt), + emptyNodeToJson, + emptyJsonToNode, + destoryXNode + ); + setFunc("CreateStreamStmt", + QUERY_NODE_CREATE_STREAM_STMT, + sizeof(SCreateStreamStmt), + createStreamStmtToJson, + jsonToCreateStreamStmt, + destoryCreateStreamStmt + ); + setFunc("DropStreamStmt", + QUERY_NODE_DROP_STREAM_STMT, + sizeof(SDropStreamStmt), + dropStreamStmtToJson, + jsonToDropStreamStmt, + destoryXNode + ); + setFunc("PauseStreamStmt", + QUERY_NODE_PAUSE_STREAM_STMT, + sizeof(SPauseStreamStmt), + emptyNodeToJson, + emptyJsonToNode, + destoryXNode + ); + setFunc("ResumeStreamStmt", + QUERY_NODE_RESUME_STREAM_STMT, + sizeof(SResumeStreamStmt), + emptyNodeToJson, + emptyJsonToNode, + destoryXNode + ); + setFunc("BalanceVgroupStmt", + QUERY_NODE_BALANCE_VGROUP_STMT, + sizeof(SBalanceVgroupStmt), + emptyNodeToJson, + emptyJsonToNode, + destoryXNode + ); + setFunc("BalanceVgroupLeaderStmt", + QUERY_NODE_BALANCE_VGROUP_LEADER_STMT, + sizeof(SBalanceVgroupLeaderStmt), + emptyNodeToJson, + emptyJsonToNode, + destoryXNode + ); + setFunc("MergeVgroupStmt", + QUERY_NODE_MERGE_VGROUP_STMT, + sizeof(SMergeVgroupStmt), + mergeVgroupStmtToJson, + jsonToMergeVgroupStmt, + destoryXNode + ); + setFunc("RedistributeVgroupStmt", + QUERY_NODE_REDISTRIBUTE_VGROUP_STMT, + sizeof(SRedistributeVgroupStmt), + redistributeVgroupStmtToJson, + jsonToRedistributeVgroupStmt, + destoryRedistributeVgroupStmt + ); + setFunc("SplitVgroupStmt", + QUERY_NODE_SPLIT_VGROUP_STMT, + sizeof(SSplitVgroupStmt), + splitVgroupStmtToJson, + jsonToSplitVgroupStmt, + destoryXNode + ); + setFunc("SyncDBStmt", + QUERY_NODE_SYNCDB_STMT, + 0, + emptyNodeToJson, + emptyJsonToNode, + destoryXNode + ); + setFunc("GrantStmt", + QUERY_NODE_GRANT_STMT, + sizeof(SGrantStmt), + grantStmtToJson, + jsonToGrantStmt, + destoryGrantStmt + ); + setFunc("RevokeStmt", + QUERY_NODE_REVOKE_STMT, + sizeof(SRevokeStmt), + revokeStmtToJson, + jsonToRevokeStmt, + destoryRevokeStmt + ); + setFunc("ShowDnodesStmt", + QUERY_NODE_SHOW_DNODES_STMT, + sizeof(SShowStmt), + showDnodesStmtToJson, + jsonToShowDnodesStmt, + destoryShowStmt + ); + setFunc("ShowMnodesStmt", + QUERY_NODE_SHOW_MNODES_STMT, + sizeof(SShowStmt), + showMnodesStmtToJson, + jsonToShowMnodesStmt, + destoryShowStmt + ); + setFunc("ShowModulesStmt", + QUERY_NODE_SHOW_MODULES_STMT, + sizeof(SShowStmt), + emptyNodeToJson, + emptyJsonToNode, + destoryShowStmt + ); + setFunc("ShowQnodesStmt", + QUERY_NODE_SHOW_QNODES_STMT, + sizeof(SShowStmt), + showQnodesStmtToJson, + jsonToShowQnodesStmt, + destoryShowStmt + ); + setFunc("ShowSnodesStmt", + QUERY_NODE_SHOW_SNODES_STMT, + sizeof(SShowStmt), + emptyNodeToJson, + emptyJsonToNode, + destoryShowStmt + ); + setFunc("ShowBnodesStmt", + QUERY_NODE_SHOW_BNODES_STMT, + sizeof(SShowStmt), + emptyNodeToJson, + emptyJsonToNode, + destoryShowStmt + ); + setFunc("ShowClusterStmt", + QUERY_NODE_SHOW_CLUSTER_STMT, + sizeof(SShowStmt), + showClusterStmtToJson, + jsonToShowClusterStmt, + destoryShowStmt + ); + setFunc("ShowDatabaseStmt", + QUERY_NODE_SHOW_DATABASES_STMT, + sizeof(SShowStmt), + showDatabasesStmtToJson, + jsonToShowDatabasesStmt, + destoryShowStmt + ); + setFunc("ShowFunctionsStmt", + QUERY_NODE_SHOW_FUNCTIONS_STMT, + sizeof(SShowStmt), + showFunctionsStmtToJson, + jsonToShowFunctionsStmt, + destoryShowStmt + ); + setFunc("ShowIndexesStmt", + QUERY_NODE_SHOW_INDEXES_STMT, + sizeof(SShowStmt), + showIndexesStmtToJson, + jsonToShowIndexesStmt, + destoryShowStmt + ); + setFunc("ShowStablesStmt", + QUERY_NODE_SHOW_STABLES_STMT, + sizeof(SShowStmt), + showStablesStmtToJson, + jsonToShowStablesStmt, + destoryShowStmt + ); + setFunc("ShowStreamsStmt", + QUERY_NODE_SHOW_STREAMS_STMT, + sizeof(SShowStmt), + showStreamsStmtToJson, + jsonToShowStreamsStmt, + destoryShowStmt + ); + setFunc("ShowTablesStmt", + QUERY_NODE_SHOW_TABLES_STMT, + sizeof(SShowStmt), + showTablesStmtToJson, + jsonToShowTablesStmt, + destoryShowStmt + ); + setFunc("ShowTagsStmt", + QUERY_NODE_SHOW_TAGS_STMT, + sizeof(SShowStmt), + showTagsStmtToJson, + jsonToShowTagsStmt, + destoryShowStmt + ); + setFunc("ShowUsersStmt", + QUERY_NODE_SHOW_USERS_STMT, + sizeof(SShowStmt), + showUsersStmtToJson, + jsonToShowUsersStmt, + destoryShowStmt + ); + setFunc("ShowLicencesStmt", + QUERY_NODE_SHOW_LICENCES_STMT, + sizeof(SShowStmt), + emptyNodeToJson, + emptyJsonToNode, + destoryShowStmt + ); + setFunc("ShowVgroupsStmt", + QUERY_NODE_SHOW_VGROUPS_STMT, + sizeof(SShowStmt), + showVgroupsStmtToJson, + jsonToShowVgroupsStmt, + destoryShowStmt + ); + setFunc("ShowTopicsStmt", + QUERY_NODE_SHOW_TOPICS_STMT, + sizeof(SShowStmt), + emptyNodeToJson, + emptyJsonToNode, + destoryShowStmt + ); + setFunc("ShowConsumersStmt", + QUERY_NODE_SHOW_CONSUMERS_STMT, + sizeof(SShowStmt), + showConsumersStmtToJson, + jsonToShowConsumersStmt, + destoryShowStmt + ); + setFunc("ShowQueriesStmt", + QUERY_NODE_SHOW_QUERIES_STMT, + sizeof(SShowStmt), + emptyNodeToJson, + emptyJsonToNode, + destoryShowStmt + ); + setFunc("ShowConnectionsStmt", + QUERY_NODE_SHOW_CONNECTIONS_STMT, + sizeof(SShowStmt), + emptyNodeToJson, + emptyJsonToNode, + destoryShowStmt + ); + setFunc("ShowAppsStmt", + QUERY_NODE_SHOW_APPS_STMT, + sizeof(SShowStmt), + emptyNodeToJson, + emptyJsonToNode, + destoryShowStmt + ); + setFunc("ShowVariablesStmt", + QUERY_NODE_SHOW_VARIABLES_STMT, + sizeof(SShowStmt), + showVariablesStmtToJson, + jsonToShowVariablesStmt, + destoryShowStmt + ); + setFunc("ShowDnodeVariablesStmt", + QUERY_NODE_SHOW_DNODE_VARIABLES_STMT, + sizeof(SShowDnodeVariablesStmt), + showDnodeVariablesStmtToJson, + jsonToShowDnodeVariablesStmt, + destoryShowDnodeVariablesStmt + ); + setFunc("ShowTransactionsStmt", + QUERY_NODE_SHOW_TRANSACTIONS_STMT, + sizeof(SShowStmt), + showTransactionsStmtToJson, + jsonToShowTransactionsStmt, + destoryShowStmt + ); + setFunc("ShowSubscriptionsStmt", + QUERY_NODE_SHOW_SUBSCRIPTIONS_STMT, + sizeof(SShowStmt), + showSubscriptionsStmtToJson, + jsonToShowSubscriptionsStmt, + destoryShowStmt + ); + setFunc("ShowVnodeStmt", + QUERY_NODE_SHOW_VNODES_STMT, + sizeof(SShowStmt), + showVnodesStmtToJson, + jsonToShowVnodesStmt, + destoryShowStmt + ); + setFunc("ShowUserPrivilegesStmt", + QUERY_NODE_SHOW_USER_PRIVILEGES_STMT, + sizeof(SShowStmt), + showUserPrivilegesStmtToJson, + jsonToShowUserPrivilegesStmt, + destoryShowStmt + ); + setFunc("ShowViewsStmt", + QUERY_NODE_SHOW_VIEWS_STMT, + sizeof(SShowStmt), + emptyNodeToJson, + emptyJsonToNode, + destoryShowStmt + ); + setFunc("ShowCreateViewStmt", + QUERY_NODE_SHOW_CREATE_VIEW_STMT, + sizeof(SShowCreateViewStmt), + showCreateViewStmtToJson, + jsonToShowCreateViewStmt, + destoryXNode + ); + setFunc("ShowCreateDatabasesStmt", + QUERY_NODE_SHOW_CREATE_DATABASE_STMT, + sizeof(SShowCreateDatabaseStmt), + showCreateDatabaseStmtToJson, + jsonToShowCreateDatabaseStmt, + destoryShowCreateDatabaseStmt + ); + setFunc("ShowCreateTablesStmt", + QUERY_NODE_SHOW_CREATE_TABLE_STMT, + sizeof(SShowCreateTableStmt), + showCreateTableStmtToJson, + jsonToShowCreateTableStmt, + destoryShowCreateTableStmt + ); + setFunc("ShowCreateStablesStmt", + QUERY_NODE_SHOW_CREATE_STABLE_STMT, + sizeof(SShowCreateTableStmt), + showCreateStableStmtToJson, + jsonToShowCreateStableStmt, + destoryShowCreateTableStmt + ); + setFunc("ShowTableDistributedStmt", + QUERY_NODE_SHOW_TABLE_DISTRIBUTED_STMT, + sizeof(SShowTableDistributedStmt), + showTableDistributedStmtToJson, + jsonToShowTableDistributedStmt, + destoryXNode + ); + setFunc("ShowLocalVariablesStmt", + QUERY_NODE_SHOW_LOCAL_VARIABLES_STMT, + sizeof(SShowStmt), + showLocalVariablesStmtToJson, + jsonToShowLocalVariablesStmt, + destoryShowStmt + ); + setFunc("ShowScoresStmt", + QUERY_NODE_SHOW_SCORES_STMT, + sizeof(SShowStmt), + emptyNodeToJson, + emptyJsonToNode, + destoryShowStmt + ); + setFunc("ShowTableTagsStmt", + QUERY_NODE_SHOW_TABLE_TAGS_STMT, + sizeof(SShowTableTagsStmt), + showTableTagsStmtToJson, + jsonToShowTableTagsStmt, + destoryShowTableTagsStmt + ); + setFunc("KillConnectionStmt", + QUERY_NODE_KILL_CONNECTION_STMT, + sizeof(SKillStmt), + emptyNodeToJson, + emptyJsonToNode, + destoryXNode + ); + setFunc("KillQueryStmt", + QUERY_NODE_KILL_QUERY_STMT, + sizeof(SKillQueryStmt), + emptyNodeToJson, + emptyJsonToNode, + destoryXNode + ); + setFunc("KillTransactionStmt", + QUERY_NODE_KILL_TRANSACTION_STMT, + sizeof(SKillStmt), + emptyNodeToJson, + emptyJsonToNode, + destoryXNode + ); + setFunc("DeleteStmt", + QUERY_NODE_DELETE_STMT, + sizeof(SDeleteStmt), + deleteStmtToJson, + jsonToDeleteStmt, + destoryDeleteStmt + ); + setFunc("InsertStmt", + QUERY_NODE_INSERT_STMT, + sizeof(SInsertStmt), + insertStmtToJson, + jsonToInsertStmt, + destoryInsertStmt + ); + setFunc("QueryNode", + QUERY_NODE_QUERY, + sizeof(SQuery), + emptyNodeToJson, + emptyJsonToNode, + destoryQueryNode + ); + setFunc("ShowDbAliveStmt", + QUERY_NODE_SHOW_DB_ALIVE_STMT, + sizeof(SShowAliveStmt), + emptyNodeToJson, + emptyJsonToNode, + destoryXNode + ); + setFunc("ShowClusterAliveStmt", + QUERY_NODE_SHOW_CLUSTER_ALIVE_STMT, + sizeof(SShowAliveStmt), + emptyNodeToJson, + emptyJsonToNode, + destoryXNode + ); + setFunc("RestoreDnodeStmt", + QUERY_NODE_RESTORE_DNODE_STMT, + sizeof(SRestoreComponentNodeStmt), + emptyNodeToJson, + jsonToRestoreDnodeStmt, + destoryXNode + ); + setFunc("RestoreQnodeStmt", + QUERY_NODE_RESTORE_QNODE_STMT, + sizeof(SRestoreComponentNodeStmt), + emptyNodeToJson, + jsonToRestoreQnodeStmt, + destoryXNode + ); + setFunc("RestoreMnodeStmt", + QUERY_NODE_RESTORE_MNODE_STMT, + sizeof(SRestoreComponentNodeStmt), + emptyNodeToJson, + jsonToRestoreMnodeStmt, + destoryXNode + ); + setFunc("RestoreVnodeStmt", + QUERY_NODE_RESTORE_VNODE_STMT, + sizeof(SRestoreComponentNodeStmt), + emptyNodeToJson, + jsonToRestoreVnodeStmt, + destoryXNode + ); + setFunc("CreateViewStmt", + QUERY_NODE_CREATE_VIEW_STMT, + sizeof(SCreateViewStmt), + emptyNodeToJson, + emptyJsonToNode, + destoryCreateViewStmt + ); + setFunc("DropViewStmt", + QUERY_NODE_DROP_VIEW_STMT, + sizeof(SDropViewStmt), + emptyNodeToJson, + emptyJsonToNode, + destoryXNode + ); + setFunc("LogicScan", + QUERY_NODE_LOGIC_PLAN_SCAN, + sizeof(SScanLogicNode), + logicScanNodeToJson, + jsonToLogicScanNode, + destoryScanLogicNode + ); + setFunc("LogicJoin", + QUERY_NODE_LOGIC_PLAN_JOIN, + sizeof(SJoinLogicNode), + logicJoinNodeToJson, + jsonToLogicJoinNode, + destoryJoinLogicNode + ); + setFunc("LogicAgg", + QUERY_NODE_LOGIC_PLAN_AGG, + sizeof(SAggLogicNode), + logicAggNodeToJson, + jsonToLogicAggNode, + destoryAggLogicNode + ); + setFunc("LogicProject", + QUERY_NODE_LOGIC_PLAN_PROJECT, + sizeof(SProjectLogicNode), + logicProjectNodeToJson, + jsonToLogicProjectNode, + destoryProjectLogicNode + ); + setFunc("LogicVnodeModify", + QUERY_NODE_LOGIC_PLAN_VNODE_MODIFY, + sizeof(SVnodeModifyLogicNode), + logicVnodeModifyNodeToJson, + jsonToLogicVnodeModifyNode, + destoryVnodeModifyLogicNode + ); + setFunc("LogicExchange", + QUERY_NODE_LOGIC_PLAN_EXCHANGE, + sizeof(SExchangeLogicNode), + logicExchangeNodeToJson, + jsonToLogicExchangeNode, + destoryExchangeLogicNode + ); + setFunc("LogicMerge", + QUERY_NODE_LOGIC_PLAN_MERGE, + sizeof(SMergeLogicNode), + logicMergeNodeToJson, + jsonToLogicMergeNode, + destoryMergeLogicNode + ); + setFunc("LogicWindow", + QUERY_NODE_LOGIC_PLAN_WINDOW, + sizeof(SWindowLogicNode), + logicWindowNodeToJson, + jsonToLogicWindowNode, + destoryWindowLogicNode + ); + setFunc("LogicFill", + QUERY_NODE_LOGIC_PLAN_FILL, + sizeof(SFillLogicNode), + logicFillNodeToJson, + jsonToLogicFillNode, + destoryFillLogicNode + ); + setFunc("LogicSort", + QUERY_NODE_LOGIC_PLAN_SORT, + sizeof(SSortLogicNode), + logicSortNodeToJson, + jsonToLogicSortNode, + destorySortLogicNode + ); + setFunc("LogicPartition", + QUERY_NODE_LOGIC_PLAN_PARTITION, + sizeof(SPartitionLogicNode), + logicPartitionNodeToJson, + jsonToLogicPartitionNode, + destoryPartitionLogicNode + ); + setFunc("LogicIndefRowsFunc", + QUERY_NODE_LOGIC_PLAN_INDEF_ROWS_FUNC, + sizeof(SIndefRowsFuncLogicNode), + logicIndefRowsFuncNodeToJson, + jsonToLogicIndefRowsFuncNode, + destoryIndefRowsFuncLogicNode + ); + setFunc("LogicInterpFunc", + QUERY_NODE_LOGIC_PLAN_INTERP_FUNC, + sizeof(SInterpFuncLogicNode), + logicInterpFuncNodeToJson, + jsonToLogicInterpFuncNode, + destoryInterpFuncLogicNode + ); + setFunc("LogicGroupCache", + QUERY_NODE_LOGIC_PLAN_GROUP_CACHE, + sizeof(SGroupCacheLogicNode), + logicGroupCacheNodeToJson, + jsonToLogicGroupCacheNode, + destoryGroupCacheLogicNode + ); + setFunc("LogicDynamicQueryCtrl", + QUERY_NODE_LOGIC_PLAN_DYN_QUERY_CTRL, + sizeof(SDynQueryCtrlLogicNode), + logicDynQueryCtrlNodeToJson, + jsonToLogicDynQueryCtrlNode, + destoryDynQueryCtrlLogicNode + ); + setFunc("LogicSubplan", + QUERY_NODE_LOGIC_SUBPLAN, + sizeof(SLogicSubplan), + logicSubplanToJson, + jsonToLogicSubplan, + destoryLogicSubplan + ); + setFunc("LogicPlan", + QUERY_NODE_LOGIC_PLAN, + sizeof(SQueryLogicPlan), + logicPlanToJson, + jsonToLogicPlan, + destoryQueryLogicPlan + ); + setFunc("PhysiTagScan", + QUERY_NODE_PHYSICAL_PLAN_TAG_SCAN, + sizeof(STagScanPhysiNode), + physiTagScanNodeToJson, + jsonToPhysiTagScanNode, + destroyScanPhysiNode + ); + setFunc("PhysiTableScan", + QUERY_NODE_PHYSICAL_PLAN_TABLE_SCAN, + sizeof(STableScanPhysiNode), + physiTableScanNodeToJson, + jsonToPhysiTableScanNode, + destoryTableScanPhysiNode + ); + setFunc("PhysiTableSeqScan", + QUERY_NODE_PHYSICAL_PLAN_TABLE_SEQ_SCAN, + sizeof(STableSeqScanPhysiNode), + emptyNodeToJson, + emptyJsonToNode, + destoryTableScanPhysiNode + ); + setFunc("PhysiTableMergeScan", + QUERY_NODE_PHYSICAL_PLAN_TABLE_MERGE_SCAN, + sizeof(STableMergeScanPhysiNode), + physiTableScanNodeToJson, + jsonToPhysiTableScanNode, + destoryTableScanPhysiNode + ); + setFunc("PhysiSreamScan", + QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN, + sizeof(SStreamScanPhysiNode), + physiTableScanNodeToJson, + jsonToPhysiTableScanNode, + destoryTableScanPhysiNode + ); + setFunc("PhysiSystemTableScan", + QUERY_NODE_PHYSICAL_PLAN_SYSTABLE_SCAN, + sizeof(SSystemTableScanPhysiNode), + physiSysTableScanNodeToJson, + jsonToPhysiSysTableScanNode, + destroyScanPhysiNode + ); + setFunc("PhysiBlockDistScan", + QUERY_NODE_PHYSICAL_PLAN_BLOCK_DIST_SCAN, + sizeof(SBlockDistScanPhysiNode), + physiScanNodeToJson, + jsonToPhysiScanNode, + destroyScanPhysiNode + ); + setFunc("PhysiLastRowScan", + QUERY_NODE_PHYSICAL_PLAN_LAST_ROW_SCAN, + sizeof(SLastRowScanPhysiNode), + physiLastRowScanNodeToJson, + jsonToPhysiLastRowScanNode, + destoryLastRowScanPhysiNode + ); + setFunc("PhysiProject", + QUERY_NODE_PHYSICAL_PLAN_PROJECT, + sizeof(SProjectPhysiNode), + physiProjectNodeToJson, + jsonToPhysiProjectNode, + destoryProjectPhysiNode + ); + setFunc("PhysiMergeJoin", + QUERY_NODE_PHYSICAL_PLAN_MERGE_JOIN, + sizeof(SSortMergeJoinPhysiNode), + physiMergeJoinNodeToJson, + jsonToPhysiMergeJoinNode, + destorySortMergeJoinPhysiNode + ); + setFunc("PhysiAgg", + QUERY_NODE_PHYSICAL_PLAN_HASH_AGG, + sizeof(SAggPhysiNode), + physiAggNodeToJson, + jsonToPhysiAggNode, + destoryAggPhysiNode + ); + setFunc("PhysiExchange", + QUERY_NODE_PHYSICAL_PLAN_EXCHANGE, + sizeof(SExchangePhysiNode), + physiExchangeNodeToJson, + jsonToPhysiExchangeNode, + destoryExchangePhysiNode + ); + setFunc("PhysiMerge", + QUERY_NODE_PHYSICAL_PLAN_MERGE, + sizeof(SMergePhysiNode), + physiMergeNodeToJson, + jsonToPhysiMergeNode, + destoryMergePhysiNode + ); + setFunc("PhysiSort", + QUERY_NODE_PHYSICAL_PLAN_SORT, + sizeof(SSortPhysiNode), + physiSortNodeToJson, + jsonToPhysiSortNode, + destorySortPhysiNode + ); + setFunc("PhysiGroupSort", + QUERY_NODE_PHYSICAL_PLAN_GROUP_SORT, + sizeof(SGroupSortPhysiNode), + physiSortNodeToJson, + jsonToPhysiSortNode, + destorySortPhysiNode + ); + setFunc("PhysiHashInterval", + QUERY_NODE_PHYSICAL_PLAN_HASH_INTERVAL, + sizeof(SIntervalPhysiNode), + physiIntervalNodeToJson, + jsonToPhysiIntervalNode, + destroyWindowPhysiNode + ); + setFunc("PhysiMergeAlignedInterval", + QUERY_NODE_PHYSICAL_PLAN_MERGE_ALIGNED_INTERVAL, + sizeof(SMergeAlignedIntervalPhysiNode), + physiIntervalNodeToJson, + jsonToPhysiIntervalNode, + destroyWindowPhysiNode + ); + setFunc("PhysiStreamInterval", + QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL, + sizeof(SStreamIntervalPhysiNode), + physiIntervalNodeToJson, + jsonToPhysiIntervalNode, + destroyWindowPhysiNode + ); + setFunc("PhysiStreamFinalInterval", + QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL, + sizeof(SStreamFinalIntervalPhysiNode), + physiIntervalNodeToJson, + jsonToPhysiIntervalNode, + destroyWindowPhysiNode + ); + setFunc("PhysiStreamSemiInterval", + QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL, + sizeof(SStreamSemiIntervalPhysiNode), + physiIntervalNodeToJson, + jsonToPhysiIntervalNode, + destroyWindowPhysiNode + ); + setFunc("PhysiFill", + QUERY_NODE_PHYSICAL_PLAN_FILL, + sizeof(SFillPhysiNode), + physiFillNodeToJson, + jsonToPhysiFillNode, + destoryFillPhysiNode + ); + setFunc("PhysiStreamFill", + QUERY_NODE_PHYSICAL_PLAN_STREAM_FILL, + sizeof(SFillPhysiNode), + physiFillNodeToJson, + jsonToPhysiFillNode, + destoryFillPhysiNode + ); + setFunc("PhysiSessionWindow", + QUERY_NODE_PHYSICAL_PLAN_MERGE_SESSION, + sizeof(SSessionWinodwPhysiNode), + physiSessionWindowNodeToJson, + jsonToPhysiSessionWindowNode, + destroyWindowPhysiNode + ); + setFunc("PhysiStreamSessionWindow", + QUERY_NODE_PHYSICAL_PLAN_STREAM_SESSION, + sizeof(SStreamSessionWinodwPhysiNode), + physiSessionWindowNodeToJson, + jsonToPhysiSessionWindowNode, + destroyWindowPhysiNode + ); + setFunc("PhysiStreamSemiSessionWindow", + QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_SESSION, + sizeof(SStreamSemiSessionWinodwPhysiNode), + physiSessionWindowNodeToJson, + jsonToPhysiSessionWindowNode, + destroyWindowPhysiNode + ); + setFunc("PhysiStreamFinalSessionWindow", + QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_SESSION, + sizeof(SStreamFinalSessionWinodwPhysiNode), + physiSessionWindowNodeToJson, + jsonToPhysiSessionWindowNode, + destroyWindowPhysiNode + ); + setFunc("PhysiStateWindow", + QUERY_NODE_PHYSICAL_PLAN_MERGE_STATE, + sizeof(SStateWinodwPhysiNode), + physiStateWindowNodeToJson, + jsonToPhysiStateWindowNode, + destoryStateWindowPhysiNode + ); + setFunc("PhysiStreamStateWindow", + QUERY_NODE_PHYSICAL_PLAN_STREAM_STATE, + sizeof(SStreamStateWinodwPhysiNode), + physiStateWindowNodeToJson, + jsonToPhysiStateWindowNode, + destoryStateWindowPhysiNode + ); + setFunc("PhysiPartition", + QUERY_NODE_PHYSICAL_PLAN_PARTITION, + sizeof(SPartitionPhysiNode), + physiPartitionNodeToJson, + jsonToPhysiPartitionNode, + destroyPartitionPhysiNode + ); + setFunc("PhysiStreamPartition", + QUERY_NODE_PHYSICAL_PLAN_STREAM_PARTITION, + sizeof(SStreamPartitionPhysiNode), + physiStreamPartitionNodeToJson, + jsonToPhysiStreamPartitionNode, + destoryStreamPartitionPhysiNode + ); + setFunc("PhysiIndefRowsFunc", + QUERY_NODE_PHYSICAL_PLAN_INDEF_ROWS_FUNC, + sizeof(SIndefRowsFuncPhysiNode), + physiIndefRowsFuncNodeToJson, + jsonToPhysiIndefRowsFuncNode, + destoryIndefRowsFuncPhysiNode + ); + setFunc("PhysiInterpFunc", + QUERY_NODE_PHYSICAL_PLAN_INTERP_FUNC, + sizeof(SInterpFuncLogicNode), + physiInterpFuncNodeToJson, + jsonToPhysiInterpFuncNode, + destoryInterpFuncPhysiNode + ); + setFunc("PhysiDispatch", + QUERY_NODE_PHYSICAL_PLAN_DISPATCH, + sizeof(SDataDispatcherNode), + physiDispatchNodeToJson, + jsonToPhysiDispatchNode, + destroyDataSinkNode + ); + setFunc("PhysiInsert", + QUERY_NODE_PHYSICAL_PLAN_INSERT, + sizeof(SDataInserterNode), + emptyNodeToJson, + emptyJsonToNode, + destoryDataInserterNode + ); + setFunc("PhysiQueryInsert", + QUERY_NODE_PHYSICAL_PLAN_QUERY_INSERT, + sizeof(SQueryInserterNode), + physiQueryInsertNodeToJson, + jsonToPhysiQueryInsertNode, + destoryQueryInserterNode + ); + setFunc("PhysiDelete", + QUERY_NODE_PHYSICAL_PLAN_DELETE, + sizeof(SDataDeleterNode), + physiDeleteNodeToJson, + jsonToPhysiDeleteNode, + destoryDataDeleterNode + ); + setFunc("PhysiGroupCache", + QUERY_NODE_PHYSICAL_PLAN_GROUP_CACHE, + sizeof(SGroupCachePhysiNode), + physiGroupCacheNodeToJson, + jsonToPhysiGroupCacheNode, + destoryGroupCachePhysiNode + ); + setFunc("PhysiDynamicQueryCtrl", + QUERY_NODE_PHYSICAL_PLAN_DYN_QUERY_CTRL, + sizeof(SDynQueryCtrlPhysiNode), + physiDynQueryCtrlNodeToJson, + jsonToPhysiDynQueryCtrlNode, + destoryDynQueryCtrlPhysiNode + ); + setFunc("PhysiSubplan", + QUERY_NODE_PHYSICAL_SUBPLAN, + sizeof(SSubplan), + subplanToJson, + jsonToSubplan, + destorySubplanNode + ); + setFunc("PhysiPlan", + QUERY_NODE_PHYSICAL_PLAN, + sizeof(SQueryPlan), + planToJson, + jsonToPlan, + destoryPlanNode + ); + setFunc("PhysiTableCountScan", + QUERY_NODE_PHYSICAL_PLAN_TABLE_COUNT_SCAN, + sizeof(STableCountScanPhysiNode), + physiLastRowScanNodeToJson, + jsonToPhysiScanNode, + destoryLastRowScanPhysiNode + ); + setFunc("PhysiMergeEventWindow", + QUERY_NODE_PHYSICAL_PLAN_MERGE_EVENT, + sizeof(SEventWinodwPhysiNode), + physiEventWindowNodeToJson, + jsonToPhysiEventWindowNode, + destoryEventWindowPhysiNode + ); + setFunc("PhysiStreamEventWindow", + QUERY_NODE_PHYSICAL_PLAN_STREAM_EVENT, + sizeof(SStreamEventWinodwPhysiNode), + physiEventWindowNodeToJson, + jsonToPhysiEventWindowNode, + destoryEventWindowPhysiNode + ); + setFunc("PhysiHashJoin", + QUERY_NODE_PHYSICAL_PLAN_HASH_JOIN, + sizeof(SHashJoinPhysiNode), + physiHashJoinNodeToJson, + jsonToPhysiHashJoinNode, + destoryHashJoinPhysiNode + ); + initNodeCode = 0; +} + +// clang-format on diff --git a/source/libs/nodes/src/nodesUtilFuncs.c b/source/libs/nodes/src/nodesUtilFuncs.c index c8de8bdfac..3a738dbbd2 100644 --- a/source/libs/nodes/src/nodesUtilFuncs.c +++ b/source/libs/nodes/src/nodesUtilFuncs.c @@ -131,7 +131,7 @@ static void destroyNodeAllocator(void* p) { } int32_t nodesInitAllocatorSet() { - initNodeName(); + nodesInit(); if (g_allocatorReqRefPool >= 0) { nodesWarn("nodes already initialized"); return TSDB_CODE_SUCCESS; @@ -238,1245 +238,14 @@ static SNode* makeNode(ENodeType type, int32_t size) { } SNode* nodesMakeNode(ENodeType type) { - switch (type) { - case QUERY_NODE_COLUMN: - return makeNode(type, sizeof(SColumnNode)); - case QUERY_NODE_VALUE: - return makeNode(type, sizeof(SValueNode)); - case QUERY_NODE_OPERATOR: - return makeNode(type, sizeof(SOperatorNode)); - case QUERY_NODE_LOGIC_CONDITION: - return makeNode(type, sizeof(SLogicConditionNode)); - case QUERY_NODE_FUNCTION: - return makeNode(type, sizeof(SFunctionNode)); - case QUERY_NODE_REAL_TABLE: - return makeNode(type, sizeof(SRealTableNode)); - case QUERY_NODE_TEMP_TABLE: - return makeNode(type, sizeof(STempTableNode)); - case QUERY_NODE_JOIN_TABLE: - return makeNode(type, sizeof(SJoinTableNode)); - case QUERY_NODE_GROUPING_SET: - return makeNode(type, sizeof(SGroupingSetNode)); - case QUERY_NODE_ORDER_BY_EXPR: - return makeNode(type, sizeof(SOrderByExprNode)); - case QUERY_NODE_LIMIT: - return makeNode(type, sizeof(SLimitNode)); - case QUERY_NODE_STATE_WINDOW: - return makeNode(type, sizeof(SStateWindowNode)); - case QUERY_NODE_SESSION_WINDOW: - return makeNode(type, sizeof(SSessionWindowNode)); - case QUERY_NODE_INTERVAL_WINDOW: - return makeNode(type, sizeof(SIntervalWindowNode)); - case QUERY_NODE_NODE_LIST: - return makeNode(type, sizeof(SNodeListNode)); - case QUERY_NODE_FILL: - return makeNode(type, sizeof(SFillNode)); - case QUERY_NODE_RAW_EXPR: - return makeNode(type, sizeof(SRawExprNode)); - case QUERY_NODE_TARGET: - return makeNode(type, sizeof(STargetNode)); - case QUERY_NODE_DATABLOCK_DESC: - return makeNode(type, sizeof(SDataBlockDescNode)); - case QUERY_NODE_SLOT_DESC: - return makeNode(type, sizeof(SSlotDescNode)); - case QUERY_NODE_COLUMN_DEF: - return makeNode(type, sizeof(SColumnDefNode)); - case QUERY_NODE_DOWNSTREAM_SOURCE: - return makeNode(type, sizeof(SDownstreamSourceNode)); - case QUERY_NODE_DATABASE_OPTIONS: - return makeNode(type, sizeof(SDatabaseOptions)); - case QUERY_NODE_TABLE_OPTIONS: - return makeNode(type, sizeof(STableOptions)); - case QUERY_NODE_INDEX_OPTIONS: - return makeNode(type, sizeof(SIndexOptions)); - case QUERY_NODE_EXPLAIN_OPTIONS: - return makeNode(type, sizeof(SExplainOptions)); - case QUERY_NODE_STREAM_OPTIONS: - return makeNode(type, sizeof(SStreamOptions)); - case QUERY_NODE_LEFT_VALUE: - return makeNode(type, sizeof(SLeftValueNode)); - case QUERY_NODE_COLUMN_REF: - return makeNode(type, sizeof(SColumnDefNode)); - case QUERY_NODE_WHEN_THEN: - return makeNode(type, sizeof(SWhenThenNode)); - case QUERY_NODE_CASE_WHEN: - return makeNode(type, sizeof(SCaseWhenNode)); - case QUERY_NODE_EVENT_WINDOW: - return makeNode(type, sizeof(SEventWindowNode)); - case QUERY_NODE_HINT: - return makeNode(type, sizeof(SHintNode)); - case QUERY_NODE_VIEW: - return makeNode(type, sizeof(SViewNode)); - case QUERY_NODE_SET_OPERATOR: - return makeNode(type, sizeof(SSetOperator)); - case QUERY_NODE_SELECT_STMT: - return makeNode(type, sizeof(SSelectStmt)); - case QUERY_NODE_VNODE_MODIFY_STMT: - return makeNode(type, sizeof(SVnodeModifyOpStmt)); - case QUERY_NODE_CREATE_DATABASE_STMT: - return makeNode(type, sizeof(SCreateDatabaseStmt)); - case QUERY_NODE_DROP_DATABASE_STMT: - return makeNode(type, sizeof(SDropDatabaseStmt)); - case QUERY_NODE_ALTER_DATABASE_STMT: - return makeNode(type, sizeof(SAlterDatabaseStmt)); - case QUERY_NODE_FLUSH_DATABASE_STMT: - return makeNode(type, sizeof(SFlushDatabaseStmt)); - case QUERY_NODE_TRIM_DATABASE_STMT: - return makeNode(type, sizeof(STrimDatabaseStmt)); - case QUERY_NODE_CREATE_TABLE_STMT: - return makeNode(type, sizeof(SCreateTableStmt)); - case QUERY_NODE_CREATE_SUBTABLE_CLAUSE: - return makeNode(type, sizeof(SCreateSubTableClause)); - case QUERY_NODE_CREATE_MULTI_TABLES_STMT: - return makeNode(type, sizeof(SCreateMultiTablesStmt)); - case QUERY_NODE_DROP_TABLE_CLAUSE: - return makeNode(type, sizeof(SDropTableClause)); - case QUERY_NODE_DROP_TABLE_STMT: - return makeNode(type, sizeof(SDropTableStmt)); - case QUERY_NODE_DROP_SUPER_TABLE_STMT: - return makeNode(type, sizeof(SDropSuperTableStmt)); - case QUERY_NODE_ALTER_TABLE_STMT: - case QUERY_NODE_ALTER_SUPER_TABLE_STMT: - return makeNode(type, sizeof(SAlterTableStmt)); - case QUERY_NODE_CREATE_USER_STMT: - return makeNode(type, sizeof(SCreateUserStmt)); - case QUERY_NODE_ALTER_USER_STMT: - return makeNode(type, sizeof(SAlterUserStmt)); - case QUERY_NODE_DROP_USER_STMT: - return makeNode(type, sizeof(SDropUserStmt)); - case QUERY_NODE_USE_DATABASE_STMT: - return makeNode(type, sizeof(SUseDatabaseStmt)); - case QUERY_NODE_CREATE_DNODE_STMT: - return makeNode(type, sizeof(SCreateDnodeStmt)); - case QUERY_NODE_DROP_DNODE_STMT: - return makeNode(type, sizeof(SDropDnodeStmt)); - case QUERY_NODE_ALTER_DNODE_STMT: - return makeNode(type, sizeof(SAlterDnodeStmt)); - case QUERY_NODE_CREATE_INDEX_STMT: - return makeNode(type, sizeof(SCreateIndexStmt)); - case QUERY_NODE_DROP_INDEX_STMT: - return makeNode(type, sizeof(SDropIndexStmt)); - case QUERY_NODE_CREATE_QNODE_STMT: - case QUERY_NODE_CREATE_BNODE_STMT: - case QUERY_NODE_CREATE_SNODE_STMT: - case QUERY_NODE_CREATE_MNODE_STMT: - return makeNode(type, sizeof(SCreateComponentNodeStmt)); - case QUERY_NODE_DROP_QNODE_STMT: - case QUERY_NODE_DROP_BNODE_STMT: - case QUERY_NODE_DROP_SNODE_STMT: - case QUERY_NODE_DROP_MNODE_STMT: - return makeNode(type, sizeof(SDropComponentNodeStmt)); - case QUERY_NODE_CREATE_TOPIC_STMT: - return makeNode(type, sizeof(SCreateTopicStmt)); - case QUERY_NODE_DROP_TOPIC_STMT: - return makeNode(type, sizeof(SDropTopicStmt)); - case QUERY_NODE_DROP_CGROUP_STMT: - return makeNode(type, sizeof(SDropCGroupStmt)); - case QUERY_NODE_ALTER_LOCAL_STMT: - return makeNode(type, sizeof(SAlterLocalStmt)); - case QUERY_NODE_EXPLAIN_STMT: - return makeNode(type, sizeof(SExplainStmt)); - case QUERY_NODE_DESCRIBE_STMT: - return makeNode(type, sizeof(SDescribeStmt)); - case QUERY_NODE_RESET_QUERY_CACHE_STMT: - return makeNode(type, sizeof(SNode)); - case QUERY_NODE_COMPACT_DATABASE_STMT: - return makeNode(type, sizeof(SCompactDatabaseStmt)); - case QUERY_NODE_CREATE_FUNCTION_STMT: - return makeNode(type, sizeof(SCreateFunctionStmt)); - case QUERY_NODE_DROP_FUNCTION_STMT: - return makeNode(type, sizeof(SDropFunctionStmt)); - case QUERY_NODE_CREATE_STREAM_STMT: - return makeNode(type, sizeof(SCreateStreamStmt)); - case QUERY_NODE_DROP_STREAM_STMT: - return makeNode(type, sizeof(SDropStreamStmt)); - case QUERY_NODE_PAUSE_STREAM_STMT: - return makeNode(type, sizeof(SPauseStreamStmt)); - case QUERY_NODE_RESUME_STREAM_STMT: - return makeNode(type, sizeof(SResumeStreamStmt)); - case QUERY_NODE_BALANCE_VGROUP_STMT: - return makeNode(type, sizeof(SBalanceVgroupStmt)); - case QUERY_NODE_BALANCE_VGROUP_LEADER_STMT: - return makeNode(type, sizeof(SBalanceVgroupLeaderStmt)); - case QUERY_NODE_MERGE_VGROUP_STMT: - return makeNode(type, sizeof(SMergeVgroupStmt)); - case QUERY_NODE_REDISTRIBUTE_VGROUP_STMT: - return makeNode(type, sizeof(SRedistributeVgroupStmt)); - case QUERY_NODE_SPLIT_VGROUP_STMT: - return makeNode(type, sizeof(SSplitVgroupStmt)); - case QUERY_NODE_SYNCDB_STMT: - break; - case QUERY_NODE_GRANT_STMT: - return makeNode(type, sizeof(SGrantStmt)); - case QUERY_NODE_REVOKE_STMT: - return makeNode(type, sizeof(SRevokeStmt)); - case QUERY_NODE_SHOW_DNODES_STMT: - case QUERY_NODE_SHOW_MNODES_STMT: - case QUERY_NODE_SHOW_MODULES_STMT: - case QUERY_NODE_SHOW_QNODES_STMT: - case QUERY_NODE_SHOW_SNODES_STMT: - case QUERY_NODE_SHOW_BNODES_STMT: - case QUERY_NODE_SHOW_CLUSTER_STMT: - case QUERY_NODE_SHOW_DATABASES_STMT: - case QUERY_NODE_SHOW_FUNCTIONS_STMT: - case QUERY_NODE_SHOW_INDEXES_STMT: - case QUERY_NODE_SHOW_STABLES_STMT: - case QUERY_NODE_SHOW_STREAMS_STMT: - case QUERY_NODE_SHOW_TABLES_STMT: - case QUERY_NODE_SHOW_USERS_STMT: - case QUERY_NODE_SHOW_LICENCES_STMT: - case QUERY_NODE_SHOW_VGROUPS_STMT: - case QUERY_NODE_SHOW_TOPICS_STMT: - case QUERY_NODE_SHOW_CONSUMERS_STMT: - case QUERY_NODE_SHOW_CONNECTIONS_STMT: - case QUERY_NODE_SHOW_QUERIES_STMT: - case QUERY_NODE_SHOW_VNODES_STMT: - case QUERY_NODE_SHOW_APPS_STMT: - case QUERY_NODE_SHOW_SCORES_STMT: - case QUERY_NODE_SHOW_VARIABLES_STMT: - case QUERY_NODE_SHOW_LOCAL_VARIABLES_STMT: - case QUERY_NODE_SHOW_TRANSACTIONS_STMT: - case QUERY_NODE_SHOW_SUBSCRIPTIONS_STMT: - case QUERY_NODE_SHOW_TAGS_STMT: - case QUERY_NODE_SHOW_USER_PRIVILEGES_STMT: - case QUERY_NODE_SHOW_VIEWS_STMT: - return makeNode(type, sizeof(SShowStmt)); - case QUERY_NODE_SHOW_TABLE_TAGS_STMT: - return makeNode(type, sizeof(SShowTableTagsStmt)); - case QUERY_NODE_SHOW_DNODE_VARIABLES_STMT: - return makeNode(type, sizeof(SShowDnodeVariablesStmt)); - case QUERY_NODE_SHOW_CREATE_DATABASE_STMT: - return makeNode(type, sizeof(SShowCreateDatabaseStmt)); - case QUERY_NODE_SHOW_DB_ALIVE_STMT: - case QUERY_NODE_SHOW_CLUSTER_ALIVE_STMT: - return makeNode(type, sizeof(SShowAliveStmt)); - case QUERY_NODE_SHOW_CREATE_TABLE_STMT: - case QUERY_NODE_SHOW_CREATE_STABLE_STMT: - return makeNode(type, sizeof(SShowCreateTableStmt)); - case QUERY_NODE_SHOW_CREATE_VIEW_STMT: - return makeNode(type, sizeof(SShowCreateViewStmt)); - case QUERY_NODE_SHOW_TABLE_DISTRIBUTED_STMT: - return makeNode(type, sizeof(SShowTableDistributedStmt)); - case QUERY_NODE_KILL_QUERY_STMT: - return makeNode(type, sizeof(SKillQueryStmt)); - case QUERY_NODE_KILL_TRANSACTION_STMT: - case QUERY_NODE_KILL_CONNECTION_STMT: - return makeNode(type, sizeof(SKillStmt)); - case QUERY_NODE_DELETE_STMT: - return makeNode(type, sizeof(SDeleteStmt)); - case QUERY_NODE_INSERT_STMT: - return makeNode(type, sizeof(SInsertStmt)); - case QUERY_NODE_QUERY: - return makeNode(type, sizeof(SQuery)); - case QUERY_NODE_RESTORE_DNODE_STMT: - case QUERY_NODE_RESTORE_QNODE_STMT: - case QUERY_NODE_RESTORE_MNODE_STMT: - case QUERY_NODE_RESTORE_VNODE_STMT: - return makeNode(type, sizeof(SRestoreComponentNodeStmt)); - case QUERY_NODE_CREATE_VIEW_STMT: - return makeNode(type, sizeof(SCreateViewStmt)); - case QUERY_NODE_DROP_VIEW_STMT: - return makeNode(type, sizeof(SDropViewStmt)); - case QUERY_NODE_LOGIC_PLAN_SCAN: - return makeNode(type, sizeof(SScanLogicNode)); - case QUERY_NODE_LOGIC_PLAN_JOIN: - return makeNode(type, sizeof(SJoinLogicNode)); - case QUERY_NODE_LOGIC_PLAN_AGG: - return makeNode(type, sizeof(SAggLogicNode)); - case QUERY_NODE_LOGIC_PLAN_PROJECT: - return makeNode(type, sizeof(SProjectLogicNode)); - case QUERY_NODE_LOGIC_PLAN_VNODE_MODIFY: - return makeNode(type, sizeof(SVnodeModifyLogicNode)); - case QUERY_NODE_LOGIC_PLAN_EXCHANGE: - return makeNode(type, sizeof(SExchangeLogicNode)); - case QUERY_NODE_LOGIC_PLAN_MERGE: - return makeNode(type, sizeof(SMergeLogicNode)); - case QUERY_NODE_LOGIC_PLAN_WINDOW: - return makeNode(type, sizeof(SWindowLogicNode)); - case QUERY_NODE_LOGIC_PLAN_FILL: - return makeNode(type, sizeof(SFillLogicNode)); - case QUERY_NODE_LOGIC_PLAN_SORT: - return makeNode(type, sizeof(SSortLogicNode)); - case QUERY_NODE_LOGIC_PLAN_PARTITION: - return makeNode(type, sizeof(SPartitionLogicNode)); - case QUERY_NODE_LOGIC_PLAN_INDEF_ROWS_FUNC: - return makeNode(type, sizeof(SIndefRowsFuncLogicNode)); - case QUERY_NODE_LOGIC_PLAN_INTERP_FUNC: - return makeNode(type, sizeof(SInterpFuncLogicNode)); - case QUERY_NODE_LOGIC_PLAN_GROUP_CACHE: - return makeNode(type, sizeof(SGroupCacheLogicNode)); - case QUERY_NODE_LOGIC_PLAN_DYN_QUERY_CTRL: - return makeNode(type, sizeof(SDynQueryCtrlLogicNode)); - case QUERY_NODE_LOGIC_SUBPLAN: - return makeNode(type, sizeof(SLogicSubplan)); - case QUERY_NODE_LOGIC_PLAN: - return makeNode(type, sizeof(SQueryLogicPlan)); - case QUERY_NODE_PHYSICAL_PLAN_TAG_SCAN: - return makeNode(type, sizeof(STagScanPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_TABLE_SCAN: - return makeNode(type, sizeof(STableScanPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_TABLE_SEQ_SCAN: - return makeNode(type, sizeof(STableSeqScanPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_TABLE_MERGE_SCAN: - return makeNode(type, sizeof(STableMergeScanPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN: - return makeNode(type, sizeof(SStreamScanPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_SYSTABLE_SCAN: - return makeNode(type, sizeof(SSystemTableScanPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_BLOCK_DIST_SCAN: - return makeNode(type, sizeof(SBlockDistScanPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_LAST_ROW_SCAN: - return makeNode(type, sizeof(SLastRowScanPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_TABLE_COUNT_SCAN: - return makeNode(type, sizeof(STableCountScanPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_PROJECT: - return makeNode(type, sizeof(SProjectPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_MERGE_JOIN: - return makeNode(type, sizeof(SSortMergeJoinPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_HASH_JOIN: - return makeNode(type, sizeof(SHashJoinPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_HASH_AGG: - return makeNode(type, sizeof(SAggPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_EXCHANGE: - return makeNode(type, sizeof(SExchangePhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_MERGE: - return makeNode(type, sizeof(SMergePhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_SORT: - return makeNode(type, sizeof(SSortPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_GROUP_SORT: - return makeNode(type, sizeof(SGroupSortPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_HASH_INTERVAL: - return makeNode(type, sizeof(SIntervalPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_MERGE_ALIGNED_INTERVAL: - return makeNode(type, sizeof(SMergeAlignedIntervalPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL: - return makeNode(type, sizeof(SStreamIntervalPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL: - return makeNode(type, sizeof(SStreamFinalIntervalPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL: - return makeNode(type, sizeof(SStreamSemiIntervalPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_FILL: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_FILL: - return makeNode(type, sizeof(SFillPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_MERGE_SESSION: - return makeNode(type, sizeof(SSessionWinodwPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_STREAM_SESSION: - return makeNode(type, sizeof(SStreamSessionWinodwPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_SESSION: - return makeNode(type, sizeof(SStreamSemiSessionWinodwPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_SESSION: - return makeNode(type, sizeof(SStreamFinalSessionWinodwPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_MERGE_STATE: - return makeNode(type, sizeof(SStateWinodwPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_STREAM_STATE: - return makeNode(type, sizeof(SStreamStateWinodwPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_MERGE_EVENT: - return makeNode(type, sizeof(SEventWinodwPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_STREAM_EVENT: - return makeNode(type, sizeof(SStreamEventWinodwPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_PARTITION: - return makeNode(type, sizeof(SPartitionPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_STREAM_PARTITION: - return makeNode(type, sizeof(SStreamPartitionPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_INDEF_ROWS_FUNC: - return makeNode(type, sizeof(SIndefRowsFuncPhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_INTERP_FUNC: - return makeNode(type, sizeof(SInterpFuncLogicNode)); - case QUERY_NODE_PHYSICAL_PLAN_DISPATCH: - return makeNode(type, sizeof(SDataDispatcherNode)); - case QUERY_NODE_PHYSICAL_PLAN_INSERT: - return makeNode(type, sizeof(SDataInserterNode)); - case QUERY_NODE_PHYSICAL_PLAN_QUERY_INSERT: - return makeNode(type, sizeof(SQueryInserterNode)); - case QUERY_NODE_PHYSICAL_PLAN_DELETE: - return makeNode(type, sizeof(SDataDeleterNode)); - case QUERY_NODE_PHYSICAL_PLAN_GROUP_CACHE: - return makeNode(type, sizeof(SGroupCachePhysiNode)); - case QUERY_NODE_PHYSICAL_PLAN_DYN_QUERY_CTRL: - return makeNode(type, sizeof(SDynQueryCtrlPhysiNode)); - case QUERY_NODE_PHYSICAL_SUBPLAN: - return makeNode(type, sizeof(SSubplan)); - case QUERY_NODE_PHYSICAL_PLAN: - return makeNode(type, sizeof(SQueryPlan)); - default: - break; + int32_t size = getNodeSize(type); + if (size > 0) { + return makeNode(type, size); } - nodesError("nodesMakeNode unknown node = %s", nodesNodeName(type)); + nodesError("nodesMakeNode unsupported type = %d", type); return NULL; } -static void destroyVgDataBlockArray(SArray* pArray) { - size_t size = taosArrayGetSize(pArray); - for (size_t i = 0; i < size; ++i) { - SVgDataBlocks* pVg = taosArrayGetP(pArray, i); - taosMemoryFreeClear(pVg->pData); - taosMemoryFreeClear(pVg); - } - taosArrayDestroy(pArray); -} - -static void destroyLogicNode(SLogicNode* pNode) { - nodesDestroyList(pNode->pTargets); - nodesDestroyNode(pNode->pConditions); - nodesDestroyList(pNode->pChildren); - nodesDestroyNode(pNode->pLimit); - nodesDestroyNode(pNode->pSlimit); - nodesDestroyList(pNode->pHint); -} - -static void destroyPhysiNode(SPhysiNode* pNode) { - nodesDestroyList(pNode->pChildren); - nodesDestroyNode(pNode->pConditions); - nodesDestroyNode((SNode*)pNode->pOutputDataBlockDesc); - nodesDestroyNode(pNode->pLimit); - nodesDestroyNode(pNode->pSlimit); -} - -static void destroyWinodwPhysiNode(SWindowPhysiNode* pNode) { - destroyPhysiNode((SPhysiNode*)pNode); - nodesDestroyList(pNode->pExprs); - nodesDestroyList(pNode->pFuncs); - nodesDestroyNode(pNode->pTspk); - nodesDestroyNode(pNode->pTsEnd); -} - -static void destroyPartitionPhysiNode(SPartitionPhysiNode* pNode) { - destroyPhysiNode((SPhysiNode*)pNode); - nodesDestroyList(pNode->pExprs); - nodesDestroyList(pNode->pPartitionKeys); - nodesDestroyList(pNode->pTargets); -} - -static void destroyScanPhysiNode(SScanPhysiNode* pNode) { - destroyPhysiNode((SPhysiNode*)pNode); - nodesDestroyList(pNode->pScanCols); - nodesDestroyList(pNode->pScanPseudoCols); -} - -static void destroyDataSinkNode(SDataSinkNode* pNode) { nodesDestroyNode((SNode*)pNode->pInputDataBlockDesc); } - -static void destroyExprNode(SExprNode* pExpr) { taosArrayDestroy(pExpr->pAssociation); } - -static void destroyTableCfg(STableCfg* pCfg) { - if (NULL == pCfg) { - return; - } - taosArrayDestroy(pCfg->pFuncs); - taosMemoryFree(pCfg->pComment); - taosMemoryFree(pCfg->pSchemas); - taosMemoryFree(pCfg->pTags); - taosMemoryFree(pCfg); -} - -static void destroySmaIndex(void* pIndex) { taosMemoryFree(((STableIndexInfo*)pIndex)->expr); } - -static void destroyHintValue(EHintOption option, void* value) { - switch (option) { - default: - break; - } - - taosMemoryFree(value); -} - -void nodesDestroyNode(SNode* pNode) { - if (NULL == pNode) { - return; - } - - switch (nodeType(pNode)) { - case QUERY_NODE_COLUMN: - destroyExprNode((SExprNode*)pNode); - break; - case QUERY_NODE_VALUE: { - SValueNode* pValue = (SValueNode*)pNode; - destroyExprNode((SExprNode*)pNode); - taosMemoryFreeClear(pValue->literal); - if (IS_VAR_DATA_TYPE(pValue->node.resType.type)) { - taosMemoryFreeClear(pValue->datum.p); - } - break; - } - case QUERY_NODE_OPERATOR: { - SOperatorNode* pOp = (SOperatorNode*)pNode; - destroyExprNode((SExprNode*)pNode); - nodesDestroyNode(pOp->pLeft); - nodesDestroyNode(pOp->pRight); - break; - } - case QUERY_NODE_LOGIC_CONDITION: - destroyExprNode((SExprNode*)pNode); - nodesDestroyList(((SLogicConditionNode*)pNode)->pParameterList); - break; - case QUERY_NODE_FUNCTION: - destroyExprNode((SExprNode*)pNode); - nodesDestroyList(((SFunctionNode*)pNode)->pParameterList); - break; - case QUERY_NODE_REAL_TABLE: { - SRealTableNode* pReal = (SRealTableNode*)pNode; - taosMemoryFreeClear(pReal->pMeta); - taosMemoryFreeClear(pReal->pVgroupList); - taosArrayDestroyEx(pReal->pSmaIndexes, destroySmaIndex); - break; - } - case QUERY_NODE_TEMP_TABLE: - nodesDestroyNode(((STempTableNode*)pNode)->pSubquery); - break; - case QUERY_NODE_JOIN_TABLE: { - SJoinTableNode* pJoin = (SJoinTableNode*)pNode; - nodesDestroyNode(pJoin->pLeft); - nodesDestroyNode(pJoin->pRight); - nodesDestroyNode(pJoin->pOnCond); - break; - } - case QUERY_NODE_GROUPING_SET: - nodesDestroyList(((SGroupingSetNode*)pNode)->pParameterList); - break; - case QUERY_NODE_ORDER_BY_EXPR: - nodesDestroyNode(((SOrderByExprNode*)pNode)->pExpr); - break; - case QUERY_NODE_LIMIT: // no pointer field - break; - case QUERY_NODE_STATE_WINDOW: { - SStateWindowNode* pState = (SStateWindowNode*)pNode; - nodesDestroyNode(pState->pCol); - nodesDestroyNode(pState->pExpr); - break; - } - case QUERY_NODE_SESSION_WINDOW: { - SSessionWindowNode* pSession = (SSessionWindowNode*)pNode; - nodesDestroyNode((SNode*)pSession->pCol); - nodesDestroyNode((SNode*)pSession->pGap); - break; - } - case QUERY_NODE_INTERVAL_WINDOW: { - SIntervalWindowNode* pJoin = (SIntervalWindowNode*)pNode; - nodesDestroyNode(pJoin->pCol); - nodesDestroyNode(pJoin->pInterval); - nodesDestroyNode(pJoin->pOffset); - nodesDestroyNode(pJoin->pSliding); - nodesDestroyNode(pJoin->pFill); - break; - } - case QUERY_NODE_NODE_LIST: - nodesDestroyList(((SNodeListNode*)pNode)->pNodeList); - break; - case QUERY_NODE_FILL: { - SFillNode* pFill = (SFillNode*)pNode; - nodesDestroyNode(pFill->pValues); - nodesDestroyNode(pFill->pWStartTs); - break; - } - case QUERY_NODE_RAW_EXPR: - nodesDestroyNode(((SRawExprNode*)pNode)->pNode); - break; - case QUERY_NODE_TARGET: - nodesDestroyNode(((STargetNode*)pNode)->pExpr); - break; - case QUERY_NODE_DATABLOCK_DESC: - nodesDestroyList(((SDataBlockDescNode*)pNode)->pSlots); - break; - case QUERY_NODE_SLOT_DESC: // no pointer field - case QUERY_NODE_COLUMN_DEF: // no pointer field - case QUERY_NODE_DOWNSTREAM_SOURCE: // no pointer field - break; - case QUERY_NODE_DATABASE_OPTIONS: { - SDatabaseOptions* pOptions = (SDatabaseOptions*)pNode; - nodesDestroyNode((SNode*)pOptions->pDaysPerFile); - nodesDestroyList(pOptions->pKeep); - nodesDestroyList(pOptions->pRetentions); - break; - } - case QUERY_NODE_TABLE_OPTIONS: { - STableOptions* pOptions = (STableOptions*)pNode; - nodesDestroyList(pOptions->pMaxDelay); - nodesDestroyList(pOptions->pWatermark); - nodesDestroyList(pOptions->pRollupFuncs); - nodesDestroyList(pOptions->pSma); - nodesDestroyList(pOptions->pDeleteMark); - break; - } - case QUERY_NODE_INDEX_OPTIONS: { - SIndexOptions* pOptions = (SIndexOptions*)pNode; - nodesDestroyList(pOptions->pFuncs); - nodesDestroyNode(pOptions->pInterval); - nodesDestroyNode(pOptions->pOffset); - nodesDestroyNode(pOptions->pSliding); - nodesDestroyNode(pOptions->pStreamOptions); - break; - } - case QUERY_NODE_EXPLAIN_OPTIONS: // no pointer field - break; - case QUERY_NODE_STREAM_OPTIONS: { - SStreamOptions* pOptions = (SStreamOptions*)pNode; - nodesDestroyNode(pOptions->pDelay); - nodesDestroyNode(pOptions->pWatermark); - nodesDestroyNode(pOptions->pDeleteMark); - break; - } - case QUERY_NODE_LEFT_VALUE: // no pointer field - case QUERY_NODE_COLUMN_REF: // no pointer field - break; - case QUERY_NODE_WHEN_THEN: { - SWhenThenNode* pWhenThen = (SWhenThenNode*)pNode; - nodesDestroyNode(pWhenThen->pWhen); - nodesDestroyNode(pWhenThen->pThen); - break; - } - case QUERY_NODE_CASE_WHEN: { - SCaseWhenNode* pCaseWhen = (SCaseWhenNode*)pNode; - nodesDestroyNode(pCaseWhen->pCase); - nodesDestroyNode(pCaseWhen->pElse); - nodesDestroyList(pCaseWhen->pWhenThenList); - break; - } - case QUERY_NODE_EVENT_WINDOW: { - SEventWindowNode* pEvent = (SEventWindowNode*)pNode; - nodesDestroyNode(pEvent->pCol); - nodesDestroyNode(pEvent->pStartCond); - nodesDestroyNode(pEvent->pEndCond); - break; - } - case QUERY_NODE_HINT: { - SHintNode* pHint = (SHintNode*)pNode; - destroyHintValue(pHint->option, pHint->value); - break; - } - case QUERY_NODE_VIEW: { - SViewNode* pView = (SViewNode*)pNode; - taosMemoryFreeClear(pView->pMeta); - taosMemoryFreeClear(pView->pVgroupList); - taosArrayDestroyEx(pView->pSmaIndexes, destroySmaIndex); - break; - } - case QUERY_NODE_SET_OPERATOR: { - SSetOperator* pStmt = (SSetOperator*)pNode; - nodesDestroyList(pStmt->pProjectionList); - nodesDestroyNode(pStmt->pLeft); - nodesDestroyNode(pStmt->pRight); - nodesDestroyList(pStmt->pOrderByList); - nodesDestroyNode(pStmt->pLimit); - break; - } - case QUERY_NODE_SELECT_STMT: { - SSelectStmt* pStmt = (SSelectStmt*)pNode; - nodesDestroyList(pStmt->pProjectionList); - nodesDestroyNode(pStmt->pFromTable); - nodesDestroyNode(pStmt->pWhere); - nodesDestroyList(pStmt->pPartitionByList); - nodesDestroyList(pStmt->pTags); - nodesDestroyNode(pStmt->pSubtable); - nodesDestroyNode(pStmt->pWindow); - nodesDestroyList(pStmt->pGroupByList); - nodesDestroyNode(pStmt->pHaving); - nodesDestroyNode(pStmt->pRange); - nodesDestroyNode(pStmt->pEvery); - nodesDestroyNode(pStmt->pFill); - nodesDestroyList(pStmt->pOrderByList); - nodesDestroyNode((SNode*)pStmt->pLimit); - nodesDestroyNode((SNode*)pStmt->pSlimit); - nodesDestroyList(pStmt->pHint); - break; - } - case QUERY_NODE_VNODE_MODIFY_STMT: { - SVnodeModifyOpStmt* pStmt = (SVnodeModifyOpStmt*)pNode; - destroyVgDataBlockArray(pStmt->pDataBlocks); - taosMemoryFreeClear(pStmt->pTableMeta); - nodesDestroyNode(pStmt->pTagCond); - taosArrayDestroy(pStmt->pTableTag); - taosHashCleanup(pStmt->pVgroupsHashObj); - taosHashCleanup(pStmt->pSubTableHashObj); - taosHashCleanup(pStmt->pTableNameHashObj); - taosHashCleanup(pStmt->pDbFNameHashObj); - if (pStmt->freeHashFunc) { - pStmt->freeHashFunc(pStmt->pTableBlockHashObj); - } - if (pStmt->freeArrayFunc) { - pStmt->freeArrayFunc(pStmt->pVgDataBlocks); - } - tdDestroySVCreateTbReq(pStmt->pCreateTblReq); - taosMemoryFreeClear(pStmt->pCreateTblReq); - if (pStmt->freeStbRowsCxtFunc) { - pStmt->freeStbRowsCxtFunc(pStmt->pStbRowsCxt); - } - taosMemoryFreeClear(pStmt->pStbRowsCxt); - taosCloseFile(&pStmt->fp); - break; - } - case QUERY_NODE_CREATE_DATABASE_STMT: - nodesDestroyNode((SNode*)((SCreateDatabaseStmt*)pNode)->pOptions); - break; - case QUERY_NODE_DROP_DATABASE_STMT: // no pointer field - break; - case QUERY_NODE_ALTER_DATABASE_STMT: - nodesDestroyNode((SNode*)((SAlterDatabaseStmt*)pNode)->pOptions); - break; - case QUERY_NODE_FLUSH_DATABASE_STMT: // no pointer field - case QUERY_NODE_TRIM_DATABASE_STMT: // no pointer field - break; - case QUERY_NODE_CREATE_TABLE_STMT: { - SCreateTableStmt* pStmt = (SCreateTableStmt*)pNode; - nodesDestroyList(pStmt->pCols); - nodesDestroyList(pStmt->pTags); - nodesDestroyNode((SNode*)pStmt->pOptions); - break; - } - case QUERY_NODE_CREATE_SUBTABLE_CLAUSE: { - SCreateSubTableClause* pStmt = (SCreateSubTableClause*)pNode; - nodesDestroyList(pStmt->pSpecificTags); - nodesDestroyList(pStmt->pValsOfTags); - nodesDestroyNode((SNode*)pStmt->pOptions); - break; - } - case QUERY_NODE_CREATE_MULTI_TABLES_STMT: - nodesDestroyList(((SCreateMultiTablesStmt*)pNode)->pSubTables); - break; - case QUERY_NODE_DROP_TABLE_CLAUSE: // no pointer field - break; - case QUERY_NODE_DROP_TABLE_STMT: - nodesDestroyList(((SDropTableStmt*)pNode)->pTables); - break; - case QUERY_NODE_DROP_SUPER_TABLE_STMT: // no pointer field - break; - case QUERY_NODE_ALTER_TABLE_STMT: - case QUERY_NODE_ALTER_SUPER_TABLE_STMT: { - SAlterTableStmt* pStmt = (SAlterTableStmt*)pNode; - nodesDestroyNode((SNode*)pStmt->pOptions); - nodesDestroyNode((SNode*)pStmt->pVal); - break; - } - case QUERY_NODE_CREATE_USER_STMT: { - SCreateUserStmt* pStmt = (SCreateUserStmt*)pNode; - taosMemoryFree(pStmt->pIpRanges); - nodesDestroyList(pStmt->pNodeListIpRanges); - break; - } - case QUERY_NODE_ALTER_USER_STMT: { - SAlterUserStmt* pStmt = (SAlterUserStmt*)pNode; - taosMemoryFree(pStmt->pIpRanges); - nodesDestroyList(pStmt->pNodeListIpRanges); - } - case QUERY_NODE_DROP_USER_STMT: // no pointer field - case QUERY_NODE_USE_DATABASE_STMT: // no pointer field - case QUERY_NODE_CREATE_DNODE_STMT: // no pointer field - case QUERY_NODE_DROP_DNODE_STMT: // no pointer field - case QUERY_NODE_ALTER_DNODE_STMT: // no pointer field - break; - case QUERY_NODE_CREATE_INDEX_STMT: { - SCreateIndexStmt* pStmt = (SCreateIndexStmt*)pNode; - nodesDestroyNode((SNode*)pStmt->pOptions); - nodesDestroyList(pStmt->pCols); - if (pStmt->pReq) { - tFreeSMCreateSmaReq(pStmt->pReq); - taosMemoryFreeClear(pStmt->pReq); - } - break; - } - case QUERY_NODE_DROP_INDEX_STMT: // no pointer field - case QUERY_NODE_CREATE_QNODE_STMT: // no pointer field - case QUERY_NODE_DROP_QNODE_STMT: // no pointer field - case QUERY_NODE_CREATE_BNODE_STMT: // no pointer field - case QUERY_NODE_DROP_BNODE_STMT: // no pointer field - case QUERY_NODE_CREATE_SNODE_STMT: // no pointer field - case QUERY_NODE_DROP_SNODE_STMT: // no pointer field - case QUERY_NODE_CREATE_MNODE_STMT: // no pointer field - case QUERY_NODE_DROP_MNODE_STMT: // no pointer field - break; - case QUERY_NODE_CREATE_TOPIC_STMT: - nodesDestroyNode(((SCreateTopicStmt*)pNode)->pQuery); - nodesDestroyNode(((SCreateTopicStmt*)pNode)->pWhere); - break; - case QUERY_NODE_DROP_TOPIC_STMT: // no pointer field - case QUERY_NODE_DROP_CGROUP_STMT: // no pointer field - case QUERY_NODE_ALTER_LOCAL_STMT: // no pointer field - break; - case QUERY_NODE_EXPLAIN_STMT: { - SExplainStmt* pStmt = (SExplainStmt*)pNode; - nodesDestroyNode((SNode*)pStmt->pOptions); - nodesDestroyNode(pStmt->pQuery); - break; - } - case QUERY_NODE_DESCRIBE_STMT: - taosMemoryFree(((SDescribeStmt*)pNode)->pMeta); - break; - case QUERY_NODE_RESET_QUERY_CACHE_STMT: // no pointer field - break; - case QUERY_NODE_COMPACT_DATABASE_STMT: { - SCompactDatabaseStmt* pStmt = (SCompactDatabaseStmt*)pNode; - nodesDestroyNode(pStmt->pStart); - nodesDestroyNode(pStmt->pEnd); - break; - } - case QUERY_NODE_CREATE_FUNCTION_STMT: // no pointer field - case QUERY_NODE_DROP_FUNCTION_STMT: // no pointer field - break; - case QUERY_NODE_CREATE_STREAM_STMT: { - SCreateStreamStmt* pStmt = (SCreateStreamStmt*)pNode; - nodesDestroyNode((SNode*)pStmt->pOptions); - nodesDestroyNode(pStmt->pQuery); - nodesDestroyList(pStmt->pTags); - nodesDestroyNode(pStmt->pSubtable); - tFreeSCMCreateStreamReq(pStmt->pReq); - taosMemoryFreeClear(pStmt->pReq); - break; - } - case QUERY_NODE_DROP_STREAM_STMT: // no pointer field - case QUERY_NODE_PAUSE_STREAM_STMT: // no pointer field - case QUERY_NODE_RESUME_STREAM_STMT: // no pointer field - case QUERY_NODE_BALANCE_VGROUP_STMT: // no pointer field - case QUERY_NODE_BALANCE_VGROUP_LEADER_STMT: // no pointer field - case QUERY_NODE_MERGE_VGROUP_STMT: // no pointer field - break; - case QUERY_NODE_REDISTRIBUTE_VGROUP_STMT: - nodesDestroyList(((SRedistributeVgroupStmt*)pNode)->pDnodes); - break; - case QUERY_NODE_SPLIT_VGROUP_STMT: // no pointer field - case QUERY_NODE_SYNCDB_STMT: // no pointer field - break; - case QUERY_NODE_GRANT_STMT: - nodesDestroyNode(((SGrantStmt*)pNode)->pTagCond); - break; - case QUERY_NODE_REVOKE_STMT: - nodesDestroyNode(((SRevokeStmt*)pNode)->pTagCond); - break; - case QUERY_NODE_SHOW_DNODES_STMT: - case QUERY_NODE_SHOW_MNODES_STMT: - case QUERY_NODE_SHOW_MODULES_STMT: - case QUERY_NODE_SHOW_QNODES_STMT: - case QUERY_NODE_SHOW_SNODES_STMT: - case QUERY_NODE_SHOW_BNODES_STMT: - case QUERY_NODE_SHOW_CLUSTER_STMT: - case QUERY_NODE_SHOW_DATABASES_STMT: - case QUERY_NODE_SHOW_FUNCTIONS_STMT: - case QUERY_NODE_SHOW_INDEXES_STMT: - case QUERY_NODE_SHOW_STABLES_STMT: - case QUERY_NODE_SHOW_STREAMS_STMT: - case QUERY_NODE_SHOW_TABLES_STMT: - case QUERY_NODE_SHOW_USERS_STMT: - case QUERY_NODE_SHOW_LICENCES_STMT: - case QUERY_NODE_SHOW_VGROUPS_STMT: - case QUERY_NODE_SHOW_TOPICS_STMT: - case QUERY_NODE_SHOW_CONSUMERS_STMT: - case QUERY_NODE_SHOW_CONNECTIONS_STMT: - case QUERY_NODE_SHOW_QUERIES_STMT: - case QUERY_NODE_SHOW_VNODES_STMT: - case QUERY_NODE_SHOW_APPS_STMT: - case QUERY_NODE_SHOW_SCORES_STMT: - case QUERY_NODE_SHOW_VARIABLES_STMT: - case QUERY_NODE_SHOW_LOCAL_VARIABLES_STMT: - case QUERY_NODE_SHOW_TRANSACTIONS_STMT: - case QUERY_NODE_SHOW_SUBSCRIPTIONS_STMT: - case QUERY_NODE_SHOW_TAGS_STMT: - case QUERY_NODE_SHOW_USER_PRIVILEGES_STMT: - case QUERY_NODE_SHOW_VIEWS_STMT: { - SShowStmt* pStmt = (SShowStmt*)pNode; - nodesDestroyNode(pStmt->pDbName); - nodesDestroyNode(pStmt->pTbName); - break; - } - case QUERY_NODE_SHOW_TABLE_TAGS_STMT: { - SShowTableTagsStmt* pStmt = (SShowTableTagsStmt*)pNode; - nodesDestroyNode(pStmt->pDbName); - nodesDestroyNode(pStmt->pTbName); - nodesDestroyList(pStmt->pTags); - break; - } - case QUERY_NODE_SHOW_DNODE_VARIABLES_STMT: - nodesDestroyNode(((SShowDnodeVariablesStmt*)pNode)->pDnodeId); - nodesDestroyNode(((SShowDnodeVariablesStmt*)pNode)->pLikePattern); - break; - case QUERY_NODE_SHOW_CREATE_DATABASE_STMT: - taosMemoryFreeClear(((SShowCreateDatabaseStmt*)pNode)->pCfg); - break; - case QUERY_NODE_SHOW_CREATE_TABLE_STMT: - case QUERY_NODE_SHOW_CREATE_STABLE_STMT: - taosMemoryFreeClear(((SShowCreateTableStmt*)pNode)->pDbCfg); - destroyTableCfg((STableCfg*)(((SShowCreateTableStmt*)pNode)->pTableCfg)); - break; - case QUERY_NODE_SHOW_CREATE_VIEW_STMT: // no pointer field - case QUERY_NODE_SHOW_TABLE_DISTRIBUTED_STMT: // no pointer field - case QUERY_NODE_KILL_CONNECTION_STMT: // no pointer field - case QUERY_NODE_KILL_QUERY_STMT: // no pointer field - case QUERY_NODE_KILL_TRANSACTION_STMT: // no pointer field - break; - case QUERY_NODE_DELETE_STMT: { - SDeleteStmt* pStmt = (SDeleteStmt*)pNode; - nodesDestroyNode(pStmt->pFromTable); - nodesDestroyNode(pStmt->pWhere); - nodesDestroyNode(pStmt->pCountFunc); - nodesDestroyNode(pStmt->pFirstFunc); - nodesDestroyNode(pStmt->pLastFunc); - nodesDestroyNode(pStmt->pTagCond); - break; - } - case QUERY_NODE_INSERT_STMT: { - SInsertStmt* pStmt = (SInsertStmt*)pNode; - nodesDestroyNode(pStmt->pTable); - nodesDestroyList(pStmt->pCols); - nodesDestroyNode(pStmt->pQuery); - break; - } - case QUERY_NODE_QUERY: { - SQuery* pQuery = (SQuery*)pNode; - nodesDestroyNode(pQuery->pPrevRoot); - nodesDestroyNode(pQuery->pRoot); - nodesDestroyNode(pQuery->pPostRoot); - taosMemoryFreeClear(pQuery->pResSchema); - if (NULL != pQuery->pCmdMsg) { - taosMemoryFreeClear(pQuery->pCmdMsg->pMsg); - taosMemoryFreeClear(pQuery->pCmdMsg); - } - taosArrayDestroy(pQuery->pDbList); - taosArrayDestroy(pQuery->pTableList); - taosArrayDestroy(pQuery->pTargetTableList); - taosArrayDestroy(pQuery->pPlaceholderValues); - nodesDestroyNode(pQuery->pPrepareRoot); - break; - } - case QUERY_NODE_RESTORE_DNODE_STMT: // no pointer field - case QUERY_NODE_RESTORE_QNODE_STMT: // no pointer field - case QUERY_NODE_RESTORE_MNODE_STMT: // no pointer field - case QUERY_NODE_RESTORE_VNODE_STMT: // no pointer field - break; - case QUERY_NODE_CREATE_VIEW_STMT: { - SCreateViewStmt* pStmt = (SCreateViewStmt*)pNode; - taosMemoryFree(pStmt->pQuerySql); - tFreeSCMCreateViewReq(&pStmt->createReq); - nodesDestroyNode(pStmt->pQuery); - break; - } - case QUERY_NODE_DROP_VIEW_STMT: - break; - case QUERY_NODE_LOGIC_PLAN_SCAN: { - SScanLogicNode* pLogicNode = (SScanLogicNode*)pNode; - destroyLogicNode((SLogicNode*)pLogicNode); - nodesDestroyList(pLogicNode->pScanCols); - nodesDestroyList(pLogicNode->pScanPseudoCols); - taosMemoryFreeClear(pLogicNode->pVgroupList); - nodesDestroyList(pLogicNode->pDynamicScanFuncs); - nodesDestroyNode(pLogicNode->pTagCond); - nodesDestroyNode(pLogicNode->pTagIndexCond); - taosArrayDestroyEx(pLogicNode->pSmaIndexes, destroySmaIndex); - nodesDestroyList(pLogicNode->pGroupTags); - nodesDestroyList(pLogicNode->pTags); - nodesDestroyNode(pLogicNode->pSubtable); - break; - } - case QUERY_NODE_LOGIC_PLAN_JOIN: { - SJoinLogicNode* pLogicNode = (SJoinLogicNode*)pNode; - destroyLogicNode((SLogicNode*)pLogicNode); - nodesDestroyNode(pLogicNode->pPrimKeyEqCond); - nodesDestroyNode(pLogicNode->pOtherOnCond); - nodesDestroyNode(pLogicNode->pColEqCond); - break; - } - case QUERY_NODE_LOGIC_PLAN_AGG: { - SAggLogicNode* pLogicNode = (SAggLogicNode*)pNode; - destroyLogicNode((SLogicNode*)pLogicNode); - nodesDestroyList(pLogicNode->pAggFuncs); - nodesDestroyList(pLogicNode->pGroupKeys); - break; - } - case QUERY_NODE_LOGIC_PLAN_PROJECT: { - SProjectLogicNode* pLogicNode = (SProjectLogicNode*)pNode; - destroyLogicNode((SLogicNode*)pLogicNode); - nodesDestroyList(pLogicNode->pProjections); - break; - } - case QUERY_NODE_LOGIC_PLAN_VNODE_MODIFY: { - SVnodeModifyLogicNode* pLogicNode = (SVnodeModifyLogicNode*)pNode; - destroyLogicNode((SLogicNode*)pLogicNode); - destroyVgDataBlockArray(pLogicNode->pDataBlocks); - // pVgDataBlocks is weak reference - nodesDestroyNode(pLogicNode->pAffectedRows); - nodesDestroyNode(pLogicNode->pStartTs); - nodesDestroyNode(pLogicNode->pEndTs); - taosMemoryFreeClear(pLogicNode->pVgroupList); - nodesDestroyList(pLogicNode->pInsertCols); - break; - } - case QUERY_NODE_LOGIC_PLAN_EXCHANGE: - destroyLogicNode((SLogicNode*)pNode); - break; - case QUERY_NODE_LOGIC_PLAN_MERGE: { - SMergeLogicNode* pLogicNode = (SMergeLogicNode*)pNode; - destroyLogicNode((SLogicNode*)pLogicNode); - nodesDestroyList(pLogicNode->pMergeKeys); - nodesDestroyList(pLogicNode->pInputs); - break; - } - case QUERY_NODE_LOGIC_PLAN_WINDOW: { - SWindowLogicNode* pLogicNode = (SWindowLogicNode*)pNode; - destroyLogicNode((SLogicNode*)pLogicNode); - nodesDestroyList(pLogicNode->pFuncs); - nodesDestroyNode(pLogicNode->pTspk); - nodesDestroyNode(pLogicNode->pTsEnd); - nodesDestroyNode(pLogicNode->pStateExpr); - nodesDestroyNode(pLogicNode->pStartCond); - nodesDestroyNode(pLogicNode->pEndCond); - break; - } - case QUERY_NODE_LOGIC_PLAN_FILL: { - SFillLogicNode* pLogicNode = (SFillLogicNode*)pNode; - destroyLogicNode((SLogicNode*)pLogicNode); - nodesDestroyNode(pLogicNode->pWStartTs); - nodesDestroyNode(pLogicNode->pValues); - nodesDestroyList(pLogicNode->pFillExprs); - nodesDestroyList(pLogicNode->pNotFillExprs); - break; - } - case QUERY_NODE_LOGIC_PLAN_SORT: { - SSortLogicNode* pLogicNode = (SSortLogicNode*)pNode; - destroyLogicNode((SLogicNode*)pLogicNode); - nodesDestroyList(pLogicNode->pSortKeys); - break; - } - case QUERY_NODE_LOGIC_PLAN_PARTITION: { - SPartitionLogicNode* pLogicNode = (SPartitionLogicNode*)pNode; - destroyLogicNode((SLogicNode*)pLogicNode); - nodesDestroyList(pLogicNode->pPartitionKeys); - nodesDestroyList(pLogicNode->pTags); - nodesDestroyNode(pLogicNode->pSubtable); - nodesDestroyList(pLogicNode->pAggFuncs); - break; - } - case QUERY_NODE_LOGIC_PLAN_INDEF_ROWS_FUNC: { - SIndefRowsFuncLogicNode* pLogicNode = (SIndefRowsFuncLogicNode*)pNode; - destroyLogicNode((SLogicNode*)pLogicNode); - nodesDestroyList(pLogicNode->pFuncs); - break; - } - case QUERY_NODE_LOGIC_PLAN_INTERP_FUNC: { - SInterpFuncLogicNode* pLogicNode = (SInterpFuncLogicNode*)pNode; - destroyLogicNode((SLogicNode*)pLogicNode); - nodesDestroyList(pLogicNode->pFuncs); - nodesDestroyNode(pLogicNode->pFillValues); - nodesDestroyNode(pLogicNode->pTimeSeries); - break; - } - case QUERY_NODE_LOGIC_PLAN_GROUP_CACHE: { - SGroupCacheLogicNode* pLogicNode = (SGroupCacheLogicNode*)pNode; - destroyLogicNode((SLogicNode*)pLogicNode); - nodesDestroyList(pLogicNode->pGroupCols); - break; - } - case QUERY_NODE_LOGIC_PLAN_DYN_QUERY_CTRL: { - SDynQueryCtrlLogicNode* pLogicNode = (SDynQueryCtrlLogicNode*)pNode; - destroyLogicNode((SLogicNode*)pLogicNode); - break; - } - case QUERY_NODE_LOGIC_SUBPLAN: { - SLogicSubplan* pSubplan = (SLogicSubplan*)pNode; - nodesDestroyList(pSubplan->pChildren); - nodesDestroyNode((SNode*)pSubplan->pNode); - nodesClearList(pSubplan->pParents); - taosMemoryFreeClear(pSubplan->pVgroupList); - break; - } - case QUERY_NODE_LOGIC_PLAN: - nodesDestroyList(((SQueryLogicPlan*)pNode)->pTopSubplans); - break; - case QUERY_NODE_PHYSICAL_PLAN_TAG_SCAN: - case QUERY_NODE_PHYSICAL_PLAN_SYSTABLE_SCAN: - case QUERY_NODE_PHYSICAL_PLAN_BLOCK_DIST_SCAN: - destroyScanPhysiNode((SScanPhysiNode*)pNode); - break; - case QUERY_NODE_PHYSICAL_PLAN_LAST_ROW_SCAN: - case QUERY_NODE_PHYSICAL_PLAN_TABLE_COUNT_SCAN: { - SLastRowScanPhysiNode* pPhyNode = (SLastRowScanPhysiNode*)pNode; - destroyScanPhysiNode((SScanPhysiNode*)pNode); - nodesDestroyList(pPhyNode->pGroupTags); - nodesDestroyList(pPhyNode->pTargets); - break; - } - case QUERY_NODE_PHYSICAL_PLAN_TABLE_SCAN: - case QUERY_NODE_PHYSICAL_PLAN_TABLE_SEQ_SCAN: - case QUERY_NODE_PHYSICAL_PLAN_TABLE_MERGE_SCAN: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN: { - STableScanPhysiNode* pPhyNode = (STableScanPhysiNode*)pNode; - destroyScanPhysiNode((SScanPhysiNode*)pNode); - nodesDestroyList(pPhyNode->pDynamicScanFuncs); - nodesDestroyList(pPhyNode->pGroupTags); - nodesDestroyList(pPhyNode->pTags); - nodesDestroyNode(pPhyNode->pSubtable); - break; - } - case QUERY_NODE_PHYSICAL_PLAN_PROJECT: { - SProjectPhysiNode* pPhyNode = (SProjectPhysiNode*)pNode; - destroyPhysiNode((SPhysiNode*)pPhyNode); - nodesDestroyList(pPhyNode->pProjections); - break; - } - case QUERY_NODE_PHYSICAL_PLAN_MERGE_JOIN: { - SSortMergeJoinPhysiNode* pPhyNode = (SSortMergeJoinPhysiNode*)pNode; - destroyPhysiNode((SPhysiNode*)pPhyNode); - nodesDestroyNode(pPhyNode->pPrimKeyCond); - nodesDestroyNode(pPhyNode->pOtherOnCond); - nodesDestroyList(pPhyNode->pTargets); - nodesDestroyNode(pPhyNode->pColEqCond); - break; - } - case QUERY_NODE_PHYSICAL_PLAN_HASH_JOIN: { - SHashJoinPhysiNode* pPhyNode = (SHashJoinPhysiNode*)pNode; - destroyPhysiNode((SPhysiNode*)pPhyNode); - nodesDestroyList(pPhyNode->pOnLeft); - nodesDestroyList(pPhyNode->pOnRight); - nodesDestroyNode(pPhyNode->pFilterConditions); - nodesDestroyList(pPhyNode->pTargets); - - nodesDestroyNode(pPhyNode->pPrimKeyCond); - nodesDestroyNode(pPhyNode->pColEqCond); - nodesDestroyNode(pPhyNode->pTagEqCond); - break; - } - case QUERY_NODE_PHYSICAL_PLAN_HASH_AGG: { - SAggPhysiNode* pPhyNode = (SAggPhysiNode*)pNode; - destroyPhysiNode((SPhysiNode*)pPhyNode); - nodesDestroyList(pPhyNode->pExprs); - nodesDestroyList(pPhyNode->pAggFuncs); - nodesDestroyList(pPhyNode->pGroupKeys); - break; - } - case QUERY_NODE_PHYSICAL_PLAN_EXCHANGE: { - SExchangePhysiNode* pPhyNode = (SExchangePhysiNode*)pNode; - destroyPhysiNode((SPhysiNode*)pPhyNode); - nodesDestroyList(pPhyNode->pSrcEndPoints); - break; - } - case QUERY_NODE_PHYSICAL_PLAN_MERGE: { - SMergePhysiNode* pPhyNode = (SMergePhysiNode*)pNode; - destroyPhysiNode((SPhysiNode*)pPhyNode); - nodesDestroyList(pPhyNode->pMergeKeys); - nodesDestroyList(pPhyNode->pTargets); - break; - } - case QUERY_NODE_PHYSICAL_PLAN_SORT: - case QUERY_NODE_PHYSICAL_PLAN_GROUP_SORT: { - SSortPhysiNode* pPhyNode = (SSortPhysiNode*)pNode; - destroyPhysiNode((SPhysiNode*)pPhyNode); - nodesDestroyList(pPhyNode->pExprs); - nodesDestroyList(pPhyNode->pSortKeys); - nodesDestroyList(pPhyNode->pTargets); - break; - } - case QUERY_NODE_PHYSICAL_PLAN_HASH_INTERVAL: - case QUERY_NODE_PHYSICAL_PLAN_MERGE_ALIGNED_INTERVAL: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL: - destroyWinodwPhysiNode((SWindowPhysiNode*)pNode); - break; - case QUERY_NODE_PHYSICAL_PLAN_FILL: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_FILL: { - SFillPhysiNode* pPhyNode = (SFillPhysiNode*)pNode; - destroyPhysiNode((SPhysiNode*)pPhyNode); - nodesDestroyList(pPhyNode->pFillExprs); - nodesDestroyList(pPhyNode->pNotFillExprs); - nodesDestroyNode(pPhyNode->pWStartTs); - nodesDestroyNode(pPhyNode->pValues); - break; - } - case QUERY_NODE_PHYSICAL_PLAN_MERGE_SESSION: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_SESSION: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_SESSION: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_SESSION: - destroyWinodwPhysiNode((SWindowPhysiNode*)pNode); - break; - case QUERY_NODE_PHYSICAL_PLAN_MERGE_STATE: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_STATE: { - SStateWinodwPhysiNode* pPhyNode = (SStateWinodwPhysiNode*)pNode; - destroyWinodwPhysiNode((SWindowPhysiNode*)pPhyNode); - nodesDestroyNode(pPhyNode->pStateKey); - break; - } - case QUERY_NODE_PHYSICAL_PLAN_MERGE_EVENT: - case QUERY_NODE_PHYSICAL_PLAN_STREAM_EVENT: { - SEventWinodwPhysiNode* pPhyNode = (SEventWinodwPhysiNode*)pNode; - destroyWinodwPhysiNode((SWindowPhysiNode*)pPhyNode); - nodesDestroyNode(pPhyNode->pStartCond); - nodesDestroyNode(pPhyNode->pEndCond); - break; - } - case QUERY_NODE_PHYSICAL_PLAN_PARTITION: { - destroyPartitionPhysiNode((SPartitionPhysiNode*)pNode); - break; - } - case QUERY_NODE_PHYSICAL_PLAN_STREAM_PARTITION: { - SStreamPartitionPhysiNode* pPhyNode = (SStreamPartitionPhysiNode*)pNode; - destroyPartitionPhysiNode((SPartitionPhysiNode*)pPhyNode); - nodesDestroyList(pPhyNode->pTags); - nodesDestroyNode(pPhyNode->pSubtable); - break; - } - case QUERY_NODE_PHYSICAL_PLAN_INDEF_ROWS_FUNC: { - SIndefRowsFuncPhysiNode* pPhyNode = (SIndefRowsFuncPhysiNode*)pNode; - destroyPhysiNode((SPhysiNode*)pPhyNode); - nodesDestroyList(pPhyNode->pExprs); - nodesDestroyList(pPhyNode->pFuncs); - break; - } - case QUERY_NODE_PHYSICAL_PLAN_INTERP_FUNC: { - SInterpFuncPhysiNode* pPhyNode = (SInterpFuncPhysiNode*)pNode; - destroyPhysiNode((SPhysiNode*)pPhyNode); - nodesDestroyList(pPhyNode->pExprs); - nodesDestroyList(pPhyNode->pFuncs); - nodesDestroyNode(pPhyNode->pFillValues); - nodesDestroyNode(pPhyNode->pTimeSeries); - break; - } - case QUERY_NODE_PHYSICAL_PLAN_DISPATCH: - destroyDataSinkNode((SDataSinkNode*)pNode); - break; - case QUERY_NODE_PHYSICAL_PLAN_INSERT: { - SDataInserterNode* pSink = (SDataInserterNode*)pNode; - destroyDataSinkNode((SDataSinkNode*)pSink); - taosMemoryFreeClear(pSink->pData); - break; - } - case QUERY_NODE_PHYSICAL_PLAN_QUERY_INSERT: { - SQueryInserterNode* pSink = (SQueryInserterNode*)pNode; - destroyDataSinkNode((SDataSinkNode*)pSink); - nodesDestroyList(pSink->pCols); - break; - } - case QUERY_NODE_PHYSICAL_PLAN_DELETE: { - SDataDeleterNode* pSink = (SDataDeleterNode*)pNode; - destroyDataSinkNode((SDataSinkNode*)pSink); - nodesDestroyNode(pSink->pAffectedRows); - nodesDestroyNode(pSink->pStartTs); - nodesDestroyNode(pSink->pEndTs); - break; - } - case QUERY_NODE_PHYSICAL_PLAN_GROUP_CACHE: { - SGroupCachePhysiNode* pPhyNode = (SGroupCachePhysiNode*)pNode; - destroyPhysiNode((SPhysiNode*)pPhyNode); - nodesDestroyList(pPhyNode->pGroupCols); - break; - } - case QUERY_NODE_PHYSICAL_PLAN_DYN_QUERY_CTRL: { - SDynQueryCtrlPhysiNode* pPhyNode = (SDynQueryCtrlPhysiNode*)pNode; - destroyPhysiNode((SPhysiNode*)pPhyNode); - break; - } - case QUERY_NODE_PHYSICAL_SUBPLAN: { - SSubplan* pSubplan = (SSubplan*)pNode; - nodesClearList(pSubplan->pChildren); - nodesDestroyNode((SNode*)pSubplan->pNode); - nodesDestroyNode((SNode*)pSubplan->pDataSink); - nodesDestroyNode((SNode*)pSubplan->pTagCond); - nodesDestroyNode((SNode*)pSubplan->pTagIndexCond); - nodesClearList(pSubplan->pParents); - break; - } - case QUERY_NODE_PHYSICAL_PLAN: - nodesDestroyList(((SQueryPlan*)pNode)->pSubplans); - break; - default: - break; - } - nodesFree(pNode); - return; -} SNodeList* nodesMakeList() { SNodeList* p = nodesCalloc(1, sizeof(SNodeList)); From 59a8fab0d76f93cc75169fd10da23ab474c66604 Mon Sep 17 00:00:00 2001 From: Bob Liu Date: Sun, 19 Nov 2023 17:11:26 +0800 Subject: [PATCH 134/195] adjust init funcNodes --- source/libs/nodes/src/nodesCodeFuncs.c | 404 ++++++++++++------------- 1 file changed, 202 insertions(+), 202 deletions(-) diff --git a/source/libs/nodes/src/nodesCodeFuncs.c b/source/libs/nodes/src/nodesCodeFuncs.c index 01d808c7d8..981e2bbaca 100644 --- a/source/libs/nodes/src/nodesCodeFuncs.c +++ b/source/libs/nodes/src/nodesCodeFuncs.c @@ -43,7 +43,7 @@ typedef struct SBuiltinNodeDefinition { FExecDestoryNode destoryFunc; } SBuiltinNodeDefinition; -SBuiltinNodeDefinition funcNodes[QUERY_NODE_END] = {}; +SBuiltinNodeDefinition funcNodes[QUERY_NODE_END] = {NULL}; static TdThreadOnce functionNodeInit = PTHREAD_ONCE_INIT; static int32_t initNodeCode = -1; @@ -7439,1407 +7439,1407 @@ void nodesDestroyNode(SNode* pNode) { // clang-format off static void doInitNodeFuncArray() { - setFunc("Column", + setFunc("Column", QUERY_NODE_COLUMN, sizeof(SColumnNode), columnNodeToJson, jsonToColumnNode, destroyColumnNode ); - setFunc("Value", + setFunc("Value", QUERY_NODE_VALUE, sizeof(SValueNode), valueNodeToJson, jsonToValueNode, destroyValueNode ); - setFunc("Operator", + setFunc("Operator", QUERY_NODE_OPERATOR, sizeof(SOperatorNode), operatorNodeToJson, jsonToOperatorNode, destroyOperatorNode ); - setFunc("LogicCondition", + setFunc("LogicCondition", QUERY_NODE_LOGIC_CONDITION, sizeof(SLogicConditionNode), logicConditionNodeToJson, jsonToLogicConditionNode, destoryLogicConditionNode ); - setFunc("Function", + setFunc("Function", QUERY_NODE_FUNCTION, sizeof(SFunctionNode), functionNodeToJson, jsonToFunctionNode, destoryFunctionNode ); - setFunc("RealTable", + setFunc("RealTable", QUERY_NODE_REAL_TABLE, sizeof(SRealTableNode), realTableNodeToJson, jsonToRealTableNode, destoryRealTableNode ); - setFunc("TempTable", + setFunc("TempTable", QUERY_NODE_TEMP_TABLE, sizeof(STempTableNode), tempTableNodeToJson, jsonToTempTableNode, destoryTempTableNode ); - setFunc("JoinTable", + setFunc("JoinTable", QUERY_NODE_JOIN_TABLE, sizeof(SJoinTableNode), joinTableNodeToJson, jsonToJoinTableNode, destoryJoinTableNode ); - setFunc("GroupingSet", + setFunc("GroupingSet", QUERY_NODE_GROUPING_SET, sizeof(SGroupingSetNode), groupingSetNodeToJson, jsonToGroupingSetNode, destoryGroupingSetNode ); - setFunc("OrderByExpr", + setFunc("OrderByExpr", QUERY_NODE_ORDER_BY_EXPR, sizeof(SOrderByExprNode), orderByExprNodeToJson, jsonToOrderByExprNode, destoryOrderByExprNode ); - setFunc("Limit", + setFunc("Limit", QUERY_NODE_LIMIT, sizeof(SLimitNode), limitNodeToJson, jsonToLimitNode, destoryXNode ); - setFunc("StateWindow", + setFunc("StateWindow", QUERY_NODE_STATE_WINDOW, sizeof(SStateWindowNode), stateWindowNodeToJson, jsonToStateWindowNode, destoryStateWindowNode ); - setFunc("SessionWinow", + setFunc("SessionWinow", QUERY_NODE_SESSION_WINDOW, sizeof(SSessionWindowNode), sessionWindowNodeToJson, jsonToSessionWindowNode, destorySessionWindowNode ); - setFunc("IntervalWindow", + setFunc("IntervalWindow", QUERY_NODE_INTERVAL_WINDOW, sizeof(SIntervalWindowNode), intervalWindowNodeToJson, jsonToIntervalWindowNode, destoryIntervalWindowNode ); - setFunc("NodeList", + setFunc("NodeList", QUERY_NODE_NODE_LIST, sizeof(SNodeListNode), nodeListNodeToJson, jsonToNodeListNode, destoryNodeListNode ); - setFunc("Fill", + setFunc("Fill", QUERY_NODE_FILL, sizeof(SFillNode), fillNodeToJson, jsonToFillNode, destoryFillNode ); - setFunc("RawExpr", + setFunc("RawExpr", QUERY_NODE_RAW_EXPR, sizeof(SRawExprNode), emptyNodeToJson, emptyJsonToNode, destoryRawExprNode ); - setFunc("Target", + setFunc("Target", QUERY_NODE_TARGET, sizeof(STargetNode), targetNodeToJson, jsonToTargetNode, destoryTargetNode ); - setFunc("DataBlockDesc", + setFunc("DataBlockDesc", QUERY_NODE_DATABLOCK_DESC, sizeof(SDataBlockDescNode), dataBlockDescNodeToJson, jsonToDataBlockDescNode, destoryDataBlockDescNode ); - setFunc("SlotDesc", + setFunc("SlotDesc", QUERY_NODE_SLOT_DESC, sizeof(SSlotDescNode), slotDescNodeToJson, jsonToSlotDescNode, destoryXNode ); - setFunc("ColumnDef", + setFunc("ColumnDef", QUERY_NODE_COLUMN_DEF, sizeof(SColumnDefNode), columnDefNodeToJson, jsonToColumnDefNode, destoryXNode ); - setFunc("DownstreamSource", + setFunc("DownstreamSource", QUERY_NODE_DOWNSTREAM_SOURCE, sizeof(SDownstreamSourceNode), downstreamSourceNodeToJson, jsonToDownstreamSourceNode, destoryXNode ); - setFunc("DatabaseOptions", + setFunc("DatabaseOptions", QUERY_NODE_DATABASE_OPTIONS, sizeof(SDatabaseOptions), databaseOptionsToJson, jsonToDatabaseOptions, destoryDatabaseOptions ); - setFunc("TableOptions", + setFunc("TableOptions", QUERY_NODE_TABLE_OPTIONS, sizeof(STableOptions), tableOptionsToJson, jsonToTableOptions, destoryTableOptions ); - setFunc("IndexOptions", + setFunc("IndexOptions", QUERY_NODE_INDEX_OPTIONS, sizeof(SIndexOptions), indexOptionsToJson, jsonToIndexOptions, destoryIndexOptions ); - setFunc("ExplainOptions", + setFunc("ExplainOptions", QUERY_NODE_EXPLAIN_OPTIONS, sizeof(SExplainOptions), explainOptionsToJson, jsonToExplainOptions, destoryXNode ); - setFunc("StreamOptions", + setFunc("StreamOptions", QUERY_NODE_STREAM_OPTIONS, sizeof(SStreamOptions), streamOptionsToJson, jsonToStreamOptions, destoryStreamOptions ); - setFunc("LeftValue", + setFunc("LeftValue", QUERY_NODE_LEFT_VALUE, sizeof(SLeftValueNode), emptyNodeToJson, emptyJsonToNode, destoryXNode ); - setFunc("ColumnRef", + setFunc("ColumnRef", QUERY_NODE_COLUMN_REF, sizeof(SColumnDefNode), emptyNodeToJson, emptyJsonToNode, destoryXNode ); - setFunc("WhenThen", + setFunc("WhenThen", QUERY_NODE_WHEN_THEN, sizeof(SWhenThenNode), whenThenNodeToJson, jsonToWhenThenNode, destoryWhenThenNode ); - setFunc("CaseWhen", + setFunc("CaseWhen", QUERY_NODE_CASE_WHEN, sizeof(SCaseWhenNode), caseWhenNodeToJson, jsonToCaseWhenNode, destoryCaseWhenNode ); - setFunc("EventWindow", + setFunc("EventWindow", QUERY_NODE_EVENT_WINDOW, sizeof(SEventWindowNode), eventWindowNodeToJson, jsonToEventWindowNode, destoryEventWindowNode ); - setFunc("HintNode", + setFunc("HintNode", QUERY_NODE_HINT, sizeof(SHintNode), emptyNodeToJson, emptyJsonToNode, destoryHintNode ); - setFunc("ViewNode", + setFunc("ViewNode", QUERY_NODE_VIEW, sizeof(SViewNode), emptyNodeToJson, emptyJsonToNode, destoryViewNode ); - setFunc("SetOperator", + setFunc("SetOperator", QUERY_NODE_SET_OPERATOR, sizeof(SSetOperator), setOperatorToJson, jsonToSetOperator, destorySetOperator ); - setFunc("SelectStmt", + setFunc("SelectStmt", QUERY_NODE_SELECT_STMT, sizeof(SSelectStmt), selectStmtToJson, jsonToSelectStmt, destorySelectStmt ); - setFunc("VnodeModifyStmt", + setFunc("VnodeModifyStmt", QUERY_NODE_VNODE_MODIFY_STMT, sizeof(SVnodeModifyOpStmt), vnodeModifyStmtToJson, jsonToVnodeModifyStmt, destoryVnodeModifyOpStmt ); - setFunc("CreateDatabaseStmt", + setFunc("CreateDatabaseStmt", QUERY_NODE_CREATE_DATABASE_STMT, sizeof(SCreateDatabaseStmt), createDatabaseStmtToJson, jsonToCreateDatabaseStmt, destoryCreateDatabaseStmt ); - setFunc("DropDatabaseStmt", + setFunc("DropDatabaseStmt", QUERY_NODE_DROP_DATABASE_STMT, sizeof(SDropDatabaseStmt), emptyNodeToJson, emptyJsonToNode, destoryXNode ); - setFunc("AlterDatabaseStmt", + setFunc("AlterDatabaseStmt", QUERY_NODE_ALTER_DATABASE_STMT, sizeof(SAlterDatabaseStmt), alterDatabaseStmtToJson, jsonToAlterDatabaseStmt, destoryAlterDatabaseStmt ); - setFunc("FlushDatabaseStmt", + setFunc("FlushDatabaseStmt", QUERY_NODE_FLUSH_DATABASE_STMT, sizeof(SFlushDatabaseStmt), emptyNodeToJson, emptyJsonToNode, destoryXNode ); - setFunc("TrimDatabaseStmt", + setFunc("TrimDatabaseStmt", QUERY_NODE_TRIM_DATABASE_STMT, sizeof(STrimDatabaseStmt), trimDatabaseStmtToJson, jsonToTrimDatabaseStmt, destoryXNode ); - setFunc("CreateTableStmt", + setFunc("CreateTableStmt", QUERY_NODE_CREATE_TABLE_STMT, sizeof(SCreateTableStmt), createTableStmtToJson, jsonToCreateTableStmt, destoryCreateTableStmt ); - setFunc("CreateSubtableClause", + setFunc("CreateSubtableClause", QUERY_NODE_CREATE_SUBTABLE_CLAUSE, sizeof(SCreateSubTableClause), createSubTableClauseToJson, jsonToCreateSubTableClause, destoryCreateSubTableClause ); - setFunc("CreateMultiTableStmt", + setFunc("CreateMultiTableStmt", QUERY_NODE_CREATE_MULTI_TABLES_STMT, sizeof(SCreateMultiTablesStmt), createMultiTablesStmtToJson, jsonToCreateMultiTablesStmt, destoryCreateMultiTablesStmt ); - setFunc("DropTableClause", + setFunc("DropTableClause", QUERY_NODE_DROP_TABLE_CLAUSE, sizeof(SDropTableClause), dropTableClauseToJson, jsonToDropTableClause, destoryXNode ); - setFunc("DropTableStmt", + setFunc("DropTableStmt", QUERY_NODE_DROP_TABLE_STMT, sizeof(SDropTableStmt), dropTableStmtToJson, jsonToDropTableStmt, destoryDropTableStmt ); - setFunc("DropSuperTableStmt", + setFunc("DropSuperTableStmt", QUERY_NODE_DROP_SUPER_TABLE_STMT, sizeof(SDropSuperTableStmt), dropStableStmtToJson, jsonToDropStableStmt, destoryXNode ); - setFunc("AlterTableStmt", + setFunc("AlterTableStmt", QUERY_NODE_ALTER_TABLE_STMT, sizeof(SAlterTableStmt), alterTableStmtToJson, jsonToAlterTableStmt, destoryAlterTableStmt ); - setFunc("AlterSuperTableStmt", + setFunc("AlterSuperTableStmt", QUERY_NODE_ALTER_SUPER_TABLE_STMT, sizeof(SAlterTableStmt), alterStableStmtToJson, jsonToAlterStableStmt, destoryAlterTableStmt ); - setFunc("CreateUserStmt", + setFunc("CreateUserStmt", QUERY_NODE_CREATE_USER_STMT, sizeof(SCreateUserStmt), createUserStmtToJson, jsonToCreateUserStmt, destoryCreateUserStmt ); - setFunc("AlterUserStmt", + setFunc("AlterUserStmt", QUERY_NODE_ALTER_USER_STMT, sizeof(SAlterUserStmt), alterUserStmtToJson, jsonToAlterUserStmt, destoryAlterUserStmt ); - setFunc("DropUserStmt", + setFunc("DropUserStmt", QUERY_NODE_DROP_USER_STMT, sizeof(SDropUserStmt), dropUserStmtToJson, jsonToDropUserStmt, destoryXNode ); - setFunc("UseDatabaseStmt", + setFunc("UseDatabaseStmt", QUERY_NODE_USE_DATABASE_STMT, sizeof(SUseDatabaseStmt), useDatabaseStmtToJson, jsonToUseDatabaseStmt, destoryXNode ); - setFunc("CreateDnodeStmt", + setFunc("CreateDnodeStmt", QUERY_NODE_CREATE_DNODE_STMT, sizeof(SCreateDnodeStmt), createDnodeStmtToJson, jsonToCreateDnodeStmt, destoryXNode ); - setFunc("DropDnodeStmt", + setFunc("DropDnodeStmt", QUERY_NODE_DROP_DNODE_STMT, sizeof(SDropDnodeStmt), dropDnodeStmtToJson, jsonToDropDnodeStmt, destoryXNode ); - setFunc("AlterDnodeStmt", + setFunc("AlterDnodeStmt", QUERY_NODE_ALTER_DNODE_STMT, sizeof(SAlterDnodeStmt), alterDnodeStmtToJson, jsonToAlterDnodeStmt, destoryXNode ); - setFunc("CreateIndexStmt", + setFunc("CreateIndexStmt", QUERY_NODE_CREATE_INDEX_STMT, sizeof(SCreateIndexStmt), createIndexStmtToJson, jsonToCreateIndexStmt, destoryCreateIndexStmt ); - setFunc("DropIndexStmt", + setFunc("DropIndexStmt", QUERY_NODE_DROP_INDEX_STMT, sizeof(SDropIndexStmt), dropIndexStmtToJson, jsonToDropIndexStmt, destoryXNode ); - setFunc("CreateQnodeStmt", + setFunc("CreateQnodeStmt", QUERY_NODE_CREATE_QNODE_STMT, sizeof(SCreateComponentNodeStmt), createQnodeStmtToJson, jsonToCreateQnodeStmt, destoryXNode ); - setFunc("DropQnodeStmt", + setFunc("DropQnodeStmt", QUERY_NODE_DROP_QNODE_STMT, sizeof(SDropComponentNodeStmt), dropQnodeStmtToJson, jsonToDropQnodeStmt, destoryXNode ); - setFunc("CreateBnodeStmt", + setFunc("CreateBnodeStmt", QUERY_NODE_CREATE_BNODE_STMT, sizeof(SCreateComponentNodeStmt), emptyNodeToJson, emptyJsonToNode, destoryXNode ); - setFunc("DropBnodeStmt", + setFunc("DropBnodeStmt", QUERY_NODE_DROP_BNODE_STMT, sizeof(SDropComponentNodeStmt), emptyNodeToJson, emptyJsonToNode, destoryXNode ); - setFunc("CreateSnodeStmt", + setFunc("CreateSnodeStmt", QUERY_NODE_CREATE_SNODE_STMT, sizeof(SCreateComponentNodeStmt), createSnodeStmtToJson, jsonToCreateSnodeStmt, destoryXNode ); - setFunc("DropSnodeStmt", + setFunc("DropSnodeStmt", QUERY_NODE_DROP_SNODE_STMT, sizeof(SDropComponentNodeStmt), dropSnodeStmtToJson, jsonToDropSnodeStmt, destoryXNode ); - setFunc("CreateMnodeStmt", + setFunc("CreateMnodeStmt", QUERY_NODE_CREATE_MNODE_STMT, sizeof(SCreateComponentNodeStmt), createMnodeStmtToJson, jsonToCreateMnodeStmt, destoryXNode ); - setFunc("DropMnodeStmt", + setFunc("DropMnodeStmt", QUERY_NODE_DROP_MNODE_STMT, sizeof(SDropComponentNodeStmt), dropMnodeStmtToJson, jsonToDropMnodeStmt, destoryXNode ); - setFunc("CreateTopicStmt", + setFunc("CreateTopicStmt", QUERY_NODE_CREATE_TOPIC_STMT, sizeof(SCreateTopicStmt), createTopicStmtToJson, jsonToCreateTopicStmt, destoryCreateTopicStmt ); - setFunc("DropTopicStmt", + setFunc("DropTopicStmt", QUERY_NODE_DROP_TOPIC_STMT, sizeof(SDropTopicStmt), dropTopicStmtToJson, jsonToDropTopicStmt, destoryXNode ); - setFunc("DropConsumerGroupStmt", + setFunc("DropConsumerGroupStmt", QUERY_NODE_DROP_CGROUP_STMT, sizeof(SDropCGroupStmt), dropConsumerGroupStmtToJson, jsonToDropConsumerGroupStmt, destoryXNode ); - setFunc("AlterLocalStmt", + setFunc("AlterLocalStmt", QUERY_NODE_ALTER_LOCAL_STMT, sizeof(SAlterLocalStmt), alterLocalStmtToJson, jsonToAlterLocalStmt, destoryXNode ); - setFunc("ExplainStmt", + setFunc("ExplainStmt", QUERY_NODE_EXPLAIN_STMT, sizeof(SExplainStmt), explainStmtToJson, jsonToExplainStmt, destoryExplainStmt ); - setFunc("DescribeStmt", + setFunc("DescribeStmt", QUERY_NODE_DESCRIBE_STMT, sizeof(SDescribeStmt), describeStmtToJson, jsonToDescribeStmt, destoryDescribeStmt ); - setFunc("QueryCacheStmt", + setFunc("QueryCacheStmt", QUERY_NODE_RESET_QUERY_CACHE_STMT, sizeof(SNode), emptyNodeToJson, emptyJsonToNode, destoryXNode ); - setFunc("CompactDatabaseStmt", + setFunc("CompactDatabaseStmt", QUERY_NODE_COMPACT_DATABASE_STMT, sizeof(SCompactDatabaseStmt), compactDatabaseStmtToJson, jsonToCompactDatabaseStmt, destoryCompactDatabaseStmt ); - setFunc("CreateFunctionStmt", + setFunc("CreateFunctionStmt", QUERY_NODE_CREATE_FUNCTION_STMT, sizeof(SCreateFunctionStmt), emptyNodeToJson, emptyJsonToNode, destoryXNode ); - setFunc("DropFunctionStmt", + setFunc("DropFunctionStmt", QUERY_NODE_DROP_FUNCTION_STMT, sizeof(SDropFunctionStmt), emptyNodeToJson, emptyJsonToNode, destoryXNode ); - setFunc("CreateStreamStmt", + setFunc("CreateStreamStmt", QUERY_NODE_CREATE_STREAM_STMT, sizeof(SCreateStreamStmt), createStreamStmtToJson, jsonToCreateStreamStmt, destoryCreateStreamStmt ); - setFunc("DropStreamStmt", + setFunc("DropStreamStmt", QUERY_NODE_DROP_STREAM_STMT, sizeof(SDropStreamStmt), dropStreamStmtToJson, jsonToDropStreamStmt, destoryXNode ); - setFunc("PauseStreamStmt", + setFunc("PauseStreamStmt", QUERY_NODE_PAUSE_STREAM_STMT, sizeof(SPauseStreamStmt), emptyNodeToJson, emptyJsonToNode, destoryXNode ); - setFunc("ResumeStreamStmt", + setFunc("ResumeStreamStmt", QUERY_NODE_RESUME_STREAM_STMT, sizeof(SResumeStreamStmt), emptyNodeToJson, emptyJsonToNode, destoryXNode ); - setFunc("BalanceVgroupStmt", + setFunc("BalanceVgroupStmt", QUERY_NODE_BALANCE_VGROUP_STMT, sizeof(SBalanceVgroupStmt), emptyNodeToJson, emptyJsonToNode, destoryXNode ); - setFunc("BalanceVgroupLeaderStmt", + setFunc("BalanceVgroupLeaderStmt", QUERY_NODE_BALANCE_VGROUP_LEADER_STMT, sizeof(SBalanceVgroupLeaderStmt), emptyNodeToJson, emptyJsonToNode, destoryXNode ); - setFunc("MergeVgroupStmt", + setFunc("MergeVgroupStmt", QUERY_NODE_MERGE_VGROUP_STMT, sizeof(SMergeVgroupStmt), mergeVgroupStmtToJson, jsonToMergeVgroupStmt, destoryXNode ); - setFunc("RedistributeVgroupStmt", + setFunc("RedistributeVgroupStmt", QUERY_NODE_REDISTRIBUTE_VGROUP_STMT, sizeof(SRedistributeVgroupStmt), redistributeVgroupStmtToJson, jsonToRedistributeVgroupStmt, destoryRedistributeVgroupStmt ); - setFunc("SplitVgroupStmt", + setFunc("SplitVgroupStmt", QUERY_NODE_SPLIT_VGROUP_STMT, sizeof(SSplitVgroupStmt), splitVgroupStmtToJson, jsonToSplitVgroupStmt, destoryXNode ); - setFunc("SyncDBStmt", + setFunc("SyncDBStmt", QUERY_NODE_SYNCDB_STMT, 0, emptyNodeToJson, emptyJsonToNode, destoryXNode ); - setFunc("GrantStmt", + setFunc("GrantStmt", QUERY_NODE_GRANT_STMT, sizeof(SGrantStmt), grantStmtToJson, jsonToGrantStmt, destoryGrantStmt ); - setFunc("RevokeStmt", + setFunc("RevokeStmt", QUERY_NODE_REVOKE_STMT, sizeof(SRevokeStmt), revokeStmtToJson, jsonToRevokeStmt, destoryRevokeStmt ); - setFunc("ShowDnodesStmt", + setFunc("ShowDnodesStmt", QUERY_NODE_SHOW_DNODES_STMT, sizeof(SShowStmt), showDnodesStmtToJson, jsonToShowDnodesStmt, destoryShowStmt ); - setFunc("ShowMnodesStmt", + setFunc("ShowMnodesStmt", QUERY_NODE_SHOW_MNODES_STMT, sizeof(SShowStmt), showMnodesStmtToJson, jsonToShowMnodesStmt, destoryShowStmt ); - setFunc("ShowModulesStmt", + setFunc("ShowModulesStmt", QUERY_NODE_SHOW_MODULES_STMT, sizeof(SShowStmt), emptyNodeToJson, emptyJsonToNode, destoryShowStmt ); - setFunc("ShowQnodesStmt", + setFunc("ShowQnodesStmt", QUERY_NODE_SHOW_QNODES_STMT, sizeof(SShowStmt), showQnodesStmtToJson, jsonToShowQnodesStmt, destoryShowStmt ); - setFunc("ShowSnodesStmt", + setFunc("ShowSnodesStmt", QUERY_NODE_SHOW_SNODES_STMT, sizeof(SShowStmt), emptyNodeToJson, emptyJsonToNode, destoryShowStmt ); - setFunc("ShowBnodesStmt", + setFunc("ShowBnodesStmt", QUERY_NODE_SHOW_BNODES_STMT, sizeof(SShowStmt), emptyNodeToJson, emptyJsonToNode, destoryShowStmt ); - setFunc("ShowClusterStmt", + setFunc("ShowClusterStmt", QUERY_NODE_SHOW_CLUSTER_STMT, sizeof(SShowStmt), showClusterStmtToJson, jsonToShowClusterStmt, destoryShowStmt ); - setFunc("ShowDatabaseStmt", + setFunc("ShowDatabaseStmt", QUERY_NODE_SHOW_DATABASES_STMT, sizeof(SShowStmt), showDatabasesStmtToJson, jsonToShowDatabasesStmt, destoryShowStmt ); - setFunc("ShowFunctionsStmt", + setFunc("ShowFunctionsStmt", QUERY_NODE_SHOW_FUNCTIONS_STMT, sizeof(SShowStmt), showFunctionsStmtToJson, jsonToShowFunctionsStmt, destoryShowStmt ); - setFunc("ShowIndexesStmt", + setFunc("ShowIndexesStmt", QUERY_NODE_SHOW_INDEXES_STMT, sizeof(SShowStmt), showIndexesStmtToJson, jsonToShowIndexesStmt, destoryShowStmt ); - setFunc("ShowStablesStmt", + setFunc("ShowStablesStmt", QUERY_NODE_SHOW_STABLES_STMT, sizeof(SShowStmt), showStablesStmtToJson, jsonToShowStablesStmt, destoryShowStmt ); - setFunc("ShowStreamsStmt", + setFunc("ShowStreamsStmt", QUERY_NODE_SHOW_STREAMS_STMT, sizeof(SShowStmt), showStreamsStmtToJson, jsonToShowStreamsStmt, destoryShowStmt ); - setFunc("ShowTablesStmt", + setFunc("ShowTablesStmt", QUERY_NODE_SHOW_TABLES_STMT, sizeof(SShowStmt), showTablesStmtToJson, jsonToShowTablesStmt, destoryShowStmt ); - setFunc("ShowTagsStmt", + setFunc("ShowTagsStmt", QUERY_NODE_SHOW_TAGS_STMT, sizeof(SShowStmt), showTagsStmtToJson, jsonToShowTagsStmt, destoryShowStmt ); - setFunc("ShowUsersStmt", + setFunc("ShowUsersStmt", QUERY_NODE_SHOW_USERS_STMT, sizeof(SShowStmt), showUsersStmtToJson, jsonToShowUsersStmt, destoryShowStmt ); - setFunc("ShowLicencesStmt", + setFunc("ShowLicencesStmt", QUERY_NODE_SHOW_LICENCES_STMT, sizeof(SShowStmt), emptyNodeToJson, emptyJsonToNode, destoryShowStmt ); - setFunc("ShowVgroupsStmt", + setFunc("ShowVgroupsStmt", QUERY_NODE_SHOW_VGROUPS_STMT, sizeof(SShowStmt), showVgroupsStmtToJson, jsonToShowVgroupsStmt, destoryShowStmt ); - setFunc("ShowTopicsStmt", + setFunc("ShowTopicsStmt", QUERY_NODE_SHOW_TOPICS_STMT, sizeof(SShowStmt), emptyNodeToJson, emptyJsonToNode, destoryShowStmt ); - setFunc("ShowConsumersStmt", + setFunc("ShowConsumersStmt", QUERY_NODE_SHOW_CONSUMERS_STMT, sizeof(SShowStmt), showConsumersStmtToJson, jsonToShowConsumersStmt, destoryShowStmt ); - setFunc("ShowQueriesStmt", + setFunc("ShowQueriesStmt", QUERY_NODE_SHOW_QUERIES_STMT, sizeof(SShowStmt), emptyNodeToJson, emptyJsonToNode, destoryShowStmt ); - setFunc("ShowConnectionsStmt", + setFunc("ShowConnectionsStmt", QUERY_NODE_SHOW_CONNECTIONS_STMT, sizeof(SShowStmt), emptyNodeToJson, emptyJsonToNode, destoryShowStmt ); - setFunc("ShowAppsStmt", + setFunc("ShowAppsStmt", QUERY_NODE_SHOW_APPS_STMT, sizeof(SShowStmt), emptyNodeToJson, emptyJsonToNode, destoryShowStmt ); - setFunc("ShowVariablesStmt", + setFunc("ShowVariablesStmt", QUERY_NODE_SHOW_VARIABLES_STMT, sizeof(SShowStmt), showVariablesStmtToJson, jsonToShowVariablesStmt, destoryShowStmt ); - setFunc("ShowDnodeVariablesStmt", + setFunc("ShowDnodeVariablesStmt", QUERY_NODE_SHOW_DNODE_VARIABLES_STMT, sizeof(SShowDnodeVariablesStmt), showDnodeVariablesStmtToJson, jsonToShowDnodeVariablesStmt, destoryShowDnodeVariablesStmt ); - setFunc("ShowTransactionsStmt", + setFunc("ShowTransactionsStmt", QUERY_NODE_SHOW_TRANSACTIONS_STMT, sizeof(SShowStmt), showTransactionsStmtToJson, jsonToShowTransactionsStmt, destoryShowStmt ); - setFunc("ShowSubscriptionsStmt", + setFunc("ShowSubscriptionsStmt", QUERY_NODE_SHOW_SUBSCRIPTIONS_STMT, sizeof(SShowStmt), showSubscriptionsStmtToJson, jsonToShowSubscriptionsStmt, destoryShowStmt ); - setFunc("ShowVnodeStmt", + setFunc("ShowVnodeStmt", QUERY_NODE_SHOW_VNODES_STMT, sizeof(SShowStmt), showVnodesStmtToJson, jsonToShowVnodesStmt, destoryShowStmt ); - setFunc("ShowUserPrivilegesStmt", + setFunc("ShowUserPrivilegesStmt", QUERY_NODE_SHOW_USER_PRIVILEGES_STMT, sizeof(SShowStmt), showUserPrivilegesStmtToJson, jsonToShowUserPrivilegesStmt, destoryShowStmt ); - setFunc("ShowViewsStmt", + setFunc("ShowViewsStmt", QUERY_NODE_SHOW_VIEWS_STMT, sizeof(SShowStmt), emptyNodeToJson, emptyJsonToNode, destoryShowStmt ); - setFunc("ShowCreateViewStmt", + setFunc("ShowCreateViewStmt", QUERY_NODE_SHOW_CREATE_VIEW_STMT, sizeof(SShowCreateViewStmt), showCreateViewStmtToJson, jsonToShowCreateViewStmt, destoryXNode ); - setFunc("ShowCreateDatabasesStmt", + setFunc("ShowCreateDatabasesStmt", QUERY_NODE_SHOW_CREATE_DATABASE_STMT, sizeof(SShowCreateDatabaseStmt), showCreateDatabaseStmtToJson, jsonToShowCreateDatabaseStmt, destoryShowCreateDatabaseStmt ); - setFunc("ShowCreateTablesStmt", + setFunc("ShowCreateTablesStmt", QUERY_NODE_SHOW_CREATE_TABLE_STMT, sizeof(SShowCreateTableStmt), showCreateTableStmtToJson, jsonToShowCreateTableStmt, destoryShowCreateTableStmt ); - setFunc("ShowCreateStablesStmt", + setFunc("ShowCreateStablesStmt", QUERY_NODE_SHOW_CREATE_STABLE_STMT, sizeof(SShowCreateTableStmt), showCreateStableStmtToJson, jsonToShowCreateStableStmt, destoryShowCreateTableStmt ); - setFunc("ShowTableDistributedStmt", + setFunc("ShowTableDistributedStmt", QUERY_NODE_SHOW_TABLE_DISTRIBUTED_STMT, sizeof(SShowTableDistributedStmt), showTableDistributedStmtToJson, jsonToShowTableDistributedStmt, destoryXNode ); - setFunc("ShowLocalVariablesStmt", + setFunc("ShowLocalVariablesStmt", QUERY_NODE_SHOW_LOCAL_VARIABLES_STMT, sizeof(SShowStmt), showLocalVariablesStmtToJson, jsonToShowLocalVariablesStmt, destoryShowStmt ); - setFunc("ShowScoresStmt", + setFunc("ShowScoresStmt", QUERY_NODE_SHOW_SCORES_STMT, sizeof(SShowStmt), emptyNodeToJson, emptyJsonToNode, destoryShowStmt ); - setFunc("ShowTableTagsStmt", + setFunc("ShowTableTagsStmt", QUERY_NODE_SHOW_TABLE_TAGS_STMT, sizeof(SShowTableTagsStmt), showTableTagsStmtToJson, jsonToShowTableTagsStmt, destoryShowTableTagsStmt ); - setFunc("KillConnectionStmt", + setFunc("KillConnectionStmt", QUERY_NODE_KILL_CONNECTION_STMT, sizeof(SKillStmt), emptyNodeToJson, emptyJsonToNode, destoryXNode ); - setFunc("KillQueryStmt", + setFunc("KillQueryStmt", QUERY_NODE_KILL_QUERY_STMT, sizeof(SKillQueryStmt), emptyNodeToJson, emptyJsonToNode, destoryXNode ); - setFunc("KillTransactionStmt", + setFunc("KillTransactionStmt", QUERY_NODE_KILL_TRANSACTION_STMT, sizeof(SKillStmt), emptyNodeToJson, emptyJsonToNode, destoryXNode ); - setFunc("DeleteStmt", + setFunc("DeleteStmt", QUERY_NODE_DELETE_STMT, sizeof(SDeleteStmt), deleteStmtToJson, jsonToDeleteStmt, destoryDeleteStmt ); - setFunc("InsertStmt", + setFunc("InsertStmt", QUERY_NODE_INSERT_STMT, sizeof(SInsertStmt), insertStmtToJson, jsonToInsertStmt, destoryInsertStmt ); - setFunc("QueryNode", + setFunc("QueryNode", QUERY_NODE_QUERY, sizeof(SQuery), emptyNodeToJson, emptyJsonToNode, destoryQueryNode ); - setFunc("ShowDbAliveStmt", + setFunc("ShowDbAliveStmt", QUERY_NODE_SHOW_DB_ALIVE_STMT, sizeof(SShowAliveStmt), emptyNodeToJson, emptyJsonToNode, destoryXNode ); - setFunc("ShowClusterAliveStmt", + setFunc("ShowClusterAliveStmt", QUERY_NODE_SHOW_CLUSTER_ALIVE_STMT, sizeof(SShowAliveStmt), emptyNodeToJson, emptyJsonToNode, destoryXNode ); - setFunc("RestoreDnodeStmt", + setFunc("RestoreDnodeStmt", QUERY_NODE_RESTORE_DNODE_STMT, sizeof(SRestoreComponentNodeStmt), emptyNodeToJson, jsonToRestoreDnodeStmt, destoryXNode ); - setFunc("RestoreQnodeStmt", + setFunc("RestoreQnodeStmt", QUERY_NODE_RESTORE_QNODE_STMT, sizeof(SRestoreComponentNodeStmt), emptyNodeToJson, jsonToRestoreQnodeStmt, destoryXNode ); - setFunc("RestoreMnodeStmt", + setFunc("RestoreMnodeStmt", QUERY_NODE_RESTORE_MNODE_STMT, sizeof(SRestoreComponentNodeStmt), emptyNodeToJson, jsonToRestoreMnodeStmt, destoryXNode ); - setFunc("RestoreVnodeStmt", + setFunc("RestoreVnodeStmt", QUERY_NODE_RESTORE_VNODE_STMT, sizeof(SRestoreComponentNodeStmt), emptyNodeToJson, jsonToRestoreVnodeStmt, destoryXNode ); - setFunc("CreateViewStmt", + setFunc("CreateViewStmt", QUERY_NODE_CREATE_VIEW_STMT, sizeof(SCreateViewStmt), emptyNodeToJson, emptyJsonToNode, destoryCreateViewStmt ); - setFunc("DropViewStmt", + setFunc("DropViewStmt", QUERY_NODE_DROP_VIEW_STMT, sizeof(SDropViewStmt), emptyNodeToJson, emptyJsonToNode, destoryXNode ); - setFunc("LogicScan", + setFunc("LogicScan", QUERY_NODE_LOGIC_PLAN_SCAN, sizeof(SScanLogicNode), logicScanNodeToJson, jsonToLogicScanNode, destoryScanLogicNode ); - setFunc("LogicJoin", + setFunc("LogicJoin", QUERY_NODE_LOGIC_PLAN_JOIN, sizeof(SJoinLogicNode), logicJoinNodeToJson, jsonToLogicJoinNode, destoryJoinLogicNode ); - setFunc("LogicAgg", + setFunc("LogicAgg", QUERY_NODE_LOGIC_PLAN_AGG, sizeof(SAggLogicNode), logicAggNodeToJson, jsonToLogicAggNode, destoryAggLogicNode ); - setFunc("LogicProject", + setFunc("LogicProject", QUERY_NODE_LOGIC_PLAN_PROJECT, sizeof(SProjectLogicNode), logicProjectNodeToJson, jsonToLogicProjectNode, destoryProjectLogicNode ); - setFunc("LogicVnodeModify", + setFunc("LogicVnodeModify", QUERY_NODE_LOGIC_PLAN_VNODE_MODIFY, sizeof(SVnodeModifyLogicNode), logicVnodeModifyNodeToJson, jsonToLogicVnodeModifyNode, destoryVnodeModifyLogicNode ); - setFunc("LogicExchange", + setFunc("LogicExchange", QUERY_NODE_LOGIC_PLAN_EXCHANGE, sizeof(SExchangeLogicNode), logicExchangeNodeToJson, jsonToLogicExchangeNode, destoryExchangeLogicNode ); - setFunc("LogicMerge", + setFunc("LogicMerge", QUERY_NODE_LOGIC_PLAN_MERGE, sizeof(SMergeLogicNode), logicMergeNodeToJson, jsonToLogicMergeNode, destoryMergeLogicNode ); - setFunc("LogicWindow", + setFunc("LogicWindow", QUERY_NODE_LOGIC_PLAN_WINDOW, sizeof(SWindowLogicNode), logicWindowNodeToJson, jsonToLogicWindowNode, destoryWindowLogicNode ); - setFunc("LogicFill", + setFunc("LogicFill", QUERY_NODE_LOGIC_PLAN_FILL, sizeof(SFillLogicNode), logicFillNodeToJson, jsonToLogicFillNode, destoryFillLogicNode ); - setFunc("LogicSort", + setFunc("LogicSort", QUERY_NODE_LOGIC_PLAN_SORT, sizeof(SSortLogicNode), logicSortNodeToJson, jsonToLogicSortNode, destorySortLogicNode ); - setFunc("LogicPartition", + setFunc("LogicPartition", QUERY_NODE_LOGIC_PLAN_PARTITION, sizeof(SPartitionLogicNode), logicPartitionNodeToJson, jsonToLogicPartitionNode, destoryPartitionLogicNode ); - setFunc("LogicIndefRowsFunc", + setFunc("LogicIndefRowsFunc", QUERY_NODE_LOGIC_PLAN_INDEF_ROWS_FUNC, sizeof(SIndefRowsFuncLogicNode), logicIndefRowsFuncNodeToJson, jsonToLogicIndefRowsFuncNode, destoryIndefRowsFuncLogicNode ); - setFunc("LogicInterpFunc", + setFunc("LogicInterpFunc", QUERY_NODE_LOGIC_PLAN_INTERP_FUNC, sizeof(SInterpFuncLogicNode), logicInterpFuncNodeToJson, jsonToLogicInterpFuncNode, destoryInterpFuncLogicNode ); - setFunc("LogicGroupCache", + setFunc("LogicGroupCache", QUERY_NODE_LOGIC_PLAN_GROUP_CACHE, sizeof(SGroupCacheLogicNode), logicGroupCacheNodeToJson, jsonToLogicGroupCacheNode, destoryGroupCacheLogicNode ); - setFunc("LogicDynamicQueryCtrl", + setFunc("LogicDynamicQueryCtrl", QUERY_NODE_LOGIC_PLAN_DYN_QUERY_CTRL, sizeof(SDynQueryCtrlLogicNode), logicDynQueryCtrlNodeToJson, jsonToLogicDynQueryCtrlNode, destoryDynQueryCtrlLogicNode ); - setFunc("LogicSubplan", + setFunc("LogicSubplan", QUERY_NODE_LOGIC_SUBPLAN, sizeof(SLogicSubplan), logicSubplanToJson, jsonToLogicSubplan, destoryLogicSubplan ); - setFunc("LogicPlan", + setFunc("LogicPlan", QUERY_NODE_LOGIC_PLAN, sizeof(SQueryLogicPlan), logicPlanToJson, jsonToLogicPlan, destoryQueryLogicPlan ); - setFunc("PhysiTagScan", + setFunc("PhysiTagScan", QUERY_NODE_PHYSICAL_PLAN_TAG_SCAN, sizeof(STagScanPhysiNode), physiTagScanNodeToJson, jsonToPhysiTagScanNode, destroyScanPhysiNode ); - setFunc("PhysiTableScan", + setFunc("PhysiTableScan", QUERY_NODE_PHYSICAL_PLAN_TABLE_SCAN, sizeof(STableScanPhysiNode), physiTableScanNodeToJson, jsonToPhysiTableScanNode, destoryTableScanPhysiNode ); - setFunc("PhysiTableSeqScan", + setFunc("PhysiTableSeqScan", QUERY_NODE_PHYSICAL_PLAN_TABLE_SEQ_SCAN, sizeof(STableSeqScanPhysiNode), emptyNodeToJson, emptyJsonToNode, destoryTableScanPhysiNode ); - setFunc("PhysiTableMergeScan", + setFunc("PhysiTableMergeScan", QUERY_NODE_PHYSICAL_PLAN_TABLE_MERGE_SCAN, sizeof(STableMergeScanPhysiNode), physiTableScanNodeToJson, jsonToPhysiTableScanNode, destoryTableScanPhysiNode ); - setFunc("PhysiSreamScan", + setFunc("PhysiSreamScan", QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN, sizeof(SStreamScanPhysiNode), physiTableScanNodeToJson, jsonToPhysiTableScanNode, destoryTableScanPhysiNode ); - setFunc("PhysiSystemTableScan", + setFunc("PhysiSystemTableScan", QUERY_NODE_PHYSICAL_PLAN_SYSTABLE_SCAN, sizeof(SSystemTableScanPhysiNode), physiSysTableScanNodeToJson, jsonToPhysiSysTableScanNode, destroyScanPhysiNode ); - setFunc("PhysiBlockDistScan", + setFunc("PhysiBlockDistScan", QUERY_NODE_PHYSICAL_PLAN_BLOCK_DIST_SCAN, sizeof(SBlockDistScanPhysiNode), physiScanNodeToJson, jsonToPhysiScanNode, destroyScanPhysiNode ); - setFunc("PhysiLastRowScan", + setFunc("PhysiLastRowScan", QUERY_NODE_PHYSICAL_PLAN_LAST_ROW_SCAN, sizeof(SLastRowScanPhysiNode), physiLastRowScanNodeToJson, jsonToPhysiLastRowScanNode, destoryLastRowScanPhysiNode ); - setFunc("PhysiProject", + setFunc("PhysiProject", QUERY_NODE_PHYSICAL_PLAN_PROJECT, sizeof(SProjectPhysiNode), physiProjectNodeToJson, jsonToPhysiProjectNode, destoryProjectPhysiNode ); - setFunc("PhysiMergeJoin", + setFunc("PhysiMergeJoin", QUERY_NODE_PHYSICAL_PLAN_MERGE_JOIN, sizeof(SSortMergeJoinPhysiNode), physiMergeJoinNodeToJson, jsonToPhysiMergeJoinNode, destorySortMergeJoinPhysiNode ); - setFunc("PhysiAgg", + setFunc("PhysiAgg", QUERY_NODE_PHYSICAL_PLAN_HASH_AGG, sizeof(SAggPhysiNode), physiAggNodeToJson, jsonToPhysiAggNode, destoryAggPhysiNode ); - setFunc("PhysiExchange", + setFunc("PhysiExchange", QUERY_NODE_PHYSICAL_PLAN_EXCHANGE, sizeof(SExchangePhysiNode), physiExchangeNodeToJson, jsonToPhysiExchangeNode, destoryExchangePhysiNode ); - setFunc("PhysiMerge", + setFunc("PhysiMerge", QUERY_NODE_PHYSICAL_PLAN_MERGE, sizeof(SMergePhysiNode), physiMergeNodeToJson, jsonToPhysiMergeNode, destoryMergePhysiNode ); - setFunc("PhysiSort", + setFunc("PhysiSort", QUERY_NODE_PHYSICAL_PLAN_SORT, sizeof(SSortPhysiNode), physiSortNodeToJson, jsonToPhysiSortNode, destorySortPhysiNode ); - setFunc("PhysiGroupSort", + setFunc("PhysiGroupSort", QUERY_NODE_PHYSICAL_PLAN_GROUP_SORT, sizeof(SGroupSortPhysiNode), physiSortNodeToJson, jsonToPhysiSortNode, destorySortPhysiNode ); - setFunc("PhysiHashInterval", + setFunc("PhysiHashInterval", QUERY_NODE_PHYSICAL_PLAN_HASH_INTERVAL, sizeof(SIntervalPhysiNode), physiIntervalNodeToJson, jsonToPhysiIntervalNode, destroyWindowPhysiNode ); - setFunc("PhysiMergeAlignedInterval", + setFunc("PhysiMergeAlignedInterval", QUERY_NODE_PHYSICAL_PLAN_MERGE_ALIGNED_INTERVAL, sizeof(SMergeAlignedIntervalPhysiNode), physiIntervalNodeToJson, jsonToPhysiIntervalNode, destroyWindowPhysiNode ); - setFunc("PhysiStreamInterval", + setFunc("PhysiStreamInterval", QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL, sizeof(SStreamIntervalPhysiNode), physiIntervalNodeToJson, jsonToPhysiIntervalNode, destroyWindowPhysiNode ); - setFunc("PhysiStreamFinalInterval", + setFunc("PhysiStreamFinalInterval", QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL, sizeof(SStreamFinalIntervalPhysiNode), physiIntervalNodeToJson, jsonToPhysiIntervalNode, destroyWindowPhysiNode ); - setFunc("PhysiStreamSemiInterval", + setFunc("PhysiStreamSemiInterval", QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL, sizeof(SStreamSemiIntervalPhysiNode), physiIntervalNodeToJson, jsonToPhysiIntervalNode, destroyWindowPhysiNode ); - setFunc("PhysiFill", + setFunc("PhysiFill", QUERY_NODE_PHYSICAL_PLAN_FILL, sizeof(SFillPhysiNode), physiFillNodeToJson, jsonToPhysiFillNode, destoryFillPhysiNode ); - setFunc("PhysiStreamFill", + setFunc("PhysiStreamFill", QUERY_NODE_PHYSICAL_PLAN_STREAM_FILL, sizeof(SFillPhysiNode), physiFillNodeToJson, jsonToPhysiFillNode, destoryFillPhysiNode ); - setFunc("PhysiSessionWindow", + setFunc("PhysiSessionWindow", QUERY_NODE_PHYSICAL_PLAN_MERGE_SESSION, sizeof(SSessionWinodwPhysiNode), physiSessionWindowNodeToJson, jsonToPhysiSessionWindowNode, destroyWindowPhysiNode ); - setFunc("PhysiStreamSessionWindow", + setFunc("PhysiStreamSessionWindow", QUERY_NODE_PHYSICAL_PLAN_STREAM_SESSION, sizeof(SStreamSessionWinodwPhysiNode), physiSessionWindowNodeToJson, jsonToPhysiSessionWindowNode, destroyWindowPhysiNode ); - setFunc("PhysiStreamSemiSessionWindow", + setFunc("PhysiStreamSemiSessionWindow", QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_SESSION, sizeof(SStreamSemiSessionWinodwPhysiNode), physiSessionWindowNodeToJson, jsonToPhysiSessionWindowNode, destroyWindowPhysiNode ); - setFunc("PhysiStreamFinalSessionWindow", + setFunc("PhysiStreamFinalSessionWindow", QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_SESSION, sizeof(SStreamFinalSessionWinodwPhysiNode), physiSessionWindowNodeToJson, jsonToPhysiSessionWindowNode, destroyWindowPhysiNode ); - setFunc("PhysiStateWindow", + setFunc("PhysiStateWindow", QUERY_NODE_PHYSICAL_PLAN_MERGE_STATE, sizeof(SStateWinodwPhysiNode), physiStateWindowNodeToJson, jsonToPhysiStateWindowNode, destoryStateWindowPhysiNode ); - setFunc("PhysiStreamStateWindow", + setFunc("PhysiStreamStateWindow", QUERY_NODE_PHYSICAL_PLAN_STREAM_STATE, sizeof(SStreamStateWinodwPhysiNode), physiStateWindowNodeToJson, jsonToPhysiStateWindowNode, destoryStateWindowPhysiNode ); - setFunc("PhysiPartition", + setFunc("PhysiPartition", QUERY_NODE_PHYSICAL_PLAN_PARTITION, sizeof(SPartitionPhysiNode), physiPartitionNodeToJson, jsonToPhysiPartitionNode, destroyPartitionPhysiNode ); - setFunc("PhysiStreamPartition", + setFunc("PhysiStreamPartition", QUERY_NODE_PHYSICAL_PLAN_STREAM_PARTITION, sizeof(SStreamPartitionPhysiNode), physiStreamPartitionNodeToJson, jsonToPhysiStreamPartitionNode, destoryStreamPartitionPhysiNode ); - setFunc("PhysiIndefRowsFunc", + setFunc("PhysiIndefRowsFunc", QUERY_NODE_PHYSICAL_PLAN_INDEF_ROWS_FUNC, sizeof(SIndefRowsFuncPhysiNode), physiIndefRowsFuncNodeToJson, jsonToPhysiIndefRowsFuncNode, destoryIndefRowsFuncPhysiNode ); - setFunc("PhysiInterpFunc", + setFunc("PhysiInterpFunc", QUERY_NODE_PHYSICAL_PLAN_INTERP_FUNC, sizeof(SInterpFuncLogicNode), physiInterpFuncNodeToJson, jsonToPhysiInterpFuncNode, destoryInterpFuncPhysiNode ); - setFunc("PhysiDispatch", + setFunc("PhysiDispatch", QUERY_NODE_PHYSICAL_PLAN_DISPATCH, sizeof(SDataDispatcherNode), physiDispatchNodeToJson, jsonToPhysiDispatchNode, destroyDataSinkNode ); - setFunc("PhysiInsert", + setFunc("PhysiInsert", QUERY_NODE_PHYSICAL_PLAN_INSERT, sizeof(SDataInserterNode), emptyNodeToJson, emptyJsonToNode, destoryDataInserterNode ); - setFunc("PhysiQueryInsert", + setFunc("PhysiQueryInsert", QUERY_NODE_PHYSICAL_PLAN_QUERY_INSERT, sizeof(SQueryInserterNode), physiQueryInsertNodeToJson, jsonToPhysiQueryInsertNode, destoryQueryInserterNode ); - setFunc("PhysiDelete", + setFunc("PhysiDelete", QUERY_NODE_PHYSICAL_PLAN_DELETE, sizeof(SDataDeleterNode), physiDeleteNodeToJson, jsonToPhysiDeleteNode, destoryDataDeleterNode ); - setFunc("PhysiGroupCache", + setFunc("PhysiGroupCache", QUERY_NODE_PHYSICAL_PLAN_GROUP_CACHE, sizeof(SGroupCachePhysiNode), physiGroupCacheNodeToJson, jsonToPhysiGroupCacheNode, destoryGroupCachePhysiNode ); - setFunc("PhysiDynamicQueryCtrl", + setFunc("PhysiDynamicQueryCtrl", QUERY_NODE_PHYSICAL_PLAN_DYN_QUERY_CTRL, sizeof(SDynQueryCtrlPhysiNode), physiDynQueryCtrlNodeToJson, jsonToPhysiDynQueryCtrlNode, destoryDynQueryCtrlPhysiNode ); - setFunc("PhysiSubplan", + setFunc("PhysiSubplan", QUERY_NODE_PHYSICAL_SUBPLAN, sizeof(SSubplan), subplanToJson, jsonToSubplan, destorySubplanNode ); - setFunc("PhysiPlan", + setFunc("PhysiPlan", QUERY_NODE_PHYSICAL_PLAN, sizeof(SQueryPlan), planToJson, jsonToPlan, destoryPlanNode ); - setFunc("PhysiTableCountScan", + setFunc("PhysiTableCountScan", QUERY_NODE_PHYSICAL_PLAN_TABLE_COUNT_SCAN, sizeof(STableCountScanPhysiNode), physiLastRowScanNodeToJson, jsonToPhysiScanNode, destoryLastRowScanPhysiNode ); - setFunc("PhysiMergeEventWindow", + setFunc("PhysiMergeEventWindow", QUERY_NODE_PHYSICAL_PLAN_MERGE_EVENT, sizeof(SEventWinodwPhysiNode), physiEventWindowNodeToJson, jsonToPhysiEventWindowNode, destoryEventWindowPhysiNode ); - setFunc("PhysiStreamEventWindow", + setFunc("PhysiStreamEventWindow", QUERY_NODE_PHYSICAL_PLAN_STREAM_EVENT, sizeof(SStreamEventWinodwPhysiNode), physiEventWindowNodeToJson, jsonToPhysiEventWindowNode, destoryEventWindowPhysiNode ); - setFunc("PhysiHashJoin", + setFunc("PhysiHashJoin", QUERY_NODE_PHYSICAL_PLAN_HASH_JOIN, sizeof(SHashJoinPhysiNode), physiHashJoinNodeToJson, From 084d0ab367b0e6c3730c9cff751e602fe3570fe1 Mon Sep 17 00:00:00 2001 From: Bob Liu Date: Sun, 19 Nov 2023 23:56:34 +0800 Subject: [PATCH 135/195] fix destoryShowCreateTableStmt --- source/libs/nodes/src/nodesCodeFuncs.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/nodes/src/nodesCodeFuncs.c b/source/libs/nodes/src/nodesCodeFuncs.c index 981e2bbaca..6771677aac 100644 --- a/source/libs/nodes/src/nodesCodeFuncs.c +++ b/source/libs/nodes/src/nodesCodeFuncs.c @@ -7045,7 +7045,7 @@ void destoryShowCreateDatabaseStmt(SNode* pNode) { void destoryShowCreateTableStmt(SNode* pNode) { STableCfg* pCfg = (STableCfg*)(((SShowCreateTableStmt*)pNode)->pTableCfg); - taosMemoryFreeClear(pCfg); + taosMemoryFreeClear(((SShowCreateTableStmt*)pNode)->pDbCfg); if (NULL == pCfg) { return; } From 734bf0731713c11acddfffe16b3876f9e06173bf Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Mon, 20 Nov 2023 10:48:49 +0800 Subject: [PATCH 136/195] Merge branch '3.0' into enh/refactorBackend --- source/dnode/mnode/impl/src/mndStream.c | 1 - source/libs/stream/src/streamBackendRocksdb.c | 8 +++++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 99fda7e81c..6a096b900f 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1250,7 +1250,6 @@ static int32_t mndCheckNodeStatus(SMnode *pMnode) { if (taosArrayGetSize(execInfo.pNodeList) == 0) { mDebug("stream task node change checking done, no vgroups exist, do nothing"); execInfo.ts = ts; - // goto _EXIT; } for (int32_t i = 0; i < taosArrayGetSize(execInfo.pTaskList); ++i) { STaskId * p = taosArrayGet(execInfo.pTaskList, i); diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index b9041198ac..a1f632d8c2 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1740,9 +1740,10 @@ STaskDbWrapper* taskDbOpenImpl(char* key, char* statePath, char* dbPath) { rocksdb_list_column_families_destroy(cfNames, nCf); } taosMemoryFree(err); + err = NULL; cfNames = rocksdb_list_column_families(pTaskDb->dbOpt, dbPath, &nCf, &err); - ASSERT(err != NULL); + ASSERT(err == NULL); } if (taskDbOpenCfs(pTaskDb, dbPath, cfNames, nCf) != 0) { @@ -1751,10 +1752,11 @@ STaskDbWrapper* taskDbOpenImpl(char* key, char* statePath, char* dbPath) { if (cfNames != NULL) { rocksdb_list_column_families_destroy(cfNames, nCf); + cfNames = NULL; } qDebug("succ to init stream backend at %s, backend:%p", dbPath, pTaskDb); - + return pTaskDb; _EXIT: taskDbDestroy(pTaskDb); @@ -1769,10 +1771,10 @@ STaskDbWrapper* taskDbOpen(char* path, char* key, int64_t chkpId) { if (rebuildDirFromChkp2(path, key, chkpId, &statePath, &dbPath) != 0) { return NULL; } - // taosMemoryFree(statePath); STaskDbWrapper* pTaskDb = taskDbOpenImpl(key, statePath, dbPath); taosMemoryFree(dbPath); + taosMemoryFree(statePath); return pTaskDb; } From 6e1e4702a5f80db2895bbd9bbf16ce76da6395ea Mon Sep 17 00:00:00 2001 From: Bob Liu Date: Mon, 20 Nov 2023 11:13:09 +0800 Subject: [PATCH 137/195] adjust --- include/libs/nodes/nodes.h | 1 - source/dnode/mgmt/node_mgmt/src/dmEnv.c | 1 - source/libs/nodes/src/nodesCodeFuncs.c | 27 +++++++++++-------------- source/libs/nodes/src/nodesUtilFuncs.c | 1 - 4 files changed, 12 insertions(+), 18 deletions(-) diff --git a/include/libs/nodes/nodes.h b/include/libs/nodes/nodes.h index 136926a856..bcbfd245f8 100644 --- a/include/libs/nodes/nodes.h +++ b/include/libs/nodes/nodes.h @@ -101,7 +101,6 @@ typedef struct SNodeList { typedef struct SNodeAllocator SNodeAllocator; -void nodesInit(); int32_t nodesInitAllocatorSet(); void nodesDestroyAllocatorSet(); int32_t nodesCreateAllocator(int64_t queryId, int32_t chunkSize, int64_t* pAllocatorId); diff --git a/source/dnode/mgmt/node_mgmt/src/dmEnv.c b/source/dnode/mgmt/node_mgmt/src/dmEnv.c index c57043b8bb..acbb3554ce 100644 --- a/source/dnode/mgmt/node_mgmt/src/dmEnv.c +++ b/source/dnode/mgmt/node_mgmt/src/dmEnv.c @@ -156,7 +156,6 @@ extern void s3End(); int32_t dmInit() { dInfo("start to init dnode env"); - nodesInit(); if (dmDiskInit() != 0) return -1; if (!dmCheckDataDirVersion()) return -1; if (!dmCheckDiskSpace()) return -1; diff --git a/source/libs/nodes/src/nodesCodeFuncs.c b/source/libs/nodes/src/nodesCodeFuncs.c index 6771677aac..cf5bc528e3 100644 --- a/source/libs/nodes/src/nodesCodeFuncs.c +++ b/source/libs/nodes/src/nodesCodeFuncs.c @@ -46,7 +46,7 @@ typedef struct SBuiltinNodeDefinition { SBuiltinNodeDefinition funcNodes[QUERY_NODE_END] = {NULL}; static TdThreadOnce functionNodeInit = PTHREAD_ONCE_INIT; -static int32_t initNodeCode = -1; +volatile int32_t initNodeCode = -1; static void setFunc(const char* name, int32_t type, int32_t nodeSize, FExecNodeToJson toJsonFunc, FExecJsonToNode toNodeFunc, FExecDestoryNode destoryFunc) { @@ -59,19 +59,16 @@ static void setFunc(const char* name, int32_t type, int32_t nodeSize, FExecNodeT static void doInitNodeFuncArray(); -void nodesInit() { - taosThreadOnce(&functionNodeInit, doInitNodeFuncArray); -} - bool funcArrayCheck(int32_t type) { - if (type < 0 || QUERY_NODE_END < (type+1)) { - nodesError("funcArrayCheck unknown type = %d", type); + if (type < 0 || QUERY_NODE_END <= type) { + nodesError("funcArrayCheck out of range type = %d", type); return false; } if (initNodeCode != 0) { - nodesInit(); + taosThreadOnce(&functionNodeInit, doInitNodeFuncArray); } if (!funcNodes[type].name) { + nodesError("funcArrayCheck unsupported type = %d", type); return false; } return true; @@ -6494,7 +6491,7 @@ static int32_t jsonToInsertStmt(const SJson* pJson, void* pObj) { int32_t specificNodeToJson(const void* pObj, SJson* pJson) { ENodeType type = nodeType(pObj); if (!funcArrayCheck(type)) { - return TSDB_CODE_SUCCESS; + return TSDB_CODE_NOT_FOUND; } if (funcNodes[type].toJsonFunc) { @@ -6508,7 +6505,7 @@ int32_t specificNodeToJson(const void* pObj, SJson* pJson) { int32_t jsonToSpecificNode(const SJson* pJson, void* pObj) { ENodeType type = nodeType(pObj); if (!funcArrayCheck(type)) { - return TSDB_CODE_SUCCESS; + return TSDB_CODE_NOT_FOUND; } if (funcNodes[type].toNodeFunc) { @@ -7423,16 +7420,16 @@ void nodesDestroyNode(SNode* pNode) { return; } - int32_t index = nodeType(pNode); - if (!funcArrayCheck(index)) { + int32_t type = nodeType(pNode); + if (!funcArrayCheck(type)) { return; } - if (funcNodes[index].destoryFunc) { - funcNodes[index].destoryFunc(pNode); + if (funcNodes[type].destoryFunc) { + funcNodes[type].destoryFunc(pNode); nodesFree(pNode); return; } - nodesError("nodesDestroyNode unknown node type = %d", nodeType(pNode)); + nodesWarn("nodesDestroyNode unknown type = %d", type); nodesFree(pNode); return; } diff --git a/source/libs/nodes/src/nodesUtilFuncs.c b/source/libs/nodes/src/nodesUtilFuncs.c index 3a738dbbd2..2ea387792b 100644 --- a/source/libs/nodes/src/nodesUtilFuncs.c +++ b/source/libs/nodes/src/nodesUtilFuncs.c @@ -131,7 +131,6 @@ static void destroyNodeAllocator(void* p) { } int32_t nodesInitAllocatorSet() { - nodesInit(); if (g_allocatorReqRefPool >= 0) { nodesWarn("nodes already initialized"); return TSDB_CODE_SUCCESS; From ffc952e31eb36caf554700e46a36df3d24453dc7 Mon Sep 17 00:00:00 2001 From: Bob Liu Date: Mon, 20 Nov 2023 14:40:46 +0800 Subject: [PATCH 138/195] adjust funcNodes init --- source/libs/nodes/src/nodesCodeFuncs.c | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/source/libs/nodes/src/nodesCodeFuncs.c b/source/libs/nodes/src/nodesCodeFuncs.c index cf5bc528e3..d4cd37750b 100644 --- a/source/libs/nodes/src/nodesCodeFuncs.c +++ b/source/libs/nodes/src/nodesCodeFuncs.c @@ -48,6 +48,7 @@ SBuiltinNodeDefinition funcNodes[QUERY_NODE_END] = {NULL}; static TdThreadOnce functionNodeInit = PTHREAD_ONCE_INIT; volatile int32_t initNodeCode = -1; + static void setFunc(const char* name, int32_t type, int32_t nodeSize, FExecNodeToJson toJsonFunc, FExecJsonToNode toNodeFunc, FExecDestoryNode destoryFunc) { funcNodes[type].name = name; @@ -64,9 +65,9 @@ bool funcArrayCheck(int32_t type) { nodesError("funcArrayCheck out of range type = %d", type); return false; } - if (initNodeCode != 0) { - taosThreadOnce(&functionNodeInit, doInitNodeFuncArray); - } + // only init once, do nothing when initNodeCode == 0 + taosThreadOnce(&functionNodeInit, doInitNodeFuncArray); + if (!funcNodes[type].name) { nodesError("funcArrayCheck unsupported type = %d", type); return false; @@ -7436,6 +7437,10 @@ void nodesDestroyNode(SNode* pNode) { // clang-format off static void doInitNodeFuncArray() { + if (initNodeCode == 0) { + return; + } + setFunc("Column", QUERY_NODE_COLUMN, sizeof(SColumnNode), From 0d724244dc6284d2d35f93ee49bc2001a7eb3451 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Mon, 20 Nov 2023 16:55:14 +0800 Subject: [PATCH 139/195] Merge branch '3.0' into enh/refactorBackend --- source/libs/stream/src/streamBackendRocksdb.c | 48 ++++++++++++++----- source/libs/stream/src/streamCheckpoint.c | 24 +++++----- 2 files changed, 47 insertions(+), 25 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index a1f632d8c2..b643e7186c 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -321,7 +321,25 @@ int32_t remoteChkp_validAndCvtMeta(char* path, SArray* list, int64_t chkpId) { return complete == 1 ? 0 : -1; } -int32_t rebuildFromRemoteChkp(char* key, char* chkpPath, int64_t chkpId, char* defaultPath) { +int32_t rebuildFromRemoteChkp_rsync(char* key, char* chkpPath, int64_t chkpId, char* defaultPath) { + // impl later + int32_t code = 0; + if (taosIsDir(chkpPath)) { + taosRemoveDir(chkpPath); + } + if (taosIsDir(defaultPath)) { + taosRemoveDir(defaultPath); + } + + code = downloadCheckpoint(key, chkpPath); + if (code != 0) { + return code; + } + code = copyFiles(chkpPath, defaultPath); + + return code; +} +int32_t rebuildFromRemoteChkp_s3(char* key, char* chkpPath, int64_t chkpId, char* defaultPath) { int32_t code = downloadCheckpoint(key, chkpPath); if (code != 0) { return code; @@ -355,6 +373,15 @@ int32_t rebuildFromRemoteChkp(char* key, char* chkpPath, int64_t chkpId, char* d taosMemoryFree(tmp); return code; } +int32_t rebuildFromRemoteChkp(char* key, char* chkpPath, int64_t chkpId, char* defaultPath) { + UPLOAD_TYPE type = getUploadType(); + if (type == UPLOAD_S3) { + return rebuildFromRemoteChkp_s3(key, chkpPath, chkpId, defaultPath); + } else if (type == UPLOAD_RSYNC) { + return rebuildFromRemoteChkp_rsync(key, chkpPath, chkpId, defaultPath); + } + return -1; +} int32_t rebuildFromLocalChkp(char* key, char* chkpPath, int64_t chkpId, char* defaultPath) { int32_t code = -1; @@ -944,7 +971,7 @@ int32_t chkpPreBuildDir(char* path, int64_t chkpId, char** chkpDir, char** chkpI sprintf(pChkpIdDir, "%s%s%s%" PRId64, pChkpDir, TD_DIRSEP, "checkpoint", chkpId); if (taosIsDir(pChkpIdDir)) { stInfo("stream rm exist checkpoint%s", pChkpIdDir); - taosRemoveFile(pChkpIdDir); + taosRemoveDir(pChkpIdDir); } *chkpDir = pChkpDir; *chkpIdDir = pChkpIdDir; @@ -1848,22 +1875,17 @@ int32_t taskDbGenChkpUploadData__rsync(STaskDbWrapper* pDb, int64_t chkpId, char if (taosAcquireRef(taskDbWrapperId, refId) == NULL) { return -1; } - char* pChkpDir = NULL; - char* pChkpIdDir = NULL; - if (chkpPreBuildDir(pDb->path, chkpId, &pChkpDir, &pChkpIdDir) != 0) { - code = -1; - } - if (taosIsDir(pChkpIdDir) && isValidCheckpoint(pChkpIdDir)) { + char* buf = taosMemoryCalloc(1, strlen(pDb->path) + 128); + sprintf(buf, "%s%s%s%s%s%" PRId64 "", pDb->path, TD_DIRSEP, "checkpoints", TD_DIRSEP, "checkpoint", chkpId); + if (taosIsDir(buf)) { code = 0; - *path = pChkpIdDir; - pChkpIdDir = NULL; + *path = buf; + } else { + taosMemoryFree(buf); } - taosMemoryFree(pChkpDir); - taosMemoryFree(pChkpIdDir); taosReleaseRef(taskDbWrapperId, refId); - return code; } diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 74c391f386..a96fbca50b 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -375,17 +375,16 @@ int32_t doUploadChkp(void* param) { SAsyncUploadArg* arg = param; char* path = NULL; int32_t code = 0; - SArray* list = taosArrayInit(4, sizeof(void*)); + SArray* toDelFiles = taosArrayInit(4, sizeof(void*)); - if ((code = taskDbGenChkpUploadData(arg->pTask->pBackend, arg->pTask->pMeta->bkdChkptMgt, arg->chkpId, - (int8_t)(arg->type), &path, list)) != 0) { + (int8_t)(arg->type), &path, toDelFiles)) != 0) { stError("s-task:%s failed to gen upload checkpoint:%" PRId64 "", arg->pTask->id.idStr, arg->chkpId); } - - code = getChkpMeta(arg->taskId, path, list); - if (code != 0) { - code = 0; + if (arg->type == UPLOAD_S3) { + if (code == 0 && (code = getChkpMeta(arg->taskId, path, toDelFiles)) != 0) { + stError("s-task:%s failed to get checkpoint:%" PRId64 " meta", arg->pTask->id.idStr, arg->chkpId); + } } if (code == 0 && (code = uploadCheckpoint(arg->taskId, path)) != 0) { @@ -393,23 +392,24 @@ int32_t doUploadChkp(void* param) { } if (code == 0) { - for (int i = 0; i < taosArrayGetSize(list); i++) { - char* p = taosArrayGetP(list, i); + for (int i = 0; i < taosArrayGetSize(toDelFiles); i++) { + char* p = taosArrayGetP(toDelFiles, i); code = deleteCheckpointFile(arg->taskId, p); - stDebug("try to del file: %s", p); + stDebug("s-task:%s try to del file: %s", arg->pTask->id.idStr, p); if (code != 0) { break; } } } - taosArrayDestroyP(list, taosMemoryFree); + taosArrayDestroyP(toDelFiles, taosMemoryFree); taosRemoveDir(path); taosMemoryFree(path); + taosMemoryFree(arg->taskId); taosMemoryFree(arg); - return 0; + return code; } int32_t streamTaskUploadChkp(SStreamTask* pTask, int64_t chkpId, char* taskId) { // async upload From 146c36b0ad168bc93a4d86d9eb569b701e662ecd Mon Sep 17 00:00:00 2001 From: Bob Liu Date: Mon, 20 Nov 2023 19:25:31 +0800 Subject: [PATCH 140/195] rm unused --- source/dnode/mgmt/node_mgmt/src/dmEnv.c | 1 - source/libs/nodes/src/nodesCodeFuncs.c | 8 +------- 2 files changed, 1 insertion(+), 8 deletions(-) diff --git a/source/dnode/mgmt/node_mgmt/src/dmEnv.c b/source/dnode/mgmt/node_mgmt/src/dmEnv.c index acbb3554ce..6f13abcebc 100644 --- a/source/dnode/mgmt/node_mgmt/src/dmEnv.c +++ b/source/dnode/mgmt/node_mgmt/src/dmEnv.c @@ -17,7 +17,6 @@ #include "dmMgmt.h" #include "audit.h" #include "libs/function/tudf.h" -#include "nodes.h" #define DM_INIT_AUDIT() \ do { \ diff --git a/source/libs/nodes/src/nodesCodeFuncs.c b/source/libs/nodes/src/nodesCodeFuncs.c index d4cd37750b..0a4f9ccdd5 100644 --- a/source/libs/nodes/src/nodesCodeFuncs.c +++ b/source/libs/nodes/src/nodesCodeFuncs.c @@ -46,7 +46,6 @@ typedef struct SBuiltinNodeDefinition { SBuiltinNodeDefinition funcNodes[QUERY_NODE_END] = {NULL}; static TdThreadOnce functionNodeInit = PTHREAD_ONCE_INIT; -volatile int32_t initNodeCode = -1; static void setFunc(const char* name, int32_t type, int32_t nodeSize, FExecNodeToJson toJsonFunc, @@ -65,7 +64,7 @@ bool funcArrayCheck(int32_t type) { nodesError("funcArrayCheck out of range type = %d", type); return false; } - // only init once, do nothing when initNodeCode == 0 + // only init once, do nothing when run funcArrayCheck again taosThreadOnce(&functionNodeInit, doInitNodeFuncArray); if (!funcNodes[type].name) { @@ -7437,10 +7436,6 @@ void nodesDestroyNode(SNode* pNode) { // clang-format off static void doInitNodeFuncArray() { - if (initNodeCode == 0) { - return; - } - setFunc("Column", QUERY_NODE_COLUMN, sizeof(SColumnNode), @@ -8848,7 +8843,6 @@ static void doInitNodeFuncArray() { jsonToPhysiHashJoinNode, destoryHashJoinPhysiNode ); - initNodeCode = 0; } // clang-format on From bb71bd43869dda25802f60fd0fdd6d18766ed7d4 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Mon, 20 Nov 2023 21:03:09 +0800 Subject: [PATCH 141/195] fix mem leak --- source/libs/executor/src/streamtimewindowoperator.c | 1 + source/libs/transport/src/transSvr.c | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/source/libs/executor/src/streamtimewindowoperator.c b/source/libs/executor/src/streamtimewindowoperator.c index 0af7413c75..6e1cfd40f0 100644 --- a/source/libs/executor/src/streamtimewindowoperator.c +++ b/source/libs/executor/src/streamtimewindowoperator.c @@ -3482,6 +3482,7 @@ void doStreamStateSaveCheckpoint(SOperatorInfo* pOperator) { len = doStreamStateEncodeOpState(&pBuf, len, pOperator, true); pInfo->streamAggSup.stateStore.streamStateSaveInfo(pInfo->streamAggSup.pState, STREAM_STATE_OP_CHECKPOINT_NAME, strlen(STREAM_STATE_OP_CHECKPOINT_NAME), buf, len); + taosMemoryFree(pBuf); } static SSDataBlock* buildStateResult(SOperatorInfo* pOperator) { diff --git a/source/libs/transport/src/transSvr.c b/source/libs/transport/src/transSvr.c index bf73c253bc..017969b4e5 100644 --- a/source/libs/transport/src/transSvr.c +++ b/source/libs/transport/src/transSvr.c @@ -159,7 +159,7 @@ static void uvStartSendResp(SSvrMsg* msg); static void uvNotifyLinkBrokenToApp(SSvrConn* conn); -static FORCE_INLINE void destroySmsg(SSvrMsg* smsg); +static FORCE_INLINE void destroySmsg(SSvrMsg* smsg); static FORCE_INLINE SSvrConn* createConn(void* hThrd); static FORCE_INLINE void destroyConn(SSvrConn* conn, bool clear /*clear handle or not*/); static FORCE_INLINE void destroyConnRegArg(SSvrConn* conn); @@ -1499,6 +1499,7 @@ int transSendResponse(const STransMsg* msg) { } SExHandle* exh = msg->info.handle; if (exh == NULL) { + rpcFreeCont(msg->pCont); return 0; } int64_t refId = msg->info.refId; From edaa33d28dbf30e230a58604d76c5ff08b14ac64 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Tue, 21 Nov 2023 10:18:38 +0800 Subject: [PATCH 142/195] fix mem leak --- source/libs/executor/src/streamtimewindowoperator.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/executor/src/streamtimewindowoperator.c b/source/libs/executor/src/streamtimewindowoperator.c index 6e1cfd40f0..893848f010 100644 --- a/source/libs/executor/src/streamtimewindowoperator.c +++ b/source/libs/executor/src/streamtimewindowoperator.c @@ -3482,7 +3482,7 @@ void doStreamStateSaveCheckpoint(SOperatorInfo* pOperator) { len = doStreamStateEncodeOpState(&pBuf, len, pOperator, true); pInfo->streamAggSup.stateStore.streamStateSaveInfo(pInfo->streamAggSup.pState, STREAM_STATE_OP_CHECKPOINT_NAME, strlen(STREAM_STATE_OP_CHECKPOINT_NAME), buf, len); - taosMemoryFree(pBuf); + taosMemoryFree(buf); } static SSDataBlock* buildStateResult(SOperatorInfo* pOperator) { From 947775155b54e5f64903979d4df27b4509be0769 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Tue, 21 Nov 2023 20:51:11 +0800 Subject: [PATCH 143/195] fix transfer crash --- source/dnode/mnode/impl/src/mndStream.c | 6 ++++-- source/dnode/vnode/src/tq/tqStreamStateSnap.c | 1 - source/libs/stream/src/streamCheckpoint.c | 2 +- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index a60b916401..a00ede7270 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -284,6 +284,8 @@ static int32_t mndStreamActionUpdate(SSdb *pSdb, SStreamObj *pOldStream, SStream pOldStream->status = pNewStream->status; pOldStream->updateTime = pNewStream->updateTime; + pOldStream->checkpointId = pNewStream->checkpointId; + pOldStream->checkpointFreq = pNewStream->checkpointFreq; taosWUnLockLatch(&pOldStream->lock); return 0; @@ -1518,7 +1520,7 @@ static int32_t mndRetrieveStream(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock *pB char dstStr[20] = {0}; STR_TO_VARSTR(dstStr, sinkQuota) pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char*) dstStr, false); + colDataSetVal(pColInfo, numOfRows, (const char *)dstStr, false); char scanHistoryIdle[20 + VARSTR_HEADER_SIZE] = {0}; strcpy(scanHistoryIdle, "100a"); @@ -1526,7 +1528,7 @@ static int32_t mndRetrieveStream(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock *pB memset(dstStr, 0, tListLen(dstStr)); STR_TO_VARSTR(dstStr, scanHistoryIdle) pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char*) dstStr, false); + colDataSetVal(pColInfo, numOfRows, (const char *)dstStr, false); numOfRows++; sdbRelease(pSdb, pStream); diff --git a/source/dnode/vnode/src/tq/tqStreamStateSnap.c b/source/dnode/vnode/src/tq/tqStreamStateSnap.c index 791f5ecfed..2ab710176d 100644 --- a/source/dnode/vnode/src/tq/tqStreamStateSnap.c +++ b/source/dnode/vnode/src/tq/tqStreamStateSnap.c @@ -106,7 +106,6 @@ int32_t streamStateSnapRead(SStreamStateReader* pReader, uint8_t** ppData) { memcpy(pHdr->data, rowData, len); taosMemoryFree(rowData); tqDebug("vgId:%d, vnode stream-state snapshot read data success", TD_VID(pReader->pTq->pVnode)); - taosMemoryFree(rowData); return code; _err: diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 37630c25ea..7e66885620 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -288,7 +288,7 @@ int32_t streamSaveTaskCheckpointInfo(SStreamTask* p, int64_t checkpointId) { taosThreadMutexLock(&p->lock); - ASSERT(p->chkInfo.checkpointId < p->chkInfo.checkpointingId && p->chkInfo.checkpointingId == checkpointId && + ASSERT(p->chkInfo.checkpointId <= p->chkInfo.checkpointingId && p->chkInfo.checkpointingId == checkpointId && p->chkInfo.checkpointVer <= p->chkInfo.processedVer); p->chkInfo.checkpointId = p->chkInfo.checkpointingId; p->chkInfo.checkpointVer = p->chkInfo.processedVer; From a2a68ce256e420a99098c054b2f4e2c1362f85b6 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 22 Nov 2023 18:28:38 +0800 Subject: [PATCH 144/195] fix(stream): fix memory leak. --- source/dnode/mnode/impl/src/mndStream.c | 2 ++ 1 file changed, 2 insertions(+) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 8ccda66212..e8a097c44e 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -168,6 +168,8 @@ void mndCleanupStream(SMnode *pMnode) { taosArrayDestroy(execInfo.pTaskList); taosHashCleanup(execInfo.pTaskMap); taosThreadMutexDestroy(&execInfo.lock); + taosHashCleanup(execInfo.transMgmt.pDBTrans); + taosHashCleanup(execInfo.transMgmt.pWaitingList); mDebug("mnd stream exec info cleanup"); } From 50f996bbf7c17b6f52b7714c68e7cd9e93808cbb Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 23 Nov 2023 21:07:58 +0800 Subject: [PATCH 145/195] Merge branch '3.0' into enh/refactorBackend --- source/libs/stream/src/streamBackendRocksdb.c | 1 + 1 file changed, 1 insertion(+) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 6ca47c8616..f07ef71003 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -190,6 +190,7 @@ int32_t getCfIdx(const char* cfName) { } bool isValidCheckpoint(const char* dir) { + return true; STaskDbWrapper* pDb = taskDbOpenImpl(NULL, NULL, (char*)dir); if (pDb == NULL) { return true; From 23ae62d268a01993a476f0b8910a53a8a1c369bc Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Fri, 24 Nov 2023 11:10:39 +0800 Subject: [PATCH 146/195] Merge branch '3.0' into enh/refactorBackend --- source/dnode/mnode/impl/src/mndStream.c | 73 +++++++++---------- source/libs/stream/src/streamBackendRocksdb.c | 2 +- 2 files changed, 37 insertions(+), 38 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index c92dca099b..602b3035b9 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -27,9 +27,9 @@ #include "tmisce.h" #include "tname.h" -#define MND_STREAM_VER_NUMBER 4 -#define MND_STREAM_RESERVE_SIZE 64 -#define MND_STREAM_MAX_NUM 60 +#define MND_STREAM_VER_NUMBER 4 +#define MND_STREAM_RESERVE_SIZE 64 +#define MND_STREAM_MAX_NUM 60 #define MND_STREAM_CHECKPOINT_NAME "stream-checkpoint" #define MND_STREAM_PAUSE_NAME "stream-pause" @@ -77,7 +77,7 @@ static SArray *mndTakeVgroupSnapshot(SMnode *pMnode, bool *allReady); static SVgroupChangeInfo mndFindChangedNodeInfo(SMnode *pMnode, const SArray *pPrevNodeList, const SArray *pNodeList); -static STrans *doCreateTrans(SMnode *pMnode, SStreamObj *pStream, SRpcMsg *pReq, const char *name, const char* pMsg); +static STrans *doCreateTrans(SMnode *pMnode, SStreamObj *pStream, SRpcMsg *pReq, const char *name, const char *pMsg); static int32_t mndPersistTransLog(SStreamObj *pStream, STrans *pTrans); static void initTransAction(STransAction *pAction, void *pCont, int32_t contLen, int32_t msgType, const SEpSet *pEpset, int32_t retryCode); @@ -85,9 +85,9 @@ static int32_t createStreamUpdateTrans(SStreamObj *pStream, SVgroupChangeInfo *p static void removeStreamTasksInBuf(SStreamObj *pStream, SStreamExecInfo *pExecNode); static void saveStreamTasksInfo(SStreamObj *pStream, SStreamExecInfo *pExecNode); static int32_t removeExpirednodeEntryAndTask(SArray *pNodeSnapshot); -static int32_t killActiveCheckpointTrans(SMnode *pMnode, const char* pDbName, size_t len); +static int32_t killActiveCheckpointTrans(SMnode *pMnode, const char *pDbName, size_t len); static int32_t setNodeEpsetExpiredFlag(const SArray *pNodeList); -static void freeCheckpointCandEntry(void*); +static void freeCheckpointCandEntry(void *); static SSdbRaw *mndStreamActionEncode(SStreamObj *pStream); static SSdbRow *mndStreamActionDecode(SSdbRaw *pRaw); @@ -168,9 +168,8 @@ void mndCleanupStream(SMnode *pMnode) { taosArrayDestroy(execInfo.pTaskList); taosHashCleanup(execInfo.pTaskMap); taosHashCleanup(execInfo.transMgmt.pDBTrans); - taosThreadMutexDestroy(&execInfo.lock); - taosHashCleanup(execInfo.transMgmt.pDBTrans); taosHashCleanup(execInfo.transMgmt.pWaitingList); + taosThreadMutexDestroy(&execInfo.lock); mDebug("mnd stream exec info cleanup"); } @@ -333,11 +332,11 @@ static void mndShowStreamStatus(char *dst, SStreamObj *pStream) { } } -SSdbRaw * mndStreamSeqActionEncode(SStreamObj *pStream) { return NULL; } -SSdbRow * mndStreamSeqActionDecode(SSdbRaw *pRaw) { return NULL; } -int32_t mndStreamSeqActionInsert(SSdb *pSdb, SStreamSeq *pStream) { return 0; } -int32_t mndStreamSeqActionDelete(SSdb *pSdb, SStreamSeq *pStream) { return 0; } -int32_t mndStreamSeqActionUpdate(SSdb *pSdb, SStreamSeq *pOldStream, SStreamSeq *pNewStream) { return 0; } +SSdbRaw *mndStreamSeqActionEncode(SStreamObj *pStream) { return NULL; } +SSdbRow *mndStreamSeqActionDecode(SSdbRaw *pRaw) { return NULL; } +int32_t mndStreamSeqActionInsert(SSdb *pSdb, SStreamSeq *pStream) { return 0; } +int32_t mndStreamSeqActionDelete(SSdb *pSdb, SStreamSeq *pStream) { return 0; } +int32_t mndStreamSeqActionUpdate(SSdb *pSdb, SStreamSeq *pOldStream, SStreamSeq *pNewStream) { return 0; } static void mndShowStreamTrigger(char *dst, SStreamObj *pStream) { int8_t trigger = pStream->conf.trigger; @@ -761,7 +760,7 @@ int32_t mndDropStreamTasks(SMnode *pMnode, STrans *pTrans, SStreamObj *pStream) static int32_t checkForNumOfStreams(SMnode *pMnode, SStreamObj *pStreamObj) { // check for number of existed tasks int32_t numOfStream = 0; SStreamObj *pStream = NULL; - void *pIter = NULL; + void * pIter = NULL; while ((pIter = sdbFetch(pMnode->pSdb, SDB_STREAM, pIter, (void **)&pStream)) != NULL) { if (pStream->sourceDbUid == pStreamObj->sourceDbUid) { @@ -1087,7 +1086,7 @@ static int32_t mndProcessStreamCheckpointTrans(SMnode *pMnode, SStreamObj *pStre } STransAction act = {0}; - SEpSet epset = mndGetVgroupEpset(pMnode, pVgObj); + SEpSet epset = mndGetVgroupEpset(pMnode, pVgObj); mndReleaseVgroup(pMnode, pVgObj); initTransAction(&act, buf, tlen, TDMT_VND_STREAM_CHECK_POINT_SOURCE, &epset, TSDB_CODE_SYN_PROPOSE_NOT_READY); @@ -1265,7 +1264,7 @@ static int32_t mndCheckNodeStatus(SMnode *pMnode) { } for (int32_t i = 0; i < taosArrayGetSize(execInfo.pTaskList); ++i) { - STaskId *p = taosArrayGet(execInfo.pTaskList, i); + STaskId * p = taosArrayGet(execInfo.pTaskList, i); STaskStatusEntry *pEntry = taosHashGet(execInfo.pTaskMap, p, sizeof(*p)); if (pEntry == NULL) { continue; @@ -1280,13 +1279,13 @@ static int32_t mndCheckNodeStatus(SMnode *pMnode) { } taosThreadMutexUnlock(&execInfo.lock); - return ready? 0:-1; + return ready ? 0 : -1; } static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { - SMnode *pMnode = pReq->info.node; - SSdb *pSdb = pMnode->pSdb; - void *pIter = NULL; + SMnode * pMnode = pReq->info.node; + SSdb * pSdb = pMnode->pSdb; + void * pIter = NULL; SStreamObj *pStream = NULL; int32_t code = 0; @@ -1308,7 +1307,7 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { static int32_t mndProcessStreamCheckpointInCandid(SRpcMsg *pReq) { SMnode *pMnode = pReq->info.node; - void *pIter = NULL; + void * pIter = NULL; int32_t code = 0; taosThreadMutexLock(&execInfo.lock); @@ -1327,7 +1326,7 @@ static int32_t mndProcessStreamCheckpointInCandid(SRpcMsg *pReq) { SCheckpointCandEntry *pEntry = pIter; SStreamObj *ps = mndAcquireStream(pMnode, pEntry->pName); - mDebug("start to launch checkpoint for stream:%s %"PRIx64" in candidate list", pEntry->pName, pEntry->streamId); + mDebug("start to launch checkpoint for stream:%s %" PRIx64 " in candidate list", pEntry->pName, pEntry->streamId); code = mndProcessStreamCheckpointTrans(pMnode, ps, pEntry->checkpointId); mndReleaseStream(pMnode, ps); @@ -2497,13 +2496,13 @@ int32_t removeExpirednodeEntryAndTask(SArray *pNodeSnapshot) { } // kill all trans in the dst DB -static void killAllCheckpointTrans(SMnode* pMnode, SVgroupChangeInfo* pChangeInfo) { - void* pIter = NULL; - while((pIter = taosHashIterate(pChangeInfo->pDBMap, pIter)) != NULL) { - char* pDb = (char*) pIter; +static void killAllCheckpointTrans(SMnode *pMnode, SVgroupChangeInfo *pChangeInfo) { + void *pIter = NULL; + while ((pIter = taosHashIterate(pChangeInfo->pDBMap, pIter)) != NULL) { + char *pDb = (char *)pIter; size_t len = 0; - void* pKey = taosHashGetKey(pDb, &len); + void * pKey = taosHashGetKey(pDb, &len); killActiveCheckpointTrans(pMnode, pKey, len); } } @@ -2652,7 +2651,7 @@ void removeStreamTasksInBuf(SStreamObj *pStream, SStreamExecInfo *pExecNode) { ASSERT(taosHashGetSize(pExecNode->pTaskMap) == taosArrayGetSize(pExecNode->pTaskList)); } -STrans *doCreateTrans(SMnode *pMnode, SStreamObj *pStream, SRpcMsg *pReq, const char *name, const char* pMsg) { +STrans *doCreateTrans(SMnode *pMnode, SStreamObj *pStream, SRpcMsg *pReq, const char *name, const char *pMsg) { STrans *pTrans = mndTransCreate(pMnode, TRN_POLICY_RETRY, TRN_CONFLICT_DB_INSIDE, pReq, name); if (pTrans == NULL) { mError("failed to build trans:%s, reason: %s", name, tstrerror(TSDB_CODE_OUT_OF_MEMORY)); @@ -2660,7 +2659,7 @@ STrans *doCreateTrans(SMnode *pMnode, SStreamObj *pStream, SRpcMsg *pReq, const return NULL; } - mDebug("s-task:0x%"PRIx64" start to build trans %s", pStream->uid, pMsg); + mDebug("s-task:0x%" PRIx64 " start to build trans %s", pStream->uid, pMsg); mndTransSetDbName(pTrans, pStream->sourceDb, pStream->targetDb); if (mndTransCheckConflict(pMnode, pTrans) != 0) { @@ -2740,9 +2739,9 @@ int32_t createStreamResetStatusTrans(SMnode *pMnode, SStreamObj *pStream) { return TSDB_CODE_ACTION_IN_PROGRESS; } -int32_t killActiveCheckpointTrans(SMnode *pMnode, const char* pDBName, size_t len) { +int32_t killActiveCheckpointTrans(SMnode *pMnode, const char *pDBName, size_t len) { // data in the hash table will be removed automatically, no need to remove it here. - SStreamTransInfo* pTransInfo = taosHashGet(execInfo.transMgmt.pDBTrans, pDBName, len); + SStreamTransInfo *pTransInfo = taosHashGet(execInfo.transMgmt.pDBTrans, pDBName, len); if (pTransInfo == NULL) { return TSDB_CODE_SUCCESS; } @@ -2753,7 +2752,7 @@ int32_t killActiveCheckpointTrans(SMnode *pMnode, const char* pDBName, size_t le return TSDB_CODE_SUCCESS; } - STrans* pTrans = mndAcquireTrans(pMnode, pTransInfo->transId); + STrans *pTrans = mndAcquireTrans(pMnode, pTransInfo->transId); if (pTrans != NULL) { mInfo("kill checkpoint transId:%d in Db:%s", pTransInfo->transId, pDBName); mndKillTrans(pMnode, pTrans); @@ -2764,7 +2763,7 @@ int32_t killActiveCheckpointTrans(SMnode *pMnode, const char* pDBName, size_t le } int32_t mndResetStatusFromCheckpoint(SMnode *pMnode, int32_t transId) { - STrans* pTrans = mndAcquireTrans(pMnode, transId); + STrans *pTrans = mndAcquireTrans(pMnode, transId); if (pTrans != NULL) { mInfo("kill checkpoint transId:%d to reset task status", transId); mndKillTrans(pMnode, pTrans); @@ -2783,8 +2782,8 @@ int32_t mndResetStatusFromCheckpoint(SMnode *pMnode, int32_t transId) { bool conflict = mndStreamTransConflictOtherTrans(pMnode, pStream->sourceDb, pStream->targetDb); if (conflict) { - mError("stream:%s other trans exists in DB:%s & %s failed to start reset-status trans", - pStream->name, pStream->sourceDb, pStream->targetDb); + mError("stream:%s other trans exists in DB:%s & %s failed to start reset-status trans", pStream->name, + pStream->sourceDb, pStream->targetDb); continue; } @@ -2928,7 +2927,7 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { return TSDB_CODE_SUCCESS; } -void freeCheckpointCandEntry(void* param) { - SCheckpointCandEntry* pEntry = param; +void freeCheckpointCandEntry(void *param) { + SCheckpointCandEntry *pEntry = param; taosMemoryFreeClear(pEntry->pName); } diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index f07ef71003..dc61bd25f2 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -190,7 +190,7 @@ int32_t getCfIdx(const char* cfName) { } bool isValidCheckpoint(const char* dir) { - return true; + // return true; STaskDbWrapper* pDb = taskDbOpenImpl(NULL, NULL, (char*)dir); if (pDb == NULL) { return true; From f28cf740b09fc4e0d539f3b722f7c5216e56a174 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 24 Nov 2023 14:44:00 +0800 Subject: [PATCH 147/195] fix(stream): fix status check for stream-scan-history status when drop stream task msg is missing. --- include/libs/stream/tstream.h | 2 ++ source/dnode/mnode/impl/src/mndStream.c | 19 +++++++++++++++-- source/util/src/tdecompress.c | 27 +++++++++++++------------ 3 files changed, 33 insertions(+), 15 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index cf9fc1d826..b8ceb31b80 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -664,6 +664,8 @@ typedef struct STaskStatusEntry { int32_t relatedHTask; // has related fill-history task int64_t activeCheckpointId; // current active checkpoint id bool checkpointFailed; // denote if the checkpoint is failed or not + bool inputQChanging; // inputQ is changing or not + int64_t inputQUnchangeCounter; double inputQUsed; // in MiB double inputRate; double sinkQuota; // existed quota size for sink task diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index db013017e3..5490d53587 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -2790,7 +2790,7 @@ static SStreamTask* mndGetStreamTask(STaskId* pId, SStreamObj* pStream) { static bool needDropRelatedFillhistoryTask(STaskStatusEntry *pTaskEntry, SStreamExecInfo *pExecNode) { if (pTaskEntry->status == TASK_STATUS__STREAM_SCAN_HISTORY && pTaskEntry->statusLastDuration >= 10) { - if (fabs(pTaskEntry->inputQUsed) <= DBL_EPSILON) { + if (!pTaskEntry->inputQChanging && pTaskEntry->inputQUnchangeCounter > 10) { int32_t numOfReady = 0; int32_t numOfTotal = 0; for (int32_t k = 0; k < taosArrayGetSize(pExecNode->pTaskList); ++k) { @@ -2933,6 +2933,7 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { bool snodeChanged = false; for (int32_t i = 0; i < req.numOfTasks; ++i) { STaskStatusEntry *p = taosArrayGet(req.pTaskStatus, i); + STaskStatusEntry *pTaskEntry = taosHashGet(execInfo.pTaskMap, &p->id, sizeof(p->id)); if (pTaskEntry == NULL) { mError("s-task:0x%" PRIx64 " not found in mnode task list", p->id.taskId); @@ -2941,8 +2942,22 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { if (pTaskEntry->stage != p->stage && pTaskEntry->stage != -1) { updateStageInfo(pTaskEntry, p->stage); - if(pTaskEntry->nodeId == SNODE_HANDLE) snodeChanged = true; + if(pTaskEntry->nodeId == SNODE_HANDLE) { + snodeChanged = true; + } } else { + // task is idle for more than 50 sec. + if (fabs(pTaskEntry->inputQUsed - p->inputQUsed) <= DBL_EPSILON) { + if (!pTaskEntry->inputQChanging) { + pTaskEntry->inputQUnchangeCounter++; + } else { + pTaskEntry->inputQChanging = false; + } + } else { + pTaskEntry->inputQChanging = true; + pTaskEntry->inputQUnchangeCounter = 0; + } + streamTaskStatusCopy(pTaskEntry, p); if (p->activeCheckpointId != 0) { if (activeCheckpointId != 0) { diff --git a/source/util/src/tdecompress.c b/source/util/src/tdecompress.c index 68841941db..19f4e1fd92 100644 --- a/source/util/src/tdecompress.c +++ b/source/util/src/tdecompress.c @@ -53,11 +53,8 @@ int32_t tsDecompressIntImpl_Hw(const char *const input, const int32_t nelements, int64_t prevValue = 0; #if __AVX2__ - while (1) { - if (_pos == nelements) break; - - uint64_t w = 0; - memcpy(&w, ip, LONG_BYTES); + while (_pos < nelements) { + uint64_t w = *(uint64_t*) ip; char selector = (char)(w & INT64MASK(4)); // selector = 4 char bit = bit_per_integer[(int32_t)selector]; // bit = 3 @@ -261,17 +258,20 @@ int32_t tsDecompressTimestampAvx2(const char* const input, const int32_t nelemen int64_t deltaOfDelta = 0; int32_t longBytes = LONG_BYTES; + // _mm_maskz_loadu_epi8 #if __AVX2__ - int32_t batch = nelements >> 2; - int32_t remainder = nelements & 0x1; + // _mm_blendv_epi8 + int32_t batch = nelements >> 4; + int32_t remainder = nelements & 0x03; - while (1) { + for(int32_t i = 0; i < batch; ++i) { uint8_t flags = input[ipos++]; // Decode dd1 uint64_t dd1 = 0; - nbytes = flags & INT8MASK(4); + nbytes = flags & INT8MASK(4); // range of nbytes starts from 0 to 7 + // __m128i mask = {};//[0], [] if (nbytes == 0) { deltaOfDelta = 0; @@ -289,10 +289,6 @@ int32_t tsDecompressTimestampAvx2(const char* const input, const int32_t nelemen prevValue += prevDelta; ostream[opos++] = prevValue; - if (opos == nelements) { - return nelements * longBytes; - } - // Decode dd2 uint64_t dd2 = 0; nbytes = (flags >> 4) & INT8MASK(4); @@ -317,6 +313,11 @@ int32_t tsDecompressTimestampAvx2(const char* const input, const int32_t nelemen return nelements * longBytes; } } + + if (remainder > 0) { + + } + #endif return 0; } \ No newline at end of file From 3deab3cfbfdfde63bf550f46bbf8d948e6c4c10f Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 24 Nov 2023 15:37:30 +0800 Subject: [PATCH 148/195] fix(stream):set failure for the related fill-history task. --- source/libs/stream/src/streamStart.c | 13 ++++++++++++- source/util/src/tdecompress.c | 2 +- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/source/libs/stream/src/streamStart.c b/source/libs/stream/src/streamStart.c index 469813defc..ce18e770b1 100644 --- a/source/libs/stream/src/streamStart.c +++ b/source/libs/stream/src/streamStart.c @@ -471,6 +471,14 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs addIntoNodeUpdateList(pTask, pRsp->downstreamNodeId); streamMetaUpdateTaskDownstreamStatus(pTask, pTask->execInfo.init, taosGetTimestampMs(), false); + // automatically set the related fill-history task to be failed. + if (HAS_RELATED_FILLHISTORY_TASK(pTask)) { + STaskId* pId = &pTask->hTaskInfo.id; + + SStreamTask* pHTask = streamMetaAcquireTask(pTask->pMeta, pId->streamId, pId->taskId); + streamMetaUpdateTaskDownstreamStatus(pHTask, pHTask->execInfo.init, taosGetTimestampMs(), false); + streamMetaReleaseTask(pTask->pMeta, pHTask); + } } else { // TASK_DOWNSTREAM_NOT_READY, let's retry in 100ms STaskRecheckInfo* pInfo = createRecheckInfo(pTask, pRsp); @@ -1072,8 +1080,9 @@ int32_t streamMetaUpdateTaskDownstreamStatus(SStreamTask* pTask, int64_t startTs taosHashPut(pDst, &id, sizeof(id), &initTs, sizeof(STaskInitTs)); int32_t numOfTotal = streamMetaGetNumOfTasks(pMeta); + int32_t numOfRecv = taosHashGetSize(pStartInfo->pReadyTaskSet) + taosHashGetSize(pStartInfo->pFailedTaskSet); - if (taosHashGetSize(pStartInfo->pReadyTaskSet) + taosHashGetSize(pStartInfo->pFailedTaskSet) == numOfTotal) { + if (numOfRecv == numOfTotal) { pStartInfo->readyTs = taosGetTimestampMs(); pStartInfo->elapsedTime = (pStartInfo->startTs != 0) ? pStartInfo->readyTs - pStartInfo->startTs : 0; @@ -1087,6 +1096,8 @@ int32_t streamMetaUpdateTaskDownstreamStatus(SStreamTask* pTask, int64_t startTs displayStatusInfo(pMeta, pStartInfo->pFailedTaskSet, false); streamMetaResetStartInfo(pStartInfo); + } else { + stDebug("vgId:%d recv check down results:%d, total:%d", pMeta->vgId, numOfRecv, numOfTotal); } streamMetaWUnLock(pMeta); diff --git a/source/util/src/tdecompress.c b/source/util/src/tdecompress.c index 19f4e1fd92..ac9d70b2e7 100644 --- a/source/util/src/tdecompress.c +++ b/source/util/src/tdecompress.c @@ -259,7 +259,7 @@ int32_t tsDecompressTimestampAvx2(const char* const input, const int32_t nelemen int32_t longBytes = LONG_BYTES; // _mm_maskz_loadu_epi8 -#if __AVX2__ +#if __AVX512F__ // _mm_blendv_epi8 int32_t batch = nelements >> 4; From 6416a6153a32043699a6970f7159a57e632dff1e Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Fri, 24 Nov 2023 16:29:09 +0800 Subject: [PATCH 149/195] Merge branch '3.0' into enh/refactorBackend --- source/dnode/mnode/impl/src/mndStream.c | 4 +- source/libs/stream/inc/streamBackendRocksdb.h | 9 ++-- source/libs/stream/src/streamBackendRocksdb.c | 53 +++++++++++-------- source/libs/stream/src/streamMeta.c | 6 +-- 4 files changed, 42 insertions(+), 30 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 602b3035b9..37c3683edb 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -942,11 +942,11 @@ int64_t mndStreamGenChkpId(SMnode *pMnode) { if (pIter == NULL) break; maxChkpId = TMAX(maxChkpId, pStream->checkpointId); - mError("stream %p checkpoint %" PRId64 "", pStream, pStream->checkpointId); + mDebug("stream %p checkpoint %" PRId64 "", pStream, pStream->checkpointId); sdbRelease(pSdb, pStream); } - mError("generated checkpoint %" PRId64 "", maxChkpId + 1); + mDebug("generated checkpoint %" PRId64 "", maxChkpId + 1); return maxChkpId + 1; } diff --git a/source/libs/stream/inc/streamBackendRocksdb.h b/source/libs/stream/inc/streamBackendRocksdb.h index a73c289440..bed0f79f02 100644 --- a/source/libs/stream/inc/streamBackendRocksdb.h +++ b/source/libs/stream/inc/streamBackendRocksdb.h @@ -139,7 +139,8 @@ void streamBackendDelCompare(void* backend, void* arg); int32_t streamStateCvtDataFormat(char* path, char* key, void* cfInst); STaskDbWrapper* taskDbOpen(char* path, char* key, int64_t chkpId); -void taskDbDestroy(void* pBackend); +void taskDbDestroy(void* pBackend, bool flush); +void taskDbDestroy2(void* pBackend); int32_t taskDbDoCheckpoint(void* arg, int64_t chkpId); void taskDbUpdateChkpId(void* pTaskDb, int64_t chkpId); @@ -217,7 +218,7 @@ int32_t streamDefaultGet_rocksdb(SStreamState* pState, const void* key, void** p int32_t streamDefaultDel_rocksdb(SStreamState* pState, const void* key); int32_t streamDefaultIterGet_rocksdb(SStreamState* pState, const void* start, const void* end, SArray* result); void* streamDefaultIterCreate_rocksdb(SStreamState* pState); -bool streamDefaultIterValid_rocksdb(void* iter); +bool streamDefaultIterValid_rocksdb(void* iter); void streamDefaultIterSeek_rocksdb(void* iter, const char* key); void streamDefaultIterNext_rocksdb(void* iter); char* streamDefaultIterKey_rocksdb(void* iter, int32_t* len); @@ -245,8 +246,8 @@ int32_t taskDbBuildSnap(void* arg, SArray* pSnap); // int32_t streamDefaultIter_rocksdb(SStreamState* pState, const void* start, const void* end, SArray* result); -STaskDbWrapper* taskDbOpen(char* path, char* key, int64_t chkpId); -void taskDbDestroy(void* pDb); +// STaskDbWrapper* taskDbOpen(char* path, char* key, int64_t chkpId); +// void taskDbDestroy(void* pDb, bool flush); int32_t taskDbDoCheckpoint(void* arg, int64_t chkpId); diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index dc61bd25f2..ccf166c390 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -190,12 +190,12 @@ int32_t getCfIdx(const char* cfName) { } bool isValidCheckpoint(const char* dir) { - // return true; + return true; STaskDbWrapper* pDb = taskDbOpenImpl(NULL, NULL, (char*)dir); if (pDb == NULL) { - return true; + return false; } - taskDbDestroy(pDb); + taskDbDestroy(pDb, false); return true; } @@ -1788,7 +1788,7 @@ STaskDbWrapper* taskDbOpenImpl(char* key, char* statePath, char* dbPath) { return pTaskDb; _EXIT: - taskDbDestroy(pTaskDb); + taskDbDestroy(pTaskDb, false); if (err) taosMemoryFree(err); if (cfNames) rocksdb_list_column_families_destroy(cfNames, nCf); return NULL; @@ -1807,7 +1807,7 @@ STaskDbWrapper* taskDbOpen(char* path, char* key, int64_t chkpId) { return pTaskDb; } -void taskDbDestroy(void* pDb) { +void taskDbDestroy(void* pDb, bool flush) { STaskDbWrapper* wrapper = pDb; qDebug("succ to destroy stream backend:%p", wrapper); @@ -1815,24 +1815,33 @@ void taskDbDestroy(void* pDb) { if (wrapper == NULL) return; - if (wrapper->db && wrapper->pCf) { - rocksdb_flushoptions_t* flushOpt = rocksdb_flushoptions_create(); - rocksdb_flushoptions_set_wait(flushOpt, 1); + if (flush) { + if (wrapper->db && wrapper->pCf) { + rocksdb_flushoptions_t* flushOpt = rocksdb_flushoptions_create(); + rocksdb_flushoptions_set_wait(flushOpt, 1); - char* err = NULL; - for (int i = 0; i < nCf; i++) { - if (wrapper->pCf[i] != NULL) rocksdb_flush_cf(wrapper->db, flushOpt, wrapper->pCf[i], &err); - if (err != NULL) { - stError("failed to flush cf:%s, reason:%s", ginitDict[i].key, err); - taosMemoryFreeClear(err); + char* err = NULL; + rocksdb_column_family_handle_t** cfs = taosMemoryCalloc(1, sizeof(rocksdb_column_family_handle_t*) * nCf); + int numOfFlushCf = 0; + for (int i = 0; i < nCf; i++) { + if (wrapper->pCf[i] != NULL) { + cfs[numOfFlushCf++] = wrapper->pCf[i]; + } } + if (numOfFlushCf != 0) { + rocksdb_flush_cfs(wrapper->db, flushOpt, cfs, numOfFlushCf, &err); + if (err != NULL) { + stError("failed to flush all cfs, reason:%s", err); + taosMemoryFreeClear(err); + } + } + taosMemoryFree(cfs); + rocksdb_flushoptions_destroy(flushOpt); } - rocksdb_flushoptions_destroy(flushOpt); - - for (int i = 0; i < nCf; i++) { - if (wrapper->pCf[i] != NULL) { - rocksdb_column_family_handle_destroy(wrapper->pCf[i]); - } + } + for (int i = 0; i < nCf; i++) { + if (wrapper->pCf[i] != NULL) { + rocksdb_column_family_handle_destroy(wrapper->pCf[i]); } } rocksdb_options_destroy(wrapper->dbOpt); @@ -1869,6 +1878,8 @@ void taskDbDestroy(void* pDb) { return; } +void taskDbDestroy2(void* pDb) { taskDbDestroy(pDb, true); } + int32_t taskDbGenChkpUploadData__rsync(STaskDbWrapper* pDb, int64_t chkpId, char** path) { int64_t st = taosGetTimestampMs(); int32_t code = -1; @@ -2007,7 +2018,7 @@ int32_t streamStateCvtDataFormat(char* path, char* key, void* pCfInst) { } _EXIT: - taskDbDestroy(pTaskDb); + taskDbDestroy(pTaskDb, true); return code; } diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 78b4814aa6..0e137a673e 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -57,7 +57,7 @@ int32_t metaRefMgtAdd(int64_t vgId, int64_t* rid); static void streamMetaEnvInit() { streamBackendId = taosOpenRef(64, streamBackendCleanup); streamBackendCfWrapperId = taosOpenRef(64, streamBackendHandleCleanup); - taskDbWrapperId = taosOpenRef(64, taskDbDestroy); + taskDbWrapperId = taosOpenRef(64, taskDbDestroy2); streamMetaId = taosOpenRef(64, streamMetaCloseImpl); @@ -1239,8 +1239,8 @@ bool streamMetaTaskInTimer(SStreamMeta* pMeta) { void streamMetaNotifyClose(SStreamMeta* pMeta) { int32_t vgId = pMeta->vgId; - stDebug("vgId:%d notify all stream tasks that the vnode is closing. isLeader:%d startHb:%" PRId64 ", totalHb:%d", vgId, - (pMeta->role == NODE_ROLE_LEADER), pMeta->pHbInfo->hbStart, pMeta->pHbInfo->hbCount); + stDebug("vgId:%d notify all stream tasks that the vnode is closing. isLeader:%d startHb:%" PRId64 ", totalHb:%d", + vgId, (pMeta->role == NODE_ROLE_LEADER), pMeta->pHbInfo->hbStart, pMeta->pHbInfo->hbCount); streamMetaWLock(pMeta); From 788194c74890585fbbf3dcdbc83a93337e6b3db0 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 24 Nov 2023 22:53:55 +0800 Subject: [PATCH 150/195] fix(stream): fix deadlock. --- source/dnode/mnode/impl/inc/mndStream.h | 2 +- source/dnode/mnode/impl/src/mndStream.c | 8 ++-- source/dnode/mnode/impl/src/mndStreamTrans.c | 41 ++++++++++++-------- 3 files changed, 30 insertions(+), 21 deletions(-) diff --git a/source/dnode/mnode/impl/inc/mndStream.h b/source/dnode/mnode/impl/inc/mndStream.h index 244a6d08dd..f2b3ce6cac 100644 --- a/source/dnode/mnode/impl/inc/mndStream.h +++ b/source/dnode/mnode/impl/inc/mndStream.h @@ -52,7 +52,7 @@ int32_t mndDropStreamByDb(SMnode *pMnode, STrans *pTrans, SDbObj *pDb); int32_t mndPersistStream(SMnode *pMnode, STrans *pTrans, SStreamObj *pStream); int32_t mndStreamRegisterTrans(STrans* pTrans, const char* pName, const char* pSrcDb, const char* pDstDb); -bool streamTransConflictOtherTrans(SMnode *pMnode, const char *pSrcDb, const char *pDstDb); +bool streamTransConflictOtherTrans(SMnode *pMnode, const char *pSrcDb, const char *pDstDb, bool lock); // for sma // TODO refactor diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 5490d53587..81cb4703e1 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1348,7 +1348,7 @@ static int32_t mndProcessDropStreamReq(SRpcMsg *pReq) { } // check if it is conflict with other trans in both sourceDb and targetDb. - bool conflict = streamTransConflictOtherTrans(pMnode, pStream->sourceDb, pStream->targetDb); + bool conflict = streamTransConflictOtherTrans(pMnode, pStream->sourceDb, pStream->targetDb, true); if (conflict) { sdbRelease(pMnode->pSdb, pStream); tFreeMDropStreamReq(&dropReq); @@ -1831,7 +1831,7 @@ static int32_t mndProcessPauseStreamReq(SRpcMsg *pReq) { } // check if it is conflict with other trans in both sourceDb and targetDb. - bool conflict = streamTransConflictOtherTrans(pMnode, pStream->sourceDb, pStream->targetDb); + bool conflict = streamTransConflictOtherTrans(pMnode, pStream->sourceDb, pStream->targetDb, true); if (conflict) { sdbRelease(pMnode->pSdb, pStream); return -1; @@ -1966,7 +1966,7 @@ static int32_t mndProcessResumeStreamReq(SRpcMsg *pReq) { } // check if it is conflict with other trans in both sourceDb and targetDb. - bool conflict = streamTransConflictOtherTrans(pMnode, pStream->sourceDb, pStream->targetDb); + bool conflict = streamTransConflictOtherTrans(pMnode, pStream->sourceDb, pStream->targetDb, true); if (conflict) { sdbRelease(pMnode->pSdb, pStream); return -1; @@ -2754,7 +2754,7 @@ static int32_t mndResetStatusFromCheckpoint(SMnode *pMnode, int32_t transId) { break; } - bool conflict = streamTransConflictOtherTrans(pMnode, pStream->sourceDb, pStream->targetDb); + bool conflict = streamTransConflictOtherTrans(pMnode, pStream->sourceDb, pStream->targetDb, false); if (conflict) { mError("stream:%s other trans exists in DB:%s & %s failed to start reset-status trans", pStream->name, pStream->sourceDb, pStream->targetDb); diff --git a/source/dnode/mnode/impl/src/mndStreamTrans.c b/source/dnode/mnode/impl/src/mndStreamTrans.c index 2345de290a..9dd9f64037 100644 --- a/source/dnode/mnode/impl/src/mndStreamTrans.c +++ b/source/dnode/mnode/impl/src/mndStreamTrans.c @@ -35,17 +35,15 @@ int32_t mndStreamRegisterTrans(STrans* pTrans, const char* pName, const char* pS } int32_t clearFinishedTrans(SMnode* pMnode) { - SArray* pList = taosArrayInit(4, sizeof(SKeyInfo)); size_t keyLen = 0; + SArray* pList = taosArrayInit(4, sizeof(SKeyInfo)); + void* pIter = NULL; - taosThreadMutexLock(&execInfo.lock); - - void* pIter = NULL; while ((pIter = taosHashIterate(execInfo.transMgmt.pDBTrans, pIter)) != NULL) { - SStreamTransInfo *pEntry = (SStreamTransInfo *)pIter; - STrans* pTrans = mndAcquireTrans(pMnode, pEntry->transId); + SStreamTransInfo* pEntry = (SStreamTransInfo*)pIter; // let's clear the finished trans + STrans* pTrans = mndAcquireTrans(pMnode, pEntry->transId); if (pTrans == NULL) { void* pKey = taosHashGetKey(pEntry, &keyLen); // key is the name of src/dst db name @@ -60,44 +58,55 @@ int32_t clearFinishedTrans(SMnode* pMnode) { } size_t num = taosArrayGetSize(pList); - for(int32_t i = 0; i < num; ++i) { + for (int32_t i = 0; i < num; ++i) { SKeyInfo* pKey = taosArrayGet(pList, i); taosHashRemove(execInfo.transMgmt.pDBTrans, pKey->pKey, pKey->keyLen); } - mDebug("clear %d finished stream-trans, remained:%d", (int32_t) num, taosHashGetSize(execInfo.transMgmt.pDBTrans)); - taosThreadMutexUnlock(&execInfo.lock); + mDebug("clear %d finished stream-trans, remained:%d", (int32_t)num, taosHashGetSize(execInfo.transMgmt.pDBTrans)); terrno = TSDB_CODE_SUCCESS; taosArrayDestroy(pList); return 0; } -bool streamTransConflictOtherTrans(SMnode* pMnode, const char* pSrcDb, const char* pDstDb) { - clearFinishedTrans(pMnode); +bool streamTransConflictOtherTrans(SMnode* pMnode, const char* pSrcDb, const char* pDstDb, bool lock) { + if (lock) { + taosThreadMutexLock(&execInfo.lock); + } - taosThreadMutexLock(&execInfo.lock); int32_t num = taosHashGetSize(execInfo.transMgmt.pDBTrans); if (num <= 0) { - taosThreadMutexUnlock(&execInfo.lock); + if (lock) { + taosThreadMutexUnlock(&execInfo.lock); + } return false; } + clearFinishedTrans(pMnode); + SStreamTransInfo *pEntry = taosHashGet(execInfo.transMgmt.pDBTrans, pSrcDb, strlen(pSrcDb)); if (pEntry != NULL) { - taosThreadMutexUnlock(&execInfo.lock); + if (lock) { + taosThreadMutexUnlock(&execInfo.lock); + } mWarn("conflict with other transId:%d in Db:%s, trans:%s", pEntry->transId, pSrcDb, pEntry->name); return true; } pEntry = taosHashGet(execInfo.transMgmt.pDBTrans, pDstDb, strlen(pDstDb)); if (pEntry != NULL) { - taosThreadMutexUnlock(&execInfo.lock); + if (lock) { + taosThreadMutexUnlock(&execInfo.lock); + } mWarn("conflict with other transId:%d in Db:%s, trans:%s", pEntry->transId, pSrcDb, pEntry->name); return true; } - taosThreadMutexUnlock(&execInfo.lock); + if (lock) { + taosThreadMutexUnlock(&execInfo.lock); + } + return false; } From 30c776cc0a2c82a2aafeb3f865caf8ffd575d310 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 25 Nov 2023 00:54:32 +0800 Subject: [PATCH 151/195] fix(stream): reset the scheduler status. --- source/dnode/vnode/src/tq/tq.c | 2 ++ 1 file changed, 2 insertions(+) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 9d16402ee6..821adc9953 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -2041,6 +2041,8 @@ int32_t tqProcessTaskDropHTask(STQ* pTq, SRpcMsg* pMsg) { SStreamTaskId id = {.streamId = pTask->hTaskInfo.id.streamId, .taskId = pTask->hTaskInfo.id.taskId}; streamBuildAndSendDropTaskMsg(pTask->pMsgCb, pMeta->vgId, &id); + // clear the scheduler status + streamTaskSetSchedStatusInactive(pTask); streamMetaReleaseTask(pMeta, pTask); return TSDB_CODE_SUCCESS; } From 9194c0c0cd929f1d78cddb7fbd532170d0756292 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 25 Nov 2023 00:56:18 +0800 Subject: [PATCH 152/195] fix(stream): add logs. --- source/dnode/vnode/src/tq/tq.c | 1 + 1 file changed, 1 insertion(+) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 821adc9953..de8305e6d9 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -2043,6 +2043,7 @@ int32_t tqProcessTaskDropHTask(STQ* pTq, SRpcMsg* pMsg) { // clear the scheduler status streamTaskSetSchedStatusInactive(pTask); + tqDebug("s-task:%s set scheduler status:%d after drop fill-history task", pTask->id.idStr, pTask->status.schedStatus); streamMetaReleaseTask(pMeta, pTask); return TSDB_CODE_SUCCESS; } From 1dd5cd17a04a4785026e48291c8088d1431850cb Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 26 Nov 2023 23:53:05 +0800 Subject: [PATCH 153/195] fix(tsdb): add simd for decompress timestamp. --- cmake/cmake.define | 4 +- include/util/tcompression.h | 1 + source/util/src/tdecompress.c | 148 ++++++++++++++++++++++------------ 3 files changed, 100 insertions(+), 53 deletions(-) diff --git a/cmake/cmake.define b/cmake/cmake.define index 7710c071eb..3eb872cfee 100644 --- a/cmake/cmake.define +++ b/cmake/cmake.define @@ -181,8 +181,8 @@ ELSE () MESSAGE(STATUS "SIMD instructions (FMA/AVX/AVX2) is ACTIVATED") IF (COMPILER_SUPPORT_AVX512F AND COMPILER_SUPPORT_AVX512BMI) - SET(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -mavx512f -mavx512vbmi") - SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -mavx512f -mavx512vbmi") + SET(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -mavx512f -mavx512vbmi -mavx512vl") + SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -mavx512f -mavx512vbmi -mavx512vl") MESSAGE(STATUS "avx512 supported by gcc") ENDIF() ENDIF() diff --git a/include/util/tcompression.h b/include/util/tcompression.h index ab0c22fc9b..7da3587f1c 100644 --- a/include/util/tcompression.h +++ b/include/util/tcompression.h @@ -139,6 +139,7 @@ int32_t getWordLength(char type); int32_t tsDecompressIntImpl_Hw(const char *const input, const int32_t nelements, char *const output, const char type); int32_t tsDecompressFloatImplAvx512(const char *const input, const int32_t nelements, char *const output); int32_t tsDecompressFloatImplAvx2(const char *const input, const int32_t nelements, char *const output); +int32_t tsDecompressTimestampAvx512(const char* const input, const int32_t nelements, char *const output, bool bigEndian); /************************************************************************* * STREAM COMPRESSION diff --git a/source/util/src/tdecompress.c b/source/util/src/tdecompress.c index ac9d70b2e7..b1c6df95c4 100644 --- a/source/util/src/tdecompress.c +++ b/source/util/src/tdecompress.c @@ -247,75 +247,121 @@ int32_t tsDecompressFloatImplAvx2(const char *const input, const int32_t nelemen return 0; } -int32_t tsDecompressTimestampAvx2(const char* const input, const int32_t nelements, char *const output, bool bigEndian) { +int32_t tsDecompressTimestampAvx512(const char* const input, const int32_t nelements, char *const output, bool bigEndian) { int64_t *ostream = (int64_t *)output; int32_t ipos = 1, opos = 0; - int8_t nbytes = 0; - - int64_t prevValue = 0; - int64_t prevDelta = 0; - - int64_t deltaOfDelta = 0; - int32_t longBytes = LONG_BYTES; + __m128i prevVal = _mm_setzero_si128(); + __m128i prevDelta = _mm_setzero_si128(); // _mm_maskz_loadu_epi8 #if __AVX512F__ - // _mm_blendv_epi8 - int32_t batch = nelements >> 4; - int32_t remainder = nelements & 0x03; + int32_t batch = nelements >> 1; + int32_t remainder = nelements & 0x01; - for(int32_t i = 0; i < batch; ++i) { + int32_t i = 0; + if (batch > 1) { + // first loop uint8_t flags = input[ipos++]; - // Decode dd1 - uint64_t dd1 = 0; - nbytes = flags & INT8MASK(4); // range of nbytes starts from 0 to 7 - // __m128i mask = {};//[0], [] + int8_t nbytes1 = flags & INT8MASK(4); // range of nbytes starts from 0 to 7 + int8_t nbytes2 = (flags >> 4) & INT8MASK(4); - if (nbytes == 0) { - deltaOfDelta = 0; - } else { - if (bigEndian) { - memcpy(((char *)(&dd1)) + longBytes - nbytes, input + ipos, nbytes); - } else { - memcpy(&dd1, input + ipos, nbytes); - } - deltaOfDelta = ZIGZAG_DECODE(int64_t, dd1); - } + __mmask16 mask2[16] = {0, 0x0001, 0x0003, 0x0007, 0x000f, 0x001f, 0x003f, 0x007f, 0x00ff}; + __m128i data1 = _mm_maskz_loadu_epi8(mask2[nbytes1], (const void*)(input + ipos)); + __m128i data2 = _mm_maskz_loadu_epi8(mask2[nbytes2], (const void*)(input + ipos + nbytes1)); + data2 = _mm_broadcastq_epi64(data2); - ipos += nbytes; - prevDelta += deltaOfDelta; - prevValue += prevDelta; - ostream[opos++] = prevValue; + __m128i zzVal = _mm_blend_epi32(data2, data1, 0x03); - // Decode dd2 - uint64_t dd2 = 0; - nbytes = (flags >> 4) & INT8MASK(4); - if (nbytes == 0) { - deltaOfDelta = 0; - } else { - if (bigEndian) { - memcpy(((char *)(&dd2)) + longBytes - nbytes, input + ipos, nbytes); - } else { - memcpy(&dd2, input + ipos, nbytes); - } - // zigzag_decoding - deltaOfDelta = ZIGZAG_DECODE(int64_t, dd2); - } + // ZIGZAG_DECODE(T, v) (((v) >> 1) ^ -((T)((v)&1))) + __m128i signmask = _mm_and_si128(_mm_set1_epi64x(1), zzVal); + signmask = _mm_sub_epi64(_mm_setzero_si128(), signmask); - ipos += nbytes; - prevDelta += deltaOfDelta; - prevValue += prevDelta; - ostream[opos++] = prevValue; + // get two zigzag values here + __m128i deltaOfDelta = _mm_xor_si128(_mm_srli_epi64(zzVal, 1), signmask); - if (opos == nelements) { - return nelements * longBytes; - } + __m128i deltaCurrent = _mm_add_epi64(deltaOfDelta, prevDelta); + deltaCurrent = _mm_add_epi64(_mm_slli_si128(deltaCurrent, 8), deltaCurrent); + + __m128i val = _mm_add_epi64(deltaCurrent, prevVal); + _mm_storeu_si128((__m128i *)&ostream[opos], val); + + // keep the previous value + prevVal = _mm_set1_epi64x(val[1]); + + // keep the previous delta of delta, for the first item + prevDelta = _mm_set1_epi64x(deltaOfDelta[1]); + + opos += 2; + ipos += nbytes1 + nbytes2; + i += 1; + } + + // the remain + for(; i < batch; ++i) { + uint8_t flags = input[ipos++]; + + int8_t nbytes1 = flags & INT8MASK(4); // range of nbytes starts from 0 to 7 + int8_t nbytes2 = (flags >> 4) & INT8MASK(4); + + __mmask16 mask2[16] = {0, 0x0001, 0x0003, 0x0007, 0x000f, 0x001f, 0x003f, 0x007f, 0x00ff}; + __m128i data1 = _mm_maskz_loadu_epi8(mask2[nbytes1], (const void*)(input + ipos)); + __m128i data2 = _mm_maskz_loadu_epi8(mask2[nbytes2], (const void*)(input + ipos + nbytes1)); + data2 = _mm_broadcastq_epi64(data2); + + __m128i zzVal = _mm_blend_epi32(data2, data1, 0x03); + + // ZIGZAG_DECODE(T, v) (((v) >> 1) ^ -((T)((v)&1))) + __m128i signmask = _mm_and_si128(_mm_set1_epi64x(1), zzVal); + signmask = _mm_sub_epi64(_mm_setzero_si128(), signmask); + + // get two zigzag values here + __m128i deltaOfDelta = _mm_xor_si128(_mm_srli_epi64(zzVal, 1), signmask); + + __m128i deltaCurrent = _mm_add_epi64(deltaOfDelta, prevDelta); + deltaCurrent = _mm_add_epi64(_mm_slli_si128(deltaCurrent, 8), deltaCurrent); + + __m128i val = _mm_add_epi64(deltaCurrent, prevVal); + _mm_storeu_si128((__m128i *)&ostream[opos], val); + + // keep the previous value + prevVal = _mm_set1_epi64x(val[1]); + + // keep the previous delta of delta + __m128i delta = _mm_add_epi64(_mm_slli_si128(deltaOfDelta, 8), deltaOfDelta); + prevDelta = _mm_set1_epi64x(_mm_add_epi64(delta, prevDelta)[1]); + + opos += 2; + ipos += nbytes1 + nbytes2; } if (remainder > 0) { + uint64_t dd = 0; + uint8_t flags = input[ipos++]; + int32_t nbytes = flags & INT8MASK(4); + int64_t deltaOfDelta = 0; + if (nbytes == 0) { + deltaOfDelta = 0; + } else { + // if (is_bigendian()) { + // memcpy(((char *)(&dd1)) + longBytes - nbytes, input + ipos, nbytes); + // } else { + memcpy(&dd, input + ipos, nbytes); + // } + deltaOfDelta = ZIGZAG_DECODE(int64_t, dd); + } + + ipos += nbytes; + if (opos == 0) { + ostream[opos++] = deltaOfDelta; + } else { + int64_t prevV = prevVal[1]; + + int64_t prevDeltaX = deltaOfDelta + prevDelta[1]; + ostream[opos++] = prevV + prevDeltaX; + } } #endif From de0c20c296ae78e4e16ee2a869e1524614e9c0a2 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 27 Nov 2023 00:02:52 +0800 Subject: [PATCH 154/195] fix(test): add test case. --- source/util/test/decompressTest.cpp | 94 +++++++++++++++++++++++++++++ 1 file changed, 94 insertions(+) create mode 100644 source/util/test/decompressTest.cpp diff --git a/source/util/test/decompressTest.cpp b/source/util/test/decompressTest.cpp new file mode 100644 index 0000000000..0c4f660002 --- /dev/null +++ b/source/util/test/decompressTest.cpp @@ -0,0 +1,94 @@ +#include +#include +#include +#include + +namespace {} // namespace + +TEST(utilTest, decompress_test) { + int64_t tsList[10] = {1700000000, 1700000100, 1700000200, 1700000300, 1700000400, + 1700000500, 1700000600, 1700000700, 1700000800, 1700000900}; + + char* pOutput[10 * sizeof(int64_t)] = {0}; + int32_t len = tsCompressTimestamp(tsList, sizeof(tsList), sizeof(tsList) / sizeof(tsList[0]), pOutput, 10, ONE_STAGE_COMP, NULL, 0); + + char* decompOutput[10 * 8] = {0}; + tsDecompressTimestamp(pOutput, len, 10, decompOutput, sizeof(int64_t)*10, ONE_STAGE_COMP, NULL, 0); + + for(int32_t i = 0; i < 10; ++i) { + std::cout<< ((int64_t*)decompOutput)[i] << std::endl; + } + + memset(decompOutput, 0, 10*8); + tsDecompressTimestampAvx512(reinterpret_cast(pOutput), 10, + reinterpret_cast(decompOutput), false); + + for(int32_t i = 0; i < 10; ++i) { + std::cout<<((int64_t*)decompOutput)[i] << std::endl; + } + + //////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + int64_t tsList1[7] = {1700000000, 1700000000, 1700000000, 1700000000, 1700000000, 1700000000, 1700000900}; + int32_t len1 = tsCompressTimestamp(tsList1, sizeof(tsList1), sizeof(tsList1) / sizeof(tsList1[0]), pOutput, 7, ONE_STAGE_COMP, NULL, 0); + + memset(decompOutput, 0, 10*8); + tsDecompressTimestampAvx512(reinterpret_cast(pOutput), 7, + reinterpret_cast(decompOutput), false); + + for(int32_t i = 0; i < 7; ++i) { + std::cout<<((int64_t*)decompOutput)[i] << std::endl; + } + + //////////////////////////////////////////////////////////////////////////////////////////////////////////////////// + int64_t tsList2[1] = {1700000000}; + int32_t len2 = tsCompressTimestamp(tsList2, sizeof(tsList2), sizeof(tsList2) / sizeof(tsList2[0]), pOutput, 1, ONE_STAGE_COMP, NULL, 0); + + memset(decompOutput, 0, 10*8); + tsDecompressTimestampAvx512(reinterpret_cast(pOutput), 1, + reinterpret_cast(decompOutput), false); + + for(int32_t i = 0; i < 1; ++i) { + std::cout<<((int64_t*)decompOutput)[i] << std::endl; + } +} + +TEST(utilTest, decompress_perf_test) { + int32_t num = 100000; + + int64_t* pList = static_cast(taosMemoryCalloc(num, sizeof(int64_t))); + int64_t iniVal = 1700000000; + + uint32_t v = 100; + + for(int32_t i = 0; i < num; ++i) { + iniVal += taosRandR(&v)%10; + pList[i] = iniVal; + } + + char* px = static_cast(taosMemoryMalloc(num * sizeof(int64_t))); + int32_t len = tsCompressTimestamp(pList, num * sizeof(int64_t), num, px, num, ONE_STAGE_COMP, NULL, 0); + + char* pOutput = static_cast(taosMemoryMalloc(num * sizeof(int64_t))); + + int64_t st = taosGetTimestampUs(); + for(int32_t k = 0; k < 10; ++k) { + tsDecompressTimestamp(px, len, num, pOutput, sizeof(int64_t) * num, ONE_STAGE_COMP, NULL, 0); + } + + int64_t el1 = taosGetTimestampUs() - st; + std::cout << "decompress elapsed time:" << el1 << " us" << std::endl; + + memset(pOutput, 0, num * sizeof(int64_t)); + st = taosGetTimestampUs(); + for(int32_t k = 0; k < 10; ++k) { + tsDecompressTimestampAvx512(px, num, pOutput, false); + } + + int64_t el2 = taosGetTimestampUs() - st; + std::cout << "SIMD decompress elapsed time:" << el2 << " us" << std::endl; + + taosMemoryFree(pList); + taosMemoryFree(pOutput); + taosMemoryFree(px); +} + From 6ab14d39bbf63619e3c92eba8366acb5c20713c0 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 27 Nov 2023 09:32:47 +0800 Subject: [PATCH 155/195] fix(test): do some internal refactor. --- cmake/cmake.define | 19 +++++++++++++------ source/util/src/tdecompress.c | 11 ++++------- source/util/test/decompressTest.cpp | 6 +++--- 3 files changed, 20 insertions(+), 16 deletions(-) diff --git a/cmake/cmake.define b/cmake/cmake.define index 3eb872cfee..7cc33b8252 100644 --- a/cmake/cmake.define +++ b/cmake/cmake.define @@ -1,5 +1,5 @@ cmake_minimum_required(VERSION 3.0) -set(CMAKE_VERBOSE_MAKEFILE FALSE) +set(CMAKE_VERBOSE_MAKEFILE TRUE) set(TD_BUILD_TAOSA_INTERNAL FALSE) #set output directory @@ -159,6 +159,7 @@ ELSE () CHECK_C_COMPILER_FLAG("-mavx2" COMPILER_SUPPORT_AVX2) CHECK_C_COMPILER_FLAG("-mavx512f" COMPILER_SUPPORT_AVX512F) CHECK_C_COMPILER_FLAG("-mavx512vbmi" COMPILER_SUPPORT_AVX512BMI) + CHECK_C_COMPILER_FLAG("-mavx512vl" COMPILER_SUPPORT_AVX512VL) IF (COMPILER_SUPPORT_SSE42) SET(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -msse4.2") @@ -166,11 +167,11 @@ ELSE () ENDIF() IF ("${SIMD_SUPPORT}" MATCHES "true") - IF (COMPILER_SUPPORT_FMA) + IF (COMPILER_SUPPORT_FMA) SET(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -mfma") SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -mfma") - ENDIF() - IF (COMPILER_SUPPORT_AVX) + ENDIF() + IF (COMPILER_SUPPORT_AVX) SET(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -mavx") SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -mavx") ENDIF() @@ -182,8 +183,14 @@ ELSE () IF (COMPILER_SUPPORT_AVX512F AND COMPILER_SUPPORT_AVX512BMI) SET(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -mavx512f -mavx512vbmi -mavx512vl") - SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -mavx512f -mavx512vbmi -mavx512vl") - MESSAGE(STATUS "avx512 supported by gcc") + SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -mavx512f -mavx512vbmi -mavx512vl") + MESSAGE(STATUS "avx512f/avx512bmi supported by gcc") + ENDIF() + + IF (COMPILER_SUPPORT_AVX512VL) + SET(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -mavx512vl") + SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -mavx512vl") + MESSAGE(STATUS "avx512vl supported by gcc") ENDIF() ENDIF() diff --git a/source/util/src/tdecompress.c b/source/util/src/tdecompress.c index b1c6df95c4..b39a340ac6 100644 --- a/source/util/src/tdecompress.c +++ b/source/util/src/tdecompress.c @@ -111,7 +111,7 @@ int32_t tsDecompressIntImpl_Hw(const char *const input, const int32_t nelements, __m256i signmask = _mm256_and_si256(_mm256_set1_epi64x(1), zigzagVal); signmask = _mm256_sub_epi64(_mm256_setzero_si256(), signmask); - // get the four zigzag values here + // get four zigzag values here __m256i delta = _mm256_xor_si256(_mm256_srli_epi64(zigzagVal, 1), signmask); // calculate the cumulative sum (prefix sum) for each number @@ -254,10 +254,11 @@ int32_t tsDecompressTimestampAvx512(const char* const input, const int32_t nelem __m128i prevDelta = _mm_setzero_si128(); // _mm_maskz_loadu_epi8 -#if __AVX512F__ +#if __AVX512VL__ int32_t batch = nelements >> 1; int32_t remainder = nelements & 0x01; + __mmask16 mask2[16] = {0, 0x0001, 0x0003, 0x0007, 0x000f, 0x001f, 0x003f, 0x007f, 0x00ff}; int32_t i = 0; if (batch > 1) { @@ -267,7 +268,6 @@ int32_t tsDecompressTimestampAvx512(const char* const input, const int32_t nelem int8_t nbytes1 = flags & INT8MASK(4); // range of nbytes starts from 0 to 7 int8_t nbytes2 = (flags >> 4) & INT8MASK(4); - __mmask16 mask2[16] = {0, 0x0001, 0x0003, 0x0007, 0x000f, 0x001f, 0x003f, 0x007f, 0x00ff}; __m128i data1 = _mm_maskz_loadu_epi8(mask2[nbytes1], (const void*)(input + ipos)); __m128i data2 = _mm_maskz_loadu_epi8(mask2[nbytes2], (const void*)(input + ipos + nbytes1)); data2 = _mm_broadcastq_epi64(data2); @@ -305,7 +305,6 @@ int32_t tsDecompressTimestampAvx512(const char* const input, const int32_t nelem int8_t nbytes1 = flags & INT8MASK(4); // range of nbytes starts from 0 to 7 int8_t nbytes2 = (flags >> 4) & INT8MASK(4); - __mmask16 mask2[16] = {0, 0x0001, 0x0003, 0x0007, 0x000f, 0x001f, 0x003f, 0x007f, 0x00ff}; __m128i data1 = _mm_maskz_loadu_epi8(mask2[nbytes1], (const void*)(input + ipos)); __m128i data2 = _mm_maskz_loadu_epi8(mask2[nbytes2], (const void*)(input + ipos + nbytes1)); data2 = _mm_broadcastq_epi64(data2); @@ -357,10 +356,8 @@ int32_t tsDecompressTimestampAvx512(const char* const input, const int32_t nelem if (opos == 0) { ostream[opos++] = deltaOfDelta; } else { - int64_t prevV = prevVal[1]; - int64_t prevDeltaX = deltaOfDelta + prevDelta[1]; - ostream[opos++] = prevV + prevDeltaX; + ostream[opos++] = prevVal[1] + prevDeltaX; } } diff --git a/source/util/test/decompressTest.cpp b/source/util/test/decompressTest.cpp index 0c4f660002..378b67edbb 100644 --- a/source/util/test/decompressTest.cpp +++ b/source/util/test/decompressTest.cpp @@ -53,7 +53,7 @@ TEST(utilTest, decompress_test) { } TEST(utilTest, decompress_perf_test) { - int32_t num = 100000; + int32_t num = 10000; int64_t* pList = static_cast(taosMemoryCalloc(num, sizeof(int64_t))); int64_t iniVal = 1700000000; @@ -71,7 +71,7 @@ TEST(utilTest, decompress_perf_test) { char* pOutput = static_cast(taosMemoryMalloc(num * sizeof(int64_t))); int64_t st = taosGetTimestampUs(); - for(int32_t k = 0; k < 10; ++k) { + for(int32_t k = 0; k < 10000; ++k) { tsDecompressTimestamp(px, len, num, pOutput, sizeof(int64_t) * num, ONE_STAGE_COMP, NULL, 0); } @@ -80,7 +80,7 @@ TEST(utilTest, decompress_perf_test) { memset(pOutput, 0, num * sizeof(int64_t)); st = taosGetTimestampUs(); - for(int32_t k = 0; k < 10; ++k) { + for(int32_t k = 0; k < 10000; ++k) { tsDecompressTimestampAvx512(px, num, pOutput, false); } From 5de59a354787b2f941bdd626238ecbde17592cc8 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 27 Nov 2023 10:32:47 +0800 Subject: [PATCH 156/195] enh(sys): add check for avx512vl --- cmake/cmake.define | 4 +- source/util/src/tcompression.c | 101 +++++++++++++++++---------------- 2 files changed, 53 insertions(+), 52 deletions(-) diff --git a/cmake/cmake.define b/cmake/cmake.define index 7cc33b8252..c685ba6161 100644 --- a/cmake/cmake.define +++ b/cmake/cmake.define @@ -184,13 +184,13 @@ ELSE () IF (COMPILER_SUPPORT_AVX512F AND COMPILER_SUPPORT_AVX512BMI) SET(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -mavx512f -mavx512vbmi -mavx512vl") SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -mavx512f -mavx512vbmi -mavx512vl") - MESSAGE(STATUS "avx512f/avx512bmi supported by gcc") + MESSAGE(STATUS "avx512f/avx512bmi supported by compiler") ENDIF() IF (COMPILER_SUPPORT_AVX512VL) SET(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -mavx512vl") SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -mavx512vl") - MESSAGE(STATUS "avx512vl supported by gcc") + MESSAGE(STATUS "avx512vl supported by compiler") ENDIF() ENDIF() diff --git a/source/util/src/tcompression.c b/source/util/src/tcompression.c index 09f50d0f6d..8a79d62cda 100644 --- a/source/util/src/tcompression.c +++ b/source/util/src/tcompression.c @@ -538,65 +538,66 @@ int32_t tsDecompressTimestampImp(const char *const input, const int32_t nelement memcpy(output, input + 1, nelements * longBytes); return nelements * longBytes; } else if (input[0] == 1) { // Decompress - int64_t *ostream = (int64_t *)output; + if (tsSIMDEnable && tsAVX512Enable) { + tsDecompressTimestampAvx512(input, nelements, output, false); + } else { + int64_t *ostream = (int64_t *)output; - int32_t ipos = 1, opos = 0; - int8_t nbytes = 0; - int64_t prev_value = 0; - int64_t prev_delta = 0; - int64_t delta_of_delta = 0; + int32_t ipos = 1, opos = 0; + int8_t nbytes = 0; + int64_t prev_value = 0; + int64_t prev_delta = 0; + int64_t delta_of_delta = 0; - while (1) { - uint8_t flags = input[ipos++]; - // Decode dd1 - uint64_t dd1 = 0; - nbytes = flags & INT8MASK(4); - if (nbytes == 0) { - delta_of_delta = 0; - } else { - if (is_bigendian()) { - memcpy(((char *)(&dd1)) + longBytes - nbytes, input + ipos, nbytes); + while (1) { + uint8_t flags = input[ipos++]; + // Decode dd1 + uint64_t dd1 = 0; + nbytes = flags & INT8MASK(4); + if (nbytes == 0) { + delta_of_delta = 0; } else { - memcpy(&dd1, input + ipos, nbytes); + if (is_bigendian()) { + memcpy(((char *)(&dd1)) + longBytes - nbytes, input + ipos, nbytes); + } else { + memcpy(&dd1, input + ipos, nbytes); + } + delta_of_delta = ZIGZAG_DECODE(int64_t, dd1); } - delta_of_delta = ZIGZAG_DECODE(int64_t, dd1); - } - ipos += nbytes; - if (opos == 0) { - prev_value = delta_of_delta; - prev_delta = 0; - ostream[opos++] = delta_of_delta; - } else { + + ipos += nbytes; + if (opos == 0) { + prev_value = delta_of_delta; + prev_delta = 0; + ostream[opos++] = delta_of_delta; + } else { + prev_delta = delta_of_delta + prev_delta; + prev_value = prev_value + prev_delta; + ostream[opos++] = prev_value; + } + if (opos == nelements) return nelements * longBytes; + + // Decode dd2 + uint64_t dd2 = 0; + nbytes = (flags >> 4) & INT8MASK(4); + if (nbytes == 0) { + delta_of_delta = 0; + } else { + if (is_bigendian()) { + memcpy(((char *)(&dd2)) + longBytes - nbytes, input + ipos, nbytes); + } else { + memcpy(&dd2, input + ipos, nbytes); + } + // zigzag_decoding + delta_of_delta = ZIGZAG_DECODE(int64_t, dd2); + } + ipos += nbytes; prev_delta = delta_of_delta + prev_delta; prev_value = prev_value + prev_delta; ostream[opos++] = prev_value; + if (opos == nelements) return nelements * longBytes; } - if (opos == nelements) return nelements * longBytes; - - // Decode dd2 - uint64_t dd2 = 0; - nbytes = (flags >> 4) & INT8MASK(4); - if (nbytes == 0) { - delta_of_delta = 0; - } else { - if (is_bigendian()) { - memcpy(((char *)(&dd2)) + longBytes - nbytes, input + ipos, nbytes); - } else { - memcpy(&dd2, input + ipos, nbytes); - } - // zigzag_decoding - delta_of_delta = ZIGZAG_DECODE(int64_t, dd2); - } - ipos += nbytes; - prev_delta = delta_of_delta + prev_delta; - prev_value = prev_value + prev_delta; - ostream[opos++] = prev_value; - if (opos == nelements) return nelements * longBytes; } - - } else { - ASSERT(0); - return -1; } } From e483570cd7a40e6a15c6e0b94dcb02662869b708 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Mon, 27 Nov 2023 11:10:56 +0800 Subject: [PATCH 157/195] Merge branch '3.0' into enh/refactorBackend --- source/dnode/mnode/impl/src/mndStream.c | 3 ++- source/dnode/mnode/impl/src/mndStreamTrans.c | 23 ++++++++++---------- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 95ea6070ae..b2c3a07681 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1340,7 +1340,8 @@ static int32_t mndProcessStreamCheckpointInCandid(SRpcMsg *pReq) { for (int32_t i = 0; i < taosArrayGetSize(pList); ++i) { int64_t *pId = taosArrayGet(pList, i); - taosHashRemove(execInfo.transMgmt.pWaitingList, &pId, sizeof(*pId)); + + taosHashRemove(execInfo.transMgmt.pWaitingList, pId, sizeof(*pId)); } int32_t remain = taosHashGetSize(execInfo.transMgmt.pWaitingList); diff --git a/source/dnode/mnode/impl/src/mndStreamTrans.c b/source/dnode/mnode/impl/src/mndStreamTrans.c index 3e2afe4ade..c1b1df85c0 100644 --- a/source/dnode/mnode/impl/src/mndStreamTrans.c +++ b/source/dnode/mnode/impl/src/mndStreamTrans.c @@ -13,8 +13,8 @@ * along with this program. If not, see . */ -#include "mndTrans.h" #include "mndStream.h" +#include "mndTrans.h" typedef struct SKeyInfo { void* pKey; @@ -42,8 +42,8 @@ int32_t clearFinishedTrans(SMnode* pMnode) { void* pIter = NULL; while ((pIter = taosHashIterate(execInfo.transMgmt.pDBTrans, pIter)) != NULL) { - SStreamTransInfo *pEntry = (SStreamTransInfo *)pIter; - STrans* pTrans = mndAcquireTrans(pMnode, pEntry->transId); + SStreamTransInfo* pEntry = (SStreamTransInfo*)pIter; + STrans* pTrans = mndAcquireTrans(pMnode, pEntry->transId); // let's clear the finished trans if (pTrans == NULL) { @@ -60,12 +60,12 @@ int32_t clearFinishedTrans(SMnode* pMnode) { } size_t num = taosArrayGetSize(pList); - for(int32_t i = 0; i < num; ++i) { + for (int32_t i = 0; i < num; ++i) { SKeyInfo* pKey = taosArrayGet(pList, i); taosHashRemove(execInfo.transMgmt.pDBTrans, pKey->pKey, pKey->keyLen); } - mDebug("clear %d finished stream-trans, remained:%d", (int32_t) num, taosHashGetSize(execInfo.transMgmt.pDBTrans)); + mDebug("clear %d finished stream-trans, remained:%d", (int32_t)num, taosHashGetSize(execInfo.transMgmt.pDBTrans)); taosThreadMutexUnlock(&execInfo.lock); terrno = TSDB_CODE_SUCCESS; @@ -83,7 +83,7 @@ bool mndStreamTransConflictOtherTrans(SMnode* pMnode, const char* pSrcDb, const return false; } - SStreamTransInfo *pEntry = taosHashGet(execInfo.transMgmt.pDBTrans, pSrcDb, strlen(pSrcDb)); + SStreamTransInfo* pEntry = taosHashGet(execInfo.transMgmt.pDBTrans, pSrcDb, strlen(pSrcDb)); if (pEntry != NULL) { taosThreadMutexUnlock(&execInfo.lock); mWarn("conflict with other transId:%d in Db:%s, trans:%s", pEntry->transId, pSrcDb, pEntry->name); @@ -112,14 +112,13 @@ int32_t mndAddtoCheckpointWaitingList(SStreamObj* pStream, int64_t checkpointId) taosHashPut(execInfo.transMgmt.pWaitingList, &pStream->uid, sizeof(pStream->uid), &entry, sizeof(entry)); int32_t size = taosHashGetSize(execInfo.transMgmt.pWaitingList); - mDebug("stream:%" PRIx64 " add into waiting list due to conflict, ts:%" PRId64 ", total in waitingList:%d", - pStream->uid, entry.checkpointTs, size); + mDebug("stream:%" PRIx64 " add into waiting list due to conflict, ts:%" PRId64 " , checkpointId: %" PRId64 + ", total in waitingList:%d", + pStream->uid, entry.checkpointTs, checkpointId, size); } else { - mDebug("stream:%" PRIx64 " ts:%" PRId64 "already in waiting list, no need to add into", pStream->uid, - pEntry->checkpointTs); + mDebug("stream:%" PRIx64 " ts:%" PRId64 ", checkpointId:%" PRId64 " already in waiting list, no need to add into", + pStream->uid, pEntry->checkpointTs, checkpointId); } return TSDB_CODE_SUCCESS; } - - From 703f323b3f289f16f00dfb4b51e85932ec62d650 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Mon, 27 Nov 2023 13:52:28 +0800 Subject: [PATCH 158/195] Merge branch '3.0' into enh/refactorBackend --- source/common/src/tglobal.c | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index a9a8217c31..c72a998f88 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -248,7 +248,7 @@ int32_t tsTtlBatchDropNum = 10000; // number of tables dropped per batch // internal int32_t tsTransPullupInterval = 2; int32_t tsMqRebalanceInterval = 2; -int32_t tsStreamCheckpointInterval = 10; +int32_t tsStreamCheckpointInterval = 40; float tsSinkDataRate = 2.0; int32_t tsStreamNodeCheckInterval = 15; int32_t tsTtlUnit = 86400; @@ -354,16 +354,24 @@ static int32_t taosLoadCfg(SConfig *pCfg, const char **envCmd, const char *input char cfgFile[PATH_MAX + 100] = {0}; taosExpandDir(inputCfgDir, cfgDir, PATH_MAX); - char lastC = cfgDir[strlen(cfgDir) - 1]; + char lastC = cfgDir[strlen(cfgDir) - 1]; char *tdDirsep = TD_DIRSEP; if (lastC == '\\' || lastC == '/') { tdDirsep = ""; } if (taosIsDir(cfgDir)) { #ifdef CUS_PROMPT - snprintf(cfgFile, sizeof(cfgFile), "%s" "%s" "%s.cfg", cfgDir, tdDirsep, CUS_PROMPT); + snprintf(cfgFile, sizeof(cfgFile), + "%s" + "%s" + "%s.cfg", + cfgDir, tdDirsep, CUS_PROMPT); #else - snprintf(cfgFile, sizeof(cfgFile), "%s" "%s" "taos.cfg", cfgDir, tdDirsep); + snprintf(cfgFile, sizeof(cfgFile), + "%s" + "%s" + "taos.cfg", + cfgDir, tdDirsep); #endif } else { tstrncpy(cfgFile, cfgDir, sizeof(cfgDir)); From 2ddb246461255ada404d0f054a73385a3f68ddb9 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Mon, 27 Nov 2023 14:10:00 +0800 Subject: [PATCH 159/195] change debug info --- source/libs/transport/src/transCli.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/libs/transport/src/transCli.c b/source/libs/transport/src/transCli.c index ef60c8a94e..772996548d 100644 --- a/source/libs/transport/src/transCli.c +++ b/source/libs/transport/src/transCli.c @@ -957,8 +957,8 @@ static void cliSendCb(uv_write_t* req, int status) { SCliMsg* pMsg = !transQueueEmpty(&pConn->cliMsgs) ? transQueueGet(&pConn->cliMsgs, 0) : NULL; if (pMsg != NULL) { int64_t cost = taosGetTimestampUs() - pMsg->st; - if (cost > 1000 * 20) { - tWarn("%s conn %p send cost:%dus, send exception", CONN_GET_INST_LABEL(pConn), pConn, (int)cost); + if (cost > 1000 * 50) { + tWarn("%s conn %p send cost:%dus ", CONN_GET_INST_LABEL(pConn), pConn, (int)cost); } } From d4c3bcc9f746baf4c7a0a31c0753715551c404b9 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Mon, 27 Nov 2023 14:10:38 +0800 Subject: [PATCH 160/195] change debug info --- source/libs/transport/src/transCli.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/transport/src/transCli.c b/source/libs/transport/src/transCli.c index 772996548d..e51c61c49d 100644 --- a/source/libs/transport/src/transCli.c +++ b/source/libs/transport/src/transCli.c @@ -958,7 +958,7 @@ static void cliSendCb(uv_write_t* req, int status) { if (pMsg != NULL) { int64_t cost = taosGetTimestampUs() - pMsg->st; if (cost > 1000 * 50) { - tWarn("%s conn %p send cost:%dus ", CONN_GET_INST_LABEL(pConn), pConn, (int)cost); + tTrace("%s conn %p send cost:%dus ", CONN_GET_INST_LABEL(pConn), pConn, (int)cost); } } From f9d717d0f2fdc7b0a9917a29831d20e6f021eabd Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 27 Nov 2023 14:42:13 +0800 Subject: [PATCH 161/195] refactor: do some internal refactor. --- cmake/cmake.define | 4 +- include/util/tcompression.h | 1 + source/util/src/tcompression.c | 4 + source/util/src/tdecompress.c | 165 ++++++++++++++++++++++++++-- source/util/test/decompressTest.cpp | 2 +- 5 files changed, 164 insertions(+), 12 deletions(-) diff --git a/cmake/cmake.define b/cmake/cmake.define index c685ba6161..7db6baafab 100644 --- a/cmake/cmake.define +++ b/cmake/cmake.define @@ -182,8 +182,8 @@ ELSE () MESSAGE(STATUS "SIMD instructions (FMA/AVX/AVX2) is ACTIVATED") IF (COMPILER_SUPPORT_AVX512F AND COMPILER_SUPPORT_AVX512BMI) - SET(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -mavx512f -mavx512vbmi -mavx512vl") - SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -mavx512f -mavx512vbmi -mavx512vl") + SET(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -mavx512f -mavx512vbmi") + SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -mavx512f -mavx512vbmi") MESSAGE(STATUS "avx512f/avx512bmi supported by compiler") ENDIF() diff --git a/include/util/tcompression.h b/include/util/tcompression.h index 7da3587f1c..75ddbb12e7 100644 --- a/include/util/tcompression.h +++ b/include/util/tcompression.h @@ -140,6 +140,7 @@ int32_t tsDecompressIntImpl_Hw(const char *const input, const int32_t nelements, int32_t tsDecompressFloatImplAvx512(const char *const input, const int32_t nelements, char *const output); int32_t tsDecompressFloatImplAvx2(const char *const input, const int32_t nelements, char *const output); int32_t tsDecompressTimestampAvx512(const char* const input, const int32_t nelements, char *const output, bool bigEndian); +int32_t tsDecompressTimestampAvx2(const char* const input, const int32_t nelements, char *const output, bool bigEndian); /************************************************************************* * STREAM COMPRESSION diff --git a/source/util/src/tcompression.c b/source/util/src/tcompression.c index 8a79d62cda..3cc00ddc7f 100644 --- a/source/util/src/tcompression.c +++ b/source/util/src/tcompression.c @@ -540,6 +540,8 @@ int32_t tsDecompressTimestampImp(const char *const input, const int32_t nelement } else if (input[0] == 1) { // Decompress if (tsSIMDEnable && tsAVX512Enable) { tsDecompressTimestampAvx512(input, nelements, output, false); + } else if (tsSIMDEnable && tsAVX2Enable) { + tsDecompressTimestampAvx2(input, nelements, output, false); } else { int64_t *ostream = (int64_t *)output; @@ -599,6 +601,8 @@ int32_t tsDecompressTimestampImp(const char *const input, const int32_t nelement } } } + + return nelements * longBytes; } /* --------------------------------------------Double Compression ---------------------------------------------- */ diff --git a/source/util/src/tdecompress.c b/source/util/src/tdecompress.c index b39a340ac6..5a5e60093c 100644 --- a/source/util/src/tdecompress.c +++ b/source/util/src/tdecompress.c @@ -247,15 +247,14 @@ int32_t tsDecompressFloatImplAvx2(const char *const input, const int32_t nelemen return 0; } -int32_t tsDecompressTimestampAvx512(const char* const input, const int32_t nelements, char *const output, bool bigEndian) { +int32_t tsDecompressTimestampAvx2(const char *const input, const int32_t nelements, char *const output, + bool bigEndian) { int64_t *ostream = (int64_t *)output; int32_t ipos = 1, opos = 0; __m128i prevVal = _mm_setzero_si128(); __m128i prevDelta = _mm_setzero_si128(); - // _mm_maskz_loadu_epi8 -#if __AVX512VL__ - +#if __AVX2__ int32_t batch = nelements >> 1; int32_t remainder = nelements & 0x01; __mmask16 mask2[16] = {0, 0x0001, 0x0003, 0x0007, 0x000f, 0x001f, 0x003f, 0x007f, 0x00ff}; @@ -268,10 +267,22 @@ int32_t tsDecompressTimestampAvx512(const char* const input, const int32_t nelem int8_t nbytes1 = flags & INT8MASK(4); // range of nbytes starts from 0 to 7 int8_t nbytes2 = (flags >> 4) & INT8MASK(4); - __m128i data1 = _mm_maskz_loadu_epi8(mask2[nbytes1], (const void*)(input + ipos)); - __m128i data2 = _mm_maskz_loadu_epi8(mask2[nbytes2], (const void*)(input + ipos + nbytes1)); - data2 = _mm_broadcastq_epi64(data2); + __m128i data1; + if (nbytes1 == 0) { + data1 = _mm_setzero_si128(); + } else { +// _mm_shuffle_epi8() + memcpy(&data1, (const void*) (input + ipos), nbytes1); + } + __m128i data2; + if (nbytes2 == 0) { + data2 = _mm_setzero_si128(); + } else { + memcpy(&data2, (const void*) (input + ipos + nbytes1), nbytes2); + } + + data2 = _mm_broadcastq_epi64(data2); __m128i zzVal = _mm_blend_epi32(data2, data1, 0x03); // ZIGZAG_DECODE(T, v) (((v) >> 1) ^ -((T)((v)&1))) @@ -305,8 +316,26 @@ int32_t tsDecompressTimestampAvx512(const char* const input, const int32_t nelem int8_t nbytes1 = flags & INT8MASK(4); // range of nbytes starts from 0 to 7 int8_t nbytes2 = (flags >> 4) & INT8MASK(4); - __m128i data1 = _mm_maskz_loadu_epi8(mask2[nbytes1], (const void*)(input + ipos)); - __m128i data2 = _mm_maskz_loadu_epi8(mask2[nbytes2], (const void*)(input + ipos + nbytes1)); +// __m128i data1 = _mm_maskz_loadu_epi8(mask2[nbytes1], (const void*)(input + ipos)); +// __m128i data2 = _mm_maskz_loadu_epi8(mask2[nbytes2], (const void*)(input + ipos + nbytes1)); + __m128i data1; + if (nbytes1 == 0) { + data1 = _mm_setzero_si128(); + } else { + int64_t dd = 0; + memcpy(&dd, (const void*) (input + ipos), nbytes1); + data1 = _mm_loadu_si64(&dd); + } + + __m128i data2; + if (nbytes2 == 0) { + data2 = _mm_setzero_si128(); + } else { + int64_t dd = 0; + memcpy(&dd, (const void*) (input + ipos + nbytes1), nbytes2); + data2 = _mm_loadu_si64(&dd); + } + data2 = _mm_broadcastq_epi64(data2); __m128i zzVal = _mm_blend_epi32(data2, data1, 0x03); @@ -335,6 +364,124 @@ int32_t tsDecompressTimestampAvx512(const char* const input, const int32_t nelem ipos += nbytes1 + nbytes2; } + if (remainder > 0) { + uint64_t dd = 0; + uint8_t flags = input[ipos++]; + + int32_t nbytes = flags & INT8MASK(4); + int64_t deltaOfDelta = 0; + if (nbytes == 0) { + deltaOfDelta = 0; + } else { + // if (is_bigendian()) { + // memcpy(((char *)(&dd1)) + longBytes - nbytes, input + ipos, nbytes); + // } else { + memcpy(&dd, input + ipos, nbytes); + // } + deltaOfDelta = ZIGZAG_DECODE(int64_t, dd); + } + + ipos += nbytes; + if (opos == 0) { + ostream[opos++] = deltaOfDelta; + } else { + int64_t prevDeltaX = deltaOfDelta + prevDelta[1]; + ostream[opos++] = prevVal[1] + prevDeltaX; + } + } +#endif + + return 0; +} + +int32_t tsDecompressTimestampAvx512(const char* const input, const int32_t nelements, char *const output, bool bigEndian) { + int64_t *ostream = (int64_t *)output; + int32_t ipos = 1, opos = 0; + __m128i prevVal = _mm_setzero_si128(); + __m128i prevDelta = _mm_setzero_si128(); + + // _mm_maskz_loadu_epi8 +#if __AVX512VL__ + + int32_t batch = nelements >> 1; + int32_t remainder = nelements & 0x01; + __mmask16 mask2[16] = {0, 0x0001, 0x0003, 0x0007, 0x000f, 0x001f, 0x003f, 0x007f, 0x00ff}; + + int32_t i = 0; + if (batch > 1) { + // first loop + uint8_t flags = input[ipos++]; + + int8_t nbytes1 = flags & INT8MASK(4); // range of nbytes starts from 0 to 7 + int8_t nbytes2 = (flags >> 4) & INT8MASK(4); + + __m128i data1 = _mm_maskz_loadu_epi8(mask2[nbytes1], (const void*)(input + ipos)); + __m128i data2 = _mm_maskz_loadu_epi8(mask2[nbytes2], (const void*)(input + ipos + nbytes1)); + data2 = _mm_broadcastq_epi64(data2); + + __m128i zzVal = _mm_blend_epi32(data2, data1, 0x03); + + // ZIGZAG_DECODE(T, v) (((v) >> 1) ^ -((T)((v)&1))) + __m128i signmask = _mm_and_si128(_mm_set1_epi64x(1), zzVal); + signmask = _mm_sub_epi64(_mm_setzero_si128(), signmask); + + // get two zigzag values here + __m128i deltaOfDelta = _mm_xor_si128(_mm_srli_epi64(zzVal, 1), signmask); + + __m128i deltaCurrent = _mm_add_epi64(deltaOfDelta, prevDelta); + deltaCurrent = _mm_add_epi64(_mm_slli_si128(deltaCurrent, 8), deltaCurrent); + + __m128i val = _mm_add_epi64(deltaCurrent, prevVal); + _mm_storeu_si128((__m128i *)&ostream[opos], val); + + // keep the previous value + prevVal = _mm_shuffle_epi32 (val, 0xEE); + + // keep the previous delta of delta, for the first item + prevDelta = _mm_shuffle_epi32(deltaOfDelta, 0xEE); + + opos += 2; + ipos += nbytes1 + nbytes2; + i += 1; + } + + // the remain + for(; i < batch; ++i) { + uint8_t flags = input[ipos++]; + + int8_t nbytes1 = flags & INT8MASK(4); // range of nbytes starts from 0 to 7 + int8_t nbytes2 = (flags >> 4) & INT8MASK(4); + + __m128i data1 = _mm_maskz_loadu_epi8(mask2[nbytes1], (const void*)(input + ipos)); + __m128i data2 = _mm_maskz_loadu_epi8(mask2[nbytes2], (const void*)(input + ipos + nbytes1)); + data2 = _mm_broadcastq_epi64(data2); + + __m128i zzVal = _mm_blend_epi32(data2, data1, 0x03); + + // ZIGZAG_DECODE(T, v) (((v) >> 1) ^ -((T)((v)&1))) + __m128i signmask = _mm_and_si128(_mm_set1_epi64x(1), zzVal); + signmask = _mm_sub_epi64(_mm_setzero_si128(), signmask); + + // get two zigzag values here + __m128i deltaOfDelta = _mm_xor_si128(_mm_srli_epi64(zzVal, 1), signmask); + + __m128i deltaCurrent = _mm_add_epi64(deltaOfDelta, prevDelta); + deltaCurrent = _mm_add_epi64(_mm_slli_si128(deltaCurrent, 8), deltaCurrent); + + __m128i val = _mm_add_epi64(deltaCurrent, prevVal); + _mm_storeu_si128((__m128i *)&ostream[opos], val); + + // keep the previous value + prevVal = _mm_shuffle_epi32 (val, 0xEE); + + // keep the previous delta of delta + __m128i delta = _mm_add_epi64(_mm_slli_si128(deltaOfDelta, 8), deltaOfDelta); + prevDelta = _mm_shuffle_epi32(_mm_add_epi64(delta, prevDelta), 0xEE); + + opos += 2; + ipos += nbytes1 + nbytes2; + } + if (remainder > 0) { uint64_t dd = 0; uint8_t flags = input[ipos++]; diff --git a/source/util/test/decompressTest.cpp b/source/util/test/decompressTest.cpp index 378b67edbb..caf8df3ba8 100644 --- a/source/util/test/decompressTest.cpp +++ b/source/util/test/decompressTest.cpp @@ -76,7 +76,7 @@ TEST(utilTest, decompress_perf_test) { } int64_t el1 = taosGetTimestampUs() - st; - std::cout << "decompress elapsed time:" << el1 << " us" << std::endl; + std::cout << "soft decompress elapsed time:" << el1 << " us" << std::endl; memset(pOutput, 0, num * sizeof(int64_t)); st = taosGetTimestampUs(); From d1193b5a7a29d3df3811d77e6bdd5d5ec84f8721 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 27 Nov 2023 14:59:12 +0800 Subject: [PATCH 162/195] refactor: opt simd perf. --- source/util/src/tdecompress.c | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/source/util/src/tdecompress.c b/source/util/src/tdecompress.c index 5a5e60093c..509afb6ed4 100644 --- a/source/util/src/tdecompress.c +++ b/source/util/src/tdecompress.c @@ -299,10 +299,10 @@ int32_t tsDecompressTimestampAvx2(const char *const input, const int32_t nelemen _mm_storeu_si128((__m128i *)&ostream[opos], val); // keep the previous value - prevVal = _mm_set1_epi64x(val[1]); + prevVal = _mm_shuffle_epi32 (val, 0xEE); // keep the previous delta of delta, for the first item - prevDelta = _mm_set1_epi64x(deltaOfDelta[1]); + prevDelta = _mm_shuffle_epi32(deltaOfDelta, 0xEE); opos += 2; ipos += nbytes1 + nbytes2; @@ -354,11 +354,11 @@ int32_t tsDecompressTimestampAvx2(const char *const input, const int32_t nelemen _mm_storeu_si128((__m128i *)&ostream[opos], val); // keep the previous value - prevVal = _mm_set1_epi64x(val[1]); + prevVal = _mm_shuffle_epi32 (val, 0xEE); // keep the previous delta of delta __m128i delta = _mm_add_epi64(_mm_slli_si128(deltaOfDelta, 8), deltaOfDelta); - prevDelta = _mm_set1_epi64x(_mm_add_epi64(delta, prevDelta)[1]); + prevDelta = _mm_shuffle_epi32(_mm_add_epi64(delta, prevDelta), 0xEE); opos += 2; ipos += nbytes1 + nbytes2; From 61111385676a910cc9b956fc111c7858da2322f3 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 27 Nov 2023 17:23:34 +0800 Subject: [PATCH 163/195] fix(stream): fix error. --- source/libs/stream/src/streamExec.c | 18 +++++++++--------- source/util/src/tdecompress.c | 10 ++++------ 2 files changed, 13 insertions(+), 15 deletions(-) diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 8b14846414..e0ee01d345 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -252,14 +252,15 @@ static void streamScanHistoryDataImpl(SStreamTask* pTask, SArray* pRes, int32_t* SScanhistoryDataInfo streamScanHistoryData(SStreamTask* pTask, int64_t st) { ASSERT(pTask->info.taskLevel == TASK_LEVEL__SOURCE); - void* exec = pTask->exec.pExecutor; - bool finished = false; + void* exec = pTask->exec.pExecutor; + bool finished = false; + const char* id = pTask->id.idStr; qSetStreamOpOpen(exec); while (1) { if (streamTaskShouldPause(pTask)) { - stDebug("s-task:%s paused from the scan-history task", pTask->id.idStr); + stDebug("s-task:%s paused from the scan-history task", id); // quit from step1, not continue to handle the step2 return (SScanhistoryDataInfo){TASK_SCANHISTORY_QUIT, 0}; } @@ -267,8 +268,7 @@ SScanhistoryDataInfo streamScanHistoryData(SStreamTask* pTask, int64_t st) { SArray* pRes = taosArrayInit(0, sizeof(SSDataBlock)); if (pRes == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; - stError("s-task:%s scan-history prepare result block failed, code:%s, retry later", pTask->id.idStr, - tstrerror(terrno)); + stError("s-task:%s scan-history prepare result block failed, code:%s, retry later", id, tstrerror(terrno)); continue; } @@ -295,8 +295,8 @@ SScanhistoryDataInfo streamScanHistoryData(SStreamTask* pTask, int64_t st) { } if (el >= STREAM_SCAN_HISTORY_TIMESLICE) { - stDebug("s-task:%s fill-history:%d time slice exhausted, elapsed time:%.2fs, retry in 100ms", - pTask->id.idStr, pTask->info.fillHistory, el / 1000.0); + stDebug("s-task:%s fill-history:%d time slice exhausted, elapsed time:%.2fs, retry in 100ms", id, + pTask->info.fillHistory, el / 1000.0); return (SScanhistoryDataInfo){TASK_SCANHISTORY_REXEC, 100}; } } @@ -543,7 +543,7 @@ int32_t streamProcessTranstateBlock(SStreamTask* pTask, SStreamDataBlock* pBlock * todo: the batch of blocks should be tuned dynamic, according to the total elapsed time of each batch of blocks, the * appropriate batch of blocks should be handled in 5 to 10 sec. */ -int32_t streamExecForAll(SStreamTask* pTask) { +int32_t doStreamExecTask(SStreamTask* pTask) { const char* id = pTask->id.idStr; // merge multiple input data if possible in the input queue. @@ -654,7 +654,7 @@ int32_t streamExecTask(SStreamTask* pTask) { int8_t schedStatus = streamTaskSetSchedStatusActive(pTask); if (schedStatus == TASK_SCHED_STATUS__WAITING) { while (1) { - int32_t code = streamExecForAll(pTask); + int32_t code = doStreamExecTask(pTask); if (code < 0) { // todo this status should be removed atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__FAILED); return -1; diff --git a/source/util/src/tdecompress.c b/source/util/src/tdecompress.c index 509afb6ed4..bb22772f7d 100644 --- a/source/util/src/tdecompress.c +++ b/source/util/src/tdecompress.c @@ -271,7 +271,6 @@ int32_t tsDecompressTimestampAvx2(const char *const input, const int32_t nelemen if (nbytes1 == 0) { data1 = _mm_setzero_si128(); } else { -// _mm_shuffle_epi8() memcpy(&data1, (const void*) (input + ipos), nbytes1); } @@ -400,15 +399,14 @@ int32_t tsDecompressTimestampAvx512(const char* const input, const int32_t nelem __m128i prevVal = _mm_setzero_si128(); __m128i prevDelta = _mm_setzero_si128(); - // _mm_maskz_loadu_epi8 #if __AVX512VL__ - int32_t batch = nelements >> 1; - int32_t remainder = nelements & 0x01; + int32_t numOfBatch = nelements >> 1; + int32_t remainder = nelements & 0x01; __mmask16 mask2[16] = {0, 0x0001, 0x0003, 0x0007, 0x000f, 0x001f, 0x003f, 0x007f, 0x00ff}; int32_t i = 0; - if (batch > 1) { + if (numOfBatch > 1) { // first loop uint8_t flags = input[ipos++]; @@ -446,7 +444,7 @@ int32_t tsDecompressTimestampAvx512(const char* const input, const int32_t nelem } // the remain - for(; i < batch; ++i) { + for(; i < numOfBatch; ++i) { uint8_t flags = input[ipos++]; int8_t nbytes1 = flags & INT8MASK(4); // range of nbytes starts from 0 to 7 From a4d475b582465d2594a3a22bcc15c249c891afeb Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Mon, 27 Nov 2023 20:27:53 +0800 Subject: [PATCH 164/195] fix rsma --- include/libs/stream/tstream.h | 23 +++++++++++++---------- source/common/src/tglobal.c | 2 +- source/dnode/vnode/src/sma/smaRollup.c | 5 ++++- source/libs/stream/inc/streamsm.h | 8 ++++---- source/libs/stream/src/streamCheckpoint.c | 4 ++++ 5 files changed, 26 insertions(+), 16 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 2e94bbdacd..f91223b863 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -108,6 +108,7 @@ typedef enum { TASK_LEVEL__SOURCE = 1, TASK_LEVEL__AGG, TASK_LEVEL__SINK, + TASK_LEVEL_SMA, } ETASK_LEVEL; enum { @@ -666,19 +667,19 @@ int32_t tDecodeStreamCheckpointReadyMsg(SDecoder* pDecoder, SStreamCheckpointRea typedef struct STaskStatusEntry { STaskId id; int32_t status; - int32_t statusLastDuration; // to record the last duration of current status + int32_t statusLastDuration; // to record the last duration of current status int64_t stage; int32_t nodeId; - int64_t verStart; // start version in WAL, only valid for source task - int64_t verEnd; // end version in WAL, only valid for source task - int64_t processedVer; // only valid for source task - int32_t relatedHTask; // has related fill-history task - int64_t activeCheckpointId; // current active checkpoint id - bool checkpointFailed; // denote if the checkpoint is failed or not - double inputQUsed; // in MiB + int64_t verStart; // start version in WAL, only valid for source task + int64_t verEnd; // end version in WAL, only valid for source task + int64_t processedVer; // only valid for source task + int32_t relatedHTask; // has related fill-history task + int64_t activeCheckpointId; // current active checkpoint id + bool checkpointFailed; // denote if the checkpoint is failed or not + double inputQUsed; // in MiB double inputRate; - double sinkQuota; // existed quota size for sink task - double sinkDataSize; // sink to dst data size + double sinkQuota; // existed quota size for sink task + double sinkDataSize; // sink to dst data size } STaskStatusEntry; typedef struct SStreamHbMsg { @@ -864,6 +865,8 @@ int32_t streamAddCheckpointSourceRspMsg(SStreamCheckpointSourceReq* pReq, SRpcHa int32_t buildCheckpointSourceRsp(SStreamCheckpointSourceReq* pReq, SRpcHandleInfo* pRpcInfo, SRpcMsg* pMsg, int8_t isSucceed); +SStreamTaskSM* streamCreateStateMachine(SStreamTask* pTask); +void* streamDestroyStateMachine(SStreamTaskSM* pSM); #ifdef __cplusplus } #endif diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index c72a998f88..e8ff530d78 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -248,7 +248,7 @@ int32_t tsTtlBatchDropNum = 10000; // number of tables dropped per batch // internal int32_t tsTransPullupInterval = 2; int32_t tsMqRebalanceInterval = 2; -int32_t tsStreamCheckpointInterval = 40; +int32_t tsStreamCheckpointInterval = 10; float tsSinkDataRate = 2.0; int32_t tsStreamNodeCheckInterval = 15; int32_t tsTtlUnit = 86400; diff --git a/source/dnode/vnode/src/sma/smaRollup.c b/source/dnode/vnode/src/sma/smaRollup.c index c7623eca78..abe4c3f2fc 100644 --- a/source/dnode/vnode/src/sma/smaRollup.c +++ b/source/dnode/vnode/src/sma/smaRollup.c @@ -297,6 +297,8 @@ static int32_t tdSetRSmaInfoItemParams(SSma *pSma, SRSmaParam *param, SRSmaStat sprintf(pStreamTask->exec.qmsg, "%s", RSMA_EXEC_TASK_FLAG); pStreamTask->chkInfo.checkpointId = streamMetaGetLatestCheckpointId(pStreamTask->pMeta); tdRSmaTaskInit(pStreamTask->pMeta, pItem, &pStreamTask->id); + pStreamTask->status.pSM = streamCreateStateMachine(pStreamTask); + pStreamState = streamStateOpen(taskInfDir, pStreamTask, true, -1, -1); if (!pStreamState) { terrno = TSDB_CODE_RSMA_STREAM_STATE_OPEN; @@ -1282,10 +1284,11 @@ _checkpoint: if (pItem && pItem->pStreamTask) { SStreamTask *pTask = pItem->pStreamTask; // atomic_store_32(&pTask->pMeta->chkptNotReadyTasks, 1); - // pTask->checkpointingId = checkpointId; + pTask->chkInfo.checkpointingId = checkpointId; pTask->chkInfo.checkpointId = checkpointId; // 1pTask->checkpointingId; pTask->chkInfo.checkpointVer = pItem->submitReqVer; pTask->info.triggerParam = pItem->fetchResultVer; + pTask->info.taskLevel = TASK_LEVEL_SMA; if (!checkpointBuilt) { // the stream states share one checkpoint diff --git a/source/libs/stream/inc/streamsm.h b/source/libs/stream/inc/streamsm.h index be3665fde7..7be655fbed 100644 --- a/source/libs/stream/inc/streamsm.h +++ b/source/libs/stream/inc/streamsm.h @@ -32,8 +32,8 @@ typedef int32_t (*__state_trans_fn)(SStreamTask*); typedef int32_t (*__state_trans_succ_fn)(SStreamTask*); typedef struct SAttachedEventInfo { - ETaskStatus status; // required status that this event can be handled - EStreamTaskEvent event; // the delayed handled event + ETaskStatus status; // required status that this event can be handled + EStreamTaskEvent event; // the delayed handled event } SAttachedEventInfo; typedef struct STaskStateTrans { @@ -64,8 +64,8 @@ typedef struct SStreamEventInfo { const char* name; } SStreamEventInfo; -SStreamTaskSM* streamCreateStateMachine(SStreamTask* pTask); -void* streamDestroyStateMachine(SStreamTaskSM* pSM); +// SStreamTaskSM* streamCreateStateMachine(SStreamTask* pTask); +// void* streamDestroyStateMachine(SStreamTaskSM* pSM); #ifdef __cplusplus } #endif diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 7473e7a411..e2561de841 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -291,6 +291,10 @@ int32_t streamSaveTaskCheckpointInfo(SStreamTask* p, int64_t checkpointId) { return code; } + if (p->info.taskLevel > TASK_LEVEL__SINK) { + return code; + } + taosThreadMutexLock(&p->lock); ASSERT(p->chkInfo.checkpointId <= p->chkInfo.checkpointingId && p->chkInfo.checkpointingId == checkpointId && From c5aba6089951ebd69af31ec09a7191c34ea5a6c9 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 28 Nov 2023 09:28:43 +0800 Subject: [PATCH 165/195] fix(stream): disable the time slice usage limitation for stream task. --- source/libs/stream/src/streamExec.c | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index e0ee01d345..8ab8f3852e 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -281,12 +281,12 @@ SScanhistoryDataInfo streamScanHistoryData(SStreamTask* pTask, int64_t st) { } // dispatch the generated results - int32_t code = handleResultBlocks(pTask, pRes, size); + /*int32_t code = */handleResultBlocks(pTask, pRes, size); int64_t el = taosGetTimestampMs() - st; // downstream task input queue is full, try in 5sec - if (pTask->inputq.status == TASK_INPUT_STATUS__BLOCKED) { + if (pTask->inputq.status == TASK_INPUT_STATUS__BLOCKED && (pTask->info.fillHistory == 1)) { return (SScanhistoryDataInfo){TASK_SCANHISTORY_REXEC, 5000}; } @@ -294,7 +294,7 @@ SScanhistoryDataInfo streamScanHistoryData(SStreamTask* pTask, int64_t st) { return (SScanhistoryDataInfo){TASK_SCANHISTORY_CONT, 0}; } - if (el >= STREAM_SCAN_HISTORY_TIMESLICE) { + if (el >= STREAM_SCAN_HISTORY_TIMESLICE && (pTask->info.fillHistory == 1)) { stDebug("s-task:%s fill-history:%d time slice exhausted, elapsed time:%.2fs, retry in 100ms", id, pTask->info.fillHistory, el / 1000.0); return (SScanhistoryDataInfo){TASK_SCANHISTORY_REXEC, 100}; From 9649db9003829a45a65b8dba14e3c63544692552 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 27 Nov 2023 18:49:22 +0800 Subject: [PATCH 166/195] fix(stream): fix invalid free. --- source/dnode/mnode/impl/src/mndStream.c | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 81cb4703e1..17c8dfe9cb 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -755,13 +755,14 @@ static int32_t checkForNumOfStreams(SMnode *pMnode, SStreamObj *pStreamObj) { / } static int32_t mndProcessCreateStreamReq(SRpcMsg *pReq) { - SMnode * pMnode = pReq->info.node; - int32_t code = -1; - SStreamObj * pStream = NULL; - SDbObj * pDb = NULL; - SCMCreateStreamReq createStreamReq = {0}; - SStreamObj streamObj = {0}; + SMnode *pMnode = pReq->info.node; + int32_t code = -1; + SStreamObj *pStream = NULL; + SStreamObj streamObj = {0}; + char *sql = NULL; + int32_t sqlLen = 0; + SCMCreateStreamReq createStreamReq = {0}; if (tDeserializeSCMCreateStreamReq(pReq->pCont, pReq->contLen, &createStreamReq) != 0) { terrno = TSDB_CODE_INVALID_MSG; goto _OVER; @@ -792,8 +793,6 @@ static int32_t mndProcessCreateStreamReq(SRpcMsg *pReq) { goto _OVER; } - char* sql = NULL; - int32_t sqlLen = 0; if(createStreamReq.sql != NULL){ sqlLen = strlen(createStreamReq.sql); sql = taosMemoryMalloc(sqlLen + 1); From 6bb30822c2711315dbb480037db337e8a06a62af Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 27 Nov 2023 18:51:30 +0800 Subject: [PATCH 167/195] refactor: do some internal refactor. --- source/dnode/mnode/impl/src/mndStream.c | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 17c8dfe9cb..feb0c3e52a 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -793,7 +793,7 @@ static int32_t mndProcessCreateStreamReq(SRpcMsg *pReq) { goto _OVER; } - if(createStreamReq.sql != NULL){ + if (createStreamReq.sql != NULL) { sqlLen = strlen(createStreamReq.sql); sql = taosMemoryMalloc(sqlLen + 1); memset(sql, 0, sqlLen + 1); @@ -880,14 +880,13 @@ static int32_t mndProcessCreateStreamReq(SRpcMsg *pReq) { // reuse this function for stream if (sql != NULL && sqlLen > 0) { - auditRecord(pReq, pMnode->clusterId, "createStream", dbname.dbname, name.dbname, sql, - sqlLen); - } - else{ + auditRecord(pReq, pMnode->clusterId, "createStream", dbname.dbname, name.dbname, sql, sqlLen); + } else { char detail[1000] = {0}; sprintf(detail, "dbname:%s, stream name:%s", dbname.dbname, name.dbname); auditRecord(pReq, pMnode->clusterId, "createStream", dbname.dbname, name.dbname, detail, strlen(detail)); } + _OVER: if (code != 0 && code != TSDB_CODE_ACTION_IN_PROGRESS) { mError("stream:%s, failed to create since %s", createStreamReq.name, terrstr()); @@ -896,7 +895,7 @@ _OVER: mndReleaseStream(pMnode, pStream); tFreeSCMCreateStreamReq(&createStreamReq); tFreeStreamObj(&streamObj); - if(sql != NULL){ + if (sql != NULL) { taosMemoryFreeClear(sql); } return code; From 2eec29861c6e5652d6496474e749315edbdee342 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 28 Nov 2023 10:06:21 +0800 Subject: [PATCH 168/195] fix(util): disable avx2 version decompress timestamp. --- source/util/src/tdecompress.c | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/source/util/src/tdecompress.c b/source/util/src/tdecompress.c index bb22772f7d..9c17000c50 100644 --- a/source/util/src/tdecompress.c +++ b/source/util/src/tdecompress.c @@ -249,6 +249,7 @@ int32_t tsDecompressFloatImplAvx2(const char *const input, const int32_t nelemen int32_t tsDecompressTimestampAvx2(const char *const input, const int32_t nelements, char *const output, bool bigEndian) { +#if 0 int64_t *ostream = (int64_t *)output; int32_t ipos = 1, opos = 0; __m128i prevVal = _mm_setzero_si128(); @@ -389,11 +390,12 @@ int32_t tsDecompressTimestampAvx2(const char *const input, const int32_t nelemen } } #endif - +#endif return 0; } -int32_t tsDecompressTimestampAvx512(const char* const input, const int32_t nelements, char *const output, bool bigEndian) { +int32_t tsDecompressTimestampAvx512(const char *const input, const int32_t nelements, char *const output, + bool UNUSED_PARAM(bigEndian)) { int64_t *ostream = (int64_t *)output; int32_t ipos = 1, opos = 0; __m128i prevVal = _mm_setzero_si128(); @@ -489,11 +491,7 @@ int32_t tsDecompressTimestampAvx512(const char* const input, const int32_t nelem if (nbytes == 0) { deltaOfDelta = 0; } else { - // if (is_bigendian()) { - // memcpy(((char *)(&dd1)) + longBytes - nbytes, input + ipos, nbytes); - // } else { memcpy(&dd, input + ipos, nbytes); - // } deltaOfDelta = ZIGZAG_DECODE(int64_t, dd); } From 786383118c9f8ee5ee7ec608fe4f0a7a63540e09 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Tue, 28 Nov 2023 10:39:17 +0800 Subject: [PATCH 169/195] fix drop stream case --- source/dnode/mnode/impl/src/mndStream.c | 3 +++ 1 file changed, 3 insertions(+) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index d2bab4eebf..15cc3eb041 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1337,6 +1337,9 @@ static int32_t mndProcessStreamCheckpointInCandid(SRpcMsg *pReq) { SCheckpointCandEntry *pEntry = pIter; SStreamObj *ps = mndAcquireStream(pMnode, pEntry->pName); + if (ps == NULL) { + continue; + } mDebug("start to launch checkpoint for stream:%s %" PRIx64 " in candidate list", pEntry->pName, pEntry->streamId); code = mndProcessStreamCheckpointTrans(pMnode, ps, pEntry->checkpointId); From f4cf898af73c7728b5dc2ea7e7124f6e4854e0f6 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 28 Nov 2023 13:59:13 +0800 Subject: [PATCH 170/195] fix(util): fix syntax error. --- source/util/src/tdecompress.c | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/source/util/src/tdecompress.c b/source/util/src/tdecompress.c index 9c17000c50..f212bf5231 100644 --- a/source/util/src/tdecompress.c +++ b/source/util/src/tdecompress.c @@ -398,11 +398,12 @@ int32_t tsDecompressTimestampAvx512(const char *const input, const int32_t nelem bool UNUSED_PARAM(bigEndian)) { int64_t *ostream = (int64_t *)output; int32_t ipos = 1, opos = 0; - __m128i prevVal = _mm_setzero_si128(); - __m128i prevDelta = _mm_setzero_si128(); #if __AVX512VL__ + __m128i prevVal = _mm_setzero_si128(); + __m128i prevDelta = _mm_setzero_si128(); + int32_t numOfBatch = nelements >> 1; int32_t remainder = nelements & 0x01; __mmask16 mask2[16] = {0, 0x0001, 0x0003, 0x0007, 0x000f, 0x001f, 0x003f, 0x007f, 0x00ff}; From b54f5b47ad4f2511cb932ebbce44bce50238d215 Mon Sep 17 00:00:00 2001 From: kailixu Date: Tue, 28 Nov 2023 17:58:54 +0800 Subject: [PATCH 171/195] fix: reserve enough buffer for user --- source/dnode/mnode/impl/src/mndUser.c | 31 +++++++++++++------- tests/system-test/0-others/user_privilege.py | 19 +++++++++--- 2 files changed, 35 insertions(+), 15 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndUser.c b/source/dnode/mnode/impl/src/mndUser.c index 66abfd6bc1..0e3b544508 100644 --- a/source/dnode/mnode/impl/src/mndUser.c +++ b/source/dnode/mnode/impl/src/mndUser.c @@ -708,9 +708,8 @@ SSdbRaw *mndUserActionEncode(SUserObj *pUser) { int32_t numOfAlterViews = taosHashGetSize(pUser->alterViews); int32_t numOfTopics = taosHashGetSize(pUser->topics); int32_t numOfUseDbs = taosHashGetSize(pUser->useDbs); - int32_t size = sizeof(SUserObj) + USER_RESERVE_SIZE + - (numOfReadDbs + numOfWriteDbs + numOfUseDbs) * TSDB_DB_FNAME_LEN + numOfTopics * TSDB_TOPIC_FNAME_LEN + - ipWhiteReserve; + int32_t size = sizeof(SUserObj) + USER_RESERVE_SIZE + (numOfReadDbs + numOfWriteDbs) * TSDB_DB_FNAME_LEN + + numOfTopics * TSDB_TOPIC_FNAME_LEN + ipWhiteReserve; char *stb = taosHashIterate(pUser->readTbs, NULL); while (stb != NULL) { @@ -720,7 +719,7 @@ SSdbRaw *mndUserActionEncode(SUserObj *pUser) { size += keyLen; size_t valueLen = 0; - valueLen = strlen(stb); + valueLen = strlen(stb) + 1; size += sizeof(int32_t); size += valueLen; stb = taosHashIterate(pUser->readTbs, stb); @@ -734,7 +733,7 @@ SSdbRaw *mndUserActionEncode(SUserObj *pUser) { size += keyLen; size_t valueLen = 0; - valueLen = strlen(stb); + valueLen = strlen(stb) + 1; size += sizeof(int32_t); size += valueLen; stb = taosHashIterate(pUser->writeTbs, stb); @@ -748,7 +747,7 @@ SSdbRaw *mndUserActionEncode(SUserObj *pUser) { size += keyLen; size_t valueLen = 0; - valueLen = strlen(stb); + valueLen = strlen(stb) + 1; size += sizeof(int32_t); size += valueLen; stb = taosHashIterate(pUser->alterTbs, stb); @@ -762,7 +761,7 @@ SSdbRaw *mndUserActionEncode(SUserObj *pUser) { size += keyLen; size_t valueLen = 0; - valueLen = strlen(stb); + valueLen = strlen(stb) + 1; size += sizeof(int32_t); size += valueLen; stb = taosHashIterate(pUser->readViews, stb); @@ -776,7 +775,7 @@ SSdbRaw *mndUserActionEncode(SUserObj *pUser) { size += keyLen; size_t valueLen = 0; - valueLen = strlen(stb); + valueLen = strlen(stb) + 1; size += sizeof(int32_t); size += valueLen; stb = taosHashIterate(pUser->writeViews, stb); @@ -790,11 +789,21 @@ SSdbRaw *mndUserActionEncode(SUserObj *pUser) { size += keyLen; size_t valueLen = 0; - valueLen = strlen(stb); + valueLen = strlen(stb) + 1; size += sizeof(int32_t); size += valueLen; stb = taosHashIterate(pUser->alterViews, stb); - } + } + + int32_t *useDb = taosHashIterate(pUser->useDbs, NULL); + while (useDb != NULL) { + size_t keyLen = 0; + void *key = taosHashGetKey(useDb, &keyLen); + size += sizeof(int32_t); + size += keyLen; + size += sizeof(int32_t); + useDb = taosHashIterate(pUser->useDbs, useDb); + } SSdbRaw *pRaw = sdbAllocRaw(SDB_USER, USER_VER_NUMBER, size); if (pRaw == NULL) goto _OVER; @@ -925,7 +934,7 @@ SSdbRaw *mndUserActionEncode(SUserObj *pUser) { stb = taosHashIterate(pUser->alterViews, stb); } - int32_t *useDb = taosHashIterate(pUser->useDbs, NULL); + useDb = taosHashIterate(pUser->useDbs, NULL); while (useDb != NULL) { size_t keyLen = 0; void *key = taosHashGetKey(useDb, &keyLen); diff --git a/tests/system-test/0-others/user_privilege.py b/tests/system-test/0-others/user_privilege.py index d1b93f6942..a731e85ddb 100644 --- a/tests/system-test/0-others/user_privilege.py +++ b/tests/system-test/0-others/user_privilege.py @@ -27,6 +27,7 @@ class TDTestCase: tdSql.init(conn.cursor()) self.setsql = TDSetSql() self.stbname = 'stb' + self.user_name = 'test' self.binary_length = 20 # the length of binary for column_dict self.nchar_length = 20 # the length of nchar for column_dict self.dbnames = ['db1', 'db2'] @@ -54,12 +55,12 @@ class TDTestCase: ] self.tbnum = 4 + self.stbnum_grant = 200 def create_user(self): - user_name = 'test' - tdSql.execute(f'create user {user_name} pass "test"') - tdSql.execute(f'grant read on {self.dbnames[0]}.{self.stbname} with t2 = "Beijing" to {user_name}') - tdSql.execute(f'grant write on {self.dbnames[1]}.{self.stbname} with t1 = 2 to {user_name}') + tdSql.execute(f'create user {self.user_name} pass "test"') + tdSql.execute(f'grant read on {self.dbnames[0]}.{self.stbname} with t2 = "Beijing" to {self.user_name}') + tdSql.execute(f'grant write on {self.dbnames[1]}.{self.stbname} with t1 = 2 to {self.user_name}') def prepare_data(self): for db in self.dbnames: @@ -70,6 +71,8 @@ class TDTestCase: tdSql.execute(f'create table {self.stbname}_{i} using {self.stbname} tags({self.tag_list[i]})') for j in self.values_list: tdSql.execute(f'insert into {self.stbname}_{i} values({j})') + for i in range(self.stbnum_grant): + tdSql.execute(f'create table {self.stbname}_grant_{i} (ts timestamp, c0 int) tags(t0 int)') def user_read_privilege_check(self, dbname): testconn = taos.connect(user='test', password='test') @@ -128,12 +131,20 @@ class TDTestCase: tdLog.exit(f"{caller.filename}({caller.lineno}) failed: sql:{sql}, expect error not occured") pass + def user_privilege_grant_check(self): + for db in self.dbnames: + tdSql.execute(f"use {db}") + for i in range(self.stbnum_grant): + tdSql.execute(f'grant read on {db}.{self.stbname}_grant_{i} to {self.user_name}') + tdSql.execute(f'grant write on {db}.{self.stbname}_grant_{i} to {self.user_name}') + def run(self): self.prepare_data() self.create_user() self.user_read_privilege_check(self.dbnames[0]) self.user_write_privilege_check(self.dbnames[1]) self.user_privilege_error_check() + self.user_privilege_grant_check() def stop(self): tdSql.close() From cc1eb2769f036f3f23e8f32fb81b34a0fe5f193b Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Tue, 28 Nov 2023 20:29:42 +0800 Subject: [PATCH 172/195] fix crash while quit --- source/libs/stream/src/streamBackendRocksdb.c | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index ccf166c390..19247166b3 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1844,6 +1844,9 @@ void taskDbDestroy(void* pDb, bool flush) { rocksdb_column_family_handle_destroy(wrapper->pCf[i]); } } + + if (wrapper->db) rocksdb_close(wrapper->db); + rocksdb_options_destroy(wrapper->dbOpt); rocksdb_readoptions_destroy(wrapper->readOpt); rocksdb_writeoptions_destroy(wrapper->writeOpt); @@ -1851,7 +1854,6 @@ void taskDbDestroy(void* pDb, bool flush) { rocksdb_cache_destroy(wrapper->cache); taosMemoryFree(wrapper->pCf); - for (int i = 0; i < nCf; i++) { rocksdb_options_t* opt = wrapper->pCfOpts[i]; rocksdb_comparator_t* compare = wrapper->pCompares[i]; @@ -1867,8 +1869,6 @@ void taskDbDestroy(void* pDb, bool flush) { taosThreadMutexDestroy(&wrapper->mutex); - if (wrapper->db) rocksdb_close(wrapper->db); - taskDbDestroyChkpOpt(wrapper); taosMemoryFree(wrapper->idstr); From a5c8eaacaac548ef67d3e395164c21d556fc1a7f Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 28 Nov 2023 23:52:19 +0800 Subject: [PATCH 173/195] fix(tsdb):clear the tsdb stt file reader flag when suspending the tsdbReader --- source/dnode/vnode/src/tsdb/tsdbRead2.c | 67 +++++++++------------- source/dnode/vnode/src/tsdb/tsdbReadUtil.c | 1 + 2 files changed, 27 insertions(+), 41 deletions(-) diff --git a/source/dnode/vnode/src/tsdb/tsdbRead2.c b/source/dnode/vnode/src/tsdb/tsdbRead2.c index 751df706ab..5d018af6c1 100644 --- a/source/dnode/vnode/src/tsdb/tsdbRead2.c +++ b/source/dnode/vnode/src/tsdb/tsdbRead2.c @@ -4148,6 +4148,20 @@ void tsdbReaderClose2(STsdbReader* pReader) { taosMemoryFreeClear(pReader); } +static void clearMemIterInfo(STableBlockScanInfo* pInfo) { + pInfo->iterInit = false; + pInfo->iter.hasVal = false; + pInfo->iiter.hasVal = false; + + if (pInfo->iter.iter != NULL) { + pInfo->iter.iter = tsdbTbDataIterDestroy(pInfo->iter.iter); + } + + if (pInfo->iiter.iter != NULL) { + pInfo->iiter.iter = tsdbTbDataIterDestroy(pInfo->iiter.iter); + } +} + int32_t tsdbReaderSuspend2(STsdbReader* pReader) { // save reader's base state & reset top state to be reconstructed from base state int32_t code = 0; @@ -4168,28 +4182,20 @@ int32_t tsdbReaderSuspend2(STsdbReader* pReader) { tsdbDataFileReaderClose(&pReader->pFileReader); SCostSummary* pCost = &pReader->cost; + pReader->status.pLDataIterArray = destroySttBlockReader(pReader->status.pLDataIterArray, &pCost->sttCost); pReader->status.pLDataIterArray = taosArrayInit(4, POINTER_BYTES); + // resetDataBlockScanInfo excluding lastKey STableBlockScanInfo** p = NULL; - int32_t iter = 0; + int32_t iter = 0; while ((p = tSimpleHashIterate(pStatus->pTableMap, p, &iter)) != NULL) { STableBlockScanInfo* pInfo = *(STableBlockScanInfo**)p; - - pInfo->iterInit = false; - pInfo->iter.hasVal = false; - pInfo->iiter.hasVal = false; - - if (pInfo->iter.iter != NULL) { - pInfo->iter.iter = tsdbTbDataIterDestroy(pInfo->iter.iter); - } - - if (pInfo->iiter.iter != NULL) { - pInfo->iiter.iter = tsdbTbDataIterDestroy(pInfo->iiter.iter); - } - + clearMemIterInfo(pInfo); + pInfo->sttKeyInfo.status = STT_FILE_READER_UNINIT; pInfo->delSkyline = taosArrayDestroy(pInfo->delSkyline); + pInfo->pFileDelData = taosArrayDestroy(pInfo->pFileDelData); } } else { @@ -4199,45 +4205,24 @@ int32_t tsdbReaderSuspend2(STsdbReader* pReader) { while ((p = tSimpleHashIterate(pStatus->pTableMap, p, &iter)) != NULL) { STableBlockScanInfo* pInfo = *(STableBlockScanInfo**)p; - - pInfo->iterInit = false; - pInfo->iter.hasVal = false; - pInfo->iiter.hasVal = false; - - if (pInfo->iter.iter != NULL) { - pInfo->iter.iter = tsdbTbDataIterDestroy(pInfo->iter.iter); - } - - if (pInfo->iiter.iter != NULL) { - pInfo->iiter.iter = tsdbTbDataIterDestroy(pInfo->iiter.iter); - } - + clearMemIterInfo(pInfo); + pInfo->sttKeyInfo.status = STT_FILE_READER_UNINIT; pInfo->delSkyline = taosArrayDestroy(pInfo->delSkyline); } - pBlockScanInfo = pStatus->pTableIter == NULL ? NULL : *pStatus->pTableIter; + pBlockScanInfo = (pStatus->pTableIter == NULL) ? NULL : *pStatus->pTableIter; if (pBlockScanInfo) { // save lastKey to restore memory iterator STimeWindow w = pReader->resBlockInfo.pResBlock->info.window; pBlockScanInfo->lastProcKey = ASCENDING_TRAVERSE(pReader->info.order) ? w.ekey : w.skey; - // reset current current table's data block scan info, - pBlockScanInfo->iterInit = false; - - pBlockScanInfo->iter.hasVal = false; - pBlockScanInfo->iiter.hasVal = false; - if (pBlockScanInfo->iter.iter != NULL) { - pBlockScanInfo->iter.iter = tsdbTbDataIterDestroy(pBlockScanInfo->iter.iter); - } - - if (pBlockScanInfo->iiter.iter != NULL) { - pBlockScanInfo->iiter.iter = tsdbTbDataIterDestroy(pBlockScanInfo->iiter.iter); - } + clearMemIterInfo(pBlockScanInfo); + pBlockScanInfo->sttKeyInfo.status = STT_FILE_READER_UNINIT; + pBlockScanInfo->delSkyline = taosArrayDestroy(pBlockScanInfo->delSkyline); pBlockScanInfo->pBlockList = taosArrayDestroy(pBlockScanInfo->pBlockList); pBlockScanInfo->pBlockIdxList = taosArrayDestroy(pBlockScanInfo->pBlockIdxList); // TODO: keep skyline for reuse - pBlockScanInfo->delSkyline = taosArrayDestroy(pBlockScanInfo->delSkyline); } } diff --git a/source/dnode/vnode/src/tsdb/tsdbReadUtil.c b/source/dnode/vnode/src/tsdb/tsdbReadUtil.c index 305399e0af..24c526a906 100644 --- a/source/dnode/vnode/src/tsdb/tsdbReadUtil.c +++ b/source/dnode/vnode/src/tsdb/tsdbReadUtil.c @@ -210,6 +210,7 @@ void clearBlockScanInfo(STableBlockScanInfo* p) { p->iterInit = false; p->iter.hasVal = false; p->iiter.hasVal = false; + p->sttKeyInfo.status = STT_FILE_READER_UNINIT; if (p->iter.iter != NULL) { p->iter.iter = tsdbTbDataIterDestroy(p->iter.iter); From 5c5efa34eb11b4e798919e46f90e9917250a7450 Mon Sep 17 00:00:00 2001 From: Bob Liu Date: Wed, 29 Nov 2023 01:53:13 +0800 Subject: [PATCH 174/195] fix crash when insert big csv --- include/libs/nodes/querynodes.h | 1 + source/libs/nodes/src/nodesUtilFuncs.c | 1 + source/libs/parser/inc/parInsertUtil.h | 2 +- source/libs/parser/src/parInsertSml.c | 2 +- source/libs/parser/src/parInsertSql.c | 52 +++++++- source/libs/parser/src/parInsertStmt.c | 2 +- source/libs/parser/src/parInsertUtil.c | 29 ++++- tests/parallel_test/cases.task | 1 + tests/system-test/1-insert/ts-4272.py | 158 +++++++++++++++++++++++++ 9 files changed, 238 insertions(+), 10 deletions(-) create mode 100644 tests/system-test/1-insert/ts-4272.py diff --git a/include/libs/nodes/querynodes.h b/include/libs/nodes/querynodes.h index 19dc8c9e4d..5c5172b9cd 100644 --- a/include/libs/nodes/querynodes.h +++ b/include/libs/nodes/querynodes.h @@ -449,6 +449,7 @@ typedef struct SVnodeModifyOpStmt { SHashObj* pSubTableHashObj; // SHashObj SHashObj* pTableNameHashObj; // set of table names for refreshing meta, sync mode SHashObj* pDbFNameHashObj; // set of db names for refreshing meta, sync mode + SHashObj* pTableCxtHashObj; // temp SHashObj for single request SArray* pVgDataBlocks; // SArray SVCreateTbReq* pCreateTblReq; TdFilePtr fp; diff --git a/source/libs/nodes/src/nodesUtilFuncs.c b/source/libs/nodes/src/nodesUtilFuncs.c index d167d81c82..e730ccf21b 100644 --- a/source/libs/nodes/src/nodesUtilFuncs.c +++ b/source/libs/nodes/src/nodesUtilFuncs.c @@ -887,6 +887,7 @@ void nodesDestroyNode(SNode* pNode) { taosHashCleanup(pStmt->pSubTableHashObj); taosHashCleanup(pStmt->pTableNameHashObj); taosHashCleanup(pStmt->pDbFNameHashObj); + taosHashCleanup(pStmt->pTableCxtHashObj); if (pStmt->freeHashFunc) { pStmt->freeHashFunc(pStmt->pTableBlockHashObj); } diff --git a/source/libs/parser/inc/parInsertUtil.h b/source/libs/parser/inc/parInsertUtil.h index ce8c2d8a3d..b20587dd43 100644 --- a/source/libs/parser/inc/parInsertUtil.h +++ b/source/libs/parser/inc/parInsertUtil.h @@ -50,7 +50,7 @@ void insCheckTableDataOrder(STableDataCxt *pTableCxt, TSKEY tsKey); int32_t insGetTableDataCxt(SHashObj *pHash, void *id, int32_t idLen, STableMeta *pTableMeta, SVCreateTbReq **pCreateTbReq, STableDataCxt **pTableCxt, bool colMode, bool ignoreColVals); int32_t initTableColSubmitData(STableDataCxt *pTableCxt); -int32_t insMergeTableDataCxt(SHashObj *pTableHash, SArray **pVgDataBlocks); +int32_t insMergeTableDataCxt(SHashObj *pTableHash, SArray **pVgDataBlocks, bool isRebuild); int32_t insBuildVgDataBlocks(SHashObj *pVgroupsHashObj, SArray *pVgDataBlocks, SArray **pDataBlocks); void insDestroyTableDataCxtHashMap(SHashObj *pTableCxtHash); void insDestroyVgroupDataCxt(SVgroupDataCxt *pVgCxt); diff --git a/source/libs/parser/src/parInsertSml.c b/source/libs/parser/src/parInsertSml.c index f2194402da..2dbba38212 100644 --- a/source/libs/parser/src/parInsertSml.c +++ b/source/libs/parser/src/parInsertSml.c @@ -425,7 +425,7 @@ SQuery* smlInitHandle() { int32_t smlBuildOutput(SQuery* handle, SHashObj* pVgHash) { SVnodeModifyOpStmt* pStmt = (SVnodeModifyOpStmt*)(handle)->pRoot; // merge according to vgId - int32_t code = insMergeTableDataCxt(pStmt->pTableBlockHashObj, &pStmt->pVgDataBlocks); + int32_t code = insMergeTableDataCxt(pStmt->pTableBlockHashObj, &pStmt->pVgDataBlocks, true); if (code != TSDB_CODE_SUCCESS) { uError("insMergeTableDataCxt failed"); return code; diff --git a/source/libs/parser/src/parInsertSql.c b/source/libs/parser/src/parInsertSql.c index 2b8516d37b..684314faef 100644 --- a/source/libs/parser/src/parInsertSql.c +++ b/source/libs/parser/src/parInsertSql.c @@ -55,6 +55,7 @@ typedef struct SInsertParseContext { bool usingDuplicateTable; bool forceUpdate; bool needTableTagVal; + bool needRequest; // whether or not request server } SInsertParseContext; typedef int32_t (*_row_append_fn_t)(SMsgBuf* pMsgBuf, const void* value, int32_t len, void* param); @@ -652,6 +653,10 @@ static int32_t parseTagValue(SInsertParseContext* pCxt, SVnodeModifyOpStmt* pStm } static int32_t buildCreateTbReq(SVnodeModifyOpStmt* pStmt, STag* pTag, SArray* pTagName) { + if (pStmt->pCreateTblReq) { + tdDestroySVCreateTbReq(pStmt->pCreateTblReq); + taosMemoryFreeClear(pStmt->pCreateTblReq); + } pStmt->pCreateTblReq = taosMemoryCalloc(1, sizeof(SVCreateTbReq)); if (NULL == pStmt->pCreateTblReq) { return TSDB_CODE_OUT_OF_MEMORY; @@ -1992,7 +1997,7 @@ static int32_t parseCsvFile(SInsertParseContext* pCxt, SVnodeModifyOpStmt* pStmt (*pNumOfRows)++; } - if (TSDB_CODE_SUCCESS == code && (*pNumOfRows) > tsMaxInsertBatchRows) { + if (TSDB_CODE_SUCCESS == code && (*pNumOfRows) >= tsMaxInsertBatchRows) { pStmt->fileProcessing = true; break; } @@ -2003,7 +2008,7 @@ static int32_t parseCsvFile(SInsertParseContext* pCxt, SVnodeModifyOpStmt* pStmt parserDebug("0x%" PRIx64 " %d rows have been parsed", pCxt->pComCxt->requestId, *pNumOfRows); - if (TSDB_CODE_SUCCESS == code && 0 == (*pNumOfRows) && + if (TSDB_CODE_SUCCESS == code && 0 == (*pNumOfRows) && 0 == pStmt->totalRowsNum && (!TSDB_QUERY_HAS_TYPE(pStmt->insertType, TSDB_QUERY_TYPE_STMT_INSERT)) && !pStmt->fileProcessing) { code = buildSyntaxErrMsg(&pCxt->msg, "no any data points", NULL); } @@ -2022,7 +2027,22 @@ static int32_t parseDataFromFileImpl(SInsertParseContext* pCxt, SVnodeModifyOpSt } else { parserDebug("0x%" PRIx64 " insert from csv. File is too large, do it in batches.", pCxt->pComCxt->requestId); } + if (pStmt->insertType != TSDB_QUERY_TYPE_FILE_INSERT) { + return buildSyntaxErrMsg(&pCxt->msg, "keyword VALUES or FILE is exclusive", NULL); + } } + + // just record pTableCxt whose data come from file + if (numOfRows > 0) { + if (NULL == pStmt->pTableCxtHashObj) { + pStmt->pTableCxtHashObj = + taosHashInit(128, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), true, HASH_NO_LOCK); + } + void* pData = rowsDataCxt.pTableDataCxt; + taosHashPut(pStmt->pTableCxtHashObj, &pStmt->pTableMeta->uid, sizeof(pStmt->pTableMeta->uid), &pData, + POINTER_BYTES); + } + return code; } @@ -2061,6 +2081,9 @@ static int32_t parseDataClause(SInsertParseContext* pCxt, SVnodeModifyOpStmt* pS NEXT_TOKEN(pStmt->pSql, token); switch (token.type) { case TK_VALUES: + if (TSDB_QUERY_HAS_TYPE(pStmt->insertType, TSDB_QUERY_TYPE_FILE_INSERT)) { + return buildSyntaxErrMsg(&pCxt->msg, "keyword VALUES or FILE is exclusive", token.z); + } return parseValuesClause(pCxt, pStmt, rowsDataCxt, &token); case TK_FILE: return parseFileClause(pCxt, pStmt, rowsDataCxt, &token); @@ -2275,8 +2298,25 @@ static int32_t parseInsertBodyBottom(SInsertParseContext* pCxt, SVnodeModifyOpSt return setStmtInfo(pCxt, pStmt); } + // release old array alloced by merge + pStmt->freeArrayFunc(pStmt->pVgDataBlocks); + pStmt->pVgDataBlocks = NULL; + + bool fileOnly = (pStmt->insertType == TSDB_QUERY_TYPE_FILE_INSERT); + if (fileOnly) { + // none data, skip merge & buildvgdata + if (0 == taosHashGetSize(pStmt->pTableCxtHashObj)) { + pCxt->needRequest = false; + return TSDB_CODE_SUCCESS; + } + } + // merge according to vgId - int32_t code = insMergeTableDataCxt(pStmt->pTableBlockHashObj, &pStmt->pVgDataBlocks); + int32_t code = insMergeTableDataCxt(fileOnly ? pStmt->pTableCxtHashObj : pStmt->pTableBlockHashObj, + &pStmt->pVgDataBlocks, pStmt->fileProcessing); + // clear tmp hashobj only + taosHashClear(pStmt->pTableCxtHashObj); + if (TSDB_CODE_SUCCESS == code) { code = insBuildVgDataBlocks(pStmt->pVgroupsHashObj, pStmt->pVgDataBlocks, &pStmt->pDataBlocks); } @@ -2718,6 +2758,7 @@ int32_t parseInsertSql(SParseContext* pCxt, SQuery** pQuery, SCatalogReq* pCatal .msg = {.buf = pCxt->pMsg, .len = pCxt->msgLen}, .missCache = false, .usingDuplicateTable = false, + .needRequest = true, .forceUpdate = (NULL != pCatalogReq ? pCatalogReq->forceUpdate : false)}; int32_t code = initInsertQuery(&context, pCatalogReq, pMetaData, pQuery); @@ -2732,5 +2773,10 @@ int32_t parseInsertSql(SParseContext* pCxt, SQuery** pQuery, SCatalogReq* pCatal code = setRefreshMeta(*pQuery); } insDestroyBoundColInfo(&context.tags); + + // if no data to insert, set emptyMode to avoid request server + if (!context.needRequest) { + (*pQuery)->execMode = QUERY_EXEC_MODE_EMPTY_RESULT; + } return code; } diff --git a/source/libs/parser/src/parInsertStmt.c b/source/libs/parser/src/parInsertStmt.c index 5137deca2e..a88aec20b3 100644 --- a/source/libs/parser/src/parInsertStmt.c +++ b/source/libs/parser/src/parInsertStmt.c @@ -58,7 +58,7 @@ int32_t qBuildStmtOutput(SQuery* pQuery, SHashObj* pVgHash, SHashObj* pBlockHash // merge according to vgId if (taosHashGetSize(pBlockHash) > 0) { - code = insMergeTableDataCxt(pBlockHash, &pVgDataBlocks); + code = insMergeTableDataCxt(pBlockHash, &pVgDataBlocks, true); } if (TSDB_CODE_SUCCESS == code) { code = insBuildVgDataBlocks(pVgHash, pVgDataBlocks, &pStmt->pDataBlocks); diff --git a/source/libs/parser/src/parInsertUtil.c b/source/libs/parser/src/parInsertUtil.c index 21b093c76c..a924ed68b0 100644 --- a/source/libs/parser/src/parInsertUtil.c +++ b/source/libs/parser/src/parInsertUtil.c @@ -289,6 +289,14 @@ static int32_t rebuildTableData(SSubmitTbData* pSrc, SSubmitTbData** pDst) { pTmp->uid = pSrc->uid; pTmp->sver = pSrc->sver; pTmp->pCreateTbReq = NULL; + if (pTmp->flags & SUBMIT_REQ_AUTO_CREATE_TABLE) { + if (pSrc->pCreateTbReq) { + cloneSVreateTbReq(pSrc->pCreateTbReq, &pTmp->pCreateTbReq); + } else { + pTmp->flags &= ~SUBMIT_REQ_AUTO_CREATE_TABLE; + } + } + if (pTmp->flags & SUBMIT_REQ_COLUMN_DATA_FORMAT) { pTmp->aCol = taosArrayInit(128, sizeof(SColData)); if (NULL == pTmp->aCol) { @@ -416,15 +424,21 @@ void insDestroyTableDataCxtHashMap(SHashObj* pTableCxtHash) { taosHashCleanup(pTableCxtHash); } -static int32_t fillVgroupDataCxt(STableDataCxt* pTableCxt, SVgroupDataCxt* pVgCxt) { +static int32_t fillVgroupDataCxt(STableDataCxt* pTableCxt, SVgroupDataCxt* pVgCxt, bool isRebuild) { if (NULL == pVgCxt->pData->aSubmitTbData) { pVgCxt->pData->aSubmitTbData = taosArrayInit(128, sizeof(SSubmitTbData)); if (NULL == pVgCxt->pData->aSubmitTbData) { return TSDB_CODE_OUT_OF_MEMORY; } } + + // push data to submit, rebuild empty data for next submit taosArrayPush(pVgCxt->pData->aSubmitTbData, pTableCxt->pData); - rebuildTableData(pTableCxt->pData, &pTableCxt->pData); + if (isRebuild) { + rebuildTableData(pTableCxt->pData, &pTableCxt->pData); + } else { + taosMemoryFreeClear(pTableCxt->pData); + } qDebug("add tableDataCxt uid:%" PRId64 " to vgId:%d", pTableCxt->pMeta->uid, pVgCxt->vgId); @@ -467,7 +481,7 @@ int insColDataComp(const void* lp, const void* rp) { return 0; } -int32_t insMergeTableDataCxt(SHashObj* pTableHash, SArray** pVgDataBlocks) { +int32_t insMergeTableDataCxt(SHashObj* pTableHash, SArray** pVgDataBlocks, bool isRebuild) { SHashObj* pVgroupHash = taosHashInit(128, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), true, false); SArray* pVgroupList = taosArrayInit(8, POINTER_BYTES); if (NULL == pVgroupHash || NULL == pVgroupList) { @@ -502,6 +516,13 @@ int32_t insMergeTableDataCxt(SHashObj* pTableHash, SArray** pVgDataBlocks) { tColDataSortMerge(pTableCxt->pData->aCol); } else { + // skip the table has no data to insert + // eg: import a csv without valid data + // if (0 == taosArrayGetSize(pTableCxt->pData->aRowP)) { + // qWarn("no row in tableDataCxt uid:%" PRId64 " ", pTableCxt->pMeta->uid); + // p = taosHashIterate(pTableHash, p); + // continue; + // } if (!pTableCxt->ordered) { code = tRowSort(pTableCxt->pData->aRowP); } @@ -520,7 +541,7 @@ int32_t insMergeTableDataCxt(SHashObj* pTableHash, SArray** pVgDataBlocks) { pVgCxt = *(SVgroupDataCxt**)pp; } if (TSDB_CODE_SUCCESS == code) { - code = fillVgroupDataCxt(pTableCxt, pVgCxt); + code = fillVgroupDataCxt(pTableCxt, pVgCxt, isRebuild); } } if (TSDB_CODE_SUCCESS == code) { diff --git a/tests/parallel_test/cases.task b/tests/parallel_test/cases.task index 7a47df97a9..d039495351 100644 --- a/tests/parallel_test/cases.task +++ b/tests/parallel_test/cases.task @@ -300,6 +300,7 @@ e ,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/precisionUS.py ,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/precisionNS.py ,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/test_ts4219.py +,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/ts-4272.py ,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/test_ts4295.py ,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/test_td27388.py ,,y,system-test,./pytest.sh python3 ./test.py -f 0-others/show.py diff --git a/tests/system-test/1-insert/ts-4272.py b/tests/system-test/1-insert/ts-4272.py new file mode 100644 index 0000000000..4e837d646d --- /dev/null +++ b/tests/system-test/1-insert/ts-4272.py @@ -0,0 +1,158 @@ + +import csv +from datetime import datetime + +import taos +from util.log import * +from util.sql import * +from util.cases import * +from util.dnodes import * +from util.common import * + +class TDTestCase: + def init(self, conn, logSql, replicaVar=1): + self.replicaVar = int(replicaVar) + self.testcasePath = os.path.split(__file__)[0] + self.testcaseFilename = os.path.split(__file__)[-1] + self.ts = 1700638570000 # 2023-11-22T07:36:10.000Z + self.tb1 = 'd001' + self.tb2 = 'd002' + self.tag1 = 'using meters(groupId) tags(1)' + self.tag2 = 'using meters(groupId) tags(2)' + self.file1 = f"{self.testcasePath}/b.csv" + self.file2 = f"{self.testcasePath}/c.csv" + + os.system("rm -rf %s/b.csv" %self.testcasePath) + tdLog.debug(f"start to excute {__file__}") + tdSql.init(conn.cursor(), logSql) + + def check_count(self, rows, records): + tdSql.execute(f"use d1;") + tdSql.query(f"select tbname,count(*) from meters group by tbname order by tbname;") + tdSql.checkRows(rows) + for i in range(rows): + tdSql.checkData(i, 1, records[i]) + + def reset_tb(self): + # create database and tables + # os.system("taos -s 'drop database if exists d1;'") + # os.system("taos -s 'create database d1;use d1;CREATE STABLE meters (ts timestamp, current float, voltage int, phase float) TAGS (location binary(64), groupId int);'") + # os.system(f"taos -s 'use d1;Create table d2001 using meters(groupId) tags(5);'") + # res = os.system(f"taos -s 'use d1;Create table d2002 using meters(groupId) tags(6);'") + # if (0 != res): + # tdLog.exit(f"create tb error") + + tdSql.execute("drop database if exists d1;") + tdSql.execute("create database d1;") + tdSql.execute("use d1;") + tdSql.execute("CREATE STABLE meters (ts timestamp, current float, voltage int, phase float) TAGS (location binary(64), groupId int);") + tdSql.execute("Create table d2001 using meters(groupId) tags(5);") + tdSql.execute("Create table d2002 using meters(groupId) tags(6);") + + def test(self, sql): + sql = "use d1;" + sql + res = os.system(f'taos -s "{sql}"') + # if (0 != res): + # tdLog.exit(f"taos sql error") + + + def check(self): + # same table, auto create + create + sql = f"INSERT INTO {self.tb1} {self.tag1} file '{self.file1}' {self.tb1} {self.tag1} file '{self.file2}';" + self.test(sql) + + # same table, create + insert + sql = f"INSERT INTO {self.tb1} {self.tag1} file '{self.file1}' {self.tb1} file '{self.file2}';" + self.test(sql) + + # same table, insert + create + sql = f"INSERT INTO {self.tb1} file '{self.file1}' {self.tb1} {self.tag1} file '{self.file2}';" + self.test(sql) + + # same table, insert + insert + sql = f"INSERT INTO {self.tb1} file '{self.file1}' {self.tb1} file '{self.file2}';" + self.test(sql) + + # diff table auto create + create + sql = f"INSERT INTO {self.tb1} {self.tag1} file '{self.file1}' {self.tb2} {self.tag2} file '{self.file2}';" + self.test(sql) + + # diff table, create + insert + sql = f"INSERT INTO {self.tb1} {self.tag1} file '{self.file1}' {self.tb2} file '{self.file2}';" + self.test(sql) + + # diff table, insert + create + sql = f"INSERT INTO {self.tb1} file '{self.file1}' {self.tb2} {self.tag2} file '{self.file2}';" + self.test(sql) + + # diff table, insert + insert + sql = f"INSERT INTO {self.tb1} file '{self.file1}' {self.tb2} file '{self.file2}';" + self.test(sql) + + # bigNum = 1010000 + # self.check_count(5, [2100, 2100, bigNum, bigNum, bigNum]) + + result = os.popen("taos -s 'select count(*) from d1.%s'" %self.tb1) + res = result.read() + if (f"OK" in res): + tdLog.info(f"check count success") + + def make_csv(self, filepath, once, qtime, startts): + f = open(filepath, 'w') + with f: + writer = csv.writer(f) + for j in range(qtime): + ts = startts + j*once + rows = [] + for i in range(once): + rows.append([ts + i, 0.3 + (i%10)/100.0, 210 + i%10, 10.0 + (i%20)/20.0]) + writer.writerows(rows) + f.close() + print(datetime.now(), filepath, " ready!") + + def test_mix(self): + #forbid use both value and file in one insert + result = os.popen(f"insert into {self.tb1} file '{self.testcasePath}/csv/2k.csv' {self.tb2} values('2021-07-13 14:06:34.630', 10.2, 219, 0.32);") + res = result.read() + if (f"error" in res): + tdLog.info(f"forbid success") + + def test_bigcsv(self): + # prepare csv + print("start csv data prepare") + once = 10000 + qtime1 = 101 + qtime2 = 100 + rowNum1 = qtime1 * once + rowNum2 = qtime2 * once + self.make_csv(self.file1, once, qtime1, self.ts - 86400000) + self.make_csv(self.file2, once, qtime2, self.ts) + print("end csv data prepare") + + # auto create + insert + sql = f"INSERT INTO d2001 using meters(groupId) tags(5) FILE '{self.file1}';" + self.test(sql) + + # only insert + sql = f"INSERT INTO d2002 FILE '{self.file2}';" + self.test(sql) + + #tdSql.execute(f"use d1;") + tdSql.query(f"select tbname,count(*) from meters group by tbname order by tbname;") + tdSql.checkRows(2) + tdSql.checkData(0, 1, rowNum1) + tdSql.checkData(1, 1, rowNum2) + + def run(self): + tdSql.prepare() + self.reset_tb() + self.test_bigcsv() + self.test_mix() + self.check() + + def stop(self): + tdSql.close() + tdLog.success(f"{__file__} successfully executed") + +tdCases.addLinux(__file__, TDTestCase()) +tdCases.addWindows(__file__, TDTestCase()) From 28b73b415cb38ba37f6d04fc5548d9fd56675022 Mon Sep 17 00:00:00 2001 From: Bob Liu Date: Wed, 29 Nov 2023 02:44:55 +0800 Subject: [PATCH 175/195] fit insert childtable by file --- source/libs/parser/src/parInsertSql.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/libs/parser/src/parInsertSql.c b/source/libs/parser/src/parInsertSql.c index 684314faef..d58eeed7ef 100644 --- a/source/libs/parser/src/parInsertSql.c +++ b/source/libs/parser/src/parInsertSql.c @@ -2033,7 +2033,7 @@ static int32_t parseDataFromFileImpl(SInsertParseContext* pCxt, SVnodeModifyOpSt } // just record pTableCxt whose data come from file - if (numOfRows > 0) { + if (!pStmt->stbSyntax && numOfRows > 0) { if (NULL == pStmt->pTableCxtHashObj) { pStmt->pTableCxtHashObj = taosHashInit(128, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), true, HASH_NO_LOCK); @@ -2302,7 +2302,7 @@ static int32_t parseInsertBodyBottom(SInsertParseContext* pCxt, SVnodeModifyOpSt pStmt->freeArrayFunc(pStmt->pVgDataBlocks); pStmt->pVgDataBlocks = NULL; - bool fileOnly = (pStmt->insertType == TSDB_QUERY_TYPE_FILE_INSERT); + bool fileOnly = (!pStmt->stbSyntax && pStmt->insertType == TSDB_QUERY_TYPE_FILE_INSERT); if (fileOnly) { // none data, skip merge & buildvgdata if (0 == taosHashGetSize(pStmt->pTableCxtHashObj)) { From ade2f322a036047142dd2548b74b20536f70f7f6 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 29 Nov 2023 11:28:03 +0800 Subject: [PATCH 176/195] change default config --- source/common/src/tglobal.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index e8ff530d78..f4d79d6e70 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -248,7 +248,7 @@ int32_t tsTtlBatchDropNum = 10000; // number of tables dropped per batch // internal int32_t tsTransPullupInterval = 2; int32_t tsMqRebalanceInterval = 2; -int32_t tsStreamCheckpointInterval = 10; +int32_t tsStreamCheckpointInterval = 60; float tsSinkDataRate = 2.0; int32_t tsStreamNodeCheckInterval = 15; int32_t tsTtlUnit = 86400; @@ -738,7 +738,7 @@ static int32_t taosAddServerCfg(SConfig *pCfg) { if (cfgAddInt64(pCfg, "streamBufferSize", tsStreamBufferSize, 0, INT64_MAX, CFG_SCOPE_SERVER, CFG_DYN_NONE) != 0) return -1; - if (cfgAddInt64(pCfg, "checkpointInterval", tsStreamCheckpointInterval, 1, 1200, CFG_SCOPE_SERVER, + if (cfgAddInt64(pCfg, "checkpointInterval", tsStreamCheckpointInterval, 60, 1200, CFG_SCOPE_SERVER, CFG_DYN_ENT_SERVER) != 0) return -1; if (cfgAddFloat(pCfg, "streamSinkDataRate", tsSinkDataRate, 0.1, 5, CFG_SCOPE_SERVER, CFG_DYN_NONE) != 0) return -1; From 03b698d8aee80b5a52fea83d9bc4cbf53f7a307e Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 29 Nov 2023 14:50:07 +0800 Subject: [PATCH 177/195] change default config --- source/common/src/tglobal.c | 2 +- source/libs/stream/src/streamSnapshot.c | 6 +++++- tests/system-test/8-stream/snode_restart_with_checkpoint.py | 2 +- 3 files changed, 7 insertions(+), 3 deletions(-) diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index f4d79d6e70..9069f5ce99 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -1477,7 +1477,7 @@ static int32_t taosCfgDynamicOptionsForServer(SConfig *pCfg, char *name) { {"supportVnodes", &tsNumOfSupportVnodes}, }; - if (taosCfgSetOption(debugOptions, tListLen(debugOptions), pItem, true) != 0) { + if (taosCfgSetOption(debugOptions, tListLen(debugOptions), pItem, true) != 0) { taosCfgSetOption(options, tListLen(options), pItem, false); } } diff --git a/source/libs/stream/src/streamSnapshot.c b/source/libs/stream/src/streamSnapshot.c index f76b49f300..e29f2ba7de 100644 --- a/source/libs/stream/src/streamSnapshot.c +++ b/source/libs/stream/src/streamSnapshot.c @@ -366,8 +366,12 @@ int32_t streamSnapRead(SStreamSnapReader* pReader, uint8_t** ppData, int64_t* si int32_t code = 0; SStreamSnapHandle* pHandle = &pReader->handle; int32_t idx = pHandle->currIdx; + SBackendSnapFile2* pSnapFile = taosArrayGet(pHandle->pDbSnapSet, idx); - SBackendFileItem* item = NULL; + if (pSnapFile == NULL) { + return 0; + } + SBackendFileItem* item = NULL; _NEXT: diff --git a/tests/system-test/8-stream/snode_restart_with_checkpoint.py b/tests/system-test/8-stream/snode_restart_with_checkpoint.py index 9567bbe439..d7bfd7b407 100644 --- a/tests/system-test/8-stream/snode_restart_with_checkpoint.py +++ b/tests/system-test/8-stream/snode_restart_with_checkpoint.py @@ -31,7 +31,7 @@ class TDTestCase: tdSql.query("use test") tdSql.query("create snode on dnode 4") tdSql.query("create stream if not exists s1 trigger at_once ignore expired 0 ignore update 0 fill_history 1 into st1 as select _wstart,sum(voltage),groupid from meters partition by groupid interval(2s)") - tdLog.debug("========create stream useing snode and insert data ok========") + tdLog.debug("========create stream using snode and insert data ok========") time.sleep(60) tdDnodes = cluster.dnodes From 1e02f823a42e023c7d439fafc2e1db72030ec83f Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao> Date: Wed, 29 Nov 2023 14:59:22 +0800 Subject: [PATCH 178/195] compact history window --- .../libs/executor/src/streameventwindowoperator.c | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/source/libs/executor/src/streameventwindowoperator.c b/source/libs/executor/src/streameventwindowoperator.c index 8029b9b156..9f1610e08d 100644 --- a/source/libs/executor/src/streameventwindowoperator.c +++ b/source/libs/executor/src/streameventwindowoperator.c @@ -603,13 +603,20 @@ void streamEventReloadState(SOperatorInfo* pOperator) { pSeKeyBuf[i].groupId, i); getSessionWindowInfoByKey(pAggSup, pSeKeyBuf + i, &curInfo.winInfo); setEventWindowFlag(pAggSup, &curInfo); - if (!curInfo.pWinFlag->startFlag || !curInfo.pWinFlag->endFlag) { + if (!curInfo.pWinFlag->startFlag || curInfo.pWinFlag->endFlag) { continue; } compactEventWindow(pOperator, &curInfo, pInfo->pSeUpdated, pInfo->pSeDeleted, false); qDebug("===stream=== reload state. save result %" PRId64 ", %" PRIu64, curInfo.winInfo.sessionWin.win.skey, curInfo.winInfo.sessionWin.groupId); + if (IS_VALID_SESSION_WIN(curInfo.winInfo)) { + saveSessionOutputBuf(pAggSup, &curInfo.winInfo); + } + + if (!curInfo.pWinFlag->endFlag) { + continue; + } if (pInfo->twAggSup.calTrigger == STREAM_TRIGGER_AT_ONCE) { saveResult(curInfo.winInfo, pInfo->pSeUpdated); @@ -621,10 +628,6 @@ void streamEventReloadState(SOperatorInfo* pOperator) { getSessionHashKey(&curInfo.winInfo.sessionWin, &key); tSimpleHashPut(pAggSup->pResultRows, &key, sizeof(SSessionKey), &curInfo.winInfo, sizeof(SResultWindowInfo)); } - - if (IS_VALID_SESSION_WIN(curInfo.winInfo)) { - saveSessionOutputBuf(pAggSup, &curInfo.winInfo); - } } taosMemoryFree(pBuf); From 9d3ce107f278f1130054f81d5f64add6428bd1d2 Mon Sep 17 00:00:00 2001 From: Bob Liu Date: Wed, 29 Nov 2023 15:32:40 +0800 Subject: [PATCH 179/195] support stable insert child batch by file --- source/libs/parser/src/parInsertSql.c | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/source/libs/parser/src/parInsertSql.c b/source/libs/parser/src/parInsertSql.c index d58eeed7ef..24cdf40d31 100644 --- a/source/libs/parser/src/parInsertSql.c +++ b/source/libs/parser/src/parInsertSql.c @@ -1824,6 +1824,10 @@ static int32_t parseOneStbRow(SInsertParseContext* pCxt, SVnodeModifyOpStmt* pSt if (TSDB_CODE_SUCCESS == code) { insCheckTableDataOrder(pTableDataCxt, TD_ROW_KEY(*pRow)); } + + void* pData = pTableDataCxt; + taosHashPut(pStmt->pTableCxtHashObj, &pStbRowsCxt->pCtbMeta->uid, sizeof(pStbRowsCxt->pCtbMeta->uid), &pData, + POINTER_BYTES); } if (code == TSDB_CODE_SUCCESS) { @@ -2016,6 +2020,12 @@ static int32_t parseCsvFile(SInsertParseContext* pCxt, SVnodeModifyOpStmt* pStmt } static int32_t parseDataFromFileImpl(SInsertParseContext* pCxt, SVnodeModifyOpStmt* pStmt, SRowsDataContext rowsDataCxt) { + // init only for file + if (NULL == pStmt->pTableCxtHashObj) { + pStmt->pTableCxtHashObj = + taosHashInit(128, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), true, HASH_NO_LOCK); + } + int32_t numOfRows = 0; int32_t code = parseCsvFile(pCxt, pStmt, rowsDataCxt, &numOfRows); if (TSDB_CODE_SUCCESS == code) { @@ -2034,10 +2044,6 @@ static int32_t parseDataFromFileImpl(SInsertParseContext* pCxt, SVnodeModifyOpSt // just record pTableCxt whose data come from file if (!pStmt->stbSyntax && numOfRows > 0) { - if (NULL == pStmt->pTableCxtHashObj) { - pStmt->pTableCxtHashObj = - taosHashInit(128, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), true, HASH_NO_LOCK); - } void* pData = rowsDataCxt.pTableDataCxt; taosHashPut(pStmt->pTableCxtHashObj, &pStmt->pTableMeta->uid, sizeof(pStmt->pTableMeta->uid), &pData, POINTER_BYTES); @@ -2302,7 +2308,7 @@ static int32_t parseInsertBodyBottom(SInsertParseContext* pCxt, SVnodeModifyOpSt pStmt->freeArrayFunc(pStmt->pVgDataBlocks); pStmt->pVgDataBlocks = NULL; - bool fileOnly = (!pStmt->stbSyntax && pStmt->insertType == TSDB_QUERY_TYPE_FILE_INSERT); + bool fileOnly = (pStmt->insertType == TSDB_QUERY_TYPE_FILE_INSERT); if (fileOnly) { // none data, skip merge & buildvgdata if (0 == taosHashGetSize(pStmt->pTableCxtHashObj)) { From 121197b8977da5136cd9124e1e5ed995c3c540ba Mon Sep 17 00:00:00 2001 From: Hui Li <52318143+plum-lihui@users.noreply.github.com> Date: Wed, 29 Nov 2023 16:02:18 +0800 Subject: [PATCH 180/195] Update set_core.sh --- packaging/tools/set_core.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/packaging/tools/set_core.sh b/packaging/tools/set_core.sh index db95aeb343..084c4465f9 100755 --- a/packaging/tools/set_core.sh +++ b/packaging/tools/set_core.sh @@ -38,3 +38,4 @@ source /etc/profile ${csudo}mkdir -p ${corePath} ||: ${csudo}sysctl -w kernel.core_pattern=${corePath}/core-%e-%p ||: ${csudo}echo "${corePath}/core-%e-%p" | ${csudo}tee /proc/sys/kernel/core_pattern ||: +${csudo}echo "kernel.core_pattern = ${corePath}/core_%e-%p" >> /etc/sysctl.conf ||: From 2e1192b97bf4c62863d5c70822f3cef8a8670624 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 29 Nov 2023 16:16:26 +0800 Subject: [PATCH 181/195] change default config --- source/common/src/tglobal.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index 9069f5ce99..ced68c134e 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -738,7 +738,7 @@ static int32_t taosAddServerCfg(SConfig *pCfg) { if (cfgAddInt64(pCfg, "streamBufferSize", tsStreamBufferSize, 0, INT64_MAX, CFG_SCOPE_SERVER, CFG_DYN_NONE) != 0) return -1; - if (cfgAddInt64(pCfg, "checkpointInterval", tsStreamCheckpointInterval, 60, 1200, CFG_SCOPE_SERVER, + if (cfgAddInt32(pCfg, "checkpointInterval", tsStreamCheckpointInterval, 60, 1200, CFG_SCOPE_SERVER, CFG_DYN_ENT_SERVER) != 0) return -1; if (cfgAddFloat(pCfg, "streamSinkDataRate", tsSinkDataRate, 0.1, 5, CFG_SCOPE_SERVER, CFG_DYN_NONE) != 0) return -1; @@ -1477,7 +1477,7 @@ static int32_t taosCfgDynamicOptionsForServer(SConfig *pCfg, char *name) { {"supportVnodes", &tsNumOfSupportVnodes}, }; - if (taosCfgSetOption(debugOptions, tListLen(debugOptions), pItem, true) != 0) { + if (taosCfgSetOption(debugOptions, tListLen(debugOptions), pItem, true) != 0) { taosCfgSetOption(options, tListLen(options), pItem, false); } } From b6881d572aff50811ca4a0a670306efcfc640a5f Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 29 Nov 2023 16:52:36 +0800 Subject: [PATCH 182/195] fix compile error on windows/mac --- source/common/src/cos.c | 1 + 1 file changed, 1 insertion(+) diff --git a/source/common/src/cos.c b/source/common/src/cos.c index 65b0a61590..7c8676e9f5 100644 --- a/source/common/src/cos.c +++ b/source/common/src/cos.c @@ -1355,5 +1355,6 @@ int32_t s3GetObjectBlock(const char *object_name, int64_t offset, int64_t size, void s3EvictCache(const char *path, long object_size) {} long s3Size(const char *object_name) { return 0; } int32_t s3GetObjectsByPrefix(const char *prefix, const char *path) { return 0; } +int32_t s3GetObjectToFile(const char *object_name, char *fileName) { return 0; } #endif From a01ccf8bc96b390120501ad83415dcaf1c62b3ed Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 29 Nov 2023 17:14:37 +0800 Subject: [PATCH 183/195] snode case forbidden --- tests/parallel_test/cases.task | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/parallel_test/cases.task b/tests/parallel_test/cases.task index 8cb23c081f..eea0904e32 100644 --- a/tests/parallel_test/cases.task +++ b/tests/parallel_test/cases.task @@ -22,7 +22,7 @@ ,,y,system-test,./pytest.sh python3 ./test.py -f 8-stream/pause_resume_test.py #,,n,system-test,python3 ./test.py -f 8-stream/vnode_restart.py -N 4 #,,n,system-test,python3 ./test.py -f 8-stream/snode_restart.py -N 4 -,,n,system-test,python3 ./test.py -f 8-stream/snode_restart_with_checkpoint.py -N 4 +#,,n,system-test,python3 ./test.py -f 8-stream/snode_restart_with_checkpoint.py -N 4 ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/tbname_vgroup.py ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/stbJoin.py From 2c9c1175295cc1e432f72e8ea23725dd176ec705 Mon Sep 17 00:00:00 2001 From: Minglei Jin Date: Wed, 29 Nov 2023 17:37:40 +0800 Subject: [PATCH 184/195] fix(tsdb/cache): flush dirty lru entries before del --- source/dnode/vnode/src/tsdb/tsdbCache.c | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/source/dnode/vnode/src/tsdb/tsdbCache.c b/source/dnode/vnode/src/tsdb/tsdbCache.c index 918d0bd7d0..e8aa755073 100644 --- a/source/dnode/vnode/src/tsdb/tsdbCache.c +++ b/source/dnode/vnode/src/tsdb/tsdbCache.c @@ -1132,6 +1132,10 @@ int32_t tsdbCacheDel(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSKEY sKey, TSKE size_t *values_list_sizes = taosMemoryCalloc(num_keys * 2, sizeof(size_t)); char **errs = taosMemoryCalloc(num_keys * 2, sizeof(char *)); taosThreadMutexLock(&pTsdb->lruMutex); + + pTsdb->flushState.flush_count = ROCKS_BATCH_SIZE; + taosLRUCacheApply(pTsdb->lruCache, tsdbCacheFlushDirty, &pTsdb->flushState); + taosThreadMutexLock(&pTsdb->rCache.rMutex); rocksMayWrite(pTsdb, true, false, false); rocksdb_multi_get(pTsdb->rCache.db, pTsdb->rCache.readoptions, num_keys * 2, (const char *const *)keys_list, From 65ccc4aa48989e0878486af8bfaee944bb527be9 Mon Sep 17 00:00:00 2001 From: Hongze Cheng Date: Wed, 29 Nov 2023 18:01:31 +0800 Subject: [PATCH 185/195] feat: backgroud task priority --- source/dnode/vnode/CMakeLists.txt | 2 + source/dnode/vnode/src/inc/tsdb.h | 7 +- source/dnode/vnode/src/inc/vnd.h | 29 +- source/dnode/vnode/src/inc/vnodeInt.h | 31 +- source/dnode/vnode/src/tsdb/tsdbFS2.c | 195 +----- source/dnode/vnode/src/tsdb/tsdbFS2.h | 5 - source/dnode/vnode/src/tsdb/tsdbFSet2.c | 11 +- source/dnode/vnode/src/tsdb/tsdbFSet2.h | 41 +- source/dnode/vnode/src/tsdb/tsdbMerge.c | 22 +- source/dnode/vnode/src/tsdb/tsdbRetention.c | 84 +-- source/dnode/vnode/src/tsdb/tsdbSnapshot.c | 7 +- source/dnode/vnode/src/vnd/vnodeAsync.c | 719 ++++++++++++++++++++ source/dnode/vnode/src/vnd/vnodeCommit.c | 27 +- source/dnode/vnode/src/vnd/vnodeHash.c | 162 +++++ source/dnode/vnode/src/vnd/vnodeHash.h | 37 + source/dnode/vnode/src/vnd/vnodeModule.c | 150 +--- source/dnode/vnode/src/vnd/vnodeOpen.c | 16 +- source/dnode/vnode/src/vnd/vnodeSnapshot.c | 23 +- 18 files changed, 1093 insertions(+), 475 deletions(-) create mode 100644 source/dnode/vnode/src/vnd/vnodeAsync.c create mode 100644 source/dnode/vnode/src/vnd/vnodeHash.c create mode 100644 source/dnode/vnode/src/vnd/vnodeHash.h diff --git a/source/dnode/vnode/CMakeLists.txt b/source/dnode/vnode/CMakeLists.txt index 9aeb14cd60..114051f02b 100644 --- a/source/dnode/vnode/CMakeLists.txt +++ b/source/dnode/vnode/CMakeLists.txt @@ -14,6 +14,8 @@ set( "src/vnd/vnodeSnapshot.c" "src/vnd/vnodeRetention.c" "src/vnd/vnodeInitApi.c" + "src/vnd/vnodeAsync.c" + "src/vnd/vnodeHash.c" # meta "src/meta/metaOpen.c" diff --git a/source/dnode/vnode/src/inc/tsdb.h b/source/dnode/vnode/src/inc/tsdb.h index ca9d22a987..88362239f5 100644 --- a/source/dnode/vnode/src/inc/tsdb.h +++ b/source/dnode/vnode/src/inc/tsdb.h @@ -309,7 +309,12 @@ int32_t tsdbTakeReadSnap2(STsdbReader *pReader, _query_reseek_func_t reseek, STs void tsdbUntakeReadSnap2(STsdbReader *pReader, STsdbReadSnap *pSnap, bool proactive); // tsdbMerge.c ============================================================================================== -int32_t tsdbSchedMerge(STsdb *tsdb, int32_t fid); +typedef struct { + STsdb *tsdb; + int32_t fid; +} SMergeArg; + +int32_t tsdbMerge(void *arg); // tsdbDiskData ============================================================================================== int32_t tDiskDataBuilderCreate(SDiskDataBuilder **ppBuilder); diff --git a/source/dnode/vnode/src/inc/vnd.h b/source/dnode/vnode/src/inc/vnd.h index 55b62dfe48..4036200d73 100644 --- a/source/dnode/vnode/src/inc/vnd.h +++ b/source/dnode/vnode/src/inc/vnd.h @@ -48,9 +48,32 @@ int32_t vnodeCheckCfg(const SVnodeCfg*); int32_t vnodeEncodeConfig(const void* pObj, SJson* pJson); int32_t vnodeDecodeConfig(const SJson* pJson, void* pObj); +// vnodeAsync.c +typedef struct SVAsync SVAsync; + +typedef enum { + EVA_PRIORITY_HIGH = 0, + EVA_PRIORITY_NORMAL, + EVA_PRIORITY_LOW, +} EVAPriority; + +#define VNODE_ASYNC_VALID_CHANNEL_ID(channelId) ((channelId) > 0) +#define VNODE_ASYNC_VALID_TASK_ID(taskId) ((taskId) > 0) + +int32_t vnodeAsyncInit(SVAsync** async, char* label); +int32_t vnodeAsyncDestroy(SVAsync** async); +int32_t vnodeAChannelInit(SVAsync* async, int64_t* channelId); +int32_t vnodeAChannelDestroy(SVAsync* async, int64_t channelId, bool waitRunning); +int32_t vnodeAsync(SVAsync* async, EVAPriority priority, int32_t (*execute)(void*), void (*complete)(void*), void* arg, + int64_t* taskId); +int32_t vnodeAsyncC(SVAsync* async, int64_t channelId, EVAPriority priority, int32_t (*execute)(void*), + void (*complete)(void*), void* arg, int64_t* taskId); +int32_t vnodeAWait(SVAsync* async, int64_t taskId); +int32_t vnodeACancel(SVAsync* async, int64_t taskId); +int32_t vnodeAsyncSetWorkers(SVAsync* async, int32_t numWorkers); + // vnodeModule.c -int vnodeScheduleTask(int (*execute)(void*), void* arg); -int vnodeScheduleTaskEx(int tpid, int (*execute)(void*), void* arg); +extern SVAsync* vnodeAsyncHandle[2]; // vnodeBufPool.c typedef struct SVBufPoolNode SVBufPoolNode; @@ -110,7 +133,7 @@ int32_t vnodeAsyncCommit(SVnode* pVnode); bool vnodeShouldRollback(SVnode* pVnode); // vnodeSync.c -int32_t vnodeSyncOpen(SVnode *pVnode, char *path, int32_t vnodeVersion); +int32_t vnodeSyncOpen(SVnode* pVnode, char* path, int32_t vnodeVersion); int32_t vnodeSyncStart(SVnode* pVnode); void vnodeSyncPreClose(SVnode* pVnode); void vnodeSyncPostClose(SVnode* pVnode); diff --git a/source/dnode/vnode/src/inc/vnodeInt.h b/source/dnode/vnode/src/inc/vnodeInt.h index 473329bfee..3a40629ff7 100644 --- a/source/dnode/vnode/src/inc/vnodeInt.h +++ b/source/dnode/vnode/src/inc/vnodeInt.h @@ -209,7 +209,7 @@ int32_t tsdbBegin(STsdb* pTsdb); // int32_t tsdbCommit(STsdb* pTsdb, SCommitInfo* pInfo); int32_t tsdbCacheCommit(STsdb* pTsdb); int32_t tsdbCompact(STsdb* pTsdb, SCompactInfo* pInfo); -int32_t tsdbRetention(STsdb *tsdb, int64_t now, int32_t sync); +int32_t tsdbRetention(STsdb* tsdb, int64_t now, int32_t sync); // int32_t tsdbFinishCommit(STsdb* pTsdb); // int32_t tsdbRollbackCommit(STsdb* pTsdb); int tsdbScanAndConvertSubmitMsg(STsdb* pTsdb, SSubmitReq2* pMsg); @@ -448,13 +448,16 @@ struct SVnode { SVBufPool* recycleTail; SVBufPool* onRecycle; + // commit variables + int64_t commitChannel; + int64_t commitTask; + SMeta* pMeta; SSma* pSma; STsdb* pTsdb; SWal* pWal; STQ* pTq; SSink* pSink; - tsem_t canCommit; int64_t sync; TdThreadMutex lock; bool blocked; @@ -494,18 +497,18 @@ struct SSma { void* pRSmaEnv; }; -#define SMA_CFG(s) (&(s)->pVnode->config) -#define SMA_TSDB_CFG(s) (&(s)->pVnode->config.tsdbCfg) -#define SMA_RETENTION(s) ((SRetention*)&(s)->pVnode->config.tsdbCfg.retentions) -#define SMA_LOCKED(s) ((s)->locked) -#define SMA_META(s) ((s)->pVnode->pMeta) -#define SMA_VID(s) TD_VID((s)->pVnode) -#define SMA_TFS(s) ((s)->pVnode->pTfs) -#define SMA_TSMA_ENV(s) ((s)->pTSmaEnv) -#define SMA_RSMA_ENV(s) ((s)->pRSmaEnv) -#define SMA_RSMA_TSDB0(s) ((s)->pVnode->pTsdb) -#define SMA_RSMA_TSDB1(s) ((s)->pRSmaTsdb[TSDB_RETENTION_L0]) -#define SMA_RSMA_TSDB2(s) ((s)->pRSmaTsdb[TSDB_RETENTION_L1]) +#define SMA_CFG(s) (&(s)->pVnode->config) +#define SMA_TSDB_CFG(s) (&(s)->pVnode->config.tsdbCfg) +#define SMA_RETENTION(s) ((SRetention*)&(s)->pVnode->config.tsdbCfg.retentions) +#define SMA_LOCKED(s) ((s)->locked) +#define SMA_META(s) ((s)->pVnode->pMeta) +#define SMA_VID(s) TD_VID((s)->pVnode) +#define SMA_TFS(s) ((s)->pVnode->pTfs) +#define SMA_TSMA_ENV(s) ((s)->pTSmaEnv) +#define SMA_RSMA_ENV(s) ((s)->pRSmaEnv) +#define SMA_RSMA_TSDB0(s) ((s)->pVnode->pTsdb) +#define SMA_RSMA_TSDB1(s) ((s)->pRSmaTsdb[TSDB_RETENTION_L0]) +#define SMA_RSMA_TSDB2(s) ((s)->pRSmaTsdb[TSDB_RETENTION_L1]) #define SMA_RSMA_GET_TSDB(pVnode, level) ((level == 0) ? pVnode->pTsdb : pVnode->pSma->pRSmaTsdb[level - 1]) // sma diff --git a/source/dnode/vnode/src/tsdb/tsdbFS2.c b/source/dnode/vnode/src/tsdb/tsdbFS2.c index 89bed6b42f..add8da52e0 100644 --- a/source/dnode/vnode/src/tsdb/tsdbFS2.c +++ b/source/dnode/vnode/src/tsdb/tsdbFS2.c @@ -20,8 +20,6 @@ #define BLOCK_COMMIT_FACTOR 3 -extern int vnodeScheduleTask(int (*execute)(void *), void *arg); -extern int vnodeScheduleTaskEx(int tpid, int (*execute)(void *), void *arg); extern void remove_file(const char *fname, bool last_level); #define TSDB_FS_EDIT_MIN TSDB_FEDIT_COMMIT @@ -651,7 +649,6 @@ _exit: static int32_t close_file_system(STFileSystem *fs) { TARRAY2_CLEAR(fs->fSetArr, tsdbTFileSetClear); TARRAY2_CLEAR(fs->fSetArrTmp, tsdbTFileSetClear); - // TODO return 0; } @@ -748,36 +745,31 @@ _exit: return code; } -static void tsdbDoWaitBgTask(STFileSystem *fs, STFSBgTask *task) { - task->numWait++; - taosThreadCondWait(task->done, &fs->tsdb->mutex); - task->numWait--; +int32_t tsdbFSCancelAllBgTask(STFileSystem *fs) { + TARRAY2(int64_t) channelArr = {0}; - if (task->numWait == 0) { - taosThreadCondDestroy(task->done); - if (task->destroy) { - task->destroy(task->arg); + // collect all open channels + taosThreadMutexLock(&fs->tsdb->mutex); + STFileSet *fset; + TARRAY2_FOREACH(fs->fSetArr, fset) { + if (VNODE_ASYNC_VALID_CHANNEL_ID(fset->bgTaskChannel)) { + TARRAY2_APPEND(&channelArr, fset->bgTaskChannel); + fset->bgTaskChannel = 0; } - taosMemoryFree(task); } -} + taosThreadMutexUnlock(&fs->tsdb->mutex); -static void tsdbDoDoneBgTask(STFileSystem *fs, STFSBgTask *task) { - if (task->numWait > 0) { - taosThreadCondBroadcast(task->done); - } else { - taosThreadCondDestroy(task->done); - if (task->destroy) { - task->destroy(task->arg); - } - taosMemoryFree(task); - } + // destroy all channels + int64_t channel; + TARRAY2_FOREACH(&channelArr, channel) { vnodeAChannelDestroy(vnodeAsyncHandle[1], channel, true); } + TARRAY2_DESTROY(&channelArr, NULL); + return 0; } int32_t tsdbCloseFS(STFileSystem **fs) { if (fs[0] == NULL) return 0; - tsdbFSDisableBgTask(fs[0]); + tsdbFSCancelAllBgTask(*fs); close_file_system(fs[0]); destroy_fs(fs); return 0; @@ -910,7 +902,20 @@ int32_t tsdbFSEditCommit(STFileSystem *fs) { } if (!skipMerge) { - code = tsdbSchedMerge(fs->tsdb, fset->fid); + code = tsdbTFileSetOpenChannel(fset); + TSDB_CHECK_CODE(code, lino, _exit); + + SMergeArg *arg = taosMemoryMalloc(sizeof(*arg)); + if (arg == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + TSDB_CHECK_CODE(code, lino, _exit); + } + + arg->tsdb = fs->tsdb; + arg->fid = fset->fid; + + code = vnodeAsyncC(vnodeAsyncHandle[1], fset->bgTaskChannel, EVA_PRIORITY_HIGH, tsdbMerge, taosMemoryFree, + arg, NULL); TSDB_CHECK_CODE(code, lino, _exit); } } @@ -939,7 +944,11 @@ int32_t tsdbFSEditCommit(STFileSystem *fs) { } } - if (tsdbTFileSetIsEmpty(fset) && fset->bgTaskRunning == NULL) { + if (tsdbTFileSetIsEmpty(fset)) { + if (VNODE_ASYNC_VALID_CHANNEL_ID(fset->bgTaskChannel)) { + vnodeAChannelDestroy(vnodeAsyncHandle[1], fset->bgTaskChannel, false); + fset->bgTaskChannel = 0; + } TARRAY2_REMOVE(fs->fSetArr, i, tsdbTFileSetClear); } else { i++; @@ -1179,136 +1188,4 @@ _out: pHash = NULL; } return code; -} - -const char *gFSBgTaskName[] = {NULL, "MERGE", "RETENTION", "COMPACT"}; - -static int32_t tsdbFSRunBgTask(void *arg) { - STFSBgTask *task = (STFSBgTask *)arg; - STFileSystem *fs = task->fs; - - task->launchTime = taosGetTimestampMs(); - task->run(task->arg); - task->finishTime = taosGetTimestampMs(); - - tsdbDebug("vgId:%d bg task:%s task id:%" PRId64 " finished, schedule time:%" PRId64 " launch time:%" PRId64 - " finish time:%" PRId64, - TD_VID(fs->tsdb->pVnode), gFSBgTaskName[task->type], task->taskid, task->scheduleTime, task->launchTime, - task->finishTime); - - taosThreadMutexLock(&fs->tsdb->mutex); - - STFileSet *fset = NULL; - tsdbFSGetFSet(fs, task->fid, &fset); - ASSERT(fset != NULL && fset->bgTaskRunning == task); - - // free last - tsdbDoDoneBgTask(fs, task); - fset->bgTaskRunning = NULL; - - // schedule next - if (fset->bgTaskNum > 0) { - if (fs->stop) { - while (fset->bgTaskNum > 0) { - STFSBgTask *nextTask = fset->bgTaskQueue->next; - nextTask->prev->next = nextTask->next; - nextTask->next->prev = nextTask->prev; - fset->bgTaskNum--; - tsdbDoDoneBgTask(fs, nextTask); - } - } else { - // pop task from head - fset->bgTaskRunning = fset->bgTaskQueue->next; - fset->bgTaskRunning->prev->next = fset->bgTaskRunning->next; - fset->bgTaskRunning->next->prev = fset->bgTaskRunning->prev; - fset->bgTaskNum--; - vnodeScheduleTaskEx(1, tsdbFSRunBgTask, fset->bgTaskRunning); - } - } - - taosThreadMutexUnlock(&fs->tsdb->mutex); - return 0; -} - -// IMPORTANT: the caller must hold the fs->tsdb->mutex -int32_t tsdbFSScheduleBgTask(STFileSystem *fs, int32_t fid, EFSBgTaskT type, int32_t (*run)(void *), - void (*destroy)(void *), void *arg, int64_t *taskid) { - if (fs->stop) { - if (destroy) { - destroy(arg); - } - return 0; - } - - STFileSet *fset; - tsdbFSGetFSet(fs, fid, &fset); - - ASSERT(fset != NULL); - - for (STFSBgTask *task = fset->bgTaskQueue->next; task != fset->bgTaskQueue; task = task->next) { - if (task->type == type) { - if (destroy) { - destroy(arg); - } - return 0; - } - } - - // do schedule task - STFSBgTask *task = taosMemoryCalloc(1, sizeof(STFSBgTask)); - if (task == NULL) return TSDB_CODE_OUT_OF_MEMORY; - taosThreadCondInit(task->done, NULL); - - task->fs = fs; - task->fid = fid; - task->type = type; - task->run = run; - task->destroy = destroy; - task->arg = arg; - task->scheduleTime = taosGetTimestampMs(); - task->taskid = ++fs->taskid; - - if (fset->bgTaskRunning == NULL && fset->bgTaskNum == 0) { - // launch task directly - fset->bgTaskRunning = task; - vnodeScheduleTaskEx(1, tsdbFSRunBgTask, task); - } else { - // add to the queue tail - fset->bgTaskNum++; - task->next = fset->bgTaskQueue; - task->prev = fset->bgTaskQueue->prev; - task->prev->next = task; - task->next->prev = task; - } - - if (taskid) *taskid = task->taskid; - return 0; -} - -int32_t tsdbFSDisableBgTask(STFileSystem *fs) { - taosThreadMutexLock(&fs->tsdb->mutex); - for (;;) { - fs->stop = true; - bool done = true; - - STFileSet *fset; - TARRAY2_FOREACH(fs->fSetArr, fset) { - if (fset->bgTaskRunning) { - tsdbDoWaitBgTask(fs, fset->bgTaskRunning); - done = false; - break; - } - } - - if (done) break; - } - taosThreadMutexUnlock(&fs->tsdb->mutex); - return 0; -} - -int32_t tsdbFSEnableBgTask(STFileSystem *fs) { - taosThreadMutexLock(&fs->tsdb->mutex); - fs->stop = false; - taosThreadMutexUnlock(&fs->tsdb->mutex); - return 0; -} +} \ No newline at end of file diff --git a/source/dnode/vnode/src/tsdb/tsdbFS2.h b/source/dnode/vnode/src/tsdb/tsdbFS2.h index a3a8e2f575..74453126cf 100644 --- a/source/dnode/vnode/src/tsdb/tsdbFS2.h +++ b/source/dnode/vnode/src/tsdb/tsdbFS2.h @@ -55,11 +55,6 @@ int64_t tsdbFSAllocEid(STFileSystem *fs); int32_t tsdbFSEditBegin(STFileSystem *fs, const TFileOpArray *opArray, EFEditT etype); int32_t tsdbFSEditCommit(STFileSystem *fs); int32_t tsdbFSEditAbort(STFileSystem *fs); -// background task -int32_t tsdbFSScheduleBgTask(STFileSystem *fs, int32_t fid, EFSBgTaskT type, int32_t (*run)(void *), - void (*destroy)(void *), void *arg, int64_t *taskid); -int32_t tsdbFSDisableBgTask(STFileSystem *fs); -int32_t tsdbFSEnableBgTask(STFileSystem *fs); // other int32_t tsdbFSGetFSet(STFileSystem *fs, int32_t fid, STFileSet **fset); int32_t tsdbFSCheckCommit(STsdb *tsdb, int32_t fid); diff --git a/source/dnode/vnode/src/tsdb/tsdbFSet2.c b/source/dnode/vnode/src/tsdb/tsdbFSet2.c index 61bedcb996..025671ff3d 100644 --- a/source/dnode/vnode/src/tsdb/tsdbFSet2.c +++ b/source/dnode/vnode/src/tsdb/tsdbFSet2.c @@ -14,6 +14,7 @@ */ #include "tsdbFSet2.h" +#include "vnd.h" int32_t tsdbSttLvlInit(int32_t level, SSttLvl **lvl) { if (!(lvl[0] = taosMemoryMalloc(sizeof(SSttLvl)))) return TSDB_CODE_OUT_OF_MEMORY; @@ -451,10 +452,7 @@ int32_t tsdbTFileSetInit(int32_t fid, STFileSet **fset) { TARRAY2_INIT(fset[0]->lvlArr); // background task queue - fset[0]->bgTaskNum = 0; - fset[0]->bgTaskQueue->next = fset[0]->bgTaskQueue; - fset[0]->bgTaskQueue->prev = fset[0]->bgTaskQueue; - fset[0]->bgTaskRunning = NULL; + fset[0]->bgTaskChannel = 0; // block commit variables taosThreadCondInit(&fset[0]->canCommit, NULL); @@ -650,3 +648,8 @@ bool tsdbTFileSetIsEmpty(const STFileSet *fset) { } return TARRAY2_SIZE(fset->lvlArr) == 0; } + +int32_t tsdbTFileSetOpenChannel(STFileSet *fset) { + if (VNODE_ASYNC_VALID_CHANNEL_ID(fset->bgTaskChannel)) return 0; + return vnodeAChannelInit(vnodeAsyncHandle[1], &fset->bgTaskChannel); +} \ No newline at end of file diff --git a/source/dnode/vnode/src/tsdb/tsdbFSet2.h b/source/dnode/vnode/src/tsdb/tsdbFSet2.h index 34f174ade7..32028db352 100644 --- a/source/dnode/vnode/src/tsdb/tsdbFSet2.h +++ b/source/dnode/vnode/src/tsdb/tsdbFSet2.h @@ -22,14 +22,12 @@ extern "C" { #endif -typedef struct STFileSet STFileSet; -typedef struct STFileOp STFileOp; -typedef struct SSttLvl SSttLvl; +typedef struct STFileOp STFileOp; +typedef struct SSttLvl SSttLvl; typedef TARRAY2(STFileObj *) TFileObjArray; typedef TARRAY2(SSttLvl *) TSttLvlArray; typedef TARRAY2(STFileOp) TFileOpArray; typedef struct STFileSystem STFileSystem; -typedef struct STFSBgTask STFSBgTask; typedef enum { TSDB_FOP_NONE = 0, @@ -72,33 +70,8 @@ bool tsdbTFileSetIsEmpty(const STFileSet *fset); // stt int32_t tsdbSttLvlInit(int32_t level, SSttLvl **lvl); int32_t tsdbSttLvlClear(SSttLvl **lvl); - -typedef enum { - TSDB_BG_TASK_MERGER = 1, - TSDB_BG_TASK_RETENTION, - TSDB_BG_TASK_COMPACT, -} EFSBgTaskT; - -struct STFSBgTask { - STFileSystem *fs; - int32_t fid; - - EFSBgTaskT type; - int32_t (*run)(void *arg); - void (*destroy)(void *arg); - void *arg; - - TdThreadCond done[1]; - int32_t numWait; - - int64_t taskid; - int64_t scheduleTime; - int64_t launchTime; - int64_t finishTime; - - struct STFSBgTask *prev; - struct STFSBgTask *next; -}; +// open channel +int32_t tsdbTFileSetOpenChannel(STFileSet *fset); struct STFileOp { tsdb_fop_t optype; @@ -118,10 +91,8 @@ struct STFileSet { STFileObj *farr[TSDB_FTYPE_MAX]; // file array TSttLvlArray lvlArr[1]; // level array - // background task queue - int32_t bgTaskNum; - STFSBgTask bgTaskQueue[1]; - STFSBgTask *bgTaskRunning; + // background task channel + int64_t bgTaskChannel; // block commit variables TdThreadCond canCommit; diff --git a/source/dnode/vnode/src/tsdb/tsdbMerge.c b/source/dnode/vnode/src/tsdb/tsdbMerge.c index 6d968d0828..b47b951b2b 100644 --- a/source/dnode/vnode/src/tsdb/tsdbMerge.c +++ b/source/dnode/vnode/src/tsdb/tsdbMerge.c @@ -17,11 +17,6 @@ #define TSDB_MAX_LEVEL 2 // means max level is 3 -typedef struct { - STsdb *tsdb; - int32_t fid; -} SMergeArg; - typedef struct { STsdb *tsdb; int32_t fid; @@ -528,7 +523,7 @@ static int32_t tsdbMergeGetFSet(SMerger *merger) { return 0; } -static int32_t tsdbMerge(void *arg) { +int32_t tsdbMerge(void *arg) { int32_t code = 0; int32_t lino = 0; SMergeArg *mergeArg = (SMergeArg *)arg; @@ -597,18 +592,3 @@ _exit: tsdbTFileSetClear(&merger->fset); return code; } - -int32_t tsdbSchedMerge(STsdb *tsdb, int32_t fid) { - SMergeArg *arg = taosMemoryMalloc(sizeof(*arg)); - if (arg == NULL) { - return TSDB_CODE_OUT_OF_MEMORY; - } - - arg->tsdb = tsdb; - arg->fid = fid; - - int32_t code = tsdbFSScheduleBgTask(tsdb->pFS, fid, TSDB_BG_TASK_MERGER, tsdbMerge, taosMemoryFree, arg, NULL); - if (code) taosMemoryFree(arg); - - return code; -} diff --git a/source/dnode/vnode/src/tsdb/tsdbRetention.c b/source/dnode/vnode/src/tsdb/tsdbRetention.c index f6888ba9cb..d8f1ad7c6c 100644 --- a/source/dnode/vnode/src/tsdb/tsdbRetention.c +++ b/source/dnode/vnode/src/tsdb/tsdbRetention.c @@ -249,7 +249,7 @@ _exit: if (code) { TSDB_ERROR_LOG(TD_VID(rtner->tsdb->pVnode), lino, code); } else { - tsdbInfo("vid:%d, cid:%" PRId64 ", %s done", TD_VID(rtner->tsdb->pVnode), rtner->cid, __func__); + tsdbDebug("vid:%d, cid:%" PRId64 ", %s done", TD_VID(rtner->tsdb->pVnode), rtner->cid, __func__); } return code; } @@ -279,7 +279,7 @@ _exit: if (code) { TSDB_ERROR_LOG(TD_VID(rtner->tsdb->pVnode), lino, code); } else { - tsdbInfo("vid:%d, cid:%" PRId64 ", %s done", TD_VID(rtner->tsdb->pVnode), rtner->cid, __func__); + tsdbDebug("vid:%d, cid:%" PRId64 ", %s done", TD_VID(rtner->tsdb->pVnode), rtner->cid, __func__); } tsdbFSDestroyCopySnapshot(&rtner->fsetArr); return code; @@ -391,32 +391,6 @@ _exit: static void tsdbFreeRtnArg(void *arg) { taosMemoryFree(arg); } -static int32_t tsdbDoRetentionSync(void *arg) { - int32_t code = 0; - int32_t lino = 0; - SRTNer rtner[1] = {0}; - - code = tsdbDoRetentionBegin(arg, rtner); - TSDB_CHECK_CODE(code, lino, _exit); - - STFileSet *fset; - TARRAY2_FOREACH(rtner->fsetArr, fset) { - code = tsdbDoRetentionOnFileSet(rtner, fset); - TSDB_CHECK_CODE(code, lino, _exit); - } - - code = tsdbDoRetentionEnd(rtner); - TSDB_CHECK_CODE(code, lino, _exit); - -_exit: - if (code) { - TSDB_ERROR_LOG(TD_VID(rtner->tsdb->pVnode), lino, code); - } - tsem_post(&((SRtnArg *)arg)->tsdb->pVnode->canCommit); - tsdbFreeRtnArg(arg); - return code; -} - static int32_t tsdbDoRetentionAsync(void *arg) { int32_t code = 0; int32_t lino = 0; @@ -454,49 +428,41 @@ _exit: int32_t tsdbRetention(STsdb *tsdb, int64_t now, int32_t sync) { int32_t code = 0; - if (sync) { // sync retention + taosThreadMutexLock(&tsdb->mutex); + + STFileSet *fset; + TARRAY2_FOREACH(tsdb->pFS->fSetArr, fset) { + code = tsdbTFileSetOpenChannel(fset); + if (code) { + taosThreadMutexUnlock(&tsdb->mutex); + return code; + } + SRtnArg *arg = taosMemoryMalloc(sizeof(*arg)); if (arg == NULL) { + taosThreadMutexUnlock(&tsdb->mutex); return TSDB_CODE_OUT_OF_MEMORY; } arg->tsdb = tsdb; arg->now = now; - arg->fid = INT32_MAX; + arg->fid = fset->fid; - tsem_wait(&tsdb->pVnode->canCommit); - code = vnodeScheduleTask(tsdbDoRetentionSync, arg); + if (sync) { + code = vnodeAsyncC(vnodeAsyncHandle[0], tsdb->pVnode->commitChannel, EVA_PRIORITY_LOW, tsdbDoRetentionAsync, + tsdbFreeRtnArg, arg, NULL); + } else { + code = vnodeAsyncC(vnodeAsyncHandle[1], fset->bgTaskChannel, EVA_PRIORITY_LOW, tsdbDoRetentionAsync, + tsdbFreeRtnArg, arg, NULL); + } if (code) { - tsem_post(&tsdb->pVnode->canCommit); - taosMemoryFree(arg); + tsdbFreeRtnArg(arg); + taosThreadMutexUnlock(&tsdb->mutex); return code; } - } else { // async retention - taosThreadMutexLock(&tsdb->mutex); - - STFileSet *fset; - TARRAY2_FOREACH(tsdb->pFS->fSetArr, fset) { - SRtnArg *arg = taosMemoryMalloc(sizeof(*arg)); - if (arg == NULL) { - taosThreadMutexUnlock(&tsdb->mutex); - return TSDB_CODE_OUT_OF_MEMORY; - } - - arg->tsdb = tsdb; - arg->now = now; - arg->fid = fset->fid; - - code = tsdbFSScheduleBgTask(tsdb->pFS, fset->fid, TSDB_BG_TASK_RETENTION, tsdbDoRetentionAsync, tsdbFreeRtnArg, - arg, NULL); - if (code) { - tsdbFreeRtnArg(arg); - taosThreadMutexUnlock(&tsdb->mutex); - return code; - } - } - - taosThreadMutexUnlock(&tsdb->mutex); } + taosThreadMutexUnlock(&tsdb->mutex); + return code; } diff --git a/source/dnode/vnode/src/tsdb/tsdbSnapshot.c b/source/dnode/vnode/src/tsdb/tsdbSnapshot.c index a9da0fbcec..104c9b2f35 100644 --- a/source/dnode/vnode/src/tsdb/tsdbSnapshot.c +++ b/source/dnode/vnode/src/tsdb/tsdbSnapshot.c @@ -1032,9 +1032,6 @@ int32_t tsdbSnapWriterOpen(STsdb* pTsdb, int64_t sver, int64_t ever, void* pRang int32_t code = 0; int32_t lino = 0; - // disable background tasks - tsdbFSDisableBgTask(pTsdb->pFS); - // start to write writer[0] = taosMemoryCalloc(1, sizeof(*writer[0])); if (writer[0] == NULL) return TSDB_CODE_OUT_OF_MEMORY; @@ -1107,7 +1104,6 @@ int32_t tsdbSnapWriterClose(STsdbSnapWriter** writer, int8_t rollback) { taosThreadMutexUnlock(&writer[0]->tsdb->mutex); } - tsdbFSEnableBgTask(tsdb->pFS); tsdbIterMergerClose(&writer[0]->ctx->tombIterMerger); tsdbIterMergerClose(&writer[0]->ctx->dataIterMerger); @@ -1595,3 +1591,6 @@ _out: return code; } + +extern int32_t tsdbFSCancelAllBgTask(STFileSystem* fs); +int32_t tsdbCancelAllBgTask(STsdb* tsdb) { return tsdbFSCancelAllBgTask(tsdb->pFS); } \ No newline at end of file diff --git a/source/dnode/vnode/src/vnd/vnodeAsync.c b/source/dnode/vnode/src/vnd/vnodeAsync.c new file mode 100644 index 0000000000..bc2d1e44c1 --- /dev/null +++ b/source/dnode/vnode/src/vnd/vnodeAsync.c @@ -0,0 +1,719 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * 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 . + */ + +#include "vnd.h" +#include "vnodeHash.h" + +typedef struct SVATask SVATask; +typedef struct SVAChannel SVAChannel; + +#define VNODE_ASYNC_DEFAULT_WORKERS 4 +#define VNODE_ASYNC_MAX_WORKERS 256 + +// priority + +#define EVA_PRIORITY_MAX (EVA_PRIORITY_LOW + 1) + +// worker +typedef enum { + EVA_WORKER_STATE_UINIT = 0, + EVA_WORKER_STATE_ACTIVE, + EVA_WORKER_STATE_IDLE, + EVA_WORKER_STATE_STOP, +} EVWorkerState; + +typedef struct { + SVAsync *async; + int32_t workerId; + EVWorkerState state; + TdThread thread; + SVATask *runningTask; +} SVWorker; + +// task +typedef enum { + EVA_TASK_STATE_WAITTING = 0, + EVA_TASK_STATE_RUNNING, +} EVATaskState; + +struct SVATask { + int64_t taskId; + EVAPriority priority; + int32_t priorScore; + SVAChannel *channel; + int32_t (*execute)(void *); + void (*complete)(void *); + void *arg; + EVATaskState state; + + // wait + int32_t numWait; + TdThreadCond waitCond; + + // queue + struct SVATask *prev; + struct SVATask *next; +}; + +#define VATASK_PIORITY(task_) ((task_)->priority - ((task_)->priorScore / 4)) + +// async channel +typedef enum { + EVA_CHANNEL_STATE_OPEN = 0, + EVA_CHANNEL_STATE_CLOSE, +} EVAChannelState; + +struct SVAChannel { + int64_t channelId; + EVAChannelState state; + SVATask queue[EVA_PRIORITY_MAX]; + SVATask *scheduled; + + SVAChannel *prev; + SVAChannel *next; +}; + +// async handle +struct SVAsync { + const char *label; + + TdThreadMutex mutex; + TdThreadCond hasTask; + bool stop; + + // worker + int32_t numWorkers; + int32_t numLaunchWorkers; + int32_t numIdleWorkers; + SVWorker workers[VNODE_ASYNC_MAX_WORKERS]; + + // channel + int64_t nextChannelId; + int32_t numChannels; + SVAChannel chList; + SVHashTable *channelTable; + + // task + int64_t nextTaskId; + int32_t numTasks; + SVATask queue[EVA_PRIORITY_MAX]; + SVHashTable *taskTable; +}; + +static int32_t vnodeAsyncTaskDone(SVAsync *async, SVATask *task) { + int32_t ret; + + if (task->channel != NULL && task->channel->scheduled == task) { + task->channel->scheduled = NULL; + if (task->channel->state == EVA_CHANNEL_STATE_CLOSE) { + taosMemoryFree(task->channel); + } else { + for (int32_t i = 0; i < EVA_PRIORITY_MAX; i++) { + SVATask *nextTask = task->channel->queue[i].next; + if (nextTask != &task->channel->queue[i]) { + if (task->channel->scheduled == NULL) { + task->channel->scheduled = nextTask; + nextTask->next->prev = nextTask->prev; + nextTask->prev->next = nextTask->next; + } else { + nextTask->priority++; + int32_t newPriority = VATASK_PIORITY(nextTask); + if (newPriority != i) { + // remove from current priority queue + nextTask->prev->next = nextTask->next; + nextTask->next->prev = nextTask->prev; + // add to new priority queue + nextTask->next = &task->channel->queue[newPriority]; + nextTask->prev = task->channel->queue[newPriority].prev; + nextTask->next->prev = nextTask; + nextTask->prev->next = nextTask; + } + } + } + } + + if (task->channel->scheduled != NULL) { + int32_t priority = VATASK_PIORITY(task->channel->scheduled); + task->channel->scheduled->next = &async->queue[priority]; + task->channel->scheduled->prev = async->queue[priority].prev; + task->channel->scheduled->next->prev = task->channel->scheduled; + task->channel->scheduled->prev->next = task->channel->scheduled; + } + } + } + + ret = vHashDrop(async->taskTable, task); + if (ret != 0) { + ASSERT(0); + } + async->numTasks--; + + // call complete callback + if (task->complete) { + task->complete(task->arg); + } + + if (task->numWait == 0) { + taosThreadCondDestroy(&task->waitCond); + taosMemoryFree(task); + } else if (task->numWait == 1) { + taosThreadCondSignal(&task->waitCond); + } else { + taosThreadCondBroadcast(&task->waitCond); + } + return 0; +} + +static int32_t vnodeAsyncCancelAllTasks(SVAsync *async) { + for (int32_t i = 0; i < EVA_PRIORITY_MAX; i++) { + while (async->queue[i].next != &async->queue[i]) { + SVATask *task = async->queue[i].next; + task->prev->next = task->next; + task->next->prev = task->prev; + vnodeAsyncTaskDone(async, task); + } + } + return 0; +} + +static void *vnodeAsyncLoop(void *arg) { + SVWorker *worker = (SVWorker *)arg; + SVAsync *async = worker->async; + + setThreadName(async->label); + + for (;;) { + taosThreadMutexLock(&async->mutex); + + // finish last running task + if (worker->runningTask != NULL) { + vnodeAsyncTaskDone(async, worker->runningTask); + worker->runningTask = NULL; + } + + for (;;) { + if (async->stop || worker->workerId >= async->numWorkers) { + if (async->stop) { // cancel all tasks + vnodeAsyncCancelAllTasks(async); + } + worker->state = EVA_WORKER_STATE_STOP; + async->numLaunchWorkers--; + taosThreadMutexUnlock(&async->mutex); + return NULL; + } + + for (int32_t i = 0; i < EVA_PRIORITY_MAX; i++) { + SVATask *task = async->queue[i].next; + if (task != &async->queue[i]) { + if (worker->runningTask == NULL) { + worker->runningTask = task; + task->prev->next = task->next; + task->next->prev = task->prev; + } else { // promote priority + task->priorScore++; + int32_t priority = VATASK_PIORITY(task); + if (priority != i) { + // remove from current priority queue + task->prev->next = task->next; + task->next->prev = task->prev; + // add to new priority queue + task->next = &async->queue[priority]; + task->prev = async->queue[priority].prev; + task->next->prev = task; + task->prev->next = task; + } + } + } + } + + if (worker->runningTask == NULL) { + worker->state = EVA_WORKER_STATE_IDLE; + async->numIdleWorkers++; + taosThreadCondWait(&async->hasTask, &async->mutex); + async->numIdleWorkers--; + worker->state = EVA_WORKER_STATE_ACTIVE; + } else { + worker->runningTask->state = EVA_TASK_STATE_RUNNING; + break; + } + } + + taosThreadMutexUnlock(&async->mutex); + + // do run the task + worker->runningTask->execute(worker->runningTask->arg); + } + + return NULL; +} + +static uint32_t vnodeAsyncTaskHash(const void *obj) { + SVATask *task = (SVATask *)obj; + return MurmurHash3_32((const char *)(&task->taskId), sizeof(task->taskId)); +} + +static int32_t vnodeAsyncTaskCompare(const void *obj1, const void *obj2) { + SVATask *task1 = (SVATask *)obj1; + SVATask *task2 = (SVATask *)obj2; + if (task1->taskId < task2->taskId) { + return -1; + } else if (task1->taskId > task2->taskId) { + return 1; + } + return 0; +} + +static uint32_t vnodeAsyncChannelHash(const void *obj) { + SVAChannel *channel = (SVAChannel *)obj; + return MurmurHash3_32((const char *)(&channel->channelId), sizeof(channel->channelId)); +} + +static int32_t vnodeAsyncChannelCompare(const void *obj1, const void *obj2) { + SVAChannel *channel1 = (SVAChannel *)obj1; + SVAChannel *channel2 = (SVAChannel *)obj2; + if (channel1->channelId < channel2->channelId) { + return -1; + } else if (channel1->channelId > channel2->channelId) { + return 1; + } + return 0; +} + +int32_t vnodeAsyncInit(SVAsync **async, char *label) { + int32_t ret; + + if (async == NULL) { + return TSDB_CODE_INVALID_PARA; + } + + if (label == NULL) { + label = "anonymous"; + } + + (*async) = (SVAsync *)taosMemoryCalloc(1, sizeof(SVAsync) + strlen(label) + 1); + if ((*async) == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + + strcpy((char *)((*async) + 1), label); + (*async)->label = (const char *)((*async) + 1); + + taosThreadMutexInit(&(*async)->mutex, NULL); + taosThreadCondInit(&(*async)->hasTask, NULL); + (*async)->stop = false; + + // worker + (*async)->numWorkers = VNODE_ASYNC_DEFAULT_WORKERS; + (*async)->numLaunchWorkers = 0; + (*async)->numIdleWorkers = 0; + for (int32_t i = 0; i < VNODE_ASYNC_MAX_WORKERS; i++) { + (*async)->workers[i].async = (*async); + (*async)->workers[i].workerId = i; + (*async)->workers[i].state = EVA_WORKER_STATE_UINIT; + (*async)->workers[i].runningTask = NULL; + } + + // channel + (*async)->nextChannelId = 0; + (*async)->numChannels = 0; + (*async)->chList.prev = &(*async)->chList; + (*async)->chList.next = &(*async)->chList; + ret = vHashInit(&(*async)->channelTable, vnodeAsyncChannelHash, vnodeAsyncChannelCompare); + if (ret != 0) { + taosThreadMutexDestroy(&(*async)->mutex); + taosThreadCondDestroy(&(*async)->hasTask); + taosMemoryFree(*async); + return ret; + } + + // task + (*async)->nextTaskId = 0; + (*async)->numTasks = 0; + for (int32_t i = 0; i < EVA_PRIORITY_MAX; i++) { + (*async)->queue[i].next = &(*async)->queue[i]; + (*async)->queue[i].prev = &(*async)->queue[i]; + } + ret = vHashInit(&(*async)->taskTable, vnodeAsyncTaskHash, vnodeAsyncTaskCompare); + if (ret != 0) { + vHashDestroy(&(*async)->channelTable); + taosThreadMutexDestroy(&(*async)->mutex); + taosThreadCondDestroy(&(*async)->hasTask); + taosMemoryFree(*async); + return ret; + } + + return 0; +} + +int32_t vnodeAsyncDestroy(SVAsync **async) { + if ((*async) == NULL) { + return TSDB_CODE_INVALID_PARA; + } + + // set stop and broadcast + taosThreadMutexLock(&(*async)->mutex); + (*async)->stop = true; + taosThreadCondBroadcast(&(*async)->hasTask); + taosThreadMutexUnlock(&(*async)->mutex); + + // join all workers + for (int32_t i = 0; i < VNODE_ASYNC_MAX_WORKERS; i++) { + taosThreadMutexLock(&(*async)->mutex); + EVWorkerState state = (*async)->workers[i].state; + taosThreadMutexUnlock(&(*async)->mutex); + + if (state == EVA_WORKER_STATE_UINIT) { + continue; + } + + taosThreadJoin((*async)->workers[i].thread, NULL); + ASSERT((*async)->workers[i].state == EVA_WORKER_STATE_STOP); + (*async)->workers[i].state = EVA_WORKER_STATE_UINIT; + } + + // close all channels + for (SVAChannel *channel = (*async)->chList.next; channel != &(*async)->chList; channel = (*async)->chList.next) { + channel->next->prev = channel->prev; + channel->prev->next = channel->next; + + int32_t ret = vHashDrop((*async)->channelTable, channel); + if (ret) { + ASSERT(0); + } + (*async)->numChannels--; + taosMemoryFree(channel); + } + + ASSERT((*async)->numLaunchWorkers == 0); + ASSERT((*async)->numIdleWorkers == 0); + ASSERT((*async)->numChannels == 0); + ASSERT((*async)->numTasks == 0); + + taosThreadMutexDestroy(&(*async)->mutex); + taosThreadCondDestroy(&(*async)->hasTask); + + vHashDestroy(&(*async)->channelTable); + vHashDestroy(&(*async)->taskTable); + taosMemoryFree(*async); + *async = NULL; + + return 0; +} + +static int32_t vnodeAsyncLaunchWorker(SVAsync *async) { + for (int32_t i = 0; i < async->numWorkers; i++) { + ASSERT(async->workers[i].state != EVA_WORKER_STATE_IDLE); + if (async->workers[i].state == EVA_WORKER_STATE_ACTIVE) { + continue; + } else if (async->workers[i].state == EVA_WORKER_STATE_STOP) { + taosThreadJoin(async->workers[i].thread, NULL); + async->workers[i].state = EVA_WORKER_STATE_UINIT; + } + + taosThreadCreate(&async->workers[i].thread, NULL, vnodeAsyncLoop, &async->workers[i]); + async->workers[i].state = EVA_WORKER_STATE_ACTIVE; + async->numLaunchWorkers++; + break; + } + return 0; +} + +int32_t vnodeAsync(SVAsync *async, EVAPriority priority, int32_t (*execute)(void *), void (*complete)(void *), + void *arg, int64_t *taskId) { + return vnodeAsyncC(async, 0, priority, execute, complete, arg, taskId); +} + +int32_t vnodeAsyncC(SVAsync *async, int64_t channelId, EVAPriority priority, int32_t (*execute)(void *), + void (*complete)(void *), void *arg, int64_t *taskId) { + if (async == NULL || execute == NULL || channelId < 0) { + return TSDB_CODE_INVALID_PARA; + } + + int64_t id; + + // create task object + SVATask *task = (SVATask *)taosMemoryCalloc(1, sizeof(SVATask)); + if (task == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + + task->priority = priority; + task->priorScore = 0; + task->execute = execute; + task->complete = complete; + task->arg = arg; + task->state = EVA_TASK_STATE_WAITTING; + task->numWait = 0; + taosThreadCondInit(&task->waitCond, NULL); + + // schedule task + taosThreadMutexLock(&async->mutex); + + if (channelId == 0) { + task->channel = NULL; + } else { + SVAChannel channel = {.channelId = channelId}; + vHashGet(async->channelTable, &channel, (void **)&task->channel); + if (task->channel == NULL) { + taosThreadMutexUnlock(&async->mutex); + taosThreadCondDestroy(&task->waitCond); + taosMemoryFree(task); + return TSDB_CODE_INVALID_PARA; + } + } + + task->taskId = id = ++async->nextTaskId; + + // add task to hash table + int32_t ret = vHashPut(async->taskTable, task); + if (ret != 0) { + taosThreadMutexUnlock(&async->mutex); + taosThreadCondDestroy(&task->waitCond); + taosMemoryFree(task); + return ret; + } + + async->numTasks++; + + // add task to queue + if (task->channel == NULL || task->channel->scheduled == NULL) { + // add task to async->queue + if (task->channel) { + task->channel->scheduled = task; + } + + task->next = &async->queue[priority]; + task->prev = async->queue[priority].prev; + task->next->prev = task; + task->prev->next = task; + + // signal worker or launch new worker + if (async->numIdleWorkers > 0) { + taosThreadCondSignal(&(async->hasTask)); + } else if (async->numLaunchWorkers < async->numWorkers) { + vnodeAsyncLaunchWorker(async); + } + } else if (task->channel->scheduled->state == EVA_TASK_STATE_RUNNING || + priority >= VATASK_PIORITY(task->channel->scheduled)) { + // add task to task->channel->queue + task->next = &task->channel->queue[priority]; + task->prev = task->channel->queue[priority].prev; + task->next->prev = task; + task->prev->next = task; + } else { + // remove task->channel->scheduled from queue + task->channel->scheduled->prev->next = task->channel->scheduled->next; + task->channel->scheduled->next->prev = task->channel->scheduled->prev; + + // promote priority and add task->channel->scheduled to task->channel->queue + task->channel->scheduled->priorScore++; + int32_t newPriority = VATASK_PIORITY(task->channel->scheduled); + task->channel->scheduled->next = &task->channel->queue[newPriority]; + task->channel->scheduled->prev = task->channel->queue[newPriority].prev; + task->channel->scheduled->next->prev = task->channel->scheduled; + task->channel->scheduled->prev->next = task->channel->scheduled; + + // add task to queue + task->channel->scheduled = task; + task->next = &async->queue[priority]; + task->prev = async->queue[priority].prev; + task->next->prev = task; + task->prev->next = task; + } + + taosThreadMutexUnlock(&async->mutex); + + if (taskId != NULL) { + *taskId = id; + } + + return 0; +} + +int32_t vnodeAWait(SVAsync *async, int64_t taskId) { + if (async == NULL || taskId <= 0) { + return TSDB_CODE_INVALID_PARA; + } + + SVATask *task = NULL; + SVATask task2 = {.taskId = taskId}; + + taosThreadMutexLock(&async->mutex); + + vHashGet(async->taskTable, &task2, (void **)&task); + if (task) { + task->numWait++; + taosThreadCondWait(&task->waitCond, &async->mutex); + task->numWait--; + + if (task->numWait == 0) { + taosThreadCondDestroy(&task->waitCond); + taosMemoryFree(task); + } + } + + taosThreadMutexUnlock(&async->mutex); + + return 0; +} + +int32_t vnodeACancel(SVAsync *async, int64_t taskId) { + if (async == NULL) { + return TSDB_CODE_INVALID_PARA; + } + + int32_t ret = 0; + SVATask *task = NULL; + SVATask task2 = {.taskId = taskId}; + + taosThreadMutexLock(&async->mutex); + + vHashGet(async->taskTable, &task2, (void **)&task); + if (task) { + if (task->state == EVA_TASK_STATE_WAITTING) { + // remove from queue + task->next->prev = task->prev; + task->prev->next = task->next; + vnodeAsyncTaskDone(async, task); + } else { + ret = 0; // task is running, should return code TSDB_CODE_BUSY ?? + } + } + + taosThreadMutexUnlock(&async->mutex); + + return ret; +} + +int32_t vnodeAsyncSetWorkers(SVAsync *async, int32_t numWorkers) { + if (async == NULL || numWorkers <= 0 || numWorkers > VNODE_ASYNC_MAX_WORKERS) { + return TSDB_CODE_INVALID_PARA; + } + + taosThreadMutexLock(&async->mutex); + async->numWorkers = numWorkers; + if (async->numIdleWorkers > 0) { + taosThreadCondBroadcast(&async->hasTask); + } + taosThreadMutexUnlock(&async->mutex); + + return 0; +} + +int32_t vnodeAChannelInit(SVAsync *async, int64_t *channelId) { + if (async == NULL || channelId == NULL) { + return TSDB_CODE_INVALID_PARA; + } + + // create channel object + SVAChannel *channel = (SVAChannel *)taosMemoryMalloc(sizeof(SVAChannel)); + if (channel == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + channel->state = EVA_CHANNEL_STATE_OPEN; + for (int32_t i = 0; i < EVA_PRIORITY_MAX; i++) { + channel->queue[i].next = &channel->queue[i]; + channel->queue[i].prev = &channel->queue[i]; + } + channel->scheduled = NULL; + + // register channel + taosThreadMutexLock(&async->mutex); + + channel->channelId = *channelId = ++async->nextChannelId; + + // add to hash table + int32_t ret = vHashPut(async->channelTable, channel); + if (ret != 0) { + taosThreadMutexUnlock(&async->mutex); + taosMemoryFree(channel); + return ret; + } + + // add to list + channel->next = &async->chList; + channel->prev = async->chList.prev; + channel->next->prev = channel; + channel->prev->next = channel; + + async->numChannels++; + + taosThreadMutexUnlock(&async->mutex); + + return 0; +} + +int32_t vnodeAChannelDestroy(SVAsync *async, int64_t channelId, bool waitRunning) { + if (async == NULL || channelId <= 0) { + return TSDB_CODE_INVALID_PARA; + } + + SVAChannel *channel = NULL; + SVAChannel channel2 = {.channelId = channelId}; + + taosThreadMutexLock(&async->mutex); + + vHashGet(async->channelTable, &channel2, (void **)&channel); + if (channel) { + // unregister channel + channel->next->prev = channel->prev; + channel->prev->next = channel->next; + vHashDrop(async->channelTable, channel); + async->numChannels--; + + // cancel all waiting tasks + for (int32_t i = 0; i < EVA_PRIORITY_MAX; i++) { + while (channel->queue[i].next != &channel->queue[i]) { + SVATask *task = channel->queue[i].next; + task->prev->next = task->next; + task->next->prev = task->prev; + vnodeAsyncTaskDone(async, task); + } + } + + // cancel or wait the scheduled task + if (channel->scheduled == NULL || channel->scheduled->state == EVA_TASK_STATE_WAITTING) { + if (channel->scheduled) { + channel->scheduled->prev->next = channel->scheduled->next; + channel->scheduled->next->prev = channel->scheduled->prev; + vnodeAsyncTaskDone(async, channel->scheduled); + } + taosMemoryFree(channel); + } else { + if (waitRunning) { + // wait task + SVATask *task = channel->scheduled; + task->numWait++; + taosThreadCondWait(&task->waitCond, &async->mutex); + task->numWait--; + if (task->numWait == 0) { + taosThreadCondDestroy(&task->waitCond); + taosMemoryFree(task); + } + + taosMemoryFree(channel); + } else { + channel->state = EVA_CHANNEL_STATE_CLOSE; + } + } + } else { + taosThreadMutexUnlock(&async->mutex); + return TSDB_CODE_INVALID_PARA; + } + + taosThreadMutexUnlock(&async->mutex); + + return 0; +} \ No newline at end of file diff --git a/source/dnode/vnode/src/vnd/vnodeCommit.c b/source/dnode/vnode/src/vnd/vnodeCommit.c index a178e1f772..c8cd167393 100644 --- a/source/dnode/vnode/src/vnd/vnodeCommit.c +++ b/source/dnode/vnode/src/vnd/vnodeCommit.c @@ -203,10 +203,8 @@ int vnodeSaveInfo(const char *dir, const SVnodeInfo *pInfo) { // free info binary taosMemoryFree(data); - vInfo("vgId:%d, vnode info is saved, fname:%s replica:%d selfIndex:%d changeVersion:%d", - pInfo->config.vgId, fname, - pInfo->config.syncCfg.replicaNum, pInfo->config.syncCfg.myIndex, - pInfo->config.syncCfg.changeVersion); + vInfo("vgId:%d, vnode info is saved, fname:%s replica:%d selfIndex:%d changeVersion:%d", pInfo->config.vgId, fname, + pInfo->config.syncCfg.replicaNum, pInfo->config.syncCfg.myIndex, pInfo->config.syncCfg.changeVersion); return 0; @@ -289,9 +287,10 @@ static int32_t vnodePrepareCommit(SVnode *pVnode, SCommitInfo *pInfo) { char dir[TSDB_FILENAME_LEN] = {0}; int64_t lastCommitted = pInfo->info.state.committed; - tsem_wait(&pVnode->canCommit); + // wait last commit task + vnodeAWait(vnodeAsyncHandle[0], pVnode->commitTask); - if(syncNodeGetConfig(pVnode->sync, &pVnode->config.syncCfg) != 0) goto _exit; + if (syncNodeGetConfig(pVnode->sync, &pVnode->config.syncCfg) != 0) goto _exit; pVnode->state.commitTerm = pVnode->state.applyTerm; @@ -379,12 +378,11 @@ static int32_t vnodeCommitTask(void *arg) { vnodeReturnBufPool(pVnode); _exit: - // end commit - tsem_post(&pVnode->canCommit); - taosMemoryFree(pInfo); return code; } +static void vnodeCompleteCommit(void *arg) { taosMemoryFree(arg); } + int vnodeAsyncCommit(SVnode *pVnode) { int32_t code = 0; @@ -401,14 +399,14 @@ int vnodeAsyncCommit(SVnode *pVnode) { } // schedule the task - code = vnodeScheduleTask(vnodeCommitTask, pInfo); + code = vnodeAsyncC(vnodeAsyncHandle[0], pVnode->commitChannel, EVA_PRIORITY_HIGH, vnodeCommitTask, + vnodeCompleteCommit, pInfo, &pVnode->commitTask); _exit: if (code) { if (NULL != pInfo) { taosMemoryFree(pInfo); } - tsem_post(&pVnode->canCommit); vError("vgId:%d, %s failed since %s, commit id:%" PRId64, TD_VID(pVnode), __func__, tstrerror(code), pVnode->state.commitID); } else { @@ -420,8 +418,7 @@ _exit: int vnodeSyncCommit(SVnode *pVnode) { vnodeAsyncCommit(pVnode); - tsem_wait(&pVnode->canCommit); - tsem_post(&pVnode->canCommit); + vnodeAWait(vnodeAsyncHandle[0], pVnode->commitTask); return 0; } @@ -501,7 +498,7 @@ _exit: } bool vnodeShouldRollback(SVnode *pVnode) { - char tFName[TSDB_FILENAME_LEN] = {0}; + char tFName[TSDB_FILENAME_LEN] = {0}; int32_t offset = 0; vnodeGetPrimaryDir(pVnode->path, pVnode->diskPrimary, pVnode->pTfs, tFName, TSDB_FILENAME_LEN); @@ -512,7 +509,7 @@ bool vnodeShouldRollback(SVnode *pVnode) { } void vnodeRollback(SVnode *pVnode) { - char tFName[TSDB_FILENAME_LEN] = {0}; + char tFName[TSDB_FILENAME_LEN] = {0}; int32_t offset = 0; vnodeGetPrimaryDir(pVnode->path, pVnode->diskPrimary, pVnode->pTfs, tFName, TSDB_FILENAME_LEN); diff --git a/source/dnode/vnode/src/vnd/vnodeHash.c b/source/dnode/vnode/src/vnd/vnodeHash.c new file mode 100644 index 0000000000..33602f6581 --- /dev/null +++ b/source/dnode/vnode/src/vnd/vnodeHash.c @@ -0,0 +1,162 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * 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 . + */ + +#include "vnodeHash.h" + +#define VNODE_HASH_DEFAULT_NUM_BUCKETS 1024 + +typedef struct SVHashEntry SVHashEntry; + +struct SVHashEntry { + SVHashEntry* next; + void* obj; +}; + +struct SVHashTable { + uint32_t (*hash)(const void*); + int32_t (*compare)(const void*, const void*); + int32_t numEntries; + uint32_t numBuckets; + SVHashEntry** buckets; +}; + +static int32_t vHashRehash(SVHashTable* ht, uint32_t newNumBuckets) { + SVHashEntry** newBuckets = (SVHashEntry**)taosMemoryCalloc(newNumBuckets, sizeof(SVHashEntry*)); + if (newBuckets == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + + for (int32_t i = 0; i < ht->numBuckets; i++) { + SVHashEntry* entry = ht->buckets[i]; + while (entry != NULL) { + SVHashEntry* next = entry->next; + uint32_t bucketIndex = ht->hash(entry->obj) % newNumBuckets; + entry->next = newBuckets[bucketIndex]; + newBuckets[bucketIndex] = entry; + entry = next; + } + } + + taosMemoryFree(ht->buckets); + ht->buckets = newBuckets; + ht->numBuckets = newNumBuckets; + + return 0; +} + +int32_t vHashInit(SVHashTable** ht, uint32_t (*hash)(const void*), int32_t (*compare)(const void*, const void*)) { + if (ht == NULL || hash == NULL || compare == NULL) { + return TSDB_CODE_INVALID_PARA; + } + + (*ht) = (SVHashTable*)taosMemoryMalloc(sizeof(SVHashTable)); + if (*ht == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + + (*ht)->hash = hash; + (*ht)->compare = compare; + (*ht)->numEntries = 0; + (*ht)->numBuckets = VNODE_HASH_DEFAULT_NUM_BUCKETS; + (*ht)->buckets = (SVHashEntry**)taosMemoryCalloc((*ht)->numBuckets, sizeof(SVHashEntry*)); + if ((*ht)->buckets == NULL) { + taosMemoryFree(*ht); + return TSDB_CODE_OUT_OF_MEMORY; + } + + return 0; +} + +int32_t vHashDestroy(SVHashTable** ht) { + if (ht == NULL) { + return TSDB_CODE_INVALID_PARA; + } + + if (*ht) { + ASSERT((*ht)->numEntries == 0); + taosMemoryFree((*ht)->buckets); + taosMemoryFree(*ht); + (*ht) = NULL; + } + return 0; +} + +int32_t vHashPut(SVHashTable* ht, void* obj) { + if (ht == NULL || obj == NULL) { + return TSDB_CODE_INVALID_PARA; + } + + uint32_t bucketIndex = ht->hash(obj) % ht->numBuckets; + for (SVHashEntry* entry = ht->buckets[bucketIndex]; entry != NULL; entry = entry->next) { + if (ht->compare(entry->obj, obj) == 0) { + return TSDB_CODE_DUP_KEY; + } + } + + if (ht->numEntries >= ht->numBuckets) { + vHashRehash(ht, ht->numBuckets * 2); + bucketIndex = ht->hash(obj) % ht->numBuckets; + } + + SVHashEntry* entry = (SVHashEntry*)taosMemoryMalloc(sizeof(SVHashEntry)); + if (entry == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + entry->obj = obj; + entry->next = ht->buckets[bucketIndex]; + ht->buckets[bucketIndex] = entry; + ht->numEntries++; + + return 0; +} + +int32_t vHashGet(SVHashTable* ht, const void* obj, void** retObj) { + if (ht == NULL || obj == NULL || retObj == NULL) { + return TSDB_CODE_INVALID_PARA; + } + + uint32_t bucketIndex = ht->hash(obj) % ht->numBuckets; + for (SVHashEntry* entry = ht->buckets[bucketIndex]; entry != NULL; entry = entry->next) { + if (ht->compare(entry->obj, obj) == 0) { + *retObj = entry->obj; + return 0; + } + } + + *retObj = NULL; + return TSDB_CODE_NOT_FOUND; +} + +int32_t vHashDrop(SVHashTable* ht, const void* obj) { + if (ht == NULL || obj == NULL) { + return TSDB_CODE_INVALID_PARA; + } + + uint32_t bucketIndex = ht->hash(obj) % ht->numBuckets; + for (SVHashEntry** entry = &ht->buckets[bucketIndex]; *entry != NULL; entry = &(*entry)->next) { + if (ht->compare((*entry)->obj, obj) == 0) { + SVHashEntry* tmp = *entry; + *entry = (*entry)->next; + taosMemoryFree(tmp); + ht->numEntries--; + if (ht->numBuckets > VNODE_HASH_DEFAULT_NUM_BUCKETS && ht->numEntries < ht->numBuckets / 4) { + vHashRehash(ht, ht->numBuckets / 2); + } + return 0; + } + } + + return TSDB_CODE_NOT_FOUND; +} \ No newline at end of file diff --git a/source/dnode/vnode/src/vnd/vnodeHash.h b/source/dnode/vnode/src/vnd/vnodeHash.h new file mode 100644 index 0000000000..86f6f9ac87 --- /dev/null +++ b/source/dnode/vnode/src/vnd/vnodeHash.h @@ -0,0 +1,37 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * 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 . + */ + +#ifndef _VNODE_HAS_H_ +#define _VNODE_HAS_H_ + +#include "vnd.h" + +#ifdef __cplusplus +extern "C" { +#endif + +typedef struct SVHashTable SVHashTable; + +int32_t vHashInit(SVHashTable** ht, uint32_t (*hash)(const void*), int32_t (*compare)(const void*, const void*)); +int32_t vHashDestroy(SVHashTable** ht); +int32_t vHashPut(SVHashTable* ht, void* obj); +int32_t vHashGet(SVHashTable* ht, const void* obj, void** retObj); +int32_t vHashDrop(SVHashTable* ht, const void* obj); + +#ifdef __cplusplus +} +#endif + +#endif /*_VNODE_HAS_H_*/ \ No newline at end of file diff --git a/source/dnode/vnode/src/vnd/vnodeModule.c b/source/dnode/vnode/src/vnd/vnodeModule.c index df08fb8a2b..4e3cee42c6 100644 --- a/source/dnode/vnode/src/vnd/vnodeModule.c +++ b/source/dnode/vnode/src/vnd/vnodeModule.c @@ -16,65 +16,25 @@ #include "cos.h" #include "vnd.h" -typedef struct SVnodeTask SVnodeTask; -struct SVnodeTask { - SVnodeTask* next; - SVnodeTask* prev; - int (*execute)(void*); - void* arg; -}; +static volatile int32_t VINIT = 0; -typedef struct { - int nthreads; - TdThread* threads; - TdThreadMutex mutex; - TdThreadCond hasTask; - SVnodeTask queue; -} SVnodeThreadPool; - -struct SVnodeGlobal { - int8_t init; - int8_t stop; - SVnodeThreadPool tp[2]; -}; - -struct SVnodeGlobal vnodeGlobal; - -static void* loop(void* arg); +SVAsync* vnodeAsyncHandle[2]; int vnodeInit(int nthreads) { - int8_t init; - int ret; + int32_t init; - init = atomic_val_compare_exchange_8(&(vnodeGlobal.init), 0, 1); + init = atomic_val_compare_exchange_32(&VINIT, 0, 1); if (init) { return 0; } - vnodeGlobal.stop = 0; - for (int32_t i = 0; i < ARRAY_SIZE(vnodeGlobal.tp); i++) { - taosThreadMutexInit(&vnodeGlobal.tp[i].mutex, NULL); - taosThreadCondInit(&vnodeGlobal.tp[i].hasTask, NULL); + // vnode-commit + vnodeAsyncInit(&vnodeAsyncHandle[0], "vnode-commit"); + vnodeAsyncSetWorkers(vnodeAsyncHandle[0], nthreads); - taosThreadMutexLock(&vnodeGlobal.tp[i].mutex); - - vnodeGlobal.tp[i].queue.next = &vnodeGlobal.tp[i].queue; - vnodeGlobal.tp[i].queue.prev = &vnodeGlobal.tp[i].queue; - - taosThreadMutexUnlock(&(vnodeGlobal.tp[i].mutex)); - - vnodeGlobal.tp[i].nthreads = nthreads; - vnodeGlobal.tp[i].threads = taosMemoryCalloc(nthreads, sizeof(TdThread)); - if (vnodeGlobal.tp[i].threads == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - vError("failed to init vnode module since:%s", tstrerror(terrno)); - return -1; - } - - for (int j = 0; j < nthreads; j++) { - taosThreadCreate(&(vnodeGlobal.tp[i].threads[j]), NULL, loop, &vnodeGlobal.tp[i]); - } - } + // vnode-merge + vnodeAsyncInit(&vnodeAsyncHandle[1], "vnode-merge"); + vnodeAsyncSetWorkers(vnodeAsyncHandle[1], nthreads); if (walInit() < 0) { return -1; @@ -90,99 +50,15 @@ int vnodeInit(int nthreads) { } void vnodeCleanup() { - int8_t init; - - init = atomic_val_compare_exchange_8(&(vnodeGlobal.init), 1, 0); + int32_t init = atomic_val_compare_exchange_32(&VINIT, 1, 0); if (init == 0) return; // set stop - vnodeGlobal.stop = 1; - for (int32_t i = 0; i < ARRAY_SIZE(vnodeGlobal.tp); i++) { - taosThreadMutexLock(&(vnodeGlobal.tp[i].mutex)); - taosThreadCondBroadcast(&(vnodeGlobal.tp[i].hasTask)); - taosThreadMutexUnlock(&(vnodeGlobal.tp[i].mutex)); - - // wait for threads - for (int j = 0; j < vnodeGlobal.tp[i].nthreads; j++) { - taosThreadJoin(vnodeGlobal.tp[i].threads[j], NULL); - } - - // clear source - taosMemoryFreeClear(vnodeGlobal.tp[i].threads); - taosThreadCondDestroy(&(vnodeGlobal.tp[i].hasTask)); - taosThreadMutexDestroy(&(vnodeGlobal.tp[i].mutex)); - } + vnodeAsyncDestroy(&vnodeAsyncHandle[0]); + vnodeAsyncDestroy(&vnodeAsyncHandle[1]); walCleanUp(); tqCleanUp(); smaCleanUp(); s3CleanUp(); } - -int vnodeScheduleTaskEx(int tpid, int (*execute)(void*), void* arg) { - SVnodeTask* pTask; - - ASSERT(!vnodeGlobal.stop); - - pTask = taosMemoryMalloc(sizeof(*pTask)); - if (pTask == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - return -1; - } - - pTask->execute = execute; - pTask->arg = arg; - - taosThreadMutexLock(&(vnodeGlobal.tp[tpid].mutex)); - pTask->next = &vnodeGlobal.tp[tpid].queue; - pTask->prev = vnodeGlobal.tp[tpid].queue.prev; - vnodeGlobal.tp[tpid].queue.prev->next = pTask; - vnodeGlobal.tp[tpid].queue.prev = pTask; - taosThreadCondSignal(&(vnodeGlobal.tp[tpid].hasTask)); - taosThreadMutexUnlock(&(vnodeGlobal.tp[tpid].mutex)); - - return 0; -} - -int vnodeScheduleTask(int (*execute)(void*), void* arg) { return vnodeScheduleTaskEx(0, execute, arg); } - -/* ------------------------ STATIC METHODS ------------------------ */ -static void* loop(void* arg) { - SVnodeThreadPool* tp = (SVnodeThreadPool*)arg; - SVnodeTask* pTask; - int ret; - - if (tp == &vnodeGlobal.tp[0]) { - setThreadName("vnode-commit"); - } else if (tp == &vnodeGlobal.tp[1]) { - setThreadName("vnode-merge"); - } - - for (;;) { - taosThreadMutexLock(&(tp->mutex)); - for (;;) { - pTask = tp->queue.next; - if (pTask == &tp->queue) { - // no task - if (vnodeGlobal.stop) { - taosThreadMutexUnlock(&(tp->mutex)); - return NULL; - } else { - taosThreadCondWait(&(tp->hasTask), &(tp->mutex)); - } - } else { - // has task - pTask->prev->next = pTask->next; - pTask->next->prev = pTask->prev; - break; - } - } - - taosThreadMutexUnlock(&(tp->mutex)); - - pTask->execute(pTask->arg); - taosMemoryFree(pTask); - } - - return NULL; -} diff --git a/source/dnode/vnode/src/vnd/vnodeOpen.c b/source/dnode/vnode/src/vnd/vnodeOpen.c index ff79e83d72..946ce9d278 100644 --- a/source/dnode/vnode/src/vnd/vnodeOpen.c +++ b/source/dnode/vnode/src/vnd/vnodeOpen.c @@ -129,8 +129,8 @@ int32_t vnodeAlterReplica(const char *path, SAlterVnodeReplicaReq *pReq, int32_t } pCfg->changeVersion = pReq->changeVersion; - vInfo("vgId:%d, save config while alter, replicas:%d totalReplicas:%d selfIndex:%d changeVersion:%d", - pReq->vgId, pCfg->replicaNum, pCfg->totalReplicaNum, pCfg->myIndex, pCfg->changeVersion); + vInfo("vgId:%d, save config while alter, replicas:%d totalReplicas:%d selfIndex:%d changeVersion:%d", pReq->vgId, + pCfg->replicaNum, pCfg->totalReplicaNum, pCfg->myIndex, pCfg->changeVersion); info.config.syncCfg = *pCfg; ret = vnodeSaveInfo(dir, &info); @@ -396,10 +396,14 @@ SVnode *vnodeOpen(const char *path, int32_t diskPrimary, STfs *pTfs, SMsgCb msgC pVnode->blocked = false; tsem_init(&pVnode->syncSem, 0, 0); - tsem_init(&(pVnode->canCommit), 0, 1); taosThreadMutexInit(&pVnode->mutex, NULL); taosThreadCondInit(&pVnode->poolNotEmpty, NULL); + if (vnodeAChannelInit(vnodeAsyncHandle[0], &pVnode->commitChannel) != 0) { + vError("vgId:%d, failed to init commit channel", TD_VID(pVnode)); + goto _err; + } + int8_t rollback = vnodeShouldRollback(pVnode); // open buffer pool @@ -487,7 +491,6 @@ _err: if (pVnode->pMeta) metaClose(&pVnode->pMeta); if (pVnode->freeList) vnodeCloseBufPool(pVnode); - tsem_destroy(&(pVnode->canCommit)); taosMemoryFree(pVnode); return NULL; } @@ -501,7 +504,8 @@ void vnodePostClose(SVnode *pVnode) { vnodeSyncPostClose(pVnode); } void vnodeClose(SVnode *pVnode) { if (pVnode) { - tsem_wait(&pVnode->canCommit); + vnodeAWait(vnodeAsyncHandle[0], pVnode->commitTask); + vnodeAChannelDestroy(vnodeAsyncHandle[0], pVnode->commitChannel, true); vnodeSyncClose(pVnode); vnodeQueryClose(pVnode); tqClose(pVnode->pTq); @@ -510,10 +514,8 @@ void vnodeClose(SVnode *pVnode) { smaClose(pVnode->pSma); if (pVnode->pMeta) metaClose(&pVnode->pMeta); vnodeCloseBufPool(pVnode); - tsem_post(&pVnode->canCommit); // destroy handle - tsem_destroy(&(pVnode->canCommit)); tsem_destroy(&pVnode->syncSem); taosThreadCondDestroy(&pVnode->poolNotEmpty); taosThreadMutexDestroy(&pVnode->mutex); diff --git a/source/dnode/vnode/src/vnd/vnodeSnapshot.c b/source/dnode/vnode/src/vnd/vnodeSnapshot.c index 91244e321f..f2ef11e9ed 100644 --- a/source/dnode/vnode/src/vnd/vnodeSnapshot.c +++ b/source/dnode/vnode/src/vnd/vnodeSnapshot.c @@ -13,8 +13,8 @@ * along with this program. If not, see . */ -#include "vnd.h" #include "tsdb.h" +#include "vnd.h" // SVSnapReader ======================================================== struct SVSnapReader { @@ -32,11 +32,11 @@ struct SVSnapReader { TSnapRangeArray *pRanges; STsdbSnapReader *pTsdbReader; // tq - int8_t tqHandleDone; - STqSnapReader *pTqSnapReader; - int8_t tqOffsetDone; - STqOffsetReader *pTqOffsetReader; - int8_t tqCheckInfoDone; + int8_t tqHandleDone; + STqSnapReader *pTqSnapReader; + int8_t tqOffsetDone; + STqOffsetReader *pTqOffsetReader; + int8_t tqCheckInfoDone; STqCheckInfoReader *pTqCheckInfoReader; // stream int8_t streamTaskDone; @@ -458,8 +458,8 @@ struct SVSnapWriter { TSnapRangeArray *pRanges; STsdbSnapWriter *pTsdbSnapWriter; // tq - STqSnapWriter *pTqSnapWriter; - STqOffsetWriter *pTqOffsetWriter; + STqSnapWriter *pTqSnapWriter; + STqOffsetWriter *pTqOffsetWriter; STqCheckInfoWriter *pTqCheckInfoWriter; // stream SStreamTaskWriter *pStreamTaskWriter; @@ -519,6 +519,8 @@ _out: return code; } +extern int32_t tsdbCancelAllBgTask(STsdb *tsdb); + int32_t vnodeSnapWriterOpen(SVnode *pVnode, SSnapshotParam *pParam, SVSnapWriter **ppWriter) { int32_t code = 0; SVSnapWriter *pWriter = NULL; @@ -526,8 +528,8 @@ int32_t vnodeSnapWriterOpen(SVnode *pVnode, SSnapshotParam *pParam, SVSnapWriter int64_t ever = pParam->end; // commit memory data - vnodeAsyncCommit(pVnode); - tsem_wait(&pVnode->canCommit); + vnodeSyncCommit(pVnode); + tsdbCancelAllBgTask(pVnode->pTsdb); // alloc pWriter = (SVSnapWriter *)taosMemoryCalloc(1, sizeof(*pWriter)); @@ -657,7 +659,6 @@ _exit: vInfo("vgId:%d, vnode snapshot writer closed, rollback:%d", TD_VID(pVnode), rollback); taosMemoryFree(pWriter); } - tsem_post(&pVnode->canCommit); return code; } From 66f25712fd4f1e9e710e296f022509461e21ebb2 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 29 Nov 2023 18:07:15 +0800 Subject: [PATCH 186/195] fix(tsdb): fix bugs in suspend/resume. adjust code to conduct the test more easily. td-27579 --- source/client/test/clientTests.cpp | 102 ++++++++------ source/dnode/vnode/src/inc/vnodeInt.h | 4 + source/dnode/vnode/src/tsdb/tsdbRead2.c | 151 ++++++++------------- source/dnode/vnode/src/tsdb/tsdbReadUtil.h | 14 +- 4 files changed, 127 insertions(+), 144 deletions(-) diff --git a/source/client/test/clientTests.cpp b/source/client/test/clientTests.cpp index addf0aa629..e78783cf3c 100644 --- a/source/client/test/clientTests.cpp +++ b/source/client/test/clientTests.cpp @@ -126,9 +126,9 @@ void queryCallback(void* param, void* res, int32_t code) { taos_fetch_raw_block_a(res, fetchCallback, param); } -void createNewTable(TAOS* pConn, int32_t index) { +void createNewTable(TAOS* pConn, int32_t index, int32_t numOfRows, int64_t startTs, const char* pVarchar) { char str[1024] = {0}; - sprintf(str, "create table tu%d using st2 tags(%d)", index, index); + sprintf(str, "create table if not exists tu%d using st2 tags(%d)", index, index); TAOS_RES* pRes = taos_query(pConn, str); if (taos_errno(pRes) != 0) { @@ -136,22 +136,43 @@ void createNewTable(TAOS* pConn, int32_t index) { } taos_free_result(pRes); - for (int32_t i = 0; i < 10000; i += 20) { - char sql[1024] = {0}; - sprintf(sql, - "insert into tu%d values(now+%da, %d)(now+%da, %d)(now+%da, %d)(now+%da, %d)" - "(now+%da, %d)(now+%da, %d)(now+%da, %d)(now+%da, %d)(now+%da, %d)(now+%da, %d)" - "(now+%da, %d)(now+%da, %d)(now+%da, %d)(now+%da, %d)" - "(now+%da, %d)(now+%da, %d)(now+%da, %d)(now+%da, %d)(now+%da, %d)(now+%da, %d)", - index, i, i, i + 1, i + 1, i + 2, i + 2, i + 3, i + 3, i + 4, i + 4, i + 5, i + 5, i + 6, i + 6, i + 7, - i + 7, i + 8, i + 8, i + 9, i + 9, i + 10, i + 10, i + 11, i + 11, i + 12, i + 12, i + 13, i + 13, i + 14, - i + 14, i + 15, i + 15, i + 16, i + 16, i + 17, i + 17, i + 18, i + 18, i + 19, i + 19); - TAOS_RES* p = taos_query(pConn, sql); - if (taos_errno(p) != 0) { - printf("failed to insert data, reason:%s\n", taos_errstr(p)); - } + if (startTs == 0) { + for (int32_t i = 0; i < numOfRows; i += 20) { + char sql[1024] = {0}; + sprintf(sql, + "insert into tu%d values(now+%da, %d)(now+%da, %d)(now+%da, %d)(now+%da, %d)" + "(now+%da, %d)(now+%da, %d)(now+%da, %d)(now+%da, %d)(now+%da, %d)(now+%da, %d)" + "(now+%da, %d)(now+%da, %d)(now+%da, %d)(now+%da, %d)" + "(now+%da, %d)(now+%da, %d)(now+%da, %d)(now+%da, %d)(now+%da, %d)(now+%da, %d)", + index, i, i, i + 1, i + 1, i + 2, i + 2, i + 3, i + 3, i + 4, i + 4, i + 5, i + 5, i + 6, i + 6, i + 7, + i + 7, i + 8, i + 8, i + 9, i + 9, i + 10, i + 10, i + 11, i + 11, i + 12, i + 12, i + 13, i + 13, i + 14, + i + 14, i + 15, i + 15, i + 16, i + 16, i + 17, i + 17, i + 18, i + 18, i + 19, i + 19); + TAOS_RES* p = taos_query(pConn, sql); + if (taos_errno(p) != 0) { + printf("failed to insert data, reason:%s\n", taos_errstr(p)); + } - taos_free_result(p); + taos_free_result(p); + } + } else { + for (int32_t i = 0; i < numOfRows; i += 20) { + char sql[1024*50] = {0}; + sprintf(sql, + "insert into tu%d values(%ld, %d, '%s')(%ld, %d, '%s')(%ld, %d, '%s')(%ld, %d, '%s')(%ld, %d, '%s')(%ld, " + "%d, '%s')(%ld, %d, '%s')(%ld, %d, '%s')(%ld, %d, '%s')(%ld, %d, '%s')(%ld, %d, '%s')(%ld, %d, '%s')(%ld, %d, '%s')(%ld, %d, '%s')(%ld, %d, '%s')(%ld, %d, " + "'%s')(%ld, %d, '%s')(%ld, %d, '%s')(%ld, %d, '%s')(%ld, %d, '%s')", + index, startTs, i, pVarchar, startTs + 1, i + 1, pVarchar, startTs + 2, i + 2, pVarchar, startTs + 3, i + 3, pVarchar, startTs + 4, i + 4, + pVarchar, startTs + 5, i + 5, pVarchar, startTs + 6, i + 6, pVarchar, startTs + 7, i + 7, pVarchar, startTs + 8, i + 8, pVarchar, startTs + 9, i + 9, + pVarchar, startTs + 10, i + 10, pVarchar, startTs + 11, i + 11, pVarchar, startTs + 12, i + 12, pVarchar, startTs + 13, i + 13, pVarchar, startTs + 14, + i + 14, pVarchar, startTs + 15, i + 15, pVarchar, startTs + 16, i + 16, pVarchar, startTs + 17, i + 17, pVarchar, startTs + 18, i + 18, + pVarchar, startTs + 19, i + 19, pVarchar); + TAOS_RES* p = taos_query(pConn, sql); + if (taos_errno(p) != 0) { + printf("failed to insert data, reason:%s\n", taos_errstr(p)); + } + + taos_free_result(p); + } } } @@ -808,14 +829,7 @@ TEST(clientCase, projection_query_tables) { TAOS_RES* pRes = taos_query(pConn, "use abc1"); taos_free_result(pRes); - pRes = taos_query(pConn, "create stable st1 (ts timestamp, k int) tags(a int)"); - if (taos_errno(pRes) != 0) { - printf("failed to create table tu, reason:%s\n", taos_errstr(pRes)); - } - - taos_free_result(pRes); - - pRes = taos_query(pConn, "create stable st2 (ts timestamp, k int) tags(a int)"); + pRes = taos_query(pConn, "create stable st2 (ts timestamp, k int, f varchar(4096)) tags(a int)"); if (taos_errno(pRes) != 0) { printf("failed to create table tu, reason:%s\n", taos_errstr(pRes)); } @@ -828,28 +842,32 @@ TEST(clientCase, projection_query_tables) { taos_free_result(pRes); int64_t start = 1685959190000; + const char* pstr = + "abcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefgh" + "ijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnop" + "qrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwx" + "yzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdef" + "ghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyz!@#$%^&&*&^^%$#@!qQWERTYUIOPASDFGHJKL:" + "QWERTYUIOP{}"; - int32_t code = -1; - for(int32_t i = 0; i < 1000000; ++i) { - char t[512] = {0}; + for(int32_t i = 0; i < 10000; ++i) { + char str[1024] = {0}; + sprintf(str, "create table if not exists tu%d using st2 tags(%d)", i, i); - sprintf(t, "insert into t1 values(now, %d)", i); - while(1) { - void* p = taos_query(pConn, t); - code = taos_errno(p); - taos_free_result(p); - if (code != 0) { - printf("insert data error, retry\n"); - } else { - break; - } + TAOS_RES* px = taos_query(pConn, str); + if (taos_errno(px) != 0) { + printf("failed to create table tu, reason:%s\n", taos_errstr(pRes)); + } + taos_free_result(px); + } + + for(int32_t j = 0; j < 5000; ++j) { + start += 20; + for (int32_t i = 0; i < 10000; ++i) { + createNewTable(pConn, i, 20, start, pstr); } } - for (int32_t i = 0; i < 1; ++i) { - printf("create table :%d\n", i); - createNewTable(pConn, i); - } // // pRes = taos_query(pConn, "select * from tu"); // if (taos_errno(pRes) != 0) { diff --git a/source/dnode/vnode/src/inc/vnodeInt.h b/source/dnode/vnode/src/inc/vnodeInt.h index 16379db053..f3f84896ad 100644 --- a/source/dnode/vnode/src/inc/vnodeInt.h +++ b/source/dnode/vnode/src/inc/vnodeInt.h @@ -93,7 +93,11 @@ typedef struct SQueryNode SQueryNode; #define VNODE_RSMA2_DIR "rsma2" #define VNODE_TQ_STREAM "stream" +#if SUSPEND_RESUME_TEST // only for test purpose +#define VNODE_BUFPOOL_SEGMENTS 1 +#else #define VNODE_BUFPOOL_SEGMENTS 3 +#endif #define VND_INFO_FNAME "vnode.json" #define VND_INFO_FNAME_TMP "vnode_tmp.json" diff --git a/source/dnode/vnode/src/tsdb/tsdbRead2.c b/source/dnode/vnode/src/tsdb/tsdbRead2.c index 5d018af6c1..4138d91f5d 100644 --- a/source/dnode/vnode/src/tsdb/tsdbRead2.c +++ b/source/dnode/vnode/src/tsdb/tsdbRead2.c @@ -48,7 +48,7 @@ static int32_t doMergeMemIMemRows(TSDBROW* pRow, TSDBROW* piRow, STableBlockScan static int32_t mergeRowsInFileBlocks(SBlockData* pBlockData, STableBlockScanInfo* pBlockScanInfo, int64_t key, STsdbReader* pReader); -static int32_t initDelSkylineIterator(STableBlockScanInfo* pBlockScanInfo, int32_t order, SCostSummary* pCost); +static int32_t initDelSkylineIterator(STableBlockScanInfo* pBlockScanInfo, int32_t order, SReadCostSummary* pCost); static STsdb* getTsdbByRetentions(SVnode* pVnode, SQueryTableDataCond* pCond, SRetention* retentions, const char* idstr, int8_t* pLevel); static SVersionRange getQueryVerRange(SVnode* pVnode, SQueryTableDataCond* pCond, int8_t level); @@ -58,6 +58,7 @@ static TSDBKEY getCurrentKeyInBuf(STableBlockScanInfo* pScanInfo, STsdbRea static bool hasDataInFileBlock(const SBlockData* pBlockData, const SFileBlockDumpInfo* pDumpInfo); static void initBlockDumpInfo(STsdbReader* pReader, SDataBlockIter* pBlockIter); static int32_t getInitialDelIndex(const SArray* pDelSkyline, int32_t order); +static void updateComposedBlockInfo(STsdbReader* pReader, double el, STableBlockScanInfo* pBlockScanInfo); static bool outOfTimeWindow(int64_t ts, STimeWindow* pWindow) { return (ts > pWindow->ekey) || (ts < pWindow->skey); } @@ -168,7 +169,7 @@ static int32_t filesetIteratorNext(SFilesetIter* pIter, STsdbReader* pReader, bo return TSDB_CODE_SUCCESS; } - SCostSummary* pCost = &pReader->cost; + SReadCostSummary* pCost = &pReader->cost; pIter->pLastBlockReader->uid = 0; tMergeTreeClose(&pIter->pLastBlockReader->mergeTree); @@ -291,11 +292,7 @@ static SSDataBlock* createResBlock(SQueryTableDataCond* pCond, int32_t capacity) } static int32_t tsdbInitReaderLock(STsdbReader* pReader) { - int32_t code = -1; - qTrace("tsdb/read: %p, pre-init read mutex: %p, code: %d", pReader, &pReader->readerMutex, code); - - code = taosThreadMutexInit(&pReader->readerMutex, NULL); - + int32_t code = taosThreadMutexInit(&pReader->readerMutex, NULL); qTrace("tsdb/read: %p, post-init read mutex: %p, code: %d", pReader, &pReader->readerMutex, code); return code; @@ -324,22 +321,14 @@ static int32_t tsdbAcquireReader(STsdbReader* pReader) { } static int32_t tsdbTryAcquireReader(STsdbReader* pReader) { - int32_t code = -1; - qTrace("tsdb/read: %p, pre-trytake read mutex: %p, code: %d", pReader, &pReader->readerMutex, code); - - code = taosThreadMutexTryLock(&pReader->readerMutex); - + int32_t code = taosThreadMutexTryLock(&pReader->readerMutex); qTrace("tsdb/read: %p, post-trytake read mutex: %p, code: %d", pReader, &pReader->readerMutex, code); return code; } static int32_t tsdbReleaseReader(STsdbReader* pReader) { - int32_t code = -1; - qTrace("tsdb/read: %p, pre-untake read mutex: %p, code: %d", pReader, &pReader->readerMutex, code); - - code = taosThreadMutexUnlock(&pReader->readerMutex); - + int32_t code = taosThreadMutexUnlock(&pReader->readerMutex); qTrace("tsdb/read: %p, post-untake read mutex: %p, code: %d", pReader, &pReader->readerMutex, code); return code; @@ -432,6 +421,7 @@ static int32_t tsdbReaderCreate(SVnode* pVnode, SQueryTableDataCond* pCond, void } tsdbInitReaderLock(pReader); + tsem_init(&pReader->resumeAfterSuspend, 0, 0); *ppReader = pReader; return code; @@ -1015,8 +1005,8 @@ static int32_t copyBlockDataToSDataBlock(STsdbReader* pReader) { // check if current block are all handled if (pDumpInfo->rowIndex >= 0 && pDumpInfo->rowIndex < pRecord->numRow) { int64_t ts = pBlockData->aTSKEY[pDumpInfo->rowIndex]; - if (outOfTimeWindow(ts, - &pReader->info.window)) { // the remain data has out of query time window, ignore current block + if (outOfTimeWindow(ts, &pReader->info.window)) { + // the remain data has out of query time window, ignore current block setBlockAllDumped(pDumpInfo, ts, pReader->info.order); } } else { @@ -1123,16 +1113,12 @@ static bool getNeighborBlockOfSameTable(SDataBlockIter* pBlockIter, SFileDataBlo } int32_t step = asc ? 1 : -1; - // *nextIndex = pBlockInfo->tbBlockIdx + step; - // *pBlockIndex = *(SBlockIndex*)taosArrayGet(pTableBlockScanInfo->pBlockList, *nextIndex); STableDataBlockIdx* pTableDataBlockIdx = taosArrayGet(pTableBlockScanInfo->pBlockIdxList, pBlockInfo->tbBlockIdx + step); SFileDataBlockInfo* p = taosArrayGet(pBlockIter->blockList, pTableDataBlockIdx->globalIndex); memcpy(pRecord, &p->record, sizeof(SBrinRecord)); *nextIndex = pBlockInfo->tbBlockIdx + step; - - // tMapDataGetItemByIdx(&pTableBlockScanInfo->mapData, pIndex->ordinalIndex, pBlock, tGetDataBlk); return true; } @@ -1376,23 +1362,19 @@ static int32_t buildDataBlockFromBuf(STsdbReader* pReader, STableBlockScanInfo* return TSDB_CODE_SUCCESS; } - SSDataBlock* pBlock = pReader->resBlockInfo.pResBlock; - int64_t st = taosGetTimestampUs(); + SSDataBlock* pBlock = pReader->resBlockInfo.pResBlock; int32_t code = buildDataBlockFromBufImpl(pBlockScanInfo, endKey, pReader->resBlockInfo.capacity, pReader); - blockDataUpdateTsWindow(pBlock, pReader->suppInfo.slotId[0]); - pBlock->info.id.uid = pBlockScanInfo->uid; + double el = (taosGetTimestampUs() - st) / 1000.0; + updateComposedBlockInfo(pReader, el, pBlockScanInfo); - setComposedBlockFlag(pReader, true); - - double elapsedTime = (taosGetTimestampUs() - st) / 1000.0; tsdbDebug("%p build data block from cache completed, elapsed time:%.2f ms, numOfRows:%" PRId64 ", brange:%" PRId64 " - %" PRId64 ", uid:%" PRIu64 ", %s", - pReader, elapsedTime, pBlock->info.rows, pBlock->info.window.skey, pBlock->info.window.ekey, + pReader, el, pBlock->info.rows, pBlock->info.window.skey, pBlock->info.window.ekey, pBlockScanInfo->uid, pReader->idStr); - pReader->cost.buildmemBlock += elapsedTime; + pReader->cost.buildmemBlock += el; return code; } @@ -2293,13 +2275,12 @@ static int32_t loadNeighborIfOverlap(SFileDataBlockInfo* pBlockInfo, STableBlock return code; } -static void updateComposedBlockInfo(STsdbReader* pReader, double el, STableBlockScanInfo* pBlockScanInfo) { +void updateComposedBlockInfo(STsdbReader* pReader, double el, STableBlockScanInfo* pBlockScanInfo) { SSDataBlock* pResBlock = pReader->resBlockInfo.pResBlock; pResBlock->info.id.uid = (pBlockScanInfo != NULL) ? pBlockScanInfo->uid : 0; pResBlock->info.dataLoad = 1; blockDataUpdateTsWindow(pResBlock, pReader->suppInfo.slotId[0]); - setComposedBlockFlag(pReader, true); pReader->cost.composedBlocks += 1; @@ -2356,7 +2337,6 @@ static int32_t buildComposedDataBlock(STsdbReader* pReader) { pBlockScanInfo = *pReader->status.pTableIter; if (pReader->pIgnoreTables && taosHashGet(*pReader->pIgnoreTables, &pBlockScanInfo->uid, sizeof(pBlockScanInfo->uid))) { - // setBlockAllDumped(pDumpInfo, pBlock->maxKey.ts, pReader->info.order); return code; } } @@ -2436,7 +2416,7 @@ int32_t getInitialDelIndex(const SArray* pDelSkyline, int32_t order) { return ASCENDING_TRAVERSE(order) ? 0 : taosArrayGetSize(pDelSkyline) - 1; } -int32_t initDelSkylineIterator(STableBlockScanInfo* pBlockScanInfo, int32_t order, SCostSummary* pCost) { +int32_t initDelSkylineIterator(STableBlockScanInfo* pBlockScanInfo, int32_t order, SReadCostSummary* pCost) { int32_t code = 0; int32_t newDelDataInFile = taosArrayGetSize(pBlockScanInfo->pFileDelData); if (newDelDataInFile == 0 && @@ -2935,6 +2915,8 @@ static int32_t buildBlockFromBufferSequentially(STsdbReader* pReader) { SReaderStatus* pStatus = &pReader->status; STableUidList* pUidList = &pStatus->uidList; + tsdbDebug("seq load data blocks from cache, %s", pReader->idStr); + while (1) { if (pReader->code != TSDB_CODE_SUCCESS) { tsdbWarn("tsdb reader is stopped ASAP, code:%s, %s", strerror(pReader->code), pReader->idStr); @@ -3043,6 +3025,8 @@ static ERetrieveType doReadDataFromLastFiles(STsdbReader* pReader) { SSDataBlock* pResBlock = pReader->resBlockInfo.pResBlock; SDataBlockIter* pBlockIter = &pReader->status.blockIter; + tsdbDebug("seq load data blocks from stt files %s", pReader->idStr); + while (1) { terrno = 0; @@ -3774,7 +3758,6 @@ int32_t buildDataBlockFromBufImpl(STableBlockScanInfo* pBlockScanInfo, int64_t e int32_t code = TSDB_CODE_SUCCESS; do { - // SRow* pTSRow = NULL; TSDBROW row = {.type = -1}; bool freeTSRow = false; tsdbGetNextRowInMem(pBlockScanInfo, pReader, &row, endKey, &freeTSRow); @@ -3792,13 +3775,17 @@ int32_t buildDataBlockFromBufImpl(STableBlockScanInfo* pBlockScanInfo, int64_t e if (code) { return code; } + + pBlockScanInfo->lastProcKey = row.pTSRow->ts; } else { code = doAppendRowFromFileBlock(pBlock, pReader, row.pBlockData, row.iRow); if (code) { break; } + pBlockScanInfo->lastProcKey = row.pBlockData->aTSKEY[row.iRow]; } + // no data in buffer, return immediately if (!(pBlockScanInfo->iter.hasVal || pBlockScanInfo->iiter.hasVal)) { break; @@ -4107,7 +4094,7 @@ void tsdbReaderClose2(STsdbReader* pReader) { tsdbDataFileReaderClose(&pReader->pFileReader); } - SCostSummary* pCost = &pReader->cost; + SReadCostSummary* pCost = &pReader->cost; SFilesetIter* pFilesetIter = &pReader->status.fileIter; if (pFilesetIter->pLastBlockReader != NULL) { SLastBlockReader* pLReader = pFilesetIter->pLastBlockReader; @@ -4122,6 +4109,7 @@ void tsdbReaderClose2(STsdbReader* pReader) { tsdbUntakeReadSnap2(pReader, pReader->pReadSnap, true); pReader->pReadSnap = NULL; + tsem_destroy(&pReader->resumeAfterSuspend); tsdbReleaseReader(pReader); tsdbUninitReaderLock(pReader); @@ -4148,26 +4136,14 @@ void tsdbReaderClose2(STsdbReader* pReader) { taosMemoryFreeClear(pReader); } -static void clearMemIterInfo(STableBlockScanInfo* pInfo) { - pInfo->iterInit = false; - pInfo->iter.hasVal = false; - pInfo->iiter.hasVal = false; - - if (pInfo->iter.iter != NULL) { - pInfo->iter.iter = tsdbTbDataIterDestroy(pInfo->iter.iter); - } - - if (pInfo->iiter.iter != NULL) { - pInfo->iiter.iter = tsdbTbDataIterDestroy(pInfo->iiter.iter); - } -} - int32_t tsdbReaderSuspend2(STsdbReader* pReader) { // save reader's base state & reset top state to be reconstructed from base state int32_t code = 0; SReaderStatus* pStatus = &pReader->status; STableBlockScanInfo* pBlockScanInfo = NULL; + pReader->status.suspendInvoked = true; // record the suspend status + if (pStatus->loadFromFile) { SFileDataBlockInfo* pBlockInfo = getCurrentBlockInfo(&pReader->status.blockIter); if (pBlockInfo != NULL) { @@ -4181,55 +4157,34 @@ int32_t tsdbReaderSuspend2(STsdbReader* pReader) { tsdbDataFileReaderClose(&pReader->pFileReader); - SCostSummary* pCost = &pReader->cost; - + SReadCostSummary* pCost = &pReader->cost; pReader->status.pLDataIterArray = destroySttBlockReader(pReader->status.pLDataIterArray, &pCost->sttCost); pReader->status.pLDataIterArray = taosArrayInit(4, POINTER_BYTES); - - // resetDataBlockScanInfo excluding lastKey - STableBlockScanInfo** p = NULL; - - int32_t iter = 0; - while ((p = tSimpleHashIterate(pStatus->pTableMap, p, &iter)) != NULL) { - STableBlockScanInfo* pInfo = *(STableBlockScanInfo**)p; - clearMemIterInfo(pInfo); - pInfo->sttKeyInfo.status = STT_FILE_READER_UNINIT; - pInfo->delSkyline = taosArrayDestroy(pInfo->delSkyline); - - pInfo->pFileDelData = taosArrayDestroy(pInfo->pFileDelData); - } - } else { - // resetDataBlockScanInfo excluding lastKey - STableBlockScanInfo** p = NULL; - int32_t iter = 0; - - while ((p = tSimpleHashIterate(pStatus->pTableMap, p, &iter)) != NULL) { - STableBlockScanInfo* pInfo = *(STableBlockScanInfo**)p; - clearMemIterInfo(pInfo); - pInfo->sttKeyInfo.status = STT_FILE_READER_UNINIT; - pInfo->delSkyline = taosArrayDestroy(pInfo->delSkyline); - } - - pBlockScanInfo = (pStatus->pTableIter == NULL) ? NULL : *pStatus->pTableIter; - if (pBlockScanInfo) { - // save lastKey to restore memory iterator - STimeWindow w = pReader->resBlockInfo.pResBlock->info.window; - pBlockScanInfo->lastProcKey = ASCENDING_TRAVERSE(pReader->info.order) ? w.ekey : w.skey; - - clearMemIterInfo(pBlockScanInfo); - pBlockScanInfo->sttKeyInfo.status = STT_FILE_READER_UNINIT; - pBlockScanInfo->delSkyline = taosArrayDestroy(pBlockScanInfo->delSkyline); - - pBlockScanInfo->pBlockList = taosArrayDestroy(pBlockScanInfo->pBlockList); - pBlockScanInfo->pBlockIdxList = taosArrayDestroy(pBlockScanInfo->pBlockIdxList); - // TODO: keep skyline for reuse - } } + // resetDataBlockScanInfo excluding lastKey + STableBlockScanInfo** p = NULL; + + int32_t step = ASCENDING_TRAVERSE(pReader->info.order)? 1:-1; + + int32_t iter = 0; + while ((p = tSimpleHashIterate(pStatus->pTableMap, p, &iter)) != NULL) { + STableBlockScanInfo* pInfo = *(STableBlockScanInfo**)p; + clearBlockScanInfo(pInfo); + pInfo->sttKeyInfo.nextProcKey = pInfo->lastProcKey + step; + } + + pStatus->uidList.currentIndex = 0; + initReaderStatus(pStatus); + tsdbUntakeReadSnap2(pReader, pReader->pReadSnap, false); pReader->pReadSnap = NULL; pReader->flag = READER_STATUS_SUSPEND; +#if SUSPEND_RESUME_TEST + tsem_post(&pReader->resumeAfterSuspend); +#endif + tsdbDebug("reader: %p suspended uid %" PRIu64 " in this query %s", pReader, pBlockScanInfo ? pBlockScanInfo->uid : 0, pReader->idStr); return code; @@ -4384,6 +4339,16 @@ int32_t tsdbNextDataBlock2(STsdbReader* pReader, bool* hasNext) { SReaderStatus* pStatus = &pReader->status; + // NOTE: the following codes is used to perform test for suspend/resume for tsdbReader when it blocks the commit + // the data should be ingested in round-robin and all the child tables should be createted before ingesting data + // the version range of query will be used to identify the correctness of suspend/resume functions. + // this function will blocked before loading the SECOND block from vnode-buffer, and restart itself from sst-files +#if SUSPEND_RESUME_TEST + if (!pReader->status.suspendInvoked && !pReader->status.loadFromFile) { + tsem_wait(&pReader->resumeAfterSuspend); + } +#endif + code = tsdbAcquireReader(pReader); qTrace("tsdb/read: %p, take read mutex, code: %d", pReader, code); diff --git a/source/dnode/vnode/src/tsdb/tsdbReadUtil.h b/source/dnode/vnode/src/tsdb/tsdbReadUtil.h index 60e6e6960a..709e311ff0 100644 --- a/source/dnode/vnode/src/tsdb/tsdbReadUtil.h +++ b/source/dnode/vnode/src/tsdb/tsdbReadUtil.h @@ -96,7 +96,7 @@ typedef struct SResultBlockInfo { int64_t capacity; } SResultBlockInfo; -typedef struct SCostSummary { +typedef struct SReadCostSummary { int64_t numOfBlocks; double blockLoadTime; double buildmemBlock; @@ -110,7 +110,7 @@ typedef struct SCostSummary { double createScanInfoList; double createSkylineIterTime; double initLastBlockReader; -} SCostSummary; +} SReadCostSummary; typedef struct STableUidList { uint64_t* tableUidList; // access table uid list in uid ascending order list @@ -122,12 +122,6 @@ typedef struct { int32_t numOfSttFiles; } SBlockNumber; -typedef struct SBlockIndex { - int32_t ordinalIndex; - int64_t inFileOffset; - STimeWindow window; // todo replace it with overlap flag. -} SBlockIndex; - typedef struct SBlockOrderWrapper { int64_t uid; int64_t offset; @@ -192,6 +186,7 @@ typedef struct SFileBlockDumpInfo { } SFileBlockDumpInfo; typedef struct SReaderStatus { + bool suspendInvoked; bool loadFromFile; // check file stage bool composedDataBlock; // the returned data block is a composed block or not SSHashObj* pTableMap; // SHash @@ -220,7 +215,8 @@ struct STsdbReader { int32_t type; // query type: 1. retrieve all data blocks, 2. retrieve direct prev|next rows SBlockLoadSuppInfo suppInfo; STsdbReadSnap* pReadSnap; - SCostSummary cost; + tsem_t resumeAfterSuspend; + SReadCostSummary cost; SHashObj** pIgnoreTables; SSHashObj* pSchemaMap; // keep the retrieved schema info, to avoid the overhead by repeatly load schema SDataFileReader* pFileReader; // the file reader From 28e17a61b932000c32c05d1ffe5c8a444e926cb3 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 29 Nov 2023 19:28:53 +0800 Subject: [PATCH 187/195] fix(tsdb): fix invalid read. --- source/dnode/vnode/src/tsdb/tsdbRead2.c | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/tsdb/tsdbRead2.c b/source/dnode/vnode/src/tsdb/tsdbRead2.c index 4138d91f5d..cb899f9ee8 100644 --- a/source/dnode/vnode/src/tsdb/tsdbRead2.c +++ b/source/dnode/vnode/src/tsdb/tsdbRead2.c @@ -3766,6 +3766,7 @@ int32_t buildDataBlockFromBufImpl(STableBlockScanInfo* pBlockScanInfo, int64_t e } if (row.type == TSDBROW_ROW_FMT) { + int64_t ts = row.pTSRow->ts;; code = doAppendRowFromTSRow(pBlock, pReader, row.pTSRow, pBlockScanInfo); if (freeTSRow) { @@ -3776,7 +3777,7 @@ int32_t buildDataBlockFromBufImpl(STableBlockScanInfo* pBlockScanInfo, int64_t e return code; } - pBlockScanInfo->lastProcKey = row.pTSRow->ts; + pBlockScanInfo->lastProcKey = ts; } else { code = doAppendRowFromFileBlock(pBlock, pReader, row.pBlockData, row.iRow); if (code) { From 08cc97697534e69f41d96319cd6f4c5d6d338a93 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 29 Nov 2023 19:57:30 +0800 Subject: [PATCH 188/195] fix crash --- source/libs/stream/src/streamBackendRocksdb.c | 1 + source/libs/stream/src/streamMeta.c | 4 ++++ 2 files changed, 5 insertions(+) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 19247166b3..630650025d 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1631,6 +1631,7 @@ void* taskDbAddRef(void* pTaskDb) { return taosAcquireRef(taskDbWrapperId, pBackend->refId); } void taskDbRemoveRef(void* pTaskDb) { + if (pTaskDb == NULL) return; STaskDbWrapper* pBackend = pTaskDb; taosReleaseRef(taskDbWrapperId, pBackend->refId); } diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 0e137a673e..bd23e41a84 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -255,6 +255,8 @@ int32_t streamTaskSetDb(SStreamMeta* pMeta, void* arg, char* key) { pTask->backendRefId = pBackend->refId; pTask->pBackend = pBackend; taosThreadMutexUnlock(&pMeta->backendMutex); + + stDebug("s-task:0x%x set backend %p", pTask->id.taskId, pBackend); return 0; } @@ -272,6 +274,8 @@ int32_t streamTaskSetDb(SStreamMeta* pMeta, void* arg, char* key) { taosHashPut(pMeta->pTaskDbUnique, key, strlen(key), &pBackend, sizeof(void*)); taosThreadMutexUnlock(&pMeta->backendMutex); + + stDebug("s-task:0x%x set backend %p", pTask->id.taskId, pBackend); return 0; } SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandFunc, int32_t vgId, int64_t stage) { From 64947f9d20d6e05171a71f656545cb5eaabf8efe Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 29 Nov 2023 20:16:41 +0800 Subject: [PATCH 189/195] fix mem leak --- source/libs/transport/src/transSvr.c | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/source/libs/transport/src/transSvr.c b/source/libs/transport/src/transSvr.c index bf73c253bc..017969b4e5 100644 --- a/source/libs/transport/src/transSvr.c +++ b/source/libs/transport/src/transSvr.c @@ -159,7 +159,7 @@ static void uvStartSendResp(SSvrMsg* msg); static void uvNotifyLinkBrokenToApp(SSvrConn* conn); -static FORCE_INLINE void destroySmsg(SSvrMsg* smsg); +static FORCE_INLINE void destroySmsg(SSvrMsg* smsg); static FORCE_INLINE SSvrConn* createConn(void* hThrd); static FORCE_INLINE void destroyConn(SSvrConn* conn, bool clear /*clear handle or not*/); static FORCE_INLINE void destroyConnRegArg(SSvrConn* conn); @@ -1499,6 +1499,7 @@ int transSendResponse(const STransMsg* msg) { } SExHandle* exh = msg->info.handle; if (exh == NULL) { + rpcFreeCont(msg->pCont); return 0; } int64_t refId = msg->info.refId; From 3069069e0f1e4dcf08aaf28e6e8686c5ac9fadb2 Mon Sep 17 00:00:00 2001 From: Bob Liu Date: Wed, 29 Nov 2023 22:47:31 +0800 Subject: [PATCH 190/195] adjust parseOneStbRow; add case for stable --- source/libs/parser/src/parInsertSql.c | 26 ++++--- tests/system-test/1-insert/ts-4272.py | 107 ++++++++++++++++++-------- 2 files changed, 92 insertions(+), 41 deletions(-) diff --git a/source/libs/parser/src/parInsertSql.c b/source/libs/parser/src/parInsertSql.c index 24cdf40d31..41d4a70675 100644 --- a/source/libs/parser/src/parInsertSql.c +++ b/source/libs/parser/src/parInsertSql.c @@ -1584,7 +1584,7 @@ typedef struct SStbRowsDataContext { bool isJsonTag; } SStbRowsDataContext; -typedef union SRowsDataContext{ +typedef struct SRowsDataContext{ STableDataCxt* pTableDataCxt; SStbRowsDataContext* pStbRowsCxt; } SRowsDataContext; @@ -1803,8 +1803,9 @@ static void clearStbRowsDataContext(SStbRowsDataContext* pStbRowsCxt) { } static int32_t parseOneStbRow(SInsertParseContext* pCxt, SVnodeModifyOpStmt* pStmt, const char** ppSql, - SStbRowsDataContext* pStbRowsCxt, bool* pGotRow, SToken* pToken) { + SRowsDataContext* pRowsDataCxt, bool* pGotRow, SToken* pToken) { bool bFirstTable = false; + SStbRowsDataContext* pStbRowsCxt = pRowsDataCxt->pStbRowsCxt; int32_t code = getStbRowValues(pCxt, pStmt, ppSql, pStbRowsCxt, pGotRow, pToken, &bFirstTable); if (code != TSDB_CODE_SUCCESS || !*pGotRow) { return code; @@ -1814,20 +1815,16 @@ static int32_t parseOneStbRow(SInsertParseContext* pCxt, SVnodeModifyOpStmt* pSt code = processCtbAutoCreationAndCtbMeta(pCxt, pStmt, pStbRowsCxt); } - STableDataCxt* pTableDataCxt = NULL; code = insGetTableDataCxt(pStmt->pTableBlockHashObj, &pStbRowsCxt->pCtbMeta->uid, sizeof(pStbRowsCxt->pCtbMeta->uid), - pStbRowsCxt->pCtbMeta, &pStbRowsCxt->pCreateCtbReq, &pTableDataCxt, false, true); - initTableColSubmitData(pTableDataCxt); + pStbRowsCxt->pCtbMeta, &pStbRowsCxt->pCreateCtbReq, &pRowsDataCxt->pTableDataCxt, false, true); + initTableColSubmitData(pRowsDataCxt->pTableDataCxt); if (code == TSDB_CODE_SUCCESS) { + STableDataCxt* pTableDataCxt = pRowsDataCxt->pTableDataCxt; SRow** pRow = taosArrayReserve(pTableDataCxt->pData->aRowP, 1); code = tRowBuild(pStbRowsCxt->aColVals, pTableDataCxt->pSchema, pRow); if (TSDB_CODE_SUCCESS == code) { insCheckTableDataOrder(pTableDataCxt, TD_ROW_KEY(*pRow)); } - - void* pData = pTableDataCxt; - taosHashPut(pStmt->pTableCxtHashObj, &pStbRowsCxt->pCtbMeta->uid, sizeof(pStbRowsCxt->pCtbMeta->uid), &pData, - POINTER_BYTES); } if (code == TSDB_CODE_SUCCESS) { @@ -1924,7 +1921,7 @@ static int32_t parseValues(SInsertParseContext* pCxt, SVnodeModifyOpStmt* pStmt, if (!pStmt->stbSyntax) { code = parseOneRow(pCxt, &pStmt->pSql, rowsDataCxt.pTableDataCxt, &gotRow, pToken); } else { - code = parseOneStbRow(pCxt, pStmt, &pStmt->pSql, rowsDataCxt.pStbRowsCxt, &gotRow, pToken); + code = parseOneStbRow(pCxt, pStmt, &pStmt->pSql, &rowsDataCxt, &gotRow, pToken); } } @@ -1988,7 +1985,14 @@ static int32_t parseCsvFile(SInsertParseContext* pCxt, SVnodeModifyOpStmt* pStmt if (!pStmt->stbSyntax) { code = parseOneRow(pCxt, (const char**)&pRow, rowsDataCxt.pTableDataCxt, &gotRow, &token); } else { - code = parseOneStbRow(pCxt, pStmt, (const char**)&pRow, rowsDataCxt.pStbRowsCxt, &gotRow, &token); + rowsDataCxt.pTableDataCxt = NULL; + code = parseOneStbRow(pCxt, pStmt, (const char**)&pRow, &rowsDataCxt, &gotRow, &token); + if (code == TSDB_CODE_SUCCESS) { + SStbRowsDataContext* pStbRowsCxt = rowsDataCxt.pStbRowsCxt; + void* pData = rowsDataCxt.pTableDataCxt; + taosHashPut(pStmt->pTableCxtHashObj, &pStbRowsCxt->pCtbMeta->uid, sizeof(pStbRowsCxt->pCtbMeta->uid), &pData, + POINTER_BYTES); + } } if (code && firstLine) { firstLine = false; diff --git a/tests/system-test/1-insert/ts-4272.py b/tests/system-test/1-insert/ts-4272.py index 4e837d646d..bb81305eb3 100644 --- a/tests/system-test/1-insert/ts-4272.py +++ b/tests/system-test/1-insert/ts-4272.py @@ -13,22 +13,26 @@ class TDTestCase: def init(self, conn, logSql, replicaVar=1): self.replicaVar = int(replicaVar) self.testcasePath = os.path.split(__file__)[0] - self.testcaseFilename = os.path.split(__file__)[-1] + self.testcasefilename = os.path.split(__file__)[-1] self.ts = 1700638570000 # 2023-11-22T07:36:10.000Z + self.db = 'db1' self.tb1 = 'd001' self.tb2 = 'd002' - self.tag1 = 'using meters(groupId) tags(1)' - self.tag2 = 'using meters(groupId) tags(2)' + self.stable0 = "meters" + self.stable1 = "stb_1" + self.stable2 = "stb_null" + self.tag1 = f'using {self.stable0}(groupId) tags(1)' + self.tag2 = f'using {self.stable0}(groupId) tags(2)' self.file1 = f"{self.testcasePath}/b.csv" self.file2 = f"{self.testcasePath}/c.csv" - os.system("rm -rf %s/b.csv" %self.testcasePath) + #os.system("rm -rf %s/b.csv" %self.testcasePath) tdLog.debug(f"start to excute {__file__}") tdSql.init(conn.cursor(), logSql) def check_count(self, rows, records): - tdSql.execute(f"use d1;") - tdSql.query(f"select tbname,count(*) from meters group by tbname order by tbname;") + tdSql.execute(f"use {self.db};") + tdSql.query(f"select tbname,count(*) from {self.stable0} group by tbname order by tbname;") tdSql.checkRows(rows) for i in range(rows): tdSql.checkData(i, 1, records[i]) @@ -36,21 +40,23 @@ class TDTestCase: def reset_tb(self): # create database and tables # os.system("taos -s 'drop database if exists d1;'") - # os.system("taos -s 'create database d1;use d1;CREATE STABLE meters (ts timestamp, current float, voltage int, phase float) TAGS (location binary(64), groupId int);'") - # os.system(f"taos -s 'use d1;Create table d2001 using meters(groupId) tags(5);'") - # res = os.system(f"taos -s 'use d1;Create table d2002 using meters(groupId) tags(6);'") + # os.system("taos -s 'create database d1;use d1;create stable meters (ts timestamp, current float, voltage int, phase float) tags (location binary(64), groupId int);'") + # os.system(f"taos -s 'use d1;create table d2001 using meters(groupId) tags(5);'") + # res = os.system(f"taos -s 'use d1;create table d2002 using meters(groupId) tags(6);'") # if (0 != res): # tdLog.exit(f"create tb error") - tdSql.execute("drop database if exists d1;") - tdSql.execute("create database d1;") - tdSql.execute("use d1;") - tdSql.execute("CREATE STABLE meters (ts timestamp, current float, voltage int, phase float) TAGS (location binary(64), groupId int);") - tdSql.execute("Create table d2001 using meters(groupId) tags(5);") - tdSql.execute("Create table d2002 using meters(groupId) tags(6);") + tdSql.execute(f"drop database if exists {self.db};") + tdSql.execute(f"create database {self.db};") + tdSql.execute(f"use {self.db};") + tdSql.execute(f"create stable {self.stable0} (ts timestamp, current float, voltage int, phase float) tags (location binary(64), groupId int);") + tdSql.execute(f"create table {self.tb1} {self.tag1};") + tdSql.execute(f"create table {self.tb2} {self.tag2};") + tdSql.execute(f"create stable {self.stable1} (ts timestamp , q_int int , q_bigint bigint , q_smallint smallint , q_tinyint tinyint , q_float float , q_double double , q_bool bool , q_binary binary(100) , q_nchar nchar(100) , q_ts timestamp , q_int_null int , q_bigint_null bigint , q_smallint_null smallint , q_tinyint_null tinyint, q_float_null float , q_double_null double , q_bool_null bool , q_binary_null binary(20) , q_nchar_null nchar(20) , q_ts_null timestamp) tags(loc nchar(100) , t_int int , t_bigint bigint , t_smallint smallint , t_tinyint tinyint, t_bool bool , t_binary binary(100) , t_nchar nchar(100) ,t_float float , t_double double , t_ts timestamp);") + tdSql.execute(f"create stable {self.stable2} (ts timestamp , q_int int , q_bigint bigint , q_smallint smallint , q_tinyint tinyint , q_float float , q_double double , q_bool bool , q_binary binary(100) , q_nchar nchar(100) , q_ts timestamp , q_int_null int , q_bigint_null bigint , q_smallint_null smallint , q_tinyint_null tinyint, q_float_null float , q_double_null double , q_bool_null bool , q_binary_null binary(20) , q_nchar_null nchar(20) , q_ts_null timestamp) tags(loc nchar(100) , t_int int , t_bigint bigint , t_smallint smallint , t_tinyint tinyint, t_bool bool , t_binary binary(100) , t_nchar nchar(100) ,t_float float , t_double double , t_ts timestamp);") def test(self, sql): - sql = "use d1;" + sql + sql = f"use {self.db};" + sql res = os.system(f'taos -s "{sql}"') # if (0 != res): # tdLog.exit(f"taos sql error") @@ -58,41 +64,41 @@ class TDTestCase: def check(self): # same table, auto create + create - sql = f"INSERT INTO {self.tb1} {self.tag1} file '{self.file1}' {self.tb1} {self.tag1} file '{self.file2}';" + sql = f"insert into {self.tb1} {self.tag1} file '{self.file1}' {self.tb1} {self.tag1} file '{self.file2}';" self.test(sql) # same table, create + insert - sql = f"INSERT INTO {self.tb1} {self.tag1} file '{self.file1}' {self.tb1} file '{self.file2}';" + sql = f"insert into {self.tb1} {self.tag1} file '{self.file1}' {self.tb1} file '{self.file2}';" self.test(sql) # same table, insert + create - sql = f"INSERT INTO {self.tb1} file '{self.file1}' {self.tb1} {self.tag1} file '{self.file2}';" + sql = f"insert into {self.tb1} file '{self.file1}' {self.tb1} {self.tag1} file '{self.file2}';" self.test(sql) # same table, insert + insert - sql = f"INSERT INTO {self.tb1} file '{self.file1}' {self.tb1} file '{self.file2}';" + sql = f"insert into {self.tb1} file '{self.file1}' {self.tb1} file '{self.file2}';" self.test(sql) # diff table auto create + create - sql = f"INSERT INTO {self.tb1} {self.tag1} file '{self.file1}' {self.tb2} {self.tag2} file '{self.file2}';" + sql = f"insert into {self.tb1} {self.tag1} file '{self.file1}' {self.tb2} {self.tag2} file '{self.file2}';" self.test(sql) # diff table, create + insert - sql = f"INSERT INTO {self.tb1} {self.tag1} file '{self.file1}' {self.tb2} file '{self.file2}';" + sql = f"insert into {self.tb1} {self.tag1} file '{self.file1}' {self.tb2} file '{self.file2}';" self.test(sql) # diff table, insert + create - sql = f"INSERT INTO {self.tb1} file '{self.file1}' {self.tb2} {self.tag2} file '{self.file2}';" + sql = f"insert into {self.tb1} file '{self.file1}' {self.tb2} {self.tag2} file '{self.file2}';" self.test(sql) # diff table, insert + insert - sql = f"INSERT INTO {self.tb1} file '{self.file1}' {self.tb2} file '{self.file2}';" + sql = f"insert into {self.tb1} file '{self.file1}' {self.tb2} file '{self.file2}';" self.test(sql) # bigNum = 1010000 # self.check_count(5, [2100, 2100, bigNum, bigNum, bigNum]) - result = os.popen("taos -s 'select count(*) from d1.%s'" %self.tb1) + result = os.popen("taos -s 'select count(*) from %s.%s'" %(self.db, self.tb1)) res = result.read() if (f"OK" in res): tdLog.info(f"check count success") @@ -112,7 +118,7 @@ class TDTestCase: def test_mix(self): #forbid use both value and file in one insert - result = os.popen(f"insert into {self.tb1} file '{self.testcasePath}/csv/2k.csv' {self.tb2} values('2021-07-13 14:06:34.630', 10.2, 219, 0.32);") + result = os.popen(f"insert into {self.tb1} file '{self.file2}' {self.tb2} values('2021-07-13 14:06:34.630', 10.2, 219, 0.32);") res = result.read() if (f"error" in res): tdLog.info(f"forbid success") @@ -130,28 +136,69 @@ class TDTestCase: print("end csv data prepare") # auto create + insert - sql = f"INSERT INTO d2001 using meters(groupId) tags(5) FILE '{self.file1}';" + sql = f"insert into {self.tb1} {self.tag1} file '{self.file1}';" self.test(sql) # only insert - sql = f"INSERT INTO d2002 FILE '{self.file2}';" + sql = f"insert into {self.tb2} file '{self.file2}';" self.test(sql) + print("end insert to table") #tdSql.execute(f"use d1;") - tdSql.query(f"select tbname,count(*) from meters group by tbname order by tbname;") + tdSql.query(f"select tbname,count(*) from {self.stable0} group by tbname order by tbname;") tdSql.checkRows(2) tdSql.checkData(0, 1, rowNum1) tdSql.checkData(1, 1, rowNum2) + print("check insert file to table success") + + def make_stable_csv(self, filepath, once, qtime, startts, table_name): + f = open(filepath, 'w') + with f: + writer = csv.writer(f) + for j in range(qtime): + offset = j*once + ts = startts + offset + rows = [] + for i in range(once): + rows.append([table_name, ts + i, offset + i, 'NULL']) + writer.writerows(rows) + f.close() + print(datetime.now(), filepath, " ready!") + + def test_stable_csv(self): + # prepare csv + print("start stable_csv data prepare") + once = 10000 + qtime1 = 101 + qtime2 = 100 + # rowNum1 = qtime1 * once + # rowNum2 = qtime2 * once + child_1 = f"{self.stable1}_1" + child_2 = f"{self.stable2}_1" + self.make_stable_csv(self.file1, once, qtime1, self.ts - 86400000, child_1) + self.make_stable_csv(self.file2, once, qtime2, self.ts, child_2) + print("end stable_csv data prepare") + + # insert create child table of stable + sql = f"insert into {self.db}.{self.stable1}(tbname,ts,q_int,q_binary) file '{self.file1}' {self.db}.{self.stable2}(tbname,ts,q_int,q_binary) file '{self.file2}';" + self.test(sql) + print("end insert to stable") + + #tdSql.execute(f"insert into {self.db}.{child_1}(ts, q_int) values(now, 1);") + tdSql.query(f"select tbname,count(*) from {self.stable1} group by tbname order by tbname;") + tdSql.checkRows(0) + print("check stable success") def run(self): tdSql.prepare() self.reset_tb() + self.test_stable_csv() self.test_bigcsv() self.test_mix() self.check() + tdSql.close() def stop(self): - tdSql.close() tdLog.success(f"{__file__} successfully executed") tdCases.addLinux(__file__, TDTestCase()) From 35ae8a4c0d8f9df9dfe9b1581ee53f550a4edc10 Mon Sep 17 00:00:00 2001 From: Bob Liu Date: Thu, 30 Nov 2023 00:25:05 +0800 Subject: [PATCH 191/195] adjust parseOneStbRow --- source/libs/parser/src/parInsertSql.c | 30 +++++++++++++-------------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/source/libs/parser/src/parInsertSql.c b/source/libs/parser/src/parInsertSql.c index 41d4a70675..31b016458a 100644 --- a/source/libs/parser/src/parInsertSql.c +++ b/source/libs/parser/src/parInsertSql.c @@ -1584,7 +1584,7 @@ typedef struct SStbRowsDataContext { bool isJsonTag; } SStbRowsDataContext; -typedef struct SRowsDataContext{ +typedef union SRowsDataContext{ STableDataCxt* pTableDataCxt; SStbRowsDataContext* pStbRowsCxt; } SRowsDataContext; @@ -1802,10 +1802,10 @@ static void clearStbRowsDataContext(SStbRowsDataContext* pStbRowsCxt) { taosMemoryFreeClear(pStbRowsCxt->pCreateCtbReq); } -static int32_t parseOneStbRow(SInsertParseContext* pCxt, SVnodeModifyOpStmt* pStmt, const char** ppSql, - SRowsDataContext* pRowsDataCxt, bool* pGotRow, SToken* pToken) { - bool bFirstTable = false; - SStbRowsDataContext* pStbRowsCxt = pRowsDataCxt->pStbRowsCxt; +static int32_t parseOneStbRow(SInsertParseContext* pCxt, SVnodeModifyOpStmt* pStmt, const char** ppSql, + SStbRowsDataContext* pStbRowsCxt, bool* pGotRow, SToken* pToken, + STableDataCxt** ppTableDataCxt) { + bool bFirstTable = false; int32_t code = getStbRowValues(pCxt, pStmt, ppSql, pStbRowsCxt, pGotRow, pToken, &bFirstTable); if (code != TSDB_CODE_SUCCESS || !*pGotRow) { return code; @@ -1816,14 +1816,13 @@ static int32_t parseOneStbRow(SInsertParseContext* pCxt, SVnodeModifyOpStmt* pSt } code = insGetTableDataCxt(pStmt->pTableBlockHashObj, &pStbRowsCxt->pCtbMeta->uid, sizeof(pStbRowsCxt->pCtbMeta->uid), - pStbRowsCxt->pCtbMeta, &pStbRowsCxt->pCreateCtbReq, &pRowsDataCxt->pTableDataCxt, false, true); - initTableColSubmitData(pRowsDataCxt->pTableDataCxt); + pStbRowsCxt->pCtbMeta, &pStbRowsCxt->pCreateCtbReq, ppTableDataCxt, false, true); + initTableColSubmitData(*ppTableDataCxt); if (code == TSDB_CODE_SUCCESS) { - STableDataCxt* pTableDataCxt = pRowsDataCxt->pTableDataCxt; - SRow** pRow = taosArrayReserve(pTableDataCxt->pData->aRowP, 1); - code = tRowBuild(pStbRowsCxt->aColVals, pTableDataCxt->pSchema, pRow); + SRow** pRow = taosArrayReserve((*ppTableDataCxt)->pData->aRowP, 1); + code = tRowBuild(pStbRowsCxt->aColVals, (*ppTableDataCxt)->pSchema, pRow); if (TSDB_CODE_SUCCESS == code) { - insCheckTableDataOrder(pTableDataCxt, TD_ROW_KEY(*pRow)); + insCheckTableDataOrder(*ppTableDataCxt, TD_ROW_KEY(*pRow)); } } @@ -1921,7 +1920,8 @@ static int32_t parseValues(SInsertParseContext* pCxt, SVnodeModifyOpStmt* pStmt, if (!pStmt->stbSyntax) { code = parseOneRow(pCxt, &pStmt->pSql, rowsDataCxt.pTableDataCxt, &gotRow, pToken); } else { - code = parseOneStbRow(pCxt, pStmt, &pStmt->pSql, &rowsDataCxt, &gotRow, pToken); + STableDataCxt* pTableDataCxt = NULL; + code = parseOneStbRow(pCxt, pStmt, &pStmt->pSql, rowsDataCxt.pStbRowsCxt, &gotRow, pToken, &pTableDataCxt); } } @@ -1985,11 +1985,11 @@ static int32_t parseCsvFile(SInsertParseContext* pCxt, SVnodeModifyOpStmt* pStmt if (!pStmt->stbSyntax) { code = parseOneRow(pCxt, (const char**)&pRow, rowsDataCxt.pTableDataCxt, &gotRow, &token); } else { - rowsDataCxt.pTableDataCxt = NULL; - code = parseOneStbRow(pCxt, pStmt, (const char**)&pRow, &rowsDataCxt, &gotRow, &token); + STableDataCxt* pTableDataCxt = NULL; + code = parseOneStbRow(pCxt, pStmt, (const char**)&pRow, rowsDataCxt.pStbRowsCxt, &gotRow, &token, &pTableDataCxt); if (code == TSDB_CODE_SUCCESS) { SStbRowsDataContext* pStbRowsCxt = rowsDataCxt.pStbRowsCxt; - void* pData = rowsDataCxt.pTableDataCxt; + void* pData = pTableDataCxt; taosHashPut(pStmt->pTableCxtHashObj, &pStbRowsCxt->pCtbMeta->uid, sizeof(pStbRowsCxt->pCtbMeta->uid), &pData, POINTER_BYTES); } From ab2c7abca1e53ea80cb6c6f8cb3d36faad82fec8 Mon Sep 17 00:00:00 2001 From: Minglei Jin Date: Thu, 30 Nov 2023 09:14:26 +0800 Subject: [PATCH 192/195] tsdb/cache-del: commit before get from rocks --- source/dnode/vnode/src/tsdb/tsdbCache.c | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/source/dnode/vnode/src/tsdb/tsdbCache.c b/source/dnode/vnode/src/tsdb/tsdbCache.c index e8aa755073..5076599753 100644 --- a/source/dnode/vnode/src/tsdb/tsdbCache.c +++ b/source/dnode/vnode/src/tsdb/tsdbCache.c @@ -1131,13 +1131,13 @@ int32_t tsdbCacheDel(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSKEY sKey, TSKE char **values_list = taosMemoryCalloc(num_keys * 2, sizeof(char *)); size_t *values_list_sizes = taosMemoryCalloc(num_keys * 2, sizeof(size_t)); char **errs = taosMemoryCalloc(num_keys * 2, sizeof(char *)); + + (void)tsdbCacheCommit(pTsdb); + taosThreadMutexLock(&pTsdb->lruMutex); - pTsdb->flushState.flush_count = ROCKS_BATCH_SIZE; - taosLRUCacheApply(pTsdb->lruCache, tsdbCacheFlushDirty, &pTsdb->flushState); - taosThreadMutexLock(&pTsdb->rCache.rMutex); - rocksMayWrite(pTsdb, true, false, false); + // rocksMayWrite(pTsdb, true, false, false); rocksdb_multi_get(pTsdb->rCache.db, pTsdb->rCache.readoptions, num_keys * 2, (const char *const *)keys_list, keys_list_sizes, values_list, values_list_sizes, errs); taosThreadMutexUnlock(&pTsdb->rCache.rMutex); From 547e6739554213f40537b29c2a9706f1950f0c93 Mon Sep 17 00:00:00 2001 From: Hongze Cheng Date: Thu, 30 Nov 2023 09:20:18 +0800 Subject: [PATCH 193/195] fix: coredump caused by typo --- source/dnode/vnode/src/vnd/vnodeAsync.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/vnd/vnodeAsync.c b/source/dnode/vnode/src/vnd/vnodeAsync.c index bc2d1e44c1..c95d2324aa 100644 --- a/source/dnode/vnode/src/vnd/vnodeAsync.c +++ b/source/dnode/vnode/src/vnd/vnodeAsync.c @@ -128,7 +128,7 @@ static int32_t vnodeAsyncTaskDone(SVAsync *async, SVATask *task) { nextTask->next->prev = nextTask->prev; nextTask->prev->next = nextTask->next; } else { - nextTask->priority++; + nextTask->priorScore++; int32_t newPriority = VATASK_PIORITY(nextTask); if (newPriority != i) { // remove from current priority queue From ca3e669bfd27184bf1f0564ea3438911f38a9d1b Mon Sep 17 00:00:00 2001 From: danielclow <106956386+danielclow@users.noreply.github.com> Date: Thu, 30 Nov 2023 10:19:42 +0800 Subject: [PATCH 194/195] docs: replace absolute links with relative links --- docs/en/05-get-started/03-package.md | 2 +- docs/en/05-get-started/_pkg_install.mdx | 2 +- docs/en/05-get-started/index.md | 2 +- docs/en/07-develop/01-connect/_connect_java.mdx | 2 +- docs/en/07-develop/01-connect/index.md | 4 ++-- docs/en/07-develop/02-model/index.mdx | 14 +++++++------- .../07-develop/03-insert-data/01-sql-writing.mdx | 8 ++++---- .../03-insert-data/20-kafka-writting.mdx | 2 +- .../07-develop/03-insert-data/30-influxdb-line.mdx | 2 +- docs/en/07-develop/04-query-data/index.mdx | 2 +- docs/en/07-develop/index.md | 4 ++-- docs/en/10-deployment/01-deploy.md | 4 ++-- docs/en/12-taos-sql/26-udf.md | 2 +- docs/en/13-operation/10-monitor.md | 2 +- docs/en/14-reference/02-rest-api/02-rest-api.mdx | 2 +- docs/en/14-reference/03-connector/03-cpp.mdx | 6 +++--- docs/en/14-reference/03-connector/04-java.mdx | 6 +++--- docs/en/14-reference/03-connector/05-go.mdx | 2 +- docs/en/14-reference/03-connector/06-rust.mdx | 2 +- docs/en/14-reference/03-connector/07-python.mdx | 12 ++++++------ docs/en/14-reference/03-connector/08-node.mdx | 4 ++-- docs/en/14-reference/03-connector/09-csharp.mdx | 4 ++-- docs/en/14-reference/03-connector/80-php.mdx | 2 +- docs/en/14-reference/03-connector/_preparation.mdx | 2 +- docs/en/14-reference/04-taosadapter.md | 4 ++-- docs/en/14-reference/06-taosdump.md | 2 +- docs/en/14-reference/08-taos-shell.md | 4 ++-- docs/en/14-reference/12-config/index.md | 2 +- .../en/14-reference/13-schemaless/13-schemaless.md | 2 +- docs/en/20-third-party/02-prometheus.md | 2 +- docs/en/20-third-party/03-telegraf.md | 4 ++-- docs/en/20-third-party/05-collectd.md | 2 +- docs/en/20-third-party/07-icinga2.md | 2 +- docs/en/20-third-party/08-tcollector.md | 2 +- docs/en/20-third-party/09-emq-broker.md | 2 +- docs/en/20-third-party/11-kafka.md | 4 ++-- docs/en/20-third-party/_deploytaosadapter.mdx | 2 +- docs/en/25-application/01-telegraf.md | 2 +- docs/en/25-application/02-collectd.md | 2 +- docs/en/25-application/_03-immigrate.md | 4 ++-- 40 files changed, 68 insertions(+), 68 deletions(-) diff --git a/docs/en/05-get-started/03-package.md b/docs/en/05-get-started/03-package.md index 3e3c04682f..d34df2c970 100644 --- a/docs/en/05-get-started/03-package.md +++ b/docs/en/05-get-started/03-package.md @@ -243,7 +243,7 @@ You can use the TDengine CLI to monitor your TDengine deployment and execute ad taos ``` -The TDengine CLI displays a welcome message and version information to indicate that its connection to the TDengine service was successful. If an error message is displayed, see the [FAQ](/train-faq/faq) for troubleshooting information. At the following prompt, you can execute SQL commands. +The TDengine CLI displays a welcome message and version information to indicate that its connection to the TDengine service was successful. If an error message is displayed, see the [FAQ](../../train-faq/faq) for troubleshooting information. At the following prompt, you can execute SQL commands. ```cmd taos> diff --git a/docs/en/05-get-started/_pkg_install.mdx b/docs/en/05-get-started/_pkg_install.mdx index 32d7c1f376..2372d2ff26 100644 --- a/docs/en/05-get-started/_pkg_install.mdx +++ b/docs/en/05-get-started/_pkg_install.mdx @@ -10,7 +10,7 @@ Between official releases, beta versions may be released that contain new featur -For information about installing TDengine, see [Install and Uninstall](/operation/pkg-install). +For information about installing TDengine, see [Install and Uninstall](../../operation/pkg-install). For information about TDengine releases, see [All Downloads](https://tdengine.com/all-downloads) diff --git a/docs/en/05-get-started/index.md b/docs/en/05-get-started/index.md index cc3b4826dd..5c4ab59f75 100644 --- a/docs/en/05-get-started/index.md +++ b/docs/en/05-get-started/index.md @@ -12,7 +12,7 @@ import StackOverflowSVG from './stackoverflow.svg' You can install and run TDengine on Linux/Windows/macOS machines as well as Docker containers. You can also deploy TDengine as a managed service with TDengine Cloud. -The full package of TDengine includes the TDengine Server (`taosd`), TDengine Client (`taosc`), taosAdapter for connecting with third-party systems and providing a RESTful interface, a command-line interface, and some tools. In addition to connectors for multiple languages, TDengine also provides a [RESTful interface](/reference/rest-api) through [taosAdapter](/reference/taosadapter). +The full package of TDengine includes the TDengine Server (`taosd`), TDengine Client (`taosc`), taosAdapter for connecting with third-party systems and providing a RESTful interface, a command-line interface, and some tools. In addition to connectors for multiple languages, TDengine also provides a [RESTful interface](../reference/rest-api) through [taosAdapter](../reference/taosadapter). ```mdx-code-block import DocCardList from '@theme/DocCardList'; diff --git a/docs/en/07-develop/01-connect/_connect_java.mdx b/docs/en/07-develop/01-connect/_connect_java.mdx index 538e27fcc3..fda86f2221 100644 --- a/docs/en/07-develop/01-connect/_connect_java.mdx +++ b/docs/en/07-develop/01-connect/_connect_java.mdx @@ -12,4 +12,4 @@ When using REST connection, the feature of bulk pulling can be enabled if the si {{#include docs/examples/java/src/main/java/com/taos/example/WSConnectExample.java:main}} ``` -More configuration about connection, please refer to [Java Connector](/reference/connector/java) +More configuration about connection, please refer to [Java Connector](../../reference/connector/java) diff --git a/docs/en/07-develop/01-connect/index.md b/docs/en/07-develop/01-connect/index.md index 3ca44783c4..3f09f9fb6a 100644 --- a/docs/en/07-develop/01-connect/index.md +++ b/docs/en/07-develop/01-connect/index.md @@ -22,7 +22,7 @@ import VerifyLinux from "../../14-reference/03-connector/_verify_linux.mdx"; import VerifyWindows from "../../14-reference/03-connector/_verify_windows.mdx"; import VerifyMacOS from "../../14-reference/03-connector/_verify_macos.mdx"; -Any application running on any platform can access TDengine through the REST API provided by TDengine. For information, see [REST API](/reference/rest-api/). Applications can also use the connectors for various programming languages, including C/C++, Java, Python, Go, Node.js, C#, and Rust, to access TDengine. These connectors support connecting to TDengine clusters using both native interfaces (taosc). Some connectors also support connecting over a REST interface. Community developers have also contributed several unofficial connectors, such as the ADO.NET connector, the Lua connector, and the PHP connector. +Any application running on any platform can access TDengine through the REST API provided by TDengine. For information, see [REST API](../../reference/rest-api/). Applications can also use the connectors for various programming languages, including C/C++, Java, Python, Go, Node.js, C#, and Rust, to access TDengine. These connectors support connecting to TDengine clusters using both native interfaces (taosc). Some connectors also support connecting over a REST interface. Community developers have also contributed several unofficial connectors, such as the ADO.NET connector, the Lua connector, and the PHP connector. ## Establish Connection @@ -36,7 +36,7 @@ For REST and native connections, connectors provide similar APIs for performing Key differences: 3. The REST connection is more accessible with cross-platform support, however it results in a 30% performance downgrade. -1. The TDengine client driver (taosc) has the highest performance with all the features of TDengine like [Parameter Binding](/reference/connector/cpp#parameter-binding-api), [Subscription](/reference/connector/cpp#subscription-and-consumption-api), etc. +1. The TDengine client driver (taosc) has the highest performance with all the features of TDengine like [Parameter Binding](../../reference/connector/cpp#parameter-binding-api), [Subscription](../../reference/connector/cpp#subscription-and-consumption-api), etc. ## Install Client Driver taosc diff --git a/docs/en/07-develop/02-model/index.mdx b/docs/en/07-develop/02-model/index.mdx index 4524a66a41..a18e1932ab 100644 --- a/docs/en/07-develop/02-model/index.mdx +++ b/docs/en/07-develop/02-model/index.mdx @@ -3,9 +3,9 @@ title: Data Model description: This document describes the data model of TDengine. --- -The data model employed by TDengine is similar to that of a relational database. You have to create databases and tables. You must design the data model based on your own business and application requirements. You should design the [STable](/concept/#super-table-stable) (an abbreviation for super table) schema to fit your data. This chapter will explain the big picture without getting into syntactical details. +The data model employed by TDengine is similar to that of a relational database. You have to create databases and tables. You must design the data model based on your own business and application requirements. You should design the [STable](../../concept/#super-table-stable) (an abbreviation for super table) schema to fit your data. This chapter will explain the big picture without getting into syntactical details. -Note: before you read this chapter, please make sure you have already read through [Key Concepts](/concept/), since TDengine introduces new concepts like "one table for one [data collection point](/concept/#data-collection-point)" and "[super table](/concept/#super-table-stable)". +Note: before you read this chapter, please make sure you have already read through [Key Concepts](../../concept/), since TDengine introduces new concepts like "one table for one [data collection point](../../concept/#data-collection-point)" and "[super table](../../concept/#super-table-stable)". ## Create Database @@ -22,7 +22,7 @@ In the above SQL statement: - a new data file will be created every 10 days - the size of the write cache pool on each VNode is 16 MB - the number of vgroups is 100 -- WAL is enabled but fsync is disabled For more details please refer to [Database](/taos-sql/database). +- WAL is enabled but fsync is disabled For more details please refer to [Database](../../taos-sql/database). After creating a database, the current database in use can be switched using SQL command `USE`. For example the SQL statement below switches the current database to `power`. @@ -41,13 +41,13 @@ Without the current database specified, table name must be preceded with the cor ## Create STable -In a time-series application, there may be multiple kinds of data collection points. For example, in the electrical power system there are meters, transformers, bus bars, switches, etc. For easy and efficient aggregation of multiple tables, one STable needs to be created for each kind of data collection point. For example, for the meters in [table 1](/concept/#model_table1), the SQL statement below can be used to create the super table. +In a time-series application, there may be multiple kinds of data collection points. For example, in the electrical power system there are meters, transformers, bus bars, switches, etc. For easy and efficient aggregation of multiple tables, one STable needs to be created for each kind of data collection point. For example, for the meters in [table 1](../../concept/#model_table1), the SQL statement below can be used to create the super table. ```sql CREATE STABLE meters (ts timestamp, current float, voltage int, phase float) TAGS (location binary(64), groupId int); ``` -Similar to creating a regular table, when creating a STable, the name and schema need to be provided. In the STable schema, the first column must always be a timestamp (like ts in the example), and the other columns (like current, voltage and phase in the example) are the data collected. The remaining columns can [contain data of type](/taos-sql/data-type/) integer, float, double, string etc. In addition, the schema for tags, like location and groupId in the example, must be provided. The tag type can be integer, float, string, etc. Tags are essentially the static properties of a data collection point. For example, properties like the location, device type, device group ID, manager ID are tags. Tags in the schema can be added, removed or updated. Please refer to [STable](/taos-sql/stable) for more details. +Similar to creating a regular table, when creating a STable, the name and schema need to be provided. In the STable schema, the first column must always be a timestamp (like ts in the example), and the other columns (like current, voltage and phase in the example) are the data collected. The remaining columns can [contain data of type](../../taos-sql/data-type/) integer, float, double, string etc. In addition, the schema for tags, like location and groupId in the example, must be provided. The tag type can be integer, float, string, etc. Tags are essentially the static properties of a data collection point. For example, properties like the location, device type, device group ID, manager ID are tags. Tags in the schema can be added, removed or updated. Please refer to [STable](../../taos-sql/stable) for more details. For each kind of data collection point, a corresponding STable must be created. There may be many STables in an application. For electrical power system, we need to create a STable respectively for meters, transformers, busbars, switches. There may be multiple kinds of data collection points on a single device, for example there may be one data collection point for electrical data like current and voltage and another data collection point for environmental data like temperature, humidity and wind direction. Multiple STables are required for these kinds of devices. @@ -61,7 +61,7 @@ A specific table needs to be created for each data collection point. Similar to CREATE TABLE d1001 USING meters TAGS ("California.SanFrancisco", 2); ``` -In the above SQL statement, "d1001" is the table name, "meters" is the STable name, followed by the value of tag "Location" and the value of tag "groupId", which are "California.SanFrancisco" and "2" respectively in the example. The tag values can be updated after the table is created. Please refer to [Tables](/taos-sql/table) for details. +In the above SQL statement, "d1001" is the table name, "meters" is the STable name, followed by the value of tag "Location" and the value of tag "groupId", which are "California.SanFrancisco" and "2" respectively in the example. The tag values can be updated after the table is created. Please refer to [Tables](../../taos-sql/table) for details. It's suggested to use the globally unique ID of a data collection point as the table name. For example the device serial number could be used as a unique ID. If a unique ID doesn't exist, multiple IDs that are not globally unique can be combined to form a globally unique ID. It's not recommended to use a globally unique ID as tag value. @@ -75,7 +75,7 @@ INSERT INTO d1001 USING meters TAGS ("California.SanFrancisco", 2) VALUES (now, In the above SQL statement, a row with value `(now, 10.2, 219, 0.32)` will be inserted into table "d1001". If table "d1001" doesn't exist, it will be created automatically using STable "meters" as template with tag value `"California.SanFrancisco", 2`. -For more details please refer to [Create Table Automatically](/taos-sql/insert#automatically-create-table-when-inserting). +For more details please refer to [Create Table Automatically](../../taos-sql/insert#automatically-create-table-when-inserting). ## Single Column vs Multiple Column diff --git a/docs/en/07-develop/03-insert-data/01-sql-writing.mdx b/docs/en/07-develop/03-insert-data/01-sql-writing.mdx index 4d1b67e451..1e719283cc 100644 --- a/docs/en/07-develop/03-insert-data/01-sql-writing.mdx +++ b/docs/en/07-develop/03-insert-data/01-sql-writing.mdx @@ -33,7 +33,7 @@ The below SQL statement is used to insert one row into table "d1001". INSERT INTO d1001 VALUES (ts1, 10.3, 219, 0.31); ``` -`ts1` is Unix timestamp, the timestamps which is larger than the difference between current time and KEEP in config is only allowed. For further detail, refer to [TDengine SQL insert timestamp section](/taos-sql/insert). +`ts1` is Unix timestamp, the timestamps which is larger than the difference between current time and KEEP in config is only allowed. For further detail, refer to [TDengine SQL insert timestamp section](../../../taos-sql/insert). ### Insert Multiple Rows @@ -43,7 +43,7 @@ Multiple rows can be inserted in a single SQL statement. The example below inser INSERT INTO d1001 VALUES (ts2, 10.2, 220, 0.23) (ts2, 10.3, 218, 0.25); ``` -`ts1` and `ts2` is Unix timestamp, the timestamps which is larger than the difference between current time and KEEP in config is only allowed. For further detail, refer to [TDengine SQL insert timestamp section](/taos-sql/insert). +`ts1` and `ts2` is Unix timestamp, the timestamps which is larger than the difference between current time and KEEP in config is only allowed. For further detail, refer to [TDengine SQL insert timestamp section](../../../taos-sql/insert). ### Insert into Multiple Tables @@ -53,9 +53,9 @@ Data can be inserted into multiple tables in the same SQL statement. The example INSERT INTO d1001 VALUES (ts1, 10.3, 219, 0.31) (ts2, 12.6, 218, 0.33) d1002 VALUES (ts3, 12.3, 221, 0.31); ``` -`ts1`, `ts2` and `ts3` is Unix timestamp, the timestamps which is larger than the difference between current time and KEEP in config is only allowed. For further detail, refer to [TDengine SQL insert timestamp section](/taos-sql/insert). +`ts1`, `ts2` and `ts3` is Unix timestamp, the timestamps which is larger than the difference between current time and KEEP in config is only allowed. For further detail, refer to [TDengine SQL insert timestamp section](../../../taos-sql/insert). -For more details about `INSERT` please refer to [INSERT](/taos-sql/insert). +For more details about `INSERT` please refer to [INSERT](../../../taos-sql/insert). :::info diff --git a/docs/en/07-develop/03-insert-data/20-kafka-writting.mdx b/docs/en/07-develop/03-insert-data/20-kafka-writting.mdx index 89ca10b669..e8de010d5b 100644 --- a/docs/en/07-develop/03-insert-data/20-kafka-writting.mdx +++ b/docs/en/07-develop/03-insert-data/20-kafka-writting.mdx @@ -35,7 +35,7 @@ bin/kafka-topics.sh --bootstrap-server=localhost:9092 --describe ## Insert into TDengine -We can write data into TDengine via SQL or Schemaless. For more information, please refer to [Insert Using SQL](/develop/insert-data/sql-writing/) or [High Performance Writing](/develop/insert-data/high-volume/) or [Schemaless Writing](/reference/schemaless/). +We can write data into TDengine via SQL or Schemaless. For more information, please refer to [Insert Using SQL](../sql-writing/) or [High Performance Writing](../high-volume/) or [Schemaless Writing](../../../reference/schemaless/). ## Examples diff --git a/docs/en/07-develop/03-insert-data/30-influxdb-line.mdx b/docs/en/07-develop/03-insert-data/30-influxdb-line.mdx index bd430d5973..b90b92510a 100644 --- a/docs/en/07-develop/03-insert-data/30-influxdb-line.mdx +++ b/docs/en/07-develop/03-insert-data/30-influxdb-line.mdx @@ -46,7 +46,7 @@ meters,location=California.LosAngeles,groupid=2 current=13.4,voltage=223,phase=0 ::: -For more details please refer to [InfluxDB Line Protocol](https://docs.influxdata.com/influxdb/v2.0/reference/syntax/line-protocol/) and [TDengine Schemaless](/reference/schemaless/#Schemaless-Line-Protocol) +For more details please refer to [InfluxDB Line Protocol](https://docs.influxdata.com/influxdb/v2.0/reference/syntax/line-protocol/) and [TDengine Schemaless](../../../reference/schemaless/#Schemaless-Line-Protocol) ## Examples diff --git a/docs/en/07-develop/04-query-data/index.mdx b/docs/en/07-develop/04-query-data/index.mdx index 5a05d22d85..413baf6b2c 100644 --- a/docs/en/07-develop/04-query-data/index.mdx +++ b/docs/en/07-develop/04-query-data/index.mdx @@ -128,7 +128,7 @@ For more information, see [Aggregate by Window](../../taos-sql/distinguished). ### Query -In the section describing [Insert](/develop/insert-data/sql-writing), a database named `power` is created and some data are inserted into STable `meters`. Below sample code demonstrates how to query the data in this STable. +In the section describing [Insert](../insert-data/sql-writing), a database named `power` is created and some data are inserted into STable `meters`. Below sample code demonstrates how to query the data in this STable. diff --git a/docs/en/07-develop/index.md b/docs/en/07-develop/index.md index 8f80b82b97..4ed5e8c19f 100644 --- a/docs/en/07-develop/index.md +++ b/docs/en/07-develop/index.md @@ -6,7 +6,7 @@ description: This document describes how to use the various components of TDengi Before creating an application to process time-series data with TDengine, consider the following: 1. Choose the method to connect to TDengine. TDengine offers a REST API that can be used with any programming language. It also has connectors for a variety of languages. -2. Design the data model based on your own use cases. Consider the main [concepts](/concept/) of TDengine, including "one table per data collection point" and the supertable. Learn about static labels, collected metrics, and subtables. Depending on the characteristics of your data and your requirements, you decide to create one or more databases and design a supertable schema that fit your data. +2. Design the data model based on your own use cases. Consider the main [concepts](../concept/) of TDengine, including "one table per data collection point" and the supertable. Learn about static labels, collected metrics, and subtables. Depending on the characteristics of your data and your requirements, you decide to create one or more databases and design a supertable schema that fit your data. 3. Decide how you will insert data. TDengine supports writing using standard SQL, but also supports schemaless writing, so that data can be written directly without creating tables manually. 4. Based on business requirements, find out what SQL query statements need to be written. You may be able to repurpose any existing SQL. 5. If you want to run real-time analysis based on time series data, including various dashboards, use the TDengine stream processing component instead of deploying complex systems such as Spark or Flink. @@ -14,7 +14,7 @@ Before creating an application to process time-series data with TDengine, consid 7. In many use cases (such as fleet management), the application needs to obtain the latest status of each data collection point. It is recommended that you use the cache function of TDengine instead of deploying Redis separately. 8. If you find that the SQL functions of TDengine cannot meet your requirements, then you can use user-defined functions to solve the problem. -This section is organized in the order described above. For ease of understanding, TDengine provides sample code for each supported programming language for each function. If you want to learn more about the use of SQL, please read the [SQL manual](/taos-sql/). For a more in-depth understanding of the use of each connector, please read the [Connector Reference Guide](/reference/connector/). If you also want to integrate TDengine with third-party systems, such as Grafana, please refer to the [third-party tools](/third-party/). +This section is organized in the order described above. For ease of understanding, TDengine provides sample code for each supported programming language for each function. If you want to learn more about the use of SQL, please read the [SQL manual](../taos-sql/). For a more in-depth understanding of the use of each connector, please read the [Connector Reference Guide](../reference/connector/). If you also want to integrate TDengine with third-party systems, such as Grafana, please refer to the [third-party tools](../third-party/). If you encounter any problems during the development process, please click ["Submit an issue"](https://github.com/taosdata/TDengine/issues/new/choose) at the bottom of each page and submit it on GitHub right away. diff --git a/docs/en/10-deployment/01-deploy.md b/docs/en/10-deployment/01-deploy.md index 4a6a3aae41..6e7e0d1b7c 100644 --- a/docs/en/10-deployment/01-deploy.md +++ b/docs/en/10-deployment/01-deploy.md @@ -72,7 +72,7 @@ For all the dnodes in a TDengine cluster, the below parameters must be configure ## Start Cluster -The first dnode can be started following the instructions in [Get Started](/get-started/). Then TDengine CLI `taos` can be launched to execute command `show dnodes`, the output is as following for example: +The first dnode can be started following the instructions in [Get Started](../../get-started/). Then TDengine CLI `taos` can be launched to execute command `show dnodes`, the output is as following for example: ``` taos> show dnodes; @@ -90,7 +90,7 @@ From the above output, it is shown that the end point of the started dnode is "h There are a few steps necessary to add other dnodes in the cluster. -Second, we can start `taosd` as instructed in [Get Started](/get-started/). +Second, we can start `taosd` as instructed in [Get Started](../../get-started/). Then, on the first dnode i.e. h1.tdengine.com in our example, use TDengine CLI `taos` to execute the following command: diff --git a/docs/en/12-taos-sql/26-udf.md b/docs/en/12-taos-sql/26-udf.md index b533b98b3d..f86b535927 100644 --- a/docs/en/12-taos-sql/26-udf.md +++ b/docs/en/12-taos-sql/26-udf.md @@ -53,7 +53,7 @@ CREATE AGGREGATE FUNCTION function_name AS library_path OUTPUTTYPE output_type [ CREATE AGGREGATE FUNCTION l2norm AS "/home/taos/udf_example/libl2norm.so" OUTPUTTYPE DOUBLE bufsize 64; ``` -For more information about user-defined functions, see [User-Defined Functions](/develop/udf). +For more information about user-defined functions, see [User-Defined Functions](../../develop/udf). ## Manage UDF diff --git a/docs/en/13-operation/10-monitor.md b/docs/en/13-operation/10-monitor.md index f1be4c5fd3..009db425a4 100644 --- a/docs/en/13-operation/10-monitor.md +++ b/docs/en/13-operation/10-monitor.md @@ -41,7 +41,7 @@ Launch `TDinsight.sh` with the command above and restart Grafana, then open Dash ## log database -The data of tdinsight dashboard is stored in `log` database (default. You can change it in taoskeeper's config file. For more infrmation, please reference to [taoskeeper document](/reference/taosKeeper)). The taoskeeper will create log database on taoskeeper startup. +The data of tdinsight dashboard is stored in `log` database (default. You can change it in taoskeeper's config file. For more infrmation, please reference to [taoskeeper document](../../reference/taosKeeper)). The taoskeeper will create log database on taoskeeper startup. ### cluster\_info table diff --git a/docs/en/14-reference/02-rest-api/02-rest-api.mdx b/docs/en/14-reference/02-rest-api/02-rest-api.mdx index 4da987213c..8f8e966f8d 100644 --- a/docs/en/14-reference/02-rest-api/02-rest-api.mdx +++ b/docs/en/14-reference/02-rest-api/02-rest-api.mdx @@ -514,4 +514,4 @@ Response body: ## Reference -[taosAdapter](/reference/taosadapter/) +[taosAdapter](../taosadapter/) diff --git a/docs/en/14-reference/03-connector/03-cpp.mdx b/docs/en/14-reference/03-connector/03-cpp.mdx index f6ebf0fe47..27adb58c12 100644 --- a/docs/en/14-reference/03-connector/03-cpp.mdx +++ b/docs/en/14-reference/03-connector/03-cpp.mdx @@ -24,7 +24,7 @@ The dynamic libraries for the TDengine client driver are located in. ## Supported platforms -Please refer to [list of supported platforms](/reference/connector#supported-platforms) +Please refer to [list of supported platforms](../#supported-platforms) ## Supported versions @@ -32,7 +32,7 @@ The version number of the TDengine client driver and the version number of the T ## Installation Steps -Please refer to the [Installation Steps](/reference/connector#installation-steps) for TDengine client driver installation +Please refer to the [Installation Steps](../#installation-steps) for TDengine client driver installation ## Establishing a connection @@ -394,7 +394,7 @@ The specific functions related to the interface are as follows (see also the [pr ### Schemaless Writing API -In addition to writing data using the SQL method or the parameter binding API, writing can also be done using schemaless writing, which eliminates the need to create a super table/data sub-table structure in advance and writes the data directly. The TDengine system automatically creates and maintains the required table structure based on the written data content. The use of schemaless writing is described in the chapter [Schemaless Writing](/reference/schemaless/), and the C/C++ API used with it is described here. +In addition to writing data using the SQL method or the parameter binding API, writing can also be done using schemaless writing, which eliminates the need to create a super table/data sub-table structure in advance and writes the data directly. The TDengine system automatically creates and maintains the required table structure based on the written data content. The use of schemaless writing is described in the chapter [Schemaless Writing](../../schemaless/), and the C/C++ API used with it is described here. - `TAOS_RES* taos_schemaless_insert(TAOS* taos, const char* lines[], int numLines, int protocol, int precision)` diff --git a/docs/en/14-reference/03-connector/04-java.mdx b/docs/en/14-reference/03-connector/04-java.mdx index f770ce0d5d..039395cc30 100644 --- a/docs/en/14-reference/03-connector/04-java.mdx +++ b/docs/en/14-reference/03-connector/04-java.mdx @@ -148,7 +148,7 @@ TDengine currently supports timestamp, number, character, Boolean type, and the **Note**: Only TAG supports JSON types Due to historical reasons, the BINARY type data in TDengine is not truly binary data and is no longer recommended for use. Please use VARBINARY type instead. -GEOMETRY type is binary data in little endian byte order, which complies with the WKB specification. For detailed information, please refer to [Data Type] (/tao-sql/data-type/#Data Types) +GEOMETRY type is binary data in little endian byte order, which complies with the WKB specification. For detailed information, please refer to [Data Type](../../../taos-sql/data-type/) For WKB specifications, please refer to [Well Known Binary (WKB)]( https://libgeos.org/specifications/wkb/ ) For Java connector, the jts library can be used to easily create GEOMETRY type objects, serialize them, and write them to TDengine. Here is an example [Geometry example](https://github.com/taosdata/TDengine/blob/3.0/examples/JDBC/JDBCDemo/src/main/java/com/taosdata/example/GeometryDemo.java) @@ -160,7 +160,7 @@ For Java connector, the jts library can be used to easily create GEOMETRY type o Before using Java Connector to connect to the database, the following conditions are required. - Java 1.8 or above runtime environment and Maven 3.6 or above installed -- TDengine client driver installed (required for native connections, not required for REST connections), please refer to [Installing Client Driver](/reference/connector#Install-Client-Driver) +- TDengine client driver installed (required for native connections, not required for REST connections), please refer to [Installing Client Driver](../#Install-Client-Driver) ### Install the connectors @@ -368,7 +368,7 @@ The configuration parameters in properties are as follows. - TSDBDriver.PROPERTY_KEY_MESSAGE_WAIT_TIMEOUT: message transmission timeout in milliseconds, the default value is 60000 ms. It only takes effect when using JDBC REST connection and batchfetch is true. - TSDBDriver.PROPERTY_KEY_USE_SSL: connecting Securely Using SSL. true: using SSL connection, false: not using SSL connection. It only takes effect when using JDBC REST connection. - TSDBDriver.HTTP_POOL_SIZE: size of REST concurrent requests. The default value is 20. - For JDBC native connections, you can specify other parameters, such as log level, SQL length, etc., by specifying URL and Properties. For more detailed configuration, please refer to [Client Configuration](/reference/config/#Client-Only). + For JDBC native connections, you can specify other parameters, such as log level, SQL length, etc., by specifying URL and Properties. For more detailed configuration, please refer to [Client Configuration](../../config/#Client-Only). ### Priority of configuration parameters diff --git a/docs/en/14-reference/03-connector/05-go.mdx b/docs/en/14-reference/03-connector/05-go.mdx index a0be7a4a02..33f7a93439 100644 --- a/docs/en/14-reference/03-connector/05-go.mdx +++ b/docs/en/14-reference/03-connector/05-go.mdx @@ -74,7 +74,7 @@ If it is a TDengine error, you can get the error code and error information in t ### Pre-installation preparation * Install Go development environment (Go 1.14 and above, GCC 4.8.5 and above) -* If you use the native connector, please install the TDengine client driver. Please refer to [Install Client Driver](/reference/connector/#install-client-driver) for specific steps +* If you use the native connector, please install the TDengine client driver. Please refer to [Install Client Driver](../#install-client-driver) for specific steps Configure the environment variables and check the command. diff --git a/docs/en/14-reference/03-connector/06-rust.mdx b/docs/en/14-reference/03-connector/06-rust.mdx index 5a44b161cb..0981df6724 100644 --- a/docs/en/14-reference/03-connector/06-rust.mdx +++ b/docs/en/14-reference/03-connector/06-rust.mdx @@ -80,7 +80,7 @@ Note: Only TAG supports JSON types ### Pre-installation preparation * Install the Rust development toolchain -* If using the native connection, please install the TDengine client driver. Please refer to [install client driver](/reference/connector#install-client-driver) +* If using the native connection, please install the TDengine client driver. Please refer to [install client driver](../#install-client-driver) ### Install the connectors diff --git a/docs/en/14-reference/03-connector/07-python.mdx b/docs/en/14-reference/03-connector/07-python.mdx index b699019b44..ccc270d3be 100644 --- a/docs/en/14-reference/03-connector/07-python.mdx +++ b/docs/en/14-reference/03-connector/07-python.mdx @@ -7,7 +7,7 @@ description: This document describes taospy, the TDengine Python connector. import Tabs from "@theme/Tabs"; import TabItem from "@theme/TabItem"; -`taospy` is the official Python connector for TDengine. taospy provides a rich API that makes it easy for Python applications to use TDengine. `taospy` wraps both the [native interface](/reference/connector/cpp) and [REST interface](/reference/rest-api) of TDengine, which correspond to the `taos` and `taosrest` modules of the `taospy` package, respectively. +`taospy` is the official Python connector for TDengine. taospy provides a rich API that makes it easy for Python applications to use TDengine. `taospy` wraps both the [native interface](../cpp) and [REST interface](../../rest-api) of TDengine, which correspond to the `taos` and `taosrest` modules of the `taospy` package, respectively. In addition to wrapping the native and REST interfaces, `taospy` also provides a set of programming interfaces that conforms to the [Python Data Access Specification (PEP 249)](https://peps.python.org/pep-0249/). It is easy to integrate `taospy` with many third-party tools, such as [SQLAlchemy](https://www.sqlalchemy.org/) and [pandas](https://pandas.pydata.org/). `taos-ws-py` is an optional package to enable using WebSocket to connect TDengine. @@ -17,7 +17,7 @@ The direct connection to the server using the native interface provided by the c The source code for the Python connector is hosted on [GitHub](https://github.com/taosdata/taos-connector-python). ## Supported platforms -- The [supported platforms](/reference/connector/#supported-platforms) for the native connection are the same as the ones supported by the TDengine client. +- The [supported platforms](../#supported-platforms) for the native connection are the same as the ones supported by the TDengine client. - REST connections are supported on all platforms that can run Python. ### Supported features @@ -95,7 +95,7 @@ TDengine currently supports timestamp, number, character, Boolean type, and the 1. Install Python. The recent taospy package requires Python 3.6.2+. The earlier versions of taospy require Python 3.7+. The taos-ws-py package requires Python 3.7+. If Python is not available on your system, refer to the [Python BeginnersGuide](https://wiki.python.org/moin/BeginnersGuide/Download) to install it. 2. Install [pip](https://pypi.org/project/pip/). In most cases, the Python installer comes with the pip utility. If not, please refer to [pip documentation](https://pip.pypa.io/en/stable/installation/) to install it. -If you use a native connection, you will also need to [Install Client Driver](/reference/connector#Install-Client-Driver). The client install package includes the TDengine client dynamic link library (`libtaos.so` or `taos.dll`) and the TDengine CLI. +If you use a native connection, you will also need to [Install Client Driver](../#Install-Client-Driver). The client install package includes the TDengine client dynamic link library (`libtaos.so` or `taos.dll`) and the TDengine CLI. ### Install via pip @@ -444,7 +444,7 @@ The best practice for TaosCursor is to create a cursor at the beginning of a que ##### Use of the RestClient class -The `RestClient` class is a direct wrapper for the [REST API](/reference/rest-api). It contains only a `sql()` method for executing arbitrary SQL statements and returning the result. +The `RestClient` class is a direct wrapper for the [REST API](../../rest-api). It contains only a `sql()` method for executing arbitrary SQL statements and returning the result. ```python title="Use of RestClient" {{#include docs/examples/python/rest_client_example.py}} @@ -501,7 +501,7 @@ The queried results can only be fetched once. For example, only one of `fetch_al -The `RestClient` class is a direct wrapper for the [REST API](/reference/rest-api). It contains only a `sql()` method for executing arbitrary SQL statements and returning the result. +The `RestClient` class is a direct wrapper for the [REST API](../../rest-api). It contains only a `sql()` method for executing arbitrary SQL statements and returning the result. ```python {{#include docs/examples/python/rest_client_example.py}} @@ -561,7 +561,7 @@ The `TaosConnection` class and the `TaosResult` class already implement all the ##### Use of the RestClient class -The `RestClient` class is a direct wrapper for the [REST API](/reference/rest-api). It contains only a `sql()` method for executing arbitrary SQL statements and returning the result. +The `RestClient` class is a direct wrapper for the [REST API](../../rest-api). It contains only a `sql()` method for executing arbitrary SQL statements and returning the result. ```python title="Use of RestClient" {{#include docs/examples/python/rest_client_with_req_id_example.py}} diff --git a/docs/en/14-reference/03-connector/08-node.mdx b/docs/en/14-reference/03-connector/08-node.mdx index a02bcddfd6..bed06477f1 100644 --- a/docs/en/14-reference/03-connector/08-node.mdx +++ b/docs/en/14-reference/03-connector/08-node.mdx @@ -28,7 +28,7 @@ The REST connector supports all platforms that can run Node.js. ## Version support -Please refer to [version support list](/reference/connector#version-support) +Please refer to [version support list](../#version-support) ## Supported features @@ -58,7 +58,7 @@ Please refer to [version support list](/reference/connector#version-support) ### Pre-installation preparation - Install the Node.js development environment -- If you are using the REST connector, skip this step. However, if you use the native connector, please install the TDengine client driver. Please refer to [Install Client Driver](/reference/connector#Install-Client-Driver) for more details. We use [node-gyp](https://github.com/nodejs/node-gyp) to interact with TDengine instances and also need to install some dependencies mentioned below depending on the specific OS. +- If you are using the REST connector, skip this step. However, if you use the native connector, please install the TDengine client driver. Please refer to [Install Client Driver](../#Install-Client-Driver) for more details. We use [node-gyp](https://github.com/nodejs/node-gyp) to interact with TDengine instances and also need to install some dependencies mentioned below depending on the specific OS. diff --git a/docs/en/14-reference/03-connector/09-csharp.mdx b/docs/en/14-reference/03-connector/09-csharp.mdx index 203d44fe02..282be3af6b 100644 --- a/docs/en/14-reference/03-connector/09-csharp.mdx +++ b/docs/en/14-reference/03-connector/09-csharp.mdx @@ -36,7 +36,7 @@ Please note TDengine does not support 32bit Windows any more. ## Version support -Please refer to [version support list](/reference/connector#version-support) +Please refer to [version support list](../#version-support) ## Supported features @@ -69,7 +69,7 @@ Please refer to [version support list](/reference/connector#version-support) * Install the [.NET SDK](https://dotnet.microsoft.com/download) * [Nuget Client](https://docs.microsoft.com/en-us/nuget/install-nuget-client-tools) (optional installation) -* Install TDengine client driver, please refer to [Install client driver](/reference/connector/#install-client-driver) for details +* Install TDengine client driver, please refer to [Install client driver](../#install-client-driver) for details ### Install `TDengine.Connector` diff --git a/docs/en/14-reference/03-connector/80-php.mdx b/docs/en/14-reference/03-connector/80-php.mdx index b3c2065b6e..bff9e8e5d5 100644 --- a/docs/en/14-reference/03-connector/80-php.mdx +++ b/docs/en/14-reference/03-connector/80-php.mdx @@ -40,7 +40,7 @@ Because the version of TDengine client driver is tightly associated with that of ### Install TDengine Client Driver -Regarding how to install TDengine client driver please refer to [Install Client Driver](/reference/connector#installation-steps) +Regarding how to install TDengine client driver please refer to [Install Client Driver](../#installation-steps) ### Install php-tdengine diff --git a/docs/en/14-reference/03-connector/_preparation.mdx b/docs/en/14-reference/03-connector/_preparation.mdx index 25b78ec134..99887ac36b 100644 --- a/docs/en/14-reference/03-connector/_preparation.mdx +++ b/docs/en/14-reference/03-connector/_preparation.mdx @@ -2,7 +2,7 @@ :::info -Since the TDengine client driver is written in C, using the native connection requires loading the client driver shared library file, which is usually included in the TDengine installer. You can install either standard TDengine server installation package or [TDengine client installation package](/get-started/). For Windows development, you need to install the corresponding Windows client, please refer to [Install TDengine](../../get-started/package). +Since the TDengine client driver is written in C, using the native connection requires loading the client driver shared library file, which is usually included in the TDengine installer. You can install either standard TDengine server installation package or [TDengine client installation package](../../get-started/). For Windows development, you need to install the corresponding Windows client, please refer to [Install TDengine](../../get-started/package). - libtaos.so: After successful installation of TDengine on a Linux system, the dependent Linux version of the client driver `libtaos.so` file will be automatically linked to `/usr/lib/libtaos.so`, which is included in the Linux scannable path and does not need to be specified separately. - taos.dll: After installing the client on Windows, the dependent Windows version of the client driver taos.dll file will be automatically copied to the system default search path C:/Windows/System32, again without the need to specify it separately. diff --git a/docs/en/14-reference/04-taosadapter.md b/docs/en/14-reference/04-taosadapter.md index c75598b0df..a9330d21c7 100644 --- a/docs/en/14-reference/04-taosadapter.md +++ b/docs/en/14-reference/04-taosadapter.md @@ -186,7 +186,7 @@ See [example/config/taosadapter.toml](https://github.com/taosdata/taosadapter/bl ### TDengine RESTful interface -You can use any client that supports the http protocol to write data to or query data from TDengine by accessing the REST interface address `http://:6041/rest/sql`. See the [official documentation](/reference/rest-api/) for details. +You can use any client that supports the http protocol to write data to or query data from TDengine by accessing the REST interface address `http://:6041/rest/sql`. See the [official documentation](../rest-api/) for details. ### InfluxDB @@ -202,7 +202,7 @@ Support InfluxDB query parameters as follows. - `precision` The time precision used by TDengine - `u` TDengine user name - `p` TDengine password -- `ttl` The time to live of automatically created sub-table. This value cannot be updated. TDengine will use the ttl value of the first data of sub-table to create sub-table. For more information, please refer [Create Table](/taos-sql/table/#create-table) +- `ttl` The time to live of automatically created sub-table. This value cannot be updated. TDengine will use the ttl value of the first data of sub-table to create sub-table. For more information, please refer [Create Table](../../taos-sql/table/#create-table) Note: InfluxDB token authorization is not supported at present. Only Basic authorization and query parameter validation are supported. Example: curl --request POST http://127.0.0.1:6041/influxdb/v1/write?db=test --user "root:taosdata" --data-binary "measurement,host=host1 field1=2i,field2=2.0 1577836800000000000" diff --git a/docs/en/14-reference/06-taosdump.md b/docs/en/14-reference/06-taosdump.md index c07465a97c..5ec26df1fa 100644 --- a/docs/en/14-reference/06-taosdump.md +++ b/docs/en/14-reference/06-taosdump.md @@ -31,7 +31,7 @@ There are two ways to install taosdump: 2. backup multiple specified databases: use `-D db1,db2,... ` parameters; 3. back up some super or normal tables in the specified database: use `dbname stbname1 stbname2 tbname1 tbname2 ... ` parameters. Note that the first parameter of this input sequence is the database name, and only one database is supported. The second and subsequent parameters are the names of super or normal tables in that database, separated by spaces. 4. back up the system log database: TDengine clusters usually contain a system database named `log`. The data in this database is the data that TDengine runs itself, and the taosdump will not back up the log database by default. If users need to back up the log database, users can use the `-a` or `-allow-sys` command-line parameter. -5. Loose mode backup: taosdump version 1.4.1 onwards provides `-n` and `-L` parameters for backing up data without using escape characters and "loose" mode, which can reduce the number of backups if table names, column names, tag names do not use escape characters. This can also reduce the backup data time and backup data footprint. If you are unsure about using `-n` and `-L` conditions, please use the default parameters for "strict" mode backup. See the [official documentation](/taos-sql/escape) for a description of escaped characters. +5. Loose mode backup: taosdump version 1.4.1 onwards provides `-n` and `-L` parameters for backing up data without using escape characters and "loose" mode, which can reduce the number of backups if table names, column names, tag names do not use escape characters. This can also reduce the backup data time and backup data footprint. If you are unsure about using `-n` and `-L` conditions, please use the default parameters for "strict" mode backup. See the [official documentation](../../taos-sql/escape) for a description of escaped characters. :::tip - taosdump versions after 1.4.1 provide the `-I` argument for parsing Avro file schema and data. If users specify `-s` then only taosdump will parse schema. diff --git a/docs/en/14-reference/08-taos-shell.md b/docs/en/14-reference/08-taos-shell.md index 7e0433a8b2..8512f5b59d 100644 --- a/docs/en/14-reference/08-taos-shell.md +++ b/docs/en/14-reference/08-taos-shell.md @@ -8,7 +8,7 @@ The TDengine command-line interface (hereafter referred to as `TDengine CLI`) is ## Installation -If executed on the TDengine server-side, there is no need for additional installation steps to install TDengine CLI as it is already included and installed automatically. To run TDengine CLI in an environment where no TDengine server is running, the TDengine client installation package needs to be installed first. For details, please refer to [Connector](/reference/connector/). +If executed on the TDengine server-side, there is no need for additional installation steps to install TDengine CLI as it is already included and installed automatically. To run TDengine CLI in an environment where no TDengine server is running, the TDengine client installation package needs to be installed first. For details, please refer to [Connector](../connector/). ## Execution @@ -18,7 +18,7 @@ To access the TDengine CLI, you can execute `taos` command-line utility from a t taos ``` -TDengine CLI will display a welcome message and version information if it successfully connected to the TDengine service. If it fails, TDengine CLI will print an error message. See [FAQ](/train-faq/faq) to solve the problem of terminal connection failure to the server. The TDengine CLI prompts as follows: +TDengine CLI will display a welcome message and version information if it successfully connected to the TDengine service. If it fails, TDengine CLI will print an error message. See [FAQ](../../train-faq/faq) to solve the problem of terminal connection failure to the server. The TDengine CLI prompts as follows: ```cmd taos> diff --git a/docs/en/14-reference/12-config/index.md b/docs/en/14-reference/12-config/index.md index ac2364a65d..65c48f9190 100755 --- a/docs/en/14-reference/12-config/index.md +++ b/docs/en/14-reference/12-config/index.md @@ -87,7 +87,7 @@ Ensure that your firewall rules do not block TCP port 6042 on any host in the c | Protocol | Default Port | Description | How to configure | | :------- | :----------- | :-------------------------------------------------------------------------------------------------------- | :--------------------------------------------------------------------------------------------- | | TCP | 6030 | Communication between client and server. In a multi-node cluster, communication between nodes. serverPort | -| TCP | 6041 | REST connection between client and server | Prior to 2.4.0.0: serverPort+11; After 2.4.0.0 refer to [taosAdapter](/reference/taosadapter/) | +| TCP | 6041 | REST connection between client and server | Prior to 2.4.0.0: serverPort+11; After 2.4.0.0 refer to [taosAdapter](../taosadapter/) | | TCP | 6043 | Service Port of taosKeeper | The parameter of taosKeeper | | TCP | 6044 | Data access port for StatsD | Configurable through taosAdapter parameters. | | UDP | 6045 | Data access for statsd | Configurable through taosAdapter parameters. | diff --git a/docs/en/14-reference/13-schemaless/13-schemaless.md b/docs/en/14-reference/13-schemaless/13-schemaless.md index eb336f4633..9b001ee79c 100644 --- a/docs/en/14-reference/13-schemaless/13-schemaless.md +++ b/docs/en/14-reference/13-schemaless/13-schemaless.md @@ -116,7 +116,7 @@ You can configure smlChildTableName in taos.cfg to specify table names, for exam 10. Taos.cfg adds the configuration of smlTsDefaultName (with a string value), which only works on the client side. After configuration, the time column name of the schemaless automatic table creation can be set through this configuration. If not configured, defaults to _ts. 11. Super table name or child table name are case sensitive. :::tip -All processing logic of schemaless will still follow TDengine's underlying restrictions on data structures, such as the total length of each row of data cannot exceed 48 KB(64 KB since version 3.0.5.0) and the total length of a tag value cannot exceed 16 KB. See [TDengine SQL Boundary Limits](/taos-sql/limit) for specific constraints in this area. +All processing logic of schemaless will still follow TDengine's underlying restrictions on data structures, such as the total length of each row of data cannot exceed 48 KB(64 KB since version 3.0.5.0) and the total length of a tag value cannot exceed 16 KB. See [TDengine SQL Boundary Limits](../../taos-sql/limit) for specific constraints in this area. ::: ## Time resolution recognition diff --git a/docs/en/20-third-party/02-prometheus.md b/docs/en/20-third-party/02-prometheus.md index bfdd3d015e..e6bd855939 100644 --- a/docs/en/20-third-party/02-prometheus.md +++ b/docs/en/20-third-party/02-prometheus.md @@ -16,7 +16,7 @@ Prometheus data can be stored in TDengine via the `remote_write` interface with To write Prometheus data to TDengine requires the following preparations. - The TDengine cluster is deployed and functioning properly -- taosAdapter is installed and running properly. Please refer to the [taosAdapter manual](/reference/taosadapter) for details. +- taosAdapter is installed and running properly. Please refer to the [taosAdapter manual](../../reference/taosadapter) for details. - Prometheus has been installed. Please refer to the [official documentation](https://prometheus.io/docs/prometheus/latest/installation/) for installing Prometheus ## Configuration steps diff --git a/docs/en/20-third-party/03-telegraf.md b/docs/en/20-third-party/03-telegraf.md index 7e99b84eab..900262f51f 100644 --- a/docs/en/20-third-party/03-telegraf.md +++ b/docs/en/20-third-party/03-telegraf.md @@ -14,7 +14,7 @@ Telegraf's data can be written to TDengine by simply adding the output configura To write Telegraf data to TDengine requires the following preparations. - The TDengine cluster is deployed and functioning properly -- taosAdapter is installed and running properly. Please refer to the [taosAdapter manual](/reference/taosadapter) for details. +- taosAdapter is installed and running properly. Please refer to the [taosAdapter manual](../../reference/taosadapter) for details. - Telegraf has been installed. Please refer to the [official documentation](https://docs.influxdata.com/telegraf/v1.22/install/) for Telegraf installation. - Telegraf collects the running status measurements of current system. You can enable [input plugins](https://docs.influxdata.com/telegraf/v1.22/plugins/) to insert [other formats](https://docs.influxdata.com/telegraf/v1.24/data_formats/input/) data to Telegraf then forward to TDengine. @@ -73,6 +73,6 @@ Query OK, 3 row(s) in set (0.013269s) - TDengine take influxdb format data and create unique ID for table names by the rule. The user can configure `smlChildTableName` parameter to generate specified table names if he/she needs. And he/she also need to insert data with specified data format. -For example, Add `smlChildTableName=tname` in the taos.cfg file. Insert data `st,tname=cpu1,t1=4 c1=3 1626006833639000000` then the table name will be cpu1. If there are multiple lines has same tname but different tag_set, the first line's tag_set will be used to automatically creating table and ignore other lines. Please refer to [TDengine Schemaless](/reference/schemaless/#Schemaless-Line-Protocol) +For example, Add `smlChildTableName=tname` in the taos.cfg file. Insert data `st,tname=cpu1,t1=4 c1=3 1626006833639000000` then the table name will be cpu1. If there are multiple lines has same tname but different tag_set, the first line's tag_set will be used to automatically creating table and ignore other lines. Please refer to [TDengine Schemaless](../../reference/schemaless/#Schemaless-Line-Protocol) ::: diff --git a/docs/en/20-third-party/05-collectd.md b/docs/en/20-third-party/05-collectd.md index d8c8e7f81d..5672852cd0 100644 --- a/docs/en/20-third-party/05-collectd.md +++ b/docs/en/20-third-party/05-collectd.md @@ -15,7 +15,7 @@ You can write the data collected by collectd to TDengine by simply modifying the Writing collectd data to the TDengine requires several preparations. - The TDengine cluster is deployed and running properly -- taosAdapter is installed and running, please refer to [taosAdapter's manual](/reference/taosadapter) for details +- taosAdapter is installed and running, please refer to [taosAdapter's manual](../../reference/taosadapter) for details - collectd has been installed. Please refer to the [official documentation](https://collectd.org/download.shtml) to install collectd ## Configuration steps diff --git a/docs/en/20-third-party/07-icinga2.md b/docs/en/20-third-party/07-icinga2.md index 540aae8689..43cd9b83ec 100644 --- a/docs/en/20-third-party/07-icinga2.md +++ b/docs/en/20-third-party/07-icinga2.md @@ -14,7 +14,7 @@ You can write the data collected by icinga2 to TDengine by simply modifying the To write icinga2 data to TDengine requires the following preparations. - The TDengine cluster is deployed and working properly -- taosAdapter is installed and running properly. Please refer to the [taosAdapter manual](/reference/taosadapter) for details. +- taosAdapter is installed and running properly. Please refer to the [taosAdapter manual](../../reference/taosadapter) for details. - icinga2 has been installed. Please refer to the [official documentation](https://icinga.com/docs/icinga-2/latest/doc/02-installation/) for icinga2 installation ## Configuration steps diff --git a/docs/en/20-third-party/08-tcollector.md b/docs/en/20-third-party/08-tcollector.md index f1c0ecd44d..83d01dda24 100644 --- a/docs/en/20-third-party/08-tcollector.md +++ b/docs/en/20-third-party/08-tcollector.md @@ -14,7 +14,7 @@ You can write the data collected by TCollector to TDengine by simply changing th To write data to the TDengine via TCollector requires the following preparations. - The TDengine cluster has been deployed and is working properly -- taosAdapter is installed and running properly. Please refer to the [taosAdapter manual](/reference/taosadapter) for details. +- taosAdapter is installed and running properly. Please refer to the [taosAdapter manual](../../reference/taosadapter) for details. - TCollector has been installed. Please refer to [official documentation](http://opentsdb.net/docs/build/html/user_guide/utilities/tcollector.html#installation-of-tcollector) for TCollector installation ## Configuration steps diff --git a/docs/en/20-third-party/09-emq-broker.md b/docs/en/20-third-party/09-emq-broker.md index 9dab6e52c9..7ca6cd4aa7 100644 --- a/docs/en/20-third-party/09-emq-broker.md +++ b/docs/en/20-third-party/09-emq-broker.md @@ -82,7 +82,7 @@ Edit the resource configuration to add the key/value pairing for Authorization. Basic cm9vdDp0YW9zZGF0YQ== ``` -Please refer to the [ TDengine REST API documentation ](/reference/rest-api/) for the authorization in details. +Please refer to the [ TDengine REST API documentation ](../../reference/rest-api/) for the authorization in details. Enter the rule engine replacement template in the message body: diff --git a/docs/en/20-third-party/11-kafka.md b/docs/en/20-third-party/11-kafka.md index b865c00bc3..cac4f5f604 100644 --- a/docs/en/20-third-party/11-kafka.md +++ b/docs/en/20-third-party/11-kafka.md @@ -94,7 +94,7 @@ The output as bellow: The role of the TDengine Sink Connector is to synchronize the data of the specified topic to TDengine. Users do not need to create databases and super tables in advance. The name of the target database can be specified manually (see the configuration parameter connection.database), or it can be generated according to specific rules (see the configuration parameter connection.database.prefix). -TDengine Sink Connector internally uses TDengine [modeless write interface](/reference/connector/cpp#modeless write-api) to write data to TDengine, currently supports data in three formats: [InfluxDB line protocol format](/develop /insert-data/influxdb-line), [OpenTSDB Telnet protocol format](/develop/insert-data/opentsdb-telnet), and [OpenTSDB JSON protocol format](/develop/insert-data/opentsdb-json). +TDengine Sink Connector internally uses TDengine [modeless write interface](../../reference/connector/cpp#modeless write-api) to write data to TDengine, currently supports data in three formats: [InfluxDB line protocol format](../../develop/insert-data/influxdb-line), [OpenTSDB Telnet protocol format](../../develop/insert-data/opentsdb-telnet), and [OpenTSDB JSON protocol format](../../develop/insert-data/opentsdb-json). The following example synchronizes the data of the topic meters to the target database power. The data format is the InfluxDB Line protocol format. @@ -213,7 +213,7 @@ If you see the above data, the synchronization is successful. If not, check the The role of the TDengine Source Connector is to push all the data of a specific TDengine database after a particular time to Kafka. The implementation principle of TDengine Source Connector is to first pull historical data in batches and then synchronize incremental data with the strategy of the regular query. At the same time, the changes in the table will be monitored, and the newly added table can be automatically synchronized. If Kafka Connect is restarted, synchronization will resume where it left off. -TDengine Source Connector will convert the data in TDengine data table into [InfluxDB Line protocol format](/develop/insert-data/influxdb-line/) or [OpenTSDB JSON protocol format](/develop/insert-data/opentsdb-json ) and then write to Kafka. +TDengine Source Connector will convert the data in TDengine data table into [InfluxDB Line protocol format](../../develop/insert-data/influxdb-line/) or [OpenTSDB JSON protocol format](../../develop/insert-data/opentsdb-json ) and then write to Kafka. The following sample program synchronizes the data in the database test to the topic tdengine-test-meters. diff --git a/docs/en/20-third-party/_deploytaosadapter.mdx b/docs/en/20-third-party/_deploytaosadapter.mdx index 840ca7640a..f8ee53af62 100644 --- a/docs/en/20-third-party/_deploytaosadapter.mdx +++ b/docs/en/20-third-party/_deploytaosadapter.mdx @@ -14,4 +14,4 @@ Check the running status of taosAdapter. systemctl status taosadapter ``` -taosAdapter Please refer to the `taosadapter --help` command output and [reference documentation](/reference/taosadapter) for detailed configuration parameters and usage of taosAdapter. +taosAdapter Please refer to the `taosadapter --help` command output and [reference documentation](../../reference/taosadapter) for detailed configuration parameters and usage of taosAdapter. diff --git a/docs/en/25-application/01-telegraf.md b/docs/en/25-application/01-telegraf.md index 1e3325b2b2..a6db826fa3 100644 --- a/docs/en/25-application/01-telegraf.md +++ b/docs/en/25-application/01-telegraf.md @@ -41,7 +41,7 @@ Download and install the [latest version of TDengine](https://docs.tdengine.com/ ### Install Grafana Plugin and Configure Data Source -Please refer to [Install Grafana Plugin and Configure Data Source](/third-party/grafana/#install-grafana-plugin-and-configure-data-source) +Please refer to [Install Grafana Plugin and Configure Data Source](../../third-party/grafana/#install-grafana-plugin-and-configure-data-source) ### Modify /etc/telegraf/telegraf.conf diff --git a/docs/en/25-application/02-collectd.md b/docs/en/25-application/02-collectd.md index ee1e944928..2c198dcdf5 100644 --- a/docs/en/25-application/02-collectd.md +++ b/docs/en/25-application/02-collectd.md @@ -44,7 +44,7 @@ Download and install the [latest version of TDengine](https://docs.tdengine.com/ ### Install Grafana Plugin and Configure Data Source -Please refer to [Install Grafana Plugin and Configure Data Source](/third-party/grafana/#install-grafana-plugin-and-configure-data-source) +Please refer to [Install Grafana Plugin and Configure Data Source](../../third-party/grafana/#install-grafana-plugin-and-configure-data-source) ### Configure collectd diff --git a/docs/en/25-application/_03-immigrate.md b/docs/en/25-application/_03-immigrate.md index 457a40614e..cdb3d5591c 100644 --- a/docs/en/25-application/_03-immigrate.md +++ b/docs/en/25-application/_03-immigrate.md @@ -70,7 +70,7 @@ You can use collectd and push the data to taosAdapter utilizing the write_tsdb p - **Tuning the Dashboard system** -After writing the data to TDengine, you can configure Grafana to visualize the data written to TDengine. To obtain and use the Grafana plugin provided by TDengine, please refer to [Links to other tools](/third-party/grafana). +After writing the data to TDengine, you can configure Grafana to visualize the data written to TDengine. To obtain and use the Grafana plugin provided by TDengine, please refer to [Links to other tools](../../third-party/grafana). TDengine provides two sets of Dashboard templates by default, and users only need to import the templates from the Grafana directory into Grafana to activate their use. @@ -396,7 +396,7 @@ Hard disk writing performance has little effect on TDengine. The TDengine writin ### Computational resource estimates -Due to the characteristics of IoT data, when the frequency of data generation is consistent, the writing process of TDengine maintains a relatively fixed amount of resource consumption (computing and storage). According to the [TDengine Operation and Maintenance Guide](/operation/) description, the system consumes less than 1 CPU core at 22,000 writes per second. +Due to the characteristics of IoT data, when the frequency of data generation is consistent, the writing process of TDengine maintains a relatively fixed amount of resource consumption (computing and storage). According to the [TDengine Operation and Maintenance Guide](../../operation/) description, the system consumes less than 1 CPU core at 22,000 writes per second. In estimating the CPU resources consumed by the query, assuming that the application requires the database to provide 10,000 QPS, the CPU time consumed by each query is about 1 ms. The query provided by each core per second is 1,000 QPS, which satisfies 10,000 QPS. The query request requires at least 10 cores. For the system as a whole system to have less than 50% CPU load, the entire cluster needs twice as many cores i.e. 20 cores. From d5989b0f749d11c8e51b560b9c5c670777dcf832 Mon Sep 17 00:00:00 2001 From: Bob Liu Date: Thu, 30 Nov 2023 11:56:33 +0800 Subject: [PATCH 195/195] revert nodes func --- include/common/tmsg.h | 3 +- include/libs/nodes/nodes.h | 1 - source/libs/nodes/src/nodesCodeFuncs.c | 3272 ++++++++---------------- source/libs/nodes/src/nodesUtilFuncs.c | 1240 ++++++++- 4 files changed, 2255 insertions(+), 2261 deletions(-) diff --git a/include/common/tmsg.h b/include/common/tmsg.h index 07e571c67f..73d1ab2473 100644 --- a/include/common/tmsg.h +++ b/include/common/tmsg.h @@ -414,8 +414,7 @@ typedef enum ENodeType { QUERY_NODE_PHYSICAL_PLAN_STREAM_EVENT, QUERY_NODE_PHYSICAL_PLAN_HASH_JOIN, QUERY_NODE_PHYSICAL_PLAN_GROUP_CACHE, - QUERY_NODE_PHYSICAL_PLAN_DYN_QUERY_CTRL, - QUERY_NODE_END + QUERY_NODE_PHYSICAL_PLAN_DYN_QUERY_CTRL } ENodeType; typedef struct { diff --git a/include/libs/nodes/nodes.h b/include/libs/nodes/nodes.h index bcbfd245f8..7fbdbfb211 100644 --- a/include/libs/nodes/nodes.h +++ b/include/libs/nodes/nodes.h @@ -110,7 +110,6 @@ int64_t nodesMakeAllocatorWeakRef(int64_t allocatorId); int64_t nodesReleaseAllocatorWeakRef(int64_t allocatorId); void nodesDestroyAllocator(int64_t allocatorId); -int32_t getNodeSize(ENodeType type); SNode* nodesMakeNode(ENodeType type); void nodesDestroyNode(SNode* pNode); void nodesFree(void* p); diff --git a/source/libs/nodes/src/nodesCodeFuncs.c b/source/libs/nodes/src/nodesCodeFuncs.c index d6193397f2..f3087dd5d4 100644 --- a/source/libs/nodes/src/nodesCodeFuncs.c +++ b/source/libs/nodes/src/nodesCodeFuncs.c @@ -27,65 +27,380 @@ static int32_t jsonToNode(const SJson* pJson, void* pObj); static int32_t jsonToNodeObject(const SJson* pJson, const char* pName, SNode** pNode); static int32_t makeNodeByJson(const SJson* pJson, SNode** pNode); - -typedef int32_t (*FExecNodeToJson)(const void* pObj, SJson* pJson); -typedef int32_t (*FExecJsonToNode)(const SJson* pJson, void* pObj); -typedef void (*FExecDestoryNode)(SNode* pNode); - -/** - * @brief Node operation to binding function set - */ -typedef struct SBuiltinNodeDefinition { - const char* name; - int32_t nodeSize; - FExecNodeToJson toJsonFunc; - FExecJsonToNode toNodeFunc; - FExecDestoryNode destoryFunc; -} SBuiltinNodeDefinition; - -SBuiltinNodeDefinition funcNodes[QUERY_NODE_END] = {NULL}; - -static TdThreadOnce functionNodeInit = PTHREAD_ONCE_INIT; - - -static void setFunc(const char* name, int32_t type, int32_t nodeSize, FExecNodeToJson toJsonFunc, - FExecJsonToNode toNodeFunc, FExecDestoryNode destoryFunc) { - funcNodes[type].name = name; - funcNodes[type].nodeSize = nodeSize; - funcNodes[type].toJsonFunc = toJsonFunc; - funcNodes[type].toNodeFunc = toNodeFunc; - funcNodes[type].destoryFunc = destoryFunc; -} - -static void doInitNodeFuncArray(); - -bool funcArrayCheck(int32_t type) { - if (type < 0 || QUERY_NODE_END <= type) { - nodesError("funcArrayCheck out of range type = %d", type); - return false; - } - // only init once, do nothing when run funcArrayCheck again - taosThreadOnce(&functionNodeInit, doInitNodeFuncArray); - - if (!funcNodes[type].name) { - nodesError("funcArrayCheck unsupported type = %d", type); - return false; - } - return true; -} - -int32_t getNodeSize(ENodeType type) { - if (!funcArrayCheck(type)) { - return 0; - } - return funcNodes[type].nodeSize; -} - const char* nodesNodeName(ENodeType type) { - if (!funcArrayCheck(type)) { - return NULL; + switch (type) { + case QUERY_NODE_COLUMN: + return "Column"; + case QUERY_NODE_VALUE: + return "Value"; + case QUERY_NODE_OPERATOR: + return "Operator"; + case QUERY_NODE_LOGIC_CONDITION: + return "LogicCondition"; + case QUERY_NODE_FUNCTION: + return "Function"; + case QUERY_NODE_REAL_TABLE: + return "RealTable"; + case QUERY_NODE_TEMP_TABLE: + return "TempTable"; + case QUERY_NODE_JOIN_TABLE: + return "JoinTable"; + case QUERY_NODE_GROUPING_SET: + return "GroupingSet"; + case QUERY_NODE_ORDER_BY_EXPR: + return "OrderByExpr"; + case QUERY_NODE_LIMIT: + return "Limit"; + case QUERY_NODE_STATE_WINDOW: + return "StateWindow"; + case QUERY_NODE_SESSION_WINDOW: + return "SessionWinow"; + case QUERY_NODE_INTERVAL_WINDOW: + return "IntervalWindow"; + case QUERY_NODE_NODE_LIST: + return "NodeList"; + case QUERY_NODE_FILL: + return "Fill"; + case QUERY_NODE_RAW_EXPR: + return "RawExpr"; + case QUERY_NODE_TARGET: + return "Target"; + case QUERY_NODE_DATABLOCK_DESC: + return "DataBlockDesc"; + case QUERY_NODE_SLOT_DESC: + return "SlotDesc"; + case QUERY_NODE_COLUMN_DEF: + return "ColumnDef"; + case QUERY_NODE_DOWNSTREAM_SOURCE: + return "DownstreamSource"; + case QUERY_NODE_DATABASE_OPTIONS: + return "DatabaseOptions"; + case QUERY_NODE_TABLE_OPTIONS: + return "TableOptions"; + case QUERY_NODE_INDEX_OPTIONS: + return "IndexOptions"; + case QUERY_NODE_EXPLAIN_OPTIONS: + return "ExplainOptions"; + case QUERY_NODE_STREAM_OPTIONS: + return "StreamOptions"; + case QUERY_NODE_LEFT_VALUE: + return "LeftValue"; + case QUERY_NODE_WHEN_THEN: + return "WhenThen"; + case QUERY_NODE_CASE_WHEN: + return "CaseWhen"; + case QUERY_NODE_EVENT_WINDOW: + return "EventWindow"; + case QUERY_NODE_SET_OPERATOR: + return "SetOperator"; + case QUERY_NODE_SELECT_STMT: + return "SelectStmt"; + case QUERY_NODE_VNODE_MODIFY_STMT: + return "VnodeModifStmt"; + case QUERY_NODE_CREATE_DATABASE_STMT: + return "CreateDatabaseStmt"; + case QUERY_NODE_DROP_DATABASE_STMT: + return "DropDatabaseStmt"; + case QUERY_NODE_ALTER_DATABASE_STMT: + return "AlterDatabaseStmt"; + case QUERY_NODE_FLUSH_DATABASE_STMT: + return "FlushDatabaseStmt"; + case QUERY_NODE_TRIM_DATABASE_STMT: + return "TrimDatabaseStmt"; + case QUERY_NODE_CREATE_TABLE_STMT: + return "CreateTableStmt"; + case QUERY_NODE_CREATE_SUBTABLE_CLAUSE: + return "CreateSubtableClause"; + case QUERY_NODE_CREATE_MULTI_TABLES_STMT: + return "CreateMultiTableStmt"; + case QUERY_NODE_DROP_TABLE_CLAUSE: + return "DropTableClause"; + case QUERY_NODE_DROP_TABLE_STMT: + return "DropTableStmt"; + case QUERY_NODE_DROP_SUPER_TABLE_STMT: + return "DropSuperTableStmt"; + case QUERY_NODE_ALTER_TABLE_STMT: + return "AlterTableStmt"; + case QUERY_NODE_ALTER_SUPER_TABLE_STMT: + return "AlterSuperTableStmt"; + case QUERY_NODE_CREATE_USER_STMT: + return "CreateUserStmt"; + case QUERY_NODE_ALTER_USER_STMT: + return "AlterUserStmt"; + case QUERY_NODE_DROP_USER_STMT: + return "DropUserStmt"; + case QUERY_NODE_USE_DATABASE_STMT: + return "UseDatabaseStmt"; + case QUERY_NODE_CREATE_DNODE_STMT: + return "CreateDnodeStmt"; + case QUERY_NODE_DROP_DNODE_STMT: + return "DropDnodeStmt"; + case QUERY_NODE_ALTER_DNODE_STMT: + return "AlterDnodeStmt"; + case QUERY_NODE_CREATE_INDEX_STMT: + return "CreateIndexStmt"; + case QUERY_NODE_DROP_INDEX_STMT: + return "DropIndexStmt"; + case QUERY_NODE_CREATE_QNODE_STMT: + return "CreateQnodeStmt"; + case QUERY_NODE_DROP_QNODE_STMT: + return "DropQnodeStmt"; + case QUERY_NODE_CREATE_SNODE_STMT: + return "CreateSnodeStmt"; + case QUERY_NODE_DROP_SNODE_STMT: + return "DropSnodeStmt"; + case QUERY_NODE_CREATE_MNODE_STMT: + return "CreateMnodeStmt"; + case QUERY_NODE_DROP_MNODE_STMT: + return "DropMnodeStmt"; + case QUERY_NODE_CREATE_TOPIC_STMT: + return "CreateTopicStmt"; + case QUERY_NODE_DROP_TOPIC_STMT: + return "DropTopicStmt"; + case QUERY_NODE_DROP_CGROUP_STMT: + return "DropConsumerGroupStmt"; + case QUERY_NODE_ALTER_LOCAL_STMT: + return "AlterLocalStmt"; + case QUERY_NODE_EXPLAIN_STMT: + return "ExplainStmt"; + case QUERY_NODE_DESCRIBE_STMT: + return "DescribeStmt"; + case QUERY_NODE_COMPACT_DATABASE_STMT: + return "CompactDatabaseStmt"; + case QUERY_NODE_CREATE_STREAM_STMT: + return "CreateStreamStmt"; + case QUERY_NODE_DROP_STREAM_STMT: + return "DropStreamStmt"; + case QUERY_NODE_PAUSE_STREAM_STMT: + return "PauseStreamStmt"; + case QUERY_NODE_RESUME_STREAM_STMT: + return "ResumeStreamStmt"; + case QUERY_NODE_BALANCE_VGROUP_STMT: + return "BalanceVgroupStmt"; + case QUERY_NODE_BALANCE_VGROUP_LEADER_STMT: + return "BalanceVgroupLeaderStmt"; + case QUERY_NODE_MERGE_VGROUP_STMT: + return "MergeVgroupStmt"; + case QUERY_NODE_SHOW_DB_ALIVE_STMT: + return "ShowDbAliveStmt"; + case QUERY_NODE_SHOW_CLUSTER_ALIVE_STMT: + return "ShowClusterAliveStmt"; + case QUERY_NODE_REDISTRIBUTE_VGROUP_STMT: + return "RedistributeVgroupStmt"; + case QUERY_NODE_SPLIT_VGROUP_STMT: + return "SplitVgroupStmt"; + case QUERY_NODE_GRANT_STMT: + return "GrantStmt"; + case QUERY_NODE_REVOKE_STMT: + return "RevokeStmt"; + case QUERY_NODE_SHOW_DNODES_STMT: + return "ShowDnodesStmt"; + case QUERY_NODE_SHOW_MNODES_STMT: + return "ShowMnodesStmt"; + case QUERY_NODE_SHOW_MODULES_STMT: + return "ShowModulesStmt"; + case QUERY_NODE_SHOW_QNODES_STMT: + return "ShowQnodesStmt"; + case QUERY_NODE_SHOW_SNODES_STMT: + return "ShowSnodesStmt"; + case QUERY_NODE_SHOW_BNODES_STMT: + return "ShowBnodesStmt"; + case QUERY_NODE_SHOW_CLUSTER_STMT: + return "ShowClusterStmt"; + case QUERY_NODE_SHOW_DATABASES_STMT: + return "ShowDatabaseStmt"; + case QUERY_NODE_SHOW_FUNCTIONS_STMT: + return "ShowFunctionsStmt"; + case QUERY_NODE_SHOW_INDEXES_STMT: + return "ShowIndexesStmt"; + case QUERY_NODE_SHOW_STABLES_STMT: + return "ShowStablesStmt"; + case QUERY_NODE_SHOW_STREAMS_STMT: + return "ShowStreamsStmt"; + case QUERY_NODE_SHOW_TABLES_STMT: + return "ShowTablesStmt"; + case QUERY_NODE_SHOW_TAGS_STMT: + return "ShowTagsStmt"; + case QUERY_NODE_SHOW_USERS_STMT: + return "ShowUsersStmt"; + case QUERY_NODE_SHOW_LICENCES_STMT: + return "ShowGrantsStmt"; + case QUERY_NODE_SHOW_VGROUPS_STMT: + return "ShowVgroupsStmt"; + case QUERY_NODE_SHOW_TOPICS_STMT: + return "ShowTopicsStmt"; + case QUERY_NODE_SHOW_CONSUMERS_STMT: + return "ShowConsumersStmt"; + case QUERY_NODE_SHOW_QUERIES_STMT: + return "ShowQueriesStmt"; + case QUERY_NODE_SHOW_VARIABLES_STMT: + return "ShowVariablesStmt"; + case QUERY_NODE_SHOW_DNODE_VARIABLES_STMT: + return "ShowDnodeVariablesStmt"; + case QUERY_NODE_SHOW_TRANSACTIONS_STMT: + return "ShowTransactionsStmt"; + case QUERY_NODE_SHOW_SUBSCRIPTIONS_STMT: + return "ShowSubscriptionsStmt"; + case QUERY_NODE_SHOW_VNODES_STMT: + return "ShowVnodeStmt"; + case QUERY_NODE_SHOW_USER_PRIVILEGES_STMT: + return "ShowUserPrivilegesStmt"; + case QUERY_NODE_SHOW_CREATE_DATABASE_STMT: + return "ShowCreateDatabasesStmt"; + case QUERY_NODE_SHOW_CREATE_TABLE_STMT: + return "ShowCreateTablesStmt"; + case QUERY_NODE_SHOW_CREATE_STABLE_STMT: + return "ShowCreateStablesStmt"; + case QUERY_NODE_SHOW_CREATE_VIEW_STMT: + return "ShowCreateViewStmt"; + case QUERY_NODE_SHOW_TABLE_DISTRIBUTED_STMT: + return "ShowTableDistributedStmt"; + case QUERY_NODE_SHOW_LOCAL_VARIABLES_STMT: + return "ShowLocalVariablesStmt"; + case QUERY_NODE_SHOW_TABLE_TAGS_STMT: + return "ShowTableTagsStmt"; + case QUERY_NODE_DELETE_STMT: + return "DeleteStmt"; + case QUERY_NODE_INSERT_STMT: + return "InsertStmt"; + case QUERY_NODE_RESTORE_DNODE_STMT: + return "RestoreDnodeStmt"; + case QUERY_NODE_RESTORE_QNODE_STMT: + return "RestoreQnodeStmt"; + case QUERY_NODE_RESTORE_MNODE_STMT: + return "RestoreMnodeStmt"; + case QUERY_NODE_RESTORE_VNODE_STMT: + return "RestoreVnodeStmt"; + case QUERY_NODE_CREATE_VIEW_STMT: + return "CreateViewStmt"; + case QUERY_NODE_DROP_VIEW_STMT: + return "DropViewStmt"; + case QUERY_NODE_LOGIC_PLAN_SCAN: + return "LogicScan"; + case QUERY_NODE_LOGIC_PLAN_JOIN: + return "LogicJoin"; + case QUERY_NODE_LOGIC_PLAN_AGG: + return "LogicAgg"; + case QUERY_NODE_LOGIC_PLAN_PROJECT: + return "LogicProject"; + case QUERY_NODE_LOGIC_PLAN_VNODE_MODIFY: + return "LogicVnodeModify"; + case QUERY_NODE_LOGIC_PLAN_EXCHANGE: + return "LogicExchange"; + case QUERY_NODE_LOGIC_PLAN_MERGE: + return "LogicMerge"; + case QUERY_NODE_LOGIC_PLAN_WINDOW: + return "LogicWindow"; + case QUERY_NODE_LOGIC_PLAN_FILL: + return "LogicFill"; + case QUERY_NODE_LOGIC_PLAN_SORT: + return "LogicSort"; + case QUERY_NODE_LOGIC_PLAN_PARTITION: + return "LogicPartition"; + case QUERY_NODE_LOGIC_PLAN_INDEF_ROWS_FUNC: + return "LogicIndefRowsFunc"; + case QUERY_NODE_LOGIC_PLAN_INTERP_FUNC: + return "LogicInterpFunc"; + case QUERY_NODE_LOGIC_PLAN_GROUP_CACHE: + return "LogicGroupCache"; + case QUERY_NODE_LOGIC_PLAN_DYN_QUERY_CTRL: + return "LogicDynamicQueryCtrl"; + case QUERY_NODE_LOGIC_SUBPLAN: + return "LogicSubplan"; + case QUERY_NODE_LOGIC_PLAN: + return "LogicPlan"; + case QUERY_NODE_PHYSICAL_PLAN_TAG_SCAN: + return "PhysiTagScan"; + case QUERY_NODE_PHYSICAL_PLAN_TABLE_SCAN: + return "PhysiTableScan"; + case QUERY_NODE_PHYSICAL_PLAN_TABLE_SEQ_SCAN: + return "PhysiTableSeqScan"; + case QUERY_NODE_PHYSICAL_PLAN_TABLE_MERGE_SCAN: + return "PhysiTableMergeScan"; + case QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN: + return "PhysiSreamScan"; + case QUERY_NODE_PHYSICAL_PLAN_SYSTABLE_SCAN: + return "PhysiSystemTableScan"; + case QUERY_NODE_PHYSICAL_PLAN_BLOCK_DIST_SCAN: + return "PhysiBlockDistScan"; + case QUERY_NODE_PHYSICAL_PLAN_LAST_ROW_SCAN: + return "PhysiLastRowScan"; + case QUERY_NODE_PHYSICAL_PLAN_TABLE_COUNT_SCAN: + return "PhysiTableCountScan"; + case QUERY_NODE_PHYSICAL_PLAN_MERGE_EVENT: + return "PhysiMergeEventWindow"; + case QUERY_NODE_PHYSICAL_PLAN_STREAM_EVENT: + return "PhysiStreamEventWindow"; + case QUERY_NODE_PHYSICAL_PLAN_PROJECT: + return "PhysiProject"; + case QUERY_NODE_PHYSICAL_PLAN_MERGE_JOIN: + return "PhysiMergeJoin"; + case QUERY_NODE_PHYSICAL_PLAN_HASH_JOIN: + return "PhysiHashJoin"; + case QUERY_NODE_PHYSICAL_PLAN_HASH_AGG: + return "PhysiAgg"; + case QUERY_NODE_PHYSICAL_PLAN_EXCHANGE: + return "PhysiExchange"; + case QUERY_NODE_PHYSICAL_PLAN_MERGE: + return "PhysiMerge"; + case QUERY_NODE_PHYSICAL_PLAN_SORT: + return "PhysiSort"; + case QUERY_NODE_PHYSICAL_PLAN_GROUP_SORT: + return "PhysiGroupSort"; + case QUERY_NODE_PHYSICAL_PLAN_HASH_INTERVAL: + return "PhysiHashInterval"; + case QUERY_NODE_PHYSICAL_PLAN_MERGE_ALIGNED_INTERVAL: + return "PhysiMergeAlignedInterval"; + case QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL: + return "PhysiStreamInterval"; + case QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL: + return "PhysiStreamFinalInterval"; + case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL: + return "PhysiStreamSemiInterval"; + case QUERY_NODE_PHYSICAL_PLAN_FILL: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_FILL: + return "PhysiFill"; + case QUERY_NODE_PHYSICAL_PLAN_MERGE_SESSION: + return "PhysiSessionWindow"; + case QUERY_NODE_PHYSICAL_PLAN_STREAM_SESSION: + return "PhysiStreamSessionWindow"; + case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_SESSION: + return "PhysiStreamSemiSessionWindow"; + case QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_SESSION: + return "PhysiStreamFinalSessionWindow"; + case QUERY_NODE_PHYSICAL_PLAN_MERGE_STATE: + return "PhysiStateWindow"; + case QUERY_NODE_PHYSICAL_PLAN_STREAM_STATE: + return "PhysiStreamStateWindow"; + case QUERY_NODE_PHYSICAL_PLAN_PARTITION: + return "PhysiPartition"; + case QUERY_NODE_PHYSICAL_PLAN_STREAM_PARTITION: + return "PhysiStreamPartition"; + case QUERY_NODE_PHYSICAL_PLAN_INDEF_ROWS_FUNC: + return "PhysiIndefRowsFunc"; + case QUERY_NODE_PHYSICAL_PLAN_INTERP_FUNC: + return "PhysiInterpFunc"; + case QUERY_NODE_PHYSICAL_PLAN_DISPATCH: + return "PhysiDispatch"; + case QUERY_NODE_PHYSICAL_PLAN_INSERT: + return "PhysiInsert"; + case QUERY_NODE_PHYSICAL_PLAN_QUERY_INSERT: + return "PhysiQueryInsert"; + case QUERY_NODE_PHYSICAL_PLAN_DELETE: + return "PhysiDelete"; + case QUERY_NODE_PHYSICAL_PLAN_GROUP_CACHE: + return "PhysiGroupCache"; + case QUERY_NODE_PHYSICAL_PLAN_DYN_QUERY_CTRL: + return "PhysiDynamicQueryCtrl"; + case QUERY_NODE_PHYSICAL_SUBPLAN: + return "PhysiSubplan"; + case QUERY_NODE_PHYSICAL_PLAN: + return "PhysiPlan"; + default: + break; } - return funcNodes[type].name; + nodesWarn("nodesNodeName unknown node = %d", type); + return "UnknownNode"; } static int32_t nodeListToJson(SJson* pJson, const char* pName, const SNodeList* pList) { @@ -6488,31 +6803,657 @@ static int32_t jsonToInsertStmt(const SJson* pJson, void* pObj) { return code; } -int32_t specificNodeToJson(const void* pObj, SJson* pJson) { - ENodeType type = nodeType(pObj); - if (!funcArrayCheck(type)) { - return TSDB_CODE_NOT_FOUND; +static int32_t specificNodeToJson(const void* pObj, SJson* pJson) { + switch (nodeType(pObj)) { + case QUERY_NODE_COLUMN: + return columnNodeToJson(pObj, pJson); + case QUERY_NODE_VALUE: + return valueNodeToJson(pObj, pJson); + case QUERY_NODE_OPERATOR: + return operatorNodeToJson(pObj, pJson); + case QUERY_NODE_LOGIC_CONDITION: + return logicConditionNodeToJson(pObj, pJson); + case QUERY_NODE_FUNCTION: + return functionNodeToJson(pObj, pJson); + case QUERY_NODE_REAL_TABLE: + return realTableNodeToJson(pObj, pJson); + case QUERY_NODE_TEMP_TABLE: + return tempTableNodeToJson(pObj, pJson); + case QUERY_NODE_JOIN_TABLE: + return joinTableNodeToJson(pObj, pJson); + case QUERY_NODE_GROUPING_SET: + return groupingSetNodeToJson(pObj, pJson); + case QUERY_NODE_ORDER_BY_EXPR: + return orderByExprNodeToJson(pObj, pJson); + case QUERY_NODE_LIMIT: + return limitNodeToJson(pObj, pJson); + case QUERY_NODE_STATE_WINDOW: + return stateWindowNodeToJson(pObj, pJson); + case QUERY_NODE_SESSION_WINDOW: + return sessionWindowNodeToJson(pObj, pJson); + case QUERY_NODE_INTERVAL_WINDOW: + return intervalWindowNodeToJson(pObj, pJson); + case QUERY_NODE_NODE_LIST: + return nodeListNodeToJson(pObj, pJson); + case QUERY_NODE_FILL: + return fillNodeToJson(pObj, pJson); + case QUERY_NODE_RAW_EXPR: + break; + case QUERY_NODE_TARGET: + return targetNodeToJson(pObj, pJson); + case QUERY_NODE_DATABLOCK_DESC: + return dataBlockDescNodeToJson(pObj, pJson); + case QUERY_NODE_SLOT_DESC: + return slotDescNodeToJson(pObj, pJson); + case QUERY_NODE_COLUMN_DEF: + return columnDefNodeToJson(pObj, pJson); + case QUERY_NODE_DOWNSTREAM_SOURCE: + return downstreamSourceNodeToJson(pObj, pJson); + case QUERY_NODE_DATABASE_OPTIONS: + return databaseOptionsToJson(pObj, pJson); + case QUERY_NODE_TABLE_OPTIONS: + return tableOptionsToJson(pObj, pJson); + case QUERY_NODE_INDEX_OPTIONS: + return indexOptionsToJson(pObj, pJson); + case QUERY_NODE_EXPLAIN_OPTIONS: + return explainOptionsToJson(pObj, pJson); + case QUERY_NODE_STREAM_OPTIONS: + return streamOptionsToJson(pObj, pJson); + case QUERY_NODE_LEFT_VALUE: + return TSDB_CODE_SUCCESS; // SLeftValueNode has no fields to serialize. + case QUERY_NODE_WHEN_THEN: + return whenThenNodeToJson(pObj, pJson); + case QUERY_NODE_CASE_WHEN: + return caseWhenNodeToJson(pObj, pJson); + case QUERY_NODE_EVENT_WINDOW: + return eventWindowNodeToJson(pObj, pJson); + case QUERY_NODE_SET_OPERATOR: + return setOperatorToJson(pObj, pJson); + case QUERY_NODE_SELECT_STMT: + return selectStmtToJson(pObj, pJson); + case QUERY_NODE_VNODE_MODIFY_STMT: + return vnodeModifyStmtToJson(pObj, pJson); + case QUERY_NODE_CREATE_DATABASE_STMT: + return createDatabaseStmtToJson(pObj, pJson); + case QUERY_NODE_ALTER_DATABASE_STMT: + return alterDatabaseStmtToJson(pObj, pJson); + case QUERY_NODE_TRIM_DATABASE_STMT: + return trimDatabaseStmtToJson(pObj, pJson); + case QUERY_NODE_CREATE_TABLE_STMT: + return createTableStmtToJson(pObj, pJson); + case QUERY_NODE_CREATE_SUBTABLE_CLAUSE: + return createSubTableClauseToJson(pObj, pJson); + case QUERY_NODE_CREATE_MULTI_TABLES_STMT: + return createMultiTablesStmtToJson(pObj, pJson); + case QUERY_NODE_DROP_TABLE_CLAUSE: + return dropTableClauseToJson(pObj, pJson); + case QUERY_NODE_DROP_TABLE_STMT: + return dropTableStmtToJson(pObj, pJson); + case QUERY_NODE_DROP_SUPER_TABLE_STMT: + return dropStableStmtToJson(pObj, pJson); + case QUERY_NODE_ALTER_TABLE_STMT: + return alterTableStmtToJson(pObj, pJson); + case QUERY_NODE_ALTER_SUPER_TABLE_STMT: + return alterStableStmtToJson(pObj, pJson); + case QUERY_NODE_CREATE_USER_STMT: + return createUserStmtToJson(pObj, pJson); + case QUERY_NODE_ALTER_USER_STMT: + return alterUserStmtToJson(pObj, pJson); + case QUERY_NODE_DROP_USER_STMT: + return dropUserStmtToJson(pObj, pJson); + case QUERY_NODE_USE_DATABASE_STMT: + return useDatabaseStmtToJson(pObj, pJson); + case QUERY_NODE_CREATE_DNODE_STMT: + return createDnodeStmtToJson(pObj, pJson); + case QUERY_NODE_DROP_DNODE_STMT: + return dropDnodeStmtToJson(pObj, pJson); + case QUERY_NODE_ALTER_DNODE_STMT: + return alterDnodeStmtToJson(pObj, pJson); + case QUERY_NODE_CREATE_INDEX_STMT: + return createIndexStmtToJson(pObj, pJson); + case QUERY_NODE_DROP_INDEX_STMT: + return dropIndexStmtToJson(pObj, pJson); + case QUERY_NODE_CREATE_QNODE_STMT: + return createQnodeStmtToJson(pObj, pJson); + case QUERY_NODE_DROP_QNODE_STMT: + return dropQnodeStmtToJson(pObj, pJson); + case QUERY_NODE_CREATE_SNODE_STMT: + return createSnodeStmtToJson(pObj, pJson); + case QUERY_NODE_DROP_SNODE_STMT: + return dropSnodeStmtToJson(pObj, pJson); + case QUERY_NODE_CREATE_MNODE_STMT: + return createMnodeStmtToJson(pObj, pJson); + case QUERY_NODE_DROP_MNODE_STMT: + return dropMnodeStmtToJson(pObj, pJson); + case QUERY_NODE_CREATE_TOPIC_STMT: + return createTopicStmtToJson(pObj, pJson); + case QUERY_NODE_DROP_TOPIC_STMT: + return dropTopicStmtToJson(pObj, pJson); + case QUERY_NODE_DROP_CGROUP_STMT: + return dropConsumerGroupStmtToJson(pObj, pJson); + case QUERY_NODE_ALTER_LOCAL_STMT: + return alterLocalStmtToJson(pObj, pJson); + case QUERY_NODE_EXPLAIN_STMT: + return explainStmtToJson(pObj, pJson); + case QUERY_NODE_DESCRIBE_STMT: + return describeStmtToJson(pObj, pJson); + case QUERY_NODE_COMPACT_DATABASE_STMT: + return compactDatabaseStmtToJson(pObj, pJson); + case QUERY_NODE_CREATE_STREAM_STMT: + return createStreamStmtToJson(pObj, pJson); + case QUERY_NODE_DROP_STREAM_STMT: + return dropStreamStmtToJson(pObj, pJson); + case QUERY_NODE_BALANCE_VGROUP_STMT: + return TSDB_CODE_SUCCESS; // SBalanceVgroupStmt has no fields to serialize. + case QUERY_NODE_BALANCE_VGROUP_LEADER_STMT: + return TSDB_CODE_SUCCESS; // SBalanceVgroupLeaderStmt has no fields to serialize. + case QUERY_NODE_MERGE_VGROUP_STMT: + return mergeVgroupStmtToJson(pObj, pJson); + case QUERY_NODE_REDISTRIBUTE_VGROUP_STMT: + return redistributeVgroupStmtToJson(pObj, pJson); + case QUERY_NODE_SPLIT_VGROUP_STMT: + return splitVgroupStmtToJson(pObj, pJson); + case QUERY_NODE_GRANT_STMT: + return grantStmtToJson(pObj, pJson); + case QUERY_NODE_REVOKE_STMT: + return revokeStmtToJson(pObj, pJson); + case QUERY_NODE_SHOW_DNODES_STMT: + return showDnodesStmtToJson(pObj, pJson); + case QUERY_NODE_SHOW_MNODES_STMT: + return showMnodesStmtToJson(pObj, pJson); + case QUERY_NODE_SHOW_QNODES_STMT: + return showQnodesStmtToJson(pObj, pJson); + case QUERY_NODE_SHOW_CLUSTER_STMT: + return showClusterStmtToJson(pObj, pJson); + case QUERY_NODE_SHOW_DATABASES_STMT: + return showDatabasesStmtToJson(pObj, pJson); + case QUERY_NODE_SHOW_FUNCTIONS_STMT: + return showFunctionsStmtToJson(pObj, pJson); + case QUERY_NODE_SHOW_INDEXES_STMT: + return showIndexesStmtToJson(pObj, pJson); + case QUERY_NODE_SHOW_STABLES_STMT: + return showStablesStmtToJson(pObj, pJson); + case QUERY_NODE_SHOW_STREAMS_STMT: + return showStreamsStmtToJson(pObj, pJson); + case QUERY_NODE_SHOW_TABLES_STMT: + return showTablesStmtToJson(pObj, pJson); + case QUERY_NODE_SHOW_TAGS_STMT: + return showTagsStmtToJson(pObj, pJson); + case QUERY_NODE_SHOW_USERS_STMT: + return showUsersStmtToJson(pObj, pJson); + case QUERY_NODE_SHOW_VGROUPS_STMT: + return showVgroupsStmtToJson(pObj, pJson); + case QUERY_NODE_SHOW_CONSUMERS_STMT: + return showConsumersStmtToJson(pObj, pJson); + case QUERY_NODE_SHOW_VARIABLES_STMT: + return showVariablesStmtToJson(pObj, pJson); + case QUERY_NODE_SHOW_DNODE_VARIABLES_STMT: + return showDnodeVariablesStmtToJson(pObj, pJson); + case QUERY_NODE_SHOW_TRANSACTIONS_STMT: + return showTransactionsStmtToJson(pObj, pJson); + case QUERY_NODE_SHOW_SUBSCRIPTIONS_STMT: + return showSubscriptionsStmtToJson(pObj, pJson); + case QUERY_NODE_SHOW_VNODES_STMT: + return showVnodesStmtToJson(pObj, pJson); + case QUERY_NODE_SHOW_USER_PRIVILEGES_STMT: + return showUserPrivilegesStmtToJson(pObj, pJson); + case QUERY_NODE_SHOW_CREATE_DATABASE_STMT: + return showCreateDatabaseStmtToJson(pObj, pJson); + case QUERY_NODE_SHOW_CREATE_TABLE_STMT: + return showCreateTableStmtToJson(pObj, pJson); + case QUERY_NODE_SHOW_CREATE_STABLE_STMT: + return showCreateStableStmtToJson(pObj, pJson); + case QUERY_NODE_SHOW_CREATE_VIEW_STMT: + return showCreateViewStmtToJson(pObj, pJson); + case QUERY_NODE_SHOW_TABLE_DISTRIBUTED_STMT: + return showTableDistributedStmtToJson(pObj, pJson); + case QUERY_NODE_SHOW_LOCAL_VARIABLES_STMT: + return showLocalVariablesStmtToJson(pObj, pJson); + case QUERY_NODE_SHOW_TABLE_TAGS_STMT: + return showTableTagsStmtToJson(pObj, pJson); + case QUERY_NODE_DELETE_STMT: + return deleteStmtToJson(pObj, pJson); + case QUERY_NODE_INSERT_STMT: + return insertStmtToJson(pObj, pJson); + case QUERY_NODE_LOGIC_PLAN_SCAN: + return logicScanNodeToJson(pObj, pJson); + case QUERY_NODE_LOGIC_PLAN_JOIN: + return logicJoinNodeToJson(pObj, pJson); + case QUERY_NODE_LOGIC_PLAN_AGG: + return logicAggNodeToJson(pObj, pJson); + case QUERY_NODE_LOGIC_PLAN_PROJECT: + return logicProjectNodeToJson(pObj, pJson); + case QUERY_NODE_LOGIC_PLAN_VNODE_MODIFY: + return logicVnodeModifyNodeToJson(pObj, pJson); + case QUERY_NODE_LOGIC_PLAN_EXCHANGE: + return logicExchangeNodeToJson(pObj, pJson); + case QUERY_NODE_LOGIC_PLAN_MERGE: + return logicMergeNodeToJson(pObj, pJson); + case QUERY_NODE_LOGIC_PLAN_WINDOW: + return logicWindowNodeToJson(pObj, pJson); + case QUERY_NODE_LOGIC_PLAN_FILL: + return logicFillNodeToJson(pObj, pJson); + case QUERY_NODE_LOGIC_PLAN_SORT: + return logicSortNodeToJson(pObj, pJson); + case QUERY_NODE_LOGIC_PLAN_PARTITION: + return logicPartitionNodeToJson(pObj, pJson); + case QUERY_NODE_LOGIC_PLAN_INDEF_ROWS_FUNC: + return logicIndefRowsFuncNodeToJson(pObj, pJson); + case QUERY_NODE_LOGIC_PLAN_INTERP_FUNC: + return logicInterpFuncNodeToJson(pObj, pJson); + case QUERY_NODE_LOGIC_PLAN_GROUP_CACHE: + return logicGroupCacheNodeToJson(pObj, pJson); + case QUERY_NODE_LOGIC_PLAN_DYN_QUERY_CTRL: + return logicDynQueryCtrlNodeToJson(pObj, pJson); + case QUERY_NODE_LOGIC_SUBPLAN: + return logicSubplanToJson(pObj, pJson); + case QUERY_NODE_LOGIC_PLAN: + return logicPlanToJson(pObj, pJson); + case QUERY_NODE_PHYSICAL_PLAN_TAG_SCAN: + return physiTagScanNodeToJson(pObj, pJson); + case QUERY_NODE_PHYSICAL_PLAN_BLOCK_DIST_SCAN: + return physiScanNodeToJson(pObj, pJson); + case QUERY_NODE_PHYSICAL_PLAN_LAST_ROW_SCAN: + case QUERY_NODE_PHYSICAL_PLAN_TABLE_COUNT_SCAN: + return physiLastRowScanNodeToJson(pObj, pJson); + case QUERY_NODE_PHYSICAL_PLAN_TABLE_SCAN: + case QUERY_NODE_PHYSICAL_PLAN_TABLE_MERGE_SCAN: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN: + return physiTableScanNodeToJson(pObj, pJson); + case QUERY_NODE_PHYSICAL_PLAN_SYSTABLE_SCAN: + return physiSysTableScanNodeToJson(pObj, pJson); + case QUERY_NODE_PHYSICAL_PLAN_PROJECT: + return physiProjectNodeToJson(pObj, pJson); + case QUERY_NODE_PHYSICAL_PLAN_MERGE_JOIN: + return physiMergeJoinNodeToJson(pObj, pJson); + case QUERY_NODE_PHYSICAL_PLAN_HASH_JOIN: + return physiHashJoinNodeToJson(pObj, pJson); + case QUERY_NODE_PHYSICAL_PLAN_HASH_AGG: + return physiAggNodeToJson(pObj, pJson); + case QUERY_NODE_PHYSICAL_PLAN_EXCHANGE: + return physiExchangeNodeToJson(pObj, pJson); + case QUERY_NODE_PHYSICAL_PLAN_MERGE: + return physiMergeNodeToJson(pObj, pJson); + case QUERY_NODE_PHYSICAL_PLAN_SORT: + case QUERY_NODE_PHYSICAL_PLAN_GROUP_SORT: + return physiSortNodeToJson(pObj, pJson); + case QUERY_NODE_PHYSICAL_PLAN_HASH_INTERVAL: + case QUERY_NODE_PHYSICAL_PLAN_MERGE_ALIGNED_INTERVAL: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL: + return physiIntervalNodeToJson(pObj, pJson); + case QUERY_NODE_PHYSICAL_PLAN_FILL: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_FILL: + return physiFillNodeToJson(pObj, pJson); + case QUERY_NODE_PHYSICAL_PLAN_MERGE_SESSION: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_SESSION: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_SESSION: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_SESSION: + return physiSessionWindowNodeToJson(pObj, pJson); + case QUERY_NODE_PHYSICAL_PLAN_MERGE_STATE: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_STATE: + return physiStateWindowNodeToJson(pObj, pJson); + case QUERY_NODE_PHYSICAL_PLAN_MERGE_EVENT: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_EVENT: + return physiEventWindowNodeToJson(pObj, pJson); + case QUERY_NODE_PHYSICAL_PLAN_PARTITION: + return physiPartitionNodeToJson(pObj, pJson); + case QUERY_NODE_PHYSICAL_PLAN_STREAM_PARTITION: + return physiStreamPartitionNodeToJson(pObj, pJson); + case QUERY_NODE_PHYSICAL_PLAN_INDEF_ROWS_FUNC: + return physiIndefRowsFuncNodeToJson(pObj, pJson); + case QUERY_NODE_PHYSICAL_PLAN_INTERP_FUNC: + return physiInterpFuncNodeToJson(pObj, pJson); + case QUERY_NODE_PHYSICAL_PLAN_DISPATCH: + return physiDispatchNodeToJson(pObj, pJson); + case QUERY_NODE_PHYSICAL_PLAN_INSERT: + break; + case QUERY_NODE_PHYSICAL_PLAN_QUERY_INSERT: + return physiQueryInsertNodeToJson(pObj, pJson); + case QUERY_NODE_PHYSICAL_PLAN_DELETE: + return physiDeleteNodeToJson(pObj, pJson); + case QUERY_NODE_PHYSICAL_PLAN_GROUP_CACHE: + return physiGroupCacheNodeToJson(pObj, pJson); + case QUERY_NODE_PHYSICAL_PLAN_DYN_QUERY_CTRL: + return physiDynQueryCtrlNodeToJson(pObj, pJson); + case QUERY_NODE_PHYSICAL_SUBPLAN: + return subplanToJson(pObj, pJson); + case QUERY_NODE_PHYSICAL_PLAN: + return planToJson(pObj, pJson); + default: + break; } - - if (funcNodes[type].toJsonFunc) { - return funcNodes[type].toJsonFunc(pObj, pJson); - } - - nodesWarn("specificNodeToJson unknown node type = %d", type); + nodesWarn("specificNodeToJson unknown node = %s", nodesNodeName(nodeType(pObj))); return TSDB_CODE_SUCCESS; } -int32_t jsonToSpecificNode(const SJson* pJson, void* pObj) { - ENodeType type = nodeType(pObj); - if (!funcArrayCheck(type)) { - return TSDB_CODE_NOT_FOUND; +static int32_t jsonToSpecificNode(const SJson* pJson, void* pObj) { + switch (nodeType(pObj)) { + case QUERY_NODE_COLUMN: + return jsonToColumnNode(pJson, pObj); + case QUERY_NODE_VALUE: + return jsonToValueNode(pJson, pObj); + case QUERY_NODE_OPERATOR: + return jsonToOperatorNode(pJson, pObj); + case QUERY_NODE_LOGIC_CONDITION: + return jsonToLogicConditionNode(pJson, pObj); + case QUERY_NODE_FUNCTION: + return jsonToFunctionNode(pJson, pObj); + case QUERY_NODE_REAL_TABLE: + return jsonToRealTableNode(pJson, pObj); + case QUERY_NODE_TEMP_TABLE: + return jsonToTempTableNode(pJson, pObj); + case QUERY_NODE_JOIN_TABLE: + return jsonToJoinTableNode(pJson, pObj); + case QUERY_NODE_GROUPING_SET: + return jsonToGroupingSetNode(pJson, pObj); + case QUERY_NODE_ORDER_BY_EXPR: + return jsonToOrderByExprNode(pJson, pObj); + case QUERY_NODE_LIMIT: + return jsonToLimitNode(pJson, pObj); + case QUERY_NODE_STATE_WINDOW: + return jsonToStateWindowNode(pJson, pObj); + case QUERY_NODE_SESSION_WINDOW: + return jsonToSessionWindowNode(pJson, pObj); + case QUERY_NODE_INTERVAL_WINDOW: + return jsonToIntervalWindowNode(pJson, pObj); + case QUERY_NODE_NODE_LIST: + return jsonToNodeListNode(pJson, pObj); + case QUERY_NODE_FILL: + return jsonToFillNode(pJson, pObj); + case QUERY_NODE_TARGET: + return jsonToTargetNode(pJson, pObj); + case QUERY_NODE_DATABLOCK_DESC: + return jsonToDataBlockDescNode(pJson, pObj); + case QUERY_NODE_SLOT_DESC: + return jsonToSlotDescNode(pJson, pObj); + case QUERY_NODE_COLUMN_DEF: + return jsonToColumnDefNode(pJson, pObj); + case QUERY_NODE_DOWNSTREAM_SOURCE: + return jsonToDownstreamSourceNode(pJson, pObj); + case QUERY_NODE_DATABASE_OPTIONS: + return jsonToDatabaseOptions(pJson, pObj); + case QUERY_NODE_TABLE_OPTIONS: + return jsonToTableOptions(pJson, pObj); + case QUERY_NODE_INDEX_OPTIONS: + return jsonToIndexOptions(pJson, pObj); + case QUERY_NODE_EXPLAIN_OPTIONS: + return jsonToExplainOptions(pJson, pObj); + case QUERY_NODE_STREAM_OPTIONS: + return jsonToStreamOptions(pJson, pObj); + case QUERY_NODE_LEFT_VALUE: + return TSDB_CODE_SUCCESS; // SLeftValueNode has no fields to deserialize. + case QUERY_NODE_WHEN_THEN: + return jsonToWhenThenNode(pJson, pObj); + case QUERY_NODE_CASE_WHEN: + return jsonToCaseWhenNode(pJson, pObj); + case QUERY_NODE_EVENT_WINDOW: + return jsonToEventWindowNode(pJson, pObj); + case QUERY_NODE_SET_OPERATOR: + return jsonToSetOperator(pJson, pObj); + case QUERY_NODE_SELECT_STMT: + return jsonToSelectStmt(pJson, pObj); + case QUERY_NODE_VNODE_MODIFY_STMT: + return jsonToVnodeModifyStmt(pJson, pObj); + case QUERY_NODE_CREATE_DATABASE_STMT: + return jsonToCreateDatabaseStmt(pJson, pObj); + case QUERY_NODE_ALTER_DATABASE_STMT: + return jsonToAlterDatabaseStmt(pJson, pObj); + case QUERY_NODE_TRIM_DATABASE_STMT: + return jsonToTrimDatabaseStmt(pJson, pObj); + case QUERY_NODE_CREATE_TABLE_STMT: + return jsonToCreateTableStmt(pJson, pObj); + case QUERY_NODE_CREATE_SUBTABLE_CLAUSE: + return jsonToCreateSubTableClause(pJson, pObj); + case QUERY_NODE_CREATE_MULTI_TABLES_STMT: + return jsonToCreateMultiTablesStmt(pJson, pObj); + case QUERY_NODE_DROP_TABLE_CLAUSE: + return jsonToDropTableClause(pJson, pObj); + case QUERY_NODE_DROP_TABLE_STMT: + return jsonToDropTableStmt(pJson, pObj); + case QUERY_NODE_DROP_SUPER_TABLE_STMT: + return jsonToDropStableStmt(pJson, pObj); + case QUERY_NODE_ALTER_TABLE_STMT: + return jsonToAlterTableStmt(pJson, pObj); + case QUERY_NODE_ALTER_SUPER_TABLE_STMT: + return jsonToAlterStableStmt(pJson, pObj); + case QUERY_NODE_CREATE_USER_STMT: + return jsonToCreateUserStmt(pJson, pObj); + case QUERY_NODE_ALTER_USER_STMT: + return jsonToAlterUserStmt(pJson, pObj); + case QUERY_NODE_DROP_USER_STMT: + return jsonToDropUserStmt(pJson, pObj); + case QUERY_NODE_USE_DATABASE_STMT: + return jsonToUseDatabaseStmt(pJson, pObj); + case QUERY_NODE_CREATE_DNODE_STMT: + return jsonToCreateDnodeStmt(pJson, pObj); + case QUERY_NODE_DROP_DNODE_STMT: + return jsonToDropDnodeStmt(pJson, pObj); + case QUERY_NODE_ALTER_DNODE_STMT: + return jsonToAlterDnodeStmt(pJson, pObj); + case QUERY_NODE_CREATE_INDEX_STMT: + return jsonToCreateIndexStmt(pJson, pObj); + case QUERY_NODE_DROP_INDEX_STMT: + return jsonToDropIndexStmt(pJson, pObj); + case QUERY_NODE_CREATE_QNODE_STMT: + return jsonToCreateQnodeStmt(pJson, pObj); + case QUERY_NODE_DROP_QNODE_STMT: + return jsonToDropQnodeStmt(pJson, pObj); + case QUERY_NODE_CREATE_SNODE_STMT: + return jsonToCreateSnodeStmt(pJson, pObj); + case QUERY_NODE_DROP_SNODE_STMT: + return jsonToDropSnodeStmt(pJson, pObj); + case QUERY_NODE_CREATE_MNODE_STMT: + return jsonToCreateMnodeStmt(pJson, pObj); + case QUERY_NODE_DROP_MNODE_STMT: + return jsonToDropMnodeStmt(pJson, pObj); + case QUERY_NODE_CREATE_TOPIC_STMT: + return jsonToCreateTopicStmt(pJson, pObj); + case QUERY_NODE_DROP_TOPIC_STMT: + return jsonToDropTopicStmt(pJson, pObj); + case QUERY_NODE_DROP_CGROUP_STMT: + return jsonToDropConsumerGroupStmt(pJson, pObj); + case QUERY_NODE_ALTER_LOCAL_STMT: + return jsonToAlterLocalStmt(pJson, pObj); + case QUERY_NODE_EXPLAIN_STMT: + return jsonToExplainStmt(pJson, pObj); + case QUERY_NODE_DESCRIBE_STMT: + return jsonToDescribeStmt(pJson, pObj); + case QUERY_NODE_COMPACT_DATABASE_STMT: + return jsonToCompactDatabaseStmt(pJson, pObj); + case QUERY_NODE_CREATE_STREAM_STMT: + return jsonToCreateStreamStmt(pJson, pObj); + case QUERY_NODE_DROP_STREAM_STMT: + return jsonToDropStreamStmt(pJson, pObj); + case QUERY_NODE_BALANCE_VGROUP_STMT: + return TSDB_CODE_SUCCESS; // SBalanceVgroupStmt has no fields to deserialize. + case QUERY_NODE_BALANCE_VGROUP_LEADER_STMT: + return TSDB_CODE_SUCCESS; // SBalanceVgroupLeaderStmt has no fields to deserialize. + case QUERY_NODE_MERGE_VGROUP_STMT: + return jsonToMergeVgroupStmt(pJson, pObj); + case QUERY_NODE_REDISTRIBUTE_VGROUP_STMT: + return jsonToRedistributeVgroupStmt(pJson, pObj); + case QUERY_NODE_SPLIT_VGROUP_STMT: + return jsonToSplitVgroupStmt(pJson, pObj); + case QUERY_NODE_GRANT_STMT: + return jsonToGrantStmt(pJson, pObj); + case QUERY_NODE_REVOKE_STMT: + return jsonToRevokeStmt(pJson, pObj); + case QUERY_NODE_SHOW_DNODES_STMT: + return jsonToShowDnodesStmt(pJson, pObj); + case QUERY_NODE_SHOW_MNODES_STMT: + return jsonToShowMnodesStmt(pJson, pObj); + case QUERY_NODE_SHOW_QNODES_STMT: + return jsonToShowQnodesStmt(pJson, pObj); + case QUERY_NODE_SHOW_CLUSTER_STMT: + return jsonToShowClusterStmt(pJson, pObj); + case QUERY_NODE_SHOW_DATABASES_STMT: + return jsonToShowDatabasesStmt(pJson, pObj); + case QUERY_NODE_SHOW_FUNCTIONS_STMT: + return jsonToShowFunctionsStmt(pJson, pObj); + case QUERY_NODE_SHOW_INDEXES_STMT: + return jsonToShowIndexesStmt(pJson, pObj); + case QUERY_NODE_SHOW_STABLES_STMT: + return jsonToShowStablesStmt(pJson, pObj); + case QUERY_NODE_SHOW_STREAMS_STMT: + return jsonToShowStreamsStmt(pJson, pObj); + case QUERY_NODE_SHOW_TABLES_STMT: + return jsonToShowTablesStmt(pJson, pObj); + case QUERY_NODE_SHOW_TAGS_STMT: + return jsonToShowTagsStmt(pJson, pObj); + case QUERY_NODE_SHOW_USERS_STMT: + return jsonToShowUsersStmt(pJson, pObj); + case QUERY_NODE_SHOW_VGROUPS_STMT: + return jsonToShowVgroupsStmt(pJson, pObj); + case QUERY_NODE_SHOW_CONSUMERS_STMT: + return jsonToShowConsumersStmt(pJson, pObj); + case QUERY_NODE_SHOW_VARIABLES_STMT: + return jsonToShowVariablesStmt(pJson, pObj); + case QUERY_NODE_SHOW_DNODE_VARIABLES_STMT: + return jsonToShowDnodeVariablesStmt(pJson, pObj); + case QUERY_NODE_SHOW_TRANSACTIONS_STMT: + return jsonToShowTransactionsStmt(pJson, pObj); + case QUERY_NODE_SHOW_SUBSCRIPTIONS_STMT: + return jsonToShowSubscriptionsStmt(pJson, pObj); + case QUERY_NODE_SHOW_VNODES_STMT: + return jsonToShowVnodesStmt(pJson, pObj); + case QUERY_NODE_SHOW_USER_PRIVILEGES_STMT: + return jsonToShowUserPrivilegesStmt(pJson, pObj); + case QUERY_NODE_SHOW_CREATE_DATABASE_STMT: + return jsonToShowCreateDatabaseStmt(pJson, pObj); + case QUERY_NODE_SHOW_CREATE_TABLE_STMT: + return jsonToShowCreateTableStmt(pJson, pObj); + case QUERY_NODE_SHOW_CREATE_STABLE_STMT: + return jsonToShowCreateStableStmt(pJson, pObj); + case QUERY_NODE_SHOW_CREATE_VIEW_STMT: + return jsonToShowCreateViewStmt(pJson, pObj); + case QUERY_NODE_SHOW_TABLE_DISTRIBUTED_STMT: + return jsonToShowTableDistributedStmt(pJson, pObj); + case QUERY_NODE_SHOW_LOCAL_VARIABLES_STMT: + return jsonToShowLocalVariablesStmt(pJson, pObj); + case QUERY_NODE_SHOW_TABLE_TAGS_STMT: + return jsonToShowTableTagsStmt(pJson, pObj); + case QUERY_NODE_DELETE_STMT: + return jsonToDeleteStmt(pJson, pObj); + case QUERY_NODE_INSERT_STMT: + return jsonToInsertStmt(pJson, pObj); + case QUERY_NODE_RESTORE_DNODE_STMT: + return jsonToRestoreDnodeStmt(pJson, pObj); + case QUERY_NODE_RESTORE_QNODE_STMT: + return jsonToRestoreQnodeStmt(pJson, pObj); + case QUERY_NODE_RESTORE_MNODE_STMT: + return jsonToRestoreMnodeStmt(pJson, pObj); + case QUERY_NODE_RESTORE_VNODE_STMT: + return jsonToRestoreVnodeStmt(pJson, pObj); + case QUERY_NODE_LOGIC_PLAN_SCAN: + return jsonToLogicScanNode(pJson, pObj); + case QUERY_NODE_LOGIC_PLAN_JOIN: + return jsonToLogicJoinNode(pJson, pObj); + case QUERY_NODE_LOGIC_PLAN_AGG: + return jsonToLogicAggNode(pJson, pObj); + case QUERY_NODE_LOGIC_PLAN_PROJECT: + return jsonToLogicProjectNode(pJson, pObj); + case QUERY_NODE_LOGIC_PLAN_VNODE_MODIFY: + return jsonToLogicVnodeModifyNode(pJson, pObj); + case QUERY_NODE_LOGIC_PLAN_EXCHANGE: + return jsonToLogicExchangeNode(pJson, pObj); + case QUERY_NODE_LOGIC_PLAN_MERGE: + return jsonToLogicMergeNode(pJson, pObj); + case QUERY_NODE_LOGIC_PLAN_WINDOW: + return jsonToLogicWindowNode(pJson, pObj); + case QUERY_NODE_LOGIC_PLAN_FILL: + return jsonToLogicFillNode(pJson, pObj); + case QUERY_NODE_LOGIC_PLAN_SORT: + return jsonToLogicSortNode(pJson, pObj); + case QUERY_NODE_LOGIC_PLAN_PARTITION: + return jsonToLogicPartitionNode(pJson, pObj); + case QUERY_NODE_LOGIC_PLAN_INDEF_ROWS_FUNC: + return jsonToLogicIndefRowsFuncNode(pJson, pObj); + case QUERY_NODE_LOGIC_PLAN_INTERP_FUNC: + return jsonToLogicInterpFuncNode(pJson, pObj); + case QUERY_NODE_LOGIC_PLAN_GROUP_CACHE: + return jsonToLogicGroupCacheNode(pJson, pObj); + case QUERY_NODE_LOGIC_PLAN_DYN_QUERY_CTRL: + return jsonToLogicDynQueryCtrlNode(pJson, pObj); + case QUERY_NODE_LOGIC_SUBPLAN: + return jsonToLogicSubplan(pJson, pObj); + case QUERY_NODE_LOGIC_PLAN: + return jsonToLogicPlan(pJson, pObj); + case QUERY_NODE_PHYSICAL_PLAN_TAG_SCAN: + return jsonToPhysiTagScanNode(pJson, pObj); + case QUERY_NODE_PHYSICAL_PLAN_BLOCK_DIST_SCAN: + case QUERY_NODE_PHYSICAL_PLAN_TABLE_COUNT_SCAN: + return jsonToPhysiScanNode(pJson, pObj); + case QUERY_NODE_PHYSICAL_PLAN_LAST_ROW_SCAN: + return jsonToPhysiLastRowScanNode(pJson, pObj); + case QUERY_NODE_PHYSICAL_PLAN_TABLE_SCAN: + case QUERY_NODE_PHYSICAL_PLAN_TABLE_MERGE_SCAN: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN: + return jsonToPhysiTableScanNode(pJson, pObj); + case QUERY_NODE_PHYSICAL_PLAN_SYSTABLE_SCAN: + return jsonToPhysiSysTableScanNode(pJson, pObj); + case QUERY_NODE_PHYSICAL_PLAN_PROJECT: + return jsonToPhysiProjectNode(pJson, pObj); + case QUERY_NODE_PHYSICAL_PLAN_MERGE_JOIN: + return jsonToPhysiMergeJoinNode(pJson, pObj); + case QUERY_NODE_PHYSICAL_PLAN_HASH_JOIN: + return jsonToPhysiHashJoinNode(pJson, pObj); + case QUERY_NODE_PHYSICAL_PLAN_HASH_AGG: + return jsonToPhysiAggNode(pJson, pObj); + case QUERY_NODE_PHYSICAL_PLAN_EXCHANGE: + return jsonToPhysiExchangeNode(pJson, pObj); + case QUERY_NODE_PHYSICAL_PLAN_MERGE: + return jsonToPhysiMergeNode(pJson, pObj); + case QUERY_NODE_PHYSICAL_PLAN_SORT: + case QUERY_NODE_PHYSICAL_PLAN_GROUP_SORT: + return jsonToPhysiSortNode(pJson, pObj); + case QUERY_NODE_PHYSICAL_PLAN_HASH_INTERVAL: + case QUERY_NODE_PHYSICAL_PLAN_MERGE_ALIGNED_INTERVAL: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL: + return jsonToPhysiIntervalNode(pJson, pObj); + case QUERY_NODE_PHYSICAL_PLAN_FILL: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_FILL: + return jsonToPhysiFillNode(pJson, pObj); + case QUERY_NODE_PHYSICAL_PLAN_MERGE_SESSION: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_SESSION: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_SESSION: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_SESSION: + return jsonToPhysiSessionWindowNode(pJson, pObj); + case QUERY_NODE_PHYSICAL_PLAN_MERGE_STATE: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_STATE: + return jsonToPhysiStateWindowNode(pJson, pObj); + case QUERY_NODE_PHYSICAL_PLAN_MERGE_EVENT: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_EVENT: + return jsonToPhysiEventWindowNode(pJson, pObj); + case QUERY_NODE_PHYSICAL_PLAN_PARTITION: + return jsonToPhysiPartitionNode(pJson, pObj); + case QUERY_NODE_PHYSICAL_PLAN_STREAM_PARTITION: + return jsonToPhysiStreamPartitionNode(pJson, pObj); + case QUERY_NODE_PHYSICAL_PLAN_INDEF_ROWS_FUNC: + return jsonToPhysiIndefRowsFuncNode(pJson, pObj); + case QUERY_NODE_PHYSICAL_PLAN_INTERP_FUNC: + return jsonToPhysiInterpFuncNode(pJson, pObj); + case QUERY_NODE_PHYSICAL_PLAN_DISPATCH: + return jsonToPhysiDispatchNode(pJson, pObj); + case QUERY_NODE_PHYSICAL_PLAN_QUERY_INSERT: + return jsonToPhysiQueryInsertNode(pJson, pObj); + case QUERY_NODE_PHYSICAL_PLAN_DELETE: + return jsonToPhysiDeleteNode(pJson, pObj); + case QUERY_NODE_PHYSICAL_PLAN_GROUP_CACHE: + return jsonToPhysiGroupCacheNode(pJson, pObj); + case QUERY_NODE_PHYSICAL_PLAN_DYN_QUERY_CTRL: + return jsonToPhysiDynQueryCtrlNode(pJson, pObj); + case QUERY_NODE_PHYSICAL_SUBPLAN: + return jsonToSubplan(pJson, pObj); + case QUERY_NODE_PHYSICAL_PLAN: + return jsonToPlan(pJson, pObj); + default: + break; } - - if (funcNodes[type].toNodeFunc) { - return funcNodes[type].toNodeFunc(pJson, pObj); - } - - nodesWarn("jsonToSpecificNode unknown node type = %d", type); + nodesWarn("jsonToSpecificNode unknown node = %s", nodesNodeName(nodeType(pObj))); return TSDB_CODE_SUCCESS; } @@ -6670,2180 +7611,3 @@ int32_t nodesStringToList(const char* pStr, SNodeList** pList) { } return TSDB_CODE_SUCCESS; } - -static int32_t emptyNodeToJson(const void* pObj, SJson* pJson) { - return TSDB_CODE_SUCCESS; -} - -static int32_t emptyJsonToNode(const SJson* pJson, void* pObj) { - return TSDB_CODE_SUCCESS; -} - -static void destroyVgDataBlockArray(SArray* pArray) { - size_t size = taosArrayGetSize(pArray); - for (size_t i = 0; i < size; ++i) { - SVgDataBlocks* pVg = taosArrayGetP(pArray, i); - taosMemoryFreeClear(pVg->pData); - taosMemoryFreeClear(pVg); - } - taosArrayDestroy(pArray); -} - -static void destroyLogicNode(SLogicNode* pNode) { - nodesDestroyList(pNode->pTargets); - nodesDestroyNode(pNode->pConditions); - nodesDestroyList(pNode->pChildren); - nodesDestroyNode(pNode->pLimit); - nodesDestroyNode(pNode->pSlimit); - nodesDestroyList(pNode->pHint); -} - -void destroyPhysiNode(SNode* pInput) { - SPhysiNode* pNode = (SPhysiNode*)pInput; - nodesDestroyList(pNode->pChildren); - nodesDestroyNode(pNode->pConditions); - nodesDestroyNode((SNode*)pNode->pOutputDataBlockDesc); - nodesDestroyNode(pNode->pLimit); - nodesDestroyNode(pNode->pSlimit); -} - -void destroyExprNode(SNode* pNode) { - SExprNode* pExpr = (SExprNode*)pNode; - taosArrayDestroy(pExpr->pAssociation); -} - -void destroyDataInSmaIndex(void* pIndex) { - taosMemoryFree(((STableIndexInfo*)pIndex)->expr); -} - -void destoryXNode(SNode* pNode) {} - -void destroyColumnNode(SNode* pNode) { - destroyExprNode(pNode); -} - -void destroyValueNode(SNode* pNode) { - SValueNode* pValue = (SValueNode*)pNode; - destroyExprNode(pNode); - taosMemoryFreeClear(pValue->literal); - if (IS_VAR_DATA_TYPE(pValue->node.resType.type)) { - taosMemoryFreeClear(pValue->datum.p); - } -} - -void destroyOperatorNode(SNode* pNode) { - SOperatorNode* pOp = (SOperatorNode*)pNode; - destroyExprNode(pNode); - nodesDestroyNode(pOp->pLeft); - nodesDestroyNode(pOp->pRight); -} - -void destoryLogicConditionNode(SNode* pNode) { - destroyExprNode(pNode); - nodesDestroyList(((SLogicConditionNode*)pNode)->pParameterList); -} - -void destoryFunctionNode(SNode* pNode) { - destroyExprNode(pNode); - nodesDestroyList(((SFunctionNode*)pNode)->pParameterList); -} - -void destoryRealTableNode(SNode* pNode) { - SRealTableNode* pReal = (SRealTableNode*)pNode; - taosMemoryFreeClear(pReal->pMeta); - taosMemoryFreeClear(pReal->pVgroupList); - taosArrayDestroyEx(pReal->pSmaIndexes, destroyDataInSmaIndex); -} - -void destoryTempTableNode(SNode* pNode) { nodesDestroyNode(((STempTableNode*)pNode)->pSubquery); } - -void destoryJoinTableNode(SNode* pNode) { - SJoinTableNode* pJoin = (SJoinTableNode*)pNode; - nodesDestroyNode(pJoin->pLeft); - nodesDestroyNode(pJoin->pRight); - nodesDestroyNode(pJoin->pOnCond); -} - -void destoryGroupingSetNode(SNode* pNode) { nodesDestroyList(((SGroupingSetNode*)pNode)->pParameterList); } - -void destoryOrderByExprNode(SNode* pNode) { nodesDestroyNode(((SOrderByExprNode*)pNode)->pExpr); } - -void destoryStateWindowNode(SNode* pNode) { - SStateWindowNode* pState = (SStateWindowNode*)pNode; - nodesDestroyNode(pState->pCol); - nodesDestroyNode(pState->pExpr); -} - -void destorySessionWindowNode(SNode* pNode) { - SSessionWindowNode* pSession = (SSessionWindowNode*)pNode; - nodesDestroyNode((SNode*)pSession->pCol); - nodesDestroyNode((SNode*)pSession->pGap); -} - -void destoryIntervalWindowNode(SNode* pNode) { - SIntervalWindowNode* pJoin = (SIntervalWindowNode*)pNode; - nodesDestroyNode(pJoin->pCol); - nodesDestroyNode(pJoin->pInterval); - nodesDestroyNode(pJoin->pOffset); - nodesDestroyNode(pJoin->pSliding); - nodesDestroyNode(pJoin->pFill); -} - -void destoryNodeListNode(SNode* pNode) { nodesDestroyList(((SNodeListNode*)pNode)->pNodeList); } - -void destoryFillNode(SNode* pNode) { - SFillNode* pFill = (SFillNode*)pNode; - nodesDestroyNode(pFill->pValues); - nodesDestroyNode(pFill->pWStartTs); -} - -void destoryRawExprNode(SNode* pNode) { nodesDestroyNode(((SRawExprNode*)pNode)->pNode); } - -void destoryTargetNode(SNode* pNode) { nodesDestroyNode(((STargetNode*)pNode)->pExpr); } - -void destoryDataBlockDescNode(SNode* pNode) { nodesDestroyList(((SDataBlockDescNode*)pNode)->pSlots); } - -void destoryDatabaseOptions(SNode* pNode) { - SDatabaseOptions* pOptions = (SDatabaseOptions*)pNode; - nodesDestroyNode((SNode*)pOptions->pDaysPerFile); - nodesDestroyList(pOptions->pKeep); - nodesDestroyList(pOptions->pRetentions); -} - -void destoryTableOptions(SNode* pNode) { - STableOptions* pOptions = (STableOptions*)pNode; - nodesDestroyList(pOptions->pMaxDelay); - nodesDestroyList(pOptions->pWatermark); - nodesDestroyList(pOptions->pRollupFuncs); - nodesDestroyList(pOptions->pSma); - nodesDestroyList(pOptions->pDeleteMark); -} - -void destoryIndexOptions(SNode* pNode) { - SIndexOptions* pOptions = (SIndexOptions*)pNode; - nodesDestroyList(pOptions->pFuncs); - nodesDestroyNode(pOptions->pInterval); - nodesDestroyNode(pOptions->pOffset); - nodesDestroyNode(pOptions->pSliding); - nodesDestroyNode(pOptions->pStreamOptions); -} - -void destoryStreamOptions(SNode* pNode) { - SStreamOptions* pOptions = (SStreamOptions*)pNode; - nodesDestroyNode(pOptions->pDelay); - nodesDestroyNode(pOptions->pWatermark); - nodesDestroyNode(pOptions->pDeleteMark); -} - -void destoryWhenThenNode(SNode* pNode) { - SWhenThenNode* pWhenThen = (SWhenThenNode*)pNode; - nodesDestroyNode(pWhenThen->pWhen); - nodesDestroyNode(pWhenThen->pThen); -} - -void destoryCaseWhenNode(SNode* pNode) { - SCaseWhenNode* pCaseWhen = (SCaseWhenNode*)pNode; - nodesDestroyNode(pCaseWhen->pCase); - nodesDestroyNode(pCaseWhen->pElse); - nodesDestroyList(pCaseWhen->pWhenThenList); -} - -void destoryEventWindowNode(SNode* pNode) { - SEventWindowNode* pEvent = (SEventWindowNode*)pNode; - nodesDestroyNode(pEvent->pCol); - nodesDestroyNode(pEvent->pStartCond); - nodesDestroyNode(pEvent->pEndCond); -} - -void destoryHintNode(SNode* pNode) { - SHintNode* pHint = (SHintNode*)pNode; - taosMemoryFree(pHint->value); -} - -void destoryViewNode(SNode* pNode) { - SViewNode* pView = (SViewNode*)pNode; - taosMemoryFreeClear(pView->pMeta); - taosMemoryFreeClear(pView->pVgroupList); - taosArrayDestroyEx(pView->pSmaIndexes, destroyDataInSmaIndex); -} - -void destorySetOperator(SNode* pNode) { - SSetOperator* pStmt = (SSetOperator*)pNode; - nodesDestroyList(pStmt->pProjectionList); - nodesDestroyNode(pStmt->pLeft); - nodesDestroyNode(pStmt->pRight); - nodesDestroyList(pStmt->pOrderByList); - nodesDestroyNode(pStmt->pLimit); -} - -void destorySelectStmt(SNode* pNode) { - SSelectStmt* pStmt = (SSelectStmt*)pNode; - nodesDestroyList(pStmt->pProjectionList); - nodesDestroyNode(pStmt->pFromTable); - nodesDestroyNode(pStmt->pWhere); - nodesDestroyList(pStmt->pPartitionByList); - nodesDestroyList(pStmt->pTags); - nodesDestroyNode(pStmt->pSubtable); - nodesDestroyNode(pStmt->pWindow); - nodesDestroyList(pStmt->pGroupByList); - nodesDestroyNode(pStmt->pHaving); - nodesDestroyNode(pStmt->pRange); - nodesDestroyNode(pStmt->pEvery); - nodesDestroyNode(pStmt->pFill); - nodesDestroyList(pStmt->pOrderByList); - nodesDestroyNode((SNode*)pStmt->pLimit); - nodesDestroyNode((SNode*)pStmt->pSlimit); - nodesDestroyList(pStmt->pHint); -} - -void destoryVnodeModifyOpStmt(SNode* pNode) { - SVnodeModifyOpStmt* pStmt = (SVnodeModifyOpStmt*)pNode; - destroyVgDataBlockArray(pStmt->pDataBlocks); - taosMemoryFreeClear(pStmt->pTableMeta); - nodesDestroyNode(pStmt->pTagCond); - taosArrayDestroy(pStmt->pTableTag); - taosHashCleanup(pStmt->pVgroupsHashObj); - taosHashCleanup(pStmt->pSubTableHashObj); - taosHashCleanup(pStmt->pTableNameHashObj); - taosHashCleanup(pStmt->pDbFNameHashObj); - taosHashCleanup(pStmt->pTableCxtHashObj); - if (pStmt->freeHashFunc) { - pStmt->freeHashFunc(pStmt->pTableBlockHashObj); - } - if (pStmt->freeArrayFunc) { - pStmt->freeArrayFunc(pStmt->pVgDataBlocks); - } - tdDestroySVCreateTbReq(pStmt->pCreateTblReq); - taosMemoryFreeClear(pStmt->pCreateTblReq); - if (pStmt->freeStbRowsCxtFunc) { - pStmt->freeStbRowsCxtFunc(pStmt->pStbRowsCxt); - } - taosMemoryFreeClear(pStmt->pStbRowsCxt); - taosCloseFile(&pStmt->fp); -} - -void destoryCreateDatabaseStmt(SNode* pNode) { nodesDestroyNode((SNode*)((SCreateDatabaseStmt*)pNode)->pOptions); } - -void destoryAlterDatabaseStmt(SNode* pNode) { nodesDestroyNode((SNode*)((SAlterDatabaseStmt*)pNode)->pOptions); } - -void destoryCreateTableStmt(SNode* pNode) { - SCreateTableStmt* pStmt = (SCreateTableStmt*)pNode; - nodesDestroyList(pStmt->pCols); - nodesDestroyList(pStmt->pTags); - nodesDestroyNode((SNode*)pStmt->pOptions); -} - -void destoryCreateSubTableClause(SNode* pNode) { - SCreateSubTableClause* pStmt = (SCreateSubTableClause*)pNode; - nodesDestroyList(pStmt->pSpecificTags); - nodesDestroyList(pStmt->pValsOfTags); - nodesDestroyNode((SNode*)pStmt->pOptions); -} - -void destoryCreateMultiTablesStmt(SNode* pNode) { - nodesDestroyList(((SCreateMultiTablesStmt*)pNode)->pSubTables); -} - -void destoryDropTableStmt(SNode* pNode) { - nodesDestroyList(((SDropTableStmt*)pNode)->pTables); -} - -void destoryAlterTableStmt(SNode* pNode) { - SAlterTableStmt* pStmt = (SAlterTableStmt*)pNode; - nodesDestroyNode((SNode*)pStmt->pOptions); - nodesDestroyNode((SNode*)pStmt->pVal); -} - -void destoryCreateUserStmt(SNode* pNode) { - SCreateUserStmt* pStmt = (SCreateUserStmt*)pNode; - taosMemoryFree(pStmt->pIpRanges); - nodesDestroyList(pStmt->pNodeListIpRanges); -} - -void destoryAlterUserStmt(SNode* pNode) { - SAlterUserStmt* pStmt = (SAlterUserStmt*)pNode; - taosMemoryFree(pStmt->pIpRanges); - nodesDestroyList(pStmt->pNodeListIpRanges); -} - -void destoryCreateIndexStmt(SNode* pNode) { - SCreateIndexStmt* pStmt = (SCreateIndexStmt*)pNode; - nodesDestroyNode((SNode*)pStmt->pOptions); - nodesDestroyList(pStmt->pCols); - if (pStmt->pReq) { - tFreeSMCreateSmaReq(pStmt->pReq); - taosMemoryFreeClear(pStmt->pReq); - } -} - -void destoryCreateTopicStmt(SNode* pNode) { - nodesDestroyNode(((SCreateTopicStmt*)pNode)->pQuery); - nodesDestroyNode(((SCreateTopicStmt*)pNode)->pWhere); -} - -void destoryExplainStmt(SNode* pNode) { - SExplainStmt* pStmt = (SExplainStmt*)pNode; - nodesDestroyNode((SNode*)pStmt->pOptions); - nodesDestroyNode(pStmt->pQuery); -} - -void destoryDescribeStmt(SNode* pNode) { - taosMemoryFree(((SDescribeStmt*)pNode)->pMeta); -} - -void destoryCompactDatabaseStmt(SNode* pNode) { - SCompactDatabaseStmt* pStmt = (SCompactDatabaseStmt*)pNode; - nodesDestroyNode(pStmt->pStart); - nodesDestroyNode(pStmt->pEnd); -} - -void destoryCreateStreamStmt(SNode* pNode) { - SCreateStreamStmt* pStmt = (SCreateStreamStmt*)pNode; - nodesDestroyNode((SNode*)pStmt->pOptions); - nodesDestroyNode(pStmt->pQuery); - nodesDestroyList(pStmt->pTags); - nodesDestroyNode(pStmt->pSubtable); - tFreeSCMCreateStreamReq(pStmt->pReq); - taosMemoryFreeClear(pStmt->pReq); -} - -void destoryRedistributeVgroupStmt(SNode* pNode) { - nodesDestroyList(((SRedistributeVgroupStmt*)pNode)->pDnodes); -} - -void destoryGrantStmt(SNode* pNode) { - nodesDestroyNode(((SGrantStmt*)pNode)->pTagCond); -} - -void destoryRevokeStmt(SNode* pNode) { - nodesDestroyNode(((SRevokeStmt*)pNode)->pTagCond); -} - -void destoryShowStmt(SNode* pNode) { - SShowStmt* pStmt = (SShowStmt*)pNode; - nodesDestroyNode(pStmt->pDbName); - nodesDestroyNode(pStmt->pTbName); -} - -void destoryShowTableTagsStmt(SNode* pNode) { - SShowTableTagsStmt* pStmt = (SShowTableTagsStmt*)pNode; - nodesDestroyNode(pStmt->pDbName); - nodesDestroyNode(pStmt->pTbName); - nodesDestroyList(pStmt->pTags); -} - -void destoryShowDnodeVariablesStmt(SNode* pNode) { - nodesDestroyNode(((SShowDnodeVariablesStmt*)pNode)->pDnodeId); - nodesDestroyNode(((SShowDnodeVariablesStmt*)pNode)->pLikePattern); -} - -void destoryShowCreateDatabaseStmt(SNode* pNode) { - taosMemoryFreeClear(((SShowCreateDatabaseStmt*)pNode)->pCfg); -} - -void destoryShowCreateTableStmt(SNode* pNode) { - STableCfg* pCfg = (STableCfg*)(((SShowCreateTableStmt*)pNode)->pTableCfg); - taosMemoryFreeClear(((SShowCreateTableStmt*)pNode)->pDbCfg); - if (NULL == pCfg) { - return; - } - taosArrayDestroy(pCfg->pFuncs); - taosMemoryFree(pCfg->pComment); - taosMemoryFree(pCfg->pSchemas); - taosMemoryFree(pCfg->pTags); - taosMemoryFree(pCfg); -} - -void destoryDeleteStmt(SNode* pNode) { - SDeleteStmt* pStmt = (SDeleteStmt*)pNode; - nodesDestroyNode(pStmt->pFromTable); - nodesDestroyNode(pStmt->pWhere); - nodesDestroyNode(pStmt->pCountFunc); - nodesDestroyNode(pStmt->pFirstFunc); - nodesDestroyNode(pStmt->pLastFunc); - nodesDestroyNode(pStmt->pTagCond); -} - -void destoryInsertStmt(SNode* pNode) { - SInsertStmt* pStmt = (SInsertStmt*)pNode; - nodesDestroyNode(pStmt->pTable); - nodesDestroyList(pStmt->pCols); - nodesDestroyNode(pStmt->pQuery); -} - -void destoryQueryNode(SNode* pNode) { - SQuery* pQuery = (SQuery*)pNode; - nodesDestroyNode(pQuery->pPrevRoot); - nodesDestroyNode(pQuery->pRoot); - nodesDestroyNode(pQuery->pPostRoot); - taosMemoryFreeClear(pQuery->pResSchema); - if (NULL != pQuery->pCmdMsg) { - taosMemoryFreeClear(pQuery->pCmdMsg->pMsg); - taosMemoryFreeClear(pQuery->pCmdMsg); - } - taosArrayDestroy(pQuery->pDbList); - taosArrayDestroy(pQuery->pTableList); - taosArrayDestroy(pQuery->pTargetTableList); - taosArrayDestroy(pQuery->pPlaceholderValues); - nodesDestroyNode(pQuery->pPrepareRoot); -} - -void destoryCreateViewStmt(SNode* pNode) { - SCreateViewStmt* pStmt = (SCreateViewStmt*)pNode; - taosMemoryFree(pStmt->pQuerySql); - tFreeSCMCreateViewReq(&pStmt->createReq); - nodesDestroyNode(pStmt->pQuery); -} - -void destoryScanLogicNode(SNode* pNode) { - SScanLogicNode* pLogicNode = (SScanLogicNode*)pNode; - destroyLogicNode((SLogicNode*)pLogicNode); - nodesDestroyList(pLogicNode->pScanCols); - nodesDestroyList(pLogicNode->pScanPseudoCols); - taosMemoryFreeClear(pLogicNode->pVgroupList); - nodesDestroyList(pLogicNode->pDynamicScanFuncs); - nodesDestroyNode(pLogicNode->pTagCond); - nodesDestroyNode(pLogicNode->pTagIndexCond); - taosArrayDestroyEx(pLogicNode->pSmaIndexes, destroyDataInSmaIndex); - nodesDestroyList(pLogicNode->pGroupTags); - nodesDestroyList(pLogicNode->pTags); - nodesDestroyNode(pLogicNode->pSubtable); -} - -void destoryJoinLogicNode(SNode* pNode) { - SJoinLogicNode* pLogicNode = (SJoinLogicNode*)pNode; - destroyLogicNode((SLogicNode*)pLogicNode); - nodesDestroyNode(pLogicNode->pPrimKeyEqCond); - nodesDestroyNode(pLogicNode->pOtherOnCond); - nodesDestroyNode(pLogicNode->pColEqCond); -} - -void destoryAggLogicNode(SNode* pNode) { - SAggLogicNode* pLogicNode = (SAggLogicNode*)pNode; - destroyLogicNode((SLogicNode*)pLogicNode); - nodesDestroyList(pLogicNode->pAggFuncs); - nodesDestroyList(pLogicNode->pGroupKeys); -} - -void destoryProjectLogicNode(SNode* pNode) { - SProjectLogicNode* pLogicNode = (SProjectLogicNode*)pNode; - destroyLogicNode((SLogicNode*)pLogicNode); - nodesDestroyList(pLogicNode->pProjections); -} - -void destoryVnodeModifyLogicNode(SNode* pNode) { - SVnodeModifyLogicNode* pLogicNode = (SVnodeModifyLogicNode*)pNode; - destroyLogicNode((SLogicNode*)pLogicNode); - destroyVgDataBlockArray(pLogicNode->pDataBlocks); - // pVgDataBlocks is weak reference - nodesDestroyNode(pLogicNode->pAffectedRows); - nodesDestroyNode(pLogicNode->pStartTs); - nodesDestroyNode(pLogicNode->pEndTs); - taosMemoryFreeClear(pLogicNode->pVgroupList); - nodesDestroyList(pLogicNode->pInsertCols); -} - -void destoryExchangeLogicNode(SNode* pNode) { - destroyLogicNode((SLogicNode*)pNode); -} - -void destoryMergeLogicNode(SNode* pNode) { - SMergeLogicNode* pLogicNode = (SMergeLogicNode*)pNode; - destroyLogicNode((SLogicNode*)pLogicNode); - nodesDestroyList(pLogicNode->pMergeKeys); - nodesDestroyList(pLogicNode->pInputs); -} - -void destoryWindowLogicNode(SNode* pNode) { - SWindowLogicNode* pLogicNode = (SWindowLogicNode*)pNode; - destroyLogicNode((SLogicNode*)pLogicNode); - nodesDestroyList(pLogicNode->pFuncs); - nodesDestroyNode(pLogicNode->pTspk); - nodesDestroyNode(pLogicNode->pTsEnd); - nodesDestroyNode(pLogicNode->pStateExpr); - nodesDestroyNode(pLogicNode->pStartCond); - nodesDestroyNode(pLogicNode->pEndCond); -} - -void destoryFillLogicNode(SNode* pNode) { - SFillLogicNode* pLogicNode = (SFillLogicNode*)pNode; - destroyLogicNode((SLogicNode*)pLogicNode); - nodesDestroyNode(pLogicNode->pWStartTs); - nodesDestroyNode(pLogicNode->pValues); - nodesDestroyList(pLogicNode->pFillExprs); - nodesDestroyList(pLogicNode->pNotFillExprs); -} - -void destorySortLogicNode(SNode* pNode) { - SSortLogicNode* pLogicNode = (SSortLogicNode*)pNode; - destroyLogicNode((SLogicNode*)pLogicNode); - nodesDestroyList(pLogicNode->pSortKeys); -} - -void destoryPartitionLogicNode(SNode* pNode) { - SPartitionLogicNode* pLogicNode = (SPartitionLogicNode*)pNode; - destroyLogicNode((SLogicNode*)pLogicNode); - nodesDestroyList(pLogicNode->pPartitionKeys); - nodesDestroyList(pLogicNode->pTags); - nodesDestroyNode(pLogicNode->pSubtable); - nodesDestroyList(pLogicNode->pAggFuncs); -} - -void destoryIndefRowsFuncLogicNode(SNode* pNode) { - SIndefRowsFuncLogicNode* pLogicNode = (SIndefRowsFuncLogicNode*)pNode; - destroyLogicNode((SLogicNode*)pLogicNode); - nodesDestroyList(pLogicNode->pFuncs); -} - -void destoryInterpFuncLogicNode(SNode* pNode) { - SInterpFuncLogicNode* pLogicNode = (SInterpFuncLogicNode*)pNode; - destroyLogicNode((SLogicNode*)pLogicNode); - nodesDestroyList(pLogicNode->pFuncs); - nodesDestroyNode(pLogicNode->pFillValues); - nodesDestroyNode(pLogicNode->pTimeSeries); -} - -void destoryGroupCacheLogicNode(SNode* pNode) { - SGroupCacheLogicNode* pLogicNode = (SGroupCacheLogicNode*)pNode; - destroyLogicNode((SLogicNode*)pLogicNode); - nodesDestroyList(pLogicNode->pGroupCols); -} - -void destoryDynQueryCtrlLogicNode(SNode* pNode) { - destroyLogicNode((SLogicNode*)pNode); -} - -void destoryLogicSubplan(SNode* pNode) { - SLogicSubplan* pSubplan = (SLogicSubplan*)pNode; - nodesDestroyList(pSubplan->pChildren); - nodesDestroyNode((SNode*)pSubplan->pNode); - nodesClearList(pSubplan->pParents); - taosMemoryFreeClear(pSubplan->pVgroupList); -} - -void destoryQueryLogicPlan(SNode* pNode) { - nodesDestroyList(((SQueryLogicPlan*)pNode)->pTopSubplans); -} - -void destroyScanPhysiNode(SNode* pInput) { - SScanPhysiNode* pNode = (SScanPhysiNode*)pInput; - destroyPhysiNode(pInput); - nodesDestroyList(pNode->pScanCols); - nodesDestroyList(pNode->pScanPseudoCols); -} - -void destoryLastRowScanPhysiNode(SNode* pNode) { - SLastRowScanPhysiNode* pPhyNode = (SLastRowScanPhysiNode*)pNode; - destroyScanPhysiNode(pNode); - nodesDestroyList(pPhyNode->pGroupTags); - nodesDestroyList(pPhyNode->pTargets); -} - -void destoryTableScanPhysiNode(SNode* pNode) { - STableScanPhysiNode* pPhyNode = (STableScanPhysiNode*)pNode; - destroyScanPhysiNode(pNode); - nodesDestroyList(pPhyNode->pDynamicScanFuncs); - nodesDestroyList(pPhyNode->pGroupTags); - nodesDestroyList(pPhyNode->pTags); - nodesDestroyNode(pPhyNode->pSubtable); -} - -void destoryProjectPhysiNode(SNode* pNode) { - SProjectPhysiNode* pPhyNode = (SProjectPhysiNode*)pNode; - destroyPhysiNode(pNode); - nodesDestroyList(pPhyNode->pProjections); -} - -void destorySortMergeJoinPhysiNode(SNode* pNode) { - SSortMergeJoinPhysiNode* pPhyNode = (SSortMergeJoinPhysiNode*)pNode; - destroyPhysiNode(pNode); - nodesDestroyNode(pPhyNode->pPrimKeyCond); - nodesDestroyNode(pPhyNode->pOtherOnCond); - nodesDestroyList(pPhyNode->pTargets); - nodesDestroyNode(pPhyNode->pColEqCond); -} - -void destoryHashJoinPhysiNode(SNode* pNode) { - SHashJoinPhysiNode* pPhyNode = (SHashJoinPhysiNode*)pNode; - destroyPhysiNode(pNode); - nodesDestroyList(pPhyNode->pOnLeft); - nodesDestroyList(pPhyNode->pOnRight); - nodesDestroyNode(pPhyNode->pFilterConditions); - nodesDestroyList(pPhyNode->pTargets); - - nodesDestroyNode(pPhyNode->pPrimKeyCond); - nodesDestroyNode(pPhyNode->pColEqCond); - nodesDestroyNode(pPhyNode->pTagEqCond); -} - -void destoryAggPhysiNode(SNode* pNode) { - SAggPhysiNode* pPhyNode = (SAggPhysiNode*)pNode; - destroyPhysiNode(pNode); - nodesDestroyList(pPhyNode->pExprs); - nodesDestroyList(pPhyNode->pAggFuncs); - nodesDestroyList(pPhyNode->pGroupKeys); -} - -void destoryExchangePhysiNode(SNode* pNode) { - SExchangePhysiNode* pPhyNode = (SExchangePhysiNode*)pNode; - destroyPhysiNode(pNode); - nodesDestroyList(pPhyNode->pSrcEndPoints); -} - -void destoryMergePhysiNode(SNode* pNode) { - SMergePhysiNode* pPhyNode = (SMergePhysiNode*)pNode; - destroyPhysiNode(pNode); - nodesDestroyList(pPhyNode->pMergeKeys); - nodesDestroyList(pPhyNode->pTargets); -} - -void destorySortPhysiNode(SNode* pNode) { - SSortPhysiNode* pPhyNode = (SSortPhysiNode*)pNode; - destroyPhysiNode(pNode); - nodesDestroyList(pPhyNode->pExprs); - nodesDestroyList(pPhyNode->pSortKeys); - nodesDestroyList(pPhyNode->pTargets); -} - -void destroyWindowPhysiNode(SNode* pInput) { - SWindowPhysiNode* pNode = (SWindowPhysiNode*)pInput; - destroyPhysiNode(pInput); - nodesDestroyList(pNode->pExprs); - nodesDestroyList(pNode->pFuncs); - nodesDestroyNode(pNode->pTspk); - nodesDestroyNode(pNode->pTsEnd); -} - -void destoryFillPhysiNode(SNode* pNode) { - SFillPhysiNode* pPhyNode = (SFillPhysiNode*)pNode; - destroyPhysiNode(pNode); - nodesDestroyList(pPhyNode->pFillExprs); - nodesDestroyList(pPhyNode->pNotFillExprs); - nodesDestroyNode(pPhyNode->pWStartTs); - nodesDestroyNode(pPhyNode->pValues); -} - -void destoryStateWindowPhysiNode(SNode* pNode) { - SStateWinodwPhysiNode* pPhyNode = (SStateWinodwPhysiNode*)pNode; - destroyWindowPhysiNode(pNode); - nodesDestroyNode(pPhyNode->pStateKey); -} - -void destoryEventWindowPhysiNode(SNode* pNode) { - SEventWinodwPhysiNode* pPhyNode = (SEventWinodwPhysiNode*)pNode; - destroyWindowPhysiNode(pNode); - nodesDestroyNode(pPhyNode->pStartCond); - nodesDestroyNode(pPhyNode->pEndCond); -} - -void destroyPartitionPhysiNode(SNode* pNode) { - SPartitionPhysiNode* pPartitionNode = (SPartitionPhysiNode*)pNode; - destroyPhysiNode(pNode); - nodesDestroyList(pPartitionNode->pExprs); - nodesDestroyList(pPartitionNode->pPartitionKeys); - nodesDestroyList(pPartitionNode->pTargets); -} - -void destoryStreamPartitionPhysiNode(SNode* pNode) { - SStreamPartitionPhysiNode* pPhyNode = (SStreamPartitionPhysiNode*)pNode; - destroyPartitionPhysiNode(pNode); - nodesDestroyList(pPhyNode->pTags); - nodesDestroyNode(pPhyNode->pSubtable); -} - -void destoryIndefRowsFuncPhysiNode(SNode* pNode) { - SIndefRowsFuncPhysiNode* pPhyNode = (SIndefRowsFuncPhysiNode*)pNode; - destroyPhysiNode(pNode); - nodesDestroyList(pPhyNode->pExprs); - nodesDestroyList(pPhyNode->pFuncs); -} - -void destoryInterpFuncPhysiNode(SNode* pNode) { - SInterpFuncPhysiNode* pPhyNode = (SInterpFuncPhysiNode*)pNode; - destroyPhysiNode(pNode); - nodesDestroyList(pPhyNode->pExprs); - nodesDestroyList(pPhyNode->pFuncs); - nodesDestroyNode(pPhyNode->pFillValues); - nodesDestroyNode(pPhyNode->pTimeSeries); -} - -void destroyDataSinkNode(SNode* pNode) { - SDataSinkNode* pDataNode = (SDataSinkNode*)pNode; - nodesDestroyNode((SNode*)pDataNode->pInputDataBlockDesc); -} - -void destoryDataInserterNode(SNode* pNode) { - SDataInserterNode* pSink = (SDataInserterNode*)pNode; - destroyDataSinkNode(pNode); - taosMemoryFreeClear(pSink->pData); -} - -void destoryQueryInserterNode(SNode* pNode) { - SQueryInserterNode* pSink = (SQueryInserterNode*)pNode; - destroyDataSinkNode(pNode); - nodesDestroyList(pSink->pCols); -} - -void destoryDataDeleterNode(SNode* pNode) { - SDataDeleterNode* pSink = (SDataDeleterNode*)pNode; - destroyDataSinkNode(pNode); - nodesDestroyNode(pSink->pAffectedRows); - nodesDestroyNode(pSink->pStartTs); - nodesDestroyNode(pSink->pEndTs); -} - -void destoryGroupCachePhysiNode(SNode* pNode) { - SGroupCachePhysiNode* pPhyNode = (SGroupCachePhysiNode*)pNode; - destroyPhysiNode(pNode); - nodesDestroyList(pPhyNode->pGroupCols); -} - -void destoryDynQueryCtrlPhysiNode(SNode* pNode) { - destroyPhysiNode(pNode); -} - -void destorySubplanNode(SNode* pNode) { - SSubplan* pSubplan = (SSubplan*)pNode; - nodesClearList(pSubplan->pChildren); - nodesDestroyNode((SNode*)pSubplan->pNode); - nodesDestroyNode((SNode*)pSubplan->pDataSink); - nodesDestroyNode((SNode*)pSubplan->pTagCond); - nodesDestroyNode((SNode*)pSubplan->pTagIndexCond); - nodesClearList(pSubplan->pParents); -} - -void destoryPlanNode(SNode* pNode) { - nodesDestroyList(((SQueryPlan*)pNode)->pSubplans); -} - -void nodesDestroyNode(SNode* pNode) { - if (NULL == pNode) { - return; - } - - int32_t type = nodeType(pNode); - if (!funcArrayCheck(type)) { - return; - } - if (funcNodes[type].destoryFunc) { - funcNodes[type].destoryFunc(pNode); - nodesFree(pNode); - return; - } - nodesWarn("nodesDestroyNode unknown type = %d", type); - nodesFree(pNode); - return; -} - -// clang-format off -static void doInitNodeFuncArray() { - setFunc("Column", - QUERY_NODE_COLUMN, - sizeof(SColumnNode), - columnNodeToJson, - jsonToColumnNode, - destroyColumnNode - ); - setFunc("Value", - QUERY_NODE_VALUE, - sizeof(SValueNode), - valueNodeToJson, - jsonToValueNode, - destroyValueNode - ); - setFunc("Operator", - QUERY_NODE_OPERATOR, - sizeof(SOperatorNode), - operatorNodeToJson, - jsonToOperatorNode, - destroyOperatorNode - ); - setFunc("LogicCondition", - QUERY_NODE_LOGIC_CONDITION, - sizeof(SLogicConditionNode), - logicConditionNodeToJson, - jsonToLogicConditionNode, - destoryLogicConditionNode - ); - setFunc("Function", - QUERY_NODE_FUNCTION, - sizeof(SFunctionNode), - functionNodeToJson, - jsonToFunctionNode, - destoryFunctionNode - ); - setFunc("RealTable", - QUERY_NODE_REAL_TABLE, - sizeof(SRealTableNode), - realTableNodeToJson, - jsonToRealTableNode, - destoryRealTableNode - ); - setFunc("TempTable", - QUERY_NODE_TEMP_TABLE, - sizeof(STempTableNode), - tempTableNodeToJson, - jsonToTempTableNode, - destoryTempTableNode - ); - setFunc("JoinTable", - QUERY_NODE_JOIN_TABLE, - sizeof(SJoinTableNode), - joinTableNodeToJson, - jsonToJoinTableNode, - destoryJoinTableNode - ); - setFunc("GroupingSet", - QUERY_NODE_GROUPING_SET, - sizeof(SGroupingSetNode), - groupingSetNodeToJson, - jsonToGroupingSetNode, - destoryGroupingSetNode - ); - setFunc("OrderByExpr", - QUERY_NODE_ORDER_BY_EXPR, - sizeof(SOrderByExprNode), - orderByExprNodeToJson, - jsonToOrderByExprNode, - destoryOrderByExprNode - ); - setFunc("Limit", - QUERY_NODE_LIMIT, - sizeof(SLimitNode), - limitNodeToJson, - jsonToLimitNode, - destoryXNode - ); - setFunc("StateWindow", - QUERY_NODE_STATE_WINDOW, - sizeof(SStateWindowNode), - stateWindowNodeToJson, - jsonToStateWindowNode, - destoryStateWindowNode - ); - setFunc("SessionWinow", - QUERY_NODE_SESSION_WINDOW, - sizeof(SSessionWindowNode), - sessionWindowNodeToJson, - jsonToSessionWindowNode, - destorySessionWindowNode - ); - setFunc("IntervalWindow", - QUERY_NODE_INTERVAL_WINDOW, - sizeof(SIntervalWindowNode), - intervalWindowNodeToJson, - jsonToIntervalWindowNode, - destoryIntervalWindowNode - ); - setFunc("NodeList", - QUERY_NODE_NODE_LIST, - sizeof(SNodeListNode), - nodeListNodeToJson, - jsonToNodeListNode, - destoryNodeListNode - ); - setFunc("Fill", - QUERY_NODE_FILL, - sizeof(SFillNode), - fillNodeToJson, - jsonToFillNode, - destoryFillNode - ); - setFunc("RawExpr", - QUERY_NODE_RAW_EXPR, - sizeof(SRawExprNode), - emptyNodeToJson, - emptyJsonToNode, - destoryRawExprNode - ); - setFunc("Target", - QUERY_NODE_TARGET, - sizeof(STargetNode), - targetNodeToJson, - jsonToTargetNode, - destoryTargetNode - ); - setFunc("DataBlockDesc", - QUERY_NODE_DATABLOCK_DESC, - sizeof(SDataBlockDescNode), - dataBlockDescNodeToJson, - jsonToDataBlockDescNode, - destoryDataBlockDescNode - ); - setFunc("SlotDesc", - QUERY_NODE_SLOT_DESC, - sizeof(SSlotDescNode), - slotDescNodeToJson, - jsonToSlotDescNode, - destoryXNode - ); - setFunc("ColumnDef", - QUERY_NODE_COLUMN_DEF, - sizeof(SColumnDefNode), - columnDefNodeToJson, - jsonToColumnDefNode, - destoryXNode - ); - setFunc("DownstreamSource", - QUERY_NODE_DOWNSTREAM_SOURCE, - sizeof(SDownstreamSourceNode), - downstreamSourceNodeToJson, - jsonToDownstreamSourceNode, - destoryXNode - ); - setFunc("DatabaseOptions", - QUERY_NODE_DATABASE_OPTIONS, - sizeof(SDatabaseOptions), - databaseOptionsToJson, - jsonToDatabaseOptions, - destoryDatabaseOptions - ); - setFunc("TableOptions", - QUERY_NODE_TABLE_OPTIONS, - sizeof(STableOptions), - tableOptionsToJson, - jsonToTableOptions, - destoryTableOptions - ); - setFunc("IndexOptions", - QUERY_NODE_INDEX_OPTIONS, - sizeof(SIndexOptions), - indexOptionsToJson, - jsonToIndexOptions, - destoryIndexOptions - ); - setFunc("ExplainOptions", - QUERY_NODE_EXPLAIN_OPTIONS, - sizeof(SExplainOptions), - explainOptionsToJson, - jsonToExplainOptions, - destoryXNode - ); - setFunc("StreamOptions", - QUERY_NODE_STREAM_OPTIONS, - sizeof(SStreamOptions), - streamOptionsToJson, - jsonToStreamOptions, - destoryStreamOptions - ); - setFunc("LeftValue", - QUERY_NODE_LEFT_VALUE, - sizeof(SLeftValueNode), - emptyNodeToJson, - emptyJsonToNode, - destoryXNode - ); - setFunc("ColumnRef", - QUERY_NODE_COLUMN_REF, - sizeof(SColumnDefNode), - emptyNodeToJson, - emptyJsonToNode, - destoryXNode - ); - setFunc("WhenThen", - QUERY_NODE_WHEN_THEN, - sizeof(SWhenThenNode), - whenThenNodeToJson, - jsonToWhenThenNode, - destoryWhenThenNode - ); - setFunc("CaseWhen", - QUERY_NODE_CASE_WHEN, - sizeof(SCaseWhenNode), - caseWhenNodeToJson, - jsonToCaseWhenNode, - destoryCaseWhenNode - ); - setFunc("EventWindow", - QUERY_NODE_EVENT_WINDOW, - sizeof(SEventWindowNode), - eventWindowNodeToJson, - jsonToEventWindowNode, - destoryEventWindowNode - ); - setFunc("HintNode", - QUERY_NODE_HINT, - sizeof(SHintNode), - emptyNodeToJson, - emptyJsonToNode, - destoryHintNode - ); - setFunc("ViewNode", - QUERY_NODE_VIEW, - sizeof(SViewNode), - emptyNodeToJson, - emptyJsonToNode, - destoryViewNode - ); - setFunc("SetOperator", - QUERY_NODE_SET_OPERATOR, - sizeof(SSetOperator), - setOperatorToJson, - jsonToSetOperator, - destorySetOperator - ); - setFunc("SelectStmt", - QUERY_NODE_SELECT_STMT, - sizeof(SSelectStmt), - selectStmtToJson, - jsonToSelectStmt, - destorySelectStmt - ); - setFunc("VnodeModifyStmt", - QUERY_NODE_VNODE_MODIFY_STMT, - sizeof(SVnodeModifyOpStmt), - vnodeModifyStmtToJson, - jsonToVnodeModifyStmt, - destoryVnodeModifyOpStmt - ); - setFunc("CreateDatabaseStmt", - QUERY_NODE_CREATE_DATABASE_STMT, - sizeof(SCreateDatabaseStmt), - createDatabaseStmtToJson, - jsonToCreateDatabaseStmt, - destoryCreateDatabaseStmt - ); - setFunc("DropDatabaseStmt", - QUERY_NODE_DROP_DATABASE_STMT, - sizeof(SDropDatabaseStmt), - emptyNodeToJson, - emptyJsonToNode, - destoryXNode - ); - setFunc("AlterDatabaseStmt", - QUERY_NODE_ALTER_DATABASE_STMT, - sizeof(SAlterDatabaseStmt), - alterDatabaseStmtToJson, - jsonToAlterDatabaseStmt, - destoryAlterDatabaseStmt - ); - setFunc("FlushDatabaseStmt", - QUERY_NODE_FLUSH_DATABASE_STMT, - sizeof(SFlushDatabaseStmt), - emptyNodeToJson, - emptyJsonToNode, - destoryXNode - ); - setFunc("TrimDatabaseStmt", - QUERY_NODE_TRIM_DATABASE_STMT, - sizeof(STrimDatabaseStmt), - trimDatabaseStmtToJson, - jsonToTrimDatabaseStmt, - destoryXNode - ); - setFunc("CreateTableStmt", - QUERY_NODE_CREATE_TABLE_STMT, - sizeof(SCreateTableStmt), - createTableStmtToJson, - jsonToCreateTableStmt, - destoryCreateTableStmt - ); - setFunc("CreateSubtableClause", - QUERY_NODE_CREATE_SUBTABLE_CLAUSE, - sizeof(SCreateSubTableClause), - createSubTableClauseToJson, - jsonToCreateSubTableClause, - destoryCreateSubTableClause - ); - setFunc("CreateMultiTableStmt", - QUERY_NODE_CREATE_MULTI_TABLES_STMT, - sizeof(SCreateMultiTablesStmt), - createMultiTablesStmtToJson, - jsonToCreateMultiTablesStmt, - destoryCreateMultiTablesStmt - ); - setFunc("DropTableClause", - QUERY_NODE_DROP_TABLE_CLAUSE, - sizeof(SDropTableClause), - dropTableClauseToJson, - jsonToDropTableClause, - destoryXNode - ); - setFunc("DropTableStmt", - QUERY_NODE_DROP_TABLE_STMT, - sizeof(SDropTableStmt), - dropTableStmtToJson, - jsonToDropTableStmt, - destoryDropTableStmt - ); - setFunc("DropSuperTableStmt", - QUERY_NODE_DROP_SUPER_TABLE_STMT, - sizeof(SDropSuperTableStmt), - dropStableStmtToJson, - jsonToDropStableStmt, - destoryXNode - ); - setFunc("AlterTableStmt", - QUERY_NODE_ALTER_TABLE_STMT, - sizeof(SAlterTableStmt), - alterTableStmtToJson, - jsonToAlterTableStmt, - destoryAlterTableStmt - ); - setFunc("AlterSuperTableStmt", - QUERY_NODE_ALTER_SUPER_TABLE_STMT, - sizeof(SAlterTableStmt), - alterStableStmtToJson, - jsonToAlterStableStmt, - destoryAlterTableStmt - ); - setFunc("CreateUserStmt", - QUERY_NODE_CREATE_USER_STMT, - sizeof(SCreateUserStmt), - createUserStmtToJson, - jsonToCreateUserStmt, - destoryCreateUserStmt - ); - setFunc("AlterUserStmt", - QUERY_NODE_ALTER_USER_STMT, - sizeof(SAlterUserStmt), - alterUserStmtToJson, - jsonToAlterUserStmt, - destoryAlterUserStmt - ); - setFunc("DropUserStmt", - QUERY_NODE_DROP_USER_STMT, - sizeof(SDropUserStmt), - dropUserStmtToJson, - jsonToDropUserStmt, - destoryXNode - ); - setFunc("UseDatabaseStmt", - QUERY_NODE_USE_DATABASE_STMT, - sizeof(SUseDatabaseStmt), - useDatabaseStmtToJson, - jsonToUseDatabaseStmt, - destoryXNode - ); - setFunc("CreateDnodeStmt", - QUERY_NODE_CREATE_DNODE_STMT, - sizeof(SCreateDnodeStmt), - createDnodeStmtToJson, - jsonToCreateDnodeStmt, - destoryXNode - ); - setFunc("DropDnodeStmt", - QUERY_NODE_DROP_DNODE_STMT, - sizeof(SDropDnodeStmt), - dropDnodeStmtToJson, - jsonToDropDnodeStmt, - destoryXNode - ); - setFunc("AlterDnodeStmt", - QUERY_NODE_ALTER_DNODE_STMT, - sizeof(SAlterDnodeStmt), - alterDnodeStmtToJson, - jsonToAlterDnodeStmt, - destoryXNode - ); - setFunc("CreateIndexStmt", - QUERY_NODE_CREATE_INDEX_STMT, - sizeof(SCreateIndexStmt), - createIndexStmtToJson, - jsonToCreateIndexStmt, - destoryCreateIndexStmt - ); - setFunc("DropIndexStmt", - QUERY_NODE_DROP_INDEX_STMT, - sizeof(SDropIndexStmt), - dropIndexStmtToJson, - jsonToDropIndexStmt, - destoryXNode - ); - setFunc("CreateQnodeStmt", - QUERY_NODE_CREATE_QNODE_STMT, - sizeof(SCreateComponentNodeStmt), - createQnodeStmtToJson, - jsonToCreateQnodeStmt, - destoryXNode - ); - setFunc("DropQnodeStmt", - QUERY_NODE_DROP_QNODE_STMT, - sizeof(SDropComponentNodeStmt), - dropQnodeStmtToJson, - jsonToDropQnodeStmt, - destoryXNode - ); - setFunc("CreateBnodeStmt", - QUERY_NODE_CREATE_BNODE_STMT, - sizeof(SCreateComponentNodeStmt), - emptyNodeToJson, - emptyJsonToNode, - destoryXNode - ); - setFunc("DropBnodeStmt", - QUERY_NODE_DROP_BNODE_STMT, - sizeof(SDropComponentNodeStmt), - emptyNodeToJson, - emptyJsonToNode, - destoryXNode - ); - setFunc("CreateSnodeStmt", - QUERY_NODE_CREATE_SNODE_STMT, - sizeof(SCreateComponentNodeStmt), - createSnodeStmtToJson, - jsonToCreateSnodeStmt, - destoryXNode - ); - setFunc("DropSnodeStmt", - QUERY_NODE_DROP_SNODE_STMT, - sizeof(SDropComponentNodeStmt), - dropSnodeStmtToJson, - jsonToDropSnodeStmt, - destoryXNode - ); - setFunc("CreateMnodeStmt", - QUERY_NODE_CREATE_MNODE_STMT, - sizeof(SCreateComponentNodeStmt), - createMnodeStmtToJson, - jsonToCreateMnodeStmt, - destoryXNode - ); - setFunc("DropMnodeStmt", - QUERY_NODE_DROP_MNODE_STMT, - sizeof(SDropComponentNodeStmt), - dropMnodeStmtToJson, - jsonToDropMnodeStmt, - destoryXNode - ); - setFunc("CreateTopicStmt", - QUERY_NODE_CREATE_TOPIC_STMT, - sizeof(SCreateTopicStmt), - createTopicStmtToJson, - jsonToCreateTopicStmt, - destoryCreateTopicStmt - ); - setFunc("DropTopicStmt", - QUERY_NODE_DROP_TOPIC_STMT, - sizeof(SDropTopicStmt), - dropTopicStmtToJson, - jsonToDropTopicStmt, - destoryXNode - ); - setFunc("DropConsumerGroupStmt", - QUERY_NODE_DROP_CGROUP_STMT, - sizeof(SDropCGroupStmt), - dropConsumerGroupStmtToJson, - jsonToDropConsumerGroupStmt, - destoryXNode - ); - setFunc("AlterLocalStmt", - QUERY_NODE_ALTER_LOCAL_STMT, - sizeof(SAlterLocalStmt), - alterLocalStmtToJson, - jsonToAlterLocalStmt, - destoryXNode - ); - setFunc("ExplainStmt", - QUERY_NODE_EXPLAIN_STMT, - sizeof(SExplainStmt), - explainStmtToJson, - jsonToExplainStmt, - destoryExplainStmt - ); - setFunc("DescribeStmt", - QUERY_NODE_DESCRIBE_STMT, - sizeof(SDescribeStmt), - describeStmtToJson, - jsonToDescribeStmt, - destoryDescribeStmt - ); - setFunc("QueryCacheStmt", - QUERY_NODE_RESET_QUERY_CACHE_STMT, - sizeof(SNode), - emptyNodeToJson, - emptyJsonToNode, - destoryXNode - ); - setFunc("CompactDatabaseStmt", - QUERY_NODE_COMPACT_DATABASE_STMT, - sizeof(SCompactDatabaseStmt), - compactDatabaseStmtToJson, - jsonToCompactDatabaseStmt, - destoryCompactDatabaseStmt - ); - setFunc("CreateFunctionStmt", - QUERY_NODE_CREATE_FUNCTION_STMT, - sizeof(SCreateFunctionStmt), - emptyNodeToJson, - emptyJsonToNode, - destoryXNode - ); - setFunc("DropFunctionStmt", - QUERY_NODE_DROP_FUNCTION_STMT, - sizeof(SDropFunctionStmt), - emptyNodeToJson, - emptyJsonToNode, - destoryXNode - ); - setFunc("CreateStreamStmt", - QUERY_NODE_CREATE_STREAM_STMT, - sizeof(SCreateStreamStmt), - createStreamStmtToJson, - jsonToCreateStreamStmt, - destoryCreateStreamStmt - ); - setFunc("DropStreamStmt", - QUERY_NODE_DROP_STREAM_STMT, - sizeof(SDropStreamStmt), - dropStreamStmtToJson, - jsonToDropStreamStmt, - destoryXNode - ); - setFunc("PauseStreamStmt", - QUERY_NODE_PAUSE_STREAM_STMT, - sizeof(SPauseStreamStmt), - emptyNodeToJson, - emptyJsonToNode, - destoryXNode - ); - setFunc("ResumeStreamStmt", - QUERY_NODE_RESUME_STREAM_STMT, - sizeof(SResumeStreamStmt), - emptyNodeToJson, - emptyJsonToNode, - destoryXNode - ); - setFunc("BalanceVgroupStmt", - QUERY_NODE_BALANCE_VGROUP_STMT, - sizeof(SBalanceVgroupStmt), - emptyNodeToJson, - emptyJsonToNode, - destoryXNode - ); - setFunc("BalanceVgroupLeaderStmt", - QUERY_NODE_BALANCE_VGROUP_LEADER_STMT, - sizeof(SBalanceVgroupLeaderStmt), - emptyNodeToJson, - emptyJsonToNode, - destoryXNode - ); - setFunc("MergeVgroupStmt", - QUERY_NODE_MERGE_VGROUP_STMT, - sizeof(SMergeVgroupStmt), - mergeVgroupStmtToJson, - jsonToMergeVgroupStmt, - destoryXNode - ); - setFunc("RedistributeVgroupStmt", - QUERY_NODE_REDISTRIBUTE_VGROUP_STMT, - sizeof(SRedistributeVgroupStmt), - redistributeVgroupStmtToJson, - jsonToRedistributeVgroupStmt, - destoryRedistributeVgroupStmt - ); - setFunc("SplitVgroupStmt", - QUERY_NODE_SPLIT_VGROUP_STMT, - sizeof(SSplitVgroupStmt), - splitVgroupStmtToJson, - jsonToSplitVgroupStmt, - destoryXNode - ); - setFunc("SyncDBStmt", - QUERY_NODE_SYNCDB_STMT, - 0, - emptyNodeToJson, - emptyJsonToNode, - destoryXNode - ); - setFunc("GrantStmt", - QUERY_NODE_GRANT_STMT, - sizeof(SGrantStmt), - grantStmtToJson, - jsonToGrantStmt, - destoryGrantStmt - ); - setFunc("RevokeStmt", - QUERY_NODE_REVOKE_STMT, - sizeof(SRevokeStmt), - revokeStmtToJson, - jsonToRevokeStmt, - destoryRevokeStmt - ); - setFunc("ShowDnodesStmt", - QUERY_NODE_SHOW_DNODES_STMT, - sizeof(SShowStmt), - showDnodesStmtToJson, - jsonToShowDnodesStmt, - destoryShowStmt - ); - setFunc("ShowMnodesStmt", - QUERY_NODE_SHOW_MNODES_STMT, - sizeof(SShowStmt), - showMnodesStmtToJson, - jsonToShowMnodesStmt, - destoryShowStmt - ); - setFunc("ShowModulesStmt", - QUERY_NODE_SHOW_MODULES_STMT, - sizeof(SShowStmt), - emptyNodeToJson, - emptyJsonToNode, - destoryShowStmt - ); - setFunc("ShowQnodesStmt", - QUERY_NODE_SHOW_QNODES_STMT, - sizeof(SShowStmt), - showQnodesStmtToJson, - jsonToShowQnodesStmt, - destoryShowStmt - ); - setFunc("ShowSnodesStmt", - QUERY_NODE_SHOW_SNODES_STMT, - sizeof(SShowStmt), - emptyNodeToJson, - emptyJsonToNode, - destoryShowStmt - ); - setFunc("ShowBnodesStmt", - QUERY_NODE_SHOW_BNODES_STMT, - sizeof(SShowStmt), - emptyNodeToJson, - emptyJsonToNode, - destoryShowStmt - ); - setFunc("ShowClusterStmt", - QUERY_NODE_SHOW_CLUSTER_STMT, - sizeof(SShowStmt), - showClusterStmtToJson, - jsonToShowClusterStmt, - destoryShowStmt - ); - setFunc("ShowDatabaseStmt", - QUERY_NODE_SHOW_DATABASES_STMT, - sizeof(SShowStmt), - showDatabasesStmtToJson, - jsonToShowDatabasesStmt, - destoryShowStmt - ); - setFunc("ShowFunctionsStmt", - QUERY_NODE_SHOW_FUNCTIONS_STMT, - sizeof(SShowStmt), - showFunctionsStmtToJson, - jsonToShowFunctionsStmt, - destoryShowStmt - ); - setFunc("ShowIndexesStmt", - QUERY_NODE_SHOW_INDEXES_STMT, - sizeof(SShowStmt), - showIndexesStmtToJson, - jsonToShowIndexesStmt, - destoryShowStmt - ); - setFunc("ShowStablesStmt", - QUERY_NODE_SHOW_STABLES_STMT, - sizeof(SShowStmt), - showStablesStmtToJson, - jsonToShowStablesStmt, - destoryShowStmt - ); - setFunc("ShowStreamsStmt", - QUERY_NODE_SHOW_STREAMS_STMT, - sizeof(SShowStmt), - showStreamsStmtToJson, - jsonToShowStreamsStmt, - destoryShowStmt - ); - setFunc("ShowTablesStmt", - QUERY_NODE_SHOW_TABLES_STMT, - sizeof(SShowStmt), - showTablesStmtToJson, - jsonToShowTablesStmt, - destoryShowStmt - ); - setFunc("ShowTagsStmt", - QUERY_NODE_SHOW_TAGS_STMT, - sizeof(SShowStmt), - showTagsStmtToJson, - jsonToShowTagsStmt, - destoryShowStmt - ); - setFunc("ShowUsersStmt", - QUERY_NODE_SHOW_USERS_STMT, - sizeof(SShowStmt), - showUsersStmtToJson, - jsonToShowUsersStmt, - destoryShowStmt - ); - setFunc("ShowLicencesStmt", - QUERY_NODE_SHOW_LICENCES_STMT, - sizeof(SShowStmt), - emptyNodeToJson, - emptyJsonToNode, - destoryShowStmt - ); - setFunc("ShowVgroupsStmt", - QUERY_NODE_SHOW_VGROUPS_STMT, - sizeof(SShowStmt), - showVgroupsStmtToJson, - jsonToShowVgroupsStmt, - destoryShowStmt - ); - setFunc("ShowTopicsStmt", - QUERY_NODE_SHOW_TOPICS_STMT, - sizeof(SShowStmt), - emptyNodeToJson, - emptyJsonToNode, - destoryShowStmt - ); - setFunc("ShowConsumersStmt", - QUERY_NODE_SHOW_CONSUMERS_STMT, - sizeof(SShowStmt), - showConsumersStmtToJson, - jsonToShowConsumersStmt, - destoryShowStmt - ); - setFunc("ShowQueriesStmt", - QUERY_NODE_SHOW_QUERIES_STMT, - sizeof(SShowStmt), - emptyNodeToJson, - emptyJsonToNode, - destoryShowStmt - ); - setFunc("ShowConnectionsStmt", - QUERY_NODE_SHOW_CONNECTIONS_STMT, - sizeof(SShowStmt), - emptyNodeToJson, - emptyJsonToNode, - destoryShowStmt - ); - setFunc("ShowAppsStmt", - QUERY_NODE_SHOW_APPS_STMT, - sizeof(SShowStmt), - emptyNodeToJson, - emptyJsonToNode, - destoryShowStmt - ); - setFunc("ShowVariablesStmt", - QUERY_NODE_SHOW_VARIABLES_STMT, - sizeof(SShowStmt), - showVariablesStmtToJson, - jsonToShowVariablesStmt, - destoryShowStmt - ); - setFunc("ShowDnodeVariablesStmt", - QUERY_NODE_SHOW_DNODE_VARIABLES_STMT, - sizeof(SShowDnodeVariablesStmt), - showDnodeVariablesStmtToJson, - jsonToShowDnodeVariablesStmt, - destoryShowDnodeVariablesStmt - ); - setFunc("ShowTransactionsStmt", - QUERY_NODE_SHOW_TRANSACTIONS_STMT, - sizeof(SShowStmt), - showTransactionsStmtToJson, - jsonToShowTransactionsStmt, - destoryShowStmt - ); - setFunc("ShowSubscriptionsStmt", - QUERY_NODE_SHOW_SUBSCRIPTIONS_STMT, - sizeof(SShowStmt), - showSubscriptionsStmtToJson, - jsonToShowSubscriptionsStmt, - destoryShowStmt - ); - setFunc("ShowVnodeStmt", - QUERY_NODE_SHOW_VNODES_STMT, - sizeof(SShowStmt), - showVnodesStmtToJson, - jsonToShowVnodesStmt, - destoryShowStmt - ); - setFunc("ShowUserPrivilegesStmt", - QUERY_NODE_SHOW_USER_PRIVILEGES_STMT, - sizeof(SShowStmt), - showUserPrivilegesStmtToJson, - jsonToShowUserPrivilegesStmt, - destoryShowStmt - ); - setFunc("ShowViewsStmt", - QUERY_NODE_SHOW_VIEWS_STMT, - sizeof(SShowStmt), - emptyNodeToJson, - emptyJsonToNode, - destoryShowStmt - ); - setFunc("ShowCreateViewStmt", - QUERY_NODE_SHOW_CREATE_VIEW_STMT, - sizeof(SShowCreateViewStmt), - showCreateViewStmtToJson, - jsonToShowCreateViewStmt, - destoryXNode - ); - setFunc("ShowCreateDatabasesStmt", - QUERY_NODE_SHOW_CREATE_DATABASE_STMT, - sizeof(SShowCreateDatabaseStmt), - showCreateDatabaseStmtToJson, - jsonToShowCreateDatabaseStmt, - destoryShowCreateDatabaseStmt - ); - setFunc("ShowCreateTablesStmt", - QUERY_NODE_SHOW_CREATE_TABLE_STMT, - sizeof(SShowCreateTableStmt), - showCreateTableStmtToJson, - jsonToShowCreateTableStmt, - destoryShowCreateTableStmt - ); - setFunc("ShowCreateStablesStmt", - QUERY_NODE_SHOW_CREATE_STABLE_STMT, - sizeof(SShowCreateTableStmt), - showCreateStableStmtToJson, - jsonToShowCreateStableStmt, - destoryShowCreateTableStmt - ); - setFunc("ShowTableDistributedStmt", - QUERY_NODE_SHOW_TABLE_DISTRIBUTED_STMT, - sizeof(SShowTableDistributedStmt), - showTableDistributedStmtToJson, - jsonToShowTableDistributedStmt, - destoryXNode - ); - setFunc("ShowLocalVariablesStmt", - QUERY_NODE_SHOW_LOCAL_VARIABLES_STMT, - sizeof(SShowStmt), - showLocalVariablesStmtToJson, - jsonToShowLocalVariablesStmt, - destoryShowStmt - ); - setFunc("ShowScoresStmt", - QUERY_NODE_SHOW_SCORES_STMT, - sizeof(SShowStmt), - emptyNodeToJson, - emptyJsonToNode, - destoryShowStmt - ); - setFunc("ShowTableTagsStmt", - QUERY_NODE_SHOW_TABLE_TAGS_STMT, - sizeof(SShowTableTagsStmt), - showTableTagsStmtToJson, - jsonToShowTableTagsStmt, - destoryShowTableTagsStmt - ); - setFunc("KillConnectionStmt", - QUERY_NODE_KILL_CONNECTION_STMT, - sizeof(SKillStmt), - emptyNodeToJson, - emptyJsonToNode, - destoryXNode - ); - setFunc("KillQueryStmt", - QUERY_NODE_KILL_QUERY_STMT, - sizeof(SKillQueryStmt), - emptyNodeToJson, - emptyJsonToNode, - destoryXNode - ); - setFunc("KillTransactionStmt", - QUERY_NODE_KILL_TRANSACTION_STMT, - sizeof(SKillStmt), - emptyNodeToJson, - emptyJsonToNode, - destoryXNode - ); - setFunc("DeleteStmt", - QUERY_NODE_DELETE_STMT, - sizeof(SDeleteStmt), - deleteStmtToJson, - jsonToDeleteStmt, - destoryDeleteStmt - ); - setFunc("InsertStmt", - QUERY_NODE_INSERT_STMT, - sizeof(SInsertStmt), - insertStmtToJson, - jsonToInsertStmt, - destoryInsertStmt - ); - setFunc("QueryNode", - QUERY_NODE_QUERY, - sizeof(SQuery), - emptyNodeToJson, - emptyJsonToNode, - destoryQueryNode - ); - setFunc("ShowDbAliveStmt", - QUERY_NODE_SHOW_DB_ALIVE_STMT, - sizeof(SShowAliveStmt), - emptyNodeToJson, - emptyJsonToNode, - destoryXNode - ); - setFunc("ShowClusterAliveStmt", - QUERY_NODE_SHOW_CLUSTER_ALIVE_STMT, - sizeof(SShowAliveStmt), - emptyNodeToJson, - emptyJsonToNode, - destoryXNode - ); - setFunc("RestoreDnodeStmt", - QUERY_NODE_RESTORE_DNODE_STMT, - sizeof(SRestoreComponentNodeStmt), - emptyNodeToJson, - jsonToRestoreDnodeStmt, - destoryXNode - ); - setFunc("RestoreQnodeStmt", - QUERY_NODE_RESTORE_QNODE_STMT, - sizeof(SRestoreComponentNodeStmt), - emptyNodeToJson, - jsonToRestoreQnodeStmt, - destoryXNode - ); - setFunc("RestoreMnodeStmt", - QUERY_NODE_RESTORE_MNODE_STMT, - sizeof(SRestoreComponentNodeStmt), - emptyNodeToJson, - jsonToRestoreMnodeStmt, - destoryXNode - ); - setFunc("RestoreVnodeStmt", - QUERY_NODE_RESTORE_VNODE_STMT, - sizeof(SRestoreComponentNodeStmt), - emptyNodeToJson, - jsonToRestoreVnodeStmt, - destoryXNode - ); - setFunc("CreateViewStmt", - QUERY_NODE_CREATE_VIEW_STMT, - sizeof(SCreateViewStmt), - emptyNodeToJson, - emptyJsonToNode, - destoryCreateViewStmt - ); - setFunc("DropViewStmt", - QUERY_NODE_DROP_VIEW_STMT, - sizeof(SDropViewStmt), - emptyNodeToJson, - emptyJsonToNode, - destoryXNode - ); - setFunc("LogicScan", - QUERY_NODE_LOGIC_PLAN_SCAN, - sizeof(SScanLogicNode), - logicScanNodeToJson, - jsonToLogicScanNode, - destoryScanLogicNode - ); - setFunc("LogicJoin", - QUERY_NODE_LOGIC_PLAN_JOIN, - sizeof(SJoinLogicNode), - logicJoinNodeToJson, - jsonToLogicJoinNode, - destoryJoinLogicNode - ); - setFunc("LogicAgg", - QUERY_NODE_LOGIC_PLAN_AGG, - sizeof(SAggLogicNode), - logicAggNodeToJson, - jsonToLogicAggNode, - destoryAggLogicNode - ); - setFunc("LogicProject", - QUERY_NODE_LOGIC_PLAN_PROJECT, - sizeof(SProjectLogicNode), - logicProjectNodeToJson, - jsonToLogicProjectNode, - destoryProjectLogicNode - ); - setFunc("LogicVnodeModify", - QUERY_NODE_LOGIC_PLAN_VNODE_MODIFY, - sizeof(SVnodeModifyLogicNode), - logicVnodeModifyNodeToJson, - jsonToLogicVnodeModifyNode, - destoryVnodeModifyLogicNode - ); - setFunc("LogicExchange", - QUERY_NODE_LOGIC_PLAN_EXCHANGE, - sizeof(SExchangeLogicNode), - logicExchangeNodeToJson, - jsonToLogicExchangeNode, - destoryExchangeLogicNode - ); - setFunc("LogicMerge", - QUERY_NODE_LOGIC_PLAN_MERGE, - sizeof(SMergeLogicNode), - logicMergeNodeToJson, - jsonToLogicMergeNode, - destoryMergeLogicNode - ); - setFunc("LogicWindow", - QUERY_NODE_LOGIC_PLAN_WINDOW, - sizeof(SWindowLogicNode), - logicWindowNodeToJson, - jsonToLogicWindowNode, - destoryWindowLogicNode - ); - setFunc("LogicFill", - QUERY_NODE_LOGIC_PLAN_FILL, - sizeof(SFillLogicNode), - logicFillNodeToJson, - jsonToLogicFillNode, - destoryFillLogicNode - ); - setFunc("LogicSort", - QUERY_NODE_LOGIC_PLAN_SORT, - sizeof(SSortLogicNode), - logicSortNodeToJson, - jsonToLogicSortNode, - destorySortLogicNode - ); - setFunc("LogicPartition", - QUERY_NODE_LOGIC_PLAN_PARTITION, - sizeof(SPartitionLogicNode), - logicPartitionNodeToJson, - jsonToLogicPartitionNode, - destoryPartitionLogicNode - ); - setFunc("LogicIndefRowsFunc", - QUERY_NODE_LOGIC_PLAN_INDEF_ROWS_FUNC, - sizeof(SIndefRowsFuncLogicNode), - logicIndefRowsFuncNodeToJson, - jsonToLogicIndefRowsFuncNode, - destoryIndefRowsFuncLogicNode - ); - setFunc("LogicInterpFunc", - QUERY_NODE_LOGIC_PLAN_INTERP_FUNC, - sizeof(SInterpFuncLogicNode), - logicInterpFuncNodeToJson, - jsonToLogicInterpFuncNode, - destoryInterpFuncLogicNode - ); - setFunc("LogicGroupCache", - QUERY_NODE_LOGIC_PLAN_GROUP_CACHE, - sizeof(SGroupCacheLogicNode), - logicGroupCacheNodeToJson, - jsonToLogicGroupCacheNode, - destoryGroupCacheLogicNode - ); - setFunc("LogicDynamicQueryCtrl", - QUERY_NODE_LOGIC_PLAN_DYN_QUERY_CTRL, - sizeof(SDynQueryCtrlLogicNode), - logicDynQueryCtrlNodeToJson, - jsonToLogicDynQueryCtrlNode, - destoryDynQueryCtrlLogicNode - ); - setFunc("LogicSubplan", - QUERY_NODE_LOGIC_SUBPLAN, - sizeof(SLogicSubplan), - logicSubplanToJson, - jsonToLogicSubplan, - destoryLogicSubplan - ); - setFunc("LogicPlan", - QUERY_NODE_LOGIC_PLAN, - sizeof(SQueryLogicPlan), - logicPlanToJson, - jsonToLogicPlan, - destoryQueryLogicPlan - ); - setFunc("PhysiTagScan", - QUERY_NODE_PHYSICAL_PLAN_TAG_SCAN, - sizeof(STagScanPhysiNode), - physiTagScanNodeToJson, - jsonToPhysiTagScanNode, - destroyScanPhysiNode - ); - setFunc("PhysiTableScan", - QUERY_NODE_PHYSICAL_PLAN_TABLE_SCAN, - sizeof(STableScanPhysiNode), - physiTableScanNodeToJson, - jsonToPhysiTableScanNode, - destoryTableScanPhysiNode - ); - setFunc("PhysiTableSeqScan", - QUERY_NODE_PHYSICAL_PLAN_TABLE_SEQ_SCAN, - sizeof(STableSeqScanPhysiNode), - emptyNodeToJson, - emptyJsonToNode, - destoryTableScanPhysiNode - ); - setFunc("PhysiTableMergeScan", - QUERY_NODE_PHYSICAL_PLAN_TABLE_MERGE_SCAN, - sizeof(STableMergeScanPhysiNode), - physiTableScanNodeToJson, - jsonToPhysiTableScanNode, - destoryTableScanPhysiNode - ); - setFunc("PhysiSreamScan", - QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN, - sizeof(SStreamScanPhysiNode), - physiTableScanNodeToJson, - jsonToPhysiTableScanNode, - destoryTableScanPhysiNode - ); - setFunc("PhysiSystemTableScan", - QUERY_NODE_PHYSICAL_PLAN_SYSTABLE_SCAN, - sizeof(SSystemTableScanPhysiNode), - physiSysTableScanNodeToJson, - jsonToPhysiSysTableScanNode, - destroyScanPhysiNode - ); - setFunc("PhysiBlockDistScan", - QUERY_NODE_PHYSICAL_PLAN_BLOCK_DIST_SCAN, - sizeof(SBlockDistScanPhysiNode), - physiScanNodeToJson, - jsonToPhysiScanNode, - destroyScanPhysiNode - ); - setFunc("PhysiLastRowScan", - QUERY_NODE_PHYSICAL_PLAN_LAST_ROW_SCAN, - sizeof(SLastRowScanPhysiNode), - physiLastRowScanNodeToJson, - jsonToPhysiLastRowScanNode, - destoryLastRowScanPhysiNode - ); - setFunc("PhysiProject", - QUERY_NODE_PHYSICAL_PLAN_PROJECT, - sizeof(SProjectPhysiNode), - physiProjectNodeToJson, - jsonToPhysiProjectNode, - destoryProjectPhysiNode - ); - setFunc("PhysiMergeJoin", - QUERY_NODE_PHYSICAL_PLAN_MERGE_JOIN, - sizeof(SSortMergeJoinPhysiNode), - physiMergeJoinNodeToJson, - jsonToPhysiMergeJoinNode, - destorySortMergeJoinPhysiNode - ); - setFunc("PhysiAgg", - QUERY_NODE_PHYSICAL_PLAN_HASH_AGG, - sizeof(SAggPhysiNode), - physiAggNodeToJson, - jsonToPhysiAggNode, - destoryAggPhysiNode - ); - setFunc("PhysiExchange", - QUERY_NODE_PHYSICAL_PLAN_EXCHANGE, - sizeof(SExchangePhysiNode), - physiExchangeNodeToJson, - jsonToPhysiExchangeNode, - destoryExchangePhysiNode - ); - setFunc("PhysiMerge", - QUERY_NODE_PHYSICAL_PLAN_MERGE, - sizeof(SMergePhysiNode), - physiMergeNodeToJson, - jsonToPhysiMergeNode, - destoryMergePhysiNode - ); - setFunc("PhysiSort", - QUERY_NODE_PHYSICAL_PLAN_SORT, - sizeof(SSortPhysiNode), - physiSortNodeToJson, - jsonToPhysiSortNode, - destorySortPhysiNode - ); - setFunc("PhysiGroupSort", - QUERY_NODE_PHYSICAL_PLAN_GROUP_SORT, - sizeof(SGroupSortPhysiNode), - physiSortNodeToJson, - jsonToPhysiSortNode, - destorySortPhysiNode - ); - setFunc("PhysiHashInterval", - QUERY_NODE_PHYSICAL_PLAN_HASH_INTERVAL, - sizeof(SIntervalPhysiNode), - physiIntervalNodeToJson, - jsonToPhysiIntervalNode, - destroyWindowPhysiNode - ); - setFunc("PhysiMergeAlignedInterval", - QUERY_NODE_PHYSICAL_PLAN_MERGE_ALIGNED_INTERVAL, - sizeof(SMergeAlignedIntervalPhysiNode), - physiIntervalNodeToJson, - jsonToPhysiIntervalNode, - destroyWindowPhysiNode - ); - setFunc("PhysiStreamInterval", - QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL, - sizeof(SStreamIntervalPhysiNode), - physiIntervalNodeToJson, - jsonToPhysiIntervalNode, - destroyWindowPhysiNode - ); - setFunc("PhysiStreamFinalInterval", - QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL, - sizeof(SStreamFinalIntervalPhysiNode), - physiIntervalNodeToJson, - jsonToPhysiIntervalNode, - destroyWindowPhysiNode - ); - setFunc("PhysiStreamSemiInterval", - QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL, - sizeof(SStreamSemiIntervalPhysiNode), - physiIntervalNodeToJson, - jsonToPhysiIntervalNode, - destroyWindowPhysiNode - ); - setFunc("PhysiFill", - QUERY_NODE_PHYSICAL_PLAN_FILL, - sizeof(SFillPhysiNode), - physiFillNodeToJson, - jsonToPhysiFillNode, - destoryFillPhysiNode - ); - setFunc("PhysiStreamFill", - QUERY_NODE_PHYSICAL_PLAN_STREAM_FILL, - sizeof(SFillPhysiNode), - physiFillNodeToJson, - jsonToPhysiFillNode, - destoryFillPhysiNode - ); - setFunc("PhysiSessionWindow", - QUERY_NODE_PHYSICAL_PLAN_MERGE_SESSION, - sizeof(SSessionWinodwPhysiNode), - physiSessionWindowNodeToJson, - jsonToPhysiSessionWindowNode, - destroyWindowPhysiNode - ); - setFunc("PhysiStreamSessionWindow", - QUERY_NODE_PHYSICAL_PLAN_STREAM_SESSION, - sizeof(SStreamSessionWinodwPhysiNode), - physiSessionWindowNodeToJson, - jsonToPhysiSessionWindowNode, - destroyWindowPhysiNode - ); - setFunc("PhysiStreamSemiSessionWindow", - QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_SESSION, - sizeof(SStreamSemiSessionWinodwPhysiNode), - physiSessionWindowNodeToJson, - jsonToPhysiSessionWindowNode, - destroyWindowPhysiNode - ); - setFunc("PhysiStreamFinalSessionWindow", - QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_SESSION, - sizeof(SStreamFinalSessionWinodwPhysiNode), - physiSessionWindowNodeToJson, - jsonToPhysiSessionWindowNode, - destroyWindowPhysiNode - ); - setFunc("PhysiStateWindow", - QUERY_NODE_PHYSICAL_PLAN_MERGE_STATE, - sizeof(SStateWinodwPhysiNode), - physiStateWindowNodeToJson, - jsonToPhysiStateWindowNode, - destoryStateWindowPhysiNode - ); - setFunc("PhysiStreamStateWindow", - QUERY_NODE_PHYSICAL_PLAN_STREAM_STATE, - sizeof(SStreamStateWinodwPhysiNode), - physiStateWindowNodeToJson, - jsonToPhysiStateWindowNode, - destoryStateWindowPhysiNode - ); - setFunc("PhysiPartition", - QUERY_NODE_PHYSICAL_PLAN_PARTITION, - sizeof(SPartitionPhysiNode), - physiPartitionNodeToJson, - jsonToPhysiPartitionNode, - destroyPartitionPhysiNode - ); - setFunc("PhysiStreamPartition", - QUERY_NODE_PHYSICAL_PLAN_STREAM_PARTITION, - sizeof(SStreamPartitionPhysiNode), - physiStreamPartitionNodeToJson, - jsonToPhysiStreamPartitionNode, - destoryStreamPartitionPhysiNode - ); - setFunc("PhysiIndefRowsFunc", - QUERY_NODE_PHYSICAL_PLAN_INDEF_ROWS_FUNC, - sizeof(SIndefRowsFuncPhysiNode), - physiIndefRowsFuncNodeToJson, - jsonToPhysiIndefRowsFuncNode, - destoryIndefRowsFuncPhysiNode - ); - setFunc("PhysiInterpFunc", - QUERY_NODE_PHYSICAL_PLAN_INTERP_FUNC, - sizeof(SInterpFuncLogicNode), - physiInterpFuncNodeToJson, - jsonToPhysiInterpFuncNode, - destoryInterpFuncPhysiNode - ); - setFunc("PhysiDispatch", - QUERY_NODE_PHYSICAL_PLAN_DISPATCH, - sizeof(SDataDispatcherNode), - physiDispatchNodeToJson, - jsonToPhysiDispatchNode, - destroyDataSinkNode - ); - setFunc("PhysiInsert", - QUERY_NODE_PHYSICAL_PLAN_INSERT, - sizeof(SDataInserterNode), - emptyNodeToJson, - emptyJsonToNode, - destoryDataInserterNode - ); - setFunc("PhysiQueryInsert", - QUERY_NODE_PHYSICAL_PLAN_QUERY_INSERT, - sizeof(SQueryInserterNode), - physiQueryInsertNodeToJson, - jsonToPhysiQueryInsertNode, - destoryQueryInserterNode - ); - setFunc("PhysiDelete", - QUERY_NODE_PHYSICAL_PLAN_DELETE, - sizeof(SDataDeleterNode), - physiDeleteNodeToJson, - jsonToPhysiDeleteNode, - destoryDataDeleterNode - ); - setFunc("PhysiGroupCache", - QUERY_NODE_PHYSICAL_PLAN_GROUP_CACHE, - sizeof(SGroupCachePhysiNode), - physiGroupCacheNodeToJson, - jsonToPhysiGroupCacheNode, - destoryGroupCachePhysiNode - ); - setFunc("PhysiDynamicQueryCtrl", - QUERY_NODE_PHYSICAL_PLAN_DYN_QUERY_CTRL, - sizeof(SDynQueryCtrlPhysiNode), - physiDynQueryCtrlNodeToJson, - jsonToPhysiDynQueryCtrlNode, - destoryDynQueryCtrlPhysiNode - ); - setFunc("PhysiSubplan", - QUERY_NODE_PHYSICAL_SUBPLAN, - sizeof(SSubplan), - subplanToJson, - jsonToSubplan, - destorySubplanNode - ); - setFunc("PhysiPlan", - QUERY_NODE_PHYSICAL_PLAN, - sizeof(SQueryPlan), - planToJson, - jsonToPlan, - destoryPlanNode - ); - setFunc("PhysiTableCountScan", - QUERY_NODE_PHYSICAL_PLAN_TABLE_COUNT_SCAN, - sizeof(STableCountScanPhysiNode), - physiLastRowScanNodeToJson, - jsonToPhysiScanNode, - destoryLastRowScanPhysiNode - ); - setFunc("PhysiMergeEventWindow", - QUERY_NODE_PHYSICAL_PLAN_MERGE_EVENT, - sizeof(SEventWinodwPhysiNode), - physiEventWindowNodeToJson, - jsonToPhysiEventWindowNode, - destoryEventWindowPhysiNode - ); - setFunc("PhysiStreamEventWindow", - QUERY_NODE_PHYSICAL_PLAN_STREAM_EVENT, - sizeof(SStreamEventWinodwPhysiNode), - physiEventWindowNodeToJson, - jsonToPhysiEventWindowNode, - destoryEventWindowPhysiNode - ); - setFunc("PhysiHashJoin", - QUERY_NODE_PHYSICAL_PLAN_HASH_JOIN, - sizeof(SHashJoinPhysiNode), - physiHashJoinNodeToJson, - jsonToPhysiHashJoinNode, - destoryHashJoinPhysiNode - ); -} - -// clang-format on diff --git a/source/libs/nodes/src/nodesUtilFuncs.c b/source/libs/nodes/src/nodesUtilFuncs.c index 2ea387792b..e730ccf21b 100644 --- a/source/libs/nodes/src/nodesUtilFuncs.c +++ b/source/libs/nodes/src/nodesUtilFuncs.c @@ -237,14 +237,1246 @@ static SNode* makeNode(ENodeType type, int32_t size) { } SNode* nodesMakeNode(ENodeType type) { - int32_t size = getNodeSize(type); - if (size > 0) { - return makeNode(type, size); + switch (type) { + case QUERY_NODE_COLUMN: + return makeNode(type, sizeof(SColumnNode)); + case QUERY_NODE_VALUE: + return makeNode(type, sizeof(SValueNode)); + case QUERY_NODE_OPERATOR: + return makeNode(type, sizeof(SOperatorNode)); + case QUERY_NODE_LOGIC_CONDITION: + return makeNode(type, sizeof(SLogicConditionNode)); + case QUERY_NODE_FUNCTION: + return makeNode(type, sizeof(SFunctionNode)); + case QUERY_NODE_REAL_TABLE: + return makeNode(type, sizeof(SRealTableNode)); + case QUERY_NODE_TEMP_TABLE: + return makeNode(type, sizeof(STempTableNode)); + case QUERY_NODE_JOIN_TABLE: + return makeNode(type, sizeof(SJoinTableNode)); + case QUERY_NODE_GROUPING_SET: + return makeNode(type, sizeof(SGroupingSetNode)); + case QUERY_NODE_ORDER_BY_EXPR: + return makeNode(type, sizeof(SOrderByExprNode)); + case QUERY_NODE_LIMIT: + return makeNode(type, sizeof(SLimitNode)); + case QUERY_NODE_STATE_WINDOW: + return makeNode(type, sizeof(SStateWindowNode)); + case QUERY_NODE_SESSION_WINDOW: + return makeNode(type, sizeof(SSessionWindowNode)); + case QUERY_NODE_INTERVAL_WINDOW: + return makeNode(type, sizeof(SIntervalWindowNode)); + case QUERY_NODE_NODE_LIST: + return makeNode(type, sizeof(SNodeListNode)); + case QUERY_NODE_FILL: + return makeNode(type, sizeof(SFillNode)); + case QUERY_NODE_RAW_EXPR: + return makeNode(type, sizeof(SRawExprNode)); + case QUERY_NODE_TARGET: + return makeNode(type, sizeof(STargetNode)); + case QUERY_NODE_DATABLOCK_DESC: + return makeNode(type, sizeof(SDataBlockDescNode)); + case QUERY_NODE_SLOT_DESC: + return makeNode(type, sizeof(SSlotDescNode)); + case QUERY_NODE_COLUMN_DEF: + return makeNode(type, sizeof(SColumnDefNode)); + case QUERY_NODE_DOWNSTREAM_SOURCE: + return makeNode(type, sizeof(SDownstreamSourceNode)); + case QUERY_NODE_DATABASE_OPTIONS: + return makeNode(type, sizeof(SDatabaseOptions)); + case QUERY_NODE_TABLE_OPTIONS: + return makeNode(type, sizeof(STableOptions)); + case QUERY_NODE_INDEX_OPTIONS: + return makeNode(type, sizeof(SIndexOptions)); + case QUERY_NODE_EXPLAIN_OPTIONS: + return makeNode(type, sizeof(SExplainOptions)); + case QUERY_NODE_STREAM_OPTIONS: + return makeNode(type, sizeof(SStreamOptions)); + case QUERY_NODE_LEFT_VALUE: + return makeNode(type, sizeof(SLeftValueNode)); + case QUERY_NODE_COLUMN_REF: + return makeNode(type, sizeof(SColumnDefNode)); + case QUERY_NODE_WHEN_THEN: + return makeNode(type, sizeof(SWhenThenNode)); + case QUERY_NODE_CASE_WHEN: + return makeNode(type, sizeof(SCaseWhenNode)); + case QUERY_NODE_EVENT_WINDOW: + return makeNode(type, sizeof(SEventWindowNode)); + case QUERY_NODE_HINT: + return makeNode(type, sizeof(SHintNode)); + case QUERY_NODE_VIEW: + return makeNode(type, sizeof(SViewNode)); + case QUERY_NODE_SET_OPERATOR: + return makeNode(type, sizeof(SSetOperator)); + case QUERY_NODE_SELECT_STMT: + return makeNode(type, sizeof(SSelectStmt)); + case QUERY_NODE_VNODE_MODIFY_STMT: + return makeNode(type, sizeof(SVnodeModifyOpStmt)); + case QUERY_NODE_CREATE_DATABASE_STMT: + return makeNode(type, sizeof(SCreateDatabaseStmt)); + case QUERY_NODE_DROP_DATABASE_STMT: + return makeNode(type, sizeof(SDropDatabaseStmt)); + case QUERY_NODE_ALTER_DATABASE_STMT: + return makeNode(type, sizeof(SAlterDatabaseStmt)); + case QUERY_NODE_FLUSH_DATABASE_STMT: + return makeNode(type, sizeof(SFlushDatabaseStmt)); + case QUERY_NODE_TRIM_DATABASE_STMT: + return makeNode(type, sizeof(STrimDatabaseStmt)); + case QUERY_NODE_CREATE_TABLE_STMT: + return makeNode(type, sizeof(SCreateTableStmt)); + case QUERY_NODE_CREATE_SUBTABLE_CLAUSE: + return makeNode(type, sizeof(SCreateSubTableClause)); + case QUERY_NODE_CREATE_MULTI_TABLES_STMT: + return makeNode(type, sizeof(SCreateMultiTablesStmt)); + case QUERY_NODE_DROP_TABLE_CLAUSE: + return makeNode(type, sizeof(SDropTableClause)); + case QUERY_NODE_DROP_TABLE_STMT: + return makeNode(type, sizeof(SDropTableStmt)); + case QUERY_NODE_DROP_SUPER_TABLE_STMT: + return makeNode(type, sizeof(SDropSuperTableStmt)); + case QUERY_NODE_ALTER_TABLE_STMT: + case QUERY_NODE_ALTER_SUPER_TABLE_STMT: + return makeNode(type, sizeof(SAlterTableStmt)); + case QUERY_NODE_CREATE_USER_STMT: + return makeNode(type, sizeof(SCreateUserStmt)); + case QUERY_NODE_ALTER_USER_STMT: + return makeNode(type, sizeof(SAlterUserStmt)); + case QUERY_NODE_DROP_USER_STMT: + return makeNode(type, sizeof(SDropUserStmt)); + case QUERY_NODE_USE_DATABASE_STMT: + return makeNode(type, sizeof(SUseDatabaseStmt)); + case QUERY_NODE_CREATE_DNODE_STMT: + return makeNode(type, sizeof(SCreateDnodeStmt)); + case QUERY_NODE_DROP_DNODE_STMT: + return makeNode(type, sizeof(SDropDnodeStmt)); + case QUERY_NODE_ALTER_DNODE_STMT: + return makeNode(type, sizeof(SAlterDnodeStmt)); + case QUERY_NODE_CREATE_INDEX_STMT: + return makeNode(type, sizeof(SCreateIndexStmt)); + case QUERY_NODE_DROP_INDEX_STMT: + return makeNode(type, sizeof(SDropIndexStmt)); + case QUERY_NODE_CREATE_QNODE_STMT: + case QUERY_NODE_CREATE_BNODE_STMT: + case QUERY_NODE_CREATE_SNODE_STMT: + case QUERY_NODE_CREATE_MNODE_STMT: + return makeNode(type, sizeof(SCreateComponentNodeStmt)); + case QUERY_NODE_DROP_QNODE_STMT: + case QUERY_NODE_DROP_BNODE_STMT: + case QUERY_NODE_DROP_SNODE_STMT: + case QUERY_NODE_DROP_MNODE_STMT: + return makeNode(type, sizeof(SDropComponentNodeStmt)); + case QUERY_NODE_CREATE_TOPIC_STMT: + return makeNode(type, sizeof(SCreateTopicStmt)); + case QUERY_NODE_DROP_TOPIC_STMT: + return makeNode(type, sizeof(SDropTopicStmt)); + case QUERY_NODE_DROP_CGROUP_STMT: + return makeNode(type, sizeof(SDropCGroupStmt)); + case QUERY_NODE_ALTER_LOCAL_STMT: + return makeNode(type, sizeof(SAlterLocalStmt)); + case QUERY_NODE_EXPLAIN_STMT: + return makeNode(type, sizeof(SExplainStmt)); + case QUERY_NODE_DESCRIBE_STMT: + return makeNode(type, sizeof(SDescribeStmt)); + case QUERY_NODE_RESET_QUERY_CACHE_STMT: + return makeNode(type, sizeof(SNode)); + case QUERY_NODE_COMPACT_DATABASE_STMT: + return makeNode(type, sizeof(SCompactDatabaseStmt)); + case QUERY_NODE_CREATE_FUNCTION_STMT: + return makeNode(type, sizeof(SCreateFunctionStmt)); + case QUERY_NODE_DROP_FUNCTION_STMT: + return makeNode(type, sizeof(SDropFunctionStmt)); + case QUERY_NODE_CREATE_STREAM_STMT: + return makeNode(type, sizeof(SCreateStreamStmt)); + case QUERY_NODE_DROP_STREAM_STMT: + return makeNode(type, sizeof(SDropStreamStmt)); + case QUERY_NODE_PAUSE_STREAM_STMT: + return makeNode(type, sizeof(SPauseStreamStmt)); + case QUERY_NODE_RESUME_STREAM_STMT: + return makeNode(type, sizeof(SResumeStreamStmt)); + case QUERY_NODE_BALANCE_VGROUP_STMT: + return makeNode(type, sizeof(SBalanceVgroupStmt)); + case QUERY_NODE_BALANCE_VGROUP_LEADER_STMT: + return makeNode(type, sizeof(SBalanceVgroupLeaderStmt)); + case QUERY_NODE_MERGE_VGROUP_STMT: + return makeNode(type, sizeof(SMergeVgroupStmt)); + case QUERY_NODE_REDISTRIBUTE_VGROUP_STMT: + return makeNode(type, sizeof(SRedistributeVgroupStmt)); + case QUERY_NODE_SPLIT_VGROUP_STMT: + return makeNode(type, sizeof(SSplitVgroupStmt)); + case QUERY_NODE_SYNCDB_STMT: + break; + case QUERY_NODE_GRANT_STMT: + return makeNode(type, sizeof(SGrantStmt)); + case QUERY_NODE_REVOKE_STMT: + return makeNode(type, sizeof(SRevokeStmt)); + case QUERY_NODE_SHOW_DNODES_STMT: + case QUERY_NODE_SHOW_MNODES_STMT: + case QUERY_NODE_SHOW_MODULES_STMT: + case QUERY_NODE_SHOW_QNODES_STMT: + case QUERY_NODE_SHOW_SNODES_STMT: + case QUERY_NODE_SHOW_BNODES_STMT: + case QUERY_NODE_SHOW_CLUSTER_STMT: + case QUERY_NODE_SHOW_DATABASES_STMT: + case QUERY_NODE_SHOW_FUNCTIONS_STMT: + case QUERY_NODE_SHOW_INDEXES_STMT: + case QUERY_NODE_SHOW_STABLES_STMT: + case QUERY_NODE_SHOW_STREAMS_STMT: + case QUERY_NODE_SHOW_TABLES_STMT: + case QUERY_NODE_SHOW_USERS_STMT: + case QUERY_NODE_SHOW_LICENCES_STMT: + case QUERY_NODE_SHOW_VGROUPS_STMT: + case QUERY_NODE_SHOW_TOPICS_STMT: + case QUERY_NODE_SHOW_CONSUMERS_STMT: + case QUERY_NODE_SHOW_CONNECTIONS_STMT: + case QUERY_NODE_SHOW_QUERIES_STMT: + case QUERY_NODE_SHOW_VNODES_STMT: + case QUERY_NODE_SHOW_APPS_STMT: + case QUERY_NODE_SHOW_SCORES_STMT: + case QUERY_NODE_SHOW_VARIABLES_STMT: + case QUERY_NODE_SHOW_LOCAL_VARIABLES_STMT: + case QUERY_NODE_SHOW_TRANSACTIONS_STMT: + case QUERY_NODE_SHOW_SUBSCRIPTIONS_STMT: + case QUERY_NODE_SHOW_TAGS_STMT: + case QUERY_NODE_SHOW_USER_PRIVILEGES_STMT: + case QUERY_NODE_SHOW_VIEWS_STMT: + return makeNode(type, sizeof(SShowStmt)); + case QUERY_NODE_SHOW_TABLE_TAGS_STMT: + return makeNode(type, sizeof(SShowTableTagsStmt)); + case QUERY_NODE_SHOW_DNODE_VARIABLES_STMT: + return makeNode(type, sizeof(SShowDnodeVariablesStmt)); + case QUERY_NODE_SHOW_CREATE_DATABASE_STMT: + return makeNode(type, sizeof(SShowCreateDatabaseStmt)); + case QUERY_NODE_SHOW_DB_ALIVE_STMT: + case QUERY_NODE_SHOW_CLUSTER_ALIVE_STMT: + return makeNode(type, sizeof(SShowAliveStmt)); + case QUERY_NODE_SHOW_CREATE_TABLE_STMT: + case QUERY_NODE_SHOW_CREATE_STABLE_STMT: + return makeNode(type, sizeof(SShowCreateTableStmt)); + case QUERY_NODE_SHOW_CREATE_VIEW_STMT: + return makeNode(type, sizeof(SShowCreateViewStmt)); + case QUERY_NODE_SHOW_TABLE_DISTRIBUTED_STMT: + return makeNode(type, sizeof(SShowTableDistributedStmt)); + case QUERY_NODE_KILL_QUERY_STMT: + return makeNode(type, sizeof(SKillQueryStmt)); + case QUERY_NODE_KILL_TRANSACTION_STMT: + case QUERY_NODE_KILL_CONNECTION_STMT: + return makeNode(type, sizeof(SKillStmt)); + case QUERY_NODE_DELETE_STMT: + return makeNode(type, sizeof(SDeleteStmt)); + case QUERY_NODE_INSERT_STMT: + return makeNode(type, sizeof(SInsertStmt)); + case QUERY_NODE_QUERY: + return makeNode(type, sizeof(SQuery)); + case QUERY_NODE_RESTORE_DNODE_STMT: + case QUERY_NODE_RESTORE_QNODE_STMT: + case QUERY_NODE_RESTORE_MNODE_STMT: + case QUERY_NODE_RESTORE_VNODE_STMT: + return makeNode(type, sizeof(SRestoreComponentNodeStmt)); + case QUERY_NODE_CREATE_VIEW_STMT: + return makeNode(type, sizeof(SCreateViewStmt)); + case QUERY_NODE_DROP_VIEW_STMT: + return makeNode(type, sizeof(SDropViewStmt)); + case QUERY_NODE_LOGIC_PLAN_SCAN: + return makeNode(type, sizeof(SScanLogicNode)); + case QUERY_NODE_LOGIC_PLAN_JOIN: + return makeNode(type, sizeof(SJoinLogicNode)); + case QUERY_NODE_LOGIC_PLAN_AGG: + return makeNode(type, sizeof(SAggLogicNode)); + case QUERY_NODE_LOGIC_PLAN_PROJECT: + return makeNode(type, sizeof(SProjectLogicNode)); + case QUERY_NODE_LOGIC_PLAN_VNODE_MODIFY: + return makeNode(type, sizeof(SVnodeModifyLogicNode)); + case QUERY_NODE_LOGIC_PLAN_EXCHANGE: + return makeNode(type, sizeof(SExchangeLogicNode)); + case QUERY_NODE_LOGIC_PLAN_MERGE: + return makeNode(type, sizeof(SMergeLogicNode)); + case QUERY_NODE_LOGIC_PLAN_WINDOW: + return makeNode(type, sizeof(SWindowLogicNode)); + case QUERY_NODE_LOGIC_PLAN_FILL: + return makeNode(type, sizeof(SFillLogicNode)); + case QUERY_NODE_LOGIC_PLAN_SORT: + return makeNode(type, sizeof(SSortLogicNode)); + case QUERY_NODE_LOGIC_PLAN_PARTITION: + return makeNode(type, sizeof(SPartitionLogicNode)); + case QUERY_NODE_LOGIC_PLAN_INDEF_ROWS_FUNC: + return makeNode(type, sizeof(SIndefRowsFuncLogicNode)); + case QUERY_NODE_LOGIC_PLAN_INTERP_FUNC: + return makeNode(type, sizeof(SInterpFuncLogicNode)); + case QUERY_NODE_LOGIC_PLAN_GROUP_CACHE: + return makeNode(type, sizeof(SGroupCacheLogicNode)); + case QUERY_NODE_LOGIC_PLAN_DYN_QUERY_CTRL: + return makeNode(type, sizeof(SDynQueryCtrlLogicNode)); + case QUERY_NODE_LOGIC_SUBPLAN: + return makeNode(type, sizeof(SLogicSubplan)); + case QUERY_NODE_LOGIC_PLAN: + return makeNode(type, sizeof(SQueryLogicPlan)); + case QUERY_NODE_PHYSICAL_PLAN_TAG_SCAN: + return makeNode(type, sizeof(STagScanPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_TABLE_SCAN: + return makeNode(type, sizeof(STableScanPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_TABLE_SEQ_SCAN: + return makeNode(type, sizeof(STableSeqScanPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_TABLE_MERGE_SCAN: + return makeNode(type, sizeof(STableMergeScanPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN: + return makeNode(type, sizeof(SStreamScanPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_SYSTABLE_SCAN: + return makeNode(type, sizeof(SSystemTableScanPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_BLOCK_DIST_SCAN: + return makeNode(type, sizeof(SBlockDistScanPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_LAST_ROW_SCAN: + return makeNode(type, sizeof(SLastRowScanPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_TABLE_COUNT_SCAN: + return makeNode(type, sizeof(STableCountScanPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_PROJECT: + return makeNode(type, sizeof(SProjectPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_MERGE_JOIN: + return makeNode(type, sizeof(SSortMergeJoinPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_HASH_JOIN: + return makeNode(type, sizeof(SHashJoinPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_HASH_AGG: + return makeNode(type, sizeof(SAggPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_EXCHANGE: + return makeNode(type, sizeof(SExchangePhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_MERGE: + return makeNode(type, sizeof(SMergePhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_SORT: + return makeNode(type, sizeof(SSortPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_GROUP_SORT: + return makeNode(type, sizeof(SGroupSortPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_HASH_INTERVAL: + return makeNode(type, sizeof(SIntervalPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_MERGE_ALIGNED_INTERVAL: + return makeNode(type, sizeof(SMergeAlignedIntervalPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL: + return makeNode(type, sizeof(SStreamIntervalPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL: + return makeNode(type, sizeof(SStreamFinalIntervalPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL: + return makeNode(type, sizeof(SStreamSemiIntervalPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_FILL: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_FILL: + return makeNode(type, sizeof(SFillPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_MERGE_SESSION: + return makeNode(type, sizeof(SSessionWinodwPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_STREAM_SESSION: + return makeNode(type, sizeof(SStreamSessionWinodwPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_SESSION: + return makeNode(type, sizeof(SStreamSemiSessionWinodwPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_SESSION: + return makeNode(type, sizeof(SStreamFinalSessionWinodwPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_MERGE_STATE: + return makeNode(type, sizeof(SStateWinodwPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_STREAM_STATE: + return makeNode(type, sizeof(SStreamStateWinodwPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_MERGE_EVENT: + return makeNode(type, sizeof(SEventWinodwPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_STREAM_EVENT: + return makeNode(type, sizeof(SStreamEventWinodwPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_PARTITION: + return makeNode(type, sizeof(SPartitionPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_STREAM_PARTITION: + return makeNode(type, sizeof(SStreamPartitionPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_INDEF_ROWS_FUNC: + return makeNode(type, sizeof(SIndefRowsFuncPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_INTERP_FUNC: + return makeNode(type, sizeof(SInterpFuncLogicNode)); + case QUERY_NODE_PHYSICAL_PLAN_DISPATCH: + return makeNode(type, sizeof(SDataDispatcherNode)); + case QUERY_NODE_PHYSICAL_PLAN_INSERT: + return makeNode(type, sizeof(SDataInserterNode)); + case QUERY_NODE_PHYSICAL_PLAN_QUERY_INSERT: + return makeNode(type, sizeof(SQueryInserterNode)); + case QUERY_NODE_PHYSICAL_PLAN_DELETE: + return makeNode(type, sizeof(SDataDeleterNode)); + case QUERY_NODE_PHYSICAL_PLAN_GROUP_CACHE: + return makeNode(type, sizeof(SGroupCachePhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_DYN_QUERY_CTRL: + return makeNode(type, sizeof(SDynQueryCtrlPhysiNode)); + case QUERY_NODE_PHYSICAL_SUBPLAN: + return makeNode(type, sizeof(SSubplan)); + case QUERY_NODE_PHYSICAL_PLAN: + return makeNode(type, sizeof(SQueryPlan)); + default: + break; } - nodesError("nodesMakeNode unsupported type = %d", type); + nodesError("nodesMakeNode unknown node = %s", nodesNodeName(type)); return NULL; } +static void destroyVgDataBlockArray(SArray* pArray) { + size_t size = taosArrayGetSize(pArray); + for (size_t i = 0; i < size; ++i) { + SVgDataBlocks* pVg = taosArrayGetP(pArray, i); + taosMemoryFreeClear(pVg->pData); + taosMemoryFreeClear(pVg); + } + taosArrayDestroy(pArray); +} + +static void destroyLogicNode(SLogicNode* pNode) { + nodesDestroyList(pNode->pTargets); + nodesDestroyNode(pNode->pConditions); + nodesDestroyList(pNode->pChildren); + nodesDestroyNode(pNode->pLimit); + nodesDestroyNode(pNode->pSlimit); + nodesDestroyList(pNode->pHint); +} + +static void destroyPhysiNode(SPhysiNode* pNode) { + nodesDestroyList(pNode->pChildren); + nodesDestroyNode(pNode->pConditions); + nodesDestroyNode((SNode*)pNode->pOutputDataBlockDesc); + nodesDestroyNode(pNode->pLimit); + nodesDestroyNode(pNode->pSlimit); +} + +static void destroyWinodwPhysiNode(SWindowPhysiNode* pNode) { + destroyPhysiNode((SPhysiNode*)pNode); + nodesDestroyList(pNode->pExprs); + nodesDestroyList(pNode->pFuncs); + nodesDestroyNode(pNode->pTspk); + nodesDestroyNode(pNode->pTsEnd); +} + +static void destroyPartitionPhysiNode(SPartitionPhysiNode* pNode) { + destroyPhysiNode((SPhysiNode*)pNode); + nodesDestroyList(pNode->pExprs); + nodesDestroyList(pNode->pPartitionKeys); + nodesDestroyList(pNode->pTargets); +} + +static void destroyScanPhysiNode(SScanPhysiNode* pNode) { + destroyPhysiNode((SPhysiNode*)pNode); + nodesDestroyList(pNode->pScanCols); + nodesDestroyList(pNode->pScanPseudoCols); +} + +static void destroyDataSinkNode(SDataSinkNode* pNode) { nodesDestroyNode((SNode*)pNode->pInputDataBlockDesc); } + +static void destroyExprNode(SExprNode* pExpr) { taosArrayDestroy(pExpr->pAssociation); } + +static void destroyTableCfg(STableCfg* pCfg) { + if (NULL == pCfg) { + return; + } + taosArrayDestroy(pCfg->pFuncs); + taosMemoryFree(pCfg->pComment); + taosMemoryFree(pCfg->pSchemas); + taosMemoryFree(pCfg->pTags); + taosMemoryFree(pCfg); +} + +static void destroySmaIndex(void* pIndex) { taosMemoryFree(((STableIndexInfo*)pIndex)->expr); } + +static void destroyHintValue(EHintOption option, void* value) { + switch (option) { + default: + break; + } + + taosMemoryFree(value); +} + +void nodesDestroyNode(SNode* pNode) { + if (NULL == pNode) { + return; + } + + switch (nodeType(pNode)) { + case QUERY_NODE_COLUMN: + destroyExprNode((SExprNode*)pNode); + break; + case QUERY_NODE_VALUE: { + SValueNode* pValue = (SValueNode*)pNode; + destroyExprNode((SExprNode*)pNode); + taosMemoryFreeClear(pValue->literal); + if (IS_VAR_DATA_TYPE(pValue->node.resType.type)) { + taosMemoryFreeClear(pValue->datum.p); + } + break; + } + case QUERY_NODE_OPERATOR: { + SOperatorNode* pOp = (SOperatorNode*)pNode; + destroyExprNode((SExprNode*)pNode); + nodesDestroyNode(pOp->pLeft); + nodesDestroyNode(pOp->pRight); + break; + } + case QUERY_NODE_LOGIC_CONDITION: + destroyExprNode((SExprNode*)pNode); + nodesDestroyList(((SLogicConditionNode*)pNode)->pParameterList); + break; + case QUERY_NODE_FUNCTION: + destroyExprNode((SExprNode*)pNode); + nodesDestroyList(((SFunctionNode*)pNode)->pParameterList); + break; + case QUERY_NODE_REAL_TABLE: { + SRealTableNode* pReal = (SRealTableNode*)pNode; + taosMemoryFreeClear(pReal->pMeta); + taosMemoryFreeClear(pReal->pVgroupList); + taosArrayDestroyEx(pReal->pSmaIndexes, destroySmaIndex); + break; + } + case QUERY_NODE_TEMP_TABLE: + nodesDestroyNode(((STempTableNode*)pNode)->pSubquery); + break; + case QUERY_NODE_JOIN_TABLE: { + SJoinTableNode* pJoin = (SJoinTableNode*)pNode; + nodesDestroyNode(pJoin->pLeft); + nodesDestroyNode(pJoin->pRight); + nodesDestroyNode(pJoin->pOnCond); + break; + } + case QUERY_NODE_GROUPING_SET: + nodesDestroyList(((SGroupingSetNode*)pNode)->pParameterList); + break; + case QUERY_NODE_ORDER_BY_EXPR: + nodesDestroyNode(((SOrderByExprNode*)pNode)->pExpr); + break; + case QUERY_NODE_LIMIT: // no pointer field + break; + case QUERY_NODE_STATE_WINDOW: { + SStateWindowNode* pState = (SStateWindowNode*)pNode; + nodesDestroyNode(pState->pCol); + nodesDestroyNode(pState->pExpr); + break; + } + case QUERY_NODE_SESSION_WINDOW: { + SSessionWindowNode* pSession = (SSessionWindowNode*)pNode; + nodesDestroyNode((SNode*)pSession->pCol); + nodesDestroyNode((SNode*)pSession->pGap); + break; + } + case QUERY_NODE_INTERVAL_WINDOW: { + SIntervalWindowNode* pJoin = (SIntervalWindowNode*)pNode; + nodesDestroyNode(pJoin->pCol); + nodesDestroyNode(pJoin->pInterval); + nodesDestroyNode(pJoin->pOffset); + nodesDestroyNode(pJoin->pSliding); + nodesDestroyNode(pJoin->pFill); + break; + } + case QUERY_NODE_NODE_LIST: + nodesDestroyList(((SNodeListNode*)pNode)->pNodeList); + break; + case QUERY_NODE_FILL: { + SFillNode* pFill = (SFillNode*)pNode; + nodesDestroyNode(pFill->pValues); + nodesDestroyNode(pFill->pWStartTs); + break; + } + case QUERY_NODE_RAW_EXPR: + nodesDestroyNode(((SRawExprNode*)pNode)->pNode); + break; + case QUERY_NODE_TARGET: + nodesDestroyNode(((STargetNode*)pNode)->pExpr); + break; + case QUERY_NODE_DATABLOCK_DESC: + nodesDestroyList(((SDataBlockDescNode*)pNode)->pSlots); + break; + case QUERY_NODE_SLOT_DESC: // no pointer field + case QUERY_NODE_COLUMN_DEF: // no pointer field + case QUERY_NODE_DOWNSTREAM_SOURCE: // no pointer field + break; + case QUERY_NODE_DATABASE_OPTIONS: { + SDatabaseOptions* pOptions = (SDatabaseOptions*)pNode; + nodesDestroyNode((SNode*)pOptions->pDaysPerFile); + nodesDestroyList(pOptions->pKeep); + nodesDestroyList(pOptions->pRetentions); + break; + } + case QUERY_NODE_TABLE_OPTIONS: { + STableOptions* pOptions = (STableOptions*)pNode; + nodesDestroyList(pOptions->pMaxDelay); + nodesDestroyList(pOptions->pWatermark); + nodesDestroyList(pOptions->pRollupFuncs); + nodesDestroyList(pOptions->pSma); + nodesDestroyList(pOptions->pDeleteMark); + break; + } + case QUERY_NODE_INDEX_OPTIONS: { + SIndexOptions* pOptions = (SIndexOptions*)pNode; + nodesDestroyList(pOptions->pFuncs); + nodesDestroyNode(pOptions->pInterval); + nodesDestroyNode(pOptions->pOffset); + nodesDestroyNode(pOptions->pSliding); + nodesDestroyNode(pOptions->pStreamOptions); + break; + } + case QUERY_NODE_EXPLAIN_OPTIONS: // no pointer field + break; + case QUERY_NODE_STREAM_OPTIONS: { + SStreamOptions* pOptions = (SStreamOptions*)pNode; + nodesDestroyNode(pOptions->pDelay); + nodesDestroyNode(pOptions->pWatermark); + nodesDestroyNode(pOptions->pDeleteMark); + break; + } + case QUERY_NODE_LEFT_VALUE: // no pointer field + case QUERY_NODE_COLUMN_REF: // no pointer field + break; + case QUERY_NODE_WHEN_THEN: { + SWhenThenNode* pWhenThen = (SWhenThenNode*)pNode; + nodesDestroyNode(pWhenThen->pWhen); + nodesDestroyNode(pWhenThen->pThen); + break; + } + case QUERY_NODE_CASE_WHEN: { + SCaseWhenNode* pCaseWhen = (SCaseWhenNode*)pNode; + nodesDestroyNode(pCaseWhen->pCase); + nodesDestroyNode(pCaseWhen->pElse); + nodesDestroyList(pCaseWhen->pWhenThenList); + break; + } + case QUERY_NODE_EVENT_WINDOW: { + SEventWindowNode* pEvent = (SEventWindowNode*)pNode; + nodesDestroyNode(pEvent->pCol); + nodesDestroyNode(pEvent->pStartCond); + nodesDestroyNode(pEvent->pEndCond); + break; + } + case QUERY_NODE_HINT: { + SHintNode* pHint = (SHintNode*)pNode; + destroyHintValue(pHint->option, pHint->value); + break; + } + case QUERY_NODE_VIEW: { + SViewNode* pView = (SViewNode*)pNode; + taosMemoryFreeClear(pView->pMeta); + taosMemoryFreeClear(pView->pVgroupList); + taosArrayDestroyEx(pView->pSmaIndexes, destroySmaIndex); + break; + } + case QUERY_NODE_SET_OPERATOR: { + SSetOperator* pStmt = (SSetOperator*)pNode; + nodesDestroyList(pStmt->pProjectionList); + nodesDestroyNode(pStmt->pLeft); + nodesDestroyNode(pStmt->pRight); + nodesDestroyList(pStmt->pOrderByList); + nodesDestroyNode(pStmt->pLimit); + break; + } + case QUERY_NODE_SELECT_STMT: { + SSelectStmt* pStmt = (SSelectStmt*)pNode; + nodesDestroyList(pStmt->pProjectionList); + nodesDestroyNode(pStmt->pFromTable); + nodesDestroyNode(pStmt->pWhere); + nodesDestroyList(pStmt->pPartitionByList); + nodesDestroyList(pStmt->pTags); + nodesDestroyNode(pStmt->pSubtable); + nodesDestroyNode(pStmt->pWindow); + nodesDestroyList(pStmt->pGroupByList); + nodesDestroyNode(pStmt->pHaving); + nodesDestroyNode(pStmt->pRange); + nodesDestroyNode(pStmt->pEvery); + nodesDestroyNode(pStmt->pFill); + nodesDestroyList(pStmt->pOrderByList); + nodesDestroyNode((SNode*)pStmt->pLimit); + nodesDestroyNode((SNode*)pStmt->pSlimit); + nodesDestroyList(pStmt->pHint); + break; + } + case QUERY_NODE_VNODE_MODIFY_STMT: { + SVnodeModifyOpStmt* pStmt = (SVnodeModifyOpStmt*)pNode; + destroyVgDataBlockArray(pStmt->pDataBlocks); + taosMemoryFreeClear(pStmt->pTableMeta); + nodesDestroyNode(pStmt->pTagCond); + taosArrayDestroy(pStmt->pTableTag); + taosHashCleanup(pStmt->pVgroupsHashObj); + taosHashCleanup(pStmt->pSubTableHashObj); + taosHashCleanup(pStmt->pTableNameHashObj); + taosHashCleanup(pStmt->pDbFNameHashObj); + taosHashCleanup(pStmt->pTableCxtHashObj); + if (pStmt->freeHashFunc) { + pStmt->freeHashFunc(pStmt->pTableBlockHashObj); + } + if (pStmt->freeArrayFunc) { + pStmt->freeArrayFunc(pStmt->pVgDataBlocks); + } + tdDestroySVCreateTbReq(pStmt->pCreateTblReq); + taosMemoryFreeClear(pStmt->pCreateTblReq); + if (pStmt->freeStbRowsCxtFunc) { + pStmt->freeStbRowsCxtFunc(pStmt->pStbRowsCxt); + } + taosMemoryFreeClear(pStmt->pStbRowsCxt); + taosCloseFile(&pStmt->fp); + break; + } + case QUERY_NODE_CREATE_DATABASE_STMT: + nodesDestroyNode((SNode*)((SCreateDatabaseStmt*)pNode)->pOptions); + break; + case QUERY_NODE_DROP_DATABASE_STMT: // no pointer field + break; + case QUERY_NODE_ALTER_DATABASE_STMT: + nodesDestroyNode((SNode*)((SAlterDatabaseStmt*)pNode)->pOptions); + break; + case QUERY_NODE_FLUSH_DATABASE_STMT: // no pointer field + case QUERY_NODE_TRIM_DATABASE_STMT: // no pointer field + break; + case QUERY_NODE_CREATE_TABLE_STMT: { + SCreateTableStmt* pStmt = (SCreateTableStmt*)pNode; + nodesDestroyList(pStmt->pCols); + nodesDestroyList(pStmt->pTags); + nodesDestroyNode((SNode*)pStmt->pOptions); + break; + } + case QUERY_NODE_CREATE_SUBTABLE_CLAUSE: { + SCreateSubTableClause* pStmt = (SCreateSubTableClause*)pNode; + nodesDestroyList(pStmt->pSpecificTags); + nodesDestroyList(pStmt->pValsOfTags); + nodesDestroyNode((SNode*)pStmt->pOptions); + break; + } + case QUERY_NODE_CREATE_MULTI_TABLES_STMT: + nodesDestroyList(((SCreateMultiTablesStmt*)pNode)->pSubTables); + break; + case QUERY_NODE_DROP_TABLE_CLAUSE: // no pointer field + break; + case QUERY_NODE_DROP_TABLE_STMT: + nodesDestroyList(((SDropTableStmt*)pNode)->pTables); + break; + case QUERY_NODE_DROP_SUPER_TABLE_STMT: // no pointer field + break; + case QUERY_NODE_ALTER_TABLE_STMT: + case QUERY_NODE_ALTER_SUPER_TABLE_STMT: { + SAlterTableStmt* pStmt = (SAlterTableStmt*)pNode; + nodesDestroyNode((SNode*)pStmt->pOptions); + nodesDestroyNode((SNode*)pStmt->pVal); + break; + } + case QUERY_NODE_CREATE_USER_STMT: { + SCreateUserStmt* pStmt = (SCreateUserStmt*)pNode; + taosMemoryFree(pStmt->pIpRanges); + nodesDestroyList(pStmt->pNodeListIpRanges); + break; + } + case QUERY_NODE_ALTER_USER_STMT: { + SAlterUserStmt* pStmt = (SAlterUserStmt*)pNode; + taosMemoryFree(pStmt->pIpRanges); + nodesDestroyList(pStmt->pNodeListIpRanges); + } + case QUERY_NODE_DROP_USER_STMT: // no pointer field + case QUERY_NODE_USE_DATABASE_STMT: // no pointer field + case QUERY_NODE_CREATE_DNODE_STMT: // no pointer field + case QUERY_NODE_DROP_DNODE_STMT: // no pointer field + case QUERY_NODE_ALTER_DNODE_STMT: // no pointer field + break; + case QUERY_NODE_CREATE_INDEX_STMT: { + SCreateIndexStmt* pStmt = (SCreateIndexStmt*)pNode; + nodesDestroyNode((SNode*)pStmt->pOptions); + nodesDestroyList(pStmt->pCols); + if (pStmt->pReq) { + tFreeSMCreateSmaReq(pStmt->pReq); + taosMemoryFreeClear(pStmt->pReq); + } + break; + } + case QUERY_NODE_DROP_INDEX_STMT: // no pointer field + case QUERY_NODE_CREATE_QNODE_STMT: // no pointer field + case QUERY_NODE_DROP_QNODE_STMT: // no pointer field + case QUERY_NODE_CREATE_BNODE_STMT: // no pointer field + case QUERY_NODE_DROP_BNODE_STMT: // no pointer field + case QUERY_NODE_CREATE_SNODE_STMT: // no pointer field + case QUERY_NODE_DROP_SNODE_STMT: // no pointer field + case QUERY_NODE_CREATE_MNODE_STMT: // no pointer field + case QUERY_NODE_DROP_MNODE_STMT: // no pointer field + break; + case QUERY_NODE_CREATE_TOPIC_STMT: + nodesDestroyNode(((SCreateTopicStmt*)pNode)->pQuery); + nodesDestroyNode(((SCreateTopicStmt*)pNode)->pWhere); + break; + case QUERY_NODE_DROP_TOPIC_STMT: // no pointer field + case QUERY_NODE_DROP_CGROUP_STMT: // no pointer field + case QUERY_NODE_ALTER_LOCAL_STMT: // no pointer field + break; + case QUERY_NODE_EXPLAIN_STMT: { + SExplainStmt* pStmt = (SExplainStmt*)pNode; + nodesDestroyNode((SNode*)pStmt->pOptions); + nodesDestroyNode(pStmt->pQuery); + break; + } + case QUERY_NODE_DESCRIBE_STMT: + taosMemoryFree(((SDescribeStmt*)pNode)->pMeta); + break; + case QUERY_NODE_RESET_QUERY_CACHE_STMT: // no pointer field + break; + case QUERY_NODE_COMPACT_DATABASE_STMT: { + SCompactDatabaseStmt* pStmt = (SCompactDatabaseStmt*)pNode; + nodesDestroyNode(pStmt->pStart); + nodesDestroyNode(pStmt->pEnd); + break; + } + case QUERY_NODE_CREATE_FUNCTION_STMT: // no pointer field + case QUERY_NODE_DROP_FUNCTION_STMT: // no pointer field + break; + case QUERY_NODE_CREATE_STREAM_STMT: { + SCreateStreamStmt* pStmt = (SCreateStreamStmt*)pNode; + nodesDestroyNode((SNode*)pStmt->pOptions); + nodesDestroyNode(pStmt->pQuery); + nodesDestroyList(pStmt->pTags); + nodesDestroyNode(pStmt->pSubtable); + tFreeSCMCreateStreamReq(pStmt->pReq); + taosMemoryFreeClear(pStmt->pReq); + break; + } + case QUERY_NODE_DROP_STREAM_STMT: // no pointer field + case QUERY_NODE_PAUSE_STREAM_STMT: // no pointer field + case QUERY_NODE_RESUME_STREAM_STMT: // no pointer field + case QUERY_NODE_BALANCE_VGROUP_STMT: // no pointer field + case QUERY_NODE_BALANCE_VGROUP_LEADER_STMT: // no pointer field + case QUERY_NODE_MERGE_VGROUP_STMT: // no pointer field + break; + case QUERY_NODE_REDISTRIBUTE_VGROUP_STMT: + nodesDestroyList(((SRedistributeVgroupStmt*)pNode)->pDnodes); + break; + case QUERY_NODE_SPLIT_VGROUP_STMT: // no pointer field + case QUERY_NODE_SYNCDB_STMT: // no pointer field + break; + case QUERY_NODE_GRANT_STMT: + nodesDestroyNode(((SGrantStmt*)pNode)->pTagCond); + break; + case QUERY_NODE_REVOKE_STMT: + nodesDestroyNode(((SRevokeStmt*)pNode)->pTagCond); + break; + case QUERY_NODE_SHOW_DNODES_STMT: + case QUERY_NODE_SHOW_MNODES_STMT: + case QUERY_NODE_SHOW_MODULES_STMT: + case QUERY_NODE_SHOW_QNODES_STMT: + case QUERY_NODE_SHOW_SNODES_STMT: + case QUERY_NODE_SHOW_BNODES_STMT: + case QUERY_NODE_SHOW_CLUSTER_STMT: + case QUERY_NODE_SHOW_DATABASES_STMT: + case QUERY_NODE_SHOW_FUNCTIONS_STMT: + case QUERY_NODE_SHOW_INDEXES_STMT: + case QUERY_NODE_SHOW_STABLES_STMT: + case QUERY_NODE_SHOW_STREAMS_STMT: + case QUERY_NODE_SHOW_TABLES_STMT: + case QUERY_NODE_SHOW_USERS_STMT: + case QUERY_NODE_SHOW_LICENCES_STMT: + case QUERY_NODE_SHOW_VGROUPS_STMT: + case QUERY_NODE_SHOW_TOPICS_STMT: + case QUERY_NODE_SHOW_CONSUMERS_STMT: + case QUERY_NODE_SHOW_CONNECTIONS_STMT: + case QUERY_NODE_SHOW_QUERIES_STMT: + case QUERY_NODE_SHOW_VNODES_STMT: + case QUERY_NODE_SHOW_APPS_STMT: + case QUERY_NODE_SHOW_SCORES_STMT: + case QUERY_NODE_SHOW_VARIABLES_STMT: + case QUERY_NODE_SHOW_LOCAL_VARIABLES_STMT: + case QUERY_NODE_SHOW_TRANSACTIONS_STMT: + case QUERY_NODE_SHOW_SUBSCRIPTIONS_STMT: + case QUERY_NODE_SHOW_TAGS_STMT: + case QUERY_NODE_SHOW_USER_PRIVILEGES_STMT: + case QUERY_NODE_SHOW_VIEWS_STMT: { + SShowStmt* pStmt = (SShowStmt*)pNode; + nodesDestroyNode(pStmt->pDbName); + nodesDestroyNode(pStmt->pTbName); + break; + } + case QUERY_NODE_SHOW_TABLE_TAGS_STMT: { + SShowTableTagsStmt* pStmt = (SShowTableTagsStmt*)pNode; + nodesDestroyNode(pStmt->pDbName); + nodesDestroyNode(pStmt->pTbName); + nodesDestroyList(pStmt->pTags); + break; + } + case QUERY_NODE_SHOW_DNODE_VARIABLES_STMT: + nodesDestroyNode(((SShowDnodeVariablesStmt*)pNode)->pDnodeId); + nodesDestroyNode(((SShowDnodeVariablesStmt*)pNode)->pLikePattern); + break; + case QUERY_NODE_SHOW_CREATE_DATABASE_STMT: + taosMemoryFreeClear(((SShowCreateDatabaseStmt*)pNode)->pCfg); + break; + case QUERY_NODE_SHOW_CREATE_TABLE_STMT: + case QUERY_NODE_SHOW_CREATE_STABLE_STMT: + taosMemoryFreeClear(((SShowCreateTableStmt*)pNode)->pDbCfg); + destroyTableCfg((STableCfg*)(((SShowCreateTableStmt*)pNode)->pTableCfg)); + break; + case QUERY_NODE_SHOW_CREATE_VIEW_STMT: // no pointer field + case QUERY_NODE_SHOW_TABLE_DISTRIBUTED_STMT: // no pointer field + case QUERY_NODE_KILL_CONNECTION_STMT: // no pointer field + case QUERY_NODE_KILL_QUERY_STMT: // no pointer field + case QUERY_NODE_KILL_TRANSACTION_STMT: // no pointer field + break; + case QUERY_NODE_DELETE_STMT: { + SDeleteStmt* pStmt = (SDeleteStmt*)pNode; + nodesDestroyNode(pStmt->pFromTable); + nodesDestroyNode(pStmt->pWhere); + nodesDestroyNode(pStmt->pCountFunc); + nodesDestroyNode(pStmt->pFirstFunc); + nodesDestroyNode(pStmt->pLastFunc); + nodesDestroyNode(pStmt->pTagCond); + break; + } + case QUERY_NODE_INSERT_STMT: { + SInsertStmt* pStmt = (SInsertStmt*)pNode; + nodesDestroyNode(pStmt->pTable); + nodesDestroyList(pStmt->pCols); + nodesDestroyNode(pStmt->pQuery); + break; + } + case QUERY_NODE_QUERY: { + SQuery* pQuery = (SQuery*)pNode; + nodesDestroyNode(pQuery->pPrevRoot); + nodesDestroyNode(pQuery->pRoot); + nodesDestroyNode(pQuery->pPostRoot); + taosMemoryFreeClear(pQuery->pResSchema); + if (NULL != pQuery->pCmdMsg) { + taosMemoryFreeClear(pQuery->pCmdMsg->pMsg); + taosMemoryFreeClear(pQuery->pCmdMsg); + } + taosArrayDestroy(pQuery->pDbList); + taosArrayDestroy(pQuery->pTableList); + taosArrayDestroy(pQuery->pTargetTableList); + taosArrayDestroy(pQuery->pPlaceholderValues); + nodesDestroyNode(pQuery->pPrepareRoot); + break; + } + case QUERY_NODE_RESTORE_DNODE_STMT: // no pointer field + case QUERY_NODE_RESTORE_QNODE_STMT: // no pointer field + case QUERY_NODE_RESTORE_MNODE_STMT: // no pointer field + case QUERY_NODE_RESTORE_VNODE_STMT: // no pointer field + break; + case QUERY_NODE_CREATE_VIEW_STMT: { + SCreateViewStmt* pStmt = (SCreateViewStmt*)pNode; + taosMemoryFree(pStmt->pQuerySql); + tFreeSCMCreateViewReq(&pStmt->createReq); + nodesDestroyNode(pStmt->pQuery); + break; + } + case QUERY_NODE_DROP_VIEW_STMT: + break; + case QUERY_NODE_LOGIC_PLAN_SCAN: { + SScanLogicNode* pLogicNode = (SScanLogicNode*)pNode; + destroyLogicNode((SLogicNode*)pLogicNode); + nodesDestroyList(pLogicNode->pScanCols); + nodesDestroyList(pLogicNode->pScanPseudoCols); + taosMemoryFreeClear(pLogicNode->pVgroupList); + nodesDestroyList(pLogicNode->pDynamicScanFuncs); + nodesDestroyNode(pLogicNode->pTagCond); + nodesDestroyNode(pLogicNode->pTagIndexCond); + taosArrayDestroyEx(pLogicNode->pSmaIndexes, destroySmaIndex); + nodesDestroyList(pLogicNode->pGroupTags); + nodesDestroyList(pLogicNode->pTags); + nodesDestroyNode(pLogicNode->pSubtable); + break; + } + case QUERY_NODE_LOGIC_PLAN_JOIN: { + SJoinLogicNode* pLogicNode = (SJoinLogicNode*)pNode; + destroyLogicNode((SLogicNode*)pLogicNode); + nodesDestroyNode(pLogicNode->pPrimKeyEqCond); + nodesDestroyNode(pLogicNode->pOtherOnCond); + nodesDestroyNode(pLogicNode->pColEqCond); + break; + } + case QUERY_NODE_LOGIC_PLAN_AGG: { + SAggLogicNode* pLogicNode = (SAggLogicNode*)pNode; + destroyLogicNode((SLogicNode*)pLogicNode); + nodesDestroyList(pLogicNode->pAggFuncs); + nodesDestroyList(pLogicNode->pGroupKeys); + break; + } + case QUERY_NODE_LOGIC_PLAN_PROJECT: { + SProjectLogicNode* pLogicNode = (SProjectLogicNode*)pNode; + destroyLogicNode((SLogicNode*)pLogicNode); + nodesDestroyList(pLogicNode->pProjections); + break; + } + case QUERY_NODE_LOGIC_PLAN_VNODE_MODIFY: { + SVnodeModifyLogicNode* pLogicNode = (SVnodeModifyLogicNode*)pNode; + destroyLogicNode((SLogicNode*)pLogicNode); + destroyVgDataBlockArray(pLogicNode->pDataBlocks); + // pVgDataBlocks is weak reference + nodesDestroyNode(pLogicNode->pAffectedRows); + nodesDestroyNode(pLogicNode->pStartTs); + nodesDestroyNode(pLogicNode->pEndTs); + taosMemoryFreeClear(pLogicNode->pVgroupList); + nodesDestroyList(pLogicNode->pInsertCols); + break; + } + case QUERY_NODE_LOGIC_PLAN_EXCHANGE: + destroyLogicNode((SLogicNode*)pNode); + break; + case QUERY_NODE_LOGIC_PLAN_MERGE: { + SMergeLogicNode* pLogicNode = (SMergeLogicNode*)pNode; + destroyLogicNode((SLogicNode*)pLogicNode); + nodesDestroyList(pLogicNode->pMergeKeys); + nodesDestroyList(pLogicNode->pInputs); + break; + } + case QUERY_NODE_LOGIC_PLAN_WINDOW: { + SWindowLogicNode* pLogicNode = (SWindowLogicNode*)pNode; + destroyLogicNode((SLogicNode*)pLogicNode); + nodesDestroyList(pLogicNode->pFuncs); + nodesDestroyNode(pLogicNode->pTspk); + nodesDestroyNode(pLogicNode->pTsEnd); + nodesDestroyNode(pLogicNode->pStateExpr); + nodesDestroyNode(pLogicNode->pStartCond); + nodesDestroyNode(pLogicNode->pEndCond); + break; + } + case QUERY_NODE_LOGIC_PLAN_FILL: { + SFillLogicNode* pLogicNode = (SFillLogicNode*)pNode; + destroyLogicNode((SLogicNode*)pLogicNode); + nodesDestroyNode(pLogicNode->pWStartTs); + nodesDestroyNode(pLogicNode->pValues); + nodesDestroyList(pLogicNode->pFillExprs); + nodesDestroyList(pLogicNode->pNotFillExprs); + break; + } + case QUERY_NODE_LOGIC_PLAN_SORT: { + SSortLogicNode* pLogicNode = (SSortLogicNode*)pNode; + destroyLogicNode((SLogicNode*)pLogicNode); + nodesDestroyList(pLogicNode->pSortKeys); + break; + } + case QUERY_NODE_LOGIC_PLAN_PARTITION: { + SPartitionLogicNode* pLogicNode = (SPartitionLogicNode*)pNode; + destroyLogicNode((SLogicNode*)pLogicNode); + nodesDestroyList(pLogicNode->pPartitionKeys); + nodesDestroyList(pLogicNode->pTags); + nodesDestroyNode(pLogicNode->pSubtable); + nodesDestroyList(pLogicNode->pAggFuncs); + break; + } + case QUERY_NODE_LOGIC_PLAN_INDEF_ROWS_FUNC: { + SIndefRowsFuncLogicNode* pLogicNode = (SIndefRowsFuncLogicNode*)pNode; + destroyLogicNode((SLogicNode*)pLogicNode); + nodesDestroyList(pLogicNode->pFuncs); + break; + } + case QUERY_NODE_LOGIC_PLAN_INTERP_FUNC: { + SInterpFuncLogicNode* pLogicNode = (SInterpFuncLogicNode*)pNode; + destroyLogicNode((SLogicNode*)pLogicNode); + nodesDestroyList(pLogicNode->pFuncs); + nodesDestroyNode(pLogicNode->pFillValues); + nodesDestroyNode(pLogicNode->pTimeSeries); + break; + } + case QUERY_NODE_LOGIC_PLAN_GROUP_CACHE: { + SGroupCacheLogicNode* pLogicNode = (SGroupCacheLogicNode*)pNode; + destroyLogicNode((SLogicNode*)pLogicNode); + nodesDestroyList(pLogicNode->pGroupCols); + break; + } + case QUERY_NODE_LOGIC_PLAN_DYN_QUERY_CTRL: { + SDynQueryCtrlLogicNode* pLogicNode = (SDynQueryCtrlLogicNode*)pNode; + destroyLogicNode((SLogicNode*)pLogicNode); + break; + } + case QUERY_NODE_LOGIC_SUBPLAN: { + SLogicSubplan* pSubplan = (SLogicSubplan*)pNode; + nodesDestroyList(pSubplan->pChildren); + nodesDestroyNode((SNode*)pSubplan->pNode); + nodesClearList(pSubplan->pParents); + taosMemoryFreeClear(pSubplan->pVgroupList); + break; + } + case QUERY_NODE_LOGIC_PLAN: + nodesDestroyList(((SQueryLogicPlan*)pNode)->pTopSubplans); + break; + case QUERY_NODE_PHYSICAL_PLAN_TAG_SCAN: + case QUERY_NODE_PHYSICAL_PLAN_SYSTABLE_SCAN: + case QUERY_NODE_PHYSICAL_PLAN_BLOCK_DIST_SCAN: + destroyScanPhysiNode((SScanPhysiNode*)pNode); + break; + case QUERY_NODE_PHYSICAL_PLAN_LAST_ROW_SCAN: + case QUERY_NODE_PHYSICAL_PLAN_TABLE_COUNT_SCAN: { + SLastRowScanPhysiNode* pPhyNode = (SLastRowScanPhysiNode*)pNode; + destroyScanPhysiNode((SScanPhysiNode*)pNode); + nodesDestroyList(pPhyNode->pGroupTags); + nodesDestroyList(pPhyNode->pTargets); + break; + } + case QUERY_NODE_PHYSICAL_PLAN_TABLE_SCAN: + case QUERY_NODE_PHYSICAL_PLAN_TABLE_SEQ_SCAN: + case QUERY_NODE_PHYSICAL_PLAN_TABLE_MERGE_SCAN: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN: { + STableScanPhysiNode* pPhyNode = (STableScanPhysiNode*)pNode; + destroyScanPhysiNode((SScanPhysiNode*)pNode); + nodesDestroyList(pPhyNode->pDynamicScanFuncs); + nodesDestroyList(pPhyNode->pGroupTags); + nodesDestroyList(pPhyNode->pTags); + nodesDestroyNode(pPhyNode->pSubtable); + break; + } + case QUERY_NODE_PHYSICAL_PLAN_PROJECT: { + SProjectPhysiNode* pPhyNode = (SProjectPhysiNode*)pNode; + destroyPhysiNode((SPhysiNode*)pPhyNode); + nodesDestroyList(pPhyNode->pProjections); + break; + } + case QUERY_NODE_PHYSICAL_PLAN_MERGE_JOIN: { + SSortMergeJoinPhysiNode* pPhyNode = (SSortMergeJoinPhysiNode*)pNode; + destroyPhysiNode((SPhysiNode*)pPhyNode); + nodesDestroyNode(pPhyNode->pPrimKeyCond); + nodesDestroyNode(pPhyNode->pOtherOnCond); + nodesDestroyList(pPhyNode->pTargets); + nodesDestroyNode(pPhyNode->pColEqCond); + break; + } + case QUERY_NODE_PHYSICAL_PLAN_HASH_JOIN: { + SHashJoinPhysiNode* pPhyNode = (SHashJoinPhysiNode*)pNode; + destroyPhysiNode((SPhysiNode*)pPhyNode); + nodesDestroyList(pPhyNode->pOnLeft); + nodesDestroyList(pPhyNode->pOnRight); + nodesDestroyNode(pPhyNode->pFilterConditions); + nodesDestroyList(pPhyNode->pTargets); + + nodesDestroyNode(pPhyNode->pPrimKeyCond); + nodesDestroyNode(pPhyNode->pColEqCond); + nodesDestroyNode(pPhyNode->pTagEqCond); + break; + } + case QUERY_NODE_PHYSICAL_PLAN_HASH_AGG: { + SAggPhysiNode* pPhyNode = (SAggPhysiNode*)pNode; + destroyPhysiNode((SPhysiNode*)pPhyNode); + nodesDestroyList(pPhyNode->pExprs); + nodesDestroyList(pPhyNode->pAggFuncs); + nodesDestroyList(pPhyNode->pGroupKeys); + break; + } + case QUERY_NODE_PHYSICAL_PLAN_EXCHANGE: { + SExchangePhysiNode* pPhyNode = (SExchangePhysiNode*)pNode; + destroyPhysiNode((SPhysiNode*)pPhyNode); + nodesDestroyList(pPhyNode->pSrcEndPoints); + break; + } + case QUERY_NODE_PHYSICAL_PLAN_MERGE: { + SMergePhysiNode* pPhyNode = (SMergePhysiNode*)pNode; + destroyPhysiNode((SPhysiNode*)pPhyNode); + nodesDestroyList(pPhyNode->pMergeKeys); + nodesDestroyList(pPhyNode->pTargets); + break; + } + case QUERY_NODE_PHYSICAL_PLAN_SORT: + case QUERY_NODE_PHYSICAL_PLAN_GROUP_SORT: { + SSortPhysiNode* pPhyNode = (SSortPhysiNode*)pNode; + destroyPhysiNode((SPhysiNode*)pPhyNode); + nodesDestroyList(pPhyNode->pExprs); + nodesDestroyList(pPhyNode->pSortKeys); + nodesDestroyList(pPhyNode->pTargets); + break; + } + case QUERY_NODE_PHYSICAL_PLAN_HASH_INTERVAL: + case QUERY_NODE_PHYSICAL_PLAN_MERGE_ALIGNED_INTERVAL: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL: + destroyWinodwPhysiNode((SWindowPhysiNode*)pNode); + break; + case QUERY_NODE_PHYSICAL_PLAN_FILL: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_FILL: { + SFillPhysiNode* pPhyNode = (SFillPhysiNode*)pNode; + destroyPhysiNode((SPhysiNode*)pPhyNode); + nodesDestroyList(pPhyNode->pFillExprs); + nodesDestroyList(pPhyNode->pNotFillExprs); + nodesDestroyNode(pPhyNode->pWStartTs); + nodesDestroyNode(pPhyNode->pValues); + break; + } + case QUERY_NODE_PHYSICAL_PLAN_MERGE_SESSION: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_SESSION: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_SESSION: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_SESSION: + destroyWinodwPhysiNode((SWindowPhysiNode*)pNode); + break; + case QUERY_NODE_PHYSICAL_PLAN_MERGE_STATE: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_STATE: { + SStateWinodwPhysiNode* pPhyNode = (SStateWinodwPhysiNode*)pNode; + destroyWinodwPhysiNode((SWindowPhysiNode*)pPhyNode); + nodesDestroyNode(pPhyNode->pStateKey); + break; + } + case QUERY_NODE_PHYSICAL_PLAN_MERGE_EVENT: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_EVENT: { + SEventWinodwPhysiNode* pPhyNode = (SEventWinodwPhysiNode*)pNode; + destroyWinodwPhysiNode((SWindowPhysiNode*)pPhyNode); + nodesDestroyNode(pPhyNode->pStartCond); + nodesDestroyNode(pPhyNode->pEndCond); + break; + } + case QUERY_NODE_PHYSICAL_PLAN_PARTITION: { + destroyPartitionPhysiNode((SPartitionPhysiNode*)pNode); + break; + } + case QUERY_NODE_PHYSICAL_PLAN_STREAM_PARTITION: { + SStreamPartitionPhysiNode* pPhyNode = (SStreamPartitionPhysiNode*)pNode; + destroyPartitionPhysiNode((SPartitionPhysiNode*)pPhyNode); + nodesDestroyList(pPhyNode->pTags); + nodesDestroyNode(pPhyNode->pSubtable); + break; + } + case QUERY_NODE_PHYSICAL_PLAN_INDEF_ROWS_FUNC: { + SIndefRowsFuncPhysiNode* pPhyNode = (SIndefRowsFuncPhysiNode*)pNode; + destroyPhysiNode((SPhysiNode*)pPhyNode); + nodesDestroyList(pPhyNode->pExprs); + nodesDestroyList(pPhyNode->pFuncs); + break; + } + case QUERY_NODE_PHYSICAL_PLAN_INTERP_FUNC: { + SInterpFuncPhysiNode* pPhyNode = (SInterpFuncPhysiNode*)pNode; + destroyPhysiNode((SPhysiNode*)pPhyNode); + nodesDestroyList(pPhyNode->pExprs); + nodesDestroyList(pPhyNode->pFuncs); + nodesDestroyNode(pPhyNode->pFillValues); + nodesDestroyNode(pPhyNode->pTimeSeries); + break; + } + case QUERY_NODE_PHYSICAL_PLAN_DISPATCH: + destroyDataSinkNode((SDataSinkNode*)pNode); + break; + case QUERY_NODE_PHYSICAL_PLAN_INSERT: { + SDataInserterNode* pSink = (SDataInserterNode*)pNode; + destroyDataSinkNode((SDataSinkNode*)pSink); + taosMemoryFreeClear(pSink->pData); + break; + } + case QUERY_NODE_PHYSICAL_PLAN_QUERY_INSERT: { + SQueryInserterNode* pSink = (SQueryInserterNode*)pNode; + destroyDataSinkNode((SDataSinkNode*)pSink); + nodesDestroyList(pSink->pCols); + break; + } + case QUERY_NODE_PHYSICAL_PLAN_DELETE: { + SDataDeleterNode* pSink = (SDataDeleterNode*)pNode; + destroyDataSinkNode((SDataSinkNode*)pSink); + nodesDestroyNode(pSink->pAffectedRows); + nodesDestroyNode(pSink->pStartTs); + nodesDestroyNode(pSink->pEndTs); + break; + } + case QUERY_NODE_PHYSICAL_PLAN_GROUP_CACHE: { + SGroupCachePhysiNode* pPhyNode = (SGroupCachePhysiNode*)pNode; + destroyPhysiNode((SPhysiNode*)pPhyNode); + nodesDestroyList(pPhyNode->pGroupCols); + break; + } + case QUERY_NODE_PHYSICAL_PLAN_DYN_QUERY_CTRL: { + SDynQueryCtrlPhysiNode* pPhyNode = (SDynQueryCtrlPhysiNode*)pNode; + destroyPhysiNode((SPhysiNode*)pPhyNode); + break; + } + case QUERY_NODE_PHYSICAL_SUBPLAN: { + SSubplan* pSubplan = (SSubplan*)pNode; + nodesClearList(pSubplan->pChildren); + nodesDestroyNode((SNode*)pSubplan->pNode); + nodesDestroyNode((SNode*)pSubplan->pDataSink); + nodesDestroyNode((SNode*)pSubplan->pTagCond); + nodesDestroyNode((SNode*)pSubplan->pTagIndexCond); + nodesClearList(pSubplan->pParents); + break; + } + case QUERY_NODE_PHYSICAL_PLAN: + nodesDestroyList(((SQueryPlan*)pNode)->pSubplans); + break; + default: + break; + } + nodesFree(pNode); + return; +} SNodeList* nodesMakeList() { SNodeList* p = nodesCalloc(1, sizeof(SNodeList));