From e92bfa558dc4c391217c0a3a14b5b34dcaf6e3e1 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 6 Mar 2023 21:16:30 +0800 Subject: [PATCH 01/65] fix(query): allow only one trans to be execute for each balance, and do some other refactor. --- source/dnode/mnode/impl/src/mndConsumer.c | 133 ++++++++------- source/dnode/mnode/impl/src/mndSubscribe.c | 186 +++++++++++++-------- 2 files changed, 188 insertions(+), 131 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndConsumer.c b/source/dnode/mnode/impl/src/mndConsumer.c index f1ef83aca5..623b375305 100644 --- a/source/dnode/mnode/impl/src/mndConsumer.c +++ b/source/dnode/mnode/impl/src/mndConsumer.c @@ -845,6 +845,53 @@ static int32_t mndConsumerActionDelete(SSdb *pSdb, SMqConsumerObj *pConsumer) { return 0; } +static void updateConsumerStatus(SMqConsumerObj* pConsumer) { + int32_t status = pConsumer->status; + + if (taosArrayGetSize(pConsumer->rebNewTopics) == 0 && taosArrayGetSize(pConsumer->rebRemovedTopics) == 0) { + if (status == MQ_CONSUMER_STATUS__MODIFY || status == MQ_CONSUMER_STATUS__MODIFY_IN_REB) { + pConsumer->status = MQ_CONSUMER_STATUS__READY; + } else if (status == MQ_CONSUMER_STATUS__LOST_IN_REB || status == MQ_CONSUMER_STATUS__LOST) { + pConsumer->status = MQ_CONSUMER_STATUS__LOST_REBD; + } + } else { + if (status == MQ_CONSUMER_STATUS__MODIFY || status == MQ_CONSUMER_STATUS__MODIFY_IN_REB) { + pConsumer->status = MQ_CONSUMER_STATUS__MODIFY; + } else if (status == MQ_CONSUMER_STATUS__LOST || status == MQ_CONSUMER_STATUS__LOST_IN_REB) { + pConsumer->status = MQ_CONSUMER_STATUS__LOST; + } + } +} + +// remove from new topic +static void removeFromNewTopicList(SMqConsumerObj* pConsumer, const char* pTopic) { + int32_t size = taosArrayGetSize(pConsumer->rebNewTopics); + for (int32_t i = 0; i < taosArrayGetSize(pConsumer->rebNewTopics); i++) { + char *p = taosArrayGetP(pConsumer->rebNewTopics, i); + if (strcmp(pTopic, p) == 0) { + taosArrayRemove(pConsumer->rebNewTopics, i); + taosMemoryFree(p); + + mDebug("consumer:0x%" PRIx64 " remove new topic:%s in the topic list, remain newTopics:%d", pConsumer->consumerId, + pTopic, (int) taosArrayGetSize(pConsumer->rebNewTopics)); + break; + } + } +} + +// remove from removed topic +static void removeFromRemoveTopicList(SMqConsumerObj* pConsumer, const char* pTopic) { + int32_t size = taosArrayGetSize(pConsumer->rebRemovedTopics); + for (int32_t i = 0; i < size; i++) { + char *p = taosArrayGetP(pConsumer->rebRemovedTopics, i); + if (strcmp(pTopic, p) == 0) { + taosArrayRemove(pConsumer->rebRemovedTopics, i); + taosMemoryFree(p); + break; + } + } +} + static int32_t mndConsumerActionUpdate(SSdb *pSdb, SMqConsumerObj *pOldConsumer, SMqConsumerObj *pNewConsumer) { mDebug("consumer:0x%" PRIx64 " perform update action, update type:%d, subscribe-time:%" PRId64 ", uptime:%" PRId64, pOldConsumer->consumerId, pNewConsumer->updateType, pOldConsumer->subscribeTime, pOldConsumer->upTime); @@ -855,6 +902,7 @@ static int32_t mndConsumerActionUpdate(SSdb *pSdb, SMqConsumerObj *pOldConsumer, /*A(taosArrayGetSize(pOldConsumer->rebNewTopics) == 0);*/ /*A(taosArrayGetSize(pOldConsumer->rebRemovedTopics) == 0);*/ + // this new consumer has identical topics with one existed consumers. if (taosArrayGetSize(pNewConsumer->rebNewTopics) == 0 && taosArrayGetSize(pNewConsumer->rebRemovedTopics) == 0) { pOldConsumer->status = MQ_CONSUMER_STATUS__READY; } else { @@ -871,7 +919,6 @@ static int32_t mndConsumerActionUpdate(SSdb *pSdb, SMqConsumerObj *pOldConsumer, pNewConsumer->assignedTopics = tmp; pOldConsumer->subscribeTime = pNewConsumer->upTime; - pOldConsumer->status = MQ_CONSUMER_STATUS__MODIFY; } } else if (pNewConsumer->updateType == CONSUMER_UPDATE__LOST) { @@ -911,71 +958,48 @@ static int32_t mndConsumerActionUpdate(SSdb *pSdb, SMqConsumerObj *pOldConsumer, pOldConsumer->rebalanceTime = pNewConsumer->upTime; } else if (pNewConsumer->updateType == CONSUMER_UPDATE__ADD) { - /*A(taosArrayGetSize(pNewConsumer->rebNewTopics) == 1);*/ - /*A(taosArrayGetSize(pNewConsumer->rebRemovedTopics) == 0);*/ + ASSERT(taosArrayGetSize(pNewConsumer->rebNewTopics) == 1 && taosArrayGetSize(pNewConsumer->rebRemovedTopics) == 0); + char *pNewTopic = taosStrdup(taosArrayGetP(pNewConsumer->rebNewTopics, 0)); - char *addedTopic = taosStrdup(taosArrayGetP(pNewConsumer->rebNewTopics, 0)); // not exist in current topic - - bool existing = false; -#if 1 + bool existing = false; int32_t numOfExistedTopics = taosArrayGetSize(pOldConsumer->currentTopics); for (int32_t i = 0; i < numOfExistedTopics; i++) { char *topic = taosArrayGetP(pOldConsumer->currentTopics, i); - if (strcmp(topic, addedTopic) == 0) { + if (strcmp(topic, pNewTopic) == 0) { existing = true; } } -#endif - // remove from new topic - for (int32_t i = 0; i < taosArrayGetSize(pOldConsumer->rebNewTopics); i++) { - char *topic = taosArrayGetP(pOldConsumer->rebNewTopics, i); - if (strcmp(addedTopic, topic) == 0) { - taosArrayRemove(pOldConsumer->rebNewTopics, i); - taosMemoryFree(topic); - break; - } - } + removeFromNewTopicList(pOldConsumer, pNewTopic); // add to current topic if (!existing) { - taosArrayPush(pOldConsumer->currentTopics, &addedTopic); + taosArrayPush(pOldConsumer->currentTopics, &pNewTopic); taosArraySort(pOldConsumer->currentTopics, taosArrayCompareString); } else { - taosMemoryFree(addedTopic); + taosMemoryFree(pNewTopic); } // set status int32_t status = pOldConsumer->status; - if (taosArrayGetSize(pOldConsumer->rebNewTopics) == 0 && taosArrayGetSize(pOldConsumer->rebRemovedTopics) == 0) { - if (status == MQ_CONSUMER_STATUS__MODIFY || status == MQ_CONSUMER_STATUS__MODIFY_IN_REB) { - pOldConsumer->status = MQ_CONSUMER_STATUS__READY; - } else if (status == MQ_CONSUMER_STATUS__LOST_IN_REB || status == MQ_CONSUMER_STATUS__LOST) { - pOldConsumer->status = MQ_CONSUMER_STATUS__LOST_REBD; - } - } else { - if (status == MQ_CONSUMER_STATUS__MODIFY || status == MQ_CONSUMER_STATUS__MODIFY_IN_REB) { - pOldConsumer->status = MQ_CONSUMER_STATUS__MODIFY_IN_REB; - } else if (status == MQ_CONSUMER_STATUS__LOST || status == MQ_CONSUMER_STATUS__LOST_IN_REB) { - pOldConsumer->status = MQ_CONSUMER_STATUS__LOST_IN_REB; - } - } + updateConsumerStatus(pOldConsumer); // the re-balance is triggered when the new consumer is launched. pOldConsumer->rebalanceTime = pNewConsumer->upTime; atomic_add_fetch_32(&pOldConsumer->epoch, 1); - mDebug("consumer:0x%" PRIx64 " state (%d)%s -> (%d)%s, new epoch:%d, reb-time:%" PRId64 ", current topics:%d", + mDebug("consumer:0x%" PRIx64 " state (%d)%s -> (%d)%s, new epoch:%d, reb-time:%" PRId64 + ", current topics:%d, newTopics:%d, removeTopics:%d", pOldConsumer->consumerId, status, mndConsumerStatusName(status), pOldConsumer->status, - mndConsumerStatusName(pOldConsumer->status), - pOldConsumer->epoch, pOldConsumer->rebalanceTime, (int)taosArrayGetSize(pOldConsumer->currentTopics)); + mndConsumerStatusName(pOldConsumer->status), pOldConsumer->epoch, pOldConsumer->rebalanceTime, + (int)taosArrayGetSize(pOldConsumer->currentTopics), (int)taosArrayGetSize(pOldConsumer->rebNewTopics), + (int)taosArrayGetSize(pOldConsumer->rebRemovedTopics)); + } else if (pNewConsumer->updateType == CONSUMER_UPDATE__REMOVE) { /*A(taosArrayGetSize(pNewConsumer->rebNewTopics) == 0);*/ /*A(taosArrayGetSize(pNewConsumer->rebRemovedTopics) == 1);*/ char *removedTopic = taosArrayGetP(pNewConsumer->rebRemovedTopics, 0); - - // not exist in new topic #if 0 for (int32_t i = 0; i < taosArrayGetSize(pOldConsumer->rebNewTopics); i++) { char *topic = taosArrayGetP(pOldConsumer->rebNewTopics, i); @@ -984,14 +1008,7 @@ static int32_t mndConsumerActionUpdate(SSdb *pSdb, SMqConsumerObj *pOldConsumer, #endif // remove from removed topic - for (int32_t i = 0; i < taosArrayGetSize(pOldConsumer->rebRemovedTopics); i++) { - char *topic = taosArrayGetP(pOldConsumer->rebRemovedTopics, i); - if (strcmp(removedTopic, topic) == 0) { - taosArrayRemove(pOldConsumer->rebRemovedTopics, i); - taosMemoryFree(topic); - break; - } - } + removeFromRemoveTopicList(pOldConsumer, removedTopic); // remove from current topic int32_t i = 0; @@ -1004,32 +1021,20 @@ static int32_t mndConsumerActionUpdate(SSdb *pSdb, SMqConsumerObj *pOldConsumer, break; } } - // must find the topic - /*A(i < sz);*/ // set status int32_t status = pOldConsumer->status; - if (taosArrayGetSize(pOldConsumer->rebNewTopics) == 0 && taosArrayGetSize(pOldConsumer->rebRemovedTopics) == 0) { - if (status == MQ_CONSUMER_STATUS__MODIFY || status == MQ_CONSUMER_STATUS__MODIFY_IN_REB) { - pOldConsumer->status = MQ_CONSUMER_STATUS__READY; - } else if (status == MQ_CONSUMER_STATUS__LOST_IN_REB || status == MQ_CONSUMER_STATUS__LOST) { - pOldConsumer->status = MQ_CONSUMER_STATUS__LOST_REBD; - } - } else { - if (status == MQ_CONSUMER_STATUS__MODIFY || status == MQ_CONSUMER_STATUS__MODIFY_IN_REB) { - pOldConsumer->status = MQ_CONSUMER_STATUS__MODIFY_IN_REB; - } else if (status == MQ_CONSUMER_STATUS__LOST || status == MQ_CONSUMER_STATUS__LOST_IN_REB) { - pOldConsumer->status = MQ_CONSUMER_STATUS__LOST_IN_REB; - } - } + updateConsumerStatus(pOldConsumer); pOldConsumer->rebalanceTime = pNewConsumer->upTime; atomic_add_fetch_32(&pOldConsumer->epoch, 1); - mDebug("consumer:0x%" PRIx64 " state %d(%s) -> %d(%s), new epoch:%d, reb-time:%" PRId64 ", current topics:%d", + mDebug("consumer:0x%" PRIx64 " state (%d)%s -> (%d)%s, new epoch:%d, reb-time:%" PRId64 + ", current topics:%d, newTopics:%d, removeTopics:%d", pOldConsumer->consumerId, status, mndConsumerStatusName(status), pOldConsumer->status, - mndConsumerStatusName(pOldConsumer->status), - pOldConsumer->epoch, pOldConsumer->rebalanceTime, (int)taosArrayGetSize(pOldConsumer->currentTopics)); + mndConsumerStatusName(pOldConsumer->status), pOldConsumer->epoch, pOldConsumer->rebalanceTime, + (int)taosArrayGetSize(pOldConsumer->currentTopics), (int)taosArrayGetSize(pOldConsumer->rebNewTopics), + (int)taosArrayGetSize(pOldConsumer->rebRemovedTopics)); } taosWUnLockLatch(&pOldConsumer->lock); diff --git a/source/dnode/mnode/impl/src/mndSubscribe.c b/source/dnode/mnode/impl/src/mndSubscribe.c index 21539a6313..5ad12a6450 100644 --- a/source/dnode/mnode/impl/src/mndSubscribe.c +++ b/source/dnode/mnode/impl/src/mndSubscribe.c @@ -197,24 +197,20 @@ static SMqRebInfo *mndGetOrCreateRebSub(SHashObj *pHash, const char *key) { return pRebSub; } -static int32_t mndDoRebalance(SMnode *pMnode, const SMqRebInputObj *pInput, SMqRebOutputObj *pOutput) { - int32_t totalVgNum = pOutput->pSub->vgNum; - const char *sub = pOutput->pSub->key; - mInfo("sub:%s mq re-balance %d vgroups", sub, pOutput->pSub->vgNum); +static void doRemoveExistedConsumers(SMqRebOutputObj *pOutput, SHashObj *pHash, const SMqRebInputObj *pInput) { + int32_t numOfRemoved = taosArrayGetSize(pInput->pRebInfo->removedConsumers); + const char *pSubKey = pOutput->pSub->key; - // 1. build temporary hash(vgId -> SMqRebOutputVg) to store modified vg - SHashObj *pHash = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), false, HASH_NO_LOCK); - - // 2. check and get actual removed consumers, put their vg into hash - int32_t removedNum = taosArrayGetSize(pInput->pRebInfo->removedConsumers); int32_t actualRemoved = 0; - for (int32_t i = 0; i < removedNum; i++) { + for (int32_t i = 0; i < numOfRemoved; i++) { uint64_t consumerId = *(uint64_t *)taosArrayGet(pInput->pRebInfo->removedConsumers, i); SMqConsumerEp *pConsumerEp = taosHashGet(pOutput->pSub->consumerHash, &consumerId, sizeof(int64_t)); + // consumer exists till now if (pConsumerEp) { actualRemoved++; + int32_t consumerVgNum = taosArrayGetSize(pConsumerEp->vgs); for (int32_t j = 0; j < consumerVgNum; j++) { SMqVgEp *pVgEp = taosArrayGetP(pConsumerEp->vgs, j); @@ -223,52 +219,66 @@ static int32_t mndDoRebalance(SMnode *pMnode, const SMqRebInputObj *pInput, SMqR .newConsumerId = -1, .pVgEp = pVgEp, }; + taosHashPut(pHash, &pVgEp->vgId, sizeof(int32_t), &outputVg, sizeof(SMqRebOutputVg)); - mInfo("sub:%s mq re-balance remove vgId:%d from consumer:%" PRIx64, sub, pVgEp->vgId, consumerId); + mInfo("sub:%s mq re-balance remove vgId:%d from consumer:%" PRIx64, pSubKey, pVgEp->vgId, consumerId); } + taosArrayDestroy(pConsumerEp->vgs); taosHashRemove(pOutput->pSub->consumerHash, &consumerId, sizeof(int64_t)); + // put into removed taosArrayPush(pOutput->removedConsumers, &consumerId); } } - if (removedNum != actualRemoved) { - mError("sub:%s mq re-balance removedNum:%d not matched with actual:%d", sub, removedNum, actualRemoved); + if (numOfRemoved != actualRemoved) { + mError("sub:%s mq re-balance removedNum:%d not matched with actual:%d", pSubKey, numOfRemoved, actualRemoved); + } else { + mInfo("sub:%s removed %d consumers", pSubKey, numOfRemoved); } +} - // if previously no consumer, there are vgs not assigned - { - int32_t consumerVgNum = taosArrayGetSize(pOutput->pSub->unassignedVgs); - for (int32_t i = 0; i < consumerVgNum; i++) { - SMqVgEp *pVgEp = *(SMqVgEp **)taosArrayPop(pOutput->pSub->unassignedVgs); - SMqRebOutputVg rebOutput = { - .oldConsumerId = -1, - .newConsumerId = -1, - .pVgEp = pVgEp, - }; - taosHashPut(pHash, &pVgEp->vgId, sizeof(int32_t), &rebOutput, sizeof(SMqRebOutputVg)); - mInfo("sub:%s mq re-balance remove vgId:%d from unassigned", sub, pVgEp->vgId); - } +static void doAddNewConsumers(SMqRebOutputObj *pOutput, const SMqRebInputObj *pInput) { + int32_t numOfNewConsumers = taosArrayGetSize(pInput->pRebInfo->newConsumers); + const char *pSubKey = pOutput->pSub->key; + + for (int32_t i = 0; i < numOfNewConsumers; i++) { + int64_t consumerId = *(int64_t *)taosArrayGet(pInput->pRebInfo->newConsumers, i); + + SMqConsumerEp newConsumerEp; + newConsumerEp.consumerId = consumerId; + newConsumerEp.vgs = taosArrayInit(0, sizeof(void *)); + + taosHashPut(pOutput->pSub->consumerHash, &consumerId, sizeof(int64_t), &newConsumerEp, sizeof(SMqConsumerEp)); + taosArrayPush(pOutput->newConsumers, &consumerId); + mInfo("sub:%s mq rebalance add new consumer:%" PRIx64, pSubKey, consumerId); } +} - // 3. calc vg number of each consumer - int32_t afterRebConsumerNum = pInput->oldConsumerNum + taosArrayGetSize(pInput->pRebInfo->newConsumers) - - taosArrayGetSize(pInput->pRebInfo->removedConsumers); - int32_t minVgCnt = 0; - int32_t imbConsumerNum = 0; - // calc num - if (afterRebConsumerNum) { - minVgCnt = totalVgNum / afterRebConsumerNum; - imbConsumerNum = totalVgNum % afterRebConsumerNum; +static void addUnassignedVgroups(SMqRebOutputObj *pOutput, SHashObj* pHash) { + const char *pSubKey = pOutput->pSub->key; + int32_t numOfVgroups = taosArrayGetSize(pOutput->pSub->unassignedVgs); + + for (int32_t i = 0; i < numOfVgroups; i++) { + SMqVgEp *pVgEp = *(SMqVgEp **)taosArrayPop(pOutput->pSub->unassignedVgs); + SMqRebOutputVg rebOutput = { + .oldConsumerId = -1, + .newConsumerId = -1, + .pVgEp = pVgEp, + }; + + taosHashPut(pHash, &pVgEp->vgId, sizeof(int32_t), &rebOutput, sizeof(SMqRebOutputVg)); + mInfo("sub:%s mq re-balance remove vgId:%d from unassigned", pSubKey, pVgEp->vgId); } +} - mInfo("sub:%s mq re-balance %d consumers: at least %d vgs each, %d consumers has more vgs", sub, - afterRebConsumerNum, minVgCnt, imbConsumerNum); +static void transferVgroupsForConsumers(SMqRebOutputObj *pOutput, SHashObj* pHash, int32_t minVgCnt, int32_t imbConsumerNum) { + const char *pSubKey = pOutput->pSub->key; - // 4. first scan: remove consumer more than wanted, put to remove hash int32_t imbCnt = 0; void *pIter = NULL; + while (1) { pIter = taosHashIterate(pOutput->pSub->consumerHash, pIter); if (pIter == NULL) { @@ -276,8 +286,8 @@ static int32_t mndDoRebalance(SMnode *pMnode, const SMqRebInputObj *pInput, SMqR } SMqConsumerEp *pConsumerEp = (SMqConsumerEp *)pIter; + int32_t consumerVgNum = taosArrayGetSize(pConsumerEp->vgs); - int32_t consumerVgNum = taosArrayGetSize(pConsumerEp->vgs); // all old consumers still existing are touched // TODO optimize: touch only consumer whose vgs changed taosArrayPush(pOutput->touchedConsumers, &pConsumerEp->consumerId); @@ -296,13 +306,13 @@ static int32_t mndDoRebalance(SMnode *pMnode, const SMqRebInputObj *pInput, SMqR .pVgEp = pVgEp, }; taosHashPut(pHash, &pVgEp->vgId, sizeof(int32_t), &outputVg, sizeof(SMqRebOutputVg)); - mInfo("sub:%s mq rebalance remove vgId:%d from consumer:0x%" PRIx64 ",(first scan)", sub, pVgEp->vgId, + mInfo("sub:%s mq rebalance remove vgId:%d from consumer:0x%" PRIx64 ",(first scan)", pSubKey, pVgEp->vgId, pConsumerEp->consumerId); } imbCnt++; } } else { - // pop until equal minVg + // all the remain consumers should only have the number of vgroups, which is equalled to the value of minVg while (taosArrayGetSize(pConsumerEp->vgs) > minVgCnt) { SMqVgEp *pVgEp = *(SMqVgEp **)taosArrayPop(pConsumerEp->vgs); SMqRebOutputVg outputVg = { @@ -311,36 +321,66 @@ static int32_t mndDoRebalance(SMnode *pMnode, const SMqRebInputObj *pInput, SMqR .pVgEp = pVgEp, }; taosHashPut(pHash, &pVgEp->vgId, sizeof(int32_t), &outputVg, sizeof(SMqRebOutputVg)); - mInfo("sub:%s mq rebalance remove vgId:%d from consumer:0x%" PRIx64 ",(first scan)", sub, pVgEp->vgId, + mInfo("sub:%s mq rebalance remove vgId:%d from consumer:0x%" PRIx64 ",(first scan)", pSubKey, pVgEp->vgId, pConsumerEp->consumerId); } } } } +} - // 5. add new consumer into sub - { - int32_t consumerNum = taosArrayGetSize(pInput->pRebInfo->newConsumers); - for (int32_t i = 0; i < consumerNum; i++) { - int64_t consumerId = *(int64_t *)taosArrayGet(pInput->pRebInfo->newConsumers, i); +static int32_t mndDoRebalance(SMnode *pMnode, const SMqRebInputObj *pInput, SMqRebOutputObj *pOutput) { + int32_t totalVgNum = pOutput->pSub->vgNum; + const char *pSubKey = pOutput->pSub->key; - SMqConsumerEp newConsumerEp; - newConsumerEp.consumerId = consumerId; - newConsumerEp.vgs = taosArrayInit(0, sizeof(void *)); - taosHashPut(pOutput->pSub->consumerHash, &consumerId, sizeof(int64_t), &newConsumerEp, sizeof(SMqConsumerEp)); - taosArrayPush(pOutput->newConsumers, &consumerId); - mInfo("sub:%s mq rebalance add new consumer:%" PRIx64, sub, consumerId); - } + int32_t numOfRemoved = taosArrayGetSize(pInput->pRebInfo->removedConsumers); + int32_t numOfAdded = taosArrayGetSize(pInput->pRebInfo->newConsumers); + mInfo("sub:%s mq re-balance %d vgroups, existed consumers:%d, added:%d, removed:%d", pSubKey, totalVgNum, + pInput->oldConsumerNum, numOfAdded, numOfRemoved); + + // 1. build temporary hash(vgId -> SMqRebOutputVg) to store modified vg + SHashObj *pHash = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), false, HASH_NO_LOCK); + + // 2. check and get actual removed consumers, put their vg into hash + doRemoveExistedConsumers(pOutput, pHash, pInput); + + // 3. if previously no consumer, there are vgs not assigned + addUnassignedVgroups(pOutput, pHash); + + // 4. calc vg number of each consumer + int32_t numOfFinal = pInput->oldConsumerNum + numOfAdded - numOfRemoved; + + int32_t minVgCnt = 0; + int32_t imbConsumerNum = 0; + + // calc num + if (numOfFinal) { + minVgCnt = totalVgNum / numOfFinal; + imbConsumerNum = totalVgNum % numOfFinal; } - // 6. second scan: find consumer do not have enough vg, extract from temporary hash and assign to new consumer. + mInfo("sub:%s mq re-balance %d consumers: at least %d vgs each, %d consumers has 1 more vgroups than avg value", + pSubKey, numOfFinal, minVgCnt, imbConsumerNum); + + // 5. first scan: remove vgroups from te consumers, who have more vgroups than the threashold value that is + // minVgCnt, and then put them into the recycled hash list + transferVgroupsForConsumers(pOutput, pHash, minVgCnt, imbConsumerNum); + + // 6. add new consumer into sub + doAddNewConsumers(pOutput, pInput); + + // 7. second scan: find consumer do not have enough vgroups, extract from temporary hash and assign to them // All related vg should be put into rebVgs SMqRebOutputVg *pRebVg = NULL; void *pRemovedIter = NULL; - pIter = NULL; + void *pIter = NULL; + while (1) { pIter = taosHashIterate(pOutput->pSub->consumerHash, pIter); - if (pIter == NULL) break; + if (pIter == NULL) { + break; + } + SMqConsumerEp *pConsumerEp = (SMqConsumerEp *)pIter; // push until equal minVg @@ -348,8 +388,8 @@ static int32_t mndDoRebalance(SMnode *pMnode, const SMqRebInputObj *pInput, SMqR // iter hash and find one vg pRemovedIter = taosHashIterate(pHash, pRemovedIter); if (pRemovedIter == NULL) { - mError("sub:%s removed iter is null", sub); - continue; + mError("sub:%s removed iter is null", pSubKey); + break; } pRebVg = (SMqRebOutputVg *)pRemovedIter; @@ -409,15 +449,15 @@ static int32_t mndDoRebalance(SMnode *pMnode, const SMqRebInputObj *pInput, SMqR taosArrayPush(pOutput->pSub->unassignedVgs, &pRebOutput->pVgEp); taosArrayPush(pOutput->rebVgs, pRebOutput); - mInfo("sub:%s mq re-balance unassign vgId:%d (second scan)", sub, pRebOutput->pVgEp->vgId); + mInfo("sub:%s mq re-balance unassign vgId:%d (second scan)", pSubKey, pRebOutput->pVgEp->vgId); } } // 8. generate logs - mInfo("sub:%s mq re-balance calculation completed, re-balanced vg", sub); + mInfo("sub:%s mq re-balance calculation completed, re-balanced vg", pSubKey); for (int32_t i = 0; i < taosArrayGetSize(pOutput->rebVgs); i++) { SMqRebOutputVg *pOutputRebVg = taosArrayGet(pOutput->rebVgs, i); - mInfo("sub:%s mq re-balance vgId:%d, moved from consumer:0x%" PRIx64 ", to consumer:0x%" PRIx64, sub, + mInfo("sub:%s mq re-balance vgId:%d, moved from consumer:0x%" PRIx64 ", to consumer:0x%" PRIx64, pSubKey, pOutputRebVg->pVgEp->vgId, pOutputRebVg->oldConsumerId, pOutputRebVg->newConsumerId); } { @@ -427,10 +467,10 @@ static int32_t mndDoRebalance(SMnode *pMnode, const SMqRebInputObj *pInput, SMqR if (pIter == NULL) break; SMqConsumerEp *pConsumerEp = (SMqConsumerEp *)pIter; int32_t sz = taosArrayGetSize(pConsumerEp->vgs); - mInfo("sub:%s mq re-balance final cfg: consumer:0x%" PRIx64 " has %d vg", sub, pConsumerEp->consumerId, sz); + mInfo("sub:%s mq re-balance final cfg: consumer:0x%" PRIx64 " has %d vg", pSubKey, pConsumerEp->consumerId, sz); for (int32_t i = 0; i < sz; i++) { SMqVgEp *pVgEp = taosArrayGetP(pConsumerEp->vgs, i); - mInfo("sub:%s mq re-balance final cfg: vg %d to consumer:0x%" PRIx64, sub, pVgEp->vgId, + mInfo("sub:%s mq re-balance final cfg: vg %d to consumer:0x%" PRIx64, pSubKey, pVgEp->vgId, pConsumerEp->consumerId); } } @@ -555,17 +595,24 @@ static int32_t mndProcessRebalanceReq(SRpcMsg *pMsg) { SMnode *pMnode = pMsg->info.node; SMqDoRebalanceMsg *pReq = pMsg->pCont; void *pIter = NULL; + bool rebalanceExec = false; // to ensure only once. - mInfo("mq re-balance start"); + mInfo("mq re-balance start, total required re-balanced trans:%d", taosHashGetSize(pReq->rebSubHash)); + // here we only handle one topic rebalance requirement to ensure the atomic execution of this transaction. while (1) { + if (rebalanceExec) { + break; + } + pIter = taosHashIterate(pReq->rebSubHash, pIter); if (pIter == NULL) { break; } - SMqRebInputObj rebInput = {0}; + taosSsleep(20); + SMqRebInputObj rebInput = {0}; SMqRebOutputObj rebOutput = {0}; rebOutput.newConsumers = taosArrayInit(0, sizeof(int64_t)); rebOutput.removedConsumers = taosArrayInit(0, sizeof(int64_t)); @@ -582,9 +629,10 @@ static int32_t mndProcessRebalanceReq(SRpcMsg *pMsg) { char topic[TSDB_TOPIC_FNAME_LEN]; char cgroup[TSDB_CGROUP_LEN]; mndSplitSubscribeKey(pRebInfo->key, topic, cgroup, true); + SMqTopicObj *pTopic = mndAcquireTopic(pMnode, topic); if (pTopic == NULL) { - mError("mq re-balance %s ignored since topic %s not exist", pRebInfo->key, topic); + mError("mq re-balance %s ignored since topic %s doesn't exist", pRebInfo->key, topic); continue; } @@ -604,11 +652,13 @@ static int32_t mndProcessRebalanceReq(SRpcMsg *pMsg) { mndReleaseTopic(pMnode, pTopic); rebInput.oldConsumerNum = 0; + mInfo("topic:%s has no consumers sub yet", topic); } else { taosRLockLatch(&pSub->lock); rebInput.oldConsumerNum = taosHashGetSize(pSub->consumerHash); rebOutput.pSub = tCloneSubscribeObj(pSub); taosRUnLockLatch(&pSub->lock); + mInfo("topic:%s has %d consumers sub till now", pRebInfo->key, rebInput.oldConsumerNum); mndReleaseSubscribe(pMnode, pSub); } @@ -633,6 +683,8 @@ static int32_t mndProcessRebalanceReq(SRpcMsg *pMsg) { taosArrayDestroy(rebOutput.rebVgs); tDeleteSubscribeObj(rebOutput.pSub); taosMemoryFree(rebOutput.pSub); + + rebalanceExec = true; } // reset flag From 1b93f980e874476c0b35f5ba5461659d233924db Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 6 Mar 2023 21:33:31 +0800 Subject: [PATCH 02/65] refactor: do some internal refactor. --- source/dnode/mnode/impl/src/mndSubscribe.c | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndSubscribe.c b/source/dnode/mnode/impl/src/mndSubscribe.c index 5ad12a6450..6368472dc9 100644 --- a/source/dnode/mnode/impl/src/mndSubscribe.c +++ b/source/dnode/mnode/impl/src/mndSubscribe.c @@ -357,11 +357,12 @@ static int32_t mndDoRebalance(SMnode *pMnode, const SMqRebInputObj *pInput, SMqR if (numOfFinal) { minVgCnt = totalVgNum / numOfFinal; imbConsumerNum = totalVgNum % numOfFinal; + mInfo("sub:%s mq re-balance %d consumers: at least %d vgs each, %d consumers has 1 more vgroups than avg value", + pSubKey, numOfFinal, minVgCnt, imbConsumerNum); + } else { + mInfo("sub:%s no consumer subscribe this topic", pSubKey); } - mInfo("sub:%s mq re-balance %d consumers: at least %d vgs each, %d consumers has 1 more vgroups than avg value", - pSubKey, numOfFinal, minVgCnt, imbConsumerNum); - // 5. first scan: remove vgroups from te consumers, who have more vgroups than the threashold value that is // minVgCnt, and then put them into the recycled hash list transferVgroupsForConsumers(pOutput, pHash, minVgCnt, imbConsumerNum); From 60cdbe37d39ede2d8aace45b32b158446907ef8f Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 6 Mar 2023 22:25:34 +0800 Subject: [PATCH 03/65] refactor: do some internal refactor. --- source/client/test/clientTests.cpp | 5 +++-- source/dnode/mnode/impl/src/mndSubscribe.c | 2 -- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/source/client/test/clientTests.cpp b/source/client/test/clientTests.cpp index 2f3d600019..cee818ccf7 100644 --- a/source/client/test/clientTests.cpp +++ b/source/client/test/clientTests.cpp @@ -925,7 +925,8 @@ TEST(clientCase, subscription_test) { // 创建订阅 topics 列表 tmq_list_t* topicList = tmq_list_new(); -// tmq_list_append(topicList, "topic_t1"); + tmq_list_append(topicList, "topic_t1"); + tmq_list_append(topicList, "topic_s2"); // 启动订阅 tmq_subscribe(tmq, topicList); @@ -954,7 +955,7 @@ TEST(clientCase, subscription_test) { printf("db: %s\n", dbName); printf("vgroup id: %d\n", vgroupId); - if (count ++ > 20) { + if (count ++ > 40) { tmq_unsubscribe(tmq); break; } diff --git a/source/dnode/mnode/impl/src/mndSubscribe.c b/source/dnode/mnode/impl/src/mndSubscribe.c index 6368472dc9..5cc2bedfdb 100644 --- a/source/dnode/mnode/impl/src/mndSubscribe.c +++ b/source/dnode/mnode/impl/src/mndSubscribe.c @@ -611,8 +611,6 @@ static int32_t mndProcessRebalanceReq(SRpcMsg *pMsg) { break; } - taosSsleep(20); - SMqRebInputObj rebInput = {0}; SMqRebOutputObj rebOutput = {0}; rebOutput.newConsumers = taosArrayInit(0, sizeof(int64_t)); From fc39a330b4405ce61b971437d63068665e547a28 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Mon, 6 Mar 2023 23:30:25 +0800 Subject: [PATCH 04/65] fix:wrong uasge of taosArrayGet --- source/dnode/mnode/impl/src/mndSubscribe.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndSubscribe.c b/source/dnode/mnode/impl/src/mndSubscribe.c index 5cc2bedfdb..402f157366 100644 --- a/source/dnode/mnode/impl/src/mndSubscribe.c +++ b/source/dnode/mnode/impl/src/mndSubscribe.c @@ -823,7 +823,7 @@ static SSdbRow *mndSubActionDecode(SSdbRaw *pRaw) { if (pSub->unassignedVgs != NULL) { int32_t size = (int32_t)taosArrayGetSize(pSub->unassignedVgs); for (int32_t i = 0; i < size; ++i) { - SMqVgEp *pMqVgEp = taosArrayGet(pSub->unassignedVgs, i); + SMqVgEp *pMqVgEp = (SMqVgEp *)taosArrayGetP(pSub->unassignedVgs, i); tmsgUpdateDnodeEpSet(&pMqVgEp->epSet); } } @@ -833,7 +833,7 @@ static SSdbRow *mndSubActionDecode(SSdbRaw *pRaw) { SMqConsumerEp *pConsumerEp = pIter; int32_t size = (int32_t)taosArrayGetSize(pConsumerEp->vgs); for (int32_t i = 0; i < size; ++i) { - SMqVgEp *pMqVgEp = taosArrayGet(pConsumerEp->vgs, i); + SMqVgEp *pMqVgEp = (SMqVgEp *)taosArrayGetP(pConsumerEp->vgs, i); tmsgUpdateDnodeEpSet(&pMqVgEp->epSet); } pIter = taosHashIterate(pSub->consumerHash, pIter); From e350f71396b91ff1905ea666c03fe9e450499a3b Mon Sep 17 00:00:00 2001 From: Benguang Zhao Date: Tue, 14 Mar 2023 19:45:31 +0800 Subject: [PATCH 05/65] enh: reset elect timer at the end of callbacks --- source/libs/sync/src/syncAppendEntries.c | 6 +++--- source/libs/sync/src/syncElection.c | 1 - source/libs/sync/src/syncMain.c | 22 ++++++++-------------- source/libs/sync/src/syncRequestVote.c | 5 ++++- source/libs/sync/src/syncSnapshot.c | 2 +- 5 files changed, 16 insertions(+), 20 deletions(-) diff --git a/source/libs/sync/src/syncAppendEntries.c b/source/libs/sync/src/syncAppendEntries.c index bd1dae54d9..9ab545075c 100644 --- a/source/libs/sync/src/syncAppendEntries.c +++ b/source/libs/sync/src/syncAppendEntries.c @@ -105,6 +105,7 @@ int32_t syncNodeOnAppendEntries(SSyncNode* ths, const SRpcMsg* pRpcMsg) { SRpcMsg rpcRsp = {0}; bool accepted = false; SSyncRaftEntry* pEntry = NULL; + bool resetElect = false; // if already drop replica, do not process if (!syncNodeInRaftGroup(ths, &(pMsg->srcId))) { @@ -137,7 +138,7 @@ int32_t syncNodeOnAppendEntries(SSyncNode* ths, const SRpcMsg* pRpcMsg) { } syncNodeStepDown(ths, pMsg->term); - syncNodeResetElectTimer(ths); + resetElect = true; if (pMsg->dataLen < sizeof(SSyncRaftEntry)) { sError("vgId:%d, incomplete append entries received. prev index:%" PRId64 ", term:%" PRId64 ", datalen:%d", @@ -184,10 +185,9 @@ _SEND_RESPONSE: // commit index, i.e. leader notice me if (syncLogBufferCommit(ths->pLogBuf, ths, ths->commitIndex) < 0) { sError("vgId:%d, failed to commit raft fsm log since %s.", ths->vgId, terrstr()); - goto _out; } -_out: + if (resetElect) syncNodeResetElectTimer(ths); return 0; _IGNORE: diff --git a/source/libs/sync/src/syncElection.c b/source/libs/sync/src/syncElection.c index e53b8ade1c..3699efbc59 100644 --- a/source/libs/sync/src/syncElection.c +++ b/source/libs/sync/src/syncElection.c @@ -115,6 +115,5 @@ int32_t syncNodeElect(SSyncNode* pSyncNode) { ASSERT(ret == 0); syncNodeResetElectTimer(pSyncNode); - return ret; } diff --git a/source/libs/sync/src/syncMain.c b/source/libs/sync/src/syncMain.c index e5fe4a2369..ed97c8abde 100644 --- a/source/libs/sync/src/syncMain.c +++ b/source/libs/sync/src/syncMain.c @@ -1593,8 +1593,8 @@ void syncNodeBecomeFollower(SSyncNode* pSyncNode, const char* debugStr) { pSyncNode->state = TAOS_SYNC_STATE_FOLLOWER; syncNodeStopHeartbeatTimer(pSyncNode); - // reset elect timer - syncNodeResetElectTimer(pSyncNode); + // trace log + sNTrace(pSyncNode, "become follower %s", debugStr); // send rsp to client syncNodeLeaderChangeRsp(pSyncNode); @@ -1610,8 +1610,8 @@ void syncNodeBecomeFollower(SSyncNode* pSyncNode, const char* debugStr) { // reset log buffer syncLogBufferReset(pSyncNode->pLogBuf, pSyncNode); - // trace log - sNTrace(pSyncNode, "become follower %s", debugStr); + // reset elect timer + syncNodeResetElectTimer(pSyncNode); } // TLA+ Spec @@ -2277,6 +2277,7 @@ static int32_t syncNodeAppendNoopOld(SSyncNode* ths) { int32_t syncNodeOnHeartbeat(SSyncNode* ths, const SRpcMsg* pRpcMsg) { SyncHeartbeat* pMsg = pRpcMsg->pCont; + bool resetElect = false; const STraceId* trace = &pRpcMsg->info.traceId; char tbuf[40] = {0}; @@ -2300,12 +2301,11 @@ int32_t syncNodeOnHeartbeat(SSyncNode* ths, const SRpcMsg* pRpcMsg) { if (pMsg->term == currentTerm && ths->state != TAOS_SYNC_STATE_LEADER) { syncIndexMgrSetRecvTime(ths->pNextIndex, &(pMsg->srcId), tsMs); + resetElect = true; - syncNodeResetElectTimer(ths); ths->minMatchIndex = pMsg->minMatchIndex; if (ths->state == TAOS_SYNC_STATE_FOLLOWER) { - // syncNodeFollowerCommit(ths, pMsg->commitIndex); SRpcMsg rpcMsgLocalCmd = {0}; (void)syncBuildLocalCmd(&rpcMsgLocalCmd, ths->vgId); @@ -2328,7 +2328,6 @@ int32_t syncNodeOnHeartbeat(SSyncNode* ths, const SRpcMsg* pRpcMsg) { } if (pMsg->term >= currentTerm && ths->state != TAOS_SYNC_STATE_FOLLOWER) { - // syncNodeStepDown(ths, pMsg->term); SRpcMsg rpcMsgLocalCmd = {0}; (void)syncBuildLocalCmd(&rpcMsgLocalCmd, ths->vgId); @@ -2348,15 +2347,10 @@ int32_t syncNodeOnHeartbeat(SSyncNode* ths, const SRpcMsg* pRpcMsg) { } } - /* - // htonl - SMsgHead* pHead = rpcMsg.pCont; - pHead->contLen = htonl(pHead->contLen); - pHead->vgId = htonl(pHead->vgId); - */ - // reply syncNodeSendMsgById(&pMsgReply->destId, ths, &rpcMsg); + + if (resetElect) syncNodeResetElectTimer(ths); return 0; } diff --git a/source/libs/sync/src/syncRequestVote.c b/source/libs/sync/src/syncRequestVote.c index 2fda2a19b8..40b747b711 100644 --- a/source/libs/sync/src/syncRequestVote.c +++ b/source/libs/sync/src/syncRequestVote.c @@ -89,6 +89,7 @@ static bool syncNodeOnRequestVoteLogOK(SSyncNode* ths, SyncRequestVote* pMsg) { int32_t syncNodeOnRequestVote(SSyncNode* ths, const SRpcMsg* pRpcMsg) { int32_t ret = 0; SyncRequestVote* pMsg = pRpcMsg->pCont; + bool resetElect = false; // if already drop replica, do not process if (!syncNodeInRaftGroup(ths, &pMsg->srcId)) { @@ -115,7 +116,7 @@ int32_t syncNodeOnRequestVote(SSyncNode* ths, const SRpcMsg* pRpcMsg) { syncNodeStepDown(ths, currentTerm); // forbid elect for this round - syncNodeResetElectTimer(ths); + resetElect = true; } // send msg @@ -134,5 +135,7 @@ int32_t syncNodeOnRequestVote(SSyncNode* ths, const SRpcMsg* pRpcMsg) { syncLogRecvRequestVote(ths, pMsg, pReply->voteGranted, ""); syncLogSendRequestVoteReply(ths, pReply, ""); syncNodeSendMsgById(&pReply->destId, ths, &rpcMsg); + + if (resetElect) syncNodeResetElectTimer(ths); return 0; } diff --git a/source/libs/sync/src/syncSnapshot.c b/source/libs/sync/src/syncSnapshot.c index 413d6adf05..a9d0ddad17 100644 --- a/source/libs/sync/src/syncSnapshot.c +++ b/source/libs/sync/src/syncSnapshot.c @@ -798,7 +798,6 @@ int32_t syncNodeOnSnapshot(SSyncNode *pSyncNode, const SRpcMsg *pRpcMsg) { if (pMsg->term > raftStoreGetTerm(pSyncNode)) { syncNodeStepDown(pSyncNode, pMsg->term); } - syncNodeResetElectTimer(pSyncNode); // state, term, seq/ack int32_t code = 0; @@ -840,6 +839,7 @@ int32_t syncNodeOnSnapshot(SSyncNode *pSyncNode, const SRpcMsg *pRpcMsg) { code = -1; } + syncNodeResetElectTimer(pSyncNode); return code; } From cc76d78673e50c04db8613001763bbb8bba2c369 Mon Sep 17 00:00:00 2001 From: Benguang Zhao Date: Tue, 14 Mar 2023 20:35:39 +0800 Subject: [PATCH 06/65] enh: separate sync read and write events --- include/common/tmsgcb.h | 2 +- source/dnode/mgmt/mgmt_mnode/inc/mmInt.h | 4 ++-- source/dnode/mgmt/mgmt_mnode/src/mmHandle.c | 17 +++++++++-------- source/dnode/mgmt/mgmt_mnode/src/mmWorker.c | 16 ++++++++-------- source/dnode/mgmt/mgmt_vnode/inc/vmInt.h | 4 ++-- source/dnode/mgmt/mgmt_vnode/src/vmHandle.c | 16 ++++++++-------- source/dnode/mgmt/mgmt_vnode/src/vmInt.c | 6 +++--- source/dnode/mgmt/mgmt_vnode/src/vmWorker.c | 20 +++++++++----------- source/dnode/mnode/impl/src/mndSync.c | 2 +- source/dnode/vnode/src/vnd/vnodeSync.c | 2 +- 10 files changed, 44 insertions(+), 45 deletions(-) diff --git a/include/common/tmsgcb.h b/include/common/tmsgcb.h index eca8740d28..9b709272b2 100644 --- a/include/common/tmsgcb.h +++ b/include/common/tmsgcb.h @@ -34,7 +34,7 @@ typedef enum { WRITE_QUEUE, APPLY_QUEUE, SYNC_QUEUE, - SYNC_CTRL_QUEUE, + SYNC_RD_QUEUE, STREAM_QUEUE, QUEUE_MAX, } EQueueType; diff --git a/source/dnode/mgmt/mgmt_mnode/inc/mmInt.h b/source/dnode/mgmt/mgmt_mnode/inc/mmInt.h index b47742b4ed..24f75e3c8b 100644 --- a/source/dnode/mgmt/mgmt_mnode/inc/mmInt.h +++ b/source/dnode/mgmt/mgmt_mnode/inc/mmInt.h @@ -34,7 +34,7 @@ typedef struct SMnodeMgmt { SSingleWorker readWorker; SSingleWorker writeWorker; SSingleWorker syncWorker; - SSingleWorker syncCtrlWorker; + SSingleWorker syncRdWorker; bool stopped; int32_t refCount; TdThreadRwlock lock; @@ -54,7 +54,7 @@ int32_t mmStartWorker(SMnodeMgmt *pMgmt); void mmStopWorker(SMnodeMgmt *pMgmt); int32_t mmPutMsgToWriteQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t mmPutMsgToSyncQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg); -int32_t mmPutMsgToSyncCtrlQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg); +int32_t mmPutMsgToSyncRdQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t mmPutMsgToReadQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t mmPutMsgToQueryQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t mmPutMsgToFetchQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg); diff --git a/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c b/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c index fe65c3dde9..c129ea21ac 100644 --- a/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c +++ b/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c @@ -188,21 +188,22 @@ SArray *mmGetMsgHandles() { if (dmSetMgmtHandle(pArray, TDMT_VND_DROP_INDEX_RSP, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_DISABLE_WRITE_RSP, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_SYNC_TIMEOUT, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_SYNC_HEARTBEAT, mmPutMsgToSyncRdQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_CLIENT_REQUEST, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_CLIENT_REQUEST_BATCH, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_SYNC_CLIENT_REQUEST_REPLY, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_REQUEST_VOTE, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_SYNC_REQUEST_VOTE_REPLY, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_APPEND_ENTRIES, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_APPEND_ENTRIES_BATCH, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_SYNC_APPEND_ENTRIES_REPLY, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_SNAPSHOT_SEND, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_SYNC_SNAPSHOT_RSP, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_PRE_SNAPSHOT, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_SYNC_PRE_SNAPSHOT_REPLY, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_SYNC_HEARTBEAT, mmPutMsgToSyncCtrlQueue, 1) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_SYNC_HEARTBEAT_REPLY, mmPutMsgToSyncCtrlQueue, 1) == NULL) goto _OVER; + + if (dmSetMgmtHandle(pArray, TDMT_SYNC_TIMEOUT, mmPutMsgToSyncRdQueue, 1) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_SYNC_REQUEST_VOTE_REPLY, mmPutMsgToSyncRdQueue, 1) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_SYNC_CLIENT_REQUEST_REPLY, mmPutMsgToSyncRdQueue, 1) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_SYNC_APPEND_ENTRIES_REPLY, mmPutMsgToSyncRdQueue, 1) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_SYNC_HEARTBEAT_REPLY, mmPutMsgToSyncRdQueue, 1) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_SYNC_SNAPSHOT_RSP, mmPutMsgToSyncRdQueue, 1) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_SYNC_PRE_SNAPSHOT_REPLY, mmPutMsgToSyncRdQueue, 1) == NULL) goto _OVER; code = 0; diff --git a/source/dnode/mgmt/mgmt_mnode/src/mmWorker.c b/source/dnode/mgmt/mgmt_mnode/src/mmWorker.c index b0810d528f..0152e5d0b1 100644 --- a/source/dnode/mgmt/mgmt_mnode/src/mmWorker.c +++ b/source/dnode/mgmt/mgmt_mnode/src/mmWorker.c @@ -111,8 +111,8 @@ int32_t mmPutMsgToSyncQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg) { return mmPutMsgToWorker(pMgmt, &pMgmt->syncWorker, pMsg); } -int32_t mmPutMsgToSyncCtrlQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg) { - return mmPutMsgToWorker(pMgmt, &pMgmt->syncCtrlWorker, pMsg); +int32_t mmPutMsgToSyncRdQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg) { + return mmPutMsgToWorker(pMgmt, &pMgmt->syncRdWorker, pMsg); } int32_t mmPutMsgToReadQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg) { @@ -151,8 +151,8 @@ int32_t mmPutMsgToQueue(SMnodeMgmt *pMgmt, EQueueType qtype, SRpcMsg *pRpc) { case SYNC_QUEUE: pWorker = &pMgmt->syncWorker; break; - case SYNC_CTRL_QUEUE: - pWorker = &pMgmt->syncCtrlWorker; + case SYNC_RD_QUEUE: + pWorker = &pMgmt->syncRdWorker; break; default: terrno = TSDB_CODE_INVALID_PARA; @@ -238,12 +238,12 @@ int32_t mmStartWorker(SMnodeMgmt *pMgmt) { SSingleWorkerCfg scCfg = { .min = 1, .max = 1, - .name = "mnode-sync-ctrl", + .name = "mnode-sync-rd", .fp = (FItem)mmProcessSyncMsg, .param = pMgmt, }; - if (tSingleWorkerInit(&pMgmt->syncCtrlWorker, &scCfg) != 0) { - dError("failed to start mnode mnode-sync-ctrl worker since %s", terrstr()); + if (tSingleWorkerInit(&pMgmt->syncRdWorker, &scCfg) != 0) { + dError("failed to start mnode mnode-sync-rd worker since %s", terrstr()); return -1; } @@ -259,6 +259,6 @@ void mmStopWorker(SMnodeMgmt *pMgmt) { tSingleWorkerCleanup(&pMgmt->readWorker); tSingleWorkerCleanup(&pMgmt->writeWorker); tSingleWorkerCleanup(&pMgmt->syncWorker); - tSingleWorkerCleanup(&pMgmt->syncCtrlWorker); + tSingleWorkerCleanup(&pMgmt->syncRdWorker); dDebug("mnode workers are closed"); } diff --git a/source/dnode/mgmt/mgmt_vnode/inc/vmInt.h b/source/dnode/mgmt/mgmt_vnode/inc/vmInt.h index e3fa2964b7..4374ae363c 100644 --- a/source/dnode/mgmt/mgmt_vnode/inc/vmInt.h +++ b/source/dnode/mgmt/mgmt_vnode/inc/vmInt.h @@ -59,7 +59,7 @@ typedef struct { SVnode *pImpl; SMultiWorker pWriteW; SMultiWorker pSyncW; - SMultiWorker pSyncCtrlW; + SMultiWorker pSyncRdW; SMultiWorker pApplyW; STaosQueue *pQueryQ; STaosQueue *pStreamQ; @@ -107,7 +107,7 @@ int32_t vmPutRpcMsgToQueue(SVnodeMgmt *pMgmt, EQueueType qtype, SRpcMsg *pRpc); int32_t vmPutMsgToWriteQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t vmPutMsgToSyncQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg); -int32_t vmPutMsgToSyncCtrlQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg); +int32_t vmPutMsgToSyncRdQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t vmPutMsgToQueryQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t vmPutMsgToFetchQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t vmPutMsgToStreamQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg); diff --git a/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c b/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c index d7f91b74a8..b60d2c3caf 100644 --- a/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c +++ b/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c @@ -549,22 +549,22 @@ SArray *vmGetMsgHandles() { if (dmSetMgmtHandle(pArray, TDMT_DND_CREATE_VNODE, vmPutMsgToMgmtQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_DND_DROP_VNODE, vmPutMsgToMgmtQueue, 0) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_SYNC_TIMEOUT, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_SYNC_HEARTBEAT, vmPutMsgToSyncRdQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_CLIENT_REQUEST, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_CLIENT_REQUEST_BATCH, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_SYNC_CLIENT_REQUEST_REPLY, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_REQUEST_VOTE, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_SYNC_REQUEST_VOTE_REPLY, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_APPEND_ENTRIES, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_APPEND_ENTRIES_BATCH, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_SYNC_APPEND_ENTRIES_REPLY, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_SNAPSHOT_SEND, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_SYNC_SNAPSHOT_RSP, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_PRE_SNAPSHOT, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_SYNC_PRE_SNAPSHOT_REPLY, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_SYNC_HEARTBEAT, vmPutMsgToSyncCtrlQueue, 0) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_SYNC_HEARTBEAT_REPLY, vmPutMsgToSyncCtrlQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_SYNC_TIMEOUT, vmPutMsgToSyncRdQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_SYNC_HEARTBEAT_REPLY, vmPutMsgToSyncRdQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_SYNC_CLIENT_REQUEST_REPLY, vmPutMsgToSyncRdQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_SYNC_APPEND_ENTRIES_REPLY, vmPutMsgToSyncRdQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_SYNC_REQUEST_VOTE_REPLY, vmPutMsgToSyncRdQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_SYNC_SNAPSHOT_RSP, vmPutMsgToSyncRdQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_SYNC_PRE_SNAPSHOT_REPLY, vmPutMsgToSyncRdQueue, 0) == NULL) goto _OVER; code = 0; diff --git a/source/dnode/mgmt/mgmt_vnode/src/vmInt.c b/source/dnode/mgmt/mgmt_vnode/src/vmInt.c index 0524e2713a..4c5b1246e7 100644 --- a/source/dnode/mgmt/mgmt_vnode/src/vmInt.c +++ b/source/dnode/mgmt/mgmt_vnode/src/vmInt.c @@ -98,9 +98,9 @@ void vmCloseVnode(SVnodeMgmt *pMgmt, SVnodeObj *pVnode, bool commitAndRemoveWal) pVnode->pSyncW.queue->threadId); tMultiWorkerCleanup(&pVnode->pSyncW); - dInfo("vgId:%d, wait for vnode sync ctrl queue:%p is empty, thread:%08" PRId64, pVnode->vgId, - pVnode->pSyncCtrlW.queue, pVnode->pSyncCtrlW.queue->threadId); - tMultiWorkerCleanup(&pVnode->pSyncCtrlW); + dInfo("vgId:%d, wait for vnode sync rd queue:%p is empty, thread:%08" PRId64, pVnode->vgId, pVnode->pSyncRdW.queue, + pVnode->pSyncRdW.queue->threadId); + tMultiWorkerCleanup(&pVnode->pSyncRdW); dInfo("vgId:%d, wait for vnode apply queue:%p is empty, thread:%08" PRId64, pVnode->vgId, pVnode->pApplyW.queue, pVnode->pApplyW.queue->threadId); diff --git a/source/dnode/mgmt/mgmt_vnode/src/vmWorker.c b/source/dnode/mgmt/mgmt_vnode/src/vmWorker.c index 7aa1c9f56a..e4e0d608de 100644 --- a/source/dnode/mgmt/mgmt_vnode/src/vmWorker.c +++ b/source/dnode/mgmt/mgmt_vnode/src/vmWorker.c @@ -216,9 +216,9 @@ static int32_t vmPutMsgToQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg, EQueueType qtyp dGTrace("vgId:%d, msg:%p put into vnode-sync queue", pVnode->vgId, pMsg); taosWriteQitem(pVnode->pSyncW.queue, pMsg); break; - case SYNC_CTRL_QUEUE: - dGTrace("vgId:%d, msg:%p put into vnode-sync-ctrl queue", pVnode->vgId, pMsg); - taosWriteQitem(pVnode->pSyncCtrlW.queue, pMsg); + case SYNC_RD_QUEUE: + dGTrace("vgId:%d, msg:%p put into vnode-sync-rd queue", pVnode->vgId, pMsg); + taosWriteQitem(pVnode->pSyncRdW.queue, pMsg); break; case APPLY_QUEUE: dGTrace("vgId:%d, msg:%p put into vnode-apply queue", pVnode->vgId, pMsg); @@ -234,9 +234,7 @@ static int32_t vmPutMsgToQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg, EQueueType qtyp return code; } -int32_t vmPutMsgToSyncCtrlQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg) { - return vmPutMsgToQueue(pMgmt, pMsg, SYNC_CTRL_QUEUE); -} +int32_t vmPutMsgToSyncRdQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg) { return vmPutMsgToQueue(pMgmt, pMsg, SYNC_RD_QUEUE); } int32_t vmPutMsgToSyncQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg) { return vmPutMsgToQueue(pMgmt, pMsg, SYNC_QUEUE); } @@ -327,18 +325,18 @@ int32_t vmGetQueueSize(SVnodeMgmt *pMgmt, int32_t vgId, EQueueType qtype) { int32_t vmAllocQueue(SVnodeMgmt *pMgmt, SVnodeObj *pVnode) { SMultiWorkerCfg wcfg = {.max = 1, .name = "vnode-write", .fp = (FItems)vnodeProposeWriteMsg, .param = pVnode->pImpl}; SMultiWorkerCfg scfg = {.max = 1, .name = "vnode-sync", .fp = (FItems)vmProcessSyncQueue, .param = pVnode}; - SMultiWorkerCfg sccfg = {.max = 1, .name = "vnode-sync-ctrl", .fp = (FItems)vmProcessSyncQueue, .param = pVnode}; + SMultiWorkerCfg sccfg = {.max = 1, .name = "vnode-sync-rd", .fp = (FItems)vmProcessSyncQueue, .param = pVnode}; SMultiWorkerCfg acfg = {.max = 1, .name = "vnode-apply", .fp = (FItems)vnodeApplyWriteMsg, .param = pVnode->pImpl}; (void)tMultiWorkerInit(&pVnode->pWriteW, &wcfg); (void)tMultiWorkerInit(&pVnode->pSyncW, &scfg); - (void)tMultiWorkerInit(&pVnode->pSyncCtrlW, &sccfg); + (void)tMultiWorkerInit(&pVnode->pSyncRdW, &sccfg); (void)tMultiWorkerInit(&pVnode->pApplyW, &acfg); pVnode->pQueryQ = tQWorkerAllocQueue(&pMgmt->queryPool, pVnode, (FItem)vmProcessQueryQueue); pVnode->pStreamQ = tAutoQWorkerAllocQueue(&pMgmt->streamPool, pVnode, (FItem)vmProcessStreamQueue); pVnode->pFetchQ = tWWorkerAllocQueue(&pMgmt->fetchPool, pVnode, (FItems)vmProcessFetchQueue); - if (pVnode->pWriteW.queue == NULL || pVnode->pSyncW.queue == NULL || pVnode->pSyncCtrlW.queue == NULL || + if (pVnode->pWriteW.queue == NULL || pVnode->pSyncW.queue == NULL || pVnode->pSyncRdW.queue == NULL || pVnode->pApplyW.queue == NULL || pVnode->pQueryQ == NULL || pVnode->pStreamQ == NULL || pVnode->pFetchQ == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; @@ -348,8 +346,8 @@ int32_t vmAllocQueue(SVnodeMgmt *pMgmt, SVnodeObj *pVnode) { pVnode->pWriteW.queue->threadId); dInfo("vgId:%d, sync-queue:%p is alloced, thread:%08" PRId64, pVnode->vgId, pVnode->pSyncW.queue, pVnode->pSyncW.queue->threadId); - dInfo("vgId:%d, sync-ctrl-queue:%p is alloced, thread:%08" PRId64, pVnode->vgId, pVnode->pSyncCtrlW.queue, - pVnode->pSyncCtrlW.queue->threadId); + dInfo("vgId:%d, sync-rd-queue:%p is alloced, thread:%08" PRId64, pVnode->vgId, pVnode->pSyncRdW.queue, + pVnode->pSyncRdW.queue->threadId); dInfo("vgId:%d, apply-queue:%p is alloced, thread:%08" PRId64, pVnode->vgId, pVnode->pApplyW.queue, pVnode->pApplyW.queue->threadId); dInfo("vgId:%d, query-queue:%p is alloced", pVnode->vgId, pVnode->pQueryQ); diff --git a/source/dnode/mnode/impl/src/mndSync.c b/source/dnode/mnode/impl/src/mndSync.c index 18548db56f..4965d5c34a 100644 --- a/source/dnode/mnode/impl/src/mndSync.c +++ b/source/dnode/mnode/impl/src/mndSync.c @@ -33,7 +33,7 @@ static int32_t mndSyncEqCtrlMsg(const SMsgCb *msgcb, SRpcMsg *pMsg) { return -1; } - int32_t code = tmsgPutToQueue(msgcb, SYNC_CTRL_QUEUE, pMsg); + int32_t code = tmsgPutToQueue(msgcb, SYNC_RD_QUEUE, pMsg); if (code != 0) { rpcFreeCont(pMsg->pCont); pMsg->pCont = NULL; diff --git a/source/dnode/vnode/src/vnd/vnodeSync.c b/source/dnode/vnode/src/vnd/vnodeSync.c index b49ca70bfa..d681f5b65e 100644 --- a/source/dnode/vnode/src/vnd/vnodeSync.c +++ b/source/dnode/vnode/src/vnd/vnodeSync.c @@ -378,7 +378,7 @@ static int32_t vnodeSyncEqCtrlMsg(const SMsgCb *msgcb, SRpcMsg *pMsg) { return -1; } - int32_t code = tmsgPutToQueue(msgcb, SYNC_CTRL_QUEUE, pMsg); + int32_t code = tmsgPutToQueue(msgcb, SYNC_RD_QUEUE, pMsg); if (code != 0) { rpcFreeCont(pMsg->pCont); pMsg->pCont = NULL; From 9f97162ef77956cfa3feb304d815e2867b60d589 Mon Sep 17 00:00:00 2001 From: Benguang Zhao Date: Tue, 14 Mar 2023 21:02:06 +0800 Subject: [PATCH 07/65] enh: separate election timer events --- include/common/tmsgdef.h | 2 +- source/dnode/mgmt/mgmt_mnode/src/mmHandle.c | 3 ++- source/dnode/mgmt/mgmt_vnode/src/vmHandle.c | 3 ++- source/libs/sync/src/syncMain.c | 3 +++ source/libs/sync/src/syncMessage.c | 4 ++-- source/libs/sync/src/syncTimeout.c | 5 ----- 6 files changed, 10 insertions(+), 10 deletions(-) diff --git a/include/common/tmsgdef.h b/include/common/tmsgdef.h index 46ca814e50..96d18d1abc 100644 --- a/include/common/tmsgdef.h +++ b/include/common/tmsgdef.h @@ -259,7 +259,7 @@ enum { TD_NEW_MSG_SEG(TDMT_SYNC_MSG) TD_DEF_MSG_TYPE(TDMT_SYNC_TIMEOUT, "sync-timer", NULL, NULL) - TD_DEF_MSG_TYPE(TDMT_SYNC_PING, "sync-ping", NULL, NULL) // no longer used + TD_DEF_MSG_TYPE(TDMT_SYNC_TIMEOUT_ELECTION, "sync-elect", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_SYNC_PING_REPLY, "sync-ping-reply", NULL, NULL) // no longer used TD_DEF_MSG_TYPE(TDMT_SYNC_CLIENT_REQUEST, "sync-client-request", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_SYNC_CLIENT_REQUEST_BATCH, "sync-client-request-batch", NULL, NULL) diff --git a/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c b/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c index c129ea21ac..ce485cb6ca 100644 --- a/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c +++ b/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c @@ -188,17 +188,18 @@ SArray *mmGetMsgHandles() { if (dmSetMgmtHandle(pArray, TDMT_VND_DROP_INDEX_RSP, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_DISABLE_WRITE_RSP, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_SYNC_TIMEOUT_ELECTION, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_HEARTBEAT, mmPutMsgToSyncRdQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_CLIENT_REQUEST, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_CLIENT_REQUEST_BATCH, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_REQUEST_VOTE, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_SYNC_REQUEST_VOTE_REPLY, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_APPEND_ENTRIES, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_APPEND_ENTRIES_BATCH, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_SNAPSHOT_SEND, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_PRE_SNAPSHOT, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_TIMEOUT, mmPutMsgToSyncRdQueue, 1) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_SYNC_REQUEST_VOTE_REPLY, mmPutMsgToSyncRdQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_CLIENT_REQUEST_REPLY, mmPutMsgToSyncRdQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_APPEND_ENTRIES_REPLY, mmPutMsgToSyncRdQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_HEARTBEAT_REPLY, mmPutMsgToSyncRdQueue, 1) == NULL) goto _OVER; diff --git a/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c b/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c index b60d2c3caf..84dfab7e6c 100644 --- a/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c +++ b/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c @@ -549,10 +549,12 @@ SArray *vmGetMsgHandles() { if (dmSetMgmtHandle(pArray, TDMT_DND_CREATE_VNODE, vmPutMsgToMgmtQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_DND_DROP_VNODE, vmPutMsgToMgmtQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_SYNC_TIMEOUT_ELECTION, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_HEARTBEAT, vmPutMsgToSyncRdQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_CLIENT_REQUEST, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_CLIENT_REQUEST_BATCH, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_REQUEST_VOTE, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_SYNC_REQUEST_VOTE_REPLY, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_APPEND_ENTRIES, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_APPEND_ENTRIES_BATCH, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_SNAPSHOT_SEND, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; @@ -562,7 +564,6 @@ SArray *vmGetMsgHandles() { if (dmSetMgmtHandle(pArray, TDMT_SYNC_HEARTBEAT_REPLY, vmPutMsgToSyncRdQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_CLIENT_REQUEST_REPLY, vmPutMsgToSyncRdQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_APPEND_ENTRIES_REPLY, vmPutMsgToSyncRdQueue, 0) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_SYNC_REQUEST_VOTE_REPLY, vmPutMsgToSyncRdQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_SNAPSHOT_RSP, vmPutMsgToSyncRdQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_PRE_SNAPSHOT_REPLY, vmPutMsgToSyncRdQueue, 0) == NULL) goto _OVER; diff --git a/source/libs/sync/src/syncMain.c b/source/libs/sync/src/syncMain.c index ed97c8abde..a617a5d293 100644 --- a/source/libs/sync/src/syncMain.c +++ b/source/libs/sync/src/syncMain.c @@ -182,6 +182,9 @@ int32_t syncProcessMsg(int64_t rid, SRpcMsg* pMsg) { case TDMT_SYNC_TIMEOUT: code = syncNodeOnTimeout(pSyncNode, pMsg); break; + case TDMT_SYNC_TIMEOUT_ELECTION: + code = syncNodeOnTimeout(pSyncNode, pMsg); + break; case TDMT_SYNC_CLIENT_REQUEST: code = syncNodeOnClientRequest(pSyncNode, pMsg, NULL); break; diff --git a/source/libs/sync/src/syncMessage.c b/source/libs/sync/src/syncMessage.c index 2a44588eef..72c8887803 100644 --- a/source/libs/sync/src/syncMessage.c +++ b/source/libs/sync/src/syncMessage.c @@ -22,7 +22,7 @@ int32_t syncBuildTimeout(SRpcMsg* pMsg, ESyncTimeoutType timeoutType, uint64_t l SSyncNode* pNode) { int32_t bytes = sizeof(SyncTimeout); pMsg->pCont = rpcMallocCont(bytes); - pMsg->msgType = TDMT_SYNC_TIMEOUT; + pMsg->msgType = (timeoutType == SYNC_TIMEOUT_ELECTION) ? TDMT_SYNC_TIMEOUT_ELECTION : TDMT_SYNC_TIMEOUT; pMsg->contLen = bytes; if (pMsg->pCont == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; @@ -31,7 +31,7 @@ int32_t syncBuildTimeout(SRpcMsg* pMsg, ESyncTimeoutType timeoutType, uint64_t l SyncTimeout* pTimeout = pMsg->pCont; pTimeout->bytes = bytes; - pTimeout->msgType = TDMT_SYNC_TIMEOUT; + pTimeout->msgType = pMsg->msgType; pTimeout->vgId = pNode->vgId; pTimeout->timeoutType = timeoutType; pTimeout->logicClock = logicClock; diff --git a/source/libs/sync/src/syncTimeout.c b/source/libs/sync/src/syncTimeout.c index 859183db95..a27be2853e 100644 --- a/source/libs/sync/src/syncTimeout.c +++ b/source/libs/sync/src/syncTimeout.c @@ -120,9 +120,6 @@ int32_t syncNodeOnTimeout(SSyncNode* ths, const SRpcMsg* pRpc) { if (atomic_load_64(&ths->pingTimerLogicClockUser) <= pMsg->logicClock) { ++(ths->pingTimerCounter); - // syncNodePingAll(ths); - // syncNodePingPeers(ths); - syncNodeTimerRoutine(ths); } @@ -138,8 +135,6 @@ int32_t syncNodeOnTimeout(SSyncNode* ths, const SRpcMsg* pRpc) { ++(ths->heartbeatTimerCounter); sTrace("vgId:%d, sync timer, type:replicate count:%" PRIu64 ", lc-user:%" PRIu64, ths->vgId, ths->heartbeatTimerCounter, ths->heartbeatTimerLogicClockUser); - - // syncNodeReplicate(ths, true); } } else { From 2185464836fc665cba783c511fb92a61d07ce78e Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao@163.com> Date: Sat, 18 Mar 2023 12:29:40 +0800 Subject: [PATCH 08/65] fix:scan of fill history ended prematurely --- source/libs/executor/src/executil.c | 2 +- source/libs/executor/src/scanoperator.c | 4 +++- source/libs/stream/src/streamExec.c | 10 ++++++++-- 3 files changed, 12 insertions(+), 4 deletions(-) diff --git a/source/libs/executor/src/executil.c b/source/libs/executor/src/executil.c index 7390cc1d26..53dd604fe9 100644 --- a/source/libs/executor/src/executil.c +++ b/source/libs/executor/src/executil.c @@ -2041,7 +2041,7 @@ int32_t createScanTableListInfo(SScanPhysiNode* pScanNode, SNodeList* pGroupTags void printDataBlock(SSDataBlock* pBlock, const char* flag) { if (!pBlock || pBlock->info.rows == 0) { - qDebug("===stream===printDataBlock: Block is Null or Empty"); + qDebug("===stream===%s: Block is Null or Empty", flag); return; } char* pBuf = NULL; diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 29a23cd90b..52bc5d4855 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -1798,6 +1798,7 @@ static SSDataBlock* doStreamScan(SOperatorInfo* pOperator) { generateScanRange(pInfo, pInfo->pUpdateDataRes, pInfo->pUpdateRes); prepareRangeScan(pInfo, pInfo->pUpdateRes, &pInfo->updateResIndex); pInfo->scanMode = STREAM_SCAN_FROM_DATAREADER_RANGE; + printDataBlock(pInfo->pUpdateRes, "recover update"); return pInfo->pUpdateRes; } break; case STREAM_SCAN_FROM_DATAREADER_RANGE: { @@ -1808,7 +1809,7 @@ static SSDataBlock* doStreamScan(SOperatorInfo* pOperator) { updateInfoSetScanRange(pInfo->pUpdateInfo, &pTableScanInfo->base.cond.twindows, pInfo->groupId, version); pSDB->info.type = pInfo->scanMode == STREAM_SCAN_FROM_DATAREADER_RANGE ? STREAM_NORMAL : STREAM_PULL_DATA; checkUpdateData(pInfo, true, pSDB, false); - // printDataBlock(pSDB, "stream scan update"); + printDataBlock(pSDB, "scan recover update"); calBlockTbName(pInfo, pSDB); return pSDB; } @@ -1833,6 +1834,7 @@ static SSDataBlock* doStreamScan(SOperatorInfo* pOperator) { } if (pInfo->pCreateTbRes->info.rows > 0) { pInfo->scanMode = STREAM_SCAN_FROM_RES; + printDataBlock(pInfo->pCreateTbRes, "recover createTbl"); return pInfo->pCreateTbRes; } qDebug("stream recover scan get block, rows %d", pInfo->pRecoverRes->info.rows); diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index cb610ad6b5..25b2656365 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -152,8 +152,14 @@ int32_t streamScanExec(SStreamTask* pTask, int32_t batchSz) { if (batchCnt >= batchSz) break; } if (taosArrayGetSize(pRes) == 0) { - taosArrayDestroy(pRes); - break; + if (finished) { + taosArrayDestroy(pRes); + qDebug("task %d finish recover exec task ", pTask->taskId); + break; + } else { + qDebug("task %d continue recover exec task ", pTask->taskId); + continue; + } } SStreamDataBlock* qRes = taosAllocateQitem(sizeof(SStreamDataBlock), DEF_QITEM, 0); if (qRes == NULL) { From a614ef777f76a85387b781f03d0de0c6472a1e86 Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao@163.com> Date: Tue, 21 Mar 2023 18:19:45 +0800 Subject: [PATCH 09/65] fix: modify pull linear --- source/libs/executor/inc/tfill.h | 4 +- source/libs/executor/src/filloperator.c | 91 ++++++++++++------------- 2 files changed, 47 insertions(+), 48 deletions(-) diff --git a/source/libs/executor/inc/tfill.h b/source/libs/executor/inc/tfill.h index fad4059515..726f0df1e8 100644 --- a/source/libs/executor/inc/tfill.h +++ b/source/libs/executor/inc/tfill.h @@ -92,8 +92,8 @@ typedef struct SResultRowData { typedef struct SStreamFillLinearInfo { TSKEY nextEnd; - SArray* pDeltaVal; // double. value for Fill(linear). - SArray* pNextDeltaVal; // double. value for Fill(linear). + SArray* pEndPoints; + SArray* pNextEndPoints; int64_t winIndex; bool hasNext; } SStreamFillLinearInfo; diff --git a/source/libs/executor/src/filloperator.c b/source/libs/executor/src/filloperator.c index 2a33e3527a..234f1a666c 100644 --- a/source/libs/executor/src/filloperator.c +++ b/source/libs/executor/src/filloperator.c @@ -447,9 +447,14 @@ void* destroyStreamFillSupporter(SStreamFillSupporter* pFillSup) { return NULL; } +void destroySPoint(void* ptr) { + SPoint* point = (SPoint*) ptr; + taosMemoryFreeClear(point->val); +} + void* destroyStreamFillLinearInfo(SStreamFillLinearInfo* pFillLinear) { - taosArrayDestroy(pFillLinear->pDeltaVal); - taosArrayDestroy(pFillLinear->pNextDeltaVal); + taosArrayDestroyEx(pFillLinear->pEndPoints, destroySPoint); + taosArrayDestroyEx(pFillLinear->pNextEndPoints, destroySPoint); taosMemoryFree(pFillLinear); return NULL; } @@ -611,19 +616,15 @@ static void calcDeltaData(SSDataBlock* pBlock, int32_t rowId, SResultRowData* pR } } -static void calcRowDeltaData(SResultRowData* pStartRow, SResultRowData* pEndRow, SArray* pDelta, SFillColInfo* pFillCol, - int32_t numOfCol, int32_t winCount) { +static void calcRowDeltaData(SResultRowData* pEndRow, SArray* pEndPoins, SFillColInfo* pFillCol, + int32_t numOfCol) { for (int32_t i = 0; i < numOfCol; i++) { if (!pFillCol[i].notFillCol) { int32_t slotId = GET_DEST_SLOT_ID(pFillCol + i); - SResultCellData* pSCell = getResultCell(pStartRow, slotId); - double start = 0.0; - GET_TYPED_DATA(start, double, pSCell->type, pSCell->pData); SResultCellData* pECell = getResultCell(pEndRow, slotId); - double end = 0.0; - GET_TYPED_DATA(end, double, pECell->type, pECell->pData); - double delta = (end - start) / winCount; - taosArraySet(pDelta, slotId, &delta); + SPoint* pPoint = taosArrayGet(pEndPoins, slotId); + pPoint->key = pEndRow->key; + memcpy(pPoint->val, pECell->pData, pECell->bytes); } } } @@ -674,10 +675,8 @@ void setDeleteFillValueInfo(TSKEY start, TSKEY end, SStreamFillSupporter* pFillS setFillKeyInfo(pFillSup->prev.key, pFillSup->next.key, &pFillSup->interval, pFillInfo); pFillInfo->pLinearInfo->hasNext = false; pFillInfo->pLinearInfo->nextEnd = INT64_MIN; - int32_t numOfWins = taosTimeCountInterval(pFillSup->prev.key, pFillSup->next.key, pFillSup->interval.sliding, - pFillSup->interval.slidingUnit, pFillSup->interval.precision); - calcRowDeltaData(&pFillSup->prev, &pFillSup->next, pFillInfo->pLinearInfo->pDeltaVal, pFillSup->pAllColInfo, - pFillSup->numOfAllCols, numOfWins); + calcRowDeltaData(&pFillSup->next, pFillInfo->pLinearInfo->pEndPoints, pFillSup->pAllColInfo, + pFillSup->numOfAllCols); pFillInfo->pResRow = &pFillSup->prev; pFillInfo->pLinearInfo->winIndex = 0; } break; @@ -780,25 +779,19 @@ void setFillValueInfo(SSDataBlock* pBlock, TSKEY ts, int32_t rowId, SStreamFillS setFillKeyInfo(prevWKey, ts, &pFillSup->interval, pFillInfo); pFillInfo->pos = FILL_POS_MID; pFillInfo->pLinearInfo->nextEnd = nextWKey; - int32_t numOfWins = taosTimeCountInterval(prevWKey, ts, pFillSup->interval.sliding, - pFillSup->interval.slidingUnit, pFillSup->interval.precision); - calcRowDeltaData(&pFillSup->prev, &pFillSup->cur, pFillInfo->pLinearInfo->pDeltaVal, pFillSup->pAllColInfo, - pFillSup->numOfAllCols, numOfWins); + calcRowDeltaData(&pFillSup->cur, pFillInfo->pLinearInfo->pEndPoints, pFillSup->pAllColInfo, + pFillSup->numOfAllCols); pFillInfo->pResRow = &pFillSup->prev; - numOfWins = taosTimeCountInterval(ts, nextWKey, pFillSup->interval.sliding, pFillSup->interval.slidingUnit, - pFillSup->interval.precision); - calcRowDeltaData(&pFillSup->cur, &pFillSup->next, pFillInfo->pLinearInfo->pNextDeltaVal, pFillSup->pAllColInfo, - pFillSup->numOfAllCols, numOfWins); + calcRowDeltaData(&pFillSup->next, pFillInfo->pLinearInfo->pNextEndPoints, pFillSup->pAllColInfo, + pFillSup->numOfAllCols); pFillInfo->pLinearInfo->hasNext = true; } else if (hasPrevWindow(pFillSup)) { setFillKeyInfo(prevWKey, ts, &pFillSup->interval, pFillInfo); pFillInfo->pos = FILL_POS_END; pFillInfo->pLinearInfo->nextEnd = INT64_MIN; - int32_t numOfWins = taosTimeCountInterval(prevWKey, ts, pFillSup->interval.sliding, - pFillSup->interval.slidingUnit, pFillSup->interval.precision); - calcRowDeltaData(&pFillSup->prev, &pFillSup->cur, pFillInfo->pLinearInfo->pDeltaVal, pFillSup->pAllColInfo, - pFillSup->numOfAllCols, numOfWins); + calcRowDeltaData(&pFillSup->cur, pFillInfo->pLinearInfo->pEndPoints, pFillSup->pAllColInfo, + pFillSup->numOfAllCols); pFillInfo->pResRow = &pFillSup->prev; pFillInfo->pLinearInfo->hasNext = false; } else { @@ -806,10 +799,8 @@ void setFillValueInfo(SSDataBlock* pBlock, TSKEY ts, int32_t rowId, SStreamFillS setFillKeyInfo(ts, nextWKey, &pFillSup->interval, pFillInfo); pFillInfo->pos = FILL_POS_START; pFillInfo->pLinearInfo->nextEnd = INT64_MIN; - int32_t numOfWins = taosTimeCountInterval(ts, nextWKey, pFillSup->interval.sliding, - pFillSup->interval.slidingUnit, pFillSup->interval.precision); - calcRowDeltaData(&pFillSup->cur, &pFillSup->next, pFillInfo->pLinearInfo->pDeltaVal, pFillSup->pAllColInfo, - pFillSup->numOfAllCols, numOfWins); + calcRowDeltaData(&pFillSup->next, pFillInfo->pLinearInfo->pEndPoints, pFillSup->pAllColInfo, + pFillSup->numOfAllCols); pFillInfo->pResRow = &pFillSup->cur; pFillInfo->pLinearInfo->hasNext = false; } @@ -906,13 +897,18 @@ static void doStreamFillLinear(SStreamFillSupporter* pFillSup, SStreamFillInfo* colDataSetNULL(pColData, index); continue; } - double* pDelta = taosArrayGet(pFillInfo->pLinearInfo->pDeltaVal, slotId); + SPoint* pEnd = taosArrayGet(pFillInfo->pLinearInfo->pEndPoints, slotId); double vCell = 0; - GET_TYPED_DATA(vCell, double, pCell->type, pCell->pData); - vCell += (*pDelta) * pFillInfo->pLinearInfo->winIndex; - int64_t result = 0; - SET_TYPED_DATA(&result, pCell->type, vCell); - colDataSetVal(pColData, index, (const char*)&result, false); + SPoint start = {0}; + start.key = pFillInfo->pResRow->key; + start.val = pCell->pData; + + SPoint cur = {0}; + cur.key = pFillInfo->current; + cur.val = taosMemoryCalloc(1, pCell->bytes); + taosGetLinearInterpolationVal(&cur, pCell->type, &start, pEnd, pCell->type); + colDataSetVal(pColData, index, (const char*)cur.val, false); + destroySPoint(&cur); } } pFillInfo->current = taosTimeAdd(pFillInfo->current, pFillSup->interval.sliding, pFillSup->interval.slidingUnit, @@ -953,8 +949,7 @@ static void doStreamFillRange(SStreamFillInfo* pFillInfo, SStreamFillSupporter* if (pFillInfo->current > pFillInfo->end && pFillInfo->pLinearInfo->hasNext) { pFillInfo->pLinearInfo->hasNext = false; pFillInfo->pLinearInfo->winIndex = 0; - taosArrayClear(pFillInfo->pLinearInfo->pDeltaVal); - taosArrayAddAll(pFillInfo->pLinearInfo->pDeltaVal, pFillInfo->pLinearInfo->pNextDeltaVal); + taosArraySwap(pFillInfo->pLinearInfo->pEndPoints, pFillInfo->pLinearInfo->pNextEndPoints); pFillInfo->pResRow = &pFillSup->cur; setFillKeyInfo(pFillSup->cur.key, pFillInfo->pLinearInfo->nextEnd, &pFillSup->interval, pFillInfo); doStreamFillLinear(pFillSup, pFillInfo, pRes); @@ -1359,15 +1354,19 @@ SStreamFillInfo* initStreamFillInfo(SStreamFillSupporter* pFillSup, SSDataBlock* pFillInfo->pLinearInfo = taosMemoryCalloc(1, sizeof(SStreamFillLinearInfo)); pFillInfo->pLinearInfo->hasNext = false; pFillInfo->pLinearInfo->nextEnd = INT64_MIN; - pFillInfo->pLinearInfo->pDeltaVal = NULL; - pFillInfo->pLinearInfo->pNextDeltaVal = NULL; + pFillInfo->pLinearInfo->pEndPoints = NULL; + pFillInfo->pLinearInfo->pNextEndPoints = NULL; if (pFillSup->type == TSDB_FILL_LINEAR) { - pFillInfo->pLinearInfo->pDeltaVal = taosArrayInit(pFillSup->numOfAllCols, sizeof(double)); - pFillInfo->pLinearInfo->pNextDeltaVal = taosArrayInit(pFillSup->numOfAllCols, sizeof(double)); + pFillInfo->pLinearInfo->pEndPoints = taosArrayInit(pFillSup->numOfAllCols, sizeof(SPoint)); + pFillInfo->pLinearInfo->pNextEndPoints = taosArrayInit(pFillSup->numOfAllCols, sizeof(SPoint)); for (int32_t i = 0; i < pFillSup->numOfAllCols; i++) { - double value = 0.0; - taosArrayPush(pFillInfo->pLinearInfo->pDeltaVal, &value); - taosArrayPush(pFillInfo->pLinearInfo->pNextDeltaVal, &value); + SColumnInfoData* pColData = taosArrayGet(pRes->pDataBlock, i); + SPoint value = {0}; + value.val = taosMemoryCalloc(1, pColData->info.bytes); + taosArrayPush(pFillInfo->pLinearInfo->pEndPoints, &value); + + value.val = taosMemoryCalloc(1, pColData->info.bytes); + taosArrayPush(pFillInfo->pLinearInfo->pNextEndPoints, &value); } } pFillInfo->pLinearInfo->winIndex = 0; From c3acab20705de933847574db3adef191b44a1c32 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 21 Mar 2023 19:38:34 +0800 Subject: [PATCH 10/65] fix(tmq): do some internal refactor. --- source/client/src/clientTmq.c | 3 +- source/dnode/vnode/src/inc/tq.h | 27 ++-- source/dnode/vnode/src/tq/tq.c | 202 +++++++++++++++-------------- source/dnode/vnode/src/tq/tqPush.c | 6 +- 4 files changed, 120 insertions(+), 118 deletions(-) diff --git a/source/client/src/clientTmq.c b/source/client/src/clientTmq.c index 77bbd0be1a..82da067d8e 100644 --- a/source/client/src/clientTmq.c +++ b/source/client/src/clientTmq.c @@ -1811,7 +1811,6 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout, bool pollIfReset) { if (pRspWrapper == NULL) { taosReadAllQitems(tmq->mqueue, tmq->qall); taosGetQitem(tmq->qall, (void**)&pRspWrapper); - if (pRspWrapper == NULL) { return NULL; } @@ -1831,7 +1830,6 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout, bool pollIfReset) { SMqDataRsp* pDataRsp = &pollRspWrapper->dataRsp; if (pDataRsp->head.epoch == consumerEpoch) { - // todo fix it: race condition SMqClientVg* pVg = pollRspWrapper->vgHandle; // update the epset @@ -1843,6 +1841,7 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout, bool pollIfReset) { pVg->epSet = *pollRspWrapper->pEpset; } + // update the local offset value only for the returned values. pVg->currentOffset = pDataRsp->rspOffset; atomic_store_32(&pVg->vgStatus, TMQ_VG_STATUS__IDLE); diff --git a/source/dnode/vnode/src/inc/tq.h b/source/dnode/vnode/src/inc/tq.h index c2b38f5cd1..379ea25ee6 100644 --- a/source/dnode/vnode/src/inc/tq.h +++ b/source/dnode/vnode/src/inc/tq.h @@ -109,23 +109,18 @@ typedef struct { } STqPushEntry; struct STQ { - SVnode* pVnode; - char* path; - int64_t walLogLastVer; - - SRWLatch pushLock; - - SHashObj* pPushMgr; // consumerId -> STqPushEntry - SHashObj* pHandle; // subKey -> STqHandle - SHashObj* pCheckInfo; // topic -> SAlterCheckInfo - + SVnode* pVnode; + char* path; + int64_t walLogLastVer; + SRWLatch lock; + SHashObj* pPushMgr; // consumerId -> STqPushEntry + SHashObj* pHandle; // subKey -> STqHandle + SHashObj* pCheckInfo; // topic -> SAlterCheckInfo STqOffsetStore* pOffsetStore; - - TDB* pMetaDB; - TTB* pExecStore; - TTB* pCheckStore; - - SStreamMeta* pStreamMeta; + TDB* pMetaDB; + TTB* pExecStore; + TTB* pCheckStore; + SStreamMeta* pStreamMeta; }; typedef struct { diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index c4800e5051..1282b0a94d 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -51,7 +51,7 @@ void tqCleanUp() { } } -static void destroySTqHandle(void* data) { +static void destroyTqHandle(void* data) { STqHandle* pData = (STqHandle*)data; qDestroyTask(pData->execHandle.task); if (pData->execHandle.subType == TOPIC_SUB_TYPE__COLUMN) { @@ -89,9 +89,9 @@ STQ* tqOpen(const char* path, SVnode* pVnode) { pTq->walLogLastVer = pVnode->pWal->vers.lastVer; pTq->pHandle = taosHashInit(64, MurmurHash3_32, true, HASH_ENTRY_LOCK); - taosHashSetFreeFp(pTq->pHandle, destroySTqHandle); + taosHashSetFreeFp(pTq->pHandle, destroyTqHandle); - taosInitRWLatch(&pTq->pushLock); + taosInitRWLatch(&pTq->lock); pTq->pPushMgr = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT), true, HASH_NO_LOCK); taosHashSetFreeFp(pTq->pPushMgr, tqPushEntryFree); @@ -236,38 +236,6 @@ int32_t tqPushDataRsp(STQ* pTq, STqPushEntry* pPushEntry) { } #endif -// int32_t len = 0; -// int32_t code = 0; -// tEncodeSize(tEncodeSMqDataRsp, pRsp, len, code); -// if (code < 0) { -// return -1; -// } -// -// int32_t tlen = sizeof(SMqRspHead) + len; -// void* buf = rpcMallocCont(tlen); -// if (buf == NULL) { -// return -1; -// } -// -// memcpy(buf, &pPushEntry->dataRsp.head, sizeof(SMqRspHead)); -// -// void* abuf = POINTER_SHIFT(buf, sizeof(SMqRspHead)); -// -// SEncoder encoder = {0}; -// tEncoderInit(&encoder, abuf, len); -// tEncodeSMqDataRsp(&encoder, pRsp); -// tEncoderClear(&encoder); -// -// SRpcMsg rsp = { -// .info = pPushEntry->pInfo, -// .pCont = buf, -// .contLen = tlen, -// .code = 0, -// }; -// -// tmsgSendRsp(&rsp); -// - SMqRspHead* pHeader = &pPushEntry->pDataRsp->head; doSendDataRsp(&pPushEntry->info, pRsp, pHeader->epoch, pHeader->consumerId, pHeader->mqMsgType); @@ -444,7 +412,7 @@ static int32_t extractResetOffsetVal(STqOffsetVal* pOffsetVal, STQ* pTq, STqHand char formatBuf[80]; tFormatOffset(formatBuf, 80, pOffsetVal); - tqDebug("tmq poll: consumer:0x%" PRIx64 ", subkey %s, vgId:%d, prev offset found, offset reset to %s and continue.", + tqDebug("tmq poll: consumer:0x%" PRIx64 ", subkey %s, vgId:%d, existed offset found, offset reset to %s and continue.", consumerId, pHandle->subKey, vgId, formatBuf); return 0; } else { @@ -502,7 +470,45 @@ static int32_t extractResetOffsetVal(STqOffsetVal* pOffsetVal, STQ* pTq, STqHand return 0; } -static int32_t extractDataForMq(STQ* pTq, STqHandle* pHandle, const SMqPollReq* pRequest, SRpcMsg* pMsg) { +#define IS_OFFSET_RESET_TYPE(_t) ((_t) < 0) + +static int32_t extractDataAndRspForNormalSubscribe(STQ* pTq, STqHandle* pHandle, const SMqPollReq* pRequest, + SRpcMsg* pMsg, STqOffsetVal* pOffset) { + int32_t code = 0; + uint64_t consumerId = pRequest->consumerId; + int32_t vgId = TD_VID(pTq->pVnode); + + SMqDataRsp dataRsp = {0}; + tqInitDataRsp(&dataRsp, pRequest, pHandle->execHandle.subType); + + // lock + taosWLockLatch(&pTq->lock); + + qSetTaskId(pHandle->execHandle.task, consumerId, pRequest->reqId); + code = tqScanData(pTq, pHandle, &dataRsp, pOffset); + + // till now, all data has been transferred to consumer, new data needs to push client once arrived. + if (dataRsp.blockNum == 0 && dataRsp.reqOffset.type == TMQ_OFFSET__LOG && + dataRsp.reqOffset.version == dataRsp.rspOffset.version && pHandle->consumerId == pRequest->consumerId) { + code = tqRegisterPushEntry(pTq, pHandle, pRequest, pMsg, &dataRsp, TMQ_MSG_TYPE__POLL_RSP); + taosWUnLockLatch(&pTq->lock); + return code; + } + + taosWUnLockLatch(&pTq->lock); + code = tqSendDataRsp(pTq, pMsg, pRequest, (SMqDataRsp*)&dataRsp, TMQ_MSG_TYPE__POLL_RSP); + + // NOTE: this pHandle->consumerId may have been changed already. + tqDebug("tmq poll: consumer:0x%" PRIx64 ", subkey %s, vgId:%d, rsp block:%d, offset type:%d, uid/version:%" PRId64 + ", ts:%" PRId64 ", reqId:0x%" PRIx64, + consumerId, pHandle->subKey, vgId, dataRsp.blockNum, dataRsp.rspOffset.type, dataRsp.rspOffset.uid, + dataRsp.rspOffset.ts, pRequest->reqId); + + tDeleteSMqDataRsp(&dataRsp); + return code; +} + +static int32_t doPollDataForMq(STQ* pTq, STqHandle* pHandle, const SMqPollReq* pRequest, SRpcMsg* pMsg) { int32_t code = -1; STqOffsetVal offset = {0}; SWalCkHead* pCkHead = NULL; @@ -512,9 +518,8 @@ static int32_t extractDataForMq(STQ* pTq, STqHandle* pHandle, const SMqPollReq* uint64_t consumerId = pRequest->consumerId; // 1. reset the offset if needed - if (reqOffset.type > 0) { - offset = reqOffset; - } else { // handle the reset offset cases, according to the consumer's choice. + if (IS_OFFSET_RESET_TYPE(reqOffset.type)) { + // handle the reset offset cases, according to the consumer's choice. bool blockReturned = false; code = extractResetOffsetVal(&offset, pTq, pHandle, pRequest, pMsg, &blockReturned); if (code != 0) { @@ -525,38 +530,41 @@ static int32_t extractDataForMq(STQ* pTq, STqHandle* pHandle, const SMqPollReq* if (blockReturned) { return 0; } + } else { // use the consumer specified offset + offset = reqOffset; } - // this is a normal subscription requirement + // this is a normal subscribe requirement if (pHandle->execHandle.subType == TOPIC_SUB_TYPE__COLUMN) { - SMqDataRsp dataRsp = {0}; - tqInitDataRsp(&dataRsp, pRequest, pHandle->execHandle.subType); - - // lock - taosWLockLatch(&pTq->pushLock); - - qSetTaskId(pHandle->execHandle.task, consumerId, pRequest->reqId); - code = tqScanData(pTq, pHandle, &dataRsp, &offset); - - // till now, all data has been transferred to consumer, new data needs to push client once arrived. - if (dataRsp.blockNum == 0 && dataRsp.reqOffset.type == TMQ_OFFSET__LOG && - dataRsp.reqOffset.version == dataRsp.rspOffset.version && pHandle->consumerId == pRequest->consumerId) { - code = tqRegisterPushEntry(pTq, pHandle, pRequest, pMsg, &dataRsp, TMQ_MSG_TYPE__POLL_RSP); - taosWUnLockLatch(&pTq->pushLock); - return code; - } - - taosWUnLockLatch(&pTq->pushLock); - code = tqSendDataRsp(pTq, pMsg, pRequest, (SMqDataRsp*)&dataRsp, TMQ_MSG_TYPE__POLL_RSP); - - // NOTE: this pHandle->consumerId may have been changed already. - tqDebug("tmq poll: consumer:0x%" PRIx64 ", subkey %s, vgId:%d, rsp block:%d, offset type:%d, uid/version:%" PRId64 - ", ts:%" PRId64", reqId:0x%"PRIx64, - consumerId, pHandle->subKey, vgId, dataRsp.blockNum, dataRsp.rspOffset.type, dataRsp.rspOffset.uid, - dataRsp.rspOffset.ts, pRequest->reqId); - - tDeleteSMqDataRsp(&dataRsp); - return code; + return extractDataAndRspForNormalSubscribe(pTq, pHandle, pRequest, pMsg, &offset); +// SMqDataRsp dataRsp = {0}; +// tqInitDataRsp(&dataRsp, pRequest, pHandle->execHandle.subType); +// +// // lock +// taosWLockLatch(&pTq->lock); +// +// qSetTaskId(pHandle->execHandle.task, consumerId, pRequest->reqId); +// code = tqScanData(pTq, pHandle, &dataRsp, &offset); +// +// // till now, all data has been transferred to consumer, new data needs to push client once arrived. +// if (dataRsp.blockNum == 0 && dataRsp.reqOffset.type == TMQ_OFFSET__LOG && +// dataRsp.reqOffset.version == dataRsp.rspOffset.version && pHandle->consumerId == pRequest->consumerId) { +// code = tqRegisterPushEntry(pTq, pHandle, pRequest, pMsg, &dataRsp, TMQ_MSG_TYPE__POLL_RSP); +// taosWUnLockLatch(&pTq->lock); +// return code; +// } +// +// taosWUnLockLatch(&pTq->lock); +// code = tqSendDataRsp(pTq, pMsg, pRequest, (SMqDataRsp*)&dataRsp, TMQ_MSG_TYPE__POLL_RSP); +// +// // NOTE: this pHandle->consumerId may have been changed already. +// tqDebug("tmq poll: consumer:0x%" PRIx64 ", subkey %s, vgId:%d, rsp block:%d, offset type:%d, uid/version:%" PRId64 +// ", ts:%" PRId64", reqId:0x%"PRIx64, +// consumerId, pHandle->subKey, vgId, dataRsp.blockNum, dataRsp.rspOffset.type, dataRsp.rspOffset.uid, +// dataRsp.rspOffset.ts, pRequest->reqId); +// +// tDeleteSMqDataRsp(&dataRsp); +// return code; } // todo handle the case where re-balance occurs. @@ -700,31 +708,31 @@ int32_t tqProcessPollReq(STQ* pTq, SRpcMsg* pMsg) { } // 2. check re-balance status - taosRLockLatch(&pTq->pushLock); + taosRLockLatch(&pTq->lock); if (pHandle->consumerId != consumerId) { tqDebug("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; - taosRUnLockLatch(&pTq->pushLock); + taosRUnLockLatch(&pTq->lock); return -1; } - taosRUnLockLatch(&pTq->pushLock); + taosRUnLockLatch(&pTq->lock); - taosWLockLatch(&pTq->pushLock); // 3. update the epoch value + taosWLockLatch(&pTq->lock); int32_t savedEpoch = pHandle->epoch; if (savedEpoch < reqEpoch) { tqDebug("tmq poll: consumer:0x%" PRIx64 " epoch update from %d to %d by poll req", consumerId, savedEpoch, reqEpoch); pHandle->epoch = reqEpoch; } - taosWUnLockLatch(&pTq->pushLock); + taosWUnLockLatch(&pTq->lock); char buf[80]; tFormatOffset(buf, 80, &reqOffset); tqDebug("tmq poll: consumer:0x%" PRIx64 " (epoch %d), subkey %s, recv poll req vgId:%d, req:%s, reqId:0x%" PRIx64, consumerId, req.epoch, pHandle->subKey, vgId, buf, req.reqId); - return extractDataForMq(pTq, pHandle, &req, pMsg); + return doPollDataForMq(pTq, pHandle, &req, pMsg); } int32_t tqProcessDeleteSubReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen) { @@ -732,12 +740,12 @@ int32_t tqProcessDeleteSubReq(STQ* pTq, int64_t sversion, char* msg, int32_t msg tqDebug("vgId:%d, tq process delete sub req %s", pTq->pVnode->config.vgId, pReq->subKey); - taosWLockLatch(&pTq->pushLock); + taosWLockLatch(&pTq->lock); int32_t code = taosHashRemove(pTq->pPushMgr, pReq->subKey, strlen(pReq->subKey)); if (code != 0) { tqDebug("vgId:%d, tq remove push handle %s", pTq->pVnode->config.vgId, pReq->subKey); } - taosWUnLockLatch(&pTq->pushLock); + taosWUnLockLatch(&pTq->lock); STqHandle* pHandle = taosHashGet(pTq->pHandle, pReq->subKey, strlen(pReq->subKey)); if (pHandle) { @@ -801,18 +809,18 @@ int32_t tqProcessSubscribeReq(STQ* pTq, int64_t sversion, char* msg, int32_t msg SVnode* pVnode = pTq->pVnode; int32_t vgId = TD_VID(pVnode); - tqDebug("vgId:%d, tq process sub req %s, Id:0x%" PRIx64 " -> Id:0x%" PRIx64, pVnode->config.vgId, req.subKey, + tqDebug("vgId:%d, tq process sub req:%s, Id:0x%" PRIx64 " -> Id:0x%" PRIx64, pVnode->config.vgId, req.subKey, req.oldConsumerId, req.newConsumerId); STqHandle* pHandle = taosHashGet(pTq->pHandle, req.subKey, strlen(req.subKey)); if (pHandle == NULL) { if (req.oldConsumerId != -1) { - tqError("vgId:%d, build new consumer handle %s for consumer:0x%" PRIx64 ", but old consumerId is %" PRId64 "", + tqError("vgId:%d, build new consumer handle %s for consumer:0x%" PRIx64 ", but old consumerId:0x%" PRIx64, req.vgId, req.subKey, req.newConsumerId, req.oldConsumerId); } if (req.newConsumerId == -1) { - tqError("vgId:%d, tq invalid rebalance request, new consumerId %" PRId64 "", req.vgId, req.newConsumerId); + tqError("vgId:%d, tq invalid re-balance request, new consumerId %" PRId64 "", req.vgId, req.newConsumerId); taosMemoryFree(req.qmsg); return 0; } @@ -902,28 +910,28 @@ int32_t tqProcessSubscribeReq(STQ* pTq, int64_t sversion, char* msg, int32_t msg atomic_add_fetch_32(&pHandle->epoch, 1); taosMemoryFree(req.qmsg); return tqMetaSaveHandle(pTq, req.subKey, pHandle); - } + } else { + tqInfo("vgId:%d switch consumer from Id:0x%" PRIx64 " to Id:0x%" PRIx64, req.vgId, pHandle->consumerId, + req.newConsumerId); - tqInfo("vgId:%d switch consumer from Id:0x%" PRIx64 " to Id:0x%" PRIx64, req.vgId, pHandle->consumerId, - req.newConsumerId); + taosWLockLatch(&pTq->lock); + atomic_store_32(&pHandle->epoch, -1); - taosWLockLatch(&pTq->pushLock); - atomic_store_32(&pHandle->epoch, -1); + // remove if it has been register in the push manager, and return one empty block to consumer + tqRemovePushEntry(pTq, req.subKey, (int32_t)strlen(req.subKey), pHandle->consumerId, true); - // remove if it has been register in the push manager, and return one empty block to consumer - tqRemovePushEntry(pTq, req.subKey, (int32_t) strlen(req.subKey), pHandle->consumerId, true); + atomic_store_64(&pHandle->consumerId, req.newConsumerId); + atomic_add_fetch_32(&pHandle->epoch, 1); - atomic_store_64(&pHandle->consumerId, req.newConsumerId); - atomic_add_fetch_32(&pHandle->epoch, 1); + if (pHandle->execHandle.subType == TOPIC_SUB_TYPE__COLUMN) { + qStreamCloseTsdbReader(pHandle->execHandle.task); + } - if (pHandle->execHandle.subType == TOPIC_SUB_TYPE__COLUMN) { - qStreamCloseTsdbReader(pHandle->execHandle.task); - } - - taosWUnLockLatch(&pTq->pushLock); - if (tqMetaSaveHandle(pTq, req.subKey, pHandle) < 0) { - taosMemoryFree(req.qmsg); - return -1; + taosWUnLockLatch(&pTq->lock); + if (tqMetaSaveHandle(pTq, req.subKey, pHandle) < 0) { + taosMemoryFree(req.qmsg); + return -1; + } } } diff --git a/source/dnode/vnode/src/tq/tqPush.c b/source/dnode/vnode/src/tq/tqPush.c index 01d8e7cf14..797aeb3f04 100644 --- a/source/dnode/vnode/src/tq/tqPush.c +++ b/source/dnode/vnode/src/tq/tqPush.c @@ -213,7 +213,7 @@ int tqPushMsg(STQ* pTq, void* msg, int32_t msgLen, tmsg_t msgType, int64_t ver) if (msgType == TDMT_VND_SUBMIT) { // lock push mgr to avoid potential msg lost - taosWLockLatch(&pTq->pushLock); + taosWLockLatch(&pTq->lock); int32_t numOfRegisteredPush = taosHashGetSize(pTq->pPushMgr); if (numOfRegisteredPush > 0) { @@ -231,7 +231,7 @@ int tqPushMsg(STQ* pTq, void* msg, int32_t msgLen, tmsg_t msgType, int64_t ver) taosArrayDestroy(cachedKeyLens); // unlock - taosWUnLockLatch(&pTq->pushLock); + taosWUnLockLatch(&pTq->lock); return -1; } @@ -320,7 +320,7 @@ int tqPushMsg(STQ* pTq, void* msg, int32_t msgLen, tmsg_t msgType, int64_t ver) taosMemoryFree(data); } // unlock - taosWUnLockLatch(&pTq->pushLock); + taosWUnLockLatch(&pTq->lock); } if (!tsDisableStream && vnodeIsRoleLeader(pTq->pVnode)) { From 69ca2b2f7b741bf30702b4b53ae86d7ed94d66fa Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 21 Mar 2023 19:40:36 +0800 Subject: [PATCH 11/65] fix(tmq): do some internal refactor. --- source/dnode/vnode/src/tq/tq.c | 214 ++++++++++++++------------------- 1 file changed, 92 insertions(+), 122 deletions(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 1282b0a94d..8279ee7aeb 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -531,159 +531,129 @@ static int32_t doPollDataForMq(STQ* pTq, STqHandle* pHandle, const SMqPollReq* p return 0; } } else { // use the consumer specified offset + // the offset value can not be monotonious increase?? offset = reqOffset; } // this is a normal subscribe requirement if (pHandle->execHandle.subType == TOPIC_SUB_TYPE__COLUMN) { return extractDataAndRspForNormalSubscribe(pTq, pHandle, pRequest, pMsg, &offset); -// SMqDataRsp dataRsp = {0}; -// tqInitDataRsp(&dataRsp, pRequest, pHandle->execHandle.subType); -// -// // lock -// taosWLockLatch(&pTq->lock); -// -// qSetTaskId(pHandle->execHandle.task, consumerId, pRequest->reqId); -// code = tqScanData(pTq, pHandle, &dataRsp, &offset); -// -// // till now, all data has been transferred to consumer, new data needs to push client once arrived. -// if (dataRsp.blockNum == 0 && dataRsp.reqOffset.type == TMQ_OFFSET__LOG && -// dataRsp.reqOffset.version == dataRsp.rspOffset.version && pHandle->consumerId == pRequest->consumerId) { -// code = tqRegisterPushEntry(pTq, pHandle, pRequest, pMsg, &dataRsp, TMQ_MSG_TYPE__POLL_RSP); -// taosWUnLockLatch(&pTq->lock); -// return code; -// } -// -// taosWUnLockLatch(&pTq->lock); -// code = tqSendDataRsp(pTq, pMsg, pRequest, (SMqDataRsp*)&dataRsp, TMQ_MSG_TYPE__POLL_RSP); -// -// // NOTE: this pHandle->consumerId may have been changed already. -// tqDebug("tmq poll: consumer:0x%" PRIx64 ", subkey %s, vgId:%d, rsp block:%d, offset type:%d, uid/version:%" PRId64 -// ", ts:%" PRId64", reqId:0x%"PRIx64, -// consumerId, pHandle->subKey, vgId, dataRsp.blockNum, dataRsp.rspOffset.type, dataRsp.rspOffset.uid, -// dataRsp.rspOffset.ts, pRequest->reqId); -// -// tDeleteSMqDataRsp(&dataRsp); -// return code; - } + } else { // for taosX + // todo handle the case where re-balance occurs. + SMqMetaRsp metaRsp = {0}; + STaosxRsp taosxRsp = {0}; + tqInitTaosxRsp(&taosxRsp, pRequest); - // todo handle the case where re-balance occurs. - // for taosx - SMqMetaRsp metaRsp = {0}; - STaosxRsp taosxRsp = {0}; - tqInitTaosxRsp(&taosxRsp, pRequest); - - if (offset.type != TMQ_OFFSET__LOG) { - if (tqScanTaosx(pTq, pHandle, &taosxRsp, &metaRsp, &offset) < 0) { - return -1; - } - - if (metaRsp.metaRspLen > 0) { - code = tqSendMetaPollRsp(pTq, pMsg, pRequest, &metaRsp); - tqDebug("tmq poll: consumer:0x%" PRIx64 " subkey:%s vgId:%d, send meta offset type:%d,uid:%" PRId64 - ",ts:%" PRId64, - consumerId, pHandle->subKey, vgId, metaRsp.rspOffset.type, metaRsp.rspOffset.uid, - metaRsp.rspOffset.ts); - taosMemoryFree(metaRsp.metaRsp); - tDeleteSTaosxRsp(&taosxRsp); - return code; - } - - if (taosxRsp.blockNum > 0) { - code = tqSendDataRsp(pTq, pMsg, pRequest, (SMqDataRsp*)&taosxRsp, TMQ_MSG_TYPE__TAOSX_RSP); - tDeleteSTaosxRsp(&taosxRsp); - return code; - } else { - offset = taosxRsp.rspOffset; - } - - tqDebug("taosx poll: consumer:0x%" PRIx64 " subkey:%s vgId:%d, send data blockNum:%d, offset type:%d,uid:%" PRId64 - ",version:%" PRId64, - consumerId, pHandle->subKey, vgId, taosxRsp.blockNum, taosxRsp.rspOffset.type, taosxRsp.rspOffset.uid, - taosxRsp.rspOffset.version); - } else { - -// if (offset.type == TMQ_OFFSET__LOG) { - int64_t fetchVer = offset.version + 1; - pCkHead = taosMemoryMalloc(sizeof(SWalCkHead) + 2048); - if (pCkHead == NULL) { - tDeleteSTaosxRsp(&taosxRsp); - terrno = TSDB_CODE_OUT_OF_MEMORY; - return -1; - } - - walSetReaderCapacity(pHandle->pWalReader, 2048); - - while (1) { - // todo refactor: this is not correct. - int32_t savedEpoch = atomic_load_32(&pHandle->epoch); - if (savedEpoch > pRequest->epoch) { - tqWarn("tmq poll: consumer:0x%" PRIx64 " (epoch %d), subkey:%s vgId:%d offset %" PRId64 - ", found new consumer epoch %d, discard req epoch %d", - consumerId, pRequest->epoch, pHandle->subKey, vgId, fetchVer, savedEpoch, pRequest->epoch); - break; + if (offset.type != TMQ_OFFSET__LOG) { + if (tqScanTaosx(pTq, pHandle, &taosxRsp, &metaRsp, &offset) < 0) { + return -1; } - if (tqFetchLog(pTq, pHandle, &fetchVer, &pCkHead, pRequest->reqId) < 0) { - tqOffsetResetToLog(&taosxRsp.rspOffset, fetchVer); - code = tqSendDataRsp(pTq, pMsg, pRequest, (SMqDataRsp*)&taosxRsp, TMQ_MSG_TYPE__TAOSX_RSP); + if (metaRsp.metaRspLen > 0) { + code = tqSendMetaPollRsp(pTq, pMsg, pRequest, &metaRsp); + tqDebug("tmq poll: consumer:0x%" PRIx64 " subkey:%s vgId:%d, send meta offset type:%d,uid:%" PRId64 + ",ts:%" PRId64, + consumerId, pHandle->subKey, vgId, metaRsp.rspOffset.type, metaRsp.rspOffset.uid, metaRsp.rspOffset.ts); + taosMemoryFree(metaRsp.metaRsp); tDeleteSTaosxRsp(&taosxRsp); - taosMemoryFreeClear(pCkHead); return code; } - SWalCont* pHead = &pCkHead->head; - tqDebug("tmq poll: consumer:0x%" PRIx64 " (epoch %d) iter log, vgId:%d offset %" PRId64 " msgType %d", consumerId, - pRequest->epoch, vgId, fetchVer, pHead->msgType); + if (taosxRsp.blockNum > 0) { + code = tqSendDataRsp(pTq, pMsg, pRequest, (SMqDataRsp*)&taosxRsp, TMQ_MSG_TYPE__TAOSX_RSP); + tDeleteSTaosxRsp(&taosxRsp); + return code; + } else { + offset = taosxRsp.rspOffset; + } - if (pHead->msgType == TDMT_VND_SUBMIT) { - SPackedData submit = { - .msgStr = POINTER_SHIFT(pHead->body, sizeof(SSubmitReq2Msg)), - .msgLen = pHead->bodyLen - sizeof(SSubmitReq2Msg), - .ver = pHead->version, - }; + tqDebug("taosx poll: consumer:0x%" PRIx64 " subkey:%s vgId:%d, send data blockNum:%d, offset type:%d,uid:%" PRId64 + ",version:%" PRId64, + consumerId, pHandle->subKey, vgId, taosxRsp.blockNum, taosxRsp.rspOffset.type, taosxRsp.rspOffset.uid, + taosxRsp.rspOffset.version); + } else { + // if (offset.type == TMQ_OFFSET__LOG) { + int64_t fetchVer = offset.version + 1; + pCkHead = taosMemoryMalloc(sizeof(SWalCkHead) + 2048); + if (pCkHead == NULL) { + tDeleteSTaosxRsp(&taosxRsp); + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; + } - if (tqTaosxScanLog(pTq, pHandle, submit, &taosxRsp) < 0) { - tqError("tmq poll: tqTaosxScanLog error %" PRId64 ", in vgId:%d, subkey %s", consumerId, vgId, - pRequest->subKey); - return -1; + walSetReaderCapacity(pHandle->pWalReader, 2048); + + while (1) { + // todo refactor: this is not correct. + int32_t savedEpoch = atomic_load_32(&pHandle->epoch); + if (savedEpoch > pRequest->epoch) { + tqWarn("tmq poll: consumer:0x%" PRIx64 " (epoch %d), subkey:%s vgId:%d offset %" PRId64 + ", found new consumer epoch %d, discard req epoch %d", + consumerId, pRequest->epoch, pHandle->subKey, vgId, fetchVer, savedEpoch, pRequest->epoch); + break; } - if (taosxRsp.blockNum > 0) { + if (tqFetchLog(pTq, pHandle, &fetchVer, &pCkHead, pRequest->reqId) < 0) { tqOffsetResetToLog(&taosxRsp.rspOffset, fetchVer); code = tqSendDataRsp(pTq, pMsg, pRequest, (SMqDataRsp*)&taosxRsp, TMQ_MSG_TYPE__TAOSX_RSP); tDeleteSTaosxRsp(&taosxRsp); taosMemoryFreeClear(pCkHead); return code; - } else { - fetchVer++; } - } else { - /*A(pHandle->fetchMeta);*/ - /*A(IS_META_MSG(pHead->msgType));*/ - tqDebug("fetch meta msg, ver:%" PRId64 ", type:%s", pHead->version, TMSG_INFO(pHead->msgType)); - tqOffsetResetToLog(&metaRsp.rspOffset, fetchVer); - metaRsp.resMsgType = pHead->msgType; - metaRsp.metaRspLen = pHead->bodyLen; - metaRsp.metaRsp = pHead->body; - if (tqSendMetaPollRsp(pTq, pMsg, pRequest, &metaRsp) < 0) { - code = -1; + SWalCont* pHead = &pCkHead->head; + tqDebug("tmq poll: consumer:0x%" PRIx64 " (epoch %d) iter log, vgId:%d offset %" PRId64 " msgType %d", + consumerId, pRequest->epoch, vgId, fetchVer, pHead->msgType); + + if (pHead->msgType == TDMT_VND_SUBMIT) { + SPackedData submit = { + .msgStr = POINTER_SHIFT(pHead->body, sizeof(SSubmitReq2Msg)), + .msgLen = pHead->bodyLen - sizeof(SSubmitReq2Msg), + .ver = pHead->version, + }; + + if (tqTaosxScanLog(pTq, pHandle, submit, &taosxRsp) < 0) { + tqError("tmq poll: tqTaosxScanLog error %" PRId64 ", in vgId:%d, subkey %s", consumerId, vgId, + pRequest->subKey); + return -1; + } + + if (taosxRsp.blockNum > 0) { + tqOffsetResetToLog(&taosxRsp.rspOffset, fetchVer); + code = tqSendDataRsp(pTq, pMsg, pRequest, (SMqDataRsp*)&taosxRsp, TMQ_MSG_TYPE__TAOSX_RSP); + tDeleteSTaosxRsp(&taosxRsp); + taosMemoryFreeClear(pCkHead); + return code; + } else { + fetchVer++; + } + + } else { + /*A(pHandle->fetchMeta);*/ + /*A(IS_META_MSG(pHead->msgType));*/ + tqDebug("fetch meta msg, ver:%" PRId64 ", type:%s", pHead->version, TMSG_INFO(pHead->msgType)); + tqOffsetResetToLog(&metaRsp.rspOffset, fetchVer); + metaRsp.resMsgType = pHead->msgType; + metaRsp.metaRspLen = pHead->bodyLen; + metaRsp.metaRsp = pHead->body; + if (tqSendMetaPollRsp(pTq, pMsg, pRequest, &metaRsp) < 0) { + code = -1; + taosMemoryFreeClear(pCkHead); + tDeleteSTaosxRsp(&taosxRsp); + return code; + } + code = 0; taosMemoryFreeClear(pCkHead); tDeleteSTaosxRsp(&taosxRsp); return code; } - code = 0; - taosMemoryFreeClear(pCkHead); - tDeleteSTaosxRsp(&taosxRsp); - return code; } } - } - tDeleteSTaosxRsp(&taosxRsp); - taosMemoryFreeClear(pCkHead); - return 0; + tDeleteSTaosxRsp(&taosxRsp); + taosMemoryFreeClear(pCkHead); + return 0; + } } int32_t tqProcessPollReq(STQ* pTq, SRpcMsg* pMsg) { From 20e2010f709cab26d7262863449fec139af99375 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Thu, 23 Mar 2023 13:35:17 +0800 Subject: [PATCH 12/65] fix:memory leak --- source/client/src/clientRawBlockWrite.c | 65 ++++++++++++++----------- 1 file changed, 36 insertions(+), 29 deletions(-) diff --git a/source/client/src/clientRawBlockWrite.c b/source/client/src/clientRawBlockWrite.c index 6bd8b01842..b1299c16e8 100644 --- a/source/client/src/clientRawBlockWrite.c +++ b/source/client/src/clientRawBlockWrite.c @@ -183,7 +183,7 @@ static char* buildAlterSTableJson(void* alterData, int32_t alterDataLen) { } string = cJSON_PrintUnformatted(json); -end: + end: cJSON_Delete(json); tFreeSMAltertbReq(&req); return string; @@ -205,7 +205,7 @@ static char* processCreateStb(SMqMetaRsp* metaRsp) { } string = buildCreateTableJson(&req.schemaRow, &req.schemaTag, req.name, req.suid, TSDB_SUPER_TABLE); uDebug("processCreateStb %s", string); -_err: + _err: tDecoderClear(&coder); return string; } @@ -227,7 +227,7 @@ static char* processAlterStb(SMqMetaRsp* metaRsp) { string = buildAlterSTableJson(req.alterOriData, req.alterOriDataLen); uDebug("processAlterStb %s", string); -_err: + _err: tDecoderClear(&coder); return string; } @@ -309,7 +309,7 @@ static void buildChildElement(cJSON* json, SVCreateTbReq* pCreateReq) { cJSON_AddItemToArray(tags, tag); } -end: + end: cJSON_AddItemToObject(json, "tags", tags); taosArrayDestroy(pTagVals); } @@ -368,7 +368,7 @@ static char* processCreateTable(SMqMetaRsp* metaRsp) { uDebug("processCreateTable :%s", string); } -_exit: + _exit: for (int32_t iReq = 0; iReq < req.nReqs; iReq++) { pCreateReq = req.pReqs + iReq; taosMemoryFreeClear(pCreateReq->comment); @@ -408,7 +408,7 @@ static char* processAutoCreateTable(STaosxRsp* rsp) { } string = buildCreateCTableJson(pCreateReq, rsp->createTableNum); uDebug("processAutoCreateTable :%s", string); -_exit: + _exit: for (int i = 0; i < rsp->createTableNum; i++) { tDecoderClear(&decoder[i]); taosMemoryFreeClear(pCreateReq[i].comment); @@ -535,7 +535,7 @@ static char* processAlterTable(SMqMetaRsp* metaRsp) { string = cJSON_PrintUnformatted(json); uDebug("processAlterTable :%s", string); -_exit: + _exit: cJSON_Delete(json); tDecoderClear(&decoder); return string; @@ -569,7 +569,7 @@ static char* processDropSTable(SMqMetaRsp* metaRsp) { string = cJSON_PrintUnformatted(json); uDebug("processDropSTable :%s", string); -_exit: + _exit: cJSON_Delete(json); tDecoderClear(&decoder); return string; @@ -609,7 +609,7 @@ static char* processDeleteTable(SMqMetaRsp* metaRsp) { string = cJSON_PrintUnformatted(json); uDebug("processDeleteTable :%s", string); -_exit: + _exit: cJSON_Delete(json); tDecoderClear(&coder); return string; @@ -652,7 +652,7 @@ static char* processDropTable(SMqMetaRsp* metaRsp) { string = cJSON_PrintUnformatted(json); uDebug("processDropTable :%s", string); -_exit: + _exit: cJSON_Delete(json); tDecoderClear(&decoder); return string; @@ -742,7 +742,7 @@ static int32_t taosCreateStb(TAOS* taos, void* meta, int32_t metaLen) { code = pRequest->code; taosMemoryFree(pCmdMsg.pMsg); -end: + end: destroyRequest(pRequest); tFreeSMCreateStbReq(&pReq); tDecoderClear(&coder); @@ -839,7 +839,7 @@ static int32_t taosDropStb(TAOS* taos, void* meta, int32_t metaLen) { code = pRequest->code; taosMemoryFree(pCmdMsg.pMsg); -end: + end: destroyRequest(pRequest); tDecoderClear(&coder); return code; @@ -901,9 +901,9 @@ static int32_t taosCreateTable(TAOS* taos, void* meta, int32_t metaLen) { taosHashSetFreeFp(pVgroupHashmap, destroyCreateTbReqBatch); SRequestConnInfo conn = {.pTrans = pTscObj->pAppInfo->pTransporter, - .requestId = pRequest->requestId, - .requestObjRefId = pRequest->self, - .mgmtEps = getEpSet_s(&pTscObj->pAppInfo->mgmtEp)}; + .requestId = pRequest->requestId, + .requestObjRefId = pRequest->self, + .mgmtEps = getEpSet_s(&pTscObj->pAppInfo->mgmtEp)}; pRequest->tableList = taosArrayInit(req.nReqs, sizeof(SName)); // loop to create table @@ -987,7 +987,7 @@ static int32_t taosCreateTable(TAOS* taos, void* meta, int32_t metaLen) { code = pRequest->code; -end: + end: for (int32_t iReq = 0; iReq < req.nReqs; iReq++) { pCreateReq = req.pReqs + iReq; taosMemoryFreeClear(pCreateReq->comment); @@ -1058,9 +1058,9 @@ static int32_t taosDropTable(TAOS* taos, void* meta, int32_t metaLen) { taosHashSetFreeFp(pVgroupHashmap, destroyDropTbReqBatch); SRequestConnInfo conn = {.pTrans = pTscObj->pAppInfo->pTransporter, - .requestId = pRequest->requestId, - .requestObjRefId = pRequest->self, - .mgmtEps = getEpSet_s(&pTscObj->pAppInfo->mgmtEp)}; + .requestId = pRequest->requestId, + .requestObjRefId = pRequest->self, + .mgmtEps = getEpSet_s(&pTscObj->pAppInfo->mgmtEp)}; pRequest->tableList = taosArrayInit(req.nReqs, sizeof(SName)); // loop to create table for (int32_t iReq = 0; iReq < req.nReqs; iReq++) { @@ -1132,7 +1132,7 @@ static int32_t taosDropTable(TAOS* taos, void* meta, int32_t metaLen) { } code = pRequest->code; -end: + end: taosHashCleanup(pVgroupHashmap); destroyRequest(pRequest); tDecoderClear(&coder); @@ -1201,7 +1201,7 @@ static int32_t taosDeleteData(TAOS* taos, void* meta, int32_t metaLen) { } taos_free_result(res); -end: + end: tDecoderClear(&coder); return code; } @@ -1249,9 +1249,9 @@ static int32_t taosAlterTable(TAOS* taos, void* meta, int32_t metaLen) { } SRequestConnInfo conn = {.pTrans = pTscObj->pAppInfo->pTransporter, - .requestId = pRequest->requestId, - .requestObjRefId = pRequest->self, - .mgmtEps = getEpSet_s(&pTscObj->pAppInfo->mgmtEp)}; + .requestId = pRequest->requestId, + .requestObjRefId = pRequest->self, + .mgmtEps = getEpSet_s(&pTscObj->pAppInfo->mgmtEp)}; SVgroupInfo pInfo = {0}; SName pName = {0}; @@ -1311,7 +1311,7 @@ static int32_t taosAlterTable(TAOS* taos, void* meta, int32_t metaLen) { code = handleAlterTbExecRes(pRes->res, pCatalog); } } -end: + end: taosArrayDestroy(pArray); if (pVgData) taosMemoryFreeClear(pVgData->pData); taosMemoryFreeClear(pVgData); @@ -1399,7 +1399,7 @@ int taos_write_raw_block_with_fields(TAOS* taos, int rows, char* pData, const ch launchQueryImpl(pRequest, pQuery, true, NULL); code = pRequest->code; -end: + end: taosMemoryFreeClear(pTableMeta); qDestroyQuery(pQuery); destroyRequest(pRequest); @@ -1481,7 +1481,7 @@ int taos_write_raw_block(TAOS* taos, int rows, char* pData, const char* tbname) launchQueryImpl(pRequest, pQuery, true, NULL); code = pRequest->code; -end: + end: taosMemoryFreeClear(pTableMeta); qDestroyQuery(pQuery); destroyRequest(pRequest); @@ -1601,6 +1601,7 @@ static int32_t tmqWriteRawDataImpl(TAOS* taos, void* data, int32_t dataLen) { uError("WriteRaw:rawBlockBindData failed"); goto end; } + taosMemoryFreeClear(pTableMeta); } code = smlBuildOutput(pQuery, pVgHash); @@ -1612,7 +1613,7 @@ static int32_t tmqWriteRawDataImpl(TAOS* taos, void* data, int32_t dataLen) { launchQueryImpl(pRequest, pQuery, true, NULL); code = pRequest->code; -end: + end: tDeleteSMqDataRsp(&rspObj.rsp); tDecoderClear(&decoder); qDestroyQuery(pQuery); @@ -1707,6 +1708,7 @@ static int32_t tmqWriteRawMetaDataImpl(TAOS* taos, void* data, int32_t dataLen) tDecoderInit(&decoderTmp, *dataTmp, *lenTmp); if (tDecodeSVCreateTbReq(&decoderTmp, &pCreateReq) < 0) { tDecoderClear(&decoderTmp); + tDestroySVCreateTbReq(&pCreateReq, TSDB_MSG_FLG_DECODE); uError("WriteRaw: tDecodeSVCreateTbReq error"); code = TSDB_CODE_TMQ_INVALID_MSG; goto end; @@ -1715,15 +1717,19 @@ static int32_t tmqWriteRawMetaDataImpl(TAOS* taos, void* data, int32_t dataLen) if (pCreateReq.type != TSDB_CHILD_TABLE) { uError("WriteRaw:pCreateReq.type != TSDB_CHILD_TABLE. table name: %s", tbName); code = TSDB_CODE_TSC_INVALID_VALUE; + tDecoderClear(&decoderTmp); + tDestroySVCreateTbReq(&pCreateReq, TSDB_MSG_FLG_DECODE); goto end; } if (strcmp(tbName, pCreateReq.name) == 0) { cloneSVreateTbReq(&pCreateReq, &pCreateReqDst); // pCreateReqDst->ctb.suid = processSuid(pCreateReqDst->ctb.suid, pRequest->pDb); tDecoderClear(&decoderTmp); + tDestroySVCreateTbReq(&pCreateReq, TSDB_MSG_FLG_DECODE); break; } tDecoderClear(&decoderTmp); + tDestroySVCreateTbReq(&pCreateReq, TSDB_MSG_FLG_DECODE); } SVgroupInfo vg; @@ -1774,6 +1780,7 @@ static int32_t tmqWriteRawMetaDataImpl(TAOS* taos, void* data, int32_t dataLen) goto end; } pCreateReqDst = NULL; + taosMemoryFreeClear(pTableMeta); } code = smlBuildOutput(pQuery, pVgHash); @@ -1785,7 +1792,7 @@ static int32_t tmqWriteRawMetaDataImpl(TAOS* taos, void* data, int32_t dataLen) launchQueryImpl(pRequest, pQuery, true, NULL); code = pRequest->code; -end: + end: tDeleteSTaosxRsp(&rspObj.rsp); tDecoderClear(&decoder); qDestroyQuery(pQuery); From 40a78bde5038a385bcf32839106202ae6f00d830 Mon Sep 17 00:00:00 2001 From: slzhou Date: Thu, 23 Mar 2023 13:49:08 +0800 Subject: [PATCH 13/65] fix: unknow db error when query ins_tables with table_name="" --- source/libs/parser/src/parUtil.c | 2 +- tests/script/tsim/query/sys_tbname.sim | 5 +++++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/source/libs/parser/src/parUtil.c b/source/libs/parser/src/parUtil.c index 793d05721e..563bc5e780 100644 --- a/source/libs/parser/src/parUtil.c +++ b/source/libs/parser/src/parUtil.c @@ -444,7 +444,7 @@ static int32_t getInsTagsTableTargetNameFromOp(int32_t acctId, SOperatorNode* pO } else if (QUERY_NODE_VALUE == nodeType(pOper->pRight)) { pVal = (SValueNode*)pOper->pRight; } - if (NULL == pCol || NULL == pVal) { + if (NULL == pCol || NULL == pVal || NULL == pVal->literal || 0 == strcmp(pVal->literal, "")) { return TSDB_CODE_SUCCESS; } diff --git a/tests/script/tsim/query/sys_tbname.sim b/tests/script/tsim/query/sys_tbname.sim index 7b3953129a..c676f2b1e0 100644 --- a/tests/script/tsim/query/sys_tbname.sim +++ b/tests/script/tsim/query/sys_tbname.sim @@ -51,6 +51,11 @@ if $data00 != @ins_stables@ then return -1 endi +sql select * from information_schema.ins_tables where table_name=''; +if $rows != 0 then + return -1 +endi + sql select tbname from information_schema.ins_tables; print $rows $data00 if $rows != 33 then From ffa59f69aa32dcb851dcac847c921046b4e83f18 Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao@163.com> Date: Thu, 23 Mar 2023 11:18:14 +0800 Subject: [PATCH 14/65] fix:add combine function for groupKey --- include/libs/function/functionMgt.h | 1 + source/libs/executor/src/timewindowoperator.c | 8 +- source/libs/function/inc/builtinsimpl.h | 1 + source/libs/function/src/builtins.c | 3 +- source/libs/function/src/builtinsimpl.c | 33 +++++ source/libs/function/src/functionMgt.c | 7 ++ .../tsim/stream/distributeInterval0.sim | 118 ++++++++++++++++++ 7 files changed, 167 insertions(+), 4 deletions(-) diff --git a/include/libs/function/functionMgt.h b/include/libs/function/functionMgt.h index 7b65c06b85..60a4a8c605 100644 --- a/include/libs/function/functionMgt.h +++ b/include/libs/function/functionMgt.h @@ -241,6 +241,7 @@ int32_t fmGetUdafExecFuncs(int32_t funcId, SFuncExecFuncs* pFpSet); int32_t fmSetInvertFunc(int32_t funcId, SFuncExecFuncs* pFpSet); int32_t fmSetNormalFunc(int32_t funcId, SFuncExecFuncs* pFpSet); bool fmIsInvertible(int32_t funcId); +char* fmGetFuncName(int32_t funcId); #ifdef __cplusplus } diff --git a/source/libs/executor/src/timewindowoperator.c b/source/libs/executor/src/timewindowoperator.c index fef588a503..f731e57dcb 100644 --- a/source/libs/executor/src/timewindowoperator.c +++ b/source/libs/executor/src/timewindowoperator.c @@ -2110,10 +2110,12 @@ void compactFunctions(SqlFunctionCtx* pDestCtx, SqlFunctionCtx* pSourceCtx, int3 } else if (functionNeedToExecute(&pDestCtx[k]) && pDestCtx[k].fpSet.combine != NULL) { int32_t code = pDestCtx[k].fpSet.combine(&pDestCtx[k], &pSourceCtx[k]); if (code != TSDB_CODE_SUCCESS) { - qError("%s apply functions error, code: %s", GET_TASKID(pTaskInfo), tstrerror(code)); - pTaskInfo->code = code; - T_LONG_JMP(pTaskInfo->env, code); + qError("%s apply combine functions error, code: %s", GET_TASKID(pTaskInfo), tstrerror(code)); } + } else if (pDestCtx[k].fpSet.combine == NULL) { + char* funName = fmGetFuncName(pDestCtx[k].functionId); + qError("%s error, combine funcion for %s is not implemented", GET_TASKID(pTaskInfo), funName); + taosMemoryFreeClear(funName); } } } diff --git a/source/libs/function/inc/builtinsimpl.h b/source/libs/function/inc/builtinsimpl.h index dc884a0581..c3afc30a7b 100644 --- a/source/libs/function/inc/builtinsimpl.h +++ b/source/libs/function/inc/builtinsimpl.h @@ -235,6 +235,7 @@ int32_t blockDistFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock); bool getGroupKeyFuncEnv(SFunctionNode* pFunc, SFuncExecEnv* pEnv); int32_t groupKeyFunction(SqlFunctionCtx* pCtx); int32_t groupKeyFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock); +int32_t groupKeyCombine(SqlFunctionCtx* pDestCtx, SqlFunctionCtx* pSourceCtx); #ifdef __cplusplus } diff --git a/source/libs/function/src/builtins.c b/source/libs/function/src/builtins.c index 0257b3d5e6..4fcc9ac1ad 100644 --- a/source/libs/function/src/builtins.c +++ b/source/libs/function/src/builtins.c @@ -2480,7 +2480,7 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { { .name = "irate", .type = FUNCTION_TYPE_IRATE, - .classification = FUNC_MGT_AGG_FUNC | FUNC_MGT_TIMELINE_FUNC | FUNC_MGT_IMPLICIT_TS_FUNC, + .classification = FUNC_MGT_AGG_FUNC | FUNC_MGT_TIMELINE_FUNC | FUNC_MGT_IMPLICIT_TS_FUNC | FUNC_MGT_FORBID_STREAM_FUNC, .translateFunc = translateIrate, .getEnvFunc = getIrateFuncEnv, .initFunc = irateFuncSetup, @@ -3234,6 +3234,7 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .initFunc = functionSetup, .processFunc = groupKeyFunction, .finalizeFunc = groupKeyFinalize, + .combineFunc = groupKeyCombine, .pPartialFunc = "_group_key", .pMergeFunc = "_group_key" }, diff --git a/source/libs/function/src/builtinsimpl.c b/source/libs/function/src/builtinsimpl.c index 35cfa1753f..3c9f2fe8ca 100644 --- a/source/libs/function/src/builtinsimpl.c +++ b/source/libs/function/src/builtinsimpl.c @@ -5900,6 +5900,39 @@ int32_t groupKeyFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock) { return pResInfo->numOfRes; } +int32_t groupKeyCombine(SqlFunctionCtx* pDestCtx, SqlFunctionCtx* pSourceCtx) { + SResultRowEntryInfo* pDResInfo = GET_RES_INFO(pDestCtx); + SGroupKeyInfo* pDBuf = GET_ROWCELL_INTERBUF(pDResInfo); + + SResultRowEntryInfo* pSResInfo = GET_RES_INFO(pSourceCtx); + SGroupKeyInfo* pSBuf = GET_ROWCELL_INTERBUF(pSResInfo); + + // escape rest of data blocks to avoid first entry to be overwritten. + if (pDBuf->hasResult) { + goto _group_key_over; + } + + if (pSBuf->isNull) { + pDBuf->isNull = true; + pDBuf->hasResult = true; + goto _group_key_over; + } + + if (IS_VAR_DATA_TYPE(pSourceCtx->resDataInfo.type)) { + memcpy(pDBuf->data, pSBuf->data, + (pSourceCtx->resDataInfo.type == TSDB_DATA_TYPE_JSON) ? getJsonValueLen(pSBuf->data) : varDataTLen(pSBuf->data)); + } else { + memcpy(pDBuf->data, pSBuf->data, pSourceCtx->resDataInfo.bytes); + } + + pDBuf->hasResult = true; + +_group_key_over: + + SET_VAL(pDResInfo, 1, 1); + return TSDB_CODE_SUCCESS; +} + int32_t cachedLastRowFunction(SqlFunctionCtx* pCtx) { int32_t numOfElems = 0; diff --git a/source/libs/function/src/functionMgt.c b/source/libs/function/src/functionMgt.c index e3127fcd7b..94ab616dda 100644 --- a/source/libs/function/src/functionMgt.c +++ b/source/libs/function/src/functionMgt.c @@ -447,3 +447,10 @@ int32_t fmGetDistMethod(const SFunctionNode* pFunc, SFunctionNode** pPartialFunc return code; } + +char* fmGetFuncName(int32_t funcId) { + if (fmIsUserDefinedFunc(funcId) || funcId < 0 || funcId >= funcMgtBuiltinsNum) { + return taosStrdup("invalid function"); + } + return taosStrdup(funcMgtBuiltins[funcId].name); +} diff --git a/tests/script/tsim/stream/distributeInterval0.sim b/tests/script/tsim/stream/distributeInterval0.sim index 1d58922928..de1c72f8dc 100644 --- a/tests/script/tsim/stream/distributeInterval0.sim +++ b/tests/script/tsim/stream/distributeInterval0.sim @@ -272,4 +272,122 @@ if $data12 != 2 then goto loop3 endi +print ===== step3 + +sql drop database if exists test4; +sql create database test4 vgroups 10; +sql use test4; +sql create stable st(ts timestamp,a int,b int,c varchar(250) ) tags(ta int,tb int,tc int); +sql create table aaa using st tags(1,1,1); +sql create table bbb using st tags(2,2,2); +sql create table ccc using st tags(3,2,2); +sql create table ddd using st tags(4,2,2); + + +sql create stream streams1 ignore expired 0 fill_history 0 watermark 3s into streamst subtable(c) as select _wstart, c , count(*) c1 from st partition by c interval(1s) ; + +sql insert into aaa values(1648791221001,2,2,"/a1/aa/aa"); +sql insert into bbb values(1648791221001,2,2,"/a1/aa/aa"); +sql insert into ccc values(1648791221001,2,2,"/a1/aa/aa"); +sql insert into ddd values(1648791221001,2,2,"/a1/aa/aa"); + +sql insert into aaa values(1648791222002,2,2,"/a2/aa/aa"); +sql insert into bbb values(1648791222002,2,2,"/a2/aa/aa"); +sql insert into ccc values(1648791222002,2,2,"/a2/aa/aa"); +sql insert into ddd values(1648791222002,2,2,"/a2/aa/aa"); + +sql insert into aaa values(1648791223003,2,2,"/a3/aa/aa"); +sql insert into bbb values(1648791223003,2,2,"/a3/aa/aa"); +sql insert into ccc values(1648791223003,2,2,"/a3/aa/aa"); +sql insert into ddd values(1648791223003,2,2,"/a3/aa/aa"); + +sql insert into aaa values(1648791224003,2,2,"/a4/aa/aa"); +sql insert into bbb values(1648791224003,2,2,"/a4/aa/aa"); +sql insert into ccc values(1648791224003,2,2,"/a4/aa/aa"); +sql insert into ddd values(1648791224003,2,2,"/a4/aa/aa"); + + +sql insert into aaa values(1648791225003,2,2,"/a5/aa/aa"); +sql insert into bbb values(1648791225003,2,2,"/a5/aa/aa"); +sql insert into ccc values(1648791225003,2,2,"/a5/aa/aa"); +sql insert into ddd values(1648791225003,2,2,"/a5/aa/aa"); + +sql insert into aaa values(1648791226003,2,2,"/a6/aa/aa"); +sql insert into bbb values(1648791226003,2,2,"/a6/aa/aa"); +sql insert into ccc values(1648791226003,2,2,"/a6/aa/aa"); +sql insert into ddd values(1648791226003,2,2,"/a6/aa/aa"); + +$loop_count = 0 + +loop4: +sleep 200 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +sql select * from streamst; + +if $rows == 0 then + goto loop4 +endi + +sql delete from aaa where ts = 1648791223003 ; + +$loop_count = 0 + +loop5: +sleep 200 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +sql select * from streamst; + +if $rows == 0 then + goto loop5 +endi + + +sql delete from ccc; + +$loop_count = 0 + +loop6: +sleep 200 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +sql select * from streamst; + +if $rows == 0 then + goto loop6 +endi + +sql delete from ddd; + +$loop_count = 0 + +loop7: +sleep 200 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +sql select * from streamst; + +if $rows == 0 then + goto loop7 +endi + +print ===== over + system sh/stop_dnodes.sh From 9be09492f0ff2deff9cd8dea5c0d889bd81ae2c8 Mon Sep 17 00:00:00 2001 From: Benguang Zhao Date: Fri, 24 Mar 2023 09:05:06 +0800 Subject: [PATCH 15/65] enh: disable timely vnodeCommit --- include/libs/sync/sync.h | 2 +- source/dnode/vnode/src/vnd/vnodeCommit.c | 9 +++------ 2 files changed, 4 insertions(+), 7 deletions(-) diff --git a/include/libs/sync/sync.h b/include/libs/sync/sync.h index 08a6be8015..1cd32f05ad 100644 --- a/include/libs/sync/sync.h +++ b/include/libs/sync/sync.h @@ -48,7 +48,7 @@ extern "C" { #define SYNC_HEARTBEAT_REPLY_SLOW_MS 1500 #define SYNC_SNAP_RESEND_MS 1000 * 60 -#define SYNC_VND_COMMIT_MIN_MS 1000 +#define SYNC_VND_COMMIT_MIN_MS 3000 #define SYNC_MAX_BATCH_SIZE 1 #define SYNC_INDEX_BEGIN 0 diff --git a/source/dnode/vnode/src/vnd/vnodeCommit.c b/source/dnode/vnode/src/vnd/vnodeCommit.c index 0779388ba9..09f1ca7877 100644 --- a/source/dnode/vnode/src/vnd/vnodeCommit.c +++ b/source/dnode/vnode/src/vnd/vnodeCommit.c @@ -156,13 +156,10 @@ int vnodeShouldCommit(SVnode *pVnode) { bool needCommit = false; taosThreadMutexLock(&pVnode->mutex); - if (!pVnode->inUse || !diskAvail) { - goto _out; + if (pVnode->inUse && diskAvail) { + needCommit = + ((pVnode->inUse->size > pVnode->inUse->node.size) && (pSched->commitMs + SYNC_VND_COMMIT_MIN_MS < nowMs)); } - needCommit = - (((pVnode->inUse->size > pVnode->inUse->node.size) && (pSched->commitMs + SYNC_VND_COMMIT_MIN_MS < nowMs)) || - (pVnode->inUse->size > 0 && pSched->commitMs + pSched->maxWaitMs < nowMs)); -_out: taosThreadMutexUnlock(&pVnode->mutex); return needCommit; } From bf78f63e22e3e3863d1bfba07dafe64522ee37bb Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Fri, 24 Mar 2023 10:10:52 +0800 Subject: [PATCH 16/65] fix:send data batch if consume wal where subscribe db --- source/dnode/vnode/src/inc/tq.h | 3 +- source/dnode/vnode/src/tq/tq.c | 56 ++++++++++++++---------------- source/dnode/vnode/src/tq/tqExec.c | 27 ++------------ 3 files changed, 30 insertions(+), 56 deletions(-) diff --git a/source/dnode/vnode/src/inc/tq.h b/source/dnode/vnode/src/inc/tq.h index c2b38f5cd1..bf390c1632 100644 --- a/source/dnode/vnode/src/inc/tq.h +++ b/source/dnode/vnode/src/inc/tq.h @@ -144,8 +144,7 @@ int32_t tqScanData(STQ* pTq, const STqHandle* pHandle, SMqDataRsp* pRsp, STqOffs int32_t tqFetchLog(STQ* pTq, STqHandle* pHandle, int64_t* fetchOffset, SWalCkHead** pHeadWithCkSum, uint64_t reqId); // tqExec -int32_t tqTaosxScanLog(STQ* pTq, STqHandle* pHandle, SPackedData submit, STaosxRsp* pRsp); -// int32_t tqTaosxScanLog(STQ* pTq, STqHandle* pHandle, SSubmitReq* pReq, STaosxRsp* pRsp); +int32_t tqTaosxScanLog(STQ* pTq, STqHandle* pHandle, SPackedData submit, STaosxRsp* pRsp, int32_t* totalRows); int32_t tqAddBlockDataToRsp(const SSDataBlock* pBlock, SMqDataRsp* pRsp, int32_t numOfCols, int8_t precision); int32_t tqSendDataRsp(STQ* pTq, const SRpcMsg* pMsg, const SMqPollReq* pReq, const SMqDataRsp* pRsp, int32_t type); int32_t tqPushDataRsp(STQ* pTq, STqPushEntry* pPushEntry); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index f48b57b74b..eb38de1c9a 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -596,8 +596,6 @@ static int32_t extractDataForMq(STQ* pTq, STqHandle* pHandle, const SMqPollReq* } if (offset.type == TMQ_OFFSET__LOG) { - -// if (offset.type == TMQ_OFFSET__LOG) { int64_t fetchVer = offset.version + 1; pCkHead = taosMemoryMalloc(sizeof(SWalCkHead) + 2048); if (pCkHead == NULL) { @@ -605,9 +603,8 @@ static int32_t extractDataForMq(STQ* pTq, STqHandle* pHandle, const SMqPollReq* terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } - walSetReaderCapacity(pHandle->pWalReader, 2048); - + int totalRows = 0; while (1) { // todo refactor: this is not correct. int32_t savedEpoch = atomic_load_32(&pHandle->epoch); @@ -630,32 +627,8 @@ static int32_t extractDataForMq(STQ* pTq, STqHandle* pHandle, const SMqPollReq* tqDebug("tmq poll: consumer:0x%" PRIx64 " (epoch %d) iter log, vgId:%d offset %" PRId64 " msgType %d", consumerId, pRequest->epoch, vgId, fetchVer, pHead->msgType); - if (pHead->msgType == TDMT_VND_SUBMIT) { - SPackedData submit = { - .msgStr = POINTER_SHIFT(pHead->body, sizeof(SSubmitReq2Msg)), - .msgLen = pHead->bodyLen - sizeof(SSubmitReq2Msg), - .ver = pHead->version, - }; - - if (tqTaosxScanLog(pTq, pHandle, submit, &taosxRsp) < 0) { - tqError("tmq poll: tqTaosxScanLog error %" PRId64 ", in vgId:%d, subkey %s", consumerId, vgId, - pRequest->subKey); - return -1; - } - - if (taosxRsp.blockNum > 0) { - tqOffsetResetToLog(&taosxRsp.rspOffset, fetchVer); - code = tqSendDataRsp(pTq, pMsg, pRequest, (SMqDataRsp*)&taosxRsp, TMQ_MSG_TYPE__TAOSX_RSP); - tDeleteSTaosxRsp(&taosxRsp); - taosMemoryFreeClear(pCkHead); - return code; - } else { - fetchVer++; - } - - } else { - /*A(pHandle->fetchMeta);*/ - /*A(IS_META_MSG(pHead->msgType));*/ + // process meta + if (pHead->msgType != TDMT_VND_SUBMIT) { tqDebug("fetch meta msg, ver:%" PRId64 ", type:%s", pHead->version, TMSG_INFO(pHead->msgType)); tqOffsetResetToLog(&metaRsp.rspOffset, fetchVer); metaRsp.resMsgType = pHead->msgType; @@ -672,6 +645,29 @@ static int32_t extractDataForMq(STQ* pTq, STqHandle* pHandle, const SMqPollReq* tDeleteSTaosxRsp(&taosxRsp); return code; } + + // process data + SPackedData submit = { + .msgStr = POINTER_SHIFT(pHead->body, sizeof(SSubmitReq2Msg)), + .msgLen = pHead->bodyLen - sizeof(SSubmitReq2Msg), + .ver = pHead->version, + }; + + if (tqTaosxScanLog(pTq, pHandle, submit, &taosxRsp, &totalRows) < 0) { + tqError("tmq poll: tqTaosxScanLog error %" PRId64 ", in vgId:%d, subkey %s", consumerId, vgId, + pRequest->subKey); + return -1; + } + + if (totalRows >= 4096 || taosxRsp.createTableNum > 0) { + tqOffsetResetToLog(&taosxRsp.rspOffset, fetchVer); + code = tqSendDataRsp(pTq, pMsg, pRequest, (SMqDataRsp*)&taosxRsp, TMQ_MSG_TYPE__TAOSX_RSP); + tDeleteSTaosxRsp(&taosxRsp); + taosMemoryFreeClear(pCkHead); + return code; + } else { + fetchVer++; + } } } diff --git a/source/dnode/vnode/src/tq/tqExec.c b/source/dnode/vnode/src/tq/tqExec.c index a62101eb47..796f150cbc 100644 --- a/source/dnode/vnode/src/tq/tqExec.c +++ b/source/dnode/vnode/src/tq/tqExec.c @@ -230,23 +230,15 @@ int32_t tqScanTaosx(STQ* pTq, const STqHandle* pHandle, STaosxRsp* pRsp, SMqMeta return 0; } -int32_t tqTaosxScanLog(STQ* pTq, STqHandle* pHandle, SPackedData submit, STaosxRsp* pRsp) { +int32_t tqTaosxScanLog(STQ* pTq, STqHandle* pHandle, SPackedData submit, STaosxRsp* pRsp, int32_t* totalRows) { STqExecHandle* pExec = &pHandle->execHandle; - /*A(pExec->subType != TOPIC_SUB_TYPE__COLUMN);*/ - SArray* pBlocks = taosArrayInit(0, sizeof(SSDataBlock)); SArray* pSchemas = taosArrayInit(0, sizeof(void*)); if (pExec->subType == TOPIC_SUB_TYPE__TABLE) { STqReader* pReader = pExec->pExecReader; - /*tqReaderSetDataMsg(pReader, pReq, 0);*/ tqReaderSetSubmitReq2(pReader, submit.msgStr, submit.msgLen, submit.ver); while (tqNextDataBlock2(pReader)) { - /*SSDataBlock block = {0};*/ - /*if (tqRetrieveDataBlock(&block, pReader) < 0) {*/ - /*if (terrno == TSDB_CODE_TQ_TABLE_SCHEMA_NOT_FOUND) continue;*/ - /*}*/ - taosArrayClear(pBlocks); taosArrayClear(pSchemas); SSubmitTbData* pSubmitTbDataRet = NULL; @@ -254,7 +246,6 @@ int32_t tqTaosxScanLog(STQ* pTq, STqHandle* pHandle, SPackedData submit, STaosxR if (terrno == TSDB_CODE_TQ_TABLE_SCHEMA_NOT_FOUND) continue; } if (pRsp->withTbName) { - /*int64_t uid = pExec->pExecReader->msgIter.uid;*/ int64_t uid = pExec->pExecReader->lastBlkUid; if (tqAddTbNameToRsp(pTq, uid, pRsp, taosArrayGetSize(pBlocks)) < 0) { taosArrayDestroyEx(pBlocks, (FDelete)blockDataFreeRes); @@ -296,6 +287,7 @@ int32_t tqTaosxScanLog(STQ* pTq, STqHandle* pHandle, SPackedData submit, STaosxR SSDataBlock* pBlock = taosArrayGet(pBlocks, i); tqAddBlockDataToRsp(pBlock, (SMqDataRsp*)pRsp, taosArrayGetSize(pBlock->pDataBlock), pTq->pVnode->config.tsdbCfg.precision); + totalRows += pBlock->info.rows; blockDataFreeRes(pBlock); SSchemaWrapper* pSW = taosArrayGetP(pSchemas, i); taosArrayPush(pRsp->blockSchema, &pSW); @@ -304,13 +296,8 @@ int32_t tqTaosxScanLog(STQ* pTq, STqHandle* pHandle, SPackedData submit, STaosxR } } else if (pExec->subType == TOPIC_SUB_TYPE__DB) { STqReader* pReader = pExec->pExecReader; - /*tqReaderSetDataMsg(pReader, pReq, 0);*/ tqReaderSetSubmitReq2(pReader, submit.msgStr, submit.msgLen, submit.ver); while (tqNextDataBlockFilterOut2(pReader, pExec->execDb.pFilterOutTbUid)) { - /*SSDataBlock block = {0};*/ - /*if (tqRetrieveDataBlock(&block, pReader) < 0) {*/ - /*if (terrno == TSDB_CODE_TQ_TABLE_SCHEMA_NOT_FOUND) continue;*/ - /*}*/ taosArrayClear(pBlocks); taosArrayClear(pSchemas); SSubmitTbData* pSubmitTbDataRet = NULL; @@ -355,15 +342,11 @@ int32_t tqTaosxScanLog(STQ* pTq, STqHandle* pHandle, SPackedData submit, STaosxR tEncoderClear(&encoder); } - /*tqAddBlockDataToRsp(&block, (SMqDataRsp*)pRsp, taosArrayGetSize(block.pDataBlock),*/ - /*pTq->pVnode->config.tsdbCfg.precision);*/ - /*blockDataFreeRes(&block);*/ - /*tqAddBlockSchemaToRsp(pExec, (SMqDataRsp*)pRsp);*/ - /*pRsp->blockNum++;*/ for (int32_t i = 0; i < taosArrayGetSize(pBlocks); i++) { SSDataBlock* pBlock = taosArrayGet(pBlocks, i); tqAddBlockDataToRsp(pBlock, (SMqDataRsp*)pRsp, taosArrayGetSize(pBlock->pDataBlock), pTq->pVnode->config.tsdbCfg.precision); + totalRows += pBlock->info.rows; blockDataFreeRes(pBlock); SSchemaWrapper* pSW = taosArrayGetP(pSchemas, i); taosArrayPush(pRsp->blockSchema, &pSW); @@ -373,9 +356,5 @@ int32_t tqTaosxScanLog(STQ* pTq, STqHandle* pHandle, SPackedData submit, STaosxR } taosArrayDestroy(pBlocks); taosArrayDestroy(pSchemas); -// if (pRsp->blockNum == 0) { -// return -1; -// } - return 0; } From a732ec7b3c96edf605a83e2f91f7c0fb0d943b31 Mon Sep 17 00:00:00 2001 From: dapan1121 Date: Fri, 24 Mar 2023 10:36:01 +0800 Subject: [PATCH 17/65] fix: change float display mode --- tools/shell/src/shellEngine.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/shell/src/shellEngine.c b/tools/shell/src/shellEngine.c index 3080b15b8c..f9602aaa4d 100644 --- a/tools/shell/src/shellEngine.c +++ b/tools/shell/src/shellEngine.c @@ -541,7 +541,7 @@ void shellPrintField(const char *val, TAOS_FIELD *field, int32_t width, int32_t printf("%*" PRIu64, width, *((uint64_t *)val)); break; case TSDB_DATA_TYPE_FLOAT: - printf("%*.5f", width, GET_FLOAT_VAL(val)); + printf("%*ef", width, GET_FLOAT_VAL(val)); break; case TSDB_DATA_TYPE_DOUBLE: n = snprintf(buf, TSDB_MAX_BYTES_PER_ROW, "%*.9f", width, GET_DOUBLE_VAL(val)); From f18cfc25d6d1d367d172a7b6895e4f3c59b6d1b2 Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao@163.com> Date: Fri, 24 Mar 2023 11:06:05 +0800 Subject: [PATCH 18/65] fix:reset parameter ignoreExpried for fill history --- source/libs/executor/inc/executorimpl.h | 3 +++ source/libs/executor/src/executor.c | 25 +++++++------------ source/libs/executor/src/timewindowoperator.c | 4 +++ 3 files changed, 16 insertions(+), 16 deletions(-) diff --git a/source/libs/executor/inc/executorimpl.h b/source/libs/executor/inc/executorimpl.h index fd610b5bcd..3f519568c4 100644 --- a/source/libs/executor/inc/executorimpl.h +++ b/source/libs/executor/inc/executorimpl.h @@ -559,6 +559,7 @@ typedef struct SStreamIntervalOperatorInfo { STimeWindowAggSupp twAggSup; bool invertible; bool ignoreExpiredData; + bool ignoreExpiredDataSaved; SArray* pDelWins; // SWinRes int32_t delIndex; SSDataBlock* pDelRes; @@ -620,6 +621,7 @@ typedef struct SStreamSessionAggOperatorInfo { SPhysiNode* pPhyNode; // create new child bool isFinal; bool ignoreExpiredData; + bool ignoreExpiredDataSaved; SArray* pUpdated; SSHashObj* pStUpdated; } SStreamSessionAggOperatorInfo; @@ -637,6 +639,7 @@ typedef struct SStreamStateAggOperatorInfo { void* pDelIterator; SArray* pChildren; // cache for children's result; bool ignoreExpiredData; + bool ignoreExpiredDataSaved; SArray* pUpdated; SSHashObj* pSeUpdated; } SStreamStateAggOperatorInfo; diff --git a/source/libs/executor/src/executor.c b/source/libs/executor/src/executor.c index 86fcd319c5..db58bd6f68 100644 --- a/source/libs/executor/src/executor.c +++ b/source/libs/executor/src/executor.c @@ -888,7 +888,8 @@ int32_t qStreamSetParamForRecover(qTaskInfo_t tinfo) { pInfo->twAggSup.deleteMarkSaved = pInfo->twAggSup.deleteMark; pInfo->twAggSup.calTrigger = STREAM_TRIGGER_AT_ONCE; pInfo->twAggSup.deleteMark = INT64_MAX; - + pInfo->ignoreExpiredDataSaved = pInfo->ignoreExpiredData; + pInfo->ignoreExpiredData = false; } else if (pOperator->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_SESSION || pOperator->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_SESSION || pOperator->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_SESSION) { @@ -904,6 +905,8 @@ int32_t qStreamSetParamForRecover(qTaskInfo_t tinfo) { pInfo->twAggSup.deleteMarkSaved = pInfo->twAggSup.deleteMark; pInfo->twAggSup.calTrigger = STREAM_TRIGGER_AT_ONCE; pInfo->twAggSup.deleteMark = INT64_MAX; + pInfo->ignoreExpiredDataSaved = pInfo->ignoreExpiredData; + pInfo->ignoreExpiredData = false; } else if (pOperator->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_STATE) { SStreamStateAggOperatorInfo* pInfo = pOperator->info; ASSERT(pInfo->twAggSup.calTrigger == STREAM_TRIGGER_AT_ONCE || @@ -917,6 +920,8 @@ int32_t qStreamSetParamForRecover(qTaskInfo_t tinfo) { pInfo->twAggSup.deleteMarkSaved = pInfo->twAggSup.deleteMark; pInfo->twAggSup.calTrigger = STREAM_TRIGGER_AT_ONCE; pInfo->twAggSup.deleteMark = INT64_MAX; + pInfo->ignoreExpiredDataSaved = pInfo->ignoreExpiredData; + pInfo->ignoreExpiredData = false; } // iterate operator tree @@ -944,35 +949,23 @@ int32_t qStreamRestoreParam(qTaskInfo_t tinfo) { pOperator->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL || pOperator->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL) { SStreamIntervalOperatorInfo* pInfo = pOperator->info; - /*ASSERT(pInfo->twAggSup.calTrigger == STREAM_TRIGGER_AT_ONCE);*/ - /*ASSERT(pInfo->twAggSup.deleteMark == INT64_MAX);*/ - pInfo->twAggSup.calTrigger = pInfo->twAggSup.calTriggerSaved; pInfo->twAggSup.deleteMark = pInfo->twAggSup.deleteMarkSaved; - /*ASSERT(pInfo->twAggSup.calTrigger == STREAM_TRIGGER_AT_ONCE ||*/ - /*pInfo->twAggSup.calTrigger == STREAM_TRIGGER_WINDOW_CLOSE);*/ + pInfo->ignoreExpiredData = pInfo->ignoreExpiredDataSaved; qInfo("restore stream param for interval: %d, %" PRId64, pInfo->twAggSup.calTrigger, pInfo->twAggSup.deleteMark); } else if (pOperator->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_SESSION || pOperator->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_SESSION || pOperator->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_SESSION) { SStreamSessionAggOperatorInfo* pInfo = pOperator->info; - /*ASSERT(pInfo->twAggSup.calTrigger == STREAM_TRIGGER_AT_ONCE);*/ - /*ASSERT(pInfo->twAggSup.deleteMark == INT64_MAX);*/ - pInfo->twAggSup.calTrigger = pInfo->twAggSup.calTriggerSaved; pInfo->twAggSup.deleteMark = pInfo->twAggSup.deleteMarkSaved; - /*ASSERT(pInfo->twAggSup.calTrigger == STREAM_TRIGGER_AT_ONCE ||*/ - /*pInfo->twAggSup.calTrigger == STREAM_TRIGGER_WINDOW_CLOSE);*/ + pInfo->ignoreExpiredData = pInfo->ignoreExpiredDataSaved; qInfo("restore stream param for session: %d, %" PRId64, pInfo->twAggSup.calTrigger, pInfo->twAggSup.deleteMark); } else if (pOperator->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_STATE) { SStreamStateAggOperatorInfo* pInfo = pOperator->info; - /*ASSERT(pInfo->twAggSup.calTrigger == STREAM_TRIGGER_AT_ONCE);*/ - /*ASSERT(pInfo->twAggSup.deleteMark == INT64_MAX);*/ - pInfo->twAggSup.calTrigger = pInfo->twAggSup.calTriggerSaved; pInfo->twAggSup.deleteMark = pInfo->twAggSup.deleteMarkSaved; - /*ASSERT(pInfo->twAggSup.calTrigger == STREAM_TRIGGER_AT_ONCE ||*/ - /*pInfo->twAggSup.calTrigger == STREAM_TRIGGER_WINDOW_CLOSE);*/ + pInfo->ignoreExpiredData = pInfo->ignoreExpiredDataSaved; qInfo("restore stream param for state: %d, %" PRId64, pInfo->twAggSup.calTrigger, pInfo->twAggSup.deleteMark); } diff --git a/source/libs/executor/src/timewindowoperator.c b/source/libs/executor/src/timewindowoperator.c index fef588a503..39cc055323 100644 --- a/source/libs/executor/src/timewindowoperator.c +++ b/source/libs/executor/src/timewindowoperator.c @@ -2769,6 +2769,7 @@ SOperatorInfo* createStreamFinalIntervalOperatorInfo(SOperatorInfo* downstream, pInfo->pPullDataMap = taosHashInit(64, hashFn, false, HASH_NO_LOCK); pInfo->pPullDataRes = createSpecialDataBlock(STREAM_RETRIEVE); pInfo->ignoreExpiredData = pIntervalPhyNode->window.igExpired; + pInfo->ignoreExpiredDataSaved = false; pInfo->pDelRes = createSpecialDataBlock(STREAM_DELETE_RESULT); pInfo->delIndex = 0; pInfo->pDelWins = taosArrayInit(4, sizeof(SWinKey)); @@ -3587,6 +3588,7 @@ SOperatorInfo* createStreamSessionAggOperatorInfo(SOperatorInfo* downstream, SPh pInfo->isFinal = false; pInfo->pPhyNode = pPhyNode; pInfo->ignoreExpiredData = pSessionNode->window.igExpired; + pInfo->ignoreExpiredDataSaved = false; pInfo->pUpdated = NULL; pInfo->pStUpdated = NULL; @@ -4112,6 +4114,7 @@ SOperatorInfo* createStreamStateAggOperatorInfo(SOperatorInfo* downstream, SPhys pInfo->pDelRes = createSpecialDataBlock(STREAM_DELETE_RESULT); pInfo->pChildren = NULL; pInfo->ignoreExpiredData = pStateNode->window.igExpired; + pInfo->ignoreExpiredDataSaved = false; pInfo->pUpdated = NULL; pInfo->pSeUpdated = NULL; @@ -4885,6 +4888,7 @@ SOperatorInfo* createStreamIntervalOperatorInfo(SOperatorInfo* downstream, SPhys pInfo->interval = interval; pInfo->twAggSup = twAggSupp; pInfo->ignoreExpiredData = pIntervalPhyNode->window.igExpired; + pInfo->ignoreExpiredDataSaved = false; pInfo->isFinal = false; SExprSupp* pSup = &pOperator->exprSupp; From dcaab41a27b1a6b40921dd35250f573a4473c690 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Fri, 24 Mar 2023 13:20:03 +0800 Subject: [PATCH 19/65] fix:error in optimize consume logic --- source/libs/parser/src/parInsertUtil.c | 68 ++- utils/test/c/tmq_taosx_ci.c | 556 ++++++++++++------------- 2 files changed, 336 insertions(+), 288 deletions(-) diff --git a/source/libs/parser/src/parInsertUtil.c b/source/libs/parser/src/parInsertUtil.c index 704f381afa..c8e4d8e6b6 100644 --- a/source/libs/parser/src/parInsertUtil.c +++ b/source/libs/parser/src/parInsertUtil.c @@ -611,8 +611,35 @@ static int bindFileds(SBoundColInfo* pBoundInfo, SSchema* pSchema, TAOS_FIELD* f return code; } +static bool isSameBindFileds(SBoundColInfo* pBoundInfo, SSchema* pSchema, TAOS_FIELD* fields, int numFields) { + int16_t lastColIdx = -1; // last column found + for (int i = 0; i < numFields; i++) { + SToken token; + token.z = fields[i].name; + token.n = strlen(fields[i].name); + + int16_t t = lastColIdx + 1; + int16_t index = insFindCol(&token, t, pBoundInfo->numOfCols, pSchema); + if (index < 0 && t > 0) { + index = insFindCol(&token, 0, t, pSchema); + } + if (index < 0) { + uError("can not find column name:%s", token.z); + return false; + } else { + lastColIdx = index; + if(pBoundInfo->pColIndex[i] != index){ + return false; + } + } + } + + return true; +} + int rawBlockBindData(SQuery* query, STableMeta* pTableMeta, void* data, SVCreateTbReq* pCreateTb, TAOS_FIELD* tFields, int numFields, bool needChangeLength) { + void* tmp = taosHashGet(((SVnodeModifyOpStmt*)(query->pRoot))->pTableBlockHashObj, &pTableMeta->uid, sizeof(pTableMeta->uid)); STableDataCxt* pTableCxt = NULL; int ret = insGetTableDataCxt(((SVnodeModifyOpStmt*)(query->pRoot))->pTableBlockHashObj, &pTableMeta->uid, sizeof(pTableMeta->uid), pTableMeta, &pCreateTb, &pTableCxt, true); @@ -620,19 +647,40 @@ int rawBlockBindData(SQuery* query, STableMeta* pTableMeta, void* data, SVCreate uError("insGetTableDataCxt error"); goto end; } - if (tFields != NULL) { - ret = bindFileds(&pTableCxt->boundColsInfo, getTableColumnSchema(pTableMeta), tFields, numFields); + + do { + if (tmp != NULL){ + if(!isSameBindFileds(&pTableCxt->boundColsInfo, getTableColumnSchema(pTableMeta), tFields, numFields)){ + char* fieldNames = (char*)taosMemoryCalloc(numFields, sizeof(tFields[0].name)); + for(int i = 0; i < numFields; i++){ + memcpy(fieldNames + i * sizeof(tFields[0].name), tFields[i].name, sizeof(tFields[0].name)); + } + ret = insGetTableDataCxt(((SVnodeModifyOpStmt*)(query->pRoot))->pTableBlockHashObj, fieldNames, + numFields * sizeof(tFields[0].name), pTableMeta, &pCreateTb, &pTableCxt, true); + taosMemoryFree(fieldNames); + if (ret != TSDB_CODE_SUCCESS) { + uError("insGetTableDataCxt inner error"); + goto end; + } + }else{ + break; + } + } + + if (tFields != NULL) { + ret = bindFileds(&pTableCxt->boundColsInfo, getTableColumnSchema(pTableMeta), tFields, numFields); + if (ret != TSDB_CODE_SUCCESS) { + uError("bindFileds error"); + goto end; + } + } + // no need to bind, because select * get all fields + ret = initTableColSubmitData(pTableCxt); if (ret != TSDB_CODE_SUCCESS) { - uError("bindFileds error"); + uError("initTableColSubmitData error"); goto end; } - } - // no need to bind, because select * get all fields - ret = initTableColSubmitData(pTableCxt); - if (ret != TSDB_CODE_SUCCESS) { - uError("initTableColSubmitData error"); - goto end; - } + }while(0); char* p = (char*)data; // | version | total length | total rows | total columns | flag seg| block group id | column schema | each column diff --git a/utils/test/c/tmq_taosx_ci.c b/utils/test/c/tmq_taosx_ci.c index 1f25eae366..a3df1106fd 100644 --- a/utils/test/c/tmq_taosx_ci.c +++ b/utils/test/c/tmq_taosx_ci.c @@ -102,284 +102,284 @@ int buildDatabase(TAOS* pConn, TAOS_RES* pRes) { /* test for TD-20612 end*/ - pRes = taos_query(pConn, - "create stable if not exists st1 (ts timestamp, c1 int, c2 float, c3 binary(16)) tags(t1 int, t3 " - "nchar(8), t4 bool)"); - if (taos_errno(pRes) != 0) { - printf("failed to create super table st1, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - pRes = taos_query(pConn, "create table if not exists ct0 using st1 tags(1000, \"ttt\", true)"); - if (taos_errno(pRes) != 0) { - printf("failed to create child table tu1, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - pRes = taos_query(pConn, "insert into ct0 values(1626006833400, 1, 2, 'a')"); - if (taos_errno(pRes) != 0) { - printf("failed to insert into ct0, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - pRes = taos_query(pConn, "create table if not exists ct1 using st1(t1) tags(2000)"); - if (taos_errno(pRes) != 0) { - printf("failed to create child table ct1, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - pRes = taos_query(pConn, "create table if not exists ct2 using st1(t1) tags(NULL)"); - if (taos_errno(pRes) != 0) { - printf("failed to create child table ct2, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - pRes = taos_query(pConn, "insert into ct1 values(1626006833600, 3, 4, 'b')"); - if (taos_errno(pRes) != 0) { - printf("failed to insert into ct1, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - pRes = taos_query(pConn, "create table if not exists ct3 using st1(t1) tags(3000)"); - if (taos_errno(pRes) != 0) { - printf("failed to create child table ct3, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - pRes = taos_query( - pConn, - "insert into ct3 values(1626006833600, 5, 6, 'c') ct1 values(1626006833601, 2, 3, 'sds') (1626006833602, 4, 5, " - "'ddd') ct0 values(1626006833603, 4, 3, 'hwj') ct1 values(now+5s, 23, 32, 's21ds')"); - if (taos_errno(pRes) != 0) { - printf("failed to insert into ct3, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - pRes = taos_query(pConn, "alter table st1 add column c4 bigint"); - if (taos_errno(pRes) != 0) { - printf("failed to alter super table st1, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - pRes = taos_query(pConn, "alter table st1 modify column c3 binary(64)"); - if (taos_errno(pRes) != 0) { - printf("failed to alter super table st1, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - pRes = taos_query(pConn, - "insert into ct3 values(1626006833605, 53, 63, 'cffffffffffffffffffffffffffff', 8989898899999) " - "(1626006833609, 51, 62, 'c333', 940)"); - if (taos_errno(pRes) != 0) { - printf("failed to insert into ct3, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - pRes = taos_query(pConn, "insert into ct3 select * from ct1"); - if (taos_errno(pRes) != 0) { - printf("failed to insert into ct3, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - pRes = taos_query(pConn, "alter table st1 add tag t2 binary(64)"); - if (taos_errno(pRes) != 0) { - printf("failed to alter super table st1, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - pRes = taos_query(pConn, "alter table ct3 set tag t1=5000"); - if (taos_errno(pRes) != 0) { - printf("failed to slter child table ct3, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - pRes = taos_query(pConn, "delete from abc1 .ct3 where ts < 1626006833606"); - if (taos_errno(pRes) != 0) { - printf("failed to insert into ct3, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - if (g_conf.dropTable) { - pRes = taos_query(pConn, "drop table ct3, ct1"); - if (taos_errno(pRes) != 0) { - printf("failed to drop child table ct3, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - pRes = taos_query(pConn, "drop table st1"); - if (taos_errno(pRes) != 0) { - printf("failed to drop super table st1, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - } - - pRes = taos_query(pConn, "create table if not exists n1(ts timestamp, c1 int, c2 nchar(4))"); - if (taos_errno(pRes) != 0) { - printf("failed to create normal table n1, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - pRes = taos_query(pConn, "alter table n1 add column c3 bigint"); - if (taos_errno(pRes) != 0) { - printf("failed to alter normal table n1, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - pRes = taos_query(pConn, "alter table n1 modify column c2 nchar(8)"); - if (taos_errno(pRes) != 0) { - printf("failed to alter normal table n1, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - pRes = taos_query(pConn, "alter table n1 rename column c3 cc3"); - if (taos_errno(pRes) != 0) { - printf("failed to alter normal table n1, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - pRes = taos_query(pConn, "alter table n1 comment 'hello'"); - if (taos_errno(pRes) != 0) { - printf("failed to alter normal table n1, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - pRes = taos_query(pConn, "alter table n1 drop column c1"); - if (taos_errno(pRes) != 0) { - printf("failed to alter normal table n1, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - pRes = taos_query(pConn, "insert into n1 values(now, 'eeee', 8989898899999) (now+9s, 'c333', 940)"); - if (taos_errno(pRes) != 0) { - printf("failed to insert into n1, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - if (g_conf.dropTable) { - pRes = taos_query(pConn, "drop table n1"); - if (taos_errno(pRes) != 0) { - printf("failed to drop normal table n1, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - } - - pRes = taos_query(pConn, "create table jt(ts timestamp, i int) tags(t json)"); - if (taos_errno(pRes) != 0) { - printf("failed to create super table jt, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - pRes = taos_query(pConn, "create table jt1 using jt tags('{\"k1\":1, \"k2\":\"hello\"}')"); - if (taos_errno(pRes) != 0) { - printf("failed to create super table jt, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - pRes = taos_query(pConn, "create table jt2 using jt tags('')"); - if (taos_errno(pRes) != 0) { - printf("failed to create super table jt2, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - pRes = taos_query(pConn, "insert into jt1 values(now, 1)"); - if (taos_errno(pRes) != 0) { - printf("failed to create super table jt1, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - pRes = taos_query(pConn, "insert into jt2 values(now, 11)"); - if (taos_errno(pRes) != 0) { - printf("failed to create super table jt2, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - if (g_conf.dropTable) { - pRes = taos_query(pConn, - "create stable if not exists st1 (ts timestamp, c1 int, c2 float, c3 binary(16)) tags(t1 int, t3 " - "nchar(8), t4 bool)"); - if (taos_errno(pRes) != 0) { - printf("failed to create super table st1, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - pRes = taos_query(pConn, "drop table st1"); - if (taos_errno(pRes) != 0) { - printf("failed to drop super table st1, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - } - - pRes = taos_query(pConn, - "create stable if not exists stt (ts timestamp, c1 int, c2 float, c3 binary(16)) tags(t1 int, t3 " - "nchar(8), t4 bool)"); - if (taos_errno(pRes) != 0) { - printf("failed to create super table stt, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - pRes = taos_query(pConn, - "create stable if not exists sttb (ts timestamp, c1 int, c2 float, c3 binary(16)) tags(t1 int, t3 " - "nchar(8), t4 bool)"); - if (taos_errno(pRes) != 0) { - printf("failed to create super table sttb, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - pRes = taos_query( - pConn, - "create table if not exists stt1 using stt tags(2, \"stt1\", true) sttb1 using sttb tags(4, \"sttb1\", true) " - "stt2 using stt tags(43, \"stt2\", false) sttb2 using sttb tags(54, \"sttb2\", true)"); - if (taos_errno(pRes) != 0) { - printf("failed to create child table stt1, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); - - pRes = - taos_query(pConn, - "insert into stt1 values(now + 2s, 3, 2, 'stt1') stt3 using stt tags(23, \"stt3\", true) values(now + " - "1s, 1, 2, 'stt3') sttb3 using sttb tags(4, \"sttb3\", true) values(now + 2s, 13, 22, 'sttb3') " - "stt4 using stt tags(433, \"stt4\", false) values(now + 3s, 21, 21, 'stt4') sttb4 using sttb " - "tags(543, \"sttb4\", true) values(now + 4s, 16, 25, 'sttb4')"); - if (taos_errno(pRes) != 0) { - printf("failed to create child table stt1, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); +// pRes = taos_query(pConn, +// "create stable if not exists st1 (ts timestamp, c1 int, c2 float, c3 binary(16)) tags(t1 int, t3 " +// "nchar(8), t4 bool)"); +// if (taos_errno(pRes) != 0) { +// printf("failed to create super table st1, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// pRes = taos_query(pConn, "create table if not exists ct0 using st1 tags(1000, \"ttt\", true)"); +// if (taos_errno(pRes) != 0) { +// printf("failed to create child table tu1, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// pRes = taos_query(pConn, "insert into ct0 values(1626006833400, 1, 2, 'a')"); +// if (taos_errno(pRes) != 0) { +// printf("failed to insert into ct0, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// pRes = taos_query(pConn, "create table if not exists ct1 using st1(t1) tags(2000)"); +// if (taos_errno(pRes) != 0) { +// printf("failed to create child table ct1, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// pRes = taos_query(pConn, "create table if not exists ct2 using st1(t1) tags(NULL)"); +// if (taos_errno(pRes) != 0) { +// printf("failed to create child table ct2, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// pRes = taos_query(pConn, "insert into ct1 values(1626006833600, 3, 4, 'b')"); +// if (taos_errno(pRes) != 0) { +// printf("failed to insert into ct1, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// pRes = taos_query(pConn, "create table if not exists ct3 using st1(t1) tags(3000)"); +// if (taos_errno(pRes) != 0) { +// printf("failed to create child table ct3, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// pRes = taos_query( +// pConn, +// "insert into ct3 values(1626006833600, 5, 6, 'c') ct1 values(1626006833601, 2, 3, 'sds') (1626006833602, 4, 5, " +// "'ddd') ct0 values(1626006833603, 4, 3, 'hwj') ct1 values(now+5s, 23, 32, 's21ds')"); +// if (taos_errno(pRes) != 0) { +// printf("failed to insert into ct3, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// pRes = taos_query(pConn, "alter table st1 add column c4 bigint"); +// if (taos_errno(pRes) != 0) { +// printf("failed to alter super table st1, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// pRes = taos_query(pConn, "alter table st1 modify column c3 binary(64)"); +// if (taos_errno(pRes) != 0) { +// printf("failed to alter super table st1, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// pRes = taos_query(pConn, +// "insert into ct3 values(1626006833605, 53, 63, 'cffffffffffffffffffffffffffff', 8989898899999) " +// "(1626006833609, 51, 62, 'c333', 940)"); +// if (taos_errno(pRes) != 0) { +// printf("failed to insert into ct3, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// pRes = taos_query(pConn, "insert into ct3 select * from ct1"); +// if (taos_errno(pRes) != 0) { +// printf("failed to insert into ct3, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// pRes = taos_query(pConn, "alter table st1 add tag t2 binary(64)"); +// if (taos_errno(pRes) != 0) { +// printf("failed to alter super table st1, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// pRes = taos_query(pConn, "alter table ct3 set tag t1=5000"); +// if (taos_errno(pRes) != 0) { +// printf("failed to slter child table ct3, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// pRes = taos_query(pConn, "delete from abc1 .ct3 where ts < 1626006833606"); +// if (taos_errno(pRes) != 0) { +// printf("failed to insert into ct3, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// if (g_conf.dropTable) { +// pRes = taos_query(pConn, "drop table ct3, ct1"); +// if (taos_errno(pRes) != 0) { +// printf("failed to drop child table ct3, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// pRes = taos_query(pConn, "drop table st1"); +// if (taos_errno(pRes) != 0) { +// printf("failed to drop super table st1, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// } +// +// pRes = taos_query(pConn, "create table if not exists n1(ts timestamp, c1 int, c2 nchar(4))"); +// if (taos_errno(pRes) != 0) { +// printf("failed to create normal table n1, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// pRes = taos_query(pConn, "alter table n1 add column c3 bigint"); +// if (taos_errno(pRes) != 0) { +// printf("failed to alter normal table n1, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// pRes = taos_query(pConn, "alter table n1 modify column c2 nchar(8)"); +// if (taos_errno(pRes) != 0) { +// printf("failed to alter normal table n1, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// pRes = taos_query(pConn, "alter table n1 rename column c3 cc3"); +// if (taos_errno(pRes) != 0) { +// printf("failed to alter normal table n1, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// pRes = taos_query(pConn, "alter table n1 comment 'hello'"); +// if (taos_errno(pRes) != 0) { +// printf("failed to alter normal table n1, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// pRes = taos_query(pConn, "alter table n1 drop column c1"); +// if (taos_errno(pRes) != 0) { +// printf("failed to alter normal table n1, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// pRes = taos_query(pConn, "insert into n1 values(now, 'eeee', 8989898899999) (now+9s, 'c333', 940)"); +// if (taos_errno(pRes) != 0) { +// printf("failed to insert into n1, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// if (g_conf.dropTable) { +// pRes = taos_query(pConn, "drop table n1"); +// if (taos_errno(pRes) != 0) { +// printf("failed to drop normal table n1, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// } +// +// pRes = taos_query(pConn, "create table jt(ts timestamp, i int) tags(t json)"); +// if (taos_errno(pRes) != 0) { +// printf("failed to create super table jt, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// pRes = taos_query(pConn, "create table jt1 using jt tags('{\"k1\":1, \"k2\":\"hello\"}')"); +// if (taos_errno(pRes) != 0) { +// printf("failed to create super table jt, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// pRes = taos_query(pConn, "create table jt2 using jt tags('')"); +// if (taos_errno(pRes) != 0) { +// printf("failed to create super table jt2, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// pRes = taos_query(pConn, "insert into jt1 values(now, 1)"); +// if (taos_errno(pRes) != 0) { +// printf("failed to create super table jt1, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// pRes = taos_query(pConn, "insert into jt2 values(now, 11)"); +// if (taos_errno(pRes) != 0) { +// printf("failed to create super table jt2, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// if (g_conf.dropTable) { +// pRes = taos_query(pConn, +// "create stable if not exists st1 (ts timestamp, c1 int, c2 float, c3 binary(16)) tags(t1 int, t3 " +// "nchar(8), t4 bool)"); +// if (taos_errno(pRes) != 0) { +// printf("failed to create super table st1, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// pRes = taos_query(pConn, "drop table st1"); +// if (taos_errno(pRes) != 0) { +// printf("failed to drop super table st1, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// } +// +// pRes = taos_query(pConn, +// "create stable if not exists stt (ts timestamp, c1 int, c2 float, c3 binary(16)) tags(t1 int, t3 " +// "nchar(8), t4 bool)"); +// if (taos_errno(pRes) != 0) { +// printf("failed to create super table stt, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// pRes = taos_query(pConn, +// "create stable if not exists sttb (ts timestamp, c1 int, c2 float, c3 binary(16)) tags(t1 int, t3 " +// "nchar(8), t4 bool)"); +// if (taos_errno(pRes) != 0) { +// printf("failed to create super table sttb, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// pRes = taos_query( +// pConn, +// "create table if not exists stt1 using stt tags(2, \"stt1\", true) sttb1 using sttb tags(4, \"sttb1\", true) " +// "stt2 using stt tags(43, \"stt2\", false) sttb2 using sttb tags(54, \"sttb2\", true)"); +// if (taos_errno(pRes) != 0) { +// printf("failed to create child table stt1, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); +// +// pRes = +// taos_query(pConn, +// "insert into stt1 values(now + 2s, 3, 2, 'stt1') stt3 using stt tags(23, \"stt3\", true) values(now + " +// "1s, 1, 2, 'stt3') sttb3 using sttb tags(4, \"sttb3\", true) values(now + 2s, 13, 22, 'sttb3') " +// "stt4 using stt tags(433, \"stt4\", false) values(now + 3s, 21, 21, 'stt4') sttb4 using sttb " +// "tags(543, \"sttb4\", true) values(now + 4s, 16, 25, 'sttb4')"); +// if (taos_errno(pRes) != 0) { +// printf("failed to create child table stt1, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); return 0; } From 5d06474f2ed3b6f06115c96e6451a394702ad3d7 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Fri, 24 Mar 2023 14:43:29 +0800 Subject: [PATCH 20/65] fix:send data batch if consume wal where subscribe db --- source/common/src/tdataformat.c | 4 + source/dnode/vnode/src/tq/tq.c | 10 + source/dnode/vnode/src/tq/tqExec.c | 2 +- source/libs/parser/src/parInsertUtil.c | 146 ++----- utils/test/c/tmq_taosx_ci.c | 556 ++++++++++++------------- 5 files changed, 324 insertions(+), 394 deletions(-) diff --git a/source/common/src/tdataformat.c b/source/common/src/tdataformat.c index 2bb8708372..5c313b3b3b 100644 --- a/source/common/src/tdataformat.c +++ b/source/common/src/tdataformat.c @@ -2439,6 +2439,10 @@ _exit: int32_t tColDataAddValueByDataBlock(SColData *pColData, int8_t type, int32_t bytes, int32_t nRows, char *lengthOrbitmap, char *data) { int32_t code = 0; + if(data == NULL){ + code = tColDataAppendValueImpl[pColData->flag][CV_FLAG_NONE](pColData, NULL, 0); + goto _exit; + } if (IS_VAR_DATA_TYPE(type)) { // var-length data type for (int32_t i = 0; i < nRows; ++i) { diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index eb38de1c9a..437870ebd6 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -629,6 +629,14 @@ static int32_t extractDataForMq(STQ* pTq, STqHandle* pHandle, const SMqPollReq* // process meta if (pHead->msgType != TDMT_VND_SUBMIT) { + if(totalRows > 0) { + tqOffsetResetToLog(&taosxRsp.rspOffset, fetchVer - 1); + code = tqSendDataRsp(pTq, pMsg, pRequest, (SMqDataRsp*)&taosxRsp, TMQ_MSG_TYPE__TAOSX_RSP); + tDeleteSTaosxRsp(&taosxRsp); + taosMemoryFreeClear(pCkHead); + return code; + } + tqDebug("fetch meta msg, ver:%" PRId64 ", type:%s", pHead->version, TMSG_INFO(pHead->msgType)); tqOffsetResetToLog(&metaRsp.rspOffset, fetchVer); metaRsp.resMsgType = pHead->msgType; @@ -656,6 +664,8 @@ static int32_t extractDataForMq(STQ* pTq, STqHandle* pHandle, const SMqPollReq* if (tqTaosxScanLog(pTq, pHandle, submit, &taosxRsp, &totalRows) < 0) { tqError("tmq poll: tqTaosxScanLog error %" PRId64 ", in vgId:%d, subkey %s", consumerId, vgId, pRequest->subKey); + taosMemoryFreeClear(pCkHead); + tDeleteSTaosxRsp(&taosxRsp); return -1; } diff --git a/source/dnode/vnode/src/tq/tqExec.c b/source/dnode/vnode/src/tq/tqExec.c index 796f150cbc..cbcd918094 100644 --- a/source/dnode/vnode/src/tq/tqExec.c +++ b/source/dnode/vnode/src/tq/tqExec.c @@ -346,7 +346,7 @@ int32_t tqTaosxScanLog(STQ* pTq, STqHandle* pHandle, SPackedData submit, STaosxR SSDataBlock* pBlock = taosArrayGet(pBlocks, i); tqAddBlockDataToRsp(pBlock, (SMqDataRsp*)pRsp, taosArrayGetSize(pBlock->pDataBlock), pTq->pVnode->config.tsdbCfg.precision); - totalRows += pBlock->info.rows; + *totalRows += pBlock->info.rows; blockDataFreeRes(pBlock); SSchemaWrapper* pSW = taosArrayGetP(pSchemas, i); taosArrayPush(pRsp->blockSchema, &pSW); diff --git a/source/libs/parser/src/parInsertUtil.c b/source/libs/parser/src/parInsertUtil.c index c8e4d8e6b6..f96c81cf85 100644 --- a/source/libs/parser/src/parInsertUtil.c +++ b/source/libs/parser/src/parInsertUtil.c @@ -566,75 +566,14 @@ int32_t insBuildVgDataBlocks(SHashObj* pVgroupsHashObj, SArray* pVgDataCxtList, return code; } -static int bindFileds(SBoundColInfo* pBoundInfo, SSchema* pSchema, TAOS_FIELD* fields, int numFields) { - bool* pUseCols = taosMemoryCalloc(pBoundInfo->numOfCols, sizeof(bool)); - if (NULL == pUseCols) { - return TSDB_CODE_OUT_OF_MEMORY; - } - - pBoundInfo->numOfBound = 0; - - int16_t lastColIdx = -1; // last column found - int32_t code = TSDB_CODE_SUCCESS; +static bool findFileds(SSchema* pSchema, TAOS_FIELD* fields, int numFields) { for (int i = 0; i < numFields; i++) { - SToken token; - token.z = fields[i].name; - token.n = strlen(fields[i].name); - - int16_t t = lastColIdx + 1; - int16_t index = insFindCol(&token, t, pBoundInfo->numOfCols, pSchema); - if (index < 0 && t > 0) { - index = insFindCol(&token, 0, t, pSchema); - } - if (index < 0) { - uError("can not find column name:%s", token.z); - code = TSDB_CODE_PAR_INVALID_COLUMN; - break; - } else if (pUseCols[index]) { - code = TSDB_CODE_PAR_INVALID_COLUMN; - uError("duplicated column name:%s", token.z); - break; - } else { - lastColIdx = index; - pUseCols[index] = true; - pBoundInfo->pColIndex[pBoundInfo->numOfBound] = index; - ++pBoundInfo->numOfBound; + if(strcmp(pSchema->name, fields[i].name) == 0){ + return true; } } - if (TSDB_CODE_SUCCESS == code && !pUseCols[0]) { - uError("primary timestamp column can not be null:"); - code = TSDB_CODE_PAR_INVALID_COLUMN; - } - - taosMemoryFree(pUseCols); - return code; -} - -static bool isSameBindFileds(SBoundColInfo* pBoundInfo, SSchema* pSchema, TAOS_FIELD* fields, int numFields) { - int16_t lastColIdx = -1; // last column found - for (int i = 0; i < numFields; i++) { - SToken token; - token.z = fields[i].name; - token.n = strlen(fields[i].name); - - int16_t t = lastColIdx + 1; - int16_t index = insFindCol(&token, t, pBoundInfo->numOfCols, pSchema); - if (index < 0 && t > 0) { - index = insFindCol(&token, 0, t, pSchema); - } - if (index < 0) { - uError("can not find column name:%s", token.z); - return false; - } else { - lastColIdx = index; - if(pBoundInfo->pColIndex[i] != index){ - return false; - } - } - } - - return true; + return false; } int rawBlockBindData(SQuery* query, STableMeta* pTableMeta, void* data, SVCreateTbReq* pCreateTb, TAOS_FIELD* tFields, @@ -648,39 +587,13 @@ int rawBlockBindData(SQuery* query, STableMeta* pTableMeta, void* data, SVCreate goto end; } - do { - if (tmp != NULL){ - if(!isSameBindFileds(&pTableCxt->boundColsInfo, getTableColumnSchema(pTableMeta), tFields, numFields)){ - char* fieldNames = (char*)taosMemoryCalloc(numFields, sizeof(tFields[0].name)); - for(int i = 0; i < numFields; i++){ - memcpy(fieldNames + i * sizeof(tFields[0].name), tFields[i].name, sizeof(tFields[0].name)); - } - ret = insGetTableDataCxt(((SVnodeModifyOpStmt*)(query->pRoot))->pTableBlockHashObj, fieldNames, - numFields * sizeof(tFields[0].name), pTableMeta, &pCreateTb, &pTableCxt, true); - taosMemoryFree(fieldNames); - if (ret != TSDB_CODE_SUCCESS) { - uError("insGetTableDataCxt inner error"); - goto end; - } - }else{ - break; - } - } - - if (tFields != NULL) { - ret = bindFileds(&pTableCxt->boundColsInfo, getTableColumnSchema(pTableMeta), tFields, numFields); - if (ret != TSDB_CODE_SUCCESS) { - uError("bindFileds error"); - goto end; - } - } - // no need to bind, because select * get all fields + if(tmp == NULL){ ret = initTableColSubmitData(pTableCxt); if (ret != TSDB_CODE_SUCCESS) { uError("initTableColSubmitData error"); goto end; } - }while(0); + } char* p = (char*)data; // | version | total length | total rows | total columns | flag seg| block group id | column schema | each column @@ -708,35 +621,38 @@ int rawBlockBindData(SQuery* query, STableMeta* pTableMeta, void* data, SVCreate SSchema* pSchema = getTableColumnSchema(pTableCxt->pMeta); SBoundColInfo* boundInfo = &pTableCxt->boundColsInfo; - if (boundInfo->numOfBound != numOfCols) { - uError("boundInfo->numOfBound:%d != numOfCols:%d", boundInfo->numOfBound, numOfCols); + if (numFields != numOfCols) { + uError("numFields:%d != numOfCols:%d", numFields, numOfCols); ret = TSDB_CODE_INVALID_PARA; goto end; } for (int c = 0; c < boundInfo->numOfBound; ++c) { - SSchema* pColSchema = &pSchema[boundInfo->pColIndex[c]]; + SSchema* pColSchema = &pSchema[c]; SColData* pCol = taosArrayGet(pTableCxt->pData->aCol, c); + if(findFileds(pColSchema, tFields, numFields)){ + if (*fields != pColSchema->type && *(int32_t*)(fields + sizeof(int8_t)) != pColSchema->bytes) { + uError("type or bytes not equal"); + ret = TSDB_CODE_INVALID_PARA; + goto end; + } - if (*fields != pColSchema->type && *(int32_t*)(fields + sizeof(int8_t)) != pColSchema->bytes) { - uError("type or bytes not equal"); - ret = TSDB_CODE_INVALID_PARA; - goto end; - } + int8_t* offset = pStart; + if (IS_VAR_DATA_TYPE(pColSchema->type)) { + pStart += numOfRows * sizeof(int32_t); + } else { + pStart += BitmapLen(numOfRows); + } + char* pData = pStart; - int8_t* offset = pStart; - if (IS_VAR_DATA_TYPE(pColSchema->type)) { - pStart += numOfRows * sizeof(int32_t); - } else { - pStart += BitmapLen(numOfRows); - } - char* pData = pStart; - - tColDataAddValueByDataBlock(pCol, pColSchema->type, pColSchema->bytes, numOfRows, offset, pData); - fields += sizeof(int8_t) + sizeof(int32_t); - if (needChangeLength) { - pStart += htonl(colLength[c]); - } else { - pStart += colLength[c]; + tColDataAddValueByDataBlock(pCol, pColSchema->type, pColSchema->bytes, numOfRows, offset, pData); + fields += sizeof(int8_t) + sizeof(int32_t); + if (needChangeLength) { + pStart += htonl(colLength[c]); + } else { + pStart += colLength[c]; + } + }else{ + tColDataAddValueByDataBlock(pCol, pColSchema->type, pColSchema->bytes, numOfRows, NULL, NULL); } } diff --git a/utils/test/c/tmq_taosx_ci.c b/utils/test/c/tmq_taosx_ci.c index a3df1106fd..1f25eae366 100644 --- a/utils/test/c/tmq_taosx_ci.c +++ b/utils/test/c/tmq_taosx_ci.c @@ -102,284 +102,284 @@ int buildDatabase(TAOS* pConn, TAOS_RES* pRes) { /* test for TD-20612 end*/ -// pRes = taos_query(pConn, -// "create stable if not exists st1 (ts timestamp, c1 int, c2 float, c3 binary(16)) tags(t1 int, t3 " -// "nchar(8), t4 bool)"); -// if (taos_errno(pRes) != 0) { -// printf("failed to create super table st1, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// pRes = taos_query(pConn, "create table if not exists ct0 using st1 tags(1000, \"ttt\", true)"); -// if (taos_errno(pRes) != 0) { -// printf("failed to create child table tu1, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// pRes = taos_query(pConn, "insert into ct0 values(1626006833400, 1, 2, 'a')"); -// if (taos_errno(pRes) != 0) { -// printf("failed to insert into ct0, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// pRes = taos_query(pConn, "create table if not exists ct1 using st1(t1) tags(2000)"); -// if (taos_errno(pRes) != 0) { -// printf("failed to create child table ct1, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// pRes = taos_query(pConn, "create table if not exists ct2 using st1(t1) tags(NULL)"); -// if (taos_errno(pRes) != 0) { -// printf("failed to create child table ct2, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// pRes = taos_query(pConn, "insert into ct1 values(1626006833600, 3, 4, 'b')"); -// if (taos_errno(pRes) != 0) { -// printf("failed to insert into ct1, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// pRes = taos_query(pConn, "create table if not exists ct3 using st1(t1) tags(3000)"); -// if (taos_errno(pRes) != 0) { -// printf("failed to create child table ct3, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// pRes = taos_query( -// pConn, -// "insert into ct3 values(1626006833600, 5, 6, 'c') ct1 values(1626006833601, 2, 3, 'sds') (1626006833602, 4, 5, " -// "'ddd') ct0 values(1626006833603, 4, 3, 'hwj') ct1 values(now+5s, 23, 32, 's21ds')"); -// if (taos_errno(pRes) != 0) { -// printf("failed to insert into ct3, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// pRes = taos_query(pConn, "alter table st1 add column c4 bigint"); -// if (taos_errno(pRes) != 0) { -// printf("failed to alter super table st1, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// pRes = taos_query(pConn, "alter table st1 modify column c3 binary(64)"); -// if (taos_errno(pRes) != 0) { -// printf("failed to alter super table st1, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// pRes = taos_query(pConn, -// "insert into ct3 values(1626006833605, 53, 63, 'cffffffffffffffffffffffffffff', 8989898899999) " -// "(1626006833609, 51, 62, 'c333', 940)"); -// if (taos_errno(pRes) != 0) { -// printf("failed to insert into ct3, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// pRes = taos_query(pConn, "insert into ct3 select * from ct1"); -// if (taos_errno(pRes) != 0) { -// printf("failed to insert into ct3, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// pRes = taos_query(pConn, "alter table st1 add tag t2 binary(64)"); -// if (taos_errno(pRes) != 0) { -// printf("failed to alter super table st1, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// pRes = taos_query(pConn, "alter table ct3 set tag t1=5000"); -// if (taos_errno(pRes) != 0) { -// printf("failed to slter child table ct3, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// pRes = taos_query(pConn, "delete from abc1 .ct3 where ts < 1626006833606"); -// if (taos_errno(pRes) != 0) { -// printf("failed to insert into ct3, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// if (g_conf.dropTable) { -// pRes = taos_query(pConn, "drop table ct3, ct1"); -// if (taos_errno(pRes) != 0) { -// printf("failed to drop child table ct3, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// pRes = taos_query(pConn, "drop table st1"); -// if (taos_errno(pRes) != 0) { -// printf("failed to drop super table st1, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// } -// -// pRes = taos_query(pConn, "create table if not exists n1(ts timestamp, c1 int, c2 nchar(4))"); -// if (taos_errno(pRes) != 0) { -// printf("failed to create normal table n1, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// pRes = taos_query(pConn, "alter table n1 add column c3 bigint"); -// if (taos_errno(pRes) != 0) { -// printf("failed to alter normal table n1, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// pRes = taos_query(pConn, "alter table n1 modify column c2 nchar(8)"); -// if (taos_errno(pRes) != 0) { -// printf("failed to alter normal table n1, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// pRes = taos_query(pConn, "alter table n1 rename column c3 cc3"); -// if (taos_errno(pRes) != 0) { -// printf("failed to alter normal table n1, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// pRes = taos_query(pConn, "alter table n1 comment 'hello'"); -// if (taos_errno(pRes) != 0) { -// printf("failed to alter normal table n1, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// pRes = taos_query(pConn, "alter table n1 drop column c1"); -// if (taos_errno(pRes) != 0) { -// printf("failed to alter normal table n1, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// pRes = taos_query(pConn, "insert into n1 values(now, 'eeee', 8989898899999) (now+9s, 'c333', 940)"); -// if (taos_errno(pRes) != 0) { -// printf("failed to insert into n1, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// if (g_conf.dropTable) { -// pRes = taos_query(pConn, "drop table n1"); -// if (taos_errno(pRes) != 0) { -// printf("failed to drop normal table n1, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// } -// -// pRes = taos_query(pConn, "create table jt(ts timestamp, i int) tags(t json)"); -// if (taos_errno(pRes) != 0) { -// printf("failed to create super table jt, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// pRes = taos_query(pConn, "create table jt1 using jt tags('{\"k1\":1, \"k2\":\"hello\"}')"); -// if (taos_errno(pRes) != 0) { -// printf("failed to create super table jt, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// pRes = taos_query(pConn, "create table jt2 using jt tags('')"); -// if (taos_errno(pRes) != 0) { -// printf("failed to create super table jt2, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// pRes = taos_query(pConn, "insert into jt1 values(now, 1)"); -// if (taos_errno(pRes) != 0) { -// printf("failed to create super table jt1, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// pRes = taos_query(pConn, "insert into jt2 values(now, 11)"); -// if (taos_errno(pRes) != 0) { -// printf("failed to create super table jt2, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// if (g_conf.dropTable) { -// pRes = taos_query(pConn, -// "create stable if not exists st1 (ts timestamp, c1 int, c2 float, c3 binary(16)) tags(t1 int, t3 " -// "nchar(8), t4 bool)"); -// if (taos_errno(pRes) != 0) { -// printf("failed to create super table st1, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// pRes = taos_query(pConn, "drop table st1"); -// if (taos_errno(pRes) != 0) { -// printf("failed to drop super table st1, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// } -// -// pRes = taos_query(pConn, -// "create stable if not exists stt (ts timestamp, c1 int, c2 float, c3 binary(16)) tags(t1 int, t3 " -// "nchar(8), t4 bool)"); -// if (taos_errno(pRes) != 0) { -// printf("failed to create super table stt, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// pRes = taos_query(pConn, -// "create stable if not exists sttb (ts timestamp, c1 int, c2 float, c3 binary(16)) tags(t1 int, t3 " -// "nchar(8), t4 bool)"); -// if (taos_errno(pRes) != 0) { -// printf("failed to create super table sttb, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// pRes = taos_query( -// pConn, -// "create table if not exists stt1 using stt tags(2, \"stt1\", true) sttb1 using sttb tags(4, \"sttb1\", true) " -// "stt2 using stt tags(43, \"stt2\", false) sttb2 using sttb tags(54, \"sttb2\", true)"); -// if (taos_errno(pRes) != 0) { -// printf("failed to create child table stt1, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); -// -// pRes = -// taos_query(pConn, -// "insert into stt1 values(now + 2s, 3, 2, 'stt1') stt3 using stt tags(23, \"stt3\", true) values(now + " -// "1s, 1, 2, 'stt3') sttb3 using sttb tags(4, \"sttb3\", true) values(now + 2s, 13, 22, 'sttb3') " -// "stt4 using stt tags(433, \"stt4\", false) values(now + 3s, 21, 21, 'stt4') sttb4 using sttb " -// "tags(543, \"sttb4\", true) values(now + 4s, 16, 25, 'sttb4')"); -// if (taos_errno(pRes) != 0) { -// printf("failed to create child table stt1, reason:%s\n", taos_errstr(pRes)); -// return -1; -// } -// taos_free_result(pRes); + pRes = taos_query(pConn, + "create stable if not exists st1 (ts timestamp, c1 int, c2 float, c3 binary(16)) tags(t1 int, t3 " + "nchar(8), t4 bool)"); + if (taos_errno(pRes) != 0) { + printf("failed to create super table st1, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, "create table if not exists ct0 using st1 tags(1000, \"ttt\", true)"); + if (taos_errno(pRes) != 0) { + printf("failed to create child table tu1, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, "insert into ct0 values(1626006833400, 1, 2, 'a')"); + if (taos_errno(pRes) != 0) { + printf("failed to insert into ct0, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, "create table if not exists ct1 using st1(t1) tags(2000)"); + if (taos_errno(pRes) != 0) { + printf("failed to create child table ct1, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, "create table if not exists ct2 using st1(t1) tags(NULL)"); + if (taos_errno(pRes) != 0) { + printf("failed to create child table ct2, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, "insert into ct1 values(1626006833600, 3, 4, 'b')"); + if (taos_errno(pRes) != 0) { + printf("failed to insert into ct1, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, "create table if not exists ct3 using st1(t1) tags(3000)"); + if (taos_errno(pRes) != 0) { + printf("failed to create child table ct3, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query( + pConn, + "insert into ct3 values(1626006833600, 5, 6, 'c') ct1 values(1626006833601, 2, 3, 'sds') (1626006833602, 4, 5, " + "'ddd') ct0 values(1626006833603, 4, 3, 'hwj') ct1 values(now+5s, 23, 32, 's21ds')"); + if (taos_errno(pRes) != 0) { + printf("failed to insert into ct3, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, "alter table st1 add column c4 bigint"); + if (taos_errno(pRes) != 0) { + printf("failed to alter super table st1, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, "alter table st1 modify column c3 binary(64)"); + if (taos_errno(pRes) != 0) { + printf("failed to alter super table st1, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, + "insert into ct3 values(1626006833605, 53, 63, 'cffffffffffffffffffffffffffff', 8989898899999) " + "(1626006833609, 51, 62, 'c333', 940)"); + if (taos_errno(pRes) != 0) { + printf("failed to insert into ct3, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, "insert into ct3 select * from ct1"); + if (taos_errno(pRes) != 0) { + printf("failed to insert into ct3, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, "alter table st1 add tag t2 binary(64)"); + if (taos_errno(pRes) != 0) { + printf("failed to alter super table st1, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, "alter table ct3 set tag t1=5000"); + if (taos_errno(pRes) != 0) { + printf("failed to slter child table ct3, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, "delete from abc1 .ct3 where ts < 1626006833606"); + if (taos_errno(pRes) != 0) { + printf("failed to insert into ct3, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + if (g_conf.dropTable) { + pRes = taos_query(pConn, "drop table ct3, ct1"); + if (taos_errno(pRes) != 0) { + printf("failed to drop child table ct3, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, "drop table st1"); + if (taos_errno(pRes) != 0) { + printf("failed to drop super table st1, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + } + + pRes = taos_query(pConn, "create table if not exists n1(ts timestamp, c1 int, c2 nchar(4))"); + if (taos_errno(pRes) != 0) { + printf("failed to create normal table n1, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, "alter table n1 add column c3 bigint"); + if (taos_errno(pRes) != 0) { + printf("failed to alter normal table n1, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, "alter table n1 modify column c2 nchar(8)"); + if (taos_errno(pRes) != 0) { + printf("failed to alter normal table n1, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, "alter table n1 rename column c3 cc3"); + if (taos_errno(pRes) != 0) { + printf("failed to alter normal table n1, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, "alter table n1 comment 'hello'"); + if (taos_errno(pRes) != 0) { + printf("failed to alter normal table n1, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, "alter table n1 drop column c1"); + if (taos_errno(pRes) != 0) { + printf("failed to alter normal table n1, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, "insert into n1 values(now, 'eeee', 8989898899999) (now+9s, 'c333', 940)"); + if (taos_errno(pRes) != 0) { + printf("failed to insert into n1, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + if (g_conf.dropTable) { + pRes = taos_query(pConn, "drop table n1"); + if (taos_errno(pRes) != 0) { + printf("failed to drop normal table n1, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + } + + pRes = taos_query(pConn, "create table jt(ts timestamp, i int) tags(t json)"); + if (taos_errno(pRes) != 0) { + printf("failed to create super table jt, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, "create table jt1 using jt tags('{\"k1\":1, \"k2\":\"hello\"}')"); + if (taos_errno(pRes) != 0) { + printf("failed to create super table jt, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, "create table jt2 using jt tags('')"); + if (taos_errno(pRes) != 0) { + printf("failed to create super table jt2, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, "insert into jt1 values(now, 1)"); + if (taos_errno(pRes) != 0) { + printf("failed to create super table jt1, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, "insert into jt2 values(now, 11)"); + if (taos_errno(pRes) != 0) { + printf("failed to create super table jt2, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + if (g_conf.dropTable) { + pRes = taos_query(pConn, + "create stable if not exists st1 (ts timestamp, c1 int, c2 float, c3 binary(16)) tags(t1 int, t3 " + "nchar(8), t4 bool)"); + if (taos_errno(pRes) != 0) { + printf("failed to create super table st1, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, "drop table st1"); + if (taos_errno(pRes) != 0) { + printf("failed to drop super table st1, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + } + + pRes = taos_query(pConn, + "create stable if not exists stt (ts timestamp, c1 int, c2 float, c3 binary(16)) tags(t1 int, t3 " + "nchar(8), t4 bool)"); + if (taos_errno(pRes) != 0) { + printf("failed to create super table stt, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, + "create stable if not exists sttb (ts timestamp, c1 int, c2 float, c3 binary(16)) tags(t1 int, t3 " + "nchar(8), t4 bool)"); + if (taos_errno(pRes) != 0) { + printf("failed to create super table sttb, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query( + pConn, + "create table if not exists stt1 using stt tags(2, \"stt1\", true) sttb1 using sttb tags(4, \"sttb1\", true) " + "stt2 using stt tags(43, \"stt2\", false) sttb2 using sttb tags(54, \"sttb2\", true)"); + if (taos_errno(pRes) != 0) { + printf("failed to create child table stt1, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = + taos_query(pConn, + "insert into stt1 values(now + 2s, 3, 2, 'stt1') stt3 using stt tags(23, \"stt3\", true) values(now + " + "1s, 1, 2, 'stt3') sttb3 using sttb tags(4, \"sttb3\", true) values(now + 2s, 13, 22, 'sttb3') " + "stt4 using stt tags(433, \"stt4\", false) values(now + 3s, 21, 21, 'stt4') sttb4 using sttb " + "tags(543, \"sttb4\", true) values(now + 4s, 16, 25, 'sttb4')"); + if (taos_errno(pRes) != 0) { + printf("failed to create child table stt1, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); return 0; } From 4167e4c00cd031ba94f4663c53f9a606fa1b366b Mon Sep 17 00:00:00 2001 From: dapan1121 Date: Fri, 24 Mar 2023 15:20:27 +0800 Subject: [PATCH 21/65] fix: some columns sma are not set issue --- source/dnode/vnode/src/tsdb/tsdbRead.c | 17 ++- tests/parallel_test/cases.task | 1 + tests/script/tsim/query/nullColSma.sim | 139 +++++++++++++++++++++++++ 3 files changed, 154 insertions(+), 3 deletions(-) create mode 100644 tests/script/tsim/query/nullColSma.sim diff --git a/source/dnode/vnode/src/tsdb/tsdbRead.c b/source/dnode/vnode/src/tsdb/tsdbRead.c index 5ad9276c6c..7432d400e6 100644 --- a/source/dnode/vnode/src/tsdb/tsdbRead.c +++ b/source/dnode/vnode/src/tsdb/tsdbRead.c @@ -4507,6 +4507,7 @@ static void doFillNullColSMA(SBlockLoadSuppInfo* pSup, int32_t numOfRows, int32_ int32_t i = 0, j = 0; int32_t size = (int32_t)taosArrayGetSize(pSup->pColAgg); taosArrayInsert(pSup->pColAgg, 0, pTsAgg); + size++; while (j < numOfCols && i < size) { SColumnDataAgg* pAgg = taosArrayGet(pSup->pColAgg, i); @@ -4519,10 +4520,20 @@ static void doFillNullColSMA(SBlockLoadSuppInfo* pSup, int32_t numOfRows, int32_ if (pSup->colId[j] != PRIMARYKEY_TIMESTAMP_COL_ID) { SColumnDataAgg nullColAgg = {.colId = pSup->colId[j], .numOfNull = numOfRows}; taosArrayInsert(pSup->pColAgg, i, &nullColAgg); + i += 1; } j += 1; } } + + while (j < numOfCols) { + if (pSup->colId[j] != PRIMARYKEY_TIMESTAMP_COL_ID) { + SColumnDataAgg nullColAgg = {.colId = pSup->colId[j], .numOfNull = numOfRows}; + taosArrayInsert(pSup->pColAgg, i, &nullColAgg); + i += 1; + } + j++; + } } int32_t tsdbRetrieveDatablockSMA(STsdbReader* pReader, SSDataBlock* pDataBlock, bool* allHave) { @@ -4602,8 +4613,8 @@ int32_t tsdbRetrieveDatablockSMA(STsdbReader* pReader, SSDataBlock* pDataBlock, } else if (pAgg->colId < pSup->colId[j]) { i += 1; } else if (pSup->colId[j] < pAgg->colId) { - // ASSERT(pSup->colId[j] == PRIMARYKEY_TIMESTAMP_COL_ID); - pResBlock->pBlockAgg[pSup->slotId[j]] = &pSup->tsColAgg; + pResBlock->pBlockAgg[pSup->slotId[j]] = NULL; + *allHave = false; j += 1; } } @@ -4996,4 +5007,4 @@ void tsdbUntakeReadSnap(STsdbReader* pReader, STsdbReadSnap* pSnap, bool proacti void tsdbReaderSetId(STsdbReader* pReader, const char* idstr) { taosMemoryFreeClear(pReader->idStr); pReader->idStr = taosStrdup(idstr); -} \ No newline at end of file +} diff --git a/tests/parallel_test/cases.task b/tests/parallel_test/cases.task index 8c8d41d5b6..58fa0cbfab 100644 --- a/tests/parallel_test/cases.task +++ b/tests/parallel_test/cases.task @@ -871,6 +871,7 @@ ,,y,script,./test.sh -f tsim/query/emptyTsRange.sim ,,y,script,./test.sh -f tsim/query/partitionby.sim ,,y,script,./test.sh -f tsim/query/tableCount.sim +,,y,script,./test.sh -f tsim/query/nullColSma.sim ,,y,script,./test.sh -f tsim/qnode/basic1.sim ,,y,script,./test.sh -f tsim/snode/basic1.sim ,,y,script,./test.sh -f tsim/mnode/basic1.sim diff --git a/tests/script/tsim/query/nullColSma.sim b/tests/script/tsim/query/nullColSma.sim new file mode 100644 index 0000000000..886274e7e5 --- /dev/null +++ b/tests/script/tsim/query/nullColSma.sim @@ -0,0 +1,139 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sql connect + +$dbPrefix = m_in_db +$tbPrefix = m_in_tb +$mtPrefix = m_in_mt +$tbNum = 1 +$rowNum = 200 +$totalNum = 400 + +print =============== step1 +$i = 0 +$db = $dbPrefix . $i +$mt = $mtPrefix . $i + +sql drop database if exists $db +sql create database $db vgroups 1 maxrows 200 minrows 10; +sql use $db +sql create table $mt (ts timestamp, f1 int, f2 float) TAGS(tgcol int) + +print ====== start create child tables and insert data +$i = 0 +while $i < $tbNum + $tb = $tbPrefix . $i + sql create table $tb using $mt tags( $i ) + + $x = 0 + while $x < $rowNum + $cc = $x * 1 + $ms = 1601481600000 + $cc + + sql insert into $tb values ($ms , NULL , $x ) + $x = $x + 1 + endw + + $i = $i + 1 +endw + +$i = 1 +$tb = $tbPrefix . $i +sql create table $tb using $mt tags( $i ) + +$x = 0 +while $x < $rowNum + $cc = $x * 1 + $ms = 1601481600000 + $cc + + sql insert into $tb values ($ms , $x , NULL ) + $x = $x + 1 +endw + +sql flush database $db + +print =============== step2 +$i = 0 +$tb = $tbPrefix . $i +sql select max(f1) from $tb +if $rows != 1 then + return -1 +endi +if $data00 != NULL then + return -1 +endi + +$i = 1 +$tb = $tbPrefix . $i +sql select max(f2) from $tb +if $rows != 1 then + return -1 +endi +if $data00 != NULL then + return -1 +endi + +$rowNum = 10 + +print ====== insert more data +$i = 0 +while $i < $tbNum + $tb = $tbPrefix . $i + + $x = 0 + while $x < $rowNum + $cc = $x * 1 + $ms = 1601481700000 + $cc + + sql insert into $tb values ($ms , $x , $x ) + $x = $x + 1 + endw + + $i = $i + 1 +endw + +$i = 1 +$tb = $tbPrefix . $i +$x = 0 +while $x < $rowNum + $cc = $x * 1 + $ms = 1601481700000 + $cc + + sql insert into $tb values ($ms , $x , $x ) + $x = $x + 1 +endw + +sql flush database $db + +print =============== step3 +$i = 0 +$tb = $tbPrefix . $i +sql select max(f1) from $tb +if $rows != 1 then + return -1 +endi +if $data00 != 9 then + return -1 +endi + +$i = 1 +$tb = $tbPrefix . $i +sql select max(f2) from $tb +if $rows != 1 then + return -1 +endi +if $data00 != 9.00000 then + print $data00 + return -1 +endi + + +print =============== clear +#sql drop database $db +#sql select * from information_schema.ins_databases +#if $rows != 0 then +# return -1 +#endi + +system sh/exec.sh -n dnode1 -s stop -x SIGINT From 8f6ac6d7715db8a57daa0049b717c43a0764e1ab Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 24 Mar 2023 15:36:42 +0800 Subject: [PATCH 22/65] fix(tmq): wait for 2mins when subscribe topics. --- source/client/src/clientTmq.c | 3 ++- source/dnode/vnode/src/inc/tq.h | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/source/client/src/clientTmq.c b/source/client/src/clientTmq.c index 82da067d8e..111ca28cdc 100644 --- a/source/client/src/clientTmq.c +++ b/source/client/src/clientTmq.c @@ -1116,6 +1116,7 @@ _failed: } int32_t tmq_subscribe(tmq_t* tmq, const tmq_list_t* topic_list) { + const int32_t MAX_RETRY_COUNT = 120 * 2; // let's wait for 2 mins at most const SArray* container = &topic_list->container; int32_t sz = taosArrayGetSize(container); void* buf = NULL; @@ -1209,7 +1210,7 @@ int32_t tmq_subscribe(tmq_t* tmq, const tmq_list_t* topic_list) { int32_t retryCnt = 0; while (TSDB_CODE_MND_CONSUMER_NOT_READY == tmqAskEp(tmq, false)) { - if (retryCnt++ > 40) { + if (retryCnt++ > MAX_RETRY_COUNT) { goto FAIL; } diff --git a/source/dnode/vnode/src/inc/tq.h b/source/dnode/vnode/src/inc/tq.h index 379ea25ee6..1d7c213e1a 100644 --- a/source/dnode/vnode/src/inc/tq.h +++ b/source/dnode/vnode/src/inc/tq.h @@ -159,7 +159,7 @@ typedef struct { int32_t size; } STqOffsetHead; -STqOffsetStore* tqOffsetOpen(); +STqOffsetStore* tqOffsetOpen(STQ* pTq); void tqOffsetClose(STqOffsetStore*); STqOffset* tqOffsetRead(STqOffsetStore* pStore, const char* subscribeKey); int32_t tqOffsetWrite(STqOffsetStore* pStore, const STqOffset* pOffset); From e0d1a4054aba7fd56f684b4b116f21bd9468841b Mon Sep 17 00:00:00 2001 From: Shuduo Sang Date: Fri, 24 Mar 2023 17:58:14 +0800 Subject: [PATCH 23/65] fix: taosbenchmark support same min/max for main (#20621) --- cmake/taostools_CMakeLists.txt.in | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/taostools_CMakeLists.txt.in b/cmake/taostools_CMakeLists.txt.in index 87f0579f44..897ccdd158 100644 --- a/cmake/taostools_CMakeLists.txt.in +++ b/cmake/taostools_CMakeLists.txt.in @@ -2,7 +2,7 @@ # taos-tools ExternalProject_Add(taos-tools GIT_REPOSITORY https://github.com/taosdata/taos-tools.git - GIT_TAG d11f210 + GIT_TAG 04296a5 SOURCE_DIR "${TD_SOURCE_DIR}/tools/taos-tools" BINARY_DIR "" #BUILD_IN_SOURCE TRUE From 481091bae685c46b8a35ee0dfa0760dbfd99435f Mon Sep 17 00:00:00 2001 From: dapan1121 Date: Fri, 24 Mar 2023 19:01:19 +0800 Subject: [PATCH 24/65] fix: sma load issue --- source/dnode/vnode/src/tsdb/tsdbRead.c | 1 + 1 file changed, 1 insertion(+) diff --git a/source/dnode/vnode/src/tsdb/tsdbRead.c b/source/dnode/vnode/src/tsdb/tsdbRead.c index 7432d400e6..96bce02b67 100644 --- a/source/dnode/vnode/src/tsdb/tsdbRead.c +++ b/source/dnode/vnode/src/tsdb/tsdbRead.c @@ -4521,6 +4521,7 @@ static void doFillNullColSMA(SBlockLoadSuppInfo* pSup, int32_t numOfRows, int32_ SColumnDataAgg nullColAgg = {.colId = pSup->colId[j], .numOfNull = numOfRows}; taosArrayInsert(pSup->pColAgg, i, &nullColAgg); i += 1; + size++; } j += 1; } From 8081572ae1a2ea068dc7f749f11faa47e33e44d1 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Sat, 25 Mar 2023 11:31:19 +0800 Subject: [PATCH 25/65] fix:modify ci case if consume too fast --- tests/system-test/7-tmq/tmqDropNtb-snapshot0.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/system-test/7-tmq/tmqDropNtb-snapshot0.py b/tests/system-test/7-tmq/tmqDropNtb-snapshot0.py index 6c49fae299..8792bdd3ba 100644 --- a/tests/system-test/7-tmq/tmqDropNtb-snapshot0.py +++ b/tests/system-test/7-tmq/tmqDropNtb-snapshot0.py @@ -100,7 +100,7 @@ class TDTestCase: tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt)) - if not ((totalConsumeRows >= expectrowcnt * 3/4) and (totalConsumeRows < expectrowcnt)): + if not ((totalConsumeRows >= expectrowcnt * 3/4) and (totalConsumeRows <= expectrowcnt)): tdLog.exit("tmq consume rows error with snapshot = 0!") tdLog.info("wait subscriptions exit ....") From 8823997941d0630d96c34cc0495f160f6a7294fa Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Sat, 25 Mar 2023 15:56:22 +0800 Subject: [PATCH 26/65] fix:case error & add NONE for rows to col mode --- source/common/src/tdataformat.c | 4 +++- source/libs/parser/src/parInsertUtil.c | 5 +++++ tests/system-test/7-tmq/tmqDropNtb-snapshot0.py | 4 ++-- 3 files changed, 10 insertions(+), 3 deletions(-) diff --git a/source/common/src/tdataformat.c b/source/common/src/tdataformat.c index 5c313b3b3b..34808aa389 100644 --- a/source/common/src/tdataformat.c +++ b/source/common/src/tdataformat.c @@ -2440,7 +2440,9 @@ int32_t tColDataAddValueByDataBlock(SColData *pColData, int8_t type, int32_t byt char *data) { int32_t code = 0; if(data == NULL){ - code = tColDataAppendValueImpl[pColData->flag][CV_FLAG_NONE](pColData, NULL, 0); + for (int32_t i = 0; i < nRows; ++i) { + code = tColDataAppendValueImpl[pColData->flag][CV_FLAG_NONE](pColData, NULL, 0); + } goto _exit; } diff --git a/source/libs/parser/src/parInsertUtil.c b/source/libs/parser/src/parInsertUtil.c index f96c81cf85..4768231fb5 100644 --- a/source/libs/parser/src/parInsertUtil.c +++ b/source/libs/parser/src/parInsertUtil.c @@ -626,6 +626,11 @@ int rawBlockBindData(SQuery* query, STableMeta* pTableMeta, void* data, SVCreate ret = TSDB_CODE_INVALID_PARA; goto end; } + if (numFields > boundInfo->numOfBound) { + uError("numFields:%d > boundInfo->numOfBound:%d", numFields, boundInfo->numOfBound); + ret = TSDB_CODE_INVALID_PARA; + goto end; + } for (int c = 0; c < boundInfo->numOfBound; ++c) { SSchema* pColSchema = &pSchema[c]; SColData* pCol = taosArrayGet(pTableCxt->pData->aCol, c); diff --git a/tests/system-test/7-tmq/tmqDropNtb-snapshot0.py b/tests/system-test/7-tmq/tmqDropNtb-snapshot0.py index 8792bdd3ba..c8bcdd6235 100644 --- a/tests/system-test/7-tmq/tmqDropNtb-snapshot0.py +++ b/tests/system-test/7-tmq/tmqDropNtb-snapshot0.py @@ -131,7 +131,7 @@ class TDTestCase: 'batchNum': 100, 'startTs': 1640966400000, # 2022-01-01 00:00:00.000 'endTs': 0, - 'pollDelay': 10, + 'pollDelay': 20, 'showMsg': 1, 'showRow': 1, 'snapshot': 0} @@ -193,7 +193,7 @@ class TDTestCase: tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt)) - if not ((totalConsumeRows >= expectrowcnt / 2 * (1 + 3/4)) and (totalConsumeRows < expectrowcnt)): + if not ((totalConsumeRows >= expectrowcnt / 2 * (1 + 3/4)) and (totalConsumeRows <= expectrowcnt)): tdLog.exit("tmq consume rows error with snapshot = 0!") tdLog.info("wait subscriptions exit ....") From 357223d0653b06dfaf7e5d4074e81560bce47b88 Mon Sep 17 00:00:00 2001 From: chenhaoran Date: Sat, 25 Mar 2023 16:17:21 +0800 Subject: [PATCH 27/65] fix: recorect ci buildPath --- tests/parallel_test/run.sh | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/parallel_test/run.sh b/tests/parallel_test/run.sh index f0880bdb04..21499a7ca8 100755 --- a/tests/parallel_test/run.sh +++ b/tests/parallel_test/run.sh @@ -167,8 +167,10 @@ function run_thread() { local case_build_san=`echo "$line"|cut -d, -f3` if [ "${case_build_san}" == "y" ]; then case_build_san="y" + DEBUGPATH="debugSan" elif [[ "${case_build_san}" == "n" ]] || [[ "${case_build_san}" == "" ]]; then case_build_san="n" + DEBUGPATH="debugNoSan" else usage exit 1 @@ -301,9 +303,9 @@ function run_thread() { if [ ! -z "$corefile" ]; then echo -e "\e[34m corefiles: $corefile \e[0m" local build_dir=$log_dir/build_${hosts[index]} - local remote_build_dir="${workdirs[index]}/TDengine/debug/build" + local remote_build_dir="${workdirs[index]}/TDengine/{DEBUGPATH}/build" if [ $ent -ne 0 ]; then - remote_build_dir="${workdirs[index]}/TDinternal/debug/build" + remote_build_dir="${workdirs[index]}/TDinternal/{DEBUGPATH}/build" fi mkdir $build_dir 2>/dev/null if [ $? -eq 0 ]; then From 0c00b4e0b54285ace0292ebd4c5b8cf2a1913f4a Mon Sep 17 00:00:00 2001 From: chenhaoran Date: Sat, 25 Mar 2023 16:19:29 +0800 Subject: [PATCH 28/65] fix: recorect ci buildPath --- tests/parallel_test/run.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/parallel_test/run.sh b/tests/parallel_test/run.sh index 21499a7ca8..de343752c6 100755 --- a/tests/parallel_test/run.sh +++ b/tests/parallel_test/run.sh @@ -303,10 +303,10 @@ function run_thread() { if [ ! -z "$corefile" ]; then echo -e "\e[34m corefiles: $corefile \e[0m" local build_dir=$log_dir/build_${hosts[index]} - local remote_build_dir="${workdirs[index]}/TDengine/{DEBUGPATH}/build" - if [ $ent -ne 0 ]; then - remote_build_dir="${workdirs[index]}/TDinternal/{DEBUGPATH}/build" - fi + local remote_build_dir="${workdirs[index]}/{DEBUGPATH}/build" + # if [ $ent -ne 0 ]; then + # remote_build_dir="${workdirs[index]}/{DEBUGPATH}/build" + # fi mkdir $build_dir 2>/dev/null if [ $? -eq 0 ]; then # scp build binary From 5a2e3c1603800d263bd886c9d190df509c1c8258 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Mon, 27 Mar 2023 10:32:30 +0800 Subject: [PATCH 29/65] fix:add test case for raw_block interface --- source/libs/parser/src/parInsertUtil.c | 6 +- tests/parallel_test/cases.task | 1 + .../7-tmq/raw_block_interface_test.py | 54 ++++++++ utils/test/c/CMakeLists.txt | 9 ++ utils/test/c/write_raw_block_test.c | 126 ++++++++++++++++++ 5 files changed, 193 insertions(+), 3 deletions(-) create mode 100644 tests/system-test/7-tmq/raw_block_interface_test.py create mode 100644 utils/test/c/write_raw_block_test.c diff --git a/source/libs/parser/src/parInsertUtil.c b/source/libs/parser/src/parInsertUtil.c index 4768231fb5..0ed32afbbc 100644 --- a/source/libs/parser/src/parInsertUtil.c +++ b/source/libs/parser/src/parInsertUtil.c @@ -621,12 +621,12 @@ int rawBlockBindData(SQuery* query, STableMeta* pTableMeta, void* data, SVCreate SSchema* pSchema = getTableColumnSchema(pTableCxt->pMeta); SBoundColInfo* boundInfo = &pTableCxt->boundColsInfo; - if (numFields != numOfCols) { + if (tFields != NULL && numFields != numOfCols) { uError("numFields:%d != numOfCols:%d", numFields, numOfCols); ret = TSDB_CODE_INVALID_PARA; goto end; } - if (numFields > boundInfo->numOfBound) { + if (tFields != NULL && numFields > boundInfo->numOfBound) { uError("numFields:%d > boundInfo->numOfBound:%d", numFields, boundInfo->numOfBound); ret = TSDB_CODE_INVALID_PARA; goto end; @@ -634,7 +634,7 @@ int rawBlockBindData(SQuery* query, STableMeta* pTableMeta, void* data, SVCreate for (int c = 0; c < boundInfo->numOfBound; ++c) { SSchema* pColSchema = &pSchema[c]; SColData* pCol = taosArrayGet(pTableCxt->pData->aCol, c); - if(findFileds(pColSchema, tFields, numFields)){ + if(tFields == NULL || findFileds(pColSchema, tFields, numFields)){ if (*fields != pColSchema->type && *(int32_t*)(fields + sizeof(int8_t)) != pColSchema->bytes) { uError("type or bytes not equal"); ret = TSDB_CODE_INVALID_PARA; diff --git a/tests/parallel_test/cases.task b/tests/parallel_test/cases.task index 58fa0cbfab..c2f8e75e79 100644 --- a/tests/parallel_test/cases.task +++ b/tests/parallel_test/cases.task @@ -93,6 +93,7 @@ ,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/dataFromTsdbNWal.py ,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/dataFromTsdbNWal-multiCtb.py ,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/tmq_taosx.py +,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/raw_block_interface_test.py ,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/stbTagFilter-multiCtb.py ,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/tmqSubscribeStb-r3.py -N 5 ,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/tmq3mnodeSwitch.py -N 6 -M 3 diff --git a/tests/system-test/7-tmq/raw_block_interface_test.py b/tests/system-test/7-tmq/raw_block_interface_test.py new file mode 100644 index 0000000000..1e89de1cce --- /dev/null +++ b/tests/system-test/7-tmq/raw_block_interface_test.py @@ -0,0 +1,54 @@ + +import taos +import sys +import time +import socket +import os +import threading + +from util.log import * +from util.sql import * +from util.cases import * +from util.dnodes import * +from util.common import * +sys.path.append("./7-tmq") +from tmqCommon import * + +class TDTestCase: + def init(self, conn, logSql, replicaVar=1): + self.replicaVar = int(replicaVar) + tdLog.debug(f"start to excute {__file__}") + tdSql.init(conn.cursor()) + + def checkData(self): + tdSql.execute('use db_raw') + tdSql.query("select * from d1") + tdSql.checkRows(1) + tdSql.checkData(0, 1, 120) + + tdSql.query("select * from d2") + tdSql.checkRows(1) + tdSql.checkData(0, 1, None) + + return + + def check(self): + buildPath = tdCom.getBuildPath() + cmdStr = '%s/build/bin/write_raw_block_test'%(buildPath) + tdLog.info(cmdStr) + os.system(cmdStr) + + self.checkData() + + return + + def run(self): + tdSql.prepare() + self.check() + + def stop(self): + tdSql.close() + tdLog.success(f"{__file__} successfully executed") + +tdCases.addLinux(__file__, TDTestCase()) +tdCases.addWindows(__file__, TDTestCase()) diff --git a/utils/test/c/CMakeLists.txt b/utils/test/c/CMakeLists.txt index 6ca266c555..eda9c70f15 100644 --- a/utils/test/c/CMakeLists.txt +++ b/utils/test/c/CMakeLists.txt @@ -3,6 +3,7 @@ add_dependencies(tmq_demo taos) add_executable(tmq_sim tmqSim.c) add_executable(create_table createTable.c) add_executable(tmq_taosx_ci tmq_taosx_ci.c) +add_executable(write_raw_block_test write_raw_block_test.c) add_executable(sml_test sml_test.c) add_executable(get_db_name_test get_db_name_test.c) target_link_libraries( @@ -34,6 +35,14 @@ target_link_libraries( PUBLIC os ) +target_link_libraries( + write_raw_block_test + PUBLIC taos_static + PUBLIC util + PUBLIC common + PUBLIC os +) + target_link_libraries( sml_test PUBLIC taos_static diff --git a/utils/test/c/write_raw_block_test.c b/utils/test/c/write_raw_block_test.c new file mode 100644 index 0000000000..998b47a66e --- /dev/null +++ b/utils/test/c/write_raw_block_test.c @@ -0,0 +1,126 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * This program is free software: you can use, redistribute, and/or modify + * it under the terms of the GNU Affero General Public License, version 3 + * or later ("AGPL"), as published by the Free Software Foundation. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +#include +#include +#include +#include "taos.h" +#include "types.h" + +int buildStable(TAOS* pConn, TAOS_RES* pRes) { + pRes = taos_query(pConn, + "CREATE STABLE `meters` (`ts` TIMESTAMP, `current` INT, `voltage` INT, `phase` FLOAT) TAGS " + "(`groupid` INT, `location` VARCHAR(16))"); + if (taos_errno(pRes) != 0) { + printf("failed to create super table meters, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, "create table d0 using meters tags(1, 'San Francisco')"); + if (taos_errno(pRes) != 0) { + printf("failed to create child table d0, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, "insert into d0 (ts, current) values (now, 120)"); + if (taos_errno(pRes) != 0) { + printf("failed to insert into table d0, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, "create table d1 using meters tags(2, 'San Francisco')"); + if (taos_errno(pRes) != 0) { + printf("failed to create child table d1, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, "create table d2 using meters tags(3, 'San Francisco')"); + if (taos_errno(pRes) != 0) { + printf("failed to create child table d1, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + return 0; +} + +int32_t init_env() { + TAOS* pConn = taos_connect("localhost", "root", "taosdata", NULL, 0); + if (pConn == NULL) { + return -1; + } + + TAOS_RES* pRes = taos_query(pConn, "drop database if exists db_raw"); + if (taos_errno(pRes) != 0) { + printf("error in drop db_taosx, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, "create database if not exists db_raw vgroups 2"); + if (taos_errno(pRes) != 0) { + printf("error in create db_taosx, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + pRes = taos_query(pConn, "use db_raw"); + if (taos_errno(pRes) != 0) { + printf("error in create db_taosx, reason:%s\n", taos_errstr(pRes)); + return -1; + } + taos_free_result(pRes); + + buildStable(pConn, pRes); + + pRes = taos_query(pConn, "select * from d0"); + if (taos_errno(pRes) != 0) { + printf("error in drop db_taosx, reason:%s\n", taos_errstr(pRes)); + return -1; + } + void *data = NULL; + int32_t numOfRows = 0; + int error_code = taos_fetch_raw_block(pRes, &numOfRows, &data); + ASSERT(error_code == 0); + ASSERT(numOfRows == 1); + + taos_write_raw_block(pConn, numOfRows, data, "d1"); + + pRes = taos_query(pConn, "select ts,phase from d0"); + if (taos_errno(pRes) != 0) { + printf("error in drop db_taosx, reason:%s\n", taos_errstr(pRes)); + return -1; + } + error_code = taos_fetch_raw_block(pRes, &numOfRows, &data); + ASSERT(error_code == 0); + ASSERT(numOfRows == 1); + + int numFields = taos_num_fields(pRes); + TAOS_FIELD *fields = taos_fetch_fields(pRes); + taos_write_raw_block_with_fields(pConn, numOfRows, data, "d2", fields, numFields); + + taos_close(pConn); + return 0; +} + +int main(int argc, char* argv[]) { + if (init_env() < 0) { + return -1; + } +} From e1b9aac4c4f00eed05d6bf6ffe44ad29cb289a42 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Mon, 27 Mar 2023 10:41:25 +0800 Subject: [PATCH 30/65] fix:add test case for raw_block interface --- utils/test/c/write_raw_block_test.c | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/test/c/write_raw_block_test.c b/utils/test/c/write_raw_block_test.c index 998b47a66e..8e5dd62752 100644 --- a/utils/test/c/write_raw_block_test.c +++ b/utils/test/c/write_raw_block_test.c @@ -101,6 +101,7 @@ int32_t init_env() { ASSERT(numOfRows == 1); taos_write_raw_block(pConn, numOfRows, data, "d1"); + taos_free_result(pRes); pRes = taos_query(pConn, "select ts,phase from d0"); if (taos_errno(pRes) != 0) { @@ -114,6 +115,7 @@ int32_t init_env() { int numFields = taos_num_fields(pRes); TAOS_FIELD *fields = taos_fetch_fields(pRes); taos_write_raw_block_with_fields(pConn, numOfRows, data, "d2", fields, numFields); + taos_free_result(pRes); taos_close(pConn); return 0; From 5f9f5be6f4e5e49bc3c6d1a1c7f55566c6b91722 Mon Sep 17 00:00:00 2001 From: Xiaoyu Wang Date: Mon, 27 Mar 2023 11:06:10 +0800 Subject: [PATCH 31/65] fix: enterprise version displays the associated community version gitinfo --- cmake/cmake.version | 19 +++++++++++++++- include/util/version.h | 1 + source/dnode/mgmt/exe/dmMain.c | 22 ++++++++++-------- source/util/src/version.c.in | 1 + tools/shell/src/shellArguments.c | 39 +++++++++++++++++--------------- 5 files changed, 54 insertions(+), 28 deletions(-) diff --git a/cmake/cmake.version b/cmake/cmake.version index de85025a8c..42de285c8e 100644 --- a/cmake/cmake.version +++ b/cmake/cmake.version @@ -16,7 +16,7 @@ find_program(HAVE_GIT NAMES git) IF (DEFINED GITINFO) SET(TD_VER_GIT ${GITINFO}) ELSEIF (HAVE_GIT) - execute_process(COMMAND git log -1 --format=%H WORKING_DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR} OUTPUT_VARIABLE GIT_COMMITID) + execute_process(COMMAND git log -1 --format=%H WORKING_DIRECTORY ${TD_COMMUNITY_DIR} OUTPUT_VARIABLE GIT_COMMITID) #message(STATUS "git log result:${GIT_COMMITID}") IF (GIT_COMMITID) string (REGEX REPLACE "[\n\t\r]" "" GIT_COMMITID ${GIT_COMMITID}) @@ -30,6 +30,23 @@ ELSE () SET(TD_VER_GIT "no git commit id") ENDIF () +IF (DEFINED GITINFOI) + SET(TD_VER_GIT_INTERNAL ${GITINFOI}) +ELSEIF (HAVE_GIT) + execute_process(COMMAND git log -1 --format=%H WORKING_DIRECTORY ${PROJECT_SOURCE_DIR} OUTPUT_VARIABLE GIT_COMMITID) + message(STATUS "git log result:${GIT_COMMITID}") + IF (GIT_COMMITID) + string (REGEX REPLACE "[\n\t\r]" "" GIT_COMMITID ${GIT_COMMITID}) + SET(TD_VER_GIT_INTERNAL ${GIT_COMMITID}) + ELSE () + message(STATUS "not a git repository") + SET(TD_VER_GIT "no git commit id") + ENDIF () +ELSE () + message(STATUS "no git cmd") + SET(TD_VER_GIT_INTERNAL "no git commit id") +ENDIF () + IF (DEFINED VERDATE) SET(TD_VER_DATE ${VERDATE}) ELSE () diff --git a/include/util/version.h b/include/util/version.h index bac29e5baf..b241dd248b 100644 --- a/include/util/version.h +++ b/include/util/version.h @@ -23,6 +23,7 @@ extern "C" { extern char version[]; extern char compatible_version[]; extern char gitinfo[]; +extern char gitinfoOfInternal[]; extern char buildinfo[]; #ifdef __cplusplus diff --git a/source/dnode/mgmt/exe/dmMain.c b/source/dnode/mgmt/exe/dmMain.c index d3cffaa185..9a092e2df5 100644 --- a/source/dnode/mgmt/exe/dmMain.c +++ b/source/dnode/mgmt/exe/dmMain.c @@ -18,6 +18,7 @@ #include "mnode.h" #include "tconfig.h" #include "tglobal.h" +#include "version.h" // clang-format off #define DM_APOLLO_URL "The apollo string to use when configuring the server, such as: -a 'jsonFile:./tests/cfg.json', cfg.json text can be '{\"fqdn\":\"td1\"}'." @@ -76,28 +77,28 @@ void dmLogCrash(int signum, void *sigInfo, void *context) { taosIgnSignal(SIGINT); taosIgnSignal(SIGBREAK); -#ifndef WINDOWS +#ifndef WINDOWS taosIgnSignal(SIGBUS); #endif taosIgnSignal(SIGABRT); taosIgnSignal(SIGFPE); taosIgnSignal(SIGSEGV); - char *pMsg = NULL; + char *pMsg = NULL; const char *flags = "UTL FATAL "; ELogLevel level = DEBUG_FATAL; int32_t dflag = 255; - int64_t msgLen= -1; - + int64_t msgLen = -1; + if (tsEnableCrashReport) { if (taosGenCrashJsonMsg(signum, &pMsg, dmGetClusterId(), global.startTime)) { taosPrintLog(flags, level, dflag, "failed to generate crash json msg"); goto _return; } else { - msgLen = strlen(pMsg); + msgLen = strlen(pMsg); } } - + _return: taosLogCrashInfo("taosd", pMsg, msgLen, signum, sigInfo); @@ -123,7 +124,7 @@ static void dmSetSignalHandle() { #ifndef WINDOWS taosSetSignal(SIGBUS, dmLogCrash); -#endif +#endif taosSetSignal(SIGABRT, dmLogCrash); taosSetSignal(SIGFPE, dmLogCrash); taosSetSignal(SIGSEGV, dmLogCrash); @@ -134,7 +135,7 @@ static int32_t dmParseArgs(int32_t argc, char const *argv[]) { int32_t cmdEnvIndex = 0; if (argc < 2) return 0; - + global.envCmd = taosMemoryMalloc((argc - 1) * sizeof(char *)); memset(global.envCmd, 0, (argc - 1) * sizeof(char *)); for (int32_t i = 1; i < argc; ++i) { @@ -203,6 +204,9 @@ static void dmPrintVersion() { #endif printf("%s version: %s compatible_version: %s\n", releaseName, version, compatible_version); printf("gitinfo: %s\n", gitinfo); +#ifdef TD_ENTERPRISE + printf("gitinfoOfInternal: %s\n", gitinfoOfInternal); +#endif printf("buildInfo: %s\n", buildinfo); } @@ -284,7 +288,7 @@ int mainWindows(int argc, char **argv) { printf("failed to init memory dbg, error:%s\n", tstrerror(code)); return code; } - tsAsyncLog = false; + tsAsyncLog = false; printf("memory dbg enabled\n"); } #endif diff --git a/source/util/src/version.c.in b/source/util/src/version.c.in index cb307b57fc..71998e3321 100644 --- a/source/util/src/version.c.in +++ b/source/util/src/version.c.in @@ -1,6 +1,7 @@ char version[64] = "${TD_VER_NUMBER}"; char compatible_version[12] = "${TD_VER_COMPATIBLE}"; char gitinfo[48] = "${TD_VER_GIT}"; +char gitinfoOfInternal[48] = "${TD_VER_GIT_INTERNAL}"; char buildinfo[64] = "Built at ${TD_VER_DATE}"; void libtaos_${TD_LIB_VER_NUMBER}_${TD_VER_OSTYPE}_${TD_VER_CPUTYPE}_${TD_VER_VERTYPE}() {}; diff --git a/tools/shell/src/shellArguments.c b/tools/shell/src/shellArguments.c index c16769ccba..8111ccc0b1 100644 --- a/tools/shell/src/shellArguments.c +++ b/tools/shell/src/shellArguments.c @@ -18,17 +18,18 @@ #endif #include "shellInt.h" +#include "version.h" #ifndef CUS_NAME - char cusName[] = "TDengine"; +char cusName[] = "TDengine"; #endif #ifndef CUS_PROMPT - char cusPrompt[] = "taos"; +char cusPrompt[] = "taos"; #endif #ifndef CUS_EMAIL - char cusEmail[] = ""; +char cusEmail[] = ""; #endif #if defined(CUS_NAME) || defined(CUS_PROMPT) || defined(CUS_EMAIL) @@ -58,9 +59,9 @@ #define SHELL_VERSION "Print program version." #ifdef WEBSOCKET -#define SHELL_DSN "The dsn to use when connecting to cloud server." -#define SHELL_REST "Use restful mode when connecting." -#define SHELL_TIMEOUT "Set the timeout for websocket query in seconds, default is 30." +#define SHELL_DSN "The dsn to use when connecting to cloud server." +#define SHELL_REST "Use restful mode when connecting." +#define SHELL_TIMEOUT "Set the timeout for websocket query in seconds, default is 30." #endif static int32_t shellParseSingleOpt(int32_t key, char *arg); @@ -145,7 +146,7 @@ static void shellParseArgsUseArgp(int argc, char *argv[]) { #endif #ifndef ARGP_ERR_UNKNOWN - #define ARGP_ERR_UNKNOWN E2BIG +#define ARGP_ERR_UNKNOWN E2BIG #endif static int32_t shellParseSingleOpt(int32_t key, char *arg) { @@ -246,8 +247,8 @@ int32_t shellParseArgsWithoutArgp(int argc, char *argv[]) { SShellArgs *pArgs = &shell.args; for (int i = 1; i < argc; i++) { - if (strcmp(argv[i], "--help") == 0 || strcmp(argv[i], "--usage") == 0 - || strcmp(argv[i], "-?") == 0 || strcmp(argv[i], "/?") == 0) { + if (strcmp(argv[i], "--help") == 0 || strcmp(argv[i], "--usage") == 0 || strcmp(argv[i], "-?") == 0 || + strcmp(argv[i], "/?") == 0) { shellParseSingleOpt('?', NULL); return 0; } @@ -263,10 +264,8 @@ int32_t shellParseArgsWithoutArgp(int argc, char *argv[]) { return -1; } - if (key[1] == 'h' || key[1] == 'P' || key[1] == 'u' - || key[1] == 'a' || key[1] == 'c' || key[1] == 's' - || key[1] == 'f' || key[1] == 'd' || key[1] == 'w' - || key[1] == 'n' || key[1] == 'l' || key[1] == 'N' + if (key[1] == 'h' || key[1] == 'P' || key[1] == 'u' || key[1] == 'a' || key[1] == 'c' || key[1] == 's' || + key[1] == 'f' || key[1] == 'd' || key[1] == 'w' || key[1] == 'n' || key[1] == 'l' || key[1] == 'N' #ifdef WEBSOCKET || key[1] == 'E' || key[1] == 'T' #endif @@ -282,12 +281,10 @@ int32_t shellParseArgsWithoutArgp(int argc, char *argv[]) { } shellParseSingleOpt(key[1], val); i++; - } else if (key[1] == 'p' || key[1] == 'A' || key[1] == 'C' - || key[1] == 'r' || key[1] == 'k' - || key[1] == 't' || key[1] == 'V' - || key[1] == '?' || key[1] == 1 + } else if (key[1] == 'p' || key[1] == 'A' || key[1] == 'C' || key[1] == 'r' || key[1] == 'k' || key[1] == 't' || + key[1] == 'V' || key[1] == '?' || key[1] == 1 #ifdef WEBSOCKET - ||key[1] == 'R' + || key[1] == 'R' #endif ) { shellParseSingleOpt(key[1], NULL); @@ -420,9 +417,15 @@ int32_t shellParseArgs(int32_t argc, char *argv[]) { sprintf(promptContinueFormat, "%%%zus> ", strlen(cusPrompt)); sprintf(shell.info.promptContinue, promptContinueFormat, " "); shell.info.promptSize = strlen(shell.info.promptHeader); +#ifdef TD_ENTERPRISE + snprintf(shell.info.programVersion, sizeof(shell.info.programVersion), + "version: %s compatible_version: %s\ngitinfo: %s\ngitinfoOfInternal: %s\nbuildInfo: %s", version, + compatible_version, gitinfo, gitinfoOfInternal, buildinfo); +#else snprintf(shell.info.programVersion, sizeof(shell.info.programVersion), "version: %s compatible_version: %s\ngitinfo: %s\nbuildInfo: %s", version, compatible_version, gitinfo, buildinfo); +#endif #if defined(_TD_WINDOWS_64) || defined(_TD_WINDOWS_32) shell.info.osname = "Windows"; From d0b6f23dc20afba8888f1e7fbc94fa3d64d38d01 Mon Sep 17 00:00:00 2001 From: Xiaoyu Wang Date: Mon, 27 Mar 2023 11:24:06 +0800 Subject: [PATCH 32/65] fix: enterprise version displays the associated community version gitinfo --- CMakeLists.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/CMakeLists.txt b/CMakeLists.txt index 566d4ad29d..fb2b306f65 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -10,6 +10,8 @@ if (NOT DEFINED TD_SOURCE_DIR) set( TD_SOURCE_DIR ${PROJECT_SOURCE_DIR} ) endif() +SET(TD_COMMUNITY_DIR ${PROJECT_SOURCE_DIR}) + set(TD_SUPPORT_DIR "${TD_SOURCE_DIR}/cmake") set(TD_CONTRIB_DIR "${TD_SOURCE_DIR}/contrib") From 3a12e36b4b8f19180cbffcd5b3ab95547b6865e0 Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao@163.com> Date: Mon, 27 Mar 2023 14:13:17 +0800 Subject: [PATCH 33/65] fix:add combine function for last row --- source/libs/function/src/builtins.c | 7 ++++--- tests/script/tsim/stream/distributeInterval0.sim | 2 +- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/source/libs/function/src/builtins.c b/source/libs/function/src/builtins.c index 4fcc9ac1ad..7924f16ee9 100644 --- a/source/libs/function/src/builtins.c +++ b/source/libs/function/src/builtins.c @@ -2500,7 +2500,8 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .sprocessFunc = firstLastScalarFunction, .pPartialFunc = "_last_row_partial", .pMergeFunc = "_last_row_merge", - .finalizeFunc = firstLastFinalize + .finalizeFunc = firstLastFinalize, + .combineFunc = lastCombine }, { .name = "_cache_last_row", @@ -2809,7 +2810,7 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { { .name = "mode", .type = FUNCTION_TYPE_MODE, - .classification = FUNC_MGT_AGG_FUNC | FUNC_MGT_SELECT_FUNC, + .classification = FUNC_MGT_AGG_FUNC | FUNC_MGT_SELECT_FUNC | FUNC_MGT_FORBID_STREAM_FUNC, .translateFunc = translateMode, .getEnvFunc = getModeFuncEnv, .initFunc = modeFunctionSetup, @@ -3212,7 +3213,7 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { { .name = "_block_dist", .type = FUNCTION_TYPE_BLOCK_DIST, - .classification = FUNC_MGT_AGG_FUNC, + .classification = FUNC_MGT_AGG_FUNC | FUNC_MGT_FORBID_STREAM_FUNC, .translateFunc = translateBlockDistFunc, .getEnvFunc = getBlockDistFuncEnv, .initFunc = blockDistSetup, diff --git a/tests/script/tsim/stream/distributeInterval0.sim b/tests/script/tsim/stream/distributeInterval0.sim index de1c72f8dc..1559d3d32b 100644 --- a/tests/script/tsim/stream/distributeInterval0.sim +++ b/tests/script/tsim/stream/distributeInterval0.sim @@ -284,7 +284,7 @@ sql create table ccc using st tags(3,2,2); sql create table ddd using st tags(4,2,2); -sql create stream streams1 ignore expired 0 fill_history 0 watermark 3s into streamst subtable(c) as select _wstart, c , count(*) c1 from st partition by c interval(1s) ; +sql create stream streams1 ignore expired 0 fill_history 0 watermark 3s into streamst subtable(c) as select _wstart, c , count(*) c1, last_row(b) c2 from st partition by c interval(1s) ; sql insert into aaa values(1648791221001,2,2,"/a1/aa/aa"); sql insert into bbb values(1648791221001,2,2,"/a1/aa/aa"); From 3c70008b4b888f0e42d5898d8f801147b22da0f3 Mon Sep 17 00:00:00 2001 From: Ganlin Zhao Date: Mon, 27 Mar 2023 15:41:50 +0800 Subject: [PATCH 34/65] fix: fix cfg load enviroment variables crash when environ is NULL --- source/util/src/tconfig.c | 2 ++ 1 file changed, 2 insertions(+) diff --git a/source/util/src/tconfig.c b/source/util/src/tconfig.c index c15bd96903..f811d2f203 100644 --- a/source/util/src/tconfig.c +++ b/source/util/src/tconfig.c @@ -637,6 +637,8 @@ int32_t cfgLoadFromEnvVar(SConfig *pConfig) { int32_t code = 0; char **pEnv = environ; line[1023] = 0; + + if (pEnv == NULL) return 0; while (*pEnv != NULL) { name = value = value2 = value3 = NULL; olen = vlen = vlen2 = vlen3 = 0; From c4791d58a65f5dcae375d25ebb1f3c8f76a8df4f Mon Sep 17 00:00:00 2001 From: Xiaoyu Wang Date: Mon, 27 Mar 2023 16:09:38 +0800 Subject: [PATCH 35/65] fix: create stream does not support event_window --- source/libs/parser/src/parCalcConst.c | 23 +++++++++++++---------- source/libs/parser/src/parTranslater.c | 12 ++++++++---- 2 files changed, 21 insertions(+), 14 deletions(-) diff --git a/source/libs/parser/src/parCalcConst.c b/source/libs/parser/src/parCalcConst.c index 33b8a686ae..a7f6d7cb3c 100644 --- a/source/libs/parser/src/parCalcConst.c +++ b/source/libs/parser/src/parCalcConst.c @@ -183,16 +183,18 @@ static int32_t calcConstProject(SNode* pProject, bool dual, SNode** pNew) { } else { code = scalarCalculateConstants(pProject, pNew); } - if (TSDB_CODE_SUCCESS == code && QUERY_NODE_VALUE == nodeType(*pNew) && NULL != pAssociation) { + if (TSDB_CODE_SUCCESS == code) { strcpy(((SExprNode*)*pNew)->aliasName, aliasName); - int32_t size = taosArrayGetSize(pAssociation); - for (int32_t i = 0; i < size; ++i) { - SNode** pCol = taosArrayGetP(pAssociation, i); - nodesDestroyNode(*pCol); - *pCol = nodesCloneNode(*pNew); - if (NULL == *pCol) { - code = TSDB_CODE_OUT_OF_MEMORY; - break; + if (QUERY_NODE_VALUE == nodeType(*pNew) && NULL != pAssociation) { + int32_t size = taosArrayGetSize(pAssociation); + for (int32_t i = 0; i < size; ++i) { + SNode** pCol = taosArrayGetP(pAssociation, i); + nodesDestroyNode(*pCol); + *pCol = nodesCloneNode(*pNew); + if (NULL == *pCol) { + code = TSDB_CODE_OUT_OF_MEMORY; + break; + } } } } @@ -383,7 +385,8 @@ static int32_t calcConstSetOpProjections(SCalcConstContext* pCxt, SSetOperator* int32_t index = 0; SNode* pProj = NULL; WHERE_EACH(pProj, pSetOp->pProjectionList) { - if (subquery && notRefByOrderBy((SColumnNode*)pProj, pSetOp->pOrderByList) && isSetUselessCol(pSetOp, index, (SExprNode*)pProj)) { + if (subquery && notRefByOrderBy((SColumnNode*)pProj, pSetOp->pOrderByList) && + isSetUselessCol(pSetOp, index, (SExprNode*)pProj)) { ERASE_NODE(pSetOp->pProjectionList); eraseSetOpChildProjection(pSetOp, index); continue; diff --git a/source/libs/parser/src/parTranslater.c b/source/libs/parser/src/parTranslater.c index 68c93f8ca9..726790443e 100644 --- a/source/libs/parser/src/parTranslater.c +++ b/source/libs/parser/src/parTranslater.c @@ -1342,8 +1342,8 @@ static bool isCountNotNullValue(SFunctionNode* pFunc) { // count(1) is rewritten as count(ts) for scannning optimization static int32_t rewriteCountNotNullValue(STranslateContext* pCxt, SFunctionNode* pCount) { SValueNode* pValue = (SValueNode*)nodesListGetNode(pCount->pParameterList, 0); - STableNode* pTable = NULL; - int32_t code = findTable(pCxt, NULL, &pTable); + STableNode* pTable = NULL; + int32_t code = findTable(pCxt, NULL, &pTable); if (TSDB_CODE_SUCCESS == code && QUERY_NODE_REAL_TABLE == nodeType(pTable)) { SColumnNode* pCol = (SColumnNode*)nodesMakeNode(QUERY_NODE_COLUMN); if (NULL == pCol) { @@ -1424,7 +1424,7 @@ static int32_t translateAggFunc(STranslateContext* pCxt, SFunctionNode* pFunc) { } if (isCountNotNullValue(pFunc)) { return rewriteCountNotNullValue(pCxt, pFunc); - } + } if (isCountTbname(pFunc)) { return rewriteCountTbname(pCxt, pFunc); } @@ -5923,11 +5923,15 @@ static int32_t addSubtableInfoToCreateStreamQuery(STranslateContext* pCxt, STabl return code; } +static bool isEventWindowQuery(SSelectStmt* pSelect) { + return NULL != pSelect->pWindow && QUERY_NODE_EVENT_WINDOW == nodeType(pSelect->pWindow); +} + static int32_t checkStreamQuery(STranslateContext* pCxt, SCreateStreamStmt* pStmt) { SSelectStmt* pSelect = (SSelectStmt*)pStmt->pQuery; if (TSDB_DATA_TYPE_TIMESTAMP != ((SExprNode*)nodesListGetNode(pSelect->pProjectionList, 0))->resType.type || !pSelect->isTimeLineResult || crossTableWithoutAggOper(pSelect) || NULL != pSelect->pOrderByList || - crossTableWithUdaf(pSelect)) { + crossTableWithUdaf(pSelect) || isEventWindowQuery(pSelect)) { return generateSyntaxErrMsgExt(&pCxt->msgBuf, TSDB_CODE_PAR_INVALID_STREAM_QUERY, "Unsupported stream query"); } if (NULL != pSelect->pSubtable && TSDB_DATA_TYPE_VARCHAR != ((SExprNode*)pSelect->pSubtable)->resType.type) { From cf2d169f751e60ffbf26824859a58d4fe985d3ba Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao@163.com> Date: Mon, 27 Mar 2023 16:15:15 +0800 Subject: [PATCH 36/65] feat:coverity scan --- include/libs/stream/tstream.h | 2 +- source/libs/stream/src/stream.c | 8 ++--- .../stream/distributeIntervalRetrive0.sim | 31 ++++++++++++++++--- 3 files changed, 31 insertions(+), 10 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 1d301623b1..dc374fa447 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -587,7 +587,7 @@ int32_t streamProcessDispatchReq(SStreamTask* pTask, SStreamDispatchReq* pReq, S int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, int32_t code); int32_t streamProcessRetrieveReq(SStreamTask* pTask, SStreamRetrieveReq* pReq, SRpcMsg* pMsg); -int32_t streamProcessRetrieveRsp(SStreamTask* pTask, SStreamRetrieveRsp* pRsp); +// int32_t streamProcessRetrieveRsp(SStreamTask* pTask, SStreamRetrieveRsp* pRsp); int32_t streamTryExec(SStreamTask* pTask); int32_t streamSchedExec(SStreamTask* pTask); diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 60729c4d0e..a4a02b7d65 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -275,7 +275,7 @@ int32_t streamProcessRetrieveReq(SStreamTask* pTask, SStreamRetrieveReq* pReq, S return 0; } -int32_t streamProcessRetrieveRsp(SStreamTask* pTask, SStreamRetrieveRsp* pRsp) { - // - return 0; -} +// int32_t streamProcessRetrieveRsp(SStreamTask* pTask, SStreamRetrieveRsp* pRsp) { +// // +// return 0; +// } diff --git a/tests/script/tsim/stream/distributeIntervalRetrive0.sim b/tests/script/tsim/stream/distributeIntervalRetrive0.sim index 529a2a1b30..a2d9c4ab45 100644 --- a/tests/script/tsim/stream/distributeIntervalRetrive0.sim +++ b/tests/script/tsim/stream/distributeIntervalRetrive0.sim @@ -3,7 +3,6 @@ system sh/deploy.sh -n dnode1 -i 1 system sh/deploy.sh -n dnode2 -i 2 system sh/exec.sh -n dnode1 -s start -#==system sh/exec.sh -n dnode1 -s start -v sleep 50 sql connect @@ -11,7 +10,6 @@ sql connect sql create dnode $hostname2 port 7200 system sh/exec.sh -n dnode2 -s start -#==system sh/exec.sh -n dnode2 -s start -v print ===== step1 $x = 0 @@ -37,14 +35,14 @@ endi print ===== step2 -sql create database test vgroups 4; +sql create database test vgroups 10; 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 IGNORE EXPIRED 0 into streamtST1 as select _wstart, count(*) c1, sum(a) c3 , max(b) c4, min(c) c5 from st interval(10s); +sql create stream stream_t1 trigger at_once IGNORE EXPIRED 0 delete_mark 10s into streamtST1 as select _wstart, count(*) c1, sum(a) c3 , max(b) c4, min(c) c5 from st interval(10s); sleep 1000 @@ -235,6 +233,29 @@ endi print loop3 over +sql insert into ts1 values(1648791200001,1,12,3,1.0); +sql insert into ts2 values(1648791200001,1,12,3,1.0); +sql insert into ts3 values(1648791200001,1,12,3,1.0); +sql insert into ts4 values(1648791200001,1,12,3,1.0); + +$loop_count = 0 +loop31: +sleep 1000 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +sql select * from streamtST1; + +if $rows <= 4 then + print =====rows=$rows + goto loop31 +endi + +print loop31 over + sql drop stream if exists streams1; sql drop database if exists test1; @@ -243,7 +264,7 @@ 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 t1 using st tags(1,1,1); sql create table t2 using st tags(2,2,2); -sql create stream streams1 trigger at_once IGNORE EXPIRED 0 into streamt1 as select _wstart as c0, count(*) c1, count(a) c2 from st interval(10s) ; +sql create stream streams1 trigger at_once IGNORE EXPIRED 0 delete_mark 20s into streamt1 as select _wstart as c0, count(*) c1, count(a) c2 from st interval(10s) ; sql insert into t1 values(1648791211000,1,2,3); From 9d207b3de2620b6927c092be4a2f119ec0751d42 Mon Sep 17 00:00:00 2001 From: Shuduo Sang Date: Mon, 27 Mar 2023 17:25:23 +0800 Subject: [PATCH 37/65] fix: taosbenchmark codacy complain for main (#20647) --- cmake/taostools_CMakeLists.txt.in | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/taostools_CMakeLists.txt.in b/cmake/taostools_CMakeLists.txt.in index 897ccdd158..40fa48b815 100644 --- a/cmake/taostools_CMakeLists.txt.in +++ b/cmake/taostools_CMakeLists.txt.in @@ -2,7 +2,7 @@ # taos-tools ExternalProject_Add(taos-tools GIT_REPOSITORY https://github.com/taosdata/taos-tools.git - GIT_TAG 04296a5 + GIT_TAG 2864326 SOURCE_DIR "${TD_SOURCE_DIR}/tools/taos-tools" BINARY_DIR "" #BUILD_IN_SOURCE TRUE From 5c128e2292c2c76c7f377797bcb53f83c0dcbd84 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 28 Mar 2023 09:27:48 +0800 Subject: [PATCH 38/65] refactor: do some internal refactor. --- include/util/tarray.h | 16 ----- source/dnode/mnode/impl/src/mndConsumer.c | 9 ++- source/dnode/vnode/src/tq/tq.c | 13 +++- source/dnode/vnode/src/tq/tqExec.c | 19 +++--- source/libs/executor/src/executorimpl.c | 2 +- source/util/src/tarray.c | 75 +---------------------- 6 files changed, 34 insertions(+), 100 deletions(-) diff --git a/include/util/tarray.h b/include/util/tarray.h index a8510e4bc8..278f9f6bab 100644 --- a/include/util/tarray.h +++ b/include/util/tarray.h @@ -69,14 +69,6 @@ void* taosArrayAddBatch(SArray* pArray, const void* pData, int32_t nEles); */ void taosArrayRemoveDuplicate(SArray* pArray, __compar_fn_t comparFn, void (*fp)(void*)); -/** - * - * @param pArray - * @param comparFn - * @param fp - */ -void taosArrayRemoveDuplicateP(SArray* pArray, __compar_fn_t comparFn, void (*fp)(void*)); - /** * add all element from the source array list into the destination * @param pArray @@ -252,14 +244,6 @@ void taosArraySortPWithExt(SArray* pArray, __ext_compar_fn_t fn, const void* par int32_t taosEncodeArray(void** buf, const SArray* pArray, FEncode encode); void* taosDecodeArray(const void* buf, SArray** pArray, FDecode decode, int32_t dataSz); -/** - * swap array - * @param a - * @param b - * @return - */ -void taosArraySwap(SArray* a, SArray* b); - #ifdef __cplusplus } #endif diff --git a/source/dnode/mnode/impl/src/mndConsumer.c b/source/dnode/mnode/impl/src/mndConsumer.c index 2f560910d4..616f69cd5b 100644 --- a/source/dnode/mnode/impl/src/mndConsumer.c +++ b/source/dnode/mnode/impl/src/mndConsumer.c @@ -601,6 +601,13 @@ static void* topicNameDup(void* p){ return taosStrdup((char*) p); } +static void freeItem(void* param) { + void* pItem = *(void**)param; + if (pItem != NULL) { + taosMemoryFree(pItem); + } +} + int32_t mndProcessSubscribeReq(SRpcMsg *pMsg) { SMnode *pMnode = pMsg->info.node; char *msgStr = pMsg->pCont; @@ -616,7 +623,7 @@ int32_t mndProcessSubscribeReq(SRpcMsg *pMsg) { int32_t code = -1; SArray *pTopicList = subscribe.topicNames; taosArraySort(pTopicList, taosArrayCompareString); - taosArrayRemoveDuplicateP(pTopicList, taosArrayCompareString, taosMemoryFree); + taosArrayRemoveDuplicate(pTopicList, taosArrayCompareString, freeItem); int32_t newTopicNum = taosArrayGetSize(pTopicList); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index de2732fcb5..5210a8cc66 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -885,6 +885,12 @@ int32_t tqProcessSubscribeReq(STQ* pTq, int64_t sversion, char* msg, int32_t msg tqInfo("vgId:%d switch consumer from Id:0x%" PRIx64 " to Id:0x%" PRIx64, req.vgId, pHandle->consumerId, req.newConsumerId); + // kill executing task + qTaskInfo_t pTaskInfo = pHandle->execHandle.task; + if (pTaskInfo != NULL) { +// qAsyncKillTask(pTaskInfo); + } + taosWLockLatch(&pTq->lock); atomic_store_32(&pHandle->epoch, -1); @@ -895,7 +901,12 @@ int32_t tqProcessSubscribeReq(STQ* pTq, int64_t sversion, char* msg, int32_t msg atomic_add_fetch_32(&pHandle->epoch, 1); if (pHandle->execHandle.subType == TOPIC_SUB_TYPE__COLUMN) { - qStreamCloseTsdbReader(pHandle->execHandle.task); + qStreamCloseTsdbReader(pTaskInfo); + } + + // reset the error code. + if (pHandle->execHandle.task != NULL) { + } taosWUnLockLatch(&pTq->lock); diff --git a/source/dnode/vnode/src/tq/tqExec.c b/source/dnode/vnode/src/tq/tqExec.c index a62101eb47..ce86a6757e 100644 --- a/source/dnode/vnode/src/tq/tqExec.c +++ b/source/dnode/vnode/src/tq/tqExec.c @@ -65,16 +65,17 @@ int32_t tqScanData(STQ* pTq, const STqHandle* pHandle, SMqDataRsp* pRsp, STqOffs const STqExecHandle* pExec = &pHandle->execHandle; qTaskInfo_t task = pExec->task; + int32_t vgId = TD_VID(pTq->pVnode); if (qStreamPrepareScan(task, pOffset, pHandle->execHandle.subType) < 0) { - tqDebug("prepare scan failed, return"); + tqDebug("prepare scan failed, return, consumer:0x%"PRIx64, pHandle->consumerId); if (pOffset->type == TMQ_OFFSET__LOG) { pRsp->rspOffset = *pOffset; return 0; } else { tqOffsetResetToLog(pOffset, pHandle->snapshotVer); if (qStreamPrepareScan(task, pOffset, pHandle->execHandle.subType) < 0) { - tqDebug("prepare scan failed, return"); + tqDebug("prepare scan failed, return, consumer:0x%"PRIx64, pHandle->consumerId); pRsp->rspOffset = *pOffset; return 0; } @@ -86,13 +87,14 @@ int32_t tqScanData(STQ* pTq, const STqHandle* pHandle, SMqDataRsp* pRsp, STqOffs SSDataBlock* pDataBlock = NULL; uint64_t ts = 0; - tqDebug("vgId:%d, tmq task start to execute", pTq->pVnode->config.vgId); + tqDebug("vgId:%d, tmq task start to execute, consumer:0x%"PRIx64, vgId, pHandle->consumerId); if (qExecTask(task, &pDataBlock, &ts) < 0) { - tqError("vgId:%d, task exec error since %s", pTq->pVnode->config.vgId, terrstr()); + tqError("vgId:%d, task exec error since %s, consumer:0x%" PRIx64, vgId, terrstr(), + pHandle->consumerId); return -1; } - tqDebug("consumer:0x%"PRIx64" vgId:%d, tmq task executed, get %p", pHandle->consumerId, pTq->pVnode->config.vgId, pDataBlock); + tqDebug("consumer:0x%"PRIx64" vgId:%d, tmq task executed, get %p", pHandle->consumerId, vgId, pDataBlock); // current scan should be stopped asap, since the rebalance occurs. if (pDataBlock == NULL) { @@ -115,15 +117,16 @@ int32_t tqScanData(STQ* pTq, const STqHandle* pHandle, SMqDataRsp* pRsp, STqOffs } if (pRsp->rspOffset.type == 0) { - tqError("expected rsp offset: type %d %" PRId64 " %" PRId64 " %" PRId64, pRsp->rspOffset.type, pRsp->rspOffset.ts, - pRsp->rspOffset.uid, pRsp->rspOffset.version); + tqError("vgId:%d, expected rsp offset: type %d %" PRId64 " %" PRId64 " %" PRId64, vgId, pRsp->rspOffset.type, + pRsp->rspOffset.ts, pRsp->rspOffset.uid, pRsp->rspOffset.version); return -1; } if (pRsp->withTbName || pRsp->withSchema) { - tqError("get column should not with meta:%d,%d", pRsp->withTbName, pRsp->withSchema); + tqError("vgId:%d, get column should not with meta:%d,%d", vgId, pRsp->withTbName, pRsp->withSchema); return -1; } + return 0; } diff --git a/source/libs/executor/src/executorimpl.c b/source/libs/executor/src/executorimpl.c index 67174c3267..24a26d575a 100644 --- a/source/libs/executor/src/executorimpl.c +++ b/source/libs/executor/src/executorimpl.c @@ -633,7 +633,7 @@ void setBlockSMAInfo(SqlFunctionCtx* pCtx, SExprInfo* pExprInfo, SSDataBlock* pB } } -bool isTaskKilled(SExecTaskInfo* pTaskInfo) { return (0 != pTaskInfo->code) ? true : false; } +bool isTaskKilled(SExecTaskInfo* pTaskInfo) { return (0 != pTaskInfo->code);} void setTaskKilled(SExecTaskInfo* pTaskInfo, int32_t rspCode) { pTaskInfo->code = rspCode; } diff --git a/source/util/src/tarray.c b/source/util/src/tarray.c index 7467fa2948..6c1d3e17bb 100644 --- a/source/util/src/tarray.c +++ b/source/util/src/tarray.c @@ -140,7 +140,7 @@ void taosArrayRemoveDuplicate(SArray* pArray, __compar_fn_t comparFn, void (*fp) taosArraySet(pArray, pos + 1, p2); memset(TARRAY_GET_ELEM(pArray, i), 0, pArray->elemSize); - pos += 1; + pos += 1; } else { pos += 1; } @@ -157,45 +157,6 @@ void taosArrayRemoveDuplicate(SArray* pArray, __compar_fn_t comparFn, void (*fp) pArray->size = pos + 1; } -void taosArrayRemoveDuplicateP(SArray* pArray, __compar_fn_t comparFn, void (*fp)(void*)) { - size_t size = pArray->size; - if (size <= 1) { - return; - } - - int32_t pos = 0; - for (int32_t i = 1; i < size; ++i) { - char* p1 = taosArrayGet(pArray, pos); - char* p2 = taosArrayGet(pArray, i); - - if (comparFn(p1, p2) == 0) { - // do nothing - } else { - if (pos + 1 != i) { - void* p = taosArrayGetP(pArray, pos + 1); - if (fp != NULL) { - fp(p); - } - - taosArraySet(pArray, pos + 1, p2); - memset(TARRAY_GET_ELEM(pArray, i), 0, pArray->elemSize); - pos += 1; - } else { - pos += 1; - } - } - } - - if (fp != NULL) { - for (int32_t i = pos + 1; i < pArray->size; ++i) { - void* p = taosArrayGetP(pArray, i); - fp(p); - } - } - - pArray->size = pos + 1; -} - void* taosArrayAddAll(SArray* pArray, const SArray* pInput) { if (pInput) { return taosArrayAddBatch(pArray, pInput->pData, (int32_t)taosArrayGetSize(pInput)); @@ -392,20 +353,6 @@ void taosArrayClearEx(SArray* pArray, void (*fp)(void*)) { pArray->size = 0; } -void taosArrayClearP(SArray* pArray, FDelete fp) { - if (pArray == NULL) return; - if (fp == NULL) { - pArray->size = 0; - return; - } - - for (int32_t i = 0; i < pArray->size; ++i) { - fp(*(void**)TARRAY_GET_ELEM(pArray, i)); - } - - pArray->size = 0; -} - void* taosArrayDestroy(SArray* pArray) { if (pArray) { taosMemoryFree(pArray->pData); @@ -495,6 +442,7 @@ static void taosArrayInsertSort(SArray* pArray, __ext_compar_fn_t fn, const void if (pArray->size <= 1) { return; } + for (int32_t i = 1; i <= pArray->size - 1; ++i) { for (int32_t j = i; j > 0; --j) { if (fn(taosArrayGetP(pArray, j), taosArrayGetP(pArray, j - 1), param) == -1) { @@ -507,7 +455,6 @@ static void taosArrayInsertSort(SArray* pArray, __ext_compar_fn_t fn, const void } } } - return; } int32_t taosEncodeArray(void** buf, const SArray* pArray, FEncode encode) { @@ -539,21 +486,3 @@ void taosArraySortPWithExt(SArray* pArray, __ext_compar_fn_t fn, const void* par taosArrayGetSize(pArray) > 8 ? taosArrayQuickSort(pArray, fn, param) : taosArrayInsertSort(pArray, fn, param); } -void taosArraySwap(SArray* a, SArray* b) { - if (a == NULL || b == NULL) return; - size_t t = a->size; - a->size = b->size; - b->size = t; - - uint32_t cap = a->capacity; - a->capacity = b->capacity; - b->capacity = cap; - - uint32_t elem = a->elemSize; - a->elemSize = b->elemSize; - b->elemSize = elem; - - void* data = a->pData; - a->pData = b->pData; - b->pData = data; -} From c158087d66c5d4e7c19cfebc5504ad2a8d8e60a3 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 28 Mar 2023 09:53:49 +0800 Subject: [PATCH 39/65] fix(tmq): kill the ongoing tsdb scans while transferring the ownership of vnode to other consumers. --- include/libs/executor/executor.h | 10 +--------- source/dnode/vnode/src/tq/tq.c | 7 +------ source/libs/executor/src/executor.c | 17 +++++++++++++++++ 3 files changed, 19 insertions(+), 15 deletions(-) diff --git a/include/libs/executor/executor.h b/include/libs/executor/executor.h index e76422ee34..19a6407b77 100644 --- a/include/libs/executor/executor.h +++ b/include/libs/executor/executor.h @@ -149,7 +149,6 @@ int32_t qGetQueryTableSchemaVersion(qTaskInfo_t tinfo, char* dbName, char* table * @param handle * @return */ - int32_t qExecTaskOpt(qTaskInfo_t tinfo, SArray* pResList, uint64_t* useconds, bool* hasMore, SLocalFetch* pLocal); int32_t qExecTask(qTaskInfo_t tinfo, SSDataBlock** pBlock, uint64_t* useconds); @@ -162,6 +161,7 @@ void qCleanExecTaskBlockBuf(qTaskInfo_t tinfo); * @return */ int32_t qAsyncKillTask(qTaskInfo_t tinfo, int32_t rspCode); +int32_t qKillTask(qTaskInfo_t tinfo, int32_t rspCode); bool qTaskIsExecuting(qTaskInfo_t qinfo); @@ -171,14 +171,6 @@ bool qTaskIsExecuting(qTaskInfo_t qinfo); */ void qDestroyTask(qTaskInfo_t tinfo); -/** - * Extract the qualified table id list, and than pass them to the TSDB driver to load the required table data blocks. - * - * @param iter the table iterator to traverse all tables belongs to a super table, or an invert index - * @return - */ -int32_t qGetQualifiedTableIdList(void* pTableList, const char* tagCond, int32_t tagCondLen, SArray* pTableIdList); - void qProcessRspMsg(void* parent, struct SRpcMsg* pMsg, struct SEpSet* pEpSet); int32_t qGetExplainExecInfo(qTaskInfo_t tinfo, SArray* pExecInfoList /*,int32_t* resNum, SExplainExecInfo** pRes*/); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 5210a8cc66..ccd8b885a0 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -888,7 +888,7 @@ int32_t tqProcessSubscribeReq(STQ* pTq, int64_t sversion, char* msg, int32_t msg // kill executing task qTaskInfo_t pTaskInfo = pHandle->execHandle.task; if (pTaskInfo != NULL) { -// qAsyncKillTask(pTaskInfo); + qKillTask(pTaskInfo, TSDB_CODE_SUCCESS); } taosWLockLatch(&pTq->lock); @@ -904,11 +904,6 @@ int32_t tqProcessSubscribeReq(STQ* pTq, int64_t sversion, char* msg, int32_t msg qStreamCloseTsdbReader(pTaskInfo); } - // reset the error code. - if (pHandle->execHandle.task != NULL) { - - } - taosWUnLockLatch(&pTq->lock); if (tqMetaSaveHandle(pTq, req.subKey, pHandle) < 0) { taosMemoryFree(req.qmsg); diff --git a/source/libs/executor/src/executor.c b/source/libs/executor/src/executor.c index 86fcd319c5..1ea2101aff 100644 --- a/source/libs/executor/src/executor.c +++ b/source/libs/executor/src/executor.c @@ -749,6 +749,23 @@ int32_t qAsyncKillTask(qTaskInfo_t qinfo, int32_t rspCode) { return TSDB_CODE_SUCCESS; } +int32_t qKillTask(qTaskInfo_t tinfo, int32_t rspCode) { + SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo; + if (pTaskInfo == NULL) { + return TSDB_CODE_QRY_INVALID_QHANDLE; + } + + qDebug("%s execTask async killed", GET_TASKID(pTaskInfo)); + setTaskKilled(pTaskInfo, rspCode); + + while(qTaskIsExecuting(pTaskInfo)) { + taosMsleep(10); + } + + pTaskInfo->code = rspCode; + return TSDB_CODE_SUCCESS; +} + bool qTaskIsExecuting(qTaskInfo_t qinfo) { SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)qinfo; if (NULL == pTaskInfo) { From 2198ae2060a856bdb32e305b2d85b8da296045cd Mon Sep 17 00:00:00 2001 From: Ganlin Zhao Date: Tue, 28 Mar 2023 10:49:49 +0800 Subject: [PATCH 40/65] fix(query): spread/elapsed using sma result for computation --- source/libs/function/src/builtins.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/libs/function/src/builtins.c b/source/libs/function/src/builtins.c index 4fcc9ac1ad..84de55653c 100644 --- a/source/libs/function/src/builtins.c +++ b/source/libs/function/src/builtins.c @@ -2375,7 +2375,7 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { { .name = "spread", .type = FUNCTION_TYPE_SPREAD, - .classification = FUNC_MGT_AGG_FUNC, + .classification = FUNC_MGT_AGG_FUNC | FUNC_MGT_SPECIAL_DATA_REQUIRED, .translateFunc = translateSpread, .dataRequiredFunc = statisDataRequired, .getEnvFunc = getSpreadFuncEnv, @@ -2417,7 +2417,7 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { { .name = "elapsed", .type = FUNCTION_TYPE_ELAPSED, - .classification = FUNC_MGT_AGG_FUNC | FUNC_MGT_TIMELINE_FUNC | FUNC_MGT_INTERVAL_INTERPO_FUNC | FUNC_MGT_FORBID_STREAM_FUNC, + .classification = FUNC_MGT_AGG_FUNC | FUNC_MGT_TIMELINE_FUNC | FUNC_MGT_INTERVAL_INTERPO_FUNC | FUNC_MGT_FORBID_STREAM_FUNC | FUNC_MGT_SPECIAL_DATA_REQUIRED, .dataRequiredFunc = statisDataRequired, .translateFunc = translateElapsed, .getEnvFunc = getElapsedFuncEnv, From 6c1515624a1e14549d9c1392399f0039e03ce050 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 28 Mar 2023 11:24:12 +0800 Subject: [PATCH 41/65] refactor: do some internal refactor. --- include/util/tarray.h | 2 ++ source/util/src/tarray.c | 18 ++++++++++++++++++ 2 files changed, 20 insertions(+) diff --git a/include/util/tarray.h b/include/util/tarray.h index 278f9f6bab..4bf24b46b9 100644 --- a/include/util/tarray.h +++ b/include/util/tarray.h @@ -208,6 +208,8 @@ void taosArrayDestroyP(SArray* pArray, FDelete fp); void taosArrayDestroyEx(SArray* pArray, FDelete fp); +void taosArraySwap(SArray* a, SArray* b); + /** * sort the array * @param pArray diff --git a/source/util/src/tarray.c b/source/util/src/tarray.c index 6c1d3e17bb..b15bb519e7 100644 --- a/source/util/src/tarray.c +++ b/source/util/src/tarray.c @@ -486,3 +486,21 @@ void taosArraySortPWithExt(SArray* pArray, __ext_compar_fn_t fn, const void* par taosArrayGetSize(pArray) > 8 ? taosArrayQuickSort(pArray, fn, param) : taosArrayInsertSort(pArray, fn, param); } +void taosArraySwap(SArray* a, SArray* b) { + if (a == NULL || b == NULL) return; + size_t t = a->size; + a->size = b->size; + b->size = t; + + uint32_t cap = a->capacity; + a->capacity = b->capacity; + b->capacity = cap; + + uint32_t elem = a->elemSize; + a->elemSize = b->elemSize; + b->elemSize = elem; + + void* data = a->pData; + a->pData = b->pData; + b->pData = data; +} \ No newline at end of file From c36c5253c59b83235b33d2b7e0fc39b92aa84039 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 28 Mar 2023 13:31:40 +0800 Subject: [PATCH 42/65] refactor: do some internal refactor. --- source/dnode/vnode/src/tsdb/tsdbRead.c | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/source/dnode/vnode/src/tsdb/tsdbRead.c b/source/dnode/vnode/src/tsdb/tsdbRead.c index 5ad9276c6c..8953e4b907 100644 --- a/source/dnode/vnode/src/tsdb/tsdbRead.c +++ b/source/dnode/vnode/src/tsdb/tsdbRead.c @@ -3929,8 +3929,13 @@ int32_t tsdbSetTableList(STsdbReader* pReader, const void* pTableList, int32_t n if (code) { return code; } - pReader->status.uidList.tableUidList = - (uint64_t*)taosMemoryRealloc(pReader->status.uidList.tableUidList, sizeof(uint64_t) * num); + + char* p1 = taosMemoryRealloc(pReader->status.uidList.tableUidList, sizeof(uint64_t) * num); + if (p1 == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + + pReader->status.uidList.tableUidList = (uint64_t*)p1; } taosHashClear(pReader->status.pTableMap); From 4fd3064fd9bd97bc2cece0579d0ca9ae15d654ab Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 28 Mar 2023 14:43:37 +0800 Subject: [PATCH 43/65] refactor: do some internal refactor. --- source/dnode/vnode/src/inc/vnodeInt.h | 2 +- source/dnode/vnode/src/tq/tq.c | 15 +-- source/dnode/vnode/src/tq/tqExec.c | 19 ++- source/dnode/vnode/src/tq/tqPush.c | 180 +++++++++++++++----------- 4 files changed, 119 insertions(+), 97 deletions(-) diff --git a/source/dnode/vnode/src/inc/vnodeInt.h b/source/dnode/vnode/src/inc/vnodeInt.h index 0fe7f9a773..8b01ba237f 100644 --- a/source/dnode/vnode/src/inc/vnodeInt.h +++ b/source/dnode/vnode/src/inc/vnodeInt.h @@ -193,7 +193,7 @@ STQ* tqOpen(const char* path, SVnode* pVnode); void tqClose(STQ*); int tqPushMsg(STQ*, void* msg, int32_t msgLen, tmsg_t msgType, int64_t ver); int tqRegisterPushEntry(STQ* pTq, void* pHandle, const SMqPollReq* pRequest, SRpcMsg* pRpcMsg, SMqDataRsp* pDataRsp, int32_t type); -int tqRemovePushEntry(STQ* pTq, const char* pKey, int32_t keyLen, uint64_t consumerId, bool rspConsumer); +int tqUnregisterPushEntry(STQ* pTq, const char* pKey, int32_t keyLen, uint64_t consumerId, bool rspConsumer); int tqCommit(STQ*); int32_t tqUpdateTbUidList(STQ* pTq, const SArray* tbUidList, bool isAdd); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index ccd8b885a0..2d2c70c84a 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -223,19 +223,6 @@ static int32_t doSendDataRsp(const SRpcHandleInfo* pRpcHandleInfo, const SMqData int32_t tqPushDataRsp(STQ* pTq, STqPushEntry* pPushEntry) { SMqDataRsp* pRsp = pPushEntry->pDataRsp; - -#if 0 - A(taosArrayGetSize(pRsp->blockData) == pRsp->blockNum); - A(taosArrayGetSize(pRsp->blockDataLen) == pRsp->blockNum); - - A(!pRsp->withSchema); - A(taosArrayGetSize(pRsp->blockSchema) == 0); - - if (pRsp->reqOffset.type == TMQ_OFFSET__LOG) { - A(pRsp->rspOffset.version > pRsp->reqOffset.version); - } -#endif - SMqRspHead* pHeader = &pPushEntry->pDataRsp->head; doSendDataRsp(&pPushEntry->info, pRsp, pHeader->epoch, pHeader->consumerId, pHeader->mqMsgType); @@ -895,7 +882,7 @@ int32_t tqProcessSubscribeReq(STQ* pTq, int64_t sversion, char* msg, int32_t msg atomic_store_32(&pHandle->epoch, -1); // remove if it has been register in the push manager, and return one empty block to consumer - tqRemovePushEntry(pTq, req.subKey, (int32_t)strlen(req.subKey), pHandle->consumerId, true); + tqUnregisterPushEntry(pTq, req.subKey, (int32_t)strlen(req.subKey), pHandle->consumerId, true); atomic_store_64(&pHandle->consumerId, req.newConsumerId); atomic_add_fetch_32(&pHandle->epoch, 1); diff --git a/source/dnode/vnode/src/tq/tqExec.c b/source/dnode/vnode/src/tq/tqExec.c index ce86a6757e..6845350346 100644 --- a/source/dnode/vnode/src/tq/tqExec.c +++ b/source/dnode/vnode/src/tq/tqExec.c @@ -62,6 +62,8 @@ static int32_t tqAddTbNameToRsp(const STQ* pTq, int64_t uid, STaosxRsp* pRsp, in } int32_t tqScanData(STQ* pTq, const STqHandle* pHandle, SMqDataRsp* pRsp, STqOffsetVal* pOffset) { + const int32_t MAX_ROWS_TO_RETURN = 4096; + const STqExecHandle* pExec = &pHandle->execHandle; qTaskInfo_t task = pExec->task; @@ -82,7 +84,8 @@ int32_t tqScanData(STQ* pTq, const STqHandle* pHandle, SMqDataRsp* pRsp, STqOffs } } - int32_t rowCnt = 0; + int32_t totalRows = 0; + while (1) { SSDataBlock* pDataBlock = NULL; uint64_t ts = 0; @@ -94,9 +97,7 @@ int32_t tqScanData(STQ* pTq, const STqHandle* pHandle, SMqDataRsp* pRsp, STqOffs return -1; } - tqDebug("consumer:0x%"PRIx64" vgId:%d, tmq task executed, get %p", pHandle->consumerId, vgId, pDataBlock); - - // current scan should be stopped asap, since the rebalance occurs. + // current scan should be stopped ASAP, since the re-balance occurs. if (pDataBlock == NULL) { break; } @@ -104,9 +105,12 @@ int32_t tqScanData(STQ* pTq, const STqHandle* pHandle, SMqDataRsp* pRsp, STqOffs tqAddBlockDataToRsp(pDataBlock, pRsp, pExec->numOfCols, pTq->pVnode->config.tsdbCfg.precision); pRsp->blockNum++; + tqDebug("vgId:%d, consumer:0x%" PRIx64 " tmq task executed, rows:%d, total blocks:%d", vgId, pHandle->consumerId, + pDataBlock->info.rows, pRsp->blockNum); + if (pOffset->type == TMQ_OFFSET__SNAPSHOT_DATA) { - rowCnt += pDataBlock->info.rows; - if (rowCnt >= 4096) { + totalRows += pDataBlock->info.rows; + if (totalRows >= MAX_ROWS_TO_RETURN) { break; } } @@ -127,6 +131,9 @@ int32_t tqScanData(STQ* pTq, const STqHandle* pHandle, SMqDataRsp* pRsp, STqOffs return -1; } + tqDebug("vgId:%d, consumer:0x%" PRIx64 " tmq task executed, rows:%d, total blocks:%d, rows:%d", vgId, pHandle->consumerId, + pRsp->blockNum, totalRows); + return 0; } diff --git a/source/dnode/vnode/src/tq/tqPush.c b/source/dnode/vnode/src/tq/tqPush.c index 797aeb3f04..e07ab06458 100644 --- a/source/dnode/vnode/src/tq/tqPush.c +++ b/source/dnode/vnode/src/tq/tqPush.c @@ -206,6 +206,87 @@ int32_t tqPushMsgNew(STQ* pTq, void* msg, int32_t msgLen, tmsg_t msgType, int64_ } #endif +typedef struct { + void* pKey; + int64_t keyLen; +} SItem; + +static void recordPushedEntry(SArray* cachedKey, void* pIter); + +static void freeItem(void* param) { + SItem* p = (SItem*) param; + taosMemoryFree(p->pKey); +} + +static void doRemovePushedEntry(SArray* pCachedKeys, STQ* pTq) { + int32_t vgId = TD_VID(pTq->pVnode); + size_t numOfKeys = taosArrayGetSize(pCachedKeys); + + for (int32_t i = 0; i < numOfKeys; i++) { + SItem* pItem = taosArrayGet(pCachedKeys, i); + if (taosHashRemove(pTq->pPushMgr, pItem->pKey, pItem->keyLen) != 0) { + tqError("vgId:%d, tq push hash remove key error, key: %s", vgId, (char*) pItem->pKey); + } + } + + if (numOfKeys > 0) { + tqDebug("vgId:%d, pushed %d items and remain:%d", vgId, numOfKeys, (int32_t)taosHashGetSize(pTq->pPushMgr)); + } +} + +static void doPushDataForEntry(void* pIter, STqExecHandle* pExec, STQ* pTq, int64_t ver, int32_t vgId, char* pData, + int32_t dataLen, SArray* pCachedKey) { + STqPushEntry* pPushEntry = *(STqPushEntry**)pIter; + + SMqDataRsp* pRsp = pPushEntry->pDataRsp; + if (pRsp->reqOffset.version >= ver) { + tqDebug("vgId:%d, push entry req version %" PRId64 ", while push version %" PRId64 ", skip", vgId, + pRsp->reqOffset.version, ver); + return; + } + + qTaskInfo_t pTaskInfo = pExec->task; + + // prepare scan mem data + SPackedData submit = { + .msgStr = pData, + .msgLen = dataLen, + .ver = ver, + }; + + if(qStreamSetScanMemData(pTaskInfo, submit) != 0){ + return; + } + + // here start to scan submit block to extract the subscribed data + int32_t totalRows = 0; + + while (1) { + SSDataBlock* pDataBlock = NULL; + uint64_t ts = 0; + if (qExecTask(pTaskInfo, &pDataBlock, &ts) < 0) { + tqDebug("vgId:%d, tq exec error since %s", vgId, terrstr()); + } + + if (pDataBlock == NULL) { + break; + } + + tqAddBlockDataToRsp(pDataBlock, pRsp, pExec->numOfCols, pTq->pVnode->config.tsdbCfg.precision); + pRsp->blockNum++; + totalRows += pDataBlock->info.rows; + } + + tqDebug("vgId:%d, tq handle push, subkey:%s, block num:%d, rows:%d", vgId, pPushEntry->subKey, pRsp->blockNum, + totalRows); + + if (pRsp->blockNum > 0) { + tqOffsetResetToLog(&pRsp->rspOffset, ver); + tqPushDataRsp(pTq, pPushEntry); + recordPushedEntry(pCachedKey, pIter); + } +} + int tqPushMsg(STQ* pTq, void* msg, int32_t msgLen, tmsg_t msgType, int64_t ver) { void* pReq = POINTER_SHIFT(msg, sizeof(SSubmitReq2Msg)); int32_t len = msgLen - sizeof(SSubmitReq2Msg); @@ -220,24 +301,19 @@ int tqPushMsg(STQ* pTq, void* msg, int32_t msgLen, tmsg_t msgType, int64_t ver) tqDebug("vgId:%d tq push msg version:%" PRId64 " type:%s, head:%p, body:%p len:%d, numOfPushed consumers:%d", vgId, ver, TMSG_INFO(msgType), msg, pReq, len, numOfRegisteredPush); - SArray* cachedKeys = taosArrayInit(0, sizeof(void*)); - SArray* cachedKeyLens = taosArrayInit(0, sizeof(size_t)); - - void* data = taosMemoryMalloc(len); + void* data = taosMemoryMalloc(len); if (data == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; - tqError("failed to copy data for stream since out of memory"); - taosArrayDestroyP(cachedKeys, (FDelete)taosMemoryFree); - taosArrayDestroy(cachedKeyLens); - - // unlock + tqError("failed to copy data for stream since out of memory, vgId:%d, consumer:0x%"PRIx64, vgId); taosWUnLockLatch(&pTq->lock); return -1; } memcpy(data, pReq, len); - void* pIter = NULL; + SArray* cachedKey = taosArrayInit(0, sizeof(SItem)); + void* pIter = NULL; + while (1) { pIter = taosHashIterate(pTq->pPushMgr, pIter); if (pIter == NULL) { @@ -248,83 +324,28 @@ int tqPushMsg(STQ* pTq, void* msg, int32_t msgLen, tmsg_t msgType, int64_t ver) STqHandle* pHandle = taosHashGet(pTq->pHandle, pPushEntry->subKey, strlen(pPushEntry->subKey)); if (pHandle == NULL) { - tqDebug("vgId:%d, cannot find handle %s", pTq->pVnode->config.vgId, pPushEntry->subKey); - continue; - } - - SMqDataRsp* pRsp = pPushEntry->pDataRsp; - if (pRsp->reqOffset.version >= ver) { - tqDebug("vgId:%d, push entry req version %" PRId64 ", while push version %" PRId64 ", skip", vgId, - pRsp->reqOffset.version, ver); + tqDebug("vgId:%d, failed to find handle %s in pushing data to consumer, ignore", pTq->pVnode->config.vgId, pPushEntry->subKey); continue; } STqExecHandle* pExec = &pHandle->execHandle; - qTaskInfo_t task = pExec->task; - - // prepare scan mem data - SPackedData submit = { - .msgStr = data, - .msgLen = len, - .ver = ver, - }; - if(qStreamSetScanMemData(task, submit) != 0){ - continue; - } - - // here start to scan submit block to extract the subscribed data - while (1) { - SSDataBlock* pDataBlock = NULL; - uint64_t ts = 0; - if (qExecTask(task, &pDataBlock, &ts) < 0) { - tqDebug("vgId:%d, tq exec error since %s", vgId, terrstr()); - } - - if (pDataBlock == NULL) { - break; - } - - tqAddBlockDataToRsp(pDataBlock, pRsp, pExec->numOfCols, pTq->pVnode->config.tsdbCfg.precision); - pRsp->blockNum++; - } - - tqDebug("vgId:%d, tq handle push, subkey:%s, block num:%d", vgId, pPushEntry->subKey, pRsp->blockNum); - if (pRsp->blockNum > 0) { - // set offset - tqOffsetResetToLog(&pRsp->rspOffset, ver); - - // remove from hash - size_t kLen; - void* key = taosHashGetKey(pIter, &kLen); - void* keyCopy = taosMemoryCalloc(1, kLen + 1); - memcpy(keyCopy, key, kLen); - - taosArrayPush(cachedKeys, &keyCopy); - taosArrayPush(cachedKeyLens, &kLen); - - tqPushDataRsp(pTq, pPushEntry); - } + doPushDataForEntry(pIter, pExec, pTq, ver, vgId, data, len, cachedKey); } - // delete entry - for (int32_t i = 0; i < taosArrayGetSize(cachedKeys); i++) { - void* key = taosArrayGetP(cachedKeys, i); - size_t kLen = *(size_t*)taosArrayGet(cachedKeyLens, i); - if (taosHashRemove(pTq->pPushMgr, key, kLen) != 0) { - tqError("vgId:%d, tq push hash remove key error, key: %s", pTq->pVnode->config.vgId, (char*)key); - } - } - - taosArrayDestroyP(cachedKeys, (FDelete)taosMemoryFree); - taosArrayDestroy(cachedKeyLens); + doRemovePushedEntry(cachedKey, pTq); + taosArrayDestroyEx(cachedKey, freeItem); taosMemoryFree(data); } + // unlock taosWUnLockLatch(&pTq->lock); } if (!tsDisableStream && vnodeIsRoleLeader(pTq->pVnode)) { - if (taosHashGetSize(pTq->pStreamMeta->pTasks) == 0) return 0; + if (taosHashGetSize(pTq->pStreamMeta->pTasks) == 0) { + return 0; + } + if (msgType == TDMT_VND_SUBMIT) { void* data = taosMemoryMalloc(len); if (data == NULL) { @@ -351,6 +372,13 @@ int tqPushMsg(STQ* pTq, void* msg, int32_t msgLen, tmsg_t msgType, int64_t ver) return 0; } +void recordPushedEntry(SArray* cachedKey, void* pIter) { + size_t kLen = 0; + void* key = taosHashGetKey(pIter, &kLen); + SItem item = {.pKey = strndup(key, kLen), .keyLen = kLen}; + taosArrayPush(cachedKey, &item); +} + int32_t tqRegisterPushEntry(STQ* pTq, void* pHandle, const SMqPollReq* pRequest, SRpcMsg* pRpcMsg, SMqDataRsp* pDataRsp, int32_t type) { uint64_t consumerId = pRequest->consumerId; @@ -388,8 +416,8 @@ int32_t tqRegisterPushEntry(STQ* pTq, void* pHandle, const SMqPollReq* pRequest, return 0; } -int32_t tqRemovePushEntry(STQ* pTq, const char* pKey, int32_t keyLen, uint64_t consumerId, bool rspConsumer) { - int32_t vgId = TD_VID(pTq->pVnode); +int32_t tqUnregisterPushEntry(STQ* pTq, const char* pKey, int32_t keyLen, uint64_t consumerId, bool rspConsumer) { + int32_t vgId = TD_VID(pTq->pVnode); STqPushEntry** pEntry = taosHashGet(pTq->pPushMgr, pKey, keyLen); if (pEntry != NULL) { From 68f310dd486421e6c577307c383054e8daf92bcc Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 28 Mar 2023 15:05:10 +0800 Subject: [PATCH 44/65] refactor: do some internal refactor. --- include/libs/stream/streamState.h | 23 +++++---- include/libs/stream/tstream.h | 5 +- source/dnode/vnode/CMakeLists.txt | 2 +- source/dnode/vnode/src/tq/tq.c | 4 ++ source/dnode/vnode/src/tq/tqPush.c | 20 +++----- .../dnode/vnode/src/tq/{tqExec.c => tqScan.c} | 47 +++++++++++-------- 6 files changed, 53 insertions(+), 48 deletions(-) rename source/dnode/vnode/src/tq/{tqExec.c => tqScan.c} (93%) diff --git a/include/libs/stream/streamState.h b/include/libs/stream/streamState.h index 6b09bf4899..fd5cec2931 100644 --- a/include/libs/stream/streamState.h +++ b/include/libs/stream/streamState.h @@ -23,20 +23,19 @@ extern "C" { #ifndef _STREAM_STATE_H_ #define _STREAM_STATE_H_ -typedef struct SStreamTask SStreamTask; - typedef bool (*state_key_cmpr_fn)(void* pKey1, void* pKey2); typedef struct STdbState { - SStreamTask* pOwner; - TDB* db; - TTB* pStateDb; - TTB* pFuncStateDb; - TTB* pFillStateDb; // todo refactor - TTB* pSessionStateDb; - TTB* pParNameDb; - TTB* pParTagDb; - TXN* txn; + struct SStreamTask* pOwner; + + TDB* db; + TTB* pStateDb; + TTB* pFuncStateDb; + TTB* pFillStateDb; // todo refactor + TTB* pSessionStateDb; + TTB* pParNameDb; + TTB* pParTagDb; + TXN* txn; } STdbState; // incremental state storage @@ -45,7 +44,7 @@ typedef struct { int32_t number; } SStreamState; -SStreamState* streamStateOpen(char* path, SStreamTask* pTask, bool specPath, int32_t szPage, int32_t pages); +SStreamState* streamStateOpen(char* path, struct SStreamTask* pTask, bool specPath, int32_t szPage, int32_t pages); void streamStateClose(SStreamState* pState); int32_t streamStateBegin(SStreamState* pState); int32_t streamStateCommit(SStreamState* pState); diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 1d301623b1..d04e965c6f 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -295,7 +295,7 @@ typedef struct { SEpSet epSet; } SStreamChildEpInfo; -typedef struct SStreamTask { +struct SStreamTask { int64_t streamId; int32_t taskId; int32_t totalLevel; @@ -362,8 +362,7 @@ typedef struct SStreamTask { int64_t checkpointingId; int32_t checkpointAlignCnt; - -} SStreamTask; +}; int32_t tEncodeStreamEpInfo(SEncoder* pEncoder, const SStreamChildEpInfo* pInfo); int32_t tDecodeStreamEpInfo(SDecoder* pDecoder, SStreamChildEpInfo* pInfo); diff --git a/source/dnode/vnode/CMakeLists.txt b/source/dnode/vnode/CMakeLists.txt index 8dc3f46ae3..9911752f8e 100644 --- a/source/dnode/vnode/CMakeLists.txt +++ b/source/dnode/vnode/CMakeLists.txt @@ -57,7 +57,7 @@ target_sources( # tq "src/tq/tq.c" - "src/tq/tqExec.c" + "src/tq/tqScan.c" "src/tq/tqMeta.c" "src/tq/tqRead.c" "src/tq/tqOffset.c" diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 2d2c70c84a..e1149b48af 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -473,6 +473,10 @@ static int32_t extractDataAndRspForNormalSubscribe(STQ* pTq, STqHandle* pHandle, qSetTaskId(pHandle->execHandle.task, consumerId, pRequest->reqId); code = tqScanData(pTq, pHandle, &dataRsp, pOffset); + if (code != TSDB_CODE_SUCCESS) { + taosWUnLockLatch(&pTq->lock); + return code; + } // till now, all data has been transferred to consumer, new data needs to push client once arrived. if (dataRsp.blockNum == 0 && dataRsp.reqOffset.type == TMQ_OFFSET__LOG && diff --git a/source/dnode/vnode/src/tq/tqPush.c b/source/dnode/vnode/src/tq/tqPush.c index e07ab06458..73d722dba5 100644 --- a/source/dnode/vnode/src/tq/tqPush.c +++ b/source/dnode/vnode/src/tq/tqPush.c @@ -220,7 +220,7 @@ static void freeItem(void* param) { static void doRemovePushedEntry(SArray* pCachedKeys, STQ* pTq) { int32_t vgId = TD_VID(pTq->pVnode); - size_t numOfKeys = taosArrayGetSize(pCachedKeys); + int32_t numOfKeys = (int32_t) taosArrayGetSize(pCachedKeys); for (int32_t i = 0; i < numOfKeys; i++) { SItem* pItem = taosArrayGet(pCachedKeys, i); @@ -248,13 +248,9 @@ static void doPushDataForEntry(void* pIter, STqExecHandle* pExec, STQ* pTq, int6 qTaskInfo_t pTaskInfo = pExec->task; // prepare scan mem data - SPackedData submit = { - .msgStr = pData, - .msgLen = dataLen, - .ver = ver, - }; + SPackedData submit = {.msgStr = pData, .msgLen = dataLen, .ver = ver}; - if(qStreamSetScanMemData(pTaskInfo, submit) != 0){ + if (qStreamSetScanMemData(pTaskInfo, submit) != 0) { return; } @@ -287,7 +283,7 @@ static void doPushDataForEntry(void* pIter, STqExecHandle* pExec, STQ* pTq, int6 } } -int tqPushMsg(STQ* pTq, void* msg, int32_t msgLen, tmsg_t msgType, int64_t ver) { +int32_t tqPushMsg(STQ* pTq, void* msg, int32_t msgLen, tmsg_t msgType, int64_t ver) { void* pReq = POINTER_SHIFT(msg, sizeof(SSubmitReq2Msg)); int32_t len = msgLen - sizeof(SSubmitReq2Msg); int32_t vgId = TD_VID(pTq->pVnode); @@ -341,6 +337,7 @@ int tqPushMsg(STQ* pTq, void* msg, int32_t msgLen, tmsg_t msgType, int64_t ver) taosWUnLockLatch(&pTq->lock); } + // push data for stream processing if (!tsDisableStream && vnodeIsRoleLeader(pTq->pVnode)) { if (taosHashGetSize(pTq->pStreamMeta->pTasks) == 0) { return 0; @@ -353,12 +350,9 @@ int tqPushMsg(STQ* pTq, void* msg, int32_t msgLen, tmsg_t msgType, int64_t ver) tqError("failed to copy data for stream since out of memory"); return -1; } + memcpy(data, pReq, len); - SPackedData submit = { - .msgStr = data, - .msgLen = len, - .ver = ver, - }; + SPackedData submit = {.msgStr = data, .msgLen = len, .ver = ver}; tqDebug("tq copy write msg %p %d %" PRId64 " from %p", data, len, ver, pReq); tqProcessSubmitReq(pTq, submit); diff --git a/source/dnode/vnode/src/tq/tqExec.c b/source/dnode/vnode/src/tq/tqScan.c similarity index 93% rename from source/dnode/vnode/src/tq/tqExec.c rename to source/dnode/vnode/src/tq/tqScan.c index 6845350346..36d0631195 100644 --- a/source/dnode/vnode/src/tq/tqExec.c +++ b/source/dnode/vnode/src/tq/tqScan.c @@ -18,7 +18,9 @@ int32_t tqAddBlockDataToRsp(const SSDataBlock* pBlock, SMqDataRsp* pRsp, int32_t numOfCols, int8_t precision) { int32_t dataStrLen = sizeof(SRetrieveTableRsp) + blockGetEncodeSize(pBlock); void* buf = taosMemoryCalloc(1, dataStrLen); - if (buf == NULL) return -1; + if (buf == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } SRetrieveTableRsp* pRetrieve = (SRetrieveTableRsp*)buf; pRetrieve->useconds = 0; @@ -31,7 +33,8 @@ int32_t tqAddBlockDataToRsp(const SSDataBlock* pBlock, SMqDataRsp* pRsp, int32_t actualLen += sizeof(SRetrieveTableRsp); taosArrayPush(pRsp->blockDataLen, &actualLen); taosArrayPush(pRsp->blockData, &buf); - return 0; + + return TSDB_CODE_SUCCESS; } static int32_t tqAddBlockSchemaToRsp(const STqExecHandle* pExec, STaosxRsp* pRsp) { @@ -63,38 +66,39 @@ static int32_t tqAddTbNameToRsp(const STQ* pTq, int64_t uid, STaosxRsp* pRsp, in int32_t tqScanData(STQ* pTq, const STqHandle* pHandle, SMqDataRsp* pRsp, STqOffsetVal* pOffset) { const int32_t MAX_ROWS_TO_RETURN = 4096; + int32_t vgId = TD_VID(pTq->pVnode); + int32_t code = 0; + int32_t totalRows = 0; const STqExecHandle* pExec = &pHandle->execHandle; - - qTaskInfo_t task = pExec->task; - int32_t vgId = TD_VID(pTq->pVnode); + qTaskInfo_t task = pExec->task; if (qStreamPrepareScan(task, pOffset, pHandle->execHandle.subType) < 0) { tqDebug("prepare scan failed, return, consumer:0x%"PRIx64, pHandle->consumerId); if (pOffset->type == TMQ_OFFSET__LOG) { pRsp->rspOffset = *pOffset; - return 0; + return code; } else { tqOffsetResetToLog(pOffset, pHandle->snapshotVer); if (qStreamPrepareScan(task, pOffset, pHandle->execHandle.subType) < 0) { tqDebug("prepare scan failed, return, consumer:0x%"PRIx64, pHandle->consumerId); pRsp->rspOffset = *pOffset; - return 0; + return code; } } } - int32_t totalRows = 0; - while (1) { SSDataBlock* pDataBlock = NULL; uint64_t ts = 0; - tqDebug("vgId:%d, tmq task start to execute, consumer:0x%"PRIx64, vgId, pHandle->consumerId); - if (qExecTask(task, &pDataBlock, &ts) < 0) { + tqDebug("vgId:%d, tmq task start to execute, consumer:0x%" PRIx64, vgId, pHandle->consumerId); + + code = qExecTask(task, &pDataBlock, &ts); + if (code != TSDB_CODE_SUCCESS) { tqError("vgId:%d, task exec error since %s, consumer:0x%" PRIx64, vgId, terrstr(), pHandle->consumerId); - return -1; + return code; } // current scan should be stopped ASAP, since the re-balance occurs. @@ -102,7 +106,12 @@ int32_t tqScanData(STQ* pTq, const STqHandle* pHandle, SMqDataRsp* pRsp, STqOffs break; } - tqAddBlockDataToRsp(pDataBlock, pRsp, pExec->numOfCols, pTq->pVnode->config.tsdbCfg.precision); + code = tqAddBlockDataToRsp(pDataBlock, pRsp, pExec->numOfCols, pTq->pVnode->config.tsdbCfg.precision); + if (code != TSDB_CODE_SUCCESS) { + tqError("vgId:%d, failed to add block to rsp msg", vgId); + return code; + } + pRsp->blockNum++; tqDebug("vgId:%d, consumer:0x%" PRIx64 " tmq task executed, rows:%d, total blocks:%d", vgId, pHandle->consumerId, @@ -116,25 +125,25 @@ int32_t tqScanData(STQ* pTq, const STqHandle* pHandle, SMqDataRsp* pRsp, STqOffs } } - if (qStreamExtractOffset(task, &pRsp->rspOffset) < 0) { - return -1; - } + qStreamExtractOffset(task, &pRsp->rspOffset); if (pRsp->rspOffset.type == 0) { + code = TSDB_CODE_INVALID_PARA; tqError("vgId:%d, expected rsp offset: type %d %" PRId64 " %" PRId64 " %" PRId64, vgId, pRsp->rspOffset.type, pRsp->rspOffset.ts, pRsp->rspOffset.uid, pRsp->rspOffset.version); - return -1; + return code; } if (pRsp->withTbName || pRsp->withSchema) { + code = TSDB_CODE_INVALID_PARA; tqError("vgId:%d, get column should not with meta:%d,%d", vgId, pRsp->withTbName, pRsp->withSchema); - return -1; + return code; } tqDebug("vgId:%d, consumer:0x%" PRIx64 " tmq task executed, rows:%d, total blocks:%d, rows:%d", vgId, pHandle->consumerId, pRsp->blockNum, totalRows); - return 0; + return code; } int32_t tqScanTaosx(STQ* pTq, const STqHandle* pHandle, STaosxRsp* pRsp, SMqMetaRsp* pMetaRsp, STqOffsetVal* pOffset) { From 22ad3129b92b089b52d14c894a5b66ecf4a46476 Mon Sep 17 00:00:00 2001 From: dapan1121 Date: Tue, 28 Mar 2023 15:21:47 +0800 Subject: [PATCH 45/65] fix: query schema is old issue --- include/common/tdatablock.h | 2 +- source/common/src/tdatablock.c | 21 +- source/dnode/vnode/inc/vnode.h | 2 +- source/dnode/vnode/src/tsdb/tsdbRead.c | 229 +++++++++++++----- source/libs/executor/src/scanoperator.c | 59 ++++- source/libs/executor/src/sysscanoperator.c | 2 +- source/libs/scalar/src/sclfunc.c | 6 +- tests/parallel_test/cases.task | 3 + tests/system-test/2-query/columnLenUpdated.py | 181 ++++++++++++++ 9 files changed, 432 insertions(+), 73 deletions(-) create mode 100644 tests/system-test/2-query/columnLenUpdated.py diff --git a/include/common/tdatablock.h b/include/common/tdatablock.h index 84d696e518..99fffa2cf1 100644 --- a/include/common/tdatablock.h +++ b/include/common/tdatablock.h @@ -178,7 +178,7 @@ int32_t getJsonValueLen(const char* data); int32_t colDataSetVal(SColumnInfoData* pColumnInfoData, uint32_t rowIndex, const char* pData, bool isNull); int32_t colDataAppend(SColumnInfoData* pColumnInfoData, uint32_t rowIndex, const char* pData, bool isNull); -int32_t colDataSetNItems(SColumnInfoData* pColumnInfoData, uint32_t rowIndex, const char* pData, uint32_t numOfRows); +int32_t colDataSetNItems(SColumnInfoData* pColumnInfoData, uint32_t rowIndex, const char* pData, uint32_t numOfRows, bool trimValue); int32_t colDataMergeCol(SColumnInfoData* pColumnInfoData, int32_t numOfRow1, int32_t* capacity, const SColumnInfoData* pSource, int32_t numOfRow2); int32_t colDataAssign(SColumnInfoData* pColumnInfoData, const SColumnInfoData* pSource, int32_t numOfRows, diff --git a/source/common/src/tdatablock.c b/source/common/src/tdatablock.c index 3c8d394b43..6e2b650b5b 100644 --- a/source/common/src/tdatablock.c +++ b/source/common/src/tdatablock.c @@ -147,9 +147,17 @@ int32_t colDataReserve(SColumnInfoData* pColumnInfoData, size_t newSize) { return TSDB_CODE_SUCCESS; } -static void doCopyNItems(struct SColumnInfoData* pColumnInfoData, int32_t currentRow, const char* pData, - int32_t itemLen, int32_t numOfRows) { - ASSERT(pColumnInfoData->info.bytes >= itemLen); +static int32_t doCopyNItems(struct SColumnInfoData* pColumnInfoData, int32_t currentRow, const char* pData, + int32_t itemLen, int32_t numOfRows, bool trimValue) { + if (pColumnInfoData->info.bytes < itemLen) { + uWarn("column/tag actual data len %d is bigger than schema len %d, trim it:%d", itemLen, pColumnInfoData->info.bytes, trimValue); + if (trimValue) { + itemLen = pColumnInfoData->info.bytes; + } else { + return TSDB_CODE_TDB_INVALID_TABLE_SCHEMA_VER; + } + } + size_t start = 1; // the first item @@ -178,10 +186,12 @@ static void doCopyNItems(struct SColumnInfoData* pColumnInfoData, int32_t curren pColumnInfoData->varmeta.length += numOfRows * itemLen; } + + return TSDB_CODE_SUCCESS; } int32_t colDataSetNItems(SColumnInfoData* pColumnInfoData, uint32_t currentRow, const char* pData, - uint32_t numOfRows) { + uint32_t numOfRows, bool trimValue) { int32_t len = pColumnInfoData->info.bytes; if (IS_VAR_DATA_TYPE(pColumnInfoData->info.type)) { len = varDataTLen(pData); @@ -193,8 +203,7 @@ int32_t colDataSetNItems(SColumnInfoData* pColumnInfoData, uint32_t currentRow, } } - doCopyNItems(pColumnInfoData, currentRow, pData, len, numOfRows); - return TSDB_CODE_SUCCESS; + return doCopyNItems(pColumnInfoData, currentRow, pData, len, numOfRows, trimValue); } static void doBitmapMerge(SColumnInfoData* pColumnInfoData, int32_t numOfRow1, const SColumnInfoData* pSource, diff --git a/source/dnode/vnode/inc/vnode.h b/source/dnode/vnode/inc/vnode.h index 2d053d04ae..7864d80a38 100644 --- a/source/dnode/vnode/inc/vnode.h +++ b/source/dnode/vnode/inc/vnode.h @@ -182,7 +182,7 @@ int32_t tsdbReaderOpen(SVnode *pVnode, SQueryTableDataCond *pCond, void *pTableL void tsdbReaderSetId(STsdbReader* pReader, const char* idstr); void tsdbReaderClose(STsdbReader *pReader); -bool tsdbNextDataBlock(STsdbReader *pReader); +int32_t tsdbNextDataBlock(STsdbReader *pReader, bool *hasNext); int32_t tsdbRetrieveDatablockSMA(STsdbReader *pReader, SSDataBlock *pDataBlock, bool *allHave); void tsdbReleaseDataBlock(STsdbReader *pReader); SSDataBlock *tsdbRetrieveDataBlock(STsdbReader *pTsdbReadHandle, SArray *pColumnIdList); diff --git a/source/dnode/vnode/src/tsdb/tsdbRead.c b/source/dnode/vnode/src/tsdb/tsdbRead.c index 96bce02b67..e11a80efb3 100644 --- a/source/dnode/vnode/src/tsdb/tsdbRead.c +++ b/source/dnode/vnode/src/tsdb/tsdbRead.c @@ -949,14 +949,17 @@ static void setBlockAllDumped(SFileBlockDumpInfo* pDumpInfo, int64_t maxKey, int pDumpInfo->lastKey = maxKey + step; } -static void doCopyColVal(SColumnInfoData* pColInfoData, int32_t rowIndex, int32_t colIndex, SColVal* pColVal, +static int32_t doCopyColVal(SColumnInfoData* pColInfoData, int32_t rowIndex, int32_t colIndex, SColVal* pColVal, SBlockLoadSuppInfo* pSup) { if (IS_VAR_DATA_TYPE(pColVal->type)) { if (!COL_VAL_IS_VALUE(pColVal)) { colDataSetNULL(pColInfoData, rowIndex); } else { varDataSetLen(pSup->buildBuf[colIndex], pColVal->value.nData); - ASSERT(pColVal->value.nData <= pColInfoData->info.bytes); + if (pColVal->value.nData > pColInfoData->info.bytes) { + tsdbWarn("column cid:%d actual data len %d is bigger than schema len %d", pColVal->cid, pColVal->value.nData, pColInfoData->info.bytes); + return TSDB_CODE_TDB_INVALID_TABLE_SCHEMA_VER; + } if (pColVal->value.nData > 0) { // pData may be null, if nData is 0 memcpy(varDataVal(pSup->buildBuf[colIndex]), pColVal->value.pData, pColVal->value.nData); } @@ -966,6 +969,8 @@ static void doCopyColVal(SColumnInfoData* pColInfoData, int32_t rowIndex, int32_ } else { colDataSetVal(pColInfoData, rowIndex, (const char*)&pColVal->value, !COL_VAL_IS_VALUE(pColVal)); } + + return TSDB_CODE_SUCCESS; } static SFileDataBlockInfo* getCurrentBlockInfo(SDataBlockIter* pBlockIter) { @@ -1167,6 +1172,7 @@ static int32_t copyBlockDataToSDataBlock(STsdbReader* pReader) { SDataBlk* pBlock = getCurrentBlock(pBlockIter); SSDataBlock* pResBlock = pReader->pResBlock; int32_t numOfOutputCols = pSupInfo->numOfCols; + int32_t code = TSDB_CODE_SUCCESS; SColVal cv = {0}; int64_t st = taosGetTimestampUs(); @@ -1244,7 +1250,10 @@ static int32_t copyBlockDataToSDataBlock(STsdbReader* pReader) { } else { // varchar/nchar type for (int32_t j = pDumpInfo->rowIndex; rowIndex < dumpedRows; j += step) { tColDataGetValue(pData, j, &cv); - doCopyColVal(pColData, rowIndex++, i, &cv, pSupInfo); + code = doCopyColVal(pColData, rowIndex++, i, &cv, pSupInfo); + if (code) { + return code; + } } } } @@ -1776,23 +1785,29 @@ static int32_t buildDataBlockFromBuf(STsdbReader* pReader, STableBlockScanInfo* } static bool tryCopyDistinctRowFromFileBlock(STsdbReader* pReader, SBlockData* pBlockData, int64_t key, - SFileBlockDumpInfo* pDumpInfo) { + SFileBlockDumpInfo* pDumpInfo, bool *copied) { // opt version // 1. it is not a border point // 2. the direct next point is not an duplicated timestamp + int32_t code = TSDB_CODE_SUCCESS; + + *copied = false; bool asc = (pReader->order == TSDB_ORDER_ASC); if ((pDumpInfo->rowIndex < pDumpInfo->totalRows - 1 && asc) || (pDumpInfo->rowIndex > 0 && (!asc))) { int32_t step = pReader->order == TSDB_ORDER_ASC ? 1 : -1; int64_t nextKey = pBlockData->aTSKEY[pDumpInfo->rowIndex + step]; if (nextKey != key) { // merge is not needed - doAppendRowFromFileBlock(pReader->pResBlock, pReader, pBlockData, pDumpInfo->rowIndex); + code = doAppendRowFromFileBlock(pReader->pResBlock, pReader, pBlockData, pDumpInfo->rowIndex); + if (code) { + return code; + } pDumpInfo->rowIndex += step; - return true; + *copied = true; } } - return false; + return code; } static bool nextRowFromLastBlocks(SLastBlockReader* pLastBlockReader, STableBlockScanInfo* pScanInfo, @@ -1819,20 +1834,34 @@ static bool nextRowFromLastBlocks(SLastBlockReader* pLastBlockReader, STableBloc } static bool tryCopyDistinctRowFromSttBlock(TSDBROW* fRow, SLastBlockReader* pLastBlockReader, - STableBlockScanInfo* pScanInfo, int64_t ts, STsdbReader* pReader) { + STableBlockScanInfo* pScanInfo, int64_t ts, STsdbReader* pReader, bool *copied) { + int32_t code = TSDB_CODE_SUCCESS; + + *copied = false; + bool hasVal = nextRowFromLastBlocks(pLastBlockReader, pScanInfo, &pReader->verRange); if (hasVal) { int64_t next1 = getCurrentKeyInLastBlock(pLastBlockReader); if (next1 != ts) { - doAppendRowFromFileBlock(pReader->pResBlock, pReader, fRow->pBlockData, fRow->iRow); - return true; + code = doAppendRowFromFileBlock(pReader->pResBlock, pReader, fRow->pBlockData, fRow->iRow); + if (code) { + return code; + } + + *copied = true; + return code; } } else { - doAppendRowFromFileBlock(pReader->pResBlock, pReader, fRow->pBlockData, fRow->iRow); - return true; + code = doAppendRowFromFileBlock(pReader->pResBlock, pReader, fRow->pBlockData, fRow->iRow); + if (code) { + return code; + } + + *copied = true; + return code; } - return false; + return code; } static FORCE_INLINE STSchema* getLatestTableSchema(STsdbReader* pReader, uint64_t uid) { @@ -2022,11 +2051,12 @@ static int32_t doMergeBufAndFileRows(STsdbReader* pReader, STableBlockScanInfo* return code; } - doAppendRowFromTSRow(pReader->pResBlock, pReader, pTSRow, pBlockScanInfo); + code = doAppendRowFromTSRow(pReader->pResBlock, pReader, pTSRow, pBlockScanInfo); taosMemoryFree(pTSRow); tsdbRowMergerClear(&merge); - return TSDB_CODE_SUCCESS; + + return code; } static int32_t doMergeFileBlockAndLastBlock(SLastBlockReader* pLastBlockReader, STsdbReader* pReader, @@ -2034,7 +2064,8 @@ static int32_t doMergeFileBlockAndLastBlock(SLastBlockReader* pLastBlockReader, bool mergeBlockData) { SFileBlockDumpInfo* pDumpInfo = &pReader->status.fBlockDumpInfo; int64_t tsLastBlock = getCurrentKeyInLastBlock(pLastBlockReader); - + bool copied = false; + int32_t code = TSDB_CODE_SUCCESS; SRow* pTSRow = NULL; SRowMerger merge = {0}; TSDBROW fRow = tMergeTreeGetRow(&pLastBlockReader->mergeTree); @@ -2042,7 +2073,12 @@ static int32_t doMergeFileBlockAndLastBlock(SLastBlockReader* pLastBlockReader, // only last block exists if ((!mergeBlockData) || (tsLastBlock != pBlockData->aTSKEY[pDumpInfo->rowIndex])) { - if (tryCopyDistinctRowFromSttBlock(&fRow, pLastBlockReader, pBlockScanInfo, tsLastBlock, pReader)) { + code = tryCopyDistinctRowFromSttBlock(&fRow, pLastBlockReader, pBlockScanInfo, tsLastBlock, pReader, &copied); + if (code) { + return code; + } + + if (copied) { pBlockScanInfo->lastKey = tsLastBlock; return TSDB_CODE_SUCCESS; } else { @@ -2060,10 +2096,15 @@ static int32_t doMergeFileBlockAndLastBlock(SLastBlockReader* pLastBlockReader, return code; } - doAppendRowFromTSRow(pReader->pResBlock, pReader, pTSRow, pBlockScanInfo); + code = doAppendRowFromTSRow(pReader->pResBlock, pReader, pTSRow, pBlockScanInfo); taosMemoryFree(pTSRow); tsdbRowMergerClear(&merge); + + if (code != TSDB_CODE_SUCCESS) { + return code; + } + } } else { // not merge block data int32_t code = tsdbRowMergerInit(&merge, &fRow, pReader->pSchema); @@ -2083,10 +2124,14 @@ static int32_t doMergeFileBlockAndLastBlock(SLastBlockReader* pLastBlockReader, return code; } - doAppendRowFromTSRow(pReader->pResBlock, pReader, pTSRow, pBlockScanInfo); + code = doAppendRowFromTSRow(pReader->pResBlock, pReader, pTSRow, pBlockScanInfo); taosMemoryFree(pTSRow); tsdbRowMergerClear(&merge); + + if (code != TSDB_CODE_SUCCESS) { + return code; + } } return TSDB_CODE_SUCCESS; @@ -2131,7 +2176,7 @@ static int32_t mergeFileBlockAndLastBlock(STsdbReader* pReader, SLastBlockReader return code; } - doAppendRowFromTSRow(pReader->pResBlock, pReader, pTSRow, pBlockScanInfo); + code = doAppendRowFromTSRow(pReader->pResBlock, pReader, pTSRow, pBlockScanInfo); taosMemoryFree(pTSRow); tsdbRowMergerClear(&merge); @@ -2353,7 +2398,7 @@ static int32_t doMergeMultiLevelRows(STsdbReader* pReader, STableBlockScanInfo* return code; } - doAppendRowFromTSRow(pReader->pResBlock, pReader, pTSRow, pBlockScanInfo); + code = doAppendRowFromTSRow(pReader->pResBlock, pReader, pTSRow, pBlockScanInfo); taosMemoryFree(pTSRow); tsdbRowMergerClear(&merge); @@ -2508,7 +2553,13 @@ bool hasDataInFileBlock(const SBlockData* pBlockData, const SFileBlockDumpInfo* int32_t mergeRowsInFileBlocks(SBlockData* pBlockData, STableBlockScanInfo* pBlockScanInfo, int64_t key, STsdbReader* pReader) { SFileBlockDumpInfo* pDumpInfo = &pReader->status.fBlockDumpInfo; - if (tryCopyDistinctRowFromFileBlock(pReader, pBlockData, key, pDumpInfo)) { + bool copied = false; + int32_t code = tryCopyDistinctRowFromFileBlock(pReader, pBlockData, key, pDumpInfo, &copied); + if (code) { + return code; + } + + if (copied) { pBlockScanInfo->lastKey = key; return TSDB_CODE_SUCCESS; } else { @@ -2528,11 +2579,11 @@ int32_t mergeRowsInFileBlocks(SBlockData* pBlockData, STableBlockScanInfo* pBloc return code; } - doAppendRowFromTSRow(pReader->pResBlock, pReader, pTSRow, pBlockScanInfo); + code = doAppendRowFromTSRow(pReader->pResBlock, pReader, pTSRow, pBlockScanInfo); taosMemoryFree(pTSRow); tsdbRowMergerClear(&merge); - return TSDB_CODE_SUCCESS; + return code; } } @@ -2649,7 +2700,10 @@ static int32_t buildComposedDataBlock(STsdbReader* pReader) { if (isCleanFileDataBlock(pReader, pBlockInfo, pBlock, pBlockScanInfo, keyInBuf, pLastBlockReader) && pBlock->nRow <= pReader->capacity) { if (asc || ((!asc) && (!hasDataInLastBlock(pLastBlockReader)))) { - copyBlockDataToSDataBlock(pReader); + code = copyBlockDataToSDataBlock(pReader); + if (code) { + goto _end; + } // record the last key value pBlockScanInfo->lastKey = asc ? pBlock->maxKey.ts : pBlock->minKey.ts; @@ -2696,8 +2750,11 @@ static int32_t buildComposedDataBlock(STsdbReader* pReader) { break; } - buildComposedDataBlockImpl(pReader, pBlockScanInfo, pBlockData, pLastBlockReader); - + code = buildComposedDataBlockImpl(pReader, pBlockScanInfo, pBlockData, pLastBlockReader); + if (code) { + goto _end; + } + // currently loaded file data block is consumed if ((pBlockData->nRow > 0) && (pDumpInfo->rowIndex >= pBlockData->nRow || pDumpInfo->rowIndex < 0)) { SDataBlk* pBlock = getCurrentBlock(&pReader->status.blockIter); @@ -2922,6 +2979,7 @@ static int32_t doLoadLastBlockSequentially(STsdbReader* pReader) { SReaderStatus* pStatus = &pReader->status; SLastBlockReader* pLastBlockReader = pStatus->fileIter.pLastBlockReader; STableUidList* pUidList = &pStatus->uidList; + int32_t code = TSDB_CODE_SUCCESS; if (taosHashGetSize(pStatus->pTableMap) == 0) { return TSDB_CODE_SUCCESS; @@ -2952,7 +3010,11 @@ static int32_t doLoadLastBlockSequentially(STsdbReader* pReader) { break; } - buildComposedDataBlockImpl(pReader, pScanInfo, &pReader->status.fileBlockData, pLastBlockReader); + code = buildComposedDataBlockImpl(pReader, pScanInfo, &pReader->status.fileBlockData, pLastBlockReader); + if (code) { + return code; + } + if (pResBlock->info.rows >= pReader->capacity) { break; } @@ -3032,7 +3094,11 @@ static int32_t doBuildDataBlock(STsdbReader* pReader) { break; } - buildComposedDataBlockImpl(pReader, pScanInfo, &pReader->status.fileBlockData, pLastBlockReader); + code = buildComposedDataBlockImpl(pReader, pScanInfo, &pReader->status.fileBlockData, pLastBlockReader); + if (code) { + return code; + } + if (pResBlock->info.rows >= pReader->capacity) { break; } @@ -3784,6 +3850,7 @@ int32_t tsdbGetNextRowInMem(STableBlockScanInfo* pBlockScanInfo, STsdbReader* pR int32_t doAppendRowFromTSRow(SSDataBlock* pBlock, STsdbReader* pReader, SRow* pTSRow, STableBlockScanInfo* pScanInfo) { int32_t outputRowIndex = pBlock->info.rows; int64_t uid = pScanInfo->uid; + int32_t code = TSDB_CODE_SUCCESS; int32_t numOfCols = (int32_t)taosArrayGetSize(pBlock->pDataBlock); @@ -3806,7 +3873,10 @@ int32_t doAppendRowFromTSRow(SSDataBlock* pBlock, STsdbReader* pReader, SRow* pT SColumnInfoData* pColInfoData = taosArrayGet(pBlock->pDataBlock, pSupInfo->slotId[i]); tRowGet(pTSRow, pSchema, j, &colVal); - doCopyColVal(pColInfoData, outputRowIndex, i, &colVal, pSupInfo); + code = doCopyColVal(pColInfoData, outputRowIndex, i, &colVal, pSupInfo); + if (code) { + return code; + } i += 1; j += 1; } else if (colId < pSchema->columns[j].colId) { @@ -3836,6 +3906,7 @@ int32_t doAppendRowFromFileBlock(SSDataBlock* pResBlock, STsdbReader* pReader, S int32_t rowIndex) { int32_t i = 0, j = 0; int32_t outputRowIndex = pResBlock->info.rows; + int32_t code = TSDB_CODE_SUCCESS; SBlockLoadSuppInfo* pSupInfo = &pReader->suppInfo; if (pReader->suppInfo.colId[i] == PRIMARYKEY_TIMESTAMP_COL_ID) { @@ -3858,7 +3929,10 @@ int32_t doAppendRowFromFileBlock(SSDataBlock* pResBlock, STsdbReader* pReader, S SColumnInfoData* pCol = TARRAY_GET_ELEM(pResBlock->pDataBlock, pSupInfo->slotId[i]); if (pData->cid == pSupInfo->colId[i]) { tColDataGetValue(pData, rowIndex, &cv); - doCopyColVal(pCol, outputRowIndex, i, &cv, pSupInfo); + code = doCopyColVal(pCol, outputRowIndex, i, &cv, pSupInfo); + if (code) { + return code; + } j += 1; } else if (pData->cid > pCol->info.colId) { // the specified column does not exist in file block, fill with null data @@ -3882,6 +3956,7 @@ int32_t doAppendRowFromFileBlock(SSDataBlock* pResBlock, STsdbReader* pReader, S int32_t buildDataBlockFromBufImpl(STableBlockScanInfo* pBlockScanInfo, int64_t endKey, int32_t capacity, STsdbReader* pReader) { SSDataBlock* pBlock = pReader->pResBlock; + int32_t code = TSDB_CODE_SUCCESS; do { // SRow* pTSRow = NULL; @@ -3893,13 +3968,20 @@ int32_t buildDataBlockFromBufImpl(STableBlockScanInfo* pBlockScanInfo, int64_t e } if (row.type == TSDBROW_ROW_FMT) { - doAppendRowFromTSRow(pBlock, pReader, row.pTSRow, pBlockScanInfo); + code = doAppendRowFromTSRow(pBlock, pReader, row.pTSRow, pBlockScanInfo); if (freeTSRow) { taosMemoryFree(row.pTSRow); } + + if (code) { + return code; + } } else { - doAppendRowFromFileBlock(pBlock, pReader, row.pBlockData, row.iRow); + code = doAppendRowFromFileBlock(pBlock, pReader, row.pBlockData, row.iRow); + if (code) { + break; + } } // no data in buffer, return immediately @@ -3912,7 +3994,7 @@ int32_t buildDataBlockFromBufImpl(STableBlockScanInfo* pBlockScanInfo, int64_t e } } while (1); - return TSDB_CODE_SUCCESS; + return code; } // TODO refactor: with createDataBlockScanInfo @@ -4394,43 +4476,52 @@ _err: return code; } -static bool doTsdbNextDataBlock(STsdbReader* pReader) { +static int32_t doTsdbNextDataBlock(STsdbReader* pReader, bool *hasNext) { + int32_t code = TSDB_CODE_SUCCESS; + // cleanup the data that belongs to the previous data block SSDataBlock* pBlock = pReader->pResBlock; blockDataCleanup(pBlock); + *hasNext = false; + SReaderStatus* pStatus = &pReader->status; if (taosHashGetSize(pStatus->pTableMap) == 0) { - return false; + return code; } if (pStatus->loadFromFile) { - int32_t code = buildBlockFromFiles(pReader); + code = buildBlockFromFiles(pReader); if (code != TSDB_CODE_SUCCESS) { - return false; + return code; } if (pBlock->info.rows > 0) { - return true; + *hasNext = true; } else { resetTableListIndex(&pReader->status); - buildBlockFromBufferSequentially(pReader); - return pBlock->info.rows > 0; + code = buildBlockFromBufferSequentially(pReader); } } else { // no data in files, let's try the buffer - buildBlockFromBufferSequentially(pReader); - return pBlock->info.rows > 0; + code = buildBlockFromBufferSequentially(pReader); + *hasNext = pBlock->info.rows > 0; } + + return code; } -bool tsdbNextDataBlock(STsdbReader* pReader) { +int32_t tsdbNextDataBlock(STsdbReader* pReader, bool *hasNext) { + int32_t code = TSDB_CODE_SUCCESS; + + *hasNext = false; + if (isEmptyQueryTimeWindow(&pReader->window) || pReader->step == EXTERNAL_ROWS_NEXT) { - return false; + return code; } SReaderStatus* pStatus = &pReader->status; - int32_t code = tsdbAcquireReader(pReader); + code = tsdbAcquireReader(pReader); qTrace("tsdb/read: %p, take read mutex, code: %d", pReader, code); if (pReader->suspended) { @@ -4438,16 +4529,21 @@ bool tsdbNextDataBlock(STsdbReader* pReader) { } if (pReader->innerReader[0] != NULL && pReader->step == 0) { - bool ret = doTsdbNextDataBlock(pReader->innerReader[0]); + code = doTsdbNextDataBlock(pReader->innerReader[0], hasNext); + if (code) { + tsdbReleaseReader(pReader); + return code; + } + pReader->step = EXTERNAL_ROWS_PREV; - if (ret) { + if (*hasNext) { pStatus = &pReader->innerReader[0]->status; if (pStatus->composedDataBlock) { qTrace("tsdb/read: %p, unlock read mutex", pReader); tsdbReleaseReader(pReader); } - return ret; + return code; } } @@ -4464,14 +4560,19 @@ bool tsdbNextDataBlock(STsdbReader* pReader) { pReader->step = EXTERNAL_ROWS_MAIN; } - bool ret = doTsdbNextDataBlock(pReader); - if (ret) { + code = doTsdbNextDataBlock(pReader, hasNext); + if (code != TSDB_CODE_SUCCESS) { + tsdbReleaseReader(pReader); + return code; + } + + if (*hasNext) { if (pStatus->composedDataBlock) { qTrace("tsdb/read: %p, unlock read mutex", pReader); tsdbReleaseReader(pReader); } - return ret; + return code; } if (pReader->step == EXTERNAL_ROWS_MAIN && pReader->innerReader[1] != NULL) { @@ -4483,23 +4584,28 @@ bool tsdbNextDataBlock(STsdbReader* pReader) { return code; } - ret = doTsdbNextDataBlock(pReader->innerReader[1]); + code = doTsdbNextDataBlock(pReader->innerReader[1], hasNext); + if (code != TSDB_CODE_SUCCESS) { + tsdbReleaseReader(pReader); + return code; + } + pReader->step = EXTERNAL_ROWS_NEXT; - if (ret) { + if (*hasNext) { pStatus = &pReader->innerReader[1]->status; if (pStatus->composedDataBlock) { qTrace("tsdb/read: %p, unlock read mutex", pReader); tsdbReleaseReader(pReader); } - return ret; + return code; } } qTrace("tsdb/read: %p, unlock read mutex", pReader); tsdbReleaseReader(pReader); - return false; + return code; } static void doFillNullColSMA(SBlockLoadSuppInfo* pSup, int32_t numOfRows, int32_t numOfCols, SColumnDataAgg* pTsAgg) { @@ -4644,20 +4750,27 @@ STableBlockScanInfo* getTableBlockScanInfo(SHashObj* pTableMap, uint64_t uid, co static SSDataBlock* doRetrieveDataBlock(STsdbReader* pReader) { SReaderStatus* pStatus = &pReader->status; + int32_t code = TSDB_CODE_SUCCESS; SFileDataBlockInfo* pBlockInfo = getCurrentBlockInfo(&pStatus->blockIter); STableBlockScanInfo* pBlockScanInfo = getTableBlockScanInfo(pStatus->pTableMap, pBlockInfo->uid, pReader->idStr); if (pBlockScanInfo == NULL) { return NULL; } - int32_t code = doLoadFileBlockData(pReader, &pStatus->blockIter, &pStatus->fileBlockData, pBlockScanInfo->uid); + code = doLoadFileBlockData(pReader, &pStatus->blockIter, &pStatus->fileBlockData, pBlockScanInfo->uid); + if (code != TSDB_CODE_SUCCESS) { + tBlockDataDestroy(&pStatus->fileBlockData); + terrno = code; + return NULL; + } + + code = copyBlockDataToSDataBlock(pReader); if (code != TSDB_CODE_SUCCESS) { tBlockDataDestroy(&pStatus->fileBlockData); terrno = code; return NULL; } - copyBlockDataToSDataBlock(pReader); return pReader->pResBlock; } diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 5dff1abb97..870c96811b 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -584,10 +584,16 @@ int32_t addTagPseudoColumnData(SReadHandle* pHandle, const SExprInfo* pExpr, int if (isNullVal) { colDataSetNNULL(pColInfoData, 0, pBlock->info.rows); } else if (pColInfoData->info.type != TSDB_DATA_TYPE_JSON) { - colDataSetNItems(pColInfoData, 0, data, pBlock->info.rows); + code = colDataSetNItems(pColInfoData, 0, data, pBlock->info.rows, false); if (IS_VAR_DATA_TYPE(((const STagVal*)p)->type)) { taosMemoryFree(data); } + if (code) { + if (freeReader) { + metaReaderClear(&mr); + } + return code; + } } else { // todo opt for json tag for (int32_t i = 0; i < pBlock->info.rows; ++i) { colDataSetVal(pColInfoData, i, data, false); @@ -634,10 +640,22 @@ static SSDataBlock* doTableScanImpl(SOperatorInfo* pOperator) { STableScanInfo* pTableScanInfo = pOperator->info; SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; SSDataBlock* pBlock = pTableScanInfo->pResBlock; + bool hasNext = false; + int32_t code = TSDB_CODE_SUCCESS; int64_t st = taosGetTimestampUs(); - while (tsdbNextDataBlock(pTableScanInfo->base.dataReader)) { + while (true) { + code = tsdbNextDataBlock(pTableScanInfo->base.dataReader, &hasNext); + if (code) { + tsdbReleaseDataBlock(pTableScanInfo->base.dataReader); + T_LONG_JMP(pTaskInfo->env, code); + } + + if (!hasNext) { + break; + } + if (isTaskKilled(pTaskInfo)) { tsdbReleaseDataBlock(pTableScanInfo->base.dataReader); T_LONG_JMP(pTaskInfo->env, pTaskInfo->code); @@ -1015,7 +1033,15 @@ static SSDataBlock* readPreVersionData(SOperatorInfo* pTableScanOp, uint64_t tbU return NULL; } - if (tsdbNextDataBlock(pReader)) { + bool hasNext = false; + code = tsdbNextDataBlock(pReader, &hasNext); + if (code != TSDB_CODE_SUCCESS) { + terrno = code; + T_LONG_JMP(pTaskInfo->env, code); + return NULL; + } + + if (hasNext) { /*SSDataBlock* p = */ tsdbRetrieveDataBlock(pReader, NULL); doSetTagColumnData(&pTableScanInfo->base, pBlock, pTaskInfo, pBlock->info.rows); pBlock->info.id.groupId = getTableGroupId(pTaskInfo->pTableInfoList, pBlock->info.id.uid); @@ -2104,12 +2130,22 @@ static SSDataBlock* doRawScan(SOperatorInfo* pOperator) { // NOTE: this operator does never check if current status is done or not SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; SStreamRawScanInfo* pInfo = pOperator->info; + int32_t code = TSDB_CODE_SUCCESS; pTaskInfo->streamInfo.metaRsp.metaRspLen = 0; // use metaRspLen !=0 to judge if data is meta pTaskInfo->streamInfo.metaRsp.metaRsp = NULL; qDebug("tmqsnap doRawScan called"); if (pTaskInfo->streamInfo.prepareStatus.type == TMQ_OFFSET__SNAPSHOT_DATA) { - if (pInfo->dataReader && tsdbNextDataBlock(pInfo->dataReader)) { + bool hasNext = false; + if (pInfo->dataReader) { + code = tsdbNextDataBlock(pInfo->dataReader, &hasNext); + if (code) { + tsdbReleaseDataBlock(pInfo->dataReader); + longjmp(pTaskInfo->env, code); + } + } + + if (pInfo->dataReader && hasNext) { if (isTaskKilled(pTaskInfo)) { tsdbReleaseDataBlock(pInfo->dataReader); longjmp(pTaskInfo->env, pTaskInfo->code); @@ -2610,8 +2646,21 @@ static SSDataBlock* getTableDataBlockImpl(void* param) { pInfo->base.dataReader = source->dataReader; STsdbReader* reader = pInfo->base.dataReader; + bool hasNext = false; qTrace("tsdb/read-table-data: %p, enter next reader", reader); - while (tsdbNextDataBlock(reader)) { + + while (true) { + code = tsdbNextDataBlock(reader, &hasNext); + if (code != 0) { + tsdbReleaseDataBlock(reader); + pInfo->base.dataReader = NULL; + T_LONG_JMP(pTaskInfo->env, code); + } + + if (!hasNext) { + break; + } + if (isTaskKilled(pTaskInfo)) { tsdbReleaseDataBlock(reader); pInfo->base.dataReader = NULL; diff --git a/source/libs/executor/src/sysscanoperator.c b/source/libs/executor/src/sysscanoperator.c index f24d3523c8..00a90cc108 100644 --- a/source/libs/executor/src/sysscanoperator.c +++ b/source/libs/executor/src/sysscanoperator.c @@ -1627,7 +1627,7 @@ static void sysTableScanFillTbName(SOperatorInfo* pOperator, const SSysTableScan char varTbName[TSDB_TABLE_FNAME_LEN - 1 + VARSTR_HEADER_SIZE] = {0}; STR_TO_VARSTR(varTbName, name); - colDataSetNItems(pColumnInfoData, 0, varTbName, pBlock->info.rows); + colDataSetNItems(pColumnInfoData, 0, varTbName, pBlock->info.rows, true); } doFilter(pBlock, pOperator->exprSupp.pFilterInfo, NULL); diff --git a/source/libs/scalar/src/sclfunc.c b/source/libs/scalar/src/sclfunc.c index 195a08525c..1affbac613 100644 --- a/source/libs/scalar/src/sclfunc.c +++ b/source/libs/scalar/src/sclfunc.c @@ -1755,7 +1755,11 @@ int32_t winEndTsFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *p int32_t qTbnameFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOutput) { char* p = colDataGetVarData(pInput->columnData, 0); - colDataSetNItems(pOutput->columnData, pOutput->numOfRows, p, pInput->numOfRows); + int32_t code = colDataSetNItems(pOutput->columnData, pOutput->numOfRows, p, pInput->numOfRows, true); + if (code) { + return code; + } + pOutput->numOfRows += pInput->numOfRows; return TSDB_CODE_SUCCESS; } diff --git a/tests/parallel_test/cases.task b/tests/parallel_test/cases.task index 58fa0cbfab..6df0651f96 100644 --- a/tests/parallel_test/cases.task +++ b/tests/parallel_test/cases.task @@ -14,6 +14,8 @@ ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/nestedQuery_math.py -Q 2 ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/nestedQuery_time.py -Q 2 ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/nestedQuery.py -Q 2 +,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/columnLenUpdated.py -Q 2 +,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/columnLenUpdated.py -Q 3 ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/nestedQuery.py -Q 3 ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/nestedQuery_str.py -Q 3 ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/nestedQuery_math.py -Q 3 @@ -22,6 +24,7 @@ ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/nestedQuery_str.py -Q 4 ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/nestedQuery_math.py -Q 4 ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/nestedQuery_time.py -Q 4 +,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/columnLenUpdated.py -Q 4 ,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/tmqShow.py ,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/tmqDropStb.py ,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/subscribeStb0.py diff --git a/tests/system-test/2-query/columnLenUpdated.py b/tests/system-test/2-query/columnLenUpdated.py new file mode 100644 index 0000000000..d6940fde8b --- /dev/null +++ b/tests/system-test/2-query/columnLenUpdated.py @@ -0,0 +1,181 @@ + +import taos +import sys +import time +import socket +import os +import platform +if platform.system().lower() == 'windows': + import wexpect as taosExpect +else: + import pexpect as taosExpect + +from util.log import * +from util.sql import * +from util.cases import * +from util.dnodes import * + +def taos_command (buildPath, key, value, expectString, sqlString=''): + if len(key) == 0: + tdLog.exit("taos test key is null!") + + if platform.system().lower() == 'windows': + taosCmd = buildPath + '\\build\\bin\\taos.exe ' + taosCmd = taosCmd.replace('\\','\\\\') + else: + taosCmd = buildPath + '/build/bin/taos ' + + cfgPath = buildPath + "/../sim/psim/cfg" + taosCmd = taosCmd + ' -c' + cfgPath + ' -' + key + if len(value) != 0: + taosCmd = taosCmd + ' ' + value + + tdLog.info ("taos cmd: %s" % taosCmd) + + child = taosExpect.spawn(taosCmd, timeout=20) + #output = child.readline() + #print (output.decode()) + if len(expectString) != 0: + i = child.expect([expectString, taosExpect.TIMEOUT, taosExpect.EOF], timeout=20) + else: + i = child.expect([taosExpect.TIMEOUT, taosExpect.EOF], timeout=20) + + if platform.system().lower() == 'windows': + retResult = child.before + else: + retResult = child.before.decode() + print(retResult) + #print(child.after.decode()) + if i == 0: + print ('taos login success! Here can run sql, taos> ') + return "TAOS_OK" + else: + return "TAOS_FAIL" + +class TDTestCase: + #updatecfgDict = {'clientCfg': {'serverPort': 7080, 'firstEp': 'trd02:7080', 'secondEp':'trd02:7080'},\ + # 'serverPort': 7080, 'firstEp': 'trd02:7080'} + hostname = socket.gethostname() + if (platform.system().lower() == 'windows' and not tdDnodes.dnodes[0].remoteIP == ""): + try: + config = eval(tdDnodes.dnodes[0].remoteIP) + hostname = config["host"] + except Exception: + hostname = tdDnodes.dnodes[0].remoteIP + serverPort = '7080' + rpcDebugFlagVal = '143' + clientCfgDict = {'serverPort': '', 'firstEp': '', 'secondEp':'', 'rpcDebugFlag':'135', 'fqdn':''} + clientCfgDict["serverPort"] = serverPort + clientCfgDict["firstEp"] = hostname + ':' + serverPort + clientCfgDict["secondEp"] = hostname + ':' + serverPort + clientCfgDict["rpcDebugFlag"] = rpcDebugFlagVal + clientCfgDict["fqdn"] = hostname + + updatecfgDict = {'clientCfg': {}, 'serverPort': '', 'firstEp': '', 'secondEp':'', 'rpcDebugFlag':'135', 'fqdn':''} + updatecfgDict["clientCfg"] = clientCfgDict + updatecfgDict["serverPort"] = serverPort + updatecfgDict["firstEp"] = hostname + ':' + serverPort + updatecfgDict["secondEp"] = hostname + ':' + serverPort + updatecfgDict["fqdn"] = hostname + + print ("===================: ", updatecfgDict) + + def init(self, conn, logSql, replicaVar=1): + self.replicaVar = int(replicaVar) + tdLog.debug(f"start to excute {__file__}") + tdSql.init(conn.cursor()) + + def getBuildPath(self): + selfPath = os.path.dirname(os.path.realpath(__file__)) + + if ("community" in selfPath): + projPath = selfPath[:selfPath.find("community")] + else: + projPath = selfPath[:selfPath.find("tests")] + + for root, dirs, files in os.walk(projPath): + if ("taosd" in files or "taosd.exe" in files): + rootRealPath = os.path.dirname(os.path.realpath(root)) + if ("packaging" not in rootRealPath): + buildPath = root[:len(root) - len("/build/bin")] + break + return buildPath + + def run(self): # sourcery skip: extract-duplicate-method, remove-redundant-fstring + tdSql.prepare() + # time.sleep(2) + tdSql.query("create user testpy pass 'testpy'") + + buildPath = self.getBuildPath() + if (buildPath == ""): + tdLog.exit("taosd not found!") + else: + tdLog.info("taosd found in %s" % buildPath) + cfgPath = buildPath + "/../sim/psim/cfg" + tdLog.info("cfgPath: %s" % cfgPath) + + checkNetworkStatus = ['0: unavailable', '1: network ok', '2: service ok', '3: service degraded', '4: exiting'] + netrole = ['client', 'server'] + + keyDict = {'h':'', 'P':'6030', 'p':'testpy', 'u':'testpy', 'a':'', 'A':'', 'c':'', 'C':'', 's':'', 'r':'', 'f':'', \ + 'k':'', 't':'', 'n':'', 'l':'1024', 'N':'100', 'V':'', 'd':'db', 'w':'30', '-help':'', '-usage':'', '?':''} + + keyDict['h'] = self.hostname + keyDict['c'] = cfgPath + keyDict['P'] = self.serverPort + + tdSql.query("drop database if exists db1") + tdSql.query("create database if not exists db1 vgroups 1") + tdSql.query("use db1") + tdSql.query("create table tba (ts timestamp, f1 binary(2))") + tdSql.query("insert into tba values (now, '22')") + tdSql.query("select * from tba") + tdSql.checkData(0, 1, '22') + + keyDict['s'] = "\"alter table db1.tba modify column f1 binary(5) \"" + retCode = taos_command(buildPath, "s", keyDict['s'], "Query OK", '') + if retCode != "TAOS_OK": + tdLog.exit("taos -s fail") + + keyDict['s'] = "\"insert into db1.tba values (now, '55555')\"" + retCode = taos_command(buildPath, "s", keyDict['s'], "Insert OK", '') + if retCode != "TAOS_OK": + tdLog.exit("taos -s fail") + + tdSql.query("select * from tba order by ts") + tdSql.checkData(0, 1, '22') + tdSql.checkData(1, 1, '55555') + + + tdSql.query("create table stb (ts timestamp, f1 int) tags (tg1 binary(2))") + tdSql.query("create table tb1 using stb tags('bb')") + tdSql.query("insert into tb1 values (now, 2)") + tdSql.query("select count(*) from stb group by tg1") + tdSql.checkData(0, 0, 1) + + keyDict['s'] = "\"alter table db1.stb modify tag tg1 binary(5) \"" + retCode = taos_command(buildPath, "s", keyDict['s'], "Query OK", '') + if retCode != "TAOS_OK": + tdLog.exit("taos -s fail") + + keyDict['s'] = "\"create table db1.tb2 using db1.stb tags('bbbbb')\"" + retCode = taos_command(buildPath, "s", keyDict['s'], "Create OK", '') + if retCode != "TAOS_OK": + tdLog.exit("taos -s fail") + + keyDict['s'] = "\"insert into db1.tb2 values (now, 2)\"" + retCode = taos_command(buildPath, "s", keyDict['s'], "Insert OK", '') + if retCode != "TAOS_OK": + tdLog.exit("taos -s fail") + + tdSql.query("select count(*) from stb group by tg1") + tdSql.checkData(0, 0, 1) + tdSql.checkData(1, 0, 1) + + + def stop(self): + tdSql.close() + tdLog.success(f"{__file__} successfully executed") + +tdCases.addLinux(__file__, TDTestCase()) +tdCases.addWindows(__file__, TDTestCase()) From 38b5f4ae1b545c5f55dd3640c82421d0881332e5 Mon Sep 17 00:00:00 2001 From: xiaolei li <85657333+xleili@users.noreply.github.com> Date: Tue, 28 Mar 2023 15:34:48 +0800 Subject: [PATCH 46/65] fix: taosToolsInstallDir if condition order (#20665) --- packaging/tools/makepkg.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/packaging/tools/makepkg.sh b/packaging/tools/makepkg.sh index 29160238ce..a2dec155e8 100755 --- a/packaging/tools/makepkg.sh +++ b/packaging/tools/makepkg.sh @@ -51,9 +51,9 @@ fi if [ -d ${top_dir}/tools/taos-tools/packaging/deb ]; then cd ${top_dir}/tools/taos-tools/packaging/deb - [ -z "$taos_tools_ver" ] && taos_tools_ver="0.1.0" - + taostools_ver=$(git for-each-ref --sort=taggerdate --format '%(tag)' refs/tags|grep -v taos | tail -1) + [ -z "$taos_tools_ver" ] && taos_tools_ver="0.1.0" taostools_install_dir="${release_dir}/${clientName2}Tools-${taostools_ver}" cd ${curr_dir} From 7342aeb750c09d6c89915ea766457dd45468de96 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 28 Mar 2023 15:42:53 +0800 Subject: [PATCH 47/65] fix(tmq): fix the syntax error. --- source/dnode/vnode/src/tq/tqScan.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/tq/tqScan.c b/source/dnode/vnode/src/tq/tqScan.c index c01108f114..1a166d326f 100644 --- a/source/dnode/vnode/src/tq/tqScan.c +++ b/source/dnode/vnode/src/tq/tqScan.c @@ -140,7 +140,7 @@ int32_t tqScanData(STQ* pTq, const STqHandle* pHandle, SMqDataRsp* pRsp, STqOffs return code; } - tqDebug("vgId:%d, consumer:0x%" PRIx64 " tmq task executed, rows:%d, total blocks:%d, rows:%d", vgId, pHandle->consumerId, + tqDebug("vgId:%d, consumer:0x%" PRIx64 " tmq task executed, total blocks:%d, rows:%d", vgId, pHandle->consumerId, pRsp->blockNum, totalRows); return code; From 0ae2034ebbc96472ee53df2ed90341c9bd7baf1b Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 28 Mar 2023 16:05:05 +0800 Subject: [PATCH 48/65] fix: fix syntax error. --- source/dnode/vnode/src/tq/tqPush.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/tq/tqPush.c b/source/dnode/vnode/src/tq/tqPush.c index 73d722dba5..1619829115 100644 --- a/source/dnode/vnode/src/tq/tqPush.c +++ b/source/dnode/vnode/src/tq/tqPush.c @@ -300,7 +300,7 @@ int32_t tqPushMsg(STQ* pTq, void* msg, int32_t msgLen, tmsg_t msgType, int64_t v void* data = taosMemoryMalloc(len); if (data == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; - tqError("failed to copy data for stream since out of memory, vgId:%d, consumer:0x%"PRIx64, vgId); + tqError("failed to copy data for stream since out of memory, vgId:%d", vgId); taosWUnLockLatch(&pTq->lock); return -1; } From b5a1246fea538de8b87e89f7de73bc5e98a2be2f Mon Sep 17 00:00:00 2001 From: Shuduo Sang Date: Tue, 28 Mar 2023 16:52:54 +0800 Subject: [PATCH 49/65] fix: post script add jemalloc (#20672) * fix: tools/post.sh to ldconfig for jemalloc * fix: rpm spec for jemalloc --- packaging/rpm/tdengine.spec | 38 +++++++++----------------- packaging/tools/post.sh | 54 +++++++++++++++++++++++++++++++++++++ 2 files changed, 67 insertions(+), 25 deletions(-) diff --git a/packaging/rpm/tdengine.spec b/packaging/rpm/tdengine.spec index e9d86219cb..c21063e6a4 100644 --- a/packaging/rpm/tdengine.spec +++ b/packaging/rpm/tdengine.spec @@ -90,45 +90,33 @@ cp %{_compiledir}/../include/libs/function/taosudf.h %{buildroot}%{homepat cp -r %{_compiledir}/../examples/* %{buildroot}%{homepath}/examples if [ -f %{_compiledir}/build/bin/jemalloc-config ]; then - mkdir -p %{buildroot}%{userlocalpath}/bin - mkdir -p %{buildroot}%{userlocalpath}/lib - mkdir -p %{buildroot}%{userlocalpath}/lib/pkgconfig - mkdir -p %{buildroot}%{userlocalpath}/include - mkdir -p %{buildroot}%{userlocalpath}/include/jemalloc - mkdir -p %{buildroot}%{userlocalpath}/share - mkdir -p %{buildroot}%{userlocalpath}/share/doc - mkdir -p %{buildroot}%{userlocalpath}/share/doc/jemalloc - mkdir -p %{buildroot}%{userlocalpath}/share/man - mkdir -p %{buildroot}%{userlocalpath}/share/man/man3 + mkdir -p %{buildroot}%{homepath}/jemalloc/ ||: + mkdir -p %{buildroot}%{homepath}/jemalloc/include/jemalloc/ ||: + mkdir -p %{buildroot}%{homepath}/jemalloc/lib/ ||: + mkdir -p %{buildroot}%{homepath}/jemalloc/lib/pkgconfig ||: - cp %{_compiledir}/build/bin/jemalloc-config %{buildroot}%{userlocalpath}/bin/ + cp %{_compiledir}/build/bin/jemalloc-config %{buildroot}%{homepath}/jemalloc/bin if [ -f %{_compiledir}/build/bin/jemalloc.sh ]; then - cp %{_compiledir}/build/bin/jemalloc.sh %{buildroot}%{userlocalpath}/bin/ + cp %{_compiledir}/build/bin/jemalloc.sh %{buildroot}%{homepath}/jemalloc/bin fi if [ -f %{_compiledir}/build/bin/jeprof ]; then - cp %{_compiledir}/build/bin/jeprof %{buildroot}%{userlocalpath}/bin/ + cp %{_compiledir}/build/bin/jeprof %{buildroot}%{homepath}/jemalloc/bin fi if [ -f %{_compiledir}/build/include/jemalloc/jemalloc.h ]; then - cp %{_compiledir}/build/include/jemalloc/jemalloc.h %{buildroot}%{userlocalpath}/include/jemalloc/ + cp %{_compiledir}/build/include/jemalloc/jemalloc.h %{buildroot}%{homepath}/jemalloc/include/jemalloc/ fi if [ -f %{_compiledir}/build/lib/libjemalloc.so.2 ]; then - cp %{_compiledir}/build/lib/libjemalloc.so.2 %{buildroot}%{userlocalpath}/lib/ - ln -sf libjemalloc.so.2 %{buildroot}%{userlocalpath}/lib/libjemalloc.so + cp %{_compiledir}/build/lib/libjemalloc.so.2 %{buildroot}%{homepath}/jemalloc/lib + ln -sf libjemalloc.so.2 %{buildroot}%{homepath}/jemalloc/lib/libjemalloc.so fi if [ -f %{_compiledir}/build/lib/libjemalloc.a ]; then - cp %{_compiledir}/build/lib/libjemalloc.a %{buildroot}%{userlocalpath}/lib/ + cp %{_compiledir}/build/lib/libjemalloc.a %{buildroot}%{homepath}/jemalloc/lib fi if [ -f %{_compiledir}/build/lib/libjemalloc_pic.a ]; then - cp %{_compiledir}/build/lib/libjemalloc_pic.a %{buildroot}%{userlocalpath}/lib/ + cp %{_compiledir}/build/lib/libjemalloc_pic.a %{buildroot}%{homepath}/jemalloc/lib fi if [ -f %{_compiledir}/build/lib/pkgconfig/jemalloc.pc ]; then - cp %{_compiledir}/build/lib/pkgconfig/jemalloc.pc %{buildroot}%{userlocalpath}/lib/pkgconfig/ - fi - if [ -f %{_compiledir}/build/share/doc/jemalloc/jemalloc.html ]; then - cp %{_compiledir}/build/share/doc/jemalloc/jemalloc.html %{buildroot}%{userlocalpath}/share/doc/jemalloc/ - fi - if [ -f %{_compiledir}/build/share/man/man3/jemalloc.3 ]; then - cp %{_compiledir}/build/share/man/man3/jemalloc.3 %{buildroot}%{userlocalpath}/share/man/man3/ + cp %{_compiledir}/build/lib/pkgconfig/jemalloc.pc %{buildroot}%{homepath}/jemalloc/lib/pkgconfig fi fi diff --git a/packaging/tools/post.sh b/packaging/tools/post.sh index c9fab51b28..35c28feb6a 100755 --- a/packaging/tools/post.sh +++ b/packaging/tools/post.sh @@ -145,6 +145,59 @@ function install_include() { log_print "install include success" } +function install_jemalloc() { + jemalloc_dir=${script_dir}/../jemalloc + + if [ -d ${jemalloc_dir} ]; then + ${csudo}/usr/bin/install -c -d /usr/local/bin + + if [ -f ${jemalloc_dir}/bin/jemalloc-config ]; then + ${csudo}/usr/bin/install -c -m 755 ${jemalloc_dir}/bin/jemalloc-config /usr/local/bin + fi + if [ -f ${jemalloc_dir}/bin/jemalloc.sh ]; then + ${csudo}/usr/bin/install -c -m 755 ${jemalloc_dir}/bin/jemalloc.sh /usr/local/bin + fi + if [ -f ${jemalloc_dir}/bin/jeprof ]; then + ${csudo}/usr/bin/install -c -m 755 ${jemalloc_dir}/bin/jeprof /usr/local/bin + fi + if [ -f ${jemalloc_dir}/include/jemalloc/jemalloc.h ]; then + ${csudo}/usr/bin/install -c -d /usr/local/include/jemalloc + ${csudo}/usr/bin/install -c -m 644 ${jemalloc_dir}/include/jemalloc/jemalloc.h /usr/local/include/jemalloc + fi + if [ -f ${jemalloc_dir}/lib/libjemalloc.so.2 ]; then + ${csudo}/usr/bin/install -c -d /usr/local/lib + ${csudo}/usr/bin/install -c -m 755 ${jemalloc_dir}/lib/libjemalloc.so.2 /usr/local/lib + ${csudo}ln -sf libjemalloc.so.2 /usr/local/lib/libjemalloc.so + ${csudo}/usr/bin/install -c -d /usr/local/lib + if [ -f ${jemalloc_dir}/lib/libjemalloc.a ]; then + ${csudo}/usr/bin/install -c -m 755 ${jemalloc_dir}/lib/libjemalloc.a /usr/local/lib + fi + if [ -f ${jemalloc_dir}/lib/libjemalloc_pic.a ]; then + ${csudo}/usr/bin/install -c -m 755 ${jemalloc_dir}/lib/libjemalloc_pic.a /usr/local/lib + fi + if [ -f ${jemalloc_dir}/lib/libjemalloc_pic.a ]; then + ${csudo}/usr/bin/install -c -d /usr/local/lib/pkgconfig + ${csudo}/usr/bin/install -c -m 644 ${jemalloc_dir}/lib/pkgconfig/jemalloc.pc /usr/local/lib/pkgconfig + fi + fi + if [ -f ${jemalloc_dir}/share/doc/jemalloc/jemalloc.html ]; then + ${csudo}/usr/bin/install -c -d /usr/local/share/doc/jemalloc + ${csudo}/usr/bin/install -c -m 644 ${jemalloc_dir}/share/doc/jemalloc/jemalloc.html /usr/local/share/doc/jemalloc + fi + if [ -f ${jemalloc_dir}/share/man/man3/jemalloc.3 ]; then + ${csudo}/usr/bin/install -c -d /usr/local/share/man/man3 + ${csudo}/usr/bin/install -c -m 644 ${jemalloc_dir}/share/man/man3/jemalloc.3 /usr/local/share/man/man3 + fi + + if [ -d /etc/ld.so.conf.d ]; then + echo "/usr/local/lib" | ${csudo}tee /etc/ld.so.conf.d/jemalloc.conf >/dev/null || echo -e "failed to write /etc/ld.so.conf.d/jemalloc.conf" + ${csudo}ldconfig + else + echo "/etc/ld.so.conf.d not found!" + fi + fi +} + function install_lib() { log_print "start install lib from ${lib_dir} to ${lib_link_dir}" ${csudo}rm -f ${lib_link_dir}/libtaos* || : @@ -663,6 +716,7 @@ function install_TDengine() { # Install include, lib, binary and service install_include && install_lib && + install_jemalloc install_bin if [[ "$?" != 0 ]];then From b11a105327f3154524bfd9cdf79d6633257dc001 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Tue, 28 Mar 2023 17:36:23 +0800 Subject: [PATCH 50/65] fix:[TD-23339] parse block error --- source/libs/parser/src/parInsertUtil.c | 53 ++++++++++++++++++++++---- 1 file changed, 45 insertions(+), 8 deletions(-) diff --git a/source/libs/parser/src/parInsertUtil.c b/source/libs/parser/src/parInsertUtil.c index 0ed32afbbc..90a40ba885 100644 --- a/source/libs/parser/src/parInsertUtil.c +++ b/source/libs/parser/src/parInsertUtil.c @@ -631,10 +631,10 @@ int rawBlockBindData(SQuery* query, STableMeta* pTableMeta, void* data, SVCreate ret = TSDB_CODE_INVALID_PARA; goto end; } - for (int c = 0; c < boundInfo->numOfBound; ++c) { - SSchema* pColSchema = &pSchema[c]; - SColData* pCol = taosArrayGet(pTableCxt->pData->aCol, c); - if(tFields == NULL || findFileds(pColSchema, tFields, numFields)){ + if(tFields == NULL){ + for (int j = 0; j < boundInfo->numOfBound; j++){ + SSchema* pColSchema = &pSchema[j]; + SColData* pCol = taosArrayGet(pTableCxt->pData->aCol, j); if (*fields != pColSchema->type && *(int32_t*)(fields + sizeof(int8_t)) != pColSchema->bytes) { uError("type or bytes not equal"); ret = TSDB_CODE_INVALID_PARA; @@ -652,12 +652,49 @@ int rawBlockBindData(SQuery* query, STableMeta* pTableMeta, void* data, SVCreate tColDataAddValueByDataBlock(pCol, pColSchema->type, pColSchema->bytes, numOfRows, offset, pData); fields += sizeof(int8_t) + sizeof(int32_t); if (needChangeLength) { - pStart += htonl(colLength[c]); + pStart += htonl(colLength[j]); } else { - pStart += colLength[c]; + pStart += colLength[j]; + } + } + }else{ + for (int i = 0; i < numFields; i++) { + for (int j = 0; j < boundInfo->numOfBound; j++){ + SSchema* pColSchema = &pSchema[j]; + SColData* pCol = taosArrayGet(pTableCxt->pData->aCol, j); + if(strcmp(pSchema->name, tFields[i].name) == 0){ + if (*fields != pColSchema->type && *(int32_t*)(fields + sizeof(int8_t)) != pColSchema->bytes) { + uError("type or bytes not equal"); + ret = TSDB_CODE_INVALID_PARA; + goto end; + } + + int8_t* offset = pStart; + if (IS_VAR_DATA_TYPE(pColSchema->type)) { + pStart += numOfRows * sizeof(int32_t); + } else { + pStart += BitmapLen(numOfRows); + } + char* pData = pStart; + + tColDataAddValueByDataBlock(pCol, pColSchema->type, pColSchema->bytes, numOfRows, offset, pData); + fields += sizeof(int8_t) + sizeof(int32_t); + if (needChangeLength) { + pStart += htonl(colLength[i]); + } else { + pStart += colLength[i]; + } + boundInfo->pColIndex[j] = -1; + } + } + + } + + for (int c = 0; c < boundInfo->numOfBound; ++c) { + if( boundInfo->pColIndex[c] != -1){ + SColData* pCol = taosArrayGet(pTableCxt->pData->aCol, c); + tColDataAddValueByDataBlock(pCol, 0, 0, numOfRows, NULL, NULL); } - }else{ - tColDataAddValueByDataBlock(pCol, pColSchema->type, pColSchema->bytes, numOfRows, NULL, NULL); } } From 78ab9d3bf9a8f38aecec1eb3eb20c61139e2fe58 Mon Sep 17 00:00:00 2001 From: Benguang Zhao Date: Tue, 28 Mar 2023 17:11:40 +0800 Subject: [PATCH 51/65] fix: process appendLog replies in vnode-sync thread --- source/dnode/mgmt/mgmt_mnode/src/mmHandle.c | 4 ++-- source/dnode/mgmt/mgmt_vnode/src/vmHandle.c | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c b/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c index ce485cb6ca..c2c9e37c8c 100644 --- a/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c +++ b/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c @@ -192,16 +192,16 @@ SArray *mmGetMsgHandles() { if (dmSetMgmtHandle(pArray, TDMT_SYNC_HEARTBEAT, mmPutMsgToSyncRdQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_CLIENT_REQUEST, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_CLIENT_REQUEST_BATCH, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_SYNC_CLIENT_REQUEST_REPLY, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_REQUEST_VOTE, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_REQUEST_VOTE_REPLY, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_APPEND_ENTRIES, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_APPEND_ENTRIES_BATCH, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_SYNC_APPEND_ENTRIES_REPLY, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_SNAPSHOT_SEND, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_PRE_SNAPSHOT, mmPutMsgToSyncQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_TIMEOUT, mmPutMsgToSyncRdQueue, 1) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_SYNC_CLIENT_REQUEST_REPLY, mmPutMsgToSyncRdQueue, 1) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_SYNC_APPEND_ENTRIES_REPLY, mmPutMsgToSyncRdQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_HEARTBEAT_REPLY, mmPutMsgToSyncRdQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_SNAPSHOT_RSP, mmPutMsgToSyncRdQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_PRE_SNAPSHOT_REPLY, mmPutMsgToSyncRdQueue, 1) == NULL) goto _OVER; diff --git a/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c b/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c index 84dfab7e6c..28cb5d2058 100644 --- a/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c +++ b/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c @@ -553,17 +553,17 @@ SArray *vmGetMsgHandles() { if (dmSetMgmtHandle(pArray, TDMT_SYNC_HEARTBEAT, vmPutMsgToSyncRdQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_CLIENT_REQUEST, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_CLIENT_REQUEST_BATCH, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_SYNC_CLIENT_REQUEST_REPLY, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_REQUEST_VOTE, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_REQUEST_VOTE_REPLY, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_APPEND_ENTRIES, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_APPEND_ENTRIES_BATCH, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_SYNC_APPEND_ENTRIES_REPLY, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_SNAPSHOT_SEND, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_PRE_SNAPSHOT, vmPutMsgToSyncQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_TIMEOUT, vmPutMsgToSyncRdQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_HEARTBEAT_REPLY, vmPutMsgToSyncRdQueue, 0) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_SYNC_CLIENT_REQUEST_REPLY, vmPutMsgToSyncRdQueue, 0) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_SYNC_APPEND_ENTRIES_REPLY, vmPutMsgToSyncRdQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_SNAPSHOT_RSP, vmPutMsgToSyncRdQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_SYNC_PRE_SNAPSHOT_REPLY, vmPutMsgToSyncRdQueue, 0) == NULL) goto _OVER; From f1ff5dcec484f87bd07f03627731ebd492d0a3c4 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 28 Mar 2023 22:53:57 +0800 Subject: [PATCH 52/65] fix(query): use hashmap to keep the multiple schema. --- include/util/tsimplehash.h | 7 +++ source/dnode/vnode/src/tsdb/tsdbRead.c | 59 ++++++++++++++------------ source/util/src/tsimplehash.c | 38 ++++++++++------- 3 files changed, 62 insertions(+), 42 deletions(-) diff --git a/include/util/tsimplehash.h b/include/util/tsimplehash.h index c9df911476..987a9fe2a8 100644 --- a/include/util/tsimplehash.h +++ b/include/util/tsimplehash.h @@ -48,6 +48,13 @@ SSHashObj *tSimpleHashInit(size_t capacity, _hash_fn_t fn); */ int32_t tSimpleHashGetSize(const SSHashObj *pHashObj); +/** + * set the free function pointer + * @param pHashObj + * @param freeFp + */ +void tSimpleHashSetFreeFp(SSHashObj* pHashObj, _hash_free_fn_t freeFp); + int32_t tSimpleHashPrint(const SSHashObj *pHashObj); /** diff --git a/source/dnode/vnode/src/tsdb/tsdbRead.c b/source/dnode/vnode/src/tsdb/tsdbRead.c index 89d3239c33..3d8b2a4031 100644 --- a/source/dnode/vnode/src/tsdb/tsdbRead.c +++ b/source/dnode/vnode/src/tsdb/tsdbRead.c @@ -15,6 +15,7 @@ #include "osDef.h" #include "tsdb.h" +#include "tsimplehash.h" #define ASCENDING_TRAVERSE(o) (o == TSDB_ORDER_ASC) @@ -177,7 +178,8 @@ struct STsdbReader { STsdbReadSnap* pReadSnap; SIOCostSummary cost; STSchema* pSchema; // the newest version schema - STSchema* pMemSchema; // the previous schema for in-memory data, to avoid load schema too many times +// STSchema* pMemSchema; // the previous schema for in-memory data, to avoid load schema too many times + SSHashObj* pSchemaMap; // keep the retrieved schema info, to avoid the overhead by repeatly load schema SDataFReader* pFileReader; // the file reader SDelFReader* pDelFReader; // the del file reader SArray* pDelIdx; // del file block index; @@ -1858,28 +1860,23 @@ static FORCE_INLINE STSchema* doGetSchemaForTSRow(int32_t sversion, STsdbReader* return pReader->pSchema; } - if (pReader->pMemSchema == NULL) { - int32_t code = - metaGetTbTSchemaEx(pReader->pTsdb->pVnode->pMeta, pReader->suid, uid, sversion, &pReader->pMemSchema); - if (code != TSDB_CODE_SUCCESS) { - terrno = code; - return NULL; - } else { - return pReader->pMemSchema; - } + void** p = tSimpleHashGet(pReader->pSchemaMap, &sversion, sizeof(sversion)); + if (p != NULL) { + return *(STSchema**) p; } - if (pReader->pMemSchema->version == sversion) { - return pReader->pMemSchema; - } - - taosMemoryFreeClear(pReader->pMemSchema); - int32_t code = metaGetTbTSchemaEx(pReader->pTsdb->pVnode->pMeta, pReader->suid, uid, sversion, &pReader->pMemSchema); - if (code != TSDB_CODE_SUCCESS || pReader->pMemSchema == NULL) { + STSchema* ptr = NULL; + int32_t code = metaGetTbTSchemaEx(pReader->pTsdb->pVnode->pMeta, pReader->suid, uid, sversion, &ptr); + if (code != TSDB_CODE_SUCCESS) { terrno = code; return NULL; } else { - return pReader->pMemSchema; + code = tSimpleHashPut(pReader->pSchemaMap, &sversion, sizeof(sversion), &ptr, POINTER_BYTES); + if (code != TSDB_CODE_SUCCESS) { + terrno = code; + return NULL; + } + return ptr; } } @@ -4002,6 +3999,11 @@ static int32_t doOpenReaderImpl(STsdbReader* pReader) { return code; } +static void freeSchemaFunc(void* param) { + void* p = *(void**) param; + taosMemoryFree(p); +} + // ====================================== EXPOSED APIs ====================================== int32_t tsdbReaderOpen(SVnode* pVnode, SQueryTableDataCond* pCond, void* pTableList, int32_t numOfTables, SSDataBlock* pResBlock, STsdbReader** ppReader, const char* idstr) { @@ -4078,6 +4080,14 @@ int32_t tsdbReaderOpen(SVnode* pVnode, SQueryTableDataCond* pCond, void* pTableL } } + pReader->pSchemaMap = tSimpleHashInit(8, taosFastHash); + if (pReader->pSchemaMap == NULL) { + tsdbError("failed init schema hash for reader", pReader->idStr); + code = TSDB_CODE_OUT_OF_MEMORY; + goto _err; + } + + tSimpleHashSetFreeFp(pReader->pSchemaMap, freeSchemaFunc); if (pReader->pSchema != NULL) { code = updateBlockSMAInfo(pReader->pSchema, &pReader->suppInfo); if (code != TSDB_CODE_SUCCESS) { @@ -4120,7 +4130,7 @@ void tsdbReaderClose(STsdbReader* pReader) { p->status.uidList.tableUidList = NULL; p->pReadSnap = NULL; p->pSchema = NULL; - p->pMemSchema = NULL; + p->pSchemaMap = NULL; p = pReader->innerReader[1]; @@ -4128,7 +4138,7 @@ void tsdbReaderClose(STsdbReader* pReader) { p->status.uidList.tableUidList = NULL; p->pReadSnap = NULL; p->pSchema = NULL; - p->pMemSchema = NULL; + p->pSchemaMap = NULL; tsdbReaderClose(pReader->innerReader[0]); tsdbReaderClose(pReader->innerReader[1]); @@ -4208,10 +4218,7 @@ void tsdbReaderClose(STsdbReader* pReader) { taosMemoryFree(pReader->idStr); taosMemoryFree(pReader->pSchema); - if (pReader->pMemSchema != pReader->pSchema) { - taosMemoryFree(pReader->pMemSchema); - } - + tSimpleHashCleanup(pReader->pSchemaMap); taosMemoryFreeClear(pReader); } @@ -4371,14 +4378,14 @@ int32_t tsdbReaderResume(STsdbReader* pReader) { pPrevReader->status.pTableMap = pReader->status.pTableMap; pPrevReader->status.uidList = pReader->status.uidList; pPrevReader->pSchema = pReader->pSchema; - pPrevReader->pMemSchema = pReader->pMemSchema; + pPrevReader->pSchemaMap = pReader->pSchemaMap; pPrevReader->pReadSnap = pReader->pReadSnap; pNextReader->capacity = 1; pNextReader->status.pTableMap = pReader->status.pTableMap; pNextReader->status.uidList = pReader->status.uidList; pNextReader->pSchema = pReader->pSchema; - pNextReader->pMemSchema = pReader->pMemSchema; + pNextReader->pSchemaMap = pReader->pSchemaMap; pNextReader->pReadSnap = pReader->pReadSnap; code = doOpenReaderImpl(pPrevReader); diff --git a/source/util/src/tsimplehash.c b/source/util/src/tsimplehash.c index 70acffed5d..310fdd27ab 100644 --- a/source/util/src/tsimplehash.c +++ b/source/util/src/tsimplehash.c @@ -28,19 +28,23 @@ #define HASH_INDEX(v, c) ((v) & ((c)-1)) -#define FREE_HASH_NODE(_n) \ - do { \ - taosMemoryFreeClear(_n); \ +#define FREE_HASH_NODE(_n, fp) \ + do { \ + if (fp) { \ + fp((_n)->data); \ + } \ + taosMemoryFreeClear(_n); \ } while (0); struct SSHashObj { - SHNode **hashList; - size_t capacity; // number of slots - int64_t size; // number of elements in hash table - _hash_fn_t hashFp; // hash function - _equal_fn_t equalFp; // equal function - SArray* pHashNodeBuf;// hash node allocation buffer, 1k size of each page by default - int32_t offset; // allocation offset in current page + SHNode **hashList; + size_t capacity; // number of slots + int64_t size; // number of elements in hash table + _hash_fn_t hashFp; // hash function + _equal_fn_t equalFp; // equal function + _hash_free_fn_t freeFp; // free function + SArray *pHashNodeBuf; // hash node allocation buffer, 1k size of each page by default + int32_t offset; // allocation offset in current page }; static FORCE_INLINE int32_t taosHashCapacity(int32_t length) { @@ -71,7 +75,6 @@ SSHashObj *tSimpleHashInit(size_t capacity, _hash_fn_t fn) { pHashObj->capacity = taosHashCapacity((int32_t)capacity); pHashObj->equalFp = memcmp; - pHashObj->pHashNodeBuf = taosArrayInit(10, sizeof(void*)); pHashObj->offset = 0; pHashObj->size = 0; @@ -92,6 +95,10 @@ int32_t tSimpleHashGetSize(const SSHashObj *pHashObj) { return (int32_t) pHashObj->size; } +void tSimpleHashSetFreeFp(SSHashObj* pHashObj, _hash_free_fn_t freeFp) { + pHashObj->freeFp = freeFp; +} + static void* doInternalAlloc(SSHashObj* pHashObj, int32_t size) { #if 0 void** p = taosArrayGetLast(pHashObj->pHashNodeBuf); @@ -306,7 +313,8 @@ int32_t tSimpleHashRemove(SSHashObj *pHashObj, const void *key, size_t keyLen) { } else { pPrev->next = pNode->next; } - FREE_HASH_NODE(pNode); + + FREE_HASH_NODE(pNode, pHashObj->freeFp); pHashObj->size -= 1; code = TSDB_CODE_SUCCESS; break; @@ -341,7 +349,7 @@ int32_t tSimpleHashIterateRemove(SSHashObj *pHashObj, const void *key, size_t ke *pIter = pPrev ? GET_SHASH_NODE_DATA(pPrev) : NULL; } - FREE_HASH_NODE(pNode); + FREE_HASH_NODE(pNode, pHashObj->freeFp); pHashObj->size -= 1; break; } @@ -370,14 +378,13 @@ void tSimpleHashClear(SSHashObj *pHashObj) { while (pNode) { pNext = pNode->next; - FREE_HASH_NODE(pNode); + FREE_HASH_NODE(pNode, pHashObj->freeFp); pNode = pNext; } pHashObj->hashList[i] = NULL; } - taosArrayClearEx(pHashObj->pHashNodeBuf, destroyItems); pHashObj->offset = 0; pHashObj->size = 0; } @@ -388,7 +395,6 @@ void tSimpleHashCleanup(SSHashObj *pHashObj) { } tSimpleHashClear(pHashObj); - taosArrayDestroy(pHashObj->pHashNodeBuf); taosMemoryFreeClear(pHashObj->hashList); taosMemoryFree(pHashObj); } From ceb160c23bcf683af37e4aa54bf89345780d9f1a Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 28 Mar 2023 23:14:11 +0800 Subject: [PATCH 53/65] fix(query): fix memory leak. --- source/dnode/mnode/impl/src/mndConsumer.c | 24 ++++++++++++++++++++-- source/dnode/mnode/impl/src/mndSubscribe.c | 1 + 2 files changed, 23 insertions(+), 2 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndConsumer.c b/source/dnode/mnode/impl/src/mndConsumer.c index 0b2b5ec35c..6acf7fb98a 100644 --- a/source/dnode/mnode/impl/src/mndConsumer.c +++ b/source/dnode/mnode/impl/src/mndConsumer.c @@ -247,6 +247,13 @@ static SMqRebInfo *mndGetOrCreateRebSub(SHashObj *pHash, const char *key) { return pRebInfo; } +static void freeRebalanceItem(void* param) { + SMqRebInfo* pInfo = param; + taosArrayDestroy(pInfo->lostConsumers); + taosArrayDestroy(pInfo->newConsumers); + taosArrayDestroy(pInfo->removedConsumers); +} + static int32_t mndProcessMqTimerMsg(SRpcMsg *pMsg) { SMnode *pMnode = pMsg->info.node; SSdb *pSdb = pMnode->pSdb; @@ -262,8 +269,21 @@ static int32_t mndProcessMqTimerMsg(SRpcMsg *pMsg) { } SMqDoRebalanceMsg *pRebMsg = rpcMallocCont(sizeof(SMqDoRebalanceMsg)); + if (pRebMsg == NULL) { + mError("failed to create the rebalance msg, size:%d, quit mq timer", sizeof(SMqDoRebalanceMsg)); + mndRebEnd(); + return TSDB_CODE_OUT_OF_MEMORY; + } + pRebMsg->rebSubHash = taosHashInit(64, MurmurHash3_32, true, HASH_NO_LOCK); - // TODO set cleanfp + if (pRebMsg->rebSubHash == NULL) { + mError("failed to create rebalance hashmap"); + rpcFreeCont(pRebMsg); + mndRebEnd(); + return TSDB_CODE_OUT_OF_MEMORY; + } + + taosHashSetFreeFp(pRebMsg->rebSubHash, freeRebalanceItem); // iterate all consumers, find all modification while (1) { @@ -356,7 +376,7 @@ static int32_t mndProcessMqTimerMsg(SRpcMsg *pMsg) { } else { taosHashCleanup(pRebMsg->rebSubHash); rpcFreeCont(pRebMsg); - mDebug("mq rebalance finished, no modification"); + mDebug("mq timer finished, no need to re-balance"); mndRebEnd(); } return 0; diff --git a/source/dnode/mnode/impl/src/mndSubscribe.c b/source/dnode/mnode/impl/src/mndSubscribe.c index b8dbec2f84..739de68e5f 100644 --- a/source/dnode/mnode/impl/src/mndSubscribe.c +++ b/source/dnode/mnode/impl/src/mndSubscribe.c @@ -611,6 +611,7 @@ static int32_t mndProcessRebalanceReq(SRpcMsg *pMsg) { break; } + // todo handle the malloc failure SMqRebInputObj rebInput = {0}; SMqRebOutputObj rebOutput = {0}; rebOutput.newConsumers = taosArrayInit(0, sizeof(int64_t)); From 70d0d7a63fa97cfe2191519ac4efafb2bd09092f Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 29 Mar 2023 09:08:07 +0800 Subject: [PATCH 54/65] fix(tmq): fix the syntax error. --- source/dnode/mnode/impl/src/mndConsumer.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/dnode/mnode/impl/src/mndConsumer.c b/source/dnode/mnode/impl/src/mndConsumer.c index 6acf7fb98a..68670eedfb 100644 --- a/source/dnode/mnode/impl/src/mndConsumer.c +++ b/source/dnode/mnode/impl/src/mndConsumer.c @@ -270,7 +270,7 @@ static int32_t mndProcessMqTimerMsg(SRpcMsg *pMsg) { SMqDoRebalanceMsg *pRebMsg = rpcMallocCont(sizeof(SMqDoRebalanceMsg)); if (pRebMsg == NULL) { - mError("failed to create the rebalance msg, size:%d, quit mq timer", sizeof(SMqDoRebalanceMsg)); + mError("failed to create the rebalance msg, size:%d, quit mq timer", (int32_t) sizeof(SMqDoRebalanceMsg)); mndRebEnd(); return TSDB_CODE_OUT_OF_MEMORY; } From d5a06da96e12312c247eeaa5948b21a540d6cab9 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Wed, 29 Mar 2023 09:31:00 +0800 Subject: [PATCH 55/65] fix:[TD-23339] parse block error --- source/libs/parser/src/parInsertUtil.c | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/source/libs/parser/src/parInsertUtil.c b/source/libs/parser/src/parInsertUtil.c index 90a40ba885..779ba091aa 100644 --- a/source/libs/parser/src/parInsertUtil.c +++ b/source/libs/parser/src/parInsertUtil.c @@ -661,8 +661,7 @@ int rawBlockBindData(SQuery* query, STableMeta* pTableMeta, void* data, SVCreate for (int i = 0; i < numFields; i++) { for (int j = 0; j < boundInfo->numOfBound; j++){ SSchema* pColSchema = &pSchema[j]; - SColData* pCol = taosArrayGet(pTableCxt->pData->aCol, j); - if(strcmp(pSchema->name, tFields[i].name) == 0){ + if(strcmp(pColSchema->name, tFields[i].name) == 0){ if (*fields != pColSchema->type && *(int32_t*)(fields + sizeof(int8_t)) != pColSchema->bytes) { uError("type or bytes not equal"); ret = TSDB_CODE_INVALID_PARA; @@ -677,6 +676,7 @@ int rawBlockBindData(SQuery* query, STableMeta* pTableMeta, void* data, SVCreate } char* pData = pStart; + SColData* pCol = taosArrayGet(pTableCxt->pData->aCol, j); tColDataAddValueByDataBlock(pCol, pColSchema->type, pColSchema->bytes, numOfRows, offset, pData); fields += sizeof(int8_t) + sizeof(int32_t); if (needChangeLength) { @@ -685,6 +685,7 @@ int rawBlockBindData(SQuery* query, STableMeta* pTableMeta, void* data, SVCreate pStart += colLength[i]; } boundInfo->pColIndex[j] = -1; + break; } } From 6f6239a7fa272e80630cf825db7deb9c2afd42b8 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Wed, 29 Mar 2023 09:44:26 +0800 Subject: [PATCH 56/65] fix:[TD-23339] parse block error --- source/libs/parser/src/parInsertUtil.c | 2 ++ 1 file changed, 2 insertions(+) diff --git a/source/libs/parser/src/parInsertUtil.c b/source/libs/parser/src/parInsertUtil.c index 779ba091aa..132a3b2618 100644 --- a/source/libs/parser/src/parInsertUtil.c +++ b/source/libs/parser/src/parInsertUtil.c @@ -695,6 +695,8 @@ int rawBlockBindData(SQuery* query, STableMeta* pTableMeta, void* data, SVCreate if( boundInfo->pColIndex[c] != -1){ SColData* pCol = taosArrayGet(pTableCxt->pData->aCol, c); tColDataAddValueByDataBlock(pCol, 0, 0, numOfRows, NULL, NULL); + }else{ + boundInfo->pColIndex[c] = c; // restore for next block } } } From 702202d4afa78ae892344c6b18f0c43e09435c6a Mon Sep 17 00:00:00 2001 From: dapan1121 Date: Wed, 29 Mar 2023 10:14:53 +0800 Subject: [PATCH 57/65] fix: fix block scan issue --- source/dnode/vnode/src/tsdb/tsdbRead.c | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/source/dnode/vnode/src/tsdb/tsdbRead.c b/source/dnode/vnode/src/tsdb/tsdbRead.c index e11a80efb3..5867580d2d 100644 --- a/source/dnode/vnode/src/tsdb/tsdbRead.c +++ b/source/dnode/vnode/src/tsdb/tsdbRead.c @@ -4496,17 +4496,16 @@ static int32_t doTsdbNextDataBlock(STsdbReader* pReader, bool *hasNext) { return code; } - if (pBlock->info.rows > 0) { - *hasNext = true; - } else { + if (pBlock->info.rows <= 0) { resetTableListIndex(&pReader->status); code = buildBlockFromBufferSequentially(pReader); } } else { // no data in files, let's try the buffer code = buildBlockFromBufferSequentially(pReader); - *hasNext = pBlock->info.rows > 0; } + *hasNext = pBlock->info.rows > 0; + return code; } From 7a050d64bdc6573201d26af6b3046495358f0e5e Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 29 Mar 2023 10:27:05 +0800 Subject: [PATCH 58/65] fix(query): set ptr to be NULL. --- source/dnode/vnode/src/tsdb/tsdbRead.c | 5 +++-- source/util/src/tsimplehash.c | 1 + 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/source/dnode/vnode/src/tsdb/tsdbRead.c b/source/dnode/vnode/src/tsdb/tsdbRead.c index 3d8b2a4031..d9f6b66d6a 100644 --- a/source/dnode/vnode/src/tsdb/tsdbRead.c +++ b/source/dnode/vnode/src/tsdb/tsdbRead.c @@ -3673,7 +3673,7 @@ int32_t doMergeMemIMemRows(TSDBROW* pRow, TSDBROW* piRow, STableBlockScanInfo* p TSDBKEY ik = TSDBROW_KEY(piRow); if (ASCENDING_TRAVERSE(pReader->order)) { // ascending order imem --> mem - STSchema* pSchema = doGetSchemaForTSRow(TSDBROW_SVERSION(pRow), pReader, pBlockScanInfo->uid); + STSchema* pSchema = doGetSchemaForTSRow(TSDBROW_SVERSION(piRow), pReader, pBlockScanInfo->uid); int32_t code = tsdbRowMergerInit(&merge, piRow, pSchema); if (code != TSDB_CODE_SUCCESS) { @@ -3686,7 +3686,8 @@ int32_t doMergeMemIMemRows(TSDBROW* pRow, TSDBROW* piRow, STableBlockScanInfo* p return code; } - tsdbRowMerge(&merge, pRow); + pSchema = doGetSchemaForTSRow(TSDBROW_SVERSION(pRow), pReader, pBlockScanInfo->uid); + tsdbRowMergerAdd(&merge, pRow, pSchema); code = doMergeRowsInBuf(&pBlockScanInfo->iter, pBlockScanInfo->uid, k.ts, pBlockScanInfo->delSkyline, &merge, pReader); if (code != TSDB_CODE_SUCCESS) { diff --git a/source/util/src/tsimplehash.c b/source/util/src/tsimplehash.c index 310fdd27ab..ec1991923f 100644 --- a/source/util/src/tsimplehash.c +++ b/source/util/src/tsimplehash.c @@ -75,6 +75,7 @@ SSHashObj *tSimpleHashInit(size_t capacity, _hash_fn_t fn) { pHashObj->capacity = taosHashCapacity((int32_t)capacity); pHashObj->equalFp = memcmp; + pHashObj->freeFp = NULL; pHashObj->offset = 0; pHashObj->size = 0; From 94c6af39daf3dfac36be05b0b62042dcfb78e3ab Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 29 Mar 2023 10:46:44 +0800 Subject: [PATCH 59/65] fix(tmq): fix the invalid free --- source/dnode/mnode/impl/src/mndSubscribe.c | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndSubscribe.c b/source/dnode/mnode/impl/src/mndSubscribe.c index 739de68e5f..64a3170d47 100644 --- a/source/dnode/mnode/impl/src/mndSubscribe.c +++ b/source/dnode/mnode/impl/src/mndSubscribe.c @@ -596,13 +596,13 @@ static int32_t mndProcessRebalanceReq(SRpcMsg *pMsg) { SMnode *pMnode = pMsg->info.node; SMqDoRebalanceMsg *pReq = pMsg->pCont; void *pIter = NULL; - bool rebalanceExec = false; // to ensure only once. + bool rebalanceOnce = false; // to ensure only once. mInfo("mq re-balance start, total required re-balanced trans:%d", taosHashGetSize(pReq->rebSubHash)); // here we only handle one topic rebalance requirement to ensure the atomic execution of this transaction. while (1) { - if (rebalanceExec) { + if (rebalanceOnce) { break; } @@ -673,10 +673,6 @@ static int32_t mndProcessRebalanceReq(SRpcMsg *pMsg) { mError("mq re-balance persist output error, possibly vnode splitted or dropped"); } - taosArrayDestroy(pRebInfo->lostConsumers); - taosArrayDestroy(pRebInfo->newConsumers); - taosArrayDestroy(pRebInfo->removedConsumers); - taosArrayDestroy(rebOutput.newConsumers); taosArrayDestroy(rebOutput.touchedConsumers); taosArrayDestroy(rebOutput.removedConsumers); @@ -684,7 +680,7 @@ static int32_t mndProcessRebalanceReq(SRpcMsg *pMsg) { tDeleteSubscribeObj(rebOutput.pSub); taosMemoryFree(rebOutput.pSub); - rebalanceExec = true; + rebalanceOnce = true; } // reset flag From 4a085db970a7d332bdade580863fb1f77f45fa33 Mon Sep 17 00:00:00 2001 From: Xiaoyu Wang Date: Wed, 29 Mar 2023 11:10:15 +0800 Subject: [PATCH 60/65] fix: hotfix version compatibility --- source/client/src/clientMsgHandler.c | 4 ++-- source/dnode/mnode/impl/src/mndProfile.c | 2 +- source/util/src/tversion.c | 19 ++++++++++++------- 3 files changed, 15 insertions(+), 10 deletions(-) diff --git a/source/client/src/clientMsgHandler.c b/source/client/src/clientMsgHandler.c index 21590022b1..a0146cfa39 100644 --- a/source/client/src/clientMsgHandler.c +++ b/source/client/src/clientMsgHandler.c @@ -76,7 +76,7 @@ int32_t processConnectRsp(void* param, SDataBuf* pMsg, int32_t code) { goto End; } - if ((code = taosCheckVersionCompatibleFromStr(version, connectRsp.sVer, 2)) != 0) { + if ((code = taosCheckVersionCompatibleFromStr(version, connectRsp.sVer, 3)) != 0) { setErrno(pRequest, code); tsem_post(&pRequest->body.rspSem); goto End; @@ -506,7 +506,7 @@ int32_t processShowVariablesRsp(void* param, SDataBuf* pMsg, int32_t code) { code = setQueryResultFromRsp(&pRequest->body.resInfo, pRes, false, true); } - if(code != 0){ + if (code != 0) { taosMemoryFree(pRes); } tFreeSShowVariablesRsp(&rsp); diff --git a/source/dnode/mnode/impl/src/mndProfile.c b/source/dnode/mnode/impl/src/mndProfile.c index 41dea50731..50e502f4ab 100644 --- a/source/dnode/mnode/impl/src/mndProfile.c +++ b/source/dnode/mnode/impl/src/mndProfile.c @@ -226,7 +226,7 @@ static int32_t mndProcessConnectReq(SRpcMsg *pReq) { goto _OVER; } - if ((code = taosCheckVersionCompatibleFromStr(connReq.sVer, version, 2)) != 0) { + if ((code = taosCheckVersionCompatibleFromStr(connReq.sVer, version, 3)) != 0) { terrno = code; goto _OVER; } diff --git a/source/util/src/tversion.c b/source/util/src/tversion.c index dc357c61a1..0f84f6bafa 100644 --- a/source/util/src/tversion.c +++ b/source/util/src/tversion.c @@ -26,8 +26,9 @@ int32_t taosVersionStrToInt(const char *vstr, int32_t *vint) { int32_t vnum[4] = {0}; int32_t len = strlen(vstr); char tmp[16] = {0}; + int32_t vpos = 0; - for (int32_t spos = 0, tpos = 0, vpos = 0; spos < len && vpos < 4; ++spos) { + for (int32_t spos = 0, tpos = 0; spos < len && vpos < 4; ++spos) { if (vstr[spos] != '.') { tmp[spos - tpos] = vstr[spos]; } else { @@ -38,6 +39,10 @@ int32_t taosVersionStrToInt(const char *vstr, int32_t *vint) { } } + if ('\0' != tmp[0] && vpos < 4) { + vnum[vpos] = atoi(tmp); + } + if (vnum[0] <= 0) { terrno = TSDB_CODE_INVALID_VERSION_STRING; return -1; @@ -66,16 +71,16 @@ int32_t taosCheckVersionCompatible(int32_t clientVer, int32_t serverVer, int32_t case 4: break; case 3: - clientVer %= 100; - serverVer %= 100; + clientVer /= 100; + serverVer /= 100; break; case 2: - clientVer %= 10000; - serverVer %= 10000; + clientVer /= 10000; + serverVer /= 10000; break; case 1: - clientVer %= 1000000; - serverVer %= 1000000; + clientVer /= 1000000; + serverVer /= 1000000; break; default: terrno = TSDB_CODE_INVALID_VERSION_NUMBER; From 87b8b4c94697a046abb7343c3bb50ece20d087a9 Mon Sep 17 00:00:00 2001 From: Shuduo Sang Date: Wed, 29 Mar 2023 11:33:45 +0800 Subject: [PATCH 61/65] fix: telnet_tcp and sml_json_all_types for main (#20661) * fix: taosbenchmark support same min/max for main * fix: enable telnet_tcp and sml_json_all_types * fix: tests/develop-test/5-taos-tools/taosbenchmark/json/sml_telnet_tcp.json * fix: taosdump free tbname in loose mode for main --- cmake/taostools_CMakeLists.txt.in | 2 +- .../5-taos-tools/taosbenchmark/json/sml_telnet_tcp.json | 2 +- tests/parallel_test/cases.task | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/cmake/taostools_CMakeLists.txt.in b/cmake/taostools_CMakeLists.txt.in index 40fa48b815..3f7a43ab2d 100644 --- a/cmake/taostools_CMakeLists.txt.in +++ b/cmake/taostools_CMakeLists.txt.in @@ -2,7 +2,7 @@ # taos-tools ExternalProject_Add(taos-tools GIT_REPOSITORY https://github.com/taosdata/taos-tools.git - GIT_TAG 2864326 + GIT_TAG e82b9fc SOURCE_DIR "${TD_SOURCE_DIR}/tools/taos-tools" BINARY_DIR "" #BUILD_IN_SOURCE TRUE diff --git a/tests/develop-test/5-taos-tools/taosbenchmark/json/sml_telnet_tcp.json b/tests/develop-test/5-taos-tools/taosbenchmark/json/sml_telnet_tcp.json index 9e1241397f..e609fcfebd 100644 --- a/tests/develop-test/5-taos-tools/taosbenchmark/json/sml_telnet_tcp.json +++ b/tests/develop-test/5-taos-tools/taosbenchmark/json/sml_telnet_tcp.json @@ -16,7 +16,7 @@ "num_of_records_per_req": 10, "databases": [{ "dbinfo": { - "name": "db", + "name": "opentsdb_telnet", "drop": "yes" }, "super_tables": [{ diff --git a/tests/parallel_test/cases.task b/tests/parallel_test/cases.task index c2f8e75e79..4d6b5ff05e 100644 --- a/tests/parallel_test/cases.task +++ b/tests/parallel_test/cases.task @@ -1103,9 +1103,9 @@ ,,n,develop-test,python3 ./test.py -f 5-taos-tools/taosbenchmark/json_tag.py ,,n,develop-test,python3 ./test.py -f 5-taos-tools/taosbenchmark/query_json.py ,,n,develop-test,python3 ./test.py -f 5-taos-tools/taosbenchmark/sample_csv_json.py -#,,n,develop-test,python3 ./test.py -f 5-taos-tools/taosbenchmark/sml_json_alltypes.py +,,n,develop-test,python3 ./test.py -f 5-taos-tools/taosbenchmark/sml_json_alltypes.py ,,n,develop-test,python3 ./test.py -f 5-taos-tools/taosbenchmark/taosdemoTestQueryWithJson.py -R -#,,n,develop-test,python3 ./test.py -f 5-taos-tools/taosbenchmark/telnet_tcp.py -R +,,n,develop-test,python3 ./test.py -f 5-taos-tools/taosbenchmark/telnet_tcp.py -R #docs-examples test ,,n,docs-examples-test,bash python.sh From 8cf8ac84d71e06f674147f99e0ff03acc6c2ef9d Mon Sep 17 00:00:00 2001 From: Minglei Jin Date: Wed, 29 Mar 2023 14:19:15 +0800 Subject: [PATCH 62/65] fix(tsdb/read): use correct scheme for mem & imem merging --- source/dnode/vnode/src/tsdb/tsdbRead.c | 37 +++++++++++++------------- 1 file changed, 19 insertions(+), 18 deletions(-) diff --git a/source/dnode/vnode/src/tsdb/tsdbRead.c b/source/dnode/vnode/src/tsdb/tsdbRead.c index 3d8b2a4031..1906ae1abd 100644 --- a/source/dnode/vnode/src/tsdb/tsdbRead.c +++ b/source/dnode/vnode/src/tsdb/tsdbRead.c @@ -177,15 +177,15 @@ struct STsdbReader { SBlockLoadSuppInfo suppInfo; STsdbReadSnap* pReadSnap; SIOCostSummary cost; - STSchema* pSchema; // the newest version schema -// STSchema* pMemSchema; // the previous schema for in-memory data, to avoid load schema too many times - SSHashObj* pSchemaMap; // keep the retrieved schema info, to avoid the overhead by repeatly load schema - SDataFReader* pFileReader; // the file reader - SDelFReader* pDelFReader; // the del file reader - SArray* pDelIdx; // del file block index; - SBlockInfoBuf blockInfoBuf; - int32_t step; - STsdbReader* innerReader[2]; + STSchema* pSchema; // the newest version schema + // STSchema* pMemSchema; // the previous schema for in-memory data, to avoid load schema too many times + SSHashObj* pSchemaMap; // keep the retrieved schema info, to avoid the overhead by repeatly load schema + SDataFReader* pFileReader; // the file reader + SDelFReader* pDelFReader; // the del file reader + SArray* pDelIdx; // del file block index; + SBlockInfoBuf blockInfoBuf; + int32_t step; + STsdbReader* innerReader[2]; }; static SFileDataBlockInfo* getCurrentBlockInfo(SDataBlockIter* pBlockIter); @@ -1862,11 +1862,11 @@ static FORCE_INLINE STSchema* doGetSchemaForTSRow(int32_t sversion, STsdbReader* void** p = tSimpleHashGet(pReader->pSchemaMap, &sversion, sizeof(sversion)); if (p != NULL) { - return *(STSchema**) p; + return *(STSchema**)p; } STSchema* ptr = NULL; - int32_t code = metaGetTbTSchemaEx(pReader->pTsdb->pVnode->pMeta, pReader->suid, uid, sversion, &ptr); + int32_t code = metaGetTbTSchemaEx(pReader->pTsdb->pVnode->pMeta, pReader->suid, uid, sversion, &ptr); if (code != TSDB_CODE_SUCCESS) { terrno = code; return NULL; @@ -3674,8 +3674,9 @@ int32_t doMergeMemIMemRows(TSDBROW* pRow, TSDBROW* piRow, STableBlockScanInfo* p if (ASCENDING_TRAVERSE(pReader->order)) { // ascending order imem --> mem STSchema* pSchema = doGetSchemaForTSRow(TSDBROW_SVERSION(pRow), pReader, pBlockScanInfo->uid); + STSchema* piSchema = doGetSchemaForTSRow(TSDBROW_SVERSION(piRow), pReader, pBlockScanInfo->uid); - int32_t code = tsdbRowMergerInit(&merge, piRow, pSchema); + int32_t code = tsdbRowMergerInit2(&merge, pSchema, piRow, piSchema); if (code != TSDB_CODE_SUCCESS) { return code; } @@ -4000,7 +4001,7 @@ static int32_t doOpenReaderImpl(STsdbReader* pReader) { } static void freeSchemaFunc(void* param) { - void* p = *(void**) param; + void* p = *(void**)param; taosMemoryFree(p); } @@ -4082,7 +4083,7 @@ int32_t tsdbReaderOpen(SVnode* pVnode, SQueryTableDataCond* pCond, void* pTableL pReader->pSchemaMap = tSimpleHashInit(8, taosFastHash); if (pReader->pSchemaMap == NULL) { - tsdbError("failed init schema hash for reader", pReader->idStr); + tsdbError("failed init schema hash for reader %s", pReader->idStr); code = TSDB_CODE_OUT_OF_MEMORY; goto _err; } @@ -4710,7 +4711,7 @@ int32_t tsdbReaderReset(STsdbReader* pReader, SQueryTableDataCond* pCond) { return TSDB_CODE_SUCCESS; } - SReaderStatus* pStatus = &pReader->status; + SReaderStatus* pStatus = &pReader->status; SDataBlockIter* pBlockIter = &pStatus->blockIter; pReader->order = pCond->order; @@ -4731,9 +4732,9 @@ int32_t tsdbReaderReset(STsdbReader* pReader, SQueryTableDataCond* pCond) { resetDataBlockIterator(pBlockIter, pReader->order); resetTableListIndex(&pReader->status); - bool asc = ASCENDING_TRAVERSE(pReader->order); - int32_t step = asc? 1:-1; - int64_t ts = asc? pReader->window.skey - 1 : pReader->window.ekey + 1; + bool asc = ASCENDING_TRAVERSE(pReader->order); + int32_t step = asc ? 1 : -1; + int64_t ts = asc ? pReader->window.skey - 1 : pReader->window.ekey + 1; resetAllDataBlockScanInfo(pStatus->pTableMap, ts, step); int32_t code = 0; From f61732dca32c228f0167de961e80012b2a85ac10 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Chappyguoxy=E2=80=9D?= <“happy_guoxy@163.com”> Date: Wed, 29 Mar 2023 16:56:53 +0800 Subject: [PATCH 63/65] test: refine query cases --- tests/system-test/2-query/columnLenUpdated.py | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/tests/system-test/2-query/columnLenUpdated.py b/tests/system-test/2-query/columnLenUpdated.py index d6940fde8b..ea01cd623c 100644 --- a/tests/system-test/2-query/columnLenUpdated.py +++ b/tests/system-test/2-query/columnLenUpdated.py @@ -147,9 +147,9 @@ class TDTestCase: tdSql.checkData(1, 1, '55555') - tdSql.query("create table stb (ts timestamp, f1 int) tags (tg1 binary(2))") + tdSql.query("create table stb (ts timestamp, f1 int, f2 binary(2)) tags (tg1 binary(2))") tdSql.query("create table tb1 using stb tags('bb')") - tdSql.query("insert into tb1 values (now, 2)") + tdSql.query("insert into tb1 values (now, 2,'22')") tdSql.query("select count(*) from stb group by tg1") tdSql.checkData(0, 0, 1) @@ -163,13 +163,23 @@ class TDTestCase: if retCode != "TAOS_OK": tdLog.exit("taos -s fail") - keyDict['s'] = "\"insert into db1.tb2 values (now, 2)\"" + keyDict['s'] = "\"insert into db1.tb2 values (now, 2,'22')\"" + retCode = taos_command(buildPath, "s", keyDict['s'], "Insert OK", '') + if retCode != "TAOS_OK": + tdLog.exit("taos -s fail") + + keyDict['s'] = "\"alter table db1.stb modify column f2 binary(5) \"" + retCode = taos_command(buildPath, "s", keyDict['s'], "Query OK", '') + if retCode != "TAOS_OK": + tdLog.exit("taos -s fail") + + keyDict['s'] = "\"insert into db1.tb2 values (now, 3,'55555')\"" retCode = taos_command(buildPath, "s", keyDict['s'], "Insert OK", '') if retCode != "TAOS_OK": tdLog.exit("taos -s fail") tdSql.query("select count(*) from stb group by tg1") - tdSql.checkData(0, 0, 1) + tdSql.checkData(0, 0, 2) tdSql.checkData(1, 0, 1) From b25b6a8333c9280b4a616a4b1d4dae348667f304 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Chappyguoxy=E2=80=9D?= <“happy_guoxy@163.com”> Date: Wed, 29 Mar 2023 18:16:46 +0800 Subject: [PATCH 64/65] test: refine query cases --- tests/parallel_test/cases.task | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/parallel_test/cases.task b/tests/parallel_test/cases.task index 260d47032a..a50503e24b 100644 --- a/tests/parallel_test/cases.task +++ b/tests/parallel_test/cases.task @@ -14,8 +14,9 @@ ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/nestedQuery_math.py -Q 2 ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/nestedQuery_time.py -Q 2 ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/nestedQuery.py -Q 2 -,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/columnLenUpdated.py -Q 2 -,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/columnLenUpdated.py -Q 3 +#,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/columnLenUpdated.py +#,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/columnLenUpdated.py -Q 2 +#,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/columnLenUpdated.py -Q 3 ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/nestedQuery.py -Q 3 ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/nestedQuery_str.py -Q 3 ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/nestedQuery_math.py -Q 3 @@ -24,7 +25,7 @@ ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/nestedQuery_str.py -Q 4 ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/nestedQuery_math.py -Q 4 ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/nestedQuery_time.py -Q 4 -,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/columnLenUpdated.py -Q 4 +#,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/columnLenUpdated.py -Q 4 ,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/tmqShow.py ,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/tmqDropStb.py ,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/subscribeStb0.py From 3810e57c1542c4deb332d6c20cafda120a7c99ea Mon Sep 17 00:00:00 2001 From: dapan1121 Date: Wed, 29 Mar 2023 19:10:10 +0800 Subject: [PATCH 65/65] fix: tag copy issue --- source/libs/executor/src/executil.c | 4 ++++ tests/parallel_test/cases.task | 8 ++++---- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/source/libs/executor/src/executil.c b/source/libs/executor/src/executil.c index 953d614951..0081ffcb0b 100644 --- a/source/libs/executor/src/executil.c +++ b/source/libs/executor/src/executil.c @@ -571,6 +571,10 @@ int32_t getColInfoResultForGroupby(void* metaHandle, SNodeList* group, STableLis memcpy(pStart, data, len); pStart += len; } else if (IS_VAR_DATA_TYPE(pValue->info.type)) { + if (varDataTLen(data) > pValue->info.bytes) { + code = TSDB_CODE_TDB_INVALID_TABLE_SCHEMA_VER; + goto end; + } memcpy(pStart, data, varDataTLen(data)); pStart += varDataTLen(data); } else { diff --git a/tests/parallel_test/cases.task b/tests/parallel_test/cases.task index a50503e24b..ab7540ee59 100644 --- a/tests/parallel_test/cases.task +++ b/tests/parallel_test/cases.task @@ -14,9 +14,9 @@ ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/nestedQuery_math.py -Q 2 ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/nestedQuery_time.py -Q 2 ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/nestedQuery.py -Q 2 -#,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/columnLenUpdated.py -#,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/columnLenUpdated.py -Q 2 -#,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/columnLenUpdated.py -Q 3 +,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/columnLenUpdated.py +,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/columnLenUpdated.py -Q 2 +,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/columnLenUpdated.py -Q 3 ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/nestedQuery.py -Q 3 ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/nestedQuery_str.py -Q 3 ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/nestedQuery_math.py -Q 3 @@ -25,7 +25,7 @@ ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/nestedQuery_str.py -Q 4 ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/nestedQuery_math.py -Q 4 ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/nestedQuery_time.py -Q 4 -#,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/columnLenUpdated.py -Q 4 +,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/columnLenUpdated.py -Q 4 ,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/tmqShow.py ,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/tmqDropStb.py ,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/subscribeStb0.py