提交 9b15bc40 编写于 作者: H Hongze Cheng

Merge branch '3.0' of https://github.com/taosdata/TDengine into feat/tsdb_refact

......@@ -234,9 +234,6 @@ int32_t buildSubmitReqFromDataBlock(SSubmitReq** pReq, const SArray* pDataBlocks
char* buildCtbNameByGroupId(const char* stbName, uint64_t groupId);
SSubmitReq* tdBlockToSubmit(const SArray* pBlocks, const STSchema* pSchema, bool createTb, int64_t suid,
const char* stbFullName, int32_t vgId);
static FORCE_INLINE int32_t blockGetEncodeSize(const SSDataBlock* pBlock) {
return blockDataGetSerialMetaSize(pBlock->info.numOfCols) + blockDataGetSize(pBlock);
}
......
......@@ -2308,6 +2308,29 @@ int32_t tDecodeSMqOffset(SDecoder* decoder, SMqOffset* pOffset);
int32_t tEncodeSMqCMCommitOffsetReq(SEncoder* encoder, const SMqCMCommitOffsetReq* pReq);
int32_t tDecodeSMqCMCommitOffsetReq(SDecoder* decoder, SMqCMCommitOffsetReq* pReq);
// tqOffset
enum {
TMQ_OFFSET__SNAPSHOT = 1,
TMQ_OFFSET__LOG,
};
typedef struct {
int8_t type;
union {
struct {
int64_t uid;
int64_t ts;
};
struct {
int64_t version;
};
};
char subKey[TSDB_SUBSCRIBE_KEY_LEN];
} STqOffset;
int32_t tEncodeSTqOffset(SEncoder* pEncoder, const STqOffset* pOffset);
int32_t tDecodeSTqOffset(SDecoder* pDecoder, STqOffset* pOffset);
typedef struct {
char name[TSDB_TABLE_FNAME_LEN];
char stb[TSDB_TABLE_FNAME_LEN];
......
......@@ -141,7 +141,7 @@ enum {
TD_DEF_MSG_TYPE(TDMT_MND_MQ_CONSUMER_RECOVER, "consumer-recover", SMqConsumerRecoverMsg, NULL)
TD_DEF_MSG_TYPE(TDMT_MND_MQ_DO_REBALANCE, "do-rebalance", SMqDoRebalanceMsg, NULL)
TD_DEF_MSG_TYPE(TDMT_MND_MQ_DROP_CGROUP, "drop-cgroup", SMqDropCGroupReq, SMqDropCGroupRsp)
TD_DEF_MSG_TYPE(TDMT_MND_MQ_COMMIT_OFFSET, "commit-offset", SMqCMCommitOffsetReq, SMqCMCommitOffsetRsp)
TD_DEF_MSG_TYPE(TDMT_MND_MQ_COMMIT_OFFSET, "mnode-commit-offset", SMqCMCommitOffsetReq, SMqCMCommitOffsetRsp)
TD_DEF_MSG_TYPE(TDMT_MND_MQ_TIMER, "mq-tmr", SMTimerReq, NULL)
TD_DEF_MSG_TYPE(TDMT_MND_TELEM_TIMER, "telem-tmr", SMTimerReq, SMTimerReq)
TD_DEF_MSG_TYPE(TDMT_MND_TRANS_TIMER, "trans-tmr", NULL, NULL)
......@@ -177,6 +177,7 @@ enum {
TD_DEF_MSG_TYPE(TDMT_VND_DROP_STB, "vnode-drop-stb", SVDropStbReq, NULL)
TD_DEF_MSG_TYPE(TDMT_VND_MQ_VG_CHANGE, "vnode-mq-vg-change", SMqRebVgReq, SMqRebVgRsp)
TD_DEF_MSG_TYPE(TDMT_VND_MQ_VG_DELETE, "vnode-mq-vg-delete", SMqVDeleteReq, SMqVDeleteRsp)
TD_DEF_MSG_TYPE(TDMT_VND_MQ_COMMIT_OFFSET, "vnode-commit-offset", STqOffset, STqOffset)
TD_DEF_MSG_TYPE(TDMT_VND_CANCEL_TASK, "vnode-cancel-task", NULL, NULL)
TD_DEF_MSG_TYPE(TDMT_VND_DROP_TASK, "vnode-drop-task", NULL, NULL)
TD_DEF_MSG_TYPE(TDMT_VND_CREATE_TOPIC, "vnode-create-topic", NULL, NULL)
......
......@@ -59,7 +59,13 @@ void mndClose(SMnode *pMnode);
* @param pMnode The mnode object.
*/
int32_t mndStart(SMnode *pMnode);
void mndStop(SMnode *pMnode);
/**
* @brief Stop mnode
*
* @param pMnode The mnode object.
*/
void mndStop(SMnode *pMnode);
/**
* @brief Get mnode monitor info.
......@@ -71,17 +77,25 @@ void mndStop(SMnode *pMnode);
* @return int32_t 0 for success, -1 for failure.
*/
int32_t mndGetMonitorInfo(SMnode *pMnode, SMonClusterInfo *pCluster, SMonVgroupInfo *pVgroup, SMonGrantInfo *pGrant);
/**
* @brief Get mnode loads for status msg.
*
* @param pMnode The mnode object.
* @param pLoad
* @return int32_t 0 for success, -1 for failure.
*/
int32_t mndGetLoad(SMnode *pMnode, SMnodeLoad *pLoad);
/**
* @brief Process the read, write, sync request.
* @brief Process the rpc, sync request.
*
* @param pMsg The request msg.
* @return int32_t 0 for success, -1 for failure.
*/
int32_t mndProcessRpcMsg(SRpcMsg *pMsg);
int32_t mndProcessSyncMsg(SRpcMsg *pMsg);
int32_t mndPreprocessQueryMsg(SMnode * pMnode, SRpcMsg * pMsg);
int32_t mndPreProcessMsg(SRpcMsg *pMsg);
/**
* @brief Generate machine code
......
......@@ -142,6 +142,10 @@ typedef enum EFunctionType {
FUNCTION_TYPE_FIRST_MERGE,
FUNCTION_TYPE_LAST_PARTIAL,
FUNCTION_TYPE_LAST_MERGE,
FUNCTION_TYPE_AVG_PARTIAL,
FUNCTION_TYPE_AVG_MERGE,
FUNCTION_TYPE_STDDEV_PARTIAL,
FUNCTION_TYPE_STDDEV_MERGE,
// user defined funcion
FUNCTION_TYPE_UDF = 10000
......
......@@ -47,8 +47,9 @@ typedef enum {
typedef enum {
TAOS_SYNC_PROPOSE_SUCCESS = 0,
TAOS_SYNC_PROPOSE_NOT_LEADER = 1,
TAOS_SYNC_PROPOSE_OTHER_ERROR = 2,
TAOS_SYNC_ONLY_ONE_REPLICA = 3,
TAOS_SYNC_ONLY_ONE_REPLICA = 2,
TAOS_SYNC_NOT_IN_NEW_CONFIG = 3,
TAOS_SYNC_OTHER_ERROR = 100,
} ESyncProposeCode;
typedef enum {
......@@ -110,6 +111,7 @@ typedef struct SSyncFSM {
void (*FpRestoreFinishCb)(struct SSyncFSM* pFsm);
void (*FpReConfigCb)(struct SSyncFSM* pFsm, const SRpcMsg* pMsg, SReConfigCbMeta cbMeta);
void (*FpLeaderTransferCb)(struct SSyncFSM* pFsm, const SRpcMsg* pMsg, SFsmCbMeta cbMeta);
int32_t (*FpGetSnapshot)(struct SSyncFSM* pFsm, SSnapshot* pSnapshot);
......@@ -199,15 +201,13 @@ bool syncIsRestoreFinish(int64_t rid);
int32_t syncGetSnapshotMeta(int64_t rid, struct SSnapshotMeta* sMeta);
int32_t syncReconfig(int64_t rid, const SSyncCfg* pNewCfg);
int32_t syncReconfigRaw(int64_t rid, const SSyncCfg* pNewCfg, SRpcMsg* pRpcMsg);
// build SRpcMsg, need to call syncPropose with SRpcMsg
int32_t syncReconfigBuild(int64_t rid, const SSyncCfg* pNewCfg, SRpcMsg* pRpcMsg);
int32_t syncLeaderTransfer(int64_t rid);
int32_t syncLeaderTransferTo(int64_t rid, SNodeInfo newLeader);
// to be moved to static
void syncStartNormal(int64_t rid);
void syncStartStandBy(int64_t rid);
#ifdef __cplusplus
}
#endif
......
......@@ -467,6 +467,7 @@ typedef struct SyncLeaderTransfer {
SRaftId srcId;
SRaftId destId;
*/
SNodeInfo newNodeInfo;
SRaftId newLeaderId;
} SyncLeaderTransfer;
......
......@@ -696,6 +696,8 @@ int32_t* taosGetErrno();
#define TSDB_CODE_RSMA_INVALID_ENV TAOS_DEF_ERROR_CODE(0, 0x3150)
#define TSDB_CODE_RSMA_INVALID_STAT TAOS_DEF_ERROR_CODE(0, 0x3151)
//index
#define TSDB_CODE_INDEX_REBUILDING TAOS_DEF_ERROR_CODE(0, 0x3200)
#ifdef __cplusplus
}
......
......@@ -206,7 +206,7 @@ int32_t execLocalCmd(SRequestObj* pRequest, SQuery* pQuery) {
SRetrieveTableRsp* pRsp = NULL;
int32_t code = qExecCommand(pQuery->pRoot, &pRsp);
if (TSDB_CODE_SUCCESS == code && NULL != pRsp) {
code = setQueryResultFromRsp(&pRequest->body.resInfo, pRsp, false, false);
code = setQueryResultFromRsp(&pRequest->body.resInfo, pRsp, false, true);
}
return code;
......@@ -1415,7 +1415,7 @@ int32_t setResultDataPtr(SReqResultInfo* pResultInfo, TAOS_FIELD* pFields, int32
int32_t bytes = *(int32_t*)p;
p += sizeof(int32_t);
// ASSERT(type == pFields[i].type && bytes == pFields[i].bytes);
/*ASSERT(type == pFields[i].type && bytes == pFields[i].bytes);*/
}
int32_t* colLength = (int32_t*)p;
......
......@@ -132,6 +132,7 @@ typedef struct {
// statistics
int64_t pollCnt;
// offset
int64_t committedOffset;
int64_t currentOffset;
// connection info
int32_t vgId;
......@@ -193,6 +194,26 @@ typedef struct {
void* userParam;
} SMqCommitCbParam;
typedef struct {
tmq_t* tmq;
int8_t automatic;
int8_t async;
int8_t freeOffsets;
int32_t waitingRspNum;
int32_t totalRspNum;
tmq_resp_err_t rspErr;
tmq_commit_cb* userCb;
SArray* successfulOffsets;
SArray* failedOffsets;
void* userParam;
tsem_t rspSem;
} SMqCommitCbParamSet;
typedef struct {
SMqCommitCbParamSet* params;
STqOffset* pOffset;
} SMqCommitCbParam2;
tmq_conf_t* tmq_conf_new() {
tmq_conf_t* conf = taosMemoryCalloc(1, sizeof(tmq_conf_t));
conf->withTbName = false;
......@@ -343,6 +364,139 @@ int32_t tmqCommitCb(void* param, const SDataBuf* pMsg, int32_t code) {
return 0;
}
int32_t tmqCommitCb2(void* param, const SDataBuf* pBuf, int32_t code) {
SMqCommitCbParam2* pParam = (SMqCommitCbParam2*)param;
SMqCommitCbParamSet* pParamSet = (SMqCommitCbParamSet*)pParam->params;
// push into array
if (code == 0) {
taosArrayPush(pParamSet->failedOffsets, &pParam->pOffset);
} else {
taosArrayPush(pParamSet->successfulOffsets, &pParam->pOffset);
}
// count down waiting rsp
int32_t waitingRspNum = atomic_sub_fetch_32(&pParamSet->waitingRspNum, 1);
ASSERT(waitingRspNum >= 0);
if (waitingRspNum == 0) {
// if no more waiting rsp
if (pParamSet->async) {
// call async cb func
if (pParamSet->automatic && pParamSet->tmq->commitCb) {
pParamSet->tmq->commitCb(pParamSet->tmq, pParamSet->rspErr, NULL, pParamSet->tmq->commitCbUserParam);
} else if (!pParamSet->automatic && pParamSet->userCb) {
// sem post
pParamSet->userCb(pParamSet->tmq, pParamSet->rspErr, NULL, pParamSet->userParam);
}
}
taosArrayDestroyP(pParamSet->successfulOffsets, taosMemoryFree);
taosArrayDestroyP(pParamSet->failedOffsets, taosMemoryFree);
}
return 0;
}
int32_t tmqCommitInner2(tmq_t* tmq, const tmq_topic_vgroup_list_t* offsets, int8_t automatic, int8_t async,
tmq_commit_cb* userCb, void* userParam) {
int32_t code = -1;
SMqCommitCbParamSet* pParamSet = taosMemoryCalloc(1, sizeof(SMqCommitCbParamSet));
if (pParamSet == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY;
return -1;
}
pParamSet->tmq = tmq;
pParamSet->automatic = automatic;
pParamSet->async = async;
pParamSet->freeOffsets = 1;
pParamSet->userCb = userCb;
pParamSet->userParam = userParam;
tsem_init(&pParamSet->rspSem, 0, 0);
for (int32_t i = 0; i < taosArrayGetSize(tmq->clientTopics); i++) {
SMqClientTopic* pTopic = taosArrayGet(tmq->clientTopics, i);
for (int32_t j = 0; j < taosArrayGetSize(pTopic->vgs); j++) {
SMqClientVg* pVg = taosArrayGet(pTopic->vgs, i);
STqOffset* pOffset = taosMemoryCalloc(1, sizeof(STqOffset));
if (pOffset == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY;
return -1;
}
int32_t tlen = strlen(tmq->groupId);
memcpy(pOffset->subKey, tmq->groupId, tlen);
pOffset->subKey[tlen] = TMQ_SEPARATOR;
strcpy(pOffset->subKey + tlen + 1, pTopic->topicName);
int32_t len;
int32_t code;
tEncodeSize(tEncodeSTqOffset, pOffset, len, code);
if (code < 0) {
ASSERT(0);
}
void* buf = taosMemoryCalloc(1, sizeof(SMsgHead) + len);
((SMsgHead*)buf)->vgId = htonl(pVg->vgId);
void* abuf = POINTER_SHIFT(buf, sizeof(SMsgHead));
SEncoder encoder;
tEncoderInit(&encoder, abuf, len);
tEncodeSTqOffset(&encoder, pOffset);
// build param
SMqCommitCbParam2* pParam = taosMemoryCalloc(1, sizeof(SMqCommitCbParam2));
pParam->params = pParamSet;
pParam->pOffset = pOffset;
// build send info
SMsgSendInfo* pMsgSendInfo = taosMemoryCalloc(1, sizeof(SMsgSendInfo));
if (pMsgSendInfo == NULL) {
// TODO
continue;
}
pMsgSendInfo->msgInfo = (SDataBuf){
.pData = buf,
.len = len,
.handle = NULL,
};
pMsgSendInfo->requestId = generateRequestId();
pMsgSendInfo->requestObjRefId = 0;
pMsgSendInfo->param = pParam;
pMsgSendInfo->fp = tmqCommitCb2;
pMsgSendInfo->msgType = TDMT_MND_MQ_COMMIT_OFFSET;
// send msg
SEpSet epSet = getEpSet_s(&tmq->pTscObj->pAppInfo->mgmtEp);
int64_t transporterId = 0;
asyncSendMsgToServer(tmq->pTscObj->pAppInfo->pTransporter, &epSet, &transporterId, pMsgSendInfo);
pParamSet->waitingRspNum++;
pParamSet->totalRspNum++;
}
}
if (!async) {
tsem_wait(&pParamSet->rspSem);
code = pParamSet->rspErr;
tsem_destroy(&pParamSet->rspSem);
} else {
code = 0;
}
if (code != 0 && async) {
if (automatic) {
tmq->commitCb(tmq, code, NULL, tmq->commitCbUserParam);
} else {
userCb(tmq, code, NULL, userParam);
}
}
if (!async) {
taosArrayDestroyP(pParamSet->successfulOffsets, taosMemoryFree);
taosArrayDestroyP(pParamSet->failedOffsets, taosMemoryFree);
}
return 0;
}
int32_t tmqCommitInner(tmq_t* tmq, const tmq_topic_vgroup_list_t* offsets, int8_t automatic, int8_t async,
tmq_commit_cb* userCb, void* userParam) {
SMqCMCommitOffsetReq req;
......@@ -890,12 +1044,13 @@ bool tmqUpdateEp(tmq_t* tmq, int32_t epoch, SMqAskEpRsp* pRsp) {
sprintf(vgKey, "%s:%d", topic.topicName, pVgEp->vgId);
int64_t* pOffset = taosHashGet(pHash, vgKey, strlen(vgKey));
int64_t offset = pVgEp->offset;
tscDebug("consumer %ld epoch %d vg %d offset og to %ld", tmq->consumerId, epoch, pVgEp->vgId, offset);
tscDebug("consumer %ld(epoch %d) original offset of vg %d is %ld", tmq->consumerId, epoch, pVgEp->vgId, offset);
if (pOffset != NULL) {
offset = *pOffset;
tscDebug("consumer %ld epoch %d vg %d found %s", tmq->consumerId, epoch, pVgEp->vgId, vgKey);
tscDebug("consumer %ld(epoch %d) receive offset of vg %d, full key is %s", tmq->consumerId, epoch, pVgEp->vgId,
vgKey);
}
tscDebug("consumer %ld epoch %d vg %d offset set to %ld", tmq->consumerId, epoch, pVgEp->vgId, offset);
tscDebug("consumer %ld(epoch %d) offset of vg %d updated to %ld", tmq->consumerId, epoch, pVgEp->vgId, offset);
SMqClientVg clientVg = {
.pollCnt = 0,
.currentOffset = offset,
......@@ -1226,9 +1381,8 @@ SMqRspObj* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout, bool pollIfReset) {
if (rspWrapper->tmqRspType == TMQ_MSG_TYPE__POLL_RSP) {
SMqPollRspWrapper* pollRspWrapper = (SMqPollRspWrapper*)rspWrapper;
/*atomic_sub_fetch_32(&tmq->readyRequest, 1);*/
/*printf("handle poll rsp %d\n", rspMsg->head.mqMsgType);*/
if (pollRspWrapper->msg.head.epoch == atomic_load_32(&tmq->epoch)) {
/*printf("epoch match\n");*/
int32_t consumerEpoch = atomic_load_32(&tmq->epoch);
if (pollRspWrapper->msg.head.epoch == consumerEpoch) {
SMqClientVg* pVg = pollRspWrapper->vgHandle;
/*printf("vg %d offset %ld up to %ld\n", pVg->vgId, pVg->currentOffset, rspMsg->msg.rspOffset);*/
pVg->currentOffset = pollRspWrapper->msg.rspOffset;
......@@ -1243,7 +1397,8 @@ SMqRspObj* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout, bool pollIfReset) {
taosFreeQitem(pollRspWrapper);
return pRsp;
} else {
/*printf("epoch mismatch\n");*/
tscDebug("msg discard since epoch mismatch: msg epoch %d, consumer epoch %d\n", pollRspWrapper->msg.head.epoch,
consumerEpoch);
taosFreeQitem(pollRspWrapper);
}
} else {
......@@ -1263,10 +1418,14 @@ TAOS_RES* tmq_consumer_poll(tmq_t* tmq, int64_t timeout) {
SMqRspObj* rspObj;
int64_t startTime = taosGetTimestampMs();
#if 0
tmqHandleAllDelayedTask(tmq);
tmqPollImpl(tmq, timeout);
rspObj = tmqHandleAllRsp(tmq, timeout, false);
if (rspObj) {
return (TAOS_RES*)rspObj;
}
#endif
// in no topic status also need process delayed task
if (atomic_load_8(&tmq->status) == TMQ_CONSUMER_STATUS__INIT) {
......@@ -1359,8 +1518,7 @@ const char* tmq_get_table_name(TAOS_RES* res) {
pRspObj->resIter >= pRspObj->rsp.blockNum) {
return NULL;
}
const char* name = taosArrayGetP(pRspObj->rsp.blockTbName, pRspObj->resIter);
return name;
return (const char*)taosArrayGetP(pRspObj->rsp.blockTbName, pRspObj->resIter);
}
return NULL;
}
......
......@@ -1708,6 +1708,7 @@ char* buildCtbNameByGroupId(const char* stbName, uint64_t groupId) {
pTag->keyLen = strlen(pTag->key);
pTag->type = TSDB_DATA_TYPE_UBIGINT;
pTag->u = groupId;
pTag->length = sizeof(uint64_t);
taosArrayPush(tags, &pTag);
void* cname = taosMemoryCalloc(1, TSDB_TABLE_NAME_LEN + 1);
......@@ -1728,173 +1729,6 @@ char* buildCtbNameByGroupId(const char* stbName, uint64_t groupId) {
return rname.childTableName;
}
SSubmitReq* tdBlockToSubmit(const SArray* pBlocks, const STSchema* pTSchema, bool createTb, int64_t suid,
const char* stbFullName, int32_t vgId) {
SSubmitReq* ret = NULL;
SArray* tagArray = taosArrayInit(1, sizeof(STagVal));
if (!tagArray) {
terrno = TSDB_CODE_OUT_OF_MEMORY;
return NULL;
}
// cal size
int32_t cap = sizeof(SSubmitReq);
int32_t sz = taosArrayGetSize(pBlocks);
for (int32_t i = 0; i < sz; i++) {
SSDataBlock* pDataBlock = taosArrayGet(pBlocks, i);
int32_t rows = pDataBlock->info.rows;
// TODO min
int32_t rowSize = pDataBlock->info.rowSize;
int32_t maxLen = TD_ROW_MAX_BYTES_FROM_SCHEMA(pTSchema);
int32_t schemaLen = 0;
if (createTb) {
SVCreateTbReq createTbReq = {0};
char* cname = buildCtbNameByGroupId(stbFullName, pDataBlock->info.groupId);
createTbReq.name = cname;
createTbReq.flags = 0;
createTbReq.type = TSDB_CHILD_TABLE;
createTbReq.ctb.suid = suid;
STagVal tagVal = {.cid = 1,
.type = TSDB_DATA_TYPE_UBIGINT,
.pData = (uint8_t*)&pDataBlock->info.groupId,
.nData = sizeof(uint64_t)};
STag* pTag = NULL;
taosArrayClear(tagArray);
taosArrayPush(tagArray, &tagVal);
tTagNew(tagArray, 1, false, &pTag);
if (pTag == NULL) {
tdDestroySVCreateTbReq(&createTbReq);
taosArrayDestroy(tagArray);
return NULL;
}
createTbReq.ctb.pTag = (uint8_t*)pTag;
int32_t code;
tEncodeSize(tEncodeSVCreateTbReq, &createTbReq, schemaLen, code);
tdDestroySVCreateTbReq(&createTbReq);
if (code < 0) {
taosArrayDestroy(tagArray);
return NULL;
}
}
cap += sizeof(SSubmitBlk) + schemaLen + rows * maxLen;
}
// assign data
// TODO
ret = taosMemoryCalloc(1, cap + 46);
ret = POINTER_SHIFT(ret, 46);
ret->header.vgId = vgId;
ret->version = htonl(1);
ret->length = sizeof(SSubmitReq);
ret->numOfBlocks = htonl(sz);
void* submitBlk = POINTER_SHIFT(ret, sizeof(SSubmitReq));
for (int32_t i = 0; i < sz; i++) {
SSDataBlock* pDataBlock = taosArrayGet(pBlocks, i);
SSubmitBlk* blkHead = submitBlk;
blkHead->numOfRows = htons(pDataBlock->info.rows);
blkHead->sversion = htonl(pTSchema->version);
// TODO
blkHead->suid = htobe64(suid);
// uid is assigned by vnode
blkHead->uid = 0;
int32_t rows = pDataBlock->info.rows;
/*int32_t maxLen = TD_ROW_MAX_BYTES_FROM_SCHEMA(pTSchema);*/
/*blkHead->dataLen = htonl(rows * maxLen);*/
blkHead->dataLen = 0;
void* blockData = POINTER_SHIFT(submitBlk, sizeof(SSubmitBlk));
int32_t schemaLen = 0;
if (createTb) {
SVCreateTbReq createTbReq = {0};
char* cname = buildCtbNameByGroupId(stbFullName, pDataBlock->info.groupId);
createTbReq.name = cname;
createTbReq.flags = 0;
createTbReq.type = TSDB_CHILD_TABLE;
createTbReq.ctb.suid = suid;
STagVal tagVal = {.cid = 1,
.type = TSDB_DATA_TYPE_UBIGINT,
.pData = (uint8_t*)&pDataBlock->info.groupId,
.nData = sizeof(uint64_t)};
taosArrayClear(tagArray);
taosArrayPush(tagArray, &tagVal);
STag* pTag = NULL;
tTagNew(tagArray, 1, false, &pTag);
if (pTag == NULL) {
tdDestroySVCreateTbReq(&createTbReq);
taosArrayDestroy(tagArray);
taosMemoryFreeClear(ret);
return NULL;
}
createTbReq.ctb.pTag = (uint8_t*)pTag;
int32_t code;
tEncodeSize(tEncodeSVCreateTbReq, &createTbReq, schemaLen, code);
if (code < 0) {
tdDestroySVCreateTbReq(&createTbReq);
taosArrayDestroy(tagArray);
taosMemoryFreeClear(ret);
return NULL;
}
SEncoder encoder = {0};
tEncoderInit(&encoder, blockData, schemaLen);
code = tEncodeSVCreateTbReq(&encoder, &createTbReq);
tEncoderClear(&encoder);
tdDestroySVCreateTbReq(&createTbReq);
if (code < 0) {
taosArrayDestroy(tagArray);
taosMemoryFreeClear(ret);
return NULL;
}
}
blkHead->schemaLen = htonl(schemaLen);
STSRow* rowData = POINTER_SHIFT(blockData, schemaLen);
for (int32_t j = 0; j < rows; j++) {
SRowBuilder rb = {0};
tdSRowInit(&rb, pTSchema->version);
tdSRowSetTpInfo(&rb, pTSchema->numOfCols, pTSchema->flen);
tdSRowResetBuf(&rb, rowData);
for (int32_t k = 0; k < pTSchema->numOfCols; k++) {
const STColumn* pColumn = &pTSchema->columns[k];
SColumnInfoData* pColData = taosArrayGet(pDataBlock->pDataBlock, k);
if (colDataIsNull_s(pColData, j)) {
tdAppendColValToRow(&rb, pColumn->colId, pColumn->type, TD_VTYPE_NONE, NULL, false, pColumn->offset, k);
} else {
void* data = colDataGetData(pColData, j);
tdAppendColValToRow(&rb, pColumn->colId, pColumn->type, TD_VTYPE_NORM, data, true, pColumn->offset, k);
}
}
int32_t rowLen = TD_ROW_LEN(rowData);
rowData = POINTER_SHIFT(rowData, rowLen);
blkHead->dataLen += rowLen;
}
int32_t dataLen = blkHead->dataLen;
blkHead->dataLen = htonl(dataLen);
ret->length += sizeof(SSubmitBlk) + schemaLen + dataLen;
blkHead = POINTER_SHIFT(blkHead, schemaLen + dataLen);
/*submitBlk = blkHead;*/
}
ret->length = htonl(ret->length);
taosArrayDestroy(tagArray);
return ret;
}
void blockCompressEncode(const SSDataBlock* pBlock, char* data, int32_t* dataLen, int32_t numOfCols,
int8_t needCompress) {
// todo extract method
......
......@@ -4851,3 +4851,32 @@ void tFreeSMAlterStbRsp(SMAlterStbRsp *pRsp) {
taosMemoryFree(pRsp->pMeta);
}
}
int32_t tEncodeSTqOffset(SEncoder *pEncoder, const STqOffset *pOffset) {
if (tEncodeI8(pEncoder, pOffset->type) < 0) return -1;
if (pOffset->type == TMQ_OFFSET__SNAPSHOT) {
if (tEncodeI64(pEncoder, pOffset->uid) < 0) return -1;
if (tEncodeI64(pEncoder, pOffset->ts) < 0) return -1;
} else if (pOffset->type == TMQ_OFFSET__LOG) {
if (tEncodeI64(pEncoder, pOffset->version) < 0) return -1;
} else {
ASSERT(0);
}
if (tEncodeCStr(pEncoder, pOffset->subKey) < 0) return -1;
return 0;
}
int32_t tDecodeSTqOffset(SDecoder *pDecoder, STqOffset *pOffset) {
if (tDecodeI8(pDecoder, &pOffset->type) < 0) return -1;
if (pOffset->type == TMQ_OFFSET__SNAPSHOT) {
if (tDecodeI64(pDecoder, &pOffset->uid) < 0) return -1;
if (tDecodeI64(pDecoder, &pOffset->ts) < 0) return -1;
} else if (pOffset->type == TMQ_OFFSET__LOG) {
if (tDecodeI64(pDecoder, &pOffset->version) < 0) return -1;
} else {
ASSERT(0);
}
if (tDecodeCStrTo(pDecoder, pOffset->subKey) < 0) return -1;
return 0;
}
......@@ -114,7 +114,8 @@ int32_t mmPutMsgToReadQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg) {
}
int32_t mmPutMsgToQueryQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg) {
if (mndPreprocessQueryMsg(pMgmt->pMnode, pMsg) != 0) {
pMsg->info.node = pMgmt->pMnode;
if (mndPreProcessMsg(pMsg) != 0) {
dError("msg:%p, failed to pre-process in mnode since %s, type:%s", pMsg, terrstr(), TMSG_INFO(pMsg->msgType));
return -1;
}
......
......@@ -344,6 +344,7 @@ SArray *vmGetMsgHandles() {
if (dmSetMgmtHandle(pArray, TDMT_VND_SUBMIT_RSMA, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_VND_MQ_VG_CHANGE, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_VND_MQ_VG_DELETE, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_VND_MQ_COMMIT_OFFSET, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_VND_CONSUME, vmPutMsgToFetchQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_VND_DELETE, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_VND_QUERY_HEARTBEAT, vmPutMsgToFetchQueue, 0) == NULL) goto _OVER;
......
......@@ -26,6 +26,8 @@ int32_t mndInitSma(SMnode *pMnode);
void mndCleanupSma(SMnode *pMnode);
SSmaObj *mndAcquireSma(SMnode *pMnode, char *smaName);
void mndReleaseSma(SMnode *pMnode, SSmaObj *pSma);
int32_t mndDropSmasByStb(SMnode *pMnode, STrans *pTrans, SDbObj *pDb, SStbObj *pStb);
int32_t mndDropSmasByDb(SMnode *pMnode, STrans *pTrans, SDbObj *pDb);
int32_t mndGetTableSma(SMnode *pMnode, char *tbFName, STableIndexRsp *rsp, bool *exist);
#ifdef __cplusplus
......
......@@ -935,6 +935,7 @@ static int32_t mndDropDb(SMnode *pMnode, SRpcMsg *pReq, SDbObj *pDb) {
if (mndDropOffsetByDB(pMnode, pTrans, pDb) != 0) goto _OVER;
if (mndDropSubByDB(pMnode, pTrans, pDb) != 0) goto _OVER;
if (mndDropTopicByDB(pMnode, pTrans, pDb) != 0) goto _OVER;
if (mndDropSmasByDb(pMnode, pTrans, pDb) != 0) goto _OVER;
if (mndSetDropDbRedoActions(pMnode, pTrans, pDb) != 0) goto _OVER;
SUserObj *pUser = mndAcquireUser(pMnode, pDb->createUser);
......
......@@ -380,22 +380,17 @@ void mndStop(SMnode *pMnode) {
int32_t mndProcessSyncMsg(SRpcMsg *pMsg) {
SMnode *pMnode = pMsg->info.node;
SSyncMgmt *pMgmt = &pMnode->syncMgmt;
int32_t code = TAOS_SYNC_PROPOSE_OTHER_ERROR;
int32_t code = TAOS_SYNC_OTHER_ERROR;
if (!syncEnvIsStart()) {
mError("failed to process sync msg:%p type:%s since syncEnv stop", pMsg, TMSG_INFO(pMsg->msgType));
return TAOS_SYNC_PROPOSE_OTHER_ERROR;
return TAOS_SYNC_OTHER_ERROR;
}
SSyncNode *pSyncNode = syncNodeAcquire(pMgmt->sync);
if (pSyncNode == NULL) {
mError("failed to process sync msg:%p type:%s since syncNode is null", pMsg, TMSG_INFO(pMsg->msgType));
return TAOS_SYNC_PROPOSE_OTHER_ERROR;
}
if (mndAcquireSyncRef(pMnode) != 0) {
mError("failed to process sync msg:%p type:%s since %s", pMsg, TMSG_INFO(pMsg->msgType), terrstr());
return TAOS_SYNC_PROPOSE_OTHER_ERROR;
return TAOS_SYNC_OTHER_ERROR;
}
char logBuf[512] = {0};
......@@ -456,7 +451,7 @@ int32_t mndProcessSyncMsg(SRpcMsg *pMsg) {
tmsgSendRsp(&rsp);
} else {
mError("failed to process msg:%p since invalid type:%s", pMsg, TMSG_INFO(pMsg->msgType));
code = TAOS_SYNC_PROPOSE_OTHER_ERROR;
code = TAOS_SYNC_OTHER_ERROR;
}
} else {
if (pMsg->msgType == TDMT_SYNC_TIMEOUT) {
......@@ -497,11 +492,10 @@ int32_t mndProcessSyncMsg(SRpcMsg *pMsg) {
tmsgSendRsp(&rsp);
} else {
mError("failed to process msg:%p since invalid type:%s", pMsg, TMSG_INFO(pMsg->msgType));
code = TAOS_SYNC_PROPOSE_OTHER_ERROR;
code = TAOS_SYNC_OTHER_ERROR;
}
}
mndReleaseSyncRef(pMnode);
return code;
}
......@@ -754,24 +748,3 @@ void mndSetStop(SMnode *pMnode) {
}
bool mndGetStop(SMnode *pMnode) { return pMnode->stopped; }
int32_t mndAcquireSyncRef(SMnode *pMnode) {
int32_t code = 0;
taosThreadRwlockRdlock(&pMnode->lock);
if (pMnode->stopped) {
terrno = TSDB_CODE_APP_NOT_READY;
code = -1;
} else {
int32_t ref = atomic_add_fetch_32(&pMnode->syncRef, 1);
// mTrace("mnode sync is acquired, ref:%d", ref);
}
taosThreadRwlockUnlock(&pMnode->lock);
return code;
}
void mndReleaseSyncRef(SMnode *pMnode) {
taosThreadRwlockRdlock(&pMnode->lock);
int32_t ref = atomic_sub_fetch_32(&pMnode->syncRef, 1);
// mTrace("mnode sync is released, ref:%d", ref);
taosThreadRwlockUnlock(&pMnode->lock);
}
......@@ -18,11 +18,10 @@
#include "mndMnode.h"
#include "qworker.h"
int32_t mndPreprocessQueryMsg(SMnode * pMnode, SRpcMsg * pMsg) {
if (TDMT_VND_QUERY != pMsg->msgType) {
return 0;
}
int32_t mndPreProcessMsg(SRpcMsg *pMsg) {
if (TDMT_VND_QUERY != pMsg->msgType) return 0;
SMnode *pMnode = pMsg->info.node;
return qWorkerPreprocessQueryMsg(pMnode->pQuery, pMsg);
}
......
......@@ -38,10 +38,10 @@ static int32_t mndSmaActionInsert(SSdb *pSdb, SSmaObj *pSma);
static int32_t mndSmaActionDelete(SSdb *pSdb, SSmaObj *pSpSmatb);
static int32_t mndSmaActionUpdate(SSdb *pSdb, SSmaObj *pOld, SSmaObj *pNew);
static int32_t mndSmaGetVgEpSet(SMnode *pMnode, SDbObj *pDb, SVgEpSet **ppVgEpSet, int32_t *numOfVgroups);
static int32_t mndProcessMCreateSmaReq(SRpcMsg *pReq);
static int32_t mndProcessMDropSmaReq(SRpcMsg *pReq);
static int32_t mndProcessCreateSmaReq(SRpcMsg *pReq);
static int32_t mndProcessDropSmaReq(SRpcMsg *pReq);
static int32_t mndProcessGetSmaReq(SRpcMsg *pReq);
static int32_t mndProcessGetTbSmaReq(SRpcMsg *pReq);
static int32_t mndProcessGetTbSmaReq(SRpcMsg *pReq);
static int32_t mndRetrieveSma(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock *pBlock, int32_t rows);
static void mndCancelGetNextSma(SMnode *pMnode, void *pIter);
......@@ -56,8 +56,8 @@ int32_t mndInitSma(SMnode *pMnode) {
.deleteFp = (SdbDeleteFp)mndSmaActionDelete,
};
mndSetMsgHandle(pMnode, TDMT_MND_CREATE_SMA, mndProcessMCreateSmaReq);
mndSetMsgHandle(pMnode, TDMT_MND_DROP_SMA, mndProcessMDropSmaReq);
mndSetMsgHandle(pMnode, TDMT_MND_CREATE_SMA, mndProcessCreateSmaReq);
mndSetMsgHandle(pMnode, TDMT_MND_DROP_SMA, mndProcessDropSmaReq);
mndSetMsgHandle(pMnode, TDMT_VND_CREATE_SMA_RSP, mndTransProcessRsp);
mndSetMsgHandle(pMnode, TDMT_VND_DROP_SMA_RSP, mndTransProcessRsp);
mndSetMsgHandle(pMnode, TDMT_MND_GET_INDEX, mndProcessGetSmaReq);
......@@ -79,7 +79,6 @@ static SSdbRaw *mndSmaActionEncode(SSmaObj *pSma) {
if (pRaw == NULL) goto _OVER;
int32_t dataPos = 0;
SDB_SET_BINARY(pRaw, dataPos, pSma->name, TSDB_TABLE_FNAME_LEN, _OVER)
SDB_SET_BINARY(pRaw, dataPos, pSma->stb, TSDB_TABLE_FNAME_LEN, _OVER)
SDB_SET_BINARY(pRaw, dataPos, pSma->db, TSDB_DB_FNAME_LEN, _OVER)
......@@ -100,6 +99,7 @@ static SSdbRaw *mndSmaActionEncode(SSmaObj *pSma) {
SDB_SET_INT32(pRaw, dataPos, pSma->tagsFilterLen, _OVER)
SDB_SET_INT32(pRaw, dataPos, pSma->sqlLen, _OVER)
SDB_SET_INT32(pRaw, dataPos, pSma->astLen, _OVER)
if (pSma->exprLen > 0) {
SDB_SET_BINARY(pRaw, dataPos, pSma->expr, pSma->exprLen, _OVER)
}
......@@ -115,6 +115,7 @@ static SSdbRaw *mndSmaActionEncode(SSmaObj *pSma) {
SDB_SET_RESERVE(pRaw, dataPos, TSDB_SMA_RESERVE_SIZE, _OVER)
SDB_SET_DATALEN(pRaw, dataPos, _OVER)
terrno = 0;
_OVER:
......@@ -193,6 +194,7 @@ static SSdbRow *mndSmaActionDecode(SSdbRaw *pRaw) {
}
SDB_GET_RESERVE(pRaw, dataPos, TSDB_SMA_RESERVE_SIZE, _OVER)
terrno = 0;
_OVER:
......@@ -383,6 +385,25 @@ static int32_t mndSetCreateSmaVgroupCommitLogs(SMnode *pMnode, STrans *pTrans, S
return 0;
}
static int32_t mndSetUpdateSmaStbCommitLogs(SMnode *pMnode, STrans *pTrans, SStbObj *pStb) {
SStbObj stbObj = {0};
taosRLockLatch(&pStb->lock);
memcpy(&stbObj, pStb, sizeof(SStbObj));
taosRUnLockLatch(&pStb->lock);
stbObj.pColumns = NULL;
stbObj.pTags = NULL;
stbObj.updateTime = taosGetTimestampMs();
stbObj.lock = 0;
stbObj.smaVer++;
SSdbRaw *pCommitRaw = mndStbActionEncode(&stbObj);
if (pCommitRaw == NULL) return -1;
if (mndTransAppendCommitlog(pTrans, pCommitRaw) != 0) return -1;
if (sdbSetRawStatus(pCommitRaw, SDB_STATUS_READY) != 0) return -1;
return 0;
}
static int32_t mndSetCreateSmaRedoActions(SMnode *pMnode, STrans *pTrans, SDbObj *pDb, SSmaObj *pSma) {
SSdb *pSdb = pMnode->pSdb;
SVgObj *pVgroup = NULL;
......@@ -457,7 +478,6 @@ static int32_t mndSetCreateSmaVgroupRedoActions(SMnode *pMnode, STrans *pTrans,
pSma->schemaTag.pSchema[0].flags = 0;
snprintf(pSma->schemaTag.pSchema[0].name, TSDB_COL_NAME_LEN, "groupId");
int32_t smaContLen = 0;
void *pSmaReq = mndBuildVCreateSmaReq(pMnode, pVgroup, pSma, &smaContLen);
if (pSmaReq == NULL) return -1;
......@@ -560,6 +580,7 @@ static int32_t mndCreateSma(SMnode *pMnode, SRpcMsg *pReq, SMCreateSmaReq *pCrea
if (mndSetCreateSmaVgroupRedoLogs(pMnode, pTrans, &streamObj.fixedSinkVg) != 0) goto _OVER;
if (mndSetCreateSmaCommitLogs(pMnode, pTrans, &smaObj) != 0) goto _OVER;
if (mndSetCreateSmaVgroupCommitLogs(pMnode, pTrans, &streamObj.fixedSinkVg) != 0) goto _OVER;
if (mndSetUpdateSmaStbCommitLogs(pMnode, pTrans, pStb) != 0) goto _OVER;
if (mndSetCreateSmaRedoActions(pMnode, pTrans, pDb, &smaObj) != 0) goto _OVER;
if (mndSetCreateSmaVgroupRedoActions(pMnode, pTrans, pDb, &streamObj.fixedSinkVg, &smaObj) != 0) goto _OVER;
if (mndAddStreamToTrans(pMnode, &streamObj, pCreate->ast, STREAM_TRIGGER_AT_ONCE, 0, pTrans) != 0) goto _OVER;
......@@ -600,7 +621,7 @@ static int32_t mndCheckCreateSmaReq(SMCreateSmaReq *pCreate) {
return 0;
}
static int32_t mndProcessMCreateSmaReq(SRpcMsg *pReq) {
static int32_t mndProcessCreateSmaReq(SRpcMsg *pReq) {
SMnode *pMnode = pReq->info.node;
int32_t code = -1;
SStbObj *pStb = NULL;
......@@ -782,13 +803,17 @@ static int32_t mndSetDropSmaVgroupRedoActions(SMnode *pMnode, STrans *pTrans, SD
}
static int32_t mndDropSma(SMnode *pMnode, SRpcMsg *pReq, SDbObj *pDb, SSmaObj *pSma) {
int32_t code = -1;
SVgObj *pVgroup = NULL;
STrans *pTrans = NULL;
int32_t code = -1;
SVgObj *pVgroup = NULL;
SStbObj *pStb = NULL;
STrans *pTrans = NULL;
pVgroup = mndAcquireVgroup(pMnode, pSma->dstVgId);
if (pVgroup == NULL) goto _OVER;
pStb = mndAcquireStb(pMnode, pSma->stb);
if (pStb == NULL) goto _OVER;
pTrans = mndTransCreate(pMnode, TRN_POLICY_ROLLBACK, TRN_CONFLICT_DB, pReq);
if (pTrans == NULL) goto _OVER;
......@@ -799,6 +824,7 @@ static int32_t mndDropSma(SMnode *pMnode, SRpcMsg *pReq, SDbObj *pDb, SSmaObj *p
if (mndSetDropSmaVgroupRedoLogs(pMnode, pTrans, pVgroup) != 0) goto _OVER;
if (mndSetDropSmaCommitLogs(pMnode, pTrans, pSma) != 0) goto _OVER;
if (mndSetDropSmaVgroupCommitLogs(pMnode, pTrans, pVgroup) != 0) goto _OVER;
if (mndSetUpdateSmaStbCommitLogs(pMnode, pTrans, pStb) != 0) goto _OVER;
if (mndSetDropSmaRedoActions(pMnode, pTrans, pDb, pSma) != 0) goto _OVER;
if (mndSetDropSmaVgroupRedoActions(pMnode, pTrans, pDb, pVgroup) != 0) goto _OVER;
if (mndTransPrepare(pMnode, pTrans) != 0) goto _OVER;
......@@ -808,10 +834,78 @@ static int32_t mndDropSma(SMnode *pMnode, SRpcMsg *pReq, SDbObj *pDb, SSmaObj *p
_OVER:
mndTransDrop(pTrans);
mndReleaseVgroup(pMnode, pVgroup);
mndReleaseStb(pMnode, pStb);
return code;
}
static int32_t mndProcessMDropSmaReq(SRpcMsg *pReq) {
int32_t mndDropSmasByStb(SMnode *pMnode, STrans *pTrans, SDbObj *pDb, SStbObj *pStb) {
SSdb *pSdb = pMnode->pSdb;
SSmaObj *pSma = NULL;
void *pIter = NULL;
SVgObj *pVgroup = NULL;
int32_t code = -1;
while (1) {
pIter = sdbFetch(pSdb, SDB_SMA, pIter, (void **)&pSma);
if (pIter == NULL) break;
if (pSma->stbUid == pStb->uid) {
pVgroup = mndAcquireVgroup(pMnode, pSma->dstVgId);
if (pVgroup == NULL) goto _OVER;
if (mndSetDropSmaVgroupCommitLogs(pMnode, pTrans, pVgroup) != 0) goto _OVER;
if (mndSetDropSmaVgroupRedoActions(pMnode, pTrans, pDb, pVgroup) != 0) goto _OVER;
if (mndSetDropSmaCommitLogs(pMnode, pTrans, pSma) != 0) goto _OVER;
if (mndSetDropSmaRedoActions(pMnode, pTrans, pDb, pSma) != 0) goto _OVER;
mndReleaseVgroup(pMnode, pVgroup);
pVgroup = NULL;
}
sdbRelease(pSdb, pSma);
}
code = 0;
_OVER:
sdbCancelFetch(pSdb, pIter);
sdbRelease(pSdb, pSma);
mndReleaseVgroup(pMnode, pVgroup);
return code;
}
int32_t mndDropSmasByDb(SMnode *pMnode, STrans *pTrans, SDbObj *pDb) {
SSdb *pSdb = pMnode->pSdb;
SSmaObj *pSma = NULL;
void *pIter = NULL;
SVgObj *pVgroup = NULL;
int32_t code = -1;
while (1) {
pIter = sdbFetch(pSdb, SDB_SMA, pIter, (void **)&pSma);
if (pIter == NULL) break;
if (pSma->dbUid == pDb->uid) {
pVgroup = mndAcquireVgroup(pMnode, pSma->dstVgId);
if (pVgroup == NULL) goto _OVER;
if (mndSetDropSmaVgroupCommitLogs(pMnode, pTrans, pVgroup) != 0) goto _OVER;
if (mndSetDropSmaVgroupRedoActions(pMnode, pTrans, pDb, pVgroup) != 0) goto _OVER;
if (mndSetDropSmaCommitLogs(pMnode, pTrans, pSma) != 0) goto _OVER;
mndReleaseVgroup(pMnode, pVgroup);
pVgroup = NULL;
}
sdbRelease(pSdb, pSma);
}
code = 0;
_OVER:
sdbCancelFetch(pSdb, pIter);
sdbRelease(pSdb, pSma);
mndReleaseVgroup(pMnode, pVgroup);
return code;
}
static int32_t mndProcessDropSmaReq(SRpcMsg *pReq) {
SMnode *pMnode = pReq->info.node;
int32_t code = -1;
SUserObj *pUser = NULL;
......@@ -901,13 +995,13 @@ static int32_t mndGetSma(SMnode *pMnode, SUserIndexReq *indexReq, SUserIndexRsp
}
int32_t mndGetTableSma(SMnode *pMnode, char *tbFName, STableIndexRsp *rsp, bool *exist) {
int32_t code = 0;
SSmaObj *pSma = NULL;
SSdb *pSdb = pMnode->pSdb;
void *pIter = NULL;
int32_t code = 0;
SSmaObj *pSma = NULL;
SSdb *pSdb = pMnode->pSdb;
void *pIter = NULL;
STableIndexInfo info;
SStbObj* pStb = mndAcquireStb(pMnode, tbFName);
SStbObj *pStb = mndAcquireStb(pMnode, tbFName);
if (NULL == pStb) {
*exist = false;
return TSDB_CODE_SUCCESS;
......@@ -936,14 +1030,14 @@ int32_t mndGetTableSma(SMnode *pMnode, char *tbFName, STableIndexRsp *rsp, bool
info.dstTbUid = pSma->dstTbUid;
info.dstVgId = pSma->dstVgId;
SVgObj* pVg = mndAcquireVgroup(pMnode, pSma->dstVgId);
SVgObj *pVg = mndAcquireVgroup(pMnode, pSma->dstVgId);
if (pVg == NULL) {
code = -1;
sdbRelease(pSdb, pSma);
return code;
}
info.epSet = mndGetVgroupEpset(pMnode, pVg);
info.expr = taosMemoryMalloc(pSma->exprLen + 1);
if (info.expr == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY;
......@@ -967,7 +1061,7 @@ int32_t mndGetTableSma(SMnode *pMnode, char *tbFName, STableIndexRsp *rsp, bool
sdbRelease(pSdb, pSma);
}
return code;
}
......@@ -1019,10 +1113,10 @@ _OVER:
static int32_t mndProcessGetTbSmaReq(SRpcMsg *pReq) {
STableIndexReq indexReq = {0};
SMnode *pMnode = pReq->info.node;
int32_t code = -1;
SMnode *pMnode = pReq->info.node;
int32_t code = -1;
STableIndexRsp rsp = {0};
bool exist = false;
bool exist = false;
if (tDeserializeSTableIndexReq(pReq->pCont, pReq->contLen, &indexReq) != 0) {
terrno = TSDB_CODE_INVALID_MSG;
......@@ -1069,7 +1163,6 @@ _OVER:
return code;
}
static int32_t mndRetrieveSma(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock *pBlock, int32_t rows) {
SMnode *pMnode = pReq->info.node;
SSdb *pSdb = pMnode->pSdb;
......
......@@ -23,6 +23,7 @@
#include "mndPerfSchema.h"
#include "mndScheduler.h"
#include "mndShow.h"
#include "mndSma.h"
#include "mndTopic.h"
#include "mndTrans.h"
#include "mndUser.h"
......@@ -37,9 +38,9 @@ static SSdbRow *mndStbActionDecode(SSdbRaw *pRaw);
static int32_t mndStbActionInsert(SSdb *pSdb, SStbObj *pStb);
static int32_t mndStbActionDelete(SSdb *pSdb, SStbObj *pStb);
static int32_t mndStbActionUpdate(SSdb *pSdb, SStbObj *pOld, SStbObj *pNew);
static int32_t mndProcessMCreateStbReq(SRpcMsg *pReq);
static int32_t mndProcessMAlterStbReq(SRpcMsg *pReq);
static int32_t mndProcessMDropStbReq(SRpcMsg *pReq);
static int32_t mndProcessCreateStbReq(SRpcMsg *pReq);
static int32_t mndProcessAlterStbReq(SRpcMsg *pReq);
static int32_t mndProcessDropStbReq(SRpcMsg *pReq);
static int32_t mndProcessTableMetaReq(SRpcMsg *pReq);
static int32_t mndRetrieveStb(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock *pBlock, int32_t rows);
static void mndCancelGetNextStb(SMnode *pMnode, void *pIter);
......@@ -55,9 +56,9 @@ int32_t mndInitStb(SMnode *pMnode) {
.deleteFp = (SdbDeleteFp)mndStbActionDelete,
};
mndSetMsgHandle(pMnode, TDMT_MND_CREATE_STB, mndProcessMCreateStbReq);
mndSetMsgHandle(pMnode, TDMT_MND_ALTER_STB, mndProcessMAlterStbReq);
mndSetMsgHandle(pMnode, TDMT_MND_DROP_STB, mndProcessMDropStbReq);
mndSetMsgHandle(pMnode, TDMT_MND_CREATE_STB, mndProcessCreateStbReq);
mndSetMsgHandle(pMnode, TDMT_MND_ALTER_STB, mndProcessAlterStbReq);
mndSetMsgHandle(pMnode, TDMT_MND_DROP_STB, mndProcessDropStbReq);
mndSetMsgHandle(pMnode, TDMT_VND_CREATE_STB_RSP, mndTransProcessRsp);
mndSetMsgHandle(pMnode, TDMT_VND_ALTER_STB_RSP, mndTransProcessRsp);
mndSetMsgHandle(pMnode, TDMT_VND_DROP_STB_RSP, mndTransProcessRsp);
......@@ -319,6 +320,7 @@ static int32_t mndStbActionUpdate(SSdb *pSdb, SStbObj *pOld, SStbObj *pNew) {
pOld->updateTime = pNew->updateTime;
pOld->tagVer = pNew->tagVer;
pOld->colVer = pNew->colVer;
pOld->smaVer = pNew->smaVer;
pOld->nextColId = pNew->nextColId;
pOld->ttl = pNew->ttl;
pOld->numOfColumns = pNew->numOfColumns;
......@@ -362,7 +364,7 @@ SDbObj *mndAcquireDbByStb(SMnode *pMnode, const char *stbName) {
return mndAcquireDb(pMnode, db);
}
static FORCE_INLINE int schemaExColIdCompare(const void *colId, const void *pSchema) {
static FORCE_INLINE int32_t schemaExColIdCompare(const void *colId, const void *pSchema) {
if (*(col_id_t *)colId < ((SSchema *)pSchema)->colId) {
return -1;
} else if (*(col_id_t *)colId > ((SSchema *)pSchema)->colId) {
......@@ -396,14 +398,14 @@ static void *mndBuildVCreateStbReq(SMnode *pMnode, SVgObj *pVgroup, SStbObj *pSt
req.pRSmaParam.xFilesFactor = pStb->xFilesFactor;
req.pRSmaParam.delay = pStb->delay;
if (pStb->ast1Len > 0) {
if (mndConvertRsmaTask(&req.pRSmaParam.qmsg1, &req.pRSmaParam.qmsg1Len, pStb->pAst1, pStb->uid, STREAM_TRIGGER_AT_ONCE, 0,
req.pRSmaParam.xFilesFactor) != TSDB_CODE_SUCCESS) {
if (mndConvertRsmaTask(&req.pRSmaParam.qmsg1, &req.pRSmaParam.qmsg1Len, pStb->pAst1, pStb->uid,
STREAM_TRIGGER_AT_ONCE, 0, req.pRSmaParam.xFilesFactor) != TSDB_CODE_SUCCESS) {
return NULL;
}
}
if (pStb->ast2Len > 0) {
if (mndConvertRsmaTask(&req.pRSmaParam.qmsg2, &req.pRSmaParam.qmsg2Len, pStb->pAst2, pStb->uid, STREAM_TRIGGER_AT_ONCE, 0,
req.pRSmaParam.xFilesFactor) != TSDB_CODE_SUCCESS) {
if (mndConvertRsmaTask(&req.pRSmaParam.qmsg2, &req.pRSmaParam.qmsg2Len, pStb->pAst2, pStb->uid,
STREAM_TRIGGER_AT_ONCE, 0, req.pRSmaParam.xFilesFactor) != TSDB_CODE_SUCCESS) {
return NULL;
}
}
......@@ -762,7 +764,7 @@ int32_t mndAddStbToTrans(SMnode *pMnode, STrans *pTrans, SDbObj *pDb, SStbObj *p
return 0;
}
static int32_t mndProcessMCreateStbReq(SRpcMsg *pReq) {
static int32_t mndProcessCreateStbReq(SRpcMsg *pReq) {
SMnode *pMnode = pReq->info.node;
int32_t code = -1;
SStbObj *pStb = NULL;
......@@ -1301,7 +1303,7 @@ static int32_t mndBuildStbSchema(SMnode *pMnode, const char *dbFName, const char
static int32_t mndBuildSMAlterStbRsp(SDbObj *pDb, const SMAlterStbReq *pAlter, SStbObj *pObj, void **pCont,
int32_t *pLen) {
int ret;
int32_t ret;
SEncoder ec = {0};
uint32_t contLen = 0;
SMAlterStbRsp alterRsp = {0};
......@@ -1420,7 +1422,7 @@ _OVER:
return code;
}
static int32_t mndProcessMAlterStbReq(SRpcMsg *pReq) {
static int32_t mndProcessAlterStbReq(SRpcMsg *pReq) {
SMnode *pMnode = pReq->info.node;
int32_t code = -1;
SDbObj *pDb = NULL;
......@@ -1550,6 +1552,7 @@ static int32_t mndDropStb(SMnode *pMnode, SRpcMsg *pReq, SDbObj *pDb, SStbObj *p
if (mndSetDropStbRedoLogs(pMnode, pTrans, pStb) != 0) goto _OVER;
if (mndSetDropStbCommitLogs(pMnode, pTrans, pStb) != 0) goto _OVER;
if (mndSetDropStbRedoActions(pMnode, pTrans, pDb, pStb) != 0) goto _OVER;
if (mndDropSmasByStb(pMnode, pTrans, pDb, pStb) != 0) goto _OVER;
if (mndTransPrepare(pMnode, pTrans) != 0) goto _OVER;
code = 0;
......@@ -1559,7 +1562,7 @@ _OVER:
return code;
}
static int32_t mndProcessMDropStbReq(SRpcMsg *pReq) {
static int32_t mndProcessDropStbReq(SRpcMsg *pReq) {
SMnode *pMnode = pReq->info.node;
int32_t code = -1;
SUserObj *pUser = NULL;
......
......@@ -236,7 +236,7 @@ int32_t mndSyncPropose(SMnode *pMnode, SSdbRaw *pRaw, int32_t transId) {
tsem_wait(&pMgmt->syncSem);
} else if (code == TAOS_SYNC_PROPOSE_NOT_LEADER) {
terrno = TSDB_CODE_APP_NOT_READY;
} else if (code == TAOS_SYNC_PROPOSE_OTHER_ERROR) {
} else if (code == TAOS_SYNC_OTHER_ERROR) {
terrno = TSDB_CODE_SYN_INTERNAL_ERROR;
} else {
terrno = TSDB_CODE_APP_ERROR;
......@@ -254,13 +254,16 @@ int32_t mndSyncPropose(SMnode *pMnode, SSdbRaw *pRaw, int32_t transId) {
void mndSyncStart(SMnode *pMnode) {
SSyncMgmt *pMgmt = &pMnode->syncMgmt;
syncSetMsgCb(pMgmt->sync, &pMnode->msgCb);
syncStart(pMgmt->sync);
mDebug("mnode sync started, id:%" PRId64 " standby:%d", pMgmt->sync, pMgmt->standby);
/*
if (pMgmt->standby) {
syncStartStandBy(pMgmt->sync);
} else {
syncStart(pMgmt->sync);
}
mDebug("mnode sync started, id:%" PRId64 " standby:%d", pMgmt->sync, pMgmt->standby);
*/
}
void mndSyncStop(SMnode *pMnode) {}
......
......@@ -41,7 +41,6 @@ extern "C" {
#define tqTrace(...) do { if (tqDebugFlag & DEBUG_TRACE) { taosPrintLog("TQ ", DEBUG_TRACE, tqDebugFlag, __VA_ARGS__); }} while(0)
// clang-format on
typedef struct STqOffsetCfg STqOffsetCfg;
typedef struct STqOffsetStore STqOffsetStore;
// tqRead
......@@ -127,14 +126,15 @@ typedef struct {
} STqHandle;
struct STQ {
char* path;
SHashObj* pushMgr; // consumerId -> STqHandle*
SHashObj* handles; // subKey -> STqHandle
SHashObj* pStreamTasks; // taksId -> SStreamTask
SVnode* pVnode;
SWal* pWal;
TDB* pMetaStore;
TTB* pExecStore;
char* path;
SHashObj* pushMgr; // consumerId -> STqHandle*
SHashObj* handles; // subKey -> STqHandle
SHashObj* pStreamTasks; // taksId -> SStreamTask
STqOffsetStore* pOffsetStore;
SVnode* pVnode;
SWal* pWal;
TDB* pMetaStore;
TTB* pExecStore;
};
typedef struct {
......@@ -157,16 +157,18 @@ int32_t tqMetaClose(STQ* pTq);
int32_t tqMetaSaveHandle(STQ* pTq, const char* key, const STqHandle* pHandle);
int32_t tqMetaDeleteHandle(STQ* pTq, const char* key);
// tqSink
void tqTableSink(SStreamTask* pTask, void* vnode, int64_t ver, void* data);
typedef struct {
int32_t size;
} STqOffsetHead;
// tqOffset
STqOffsetStore* tqOffsetOpen(STqOffsetCfg*);
STqOffsetStore* tqOffsetOpen();
void tqOffsetClose(STqOffsetStore*);
int64_t tqOffsetFetch(STqOffsetStore* pStore, const char* subscribeKey);
int32_t tqOffsetCommit(STqOffsetStore* pStore, const char* subscribeKey, int64_t offset);
int32_t tqOffsetPersist(STqOffsetStore* pStore, const char* subscribeKey);
int32_t tqOffsetPersistAll(STqOffsetStore* pStore);
STqOffset* tqOffsetRead(STqOffsetStore* pStore, const char* subscribeKey);
int32_t tqOffsetWrite(STqOffsetStore* pStore, const STqOffset* pOffset);
int32_t tqOffsetSnapshot(STqOffsetStore* pStore);
// tqSink
void tqTableSink(SStreamTask* pTask, void* vnode, int64_t ver, void* data);
#ifdef __cplusplus
}
......
......@@ -137,6 +137,7 @@ int tqCommit(STQ*);
int32_t tqUpdateTbUidList(STQ* pTq, const SArray* tbUidList, bool isAdd);
int32_t tqProcessVgChangeReq(STQ* pTq, char* msg, int32_t msgLen);
int32_t tqProcessVgDeleteReq(STQ* pTq, char* msg, int32_t msgLen);
int32_t tqProcessOffsetCommitReq(STQ* pTq, char* msg, int32_t msgLen);
int32_t tqProcessPollReq(STQ* pTq, SRpcMsg* pMsg, int32_t workerId);
int32_t tqProcessTaskDeploy(STQ* pTq, char* msg, int32_t msgLen);
int32_t tqProcessStreamTrigger(STQ* pTq, SSubmitReq* data);
......
......@@ -47,7 +47,7 @@ void tqCleanUp() {
}
STQ* tqOpen(const char* path, SVnode* pVnode, SWal* pWal) {
STQ* pTq = taosMemoryMalloc(sizeof(STQ));
STQ* pTq = taosMemoryCalloc(1, sizeof(STQ));
if (pTq == NULL) {
terrno = TSDB_CODE_TQ_OUT_OF_MEMORY;
return NULL;
......@@ -66,19 +66,23 @@ STQ* tqOpen(const char* path, SVnode* pVnode, SWal* pWal) {
ASSERT(0);
}
if (tqOffsetOpen(pTq) < 0) {
ASSERT(0);
}
return pTq;
}
void tqClose(STQ* pTq) {
if (pTq) {
taosMemoryFreeClear(pTq->path);
tqOffsetClose(pTq->pOffsetStore);
taosHashCleanup(pTq->handles);
taosHashCleanup(pTq->pStreamTasks);
taosHashCleanup(pTq->pushMgr);
taosMemoryFree(pTq->path);
tqMetaClose(pTq);
taosMemoryFree(pTq);
}
// TODO
}
int32_t tqSendPollRsp(STQ* pTq, const SRpcMsg* pMsg, const SMqPollReq* pReq, const SMqDataBlkRsp* pRsp) {
......@@ -109,6 +113,33 @@ int32_t tqSendPollRsp(STQ* pTq, const SRpcMsg* pMsg, const SMqPollReq* pReq, con
return 0;
}
int32_t tqProcessOffsetCommitReq(STQ* pTq, char* msg, int32_t msgLen) {
STqOffset offset = {0};
SDecoder decoder;
tDecoderInit(&decoder, msg, msgLen);
if (tDecodeSTqOffset(&decoder, &offset) < 0) {
ASSERT(0);
return -1;
}
tDecoderClear(&decoder);
if (offset.type == TMQ_OFFSET__SNAPSHOT) {
tqDebug("receive offset commit msg to %s, offset(type:snapshot) uid: %ld, ts: %ld", offset.subKey, offset.uid,
offset.ts);
} else if (offset.type == TMQ_OFFSET__LOG) {
tqDebug("receive offset commit msg to %s, offset(type:log) version: %ld", offset.subKey, offset.version);
} else {
ASSERT(0);
}
if (tqOffsetWrite(pTq->pOffsetStore, &offset) < 0) {
ASSERT(0);
return -1;
}
return 0;
}
int32_t tqProcessPollReq(STQ* pTq, SRpcMsg* pMsg, int32_t workerId) {
SMqPollReq* pReq = pMsg->pCont;
int64_t consumerId = pReq->consumerId;
......
......@@ -16,26 +16,128 @@
#include "tq.h"
enum ETqOffsetPersist {
TQ_OFFSET_PERSIST__LAZY = 1,
TQ_OFFSET_PERSIST__EAGER,
};
struct STqOffsetCfg {
int8_t persistPolicy;
};
struct STqOffsetStore {
STqOffsetCfg cfg;
SHashObj* pHash; // SHashObj<subscribeKey, offset>
char* fname;
STQ* pTq;
SHashObj* pHash; // SHashObj<subscribeKey, offset>
};
STqOffsetStore* tqOffsetOpen(STqOffsetCfg* pCfg) {
STqOffsetStore* pStore = taosMemoryMalloc(sizeof(STqOffsetStore));
static char* buildFileName(const char* path) {
int32_t len = strlen(path);
char* fname = taosMemoryCalloc(1, len + 20);
snprintf(fname, len + 20, "%s/offset", path);
return fname;
}
STqOffsetStore* tqOffsetOpen(STQ* pTq) {
STqOffsetStore* pStore = taosMemoryCalloc(1, sizeof(STqOffsetStore));
if (pStore == NULL) {
return NULL;
}
memcpy(&pStore->cfg, pCfg, sizeof(STqOffsetCfg));
pStore->pTq = pTq;
pTq->pOffsetStore = pStore;
pStore->pHash = taosHashInit(64, MurmurHash3_32, true, HASH_NO_LOCK);
if (pStore->pHash == NULL) {
if (pStore->pHash) taosHashCleanup(pStore->pHash);
return NULL;
}
char* fname = buildFileName(pStore->pTq->path);
TdFilePtr pFile = taosOpenFile(fname, TD_FILE_READ);
if (pFile != NULL) {
STqOffsetHead head = {0};
int64_t code;
while (1) {
if ((code = taosReadFile(pFile, &head, sizeof(STqOffsetHead))) != sizeof(STqOffsetHead)) {
if (code == 0) {
break;
} else {
ASSERT(0);
// TODO handle error
}
}
int32_t size = htonl(head.size);
void* memBuf = taosMemoryCalloc(1, size);
if ((code = taosReadFile(pFile, memBuf, size)) != size) {
ASSERT(0);
// TODO handle error
}
STqOffset offset;
SDecoder decoder;
tDecoderInit(&decoder, memBuf, size);
if (tDecodeSTqOffset(&decoder, &offset) < 0) {
ASSERT(0);
}
tDecoderClear(&decoder);
if (taosHashPut(pStore->pHash, offset.subKey, strlen(offset.subKey), &offset, sizeof(STqOffset)) < 0) {
ASSERT(0);
// TODO
}
}
taosCloseFile(&pFile);
taosMemoryFree(fname);
}
return pStore;
}
void tqOffsetClose(STqOffsetStore* pStore) {
tqOffsetSnapshot(pStore);
taosHashCleanup(pStore->pHash);
}
STqOffset* tqOffsetRead(STqOffsetStore* pStore, const char* subscribeKey) {
return (STqOffset*)taosHashGet(pStore->pHash, subscribeKey, strlen(subscribeKey));
}
int32_t tqOffsetWrite(STqOffsetStore* pStore, const STqOffset* pOffset) {
return taosHashPut(pStore->pHash, pOffset->subKey, strlen(pOffset->subKey), pOffset, sizeof(STqOffset));
}
int32_t tqOffsetSnapshot(STqOffsetStore* pStore) {
// open file
// TODO file name should be with a version
char* fname = buildFileName(pStore->pTq->path);
TdFilePtr pFile = taosOpenFile(fname, TD_FILE_CREATE | TD_FILE_WRITE | TD_FILE_APPEND);
if (pFile == NULL) {
ASSERT(0);
return -1;
}
void* pIter = NULL;
while (1) {
pIter = taosHashIterate(pStore->pHash, pIter);
if (pIter == NULL) break;
STqOffset* pOffset = (STqOffset*)pIter;
int32_t bodyLen;
int32_t code;
tEncodeSize(tEncodeSTqOffset, pOffset, bodyLen, code);
ASSERT(code == 0);
if (code < 0) {
ASSERT(0);
taosHashCancelIterate(pStore->pHash, pIter);
return -1;
}
int32_t totLen = sizeof(STqOffsetHead) + bodyLen;
void* buf = taosMemoryCalloc(1, totLen);
void* abuf = POINTER_SHIFT(buf, sizeof(STqOffsetHead));
((STqOffsetHead*)buf)->size = htonl(bodyLen);
SEncoder encoder;
tEncoderInit(&encoder, abuf, bodyLen);
tEncodeSTqOffset(&encoder, pOffset);
// write file
int64_t writeLen;
if ((writeLen = taosWriteFile(pFile, buf, totLen)) != bodyLen) {
ASSERT(0);
tqError("write offset incomplete, len %d, write len %ld", bodyLen, writeLen);
taosHashCancelIterate(pStore->pHash, pIter);
return -1;
}
}
// close and rename file
taosCloseFile(&pFile);
taosMemoryFree(fname);
return 0;
}
......@@ -15,6 +15,177 @@
#include "tq.h"
static SSubmitReq* tdBlockToSubmit(const SArray* pBlocks, const STSchema* pSchema, bool createTb, int64_t suid,
const char* stbFullName, int32_t vgId);
static SSubmitReq* tdBlockToSubmit(const SArray* pBlocks, const STSchema* pTSchema, bool createTb, int64_t suid,
const char* stbFullName, int32_t vgId) {
SSubmitReq* ret = NULL;
SArray* tagArray = taosArrayInit(1, sizeof(STagVal));
if (!tagArray) {
terrno = TSDB_CODE_OUT_OF_MEMORY;
return NULL;
}
// cal size
int32_t cap = sizeof(SSubmitReq);
int32_t sz = taosArrayGetSize(pBlocks);
for (int32_t i = 0; i < sz; i++) {
SSDataBlock* pDataBlock = taosArrayGet(pBlocks, i);
int32_t rows = pDataBlock->info.rows;
// TODO min
int32_t rowSize = pDataBlock->info.rowSize;
int32_t maxLen = TD_ROW_MAX_BYTES_FROM_SCHEMA(pTSchema);
int32_t schemaLen = 0;
if (createTb) {
SVCreateTbReq createTbReq = {0};
char* cname = buildCtbNameByGroupId(stbFullName, pDataBlock->info.groupId);
createTbReq.name = cname;
createTbReq.flags = 0;
createTbReq.type = TSDB_CHILD_TABLE;
createTbReq.ctb.suid = suid;
STagVal tagVal = {
.cid = pDataBlock->info.numOfCols + 1,
.type = TSDB_DATA_TYPE_UBIGINT,
.i64 = (int64_t)pDataBlock->info.groupId,
};
STag* pTag = NULL;
taosArrayClear(tagArray);
taosArrayPush(tagArray, &tagVal);
tTagNew(tagArray, 1, false, &pTag);
if (pTag == NULL) {
tdDestroySVCreateTbReq(&createTbReq);
taosArrayDestroy(tagArray);
return NULL;
}
createTbReq.ctb.pTag = (uint8_t*)pTag;
int32_t code;
tEncodeSize(tEncodeSVCreateTbReq, &createTbReq, schemaLen, code);
tdDestroySVCreateTbReq(&createTbReq);
if (code < 0) {
taosArrayDestroy(tagArray);
return NULL;
}
}
cap += sizeof(SSubmitBlk) + schemaLen + rows * maxLen;
}
// assign data
// TODO
ret = rpcMallocCont(cap);
ret->header.vgId = vgId;
ret->version = htonl(1);
ret->length = sizeof(SSubmitReq);
ret->numOfBlocks = htonl(sz);
void* submitBlk = POINTER_SHIFT(ret, sizeof(SSubmitReq));
for (int32_t i = 0; i < sz; i++) {
SSDataBlock* pDataBlock = taosArrayGet(pBlocks, i);
SSubmitBlk* blkHead = submitBlk;
blkHead->numOfRows = htons(pDataBlock->info.rows);
blkHead->sversion = htonl(pTSchema->version);
// TODO
blkHead->suid = htobe64(suid);
// uid is assigned by vnode
blkHead->uid = 0;
int32_t rows = pDataBlock->info.rows;
/*int32_t maxLen = TD_ROW_MAX_BYTES_FROM_SCHEMA(pTSchema);*/
/*blkHead->dataLen = htonl(rows * maxLen);*/
blkHead->dataLen = 0;
void* blockData = POINTER_SHIFT(submitBlk, sizeof(SSubmitBlk));
int32_t schemaLen = 0;
if (createTb) {
SVCreateTbReq createTbReq = {0};
char* cname = buildCtbNameByGroupId(stbFullName, pDataBlock->info.groupId);
createTbReq.name = cname;
createTbReq.flags = 0;
createTbReq.type = TSDB_CHILD_TABLE;
createTbReq.ctb.suid = suid;
STagVal tagVal = {
.cid = pDataBlock->info.numOfCols + 1,
.type = TSDB_DATA_TYPE_UBIGINT,
.i64 = (int64_t)pDataBlock->info.groupId,
};
taosArrayClear(tagArray);
taosArrayPush(tagArray, &tagVal);
STag* pTag = NULL;
tTagNew(tagArray, 1, false, &pTag);
if (pTag == NULL) {
tdDestroySVCreateTbReq(&createTbReq);
taosArrayDestroy(tagArray);
taosMemoryFreeClear(ret);
return NULL;
}
createTbReq.ctb.pTag = (uint8_t*)pTag;
int32_t code;
tEncodeSize(tEncodeSVCreateTbReq, &createTbReq, schemaLen, code);
if (code < 0) {
tdDestroySVCreateTbReq(&createTbReq);
taosArrayDestroy(tagArray);
taosMemoryFreeClear(ret);
return NULL;
}
SEncoder encoder = {0};
tEncoderInit(&encoder, blockData, schemaLen);
code = tEncodeSVCreateTbReq(&encoder, &createTbReq);
tEncoderClear(&encoder);
tdDestroySVCreateTbReq(&createTbReq);
if (code < 0) {
taosArrayDestroy(tagArray);
taosMemoryFreeClear(ret);
return NULL;
}
}
blkHead->schemaLen = htonl(schemaLen);
STSRow* rowData = POINTER_SHIFT(blockData, schemaLen);
for (int32_t j = 0; j < rows; j++) {
SRowBuilder rb = {0};
tdSRowInit(&rb, pTSchema->version);
tdSRowSetTpInfo(&rb, pTSchema->numOfCols, pTSchema->flen);
tdSRowResetBuf(&rb, rowData);
for (int32_t k = 0; k < pTSchema->numOfCols; k++) {
const STColumn* pColumn = &pTSchema->columns[k];
SColumnInfoData* pColData = taosArrayGet(pDataBlock->pDataBlock, k);
if (colDataIsNull_s(pColData, j)) {
tdAppendColValToRow(&rb, pColumn->colId, pColumn->type, TD_VTYPE_NONE, NULL, false, pColumn->offset, k);
} else {
void* data = colDataGetData(pColData, j);
tdAppendColValToRow(&rb, pColumn->colId, pColumn->type, TD_VTYPE_NORM, data, true, pColumn->offset, k);
}
}
int32_t rowLen = TD_ROW_LEN(rowData);
rowData = POINTER_SHIFT(rowData, rowLen);
blkHead->dataLen += rowLen;
}
int32_t dataLen = blkHead->dataLen;
blkHead->dataLen = htonl(dataLen);
ret->length += sizeof(SSubmitBlk) + schemaLen + dataLen;
blkHead = POINTER_SHIFT(blkHead, schemaLen + dataLen);
/*submitBlk = blkHead;*/
}
ret->length = htonl(ret->length);
taosArrayDestroy(tagArray);
return ret;
}
void tqTableSink(SStreamTask* pTask, void* vnode, int64_t ver, void* data) {
const SArray* pRes = (const SArray*)data;
SVnode* pVnode = (SVnode*)vnode;
......
......@@ -148,17 +148,24 @@ int32_t vnodeProcessWriteReq(SVnode *pVnode, SRpcMsg *pMsg, int64_t version, SRp
case TDMT_VND_MQ_VG_CHANGE:
if (tqProcessVgChangeReq(pVnode->pTq, POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)),
pMsg->contLen - sizeof(SMsgHead)) < 0) {
// TODO: handle error
goto _err;
}
break;
case TDMT_VND_MQ_VG_DELETE:
if (tqProcessVgDeleteReq(pVnode->pTq, pMsg->pCont, pMsg->contLen) < 0) {
// TODO: handle error
goto _err;
}
break;
case TDMT_VND_MQ_COMMIT_OFFSET:
if (tqProcessOffsetCommitReq(pVnode->pTq, POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)),
pMsg->contLen - sizeof(SMsgHead)) < 0) {
goto _err;
}
break;
case TDMT_STREAM_TASK_DEPLOY: {
if (tqProcessTaskDeploy(pVnode->pTq, POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)),
pMsg->contLen - sizeof(SMsgHead)) < 0) {
goto _err;
}
} break;
case TDMT_VND_ALTER_CONFIRM:
......@@ -289,7 +296,7 @@ void vnodeUpdateMetaRsp(SVnode *pVnode, STableMetaRsp *pMetaRsp) {
}
int32_t vnodeProcessSyncReq(SVnode *pVnode, SRpcMsg *pMsg, SRpcMsg **pRsp) {
int32_t ret = TAOS_SYNC_PROPOSE_OTHER_ERROR;
int32_t ret = TAOS_SYNC_OTHER_ERROR;
if (syncEnvIsStart()) {
SSyncNode *pSyncNode = syncNodeAcquire(pVnode->sync);
......@@ -370,13 +377,13 @@ int32_t vnodeProcessSyncReq(SVnode *pVnode, SRpcMsg *pMsg, SRpcMsg **pRsp) {
} else {
vError("==vnodeProcessSyncReq== error msg type:%d", pRpcMsg->msgType);
ret = TAOS_SYNC_PROPOSE_OTHER_ERROR;
ret = TAOS_SYNC_OTHER_ERROR;
}
syncNodeRelease(pSyncNode);
} else {
vError("==vnodeProcessSyncReq== error syncEnv stop");
ret = TAOS_SYNC_PROPOSE_OTHER_ERROR;
ret = TAOS_SYNC_OTHER_ERROR;
}
return ret;
......@@ -901,8 +908,8 @@ static int32_t vnodeProcessAlterConfirmReq(SVnode *pVnode, int64_t version, void
static int32_t vnodeProcessAlterHasnRangeReq(SVnode *pVnode, int64_t version, void *pReq, int32_t len, SRpcMsg *pRsp) {
vInfo("vgId:%d, alter hashrange msg will be processed", TD_VID(pVnode));
// todo
// 1. stop work
// todo
// 1. stop work
// 2. adjust hash range / compact / remove wals / rename vgroups
// 3. reload sync
return 0;
......
......@@ -283,11 +283,14 @@ int32_t vnodeSyncOpen(SVnode *pVnode, char *path) {
void vnodeSyncStart(SVnode *pVnode) {
syncSetMsgCb(pVnode->sync, &pVnode->msgCb);
syncStart(pVnode->sync);
/*
if (pVnode->config.standby) {
syncStartStandBy(pVnode->sync);
} else {
syncStart(pVnode->sync);
}
*/
}
void vnodeSyncClose(SVnode *pVnode) { syncStop(pVnode->sync); }
......@@ -27,10 +27,7 @@ typedef struct {
int32_t bytes;
} SGroupKeys, SStateKeys;
int32_t initGroupOptrInfo(SArray** pGroupColVals, int32_t* keyLen, char** keyBuf, const SArray* pGroupColList);
uint64_t calcGroupId(char* pData, int32_t len);
void recordNewGroupKeys(SArray* pGroupCols, SArray* pGroupColVals, SSDataBlock* pBlock, int32_t rowIndex);
int32_t buildGroupKeys(void* pKey, const SArray* pGroupColVals);
#ifdef __cplusplus
}
#endif
......
......@@ -336,12 +336,6 @@ typedef struct STableScanInfo {
int32_t dataBlockLoadFlag;
SInterval interval; // if the upstream is an interval operator, the interval info is also kept here to get the time window to check if current data block needs to be loaded.
SArray* pGroupCols;
SArray* pGroupColVals; // current group column values, SArray<SGroupKeys>
char* keyBuf; // group by keys for hash
int32_t groupKeyLen; // total group by column width
SHashObj* pGroupSet; // quick locate the window object for each result
SSampleExecInfo sample; // sample execution info
int32_t curTWinIdx;
} STableScanInfo;
......@@ -789,7 +783,7 @@ SResultRow* doSetResultOutBufByKey(SDiskbasedBuf* pResultBuf, SResultRowInfo* pR
SOperatorInfo* createExchangeOperatorInfo(void* pTransporter, SExchangePhysiNode* pExNode, SExecTaskInfo* pTaskInfo);
SOperatorInfo* createTableScanOperatorInfo(STableScanPhysiNode* pTableScanNode, tsdbReaderT pDataReader, SReadHandle* pHandle, SArray* groupKyes, SExecTaskInfo* pTaskInfo);
SOperatorInfo* createTableScanOperatorInfo(STableScanPhysiNode* pTableScanNode, tsdbReaderT pDataReader, SReadHandle* pHandle, SExecTaskInfo* pTaskInfo);
SOperatorInfo* createTagScanOperatorInfo(SReadHandle* pReadHandle, STagScanPhysiNode* pPhyNode,
STableListInfo* pTableListInfo, SExecTaskInfo* pTaskInfo);
SOperatorInfo* createSysTableScanOperatorInfo(void* readHandle, SSystemTableScanPhysiNode *pScanPhyNode, SExecTaskInfo* pTaskInfo);
......@@ -827,7 +821,7 @@ SOperatorInfo* createGroupOperatorInfo(SOperatorInfo* downstream, SExprInfo* pEx
SExprInfo* pScalarExprInfo, int32_t numOfScalarExpr, SExecTaskInfo* pTaskInfo);
SOperatorInfo* createDataBlockInfoScanOperator(void* dataReader, SExecTaskInfo* pTaskInfo);
SOperatorInfo* createStreamScanOperatorInfo(void* pDataReader, SReadHandle* pHandle, SArray* pTableIdList,
SOperatorInfo* createStreamScanOperatorInfo(void* pDataReader, SReadHandle* pHandle,
STableScanPhysiNode* pTableScanNode, SExecTaskInfo* pTaskInfo, STimeWindowAggSupp* pTwSup);
......
......@@ -4558,7 +4558,6 @@ static tsdbReaderT doCreateDataReader(STableScanPhysiNode* pTableScanNode, SRead
static int32_t getTableList(void* metaHandle, int32_t tableType, uint64_t tableUid, STableListInfo* pListInfo,
SNode* pTagCond);
static SArray* extractTableIdList(const STableListInfo* pTableGroupInfo);
static SArray* extractColumnInfo(SNodeList* pNodeList);
static SArray* createSortInfo(SNodeList* pNodeList);
......@@ -4592,6 +4591,85 @@ int32_t extractTableSchemaVersion(SReadHandle* pHandle, uint64_t uid, SExecTaskI
return TSDB_CODE_SUCCESS;
}
int32_t generateGroupIdMap(STableListInfo* pTableListInfo, SReadHandle* pHandle, SArray* groupKey){
if(groupKey == NULL) {
return TDB_CODE_SUCCESS;
}
pTableListInfo->map = taosHashInit(32, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), false, HASH_NO_LOCK);
if (pTableListInfo->map == NULL) {
return TSDB_CODE_OUT_OF_MEMORY;
}
int32_t keyLen = 0;
void *keyBuf = NULL;
int32_t numOfGroupCols = taosArrayGetSize(groupKey);
for (int32_t j = 0; j < numOfGroupCols; ++j) {
SColumn* pCol = taosArrayGet(groupKey, j);
keyLen += pCol->bytes; // actual data + null_flag
}
int32_t nullFlagSize = sizeof(int8_t) * numOfGroupCols;
keyLen += nullFlagSize;
keyBuf = taosMemoryCalloc(1, keyLen);
if (keyBuf == NULL) {
return TSDB_CODE_OUT_OF_MEMORY;
}
for(int32_t i = 0; i < taosArrayGetSize(pTableListInfo->pTableList); i++){
STableKeyInfo *info = taosArrayGet(pTableListInfo->pTableList, i);
SMetaReader mr = {0};
metaReaderInit(&mr, pHandle->meta, 0);
metaGetTableEntryByUid(&mr, info->uid);
char* isNull = (char*)keyBuf;
char* pStart = (char*)keyBuf + sizeof(int8_t) * numOfGroupCols;
for (int32_t j = 0; j < numOfGroupCols; ++j) {
SColumn* pCol = taosArrayGet(groupKey, j);
if(strcmp(pCol->name, "tbname") == 0){
isNull[i] = 0;
memcpy(pStart, mr.me.name, strlen(mr.me.name));
pStart += strlen(mr.me.name);
}else{
STagVal tagVal = {0};
tagVal.cid = pCol->colId;
const char* p = metaGetTableTagVal(&mr.me, pCol->type, &tagVal);
if(p == NULL){
isNull[j] = 1;
continue;
}
isNull[i] = 0;
if (pCol->type == TSDB_DATA_TYPE_JSON) {
// int32_t dataLen = getJsonValueLen(pkey->pData);
// memcpy(pStart, (pkey->pData), dataLen);
// pStart += dataLen;
} else if (IS_VAR_DATA_TYPE(pCol->type)) {
memcpy(pStart, tagVal.pData, tagVal.nData);
pStart += tagVal.nData;
ASSERT(tagVal.nData <= pCol->bytes);
} else {
memcpy(pStart, &(tagVal.i64), pCol->bytes);
pStart += pCol->bytes;
}
}
}
int32_t len = (int32_t) (pStart - (char*)keyBuf);
uint64_t* groupId = taosHashGet(pTableListInfo->map, keyBuf, len);
if (groupId) {
taosHashPut(pTableListInfo->map, &(info->uid), sizeof(uint64_t), groupId, sizeof(uint64_t));
} else {
uint64_t tmpId = calcGroupId(keyBuf, len);
taosHashPut(pTableListInfo->map, &(info->uid), sizeof(uint64_t), &tmpId, sizeof(uint64_t));
}
metaReaderClear(&mr);
}
taosMemoryFree(keyBuf);
return TDB_CODE_SUCCESS;
}
SOperatorInfo* createOperatorTree(SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo, SReadHandle* pHandle,
uint64_t queryId, uint64_t taskId, STableListInfo* pTableListInfo, SNode* pTagCond) {
int32_t type = nodeType(pPhyNode);
......@@ -4605,15 +4683,23 @@ SOperatorInfo* createOperatorTree(SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo
if (pDataReader == NULL && terrno != 0) {
return NULL;
}
SArray* groupKyes = extractPartitionColInfo(pTableScanNode->pPartitionKeys);
int32_t code = extractTableSchemaVersion(pHandle, pTableScanNode->scan.uid, pTaskInfo);
if (code) {
tsdbCleanupReadHandle(pDataReader);
return NULL;
}
SArray* groupKeys = extractPartitionColInfo(pTableScanNode->pPartitionKeys);
code = generateGroupIdMap(pTableListInfo, pHandle, groupKeys); //todo for json
taosArrayDestroy(groupKeys);
if (code){
tsdbCleanupReadHandle(pDataReader);
return NULL;
}
SOperatorInfo* pOperator =
createTableScanOperatorInfo(pTableScanNode, pDataReader, pHandle, groupKyes, pTaskInfo);
createTableScanOperatorInfo(pTableScanNode, pDataReader, pHandle, pTaskInfo);
STableScanInfo* pScanInfo = pOperator->info;
pTaskInfo->cost.pRecoder = &pScanInfo->readRecorder;
......@@ -4639,12 +4725,18 @@ SOperatorInfo* createOperatorTree(SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo
} else {
qDebug("%s pDataReader is not NULL", GET_TASKID(pTaskInfo));
}
SArray* tableIdList = extractTableIdList(pTableListInfo);
SArray* groupKeys = extractPartitionColInfo(pTableScanNode->pPartitionKeys);
int32_t code = generateGroupIdMap(pTableListInfo, pHandle, groupKeys); //todo for json
taosArrayDestroy(groupKeys);
if (code){
tsdbCleanupReadHandle(pDataReader);
return NULL;
}
SOperatorInfo* pOperator =
createStreamScanOperatorInfo(pDataReader, pHandle, tableIdList, pTableScanNode, pTaskInfo, &twSup);
createStreamScanOperatorInfo(pDataReader, pHandle, pTableScanNode, pTaskInfo, &twSup);
taosArrayDestroy(tableIdList);
return pOperator;
} else if (QUERY_NODE_PHYSICAL_PLAN_SYSTABLE_SCAN == type) {
SSystemTableScanPhysiNode* pSysScanPhyNode = (SSystemTableScanPhysiNode*)pPhyNode;
......@@ -4969,6 +5061,7 @@ SArray* extractColumnInfo(SNodeList* pNodeList) {
}
SArray* extractPartitionColInfo(SNodeList* pNodeList) {
if(!pNodeList) return NULL;
size_t numOfCols = LIST_LENGTH(pNodeList);
SArray* pList = taosArrayInit(numOfCols, sizeof(SColumn));
if (pList == NULL) {
......@@ -5073,7 +5166,9 @@ int32_t getTableList(void* metaHandle, int32_t tableType, uint64_t tableUid, STa
SArray* res = taosArrayInit(8, sizeof(uint64_t));
code = doFilterTag(pTagCond, &metaArg, res);
if (code != TSDB_CODE_SUCCESS) {
if (code == TSDB_CODE_INDEX_REBUILDING){ // todo
// doFilter();
} else if (code != TSDB_CODE_SUCCESS) {
qError("failed to get tableIds, reason: %s, suid: %" PRIu64 "", tstrerror(code), tableUid);
taosArrayDestroy(res);
terrno = code;
......@@ -5081,6 +5176,7 @@ int32_t getTableList(void* metaHandle, int32_t tableType, uint64_t tableUid, STa
} else {
qDebug("sucess to get tableIds, size: %d, suid: %" PRIu64 "", (int)taosArrayGetSize(res), tableUid);
}
for (int i = 0; i < taosArrayGetSize(res); i++) {
STableKeyInfo info = {.lastKey = TSKEY_INITIAL_VAL, .uid = *(uint64_t*)taosArrayGet(res, i)};
taosArrayPush(pListInfo->pTableList, &info);
......@@ -5097,18 +5193,6 @@ int32_t getTableList(void* metaHandle, int32_t tableType, uint64_t tableUid, STa
return code;
}
SArray* extractTableIdList(const STableListInfo* pTableGroupInfo) {
SArray* tableIdList = taosArrayInit(4, sizeof(uint64_t));
// Transfer the Array of STableKeyInfo into uid list.
for (int32_t i = 0; i < taosArrayGetSize(pTableGroupInfo->pTableList); ++i) {
STableKeyInfo* pkeyInfo = taosArrayGet(pTableGroupInfo->pTableList, i);
taosArrayPush(tableIdList, &pkeyInfo->uid);
}
return tableIdList;
}
tsdbReaderT doCreateDataReader(STableScanPhysiNode* pTableScanNode, SReadHandle* pHandle,
STableListInfo* pTableListInfo, uint64_t queryId, uint64_t taskId, SNode* pTagCond) {
int32_t code =
......
......@@ -37,7 +37,7 @@ static void destroyGroupOperatorInfo(void* param, int32_t numOfOutput) {
taosArrayDestroy(pInfo->pGroupColVals);
}
int32_t initGroupOptrInfo(SArray** pGroupColVals, int32_t* keyLen, char** keyBuf, const SArray* pGroupColList) {
static int32_t initGroupOptrInfo(SArray** pGroupColVals, int32_t* keyLen, char** keyBuf, const SArray* pGroupColList) {
*pGroupColVals = taosArrayInit(4, sizeof(SGroupKeys));
if ((*pGroupColVals) == NULL) {
return TSDB_CODE_OUT_OF_MEMORY;
......@@ -118,7 +118,7 @@ static bool groupKeyCompare(SArray* pGroupCols, SArray* pGroupColVals, SSDataBlo
return true;
}
void recordNewGroupKeys(SArray* pGroupCols, SArray* pGroupColVals, SSDataBlock* pBlock, int32_t rowIndex) {
static void recordNewGroupKeys(SArray* pGroupCols, SArray* pGroupColVals, SSDataBlock* pBlock, int32_t rowIndex) {
SColumnDataAgg* pColAgg = NULL;
size_t numOfGroupCols = taosArrayGetSize(pGroupCols);
......@@ -150,7 +150,7 @@ void recordNewGroupKeys(SArray* pGroupCols, SArray* pGroupColVals, SSDataBlock*
}
}
int32_t buildGroupKeys(void* pKey, const SArray* pGroupColVals) {
static int32_t buildGroupKeys(void* pKey, const SArray* pGroupColVals) {
ASSERT(pKey != NULL);
size_t numOfGroupCols = taosArrayGetSize(pGroupColVals);
......
......@@ -391,22 +391,16 @@ static SSDataBlock* doTableScanImpl(SOperatorInfo* pOperator) {
longjmp(pOperator->pTaskInfo->env, code);
}
recordNewGroupKeys(pTableScanInfo->pGroupCols, pTableScanInfo->pGroupColVals, pBlock, 0);
int32_t len = buildGroupKeys(pTableScanInfo->keyBuf, pTableScanInfo->pGroupColVals);
uint64_t* groupId = taosHashGet(pTableScanInfo->pGroupSet, pTableScanInfo->keyBuf, len);
if (groupId) {
pBlock->info.groupId = *groupId;
} else if (len != 0) {
pBlock->info.groupId = calcGroupId(pTableScanInfo->keyBuf, len);
taosHashPut(pTableScanInfo->pGroupSet, pTableScanInfo->keyBuf, len, &pBlock->info.groupId, sizeof(uint64_t));
}
// current block is filter out according to filter condition, continue load the next block
if (status == FUNC_DATA_REQUIRED_FILTEROUT || pBlock->info.rows == 0) {
continue;
}
uint64_t* groupId = taosHashGet(pOperator->pTaskInfo->tableqinfoList.map, &pBlock->info.uid, sizeof(int64_t));
if (groupId) {
pBlock->info.groupId = *groupId;
}
pOperator->resultInfo.totalRows = pTableScanInfo->readRecorder.totalRows;
pTableScanInfo->readRecorder.elapsedTime += (taosGetTimestampUs() - st) / 1000.0;
......@@ -530,21 +524,13 @@ static void destroyTableScanOperatorInfo(void* param, int32_t numOfOutput) {
tsdbCleanupReadHandle(pTableScanInfo->dataReader);
taosArrayDestroy(pTableScanInfo->pGroupCols);
for (int i = 0; i < taosArrayGetSize(pTableScanInfo->pGroupColVals); i++) {
SGroupKeys key = *(SGroupKeys*)taosArrayGet(pTableScanInfo->pGroupColVals, i);
taosMemoryFree(key.pData);
}
taosArrayDestroy(pTableScanInfo->pGroupColVals);
taosMemoryFree(pTableScanInfo->keyBuf);
taosHashCleanup(pTableScanInfo->pGroupSet);
if (pTableScanInfo->pColMatchInfo != NULL) {
taosArrayDestroy(pTableScanInfo->pColMatchInfo);
}
}
SOperatorInfo* createTableScanOperatorInfo(STableScanPhysiNode* pTableScanNode, tsdbReaderT pDataReader,
SReadHandle* readHandle, SArray* groupKyes, SExecTaskInfo* pTaskInfo) {
SReadHandle* readHandle, SExecTaskInfo* pTaskInfo) {
STableScanInfo* pInfo = taosMemoryCalloc(1, sizeof(STableScanInfo));
SOperatorInfo* pOperator = taosMemoryCalloc(1, sizeof(SOperatorInfo));
if (pInfo == NULL || pOperator == NULL) {
......@@ -591,18 +577,6 @@ SOperatorInfo* createTableScanOperatorInfo(STableScanPhysiNode* pTableScanNode,
pOperator->numOfExprs = numOfCols;
pOperator->pTaskInfo = pTaskInfo;
// for table group
pInfo->pGroupCols = groupKyes;
_hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY);
pInfo->pGroupSet = taosHashInit(100, hashFn, false, HASH_NO_LOCK);
if (pInfo->pGroupSet == NULL) {
goto _error;
}
code = initGroupOptrInfo(&pInfo->pGroupColVals, &pInfo->groupKeyLen, &pInfo->keyBuf, groupKyes);
if (code != TSDB_CODE_SUCCESS) {
goto _error;
}
pOperator->fpSet = createOperatorFpSet(operatorDummyOpenFn, doTableScan, NULL, NULL, destroyTableScanOperatorInfo,
NULL, NULL, getTableScannerExecInfo);
......@@ -912,6 +886,11 @@ static SSDataBlock* doStreamBlockScan(SOperatorInfo* pOperator) {
pInfo->pRes->info.groupId = groupId;
}
uint64_t* groupIdPre = taosHashGet(pOperator->pTaskInfo->tableqinfoList.map, &uid, sizeof(int64_t));
if (groupIdPre) {
pInfo->pRes->info.groupId = *groupIdPre;
}
for (int32_t i = 0; i < taosArrayGetSize(pInfo->pColMatchInfo); ++i) {
SColMatchInfo* pColMatchInfo = taosArrayGet(pInfo->pColMatchInfo, i);
if (!pColMatchInfo->output) {
......@@ -979,11 +958,24 @@ static SSDataBlock* doStreamBlockScan(SOperatorInfo* pOperator) {
}
}
SOperatorInfo* createStreamScanOperatorInfo(void* pDataReader, SReadHandle* pHandle, SArray* pTableIdList,
static SArray* extractTableIdList(const STableListInfo* pTableGroupInfo) {
SArray* tableIdList = taosArrayInit(4, sizeof(uint64_t));
// Transfer the Array of STableKeyInfo into uid list.
for (int32_t i = 0; i < taosArrayGetSize(pTableGroupInfo->pTableList); ++i) {
STableKeyInfo* pkeyInfo = taosArrayGet(pTableGroupInfo->pTableList, i);
taosArrayPush(tableIdList, &pkeyInfo->uid);
}
return tableIdList;
}
SOperatorInfo* createStreamScanOperatorInfo(void* pDataReader, SReadHandle* pHandle,
STableScanPhysiNode* pTableScanNode, SExecTaskInfo* pTaskInfo,
STimeWindowAggSupp* pTwSup) {
SStreamBlockScanInfo* pInfo = taosMemoryCalloc(1, sizeof(SStreamBlockScanInfo));
SOperatorInfo* pOperator = taosMemoryCalloc(1, sizeof(SOperatorInfo));
if (pInfo == NULL || pOperator == NULL) {
terrno = TSDB_CODE_QRY_OUT_OF_MEMORY;
goto _error;
......@@ -992,7 +984,7 @@ SOperatorInfo* createStreamScanOperatorInfo(void* pDataReader, SReadHandle* pHan
SScanPhysiNode* pScanPhyNode = &pTableScanNode->scan;
SDataBlockDescNode* pDescNode = pScanPhyNode->node.pOutputDataBlockDesc;
SOperatorInfo* pTableScanDummy = createTableScanOperatorInfo(pTableScanNode, pDataReader, pHandle, NULL, pTaskInfo);
SOperatorInfo* pTableScanDummy = createTableScanOperatorInfo(pTableScanNode, pDataReader, pHandle, pTaskInfo);
STableScanInfo* pSTInfo = (STableScanInfo*)pTableScanDummy->info;
......@@ -1014,10 +1006,13 @@ SOperatorInfo* createStreamScanOperatorInfo(void* pDataReader, SReadHandle* pHan
// set the extract column id to streamHandle
tqReadHandleSetColIdList((STqReadHandle*)pHandle->reader, pColIds);
int32_t code = tqReadHandleSetTbUidList(pHandle->reader, pTableIdList);
SArray* tableIdList = extractTableIdList(&pTaskInfo->tableqinfoList);
int32_t code = tqReadHandleSetTbUidList(pHandle->reader, tableIdList);
if (code != 0) {
taosArrayDestroy(tableIdList);
goto _error;
}
taosArrayDestroy(tableIdList);
pInfo->pBlockLists = taosArrayInit(4, POINTER_BYTES);
if (pInfo->pBlockLists == NULL) {
......
......@@ -55,16 +55,22 @@ int32_t maxCombine(SqlFunctionCtx* pDestCtx, SqlFunctionCtx* pSourceCtx);
bool getAvgFuncEnv(struct SFunctionNode* pFunc, SFuncExecEnv* pEnv);
bool avgFunctionSetup(SqlFunctionCtx *pCtx, SResultRowEntryInfo* pResultInfo);
int32_t avgFunction(SqlFunctionCtx* pCtx);
int32_t avgFunctionMerge(SqlFunctionCtx* pCtx);
int32_t avgFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock);
int32_t avgPartialFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock);
int32_t avgInvertFunction(SqlFunctionCtx* pCtx);
int32_t avgCombine(SqlFunctionCtx* pDestCtx, SqlFunctionCtx* pSourceCtx);
int32_t getAvgInfoSize();
bool getStddevFuncEnv(struct SFunctionNode* pFunc, SFuncExecEnv* pEnv);
bool stddevFunctionSetup(SqlFunctionCtx *pCtx, SResultRowEntryInfo* pResultInfo);
int32_t stddevFunction(SqlFunctionCtx* pCtx);
int32_t stddevFunctionMerge(SqlFunctionCtx* pCtx);
int32_t stddevFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock);
int32_t stddevPartialFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock);
int32_t stddevInvertFunction(SqlFunctionCtx* pCtx);
int32_t stddevCombine(SqlFunctionCtx* pDestCtx, SqlFunctionCtx* pSourceCtx);
int32_t getStddevInfoSize();
bool getLeastSQRFuncEnv(struct SFunctionNode* pFunc, SFuncExecEnv* pEnv);
bool leastSQRFunctionSetup(SqlFunctionCtx *pCtx, SResultRowEntryInfo* pResultInfo);
......
......@@ -155,6 +155,64 @@ static int32_t translateSum(SFunctionNode* pFunc, char* pErrBuf, int32_t len) {
return TSDB_CODE_SUCCESS;
}
static int32_t translateAvgPartial(SFunctionNode* pFunc, char* pErrBuf, int32_t len) {
if (1 != LIST_LENGTH(pFunc->pParameterList)) {
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
}
uint8_t paraType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type;
if (!IS_NUMERIC_TYPE(paraType) && !IS_NULL_TYPE(paraType)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
}
pFunc->node.resType = (SDataType){.bytes = getAvgInfoSize() + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_BINARY};
return TSDB_CODE_SUCCESS;
}
static int32_t translateAvgMerge(SFunctionNode* pFunc, char* pErrBuf, int32_t len) {
if (1 != LIST_LENGTH(pFunc->pParameterList)) {
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
}
uint8_t paraType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type;
if (TSDB_DATA_TYPE_BINARY != paraType) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
}
pFunc->node.resType = (SDataType){.bytes = tDataTypes[TSDB_DATA_TYPE_DOUBLE].bytes, .type = TSDB_DATA_TYPE_DOUBLE};
return TSDB_CODE_SUCCESS;
}
static int32_t translateStddevPartial(SFunctionNode* pFunc, char* pErrBuf, int32_t len) {
if (1 != LIST_LENGTH(pFunc->pParameterList)) {
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
}
uint8_t paraType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type;
if (!IS_NUMERIC_TYPE(paraType) && !IS_NULL_TYPE(paraType)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
}
pFunc->node.resType = (SDataType){.bytes = getStddevInfoSize() + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_BINARY};
return TSDB_CODE_SUCCESS;
}
static int32_t translateStddevMerge(SFunctionNode* pFunc, char* pErrBuf, int32_t len) {
if (1 != LIST_LENGTH(pFunc->pParameterList)) {
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
}
uint8_t paraType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type;
if (TSDB_DATA_TYPE_BINARY != paraType) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
}
pFunc->node.resType = (SDataType){.bytes = tDataTypes[TSDB_DATA_TYPE_DOUBLE].bytes, .type = TSDB_DATA_TYPE_DOUBLE};
return TSDB_CODE_SUCCESS;
}
static int32_t translateWduration(SFunctionNode* pFunc, char* pErrBuf, int32_t len) {
// pseudo column do not need to check parameters
pFunc->node.resType = (SDataType){.bytes = sizeof(int64_t), .type = TSDB_DATA_TYPE_BIGINT};
......@@ -393,7 +451,8 @@ static int32_t translateTopBotImpl(SFunctionNode* pFunc, char* pErrBuf, int32_t
pValue->notReserved = true;
// set result type
pFunc->node.resType = (SDataType){.bytes = getTopBotInfoSize(pValue->datum.i) + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_BINARY};
pFunc->node.resType =
(SDataType){.bytes = getTopBotInfoSize(pValue->datum.i) + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_BINARY};
} else {
if (1 != numOfParams) {
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
......@@ -956,11 +1015,11 @@ static int32_t translateFirstLast(SFunctionNode* pFunc, char* pErrBuf, int32_t l
static int32_t translateFirstLastImpl(SFunctionNode* pFunc, char* pErrBuf, int32_t len, bool isPartial) {
// first(col_list) will be rewritten as first(col)
if (2 != LIST_LENGTH(pFunc->pParameterList)) { //input has two params c0,ts, is this a bug?
if (2 != LIST_LENGTH(pFunc->pParameterList)) { // input has two params c0,ts, is this a bug?
return TSDB_CODE_SUCCESS;
}
SNode* pPara = nodesListGetNode(pFunc->pParameterList, 0);
SNode* pPara = nodesListGetNode(pFunc->pParameterList, 0);
uint8_t paraType = ((SExprNode*)pPara)->resType.type;
int32_t paraBytes = ((SExprNode*)pPara)->resType.bytes;
if (isPartial) {
......@@ -969,8 +1028,8 @@ static int32_t translateFirstLastImpl(SFunctionNode* pFunc, char* pErrBuf, int32
"The parameters of first/last can only be columns");
}
pFunc->node.resType = (SDataType){.bytes = getFirstLastInfoSize(paraBytes) + VARSTR_HEADER_SIZE,
.type = TSDB_DATA_TYPE_BINARY};
pFunc->node.resType =
(SDataType){.bytes = getFirstLastInfoSize(paraBytes) + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_BINARY};
} else {
if (TSDB_DATA_TYPE_BINARY != paraType) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
......@@ -1493,6 +1552,32 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = {
.finalizeFunc = stddevFinalize,
.invertFunc = stddevInvertFunction,
.combineFunc = stddevCombine,
.pPartialFunc = "_stddev_partial",
.pMergeFunc = "_stddev_merge"
},
{
.name = "_stddev_partial",
.type = FUNCTION_TYPE_STDDEV_PARTIAL,
.classification = FUNC_MGT_AGG_FUNC,
.translateFunc = translateStddevPartial,
.getEnvFunc = getStddevFuncEnv,
.initFunc = stddevFunctionSetup,
.processFunc = stddevFunction,
.finalizeFunc = stddevPartialFinalize,
.invertFunc = stddevInvertFunction,
.combineFunc = stddevCombine,
},
{
.name = "_stddev_merge",
.type = FUNCTION_TYPE_STDDEV_MERGE,
.classification = FUNC_MGT_AGG_FUNC,
.translateFunc = translateStddevMerge,
.getEnvFunc = getStddevFuncEnv,
.initFunc = stddevFunctionSetup,
.processFunc = stddevFunctionMerge,
.finalizeFunc = stddevFinalize,
.invertFunc = stddevInvertFunction,
.combineFunc = stddevCombine,
},
{
.name = "leastsquares",
......@@ -1517,11 +1602,37 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = {
.finalizeFunc = avgFinalize,
.invertFunc = avgInvertFunction,
.combineFunc = avgCombine,
.pPartialFunc = "_avg_partial",
.pMergeFunc = "_avg_merge"
},
{
.name = "_avg_partial",
.type = FUNCTION_TYPE_AVG_PARTIAL,
.classification = FUNC_MGT_AGG_FUNC,
.translateFunc = translateAvgPartial,
.getEnvFunc = getAvgFuncEnv,
.initFunc = avgFunctionSetup,
.processFunc = avgFunction,
.finalizeFunc = avgPartialFinalize,
.invertFunc = avgInvertFunction,
.combineFunc = avgCombine,
},
{
.name = "_avg_merge",
.type = FUNCTION_TYPE_AVG_MERGE,
.classification = FUNC_MGT_AGG_FUNC,
.translateFunc = translateAvgMerge,
.getEnvFunc = getAvgFuncEnv,
.initFunc = avgFunctionSetup,
.processFunc = avgFunctionMerge,
.finalizeFunc = avgFinalize,
.invertFunc = avgInvertFunction,
.combineFunc = avgCombine,
},
{
.name = "percentile",
.type = FUNCTION_TYPE_PERCENTILE,
.classification = FUNC_MGT_AGG_FUNC | FUNC_MGT_REPEAT_SCAN_FUNC,
.classification = FUNC_MGT_AGG_FUNC | FUNC_MGT_REPEAT_SCAN_FUNC | FUNC_MGT_FORBID_STREAM_FUNC,
.translateFunc = translatePercentile,
.getEnvFunc = getPercentileFuncEnv,
.initFunc = percentileFunctionSetup,
......@@ -1682,7 +1793,7 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = {
{
.name = "elapsed",
.type = FUNCTION_TYPE_ELAPSED,
.classification = FUNC_MGT_AGG_FUNC,
.classification = FUNC_MGT_AGG_FUNC | FUNC_MGT_FORBID_STREAM_FUNC,
.dataRequiredFunc = statisDataRequired,
.translateFunc = translateElapsed,
.getEnvFunc = getElapsedFuncEnv,
......@@ -1803,7 +1914,7 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = {
{
.name = "twa",
.type = FUNCTION_TYPE_TWA,
.classification = FUNC_MGT_AGG_FUNC | FUNC_MGT_TIMELINE_FUNC,
.classification = FUNC_MGT_AGG_FUNC | FUNC_MGT_TIMELINE_FUNC | FUNC_MGT_FORBID_STREAM_FUNC,
.translateFunc = translateInNumOutDou,
.getEnvFunc = getTwaFuncEnv,
.initFunc = twaFunctionSetup,
......@@ -1889,7 +2000,7 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = {
{
.name = "diff",
.type = FUNCTION_TYPE_DIFF,
.classification = FUNC_MGT_INDEFINITE_ROWS_FUNC | FUNC_MGT_TIMELINE_FUNC,
.classification = FUNC_MGT_INDEFINITE_ROWS_FUNC | FUNC_MGT_TIMELINE_FUNC | FUNC_MGT_FORBID_STREAM_FUNC,
.translateFunc = translateDiff,
.getEnvFunc = getDiffFuncEnv,
.initFunc = diffFunctionSetup,
......@@ -1899,7 +2010,7 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = {
{
.name = "statecount",
.type = FUNCTION_TYPE_STATE_COUNT,
.classification = FUNC_MGT_INDEFINITE_ROWS_FUNC,
.classification = FUNC_MGT_INDEFINITE_ROWS_FUNC | FUNC_MGT_FORBID_STREAM_FUNC,
.translateFunc = translateStateCount,
.getEnvFunc = getStateFuncEnv,
.initFunc = functionSetup,
......@@ -1909,7 +2020,7 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = {
{
.name = "stateduration",
.type = FUNCTION_TYPE_STATE_DURATION,
.classification = FUNC_MGT_INDEFINITE_ROWS_FUNC | FUNC_MGT_TIMELINE_FUNC,
.classification = FUNC_MGT_INDEFINITE_ROWS_FUNC | FUNC_MGT_TIMELINE_FUNC | FUNC_MGT_FORBID_STREAM_FUNC,
.translateFunc = translateStateDuration,
.getEnvFunc = getStateFuncEnv,
.initFunc = functionSetup,
......@@ -1919,7 +2030,7 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = {
{
.name = "csum",
.type = FUNCTION_TYPE_CSUM,
.classification = FUNC_MGT_INDEFINITE_ROWS_FUNC | FUNC_MGT_TIMELINE_FUNC,
.classification = FUNC_MGT_INDEFINITE_ROWS_FUNC | FUNC_MGT_TIMELINE_FUNC | FUNC_MGT_FORBID_STREAM_FUNC,
.translateFunc = translateCsum,
.getEnvFunc = getCsumFuncEnv,
.initFunc = functionSetup,
......@@ -1929,7 +2040,7 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = {
{
.name = "mavg",
.type = FUNCTION_TYPE_MAVG,
.classification = FUNC_MGT_INDEFINITE_ROWS_FUNC | FUNC_MGT_TIMELINE_FUNC,
.classification = FUNC_MGT_INDEFINITE_ROWS_FUNC | FUNC_MGT_TIMELINE_FUNC | FUNC_MGT_FORBID_STREAM_FUNC,
.translateFunc = translateMavg,
.getEnvFunc = getMavgFuncEnv,
.initFunc = mavgFunctionSetup,
......@@ -1939,7 +2050,7 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = {
{
.name = "sample",
.type = FUNCTION_TYPE_SAMPLE,
.classification = FUNC_MGT_AGG_FUNC | FUNC_MGT_SELECT_FUNC | FUNC_MGT_INDEFINITE_ROWS_FUNC,
.classification = FUNC_MGT_AGG_FUNC | FUNC_MGT_SELECT_FUNC | FUNC_MGT_INDEFINITE_ROWS_FUNC | FUNC_MGT_FORBID_STREAM_FUNC,
.translateFunc = translateSample,
.getEnvFunc = getSampleFuncEnv,
.initFunc = sampleFunctionSetup,
......@@ -1949,7 +2060,7 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = {
{
.name = "tail",
.type = FUNCTION_TYPE_TAIL,
.classification = FUNC_MGT_INDEFINITE_ROWS_FUNC | FUNC_MGT_TIMELINE_FUNC,
.classification = FUNC_MGT_INDEFINITE_ROWS_FUNC | FUNC_MGT_TIMELINE_FUNC | FUNC_MGT_FORBID_STREAM_FUNC,
.translateFunc = translateTail,
.getEnvFunc = getTailFuncEnv,
.initFunc = tailFunctionSetup,
......@@ -1959,7 +2070,7 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = {
{
.name = "unique",
.type = FUNCTION_TYPE_UNIQUE,
.classification = FUNC_MGT_INDEFINITE_ROWS_FUNC | FUNC_MGT_TIMELINE_FUNC,
.classification = FUNC_MGT_INDEFINITE_ROWS_FUNC | FUNC_MGT_TIMELINE_FUNC | FUNC_MGT_FORBID_STREAM_FUNC,
.translateFunc = translateUnique,
.getEnvFunc = getUniqueFuncEnv,
.initFunc = uniqueFunctionSetup,
......
......@@ -51,6 +51,7 @@ typedef struct SAvgRes {
double result;
SSumRes sum;
int64_t count;
int16_t type; // store the original input type, used in merge function
} SAvgRes;
typedef struct STuplePos {
......@@ -88,6 +89,7 @@ typedef struct SStddevRes {
double dsum;
int64_t isum;
};
int16_t type;
} SStddevRes;
typedef struct SLeastSQRInfo {
......@@ -624,6 +626,8 @@ bool getSumFuncEnv(SFunctionNode* UNUSED_PARAM(pFunc), SFuncExecEnv* pEnv) {
return true;
}
int32_t getAvgInfoSize() { return (int32_t)sizeof(SAvgRes); }
bool getAvgFuncEnv(SFunctionNode* UNUSED_PARAM(pFunc), SFuncExecEnv* pEnv) {
pEnv->calcMemSize = sizeof(SAvgRes);
return true;
......@@ -642,11 +646,12 @@ bool avgFunctionSetup(SqlFunctionCtx* pCtx, SResultRowEntryInfo* pResultInfo) {
int32_t avgFunction(SqlFunctionCtx* pCtx) {
int32_t numOfElem = 0;
// Only the pre-computing information loaded and actual data does not loaded
SInputColumnInfoData* pInput = &pCtx->input;
SColumnDataAgg* pAgg = pInput->pColumnDataAgg[0];
int32_t type = pInput->pData[0]->info.type;
SAvgRes* pAvgRes = GET_ROWCELL_INTERBUF(GET_RES_INFO(pCtx));
pAvgRes->type = type;
// computing based on the true data block
SColumnInfoData* pCol = pInput->pData[0];
......@@ -660,95 +665,107 @@ int32_t avgFunction(SqlFunctionCtx* pCtx) {
goto _avg_over;
}
switch (type) {
case TSDB_DATA_TYPE_TINYINT: {
int8_t* plist = (int8_t*)pCol->pData;
for (int32_t i = start; i < numOfRows + pInput->startRowIndex; ++i) {
if (pCol->hasNull && colDataIsNull_f(pCol->nullbitmap, i)) {
continue;
}
numOfElem += 1;
pAvgRes->count += 1;
pAvgRes->sum.isum += plist[i];
}
if (pInput->colDataAggIsSet) {
numOfElem = numOfRows - pAgg->numOfNull;
ASSERT(numOfElem >= 0);
break;
pAvgRes->count += numOfElem;
if (IS_INTEGER_TYPE(type)) {
pAvgRes->sum.isum += pAgg->sum;
} else if (IS_FLOAT_TYPE(type)) {
pAvgRes->sum.dsum += GET_DOUBLE_VAL((const char*)&(pAgg->sum));
}
} else { // computing based on the true data block
switch (type) {
case TSDB_DATA_TYPE_TINYINT: {
int8_t* plist = (int8_t*)pCol->pData;
for (int32_t i = start; i < numOfRows + pInput->startRowIndex; ++i) {
if (pCol->hasNull && colDataIsNull_f(pCol->nullbitmap, i)) {
continue;
}
case TSDB_DATA_TYPE_SMALLINT: {
int16_t* plist = (int16_t*)pCol->pData;
for (int32_t i = start; i < numOfRows + pInput->startRowIndex; ++i) {
if (pCol->hasNull && colDataIsNull_f(pCol->nullbitmap, i)) {
continue;
numOfElem += 1;
pAvgRes->count += 1;
pAvgRes->sum.isum += plist[i];
}
numOfElem += 1;
pAvgRes->count += 1;
pAvgRes->sum.isum += plist[i];
break;
}
break;
}
case TSDB_DATA_TYPE_INT: {
int32_t* plist = (int32_t*)pCol->pData;
for (int32_t i = start; i < numOfRows + pInput->startRowIndex; ++i) {
if (pCol->hasNull && colDataIsNull_f(pCol->nullbitmap, i)) {
continue;
}
case TSDB_DATA_TYPE_SMALLINT: {
int16_t* plist = (int16_t*)pCol->pData;
for (int32_t i = start; i < numOfRows + pInput->startRowIndex; ++i) {
if (pCol->hasNull && colDataIsNull_f(pCol->nullbitmap, i)) {
continue;
}
numOfElem += 1;
pAvgRes->count += 1;
pAvgRes->sum.isum += plist[i];
numOfElem += 1;
pAvgRes->count += 1;
pAvgRes->sum.isum += plist[i];
}
break;
}
break;
}
case TSDB_DATA_TYPE_INT: {
int32_t* plist = (int32_t*)pCol->pData;
for (int32_t i = start; i < numOfRows + pInput->startRowIndex; ++i) {
if (pCol->hasNull && colDataIsNull_f(pCol->nullbitmap, i)) {
continue;
}
case TSDB_DATA_TYPE_BIGINT: {
int64_t* plist = (int64_t*)pCol->pData;
for (int32_t i = start; i < numOfRows + pInput->startRowIndex; ++i) {
if (pCol->hasNull && colDataIsNull_f(pCol->nullbitmap, i)) {
continue;
numOfElem += 1;
pAvgRes->count += 1;
pAvgRes->sum.isum += plist[i];
}
numOfElem += 1;
pAvgRes->count += 1;
pAvgRes->sum.isum += plist[i];
break;
}
break;
}
case TSDB_DATA_TYPE_FLOAT: {
float* plist = (float*)pCol->pData;
for (int32_t i = start; i < numOfRows + pInput->startRowIndex; ++i) {
if (pCol->hasNull && colDataIsNull_f(pCol->nullbitmap, i)) {
continue;
}
case TSDB_DATA_TYPE_BIGINT: {
int64_t* plist = (int64_t*)pCol->pData;
for (int32_t i = start; i < numOfRows + pInput->startRowIndex; ++i) {
if (pCol->hasNull && colDataIsNull_f(pCol->nullbitmap, i)) {
continue;
}
numOfElem += 1;
pAvgRes->count += 1;
pAvgRes->sum.dsum += plist[i];
numOfElem += 1;
pAvgRes->count += 1;
pAvgRes->sum.isum += plist[i];
}
break;
}
break;
}
case TSDB_DATA_TYPE_DOUBLE: {
double* plist = (double*)pCol->pData;
for (int32_t i = start; i < numOfRows + pInput->startRowIndex; ++i) {
if (pCol->hasNull && colDataIsNull_f(pCol->nullbitmap, i)) {
continue;
case TSDB_DATA_TYPE_FLOAT: {
float* plist = (float*)pCol->pData;
for (int32_t i = start; i < numOfRows + pInput->startRowIndex; ++i) {
if (pCol->hasNull && colDataIsNull_f(pCol->nullbitmap, i)) {
continue;
}
numOfElem += 1;
pAvgRes->count += 1;
pAvgRes->sum.dsum += plist[i];
}
break;
}
numOfElem += 1;
pAvgRes->count += 1;
pAvgRes->sum.dsum += plist[i];
case TSDB_DATA_TYPE_DOUBLE: {
double* plist = (double*)pCol->pData;
for (int32_t i = start; i < numOfRows + pInput->startRowIndex; ++i) {
if (pCol->hasNull && colDataIsNull_f(pCol->nullbitmap, i)) {
continue;
}
numOfElem += 1;
pAvgRes->count += 1;
pAvgRes->sum.dsum += plist[i];
}
break;
}
break;
}
default:
break;
default:
break;
}
}
_avg_over:
......@@ -757,6 +774,37 @@ _avg_over:
return TSDB_CODE_SUCCESS;
}
static void avgTransferInfo(SAvgRes* pInput, SAvgRes* pOutput) {
pOutput->type = pInput->type;
if (IS_INTEGER_TYPE(pOutput->type)) {
pOutput->sum.isum += pInput->sum.isum;
} else {
pOutput->sum.dsum += pInput->sum.dsum;
}
pOutput->count += pInput->count;
return;
}
int32_t avgFunctionMerge(SqlFunctionCtx* pCtx) {
SInputColumnInfoData* pInput = &pCtx->input;
SColumnInfoData* pCol = pInput->pData[0];
ASSERT(pCol->info.type == TSDB_DATA_TYPE_BINARY);
SAvgRes* pInfo = GET_ROWCELL_INTERBUF(GET_RES_INFO(pCtx));
int32_t start = pInput->startRowIndex;
char* data = colDataGetData(pCol, start);
SAvgRes* pInputInfo = (SAvgRes*)varDataVal(data);
avgTransferInfo(pInputInfo, pInfo);
SET_VAL(GET_RES_INFO(pCtx), 1, 1);
return TSDB_CODE_SUCCESS;
}
#define LIST_AVG_N(sumT, T) \
do { \
T* plist = (T*)pCol->pData; \
......@@ -841,8 +889,8 @@ int32_t avgCombine(SqlFunctionCtx* pDestCtx, SqlFunctionCtx* pSourceCtx) {
int32_t avgFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock) {
SInputColumnInfoData* pInput = &pCtx->input;
int32_t type = pInput->pData[0]->info.type;
SAvgRes* pAvgRes = GET_ROWCELL_INTERBUF(GET_RES_INFO(pCtx));
int32_t type = pAvgRes->type;
if (IS_INTEGER_TYPE(type)) {
pAvgRes->result = pAvgRes->sum.isum / ((double)pAvgRes->count);
......@@ -858,6 +906,24 @@ int32_t avgFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock) {
return functionFinalize(pCtx, pBlock);
}
int32_t avgPartialFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock) {
SResultRowEntryInfo* pResInfo = GET_RES_INFO(pCtx);
SAvgRes* pInfo = GET_ROWCELL_INTERBUF(GET_RES_INFO(pCtx));
int32_t resultBytes = getAvgInfoSize();
char* res = taosMemoryCalloc(resultBytes + VARSTR_HEADER_SIZE, sizeof(char));
memcpy(varDataVal(res), pInfo, resultBytes);
varDataSetLen(res, resultBytes);
int32_t slotId = pCtx->pExpr->base.resSchema.slotId;
SColumnInfoData* pCol = taosArrayGet(pBlock->pDataBlock, slotId);
colDataAppend(pCol, pBlock->info.rows, res, false);
taosMemoryFree(res);
return pResInfo->numOfRes;
}
EFuncDataRequired statisDataRequired(SFunctionNode* pFunc, STimeWindow* pTimeWindow) {
return FUNC_DATA_REQUIRED_STATIS_LOAD;
}
......@@ -1423,6 +1489,8 @@ int32_t maxCombine(SqlFunctionCtx* pDestCtx, SqlFunctionCtx* pSourceCtx) {
return minMaxCombine(pDestCtx, pSourceCtx, 0);
}
int32_t getStddevInfoSize() { return (int32_t)sizeof(SStddevRes); }
bool getStddevFuncEnv(SFunctionNode* pFunc, SFuncExecEnv* pEnv) {
pEnv->calcMemSize = sizeof(SStddevRes);
return true;
......@@ -1446,6 +1514,7 @@ int32_t stddevFunction(SqlFunctionCtx* pCtx) {
int32_t type = pInput->pData[0]->info.type;
SStddevRes* pStddevRes = GET_ROWCELL_INTERBUF(GET_RES_INFO(pCtx));
pStddevRes->type = type;
// computing based on the true data block
SColumnInfoData* pCol = pInput->pData[0];
......@@ -1562,6 +1631,39 @@ _stddev_over:
return TSDB_CODE_SUCCESS;
}
static void stddevTransferInfo(SStddevRes* pInput, SStddevRes* pOutput) {
pOutput->type = pInput->type;
if (IS_INTEGER_TYPE(pOutput->type)) {
pOutput->quadraticISum += pInput->quadraticISum;
pOutput->isum += pInput->isum;
} else {
pOutput->quadraticDSum += pInput->quadraticDSum;
pOutput->dsum += pInput->dsum;
}
pOutput->count += pInput->count;
return;
}
int32_t stddevFunctionMerge(SqlFunctionCtx* pCtx) {
SInputColumnInfoData* pInput = &pCtx->input;
SColumnInfoData* pCol = pInput->pData[0];
ASSERT(pCol->info.type == TSDB_DATA_TYPE_BINARY);
SStddevRes* pInfo = GET_ROWCELL_INTERBUF(GET_RES_INFO(pCtx));
int32_t start = pInput->startRowIndex;
char* data = colDataGetData(pCol, start);
SStddevRes* pInputInfo = (SStddevRes*)varDataVal(data);
stddevTransferInfo(pInputInfo, pInfo);
SET_VAL(GET_RES_INFO(pCtx), 1, 1);
return TSDB_CODE_SUCCESS;
}
#define LIST_STDDEV_SUB_N(sumT, T) \
do { \
T* plist = (T*)pCol->pData; \
......@@ -1627,9 +1729,10 @@ int32_t stddevInvertFunction(SqlFunctionCtx* pCtx) {
int32_t stddevFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock) {
SInputColumnInfoData* pInput = &pCtx->input;
int32_t type = pInput->pData[0]->info.type;
SStddevRes* pStddevRes = GET_ROWCELL_INTERBUF(GET_RES_INFO(pCtx));
int32_t type = pStddevRes->type;
double avg;
if (IS_INTEGER_TYPE(type)) {
avg = pStddevRes->isum / ((double)pStddevRes->count);
pStddevRes->result = sqrt(pStddevRes->quadraticISum / ((double)pStddevRes->count) - avg * avg);
......@@ -1641,6 +1744,24 @@ int32_t stddevFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock) {
return functionFinalize(pCtx, pBlock);
}
int32_t stddevPartialFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock) {
SResultRowEntryInfo* pResInfo = GET_RES_INFO(pCtx);
SStddevRes* pInfo = GET_ROWCELL_INTERBUF(GET_RES_INFO(pCtx));
int32_t resultBytes = getStddevInfoSize();
char* res = taosMemoryCalloc(resultBytes + VARSTR_HEADER_SIZE, sizeof(char));
memcpy(varDataVal(res), pInfo, resultBytes);
varDataSetLen(res, resultBytes);
int32_t slotId = pCtx->pExpr->base.resSchema.slotId;
SColumnInfoData* pCol = taosArrayGet(pBlock->pDataBlock, slotId);
colDataAppend(pCol, pBlock->info.rows, res, false);
taosMemoryFree(res);
return pResInfo->numOfRes;
}
int32_t stddevCombine(SqlFunctionCtx* pDestCtx, SqlFunctionCtx* pSourceCtx) {
SResultRowEntryInfo* pDResInfo = GET_RES_INFO(pDestCtx);
SStddevRes* pDBuf = GET_ROWCELL_INTERBUF(pDResInfo);
......@@ -3226,11 +3347,10 @@ int32_t spreadFunctionMerge(SqlFunctionCtx* pCtx) {
ASSERT(pCol->info.type == TSDB_DATA_TYPE_BINARY);
SSpreadInfo* pInfo = GET_ROWCELL_INTERBUF(GET_RES_INFO(pCtx));
SSpreadInfo* pInputInfo;
int32_t start = pInput->startRowIndex;
char* data = colDataGetData(pCol, start);
pInputInfo = (SSpreadInfo*)varDataVal(data);
SSpreadInfo* pInputInfo = (SSpreadInfo*)varDataVal(data);
spreadTransferInfo(pInputInfo, pInfo);
......
......@@ -168,6 +168,7 @@ SSyncNode* syncNodeOpen(const SSyncInfo* pSyncInfo);
void syncNodeStart(SSyncNode* pSyncNode);
void syncNodeStartStandBy(SSyncNode* pSyncNode);
void syncNodeClose(SSyncNode* pSyncNode);
int32_t syncNodePropose(SSyncNode* pSyncNode, const SRpcMsg* pMsg, bool isWeak);
// option
bool syncNodeSnapshotEnable(SSyncNode* pSyncNode);
......@@ -232,6 +233,9 @@ int32_t syncNodeCommit(SSyncNode* ths, SyncIndex beginIndex, SyncIndex endIndex,
bool syncNodeInRaftGroup(SSyncNode* ths, SRaftId* pRaftId);
SSyncSnapshotSender* syncNodeGetSnapshotSender(SSyncNode* ths, SRaftId* pDestId);
void syncStartNormal(int64_t rid);
void syncStartStandBy(int64_t rid);
// for debug --------------
void syncNodePrint(SSyncNode* pObj);
void syncNodePrint2(char* s, SSyncNode* pObj);
......
......@@ -995,7 +995,7 @@ int32_t syncNodeOnAppendEntriesSnapshotCb(SSyncNode* ths, SyncAppendEntries* pMs
ths->commitIndex = snapshot.lastApplyIndex;
sDebug("vgId:%d sync event commit by snapshot from index:%ld to index:%ld, %s", ths->vgId, commitBegin,
commitEnd, syncUtilState2String(ths->state));
commitEnd, syncUtilState2String(ths->state));
}
SyncIndex beginIndex = ths->commitIndex + 1;
......
......@@ -57,7 +57,7 @@ void syncMaybeAdvanceCommitIndex(SSyncNode* pSyncNode) {
pSyncNode->commitIndex = snapshot.lastApplyIndex;
sDebug("vgId:%d sync event commit by snapshot from index:%ld to index:%ld, %s", pSyncNode->vgId,
pSyncNode->commitIndex, snapshot.lastApplyIndex, syncUtilState2String(pSyncNode->state));
pSyncNode->commitIndex, snapshot.lastApplyIndex, syncUtilState2String(pSyncNode->state));
}
// update commit index
......
......@@ -149,12 +149,12 @@ void syncStop(int64_t rid) {
int32_t syncSetStandby(int64_t rid) {
SSyncNode* pSyncNode = (SSyncNode*)taosAcquireRef(tsNodeRefId, rid);
if (pSyncNode == NULL) {
return -1;
return TAOS_SYNC_OTHER_ERROR;
}
if (pSyncNode->state == TAOS_SYNC_STATE_LEADER) {
taosReleaseRef(tsNodeRefId, pSyncNode->rid);
return -1;
return TAOS_SYNC_OTHER_ERROR;
}
// state change
......@@ -174,14 +174,88 @@ int32_t syncSetStandby(int64_t rid) {
return 0;
}
int32_t syncReconfig(int64_t rid, const SSyncCfg* pSyncCfg) {
int32_t syncReconfigBuild(int64_t rid, const SSyncCfg* pNewCfg, SRpcMsg* pRpcMsg) {
SSyncNode* pSyncNode = (SSyncNode*)taosAcquireRef(tsNodeRefId, rid);
if (pSyncNode == NULL) {
return TAOS_SYNC_OTHER_ERROR;
}
ASSERT(rid == pSyncNode->rid);
int32_t ret = 0;
char* newconfig = syncCfg2Str((SSyncCfg*)pSyncCfg);
bool IamInNew = false;
for (int i = 0; i < pNewCfg->replicaNum; ++i) {
if (strcmp((pNewCfg->nodeInfo)[i].nodeFqdn, pSyncNode->myNodeInfo.nodeFqdn) == 0 &&
(pNewCfg->nodeInfo)[i].nodePort == pSyncNode->myNodeInfo.nodePort) {
IamInNew = true;
}
/*
SRaftId newId;
newId.addr = syncUtilAddr2U64((pNewCfg->nodeInfo)[i].nodeFqdn, (pNewCfg->nodeInfo)[i].nodePort);
newId.vgId = pSyncNode->vgId;
if (syncUtilSameId(&(pSyncNode->myRaftId), &newId)) {
IamInNew = true;
}
*/
}
if (!IamInNew) {
taosReleaseRef(tsNodeRefId, pSyncNode->rid);
return TAOS_SYNC_NOT_IN_NEW_CONFIG;
}
char* newconfig = syncCfg2Str((SSyncCfg*)pNewCfg);
pRpcMsg->msgType = TDMT_SYNC_CONFIG_CHANGE;
pRpcMsg->info.noResp = 1;
pRpcMsg->contLen = strlen(newconfig) + 1;
pRpcMsg->pCont = rpcMallocCont(pRpcMsg->contLen);
snprintf(pRpcMsg->pCont, pRpcMsg->contLen, "%s", newconfig);
taosMemoryFree(newconfig);
taosReleaseRef(tsNodeRefId, pSyncNode->rid);
return ret;
}
int32_t syncReconfig(int64_t rid, const SSyncCfg* pNewCfg) {
SSyncNode* pSyncNode = (SSyncNode*)taosAcquireRef(tsNodeRefId, rid);
if (pSyncNode == NULL) {
return TAOS_SYNC_OTHER_ERROR;
}
ASSERT(rid == pSyncNode->rid);
bool IamInNew = false;
for (int i = 0; i < pNewCfg->replicaNum; ++i) {
if (strcmp((pNewCfg->nodeInfo)[i].nodeFqdn, pSyncNode->myNodeInfo.nodeFqdn) == 0 &&
(pNewCfg->nodeInfo)[i].nodePort == pSyncNode->myNodeInfo.nodePort) {
IamInNew = true;
}
/*
// some problem in inet_addr
SRaftId newId = EMPTY_RAFT_ID;
newId.addr = syncUtilAddr2U64((pNewCfg->nodeInfo)[i].nodeFqdn, (pNewCfg->nodeInfo)[i].nodePort);
newId.vgId = pSyncNode->vgId;
if (syncUtilSameId(&(pSyncNode->myRaftId), &newId)) {
IamInNew = true;
}
*/
}
if (!IamInNew) {
sError("sync reconfig error, not in new config");
taosReleaseRef(tsNodeRefId, pSyncNode->rid);
return TAOS_SYNC_NOT_IN_NEW_CONFIG;
}
char* newconfig = syncCfg2Str((SSyncCfg*)pNewCfg);
if (gRaftDetailLog) {
sInfo("==syncReconfig== newconfig:%s", newconfig);
}
int32_t ret = 0;
SRpcMsg rpcMsg = {0};
rpcMsg.msgType = TDMT_SYNC_CONFIG_CHANGE;
rpcMsg.info.noResp = 1;
......@@ -189,58 +263,59 @@ int32_t syncReconfig(int64_t rid, const SSyncCfg* pSyncCfg) {
rpcMsg.pCont = rpcMallocCont(rpcMsg.contLen);
snprintf(rpcMsg.pCont, rpcMsg.contLen, "%s", newconfig);
taosMemoryFree(newconfig);
ret = syncPropose(rid, &rpcMsg, false);
ret = syncNodePropose(pSyncNode, &rpcMsg, false);
taosReleaseRef(tsNodeRefId, pSyncNode->rid);
return ret;
}
int32_t syncLeaderTransfer(int64_t rid) {
int32_t ret = 0;
SSyncNode* pSyncNode = (SSyncNode*)taosAcquireRef(tsNodeRefId, rid);
if (pSyncNode == NULL) {
return TAOS_SYNC_OTHER_ERROR;
}
ASSERT(rid == pSyncNode->rid);
if (pSyncNode->peersNum == 0) {
taosReleaseRef(tsNodeRefId, pSyncNode->rid);
return TAOS_SYNC_OTHER_ERROR;
}
SNodeInfo newLeader = (pSyncNode->peersNodeInfo)[0];
taosReleaseRef(tsNodeRefId, pSyncNode->rid);
int32_t ret = syncLeaderTransferTo(rid, newLeader);
return ret;
}
int32_t syncLeaderTransferTo(int64_t rid, SNodeInfo newLeader) {
SSyncNode* pSyncNode = (SSyncNode*)taosAcquireRef(tsNodeRefId, rid);
if (pSyncNode == NULL) {
return false;
return TAOS_SYNC_OTHER_ERROR;
}
assert(rid == pSyncNode->rid);
ASSERT(rid == pSyncNode->rid);
int32_t ret = 0;
if (pSyncNode->replicaNum == 1) {
taosReleaseRef(tsNodeRefId, pSyncNode->rid);
sError("only one replica, cannot drop leader");
taosReleaseRef(tsNodeRefId, pSyncNode->rid);
return TAOS_SYNC_ONLY_ONE_REPLICA;
}
SyncLeaderTransfer* pMsg = syncLeaderTransferBuild(pSyncNode->vgId);
pMsg->newLeaderId.addr = syncUtilAddr2U64(newLeader.nodeFqdn, newLeader.nodePort);
pMsg->newLeaderId.vgId = pSyncNode->vgId;
pMsg->newNodeInfo = newLeader;
ASSERT(pMsg != NULL);
SRpcMsg rpcMsg = {0};
syncLeaderTransfer2RpcMsg(pMsg, &rpcMsg);
syncLeaderTransferDestroy(pMsg);
ret = syncPropose(rid, &rpcMsg, false);
ret = syncNodePropose(pSyncNode, &rpcMsg, false);
taosReleaseRef(tsNodeRefId, pSyncNode->rid);
return ret;
}
int32_t syncReconfigRaw(int64_t rid, const SSyncCfg* pNewCfg, SRpcMsg* pRpcMsg) {
int32_t ret = 0;
char* newconfig = syncCfg2Str((SSyncCfg*)pNewCfg);
pRpcMsg->msgType = TDMT_SYNC_CONFIG_CHANGE;
pRpcMsg->info.noResp = 1;
pRpcMsg->contLen = strlen(newconfig) + 1;
pRpcMsg->pCont = rpcMallocCont(pRpcMsg->contLen);
snprintf(pRpcMsg->pCont, pRpcMsg->contLen, "%s", newconfig);
taosMemoryFree(newconfig);
return ret;
}
bool syncCanLeaderTransfer(int64_t rid) {
SSyncNode* pSyncNode = (SSyncNode*)taosAcquireRef(tsNodeRefId, rid);
if (pSyncNode == NULL) {
......@@ -273,8 +348,6 @@ bool syncCanLeaderTransfer(int64_t rid) {
return matchOK;
}
int32_t syncGiveUpLeader(int64_t rid) { return 0; }
int32_t syncForwardToPeer(int64_t rid, const SRpcMsg* pMsg, bool isWeak) {
int32_t ret = syncPropose(rid, pMsg, isWeak);
return ret;
......@@ -469,16 +542,26 @@ int32_t syncPropose(int64_t rid, const SRpcMsg* pMsg, bool isWeak) {
SSyncNode* pSyncNode = taosAcquireRef(tsNodeRefId, rid);
if (pSyncNode == NULL) {
return TAOS_SYNC_PROPOSE_OTHER_ERROR;
return TAOS_SYNC_OTHER_ERROR;
}
assert(rid == pSyncNode->rid);
sDebug("vgId:%d sync event propose msgType:%s", pSyncNode->vgId, TMSG_INFO(pMsg->msgType));
ret = syncNodePropose(pSyncNode, pMsg, isWeak);
taosReleaseRef(tsNodeRefId, pSyncNode->rid);
return ret;
}
int32_t syncNodePropose(SSyncNode* pSyncNode, const SRpcMsg* pMsg, bool isWeak) {
int32_t ret = TAOS_SYNC_PROPOSE_SUCCESS;
sDebug("vgId:%d sync event propose msgType:%s", pSyncNode->vgId, TMSG_INFO(pMsg->msgType));
if (pSyncNode->state == TAOS_SYNC_STATE_LEADER) {
SRespStub stub;
stub.createTime = taosGetTimestampMs();
stub.rpcMsg = *pMsg;
uint64_t seqNum = syncRespMgrAdd(pSyncNode->pSyncRespMgr, &stub);
sDebug("vgId:%d, sync event propose, type:%s seq:%" PRIu64 " handle:%p", pSyncNode->vgId, TMSG_INFO(pMsg->msgType),
sDebug("vgId:%d sync event propose, type:%s seq:%" PRIu64 " handle:%p", pSyncNode->vgId, TMSG_INFO(pMsg->msgType),
seqNum, pMsg->info.handle);
SyncClientRequest* pSyncMsg = syncClientRequestBuild2(pMsg, seqNum, isWeak, pSyncNode->vgId);
......@@ -488,16 +571,14 @@ int32_t syncPropose(int64_t rid, const SRpcMsg* pMsg, bool isWeak) {
if (pSyncNode->FpEqMsg != NULL && (*pSyncNode->FpEqMsg)(pSyncNode->msgcb, &rpcMsg) == 0) {
ret = TAOS_SYNC_PROPOSE_SUCCESS;
} else {
sTrace("syncPropose pSyncNode->FpEqMsg is NULL");
sError("syncPropose pSyncNode->FpEqMsg is NULL");
}
syncClientRequestDestroy(pSyncMsg);
} else {
sDebug("vgId:%d, failed to propose since not leader, type:%s handle:%p %s", pSyncNode->vgId,
TMSG_INFO(pMsg->msgType), pMsg->info.handle, syncUtilState2String(pSyncNode->state));
sError("syncPropose not leader, %s", syncUtilState2String(pSyncNode->state));
ret = TAOS_SYNC_PROPOSE_NOT_LEADER;
}
taosReleaseRef(tsNodeRefId, pSyncNode->rid);
return ret;
}
......@@ -1162,8 +1243,14 @@ void syncNodeUpdateConfig(SSyncNode* pSyncNode, SSyncCfg* newConfig, SyncIndex l
int32_t oldReplicaNum = pSyncNode->replicaNum;
SRaftId oldReplicasId[TSDB_MAX_REPLICA];
memcpy(oldReplicasId, pSyncNode->replicasId, sizeof(oldReplicasId));
SSyncSnapshotSender* oldSenders[TSDB_MAX_REPLICA];
memcpy(oldSenders, pSyncNode->senders, sizeof(oldSenders));
SSyncSnapshotSender* oldSenders[TSDB_MAX_REPLICA];
for (int i = 0; i < TSDB_MAX_REPLICA; ++i) {
oldSenders[i] = (pSyncNode->senders)[i];
sDebug("vgId:%d sync event save senders %d, %p", pSyncNode->vgId, i, oldSenders[i]);
if (gRaftDetailLog) {
;
}
}
// init internal
pSyncNode->myNodeInfo = pSyncNode->pRaftCfg->cfg.nodeInfo[pSyncNode->pRaftCfg->cfg.myIndex];
......@@ -1195,24 +1282,51 @@ void syncNodeUpdateConfig(SSyncNode* pSyncNode, SSyncCfg* newConfig, SyncIndex l
pSyncNode->quorum = syncUtilQuorum(pSyncNode->pRaftCfg->cfg.replicaNum);
// reset snapshot senders, memory leak
// reset snapshot senders
// clear new
for (int i = 0; i < TSDB_MAX_REPLICA; ++i) {
(pSyncNode->senders)[i] = NULL;
}
// reset new
for (int i = 0; i < pSyncNode->replicaNum; ++i) {
// reset sender
bool reset = false;
for (int j = 0; j < TSDB_MAX_REPLICA; ++j) {
if (syncUtilSameId(&(pSyncNode->replicasId)[i], &oldReplicasId[j])) {
char host[128];
char host[128];
uint16_t port;
syncUtilU642Addr((pSyncNode->replicasId)[i].addr, host, sizeof(host), &port);
sDebug("vgId:%d sync event reset sender for %lu, %s:%d", pSyncNode->vgId, (pSyncNode->replicasId)[i].addr, host, port);
sDebug("vgId:%d sync event reset sender for %lu, newIndex:%d, %s:%d, %p", pSyncNode->vgId,
(pSyncNode->replicasId)[i].addr, i, host, port, oldSenders[j]);
(pSyncNode->senders)[i] = oldSenders[j];
oldSenders[j] = NULL;
reset = true;
// reset replicaIndex
int32_t oldreplicaIndex = (pSyncNode->senders)[i]->replicaIndex;
(pSyncNode->senders)[i]->replicaIndex = i;
sDebug("vgId:%d sync event udpate replicaIndex from %d to %d, %s:%d, %p, reset:%d", pSyncNode->vgId,
oldreplicaIndex, i, host, port, (pSyncNode->senders)[i], reset);
}
}
}
// create new
for (int i = 0; i < TSDB_MAX_REPLICA; ++i) {
if ((pSyncNode->senders)[i] == NULL) {
(pSyncNode->senders)[i] = snapshotSenderCreate(pSyncNode, i);
sDebug("vgId:%d sync event create new sender %p replicaIndex:%d", pSyncNode->vgId, (pSyncNode->senders)[i], i);
}
}
// free old
for (int i = 0; i < TSDB_MAX_REPLICA; ++i) {
if (oldSenders[i] != NULL) {
snapshotSenderDestroy(oldSenders[i]);
sDebug("vgId:%d sync event delete old sender %p replicaIndex:%d", pSyncNode->vgId, oldSenders[i], i);
oldSenders[i] = NULL;
}
}
......@@ -1272,8 +1386,8 @@ void syncNodeUpdateTerm(SSyncNode* pSyncNode, SyncTerm term) {
}
void syncNodeBecomeFollower(SSyncNode* pSyncNode, const char* debugStr) {
sDebug("vgId:%d sync event become follower, isStandBy:%d, %s", pSyncNode->vgId, pSyncNode->pRaftCfg->isStandBy,
debugStr);
sDebug("vgId:%d sync event become follower, isStandBy:%d, replicaNum:%d, %s", pSyncNode->vgId,
pSyncNode->pRaftCfg->isStandBy, pSyncNode->replicaNum, debugStr);
// maybe clear leader cache
if (pSyncNode->state == TAOS_SYNC_STATE_LEADER) {
......@@ -1307,8 +1421,8 @@ void syncNodeBecomeFollower(SSyncNode* pSyncNode, const char* debugStr) {
// /\ UNCHANGED <<messages, currentTerm, votedFor, candidateVars, logVars>>
//
void syncNodeBecomeLeader(SSyncNode* pSyncNode, const char* debugStr) {
sDebug("vgId:%d sync event become leader, isStandBy:%d, %s", pSyncNode->vgId, pSyncNode->pRaftCfg->isStandBy,
debugStr);
sDebug("vgId:%d sync event become leader, isStandBy:%d, replicaNum:%d %s", pSyncNode->vgId,
pSyncNode->pRaftCfg->isStandBy, pSyncNode->replicaNum, debugStr);
// state change
pSyncNode->state = TAOS_SYNC_STATE_LEADER;
......@@ -1857,10 +1971,52 @@ const char* syncStr(ESyncState state) {
}
static int32_t syncDoLeaderTransfer(SSyncNode* ths, SRpcMsg* pRpcMsg, SSyncRaftEntry* pEntry) {
SyncLeaderTransfer* pSyncLeaderTransfer;
if (syncUtilSameId(&(pSyncLeaderTransfer->newLeaderId), &(ths->myRaftId))) {
SyncLeaderTransfer* pSyncLeaderTransfer = syncLeaderTransferFromRpcMsg2(pRpcMsg);
/*
char host[128];
uint16_t port;
syncUtilU642Addr(pSyncLeaderTransfer->newLeaderId.addr, host, sizeof(host), &port);
sDebug("vgId:%d sync event, maybe leader transfer to %s:%d %lu", ths->vgId, host, port,
pSyncLeaderTransfer->newLeaderId.addr);
*/
sDebug("vgId:%d sync event, begin leader transfer", ths->vgId);
if (strcmp(pSyncLeaderTransfer->newNodeInfo.nodeFqdn, ths->myNodeInfo.nodeFqdn) == 0 &&
pSyncLeaderTransfer->newNodeInfo.nodePort == ths->myNodeInfo.nodePort) {
sDebug("vgId:%d sync event, maybe leader transfer to %s:%d %lu", ths->vgId,
pSyncLeaderTransfer->newNodeInfo.nodeFqdn, pSyncLeaderTransfer->newNodeInfo.nodePort,
pSyncLeaderTransfer->newLeaderId.addr);
// reset elect timer now!
int32_t electMS = 1;
int32_t ret = syncNodeRestartElectTimer(ths, electMS);
ASSERT(ret == 0);
}
/*
if (syncUtilSameId(&(pSyncLeaderTransfer->newLeaderId), &(ths->myRaftId))) {
// reset elect timer now!
int32_t electMS = 1;
int32_t ret = syncNodeRestartElectTimer(ths, electMS);
ASSERT(ret == 0);
}
*/
if (ths->pFsm->FpLeaderTransferCb != NULL) {
SFsmCbMeta cbMeta;
cbMeta.code = 0;
cbMeta.currentTerm = ths->pRaftStore->currentTerm;
cbMeta.flag = 0;
cbMeta.index = pEntry->index;
cbMeta.isWeak = pEntry->isWeak;
cbMeta.seqNum = pEntry->seqNum;
cbMeta.state = ths->state;
cbMeta.term = pEntry->term;
ths->pFsm->FpLeaderTransferCb(ths->pFsm, pRpcMsg, cbMeta);
}
syncLeaderTransferDestroy(pSyncLeaderTransfer);
return 0;
}
......@@ -1884,26 +2040,31 @@ static int32_t syncNodeConfigChange(SSyncNode* ths, SRpcMsg* pRpcMsg, SSyncRaftE
bool isDrop;
//if (IamInNew || (!IamInNew && ths->state != TAOS_SYNC_STATE_LEADER)) {
// if (IamInNew || (!IamInNew && ths->state != TAOS_SYNC_STATE_LEADER)) {
if (IamInNew) {
syncNodeUpdateConfig(ths, &newSyncCfg, pEntry->index, &isDrop);
// change isStandBy to normal
if (!isDrop) {
char tmpbuf[128];
snprintf(tmpbuf, sizeof(tmpbuf), "config change from %d to %d", oldSyncCfg.replicaNum, newSyncCfg.replicaNum);
if (ths->state == TAOS_SYNC_STATE_LEADER) {
syncNodeBecomeLeader(ths, "config change");
syncNodeBecomeLeader(ths, tmpbuf);
} else {
syncNodeBecomeFollower(ths, "config change");
syncNodeBecomeFollower(ths, tmpbuf);
}
}
} else {
syncNodeBecomeFollower(ths, "config change2");
char tmpbuf[128];
snprintf(tmpbuf, sizeof(tmpbuf), "config change2 from %d to %d", oldSyncCfg.replicaNum, newSyncCfg.replicaNum);
syncNodeBecomeFollower(ths, tmpbuf);
}
if (gRaftDetailLog) {
char* sOld = syncCfg2Str(&oldSyncCfg);
char* sNew = syncCfg2Str(&newSyncCfg);
sInfo("==config change== 0x11 old:%s new:%s isDrop:%d index:%ld \n", sOld, sNew, isDrop, pEntry->index);
sInfo("==config change== 0x11 old:%s new:%s isDrop:%d index:%ld IamInNew:%d \n", sOld, sNew, isDrop, pEntry->index,
IamInNew);
taosMemoryFree(sOld);
taosMemoryFree(sNew);
}
......@@ -1965,7 +2126,7 @@ int32_t syncNodeCommit(SSyncNode* ths, SyncIndex beginIndex, SyncIndex endIndex,
ASSERT(code == 0);
}
// config change
// leader transfer
if (pEntry->originalRpcType == TDMT_SYNC_LEADER_TRANSFER) {
code = syncDoLeaderTransfer(ths, &rpcMsg, pEntry);
ASSERT(code == 0);
......
......@@ -393,7 +393,7 @@ cJSON *snapshotSender2Json(SSyncSnapshotSender *pSender) {
char *snapshotSender2Str(SSyncSnapshotSender *pSender) {
cJSON *pJson = snapshotSender2Json(pSender);
char * serialized = cJSON_Print(pJson);
char *serialized = cJSON_Print(pJson);
cJSON_Delete(pJson);
return serialized;
}
......@@ -514,7 +514,7 @@ cJSON *snapshotReceiver2Json(SSyncSnapshotReceiver *pReceiver) {
cJSON_AddStringToObject(pFromId, "addr", u64buf);
{
uint64_t u64 = pReceiver->fromId.addr;
cJSON * pTmp = pFromId;
cJSON *pTmp = pFromId;
char host[128] = {0};
uint16_t port;
syncUtilU642Addr(u64, host, sizeof(host), &port);
......@@ -538,7 +538,7 @@ cJSON *snapshotReceiver2Json(SSyncSnapshotReceiver *pReceiver) {
char *snapshotReceiver2Str(SSyncSnapshotReceiver *pReceiver) {
cJSON *pJson = snapshotReceiver2Json(pReceiver);
char * serialized = cJSON_Print(pJson);
char *serialized = cJSON_Print(pJson);
cJSON_Delete(pJson);
return serialized;
}
......@@ -588,6 +588,8 @@ int32_t syncNodeOnSnapshotSendCb(SSyncNode *pSyncNode, SyncSnapshotSend *pMsg) {
// maybe update lastconfig
if (pMsg->lastConfigIndex >= SYNC_INDEX_BEGIN) {
int32_t oldReplicaNum = pSyncNode->replicaNum;
// update new config myIndex
bool IamInNew = false;
SSyncCfg newSyncCfg = pMsg->lastConfig;
......@@ -614,10 +616,12 @@ int32_t syncNodeOnSnapshotSendCb(SSyncNode *pSyncNode, SyncSnapshotSend *pMsg) {
// change isStandBy to normal
if (!isDrop) {
char tmpbuf[128];
snprintf(tmpbuf, sizeof(tmpbuf), "config change3 from %d to %d", oldReplicaNum, newSyncCfg.replicaNum);
if (pSyncNode->state == TAOS_SYNC_STATE_LEADER) {
syncNodeBecomeLeader(pSyncNode, "config change");
syncNodeBecomeLeader(pSyncNode, tmpbuf);
} else {
syncNodeBecomeFollower(pSyncNode, "config change");
syncNodeBecomeFollower(pSyncNode, tmpbuf);
}
}
}
......
add_executable(syncTest "")
add_executable(syncRaftIdCheck "")
add_executable(syncEnvTest "")
add_executable(syncPingTimerTest "")
add_executable(syncIOTickQTest "")
......@@ -54,6 +55,10 @@ target_sources(syncTest
PRIVATE
"syncTest.cpp"
)
target_sources(syncRaftIdCheck
PRIVATE
"syncRaftIdCheck.cpp"
)
target_sources(syncEnvTest
PRIVATE
"syncEnvTest.cpp"
......@@ -257,6 +262,11 @@ target_include_directories(syncTest
"${TD_SOURCE_DIR}/include/libs/sync"
"${CMAKE_CURRENT_SOURCE_DIR}/../inc"
)
target_include_directories(syncRaftIdCheck
PUBLIC
"${TD_SOURCE_DIR}/include/libs/sync"
"${CMAKE_CURRENT_SOURCE_DIR}/../inc"
)
target_include_directories(syncEnvTest
PUBLIC
"${TD_SOURCE_DIR}/include/libs/sync"
......@@ -508,6 +518,10 @@ target_link_libraries(syncTest
sync
gtest_main
)
target_link_libraries(syncRaftIdCheck
sync
gtest_main
)
target_link_libraries(syncEnvTest
sync
gtest_main
......
#include <gtest/gtest.h>
#include <stdio.h>
#include "syncIO.h"
#include "syncInt.h"
#include "syncUtil.h"
void usage(char* exe) {
printf("Usage: %s host port \n", exe);
printf("Usage: %s u64 \n", exe);
}
int main(int argc, char** argv) {
if (argc == 2) {
uint64_t u64 = atoll(argv[1]);
char host[128];
uint16_t port;
syncUtilU642Addr(u64, host, sizeof(host), &port);
printf("%lu -> %s:%d \n", u64, host, port);
} else if (argc == 3) {
uint64_t u64;
char* host = argv[1];
uint16_t port = atoi(argv[2]);
u64 = syncUtilAddr2U64(host, port);
printf("%s:%d -> %lu \n", host, port, u64);
} else {
usage(argv[0]);
exit(-1);
}
return 0;
}
......@@ -153,6 +153,16 @@ void ReConfigCb(struct SSyncFSM* pFsm, const SRpcMsg* pMsg, SReConfigCbMeta cbMe
taosMemoryFree(s);
}
void LeaderTransferCb(struct SSyncFSM* pFsm, const SRpcMsg* pMsg, SFsmCbMeta cbMeta) {
char logBuf[256] = {0};
snprintf(logBuf, sizeof(logBuf),
"==callback== ==LeaderTransferCb== pFsm:%p, index:%ld, isWeak:%d, code:%d, state:%d %s, flag:%lu, term:%lu "
"currentTerm:%lu \n",
pFsm, cbMeta.index, cbMeta.isWeak, cbMeta.code, cbMeta.state, syncUtilState2String(cbMeta.state),
cbMeta.flag, cbMeta.term, cbMeta.currentTerm);
syncRpcMsgLog2(logBuf, (SRpcMsg*)pMsg);
}
SSyncFSM* createFsm() {
SSyncFSM* pFsm = (SSyncFSM*)taosMemoryMalloc(sizeof(SSyncFSM));
memset(pFsm, 0, sizeof(*pFsm));
......@@ -172,6 +182,8 @@ SSyncFSM* createFsm() {
pFsm->FpSnapshotStopWrite = SnapshotStopWrite;
pFsm->FpSnapshotDoWrite = SnapshotDoWrite;
pFsm->FpLeaderTransferCb = LeaderTransferCb;
return pFsm;
}
......@@ -277,7 +289,8 @@ void usage(char* exe) {
printf(
"usage: %s replicaNum(1-5) myIndex(0-..) enableSnapshot(0/1) lastApplyIndex(>=-1) lastApplyTerm(>=0) "
"writeRecordNum(>=0) "
"isStandBy(0/1) isConfigChange(0-5) iterTimes(>=0) finishLastApplyIndex(>=-1) finishLastApplyTerm(>=0) \n",
"isStandBy(0/1) isConfigChange(0-5) iterTimes(>=0) finishLastApplyIndex(>=-1) finishLastApplyTerm(>=0) "
"leaderTransfer(0/1) \n",
exe);
}
......@@ -294,9 +307,9 @@ SRpcMsg* createRpcMsg(int i, int count, int myIndex) {
int main(int argc, char** argv) {
sprintf(tsTempDir, "%s", ".");
tsAsyncLog = 0;
sDebugFlag = DEBUG_SCREEN + DEBUG_FILE + DEBUG_TRACE + DEBUG_INFO + DEBUG_ERROR;
sDebugFlag = DEBUG_SCREEN + DEBUG_FILE + DEBUG_TRACE + DEBUG_INFO + DEBUG_ERROR + DEBUG_DEBUG;
if (argc != 12) {
if (argc != 13) {
usage(argv[0]);
exit(-1);
}
......@@ -312,12 +325,14 @@ int main(int argc, char** argv) {
int32_t iterTimes = atoi(argv[9]);
int32_t finishLastApplyIndex = atoi(argv[10]);
int32_t finishLastApplyTerm = atoi(argv[11]);
int32_t leaderTransfer = atoi(argv[12]);
sTrace(
sInfo(
"args: replicaNum:%d, myIndex:%d, enableSnapshot:%d, lastApplyIndex:%d, lastApplyTerm:%d, writeRecordNum:%d, "
"isStandBy:%d, isConfigChange:%d, iterTimes:%d, finishLastApplyIndex:%d, finishLastApplyTerm:%d",
"isStandBy:%d, isConfigChange:%d, iterTimes:%d, finishLastApplyIndex:%d, finishLastApplyTerm:%d, "
"leaderTransfer:%d",
replicaNum, myIndex, enableSnapshot, lastApplyIndex, lastApplyTerm, writeRecordNum, isStandBy, isConfigChange,
iterTimes, finishLastApplyIndex, finishLastApplyTerm);
iterTimes, finishLastApplyIndex, finishLastApplyTerm, leaderTransfer);
// check parameter
assert(replicaNum >= 1 && replicaNum <= 5);
......@@ -363,24 +378,31 @@ int main(int argc, char** argv) {
//---------------------------
int32_t alreadySend = 0;
int32_t leaderTransferWait = 0;
while (1) {
char* simpleStr = syncNode2SimpleStr(pSyncNode);
leaderTransferWait++;
if (leaderTransferWait == 7) {
sTrace("begin leader transfer ...");
int32_t ret = syncLeaderTransfer(rid);
}
if (alreadySend < writeRecordNum) {
SRpcMsg* pRpcMsg = createRpcMsg(alreadySend, writeRecordNum, myIndex);
int32_t ret = syncPropose(rid, pRpcMsg, false);
if (ret == TAOS_SYNC_PROPOSE_NOT_LEADER) {
sTrace("%s value%d write not leader", simpleStr, alreadySend);
sTrace("%s value%d write not leader, leaderTransferWait:%d", simpleStr, alreadySend, leaderTransferWait);
} else {
assert(ret == 0);
sTrace("%s value%d write ok", simpleStr, alreadySend);
sTrace("%s value%d write ok, leaderTransferWait:%d", simpleStr, alreadySend, leaderTransferWait);
}
alreadySend++;
rpcFreeCont(pRpcMsg->pCont);
taosMemoryFree(pRpcMsg);
} else {
sTrace("%s", simpleStr);
sTrace("%s, leaderTransferWait:%d", simpleStr, leaderTransferWait);
}
taosMsleep(1000);
......
......@@ -45,13 +45,13 @@ int32_t BUILDIN_CLZL(uint64_t val) {
#else
_MyBitScanReverse64(&r, val);
#endif
return (int)(r >> 3);
return (int)(63 - r);
}
int32_t BUILDIN_CLZ(uint32_t val) {
unsigned long r = 0;
_BitScanReverse(&r, val);
return (int)(r >> 3);
return (int)(31 - r);
}
int32_t BUILDIN_CTZL(uint64_t val) {
......@@ -61,13 +61,13 @@ int32_t BUILDIN_CTZL(uint64_t val) {
#else
_MyBitScanForward64(&r, val);
#endif
return (int)(r >> 3);
return (int)(r);
}
int32_t BUILDIN_CTZ(uint32_t val) {
unsigned long r = 0;
_BitScanForward(&r, val);
return (int)(r >> 3);
return (int)(r);
}
#endif
......@@ -567,6 +567,7 @@ TAOS_DEFINE_ERROR(TSDB_CODE_TSMA_RM_SKEY_IN_HASH, "Rm tsma skey in cac
TAOS_DEFINE_ERROR(TSDB_CODE_RSMA_INVALID_ENV, "Invalid rsma env")
TAOS_DEFINE_ERROR(TSDB_CODE_RSMA_INVALID_STAT, "Invalid rsma state")
TAOS_DEFINE_ERROR(TSDB_CODE_INDEX_REBUILDING, "Index is rebuilding")
#ifdef TAOS_ERROR_C
};
......
......@@ -78,6 +78,7 @@
# ./test.sh -f tsim/stream/state0.sim
# ./test.sh -f tsim/stream/triggerInterval0.sim
# ./test.sh -f tsim/stream/triggerSession0.sim
# ./test.sh -f tsim/stream/partitionby.sim
# ---- transaction
......
system sh/stop_dnodes.sh
system sh/deploy.sh -n dnode1 -i 1
system sh/exec.sh -n dnode1 -s start
sleep 50
sql connect
sql create database test vgroups 4;
sql use test;
sql create stable st(ts timestamp, a int, b int , c int, d double) tags(ta int,tb int,tc int);
sql create table ts1 using st tags(1,1,1);
sql create table ts2 using st tags(2,2,2);
sql create table ts3 using st tags(3,2,2);
sql create table ts4 using st tags(4,2,2);
sql create stream stream_t1 trigger at_once into streamtST1 as select _wstartts, count(*) c1, count(d) c2 , sum(a) c3 , max(b) c4, min(c) c5 from st partition by ta,tb,tc interval(10s);
sql insert into ts1 values(1648791213001,1,12,3,1.0);
sql insert into ts2 values(1648791213001,1,12,3,1.0);
sql insert into ts3 values(1648791213001,1,12,3,1.0);
sql insert into ts4 values(1648791213001,1,12,3,1.0);
$loop_count = 0
loop0:
sleep 300
sql select * from streamtST1;
$loop_count = $loop_count + 1
if $loop_count == 10 then
return -1
endi
if $rows != 4 then
print =====rows=$rows
goto loop0
endi
sql create database test1 vgroups 1;
sql use test1;
sql create stable st(ts timestamp,a int,b int,c int) tags(ta int,tb int,tc int);
sql create table ts1 using st tags(1,2,3);
sql create table ts2 using st tags(1,3,4);
sql create table ts3 using st tags(1,4,5);
sql create stream streams1 trigger at_once into streamt as select _wstartts, count(*) c1, count(a) c2 from st partition by ta,tb,tc interval(10s);
sql insert into ts1 values(1648791211000,1,2,3);
sql insert into ts2 values(1648791211000,1,2,3);
$loop_count = 0
loop0:
sleep 300
sql select * from streamt;
$loop_count = $loop_count + 1
if $loop_count == 10 then
return -1
endi
if $rows != 2 then
print =====rows=$rows
goto loop0
endi
system sh/exec.sh -n dnode1 -s stop -x SIGINT
\ No newline at end of file
......@@ -437,12 +437,12 @@ class TDTestCase:
event.wait()
tdLog.info("start consume processor")
pollDelay = 100
pollDelay = 20
showMsg = 1
showRow = 1
self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow)
time.sleep(5)
time.sleep(3)
tdLog.info("pkill consume processor")
if (platform.system().lower() == 'windows'):
os.system("TASKKILL /F /IM tmq_sim.exe")
......@@ -465,7 +465,7 @@ class TDTestCase:
totalConsumeRows += resultList[i]
if totalConsumeRows >= expectrowcnt or totalConsumeRows <= 0:
tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt))
tdLog.info("act consume rows: %d, expect consume rows between %d and 0"%(totalConsumeRows, expectrowcnt))
tdLog.exit("tmq consume rows error!")
time.sleep(15)
......
......@@ -104,7 +104,7 @@ python3 ./test.py -f 6-cluster/5dnode3mnodeStop.py
python3 ./test.py -f 7-tmq/basic5.py
python3 ./test.py -f 7-tmq/subscribeDb.py
python3 ./test.py -f 7-tmq/subscribeDb0.py
# python3 ./test.py -f 7-tmq/subscribeDb1.py
python3 ./test.py -f 7-tmq/subscribeDb1.py
python3 ./test.py -f 7-tmq/subscribeStb.py
python3 ./test.py -f 7-tmq/subscribeStb0.py
python3 ./test.py -f 7-tmq/subscribeStb1.py
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册