Merge remote-tracking branch 'origin/main' into fix/main_bugfix_wxy

This commit is contained in:
Xiaoyu Wang 2023-01-16 09:44:32 +08:00
commit aa98d1d3fd
36 changed files with 402 additions and 99 deletions

View File

@ -2,7 +2,7 @@
# taosadapter
ExternalProject_Add(taosadapter
GIT_REPOSITORY https://github.com/taosdata/taosadapter.git
GIT_TAG 69eee2e
GIT_TAG 213f8b3
SOURCE_DIR "${TD_SOURCE_DIR}/tools/taosadapter"
BINARY_DIR ""
#BUILD_IN_SOURCE TRUE

View File

@ -2,7 +2,7 @@
# taos-tools
ExternalProject_Add(taos-tools
GIT_REPOSITORY https://github.com/taosdata/taos-tools.git
GIT_TAG 5aa25e9
GIT_TAG f80dd7e
SOURCE_DIR "${TD_SOURCE_DIR}/tools/taos-tools"
BINARY_DIR ""
#BUILD_IN_SOURCE TRUE

View File

@ -39,7 +39,7 @@ typedef enum {
QUEUE_MAX,
} EQueueType;
typedef int32_t (*UpdateDnodeInfoFp)(void* pData, int32_t* dnodeId, int64_t* clusterId, char* fqdn, uint16_t* port);
typedef void (*UpdateDnodeInfoFp)(void* pData, int32_t* dnodeId, int64_t* clusterId, char* fqdn, uint16_t* port);
typedef int32_t (*PutToQueueFp)(void* pMgmt, EQueueType qtype, SRpcMsg* pMsg);
typedef int32_t (*GetQueueSizeFp)(void* pMgmt, int32_t vgId, EQueueType qtype);
typedef int32_t (*SendReqFp)(const SEpSet* pEpSet, SRpcMsg* pMsg);
@ -70,7 +70,8 @@ void tmsgSendRsp(SRpcMsg* pMsg);
void tmsgRegisterBrokenLinkArg(SRpcMsg* pMsg);
void tmsgReleaseHandle(SRpcHandleInfo* pHandle, int8_t type);
void tmsgReportStartup(const char* name, const char* desc);
int32_t tmsgUpdateDnodeInfo(int32_t* dnodeId, int64_t* clusterId, char* fqdn, uint16_t* port);
void tmsgUpdateDnodeInfo(int32_t* dnodeId, int64_t* clusterId, char* fqdn, uint16_t* port);
void tmsgUpdateDnodeEpSet(SEpSet* epset);
#ifdef __cplusplus
}

View File

@ -354,7 +354,8 @@ int32_t tDecodeSStreamTask(SDecoder* pDecoder, SStreamTask* pTask);
void tFreeSStreamTask(SStreamTask* pTask);
static FORCE_INLINE int32_t streamTaskInput(SStreamTask* pTask, SStreamQueueItem* pItem) {
if (pItem->type == STREAM_INPUT__DATA_SUBMIT) {
int8_t type = pItem->type;
if (type == STREAM_INPUT__DATA_SUBMIT) {
SStreamDataSubmit* pSubmitClone = streamSubmitRefClone((SStreamDataSubmit*)pItem);
if (pSubmitClone == NULL) {
qDebug("task %d %p submit enqueue failed since out of memory", pTask->taskId, pTask);
@ -365,19 +366,19 @@ static FORCE_INLINE int32_t streamTaskInput(SStreamTask* pTask, SStreamQueueItem
qDebug("task %d %p submit enqueue %p %p %p", pTask->taskId, pTask, pItem, pSubmitClone, pSubmitClone->data);
taosWriteQitem(pTask->inputQueue->queue, pSubmitClone);
// qStreamInput(pTask->exec.executor, pSubmitClone);
} else if (pItem->type == STREAM_INPUT__DATA_BLOCK || pItem->type == STREAM_INPUT__DATA_RETRIEVE ||
pItem->type == STREAM_INPUT__REF_DATA_BLOCK) {
} else if (type == STREAM_INPUT__DATA_BLOCK || type == STREAM_INPUT__DATA_RETRIEVE ||
type == STREAM_INPUT__REF_DATA_BLOCK) {
taosWriteQitem(pTask->inputQueue->queue, pItem);
// qStreamInput(pTask->exec.executor, pItem);
} else if (pItem->type == STREAM_INPUT__CHECKPOINT) {
} else if (type == STREAM_INPUT__CHECKPOINT) {
taosWriteQitem(pTask->inputQueue->queue, pItem);
// qStreamInput(pTask->exec.executor, pItem);
} else if (pItem->type == STREAM_INPUT__GET_RES) {
} else if (type == STREAM_INPUT__GET_RES) {
taosWriteQitem(pTask->inputQueue->queue, pItem);
// qStreamInput(pTask->exec.executor, pItem);
}
if (pItem->type != STREAM_INPUT__GET_RES && pItem->type != STREAM_INPUT__CHECKPOINT && pTask->triggerParam != 0) {
if (type != STREAM_INPUT__GET_RES && type != STREAM_INPUT__CHECKPOINT && pTask->triggerParam != 0) {
atomic_val_compare_exchange_8(&pTask->triggerStatus, TASK_TRIGGER_STATUS__INACTIVE, TASK_TRIGGER_STATUS__ACTIVE);
}

View File

@ -499,7 +499,7 @@ enum {
#define DEFAULT_PAGESIZE 4096
#define VNODE_TIMEOUT_SEC 60
#define MNODE_TIMEOUT_SEC 10
#define MNODE_TIMEOUT_SEC 60
#ifdef __cplusplus
}

View File

@ -137,7 +137,7 @@ static void vmGenerateVnodeCfg(SCreateVnodeReq *pCreate, SVnodeCfg *pCfg) {
pNode->nodeId = pCreate->replicas[i].id;
pNode->nodePort = pCreate->replicas[i].port;
tstrncpy(pNode->nodeFqdn, pCreate->replicas[i].fqdn, TSDB_FQDN_LEN);
(void)tmsgUpdateDnodeInfo(&pNode->nodeId, &pNode->clusterId, pNode->nodeFqdn, &pNode->nodePort);
tmsgUpdateDnodeInfo(&pNode->nodeId, &pNode->clusterId, pNode->nodeFqdn, &pNode->nodePort);
}
}

View File

@ -343,13 +343,12 @@ static void vmCheckSyncTimeout(SVnodeMgmt *pMgmt) {
int32_t numOfVnodes = 0;
SVnodeObj **ppVnodes = vmGetVnodeListFromHash(pMgmt, &numOfVnodes);
for (int32_t i = 0; i < numOfVnodes; ++i) {
SVnodeObj *pVnode = ppVnodes[i];
vnodeSyncCheckTimeout(pVnode->pImpl);
vmReleaseVnode(pMgmt, pVnode);
}
if (ppVnodes != NULL) {
for (int32_t i = 0; i < numOfVnodes; ++i) {
SVnodeObj *pVnode = ppVnodes[i];
vnodeSyncCheckTimeout(pVnode->pImpl);
vmReleaseVnode(pMgmt, pVnode);
}
taosMemoryFree(ppVnodes);
}
}

View File

@ -100,6 +100,7 @@ typedef struct {
bool stopped;
SEpSet mnodeEps;
SArray *dnodeEps;
SArray *oldDnodeEps;
SHashObj *dnodeHash;
TdThreadRwlock lock;
SMsgCb msgCb;
@ -167,7 +168,7 @@ void dmUpdateEps(SDnodeData *pData, SArray *pDnodeEps);
void dmGetMnodeEpSet(SDnodeData *pData, SEpSet *pEpSet);
void dmGetMnodeEpSetForRedirect(SDnodeData *pData, SRpcMsg *pMsg, SEpSet *pEpSet);
void dmSetMnodeEpSet(SDnodeData *pData, SEpSet *pEpSet);
int32_t dmUpdateDnodeInfo(void *pData, int32_t *dnodeId, int64_t *clusterId, char *fqdn, uint16_t *port);
void dmUpdateDnodeInfo(void *pData, int32_t *dnodeId, int64_t *clusterId, char *fqdn, uint16_t *port);
#ifdef __cplusplus
}

View File

@ -332,40 +332,48 @@ void dmSetMnodeEpSet(SDnodeData *pData, SEpSet *pEpSet) {
}
}
int32_t dmUpdateDnodeInfo(void *data, int32_t *dnodeId, int64_t *clusterId, char *fqdn, uint16_t *port) {
void dmUpdateDnodeInfo(void *data, int32_t *did, int64_t *clusterId, char *fqdn, uint16_t *port) {
SDnodeData *pData = data;
int32_t ret = -1;
int32_t dnodeId = -1;
if (did != NULL) dnodeId = *did;
taosThreadRwlockRdlock(&pData->lock);
if (*dnodeId <= 0) {
for (int32_t i = 0; i < (int32_t)taosArrayGetSize(pData->dnodeEps); ++i) {
SDnodeEp *pDnodeEp = taosArrayGet(pData->dnodeEps, i);
if (pData->oldDnodeEps != NULL) {
int32_t size = (int32_t)taosArrayGetSize(pData->oldDnodeEps);
for (int32_t i = 0; i < size; ++i) {
SDnodeEp *pDnodeEp = taosArrayGet(pData->oldDnodeEps, i);
if (strcmp(pDnodeEp->ep.fqdn, fqdn) == 0 && pDnodeEp->ep.port == *port) {
dInfo("dnode:%s:%u, update dnodeId from %d to %d", fqdn, *port, *dnodeId, pDnodeEp->id);
*dnodeId = pDnodeEp->id;
*clusterId = pData->clusterId;
ret = 0;
}
}
if (ret != 0) {
dInfo("dnode:%s:%u, failed to update dnodeId:%d", fqdn, *port, *dnodeId);
}
} else {
SDnodeEp *pDnodeEp = taosHashGet(pData->dnodeHash, dnodeId, sizeof(int32_t));
if (pDnodeEp) {
if (strcmp(pDnodeEp->ep.fqdn, fqdn) != 0) {
dInfo("dnode:%d, update port from %s to %s", *dnodeId, fqdn, pDnodeEp->ep.fqdn);
dInfo("dnode:%d, update ep:%s:%u to %s:%u", dnodeId, fqdn, *port, pDnodeEp->ep.fqdn, pDnodeEp->ep.port);
tstrncpy(fqdn, pDnodeEp->ep.fqdn, TSDB_FQDN_LEN);
}
if (pDnodeEp->ep.port != *port) {
dInfo("dnode:%d, update port from %u to %u", *dnodeId, *port, pDnodeEp->ep.port);
*port = pDnodeEp->ep.port;
}
*clusterId = pData->clusterId;
ret = 0;
} else {
dInfo("dnode:%d, failed to update dnode info", *dnodeId);
}
}
if (did != NULL && dnodeId <= 0) {
int32_t size = (int32_t)taosArrayGetSize(pData->dnodeEps);
for (int32_t i = 0; i < size; ++i) {
SDnodeEp *pDnodeEp = taosArrayGet(pData->dnodeEps, i);
if (strcmp(pDnodeEp->ep.fqdn, fqdn) == 0 && pDnodeEp->ep.port == *port) {
dInfo("dnode:%s:%u, update dnodeId to dnode:%d", fqdn, *port, pDnodeEp->id);
*did = pDnodeEp->id;
if (clusterId != NULL) *clusterId = pData->clusterId;
}
}
}
if (dnodeId > 0) {
SDnodeEp *pDnodeEp = taosHashGet(pData->dnodeHash, &dnodeId, sizeof(int32_t));
if (pDnodeEp) {
if (strcmp(pDnodeEp->ep.fqdn, fqdn) != 0 || pDnodeEp->ep.port != *port) {
dInfo("dnode:%d, update ep:%s:%u to %s:%u", dnodeId, fqdn, *port, pDnodeEp->ep.fqdn, pDnodeEp->ep.port);
tstrncpy(fqdn, pDnodeEp->ep.fqdn, TSDB_FQDN_LEN);
*port = pDnodeEp->ep.port;
}
if (clusterId != NULL) *clusterId = pData->clusterId;
}
}
taosThreadRwlockUnlock(&pData->lock);
return ret;
}

View File

@ -742,6 +742,7 @@ SSdbRow *mndConsumerActionDecode(SSdbRaw *pRaw) {
if (tDecodeSMqConsumerObj(buf, pConsumer) == NULL) {
goto CM_DECODE_OVER;
}
tmsgUpdateDnodeEpSet(&pConsumer->ep);
terrno = TSDB_CODE_SUCCESS;

View File

@ -180,6 +180,7 @@ static SSdbRow *mndDnodeActionDecode(SSdbRaw *pRaw) {
SDB_GET_RESERVE(pRaw, dataPos, TSDB_DNODE_RESERVE_SIZE, _OVER)
terrno = 0;
tmsgUpdateDnodeInfo(&pDnode->id, NULL, pDnode->fqdn, &pDnode->port);
_OVER:
if (terrno != 0) {

View File

@ -293,7 +293,7 @@ static int32_t mndProcessCreateFuncReq(SRpcMsg *pReq) {
goto _OVER;
}
mInfo("func:%s, start to create", createReq.name);
mInfo("func:%s, start to create, size:%d", createReq.name, createReq.codeLen);
if (mndCheckOperPrivilege(pMnode, pReq->info.conn.user, MND_OPER_CREATE_FUNC) != 0) {
goto _OVER;
}

View File

@ -15,13 +15,13 @@
#define _DEFAULT_SOURCE
#include "mndMnode.h"
#include "mndCluster.h"
#include "mndDnode.h"
#include "mndPrivilege.h"
#include "mndShow.h"
#include "mndSync.h"
#include "mndTrans.h"
#include "tmisce.h"
#include "mndCluster.h"
#define MNODE_VER_NUMBER 1
#define MNODE_RESERVE_SIZE 64
@ -181,9 +181,8 @@ _OVER:
static int32_t mndMnodeActionInsert(SSdb *pSdb, SMnodeObj *pObj) {
mTrace("mnode:%d, perform insert action, row:%p", pObj->id, pObj);
pObj->pDnode = sdbAcquire(pSdb, SDB_DNODE, &pObj->id);
pObj->pDnode = sdbAcquireNotReadyObj(pSdb, SDB_DNODE, &pObj->id);
if (pObj->pDnode == NULL) {
terrno = TSDB_CODE_MND_DNODE_NOT_EXIST;
mError("mnode:%d, failed to perform insert action since %s", pObj->id, terrstr());
return -1;
}
@ -748,7 +747,7 @@ static void mndReloadSyncConfig(SMnode *pMnode) {
pNode->clusterId = mndGetClusterId(pMnode);
pNode->nodePort = pObj->pDnode->port;
tstrncpy(pNode->nodeFqdn, pObj->pDnode->fqdn, TSDB_FQDN_LEN);
(void)tmsgUpdateDnodeInfo(&pNode->nodeId, &pNode->clusterId, pNode->nodeFqdn, &pNode->nodePort);
tmsgUpdateDnodeInfo(&pNode->nodeId, &pNode->clusterId, pNode->nodeFqdn, &pNode->nodePort);
mInfo("vgId:1, ep:%s:%u dnode:%d", pNode->nodeFqdn, pNode->nodePort, pNode->nodeId);
if (pObj->pDnode->id == pMnode->selfDnodeId) {
cfg.myIndex = cfg.replicaNum;

View File

@ -202,11 +202,13 @@ static SSdbRow *mndSmaActionDecode(SSdbRaw *pRaw) {
_OVER:
if (terrno != 0) {
mError("sma:%s, failed to decode from raw:%p since %s", pSma == NULL ? "null" : pSma->name, pRaw, terrstr());
taosMemoryFreeClear(pSma->expr);
taosMemoryFreeClear(pSma->tagsFilter);
taosMemoryFreeClear(pSma->sql);
taosMemoryFreeClear(pSma->ast);
if (pSma != NULL) {
mError("sma:%s, failed to decode from raw:%p since %s", pSma->name, pRaw, terrstr());
taosMemoryFreeClear(pSma->expr);
taosMemoryFreeClear(pSma->tagsFilter);
taosMemoryFreeClear(pSma->sql);
taosMemoryFreeClear(pSma->ast);
}
taosMemoryFreeClear(pRow);
return NULL;
}

View File

@ -760,6 +760,27 @@ static SSdbRow *mndSubActionDecode(SSdbRaw *pRaw) {
goto SUB_DECODE_OVER;
}
// update epset saved in mnode
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);
tmsgUpdateDnodeEpSet(&pMqVgEp->epSet);
}
}
if (pSub->consumerHash != NULL) {
void *pIter = taosHashIterate(pSub->consumerHash, NULL);
while (pIter) {
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);
tmsgUpdateDnodeEpSet(&pMqVgEp->epSet);
}
pIter = taosHashIterate(pSub->consumerHash, pIter);
}
}
terrno = TSDB_CODE_SUCCESS;
SUB_DECODE_OVER:

View File

@ -271,9 +271,11 @@ SSyncFSM *mndSyncMakeFsm(SMnode *pMnode) {
int32_t mndInitSync(SMnode *pMnode) {
SSyncMgmt *pMgmt = &pMnode->syncMgmt;
taosThreadMutexInit(&pMgmt->lock, NULL);
taosThreadMutexLock(&pMgmt->lock);
pMgmt->transId = 0;
pMgmt->transSec = 0;
pMgmt->transSeq = 0;
taosThreadMutexUnlock(&pMgmt->lock);
SSyncInfo syncInfo = {
.snapshotStrategy = SYNC_STRATEGY_STANDARD_SNAPSHOT,
@ -301,7 +303,7 @@ int32_t mndInitSync(SMnode *pMnode) {
pNode->nodeId = pMgmt->replicas[i].id;
pNode->nodePort = pMgmt->replicas[i].port;
tstrncpy(pNode->nodeFqdn, pMgmt->replicas[i].fqdn, sizeof(pNode->nodeFqdn));
(void)tmsgUpdateDnodeInfo(&pNode->nodeId, &pNode->clusterId, pNode->nodeFqdn, &pNode->nodePort);
tmsgUpdateDnodeInfo(&pNode->nodeId, &pNode->clusterId, pNode->nodeFqdn, &pNode->nodePort);
mInfo("vgId:1, index:%d ep:%s:%u dnode:%d cluster:%" PRId64, i, pNode->nodeFqdn, pNode->nodePort, pNode->nodeId,
pNode->clusterId);
}
@ -369,6 +371,7 @@ int32_t mndSyncPropose(SMnode *pMnode, SSdbRaw *pRaw, int32_t transId) {
if (pMgmt->transId != 0) {
mError("trans:%d, can't be proposed since trans:%d already waiting for confirm", transId, pMgmt->transId);
taosThreadMutexUnlock(&pMgmt->lock);
rpcFreeCont(req.pCont);
terrno = TSDB_CODE_MND_LAST_TRANS_NOT_FINISHED;
return terrno;
}

View File

@ -329,6 +329,7 @@ static SSdbRow *mndTransActionDecode(SSdbRaw *pRaw) {
action.pRaw = NULL;
} else if (action.actionType == TRANS_ACTION_MSG) {
SDB_GET_BINARY(pRaw, dataPos, (void *)&action.epSet, sizeof(SEpSet), _OVER);
tmsgUpdateDnodeEpSet(&action.epSet);
SDB_GET_INT16(pRaw, dataPos, &action.msgType, _OVER)
SDB_GET_INT8(pRaw, dataPos, &unused /*&action.msgSent*/, _OVER)
SDB_GET_INT8(pRaw, dataPos, &unused /*&action.msgReceived*/, _OVER)

View File

@ -1441,10 +1441,10 @@ static int32_t mndRedistributeVgroup(SMnode *pMnode, SRpcMsg *pReq, SDbObj *pDb,
{
SSdbRaw *pRaw = mndVgroupActionEncode(&newVg);
if (pRaw == NULL) return -1;
if (pRaw == NULL) goto _OVER;
if (mndTransAppendCommitlog(pTrans, pRaw) != 0) {
sdbFreeRaw(pRaw);
return -1;
goto _OVER;
}
(void)sdbSetRawStatus(pRaw, SDB_STATUS_READY);
}

View File

@ -291,6 +291,7 @@ int32_t sdbWriteWithoutFree(SSdb *pSdb, SSdbRaw *pRaw);
* @return void* The object of the row.
*/
void *sdbAcquire(SSdb *pSdb, ESdbType type, const void *pKey);
void *sdbAcquireNotReadyObj(SSdb *pSdb, ESdbType type, const void *pKey);
/**
* @brief Release a row from sdb.

View File

@ -228,11 +228,12 @@ static int32_t sdbReadFileImp(SSdb *pSdb) {
int32_t readLen = 0;
int64_t ret = 0;
char file[PATH_MAX] = {0};
int32_t bufLen = TSDB_MAX_MSG_SIZE;
snprintf(file, sizeof(file), "%s%ssdb.data", pSdb->currDir, TD_DIRSEP);
mInfo("start to read sdb file:%s", file);
SSdbRaw *pRaw = taosMemoryMalloc(TSDB_MAX_MSG_SIZE + 100);
SSdbRaw *pRaw = taosMemoryMalloc(bufLen + 100);
if (pRaw == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY;
mError("failed read sdb file since %s", terrstr());
@ -275,14 +276,15 @@ static int32_t sdbReadFileImp(SSdb *pSdb) {
}
readLen = pRaw->dataLen + sizeof(int32_t);
if (readLen >= pRaw->dataLen) {
SSdbRaw *pNewRaw = taosMemoryMalloc(pRaw->dataLen + TSDB_MAX_MSG_SIZE);
if (readLen >= bufLen) {
bufLen = pRaw->dataLen * 2;
SSdbRaw *pNewRaw = taosMemoryMalloc(bufLen + 100);
if (pNewRaw == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY;
mError("failed read sdb file since malloc new sdbRaw size:%d failed", pRaw->dataLen + TSDB_MAX_MSG_SIZE);
mError("failed read sdb file since malloc new sdbRaw size:%d failed", bufLen);
goto _OVER;
}
mInfo("malloc new sdbRaw size:%d, type:%d", pRaw->dataLen + TSDB_MAX_MSG_SIZE, pRaw->type);
mInfo("malloc new sdb raw size:%d, type:%d", bufLen, pRaw->type);
memcpy(pNewRaw, pRaw, sizeof(SSdbRaw));
sdbFreeRaw(pRaw);
pRaw = pNewRaw;

View File

@ -270,7 +270,7 @@ int32_t sdbWrite(SSdb *pSdb, SSdbRaw *pRaw) {
return code;
}
void *sdbAcquire(SSdb *pSdb, ESdbType type, const void *pKey) {
void *sdbAcquireAll(SSdb *pSdb, ESdbType type, const void *pKey, bool onlyReady) {
terrno = 0;
SHashObj *hash = sdbGetHash(pSdb, type);
@ -306,10 +306,24 @@ void *sdbAcquire(SSdb *pSdb, ESdbType type, const void *pKey) {
break;
}
if (pRet == NULL) {
if (!onlyReady) {
terrno = 0;
atomic_add_fetch_32(&pRow->refCount, 1);
pRet = pRow->pObj;
sdbPrintOper(pSdb, pRow, "acquire");
}
}
sdbUnLock(pSdb, type);
return pRet;
}
void *sdbAcquire(SSdb *pSdb, ESdbType type, const void *pKey) { return sdbAcquireAll(pSdb, type, pKey, true); }
void *sdbAcquireNotReadyObj(SSdb *pSdb, ESdbType type, const void *pKey) {
return sdbAcquireAll(pSdb, type, pKey, false);
}
static void sdbCheckRow(SSdb *pSdb, SSdbRow *pRow) {
int32_t type = pRow->type;
sdbWriteLock(pSdb, type);

View File

@ -86,7 +86,7 @@ int32_t vnodeAlter(const char *path, SAlterVnodeReplicaReq *pReq, STfs *pTfs) {
pNode->nodeId = pReq->replicas[i].id;
pNode->nodePort = pReq->replicas[i].port;
tstrncpy(pNode->nodeFqdn, pReq->replicas[i].fqdn, sizeof(pNode->nodeFqdn));
(void)tmsgUpdateDnodeInfo(&pNode->nodeId, &pNode->clusterId, pNode->nodeFqdn, &pNode->nodePort);
tmsgUpdateDnodeInfo(&pNode->nodeId, &pNode->clusterId, pNode->nodeFqdn, &pNode->nodePort);
vInfo("vgId:%d, replica:%d ep:%s:%u dnode:%d", pReq->vgId, i, pNode->nodeFqdn, pNode->nodePort, pNode->nodeId);
}

View File

@ -3222,7 +3222,9 @@ static void buildVnodeGroupedNtbTableCount(STableCountScanOperatorInfo* pInfo, S
uint64_t groupId = calcGroupId(fullStbName, strlen(fullStbName));
pRes->info.id.groupId = groupId;
int64_t ntbNum = metaGetNtbNum(pInfo->readHandle.meta);
fillTableCountScanDataBlock(pSupp, dbName, "", ntbNum, pRes);
if (ntbNum != 0) {
fillTableCountScanDataBlock(pSupp, dbName, "", ntbNum, pRes);
}
}
static void buildVnodeGroupedStbTableCount(STableCountScanOperatorInfo* pInfo, STableCountScanSupp* pSupp,

View File

@ -898,7 +898,7 @@ SSyncNode* syncNodeOpen(SSyncInfo* pSyncInfo) {
sInfo("vgId:%d, start to open sync node, replica:%d selfIndex:%d", pSyncNode->vgId, pCfg->replicaNum, pCfg->myIndex);
for (int32_t i = 0; i < pCfg->replicaNum; ++i) {
SNodeInfo* pNode = &pCfg->nodeInfo[i];
(void)tmsgUpdateDnodeInfo(&pNode->nodeId, &pNode->clusterId, pNode->nodeFqdn, &pNode->nodePort);
tmsgUpdateDnodeInfo(&pNode->nodeId, &pNode->clusterId, pNode->nodeFqdn, &pNode->nodePort);
sInfo("vgId:%d, index:%d ep:%s:%u dnode:%d cluster:%" PRId64, pSyncNode->vgId, i, pNode->nodeFqdn, pNode->nodePort,
pNode->nodeId, pNode->clusterId);
}
@ -1703,8 +1703,7 @@ void syncNodeDoConfigChange(SSyncNode* pSyncNode, SSyncCfg* pNewConfig, SyncInde
_END:
// log end config change
sNInfo(pSyncNode, "end do config change, from %d to %d", pSyncNode->vgId, oldConfig.replicaNum,
pNewConfig->replicaNum);
sNInfo(pSyncNode, "end do config change, from %d to %d", oldConfig.replicaNum, pNewConfig->replicaNum);
}
// raft state change --------------

View File

@ -59,6 +59,12 @@ void tmsgReleaseHandle(SRpcHandleInfo* pHandle, int8_t type) { (*defaultMsgCb.re
void tmsgReportStartup(const char* name, const char* desc) { (*defaultMsgCb.reportStartupFp)(name, desc); }
int32_t tmsgUpdateDnodeInfo(int32_t* dnodeId, int64_t* clusterId, char* fqdn, uint16_t* port) {
return (*defaultMsgCb.updateDnodeInfoFp)(defaultMsgCb.data, dnodeId, clusterId, fqdn, port);
void tmsgUpdateDnodeInfo(int32_t* dnodeId, int64_t* clusterId, char* fqdn, uint16_t* port) {
(*defaultMsgCb.updateDnodeInfoFp)(defaultMsgCb.data, dnodeId, clusterId, fqdn, port);
}
void tmsgUpdateDnodeEpSet(SEpSet* epset) {
for (int32_t i = 0; i < epset->numOfEps; ++i) {
tmsgUpdateDnodeInfo(NULL, NULL, epset->eps[i].fqdn, &epset->eps[i].port);
}
}

View File

@ -227,6 +227,7 @@ STaosQueue *tAutoQWorkerAllocQueue(SAutoQWorkerPool *pool, void *ahandle, FItem
uError("worker:%s:%d failed to create", pool->name, curWorkerNum);
taosMemoryFree(worker);
taosCloseQueue(queue);
taosThreadMutexUnlock(&pool->mutex);
terrno = TSDB_CODE_OUT_OF_MEMORY;
return NULL;
}

View File

@ -0,0 +1,238 @@
import sys
from util.log import *
from util.cases import *
from util.sql import *
from util.dnodes import tdDnodes
from math import inf
class TDTestCase:
def caseDescription(self):
'''
case1<shenglian zhou>: [TD-21890] table count scan test case
'''
return
def init(self, conn, logSql, replicaVer=1):
tdLog.debug("start to execute %s" % __file__)
tdSql.init(conn.cursor(), False)
self._conn = conn
def restartTaosd(self, index=1, dbname="db"):
tdDnodes.stop(index)
tdDnodes.startWithoutSleep(index)
tdSql.execute(f"use tbl_count")
def run(self):
print("running {}".format(__file__))
tdSql.execute("drop database if exists tbl_count")
tdSql.execute("create database if not exists tbl_count")
tdSql.execute('use tbl_count')
tdSql.execute('create table stb1 (ts timestamp, c1 bool, c2 tinyint, c3 smallint, c4 int, c5 bigint, c6 float, c7 double, c8 binary(10), c9 nchar(10), c10 tinyint unsigned, c11 smallint unsigned, c12 int unsigned, c13 bigint unsigned) TAGS(t1 int, t2 binary(10), t3 double);')
tdSql.execute("create table tb1 using stb1 tags(1,'1',1.0);")
tdSql.execute("create table tb2 using stb1 tags(2,'2',2.0);")
tdSql.execute("create table tb3 using stb1 tags(3,'3',3.0);")
tdSql.execute('insert into tb1 values (\'2021-11-11 09:00:00\',true,1,1,1,1,1,1,"123","1234",1,1,1,1);')
tdSql.execute("insert into tb1 values ('2021-11-11 09:00:01',true,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL);")
tdSql.execute('insert into tb1 values (\'2021-11-11 09:00:02\',true,2,NULL,2,NULL,2,NULL,"234",NULL,2,NULL,2,NULL);')
tdSql.execute('insert into tb1 values (\'2021-11-11 09:00:03\',false,NULL,3,NULL,3,NULL,3,NULL,"3456",NULL,3,NULL,3);')
tdSql.execute('insert into tb1 values (\'2021-11-11 09:00:04\',true,4,4,4,4,4,4,"456","4567",4,4,4,4);')
tdSql.execute('insert into tb1 values (\'2021-11-11 09:00:05\',true,127,32767,2147483647,9223372036854775807,3.402823466e+38,1.79769e+308,"567","5678",254,65534,4294967294,9223372036854775807);')
tdSql.execute('insert into tb1 values (\'2021-11-11 09:00:06\',true,-127,-32767,-2147483647,-9223372036854775807,-3.402823466e+38,-1.79769e+308,"678","6789",0,0,0,0);')
tdSql.execute('insert into tb2 values (\'2021-11-11 09:00:00\',true,1,1,1,1,1,1,"111","1111",1,1,1,1);')
tdSql.execute('insert into tb2 values (\'2021-11-11 09:00:01\',true,2,2,2,2,2,2,"222","2222",2,2,2,2);')
tdSql.execute('insert into tb2 values (\'2021-11-11 09:00:02\',true,3,3,2,3,3,3,"333","3333",3,3,3,3);')
tdSql.execute('insert into tb2 values (\'2021-11-11 09:00:03\',false,4,4,4,4,4,4,"444","4444",4,4,4,4);')
tdSql.execute('insert into tb2 values (\'2021-11-11 09:00:04\',true,5,5,5,5,5,5,"555","5555",5,5,5,5);')
tdSql.execute('insert into tb2 values (\'2021-11-11 09:00:05\',true,6,6,6,6,6,6,"666","6666",6,6,6,6);')
tdSql.execute('insert into tb2 values (\'2021-11-11 09:00:06\',true,7,7,7,7,7,7,"777","7777",7,7,7,7);')
tdSql.query('select count(*),db_name, stable_name from information_schema.ins_tables group by db_name, stable_name;')
tdSql.checkRows(3)
tdSql.checkData(0, 0, 23)
tdSql.checkData(0, 1, 'information_schema')
tdSql.checkData(0, 2, None)
tdSql.checkData(1, 0, 3)
tdSql.checkData(1, 1, 'tbl_count')
tdSql.checkData(1, 2, 'stb1')
tdSql.checkData(2, 0, 5)
tdSql.checkData(2, 1, 'performance_schema')
tdSql.checkData(2, 2, None)
tdSql.query('select count(1),db_name, stable_name from information_schema.ins_tables group by db_name, stable_name;')
tdSql.checkRows(3)
tdSql.checkData(0, 0, 23)
tdSql.checkData(0, 1, 'information_schema')
tdSql.checkData(0, 2, None)
tdSql.checkData(1, 0, 5)
tdSql.checkData(1, 1, 'performance_schema')
tdSql.checkData(1, 2, None)
tdSql.checkData(2, 0, 3)
tdSql.checkData(2, 1, 'tbl_count')
tdSql.checkData(2, 2, 'stb1')
tdSql.query('select count(1),db_name from information_schema.ins_tables group by db_name')
tdSql.checkRows(3)
tdSql.checkData(0, 0, 5)
tdSql.checkData(0, 1, 'performance_schema')
tdSql.checkData(1, 0, 3)
tdSql.checkData(1, 1, 'tbl_count')
tdSql.checkData(2, 0, 23)
tdSql.checkData(2, 1, 'information_schema')
tdSql.query("select count(*) from information_schema.ins_tables where db_name='tbl_count'")
tdSql.checkRows(1)
tdSql.checkData(0, 0, 3)
tdSql.query('select count(*) from information_schema.ins_tables where db_name=\'tbl_count\' and stable_name="stb1";')
tdSql.checkRows(1)
tdSql.checkData(0, 0, 3)
tdSql.query('select count(*) from information_schema.ins_tables')
tdSql.checkRows(1)
tdSql.checkData(0, 0, 31)
tdSql.execute('create table stba (ts timestamp, c1 bool, c2 tinyint, c3 smallint, c4 int, c5 bigint, c6 float, c7 double, c8 binary(10), c9 nchar(10), c10 tinyint unsigned, c11 smallint unsigned, c12 int unsigned, c13 bigint unsigned) TAGS(t1 int, t2 binary(10), t3 double);')
tdSql.execute("create table tba1 using stba tags(1,'1',1.0);")
tdSql.execute('insert into tba1 values (\'2021-11-11 09:00:00\',true, 1,1,1,1,1,1,"111","1111",1,1,1,1);')
tdSql.execute('insert into tba1 values (\'2021-11-11 09:00:01\',true, 2,2,2,2,2,2,"222","2222",2,2,2,2);')
tdSql.execute('insert into tba1 values (\'2021-11-11 09:00:02\',true, 3,3,2,3,3,3,"333","3333",3,3,3,3);')
tdSql.execute('insert into tba1 values (\'2021-11-11 09:00:03\',false,4,4,4,4,4,4,"444","4444",4,4,4,4);')
tdSql.execute('insert into tba1 values (\'2021-11-11 09:00:04\',true, 5,5,5,5,5,5,"555","5555",5,5,5,5);')
tdSql.execute('insert into tba1 values (\'2021-11-11 09:00:05\',true, 6,6,6,6,6,6,"666","6666",6,6,6,6);')
tdSql.execute('insert into tba1 values (\'2021-11-11 09:00:06\',true, 7,7,7,7,7,7,"777","7777",7,7,7,7);')
tdSql.execute('insert into tba1 values (\'2021-11-11 09:00:07\',true, 8,8,8,8,8,8,"888","8888",8,8,8,8);')
tdSql.execute('insert into tba1 values (\'2021-11-11 09:00:08\',true, 9,9,9,9,9,9,"999","9999",9,9,9,9);')
tdSql.execute('insert into tba1 values (\'2021-11-11 09:00:09\',true, 0,0,0,0,0,0,"000","0000",0,0,0,0);')
self.restartTaosd(1, dbname='tbl_count')
tdSql.execute('insert into tba1 values (\'2021-11-11 09:00:10\',true, 1,1,1,1,1,1,"111","1111",1,1,1,1);')
tdSql.execute('insert into tba1 values (\'2021-11-11 09:00:11\',true, 2,2,2,2,2,2,"222","2222",2,2,2,2);')
tdSql.execute('insert into tba1 values (\'2021-11-11 09:00:12\',true, 3,3,2,3,3,3,"333","3333",3,3,3,3);')
tdSql.execute('insert into tba1 values (\'2021-11-11 09:00:13\',false,4,4,4,4,4,4,"444","4444",4,4,4,4);')
tdSql.execute('insert into tba1 values (\'2021-11-11 09:00:14\',true, 5,5,5,5,5,5,"555","5555",5,5,5,5);')
tdSql.execute('insert into tba1 values (\'2021-11-11 09:00:15\',true, 6,6,6,6,6,6,"666","6666",6,6,6,6);')
tdSql.execute('insert into tba1 values (\'2021-11-11 09:00:16\',true, 7,7,7,7,7,7,"777","7777",7,7,7,7);')
tdSql.execute('insert into tba1 values (\'2021-11-11 09:00:17\',true, 8,8,8,8,8,8,"888","8888",8,8,8,8);')
tdSql.execute('insert into tba1 values (\'2021-11-11 09:00:18\',true, 9,9,9,9,9,9,"999","9999",9,9,9,9);')
tdSql.execute('insert into tba1 values (\'2021-11-11 09:00:19\',true, 0,0,0,0,0,0,"000","0000",0,0,0,0);')
self.restartTaosd(1, dbname='tbl_count')
tdSql.execute('insert into tba1 values (\'2021-11-11 09:00:20\',true, 1,1,1,1,1,1,"111","1111",1,1,1,1);')
tdSql.execute('insert into tba1 values (\'2021-11-11 09:00:21\',true, 2,2,2,2,2,2,"222","2222",2,2,2,2);')
tdSql.execute('insert into tba1 values (\'2021-11-11 09:00:22\',true, 3,3,2,3,3,3,"333","3333",3,3,3,3);')
tdSql.execute('insert into tba1 values (\'2021-11-11 09:00:23\',false,4,4,4,4,4,4,"444","4444",4,4,4,4);')
tdSql.execute('insert into tba1 values (\'2021-11-11 09:00:24\',true, 5,5,5,5,5,5,"555","5555",5,5,5,5);')
tdSql.execute('insert into tba1 values (\'2021-11-11 09:00:25\',true, 6,6,6,6,6,6,"666","6666",6,6,6,6);')
tdSql.execute('insert into tba1 values (\'2021-11-11 09:00:26\',true, 7,7,7,7,7,7,"777","7777",7,7,7,7);')
tdSql.execute('insert into tba1 values (\'2021-11-11 09:00:27\',true, 8,8,8,8,8,8,"888","8888",8,8,8,8);')
tdSql.execute('insert into tba1 values (\'2021-11-11 09:00:28\',true, 9,9,9,9,9,9,"999","9999",9,9,9,9);')
tdSql.execute('insert into tba1 values (\'2021-11-11 09:00:29\',true, 0,0,0,0,0,0,"000","0000",0,0,0,0);')
tdSql.query('select count(*),db_name, stable_name from information_schema.ins_tables group by db_name, stable_name;')
tdSql.checkRows(4)
tdSql.checkData(0, 0, 1)
tdSql.checkData(0, 1, 'tbl_count')
tdSql.checkData(0, 2, 'stba')
tdSql.checkData(1, 0, 23)
tdSql.checkData(1, 1, 'information_schema')
tdSql.checkData(1, 2, None)
tdSql.checkData(2, 0, 3)
tdSql.checkData(2, 1, 'tbl_count')
tdSql.checkData(2, 2, 'stb1')
tdSql.checkData(3, 0, 5)
tdSql.checkData(3, 1, 'performance_schema')
tdSql.checkData(3, 2, None)
tdSql.query('select count(1),db_name, stable_name from information_schema.ins_tables group by db_name, stable_name;')
tdSql.checkRows(4)
tdSql.checkData(0, 0, 23)
tdSql.checkData(0, 1, 'information_schema')
tdSql.checkData(0, 2, None)
tdSql.checkData(1, 0, 5)
tdSql.checkData(1, 1, 'performance_schema')
tdSql.checkData(1, 2, None)
tdSql.checkData(2, 0, 1)
tdSql.checkData(2, 1, 'tbl_count')
tdSql.checkData(2, 2, 'stba')
tdSql.checkData(3, 0, 3)
tdSql.checkData(3, 1, 'tbl_count')
tdSql.checkData(3, 2, 'stb1')
tdSql.query('select count(1),db_name from information_schema.ins_tables group by db_name')
tdSql.checkRows(3)
tdSql.checkData(0, 0, 5)
tdSql.checkData(0, 1, 'performance_schema')
tdSql.checkData(1, 0, 4)
tdSql.checkData(1, 1, 'tbl_count')
tdSql.checkData(2, 0, 23)
tdSql.checkData(2, 1, 'information_schema')
tdSql.query("select count(*) from information_schema.ins_tables where db_name='tbl_count'")
tdSql.checkRows(1)
tdSql.checkData(0, 0, 4)
tdSql.query('select count(*) from information_schema.ins_tables where db_name=\'tbl_count\' and stable_name="stb1";')
tdSql.checkRows(1)
tdSql.checkData(0, 0, 3)
tdSql.query('select count(*) from information_schema.ins_tables')
tdSql.checkRows(1)
tdSql.checkData(0, 0, 32)
tdSql.execute('drop database tbl_count')
def stop(self):
tdSql.close()
tdLog.success("%s successfully executed" % __file__)
tdCases.addWindows(__file__, TDTestCase())
tdCases.addLinux(__file__, TDTestCase())

View File

@ -669,7 +669,7 @@
,,y,system-test,./pytest.sh python3 ./test.py -f 6-cluster/5dnode3mnodeRestartDnodeInsertDataAsync.py -N 6 -M 3 -n 3
,,y,system-test,./pytest.sh python3 ./test.py -f 6-cluster/5dnode3mnodeAdd1Ddnoe.py -N 7 -M 3 -C 6
,,y,system-test,./pytest.sh python3 ./test.py -f 6-cluster/5dnode3mnodeAdd1Ddnoe.py -N 7 -M 3 -C 6 -n 3
,,y,system-test,./pytest.sh python3 ./test.py -f 6-cluster/5dnode3mnodeDrop.py -N 5
#,,y,system-test,./pytest.sh python3 ./test.py -f 6-cluster/5dnode3mnodeDrop.py -N 5
,,y,system-test,./pytest.sh python3 ./test.py -f 6-cluster/5dnode3mnodeRecreateMnode.py -N 5 -M 3
,,y,system-test,./pytest.sh python3 ./test.py -f 6-cluster/5dnode3mnodeStopFollowerLeader.py -N 5 -M 3
,,y,system-test,./pytest.sh python3 ./test.py -f 6-cluster/5dnode3mnodeStop2Follower.py -N 5 -M 3
@ -1050,6 +1050,7 @@
,,y,system-test,./pytest.sh python3 ./test.py -f 99-TDcase/TD-20582.py
#develop test
,,n,develop-test,python3 ./test.py -f 2-query/table_count_scan.py
,,n,develop-test,python3 ./test.py -f 5-taos-tools/taosbenchmark/auto_create_table_json.py
,,n,develop-test,python3 ./test.py -f 5-taos-tools/taosbenchmark/custom_col_tag.py
,,n,develop-test,python3 ./test.py -f 5-taos-tools/taosbenchmark/default_json.py

View File

@ -39,7 +39,7 @@ python_error=`cat ${LOG_DIR}/*.info | grep -w "stack" | wc -l`
# /root/TDengine/source/libs/scalar/src/sclvector.c:1075:66: runtime error: signed integer overflow: 9223372034707292160 + 1668838476672 cannot be represented in type 'long int'
# /root/TDengine/source/common/src/tdataformat.c:1876:7: runtime error: signed integer overflow: 8252423483843671206 + 2406154664059062870 cannot be represented in type 'long int'
runtime_error=`cat ${LOG_DIR}/*.asan | grep "runtime error" | grep -v "trees.c:873" | grep -v "sclfunc.c.*outside the range of representable values of type"| grep -v "signed integer overflow" | wc -l`
runtime_error=`cat ${LOG_DIR}/*.asan | grep "runtime error" | grep -v "trees.c:873" | grep -v "sclfunc.c.*outside the range of representable values of type"| grep -v "signed integer overflow" |grep -v "strerror.c"| grep -v "asan_malloc_linux.cc" |wc -l`
echo -e "\033[44;32;1m"asan error_num: $error_num"\033[0m"
echo -e "\033[44;32;1m"asan memory_leak: $memory_leak"\033[0m"

View File

@ -24,7 +24,7 @@ from util.dnodes import tdDnodes
from util.dnodes import *
class TDTestCase:
updatecfgDict = {'maxSQLLength':1048576,'debugFlag': 143 ,"querySmaOptimize":1}
updatecfgDict = {'maxSQLLength':1048576,'debugFlag': 131 ,"querySmaOptimize":1}
def init(self, conn, logSql, replicaVar):
tdLog.debug("start to execute %s" % __file__)

View File

@ -24,9 +24,9 @@ from util.dnodes import tdDnodes
from util.dnodes import *
class TDTestCase:
updatecfgDict = {'maxSQLLength':1048576,'debugFlag': 143 ,"cDebugFlag":143,"uDebugFlag":143 ,"rpcDebugFlag":143 , "tmrDebugFlag":143 ,
"jniDebugFlag":143 ,"simDebugFlag":143,"dDebugFlag":143, "dDebugFlag":143,"vDebugFlag":143,"mDebugFlag":143,"qDebugFlag":143,
"wDebugFlag":143,"sDebugFlag":143,"tsdbDebugFlag":143,"tqDebugFlag":143 ,"fsDebugFlag":143 ,"fnDebugFlag":143}
updatecfgDict = {'maxSQLLength':1048576,'debugFlag': 131 ,"cDebugFlag":131,"uDebugFlag":131 ,"rpcDebugFlag":131 , "tmrDebugFlag":131 ,
"jniDebugFlag":131 ,"simDebugFlag":131,"dDebugFlag":131, "dDebugFlag":131,"vDebugFlag":131,"mDebugFlag":131,"qDebugFlag":131,
"wDebugFlag":131,"sDebugFlag":131,"tsdbDebugFlag":131,"tqDebugFlag":131 ,"fsDebugFlag":131 ,"fnDebugFlag":131}
def init(self, conn, logSql, replicaVar=1):
self.replicaVar = int(replicaVar)

View File

@ -24,9 +24,9 @@ from util.dnodes import tdDnodes
from util.dnodes import *
class TDTestCase:
updatecfgDict = {'maxSQLLength':1048576,'debugFlag': 143 ,"cDebugFlag":143,"uDebugFlag":143 ,"rpcDebugFlag":143 , "tmrDebugFlag":143 ,
"jniDebugFlag":143 ,"simDebugFlag":143,"dDebugFlag":143, "dDebugFlag":143,"vDebugFlag":143,"mDebugFlag":143,"qDebugFlag":143,
"wDebugFlag":143,"sDebugFlag":143,"tsdbDebugFlag":143,"tqDebugFlag":143 ,"fsDebugFlag":143 ,"fnDebugFlag":143}
updatecfgDict = {'maxSQLLength':1048576,'debugFlag': 131 ,"cDebugFlag":131,"uDebugFlag":131 ,"rpcDebugFlag":131 , "tmrDebugFlag":131 ,
"jniDebugFlag":131 ,"simDebugFlag":131,"dDebugFlag":131, "dDebugFlag":131,"vDebugFlag":131,"mDebugFlag":131,"qDebugFlag":131,
"wDebugFlag":131,"sDebugFlag":131,"tsdbDebugFlag":131,"tqDebugFlag":131 ,"fsDebugFlag":131 ,"fnDebugFlag":131}
def init(self, conn, logSql, replicaVar=1):
self.replicaVar = int(replicaVar)

View File

@ -121,7 +121,7 @@ class TDTestCase:
def check3mnode(self):
count=0
while count < 10:
time.sleep(1)
time.sleep(0.1)
tdSql.query("select * from information_schema.ins_mnodes;")
if tdSql.checkRows(3) :
tdLog.debug("mnode is three nodes")
@ -158,7 +158,7 @@ class TDTestCase:
def check3mnode1off(self):
count=0
while count < 10:
time.sleep(1)
time.sleep(0.1)
tdSql.query("select * from information_schema.ins_mnodes;")
if tdSql.checkRows(3) :
tdLog.debug("mnode is three nodes")
@ -190,7 +190,7 @@ class TDTestCase:
def check3mnode2off(self):
count=0
while count < 40:
time.sleep(1)
time.sleep(0.1)
tdSql.query("select * from information_schema.ins_mnodes;")
if tdSql.checkRows(3) :
tdLog.debug("mnode is three nodes")
@ -220,7 +220,7 @@ class TDTestCase:
def check3mnode3off(self):
count=0
while count < 10:
time.sleep(1)
time.sleep(0.1)
tdSql.query("select * from information_schema.ins_mnodes;")
if tdSql.checkRows(3) :
tdLog.debug("mnode is three nodes")
@ -280,32 +280,32 @@ class TDTestCase:
# drop follower of mnode
dropcount =0
while dropcount <= 10:
while dropcount <= 5:
for i in range(1,3):
tdLog.debug("drop mnode on dnode %d"%(i+1))
tdSql.execute("drop mnode on dnode %d"%(i+1))
tdSql.query("select * from information_schema.ins_mnodes;")
count=0
while count<10:
time.sleep(1)
time.sleep(0.1)
tdSql.query("select * from information_schema.ins_mnodes;")
if tdSql.checkRows(2):
if tdSql.queryRows == 2:
tdLog.debug("drop mnode %d successfully"%(i+1))
break
count+=1
self.wait_for_transactions(20)
self.wait_for_transactions(100)
tdLog.debug("create mnode on dnode %d"%(i+1))
tdSql.execute("create mnode on dnode %d"%(i+1))
count=0
while count<10:
time.sleep(1)
time.sleep(0.1)
tdSql.query("select * from information_schema.ins_mnodes;")
if tdSql.checkRows(3):
if tdSql.queryRows == 3:
tdLog.debug("create mnode %d successfully"%(i+1))
break
count+=1
self.wait_for_transactions(20)
self.wait_for_transactions(100)
dropcount+=1
self.check3mnode()
@ -314,12 +314,13 @@ class TDTestCase:
while count<timeout:
time.sleep(1)
tdSql.query("show transactions;")
if tdSql.checkRows(0):
# print(tdSql.queryRows)
if tdSql.queryRows == 0 :
tdLog.debug("transactions completed successfully")
break
count+=1
if count >= timeout:
tdLog.debug("transactions not finished before timeout (%d secs)", timeout)
tdLog.debug("transactions not finished before timeout (%d secs)"%timeout)
def getConnection(self, dnode):
host = dnode.cfgDict["fqdn"]

View File

@ -130,7 +130,7 @@ class TDTestCase:
tdLog.info("expect consume rows: %d, act consume rows: %d"%(expectRowsList[0], resultList[0]))
tdLog.exit("%d tmq consume rows error!"%consumerId)
tmqCom.checkFileContent(consumerId, queryString)
# tmqCom.checkFileContent(consumerId, queryString)
time.sleep(10)
for i in range(len(topicNameList)):

View File

@ -116,7 +116,7 @@ class TDTestCase:
topicList = topicNameList[0]
ifcheckdata = 1
ifManualCommit = 1
keyList = 'group.id:cgrp1, enable.auto.commit:true, auto.commit.interval.ms:1000, auto.offset.reset:earliest'
keyList = 'group.id:cgrp1, enable.auto.commit:true, auto.commit.interval.ms:200, auto.offset.reset:earliest'
tmqCom.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit)
consumerId = 4
@ -188,7 +188,7 @@ class TDTestCase:
topicList = topicNameList[0]
ifcheckdata = 1
ifManualCommit = 1
keyList = 'group.id:cgrp1, enable.auto.commit:true, auto.commit.interval.ms:1000, auto.offset.reset:earliest'
keyList = 'group.id:cgrp1, enable.auto.commit:true, auto.commit.interval.ms:200, auto.offset.reset:earliest'
tmqCom.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit)
tdLog.info("start consume processor 0")

View File

@ -116,7 +116,7 @@ class TDTestCase:
topicList = topicNameList[0]
ifcheckdata = 1
ifManualCommit = 1
keyList = 'group.id:cgrp1, enable.auto.commit:true, auto.commit.interval.ms:1000, auto.offset.reset:earliest'
keyList = 'group.id:cgrp1, enable.auto.commit:true, auto.commit.interval.ms:200, auto.offset.reset:earliest'
tmqCom.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit)
consumerId = 4
@ -188,7 +188,7 @@ class TDTestCase:
topicList = topicNameList[0]
ifcheckdata = 1
ifManualCommit = 1
keyList = 'group.id:cgrp1, enable.auto.commit:true, auto.commit.interval.ms:1000, auto.offset.reset:earliest'
keyList = 'group.id:cgrp1, enable.auto.commit:true, auto.commit.interval.ms:200, auto.offset.reset:earliest'
tmqCom.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit)
tdLog.info("start consume processor 0")