提交 3df96204 编写于 作者: H Hongze Cheng

Merge branch '3.0' of https://github.com/taosdata/TDengine into enh/tsdb_optimize

......@@ -2807,39 +2807,49 @@ typedef struct {
int64_t suid;
} SMqRebVgReq;
static FORCE_INLINE int32_t tEncodeSMqRebVgReq(void** buf, const SMqRebVgReq* pReq) {
int32_t tlen = 0;
tlen += taosEncodeFixedI64(buf, pReq->leftForVer);
tlen += taosEncodeFixedI32(buf, pReq->vgId);
tlen += taosEncodeFixedI64(buf, pReq->oldConsumerId);
tlen += taosEncodeFixedI64(buf, pReq->newConsumerId);
tlen += taosEncodeString(buf, pReq->subKey);
tlen += taosEncodeFixedI8(buf, pReq->subType);
tlen += taosEncodeFixedI8(buf, pReq->withMeta);
static FORCE_INLINE int tEncodeSMqRebVgReq(SEncoder *pCoder, const SMqRebVgReq* pReq) {
if (tStartEncode(pCoder) < 0) return -1;
if (tEncodeI64(pCoder, pReq->leftForVer) < 0) return -1;
if (tEncodeI32(pCoder, pReq->vgId) < 0) return -1;
if (tEncodeI64(pCoder, pReq->oldConsumerId) < 0) return -1;
if (tEncodeI64(pCoder, pReq->newConsumerId) < 0) return -1;
if (tEncodeCStr(pCoder, pReq->subKey) < 0) return -1;
if (tEncodeI8(pCoder, pReq->subType) < 0) return -1;
if (tEncodeI8(pCoder, pReq->withMeta) < 0) return -1;
if (pReq->subType == TOPIC_SUB_TYPE__COLUMN) {
tlen += taosEncodeString(buf, pReq->qmsg);
if (tEncodeCStr(pCoder, pReq->qmsg) < 0) return -1;
} else if (pReq->subType == TOPIC_SUB_TYPE__TABLE) {
tlen += taosEncodeFixedI64(buf, pReq->suid);
tlen += taosEncodeString(buf, pReq->qmsg);
if (tEncodeI64(pCoder, pReq->suid) < 0) return -1;
if (tEncodeCStr(pCoder, pReq->qmsg) < 0) return -1;
}
return tlen;
tEndEncode(pCoder);
return 0;
}
static FORCE_INLINE void* tDecodeSMqRebVgReq(const void* buf, SMqRebVgReq* pReq) {
buf = taosDecodeFixedI64(buf, &pReq->leftForVer);
buf = taosDecodeFixedI32(buf, &pReq->vgId);
buf = taosDecodeFixedI64(buf, &pReq->oldConsumerId);
buf = taosDecodeFixedI64(buf, &pReq->newConsumerId);
buf = taosDecodeStringTo(buf, pReq->subKey);
buf = taosDecodeFixedI8(buf, &pReq->subType);
buf = taosDecodeFixedI8(buf, &pReq->withMeta);
static FORCE_INLINE int tDecodeSMqRebVgReq(SDecoder *pCoder, SMqRebVgReq* pReq) {
if (tStartDecode(pCoder) < 0) return -1;
if (tDecodeI64(pCoder, &pReq->leftForVer) < 0) return -1;
if (tDecodeI32(pCoder, &pReq->vgId) < 0) return -1;
if (tDecodeI64(pCoder, &pReq->oldConsumerId) < 0) return -1;
if (tDecodeI64(pCoder, &pReq->newConsumerId) < 0) return -1;
if (tDecodeCStrTo(pCoder, pReq->subKey) < 0) return -1;
if (tDecodeI8(pCoder, &pReq->subType) < 0) return -1;
if (tDecodeI8(pCoder, &pReq->withMeta) < 0) return -1;
if (pReq->subType == TOPIC_SUB_TYPE__COLUMN) {
buf = taosDecodeString(buf, &pReq->qmsg);
if (tDecodeCStr(pCoder, &pReq->qmsg) < 0) return -1;
} else if (pReq->subType == TOPIC_SUB_TYPE__TABLE) {
buf = taosDecodeFixedI64(buf, &pReq->suid);
buf = taosDecodeString(buf, &pReq->qmsg);
if (tDecodeI64(pCoder, &pReq->suid) < 0) return -1;
if (!tDecodeIsEnd(pCoder)){
if (tDecodeCStr(pCoder, &pReq->qmsg) < 0) return -1;
}
}
return (void*)buf;
tEndDecode(pCoder);
return 0;
}
typedef struct {
......
......@@ -261,6 +261,9 @@ function install_lib() {
${csudo}ln -sf ${install_main_dir}/driver/librocksdb.* ${lib_link_dir}/librocksdb.so.8
${csudo}ln -sf ${lib_link_dir}/librocksdb.so.8 ${lib_link_dir}/librocksdb.so
${csudo}ln -sf ${install_main_dir}/driver/librocksdb.* ${lib_link_dir}/librocksdb.so.8
${csudo}ln -sf ${lib_link_dir}/librocksdb.so.8 ${lib_link_dir}/librocksdb.so
[ -f ${install_main_dir}/driver/libtaosws.so ] && ${csudo}ln -sf ${install_main_dir}/driver/libtaosws.so ${lib_link_dir}/libtaosws.so || :
......
......@@ -207,6 +207,10 @@ function install_lib() {
[ -f ${lib_link_dir}/librocksdb* ] && ${csudo}rm -f ${lib_link_dir}/librocksdb* || :
[ -f ${lib64_link_dir}/librocksdb* ] && ${csudo}rm -f ${lib64_link_dir}/librocksdb* || :
#rocksdb
[ -f ${lib_link_dir}/librocksdb* ] && ${csudo}rm -f ${lib_link_dir}/librocksdb* || :
[ -f ${lib64_link_dir}/librocksdb* ] && ${csudo}rm -f ${lib64_link_dir}/librocksdb* || :
[ -f ${lib_link_dir}/libtaosws.${lib_file_ext} ] && ${csudo}rm -f ${lib_link_dir}/libtaosws.${lib_file_ext} || :
[ -f ${lib64_link_dir}/libtaosws.${lib_file_ext} ] && ${csudo}rm -f ${lib64_link_dir}/libtaosws.${lib_file_ext} || :
......
......@@ -1868,7 +1868,10 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout, bool pollIfReset) {
// update the local offset value only for the returned values, only when the local offset is NOT updated
// by tmq_offset_seek function
if (!pVg->seekUpdated) {
tscDebug("consumer:0x%" PRIx64" local offset is update, since seekupdate not set", tmq->consumerId);
pVg->offsetInfo.currentOffset = pDataRsp->rspOffset;
} else {
tscDebug("consumer:0x%" PRIx64" local offset is NOT update, since seekupdate is set", tmq->consumerId);
}
// update the status
......@@ -1952,8 +1955,15 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout, bool pollIfReset) {
return NULL;
}
if(pollRspWrapper->taosxRsp.rspOffset.type != 0){ // if offset is validate
pVg->offsetInfo.currentOffset = pollRspWrapper->taosxRsp.rspOffset;
// update the local offset value only for the returned values, only when the local offset is NOT updated
// by tmq_offset_seek function
if (!pVg->seekUpdated) {
if(pollRspWrapper->taosxRsp.rspOffset.type != 0) { // if offset is validate
tscDebug("consumer:0x%" PRIx64" local offset is update, since seekupdate not set", tmq->consumerId);
pVg->offsetInfo.currentOffset = pollRspWrapper->taosxRsp.rspOffset;
}
} else {
tscDebug("consumer:0x%" PRIx64" local offset is NOT update, since seekupdate is set", tmq->consumerId);
}
atomic_store_32(&pVg->vgStatus, TMQ_VG_STATUS__IDLE);
......
......@@ -111,7 +111,14 @@ static int32_t mndBuildSubChangeReq(void **pBuf, int32_t *pLen, const SMqSubscri
req.suid = pSub->stbUid;
tstrncpy(req.subKey, pSub->key, TSDB_SUBSCRIBE_KEY_LEN);
int32_t tlen = sizeof(SMsgHead) + tEncodeSMqRebVgReq(NULL, &req);
int32_t tlen = 0;
int32_t ret = 0;
tEncodeSize(tEncodeSMqRebVgReq, &req, tlen, ret);
if (ret < 0) {
return -1;
}
tlen += sizeof(SMsgHead);
void *buf = taosMemoryMalloc(tlen);
if (buf == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY;
......@@ -123,8 +130,14 @@ static int32_t mndBuildSubChangeReq(void **pBuf, int32_t *pLen, const SMqSubscri
pMsgHead->contLen = htonl(tlen);
pMsgHead->vgId = htonl(pRebVg->pVgEp->vgId);
void *abuf = POINTER_SHIFT(buf, sizeof(SMsgHead));
tEncodeSMqRebVgReq(&abuf, &req);
SEncoder encoder = {0};
tEncoderInit(&encoder, POINTER_SHIFT(buf, sizeof(SMsgHead)), tlen);
if (tEncodeSMqRebVgReq(&encoder, &req) < 0) {
taosMemoryFreeClear(buf);
tEncoderClear(&encoder);
return -1;
}
tEncoderClear(&encoder);
*pBuf = buf;
*pLen = tlen;
......
......@@ -200,7 +200,7 @@ STqReader *tqReaderOpen(SVnode *pVnode);
void tqReaderClose(STqReader *);
void tqReaderSetColIdList(STqReader *pReader, SArray *pColIdList);
int32_t tqReaderSetTbUidList(STqReader *pReader, const SArray *tbUidList);
int32_t tqReaderSetTbUidList(STqReader *pReader, const SArray *tbUidList, const char* id);
int32_t tqReaderAddTbUidList(STqReader *pReader, const SArray *pTableUidList);
int32_t tqReaderRemoveTbUidList(STqReader *pReader, const SArray *tbUidList);
......
......@@ -297,6 +297,7 @@ int32_t tsdbUpdateDelFileHdr(SDelFWriter *pWriter);
// SDelFReader
int32_t tsdbDelFReaderOpen(SDelFReader **ppReader, SDelFile *pFile, STsdb *pTsdb);
int32_t tsdbDelFReaderClose(SDelFReader **ppReader);
int32_t tsdbReadDelDatav1(SDelFReader *pReader, SDelIdx *pDelIdx, SArray *aDelData, int64_t maxVer);
int32_t tsdbReadDelData(SDelFReader *pReader, SDelIdx *pDelIdx, SArray *aDelData);
int32_t tsdbReadDelIdx(SDelFReader *pReader, SArray *aDelIdx);
// tsdbRead.c ==============================================================================================
......
......@@ -23,8 +23,8 @@
static int32_t tqInitialize(STQ* pTq);
static FORCE_INLINE bool tqIsHandleExec(STqHandle* pHandle) { return TMQ_HANDLE_STATUS_EXEC == pHandle->status; }
static FORCE_INLINE void tqSetHandleExec(STqHandle* pHandle) {pHandle->status = TMQ_HANDLE_STATUS_EXEC;}
static FORCE_INLINE void tqSetHandleIdle(STqHandle* pHandle) {pHandle->status = TMQ_HANDLE_STATUS_IDLE;}
static FORCE_INLINE void tqSetHandleExec(STqHandle* pHandle) { pHandle->status = TMQ_HANDLE_STATUS_EXEC; }
static FORCE_INLINE void tqSetHandleIdle(STqHandle* pHandle) { pHandle->status = TMQ_HANDLE_STATUS_IDLE; }
int32_t tqInit() {
int8_t old;
......@@ -78,7 +78,7 @@ static void destroyTqHandle(void* data) {
taosMemoryFreeClear(pData->execHandle.execTb.qmsg);
nodesDestroyNode(pData->execHandle.execTb.node);
}
if(pData->msg != NULL) {
if (pData->msg != NULL) {
rpcFreeCont(pData->msg->pCont);
taosMemoryFree(pData->msg);
pData->msg = NULL;
......@@ -240,14 +240,15 @@ int32_t tqPushDataRsp(STqHandle* pHandle, int32_t vgId) {
int64_t sver = 0, ever = 0;
walReaderValidVersionRange(pHandle->execHandle.pTqReader->pWalReader, &sver, &ever);
tqDoSendDataRsp(&pHandle->msg->info, &dataRsp, pHandle->epoch, pHandle->consumerId, TMQ_MSG_TYPE__POLL_RSP, sver, ever);
tqDoSendDataRsp(&pHandle->msg->info, &dataRsp, pHandle->epoch, pHandle->consumerId, TMQ_MSG_TYPE__POLL_RSP, sver,
ever);
char buf1[80] = {0};
char buf2[80] = {0};
tFormatOffset(buf1, tListLen(buf1), &dataRsp.reqOffset);
tFormatOffset(buf2, tListLen(buf2), &dataRsp.rspOffset);
tqDebug("vgId:%d, from consumer:0x%" PRIx64 " (epoch %d) push rsp, block num: %d, req:%s, rsp:%s",
vgId, dataRsp.head.consumerId, dataRsp.head.epoch, dataRsp.blockNum, buf1, buf2);
tqDebug("vgId:%d, from consumer:0x%" PRIx64 " (epoch %d) push rsp, block num: %d, req:%s, rsp:%s", vgId,
dataRsp.head.consumerId, dataRsp.head.epoch, dataRsp.blockNum, buf1, buf2);
return 0;
}
......@@ -263,8 +264,8 @@ int32_t tqSendDataRsp(STqHandle* pHandle, const SRpcMsg* pMsg, const SMqPollReq*
tFormatOffset(buf1, 80, &pRsp->reqOffset);
tFormatOffset(buf2, 80, &pRsp->rspOffset);
tqDebug("vgId:%d consumer:0x%" PRIx64 " (epoch %d) send rsp, block num:%d, req:%s, rsp:%s, reqId:0x%" PRIx64,
vgId, pReq->consumerId, pReq->epoch, pRsp->blockNum, buf1, buf2, pReq->reqId);
tqDebug("vgId:%d consumer:0x%" PRIx64 " (epoch %d) send rsp, block num:%d, req:%s, rsp:%s, reqId:0x%" PRIx64, vgId,
pReq->consumerId, pReq->epoch, pRsp->blockNum, buf1, buf2, pReq->reqId);
return 0;
}
......@@ -336,8 +337,7 @@ int32_t tqProcessSeekReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen)
STqHandle* pHandle = taosHashGet(pTq->pHandle, pOffset->subKey, strlen(pOffset->subKey));
if (pHandle == NULL) {
tqError("tmq seek: consumer:0x%" PRIx64 " vgId:%d subkey %s not found", vgOffset.consumerId, vgId,
pOffset->subKey);
tqError("tmq seek: consumer:0x%" PRIx64 " vgId:%d subkey %s not found", vgOffset.consumerId, vgId, pOffset->subKey);
terrno = TSDB_CODE_INVALID_MSG;
return -1;
}
......@@ -353,7 +353,7 @@ int32_t tqProcessSeekReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen)
}
taosRUnLockLatch(&pTq->lock);
//3. check the offset info
// 3. check the offset info
STqOffset* pSavedOffset = tqOffsetRead(pTq->pOffsetStore, pOffset->subKey);
if (pSavedOffset != NULL) {
if (pSavedOffset->val.type != TMQ_OFFSET__LOG) {
......@@ -381,7 +381,7 @@ int32_t tqProcessSeekReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen)
tqDebug("vgId:%d sub:%s seek to:%" PRId64 " prev offset:%" PRId64, vgId, pOffset->subKey, pOffset->val.version,
pSavedOffset->val.version);
} else {
tqDebug("vgId:%d sub:%s seek to:%"PRId64" not saved yet", vgId, pOffset->subKey, pOffset->val.version);
tqDebug("vgId:%d sub:%s seek to:%" PRId64 " not saved yet", vgId, pOffset->subKey, pOffset->val.version);
}
if (tqOffsetWrite(pTq->pOffsetStore, pOffset) < 0) {
......@@ -423,7 +423,7 @@ int32_t tqCheckColModifiable(STQ* pTq, int64_t tbUid, int32_t colId) {
int32_t tqProcessPollReq(STQ* pTq, SRpcMsg* pMsg) {
SMqPollReq req = {0};
int code = 0;
int code = 0;
if (tDeserializeSMqPollReq(pMsg->pCont, pMsg->contLen, &req) < 0) {
tqError("tDeserializeSMqPollReq %d failed", pMsg->contLen);
terrno = TSDB_CODE_INVALID_MSG;
......@@ -449,7 +449,8 @@ int32_t tqProcessPollReq(STQ* pTq, SRpcMsg* pMsg) {
// 2. check re-balance status
if (pHandle->consumerId != consumerId) {
tqError("ERROR tmq poll: consumer:0x%" PRIx64 " vgId:%d, subkey %s, mismatch for saved handle consumer:0x%" PRIx64,
tqError("ERROR tmq poll: consumer:0x%" PRIx64
" vgId:%d, subkey %s, mismatch for saved handle consumer:0x%" PRIx64,
consumerId, TD_VID(pTq->pVnode), req.subKey, pHandle->consumerId);
terrno = TSDB_CODE_TMQ_CONSUMER_MISMATCH;
taosWUnLockLatch(&pTq->lock);
......@@ -457,22 +458,26 @@ int32_t tqProcessPollReq(STQ* pTq, SRpcMsg* pMsg) {
}
bool exec = tqIsHandleExec(pHandle);
if(!exec) {
if (!exec) {
tqSetHandleExec(pHandle);
// qSetTaskCode(pHandle->execHandle.task, TDB_CODE_SUCCESS);
tqDebug("tmq poll: consumer:0x%" PRIx64 "vgId:%d, topic:%s, set handle exec, pHandle:%p", consumerId, vgId, req.subKey, pHandle);
// qSetTaskCode(pHandle->execHandle.task, TDB_CODE_SUCCESS);
tqDebug("tmq poll: consumer:0x%" PRIx64 "vgId:%d, topic:%s, set handle exec, pHandle:%p", consumerId, vgId,
req.subKey, pHandle);
taosWUnLockLatch(&pTq->lock);
break;
}
taosWUnLockLatch(&pTq->lock);
tqDebug("tmq poll: consumer:0x%" PRIx64 "vgId:%d, topic:%s, subscription is executing, wait for 10ms and retry, pHandle:%p", consumerId, vgId, req.subKey, pHandle);
tqDebug("tmq poll: consumer:0x%" PRIx64
"vgId:%d, topic:%s, subscription is executing, wait for 10ms and retry, pHandle:%p",
consumerId, vgId, req.subKey, pHandle);
taosMsleep(10);
}
// 3. update the epoch value
if (pHandle->epoch < reqEpoch) {
tqDebug("tmq poll: consumer:0x%" PRIx64 " epoch update from %d to %d by poll req", consumerId, pHandle->epoch, reqEpoch);
tqDebug("tmq poll: consumer:0x%" PRIx64 " epoch update from %d to %d by poll req", consumerId, pHandle->epoch,
reqEpoch);
pHandle->epoch = reqEpoch;
}
......@@ -484,7 +489,8 @@ int32_t tqProcessPollReq(STQ* pTq, SRpcMsg* pMsg) {
code = tqExtractDataForMq(pTq, pHandle, &req, pMsg);
tqSetHandleIdle(pHandle);
tqDebug("tmq poll: consumer:0x%" PRIx64 "vgId:%d, topic:%s, , set handle idle, pHandle:%p", consumerId, vgId, req.subKey, pHandle);
tqDebug("tmq poll: consumer:0x%" PRIx64 "vgId:%d, topic:%s, , set handle idle, pHandle:%p", consumerId, vgId,
req.subKey, pHandle);
return code;
}
......@@ -548,7 +554,7 @@ int32_t tqProcessVgWalInfoReq(STQ* pTq, SRpcMsg* pMsg) {
if (reqOffset.type == TMQ_OFFSET__LOG) {
int64_t currentVer = walReaderGetCurrentVer(pHandle->execHandle.pTqReader->pWalReader);
if (currentVer == -1) { // not start to read data from wal yet, return req offset directly
if (currentVer == -1) { // not start to read data from wal yet, return req offset directly
dataRsp.rspOffset.version = reqOffset.version;
} else {
dataRsp.rspOffset.version = currentVer; // return current consume offset value
......@@ -572,7 +578,7 @@ int32_t tqProcessVgWalInfoReq(STQ* pTq, SRpcMsg* pMsg) {
int32_t tqProcessDeleteSubReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen) {
SMqVDeleteReq* pReq = (SMqVDeleteReq*)msg;
int32_t vgId = TD_VID(pTq->pVnode);
int32_t vgId = TD_VID(pTq->pVnode);
tqDebug("vgId:%d, tq process delete sub req %s", vgId, pReq->subKey);
int32_t code = 0;
......@@ -581,7 +587,8 @@ int32_t tqProcessDeleteSubReq(STQ* pTq, int64_t sversion, char* msg, int32_t msg
STqHandle* pHandle = taosHashGet(pTq->pHandle, pReq->subKey, strlen(pReq->subKey));
if (pHandle) {
while (tqIsHandleExec(pHandle)) {
tqDebug("vgId:%d, topic:%s, subscription is executing, wait for 10ms and retry, pHandle:%p", vgId, pHandle->subKey, pHandle);
tqDebug("vgId:%d, topic:%s, subscription is executing, wait for 10ms and retry, pHandle:%p", vgId,
pHandle->subKey, pHandle);
taosMsleep(10);
}
......@@ -641,9 +648,18 @@ int32_t tqProcessDelCheckInfoReq(STQ* pTq, int64_t sversion, char* msg, int32_t
}
int32_t tqProcessSubscribeReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen) {
int ret = 0;
int ret = 0;
SMqRebVgReq req = {0};
tDecodeSMqRebVgReq(msg, &req);
SDecoder dc = {0};
tDecoderInit(&dc, msg, msgLen);
// decode req
if (tDecodeSMqRebVgReq(&dc, &req) < 0) {
terrno = TSDB_CODE_INVALID_MSG;
tDecoderClear(&dc);
return -1;
}
SVnode* pVnode = pTq->pVnode;
int32_t vgId = TD_VID(pVnode);
......@@ -689,8 +705,7 @@ int32_t tqProcessSubscribeReq(STQ* pTq, int64_t sversion, char* msg, int32_t msg
pHandle->snapshotVer = ver;
if (pHandle->execHandle.subType == TOPIC_SUB_TYPE__COLUMN) {
pHandle->execHandle.execCol.qmsg = req.qmsg;
req.qmsg = NULL;
pHandle->execHandle.execCol.qmsg = taosStrdup(req.qmsg);
pHandle->execHandle.task = qCreateQueueExecTaskInfo(pHandle->execHandle.execCol.qmsg, &handle, vgId,
&pHandle->execHandle.numOfCols, req.newConsumerId);
......@@ -710,10 +725,9 @@ int32_t tqProcessSubscribeReq(STQ* pTq, int64_t sversion, char* msg, int32_t msg
} else if (pHandle->execHandle.subType == TOPIC_SUB_TYPE__TABLE) {
pHandle->pWalReader = walOpenReader(pVnode->pWal, NULL);
pHandle->execHandle.execTb.suid = req.suid;
pHandle->execHandle.execTb.qmsg = req.qmsg;
req.qmsg = NULL;
pHandle->execHandle.execTb.qmsg = taosStrdup(req.qmsg);
if(strcmp(pHandle->execHandle.execTb.qmsg, "") != 0) {
if (strcmp(pHandle->execHandle.execTb.qmsg, "") != 0) {
if (nodesStringToNode(pHandle->execHandle.execTb.qmsg, &pHandle->execHandle.execTb.node) != 0) {
tqError("nodesStringToNode error in sub stable, since %s, vgId:%d, subkey:%s consumer:0x%" PRIx64, terrstr(),
pVnode->config.vgId, req.subKey, pHandle->consumerId);
......@@ -727,45 +741,47 @@ int32_t tqProcessSubscribeReq(STQ* pTq, int64_t sversion, char* msg, int32_t msg
SArray* tbUidList = NULL;
ret = qGetTableList(req.suid, pVnode, pHandle->execHandle.execTb.node, &tbUidList, pHandle->execHandle.task);
if(ret != TDB_CODE_SUCCESS) {
tqError("qGetTableList error:%d vgId:%d, subkey:%s consumer:0x%" PRIx64, ret, pVnode->config.vgId, req.subKey, pHandle->consumerId);
if (ret != TDB_CODE_SUCCESS) {
tqError("qGetTableList error:%d vgId:%d, subkey:%s consumer:0x%" PRIx64, ret, pVnode->config.vgId, req.subKey,
pHandle->consumerId);
taosArrayDestroy(tbUidList);
goto end;
}
tqDebug("tq try to get ctb for stb subscribe, vgId:%d, subkey:%s consumer:0x%" PRIx64 " suid:%" PRId64, pVnode->config.vgId, req.subKey, pHandle->consumerId, req.suid);
tqDebug("tq try to get ctb for stb subscribe, vgId:%d, subkey:%s consumer:0x%" PRIx64 " suid:%" PRId64,
pVnode->config.vgId, req.subKey, pHandle->consumerId, req.suid);
pHandle->execHandle.pTqReader = tqReaderOpen(pVnode);
tqReaderSetTbUidList(pHandle->execHandle.pTqReader, tbUidList);
tqReaderSetTbUidList(pHandle->execHandle.pTqReader, tbUidList, NULL);
taosArrayDestroy(tbUidList);
}
taosHashPut(pTq->pHandle, req.subKey, strlen(req.subKey), pHandle, sizeof(STqHandle));
tqDebug("try to persist handle %s consumer:0x%" PRIx64, req.subKey,
pHandle->consumerId);
tqDebug("try to persist handle %s consumer:0x%" PRIx64, req.subKey, pHandle->consumerId);
ret = tqMetaSaveHandle(pTq, req.subKey, pHandle);
goto end;
} else {
taosWLockLatch(&pTq->lock);
if (pHandle->consumerId == req.newConsumerId) { // do nothing
tqInfo("vgId:%d consumer:0x%" PRIx64 " remains, no switch occurs, should not reach here", req.vgId, req.newConsumerId);
tqInfo("vgId:%d consumer:0x%" PRIx64 " remains, no switch occurs, should not reach here", req.vgId,
req.newConsumerId);
} else {
tqInfo("vgId:%d switch consumer from Id:0x%" PRIx64 " to Id:0x%" PRIx64, req.vgId, pHandle->consumerId,
req.newConsumerId);
atomic_store_64(&pHandle->consumerId, req.newConsumerId);
}
// atomic_add_fetch_32(&pHandle->epoch, 1);
// atomic_add_fetch_32(&pHandle->epoch, 1);
// kill executing task
// if(tqIsHandleExec(pHandle)) {
// qTaskInfo_t pTaskInfo = pHandle->execHandle.task;
// if (pTaskInfo != NULL) {
// qKillTask(pTaskInfo, TSDB_CODE_SUCCESS);
// }
// if (pHandle->execHandle.subType == TOPIC_SUB_TYPE__COLUMN) {
// qStreamCloseTsdbReader(pTaskInfo);
// }
// }
// if(tqIsHandleExec(pHandle)) {
// qTaskInfo_t pTaskInfo = pHandle->execHandle.task;
// if (pTaskInfo != NULL) {
// qKillTask(pTaskInfo, TSDB_CODE_SUCCESS);
// }
// if (pHandle->execHandle.subType == TOPIC_SUB_TYPE__COLUMN) {
// qStreamCloseTsdbReader(pTaskInfo);
// }
// }
// remove if it has been register in the push manager, and return one empty block to consumer
tqUnregisterPushHandle(pTq, pHandle);
taosWUnLockLatch(&pTq->lock);
......@@ -773,13 +789,11 @@ int32_t tqProcessSubscribeReq(STQ* pTq, int64_t sversion, char* msg, int32_t msg
}
end:
taosMemoryFree(req.qmsg);
tDecoderClear(&dc);
return ret;
}
void freePtr(void *ptr) {
taosMemoryFree(*(void**)ptr);
}
void freePtr(void* ptr) { taosMemoryFree(*(void**)ptr); }
int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) {
int32_t vgId = TD_VID(pTq->pVnode);
......@@ -802,7 +816,7 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) {
pTask->chkInfo.currentVer = ver;
// expand executor
pTask->status.taskStatus = (pTask->fillHistory)? TASK_STATUS__WAIT_DOWNSTREAM:TASK_STATUS__NORMAL;
pTask->status.taskStatus = (pTask->fillHistory) ? TASK_STATUS__WAIT_DOWNSTREAM : TASK_STATUS__NORMAL;
if (pTask->taskLevel == TASK_LEVEL__SOURCE) {
pTask->pState = streamStateOpen(pTq->pStreamMeta->path, pTask, false, -1, -1);
......@@ -868,8 +882,8 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) {
streamSetupTrigger(pTask);
tqInfo("vgId:%d expand stream task, s-task:%s, checkpoint ver:%" PRId64 " child id:%d, level:%d", vgId, pTask->id.idStr,
pTask->chkInfo.version, pTask->selfChildId, pTask->taskLevel);
tqInfo("vgId:%d expand stream task, s-task:%s, checkpoint ver:%" PRId64 " child id:%d, level:%d", vgId,
pTask->id.idStr, pTask->chkInfo.version, pTask->selfChildId, pTask->taskLevel);
// next valid version will add one
pTask->chkInfo.version += 1;
......@@ -982,7 +996,8 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms
SStreamTask* pTask = taosMemoryCalloc(1, sizeof(SStreamTask));
if (pTask == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY;
tqError("vgId:%d failed to create stream task due to out of memory, alloc size:%d", vgId, (int32_t) sizeof(SStreamTask));
tqError("vgId:%d failed to create stream task due to out of memory, alloc size:%d", vgId,
(int32_t)sizeof(SStreamTask));
return -1;
}
......@@ -1097,7 +1112,7 @@ int32_t tqProcessTaskRecover2Req(STQ* pTq, int64_t sversion, char* msg, int32_t
// do recovery step 2
int64_t st = taosGetTimestampMs();
tqDebug("s-task:%s start step2 recover, ts:%"PRId64, pTask->id.idStr, st);
tqDebug("s-task:%s start step2 recover, ts:%" PRId64, pTask->id.idStr, st);
code = streamSourceRecoverScanStep2(pTask, sversion);
if (code < 0) {
......@@ -1105,7 +1120,8 @@ int32_t tqProcessTaskRecover2Req(STQ* pTq, int64_t sversion, char* msg, int32_t
return -1;
}
qDebug("s-task:%s set the start wal offset to be:%"PRId64, pTask->id.idStr, sversion);
qDebug("s-task:%s set start wal scan start ver:%"PRId64, pTask->id.idStr, sversion);
walReaderSeekVer(pTask->exec.pWalReader, sversion);
pTask->chkInfo.currentVer = sversion;
......@@ -1129,7 +1145,7 @@ int32_t tqProcessTaskRecover2Req(STQ* pTq, int64_t sversion, char* msg, int32_t
return -1;
}
double el = (taosGetTimestampMs() - st)/ 1000.0;
double el = (taosGetTimestampMs() - st) / 1000.0;
tqDebug("s-task:%s step2 recover finished, el:%.2fs", pTask->id.idStr, el);
// dispatch recover finish req to all related downstream task
......@@ -1245,8 +1261,8 @@ int32_t tqProcessTaskRunReq(STQ* pTq, SRpcMsg* pMsg) {
SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, taskId);
if (pTask != NULL) {
if (pTask->status.taskStatus == TASK_STATUS__NORMAL) {
tqDebug("vgId:%d s-task:%s start to process block from wal, last chk point:%" PRId64, vgId,
pTask->id.idStr, pTask->chkInfo.version);
tqDebug("vgId:%d s-task:%s start to process block from wal, last chk point:%" PRId64, vgId, pTask->id.idStr,
pTask->chkInfo.version);
streamProcessRunReq(pTask);
} else {
if (streamTaskShouldPause(&pTask->status)) {
......@@ -1265,9 +1281,9 @@ int32_t tqProcessTaskRunReq(STQ* pTq, SRpcMsg* pMsg) {
}
int32_t tqProcessTaskDispatchReq(STQ* pTq, SRpcMsg* pMsg, bool exec) {
char* msgStr = pMsg->pCont;
char* msgBody = POINTER_SHIFT(msgStr, sizeof(SMsgHead));
int32_t msgLen = pMsg->contLen - sizeof(SMsgHead);
char* msgStr = pMsg->pCont;
char* msgBody = POINTER_SHIFT(msgStr, sizeof(SMsgHead));
int32_t msgLen = pMsg->contLen - sizeof(SMsgHead);
SStreamDispatchReq req = {0};
......@@ -1313,7 +1329,7 @@ int32_t tqProcessTaskDropReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgL
int32_t tqProcessTaskPauseReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen) {
SVPauseStreamTaskReq* pReq = (SVPauseStreamTaskReq*)msg;
SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, pReq->taskId);
SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, pReq->taskId);
if (pTask) {
tqDebug("vgId:%d s-task:%s set pause flag", pTq->pStreamMeta->vgId, pTask->id.idStr);
atomic_store_8(&pTask->status.keepTaskStatus, pTask->status.taskStatus);
......
......@@ -362,7 +362,7 @@ int32_t tqMetaRestoreHandle(STQ* pTq) {
}
tqDebug("vgId:%d, tq try to get ctb for stb subscribe, suid:%" PRId64, pTq->pVnode->config.vgId, handle.execHandle.execTb.suid);
handle.execHandle.pTqReader = tqReaderOpen(pTq->pVnode);
tqReaderSetTbUidList(handle.execHandle.pTqReader, tbUidList);
tqReaderSetTbUidList(handle.execHandle.pTqReader, tbUidList, NULL);
taosArrayDestroy(tbUidList);
}
tqDebug("tq restore %s consumer %" PRId64 " vgId:%d", handle.subKey, handle.consumerId, vgId);
......
......@@ -394,8 +394,8 @@ bool tqNextBlockInWal(STqReader* pReader, const char* id) {
SSubmitTbData* pSubmitTbData = taosArrayGet(pReader->submit.aSubmitTbData, pReader->nextBlk);
if (pReader->tbIdHash == NULL) {
SSDataBlock* pRes = NULL;
int32_t code = tqRetrieveDataBlock(pReader, &pRes, NULL);
SSDataBlock* pRes = NULL;
int32_t code = tqRetrieveDataBlock(pReader, &pRes, NULL);
if (code == TSDB_CODE_SUCCESS && pRes->info.rows > 0) {
return true;
}
......@@ -457,7 +457,7 @@ bool tqNextBlockImpl(STqReader* pReader, const char* idstr) {
int32_t numOfBlocks = taosArrayGetSize(pReader->submit.aSubmitTbData);
while (pReader->nextBlk < numOfBlocks) {
tqDebug("tq reader next data block, len:%d ver:%" PRId64 " index:%d/%d, %s", pReader->msg.msgLen, pReader->msg.ver,
tqDebug("try next data block, len:%d ver:%" PRId64 " index:%d/%d, %s", pReader->msg.msgLen, pReader->msg.ver,
pReader->nextBlk, numOfBlocks, idstr);
SSubmitTbData* pSubmitTbData = taosArrayGet(pReader->submit.aSubmitTbData, pReader->nextBlk);
......@@ -467,10 +467,11 @@ bool tqNextBlockImpl(STqReader* pReader, const char* idstr) {
void* ret = taosHashGet(pReader->tbIdHash, &pSubmitTbData->uid, sizeof(int64_t));
if (ret != NULL) {
tqDebug("tq reader block found, ver:%" PRId64 ", uid:%" PRId64, pReader->msg.ver, pSubmitTbData->uid);
tqDebug("block found, ver:%" PRId64 ", uid:%" PRId64", %s", pReader->msg.ver, pSubmitTbData->uid, idstr);
return true;
} else {
tqDebug("tq reader discard submit block, uid:%" PRId64 ", continue", pSubmitTbData->uid);
tqDebug("discard submit block, uid:%" PRId64 ", total queried tables:%d continue %s", pSubmitTbData->uid,
taosHashGetSize(pReader->tbIdHash), idstr);
}
pReader->nextBlk++;
......@@ -604,7 +605,6 @@ static int32_t doSetVal(SColumnInfoData* pColumnInfoData, int32_t rowIndex, SCol
int32_t tqRetrieveDataBlock(STqReader* pReader, SSDataBlock** pRes, const char* id) {
tqDebug("tq reader retrieve data block %p, index:%d", pReader->msg.msgStr, pReader->nextBlk);
SSubmitTbData* pSubmitTbData = taosArrayGet(pReader->submit.aSubmitTbData, pReader->nextBlk++);
SSDataBlock* pBlock = pReader->pResBlock;
......@@ -612,6 +612,7 @@ int32_t tqRetrieveDataBlock(STqReader* pReader, SSDataBlock** pRes, const char*
blockDataCleanup(pBlock);
int32_t vgId = pReader->pWalReader->pWal->cfg.vgId;
int32_t sversion = pSubmitTbData->sver;
int64_t suid = pSubmitTbData->suid;
int64_t uid = pSubmitTbData->uid;
......@@ -628,7 +629,7 @@ int32_t tqRetrieveDataBlock(STqReader* pReader, SSDataBlock** pRes, const char*
if (pReader->pSchemaWrapper == NULL) {
tqWarn("vgId:%d, cannot found schema wrapper for table: suid:%" PRId64 ", uid:%" PRId64
"version %d, possibly dropped table",
pReader->pWalReader->pWal->cfg.vgId, suid, uid, pReader->cachedSchemaVer);
vgId, suid, uid, pReader->cachedSchemaVer);
pReader->cachedSchemaSuid = 0;
terrno = TSDB_CODE_TQ_TABLE_SCHEMA_NOT_FOUND;
return -1;
......@@ -642,6 +643,7 @@ int32_t tqRetrieveDataBlock(STqReader* pReader, SSDataBlock** pRes, const char*
if (blockDataGetNumOfCols(pBlock) == 0) {
int32_t code = buildResSDataBlock(pReader->pResBlock, pReader->pSchemaWrapper, pReader->pColIdList);
if (code != TSDB_CODE_SUCCESS) {
tqError("vgId:%d failed to build data block, code:%s", vgId, tstrerror(code));
return code;
}
}
......@@ -998,7 +1000,7 @@ FAIL:
void tqReaderSetColIdList(STqReader* pReader, SArray* pColIdList) { pReader->pColIdList = pColIdList; }
int tqReaderSetTbUidList(STqReader* pReader, const SArray* tbUidList) {
int tqReaderSetTbUidList(STqReader* pReader, const SArray* tbUidList, const char* id) {
if (pReader->tbIdHash) {
taosHashClear(pReader->tbIdHash);
} else {
......@@ -1015,6 +1017,7 @@ int tqReaderSetTbUidList(STqReader* pReader, const SArray* tbUidList) {
taosHashPut(pReader->tbIdHash, pKey, sizeof(int64_t), NULL, 0);
}
tqDebug("s-task:%s %d tables are set to be queried target table", id, (int32_t) taosArrayGetSize(tbUidList));
return 0;
}
......@@ -1089,7 +1092,7 @@ int32_t tqUpdateTbUidList(STQ* pTq, const SArray* tbUidList, bool isAdd) {
taosArrayDestroy(list);
return ret;
}
tqReaderSetTbUidList(pTqHandle->execHandle.pTqReader, list);
tqReaderSetTbUidList(pTqHandle->execHandle.pTqReader, list, NULL);
taosArrayDestroy(list);
} else {
tqReaderRemoveTbUidList(pTqHandle->execHandle.pTqReader, tbUidList);
......
......@@ -1454,7 +1454,7 @@ static int32_t getTableDelDataFromDelIdx(SDelFReader *pDelReader, SDelIdx *pDelI
int32_t code = 0;
if (pDelIdx) {
code = tsdbReadDelData(pDelReader, pDelIdx, aDelData);
code = tsdbReadDelDatav1(pDelReader, pDelIdx, aDelData, INT64_MAX);
}
return code;
......
......@@ -266,7 +266,7 @@ static int32_t tsdbCommitTableDel(SCommitter *pCommitter, STbData *pTbData, SDel
suid = pDelIdx->suid;
uid = pDelIdx->uid;
code = tsdbReadDelData(pCommitter->pDelFReader, pDelIdx, pCommitter->aDelData);
code = tsdbReadDelDatav1(pCommitter->pDelFReader, pDelIdx, pCommitter->aDelData, INT64_MAX);
TSDB_CHECK_CODE(code, lino, _exit);
} else {
taosArrayClear(pCommitter->aDelData);
......
......@@ -412,7 +412,7 @@ static int32_t tsdbTombFileDataIterNext(STsdbDataIter2* pIter, STsdbFilterInfo*
}
}
code = tsdbReadDelData(pIter->tIter.pReader, pDelIdx, pIter->tIter.aDelData);
code = tsdbReadDelDatav1(pIter->tIter.pReader, pDelIdx, pIter->tIter.aDelData, INT64_MAX);
TSDB_CHECK_CODE(code, lino, _exit);
pIter->delInfo.suid = pDelIdx->suid;
......
......@@ -2967,7 +2967,7 @@ int32_t initDelSkylineIterator(STableBlockScanInfo* pBlockScanInfo, STsdbReader*
SDelIdx* pIdx = taosArraySearch(pReader->pDelIdx, &idx, tCmprDelIdx, TD_EQ);
if (pIdx != NULL) {
code = tsdbReadDelData(pReader->pDelFReader, pIdx, pDelData);
code = tsdbReadDelDatav1(pReader->pDelFReader, pIdx, pDelData, pReader->verRange.maxVer);
}
if (code != TSDB_CODE_SUCCESS) {
goto _err;
......@@ -2978,7 +2978,10 @@ int32_t initDelSkylineIterator(STableBlockScanInfo* pBlockScanInfo, STsdbReader*
if (pMemTbData != NULL) {
p = pMemTbData->pHead;
while (p) {
taosArrayPush(pDelData, p);
if (p->version <= pReader->verRange.maxVer) {
taosArrayPush(pDelData, p);
}
p = p->pNext;
}
}
......@@ -2986,7 +2989,9 @@ int32_t initDelSkylineIterator(STableBlockScanInfo* pBlockScanInfo, STsdbReader*
if (piMemTbData != NULL) {
p = piMemTbData->pHead;
while (p) {
taosArrayPush(pDelData, p);
if (p->version <= pReader->verRange.maxVer) {
taosArrayPush(pDelData, p);
}
p = p->pNext;
}
}
......@@ -4558,7 +4563,11 @@ int32_t tsdbReaderOpen(void* pVnode, SQueryTableDataCond* pCond, void* pTableLis
pReader->pIgnoreTables = pIgnoreTables;
tsdbDebug("%p total numOfTable:%d in this query %s", pReader, numOfTables, pReader->idStr);
tsdbDebug("%p total numOfTable:%d, window:%" PRId64 " - %" PRId64 ", verRange:%" PRId64 " - %" PRId64
" in this query %s",
pReader, numOfTables, pReader->window.skey, pReader->window.ekey, pReader->verRange.minVer,
pReader->verRange.maxVer, pReader->idStr);
return code;
_err:
......
......@@ -1489,6 +1489,10 @@ int32_t tsdbDelFReaderClose(SDelFReader **ppReader) {
}
int32_t tsdbReadDelData(SDelFReader *pReader, SDelIdx *pDelIdx, SArray *aDelData) {
return tsdbReadDelDatav1(pReader, pDelIdx, aDelData, INT64_MAX);
}
int32_t tsdbReadDelDatav1(SDelFReader *pReader, SDelIdx *pDelIdx, SArray *aDelData, int64_t maxVer) {
int32_t code = 0;
int64_t offset = pDelIdx->offset;
int64_t size = pDelIdx->size;
......@@ -1510,11 +1514,15 @@ int32_t tsdbReadDelData(SDelFReader *pReader, SDelIdx *pDelIdx, SArray *aDelData
SDelData delData;
n += tGetDelData(pReader->aBuf[0] + n, &delData);
if (taosArrayPush(aDelData, &delData) == NULL) {
code = TSDB_CODE_OUT_OF_MEMORY;
goto _err;
if (delData.version > maxVer) {
continue;
}
if (taosArrayPush(aDelData, &delData) == NULL) {
code = TSDB_CODE_OUT_OF_MEMORY;
goto _err;
}
}
ASSERT(n == size);
return code;
......
......@@ -1166,7 +1166,7 @@ static int32_t tsdbSnapWriteDelTableDataStart(STsdbSnapWriter* pWriter, TABLEID*
int32_t c = tTABLEIDCmprFn(pDelIdx, &pWriter->tbid);
if (c < 0) {
code = tsdbReadDelData(pWriter->pDelFReader, pDelIdx, pWriter->pTIter->tIter.aDelData);
code = tsdbReadDelDatav1(pWriter->pDelFReader, pDelIdx, pWriter->pTIter->tIter.aDelData, INT64_MAX);
TSDB_CHECK_CODE(code, lino, _exit);
SDelIdx* pDelIdxNew = taosArrayReserve(pWriter->aDelIdx, 1);
......@@ -1183,7 +1183,7 @@ static int32_t tsdbSnapWriteDelTableDataStart(STsdbSnapWriter* pWriter, TABLEID*
pWriter->pTIter->tIter.iDelIdx++;
} else if (c == 0) {
code = tsdbReadDelData(pWriter->pDelFReader, pDelIdx, pWriter->aDelData);
code = tsdbReadDelDatav1(pWriter->pDelFReader, pDelIdx, pWriter->aDelData, INT64_MAX);
TSDB_CHECK_CODE(code, lino, _exit);
pWriter->pTIter->tIter.iDelIdx++;
......
......@@ -456,6 +456,7 @@ typedef struct SStreamIntervalOperatorInfo {
SSHashObj* pUpdatedMap;
int64_t dataVersion;
SStateStore statestore;
bool recvGetAll;
} SStreamIntervalOperatorInfo;
typedef struct SDataGroupInfo {
......
......@@ -406,7 +406,7 @@ int32_t qUpdateTableListForStreamScanner(qTaskInfo_t tinfo, const SArray* tableI
int32_t code = 0;
if (isAdd) {
qDebug("add %d tables id into query list, %s", (int32_t)taosArrayGetSize(tableIdList), id);
qDebug("try to add %d tables id into query list, %s", (int32_t)taosArrayGetSize(tableIdList), id);
}
// traverse to the stream scanner node to add this table id
......
......@@ -13,6 +13,8 @@
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
// clang-format off
#include "executorInt.h"
#include "filter.h"
#include "function.h"
......@@ -1872,6 +1874,7 @@ static SSDataBlock* doStreamScan(SOperatorInfo* pOperator) {
printDataBlock(pInfo->pCreateTbRes, "recover createTbl");
return pInfo->pCreateTbRes;
}
qDebug("stream recover scan get block, rows %" PRId64, pInfo->pRecoverRes->info.rows);
printDataBlock(pInfo->pRecoverRes, "scan recover");
return pInfo->pRecoverRes;
......@@ -1980,7 +1983,7 @@ FETCH_NEXT_BLOCK:
// printDataBlock(pBlock, "stream scan recv");
return pBlock;
} else if (pInfo->blockType == STREAM_INPUT__DATA_SUBMIT) {
qDebug("scan mode %d", pInfo->scanMode);
qDebug("stream scan mode:%d, %s", pInfo->scanMode, id);
switch (pInfo->scanMode) {
case STREAM_SCAN_FROM_RES: {
pInfo->scanMode = STREAM_SCAN_FROM_READERHANDLE;
......@@ -2064,8 +2067,13 @@ FETCH_NEXT_BLOCK:
while (pAPI->tqReaderFn.tqNextBlockImpl(pInfo->tqReader, id)) {
SSDataBlock* pRes = NULL;
int32_t code = pAPI->tqReaderFn.tqRetrieveBlock(pInfo->tqReader, &pRes, id);
qDebug("retrieve data from submit completed code:%s, rows:%" PRId64 " %s", tstrerror(code), pRes->info.rows,
id);
if (code != TSDB_CODE_SUCCESS || pRes->info.rows == 0) {
qDebug("retrieve data failed, try next block in submit block, %s", id);
continue;
}
......@@ -2073,6 +2081,7 @@ FETCH_NEXT_BLOCK:
if (pInfo->pCreateTbRes->info.rows > 0) {
pInfo->scanMode = STREAM_SCAN_FROM_RES;
qDebug("create table res exists, rows:%"PRId64" return from stream scan, %s", pInfo->pCreateTbRes->info.rows, id);
return pInfo->pCreateTbRes;
}
......@@ -2081,6 +2090,8 @@ FETCH_NEXT_BLOCK:
pBlock->info.dataLoad = 1;
blockDataUpdateTsWindow(pBlock, pInfo->primaryTsIndex);
qDebug("%" PRId64 " rows in datablock, update res:%" PRId64 " %s", pBlockInfo->rows,
pInfo->pUpdateDataRes->info.rows, id);
if (pBlockInfo->rows > 0 || pInfo->pUpdateDataRes->info.rows > 0) {
break;
}
......@@ -2097,7 +2108,7 @@ FETCH_NEXT_BLOCK:
pInfo->numOfExec++;
pOperator->resultInfo.totalRows += pBlockInfo->rows;
qDebug("stream scan get source rows:%" PRId64", %s", pBlockInfo->rows, id);
qDebug("stream scan completed, and return source rows:%" PRId64", %s", pBlockInfo->rows, id);
if (pBlockInfo->rows > 0) {
return pBlock;
}
......@@ -2283,7 +2294,8 @@ SOperatorInfo* createStreamScanOperatorInfo(SReadHandle* pHandle, STableScanPhys
SArray* pColIds = NULL;
SStreamScanInfo* pInfo = taosMemoryCalloc(1, sizeof(SStreamScanInfo));
SOperatorInfo* pOperator = taosMemoryCalloc(1, sizeof(SOperatorInfo));
SStorageAPI* pAPI = &pTaskInfo->storageAPI;
SStorageAPI* pAPI = &pTaskInfo->storageAPI;
const char* idstr = pTaskInfo->id.str;
if (pInfo == NULL || pOperator == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY;
......@@ -2394,7 +2406,7 @@ SOperatorInfo* createStreamScanOperatorInfo(SReadHandle* pHandle, STableScanPhys
// set the extract column id to streamHandle
pAPI->tqReaderFn.tqReaderSetColIdList(pInfo->tqReader, pColIds);
SArray* tableIdList = extractTableIdList(((STableScanInfo*)(pInfo->pTableScanOp->info))->base.pTableListInfo);
code = pAPI->tqReaderFn.tqReaderSetQueryTableList(pInfo->tqReader, tableIdList);
code = pAPI->tqReaderFn.tqReaderSetQueryTableList(pInfo->tqReader, tableIdList, idstr);
if (code != 0) {
taosArrayDestroy(tableIdList);
goto _error;
......@@ -2444,6 +2456,7 @@ SOperatorInfo* createStreamScanOperatorInfo(SReadHandle* pHandle, STableScanPhys
int32_t len = 0;
pAPI->stateStore.streamStateGetInfo(pTaskInfo->streamInfo.pState, STREAM_SCAN_OP_NAME, strlen(STREAM_SCAN_OP_NAME), &buff, &len);
streamScanOperatorDecode(buff, len, pInfo);
taosMemoryFree(buff);
}
setOperatorInfo(pOperator, STREAM_SCAN_OP_NAME, QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN, false, OP_NOT_OPENED, pInfo,
......@@ -3458,3 +3471,5 @@ static void destoryTableCountScanOperator(void* param) {
taosArrayDestroy(pTableCountScanInfo->stbUidList);
taosMemoryFreeClear(param);
}
// clang-format on
......@@ -2439,6 +2439,15 @@ static inline int winPosCmprImpl(const void* pKey1, const void* pKey2) {
return 0;
}
static void resetUnCloseWinInfo(SSHashObj* winMap) {
void* pIte = NULL;
int32_t iter = 0;
while ((pIte = tSimpleHashIterate(winMap, pIte, &iter)) != NULL) {
SRowBuffPos* pPos = *(SRowBuffPos**)pIte;
pPos->beUsed = true;
}
}
static SSDataBlock* doStreamFinalIntervalAgg(SOperatorInfo* pOperator) {
SStreamIntervalOperatorInfo* pInfo = pOperator->info;
SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo;
......@@ -2472,6 +2481,11 @@ static SSDataBlock* doStreamFinalIntervalAgg(SOperatorInfo* pOperator) {
return pInfo->binfo.pRes;
}
if (pInfo->recvGetAll) {
pInfo->recvGetAll = false;
resetUnCloseWinInfo(pInfo->aggSup.pResultRowHashTable);
}
setOperatorCompleted(pOperator);
if (!IS_FINAL_OP(pInfo)) {
clearFunctionContext(&pOperator->exprSupp);
......@@ -2565,6 +2579,7 @@ static SSDataBlock* doStreamFinalIntervalAgg(SOperatorInfo* pOperator) {
break;
} else if (pBlock->info.type == STREAM_GET_ALL && IS_FINAL_OP(pInfo)) {
pInfo->recvGetAll = true;
getAllIntervalWindow(pInfo->aggSup.pResultRowHashTable, pInfo->pUpdatedMap);
continue;
} else if (pBlock->info.type == STREAM_RETRIEVE && !IS_FINAL_OP(pInfo)) {
......@@ -2773,6 +2788,7 @@ SOperatorInfo* createStreamFinalIntervalOperatorInfo(SOperatorInfo* downstream,
compareTs, pInfo->pState, pInfo->twAggSup.deleteMark);
pInfo->dataVersion = 0;
pInfo->statestore = pTaskInfo->storageAPI.stateStore;
pInfo->recvGetAll = false;
pOperator->operatorType = pPhyNode->type;
pOperator->blocking = true;
......@@ -4751,6 +4767,12 @@ static SSDataBlock* doStreamIntervalAgg(SOperatorInfo* pOperator) {
printDataBlock(pInfo->binfo.pRes, "single interval");
return pInfo->binfo.pRes;
}
if (pInfo->recvGetAll) {
pInfo->recvGetAll = false;
resetUnCloseWinInfo(pInfo->aggSup.pResultRowHashTable);
}
setOperatorCompleted(pOperator);
if (pInfo->twAggSup.maxTs > 0 &&
pInfo->twAggSup.maxTs - pInfo->twAggSup.checkPointInterval > pInfo->twAggSup.checkPointTs) {
......@@ -4790,6 +4812,7 @@ static SSDataBlock* doStreamIntervalAgg(SOperatorInfo* pOperator) {
continue;
} else if (pBlock->info.type == STREAM_GET_ALL) {
qDebug("===stream===single interval recv|block type STREAM_GET_ALL");
pInfo->recvGetAll = true;
getAllIntervalWindow(pInfo->aggSup.pResultRowHashTable, pInfo->pUpdatedMap);
continue;
} else if (pBlock->info.type == STREAM_CREATE_CHILD_TABLE) {
......@@ -4960,6 +4983,8 @@ SOperatorInfo* createStreamIntervalOperatorInfo(SOperatorInfo* downstream, SPhys
destroyStreamFinalIntervalOperatorInfo, optrDefaultBufFn, NULL);
pInfo->statestore = pTaskInfo->storageAPI.stateStore;
pInfo->recvGetAll = false;
initIntervalDownStream(downstream, pPhyNode->type, pInfo);
code = appendDownstream(pOperator, &downstream, 1);
if (code != TSDB_CODE_SUCCESS) {
......
......@@ -296,11 +296,8 @@ int32_t tAppendDataToInputQueue(SStreamTask* pTask, SStreamQueueItem* pItem) {
if (type == STREAM_INPUT__DATA_SUBMIT) {
SStreamDataSubmit* px = (SStreamDataSubmit*)pItem;
qDebug("s-task:%s submit enqueue msgLen:%d ver:%" PRId64 ", total in queue:%d, size:%.2fMiB", pTask->id.idStr,
px->submit.msgLen, px->submit.ver, total, size);
if ((pTask->taskLevel == TASK_LEVEL__SOURCE) && tInputQueueIsFull(pTask)) {
qError("s-task:%s input queue is full, capacity(size:%d num:%dMiB), current(blocks:%d, size:%.2fMiB) abort",
qError("s-task:%s input queue is full, capacity(size:%d num:%dMiB), current(blocks:%d, size:%.2fMiB) stop to push data",
pTask->id.idStr, STREAM_TASK_INPUT_QUEUEU_CAPACITY, STREAM_TASK_INPUT_QUEUEU_CAPACITY_IN_SIZE, total,
size);
streamDataSubmitDestroy(px);
......@@ -314,9 +311,12 @@ int32_t tAppendDataToInputQueue(SStreamTask* pTask, SStreamQueueItem* pItem) {
taosFreeQitem(pItem);
return code;
}
qDebug("s-task:%s submit enqueue msgLen:%d ver:%" PRId64 ", total in queue:%d, size:%.2fMiB", pTask->id.idStr,
px->submit.msgLen, px->submit.ver, total, size + px->submit.msgLen/1048576.0);
} else if (type == STREAM_INPUT__DATA_BLOCK || type == STREAM_INPUT__DATA_RETRIEVE ||
type == STREAM_INPUT__REF_DATA_BLOCK) {
if (/*(pTask->taskLevel == TASK_LEVEL__SOURCE) && */(tInputQueueIsFull(pTask))) {
if ((pTask->taskLevel == TASK_LEVEL__SOURCE) && (tInputQueueIsFull(pTask))) {
qError("s-task:%s input queue is full, capacity:%d size:%d MiB, current(blocks:%d, size:%.2fMiB) abort",
pTask->id.idStr, STREAM_TASK_INPUT_QUEUEU_CAPACITY, STREAM_TASK_INPUT_QUEUEU_CAPACITY_IN_SIZE, total,
size);
......
......@@ -87,8 +87,6 @@ void* streamBackendInit(const char* path) {
pHandle->cfInst = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), false, HASH_NO_LOCK);
rocksdb_env_t* env = rocksdb_create_default_env(); // rocksdb_envoptions_create();
rocksdb_env_set_low_priority_background_threads(env, 4);
rocksdb_env_set_high_priority_background_threads(env, 2);
rocksdb_cache_t* cache = rocksdb_cache_create_lru(64 << 20);
......@@ -574,9 +572,14 @@ int32_t decodeValueFunc(void* value, int32_t vlen, int64_t* ttl, char** dest) {
*dest = NULL;
return -1;
}
int64_t now = taosGetTimestampMs();
p = taosDecodeFixedI64(p, &key.unixTimestamp);
p = taosDecodeFixedI32(p, &key.len);
if (vlen != (sizeof(int64_t) + sizeof(int32_t) + key.len)) {
if (dest != NULL) *dest = NULL;
qError("vlen: %d, read len: %d", vlen, key.len);
return -1;
}
if (key.len == 0) {
key.data = NULL;
} else {
......@@ -584,6 +587,7 @@ int32_t decodeValueFunc(void* value, int32_t vlen, int64_t* ttl, char** dest) {
}
if (ttl != NULL) {
int64_t now = taosGetTimestampMs();
*ttl = key.unixTimestamp == 0 ? 0 : key.unixTimestamp - now;
}
if (dest != NULL) {
......@@ -1005,35 +1009,35 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfNa
((rocksdb_column_family_handle_t**)pState->pTdbState->pHandle)[idx]);
}
#define STREAM_STATE_PUT_ROCKSDB(pState, funcname, key, value, vLen) \
do { \
code = 0; \
char buf[128] = {0}; \
char* err = NULL; \
int i = streamGetInit(pState, funcname); \
if (i < 0) { \
qWarn("streamState failed to get cf name: %s", funcname); \
code = -1; \
break; \
} \
char toString[128] = {0}; \
if (qDebugFlag & DEBUG_TRACE) ginitDict[i].toStrFunc((void*)key, toString); \
int32_t klen = ginitDict[i].enFunc((void*)key, buf); \
rocksdb_column_family_handle_t* pHandle = \
((rocksdb_column_family_handle_t**)pState->pTdbState->pHandle)[ginitDict[i].idx]; \
rocksdb_t* db = pState->pTdbState->rocksdb; \
rocksdb_writeoptions_t* opts = pState->pTdbState->writeOpts; \
char* ttlV = NULL; \
int32_t ttlVLen = ginitDict[i].enValueFunc((char*)value, vLen, 0, &ttlV); \
rocksdb_put_cf(db, opts, pHandle, (const char*)buf, klen, (const char*)ttlV, (size_t)ttlVLen, &err); \
if (err != NULL) { \
taosMemoryFree(err); \
qError("streamState str: %s failed to write to %s, err: %s", toString, funcname, err); \
code = -1; \
} else { \
qTrace("streamState str:%s succ to write to %s, valLen:%d", toString, funcname, vLen); \
} \
taosMemoryFree(ttlV); \
#define STREAM_STATE_PUT_ROCKSDB(pState, funcname, key, value, vLen) \
do { \
code = 0; \
char buf[128] = {0}; \
char* err = NULL; \
int i = streamGetInit(pState, funcname); \
if (i < 0) { \
qWarn("streamState failed to get cf name: %s", funcname); \
code = -1; \
break; \
} \
char toString[128] = {0}; \
if (qDebugFlag & DEBUG_TRACE) ginitDict[i].toStrFunc((void*)key, toString); \
int32_t klen = ginitDict[i].enFunc((void*)key, buf); \
rocksdb_column_family_handle_t* pHandle = \
((rocksdb_column_family_handle_t**)pState->pTdbState->pHandle)[ginitDict[i].idx]; \
rocksdb_t* db = pState->pTdbState->rocksdb; \
rocksdb_writeoptions_t* opts = pState->pTdbState->writeOpts; \
char* ttlV = NULL; \
int32_t ttlVLen = ginitDict[i].enValueFunc((char*)value, vLen, 0, &ttlV); \
rocksdb_put_cf(db, opts, pHandle, (const char*)buf, klen, (const char*)ttlV, (size_t)ttlVLen, &err); \
if (err != NULL) { \
taosMemoryFree(err); \
qError("streamState str: %s failed to write to %s, err: %s", toString, funcname, err); \
code = -1; \
} else { \
qTrace("streamState str:%s succ to write to %s, rowValLen:%d, ttlValLen:%d", toString, funcname, vLen, ttlVLen); \
} \
taosMemoryFree(ttlV); \
} while (0);
#define STREAM_STATE_GET_ROCKSDB(pState, funcname, key, pVal, vLen) \
......@@ -1056,7 +1060,7 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfNa
rocksdb_readoptions_t* opts = pState->pTdbState->readOpts; \
size_t len = 0; \
char* val = rocksdb_get_cf(db, opts, pHandle, (const char*)buf, klen, (size_t*)&len, &err); \
if (val == NULL) { \
if (val == NULL || len == 0) { \
if (err == NULL) { \
qTrace("streamState str: %s failed to read from %s_%s, err: not exist", toString, pState->pTdbState->idstr, \
funcname); \
......@@ -1068,17 +1072,17 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfNa
code = -1; \
} else { \
char* p = NULL; \
int32_t len = ginitDict[i].deValueFunc(val, len, NULL, (char**)pVal); \
if (len < 0) { \
int32_t tlen = ginitDict[i].deValueFunc(val, len, NULL, (char**)pVal); \
if (tlen <= 0) { \
qError("streamState str: %s failed to read from %s_%s, err: already ttl ", toString, pState->pTdbState->idstr, \
funcname); \
code = -1; \
} else { \
qTrace("streamState str: %s succ to read from %s_%s, valLen:%d", toString, pState->pTdbState->idstr, funcname, \
len); \
tlen); \
} \
taosMemoryFree(val); \
if (vLen != NULL) *vLen = len; \
if (vLen != NULL) *vLen = tlen; \
} \
if (code == 0) \
qDebug("streamState str: %s succ to read from %s_%s", toString, pState->pTdbState->idstr, funcname); \
......@@ -1924,17 +1928,17 @@ int32_t streamStateGetParName_rocksdb(SStreamState* pState, int64_t groupId, voi
int32_t streamDefaultPut_rocksdb(SStreamState* pState, const void* key, void* pVal, int32_t pVLen) {
int code = 0;
STREAM_STATE_PUT_ROCKSDB(pState, "default", &key, pVal, pVLen);
STREAM_STATE_PUT_ROCKSDB(pState, "default", key, pVal, pVLen);
return code;
}
int32_t streamDefaultGet_rocksdb(SStreamState* pState, const void* key, void** pVal, int32_t* pVLen) {
int code = 0;
STREAM_STATE_GET_ROCKSDB(pState, "default", &key, pVal, pVLen);
STREAM_STATE_GET_ROCKSDB(pState, "default", key, pVal, pVLen);
return code;
}
int32_t streamDefaultDel_rocksdb(SStreamState* pState, const void* key) {
int code = 0;
STREAM_STATE_DEL_ROCKSDB(pState, "default", &key);
STREAM_STATE_DEL_ROCKSDB(pState, "default", key);
return code;
}
......
......@@ -283,7 +283,7 @@ int32_t streamDispatchOneRecoverFinishReq(SStreamTask* pTask, const SStreamRecov
msg.info.noResp = 1;
tmsgSendReq(pEpSet, &msg);
qDebug("s-task:%s dispatch recover finish msg to taskId:%d node %d: recover finish msg", pTask->id.idStr,
qDebug("s-task:%s dispatch recover finish msg to downstream taskId:0x%x node %d: recover finish msg", pTask->id.idStr,
pReq->taskId, vgId);
return 0;
......@@ -318,7 +318,7 @@ int32_t doSendDispatchMsg(SStreamTask* pTask, const SStreamDispatchReq* pReq, in
msg.pCont = buf;
msg.msgType = pTask->dispatchMsgType;
qDebug("dispatch from s-task:%s to taskId:0x%x vgId:%d data msg", pTask->id.idStr, pReq->taskId, vgId);
qDebug("s-task:%s dispatch msg to taskId:0x%x vgId:%d data msg", pTask->id.idStr, pReq->taskId, vgId);
tmsgSendReq(pEpSet, &msg);
code = 0;
......@@ -414,7 +414,7 @@ int32_t streamDispatchAllBlocks(SStreamTask* pTask, const SStreamDataBlock* pDat
req.taskId = downstreamTaskId;
qDebug("s-task:%s (child taskId:%d) fix-dispatch blocks:%d to down stream s-task:%d in vgId:%d", pTask->id.idStr,
qDebug("s-task:%s (child taskId:%d) fix-dispatch %d block(s) to down stream s-task:0x%x in vgId:%d", pTask->id.idStr,
pTask->selfChildId, numOfBlocks, downstreamTaskId, vgId);
code = doSendDispatchMsg(pTask, &req, vgId, pEpSet);
......@@ -514,7 +514,7 @@ int32_t streamDispatchStreamBlock(SStreamTask* pTask) {
return 0;
}
qDebug("s-task:%s start to dispatch msg, output status:%d", pTask->id.idStr, pTask->outputStatus);
qDebug("s-task:%s start to dispatch msg, set output status:%d", pTask->id.idStr, pTask->outputStatus);
SStreamDataBlock* pDispatchedBlock = streamQueueNextItem(pTask->outputQueue);
if (pDispatchedBlock == NULL) {
......
......@@ -17,10 +17,10 @@
// maximum allowed processed block batches. One block may include several submit blocks
#define MAX_STREAM_EXEC_BATCH_NUM 32
#define MIN_STREAM_EXEC_BATCH_NUM 8
#define MIN_STREAM_EXEC_BATCH_NUM 4
#define MAX_STREAM_RESULT_DUMP_THRESHOLD 100
static int32_t updateCheckPointInfo (SStreamTask* pTask);
static int32_t updateCheckPointInfo(SStreamTask* pTask);
bool streamTaskShouldStop(const SStreamStatus* pStatus) {
int32_t status = atomic_load_8((int8_t*)&pStatus->taskStatus);
......@@ -44,14 +44,16 @@ static int32_t doDumpResult(SStreamTask* pTask, SStreamQueueItem* pItem, SArray*
if (numOfBlocks > 0) {
SStreamDataBlock* pStreamBlocks = createStreamBlockFromResults(pItem, pTask, size, pRes);
if (pStreamBlocks == NULL) {
qError("s-task:%s failed to create result stream data block, code:%s", pTask->id.idStr, tstrerror(terrno));
taosArrayDestroyEx(pRes, (FDelete)blockDataFreeRes);
return -1;
}
qDebug("s-task:%s dump stream result data blocks, num:%d, size:%.2fMiB", pTask->id.idStr, numOfBlocks, size/1048576.0);
qDebug("s-task:%s dump stream result data blocks, num:%d, size:%.2fMiB", pTask->id.idStr, numOfBlocks,
size / 1048576.0);
code = streamTaskOutputResultBlock(pTask, pStreamBlocks);
if (code == TSDB_CODE_UTIL_QUEUE_OUT_OF_MEMORY) { // back pressure and record position
if (code == TSDB_CODE_UTIL_QUEUE_OUT_OF_MEMORY) { // back pressure and record position
destroyStreamDataBlock(pStreamBlocks);
return -1;
}
......@@ -65,7 +67,8 @@ static int32_t doDumpResult(SStreamTask* pTask, SStreamQueueItem* pItem, SArray*
return TSDB_CODE_SUCCESS;
}
static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, int64_t* totalSize, int32_t* totalBlocks) {
static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, int64_t* totalSize,
int32_t* totalBlocks) {
int32_t code = TSDB_CODE_SUCCESS;
void* pExecutor = pTask->exec.pExecutor;
......@@ -82,7 +85,7 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, i
}
if (streamTaskShouldStop(&pTask->status)) {
taosArrayDestroy(pRes); // memory leak
taosArrayDestroyEx(pRes, (FDelete)blockDataFreeRes);
return 0;
}
......@@ -99,9 +102,8 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, i
if (output == NULL) {
if (pItem->type == STREAM_INPUT__DATA_RETRIEVE) {
SSDataBlock block = {0};
const SStreamDataBlock* pRetrieveBlock = (const SStreamDataBlock*) pItem;
SSDataBlock block = {0};
const SStreamDataBlock* pRetrieveBlock = (const SStreamDataBlock*)pItem;
ASSERT(taosArrayGetSize(pRetrieveBlock->blocks) == 1);
assignOneDataBlock(&block, taosArrayGet(pRetrieveBlock->blocks, 0));
......@@ -132,7 +134,7 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, i
taosArrayPush(pRes, &block);
qDebug("s-task:%s (child %d) executed and get block, total blocks:%d, size:%.2fMiB", pTask->id.idStr,
qDebug("s-task:%s (child %d) executed and get %d result blocks, size:%.2fMiB", pTask->id.idStr,
pTask->selfChildId, numOfBlocks, size / 1048576.0);
// current output should be dispatched to down stream nodes
......@@ -153,7 +155,7 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, i
ASSERT(numOfBlocks == taosArrayGetSize(pRes));
code = doDumpResult(pTask, pItem, pRes, size, totalSize, totalBlocks);
} else {
taosArrayDestroy(pRes);
taosArrayDestroyEx(pRes, (FDelete)blockDataFreeRes);
}
return code;
......@@ -235,11 +237,11 @@ int32_t streamScanExec(SStreamTask* pTask, int32_t batchSz) {
taosFreeQitem(qRes);
return code;
}
if (pTask->outputType == TASK_OUTPUT__FIXED_DISPATCH || pTask->outputType == TASK_OUTPUT__SHUFFLE_DISPATCH) {
qDebug("s-task:%s scan exec dispatch blocks:%d", pTask->id.idStr, batchCnt);
streamDispatchStreamBlock(pTask);
}
//
// if (pTask->outputType == TASK_OUTPUT__FIXED_DISPATCH || pTask->outputType == TASK_OUTPUT__SHUFFLE_DISPATCH) {
// qDebug("s-task:%s scan exec dispatch blocks:%d", pTask->id.idStr, batchCnt);
// streamDispatchStreamBlock(pTask);
// }
if (finished) {
break;
......@@ -286,7 +288,7 @@ int32_t streamBatchExec(SStreamTask* pTask, int32_t batchLimit) {
}
#endif
int32_t updateCheckPointInfo (SStreamTask* pTask) {
int32_t updateCheckPointInfo(SStreamTask* pTask) {
int64_t ckId = 0;
int64_t dataVer = 0;
qGetCheckpointVersion(pTask->exec.pExecutor, &dataVer, &ckId);
......@@ -294,7 +296,8 @@ int32_t updateCheckPointInfo (SStreamTask* pTask) {
SCheckpointInfo* pCkInfo = &pTask->chkInfo;
if (ckId > pCkInfo->id) { // save it since the checkpoint is updated
qDebug("s-task:%s exec end, start to update check point, ver from %" PRId64 " to %" PRId64
", checkPoint id:%" PRId64 " -> %" PRId64, pTask->id.idStr, pCkInfo->version, dataVer, pCkInfo->id, ckId);
", checkPoint id:%" PRId64 " -> %" PRId64,
pTask->id.idStr, pCkInfo->version, dataVer, pCkInfo->id, ckId);
pTask->chkInfo = (SCheckpointInfo){.version = dataVer, .id = ckId, .currentVer = pCkInfo->currentVer};
......@@ -314,8 +317,13 @@ int32_t updateCheckPointInfo (SStreamTask* pTask) {
return TSDB_CODE_SUCCESS;
}
/**
* todo: the batch of blocks should be tuned dynamic, according to the total elapsed time of each batch of blocks, the
* appropriate batch of blocks should be handled in 5 to 10 sec.
*/
int32_t streamExecForAll(SStreamTask* pTask) {
int32_t code = 0;
const char* id = pTask->id.idStr;
while (1) {
int32_t batchSize = 1;
int16_t times = 0;
......@@ -323,7 +331,7 @@ int32_t streamExecForAll(SStreamTask* pTask) {
SStreamQueueItem* pInput = NULL;
// merge multiple input data if possible in the input queue.
qDebug("s-task:%s start to extract data block from inputQ", pTask->id.idStr);
qDebug("s-task:%s start to extract data block from inputQ", id);
while (1) {
if (streamTaskShouldPause(&pTask->status)) {
......@@ -338,7 +346,7 @@ int32_t streamExecForAll(SStreamTask* pTask) {
if (qItem == NULL) {
if (pTask->taskLevel == TASK_LEVEL__SOURCE && batchSize < MIN_STREAM_EXEC_BATCH_NUM && times < 5) {
times++;
taosMsleep(1);
taosMsleep(10);
qDebug("===stream===try again batchSize:%d", batchSize);
continue;
}
......@@ -363,8 +371,10 @@ int32_t streamExecForAll(SStreamTask* pTask) {
batchSize++;
pInput = newRet;
streamQueueProcessSuccess(pTask->inputQueue);
if (batchSize > MAX_STREAM_EXEC_BATCH_NUM) {
qDebug("maximum batch limit:%d reached, processing, %s", MAX_STREAM_EXEC_BATCH_NUM, pTask->id.idStr);
qDebug("s-task:%s batch size limit:%d reached, start to process blocks", id,
MAX_STREAM_EXEC_BATCH_NUM);
break;
}
}
......@@ -375,7 +385,6 @@ int32_t streamExecForAll(SStreamTask* pTask) {
if (pInput) {
streamFreeQitem(pInput);
}
return 0;
}
......@@ -385,7 +394,7 @@ int32_t streamExecForAll(SStreamTask* pTask) {
if (pTask->taskLevel == TASK_LEVEL__SINK) {
ASSERT(pInput->type == STREAM_INPUT__DATA_BLOCK);
qDebug("s-task:%s sink task start to sink %d blocks", pTask->id.idStr, batchSize);
qDebug("s-task:%s sink task start to sink %d blocks", id, batchSize);
streamTaskOutputResultBlock(pTask, (SStreamDataBlock*)pInput);
continue;
}
......@@ -394,16 +403,16 @@ int32_t streamExecForAll(SStreamTask* pTask) {
while (pTask->taskLevel == TASK_LEVEL__SOURCE) {
int8_t status = atomic_load_8(&pTask->status.taskStatus);
if (status != TASK_STATUS__NORMAL && status != TASK_STATUS__PAUSE) {
qError("stream task wait for the end of fill history, s-task:%s, status:%d", pTask->id.idStr,
qError("stream task wait for the end of fill history, s-task:%s, status:%d", id,
atomic_load_8(&pTask->status.taskStatus));
taosMsleep(2);
taosMsleep(100);
} else {
break;
}
}
int64_t st = taosGetTimestampMs();
qDebug("s-task:%s start to execute, block batches:%d", pTask->id.idStr, batchSize);
qDebug("s-task:%s start to process batch of blocks, num:%d", id, batchSize);
{
// set input
......@@ -417,21 +426,21 @@ int32_t streamExecForAll(SStreamTask* pTask) {
ASSERT(pTask->taskLevel == TASK_LEVEL__SOURCE);
const SStreamDataSubmit* pSubmit = (const SStreamDataSubmit*)pInput;
qSetMultiStreamInput(pExecutor, &pSubmit->submit, 1, STREAM_INPUT__DATA_SUBMIT);
qDebug("s-task:%s set submit blocks as source block completed, %p %p len:%d ver:%" PRId64, pTask->id.idStr, pSubmit,
qDebug("s-task:%s set submit blocks as source block completed, %p %p len:%d ver:%" PRId64, id, pSubmit,
pSubmit->submit.msgStr, pSubmit->submit.msgLen, pSubmit->submit.ver);
} else if (pItem->type == STREAM_INPUT__DATA_BLOCK || pItem->type == STREAM_INPUT__DATA_RETRIEVE) {
const SStreamDataBlock* pBlock = (const SStreamDataBlock*)pInput;
SArray* pBlockList = pBlock->blocks;
int32_t numOfBlocks = taosArrayGetSize(pBlockList);
qDebug("s-task:%s set sdata blocks as input num:%d, ver:%" PRId64, pTask->id.idStr, numOfBlocks, pBlock->sourceVer);
qDebug("s-task:%s set sdata blocks as input num:%d, ver:%" PRId64, id, numOfBlocks, pBlock->sourceVer);
qSetMultiStreamInput(pExecutor, pBlockList->pData, numOfBlocks, STREAM_INPUT__DATA_BLOCK);
} else if (pItem->type == STREAM_INPUT__MERGED_SUBMIT) {
const SStreamMergedSubmit* pMerged = (const SStreamMergedSubmit*)pInput;
SArray* pBlockList = pMerged->submits;
int32_t numOfBlocks = taosArrayGetSize(pBlockList);
qDebug("s-task:%s %p set submit input (merged), batch num:%d", pTask->id.idStr, pTask, numOfBlocks);
qDebug("s-task:%s %p set (merged) submit blocks as a batch, numOfBlocks:%d", id, pTask, numOfBlocks);
qSetMultiStreamInput(pExecutor, pBlockList->pData, numOfBlocks, STREAM_INPUT__MERGED_SUBMIT);
} else if (pItem->type == STREAM_INPUT__REF_DATA_BLOCK) {
const SStreamRefDataBlock* pRefBlock = (const SStreamRefDataBlock*)pInput;
......@@ -446,7 +455,8 @@ int32_t streamExecForAll(SStreamTask* pTask) {
streamTaskExecImpl(pTask, pInput, &resSize, &totalBlocks);
double el = (taosGetTimestampMs() - st) / 1000.0;
qDebug("s-task:%s exec end, elapsed time:%.2fs, result size:%.2fMiB, numOfBlocks:%d", pTask->id.idStr, el, resSize / 1048576.0, totalBlocks);
qDebug("s-task:%s batch of input blocks exec end, elapsed time:%.2fs, result size:%.2fMiB, numOfBlocks:%d",
id, el, resSize / 1048576.0, totalBlocks);
streamFreeQitem(pInput);
}
......
......@@ -20,9 +20,9 @@
#include "ttime.h"
#define DEFAULT_FALSE_POSITIVE 0.01
#define DEFAULT_BUCKET_SIZE 1310720
#define DEFAULT_MAP_CAPACITY 1310720
#define DEFAULT_MAP_SIZE (DEFAULT_MAP_CAPACITY * 10)
#define DEFAULT_BUCKET_SIZE 131072
#define DEFAULT_MAP_CAPACITY 131072
#define DEFAULT_MAP_SIZE (DEFAULT_MAP_CAPACITY * 100)
#define ROWS_PER_MILLISECOND 1
#define MAX_NUM_SCALABLE_BF 100000
#define MIN_NUM_SCALABLE_BF 10
......@@ -44,8 +44,8 @@ static void windowSBfAdd(SUpdateInfo *pInfo, uint64_t count) {
}
}
static void clearItemHelper(void* p) {
SScalableBf** pBf = p;
static void clearItemHelper(void *p) {
SScalableBf **pBf = p;
tScalableBfDestroy(*pBf);
}
......@@ -274,7 +274,7 @@ void updateInfoDestoryColseWinSBF(SUpdateInfo *pInfo) {
}
int32_t updateInfoSerialize(void *buf, int32_t bufLen, const SUpdateInfo *pInfo) {
if(!pInfo) {
if (!pInfo) {
return 0;
}
......
......@@ -16,12 +16,12 @@
#include "tstreamFileState.h"
#include "query.h"
#include "storageapi.h"
#include "streamBackendRocksdb.h"
#include "taos.h"
#include "tcommon.h"
#include "thash.h"
#include "tsimplehash.h"
#include "storageapi.h"
#define FLUSH_RATIO 0.5
#define FLUSH_NUM 4
......@@ -137,7 +137,7 @@ void clearExpiredRowBuff(SStreamFileState* pFileState, TSKEY ts, bool all) {
SListNode* pNode = NULL;
while ((pNode = tdListNext(&iter)) != NULL) {
SRowBuffPos* pPos = *(SRowBuffPos**)(pNode->data);
if (all || (pFileState->getTs(pPos->pKey) < ts)) {
if (all || (pFileState->getTs(pPos->pKey) < ts && !pPos->beUsed)) {
ASSERT(pPos->pRowBuff != NULL);
tdListAppend(pFileState->freeBuffs, &(pPos->pRowBuff));
pPos->pRowBuff = NULL;
......@@ -416,10 +416,13 @@ int32_t deleteExpiredCheckPoint(SStreamFileState* pFileState, TSKEY mark) {
int32_t len = 0;
memcpy(buf, taskKey, strlen(taskKey));
code = streamDefaultGet_rocksdb(pFileState->pFileStore, buf, &val, &len);
if (code != 0) {
if (code != 0 || len == 0 || val == NULL) {
return TSDB_CODE_FAILED;
}
sscanf(val, "%" PRId64 "", &maxCheckPointId);
memcpy(val, buf, len);
buf[len] = 0;
maxCheckPointId = atol((char*)buf);
taosMemoryFree(val);
}
for (int64_t i = maxCheckPointId; i > 0; i--) {
char buf[128] = {0};
......@@ -430,13 +433,16 @@ int32_t deleteExpiredCheckPoint(SStreamFileState* pFileState, TSKEY mark) {
if (code != 0) {
return TSDB_CODE_FAILED;
}
memcpy(val, buf, len);
buf[len] = 0;
taosMemoryFree(val);
TSKEY ts;
sscanf(val, "%" PRId64 "", &ts);
ts = atol((char*)buf);
if (ts < mark) {
// statekey winkey.ts < mark
forceRemoveCheckpoint(pFileState, i);
break;
} else {
}
}
return code;
......
......@@ -333,7 +333,7 @@ static int32_t walFetchBodyNew(SWalReader *pReader) {
return -1;
}
wDebug("vgId:%d, index:%" PRId64 " is fetched, cursor advance", pReader->pWal->cfg.vgId, ver);
wDebug("vgId:%d, index:%" PRId64 " is fetched, type:%d, cursor advance", pReader->pWal->cfg.vgId, ver, pReader->pHead->head.msgType);
pReader->curVersion = ver + 1;
return 0;
}
......
......@@ -564,7 +564,7 @@
,,y,system-test,./pytest.sh python3 ./test.py -f 0-others/user_privilege.py
,,y,system-test,./pytest.sh python3 ./test.py -f 0-others/fsync.py
,,y,system-test,./pytest.sh python3 ./test.py -f 0-others/multilevel.py
#,,n,system-test,python3 ./test.py -f 0-others/compatibility.py
,,n,system-test,python3 ./test.py -f 0-others/compatibility.py
,,n,system-test,python3 ./test.py -f 0-others/tag_index_basic.py
,,n,system-test,python3 ./test.py -f 0-others/udfpy_main.py
,,n,system-test,python3 ./test.py -N 3 -f 0-others/walRetention.py
......@@ -820,6 +820,8 @@
,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/function_diff.py
,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/tagFilter.py
,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/projectionDesc.py
,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/ts_3398.py -N 3 -n 3
,,n,system-test,python3 ./test.py -f 2-query/queryQnode.py
,,y,system-test,./pytest.sh python3 ./test.py -f 6-cluster/5dnode1mnode.py
,,y,system-test,./pytest.sh python3 ./test.py -f 6-cluster/5dnode2mnode.py -N 5
......
......@@ -11,7 +11,7 @@
"confirm_parameter_prompt": "no",
"insert_interval": 0,
"interlace_rows": 0,
"num_of_records_per_req": 100000,
"num_of_records_per_req": 10000,
"databases": [
{
"dbinfo": {
......@@ -73,4 +73,4 @@
]
}
]
}
\ No newline at end of file
}
from util.log import *
from util.sql import *
from util.cases import *
from util.sqlset import *
import datetime
class TDTestCase:
"""This test case is used to verify the aliasName of Node structure is not truncated
when sum clause is more than 65 bits.
"""
def init(self, conn, logSql, replicaVar=1):
self.replicaVar = int(replicaVar)
tdLog.debug("start to execute %s" % __file__)
tdSql.init(conn.cursor(), False)
def run(self):
# test case for https://jira.taosdata.com:18080/browse/TS-3405:
# create db
ret = tdSql.execute("CREATE DATABASE IF NOT EXISTS statistics1 REPLICA {} DURATION 14400m KEEP 5256000m,5256000m,5256000m PRECISION 'ms' MINROWS 100 MAXROWS 4096 COMP 2;".format(self.replicaVar))
tdSql.execute("use statistics1;")
# create stable
ret = tdSql.execute("CREATE STABLE IF NOT EXISTS statistics1.`g`(`day` timestamp,`run_state` tinyint) TAGS(`vin` binary(32));")
ret = tdSql.execute("CREATE STABLE IF NOT EXISTS statistics1.`b`(`day` timestamp, `total_heart` int) TAGS(`vin` binary(32));")
ret = tdSql.execute("CREATE STABLE IF NOT EXISTS statistics1.`tg`(`day` timestamp,`lt_4177` int,`f30_4177` int, `f35_4177` int) TAGS(`vin` binary(32));")
# insert the data to table
ret = tdSql.execute("insert into d1001 using statistics1.`g` tags('NJHYNBSAS0000061') values (%s, %d)" % ("'2023-05-01'", 99))
ret = tdSql.execute("insert into d2001 using statistics1.`b` tags('NJHYNBSAS0000061') values (%s, %d)" % ("'2023-05-01'", 99))
ret = tdSql.execute("insert into d3001 using statistics1.`tg` tags('NJHYNBSAS0000061') values (%s, %d, %d, %d)" % ("'2023-05-01'", 99, 99, 99))
# execute the sql statements
ret = tdSql.query("SELECT b.`day` `day`,sum(CASE WHEN tg.lt_4177 IS NULL THEN 0 ELSE tg.lt_4177 END \
+ CASE WHEN tg.f35_4177 IS NULL THEN 0 ELSE tg.f35_4177 END) / 3600 es0,sum(CASE WHEN tg.lt_4177 \
IS NULL THEN 0 ELSE tg.lt_4177 END + CASE WHEN tg.f35_4177 IS NULL THEN 0 ELSE tg.f35_4177 \
END + CASE WHEN tg.f30_4177 IS NULL THEN 0 ELSE tg.f30_4177 END) / 3600 es1 FROM \
statistics1.b b,statistics1.tg tg,statistics1.g g WHERE b.`day` = tg.`day` AND g.`day` = b.`day` \
AND b.vin = tg.vin AND b.vin = g.vin AND b.`day` BETWEEN '2023-05-01' AND '2023-05-05' \
AND b.vin = 'NJHYNBSAS0000061' AND g.vin IS NOT NULL AND b.vin IS NOT NULL AND tg.vin IS NOT NULL \
GROUP BY b.`day`;")
# check the result
if 0.055 in tdSql.queryResult[0] and 0.0825 in tdSql.queryResult[0]:
tdLog.info("query result is correct")
else:
tdLog.info("query result is wrong")
def stop(self):
# clear the db
tdSql.execute("drop database if exists statistics1;")
tdSql.close()
tdLog.success("%s successfully executed" % __file__)
tdCases.addWindows(__file__, TDTestCase())
tdCases.addLinux(__file__, TDTestCase())
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册