From 077745390dc75dfc19d311e92f0c6e29a115e403 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 3 Aug 2024 16:34:26 +0800 Subject: [PATCH 01/46] fix(stream): add more check in tmr. --- source/libs/stream/src/streamCheckStatus.c | 7 ------- source/libs/stream/src/streamCheckpoint.c | 22 ++++++++++++++++++++++ source/libs/stream/src/streamDispatch.c | 13 ++++++++++++- 3 files changed, 34 insertions(+), 8 deletions(-) diff --git a/source/libs/stream/src/streamCheckStatus.c b/source/libs/stream/src/streamCheckStatus.c index c9ba6ffcfe..b7661e72d4 100644 --- a/source/libs/stream/src/streamCheckStatus.c +++ b/source/libs/stream/src/streamCheckStatus.c @@ -74,13 +74,6 @@ int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_ } if (pInfo->stage != stage) { - streamMutexLock(&pTask->lock); - ETaskStatus status = streamTaskGetStatus(pTask).state; - if (status == TASK_STATUS__CK) { - streamTaskSetFailedCheckpointId(pTask); - } - streamMutexUnlock(&pTask->lock); - return TASK_UPSTREAM_NEW_STAGE; } else if (pTask->status.downstreamReady != 1) { stDebug("s-task:%s vgId:%d leader:%d, downstream not ready", id, vgId, (pTask->pMeta->role == NODE_ROLE_LEADER)); diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index f817447099..741e3cc882 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -838,6 +838,28 @@ void checkpointTriggerMonitorFn(void* param, void* tmrId) { return; } + if ((pTmrInfo->launchChkptId != pActiveInfo->activeId) || (pActiveInfo->activeId == 0)) { + streamMutexUnlock(&pActiveInfo->lock); + int32_t ref = streamCleanBeforeQuitTmr(pTmrInfo, pTask); + stWarn("s-task:%s vgId:%d checkpoint-trigger retrieve by previous checkpoint procedure, checkpointId:%" PRId64 + ", quit, ref:%d", + id, vgId, pTmrInfo->launchChkptId, ref); + + streamMetaReleaseTask(pTask->pMeta, pTask); + return; + } + + // active checkpoint info is cleared for now + if ((pActiveInfo->activeId == 0) || (pActiveInfo->transId == 0) || (pTask->chkInfo.startTs == 0)) { + streamMutexUnlock(&pActiveInfo->lock); + int32_t ref = streamCleanBeforeQuitTmr(pTmrInfo, pTask); + stWarn("s-task:%s vgId:%d active checkpoint may be cleared, quit from retrieve checkpoint-trigger send tmr, ref:%d", + id, vgId, ref); + + streamMetaReleaseTask(pTask->pMeta, pTask); + return; + } + for (int32_t i = 0; i < taosArrayGetSize(pList); ++i) { SStreamUpstreamEpInfo* pInfo = taosArrayGetP(pList, i); diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 255afb44f9..9e07059e53 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -819,8 +819,19 @@ static void checkpointReadyMsgSendMonitorFn(void* param, void* tmrId) { SArray* pList = pActiveInfo->pReadyMsgList; int32_t num = taosArrayGetSize(pList); + if (pTmrInfo->launchChkptId != pActiveInfo->activeId) { + streamMutexUnlock(&pActiveInfo->lock); + int32_t ref = streamCleanBeforeQuitTmr(pTmrInfo, pTask); + stWarn("s-task:%s vgId:%d ready-msg send tmr launched by previous checkpoint procedure, checkpointId:%" PRId64 + ", quit, ref:%d", + id, vgId, pTmrInfo->launchChkptId, ref); + + streamMetaReleaseTask(pTask->pMeta, pTask); + return; + } + // active checkpoint info is cleared for now - if ((pActiveInfo->activeId == 0) && (pActiveInfo->transId == 0) && (num == 0) && (pTask->chkInfo.startTs == 0)) { + if ((pActiveInfo->activeId == 0) || (pActiveInfo->transId == 0) || (num == 0) || (pTask->chkInfo.startTs == 0)) { streamMutexUnlock(&pActiveInfo->lock); int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); stWarn("s-task:%s vgId:%d active checkpoint may be cleared, quit from readyMsg send tmr, ref:%d", id, vgId, ref); From 6a6ab9ff6a0b57426d2ab290192df0a5a5b9f02e Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 4 Aug 2024 11:37:23 +0800 Subject: [PATCH 02/46] fix(stream): add check for checkpointId in retrieve-checkpoint id msg. --- source/dnode/vnode/src/tqCommon/tqCommon.c | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index b56c474ed5..11d38dde87 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -989,7 +989,12 @@ int32_t tqStreamTaskProcessRetrieveTriggerReq(SStreamMeta* pMeta, SRpcMsg* pMsg) int64_t checkpointId = 0; streamTaskGetActiveCheckpointInfo(pTask, &transId, &checkpointId); - ASSERT(checkpointId == pReq->checkpointId); + if (checkpointId != pReq->checkpointId) { + tqError("s-task:%s invalid checkpoint-trigger retrieve msg from %x, current checkpointId:%"PRId64" req:%"PRId64, + pTask->id.idStr, pReq->downstreamTaskId, checkpointId, pReq->checkpointId); + streamMetaReleaseTask(pMeta, pTask); + return TSDB_CODE_INVALID_MSG; + } if (streamTaskAlreadySendTrigger(pTask, pReq->downstreamNodeId)) { // re-send the lost checkpoint-trigger msg to downstream task From d5b990e4ed2e77eaf23b2a865b6716d4b901799a Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Thu, 8 Aug 2024 16:38:11 +0800 Subject: [PATCH 03/46] fix:[TS-5156]add user in show consumers --- include/common/tmsg.h | 3 +++ source/client/src/clientTmq.c | 3 +++ source/dnode/mnode/impl/inc/mndDef.h | 1 + source/dnode/mnode/impl/src/mndConsumer.c | 8 ++++++++ source/dnode/mnode/impl/src/mndDef.c | 2 ++ 5 files changed, 17 insertions(+) diff --git a/include/common/tmsg.h b/include/common/tmsg.h index 70cf9c8b58..63fe4271d9 100644 --- a/include/common/tmsg.h +++ b/include/common/tmsg.h @@ -2837,6 +2837,7 @@ typedef struct { int64_t consumerId; char cgroup[TSDB_CGROUP_LEN]; char clientId[TSDB_CLIENT_ID_LEN]; + char user[TSDB_USER_LEN]; SArray* topicNames; // SArray int8_t withTbName; @@ -2870,6 +2871,7 @@ static FORCE_INLINE int32_t tSerializeSCMSubscribeReq(void** buf, const SCMSubsc tlen += taosEncodeFixedI8(buf, pReq->enableBatchMeta); tlen += taosEncodeFixedI32(buf, pReq->sessionTimeoutMs); tlen += taosEncodeFixedI32(buf, pReq->maxPollIntervalMs); + tlen += taosEncodeString(buf, pReq->user); return tlen; } @@ -2904,6 +2906,7 @@ static FORCE_INLINE int32_t tDeserializeSCMSubscribeReq(void* buf, SCMSubscribeR if ((char*)buf - (char*)start < len) { buf = taosDecodeFixedI32(buf, &pReq->sessionTimeoutMs); buf = taosDecodeFixedI32(buf, &pReq->maxPollIntervalMs); + buf = taosDecodeStringTo(buf, pReq->user); } else { pReq->sessionTimeoutMs = DEFAULT_SESSION_TIMEOUT; pReq->maxPollIntervalMs = DEFAULT_MAX_POLL_INTERVAL; diff --git a/source/client/src/clientTmq.c b/source/client/src/clientTmq.c index 197a65add8..61037c2e68 100644 --- a/source/client/src/clientTmq.c +++ b/source/client/src/clientTmq.c @@ -82,6 +82,7 @@ struct tmq_t { int64_t refId; char groupId[TSDB_CGROUP_LEN]; char clientId[TSDB_CLIENT_ID_LEN]; + char user[TSDB_USER_LEN]; int8_t withTbName; int8_t useSnapshot; int8_t autoCommit; @@ -1265,6 +1266,7 @@ tmq_t* tmq_consumer_new(tmq_conf_t* conf, char* errstr, int32_t errstrLen) { pTmq->replayEnable = conf->replayEnable; pTmq->sourceExcluded = conf->sourceExcluded; pTmq->enableBatchMeta = conf->enableBatchMeta; + tstrncpy(pTmq->user, user, TSDB_USER_LEN); if (conf->replayEnable) { pTmq->autoCommit = false; } @@ -1332,6 +1334,7 @@ int32_t tmq_subscribe(tmq_t* tmq, const tmq_list_t* topic_list) { req.consumerId = tmq->consumerId; tstrncpy(req.clientId, tmq->clientId, TSDB_CLIENT_ID_LEN); tstrncpy(req.cgroup, tmq->groupId, TSDB_CGROUP_LEN); + tstrncpy(req.user, tmq->user, TSDB_USER_LEN); req.topicNames = taosArrayInit(sz, sizeof(void*)); if (req.topicNames == NULL) { diff --git a/source/dnode/mnode/impl/inc/mndDef.h b/source/dnode/mnode/impl/inc/mndDef.h index 62e77867f6..0505f604a2 100644 --- a/source/dnode/mnode/impl/inc/mndDef.h +++ b/source/dnode/mnode/impl/inc/mndDef.h @@ -597,6 +597,7 @@ typedef struct { int64_t consumerId; char cgroup[TSDB_CGROUP_LEN]; char clientId[TSDB_CLIENT_ID_LEN]; + char user[TSDB_USER_LEN]; int8_t updateType; // used only for update int32_t epoch; int32_t status; diff --git a/source/dnode/mnode/impl/src/mndConsumer.c b/source/dnode/mnode/impl/src/mndConsumer.c index 6116d2da19..5f3f794ef9 100644 --- a/source/dnode/mnode/impl/src/mndConsumer.c +++ b/source/dnode/mnode/impl/src/mndConsumer.c @@ -903,6 +903,14 @@ static int32_t mndRetrieveConsumer(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock * MND_TMQ_NULL_CHECK(pColInfo); MND_TMQ_RETURN_CHECK(colDataSetVal(pColInfo, numOfRows, (const char *)clientId, false)); + // user + char user[TSDB_USER_LEN + VARSTR_HEADER_SIZE] = {0}; + STR_TO_VARSTR(user, pConsumer->user); + + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + MND_TMQ_NULL_CHECK(pColInfo); + MND_TMQ_RETURN_CHECK(colDataSetVal(pColInfo, numOfRows, (const char *)user, false)); + // status const char *pStatusName = mndConsumerStatusName(pConsumer->status); status = taosMemoryCalloc(1, pShow->pMeta->pSchemas[cols].bytes); diff --git a/source/dnode/mnode/impl/src/mndDef.c b/source/dnode/mnode/impl/src/mndDef.c index 695bf4d30d..c7de16d824 100644 --- a/source/dnode/mnode/impl/src/mndDef.c +++ b/source/dnode/mnode/impl/src/mndDef.c @@ -429,6 +429,7 @@ int32_t tEncodeSMqConsumerObj(void **buf, const SMqConsumerObj *pConsumer) { tlen += taosEncodeFixedI32(buf, pConsumer->resetOffsetCfg); tlen += taosEncodeFixedI32(buf, pConsumer->maxPollIntervalMs); tlen += taosEncodeFixedI32(buf, pConsumer->sessionTimeoutMs); + tlen += taosEncodeString(buf, pConsumer->user); return tlen; } @@ -503,6 +504,7 @@ void *tDecodeSMqConsumerObj(const void *buf, SMqConsumerObj *pConsumer, int8_t s if (sver > 2){ buf = taosDecodeFixedI32(buf, &pConsumer->maxPollIntervalMs); buf = taosDecodeFixedI32(buf, &pConsumer->sessionTimeoutMs); + buf = taosDecodeStringTo(buf, pConsumer->user); } else{ pConsumer->maxPollIntervalMs = DEFAULT_MAX_POLL_INTERVAL; pConsumer->sessionTimeoutMs = DEFAULT_SESSION_TIMEOUT; From 9e018e8e58d6ce05db6bb3443ae487017e6b9fa7 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Fri, 9 Aug 2024 14:28:58 +0800 Subject: [PATCH 04/46] fix:[TS-5156]add user/fqdn in show consumers --- include/common/tmsg.h | 3 +++ source/client/src/clientTmq.c | 5 ++++ source/common/src/systable.c | 7 +++-- source/dnode/mnode/impl/inc/mndDef.h | 1 + source/dnode/mnode/impl/src/mndConsumer.c | 8 ++++++ source/dnode/mnode/impl/src/mndDef.c | 4 +++ source/dnode/mnode/impl/src/mndSubscribe.c | 31 ++++++++++++++++++---- 7 files changed, 52 insertions(+), 7 deletions(-) diff --git a/include/common/tmsg.h b/include/common/tmsg.h index 63fe4271d9..736267c3b9 100644 --- a/include/common/tmsg.h +++ b/include/common/tmsg.h @@ -2838,6 +2838,7 @@ typedef struct { char cgroup[TSDB_CGROUP_LEN]; char clientId[TSDB_CLIENT_ID_LEN]; char user[TSDB_USER_LEN]; + char fqdn[TSDB_FQDN_LEN]; SArray* topicNames; // SArray int8_t withTbName; @@ -2872,6 +2873,7 @@ static FORCE_INLINE int32_t tSerializeSCMSubscribeReq(void** buf, const SCMSubsc tlen += taosEncodeFixedI32(buf, pReq->sessionTimeoutMs); tlen += taosEncodeFixedI32(buf, pReq->maxPollIntervalMs); tlen += taosEncodeString(buf, pReq->user); + tlen += taosEncodeString(buf, pReq->fqdn); return tlen; } @@ -2907,6 +2909,7 @@ static FORCE_INLINE int32_t tDeserializeSCMSubscribeReq(void* buf, SCMSubscribeR buf = taosDecodeFixedI32(buf, &pReq->sessionTimeoutMs); buf = taosDecodeFixedI32(buf, &pReq->maxPollIntervalMs); buf = taosDecodeStringTo(buf, pReq->user); + buf = taosDecodeStringTo(buf, pReq->fqdn); } else { pReq->sessionTimeoutMs = DEFAULT_SESSION_TIMEOUT; pReq->maxPollIntervalMs = DEFAULT_MAX_POLL_INTERVAL; diff --git a/source/client/src/clientTmq.c b/source/client/src/clientTmq.c index 61037c2e68..a69af05900 100644 --- a/source/client/src/clientTmq.c +++ b/source/client/src/clientTmq.c @@ -83,6 +83,7 @@ struct tmq_t { char groupId[TSDB_CGROUP_LEN]; char clientId[TSDB_CLIENT_ID_LEN]; char user[TSDB_USER_LEN]; + char fqdn[TSDB_FQDN_LEN]; int8_t withTbName; int8_t useSnapshot; int8_t autoCommit; @@ -1267,6 +1268,9 @@ tmq_t* tmq_consumer_new(tmq_conf_t* conf, char* errstr, int32_t errstrLen) { pTmq->sourceExcluded = conf->sourceExcluded; pTmq->enableBatchMeta = conf->enableBatchMeta; tstrncpy(pTmq->user, user, TSDB_USER_LEN); + if (taosGetFqdn(pTmq->fqdn) != 0) { + (void)strcpy(pTmq->fqdn, "localhost"); + } if (conf->replayEnable) { pTmq->autoCommit = false; } @@ -1335,6 +1339,7 @@ int32_t tmq_subscribe(tmq_t* tmq, const tmq_list_t* topic_list) { tstrncpy(req.clientId, tmq->clientId, TSDB_CLIENT_ID_LEN); tstrncpy(req.cgroup, tmq->groupId, TSDB_CGROUP_LEN); tstrncpy(req.user, tmq->user, TSDB_USER_LEN); + tstrncpy(req.fqdn, tmq->fqdn, TSDB_FQDN_LEN); req.topicNames = taosArrayInit(sz, sizeof(void*)); if (req.topicNames == NULL) { diff --git a/source/common/src/systable.c b/source/common/src/systable.c index 2d69a687a6..3f27ab2b2b 100644 --- a/source/common/src/systable.c +++ b/source/common/src/systable.c @@ -344,7 +344,9 @@ static const SSysDbTableSchema subscriptionSchema[] = { {.name = "topic_name", .bytes = TSDB_TOPIC_FNAME_LEN + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_BINARY, .sysInfo = false}, {.name = "consumer_group", .bytes = TSDB_CGROUP_LEN + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_BINARY, .sysInfo = false}, {.name = "vgroup_id", .bytes = 4, .type = TSDB_DATA_TYPE_INT, .sysInfo = false}, - {.name = "consumer_id", .bytes = 32, .type = TSDB_DATA_TYPE_BINARY, .sysInfo = false}, + {.name = "consumer_id", .bytes = TSDB_CLIENT_ID_LEN + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_BINARY, .sysInfo = false}, + {.name = "user", .bytes = TSDB_USER_LEN + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_BINARY, .sysInfo = false}, + {.name = "fqdn", .bytes = TSDB_FQDN_LEN + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_BINARY, .sysInfo = false}, {.name = "offset", .bytes = TSDB_OFFSET_LEN + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_BINARY, .sysInfo = false}, {.name = "rows", .bytes = 8, .type = TSDB_DATA_TYPE_BIGINT, .sysInfo = false}, }; @@ -480,11 +482,12 @@ static const SSysDbTableSchema connectionsSchema[] = { {.name = "last_access", .bytes = 8, .type = TSDB_DATA_TYPE_TIMESTAMP, .sysInfo = false}, }; - static const SSysDbTableSchema consumerSchema[] = { {.name = "consumer_id", .bytes = TSDB_CONSUMER_ID_LEN + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_BINARY, .sysInfo = false}, {.name = "consumer_group", .bytes = TSDB_CGROUP_LEN + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_BINARY, .sysInfo = false}, {.name = "client_id", .bytes = TSDB_CLIENT_ID_LEN + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_BINARY, .sysInfo = false}, + {.name = "user", .bytes = TSDB_USER_LEN + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_BINARY, .sysInfo = false}, + {.name = "fqdn", .bytes = TSDB_FQDN_LEN + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_BINARY, .sysInfo = false}, {.name = "status", .bytes = 20 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_BINARY, .sysInfo = false}, {.name = "topics", .bytes = TSDB_TOPIC_FNAME_LEN + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_BINARY, .sysInfo = false}, /*{.name = "end_point", .bytes = TSDB_IPv4ADDR_LEN + 6 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false},*/ diff --git a/source/dnode/mnode/impl/inc/mndDef.h b/source/dnode/mnode/impl/inc/mndDef.h index 0505f604a2..99e59662ac 100644 --- a/source/dnode/mnode/impl/inc/mndDef.h +++ b/source/dnode/mnode/impl/inc/mndDef.h @@ -598,6 +598,7 @@ typedef struct { char cgroup[TSDB_CGROUP_LEN]; char clientId[TSDB_CLIENT_ID_LEN]; char user[TSDB_USER_LEN]; + char fqdn[TSDB_FQDN_LEN]; int8_t updateType; // used only for update int32_t epoch; int32_t status; diff --git a/source/dnode/mnode/impl/src/mndConsumer.c b/source/dnode/mnode/impl/src/mndConsumer.c index 5f3f794ef9..37eb899ed5 100644 --- a/source/dnode/mnode/impl/src/mndConsumer.c +++ b/source/dnode/mnode/impl/src/mndConsumer.c @@ -911,6 +911,14 @@ static int32_t mndRetrieveConsumer(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock * MND_TMQ_NULL_CHECK(pColInfo); MND_TMQ_RETURN_CHECK(colDataSetVal(pColInfo, numOfRows, (const char *)user, false)); + // fqdn + char fqdn[TSDB_FQDN_LEN + VARSTR_HEADER_SIZE] = {0}; + STR_TO_VARSTR(fqdn, pConsumer->fqdn); + + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + MND_TMQ_NULL_CHECK(pColInfo); + MND_TMQ_RETURN_CHECK(colDataSetVal(pColInfo, numOfRows, (const char *)fqdn, false)); + // status const char *pStatusName = mndConsumerStatusName(pConsumer->status); status = taosMemoryCalloc(1, pShow->pMeta->pSchemas[cols].bytes); diff --git a/source/dnode/mnode/impl/src/mndDef.c b/source/dnode/mnode/impl/src/mndDef.c index c7de16d824..c604e58588 100644 --- a/source/dnode/mnode/impl/src/mndDef.c +++ b/source/dnode/mnode/impl/src/mndDef.c @@ -325,6 +325,8 @@ int32_t tNewSMqConsumerObj(int64_t consumerId, char *cgroup, int8_t updateType, pConsumer->resetOffsetCfg = subscribe->resetOffsetCfg; pConsumer->maxPollIntervalMs = subscribe->maxPollIntervalMs; pConsumer->sessionTimeoutMs = subscribe->sessionTimeoutMs; + tstrncpy(pConsumer->user, subscribe->user, TSDB_USER_LEN); + tstrncpy(pConsumer->fqdn, subscribe->fqdn, TSDB_FQDN_LEN); pConsumer->rebNewTopics = taosArrayDup(subscribe->topicNames, topicNameDup); if (pConsumer->rebNewTopics == NULL){ @@ -430,6 +432,7 @@ int32_t tEncodeSMqConsumerObj(void **buf, const SMqConsumerObj *pConsumer) { tlen += taosEncodeFixedI32(buf, pConsumer->maxPollIntervalMs); tlen += taosEncodeFixedI32(buf, pConsumer->sessionTimeoutMs); tlen += taosEncodeString(buf, pConsumer->user); + tlen += taosEncodeString(buf, pConsumer->fqdn); return tlen; } @@ -505,6 +508,7 @@ void *tDecodeSMqConsumerObj(const void *buf, SMqConsumerObj *pConsumer, int8_t s buf = taosDecodeFixedI32(buf, &pConsumer->maxPollIntervalMs); buf = taosDecodeFixedI32(buf, &pConsumer->sessionTimeoutMs); buf = taosDecodeStringTo(buf, pConsumer->user); + buf = taosDecodeStringTo(buf, pConsumer->fqdn); } else{ pConsumer->maxPollIntervalMs = DEFAULT_MAX_POLL_INTERVAL; pConsumer->sessionTimeoutMs = DEFAULT_SESSION_TIMEOUT; diff --git a/source/dnode/mnode/impl/src/mndSubscribe.c b/source/dnode/mnode/impl/src/mndSubscribe.c index bff313dbaf..db5bb2eacd 100644 --- a/source/dnode/mnode/impl/src/mndSubscribe.c +++ b/source/dnode/mnode/impl/src/mndSubscribe.c @@ -1330,8 +1330,8 @@ END: TAOS_RETURN(code); } -static int32_t buildResult(SSDataBlock *pBlock, int32_t *numOfRows, int64_t consumerId, const char *topic, - const char *cgroup, SArray *vgs, SArray *offsetRows) { +static int32_t buildResult(SSDataBlock *pBlock, int32_t *numOfRows, int64_t consumerId, const char* user, const char* fqdn, + const char *topic, const char *cgroup, SArray *vgs, SArray *offsetRows) { int32_t code = 0; int32_t sz = taosArrayGetSize(vgs); for (int32_t j = 0; j < sz; j++) { @@ -1355,7 +1355,7 @@ static int32_t buildResult(SSDataBlock *pBlock, int32_t *numOfRows, int64_t cons MND_TMQ_RETURN_CHECK(colDataSetVal(pColInfo, *numOfRows, (const char *)&pVgEp->vgId, false)); // consumer id - char consumerIdHex[32] = {0}; + char consumerIdHex[TSDB_CONSUMER_ID_LEN] = {0}; (void)sprintf(varDataVal(consumerIdHex), "0x%" PRIx64, consumerId); varDataSetLen(consumerIdHex, strlen(varDataVal(consumerIdHex))); @@ -1363,6 +1363,18 @@ static int32_t buildResult(SSDataBlock *pBlock, int32_t *numOfRows, int64_t cons MND_TMQ_NULL_CHECK(pColInfo); MND_TMQ_RETURN_CHECK(colDataSetVal(pColInfo, *numOfRows, (const char *)consumerIdHex, consumerId == -1)); + char userStr[TSDB_USER_LEN + VARSTR_HEADER_SIZE] = {0}; + if (user) STR_TO_VARSTR(userStr, user); + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + MND_TMQ_NULL_CHECK(pColInfo); + MND_TMQ_RETURN_CHECK(colDataSetVal(pColInfo, *numOfRows, userStr, user == NULL)); + + char fqdnStr[TSDB_FQDN_LEN + VARSTR_HEADER_SIZE] = {0}; + if (fqdn) STR_TO_VARSTR(fqdnStr, fqdn); + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + MND_TMQ_NULL_CHECK(pColInfo); + MND_TMQ_RETURN_CHECK(colDataSetVal(pColInfo, *numOfRows, fqdnStr, fqdn == NULL)); + mInfo("mnd show subscriptions: topic %s, consumer:0x%" PRIx64 " cgroup %s vgid %d", varDataVal(topic), consumerId, varDataVal(cgroup), pVgEp->vgId); @@ -1435,16 +1447,25 @@ int32_t mndRetrieveSubscribe(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock *pBlock SMqConsumerEp *pConsumerEp = NULL; void *pIter = NULL; + while (1) { pIter = taosHashIterate(pSub->consumerHash, pIter); if (pIter == NULL) break; pConsumerEp = (SMqConsumerEp *)pIter; - MND_TMQ_RETURN_CHECK(buildResult(pBlock, &numOfRows, pConsumerEp->consumerId, topic, cgroup, pConsumerEp->vgs, + char *user = NULL; + char *fqdn = NULL; + SMqConsumerObj *pConsumer = sdbAcquire(pSdb, SDB_CONSUMER, &pConsumerEp->consumerId); + if (pConsumer != NULL) { + user = pConsumer->user; + fqdn = pConsumer->fqdn; + sdbRelease(pSdb, pConsumer); + } + MND_TMQ_RETURN_CHECK(buildResult(pBlock, &numOfRows, pConsumerEp->consumerId, user, fqdn, topic, cgroup, pConsumerEp->vgs, pConsumerEp->offsetRows)); } - MND_TMQ_RETURN_CHECK(buildResult(pBlock, &numOfRows, -1, topic, cgroup, pSub->unassignedVgs, pSub->offsetRows)); + MND_TMQ_RETURN_CHECK(buildResult(pBlock, &numOfRows, -1, NULL, NULL, topic, cgroup, pSub->unassignedVgs, pSub->offsetRows)); pBlock->info.rows = numOfRows; From 78f991b2c9db9aac551a4b0cbde767f6990da936 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Fri, 9 Aug 2024 16:56:35 +0800 Subject: [PATCH 05/46] fix:[TS-5156]case error --- tests/system-test/0-others/information_schema.py | 4 ++-- tests/system-test/7-tmq/tmq_primary_key.py | 10 +++++----- tests/system-test/7-tmq/tmq_taosx.py | 6 +++--- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/system-test/0-others/information_schema.py b/tests/system-test/0-others/information_schema.py index d7a5540544..616cd034ab 100644 --- a/tests/system-test/0-others/information_schema.py +++ b/tests/system-test/0-others/information_schema.py @@ -222,10 +222,10 @@ class TDTestCase: tdSql.query("select * from information_schema.ins_columns where db_name ='information_schema'") tdLog.info(len(tdSql.queryResult)) - tdSql.checkEqual(True, len(tdSql.queryResult) in range(261, 269)) + tdSql.checkEqual(True, len(tdSql.queryResult) in range(261, 271)) tdSql.query("select * from information_schema.ins_columns where db_name ='performance_schema'") - tdSql.checkEqual(54, len(tdSql.queryResult)) + tdSql.checkEqual(56, len(tdSql.queryResult)) def ins_dnodes_check(self): tdSql.execute('drop database if exists db2') diff --git a/tests/system-test/7-tmq/tmq_primary_key.py b/tests/system-test/7-tmq/tmq_primary_key.py index 80888ddbe6..13d6bd565d 100644 --- a/tests/system-test/7-tmq/tmq_primary_key.py +++ b/tests/system-test/7-tmq/tmq_primary_key.py @@ -85,7 +85,7 @@ class TDTestCase: time.sleep(4) # wait for heart beat tdSql.query(f'show subscriptions;') - sub = tdSql.getData(0, 4); + sub = tdSql.getData(0, 6); print(sub) if not sub.startswith("tsdb"): tdLog.exit(f"show subscriptions error") @@ -196,7 +196,7 @@ class TDTestCase: time.sleep(4) # wait for heart beat tdSql.query(f'show subscriptions;') - sub = tdSql.getData(0, 4); + sub = tdSql.getData(0, 6); print(sub) if not sub.startswith("tsdb"): tdLog.exit(f"show subscriptions error") @@ -306,7 +306,7 @@ class TDTestCase: time.sleep(4) # wait for heart beat tdSql.query(f'show subscriptions;') - sub = tdSql.getData(0, 4); + sub = tdSql.getData(0, 6); print(sub) if not sub.startswith("tsdb"): tdLog.exit(f"show subscriptions error") @@ -416,7 +416,7 @@ class TDTestCase: time.sleep(4) # wait for heart beat tdSql.query(f'show subscriptions;') - sub = tdSql.getData(0, 4); + sub = tdSql.getData(0, 6); print(sub) if not sub.startswith("tsdb"): tdLog.exit(f"show subscriptions error") @@ -517,7 +517,7 @@ class TDTestCase: consumer.close() tdSql.query(f'show subscriptions;') - sub = tdSql.getData(0, 4); + sub = tdSql.getData(0, 6); print(sub) if not sub.startswith("tsdb"): tdLog.exit(f"show subscriptions error") diff --git a/tests/system-test/7-tmq/tmq_taosx.py b/tests/system-test/7-tmq/tmq_taosx.py index d0e682cffb..4e90aefe7c 100644 --- a/tests/system-test/7-tmq/tmq_taosx.py +++ b/tests/system-test/7-tmq/tmq_taosx.py @@ -598,12 +598,12 @@ class TDTestCase: tdSql.query(f'show consumers') tdSql.checkRows(1) tdSql.checkData(0, 1, 'g1') - tdSql.checkData(0, 4, 't2') + tdSql.checkData(0, 6, 't2') tdSql.execute(f'drop consumer group g1 on t1') tdSql.query(f'show consumers') tdSql.checkRows(1) tdSql.checkData(0, 1, 'g1') - tdSql.checkData(0, 4, 't2') + tdSql.checkData(0, 6, 't2') tdSql.query(f'show subscriptions') tdSql.checkRows(1) @@ -641,7 +641,7 @@ class TDTestCase: tdSql.query(f'show consumers') tdSql.checkRows(1) tdSql.checkData(0, 1, 'g1') - tdSql.checkData(0, 4, 't2') + tdSql.checkData(0, 6, 't2') tdSql.execute(f'insert into t4 using st tags(3) values(now, 1)') try: From 7bf320607630e65cbd1fd7b9668f45576d4de70d Mon Sep 17 00:00:00 2001 From: dmchen Date: Tue, 13 Aug 2024 08:29:09 +0000 Subject: [PATCH 06/46] fix/TS-5262-conflict-return-error --- source/dnode/mnode/impl/src/mndSync.c | 9 ++++++--- source/dnode/mnode/impl/src/mndTrans.c | 5 ++++- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndSync.c b/source/dnode/mnode/impl/src/mndSync.c index 89f3c6e253..282ae677fe 100644 --- a/source/dnode/mnode/impl/src/mndSync.c +++ b/source/dnode/mnode/impl/src/mndSync.c @@ -183,7 +183,7 @@ int32_t mndProcessWriteMsg(SMnode *pMnode, SRpcMsg *pMsg, SFsmCbMeta *pMeta) { code = mndTransValidate(pMnode, pRaw); if (code != 0) { mError("trans:%d, failed to validate requested trans since %s", transId, terrstr()); - code = 0; + // code = 0; pMeta->code = code; goto _OUT; } @@ -191,7 +191,7 @@ int32_t mndProcessWriteMsg(SMnode *pMnode, SRpcMsg *pMsg, SFsmCbMeta *pMeta) { code = sdbWriteWithoutFree(pMnode->pSdb, pRaw); if (code != 0) { mError("trans:%d, failed to write to sdb since %s", transId, terrstr()); - code = 0; + // code = 0; pMeta->code = code; goto _OUT; } @@ -206,7 +206,10 @@ int32_t mndProcessWriteMsg(SMnode *pMnode, SRpcMsg *pMsg, SFsmCbMeta *pMeta) { if (pTrans->stage == TRN_STAGE_PREPARE) { bool continueExec = mndTransPerformPrepareStage(pMnode, pTrans, false); - if (!continueExec) goto _OUT; + if (!continueExec) { + if (terrno != 0) code = terrno; + goto _OUT; + } } mndTransRefresh(pMnode, pTrans); diff --git a/source/dnode/mnode/impl/src/mndTrans.c b/source/dnode/mnode/impl/src/mndTrans.c index 7f8d63c8e0..ab236007a1 100644 --- a/source/dnode/mnode/impl/src/mndTrans.c +++ b/source/dnode/mnode/impl/src/mndTrans.c @@ -1569,6 +1569,7 @@ static int32_t mndTransExecuteUndoActionsSerial(SMnode *pMnode, STrans *pTrans, bool mndTransPerformPrepareStage(SMnode *pMnode, STrans *pTrans, bool topHalf) { bool continueExec = true; int32_t code = 0; + terrno = 0; int32_t numOfActions = taosArrayGetSize(pTrans->prepareActions); if (numOfActions == 0) goto _OVER; @@ -1579,7 +1580,9 @@ bool mndTransPerformPrepareStage(SMnode *pMnode, STrans *pTrans, bool topHalf) { STransAction *pAction = taosArrayGet(pTrans->prepareActions, action); code = mndTransExecSingleAction(pMnode, pTrans, pAction, topHalf); if (code != 0) { - mError("trans:%d, failed to execute prepare action:%d, numOfActions:%d", pTrans->id, action, numOfActions); + terrno = code; + mError("trans:%d, failed to execute prepare action:%d, numOfActions:%d, since %s", pTrans->id, action, + numOfActions, tstrerror(code)); return false; } } From 61f0aec698e5b0f62a6f573a69ea209a944abf1a Mon Sep 17 00:00:00 2001 From: dapan1121 Date: Wed, 14 Aug 2024 10:05:03 +0800 Subject: [PATCH 07/46] fix: response message memory leak --- source/client/src/clientMsgHandler.c | 11 ++++++++++- source/client/src/clientTmq.c | 28 ++++++++++++++++++++++------ 2 files changed, 32 insertions(+), 7 deletions(-) diff --git a/source/client/src/clientMsgHandler.c b/source/client/src/clientMsgHandler.c index cc1ed7f3fa..4dea9c17b0 100644 --- a/source/client/src/clientMsgHandler.c +++ b/source/client/src/clientMsgHandler.c @@ -297,6 +297,9 @@ int32_t processUseDbRsp(void* param, SDataBuf* pMsg, int32_t code) { } if (strlen(usedbRsp.db) == 0) { + taosMemoryFree(pMsg->pData); + taosMemoryFree(pMsg->pEpSet); + if (usedbRsp.errCode != 0) { return usedbRsp.errCode; } else { @@ -366,9 +369,15 @@ int32_t processUseDbRsp(void* param, SDataBuf* pMsg, int32_t code) { } int32_t processCreateSTableRsp(void* param, SDataBuf* pMsg, int32_t code) { - if (pMsg == NULL || param == NULL) { + if (pMsg == NULL) { return TSDB_CODE_TSC_INVALID_INPUT; } + if (param == NULL) { + taosMemoryFree(pMsg->pEpSet); + taosMemoryFree(pMsg->pData); + return TSDB_CODE_TSC_INVALID_INPUT; + } + SRequestObj* pRequest = param; if (code != TSDB_CODE_SUCCESS) { diff --git a/source/client/src/clientTmq.c b/source/client/src/clientTmq.c index 8f35a2fad1..73d21c0e22 100644 --- a/source/client/src/clientTmq.c +++ b/source/client/src/clientTmq.c @@ -823,15 +823,17 @@ void tmqAssignDelayedCommitTask(void* param, void* tmrId) { int32_t tmqHbCb(void* param, SDataBuf* pMsg, int32_t code) { if (code != 0){ - return code; + goto _return; } if (pMsg == NULL || param == NULL) { - return TSDB_CODE_INVALID_PARA; + code = TSDB_CODE_INVALID_PARA; + goto _return; } + SMqHbRsp rsp = {0}; code = tDeserializeSMqHbRsp(pMsg->pData, pMsg->len, &rsp); if (code != 0) { - return code; + goto _return; } int64_t refId = (int64_t)param; @@ -854,10 +856,14 @@ int32_t tmqHbCb(void* param, SDataBuf* pMsg, int32_t code) { taosWUnLockLatch(&tmq->lock); (void)taosReleaseRef(tmqMgmt.rsetId, refId); } + tDestroySMqHbRsp(&rsp); + +_return: + taosMemoryFree(pMsg->pData); taosMemoryFree(pMsg->pEpSet); - return 0; + return code; } void tmqSendHbReq(void* param, void* tmrId) { @@ -1504,7 +1510,12 @@ static void setVgIdle(tmq_t* tmq, char* topicName, int32_t vgId) { int32_t tmqPollCb(void* param, SDataBuf* pMsg, int32_t code) { tmq_t* tmq = NULL; SMqPollCbParam* pParam = (SMqPollCbParam*)param; - if (pParam == NULL || pMsg == NULL) { + if (pMsg == NULL) { + return TSDB_CODE_TSC_INTERNAL_ERROR; + } + if (pParam == NULL) { + taosMemoryFreeClear(pMsg->pData); + taosMemoryFreeClear(pMsg->pEpSet); return TSDB_CODE_TSC_INTERNAL_ERROR; } int64_t refId = pParam->refId; @@ -1512,6 +1523,8 @@ int32_t tmqPollCb(void* param, SDataBuf* pMsg, int32_t code) { uint64_t requestId = pParam->requestId; tmq = taosAcquireRef(tmqMgmt.rsetId, refId); if (tmq == NULL) { + taosMemoryFreeClear(pMsg->pData); + taosMemoryFreeClear(pMsg->pEpSet); return TSDB_CODE_TMQ_CONSUMER_CLOSED; } @@ -2809,7 +2822,10 @@ end: } int32_t askEpCb(void* param, SDataBuf* pMsg, int32_t code) { - if (param == NULL) return code; + if (param == NULL) { + goto FAIL; + } + SMqAskEpCbParam* pParam = (SMqAskEpCbParam*)param; tmq_t* tmq = taosAcquireRef(tmqMgmt.rsetId, pParam->refId); if (tmq == NULL) { From e08d7240efe4986bb4ba8895dc3adf83aef07029 Mon Sep 17 00:00:00 2001 From: t_max <1172915550@qq.com> Date: Wed, 14 Aug 2024 14:14:17 +0800 Subject: [PATCH 08/46] fix: check transactions and reset query cache before running the example --- tests/docs-examples-test/csharp.sh | 65 ++++++++++++++++++++++++------ tests/docs-examples-test/go.sh | 30 ++++++++++++++ 2 files changed, 82 insertions(+), 13 deletions(-) diff --git a/tests/docs-examples-test/csharp.sh b/tests/docs-examples-test/csharp.sh index 0805b425b4..497cb074d9 100644 --- a/tests/docs-examples-test/csharp.sh +++ b/tests/docs-examples-test/csharp.sh @@ -2,6 +2,32 @@ set -e +check_transactions() { + for i in {1..30} + do + output=$(taos -s "show transactions;") + if [[ $output == *"Query OK, 0 row(s)"* ]]; then + echo "Success: No transactions are in progress." + return 0 + fi + sleep 1 + done + + echo "Error: Transactions are still in progress after 30 attempts." + return 1 +} + +reset_cache() { + response=$(curl --location -uroot:taosdata 'http://127.0.0.1:6041/rest/sql' --data 'reset query cache') + + if [[ $response == \{\"code\":0* ]]; then + echo "Success: Query cache reset successfully." + else + echo "Error: Failed to reset query cache. Response: $response" + return 1 + fi +} + pgrep taosd || taosd >> /dev/null 2>&1 & pgrep taosadapter || taosadapter >> /dev/null 2>&1 & cd ../../docs/examples/csharp @@ -10,56 +36,69 @@ dotnet run --project connect/connect.csproj dotnet run --project wsConnect/wsConnect.csproj taos -s "drop database if exists test" -sleep 1 +check_transactions || exit 1 +reset_cache || exit 1 dotnet run --project influxdbLine/influxdbline.csproj taos -s "drop database if exists test" -sleep 1 +check_transactions || exit 1 +reset_cache || exit 1 dotnet run --project optsTelnet/optstelnet.csproj taos -s "drop database if exists test" -sleep 1 +check_transactions || exit 1 +reset_cache || exit 1 dotnet run --project optsJSON/optsJSON.csproj # query taos -s "drop database if exists power" -sleep 1 +check_transactions || exit 1 +reset_cache || exit 1 dotnet run --project wsInsert/wsInsert.csproj dotnet run --project wsQuery/wsQuery.csproj taos -s "drop database if exists power" -sleep 1 +check_transactions || exit 1 +reset_cache || exit 1 dotnet run --project sqlInsert/sqlinsert.csproj dotnet run --project query/query.csproj # stmt taos -s "drop database if exists power" -sleep 1 +check_transactions || exit 1 +reset_cache || exit 1 dotnet run --project wsStmt/wsStmt.csproj taos -s "drop database if exists power" -sleep 1 +check_transactions || exit 1 +reset_cache || exit 1 dotnet run --project stmtInsert/stmtinsert.csproj # schemaless taos -s "drop database if exists power" -sleep 1 +check_transactions || exit 1 +reset_cache || exit 1 dotnet run --project wssml/wssml.csproj taos -s "drop database if exists power" -sleep 1 +check_transactions || exit 1 +reset_cache || exit 1 dotnet run --project nativesml/nativesml.csproj # subscribe taos -s "drop topic if exists topic_meters" -sleep 1 +check_transactions || exit 1 +reset_cache || exit 1 taos -s "drop database if exists power" -sleep 1 +check_transactions || exit 1 +reset_cache || exit 1 dotnet run --project wssubscribe/wssubscribe.csproj taos -s "drop topic if exists topic_meters" -sleep 1 +check_transactions || exit 1 +reset_cache || exit 1 taos -s "drop database if exists power" -sleep 1 +check_transactions || exit 1 +reset_cache || exit 1 dotnet run --project subscribe/subscribe.csproj diff --git a/tests/docs-examples-test/go.sh b/tests/docs-examples-test/go.sh index ea19d3212a..606265435d 100644 --- a/tests/docs-examples-test/go.sh +++ b/tests/docs-examples-test/go.sh @@ -17,6 +17,17 @@ check_transactions() { return 1 } +reset_cache() { + response=$(curl --location -uroot:taosdata 'http://127.0.0.1:6041/rest/sql' --data 'reset query cache') + + if [[ $response == \{\"code\":0* ]]; then + echo "Success: Query cache reset successfully." + else + echo "Error: Failed to reset query cache. Response: $response" + return 1 + fi +} + taosd >>/dev/null 2>&1 & taosadapter >>/dev/null 2>&1 & sleep 1 @@ -31,64 +42,83 @@ go run ./connect/connpool/main.go go run ./connect/wsexample/main.go taos -s "drop database if exists power" +check_transactions || exit 1 +reset_cache || exit 1 go run ./sqlquery/main.go taos -s "drop database if exists power" check_transactions || exit 1 +reset_cache || exit 1 go run ./queryreqid/main.go taos -s "drop database if exists power" check_transactions || exit 1 +reset_cache || exit 1 go run ./stmt/native/main.go taos -s "drop database if exists power" check_transactions || exit 1 +reset_cache || exit 1 go run ./stmt/ws/main.go taos -s "drop database if exists power" check_transactions || exit 1 +reset_cache || exit 1 sleep 3 go run ./schemaless/native/main.go taos -s "drop database if exists power" check_transactions || exit 1 +reset_cache || exit 1 go run ./schemaless/ws/main.go taos -s "drop topic if exists topic_meters" check_transactions || exit 1 +reset_cache || exit 1 taos -s "drop database if exists power" check_transactions || exit 1 +reset_cache || exit 1 go run ./tmq/native/main.go taos -s "drop topic if exists topic_meters" check_transactions || exit 1 +reset_cache || exit 1 taos -s "drop database if exists power" check_transactions || exit 1 +reset_cache || exit 1 go run ./tmq/ws/main.go taos -s "drop database if exists test" check_transactions || exit 1 +reset_cache || exit 1 go run ./insert/json/main.go taos -s "drop database if exists test" check_transactions || exit 1 +reset_cache || exit 1 go run ./insert/line/main.go taos -s "drop topic if exists topic_meters" check_transactions || exit 1 +reset_cache || exit 1 taos -s "drop database if exists power" check_transactions || exit 1 +reset_cache || exit 1 go run ./insert/sql/main.go taos -s "drop database if exists power" check_transactions || exit 1 +reset_cache || exit 1 go run ./insert/stmt/main.go taos -s "drop database if exists test" check_transactions || exit 1 +reset_cache || exit 1 go run ./insert/telnet/main.go go run ./query/sync/main.go taos -s "drop topic if exists example_tmq_topic" check_transactions || exit 1 +reset_cache || exit 1 taos -s "drop database if exists example_tmq" check_transactions || exit 1 +reset_cache || exit 1 go run ./sub/main.go From 19dcc5bddd3c55cd2d9765dbdfeacd85c41a4ec1 Mon Sep 17 00:00:00 2001 From: dapan1121 Date: Wed, 14 Aug 2024 14:50:57 +0800 Subject: [PATCH 09/46] fix: error log issue --- source/client/src/clientTmq.c | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/source/client/src/clientTmq.c b/source/client/src/clientTmq.c index 73d21c0e22..929debf16d 100644 --- a/source/client/src/clientTmq.c +++ b/source/client/src/clientTmq.c @@ -1509,6 +1509,10 @@ static void setVgIdle(tmq_t* tmq, char* topicName, int32_t vgId) { int32_t tmqPollCb(void* param, SDataBuf* pMsg, int32_t code) { tmq_t* tmq = NULL; + SMqPollRspWrapper* pRspWrapper = NULL; + int8_t rspType = 0; + int32_t vgId = 0; + uint64_t requestId = 0; SMqPollCbParam* pParam = (SMqPollCbParam*)param; if (pMsg == NULL) { return TSDB_CODE_TSC_INTERNAL_ERROR; @@ -1519,8 +1523,8 @@ int32_t tmqPollCb(void* param, SDataBuf* pMsg, int32_t code) { return TSDB_CODE_TSC_INTERNAL_ERROR; } int64_t refId = pParam->refId; - int32_t vgId = pParam->vgId; - uint64_t requestId = pParam->requestId; + vgId = pParam->vgId; + requestId = pParam->requestId; tmq = taosAcquireRef(tmqMgmt.rsetId, refId); if (tmq == NULL) { taosMemoryFreeClear(pMsg->pData); @@ -1528,7 +1532,6 @@ int32_t tmqPollCb(void* param, SDataBuf* pMsg, int32_t code) { return TSDB_CODE_TMQ_CONSUMER_CLOSED; } - SMqPollRspWrapper* pRspWrapper = NULL; int32_t ret = taosAllocateQitem(sizeof(SMqPollRspWrapper), DEF_QITEM, 0, (void**)&pRspWrapper); if (ret) { code = ret; @@ -1559,7 +1562,7 @@ int32_t tmqPollCb(void* param, SDataBuf* pMsg, int32_t code) { ASSERT(msgEpoch == clientEpoch); // handle meta rsp - int8_t rspType = ((SMqRspHead*)pMsg->pData)->mqMsgType; + rspType = ((SMqRspHead*)pMsg->pData)->mqMsgType; pRspWrapper->tmqRspType = rspType; pRspWrapper->reqId = requestId; pRspWrapper->pEpset = pMsg->pEpSet; @@ -1627,7 +1630,7 @@ END: } int32_t total = taosQueueItemSize(tmq->mqueue); tscDebug("consumer:0x%" PRIx64 " put poll res into mqueue, type:%d, vgId:%d, total in queue:%d, reqId:0x%" PRIx64, - tmq->consumerId, rspType, vgId, total, requestId); + tmq ? tmq->consumerId : 0, rspType, vgId, total, requestId); if (tmq) (void)tsem2_post(&tmq->rspSem); if (pMsg) taosMemoryFreeClear(pMsg->pData); From daf8ed7e06011ac27d26aae70b5be3b1e9e4afa1 Mon Sep 17 00:00:00 2001 From: wangjiaming0909 <604227650@qq.com> Date: Wed, 14 Aug 2024 16:23:08 +0800 Subject: [PATCH 10/46] fix db tsma version not updated by heart beat --- include/common/tmsg.h | 2 + include/libs/catalog/catalog.h | 2 + source/client/src/clientHb.c | 2 + source/common/src/tmsg.c | 7 +- source/dnode/mnode/impl/src/mndDb.c | 2 + source/libs/catalog/inc/catalogInt.h | 4 ++ source/libs/catalog/src/catalog.c | 12 ++++ source/libs/catalog/src/ctgCache.c | 102 +++++++++++++++++++++------ 8 files changed, 110 insertions(+), 23 deletions(-) diff --git a/include/common/tmsg.h b/include/common/tmsg.h index 70cf9c8b58..1d926ba42c 100644 --- a/include/common/tmsg.h +++ b/include/common/tmsg.h @@ -1547,6 +1547,8 @@ typedef struct { SDbCfgRsp* cfgRsp; STableTSMAInfoRsp* pTsmaRsp; int32_t dbTsmaVersion; + char db[TSDB_DB_FNAME_LEN]; + int64_t dbId; } SDbHbRsp; typedef struct { diff --git a/include/libs/catalog/catalog.h b/include/libs/catalog/catalog.h index 3f1cf74cfa..11ed7c7da6 100644 --- a/include/libs/catalog/catalog.h +++ b/include/libs/catalog/catalog.h @@ -415,6 +415,8 @@ int32_t catalogGetTableTsmas(SCatalog* pCtg, SRequestConnInfo* pConn, const SNam int32_t catalogGetTsma(SCatalog* pCtg, SRequestConnInfo* pConn, const SName* pTsmaName, STableTSMAInfo** pTsma); +int32_t catalogAsyncUpdateDbTsmaVersion(SCatalog* pCtg, int32_t tsmaVersion, const char* dbFName, int64_t dbId); + /** * Destroy catalog and relase all resources */ diff --git a/source/client/src/clientHb.c b/source/client/src/clientHb.c index 415c2d6685..70a519d8ae 100644 --- a/source/client/src/clientHb.c +++ b/source/client/src/clientHb.c @@ -269,6 +269,8 @@ static int32_t hbProcessDBInfoRsp(void *value, int32_t valueLen, struct SCatalog TSC_ERR_JRET(catalogAsyncUpdateTSMA(pCatalog, &pTsma, rsp->dbTsmaVersion)); } taosArrayClear(rsp->pTsmaRsp->pTsmas); + } else { + TSC_ERR_JRET(catalogAsyncUpdateDbTsmaVersion(pCatalog, rsp->dbTsmaVersion, rsp->db, rsp->dbId)); } } } diff --git a/source/common/src/tmsg.c b/source/common/src/tmsg.c index 740e517e35..4dc59bf6fe 100644 --- a/source/common/src/tmsg.c +++ b/source/common/src/tmsg.c @@ -3824,7 +3824,8 @@ int32_t tSerializeSDbHbRspImp(SEncoder *pEncoder, const SDbHbRsp *pRsp) { if (tEncodeI8(pEncoder, 0) < 0) return -1; } if (tEncodeI32(pEncoder, pRsp->dbTsmaVersion) < 0) return -1; - + if (tEncodeCStr(pEncoder, pRsp->db) < 0) return -1; + if (tEncodeI64(pEncoder, pRsp->dbId) < 0) return -1; return 0; } @@ -3915,6 +3916,10 @@ int32_t tDeserializeSDbHbRspImp(SDecoder *decoder, SDbHbRsp *pRsp) { if (!tDecodeIsEnd(decoder)) { if (tDecodeI32(decoder, &pRsp->dbTsmaVersion) < 0) return -1; } + if (!tDecodeIsEnd(decoder)) { + if (tDecodeCStrTo(decoder, pRsp->db) < 0) return -1; + if (tDecodeI64(decoder, &pRsp->dbId) < 0) return -1; + } return 0; } diff --git a/source/dnode/mnode/impl/src/mndDb.c b/source/dnode/mnode/impl/src/mndDb.c index dd3f89c9d0..fe5c12419c 100644 --- a/source/dnode/mnode/impl/src/mndDb.c +++ b/source/dnode/mnode/impl/src/mndDb.c @@ -1843,6 +1843,8 @@ int32_t mndValidateDbInfo(SMnode *pMnode, SDbCacheInfo *pDbs, int32_t numOfDbs, pDbCacheInfo->tsmaVersion = htonl(pDbCacheInfo->tsmaVersion); SDbHbRsp rsp = {0}; + (void)memcpy(rsp.db, pDbCacheInfo->dbFName, TSDB_DB_FNAME_LEN); + rsp.dbId = pDbCacheInfo->dbId; if ((0 == strcasecmp(pDbCacheInfo->dbFName, TSDB_INFORMATION_SCHEMA_DB) || (0 == strcasecmp(pDbCacheInfo->dbFName, TSDB_PERFORMANCE_SCHEMA_DB)))) { diff --git a/source/libs/catalog/inc/catalogInt.h b/source/libs/catalog/inc/catalogInt.h index f70cfff71d..f3b1852ce1 100644 --- a/source/libs/catalog/inc/catalogInt.h +++ b/source/libs/catalog/inc/catalogInt.h @@ -108,6 +108,7 @@ enum { CTG_OP_UPDATE_TB_TSMA, CTG_OP_DROP_TB_TSMA, CTG_OP_CLEAR_CACHE, + CTG_OP_UPDATE_DB_TSMA_VERSION, CTG_OP_MAX }; @@ -603,6 +604,7 @@ typedef struct SCtgUpdateTbTSMAMsg { STableTSMAInfo* pTsma; int32_t dbTsmaVersion; uint64_t dbId; + char dbFName[TSDB_DB_FNAME_LEN]; } SCtgUpdateTbTSMAMsg; typedef struct SCtgDropTbTSMAMsg { @@ -1167,6 +1169,8 @@ int32_t ctgGetStreamProgressFromVnode(SCatalog* pCtg, SRequestConnInfo* pConn, void* bInput); int32_t ctgAddTSMAFetch(SArray** pFetchs, int32_t dbIdx, int32_t tbIdx, int32_t* fetchIdx, int32_t resIdx, int32_t flag, CTG_TSMA_FETCH_TYPE fetchType, const SName* sourceTbName); +int32_t ctgOpUpdateDbTsmaVersion(SCtgCacheOperation* pOper); +int32_t ctgUpdateDbTsmaVersionEnqueue(SCatalog* pCtg, int32_t tsmaVersion, const char* dbFName, int64_t dbId, bool syncOper); void ctgFreeTask(SCtgTask* pTask, bool freeRes); extern SCatalogMgmt gCtgMgmt; diff --git a/source/libs/catalog/src/catalog.c b/source/libs/catalog/src/catalog.c index 27a7ce1022..d4c79a6c8d 100644 --- a/source/libs/catalog/src/catalog.c +++ b/source/libs/catalog/src/catalog.c @@ -1933,6 +1933,18 @@ _return: CTG_API_LEAVE(code); } +int32_t catalogAsyncUpdateDbTsmaVersion(SCatalog* pCtg, int32_t tsmaVersion, const char* dbFName, int64_t dbId) { + CTG_API_ENTER(); + if (!pCtg || !dbFName) { + CTG_API_LEAVE(TSDB_CODE_CTG_INVALID_INPUT); + } + int32_t code = 0; + CTG_ERR_JRET(ctgUpdateDbTsmaVersionEnqueue(pCtg, tsmaVersion, dbFName, dbId, false)); + +_return: + CTG_API_LEAVE(code); +} + int32_t catalogClearCache(void) { CTG_API_ENTER_NOLOCK(); diff --git a/source/libs/catalog/src/ctgCache.c b/source/libs/catalog/src/ctgCache.c index 8e5aba26af..689bf900e2 100644 --- a/source/libs/catalog/src/ctgCache.c +++ b/source/libs/catalog/src/ctgCache.c @@ -34,7 +34,8 @@ SCtgOperation gCtgCacheOperation[CTG_OP_MAX] = {{CTG_OP_UPDATE_VGROUP, "update v {CTG_OP_DROP_VIEW_META, "drop viewMeta", ctgOpDropViewMeta}, {CTG_OP_UPDATE_TB_TSMA, "update tbTSMA", ctgOpUpdateTbTSMA}, {CTG_OP_DROP_TB_TSMA, "drop tbTSMA", ctgOpDropTbTSMA}, - {CTG_OP_CLEAR_CACHE, "clear cache", ctgOpClearCache}}; + {CTG_OP_CLEAR_CACHE, "clear cache", ctgOpClearCache}, + {CTG_OP_UPDATE_DB_TSMA_VERSION, "update dbTsmaVersion", ctgOpUpdateDbTsmaVersion}}; SCtgCacheItemInfo gCtgStatItem[CTG_CI_MAX_VALUE] = { {"Cluster ", CTG_CI_FLAG_LEVEL_GLOBAL}, //CTG_CI_CLUSTER @@ -1628,6 +1629,41 @@ _return: CTG_RET(code); } +int32_t ctgUpdateDbTsmaVersionEnqueue(SCatalog* pCtg, int32_t tsmaVersion, const char* dbFName, int64_t dbId, bool syncOp) { + int32_t code = 0; + SCtgCacheOperation *op = taosMemoryCalloc(1, sizeof(SCtgCacheOperation)); + if (NULL == op) { + ctgError("malloc %d failed", (int32_t)sizeof(SCtgCacheOperation)); + CTG_ERR_RET(TSDB_CODE_OUT_OF_MEMORY); + } + + op->opId = CTG_OP_UPDATE_DB_TSMA_VERSION; + op->syncOp = syncOp; + + SCtgUpdateTbTSMAMsg *msg = taosMemoryMalloc(sizeof(SCtgUpdateTbTSMAMsg)); + if (NULL == msg) { + ctgError("malloc %d failed", (int32_t)sizeof(SCtgUpdateTbTSMAMsg)); + taosMemoryFree(op); + CTG_ERR_JRET(TSDB_CODE_OUT_OF_MEMORY); + } + + msg->pCtg = pCtg; + msg->pTsma = NULL; + msg->dbTsmaVersion = tsmaVersion; + msg->dbId = dbId; + memcpy(msg->dbFName, dbFName, TSDB_DB_FNAME_LEN); + + op->data = msg; + + CTG_ERR_JRET(ctgEnqueue(pCtg, op)); + + return TSDB_CODE_SUCCESS; + +_return: + + CTG_RET(code); +} + int32_t ctgAddNewDBCache(SCatalog *pCtg, const char *dbFName, uint64_t dbId) { int32_t code = 0; @@ -3010,6 +3046,32 @@ _return: CTG_RET(code); } +static int32_t ctgOpUpdateDbRentForTsmaVersion(SCtgDBCache* pDbCache, SCtgUpdateTbTSMAMsg* pMsg) { + int32_t code = TSDB_CODE_SUCCESS; + if (pDbCache && pMsg->dbTsmaVersion > 0) { + pDbCache->tsmaVersion = pMsg->dbTsmaVersion; + SDbCacheInfo cacheInfo = {0}; + cacheInfo.dbId = pDbCache->dbId; + + if (pDbCache->cfgCache.cfgInfo) { + cacheInfo.cfgVersion = pDbCache->cfgCache.cfgInfo->cfgVersion; + tstrncpy(cacheInfo.dbFName, pDbCache->cfgCache.cfgInfo->db, TSDB_DB_FNAME_LEN); + } + + if (pDbCache->vgCache.vgInfo) { + cacheInfo.vgVersion = pDbCache->vgCache.vgInfo->vgVersion; + cacheInfo.numOfTable = pDbCache->vgCache.vgInfo->numOfTable; + cacheInfo.stateTs = pDbCache->vgCache.vgInfo->stateTs; + } + + cacheInfo.tsmaVersion = pDbCache->tsmaVersion; + CTG_ERR_JRET(ctgMetaRentUpdate(&pMsg->pCtg->dbRent, &cacheInfo, cacheInfo.dbId, sizeof(SDbCacheInfo), + ctgDbCacheInfoSortCompare, ctgDbCacheInfoSearchCompare)); + } +_return: + CTG_RET(code); +} + int32_t ctgOpUpdateTbTSMA(SCtgCacheOperation *operation) { int32_t code = 0; SCtgUpdateTbTSMAMsg *msg = operation->data; @@ -3023,27 +3085,7 @@ int32_t ctgOpUpdateTbTSMA(SCtgCacheOperation *operation) { CTG_ERR_JRET(ctgGetAddDBCache(pCtg, pTsmaInfo->dbFName, pTsmaInfo->dbId, &dbCache)); CTG_ERR_JRET(ctgWriteTbTSMAToCache(pCtg, dbCache, pTsmaInfo->dbFName, pTsmaInfo->tb, &pTsmaInfo)); - - if (dbCache && msg->dbTsmaVersion > 0) { - dbCache->tsmaVersion = msg->dbTsmaVersion; - SDbCacheInfo cacheInfo = {0}; - cacheInfo.dbId = dbCache->dbId; - - if (dbCache->cfgCache.cfgInfo) { - cacheInfo.cfgVersion = dbCache->cfgCache.cfgInfo->cfgVersion; - tstrncpy(cacheInfo.dbFName, dbCache->cfgCache.cfgInfo->db, TSDB_DB_FNAME_LEN); - } - - if (dbCache->vgCache.vgInfo) { - cacheInfo.vgVersion = dbCache->vgCache.vgInfo->vgVersion; - cacheInfo.numOfTable = dbCache->vgCache.vgInfo->numOfTable; - cacheInfo.stateTs = dbCache->vgCache.vgInfo->stateTs; - } - - cacheInfo.tsmaVersion = dbCache->tsmaVersion; - CTG_ERR_JRET(ctgMetaRentUpdate(&msg->pCtg->dbRent, &cacheInfo, cacheInfo.dbId, sizeof(SDbCacheInfo), - ctgDbCacheInfoSortCompare, ctgDbCacheInfoSearchCompare)); - } + CTG_ERR_JRET(ctgOpUpdateDbRentForTsmaVersion(dbCache, msg)); _return: @@ -3057,6 +3099,22 @@ _return: CTG_RET(code); } +int32_t ctgOpUpdateDbTsmaVersion(SCtgCacheOperation *pOper) { + int32_t code = 0; + SCtgUpdateTbTSMAMsg *pMsg = pOper->data; + SCatalog *pCtg = pMsg->pCtg; + SCtgDBCache *pDbCache = NULL; + + if (pCtg->stopUpdate) goto _return; + + CTG_ERR_JRET(ctgGetAddDBCache(pCtg, pMsg->dbFName, pMsg->dbId, &pDbCache)); + CTG_ERR_JRET(ctgOpUpdateDbRentForTsmaVersion(pDbCache, pMsg)); + +_return: + taosMemoryFreeClear(pMsg); + CTG_RET(code); +} + void ctgFreeCacheOperationData(SCtgCacheOperation *op) { if (NULL == op || NULL == op->data) { From 2b40ecf71892604fa4ba7a0c6228d7819bcc6162 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 14 Aug 2024 20:41:28 +0800 Subject: [PATCH 11/46] fix meta deadlock --- source/dnode/vnode/src/meta/metaQuery.c | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/meta/metaQuery.c b/source/dnode/vnode/src/meta/metaQuery.c index 3abd185f0f..47ed7743cb 100644 --- a/source/dnode/vnode/src/meta/metaQuery.c +++ b/source/dnode/vnode/src/meta/metaQuery.c @@ -280,7 +280,11 @@ int32_t metaResumeTbCursor(SMTbCursor *pTbCur, int8_t first, int8_t move) { metaReaderDoInit(&pTbCur->mr, pTbCur->pMeta, META_READER_LOCK); code = tdbTbcOpen(((SMeta *)pTbCur->pMeta)->pUidIdx, (TBC **)&pTbCur->pDbc, NULL); - TSDB_CHECK_CODE(code, lino, _exit); + if (code != 0) { + metaReaderReleaseLock(&pTbCur->mr); + pTbCur->paused = 1; + TSDB_CHECK_CODE(code, lino, _exit); + } if (first) { code = tdbTbcMoveToFirst((TBC *)pTbCur->pDbc); From ecb48dd31707aa81ee75643bb8b64259e96da461 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 14 Aug 2024 21:09:24 +0800 Subject: [PATCH 12/46] fix meta deadlock --- source/dnode/vnode/src/meta/metaQuery.c | 4 ++-- source/libs/executor/src/sysscanoperator.c | 17 ++++++++++++++--- 2 files changed, 16 insertions(+), 5 deletions(-) diff --git a/source/dnode/vnode/src/meta/metaQuery.c b/source/dnode/vnode/src/meta/metaQuery.c index 47ed7743cb..7d7b12bef3 100644 --- a/source/dnode/vnode/src/meta/metaQuery.c +++ b/source/dnode/vnode/src/meta/metaQuery.c @@ -282,7 +282,6 @@ int32_t metaResumeTbCursor(SMTbCursor *pTbCur, int8_t first, int8_t move) { code = tdbTbcOpen(((SMeta *)pTbCur->pMeta)->pUidIdx, (TBC **)&pTbCur->pDbc, NULL); if (code != 0) { metaReaderReleaseLock(&pTbCur->mr); - pTbCur->paused = 1; TSDB_CHECK_CODE(code, lino, _exit); } @@ -1311,7 +1310,8 @@ int32_t metaFilterTableIds(void *pVnode, SMetaFltParam *arg, SArray *pUids) { } TAOS_CHECK_GOTO(metaCreateTagIdxKey(pCursor->suid, pCursor->cid, tagData, nTagData, pCursor->type, - param->reverse ? INT64_MAX : INT64_MIN, &pKey, &nKey), NULL, END); + param->reverse ? INT64_MAX : INT64_MIN, &pKey, &nKey), + NULL, END); int cmp = 0; TAOS_CHECK_GOTO(tdbTbcMoveTo(pCursor->pCur, pKey, nKey, &cmp), 0, END); diff --git a/source/libs/executor/src/sysscanoperator.c b/source/libs/executor/src/sysscanoperator.c index e11ee6b0dc..68be3b4ae8 100644 --- a/source/libs/executor/src/sysscanoperator.c +++ b/source/libs/executor/src/sysscanoperator.c @@ -568,7 +568,12 @@ static SSDataBlock* sysTableScanUserCols(SOperatorInfo* pOperator) { if (pInfo->pCur == NULL) { pInfo->pCur = pAPI->metaFn.openTableMetaCursor(pInfo->readHandle.vnode); } else { - (void)pAPI->metaFn.resumeTableMetaCursor(pInfo->pCur, 0, 0); + code = pAPI->metaFn.resumeTableMetaCursor(pInfo->pCur, 0, 0); + if (code != 0) { + pAPI->metaFn.closeTableMetaCursor(pInfo->pCur); + pInfo->pCur = NULL; + QUERY_CHECK_CODE(code, lino, _end); + } } if (pInfo->pSchema == NULL) { @@ -782,7 +787,8 @@ static SSDataBlock* sysTableScanUserTags(SOperatorInfo* pOperator) { pInfo->pCur = pAPI->metaFn.openTableMetaCursor(pInfo->readHandle.vnode); QUERY_CHECK_NULL(pInfo->pCur, code, lino, _end, terrno); } else { - (void)pAPI->metaFn.resumeTableMetaCursor(pInfo->pCur, 0, 0); + code = pAPI->metaFn.resumeTableMetaCursor(pInfo->pCur, 0, 0); + QUERY_CHECK_CODE(code, lino, _end); } while ((ret = pAPI->metaFn.cursorNext(pInfo->pCur, TSDB_SUPER_TABLE)) == 0) { @@ -1583,7 +1589,12 @@ static SSDataBlock* sysTableBuildUserTables(SOperatorInfo* pOperator) { firstMetaCursor = 1; } if (!firstMetaCursor) { - (void)pAPI->metaFn.resumeTableMetaCursor(pInfo->pCur, 0, 1); + code = pAPI->metaFn.resumeTableMetaCursor(pInfo->pCur, 0, 1); + if (code != 0) { + pAPI->metaFn.closeTableMetaCursor(pInfo->pCur); + pInfo->pCur = NULL; + QUERY_CHECK_CODE(code, lino, _end); + } } blockDataCleanup(pInfo->pRes); From bc2cbea8d12d7f0ec195eec6d60140bc592cb029 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 15 Aug 2024 09:19:34 +0800 Subject: [PATCH 13/46] fix meta deadlock --- source/dnode/vnode/src/meta/metaQuery.c | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/source/dnode/vnode/src/meta/metaQuery.c b/source/dnode/vnode/src/meta/metaQuery.c index 7d7b12bef3..7e9ce5b61f 100644 --- a/source/dnode/vnode/src/meta/metaQuery.c +++ b/source/dnode/vnode/src/meta/metaQuery.c @@ -275,13 +275,12 @@ void metaPauseTbCursor(SMTbCursor *pTbCur) { int32_t metaResumeTbCursor(SMTbCursor *pTbCur, int8_t first, int8_t move) { int32_t code = 0; int32_t lino; - + int8_t locked = 0; if (pTbCur->paused) { metaReaderDoInit(&pTbCur->mr, pTbCur->pMeta, META_READER_LOCK); - + locked = 1; code = tdbTbcOpen(((SMeta *)pTbCur->pMeta)->pUidIdx, (TBC **)&pTbCur->pDbc, NULL); if (code != 0) { - metaReaderReleaseLock(&pTbCur->mr); TSDB_CHECK_CODE(code, lino, _exit); } @@ -307,6 +306,9 @@ int32_t metaResumeTbCursor(SMTbCursor *pTbCur, int8_t first, int8_t move) { } _exit: + if (locked) { + metaReaderReleaseLock(&pTbCur->mr); + } return code; } @@ -794,6 +796,7 @@ void metaCloseSmaCursor(SMSmaCursor *pSmaCur) { if (pSmaCur->pMeta) metaULock(pSmaCur->pMeta); if (pSmaCur->pCur) { (void)tdbTbcClose(pSmaCur->pCur); + pSmaCur->pCur = NULL; tdbFree(pSmaCur->pKey); tdbFree(pSmaCur->pVal); From b9d085764f5e95d4ffda87b05f22e9f8f00bf2f8 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 15 Aug 2024 09:23:19 +0800 Subject: [PATCH 14/46] fix meta deadlock --- source/dnode/vnode/src/meta/metaQuery.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/meta/metaQuery.c b/source/dnode/vnode/src/meta/metaQuery.c index 7e9ce5b61f..bee4727260 100644 --- a/source/dnode/vnode/src/meta/metaQuery.c +++ b/source/dnode/vnode/src/meta/metaQuery.c @@ -306,7 +306,7 @@ int32_t metaResumeTbCursor(SMTbCursor *pTbCur, int8_t first, int8_t move) { } _exit: - if (locked) { + if (code != 0 && locked) { metaReaderReleaseLock(&pTbCur->mr); } return code; From 0c025d54a70b011e195949629e4bc3b750252ea9 Mon Sep 17 00:00:00 2001 From: t_max <1172915550@qq.com> Date: Thu, 15 Aug 2024 10:01:13 +0800 Subject: [PATCH 15/46] docs: the download address uses absolute url --- docs/zh/04-get-started/03-package.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/04-get-started/03-package.md b/docs/zh/04-get-started/03-package.md index 4906a2fcfa..7df41af831 100644 --- a/docs/zh/04-get-started/03-package.md +++ b/docs/zh/04-get-started/03-package.md @@ -146,7 +146,7 @@ Note: 从 3.0.1.7 开始,只提供 TDengine 客户端的 Windows 客户端的 :::info -下载其他组件、最新 Beta 版及之前版本的安装包,请点击[发布历史页面](../../releases/tdengine)。 +下载其他组件、最新 Beta 版及之前版本的安装包,请点击[发布历史页面](https://docs.taosdata.com/releases/tdengine/)。 ::: :::note From 2fec464eb246a881a1a0f80cc3bef053792c206e Mon Sep 17 00:00:00 2001 From: sheyanjie-qq <249478495@qq.com> Date: Wed, 14 Aug 2024 19:46:39 +0800 Subject: [PATCH 16/46] fix link error --- docs/zh/08-develop/01-connect/index.md | 2 +- docs/zh/26-tdinternal/03-storage.md | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/zh/08-develop/01-connect/index.md b/docs/zh/08-develop/01-connect/index.md index 755a9e7f74..160e4cd40c 100644 --- a/docs/zh/08-develop/01-connect/index.md +++ b/docs/zh/08-develop/01-connect/index.md @@ -99,7 +99,7 @@ TDengine 提供了丰富的应用程序开发接口,为了便于用户快速 - **安装前准备** - 安装 Python。新近版本 taospy 包要求 Python 3.6.2+。早期版本 taospy 包要求 Python 3.7+。taos-ws-py 包要求 Python 3.7+。如果系统上还没有 Python 可参考 [Python BeginnersGuide](https://wiki.python.org/moin/BeginnersGuide/Download) 安装。 - 安装 [pip](https://pypi.org/project/pip/)。大部分情况下 Python 的安装包都自带了 pip 工具, 如果没有请参考 [pip documentation](https://pip.pypa.io/en/stable/installation/) 安装。 - - 如果使用原生连接,还需[安装客户端驱动](../#安装客户端驱动)。客户端软件包含了 TDengine 客户端动态链接库(libtaos.so 或 taos.dll) 和 TDengine CLI。 + - 如果使用原生连接,还需[安装客户端驱动](../connect/#安装客户端驱动-taosc)。客户端软件包含了 TDengine 客户端动态链接库(libtaos.so 或 taos.dll) 和 TDengine CLI。 - **使用 pip 安装** - 卸载旧版本 diff --git a/docs/zh/26-tdinternal/03-storage.md b/docs/zh/26-tdinternal/03-storage.md index f65f06e85b..e402babdd7 100644 --- a/docs/zh/26-tdinternal/03-storage.md +++ b/docs/zh/26-tdinternal/03-storage.md @@ -101,7 +101,7 @@ head 文件是时序数据存储文件(data 文件)的 BRIN(Block Range In head 文件中存储了多个 BRIN 记录块及其索引。BRIN 记录块采用列存压缩的方式,这种方式可以大大减少空间占用,同时保持较高的查询性能。BRIN 索引结构如下图所示: -![BRIN 索引结构](./brin.png) +![BRIN 索引结构](./brin.png) #### data 文件 @@ -121,4 +121,4 @@ data 文件是实际存储时序数据的文件。在 data 文件中,时序数 在少表高频的场景下,系统仅维护一个 stt 文件。该文件专门用于存储每次数据落盘后剩余的碎片数据。这样,在下一次数据落盘时,这些碎片数据可以与内存中的新数据合并,形成较大的数据块,随后一并写入 data 文件中。这种机制有效地避免了数据文件的碎片化,确保了数据存储的连续性和高效性。 -对于多表低频的场景,建议配置多个 stt 文件。这种场景下的核心思想是,尽管单张表每次落盘的数据量可能不大,但同一超级表下的所有子表累积的数据量却相当可观。通过合并这些数据,可以生成较大的数据块,从而减少数据块的碎片化。这不仅提升了数据的写入效率,还能显著提高查询性能,因为连续的数据存储更有利于快速的数据检索和访问。 \ No newline at end of file +对于多表低频的场景,建议配置多个 stt 文件。这种场景下的核心思想是,尽管单张表每次落盘的数据量可能不大,但同一超级表下的所有子表累积的数据量却相当可观。通过合并这些数据,可以生成较大的数据块,从而减少数据块的碎片化。这不仅提升了数据的写入效率,还能显著提高查询性能,因为连续的数据存储更有利于快速的数据检索和访问。 From b746a7104690daf9811260804c18b4b693d7e5d7 Mon Sep 17 00:00:00 2001 From: kailixu Date: Thu, 15 Aug 2024 10:23:17 +0800 Subject: [PATCH 17/46] fix(insert): return error when parsing csv file --- source/libs/parser/src/parInsertSql.c | 2 ++ 1 file changed, 2 insertions(+) diff --git a/source/libs/parser/src/parInsertSql.c b/source/libs/parser/src/parInsertSql.c index 70bd43559c..cb94cd42f7 100644 --- a/source/libs/parser/src/parInsertSql.c +++ b/source/libs/parser/src/parInsertSql.c @@ -2238,6 +2238,8 @@ static int32_t parseDataFromFileImpl(SInsertParseContext* pCxt, SVnodeModifyOpSt if (pStmt->insertType != TSDB_QUERY_TYPE_FILE_INSERT) { return buildSyntaxErrMsg(&pCxt->msg, "keyword VALUES or FILE is exclusive", NULL); } + } else { + return buildInvalidOperationMsg(&pCxt->msg, tstrerror(code)); } // just record pTableCxt whose data come from file From 5589799e90a7dce1a7ab013d9536d923961bb209 Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao> Date: Thu, 15 Aug 2024 10:58:53 +0800 Subject: [PATCH 18/46] fix issue --- source/libs/executor/src/tfill.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/libs/executor/src/tfill.c b/source/libs/executor/src/tfill.c index e346946a7a..a7e2ea3429 100644 --- a/source/libs/executor/src/tfill.c +++ b/source/libs/executor/src/tfill.c @@ -570,8 +570,8 @@ int32_t taosCreateFillInfo(TSKEY skey, int32_t numOfFillCols, int32_t numOfNotFi _end: if (code != TSDB_CODE_SUCCESS) { - taosArrayDestroy(pFillInfo->next.pRowVal); - taosArrayDestroy(pFillInfo->prev.pRowVal); + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + pFillInfo = taosDestroyFillInfo(pFillInfo); } (*ppFillInfo) = pFillInfo; return code; From d0059d2d9dfa0bd681bcf3a7dec22764519ddc29 Mon Sep 17 00:00:00 2001 From: kailixu Date: Thu, 15 Aug 2024 13:25:04 +0800 Subject: [PATCH 19/46] enh: support config randErrorScope dynamically --- include/os/os.h | 1 + include/util/tdef.h | 7 +++++++ source/common/src/tglobal.c | 23 +++++++++++++++-------- source/os/src/osFile.c | 17 +++++++++-------- source/os/src/osMemory.c | 12 +++++++----- 5 files changed, 39 insertions(+), 21 deletions(-) diff --git a/include/os/os.h b/include/os/os.h index 08b68f36d4..9e5e9221e4 100644 --- a/include/os/os.h +++ b/include/os/os.h @@ -126,6 +126,7 @@ extern "C" { extern int32_t tsRandErrChance; extern int64_t tsRandErrDivisor; +extern int64_t tsRandErrScope; extern threadlocal bool tsEnableRandErr; #ifdef __cplusplus diff --git a/include/util/tdef.h b/include/util/tdef.h index 890f1d8f95..35c4adab50 100644 --- a/include/util/tdef.h +++ b/include/util/tdef.h @@ -568,6 +568,13 @@ enum { SND_WORKER_TYPE__UNIQUE, }; +enum { + RAND_ERR_MEMORY = 1, + RAND_ERR_FILE = 2, + // RAND_ERR_SCOPE_XXX... = 4, + // ... +}; + #define DEFAULT_HANDLE 0 #define MNODE_HANDLE 1 #define QNODE_HANDLE -1 diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index 46ad263d3d..a013c98b73 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -592,6 +592,9 @@ static int32_t taosAddClientCfg(SConfig *pCfg) { CFG_SCOPE_CLIENT, CFG_DYN_NONE)); TAOS_CHECK_RETURN( cfgAddInt32(pCfg, "metaCacheMaxSize", tsMetaCacheMaxSize, -1, INT32_MAX, CFG_SCOPE_CLIENT, CFG_DYN_CLIENT)); + TAOS_CHECK_RETURN(cfgAddInt32(pCfg, "randErrorChance", tsRandErrChance, 0, 10000, CFG_SCOPE_BOTH, CFG_DYN_NONE)); + TAOS_CHECK_RETURN(cfgAddInt64(pCfg, "randErrorDivisor", tsRandErrDivisor, 1, INT64_MAX, CFG_SCOPE_BOTH, CFG_DYN_BOTH)); + TAOS_CHECK_RETURN(cfgAddInt64(pCfg, "randErrorScope", tsRandErrScope, 0, INT64_MAX, CFG_SCOPE_BOTH, CFG_DYN_BOTH)); tsNumOfRpcThreads = tsNumOfCores / 2; tsNumOfRpcThreads = TRANGE(tsNumOfRpcThreads, 2, TSDB_MAX_RPC_THREADS); @@ -774,8 +777,6 @@ static int32_t taosAddServerCfg(SConfig *pCfg) { TAOS_CHECK_RETURN(cfgAddInt32(pCfg, "transPullupInterval", tsTransPullupInterval, 1, 10000, CFG_SCOPE_SERVER, CFG_DYN_ENT_SERVER)); TAOS_CHECK_RETURN(cfgAddInt32(pCfg, "compactPullupInterval", tsCompactPullupInterval, 1, 10000, CFG_SCOPE_SERVER, CFG_DYN_ENT_SERVER)); TAOS_CHECK_RETURN(cfgAddInt32(pCfg, "mqRebalanceInterval", tsMqRebalanceInterval, 1, 10000, CFG_SCOPE_SERVER, CFG_DYN_ENT_SERVER)); - TAOS_CHECK_RETURN(cfgAddInt32(pCfg, "randErrorChance", tsRandErrChance, 0, 10000, CFG_SCOPE_BOTH, CFG_DYN_NONE)); - TAOS_CHECK_RETURN(cfgAddInt64(pCfg, "randErrorDivisor", tsRandErrDivisor, 1, INT64_MAX, CFG_SCOPE_BOTH, CFG_DYN_SERVER)); TAOS_CHECK_RETURN(cfgAddInt32(pCfg, "ttlUnit", tsTtlUnit, 1, 86400 * 365, CFG_SCOPE_SERVER, CFG_DYN_NONE)); TAOS_CHECK_RETURN(cfgAddInt32(pCfg, "ttlPushInterval", tsTtlPushIntervalSec, 1, 100000, CFG_SCOPE_SERVER, CFG_DYN_ENT_SERVER)); @@ -1210,6 +1211,15 @@ static int32_t taosSetClientCfg(SConfig *pCfg) { TAOS_CHECK_GET_CFG_ITEM(pCfg, pItem, "metaCacheMaxSize"); tsMetaCacheMaxSize = pItem->i32; + TAOS_CHECK_GET_CFG_ITEM(pCfg, pItem, "randErrorChance"); + tsRandErrChance = pItem->i32; + + TAOS_CHECK_GET_CFG_ITEM(pCfg, pItem, "randErrorDivisor"); + tsRandErrDivisor = pItem->i64; + + TAOS_CHECK_GET_CFG_ITEM(pCfg, pItem, "randErrorScope"); + tsRandErrScope = pItem->i64; + TAOS_CHECK_GET_CFG_ITEM(pCfg, pItem, "countAlwaysReturnValue"); tsCountAlwaysReturnValue = pItem->i32; @@ -1466,12 +1476,6 @@ static int32_t taosSetServerCfg(SConfig *pCfg) { TAOS_CHECK_GET_CFG_ITEM(pCfg, pItem, "mqRebalanceInterval"); tsMqRebalanceInterval = pItem->i32; - TAOS_CHECK_GET_CFG_ITEM(pCfg, pItem, "randErrorChance"); - tsRandErrChance = pItem->i32; - - TAOS_CHECK_GET_CFG_ITEM(pCfg, pItem, "randErrorDivisor"); - tsRandErrDivisor = pItem->i64; - TAOS_CHECK_GET_CFG_ITEM(pCfg, pItem, "ttlUnit"); tsTtlUnit = pItem->i32; @@ -1927,6 +1931,7 @@ static int32_t taosCfgDynamicOptionsForServer(SConfig *pCfg, const char *name) { {"mndSdbWriteDelta", &tsMndSdbWriteDelta}, {"minDiskFreeSize", &tsMinDiskFreeSize}, {"randErrorDivisor", &tsRandErrDivisor}, + {"randErrorScope", &tsRandErrScope}, {"cacheLazyLoadThreshold", &tsCacheLazyLoadThreshold}, {"checkpointInterval", &tsStreamCheckpointInterval}, @@ -2205,6 +2210,8 @@ static int32_t taosCfgDynamicOptionsForClient(SConfig *pCfg, const char *name) { {"queryPlannerTrace", &tsQueryPlannerTrace}, {"queryNodeChunkSize", &tsQueryNodeChunkSize}, {"queryUseNodeAllocator", &tsQueryUseNodeAllocator}, + {"randErrorDivisor", &tsRandErrDivisor}, + {"randErrorScope", &tsRandErrScope}, {"smlDot2Underline", &tsSmlDot2Underline}, {"shellActivityTimer", &tsShellActivityTimer}, {"useAdapter", &tsUseAdapter}, diff --git a/source/os/src/osFile.c b/source/os/src/osFile.c index b8160a14b7..a5df4f63f3 100644 --- a/source/os/src/osFile.c +++ b/source/os/src/osFile.c @@ -15,6 +15,7 @@ #define ALLOW_FORBID_FUNC #include "os.h" #include "osSemaphore.h" +#include "tdef.h" #include "zlib.h" #ifdef WINDOWS @@ -65,14 +66,14 @@ typedef struct TdFile { #define FILE_WITH_LOCK 1 #ifdef BUILD_WITH_RAND_ERR -#define STUB_RAND_IO_ERR(ret) \ - if (tsEnableRandErr) { \ - uint32_t r = taosRand() % tsRandErrDivisor; \ - if ((r + 1) <= tsRandErrChance) { \ - errno = EIO; \ - terrno = TAOS_SYSTEM_ERROR(errno); \ - return (ret); \ - } \ +#define STUB_RAND_IO_ERR(ret) \ + if (tsEnableRandErr && (tsRandErrScope & RAND_ERR_FILE)) { \ + uint32_t r = taosRand() % tsRandErrDivisor; \ + if ((r + 1) <= tsRandErrChance) { \ + errno = EIO; \ + terrno = TAOS_SYSTEM_ERROR(errno); \ + return (ret); \ + } \ } #else #define STUB_RAND_IO_ERR(ret) diff --git a/source/os/src/osMemory.c b/source/os/src/osMemory.c index 297b17b957..7a5a547354 100644 --- a/source/os/src/osMemory.c +++ b/source/os/src/osMemory.c @@ -20,9 +20,11 @@ #include #endif #include "os.h" +#include "tdef.h" int32_t tsRandErrChance = 1; int64_t tsRandErrDivisor = 10001; +int64_t tsRandErrScope = (RAND_ERR_MEMORY | RAND_ERR_FILE); threadlocal bool tsEnableRandErr = 0; #if defined(USE_TD_MEMORY) || defined(USE_ADDR2LINE) @@ -272,7 +274,7 @@ void *taosMemoryMalloc(int64_t size) { #else #ifdef BUILD_WITH_RAND_ERR - if (tsEnableRandErr) { + if (tsEnableRandErr && (tsRandErrScope & RAND_ERR_MEMORY)) { uint32_t r = taosRand() % tsRandErrDivisor; if ((r + 1) <= tsRandErrChance) { terrno = TSDB_CODE_OUT_OF_MEMORY; @@ -302,7 +304,7 @@ void *taosMemoryCalloc(int64_t num, int64_t size) { return (char *)tmp + sizeof(TdMemoryInfo); #else #ifdef BUILD_WITH_RAND_ERR - if (tsEnableRandErr) { + if (tsEnableRandErr && (tsRandErrScope & RAND_ERR_MEMORY)) { uint32_t r = taosRand() % tsRandErrDivisor; if ((r + 1) <= tsRandErrChance) { terrno = TSDB_CODE_OUT_OF_MEMORY; @@ -342,7 +344,7 @@ void *taosMemoryRealloc(void *ptr, int64_t size) { return (char *)tmp + sizeof(TdMemoryInfo); #else #ifdef BUILD_WITH_RAND_ERR - if (tsEnableRandErr) { + if (tsEnableRandErr && (tsRandErrScope & RAND_ERR_MEMORY)) { uint32_t r = taosRand() % tsRandErrDivisor; if ((r + 1) <= tsRandErrChance) { terrno = TSDB_CODE_OUT_OF_MEMORY; @@ -377,7 +379,7 @@ char *taosStrdup(const char *ptr) { return (char *)tmp + sizeof(TdMemoryInfo); #else #ifdef BUILD_WITH_RAND_ERR - if (tsEnableRandErr) { + if (tsEnableRandErr && (tsRandErrScope & RAND_ERR_MEMORY)) { uint32_t r = taosRand() % tsRandErrDivisor; if ((r + 1) <= tsRandErrChance) { terrno = TSDB_CODE_OUT_OF_MEMORY; @@ -443,7 +445,7 @@ void *taosMemoryMallocAlign(uint32_t alignment, int64_t size) { #else #if defined(LINUX) #ifdef BUILD_WITH_RAND_ERR - if (tsEnableRandErr) { + if (tsEnableRandErr && (tsRandErrScope & RAND_ERR_MEMORY)) { uint32_t r = taosRand() % tsRandErrDivisor; if ((r + 1) <= tsRandErrChance) { terrno = TSDB_CODE_OUT_OF_MEMORY; From ef5f69e3cf97eb77fe7cb46ff478c16768a6d0b3 Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao> Date: Thu, 15 Aug 2024 14:13:08 +0800 Subject: [PATCH 20/46] fix issue --- source/libs/executor/src/timesliceoperator.c | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/source/libs/executor/src/timesliceoperator.c b/source/libs/executor/src/timesliceoperator.c index 8cd547e333..b14f4f0266 100644 --- a/source/libs/executor/src/timesliceoperator.c +++ b/source/libs/executor/src/timesliceoperator.c @@ -1249,10 +1249,11 @@ void destroyTimeSliceOperatorInfo(void* param) { } cleanupExprSupp(&pInfo->scalarSup); - - for (int32_t i = 0; i < pInfo->pFillColInfo->numOfFillExpr; ++i) { - taosVariantDestroy(&pInfo->pFillColInfo[i].fillVal); + if (pInfo->pFillColInfo != NULL) { + for (int32_t i = 0; i < pInfo->pFillColInfo->numOfFillExpr; ++i) { + taosVariantDestroy(&pInfo->pFillColInfo[i].fillVal); + } + taosMemoryFree(pInfo->pFillColInfo); } - taosMemoryFree(pInfo->pFillColInfo); taosMemoryFreeClear(param); } From 41d8c20117051c3dc069afa82bc6d59ce5e27f0e Mon Sep 17 00:00:00 2001 From: Shungang Li Date: Thu, 15 Aug 2024 14:49:48 +0800 Subject: [PATCH 21/46] fix: add miss hostname for ep funcs --- source/common/src/cos.c | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/source/common/src/cos.c b/source/common/src/cos.c index db0dadbc46..aa587bf07b 100644 --- a/source/common/src/cos.c +++ b/source/common/src/cos.c @@ -974,7 +974,7 @@ int32_t s3PutObjectFromFile2ByEp(const char *file, const char *object_name, int8 data.totalContentLength = data.totalOriginalContentLength = data.contentLength = data.originalContentLength = contentLength; - S3BucketContext bucketContext = {0, + S3BucketContext bucketContext = {tsS3Hostname[epIndex], tsS3BucketName[epIndex], protocolG, uriStyleG, @@ -1057,7 +1057,7 @@ static int32_t s3PutObjectFromFileOffsetByEp(const char *file, const char *objec data.totalContentLength = data.totalOriginalContentLength = data.contentLength = data.originalContentLength = contentLength; - S3BucketContext bucketContext = {0, + S3BucketContext bucketContext = {tsS3Hostname[epIndex], tsS3BucketName[epIndex], protocolG, uriStyleG, @@ -1153,7 +1153,7 @@ static void s3FreeObjectKey(void *pItem) { } static SArray *getListByPrefixByEp(const char *prefix, int8_t epIndex) { - S3BucketContext bucketContext = {0, + S3BucketContext bucketContext = {tsS3Hostname[epIndex], tsS3BucketName[epIndex], protocolG, uriStyleG, @@ -1221,7 +1221,7 @@ static SArray *getListByPrefix(const char *prefix) { static int32_t s3DeleteObjectsByEp(const char *object_name[], int nobject, int8_t epIndex) { int32_t code = 0; - S3BucketContext bucketContext = {0, + S3BucketContext bucketContext = {tsS3Hostname[epIndex], tsS3BucketName[epIndex], protocolG, uriStyleG, @@ -1297,7 +1297,7 @@ static int32_t s3GetObjectBlockByEp(const char *object_name, int64_t offset, int int64_t ifModifiedSince = -1, ifNotModifiedSince = -1; const char *ifMatch = 0, *ifNotMatch = 0; - S3BucketContext bucketContext = {0, + S3BucketContext bucketContext = {tsS3Hostname[epIndex], tsS3BucketName[epIndex], protocolG, uriStyleG, @@ -1370,7 +1370,7 @@ static int32_t s3GetObjectToFileByEp(const char *object_name, const char *fileNa int64_t ifModifiedSince = -1, ifNotModifiedSince = -1; const char *ifMatch = 0, *ifNotMatch = 0; - S3BucketContext bucketContext = {0, + S3BucketContext bucketContext = {tsS3Hostname[epIndex], tsS3BucketName[epIndex], protocolG, uriStyleG, @@ -1447,7 +1447,7 @@ static long s3SizeByEp(const char *object_name, int8_t epIndex) { long size = 0; int status = 0; - S3BucketContext bucketContext = {0, + S3BucketContext bucketContext = {tsS3Hostname[epIndex], tsS3BucketName[epIndex], protocolG, uriStyleG, From fd53940b33407b0a6c146893bede76ec7352fb3e Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 15 Aug 2024 14:55:29 +0800 Subject: [PATCH 22/46] fix(query): release lock when error occurs. --- source/libs/executor/src/scanoperator.c | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 54c1c78844..b25ac8e928 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -440,7 +440,10 @@ static int32_t loadDataBlock(SOperatorInfo* pOperator, STableScanBase* pTableSca // try to filter data block according to current results code = doDynamicPruneDataBlock(pOperator, pBlockInfo, status); - QUERY_CHECK_CODE(code, lino, _end); + if (code) { + pAPI->tsdReader.tsdReaderReleaseDataBlock(pTableScanInfo->dataReader); + QUERY_CHECK_CODE(code, lino, _end); + } if (*status == FUNC_DATA_REQUIRED_NOT_LOAD) { qDebug("%s data block skipped due to dynamic prune, brange:%" PRId64 "-%" PRId64 ", rows:%" PRId64, From 2e51d5409ab504efc0dce859d679542e9aa5f1ec Mon Sep 17 00:00:00 2001 From: sima Date: Thu, 15 Aug 2024 14:34:41 +0800 Subject: [PATCH 23/46] fix:[TD-31469] Fix trim sql syntax error. --- source/libs/parser/inc/sql.y | 3 +- source/libs/parser/src/sql.c | 1808 +++++++++++++++++----------------- 2 files changed, 909 insertions(+), 902 deletions(-) diff --git a/source/libs/parser/inc/sql.y b/source/libs/parser/inc/sql.y index 05c1c95aad..9228e16ff9 100644 --- a/source/libs/parser/inc/sql.y +++ b/source/libs/parser/inc/sql.y @@ -1195,6 +1195,8 @@ function_expression(A) ::= TRIM(B) NK_LP expr_or_subquery(C) NK_RP(D). { A = createRawExprNodeExt(pCxt, &B, &D, createTrimFunctionNode(pCxt, releaseRawExprNode(pCxt, C), TRIM_TYPE_BOTH)); } function_expression(A) ::= TRIM(B) NK_LP trim_specification_type(C) FROM expr_or_subquery(D) NK_RP(E). { A = createRawExprNodeExt(pCxt, &B, &E, createTrimFunctionNode(pCxt, releaseRawExprNode(pCxt, D), C)); } +function_expression(A) ::= + TRIM(B) NK_LP expr_or_subquery(C) FROM expr_or_subquery(D) NK_RP(E). { A = createRawExprNodeExt(pCxt, &B, &E, createTrimFunctionNodeExt(pCxt, releaseRawExprNode(pCxt, C), releaseRawExprNode(pCxt, D), TRIM_TYPE_BOTH)); } function_expression(A) ::= TRIM(B) NK_LP trim_specification_type(C) expr_or_subquery(D) FROM expr_or_subquery(E) NK_RP(F). { A = createRawExprNodeExt(pCxt, &B, &F, createTrimFunctionNodeExt(pCxt, releaseRawExprNode(pCxt, D), releaseRawExprNode(pCxt, E), C)); } function_expression(A) ::= @@ -1217,7 +1219,6 @@ substr_func(A) ::= SUBSTRING(B). %type trim_specification_type ETrimType %destructor trim_specification_type { } -trim_specification_type(A) ::= . { A = TRIM_TYPE_BOTH; } trim_specification_type(A) ::= BOTH. { A = TRIM_TYPE_BOTH; } trim_specification_type(A) ::= TRAILING. { A = TRIM_TYPE_TRAILING; } trim_specification_type(A) ::= LEADING. { A = TRIM_TYPE_LEADING; } diff --git a/source/libs/parser/src/sql.c b/source/libs/parser/src/sql.c index 0492288fd6..4ff7510e92 100644 --- a/source/libs/parser/src/sql.c +++ b/source/libs/parser/src/sql.c @@ -536,18 +536,18 @@ typedef union { #define ParseCTX_FETCH #define ParseCTX_STORE #define YYFALLBACK 1 -#define YYNSTATE 1004 +#define YYNSTATE 1006 #define YYNRULE 770 #define YYNRULE_WITH_ACTION 770 #define YYNTOKEN 385 -#define YY_MAX_SHIFT 1003 -#define YY_MIN_SHIFTREDUCE 1487 -#define YY_MAX_SHIFTREDUCE 2256 -#define YY_ERROR_ACTION 2257 -#define YY_ACCEPT_ACTION 2258 -#define YY_NO_ACTION 2259 -#define YY_MIN_REDUCE 2260 -#define YY_MAX_REDUCE 3029 +#define YY_MAX_SHIFT 1005 +#define YY_MIN_SHIFTREDUCE 1490 +#define YY_MAX_SHIFTREDUCE 2259 +#define YY_ERROR_ACTION 2260 +#define YY_ACCEPT_ACTION 2261 +#define YY_NO_ACTION 2262 +#define YY_MIN_REDUCE 2263 +#define YY_MAX_REDUCE 3032 #define YY_MIN_DSTRCTR 386 #define YY_MAX_DSTRCTR 566 /************* End control #defines *******************************************/ @@ -632,402 +632,404 @@ typedef union { ** yy_default[] Default action for each state. ** *********** Begin parsing tables **********************************************/ -#define YY_ACTTAB_COUNT (3934) +#define YY_ACTTAB_COUNT (3956) static const YYACTIONTYPE yy_action[] = { - /* 0 */ 819, 668, 3000, 676, 669, 2308, 669, 2308, 2995, 2472, - /* 10 */ 2995, 954, 57, 55, 2427, 56, 54, 53, 52, 51, - /* 20 */ 497, 2261, 1979, 2623, 2004, 462, 818, 228, 769, 2999, - /* 30 */ 217, 2996, 820, 2996, 2998, 501, 1977, 490, 2081, 2352, - /* 40 */ 217, 2790, 148, 2620, 880, 147, 146, 145, 144, 143, - /* 50 */ 142, 141, 140, 139, 580, 2592, 834, 2596, 50, 49, - /* 60 */ 831, 167, 56, 54, 53, 52, 51, 2596, 2076, 148, - /* 70 */ 765, 863, 147, 146, 145, 144, 143, 142, 141, 140, - /* 80 */ 139, 893, 1985, 50, 49, 2808, 755, 56, 54, 53, - /* 90 */ 52, 51, 50, 49, 2695, 2008, 56, 54, 53, 52, - /* 100 */ 51, 2755, 749, 875, 753, 751, 298, 297, 244, 688, - /* 110 */ 678, 2662, 1000, 893, 691, 58, 971, 970, 969, 968, - /* 120 */ 526, 248, 967, 966, 172, 961, 960, 959, 958, 957, - /* 130 */ 956, 955, 171, 949, 948, 947, 525, 524, 944, 943, - /* 140 */ 942, 208, 207, 941, 521, 940, 939, 938, 2789, 42, - /* 150 */ 350, 2836, 2084, 2085, 808, 131, 2791, 879, 2793, 2794, - /* 160 */ 874, 2459, 2748, 862, 898, 514, 2217, 104, 2610, 210, - /* 170 */ 689, 2897, 103, 50, 49, 492, 2893, 56, 54, 53, - /* 180 */ 52, 51, 2790, 2808, 204, 2905, 830, 768, 159, 829, - /* 190 */ 3000, 2040, 2050, 341, 342, 229, 2995, 876, 340, 766, - /* 200 */ 2060, 2083, 2086, 2944, 2260, 50, 49, 2007, 60, 56, - /* 210 */ 54, 53, 52, 51, 818, 228, 1980, 2170, 1978, 2996, - /* 220 */ 820, 2177, 9, 861, 503, 503, 2808, 2004, 157, 156, - /* 230 */ 155, 154, 153, 152, 151, 150, 149, 898, 898, 102, - /* 240 */ 454, 893, 2755, 2258, 875, 935, 184, 183, 932, 931, - /* 250 */ 930, 181, 1983, 1984, 2037, 807, 2039, 2042, 2043, 2044, - /* 260 */ 2045, 2046, 2047, 2048, 2049, 871, 864, 2004, 224, 896, - /* 270 */ 895, 2067, 2068, 2070, 2071, 2072, 2075, 2077, 2078, 2079, - /* 280 */ 2080, 2082, 2, 57, 55, 2173, 684, 2790, 673, 2789, - /* 290 */ 2111, 497, 2836, 1979, 670, 687, 131, 2791, 879, 2793, - /* 300 */ 2794, 874, 873, 2241, 862, 898, 742, 1977, 169, 2081, - /* 310 */ 178, 2868, 2897, 2189, 831, 167, 492, 2893, 50, 49, - /* 320 */ 893, 756, 56, 54, 53, 52, 51, 3000, 571, 2009, - /* 330 */ 2623, 2808, 529, 2148, 72, 2995, 60, 528, 219, 2076, - /* 340 */ 299, 2148, 863, 33, 499, 2004, 19, 2755, 665, 875, - /* 350 */ 2620, 880, 43, 1985, 2999, 2112, 745, 663, 2996, 2997, - /* 360 */ 659, 655, 72, 739, 737, 2749, 57, 55, 72, 264, - /* 370 */ 296, 894, 2468, 671, 497, 2316, 1979, 2206, 633, 631, - /* 380 */ 332, 434, 769, 1000, 242, 76, 15, 2565, 785, 1532, - /* 390 */ 1977, 158, 2081, 202, 2789, 327, 2995, 2836, 2041, 714, - /* 400 */ 334, 429, 2791, 879, 2793, 2794, 874, 872, 1539, 862, - /* 410 */ 898, 854, 2862, 2355, 3001, 228, 82, 124, 12, 2996, - /* 420 */ 820, 81, 2076, 2084, 2085, 863, 2218, 518, 503, 19, - /* 430 */ 2521, 2523, 2117, 1534, 1537, 1538, 1985, 833, 197, 2905, - /* 440 */ 2906, 898, 165, 2910, 801, 800, 2204, 2205, 2207, 2208, - /* 450 */ 2209, 41, 494, 2106, 2107, 2108, 2109, 2110, 2114, 2115, - /* 460 */ 2116, 2038, 2040, 2050, 771, 2662, 1000, 2005, 562, 15, - /* 470 */ 561, 61, 2083, 2086, 2917, 2145, 2146, 2147, 2917, 2917, - /* 480 */ 2917, 2917, 2917, 2145, 2146, 2147, 2765, 1980, 1985, 1978, - /* 490 */ 735, 734, 733, 194, 861, 192, 72, 725, 164, 729, - /* 500 */ 809, 2401, 560, 728, 516, 2470, 2084, 2085, 727, 732, - /* 510 */ 472, 471, 904, 1650, 726, 2769, 1825, 1826, 470, 722, - /* 520 */ 721, 720, 2169, 1983, 1984, 2037, 334, 2039, 2042, 2043, - /* 530 */ 2044, 2045, 2046, 2047, 2048, 2049, 871, 864, 2452, 441, - /* 540 */ 896, 895, 2067, 2068, 182, 2040, 2050, 2075, 2077, 2078, - /* 550 */ 2079, 2080, 2082, 2, 334, 2083, 2086, 685, 608, 1652, - /* 560 */ 334, 469, 468, 607, 1712, 2771, 2773, 493, 302, 2148, - /* 570 */ 1980, 606, 1978, 221, 831, 167, 2445, 861, 898, 1703, - /* 580 */ 927, 926, 925, 1707, 924, 1709, 1710, 923, 920, 2515, - /* 590 */ 1718, 917, 1720, 1721, 914, 911, 908, 50, 49, 1751, - /* 600 */ 1752, 56, 54, 53, 52, 51, 1983, 1984, 2037, 173, - /* 610 */ 2039, 2042, 2043, 2044, 2045, 2046, 2047, 2048, 2049, 871, - /* 620 */ 864, 686, 2616, 896, 895, 2067, 2068, 64, 2912, 2623, - /* 630 */ 2075, 2077, 2078, 2079, 2080, 2082, 2, 12, 57, 55, - /* 640 */ 509, 467, 466, 391, 716, 2790, 497, 301, 1979, 2621, - /* 650 */ 880, 300, 804, 1906, 1907, 1712, 2909, 831, 167, 195, - /* 660 */ 876, 2272, 1977, 334, 2081, 718, 903, 902, 901, 717, - /* 670 */ 1703, 927, 926, 925, 1707, 924, 1709, 1710, 870, 869, - /* 680 */ 2790, 1718, 868, 1720, 1721, 867, 911, 908, 334, 2808, - /* 690 */ 1559, 2283, 1558, 2005, 2076, 876, 2548, 863, 137, 2905, - /* 700 */ 2906, 19, 165, 2910, 2671, 2755, 163, 875, 1985, 217, - /* 710 */ 2917, 2145, 2146, 2147, 2917, 2917, 2917, 2917, 2917, 894, - /* 720 */ 2468, 57, 55, 2087, 2808, 113, 548, 2765, 1560, 497, - /* 730 */ 449, 1979, 2282, 477, 2037, 757, 2597, 2092, 1000, 520, - /* 740 */ 2755, 15, 875, 2004, 134, 1977, 1559, 2081, 1558, 1562, - /* 750 */ 1563, 2755, 2789, 516, 2470, 2836, 2769, 586, 2592, 416, - /* 760 */ 2791, 879, 2793, 2794, 874, 723, 31, 862, 898, 810, - /* 770 */ 805, 798, 794, 894, 2468, 617, 2592, 2076, 2084, 2085, - /* 780 */ 863, 198, 2905, 2906, 1560, 165, 2910, 2789, 1643, 2708, - /* 790 */ 2836, 1985, 2755, 158, 131, 2791, 879, 2793, 2794, 874, - /* 800 */ 500, 719, 862, 898, 479, 2670, 2771, 2774, 2872, 191, - /* 810 */ 2897, 246, 894, 2468, 492, 2893, 94, 2040, 2050, 898, - /* 820 */ 2473, 1000, 1948, 937, 58, 1852, 1853, 2083, 2086, 251, - /* 830 */ 50, 49, 233, 170, 56, 54, 53, 52, 51, 735, - /* 840 */ 734, 733, 1980, 2460, 1978, 785, 725, 164, 729, 861, - /* 850 */ 928, 610, 728, 2995, 508, 507, 46, 727, 732, 472, - /* 860 */ 471, 2084, 2085, 726, 552, 2709, 609, 470, 722, 721, - /* 870 */ 720, 3001, 228, 2281, 1851, 1854, 2996, 820, 1983, 1984, - /* 880 */ 2037, 110, 2039, 2042, 2043, 2044, 2045, 2046, 2047, 2048, - /* 890 */ 2049, 871, 864, 554, 550, 896, 895, 2067, 2068, 465, - /* 900 */ 2040, 2050, 2075, 2077, 2078, 2079, 2080, 2082, 2, 2463, - /* 910 */ 2083, 2086, 50, 49, 2041, 174, 56, 54, 53, 52, - /* 920 */ 51, 785, 266, 894, 2468, 1980, 671, 1978, 2316, 2995, - /* 930 */ 2008, 519, 861, 2755, 894, 2468, 2528, 12, 512, 10, - /* 940 */ 191, 564, 2007, 65, 460, 2528, 563, 3001, 228, 2008, - /* 950 */ 762, 2473, 2996, 820, 568, 2526, 894, 2468, 384, 523, - /* 960 */ 522, 1983, 1984, 2037, 838, 2039, 2042, 2043, 2044, 2045, - /* 970 */ 2046, 2047, 2048, 2049, 871, 864, 569, 2038, 896, 895, - /* 980 */ 2067, 2068, 2528, 1986, 724, 2075, 2077, 2078, 2079, 2080, - /* 990 */ 2082, 2, 57, 55, 1979, 2790, 2528, 785, 1896, 1662, - /* 1000 */ 497, 846, 1979, 784, 488, 2995, 785, 1641, 1977, 1539, - /* 1010 */ 834, 894, 2468, 1661, 2995, 2526, 1977, 855, 2081, 2869, - /* 1020 */ 2057, 894, 2468, 3001, 228, 894, 2468, 37, 2996, 820, - /* 1030 */ 2248, 588, 3001, 228, 819, 1537, 1538, 2996, 820, 2808, - /* 1040 */ 2280, 602, 2995, 168, 2790, 690, 2868, 391, 2076, 2522, - /* 1050 */ 2523, 863, 894, 2468, 1985, 2755, 857, 875, 2869, 876, - /* 1060 */ 818, 228, 1985, 50, 49, 2996, 820, 56, 54, 53, - /* 1070 */ 52, 51, 603, 50, 49, 57, 55, 56, 54, 53, - /* 1080 */ 52, 51, 866, 497, 1000, 1979, 710, 709, 2808, 638, - /* 1090 */ 712, 711, 1000, 865, 123, 58, 894, 2468, 2530, 1977, - /* 1100 */ 2755, 2081, 2789, 223, 2755, 2836, 875, 478, 2670, 131, - /* 1110 */ 2791, 879, 2793, 2794, 874, 332, 604, 862, 898, 2279, - /* 1120 */ 1947, 2461, 231, 210, 627, 2897, 53, 52, 51, 492, - /* 1130 */ 2893, 2076, 2084, 2085, 863, 313, 38, 2150, 2151, 2152, - /* 1140 */ 2153, 2154, 50, 49, 70, 1985, 56, 54, 53, 52, - /* 1150 */ 51, 2789, 511, 510, 2836, 782, 2247, 2945, 196, 2791, - /* 1160 */ 879, 2793, 2794, 874, 2009, 1989, 862, 898, 2528, 894, - /* 1170 */ 2468, 2040, 2050, 894, 2468, 1000, 502, 2004, 58, 2755, - /* 1180 */ 191, 2083, 2086, 2009, 894, 2468, 115, 2526, 1980, 2465, - /* 1190 */ 1978, 2474, 253, 304, 894, 2468, 1980, 2457, 1978, 40, - /* 1200 */ 894, 2468, 2676, 861, 312, 50, 49, 786, 2955, 56, - /* 1210 */ 54, 53, 52, 51, 837, 2084, 2085, 626, 252, 2912, - /* 1220 */ 345, 2455, 50, 49, 1983, 1984, 56, 54, 53, 52, - /* 1230 */ 51, 624, 1983, 1984, 2037, 2278, 2039, 2042, 2043, 2044, - /* 1240 */ 2045, 2046, 2047, 2048, 2049, 871, 864, 2908, 135, 896, - /* 1250 */ 895, 2067, 2068, 636, 2040, 2050, 2075, 2077, 2078, 2079, - /* 1260 */ 2080, 2082, 2, 303, 2083, 2086, 50, 49, 1666, 1541, - /* 1270 */ 56, 54, 53, 52, 51, 2003, 284, 894, 2468, 1980, - /* 1280 */ 2444, 1978, 1665, 50, 49, 2442, 861, 56, 54, 53, - /* 1290 */ 52, 51, 203, 894, 2468, 2755, 390, 851, 894, 2468, - /* 1300 */ 192, 708, 704, 700, 696, 44, 283, 2441, 731, 730, - /* 1310 */ 2471, 894, 2468, 352, 2999, 1983, 1984, 2037, 887, 2039, - /* 1320 */ 2042, 2043, 2044, 2045, 2046, 2047, 2048, 2049, 871, 864, - /* 1330 */ 2061, 888, 896, 895, 2067, 2068, 14, 13, 2113, 2075, - /* 1340 */ 2077, 2078, 2079, 2080, 2082, 2, 57, 55, 2041, 2277, - /* 1350 */ 2198, 438, 111, 2002, 497, 767, 1979, 281, 640, 937, - /* 1360 */ 615, 50, 49, 464, 2199, 56, 54, 53, 52, 51, - /* 1370 */ 1977, 635, 2081, 1003, 50, 49, 894, 2468, 56, 54, - /* 1380 */ 53, 52, 51, 894, 2468, 2528, 398, 594, 2276, 637, - /* 1390 */ 2505, 2058, 388, 517, 439, 596, 892, 951, 965, 963, - /* 1400 */ 45, 2275, 2076, 380, 2526, 863, 574, 991, 218, 2755, - /* 1410 */ 2197, 2038, 2912, 2699, 2274, 2118, 1985, 987, 983, 979, - /* 1420 */ 975, 2008, 383, 2004, 269, 935, 184, 183, 932, 931, - /* 1430 */ 930, 181, 110, 280, 39, 174, 929, 271, 278, 2519, - /* 1440 */ 2907, 543, 933, 276, 682, 2519, 1000, 463, 2755, 15, - /* 1450 */ 1988, 2125, 935, 184, 183, 932, 931, 930, 181, 582, - /* 1460 */ 2464, 2755, 268, 2271, 2270, 235, 953, 536, 130, 2269, - /* 1470 */ 2268, 50, 49, 356, 2755, 56, 54, 53, 52, 51, - /* 1480 */ 309, 770, 2056, 835, 2267, 823, 2084, 2085, 2266, 934, - /* 1490 */ 2603, 2582, 2519, 623, 622, 621, 620, 619, 614, 613, - /* 1500 */ 612, 611, 446, 813, 847, 601, 600, 599, 598, 597, - /* 1510 */ 591, 590, 589, 2265, 584, 583, 461, 2264, 1904, 2528, - /* 1520 */ 575, 1813, 1814, 2755, 2755, 2040, 2050, 1832, 114, 2755, - /* 1530 */ 2755, 2288, 993, 2250, 2251, 2083, 2086, 785, 2527, 785, - /* 1540 */ 2575, 2263, 792, 289, 2755, 2995, 287, 2995, 2755, 354, - /* 1550 */ 1980, 853, 1978, 3, 337, 88, 718, 861, 222, 336, - /* 1560 */ 717, 160, 1880, 3001, 228, 3001, 228, 63, 2996, 820, - /* 1570 */ 2996, 820, 2159, 2755, 759, 100, 758, 2755, 306, 2055, - /* 1580 */ 291, 1645, 293, 290, 578, 292, 1983, 1984, 2037, 1623, - /* 1590 */ 2039, 2042, 2043, 2044, 2045, 2046, 2047, 2048, 2049, 871, - /* 1600 */ 864, 2755, 2339, 896, 895, 2067, 2068, 59, 826, 101, - /* 1610 */ 2075, 2077, 2078, 2079, 2080, 2082, 2, 193, 59, 211, - /* 1620 */ 295, 2790, 404, 294, 736, 2193, 397, 1646, 2337, 2328, - /* 1630 */ 2326, 2776, 1991, 182, 339, 1624, 876, 87, 2318, 59, - /* 1640 */ 822, 402, 86, 14, 13, 85, 74, 2790, 1596, 59, - /* 1650 */ 738, 740, 743, 860, 1987, 2009, 440, 2038, 59, 59, - /* 1660 */ 359, 358, 876, 2472, 2952, 2808, 2273, 87, 262, 650, - /* 1670 */ 648, 645, 643, 361, 360, 363, 362, 179, 365, 364, - /* 1680 */ 129, 2755, 126, 875, 160, 182, 367, 366, 369, 368, - /* 1690 */ 2203, 2808, 371, 370, 1597, 2778, 373, 372, 375, 374, - /* 1700 */ 945, 2202, 318, 377, 376, 379, 378, 2755, 84, 875, - /* 1710 */ 906, 180, 182, 2402, 72, 161, 836, 343, 179, 202, - /* 1720 */ 843, 2958, 2059, 1615, 946, 2320, 328, 802, 2789, 2119, - /* 1730 */ 320, 2836, 2062, 832, 2809, 131, 2791, 879, 2793, 2794, - /* 1740 */ 874, 2051, 1849, 862, 898, 162, 2394, 1613, 2393, 3015, - /* 1750 */ 1839, 2897, 2309, 73, 2789, 492, 2893, 2836, 2948, 2790, - /* 1760 */ 355, 131, 2791, 879, 2793, 2794, 874, 891, 1694, 862, - /* 1770 */ 898, 799, 484, 806, 876, 3015, 2965, 2897, 480, 840, - /* 1780 */ 989, 492, 2893, 824, 2601, 527, 545, 778, 2315, 814, - /* 1790 */ 2516, 396, 2949, 1725, 1733, 1740, 2959, 815, 1738, 330, - /* 1800 */ 325, 185, 333, 2808, 2602, 2790, 5, 98, 97, 567, - /* 1810 */ 2428, 535, 241, 530, 458, 2002, 2012, 544, 237, 2755, - /* 1820 */ 876, 875, 796, 556, 555, 559, 557, 558, 1873, 239, - /* 1830 */ 2103, 2003, 523, 522, 389, 2790, 1990, 236, 437, 572, - /* 1840 */ 250, 546, 1993, 579, 542, 538, 534, 531, 560, 2808, - /* 1850 */ 876, 581, 2988, 585, 629, 587, 1986, 605, 2081, 592, - /* 1860 */ 616, 2594, 625, 618, 628, 2755, 2789, 875, 641, 2836, - /* 1870 */ 630, 642, 639, 131, 2791, 879, 2793, 2794, 874, 2808, - /* 1880 */ 255, 862, 898, 256, 644, 646, 2790, 3015, 2076, 2897, - /* 1890 */ 647, 259, 2010, 492, 2893, 2755, 649, 875, 666, 651, - /* 1900 */ 4, 876, 1985, 2929, 667, 674, 334, 675, 827, 2005, - /* 1910 */ 677, 679, 2789, 267, 2011, 2836, 680, 106, 270, 131, - /* 1920 */ 2791, 879, 2793, 2794, 874, 2013, 273, 862, 898, 681, - /* 1930 */ 2808, 683, 859, 3015, 275, 2897, 2014, 2617, 2015, 492, - /* 1940 */ 2893, 107, 2789, 108, 2611, 2836, 2755, 109, 875, 131, - /* 1950 */ 2791, 879, 2793, 2794, 874, 282, 692, 862, 898, 713, - /* 1960 */ 136, 444, 443, 3015, 746, 2897, 715, 2458, 747, 492, - /* 1970 */ 2893, 504, 2685, 286, 2454, 288, 432, 2682, 187, 133, - /* 1980 */ 2456, 2451, 188, 189, 761, 513, 2681, 2081, 112, 763, - /* 1990 */ 2006, 305, 175, 2789, 773, 2790, 2836, 392, 2663, 774, - /* 2000 */ 131, 2791, 879, 2793, 2794, 874, 772, 308, 862, 898, - /* 2010 */ 876, 777, 310, 780, 3015, 789, 2897, 2076, 803, 2964, - /* 2020 */ 492, 2893, 841, 315, 2963, 2790, 812, 779, 317, 8, - /* 2030 */ 790, 201, 321, 2936, 319, 788, 1994, 322, 1989, 2808, - /* 2040 */ 876, 787, 817, 324, 323, 816, 3018, 485, 828, 825, - /* 2050 */ 2916, 2994, 326, 166, 329, 2755, 2007, 875, 2913, 2790, - /* 2060 */ 2167, 2165, 1, 214, 335, 230, 176, 839, 2631, 2808, - /* 2070 */ 2630, 2629, 1997, 1999, 876, 489, 393, 394, 844, 177, - /* 2080 */ 845, 849, 852, 71, 2878, 2755, 883, 875, 395, 896, - /* 2090 */ 895, 881, 348, 885, 2747, 2469, 2075, 2077, 2078, 2079, - /* 2100 */ 2080, 2082, 2789, 2808, 886, 2836, 353, 122, 125, 131, - /* 2110 */ 2791, 879, 2793, 2794, 874, 2746, 2742, 862, 898, 2755, - /* 2120 */ 2741, 875, 382, 2870, 995, 2897, 1511, 996, 399, 492, - /* 2130 */ 2893, 900, 2789, 997, 186, 2836, 2733, 2732, 2724, 131, - /* 2140 */ 2791, 879, 2793, 2794, 874, 385, 2790, 862, 898, 2723, - /* 2150 */ 992, 999, 2739, 856, 2738, 2897, 386, 2730, 2729, 492, - /* 2160 */ 2893, 876, 2718, 2717, 450, 1970, 2789, 1946, 2736, 2836, - /* 2170 */ 2735, 2727, 2726, 132, 2791, 879, 2793, 2794, 874, 2715, - /* 2180 */ 2714, 862, 898, 2712, 442, 2711, 2520, 420, 431, 2897, - /* 2190 */ 2808, 421, 2790, 2896, 2893, 62, 401, 451, 445, 506, - /* 2200 */ 505, 1971, 403, 2707, 2706, 2705, 2755, 876, 875, 95, - /* 2210 */ 2700, 532, 765, 1930, 433, 533, 1931, 234, 896, 895, - /* 2220 */ 537, 2698, 2790, 539, 540, 2075, 2077, 2078, 2079, 2080, - /* 2230 */ 2082, 541, 1929, 2697, 2696, 459, 2808, 876, 2694, 2693, - /* 2240 */ 2692, 551, 547, 549, 2691, 553, 1917, 2667, 238, 2666, - /* 2250 */ 240, 1876, 2755, 2789, 875, 96, 2836, 1875, 2644, 2643, - /* 2260 */ 132, 2791, 879, 2793, 2794, 874, 2808, 2642, 862, 898, - /* 2270 */ 565, 2641, 566, 2640, 2584, 570, 2897, 1812, 2581, 573, - /* 2280 */ 858, 2893, 2755, 2580, 875, 2574, 2790, 576, 2571, 577, - /* 2290 */ 2570, 2569, 243, 99, 2568, 2573, 245, 2572, 2567, 877, - /* 2300 */ 2566, 876, 2836, 2564, 2563, 2562, 132, 2791, 879, 2793, - /* 2310 */ 2794, 874, 247, 2790, 862, 898, 593, 2561, 595, 2559, - /* 2320 */ 2558, 2557, 2897, 2556, 2555, 2579, 453, 2893, 876, 2789, - /* 2330 */ 2808, 2554, 2836, 2553, 2552, 2577, 199, 2791, 879, 2793, - /* 2340 */ 2794, 874, 2560, 2551, 862, 898, 2755, 2550, 875, 2549, - /* 2350 */ 2790, 2547, 2546, 2545, 2544, 2543, 2542, 2808, 249, 2541, - /* 2360 */ 105, 2540, 2539, 2538, 2537, 876, 1818, 2534, 2533, 254, - /* 2370 */ 632, 2532, 634, 2755, 2609, 875, 2578, 2576, 2536, 2535, - /* 2380 */ 2531, 2529, 1663, 2359, 447, 2358, 257, 1667, 448, 2357, - /* 2390 */ 258, 2356, 2354, 2789, 2808, 260, 2836, 821, 3016, 2790, - /* 2400 */ 132, 2791, 879, 2793, 2794, 874, 1659, 261, 862, 898, - /* 2410 */ 2755, 2351, 875, 654, 876, 2350, 2897, 658, 652, 2343, - /* 2420 */ 2789, 2894, 656, 2836, 660, 2330, 653, 196, 2791, 879, - /* 2430 */ 2793, 2794, 874, 664, 482, 862, 898, 662, 657, 2304, - /* 2440 */ 91, 661, 263, 2808, 1540, 209, 2303, 92, 2790, 2775, - /* 2450 */ 220, 672, 265, 2665, 2661, 2651, 2639, 2789, 272, 2755, - /* 2460 */ 2836, 875, 274, 876, 430, 2791, 879, 2793, 2794, 874, - /* 2470 */ 2638, 277, 862, 898, 2615, 279, 2790, 2956, 693, 2347, - /* 2480 */ 1589, 2353, 2349, 483, 2608, 2446, 694, 695, 697, 699, - /* 2490 */ 2790, 876, 2808, 2345, 698, 701, 2342, 705, 702, 2325, - /* 2500 */ 2323, 703, 706, 2324, 707, 876, 2789, 2322, 2755, 2836, - /* 2510 */ 875, 1745, 2790, 430, 2791, 879, 2793, 2794, 874, 2300, - /* 2520 */ 2808, 862, 898, 2448, 1744, 2447, 83, 873, 285, 2340, - /* 2530 */ 1649, 1648, 2338, 1647, 2808, 1631, 2755, 1644, 875, 2329, - /* 2540 */ 1642, 1640, 1639, 473, 474, 1638, 1637, 962, 1636, 964, - /* 2550 */ 2755, 1633, 875, 1632, 1630, 2789, 2808, 475, 2836, 2327, - /* 2560 */ 476, 2299, 423, 2791, 879, 2793, 2794, 874, 2298, 744, - /* 2570 */ 862, 898, 2755, 2297, 875, 754, 741, 748, 2296, 750, - /* 2580 */ 2295, 2790, 752, 2789, 2294, 138, 2836, 1911, 1913, 1915, - /* 2590 */ 199, 2791, 879, 2793, 2794, 874, 876, 2789, 862, 898, - /* 2600 */ 2836, 32, 1910, 2664, 415, 2791, 879, 2793, 2794, 874, - /* 2610 */ 66, 307, 862, 898, 2660, 811, 67, 2790, 764, 2789, - /* 2620 */ 2650, 1882, 2836, 1901, 1884, 2808, 429, 2791, 879, 2793, - /* 2630 */ 2794, 874, 876, 1886, 862, 898, 311, 2863, 775, 77, - /* 2640 */ 190, 2755, 776, 875, 1861, 2790, 2637, 2636, 3000, 17, - /* 2650 */ 781, 1860, 3017, 783, 21, 34, 6, 7, 2220, 314, - /* 2660 */ 876, 2808, 791, 2194, 481, 495, 316, 797, 795, 793, - /* 2670 */ 22, 23, 213, 2201, 2188, 225, 2776, 2755, 200, 875, - /* 2680 */ 2158, 36, 212, 2160, 35, 93, 226, 24, 2789, 2808, - /* 2690 */ 2162, 2836, 227, 75, 25, 430, 2791, 879, 2793, 2794, - /* 2700 */ 874, 491, 2240, 862, 898, 2755, 2241, 875, 2235, 2234, - /* 2710 */ 486, 2239, 2238, 487, 331, 2142, 2141, 69, 18, 205, - /* 2720 */ 2635, 2614, 68, 116, 2789, 117, 2790, 2836, 338, 496, - /* 2730 */ 215, 413, 2791, 879, 2793, 2794, 874, 2613, 2196, 862, - /* 2740 */ 898, 876, 344, 118, 842, 79, 2607, 848, 347, 346, - /* 2750 */ 119, 26, 2789, 2094, 2790, 2836, 11, 13, 850, 430, - /* 2760 */ 2791, 879, 2793, 2794, 874, 2093, 349, 862, 898, 876, - /* 2770 */ 2808, 1995, 27, 28, 20, 47, 206, 913, 2104, 216, - /* 2780 */ 2054, 2053, 916, 2030, 919, 922, 2755, 2052, 875, 878, - /* 2790 */ 2790, 884, 120, 48, 16, 2606, 2443, 29, 2808, 2022, - /* 2800 */ 30, 80, 889, 882, 351, 876, 121, 2256, 357, 126, - /* 2810 */ 498, 890, 89, 2847, 2755, 2846, 875, 2064, 897, 78, - /* 2820 */ 899, 2255, 2254, 2790, 1726, 2253, 905, 907, 515, 1723, - /* 2830 */ 909, 910, 1717, 2789, 2808, 1722, 2836, 912, 876, 1719, - /* 2840 */ 430, 2791, 879, 2793, 2794, 874, 915, 1713, 862, 898, - /* 2850 */ 2755, 918, 875, 921, 1711, 127, 381, 128, 1739, 1716, - /* 2860 */ 90, 760, 1735, 1715, 2836, 1587, 2790, 2808, 425, 2791, - /* 2870 */ 879, 2793, 2794, 874, 1627, 1626, 862, 898, 1714, 936, - /* 2880 */ 1625, 876, 1622, 2755, 1619, 875, 1618, 1617, 1657, 1616, - /* 2890 */ 1614, 1612, 1611, 1610, 1656, 950, 952, 2789, 232, 2790, - /* 2900 */ 2836, 1653, 1608, 1605, 407, 2791, 879, 2793, 2794, 874, - /* 2910 */ 2808, 1607, 862, 898, 876, 1606, 1604, 1593, 1603, 1602, - /* 2920 */ 1651, 1599, 1598, 1595, 1594, 1592, 2755, 2348, 875, 972, - /* 2930 */ 2789, 973, 2346, 2836, 974, 976, 978, 405, 2791, 879, - /* 2940 */ 2793, 2794, 874, 2808, 2344, 862, 898, 980, 977, 982, - /* 2950 */ 981, 2341, 984, 986, 2321, 985, 988, 2319, 990, 2755, - /* 2960 */ 1529, 875, 2293, 2790, 1512, 1517, 994, 1519, 387, 998, - /* 2970 */ 2259, 1981, 400, 2789, 1001, 1002, 2836, 2259, 876, 2259, - /* 2980 */ 408, 2791, 879, 2793, 2794, 874, 2790, 2259, 862, 898, - /* 2990 */ 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, - /* 3000 */ 2259, 876, 2259, 2259, 2259, 2259, 2789, 2808, 2259, 2836, - /* 3010 */ 2259, 2259, 2259, 422, 2791, 879, 2793, 2794, 874, 2259, - /* 3020 */ 2259, 862, 898, 2755, 2259, 875, 2259, 2259, 2259, 2259, - /* 3030 */ 2808, 2259, 2790, 2259, 2259, 2259, 2259, 2259, 2259, 2259, - /* 3040 */ 2259, 2259, 2259, 2259, 2259, 2259, 2755, 876, 875, 2259, - /* 3050 */ 2259, 2259, 2259, 2259, 2790, 2259, 2259, 2259, 2259, 2259, - /* 3060 */ 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2790, 876, - /* 3070 */ 2789, 2259, 2259, 2836, 2259, 2259, 2808, 409, 2791, 879, - /* 3080 */ 2793, 2794, 874, 876, 2259, 862, 898, 2259, 2259, 2259, - /* 3090 */ 2259, 2259, 2755, 2789, 875, 2259, 2836, 2259, 2808, 2259, - /* 3100 */ 426, 2791, 879, 2793, 2794, 874, 2259, 2259, 862, 898, - /* 3110 */ 2259, 2259, 2808, 2259, 2755, 2259, 875, 2259, 2259, 2259, - /* 3120 */ 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2755, 2259, - /* 3130 */ 875, 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2789, - /* 3140 */ 2259, 2259, 2836, 2259, 2790, 2259, 410, 2791, 879, 2793, - /* 3150 */ 2794, 874, 2259, 2259, 862, 898, 2259, 2259, 2259, 876, - /* 3160 */ 2259, 2789, 2259, 2259, 2836, 2259, 2259, 2259, 427, 2791, - /* 3170 */ 879, 2793, 2794, 874, 2259, 2789, 862, 898, 2836, 2790, - /* 3180 */ 2259, 2259, 411, 2791, 879, 2793, 2794, 874, 2808, 2259, - /* 3190 */ 862, 898, 2259, 2259, 876, 2259, 2259, 2259, 2259, 2259, - /* 3200 */ 2259, 2259, 2259, 2259, 2755, 2259, 875, 2259, 2259, 2259, - /* 3210 */ 2259, 2259, 2790, 2259, 2259, 2259, 2259, 2259, 2259, 2259, - /* 3220 */ 2259, 2259, 2259, 2808, 2259, 2259, 2259, 876, 2259, 2259, - /* 3230 */ 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2755, - /* 3240 */ 2259, 875, 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, - /* 3250 */ 2259, 2789, 2259, 2259, 2836, 2790, 2808, 2259, 428, 2791, - /* 3260 */ 879, 2793, 2794, 874, 2259, 2259, 862, 898, 2259, 2259, - /* 3270 */ 876, 2259, 2755, 2259, 875, 2259, 2259, 2259, 2259, 2259, - /* 3280 */ 2259, 2259, 2259, 2259, 2259, 2259, 2789, 2259, 2790, 2836, - /* 3290 */ 2259, 2259, 2259, 412, 2791, 879, 2793, 2794, 874, 2808, - /* 3300 */ 2259, 862, 898, 876, 2259, 2259, 2259, 2259, 2259, 2259, - /* 3310 */ 2259, 2259, 2259, 2259, 2259, 2755, 2259, 875, 2259, 2789, - /* 3320 */ 2259, 2259, 2836, 2259, 2259, 2259, 406, 2791, 879, 2793, - /* 3330 */ 2794, 874, 2808, 2259, 862, 898, 2259, 2259, 2259, 2259, - /* 3340 */ 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2755, 2259, - /* 3350 */ 875, 2259, 2790, 2259, 2259, 2259, 2259, 2259, 2259, 2259, - /* 3360 */ 2259, 2259, 2789, 2259, 2259, 2836, 2259, 876, 2259, 414, - /* 3370 */ 2791, 879, 2793, 2794, 874, 2790, 2259, 862, 898, 2259, - /* 3380 */ 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, - /* 3390 */ 876, 2259, 2259, 2259, 2259, 2789, 2808, 2259, 2836, 2259, - /* 3400 */ 2259, 2259, 417, 2791, 879, 2793, 2794, 874, 2259, 2259, - /* 3410 */ 862, 898, 2755, 2259, 875, 2259, 2259, 2259, 2259, 2808, - /* 3420 */ 2259, 2790, 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, - /* 3430 */ 2259, 2259, 2259, 2259, 2259, 2755, 876, 875, 2259, 2259, - /* 3440 */ 2259, 2259, 2259, 2790, 2259, 2259, 2259, 2259, 2259, 2259, - /* 3450 */ 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2790, 876, 2789, - /* 3460 */ 2259, 2259, 2836, 2259, 2259, 2808, 418, 2791, 879, 2793, - /* 3470 */ 2794, 874, 876, 2259, 862, 898, 2259, 2259, 2259, 2259, - /* 3480 */ 2259, 2755, 2789, 875, 2259, 2836, 2259, 2808, 2259, 419, - /* 3490 */ 2791, 879, 2793, 2794, 874, 2259, 2259, 862, 898, 2259, - /* 3500 */ 2259, 2808, 2259, 2755, 2259, 875, 2259, 2259, 2259, 2259, - /* 3510 */ 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2755, 2259, 875, - /* 3520 */ 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2789, 2259, - /* 3530 */ 2259, 2836, 2259, 2790, 2259, 435, 2791, 879, 2793, 2794, - /* 3540 */ 874, 2259, 2259, 862, 898, 2259, 2259, 2259, 876, 2259, - /* 3550 */ 2789, 2259, 2259, 2836, 2259, 2259, 2259, 436, 2791, 879, - /* 3560 */ 2793, 2794, 874, 2259, 2789, 862, 898, 2836, 2790, 2259, - /* 3570 */ 2259, 2802, 2791, 879, 2793, 2794, 874, 2808, 2259, 862, - /* 3580 */ 898, 2259, 2259, 876, 2259, 2259, 2259, 2259, 2259, 2259, - /* 3590 */ 2259, 2259, 2259, 2755, 2259, 875, 2259, 2259, 2259, 2259, - /* 3600 */ 2259, 2790, 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, - /* 3610 */ 2259, 2259, 2808, 2259, 2259, 2259, 876, 2259, 2259, 2259, - /* 3620 */ 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2755, 2259, - /* 3630 */ 875, 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, - /* 3640 */ 2789, 2259, 2259, 2836, 2790, 2808, 2259, 2801, 2791, 879, - /* 3650 */ 2793, 2794, 874, 2259, 2259, 862, 898, 2259, 2259, 876, - /* 3660 */ 2259, 2755, 2259, 875, 2259, 2259, 2259, 2259, 2259, 2259, - /* 3670 */ 2259, 2259, 2259, 2259, 2259, 2789, 2259, 2790, 2836, 2259, - /* 3680 */ 2259, 2259, 2800, 2791, 879, 2793, 2794, 874, 2808, 2259, - /* 3690 */ 862, 898, 876, 2259, 2259, 2259, 2259, 2259, 2259, 2259, - /* 3700 */ 2259, 2259, 2259, 2259, 2755, 2259, 875, 2259, 2789, 2259, - /* 3710 */ 2259, 2836, 2259, 2259, 2259, 455, 2791, 879, 2793, 2794, - /* 3720 */ 874, 2808, 2259, 862, 898, 2259, 2259, 2259, 2259, 2259, - /* 3730 */ 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2755, 2259, 875, - /* 3740 */ 2259, 2790, 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, - /* 3750 */ 2259, 2789, 2259, 2259, 2836, 2259, 876, 2259, 456, 2791, - /* 3760 */ 879, 2793, 2794, 874, 2790, 2259, 862, 898, 2259, 2259, - /* 3770 */ 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, 876, - /* 3780 */ 2259, 2259, 2259, 2259, 2789, 2808, 2259, 2836, 2259, 2259, - /* 3790 */ 2259, 452, 2791, 879, 2793, 2794, 874, 2259, 2259, 862, - /* 3800 */ 898, 2755, 2259, 875, 2259, 2259, 2259, 2259, 2808, 2259, - /* 3810 */ 2790, 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, - /* 3820 */ 2259, 2259, 2259, 2259, 2755, 876, 875, 2259, 2259, 2259, - /* 3830 */ 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, - /* 3840 */ 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2789, 2259, - /* 3850 */ 2259, 2836, 2259, 2259, 2808, 457, 2791, 879, 2793, 2794, - /* 3860 */ 874, 2259, 2259, 862, 898, 2259, 2259, 2259, 2259, 2259, - /* 3870 */ 2755, 877, 875, 2259, 2836, 2259, 2259, 2259, 425, 2791, - /* 3880 */ 879, 2793, 2794, 874, 2259, 2259, 862, 898, 2259, 2259, - /* 3890 */ 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, - /* 3900 */ 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2259, - /* 3910 */ 2259, 2259, 2259, 2259, 2259, 2259, 2259, 2789, 2259, 2259, - /* 3920 */ 2836, 2259, 2259, 2259, 424, 2791, 879, 2793, 2794, 874, - /* 3930 */ 2259, 2259, 862, 898, + /* 0 */ 821, 670, 3003, 678, 671, 2311, 671, 2311, 2998, 2475, + /* 10 */ 2998, 956, 58, 56, 2430, 57, 55, 54, 53, 52, + /* 20 */ 499, 2264, 1982, 2626, 2007, 464, 820, 229, 771, 3002, + /* 30 */ 218, 2999, 822, 2999, 3001, 503, 1980, 492, 2084, 2355, + /* 40 */ 218, 2793, 149, 2623, 882, 148, 147, 146, 145, 144, + /* 50 */ 143, 142, 141, 140, 582, 2595, 836, 2599, 51, 50, + /* 60 */ 833, 168, 57, 55, 54, 53, 52, 2599, 2079, 149, + /* 70 */ 767, 865, 148, 147, 146, 145, 144, 143, 142, 141, + /* 80 */ 140, 895, 1988, 51, 50, 2811, 757, 57, 55, 54, + /* 90 */ 53, 52, 51, 50, 2698, 2011, 57, 55, 54, 53, + /* 100 */ 52, 2758, 751, 877, 755, 753, 299, 298, 245, 690, + /* 110 */ 680, 2665, 1002, 895, 693, 59, 973, 972, 971, 970, + /* 120 */ 528, 249, 969, 968, 173, 963, 962, 961, 960, 959, + /* 130 */ 958, 957, 172, 951, 950, 949, 527, 526, 946, 945, + /* 140 */ 944, 209, 208, 943, 523, 942, 941, 940, 2792, 43, + /* 150 */ 351, 2840, 2087, 2088, 810, 132, 2794, 881, 2796, 2797, + /* 160 */ 876, 2462, 2751, 864, 900, 516, 2220, 105, 2613, 211, + /* 170 */ 691, 2900, 104, 51, 50, 494, 2896, 57, 55, 54, + /* 180 */ 53, 52, 2793, 2811, 205, 2908, 832, 770, 160, 831, + /* 190 */ 3003, 2043, 2053, 342, 343, 230, 2998, 878, 341, 768, + /* 200 */ 2064, 2086, 2089, 2947, 2263, 51, 50, 2010, 61, 57, + /* 210 */ 55, 54, 53, 52, 820, 229, 1983, 2173, 1981, 2999, + /* 220 */ 822, 2180, 9, 863, 505, 505, 2811, 2007, 158, 157, + /* 230 */ 156, 155, 154, 153, 152, 151, 150, 900, 900, 103, + /* 240 */ 456, 895, 2758, 2261, 877, 937, 185, 184, 934, 933, + /* 250 */ 932, 182, 1986, 1987, 2040, 809, 2042, 2045, 2046, 2047, + /* 260 */ 2048, 2049, 2050, 2051, 2052, 873, 866, 2007, 225, 898, + /* 270 */ 897, 2071, 2072, 2073, 2074, 2075, 2078, 2080, 2081, 2082, + /* 280 */ 2083, 2085, 2, 58, 56, 2176, 686, 2793, 675, 2792, + /* 290 */ 2114, 499, 2840, 1982, 672, 689, 132, 2794, 881, 2796, + /* 300 */ 2797, 876, 875, 2244, 864, 900, 744, 1980, 170, 2084, + /* 310 */ 179, 2871, 2900, 2192, 833, 168, 494, 2896, 51, 50, + /* 320 */ 895, 758, 57, 55, 54, 53, 52, 3003, 573, 2012, + /* 330 */ 2626, 2811, 531, 2151, 73, 2998, 61, 530, 220, 2079, + /* 340 */ 300, 2151, 865, 33, 501, 2007, 19, 2758, 667, 877, + /* 350 */ 2623, 882, 44, 1988, 3002, 2115, 747, 665, 2999, 3000, + /* 360 */ 661, 657, 73, 741, 739, 2752, 58, 56, 73, 265, + /* 370 */ 297, 896, 2471, 673, 499, 2319, 1982, 2209, 635, 633, + /* 380 */ 333, 436, 771, 1002, 243, 77, 15, 2568, 787, 1535, + /* 390 */ 1980, 159, 2084, 203, 2792, 328, 2998, 2840, 2044, 716, + /* 400 */ 335, 431, 2794, 881, 2796, 2797, 876, 874, 1542, 864, + /* 410 */ 900, 856, 2865, 2358, 3004, 229, 83, 125, 12, 2999, + /* 420 */ 822, 82, 2079, 2087, 2088, 865, 2221, 520, 505, 19, + /* 430 */ 2524, 2526, 2120, 1537, 1540, 1541, 1988, 835, 198, 2908, + /* 440 */ 2909, 900, 166, 2913, 803, 802, 2207, 2208, 2210, 2211, + /* 450 */ 2212, 42, 496, 2109, 2110, 2111, 2112, 2113, 2117, 2118, + /* 460 */ 2119, 2041, 2043, 2053, 773, 2665, 1002, 2008, 564, 15, + /* 470 */ 563, 62, 2086, 2089, 2920, 2148, 2149, 2150, 2920, 2920, + /* 480 */ 2920, 2920, 2920, 2148, 2149, 2150, 2768, 1983, 1988, 1981, + /* 490 */ 737, 736, 735, 195, 863, 193, 73, 727, 165, 731, + /* 500 */ 811, 2404, 562, 730, 518, 2473, 2087, 2088, 729, 734, + /* 510 */ 474, 473, 906, 1653, 728, 2772, 1828, 1829, 472, 724, + /* 520 */ 723, 722, 2172, 1986, 1987, 2040, 335, 2042, 2045, 2046, + /* 530 */ 2047, 2048, 2049, 2050, 2051, 2052, 873, 866, 2455, 443, + /* 540 */ 898, 897, 2071, 2072, 183, 2043, 2053, 2078, 2080, 2081, + /* 550 */ 2082, 2083, 2085, 2, 335, 2086, 2089, 687, 610, 1655, + /* 560 */ 335, 471, 470, 609, 1715, 2774, 2776, 495, 303, 2151, + /* 570 */ 1983, 608, 1981, 222, 833, 168, 2010, 863, 900, 1706, + /* 580 */ 929, 928, 927, 1710, 926, 1712, 1713, 925, 922, 2518, + /* 590 */ 1721, 919, 1723, 1724, 916, 913, 910, 51, 50, 1754, + /* 600 */ 1755, 57, 55, 54, 53, 52, 1986, 1987, 2040, 930, + /* 610 */ 2042, 2045, 2046, 2047, 2048, 2049, 2050, 2051, 2052, 873, + /* 620 */ 866, 688, 2619, 898, 897, 2071, 2072, 65, 2915, 2626, + /* 630 */ 2078, 2080, 2081, 2082, 2083, 2085, 2, 12, 58, 56, + /* 640 */ 511, 469, 468, 392, 718, 2793, 499, 302, 1982, 2624, + /* 650 */ 882, 301, 806, 1909, 1910, 1715, 2912, 833, 168, 196, + /* 660 */ 878, 2275, 1980, 335, 2084, 720, 905, 904, 903, 719, + /* 670 */ 1706, 929, 928, 927, 1710, 926, 1712, 1713, 872, 871, + /* 680 */ 2793, 1721, 870, 1723, 1724, 869, 913, 910, 335, 2811, + /* 690 */ 1562, 2286, 1561, 2008, 2079, 878, 2551, 865, 138, 2908, + /* 700 */ 2909, 19, 166, 2913, 2674, 2758, 164, 877, 1988, 218, + /* 710 */ 2920, 2148, 2149, 2150, 2920, 2920, 2920, 2920, 2920, 896, + /* 720 */ 2471, 58, 56, 2090, 2811, 114, 550, 2768, 1563, 499, + /* 730 */ 451, 1982, 2285, 479, 2040, 759, 2600, 2095, 1002, 522, + /* 740 */ 2758, 15, 877, 2007, 135, 1980, 1562, 2084, 1561, 333, + /* 750 */ 2011, 2758, 2792, 518, 2473, 2840, 2772, 588, 2595, 408, + /* 760 */ 2794, 881, 2796, 2797, 876, 725, 31, 864, 900, 812, + /* 770 */ 807, 800, 796, 896, 2471, 619, 2595, 2079, 2087, 2088, + /* 780 */ 865, 199, 2908, 2909, 1563, 166, 2913, 2792, 1646, 2711, + /* 790 */ 2840, 1988, 2758, 159, 132, 2794, 881, 2796, 2797, 876, + /* 800 */ 502, 721, 864, 900, 481, 2673, 2774, 2777, 2875, 192, + /* 810 */ 2900, 247, 896, 2471, 494, 2896, 95, 2043, 2053, 900, + /* 820 */ 2476, 1002, 1951, 939, 59, 1855, 1856, 2086, 2089, 252, + /* 830 */ 51, 50, 234, 171, 57, 55, 54, 53, 52, 737, + /* 840 */ 736, 735, 1983, 2463, 1981, 787, 727, 165, 731, 863, + /* 850 */ 2445, 612, 730, 2998, 510, 509, 47, 729, 734, 474, + /* 860 */ 473, 2087, 2088, 728, 554, 2712, 611, 472, 724, 723, + /* 870 */ 722, 3004, 229, 2011, 1854, 1857, 2999, 822, 1986, 1987, + /* 880 */ 2040, 111, 2042, 2045, 2046, 2047, 2048, 2049, 2050, 2051, + /* 890 */ 2052, 873, 866, 556, 552, 898, 897, 2071, 2072, 467, + /* 900 */ 2043, 2053, 2078, 2080, 2081, 2082, 2083, 2085, 2, 2466, + /* 910 */ 2086, 2089, 51, 50, 2044, 868, 57, 55, 54, 53, + /* 920 */ 52, 787, 392, 224, 939, 1983, 867, 1981, 629, 2998, + /* 930 */ 896, 2471, 863, 267, 525, 524, 2531, 673, 514, 2319, + /* 940 */ 521, 566, 1665, 1982, 462, 2531, 565, 3004, 229, 192, + /* 950 */ 66, 46, 2999, 822, 2342, 2529, 1664, 1980, 1989, 2284, + /* 960 */ 2476, 1986, 1987, 2040, 840, 2042, 2045, 2046, 2047, 2048, + /* 970 */ 2049, 2050, 2051, 2052, 873, 866, 738, 2041, 898, 897, + /* 980 */ 2071, 2072, 480, 2673, 2012, 2078, 2080, 2081, 2082, 2083, + /* 990 */ 2085, 2, 58, 56, 2201, 2793, 254, 787, 2531, 12, + /* 1000 */ 499, 10, 1982, 1988, 2283, 2998, 490, 2116, 2202, 115, + /* 1010 */ 836, 2153, 2154, 2155, 2156, 2157, 1980, 2529, 2084, 2758, + /* 1020 */ 2059, 628, 253, 3004, 229, 896, 2471, 37, 2999, 822, + /* 1030 */ 2251, 2282, 640, 1002, 821, 626, 896, 2471, 314, 2811, + /* 1040 */ 2525, 2526, 2998, 1883, 2793, 692, 232, 2679, 2079, 896, + /* 1050 */ 2471, 865, 896, 2471, 2200, 2758, 570, 877, 2915, 878, + /* 1060 */ 820, 229, 1988, 2531, 2758, 2999, 822, 896, 2471, 571, + /* 1070 */ 896, 2471, 590, 896, 2471, 58, 56, 896, 2471, 385, + /* 1080 */ 896, 2471, 848, 499, 2121, 1982, 2911, 604, 2811, 116, + /* 1090 */ 605, 2758, 1002, 606, 720, 59, 2915, 2468, 719, 1980, + /* 1100 */ 305, 2084, 2792, 40, 2758, 2840, 877, 2012, 304, 132, + /* 1110 */ 2794, 881, 2796, 2797, 876, 896, 2471, 864, 900, 2531, + /* 1120 */ 1950, 896, 2471, 211, 2910, 2900, 2281, 504, 953, 494, + /* 1130 */ 2896, 2079, 2087, 2088, 865, 313, 39, 1983, 2529, 1981, + /* 1140 */ 1992, 839, 51, 50, 192, 1988, 57, 55, 54, 53, + /* 1150 */ 52, 2792, 513, 512, 2840, 2477, 2250, 2948, 197, 2794, + /* 1160 */ 881, 2796, 2797, 876, 1565, 1566, 864, 900, 54, 53, + /* 1170 */ 52, 2043, 2053, 1986, 1987, 1002, 2460, 2007, 59, 896, + /* 1180 */ 2471, 2086, 2089, 51, 50, 2531, 2758, 57, 55, 54, + /* 1190 */ 53, 52, 124, 519, 712, 711, 1983, 955, 1981, 346, + /* 1200 */ 714, 713, 2458, 863, 2529, 51, 50, 788, 2958, 57, + /* 1210 */ 55, 54, 53, 52, 41, 2087, 2088, 2280, 2279, 2464, + /* 1220 */ 51, 50, 14, 13, 57, 55, 54, 53, 52, 733, + /* 1230 */ 732, 399, 1986, 1987, 2040, 2508, 2042, 2045, 2046, 2047, + /* 1240 */ 2048, 2049, 2050, 2051, 2052, 873, 866, 111, 2278, 898, + /* 1250 */ 897, 2071, 2072, 175, 2043, 2053, 2078, 2080, 2081, 2082, + /* 1260 */ 2083, 2085, 2, 786, 2086, 2089, 51, 50, 896, 2471, + /* 1270 */ 57, 55, 54, 53, 52, 2467, 285, 2758, 2758, 1983, + /* 1280 */ 2447, 1981, 857, 859, 2872, 2872, 863, 825, 853, 51, + /* 1290 */ 50, 2060, 204, 57, 55, 54, 53, 52, 896, 2471, + /* 1300 */ 391, 710, 706, 702, 698, 38, 284, 2444, 2758, 169, + /* 1310 */ 896, 2471, 2871, 2062, 2011, 1986, 1987, 2040, 353, 2042, + /* 1320 */ 2045, 2046, 2047, 2048, 2049, 2050, 2051, 2052, 873, 866, + /* 1330 */ 889, 2007, 898, 897, 2071, 2072, 1899, 2128, 2277, 2078, + /* 1340 */ 2080, 2081, 2082, 2083, 2085, 2, 58, 56, 2044, 2274, + /* 1350 */ 1669, 440, 112, 2005, 499, 769, 1982, 282, 3002, 1542, + /* 1360 */ 617, 51, 50, 466, 1668, 57, 55, 54, 53, 52, + /* 1370 */ 1980, 637, 2084, 1005, 51, 50, 896, 2471, 57, 55, + /* 1380 */ 54, 53, 52, 896, 2471, 1540, 1541, 596, 545, 639, + /* 1390 */ 896, 2471, 389, 1544, 441, 598, 890, 815, 2758, 2006, + /* 1400 */ 2273, 2272, 2079, 894, 71, 865, 576, 993, 219, 2758, + /* 1410 */ 381, 2041, 236, 45, 2271, 784, 1988, 989, 985, 981, + /* 1420 */ 977, 2270, 384, 2269, 270, 937, 185, 184, 934, 933, + /* 1430 */ 932, 182, 193, 281, 967, 965, 931, 272, 279, 2522, + /* 1440 */ 642, 310, 2474, 277, 684, 828, 1002, 465, 2531, 15, + /* 1450 */ 2291, 995, 937, 185, 184, 934, 933, 932, 182, 584, + /* 1460 */ 2758, 2758, 269, 2268, 2267, 2702, 2162, 2530, 131, 2065, + /* 1470 */ 2266, 51, 50, 357, 2758, 57, 55, 54, 53, 52, + /* 1480 */ 2533, 2758, 935, 2758, 89, 2522, 2087, 2088, 2578, 3, + /* 1490 */ 2606, 2585, 175, 625, 624, 623, 622, 621, 616, 615, + /* 1500 */ 614, 613, 448, 64, 849, 603, 602, 601, 600, 599, + /* 1510 */ 593, 592, 591, 2448, 586, 585, 463, 1648, 2340, 538, + /* 1520 */ 577, 1816, 1817, 2758, 2758, 2043, 2053, 1835, 936, 161, + /* 1530 */ 2758, 2522, 580, 2331, 290, 2086, 2089, 288, 102, 292, + /* 1540 */ 740, 294, 291, 101, 293, 726, 174, 2329, 2012, 355, + /* 1550 */ 1983, 855, 1981, 296, 338, 742, 295, 863, 761, 337, + /* 1560 */ 760, 794, 1626, 1649, 398, 2041, 51, 50, 1644, 745, + /* 1570 */ 57, 55, 54, 53, 52, 1907, 2779, 60, 307, 2061, + /* 1580 */ 2253, 2254, 2475, 60, 212, 826, 1986, 1987, 2040, 223, + /* 1590 */ 2042, 2045, 2046, 2047, 2048, 2049, 2050, 2051, 2052, 873, + /* 1600 */ 866, 14, 13, 898, 897, 2071, 2072, 183, 1627, 340, + /* 1610 */ 2078, 2080, 2081, 2082, 2083, 2085, 2, 194, 88, 51, + /* 1620 */ 50, 2793, 405, 57, 55, 54, 53, 52, 2405, 764, + /* 1630 */ 60, 360, 359, 2276, 862, 638, 878, 75, 2321, 60, + /* 1640 */ 2781, 403, 87, 60, 2196, 86, 60, 2793, 1991, 88, + /* 1650 */ 180, 161, 362, 361, 2961, 203, 442, 364, 363, 772, + /* 1660 */ 2206, 130, 878, 127, 2955, 2811, 2205, 319, 263, 652, + /* 1670 */ 650, 647, 645, 329, 2058, 183, 947, 366, 365, 368, + /* 1680 */ 367, 2758, 1990, 877, 824, 787, 804, 370, 369, 321, + /* 1690 */ 838, 2811, 344, 2998, 372, 371, 837, 374, 373, 1618, + /* 1700 */ 834, 845, 85, 376, 375, 378, 377, 2758, 163, 877, + /* 1710 */ 2812, 3004, 229, 2063, 73, 787, 2999, 822, 380, 379, + /* 1720 */ 2122, 2312, 2066, 2998, 2397, 2396, 2054, 908, 2792, 1852, + /* 1730 */ 181, 2840, 1842, 356, 893, 132, 2794, 881, 2796, 2797, + /* 1740 */ 876, 3004, 229, 864, 900, 829, 2999, 822, 183, 3018, + /* 1750 */ 948, 2900, 787, 74, 2792, 494, 2896, 2840, 1697, 2793, + /* 1760 */ 2998, 132, 2794, 881, 2796, 2797, 876, 801, 162, 864, + /* 1770 */ 900, 180, 1599, 1616, 878, 3018, 2968, 2900, 3004, 229, + /* 1780 */ 486, 494, 2896, 2999, 822, 397, 2951, 808, 482, 842, + /* 1790 */ 2604, 529, 547, 2318, 2519, 780, 326, 2952, 2962, 816, + /* 1800 */ 817, 331, 2605, 2811, 136, 2793, 2323, 99, 98, 569, + /* 1810 */ 1728, 2106, 242, 1736, 334, 2431, 5, 537, 1600, 2758, + /* 1820 */ 878, 877, 798, 532, 460, 561, 559, 2005, 546, 2015, + /* 1830 */ 1994, 1743, 525, 524, 557, 2793, 558, 237, 439, 1876, + /* 1840 */ 238, 548, 1996, 560, 544, 540, 536, 533, 562, 2811, + /* 1850 */ 878, 1741, 2991, 240, 186, 390, 1989, 574, 2084, 2006, + /* 1860 */ 581, 991, 251, 583, 1993, 2758, 2792, 877, 589, 2840, + /* 1870 */ 587, 631, 594, 132, 2794, 881, 2796, 2797, 876, 2811, + /* 1880 */ 607, 864, 900, 643, 620, 618, 2793, 3018, 2079, 2900, + /* 1890 */ 627, 2597, 630, 494, 2896, 2758, 641, 877, 632, 644, + /* 1900 */ 256, 878, 1988, 2932, 257, 646, 335, 648, 649, 260, + /* 1910 */ 651, 653, 2792, 2013, 668, 2840, 4, 676, 679, 132, + /* 1920 */ 2794, 881, 2796, 2797, 876, 268, 669, 864, 900, 677, + /* 1930 */ 2811, 107, 861, 3018, 2008, 2900, 2014, 681, 682, 494, + /* 1940 */ 2896, 2016, 2792, 685, 683, 2840, 2758, 271, 877, 132, + /* 1950 */ 2794, 881, 2796, 2797, 876, 2017, 274, 864, 900, 276, + /* 1960 */ 2620, 446, 445, 3018, 108, 2900, 2018, 109, 110, 494, + /* 1970 */ 2896, 506, 2614, 694, 283, 715, 763, 748, 749, 137, + /* 1980 */ 765, 717, 113, 434, 2461, 515, 287, 2084, 2688, 2685, + /* 1990 */ 2684, 393, 2457, 2792, 289, 2793, 2840, 188, 134, 2459, + /* 2000 */ 132, 2794, 881, 2796, 2797, 876, 2009, 2454, 864, 900, + /* 2010 */ 878, 189, 176, 190, 3018, 775, 2900, 2079, 306, 774, + /* 2020 */ 494, 2896, 2666, 779, 311, 2793, 776, 805, 782, 2967, + /* 2030 */ 791, 843, 316, 8, 2939, 814, 1997, 309, 1992, 2811, + /* 2040 */ 878, 792, 320, 781, 790, 2966, 318, 202, 819, 322, + /* 2050 */ 818, 789, 2919, 487, 323, 2758, 327, 877, 324, 2793, + /* 2060 */ 830, 827, 167, 3021, 325, 2010, 2170, 2168, 231, 2811, + /* 2070 */ 2916, 330, 2000, 2002, 878, 336, 1, 2997, 215, 177, + /* 2080 */ 841, 2634, 394, 2633, 2632, 2758, 491, 877, 395, 898, + /* 2090 */ 897, 846, 847, 178, 72, 851, 2078, 2080, 2081, 2082, + /* 2100 */ 2083, 2085, 2792, 2811, 854, 2840, 885, 349, 883, 132, + /* 2110 */ 2794, 881, 2796, 2797, 876, 2881, 887, 864, 900, 2758, + /* 2120 */ 354, 877, 888, 2873, 396, 2900, 2472, 2750, 123, 494, + /* 2130 */ 2896, 2749, 2792, 126, 2745, 2840, 2744, 2736, 2735, 132, + /* 2140 */ 2794, 881, 2796, 2797, 876, 1514, 2793, 864, 900, 2727, + /* 2150 */ 2726, 2742, 2741, 858, 400, 2900, 2733, 997, 2732, 494, + /* 2160 */ 2896, 878, 902, 998, 2721, 1973, 2792, 1949, 2720, 2840, + /* 2170 */ 2739, 383, 2738, 133, 2794, 881, 2796, 2797, 876, 187, + /* 2180 */ 2730, 864, 900, 999, 386, 994, 767, 387, 402, 2900, + /* 2190 */ 2811, 2729, 2793, 2899, 2896, 2718, 2717, 2715, 2714, 508, + /* 2200 */ 507, 1974, 2523, 1001, 63, 444, 2758, 878, 877, 422, + /* 2210 */ 452, 447, 433, 453, 435, 404, 2710, 423, 898, 897, + /* 2220 */ 2709, 2708, 2793, 96, 2703, 2078, 2080, 2081, 2082, 2083, + /* 2230 */ 2085, 534, 535, 1933, 1934, 235, 2811, 878, 539, 2701, + /* 2240 */ 541, 542, 543, 1932, 2700, 2699, 461, 2697, 549, 2696, + /* 2250 */ 551, 2695, 2758, 2792, 877, 553, 2840, 1920, 555, 2670, + /* 2260 */ 133, 2794, 881, 2796, 2797, 876, 2811, 2694, 864, 900, + /* 2270 */ 239, 2669, 241, 1879, 97, 1878, 2900, 2647, 2646, 2645, + /* 2280 */ 860, 2896, 2758, 567, 877, 568, 2793, 2644, 2643, 2587, + /* 2290 */ 2584, 572, 1815, 2583, 575, 2577, 578, 579, 2574, 879, + /* 2300 */ 2573, 878, 2840, 244, 2572, 2571, 133, 2794, 881, 2796, + /* 2310 */ 2797, 876, 100, 2793, 864, 900, 2576, 246, 2575, 2570, + /* 2320 */ 2569, 2567, 2900, 2566, 2565, 248, 455, 2896, 878, 2792, + /* 2330 */ 2811, 2564, 2840, 595, 597, 2562, 200, 2794, 881, 2796, + /* 2340 */ 2797, 876, 2561, 2560, 864, 900, 2758, 2559, 877, 2558, + /* 2350 */ 2793, 2582, 2557, 2556, 2555, 2580, 2563, 2811, 2554, 2553, + /* 2360 */ 2552, 2550, 2549, 2548, 2547, 878, 250, 2544, 2543, 106, + /* 2370 */ 2542, 2541, 2540, 2758, 2546, 877, 2545, 2612, 2581, 2579, + /* 2380 */ 2539, 2538, 2537, 255, 2536, 634, 2535, 1821, 636, 2534, + /* 2390 */ 2532, 1666, 449, 2792, 2811, 1670, 2840, 823, 3019, 2793, + /* 2400 */ 133, 2794, 881, 2796, 2797, 876, 2362, 258, 864, 900, + /* 2410 */ 2758, 2361, 877, 259, 878, 2360, 2900, 2359, 261, 1662, + /* 2420 */ 2792, 2897, 2357, 2840, 2354, 262, 2353, 197, 2794, 881, + /* 2430 */ 2796, 2797, 876, 654, 484, 864, 900, 450, 656, 658, + /* 2440 */ 660, 655, 659, 2811, 2346, 662, 663, 2333, 2793, 664, + /* 2450 */ 666, 2307, 92, 210, 264, 2778, 1543, 2792, 2306, 2758, + /* 2460 */ 2840, 877, 93, 878, 432, 2794, 881, 2796, 2797, 876, + /* 2470 */ 221, 2668, 864, 900, 674, 266, 2793, 2959, 273, 275, + /* 2480 */ 2641, 2618, 278, 485, 2664, 2654, 2642, 280, 2611, 2449, + /* 2490 */ 2793, 878, 2811, 2356, 2352, 695, 697, 1592, 2350, 696, + /* 2500 */ 699, 701, 700, 2348, 703, 878, 2792, 704, 2758, 2840, + /* 2510 */ 877, 705, 2793, 432, 2794, 881, 2796, 2797, 876, 2345, + /* 2520 */ 2811, 864, 900, 707, 708, 2328, 709, 875, 2326, 2327, + /* 2530 */ 2325, 2303, 2451, 1748, 2811, 84, 2758, 1747, 877, 286, + /* 2540 */ 2450, 1651, 1652, 1650, 1647, 1645, 1643, 964, 1642, 1641, + /* 2550 */ 2758, 2343, 877, 1640, 1639, 2792, 2811, 966, 2840, 1636, + /* 2560 */ 1634, 475, 425, 2794, 881, 2796, 2797, 876, 2341, 1635, + /* 2570 */ 864, 900, 2758, 1633, 877, 476, 2332, 477, 743, 2330, + /* 2580 */ 478, 2793, 746, 2792, 2302, 2301, 2840, 2300, 750, 2299, + /* 2590 */ 200, 2794, 881, 2796, 2797, 876, 878, 2792, 864, 900, + /* 2600 */ 2840, 752, 2298, 754, 413, 2794, 881, 2796, 2797, 876, + /* 2610 */ 2297, 139, 864, 900, 756, 813, 1914, 2793, 1916, 2792, + /* 2620 */ 1918, 32, 2840, 2667, 1913, 2811, 431, 2794, 881, 2796, + /* 2630 */ 2797, 876, 878, 67, 864, 900, 78, 2866, 2663, 68, + /* 2640 */ 766, 2758, 1904, 877, 308, 2793, 1887, 1885, 1889, 2653, + /* 2650 */ 777, 191, 3020, 778, 312, 2640, 2639, 783, 1864, 1863, + /* 2660 */ 878, 2811, 3003, 21, 785, 497, 2223, 793, 483, 34, + /* 2670 */ 6, 315, 7, 2197, 795, 797, 317, 2758, 22, 877, + /* 2680 */ 799, 23, 2204, 214, 226, 201, 213, 17, 2792, 2811, + /* 2690 */ 2191, 2840, 35, 2779, 36, 432, 2794, 881, 2796, 2797, + /* 2700 */ 876, 493, 2161, 864, 900, 2758, 227, 877, 2163, 94, + /* 2710 */ 24, 228, 2165, 2243, 76, 2244, 25, 2238, 2237, 2145, + /* 2720 */ 488, 2242, 2241, 489, 2792, 2144, 2793, 2840, 332, 498, + /* 2730 */ 70, 417, 2794, 881, 2796, 2797, 876, 69, 2638, 864, + /* 2740 */ 900, 878, 206, 2617, 117, 18, 118, 2616, 119, 347, + /* 2750 */ 339, 2199, 2792, 2610, 2793, 2840, 216, 844, 345, 432, + /* 2760 */ 2794, 881, 2796, 2797, 876, 120, 80, 864, 900, 878, + /* 2770 */ 2811, 2097, 26, 850, 852, 348, 350, 2096, 13, 11, + /* 2780 */ 1998, 2057, 27, 28, 20, 48, 2758, 2107, 877, 915, + /* 2790 */ 2793, 2056, 207, 918, 921, 217, 2033, 880, 2811, 2609, + /* 2800 */ 886, 121, 2446, 924, 49, 878, 2055, 16, 29, 30, + /* 2810 */ 500, 2025, 884, 81, 2758, 352, 877, 891, 122, 892, + /* 2820 */ 358, 90, 127, 2793, 2850, 2849, 2259, 2258, 2068, 899, + /* 2830 */ 79, 2257, 1729, 2792, 2811, 2256, 2840, 907, 878, 901, + /* 2840 */ 432, 2794, 881, 2796, 2797, 876, 517, 909, 864, 900, + /* 2850 */ 2758, 1726, 877, 911, 914, 912, 917, 920, 1725, 923, + /* 2860 */ 1720, 762, 1722, 1719, 2840, 1716, 2793, 2811, 427, 2794, + /* 2870 */ 881, 2796, 2797, 876, 1718, 1714, 864, 900, 128, 382, + /* 2880 */ 129, 878, 1717, 2758, 1742, 877, 91, 1738, 1590, 938, + /* 2890 */ 1630, 1629, 1628, 1625, 1622, 1621, 1620, 2792, 1619, 2793, + /* 2900 */ 2840, 1660, 1617, 952, 409, 2794, 881, 2796, 2797, 876, + /* 2910 */ 2811, 1615, 864, 900, 878, 1614, 1613, 1659, 233, 1611, + /* 2920 */ 954, 1608, 1610, 1609, 1607, 1606, 2758, 1605, 877, 1656, + /* 2930 */ 2792, 1654, 1602, 2840, 1601, 1596, 2351, 406, 2794, 881, + /* 2940 */ 2796, 2797, 876, 2811, 1598, 864, 900, 1597, 1595, 976, + /* 2950 */ 974, 975, 2349, 978, 980, 979, 2347, 982, 983, 2758, + /* 2960 */ 984, 877, 2344, 2793, 986, 2324, 988, 990, 2322, 987, + /* 2970 */ 992, 1532, 2296, 2792, 996, 1515, 2840, 388, 878, 1520, + /* 2980 */ 410, 2794, 881, 2796, 2797, 876, 2793, 1522, 864, 900, + /* 2990 */ 1000, 2262, 1984, 401, 1003, 1004, 2262, 2262, 2262, 2262, + /* 3000 */ 2262, 878, 2262, 2262, 2262, 2262, 2792, 2811, 2262, 2840, + /* 3010 */ 2262, 2262, 2262, 424, 2794, 881, 2796, 2797, 876, 2262, + /* 3020 */ 2262, 864, 900, 2758, 2262, 877, 2262, 2262, 2262, 2262, + /* 3030 */ 2811, 2262, 2793, 2262, 2262, 2262, 2262, 2262, 2262, 2262, + /* 3040 */ 2262, 2262, 2262, 2262, 2262, 2262, 2758, 878, 877, 2262, + /* 3050 */ 2262, 2262, 2262, 2262, 2793, 2262, 2262, 2262, 2262, 2262, + /* 3060 */ 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2793, 878, + /* 3070 */ 2792, 2262, 2262, 2840, 2262, 2262, 2811, 411, 2794, 881, + /* 3080 */ 2796, 2797, 876, 878, 2262, 864, 900, 2262, 2262, 2262, + /* 3090 */ 2262, 2262, 2758, 2792, 877, 2262, 2840, 2262, 2811, 2262, + /* 3100 */ 412, 2794, 881, 2796, 2797, 876, 2262, 2262, 864, 900, + /* 3110 */ 2262, 2262, 2811, 2262, 2758, 2262, 877, 2262, 2262, 2262, + /* 3120 */ 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2758, 2262, + /* 3130 */ 877, 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2792, + /* 3140 */ 2262, 2262, 2840, 2262, 2793, 2262, 428, 2794, 881, 2796, + /* 3150 */ 2797, 876, 2262, 2262, 864, 900, 2262, 2262, 2262, 878, + /* 3160 */ 2262, 2792, 2262, 2262, 2840, 2262, 2262, 2262, 414, 2794, + /* 3170 */ 881, 2796, 2797, 876, 2262, 2792, 864, 900, 2840, 2793, + /* 3180 */ 2262, 2262, 429, 2794, 881, 2796, 2797, 876, 2811, 2262, + /* 3190 */ 864, 900, 2262, 2262, 878, 2262, 2262, 2262, 2262, 2262, + /* 3200 */ 2262, 2262, 2262, 2262, 2758, 2262, 877, 2262, 2262, 2262, + /* 3210 */ 2262, 2262, 2793, 2262, 2262, 2262, 2262, 2262, 2262, 2262, + /* 3220 */ 2262, 2262, 2262, 2811, 2262, 2262, 2262, 878, 2262, 2262, + /* 3230 */ 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2758, + /* 3240 */ 2262, 877, 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2262, + /* 3250 */ 2262, 2792, 2262, 2262, 2840, 2793, 2811, 2262, 415, 2794, + /* 3260 */ 881, 2796, 2797, 876, 2262, 2262, 864, 900, 2262, 2262, + /* 3270 */ 878, 2262, 2758, 2262, 877, 2262, 2262, 2262, 2262, 2262, + /* 3280 */ 2262, 2262, 2262, 2262, 2262, 2262, 2792, 2262, 2793, 2840, + /* 3290 */ 2262, 2262, 2262, 430, 2794, 881, 2796, 2797, 876, 2811, + /* 3300 */ 2262, 864, 900, 878, 2262, 2262, 2262, 2262, 2262, 2262, + /* 3310 */ 2262, 2262, 2262, 2262, 2262, 2758, 2262, 877, 2262, 2792, + /* 3320 */ 2262, 2262, 2840, 2262, 2262, 2262, 416, 2794, 881, 2796, + /* 3330 */ 2797, 876, 2811, 2262, 864, 900, 2262, 2262, 2262, 2262, + /* 3340 */ 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2758, 2262, + /* 3350 */ 877, 2262, 2793, 2262, 2262, 2262, 2262, 2262, 2262, 2262, + /* 3360 */ 2262, 2262, 2792, 2262, 2262, 2840, 2262, 878, 2262, 407, + /* 3370 */ 2794, 881, 2796, 2797, 876, 2793, 2262, 864, 900, 2262, + /* 3380 */ 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2262, + /* 3390 */ 878, 2262, 2262, 2262, 2262, 2792, 2811, 2262, 2840, 2262, + /* 3400 */ 2262, 2262, 418, 2794, 881, 2796, 2797, 876, 2262, 2262, + /* 3410 */ 864, 900, 2758, 2262, 877, 2262, 2262, 2262, 2262, 2811, + /* 3420 */ 2262, 2793, 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2262, + /* 3430 */ 2262, 2262, 2262, 2262, 2262, 2758, 878, 877, 2262, 2262, + /* 3440 */ 2262, 2262, 2262, 2793, 2262, 2262, 2262, 2262, 2262, 2262, + /* 3450 */ 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2793, 878, 2792, + /* 3460 */ 2262, 2262, 2840, 2262, 2262, 2811, 419, 2794, 881, 2796, + /* 3470 */ 2797, 876, 878, 2262, 864, 900, 2262, 2262, 2262, 2262, + /* 3480 */ 2262, 2758, 2792, 877, 2262, 2840, 2262, 2811, 2262, 420, + /* 3490 */ 2794, 881, 2796, 2797, 876, 2262, 2262, 864, 900, 2262, + /* 3500 */ 2262, 2811, 2262, 2758, 2262, 877, 2262, 2262, 2262, 2262, + /* 3510 */ 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2758, 2262, 877, + /* 3520 */ 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2792, 2262, + /* 3530 */ 2262, 2840, 2262, 2793, 2262, 421, 2794, 881, 2796, 2797, + /* 3540 */ 876, 2262, 2262, 864, 900, 2262, 2262, 2262, 878, 2262, + /* 3550 */ 2792, 2262, 2262, 2840, 2262, 2262, 2262, 437, 2794, 881, + /* 3560 */ 2796, 2797, 876, 2262, 2792, 864, 900, 2840, 2793, 2262, + /* 3570 */ 2262, 438, 2794, 881, 2796, 2797, 876, 2811, 2262, 864, + /* 3580 */ 900, 2262, 2262, 878, 2262, 2262, 2262, 2262, 2262, 2262, + /* 3590 */ 2262, 2262, 2262, 2758, 2262, 877, 2262, 2262, 2262, 2262, + /* 3600 */ 2262, 2793, 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2262, + /* 3610 */ 2262, 2262, 2811, 2262, 2262, 2262, 878, 2262, 2262, 2262, + /* 3620 */ 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2758, 2262, + /* 3630 */ 877, 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2262, + /* 3640 */ 2792, 2262, 2262, 2840, 2793, 2811, 2262, 2805, 2794, 881, + /* 3650 */ 2796, 2797, 876, 2262, 2262, 864, 900, 2262, 2262, 878, + /* 3660 */ 2262, 2758, 2262, 877, 2262, 2262, 2262, 2262, 2262, 2262, + /* 3670 */ 2262, 2262, 2262, 2262, 2262, 2792, 2262, 2793, 2840, 2262, + /* 3680 */ 2262, 2262, 2804, 2794, 881, 2796, 2797, 876, 2811, 2262, + /* 3690 */ 864, 900, 878, 2262, 2262, 2262, 2262, 2262, 2262, 2262, + /* 3700 */ 2262, 2262, 2262, 2262, 2758, 2262, 877, 2262, 2792, 2262, + /* 3710 */ 2262, 2840, 2262, 2262, 2262, 2803, 2794, 881, 2796, 2797, + /* 3720 */ 876, 2811, 2262, 864, 900, 2262, 2262, 2262, 2262, 2262, + /* 3730 */ 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2758, 2262, 877, + /* 3740 */ 2262, 2793, 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2262, + /* 3750 */ 2262, 2792, 2262, 2262, 2840, 2262, 878, 2262, 457, 2794, + /* 3760 */ 881, 2796, 2797, 876, 2793, 2262, 864, 900, 2262, 2262, + /* 3770 */ 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2262, 878, + /* 3780 */ 2262, 2262, 2262, 2262, 2792, 2811, 2262, 2840, 2262, 2262, + /* 3790 */ 2262, 458, 2794, 881, 2796, 2797, 876, 2262, 2262, 864, + /* 3800 */ 900, 2758, 2262, 877, 2262, 2262, 2262, 2262, 2811, 2262, + /* 3810 */ 2793, 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2262, + /* 3820 */ 2262, 2262, 2262, 2262, 2758, 878, 877, 2262, 2262, 2262, + /* 3830 */ 2262, 2262, 2793, 2262, 2262, 2262, 2262, 2262, 2262, 2262, + /* 3840 */ 2262, 2262, 2262, 2262, 2262, 2262, 2262, 878, 2792, 2262, + /* 3850 */ 2262, 2840, 2262, 2262, 2811, 454, 2794, 881, 2796, 2797, + /* 3860 */ 876, 2262, 2262, 864, 900, 2262, 2262, 2262, 2262, 2262, + /* 3870 */ 2758, 2792, 877, 2262, 2840, 2262, 2811, 2262, 459, 2794, + /* 3880 */ 881, 2796, 2797, 876, 2262, 2262, 864, 900, 2262, 2262, + /* 3890 */ 2262, 2262, 2758, 2262, 877, 2262, 2262, 2262, 2262, 2262, + /* 3900 */ 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2262, 2262, + /* 3910 */ 2262, 2262, 2262, 2262, 2262, 2262, 2262, 879, 2262, 2262, + /* 3920 */ 2840, 2262, 2262, 2262, 427, 2794, 881, 2796, 2797, 876, + /* 3930 */ 2262, 2262, 864, 900, 2262, 2262, 2262, 2262, 2262, 2792, + /* 3940 */ 2262, 2262, 2840, 2262, 2262, 2262, 426, 2794, 881, 2796, + /* 3950 */ 2797, 876, 2262, 2262, 864, 900, }; static const YYCODETYPE yy_lookahead[] = { /* 0 */ 530, 397, 530, 397, 400, 401, 400, 401, 538, 433, @@ -1087,10 +1089,10 @@ static const YYCODETYPE yy_lookahead[] = { /* 540 */ 269, 270, 271, 272, 33, 191, 192, 276, 277, 278, /* 550 */ 279, 280, 281, 282, 307, 201, 202, 402, 171, 82, /* 560 */ 307, 41, 42, 176, 112, 498, 499, 500, 146, 155, - /* 570 */ 216, 184, 218, 431, 402, 403, 0, 223, 511, 127, + /* 570 */ 216, 184, 218, 431, 402, 403, 20, 223, 511, 127, /* 580 */ 128, 129, 130, 131, 132, 133, 134, 135, 136, 447, /* 590 */ 138, 139, 140, 141, 142, 143, 144, 8, 9, 152, - /* 600 */ 153, 12, 13, 14, 15, 16, 252, 253, 254, 33, + /* 600 */ 153, 12, 13, 14, 15, 16, 252, 253, 254, 126, /* 610 */ 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, /* 620 */ 266, 466, 467, 269, 270, 271, 272, 116, 501, 450, /* 630 */ 276, 277, 278, 279, 280, 281, 282, 283, 12, 13, @@ -1104,8 +1106,8 @@ static const YYCODETYPE yy_lookahead[] = { /* 710 */ 296, 297, 298, 299, 300, 301, 302, 303, 304, 402, /* 720 */ 403, 12, 13, 14, 432, 225, 73, 419, 58, 20, /* 730 */ 230, 22, 388, 233, 254, 235, 459, 14, 112, 422, - /* 740 */ 448, 115, 450, 20, 432, 36, 20, 38, 22, 59, - /* 750 */ 60, 448, 495, 441, 442, 498, 448, 402, 403, 502, + /* 740 */ 448, 115, 450, 20, 432, 36, 20, 38, 22, 193, + /* 750 */ 20, 448, 495, 441, 442, 498, 448, 402, 403, 502, /* 760 */ 503, 504, 505, 506, 507, 13, 509, 510, 511, 313, /* 770 */ 314, 315, 316, 402, 403, 402, 403, 68, 152, 153, /* 780 */ 71, 526, 527, 528, 58, 530, 531, 495, 36, 474, @@ -1115,221 +1117,221 @@ static const YYCODETYPE yy_lookahead[] = { /* 820 */ 443, 112, 218, 74, 115, 152, 153, 201, 202, 456, /* 830 */ 8, 9, 422, 426, 12, 13, 14, 15, 16, 77, /* 840 */ 78, 79, 216, 436, 218, 530, 84, 85, 86, 223, - /* 850 */ 126, 155, 90, 538, 250, 251, 267, 95, 96, 97, + /* 850 */ 0, 155, 90, 538, 250, 251, 267, 95, 96, 97, /* 860 */ 98, 152, 153, 101, 211, 474, 170, 105, 106, 107, - /* 870 */ 108, 556, 557, 388, 201, 202, 561, 562, 252, 253, + /* 870 */ 108, 556, 557, 20, 201, 202, 561, 562, 252, 253, /* 880 */ 254, 411, 256, 257, 258, 259, 260, 261, 262, 263, /* 890 */ 264, 265, 266, 240, 241, 269, 270, 271, 272, 429, /* 900 */ 191, 192, 276, 277, 278, 279, 280, 281, 282, 439, - /* 910 */ 201, 202, 8, 9, 191, 33, 12, 13, 14, 15, - /* 920 */ 16, 530, 398, 402, 403, 216, 402, 218, 404, 538, - /* 930 */ 20, 423, 223, 448, 402, 403, 432, 283, 36, 285, - /* 940 */ 432, 474, 20, 422, 440, 432, 479, 556, 557, 20, - /* 950 */ 474, 443, 561, 562, 422, 451, 402, 403, 34, 12, - /* 960 */ 13, 252, 253, 254, 451, 256, 257, 258, 259, 260, - /* 970 */ 261, 262, 263, 264, 265, 266, 422, 254, 269, 270, - /* 980 */ 271, 272, 432, 36, 13, 276, 277, 278, 279, 280, - /* 990 */ 281, 282, 12, 13, 22, 388, 432, 530, 116, 22, - /* 1000 */ 20, 451, 22, 52, 440, 538, 530, 36, 36, 23, - /* 1010 */ 403, 402, 403, 36, 538, 451, 36, 515, 38, 517, - /* 1020 */ 116, 402, 403, 556, 557, 402, 403, 47, 561, 562, - /* 1030 */ 208, 422, 556, 557, 530, 49, 50, 561, 562, 432, - /* 1040 */ 388, 422, 538, 514, 388, 422, 517, 432, 68, 449, - /* 1050 */ 450, 71, 402, 403, 82, 448, 515, 450, 517, 403, - /* 1060 */ 556, 557, 82, 8, 9, 561, 562, 12, 13, 14, - /* 1070 */ 15, 16, 422, 8, 9, 12, 13, 12, 13, 14, - /* 1080 */ 15, 16, 433, 20, 112, 22, 407, 408, 432, 112, - /* 1090 */ 407, 408, 112, 444, 409, 115, 402, 403, 0, 36, - /* 1100 */ 448, 38, 495, 193, 448, 498, 450, 492, 493, 502, - /* 1110 */ 503, 504, 505, 506, 507, 193, 422, 510, 511, 388, - /* 1120 */ 218, 436, 193, 516, 91, 518, 14, 15, 16, 522, - /* 1130 */ 523, 68, 152, 153, 71, 68, 2, 300, 301, 302, - /* 1140 */ 303, 304, 8, 9, 193, 82, 12, 13, 14, 15, - /* 1150 */ 16, 495, 250, 251, 498, 204, 334, 550, 502, 503, - /* 1160 */ 504, 505, 506, 507, 254, 218, 510, 511, 432, 402, - /* 1170 */ 403, 191, 192, 402, 403, 112, 440, 20, 115, 448, - /* 1180 */ 432, 201, 202, 254, 402, 403, 119, 451, 216, 422, - /* 1190 */ 218, 443, 159, 422, 402, 403, 216, 433, 218, 2, - /* 1200 */ 402, 403, 428, 223, 422, 8, 9, 551, 552, 12, - /* 1210 */ 13, 14, 15, 16, 422, 152, 153, 184, 185, 501, - /* 1220 */ 422, 433, 8, 9, 252, 253, 12, 13, 14, 15, - /* 1230 */ 16, 198, 252, 253, 254, 388, 256, 257, 258, 259, - /* 1240 */ 260, 261, 262, 263, 264, 265, 266, 529, 193, 269, - /* 1250 */ 270, 271, 272, 155, 191, 192, 276, 277, 278, 279, - /* 1260 */ 280, 281, 282, 489, 201, 202, 8, 9, 22, 14, - /* 1270 */ 12, 13, 14, 15, 16, 20, 38, 402, 403, 216, - /* 1280 */ 0, 218, 36, 8, 9, 0, 223, 12, 13, 14, - /* 1290 */ 15, 16, 54, 402, 403, 448, 433, 422, 402, 403, - /* 1300 */ 432, 63, 64, 65, 66, 47, 68, 0, 416, 417, - /* 1310 */ 442, 402, 403, 422, 3, 252, 253, 254, 422, 256, + /* 910 */ 201, 202, 8, 9, 191, 433, 12, 13, 14, 15, + /* 920 */ 16, 530, 432, 193, 74, 216, 444, 218, 91, 538, + /* 930 */ 402, 403, 223, 398, 12, 13, 432, 402, 36, 404, + /* 940 */ 423, 474, 22, 22, 440, 432, 479, 556, 557, 432, + /* 950 */ 422, 47, 561, 562, 0, 451, 36, 36, 36, 388, + /* 960 */ 443, 252, 253, 254, 451, 256, 257, 258, 259, 260, + /* 970 */ 261, 262, 263, 264, 265, 266, 22, 254, 269, 270, + /* 980 */ 271, 272, 492, 493, 254, 276, 277, 278, 279, 280, + /* 990 */ 281, 282, 12, 13, 22, 388, 159, 530, 432, 283, + /* 1000 */ 20, 285, 22, 82, 388, 538, 440, 190, 36, 186, + /* 1010 */ 403, 300, 301, 302, 303, 304, 36, 451, 38, 448, + /* 1020 */ 116, 184, 185, 556, 557, 402, 403, 47, 561, 562, + /* 1030 */ 208, 388, 112, 112, 530, 198, 402, 403, 68, 432, + /* 1040 */ 449, 450, 538, 220, 388, 422, 193, 428, 68, 402, + /* 1050 */ 403, 71, 402, 403, 82, 448, 422, 450, 501, 403, + /* 1060 */ 556, 557, 82, 432, 448, 561, 562, 402, 403, 422, + /* 1070 */ 402, 403, 422, 402, 403, 12, 13, 402, 403, 34, + /* 1080 */ 402, 403, 451, 20, 267, 22, 529, 422, 432, 119, + /* 1090 */ 422, 448, 112, 422, 145, 115, 501, 422, 149, 36, + /* 1100 */ 422, 38, 495, 286, 448, 498, 450, 254, 489, 502, + /* 1110 */ 503, 504, 505, 506, 507, 402, 403, 510, 511, 432, + /* 1120 */ 218, 402, 403, 516, 529, 518, 388, 440, 13, 522, + /* 1130 */ 523, 68, 152, 153, 71, 422, 2, 216, 451, 218, + /* 1140 */ 218, 422, 8, 9, 432, 82, 12, 13, 14, 15, + /* 1150 */ 16, 495, 250, 251, 498, 443, 334, 550, 502, 503, + /* 1160 */ 504, 505, 506, 507, 59, 60, 510, 511, 14, 15, + /* 1170 */ 16, 191, 192, 252, 253, 112, 433, 20, 115, 402, + /* 1180 */ 403, 201, 202, 8, 9, 432, 448, 12, 13, 14, + /* 1190 */ 15, 16, 409, 440, 407, 408, 216, 82, 218, 422, + /* 1200 */ 407, 408, 433, 223, 451, 8, 9, 551, 552, 12, + /* 1210 */ 13, 14, 15, 16, 2, 152, 153, 388, 388, 436, + /* 1220 */ 8, 9, 1, 2, 12, 13, 14, 15, 16, 416, + /* 1230 */ 417, 424, 252, 253, 254, 428, 256, 257, 258, 259, + /* 1240 */ 260, 261, 262, 263, 264, 265, 266, 411, 388, 269, + /* 1250 */ 270, 271, 272, 33, 191, 192, 276, 277, 278, 279, + /* 1260 */ 280, 281, 282, 52, 201, 202, 8, 9, 402, 403, + /* 1270 */ 12, 13, 14, 15, 16, 439, 38, 448, 448, 216, + /* 1280 */ 0, 218, 515, 515, 517, 517, 223, 33, 422, 8, + /* 1290 */ 9, 116, 54, 12, 13, 14, 15, 16, 402, 403, + /* 1300 */ 433, 63, 64, 65, 66, 47, 68, 0, 448, 514, + /* 1310 */ 402, 403, 517, 116, 20, 252, 253, 254, 422, 256, /* 1320 */ 257, 258, 259, 260, 261, 262, 263, 264, 265, 266, - /* 1330 */ 116, 422, 269, 270, 271, 272, 1, 2, 190, 276, + /* 1330 */ 422, 20, 269, 270, 271, 272, 116, 116, 388, 276, /* 1340 */ 277, 278, 279, 280, 281, 282, 12, 13, 191, 388, - /* 1350 */ 22, 18, 114, 20, 20, 1, 22, 119, 112, 74, + /* 1350 */ 22, 18, 114, 20, 20, 1, 22, 119, 3, 23, /* 1360 */ 27, 8, 9, 30, 36, 12, 13, 14, 15, 16, /* 1370 */ 36, 38, 38, 19, 8, 9, 402, 403, 12, 13, - /* 1380 */ 14, 15, 16, 402, 403, 432, 424, 54, 388, 56, - /* 1390 */ 428, 116, 38, 440, 61, 62, 422, 13, 416, 417, - /* 1400 */ 47, 388, 68, 422, 451, 71, 73, 53, 54, 448, - /* 1410 */ 82, 254, 501, 0, 388, 267, 82, 63, 64, 65, - /* 1420 */ 66, 20, 68, 20, 186, 145, 146, 147, 148, 149, - /* 1430 */ 150, 151, 411, 195, 286, 33, 445, 199, 200, 448, - /* 1440 */ 529, 44, 445, 205, 206, 448, 112, 114, 448, 115, - /* 1450 */ 36, 116, 145, 146, 147, 148, 149, 150, 151, 126, - /* 1460 */ 439, 448, 224, 388, 388, 68, 82, 54, 114, 388, + /* 1380 */ 14, 15, 16, 402, 403, 49, 50, 54, 44, 56, + /* 1390 */ 402, 403, 38, 14, 61, 62, 422, 13, 448, 20, + /* 1400 */ 388, 388, 68, 422, 193, 71, 73, 53, 54, 448, + /* 1410 */ 422, 254, 68, 47, 388, 204, 82, 63, 64, 65, + /* 1420 */ 66, 388, 68, 388, 186, 145, 146, 147, 148, 149, + /* 1430 */ 150, 151, 432, 195, 416, 417, 445, 199, 200, 448, + /* 1440 */ 112, 433, 442, 205, 206, 33, 112, 114, 432, 115, + /* 1450 */ 391, 392, 145, 146, 147, 148, 149, 150, 151, 126, + /* 1460 */ 448, 448, 224, 388, 388, 0, 82, 451, 114, 116, /* 1470 */ 388, 8, 9, 119, 448, 12, 13, 14, 15, 16, - /* 1480 */ 433, 474, 116, 474, 388, 33, 152, 153, 388, 445, - /* 1490 */ 157, 158, 448, 160, 161, 162, 163, 164, 165, 166, - /* 1500 */ 167, 168, 169, 13, 150, 172, 173, 174, 175, 176, - /* 1510 */ 177, 178, 179, 388, 181, 182, 183, 388, 116, 432, - /* 1520 */ 187, 188, 189, 448, 448, 191, 192, 194, 186, 448, - /* 1530 */ 448, 391, 392, 152, 153, 201, 202, 530, 451, 530, - /* 1540 */ 0, 388, 33, 120, 448, 538, 123, 538, 448, 195, - /* 1550 */ 216, 197, 218, 33, 200, 126, 145, 223, 480, 205, - /* 1560 */ 149, 33, 220, 556, 557, 556, 557, 47, 561, 562, - /* 1570 */ 561, 562, 82, 448, 234, 47, 236, 448, 224, 116, - /* 1580 */ 120, 36, 120, 123, 44, 123, 252, 253, 254, 36, + /* 1480 */ 0, 448, 445, 448, 126, 448, 152, 153, 0, 33, + /* 1490 */ 157, 158, 33, 160, 161, 162, 163, 164, 165, 166, + /* 1500 */ 167, 168, 169, 47, 150, 172, 173, 174, 175, 176, + /* 1510 */ 177, 178, 179, 0, 181, 182, 183, 36, 0, 54, + /* 1520 */ 187, 188, 189, 448, 448, 191, 192, 194, 445, 33, + /* 1530 */ 448, 448, 44, 0, 120, 201, 202, 123, 180, 120, + /* 1540 */ 22, 120, 123, 47, 123, 13, 33, 0, 254, 195, + /* 1550 */ 216, 197, 218, 120, 200, 22, 123, 223, 234, 205, + /* 1560 */ 236, 33, 36, 82, 433, 254, 8, 9, 36, 22, + /* 1570 */ 12, 13, 14, 15, 16, 116, 51, 33, 224, 116, + /* 1580 */ 152, 153, 433, 33, 33, 331, 252, 253, 254, 480, /* 1590 */ 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, - /* 1600 */ 266, 448, 0, 269, 270, 271, 272, 33, 33, 180, - /* 1610 */ 276, 277, 278, 279, 280, 281, 282, 18, 33, 33, - /* 1620 */ 120, 388, 23, 123, 22, 116, 433, 82, 0, 0, - /* 1630 */ 0, 51, 218, 33, 33, 82, 403, 33, 405, 33, - /* 1640 */ 329, 42, 43, 1, 2, 46, 33, 388, 36, 33, - /* 1650 */ 22, 22, 22, 75, 36, 254, 57, 254, 33, 33, - /* 1660 */ 12, 13, 403, 433, 405, 432, 389, 33, 69, 70, - /* 1670 */ 71, 72, 73, 12, 13, 12, 13, 33, 12, 13, - /* 1680 */ 115, 448, 117, 450, 33, 33, 12, 13, 12, 13, - /* 1690 */ 116, 432, 12, 13, 82, 115, 12, 13, 12, 13, - /* 1700 */ 13, 116, 116, 12, 13, 12, 13, 448, 33, 450, - /* 1710 */ 33, 33, 33, 420, 115, 33, 116, 116, 33, 532, - /* 1720 */ 116, 460, 116, 36, 13, 0, 565, 554, 495, 116, - /* 1730 */ 547, 498, 116, 533, 432, 502, 503, 504, 505, 506, - /* 1740 */ 507, 116, 116, 510, 511, 406, 419, 36, 419, 516, - /* 1750 */ 116, 518, 401, 154, 495, 522, 523, 498, 460, 388, - /* 1760 */ 116, 502, 503, 504, 505, 506, 507, 116, 116, 510, - /* 1770 */ 511, 553, 553, 553, 403, 516, 405, 518, 473, 553, - /* 1780 */ 55, 522, 523, 331, 460, 406, 496, 481, 403, 537, - /* 1790 */ 447, 116, 460, 116, 116, 116, 460, 537, 116, 558, - /* 1800 */ 524, 116, 540, 432, 460, 388, 310, 208, 209, 210, - /* 1810 */ 421, 54, 213, 475, 497, 20, 20, 402, 411, 448, - /* 1820 */ 403, 450, 405, 486, 233, 226, 227, 486, 214, 411, - /* 1830 */ 252, 20, 12, 13, 477, 388, 218, 491, 239, 402, - /* 1840 */ 47, 242, 22, 403, 245, 246, 247, 248, 249, 432, - /* 1850 */ 403, 457, 405, 403, 190, 457, 36, 402, 38, 454, - /* 1860 */ 403, 402, 454, 457, 454, 448, 495, 450, 113, 498, - /* 1870 */ 454, 415, 111, 502, 503, 504, 505, 506, 507, 432, - /* 1880 */ 414, 510, 511, 402, 402, 110, 388, 516, 68, 518, - /* 1890 */ 413, 402, 20, 522, 523, 448, 402, 450, 395, 402, - /* 1900 */ 52, 403, 82, 405, 399, 395, 307, 399, 333, 20, - /* 1910 */ 486, 450, 495, 411, 20, 498, 404, 411, 411, 502, - /* 1920 */ 503, 504, 505, 506, 507, 20, 411, 510, 511, 476, - /* 1930 */ 432, 404, 112, 516, 411, 518, 20, 467, 20, 522, - /* 1940 */ 523, 411, 495, 411, 461, 498, 448, 411, 450, 502, - /* 1950 */ 503, 504, 505, 506, 507, 411, 402, 510, 511, 395, - /* 1960 */ 402, 12, 13, 516, 391, 518, 432, 432, 391, 522, - /* 1970 */ 523, 22, 448, 432, 432, 432, 395, 448, 432, 432, - /* 1980 */ 432, 432, 432, 432, 237, 36, 448, 38, 115, 490, - /* 1990 */ 20, 409, 488, 495, 222, 388, 498, 486, 485, 483, - /* 2000 */ 502, 503, 504, 505, 506, 507, 221, 482, 510, 511, - /* 2010 */ 403, 450, 409, 402, 516, 448, 518, 68, 318, 546, - /* 2020 */ 522, 523, 317, 468, 546, 388, 207, 475, 468, 326, - /* 2030 */ 328, 546, 545, 549, 548, 327, 216, 544, 218, 432, - /* 2040 */ 403, 311, 306, 475, 543, 305, 566, 335, 332, 330, - /* 2050 */ 536, 560, 535, 403, 559, 448, 20, 450, 501, 388, - /* 2060 */ 126, 308, 541, 404, 409, 539, 409, 448, 448, 432, - /* 2070 */ 448, 448, 252, 253, 403, 448, 468, 468, 199, 409, - /* 2080 */ 465, 448, 461, 115, 521, 448, 448, 450, 428, 269, - /* 2090 */ 270, 199, 409, 462, 448, 403, 276, 277, 278, 279, - /* 2100 */ 280, 281, 495, 432, 461, 498, 409, 409, 115, 502, - /* 2110 */ 503, 504, 505, 506, 507, 448, 448, 510, 511, 448, - /* 2120 */ 448, 450, 409, 516, 390, 518, 22, 35, 402, 522, - /* 2130 */ 523, 437, 495, 37, 393, 498, 448, 448, 448, 502, - /* 2140 */ 503, 504, 505, 506, 507, 394, 388, 510, 511, 448, - /* 2150 */ 40, 395, 448, 516, 448, 518, 396, 448, 448, 522, - /* 2160 */ 523, 403, 448, 448, 469, 216, 495, 218, 448, 498, - /* 2170 */ 448, 448, 448, 502, 503, 504, 505, 506, 507, 448, - /* 2180 */ 448, 510, 511, 448, 434, 448, 448, 425, 425, 518, - /* 2190 */ 432, 425, 388, 522, 523, 478, 410, 469, 434, 250, - /* 2200 */ 251, 252, 386, 0, 0, 0, 448, 403, 450, 47, - /* 2210 */ 0, 36, 494, 36, 487, 243, 36, 36, 269, 270, - /* 2220 */ 243, 0, 388, 36, 36, 276, 277, 278, 279, 280, - /* 2230 */ 281, 243, 36, 0, 0, 243, 432, 403, 0, 0, - /* 2240 */ 0, 22, 36, 36, 0, 36, 238, 0, 224, 0, - /* 2250 */ 224, 218, 448, 495, 450, 225, 498, 216, 0, 0, + /* 1600 */ 266, 1, 2, 269, 270, 271, 272, 33, 82, 33, + /* 1610 */ 276, 277, 278, 279, 280, 281, 282, 18, 33, 8, + /* 1620 */ 9, 388, 23, 12, 13, 14, 15, 16, 420, 474, + /* 1630 */ 33, 12, 13, 389, 75, 155, 403, 33, 405, 33, + /* 1640 */ 115, 42, 43, 33, 116, 46, 33, 388, 36, 33, + /* 1650 */ 33, 33, 12, 13, 460, 532, 57, 12, 13, 474, + /* 1660 */ 116, 115, 403, 117, 405, 432, 116, 116, 69, 70, + /* 1670 */ 71, 72, 73, 565, 116, 33, 13, 12, 13, 12, + /* 1680 */ 13, 448, 36, 450, 329, 530, 554, 12, 13, 547, + /* 1690 */ 116, 432, 116, 538, 12, 13, 474, 12, 13, 36, + /* 1700 */ 533, 116, 33, 12, 13, 12, 13, 448, 406, 450, + /* 1710 */ 432, 556, 557, 116, 115, 530, 561, 562, 12, 13, + /* 1720 */ 116, 401, 116, 538, 419, 419, 116, 33, 495, 116, + /* 1730 */ 33, 498, 116, 116, 116, 502, 503, 504, 505, 506, + /* 1740 */ 507, 556, 557, 510, 511, 333, 561, 562, 33, 516, + /* 1750 */ 13, 518, 530, 154, 495, 522, 523, 498, 116, 388, + /* 1760 */ 538, 502, 503, 504, 505, 506, 507, 553, 33, 510, + /* 1770 */ 511, 33, 36, 36, 403, 516, 405, 518, 556, 557, + /* 1780 */ 553, 522, 523, 561, 562, 116, 460, 553, 473, 553, + /* 1790 */ 460, 406, 496, 403, 447, 481, 524, 460, 460, 537, + /* 1800 */ 537, 558, 460, 432, 193, 388, 0, 208, 209, 210, + /* 1810 */ 116, 252, 213, 116, 540, 421, 310, 54, 82, 448, + /* 1820 */ 403, 450, 405, 475, 497, 226, 227, 20, 402, 20, + /* 1830 */ 218, 116, 12, 13, 233, 388, 486, 491, 239, 214, + /* 1840 */ 411, 242, 22, 486, 245, 246, 247, 248, 249, 432, + /* 1850 */ 403, 116, 405, 411, 116, 477, 36, 402, 38, 20, + /* 1860 */ 403, 55, 47, 457, 218, 448, 495, 450, 457, 498, + /* 1870 */ 403, 190, 454, 502, 503, 504, 505, 506, 507, 432, + /* 1880 */ 402, 510, 511, 113, 457, 403, 388, 516, 68, 518, + /* 1890 */ 454, 402, 454, 522, 523, 448, 111, 450, 454, 415, + /* 1900 */ 414, 403, 82, 405, 402, 402, 307, 110, 413, 402, + /* 1910 */ 402, 402, 495, 20, 395, 498, 52, 395, 486, 502, + /* 1920 */ 503, 504, 505, 506, 507, 411, 399, 510, 511, 399, + /* 1930 */ 432, 411, 112, 516, 20, 518, 20, 450, 404, 522, + /* 1940 */ 523, 20, 495, 404, 476, 498, 448, 411, 450, 502, + /* 1950 */ 503, 504, 505, 506, 507, 20, 411, 510, 511, 411, + /* 1960 */ 467, 12, 13, 516, 411, 518, 20, 411, 411, 522, + /* 1970 */ 523, 22, 461, 402, 411, 395, 237, 391, 391, 402, + /* 1980 */ 490, 432, 115, 395, 432, 36, 432, 38, 448, 448, + /* 1990 */ 448, 486, 432, 495, 432, 388, 498, 432, 432, 432, + /* 2000 */ 502, 503, 504, 505, 506, 507, 20, 432, 510, 511, + /* 2010 */ 403, 432, 488, 432, 516, 222, 518, 68, 409, 221, + /* 2020 */ 522, 523, 485, 450, 409, 388, 483, 318, 402, 546, + /* 2030 */ 448, 317, 468, 326, 549, 207, 216, 482, 218, 432, + /* 2040 */ 403, 328, 548, 475, 327, 546, 468, 546, 306, 545, + /* 2050 */ 305, 311, 536, 335, 544, 448, 535, 450, 543, 388, + /* 2060 */ 332, 330, 403, 566, 475, 20, 126, 308, 539, 432, + /* 2070 */ 501, 559, 252, 253, 403, 409, 541, 560, 404, 409, + /* 2080 */ 448, 448, 468, 448, 448, 448, 448, 450, 468, 269, + /* 2090 */ 270, 199, 465, 409, 115, 448, 276, 277, 278, 279, + /* 2100 */ 280, 281, 495, 432, 461, 498, 448, 409, 199, 502, + /* 2110 */ 503, 504, 505, 506, 507, 521, 462, 510, 511, 448, + /* 2120 */ 409, 450, 461, 516, 428, 518, 403, 448, 409, 522, + /* 2130 */ 523, 448, 495, 115, 448, 498, 448, 448, 448, 502, + /* 2140 */ 503, 504, 505, 506, 507, 22, 388, 510, 511, 448, + /* 2150 */ 448, 448, 448, 516, 402, 518, 448, 390, 448, 522, + /* 2160 */ 523, 403, 437, 35, 448, 216, 495, 218, 448, 498, + /* 2170 */ 448, 409, 448, 502, 503, 504, 505, 506, 507, 393, + /* 2180 */ 448, 510, 511, 37, 394, 40, 494, 396, 410, 518, + /* 2190 */ 432, 448, 388, 522, 523, 448, 448, 448, 448, 250, + /* 2200 */ 251, 252, 448, 395, 478, 434, 448, 403, 450, 425, + /* 2210 */ 469, 434, 425, 469, 487, 386, 0, 425, 269, 270, + /* 2220 */ 0, 0, 388, 47, 0, 276, 277, 278, 279, 280, + /* 2230 */ 281, 36, 243, 36, 36, 36, 432, 403, 243, 0, + /* 2240 */ 36, 36, 243, 36, 0, 0, 243, 0, 36, 0, + /* 2250 */ 36, 0, 448, 495, 450, 22, 498, 238, 36, 0, /* 2260 */ 502, 503, 504, 505, 506, 507, 432, 0, 510, 511, - /* 2270 */ 212, 0, 211, 0, 158, 51, 518, 51, 0, 36, - /* 2280 */ 522, 523, 448, 0, 450, 0, 388, 36, 0, 54, - /* 2290 */ 0, 0, 51, 47, 0, 0, 51, 0, 0, 495, - /* 2300 */ 0, 403, 498, 0, 0, 0, 502, 503, 504, 505, - /* 2310 */ 506, 507, 176, 388, 510, 511, 36, 0, 176, 0, - /* 2320 */ 0, 0, 518, 0, 0, 0, 522, 523, 403, 495, - /* 2330 */ 432, 0, 498, 0, 0, 0, 502, 503, 504, 505, + /* 2270 */ 224, 0, 224, 218, 225, 216, 518, 0, 0, 0, + /* 2280 */ 522, 523, 448, 212, 450, 211, 388, 0, 0, 158, + /* 2290 */ 0, 51, 51, 0, 36, 0, 36, 54, 0, 495, + /* 2300 */ 0, 403, 498, 51, 0, 0, 502, 503, 504, 505, + /* 2310 */ 506, 507, 47, 388, 510, 511, 0, 51, 0, 0, + /* 2320 */ 0, 0, 518, 0, 0, 176, 522, 523, 403, 495, + /* 2330 */ 432, 0, 498, 36, 176, 0, 502, 503, 504, 505, /* 2340 */ 506, 507, 0, 0, 510, 511, 448, 0, 450, 0, - /* 2350 */ 388, 0, 0, 0, 0, 0, 0, 432, 51, 0, - /* 2360 */ 47, 0, 0, 0, 0, 403, 22, 0, 0, 158, - /* 2370 */ 157, 0, 156, 448, 0, 450, 0, 0, 0, 0, - /* 2380 */ 0, 0, 22, 0, 52, 0, 68, 22, 52, 0, - /* 2390 */ 68, 0, 0, 495, 432, 68, 498, 563, 564, 388, - /* 2400 */ 502, 503, 504, 505, 506, 507, 36, 68, 510, 511, - /* 2410 */ 448, 0, 450, 44, 403, 0, 518, 44, 36, 0, - /* 2420 */ 495, 523, 36, 498, 36, 0, 54, 502, 503, 504, - /* 2430 */ 505, 506, 507, 36, 472, 510, 511, 44, 54, 0, - /* 2440 */ 44, 54, 47, 432, 14, 33, 0, 44, 388, 51, - /* 2450 */ 51, 51, 45, 0, 0, 0, 0, 495, 44, 448, - /* 2460 */ 498, 450, 207, 403, 502, 503, 504, 505, 506, 507, - /* 2470 */ 0, 51, 510, 511, 0, 51, 388, 552, 36, 0, - /* 2480 */ 76, 0, 0, 472, 0, 0, 54, 44, 36, 44, - /* 2490 */ 388, 403, 432, 0, 54, 36, 0, 36, 54, 0, - /* 2500 */ 0, 44, 54, 0, 44, 403, 495, 0, 448, 498, - /* 2510 */ 450, 36, 388, 502, 503, 504, 505, 506, 507, 0, - /* 2520 */ 432, 510, 511, 0, 22, 0, 125, 403, 123, 0, - /* 2530 */ 22, 36, 0, 36, 432, 22, 448, 36, 450, 0, - /* 2540 */ 36, 36, 36, 22, 22, 36, 36, 33, 36, 33, - /* 2550 */ 448, 36, 450, 36, 36, 495, 432, 22, 498, 0, - /* 2560 */ 22, 0, 502, 503, 504, 505, 506, 507, 0, 36, - /* 2570 */ 510, 511, 448, 0, 450, 22, 56, 36, 0, 36, - /* 2580 */ 0, 388, 36, 495, 0, 20, 498, 36, 36, 116, + /* 2350 */ 388, 0, 0, 0, 0, 0, 0, 432, 0, 0, + /* 2360 */ 0, 0, 0, 0, 0, 403, 51, 0, 0, 47, + /* 2370 */ 0, 0, 0, 448, 0, 450, 0, 0, 0, 0, + /* 2380 */ 0, 0, 0, 158, 0, 157, 0, 22, 156, 0, + /* 2390 */ 0, 22, 52, 495, 432, 22, 498, 563, 564, 388, + /* 2400 */ 502, 503, 504, 505, 506, 507, 0, 68, 510, 511, + /* 2410 */ 448, 0, 450, 68, 403, 0, 518, 0, 68, 36, + /* 2420 */ 495, 523, 0, 498, 0, 68, 0, 502, 503, 504, + /* 2430 */ 505, 506, 507, 36, 472, 510, 511, 52, 44, 36, + /* 2440 */ 44, 54, 54, 432, 0, 36, 54, 0, 388, 44, + /* 2450 */ 36, 0, 44, 33, 47, 51, 14, 495, 0, 448, + /* 2460 */ 498, 450, 44, 403, 502, 503, 504, 505, 506, 507, + /* 2470 */ 51, 0, 510, 511, 51, 45, 388, 552, 44, 207, + /* 2480 */ 0, 0, 51, 472, 0, 0, 0, 51, 0, 0, + /* 2490 */ 388, 403, 432, 0, 0, 36, 44, 76, 0, 54, + /* 2500 */ 36, 44, 54, 0, 36, 403, 495, 54, 448, 498, + /* 2510 */ 450, 44, 388, 502, 503, 504, 505, 506, 507, 0, + /* 2520 */ 432, 510, 511, 36, 54, 0, 44, 403, 0, 0, + /* 2530 */ 0, 0, 0, 36, 432, 125, 448, 22, 450, 123, + /* 2540 */ 0, 36, 22, 36, 36, 36, 36, 33, 36, 36, + /* 2550 */ 448, 0, 450, 36, 36, 495, 432, 33, 498, 36, + /* 2560 */ 22, 22, 502, 503, 504, 505, 506, 507, 0, 36, + /* 2570 */ 510, 511, 448, 36, 450, 22, 0, 22, 56, 0, + /* 2580 */ 22, 388, 36, 495, 0, 0, 498, 0, 36, 0, /* 2590 */ 502, 503, 504, 505, 506, 507, 403, 495, 510, 511, - /* 2600 */ 498, 115, 36, 0, 502, 503, 504, 505, 506, 507, - /* 2610 */ 193, 51, 510, 511, 0, 555, 193, 388, 228, 495, - /* 2620 */ 0, 36, 498, 229, 22, 432, 502, 503, 504, 505, - /* 2630 */ 506, 507, 403, 223, 510, 511, 199, 513, 22, 115, - /* 2640 */ 219, 448, 193, 450, 193, 388, 0, 0, 3, 312, - /* 2650 */ 203, 193, 564, 203, 33, 115, 52, 52, 116, 115, - /* 2660 */ 403, 432, 36, 116, 36, 472, 116, 111, 113, 115, - /* 2670 */ 33, 33, 33, 116, 116, 51, 51, 448, 115, 450, - /* 2680 */ 116, 33, 115, 82, 115, 115, 33, 312, 495, 432, - /* 2690 */ 36, 498, 115, 3, 33, 502, 503, 504, 505, 506, - /* 2700 */ 507, 472, 116, 510, 511, 448, 116, 450, 36, 36, - /* 2710 */ 36, 36, 36, 36, 51, 116, 116, 33, 312, 51, - /* 2720 */ 0, 0, 296, 115, 495, 44, 388, 498, 116, 472, - /* 2730 */ 115, 502, 503, 504, 505, 506, 507, 0, 116, 510, - /* 2740 */ 511, 403, 115, 44, 196, 115, 0, 116, 115, 200, - /* 2750 */ 44, 33, 495, 113, 388, 498, 284, 2, 196, 502, - /* 2760 */ 503, 504, 505, 506, 507, 113, 195, 510, 511, 403, - /* 2770 */ 432, 22, 115, 115, 115, 115, 51, 115, 252, 51, - /* 2780 */ 116, 116, 115, 22, 115, 115, 448, 116, 450, 255, - /* 2790 */ 388, 196, 44, 115, 115, 0, 0, 115, 432, 116, - /* 2800 */ 115, 115, 22, 116, 115, 403, 115, 22, 51, 117, - /* 2810 */ 472, 118, 115, 115, 448, 115, 450, 116, 115, 115, - /* 2820 */ 126, 22, 22, 388, 116, 229, 36, 115, 36, 116, - /* 2830 */ 36, 115, 137, 495, 432, 116, 498, 36, 403, 116, - /* 2840 */ 502, 503, 504, 505, 506, 507, 36, 116, 510, 511, - /* 2850 */ 448, 36, 450, 36, 116, 115, 33, 115, 36, 137, - /* 2860 */ 115, 495, 22, 137, 498, 76, 388, 432, 502, 503, - /* 2870 */ 504, 505, 506, 507, 22, 36, 510, 511, 137, 75, - /* 2880 */ 36, 403, 36, 448, 36, 450, 36, 36, 82, 36, - /* 2890 */ 36, 36, 36, 36, 82, 109, 109, 495, 33, 388, - /* 2900 */ 498, 82, 36, 22, 502, 503, 504, 505, 506, 507, - /* 2910 */ 432, 36, 510, 511, 403, 36, 36, 22, 36, 36, - /* 2920 */ 36, 36, 36, 36, 36, 36, 448, 0, 450, 36, - /* 2930 */ 495, 54, 0, 498, 44, 36, 44, 502, 503, 504, - /* 2940 */ 505, 506, 507, 432, 0, 510, 511, 36, 54, 44, - /* 2950 */ 54, 0, 36, 44, 0, 54, 36, 0, 22, 448, - /* 2960 */ 36, 450, 0, 388, 22, 36, 33, 36, 22, 21, - /* 2970 */ 567, 22, 22, 495, 21, 20, 498, 567, 403, 567, - /* 2980 */ 502, 503, 504, 505, 506, 507, 388, 567, 510, 511, - /* 2990 */ 567, 567, 567, 567, 567, 567, 567, 567, 567, 567, + /* 2600 */ 498, 36, 0, 36, 502, 503, 504, 505, 506, 507, + /* 2610 */ 0, 20, 510, 511, 22, 555, 36, 388, 36, 495, + /* 2620 */ 116, 115, 498, 0, 36, 432, 502, 503, 504, 505, + /* 2630 */ 506, 507, 403, 193, 510, 511, 115, 513, 0, 193, + /* 2640 */ 228, 448, 229, 450, 51, 388, 22, 36, 223, 0, + /* 2650 */ 22, 219, 564, 193, 199, 0, 0, 203, 193, 193, + /* 2660 */ 403, 432, 3, 33, 203, 472, 116, 36, 36, 115, + /* 2670 */ 52, 115, 52, 116, 115, 113, 116, 448, 33, 450, + /* 2680 */ 111, 33, 116, 33, 51, 115, 115, 312, 495, 432, + /* 2690 */ 116, 498, 115, 51, 33, 502, 503, 504, 505, 506, + /* 2700 */ 507, 472, 116, 510, 511, 448, 33, 450, 82, 115, + /* 2710 */ 312, 115, 36, 116, 3, 116, 33, 36, 36, 116, + /* 2720 */ 36, 36, 36, 36, 495, 116, 388, 498, 51, 472, + /* 2730 */ 33, 502, 503, 504, 505, 506, 507, 296, 0, 510, + /* 2740 */ 511, 403, 51, 0, 115, 312, 44, 0, 44, 200, + /* 2750 */ 116, 116, 495, 0, 388, 498, 115, 196, 115, 502, + /* 2760 */ 503, 504, 505, 506, 507, 44, 115, 510, 511, 403, + /* 2770 */ 432, 113, 33, 116, 196, 115, 195, 113, 2, 284, + /* 2780 */ 22, 116, 115, 115, 115, 115, 448, 252, 450, 115, + /* 2790 */ 388, 116, 51, 115, 115, 51, 22, 255, 432, 0, + /* 2800 */ 196, 44, 0, 115, 115, 403, 116, 115, 115, 115, + /* 2810 */ 472, 116, 116, 115, 448, 115, 450, 22, 115, 118, + /* 2820 */ 51, 115, 117, 388, 115, 115, 22, 22, 116, 115, + /* 2830 */ 115, 22, 116, 495, 432, 229, 498, 36, 403, 126, + /* 2840 */ 502, 503, 504, 505, 506, 507, 36, 115, 510, 511, + /* 2850 */ 448, 116, 450, 36, 36, 115, 36, 36, 116, 36, + /* 2860 */ 137, 495, 116, 137, 498, 116, 388, 432, 502, 503, + /* 2870 */ 504, 505, 506, 507, 137, 116, 510, 511, 115, 33, + /* 2880 */ 115, 403, 137, 448, 36, 450, 115, 22, 76, 75, + /* 2890 */ 22, 36, 36, 36, 36, 36, 36, 495, 36, 388, + /* 2900 */ 498, 82, 36, 109, 502, 503, 504, 505, 506, 507, + /* 2910 */ 432, 36, 510, 511, 403, 36, 36, 82, 33, 36, + /* 2920 */ 109, 22, 36, 36, 36, 36, 448, 36, 450, 82, + /* 2930 */ 495, 36, 36, 498, 36, 22, 0, 502, 503, 504, + /* 2940 */ 505, 506, 507, 432, 36, 510, 511, 36, 36, 44, + /* 2950 */ 36, 54, 0, 36, 44, 54, 0, 36, 54, 448, + /* 2960 */ 44, 450, 0, 388, 36, 0, 44, 36, 0, 54, + /* 2970 */ 22, 36, 0, 495, 33, 22, 498, 22, 403, 36, + /* 2980 */ 502, 503, 504, 505, 506, 507, 388, 36, 510, 511, + /* 2990 */ 21, 567, 22, 22, 21, 20, 567, 567, 567, 567, /* 3000 */ 567, 403, 567, 567, 567, 567, 495, 432, 567, 498, /* 3010 */ 567, 567, 567, 502, 503, 504, 505, 506, 507, 567, /* 3020 */ 567, 510, 511, 448, 567, 450, 567, 567, 567, 567, @@ -1413,19 +1415,19 @@ static const YYCODETYPE yy_lookahead[] = { /* 3800 */ 511, 448, 567, 450, 567, 567, 567, 567, 432, 567, /* 3810 */ 388, 567, 567, 567, 567, 567, 567, 567, 567, 567, /* 3820 */ 567, 567, 567, 567, 448, 403, 450, 567, 567, 567, - /* 3830 */ 567, 567, 567, 567, 567, 567, 567, 567, 567, 567, - /* 3840 */ 567, 567, 567, 567, 567, 567, 567, 567, 495, 567, + /* 3830 */ 567, 567, 388, 567, 567, 567, 567, 567, 567, 567, + /* 3840 */ 567, 567, 567, 567, 567, 567, 567, 403, 495, 567, /* 3850 */ 567, 498, 567, 567, 432, 502, 503, 504, 505, 506, /* 3860 */ 507, 567, 567, 510, 511, 567, 567, 567, 567, 567, - /* 3870 */ 448, 495, 450, 567, 498, 567, 567, 567, 502, 503, + /* 3870 */ 448, 495, 450, 567, 498, 567, 432, 567, 502, 503, /* 3880 */ 504, 505, 506, 507, 567, 567, 510, 511, 567, 567, - /* 3890 */ 567, 567, 567, 567, 567, 567, 567, 567, 567, 567, + /* 3890 */ 567, 567, 448, 567, 450, 567, 567, 567, 567, 567, /* 3900 */ 567, 567, 567, 567, 567, 567, 567, 567, 567, 567, /* 3910 */ 567, 567, 567, 567, 567, 567, 567, 495, 567, 567, /* 3920 */ 498, 567, 567, 567, 502, 503, 504, 505, 506, 507, - /* 3930 */ 567, 567, 510, 511, 385, 385, 385, 385, 385, 385, - /* 3940 */ 385, 385, 385, 385, 385, 385, 385, 385, 385, 385, - /* 3950 */ 385, 385, 385, 385, 385, 385, 385, 385, 385, 385, + /* 3930 */ 567, 567, 510, 511, 567, 567, 567, 567, 567, 495, + /* 3940 */ 567, 567, 498, 567, 567, 567, 502, 503, 504, 505, + /* 3950 */ 506, 507, 567, 567, 510, 511, 385, 385, 385, 385, /* 3960 */ 385, 385, 385, 385, 385, 385, 385, 385, 385, 385, /* 3970 */ 385, 385, 385, 385, 385, 385, 385, 385, 385, 385, /* 3980 */ 385, 385, 385, 385, 385, 385, 385, 385, 385, 385, @@ -1461,11 +1463,14 @@ static const YYCODETYPE yy_lookahead[] = { /* 4280 */ 385, 385, 385, 385, 385, 385, 385, 385, 385, 385, /* 4290 */ 385, 385, 385, 385, 385, 385, 385, 385, 385, 385, /* 4300 */ 385, 385, 385, 385, 385, 385, 385, 385, 385, 385, - /* 4310 */ 385, 385, 385, 385, 385, 385, 385, 385, 385, + /* 4310 */ 385, 385, 385, 385, 385, 385, 385, 385, 385, 385, + /* 4320 */ 385, 385, 385, 385, 385, 385, 385, 385, 385, 385, + /* 4330 */ 385, 385, 385, 385, 385, 385, 385, 385, 385, 385, + /* 4340 */ 385, }; -#define YY_SHIFT_COUNT (1003) +#define YY_SHIFT_COUNT (1005) #define YY_SHIFT_MIN (0) -#define YY_SHIFT_MAX (2962) +#define YY_SHIFT_MAX (2975) static const unsigned short int yy_shift_ofst[] = { /* 0 */ 1599, 271, 354, 271, 626, 626, 626, 626, 626, 626, /* 10 */ 626, 626, 626, 626, 626, 626, 709, 1063, 1063, 1334, @@ -1473,105 +1478,105 @@ static const unsigned short int yy_shift_ofst[] = { /* 30 */ 1063, 980, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, /* 40 */ 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, /* 50 */ 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, - /* 60 */ 93, 247, 253, 221, 219, 356, 219, 219, 221, 221, - /* 70 */ 219, 1820, 219, 1333, 1820, 381, 219, 4, 1949, 673, - /* 80 */ 673, 61, 61, 1949, 1949, 385, 385, 673, 325, 325, - /* 90 */ 447, 274, 274, 480, 300, 61, 61, 61, 61, 61, - /* 100 */ 61, 61, 61, 61, 61, 61, 167, 266, 275, 61, - /* 110 */ 61, 40, 4, 61, 167, 61, 4, 61, 61, 61, - /* 120 */ 61, 4, 61, 61, 61, 4, 61, 4, 4, 4, - /* 130 */ 749, 165, 165, 452, 452, 543, 762, 178, 48, 972, - /* 140 */ 972, 972, 972, 972, 972, 972, 972, 972, 972, 972, - /* 150 */ 972, 972, 972, 972, 972, 972, 972, 972, 520, 187, - /* 160 */ 325, 447, 690, 690, 477, 922, 922, 922, 654, 654, - /* 170 */ 1285, 1384, 477, 40, 4, 73, 4, 4, 135, 4, - /* 180 */ 4, 406, 4, 406, 406, 724, 924, 452, 452, 452, - /* 190 */ 452, 452, 452, 1354, 413, 21, 75, 414, 414, 822, - /* 200 */ 125, 456, 837, 670, 186, 207, 723, 947, 947, 986, - /* 210 */ 910, 1328, 1328, 1328, 951, 1328, 1157, 929, 726, 1397, - /* 220 */ 1255, 1411, 1342, 1401, 1401, 1403, 1490, 1490, 1311, 1520, - /* 230 */ 213, 1401, 1384, 1496, 1757, 1795, 1796, 1591, 40, 1796, - /* 240 */ 40, 1614, 1795, 1811, 1793, 1811, 1793, 1664, 1795, 1811, - /* 250 */ 1795, 1793, 1664, 1664, 1664, 1755, 1761, 1795, 1795, 1775, - /* 260 */ 1795, 1795, 1795, 1872, 1848, 1872, 1848, 1796, 40, 40, - /* 270 */ 1889, 40, 1894, 1905, 40, 1894, 40, 1916, 40, 1918, - /* 280 */ 40, 40, 1795, 40, 1872, 4, 4, 4, 4, 4, - /* 290 */ 4, 4, 4, 4, 4, 4, 1795, 924, 924, 1872, - /* 300 */ 406, 406, 406, 1747, 1873, 1796, 749, 1970, 1772, 1785, - /* 310 */ 1889, 749, 1496, 1795, 406, 1700, 1705, 1700, 1705, 1703, - /* 320 */ 1819, 1700, 1702, 1708, 1730, 1496, 1736, 1740, 1712, 1716, - /* 330 */ 1719, 1811, 2036, 1934, 1753, 1894, 749, 749, 1705, 406, - /* 340 */ 406, 406, 406, 1705, 406, 1879, 749, 406, 1918, 749, - /* 350 */ 1968, 406, 1892, 1918, 749, 724, 749, 1811, 406, 406, + /* 60 */ 1063, 93, 247, 253, 221, 219, 356, 219, 219, 221, + /* 70 */ 221, 219, 1820, 219, 1333, 1820, 381, 219, 4, 1949, + /* 80 */ 673, 673, 61, 61, 1949, 1949, 385, 385, 673, 325, + /* 90 */ 325, 447, 274, 274, 480, 300, 61, 61, 61, 61, + /* 100 */ 61, 61, 61, 61, 61, 61, 61, 167, 266, 275, + /* 110 */ 61, 61, 40, 4, 61, 167, 61, 4, 61, 61, + /* 120 */ 61, 61, 4, 61, 61, 61, 4, 61, 4, 4, + /* 130 */ 4, 749, 165, 165, 452, 452, 543, 762, 178, 48, + /* 140 */ 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, + /* 150 */ 921, 921, 921, 921, 921, 921, 921, 921, 921, 520, + /* 160 */ 187, 325, 447, 1105, 1105, 477, 556, 556, 556, 716, + /* 170 */ 716, 850, 1115, 477, 40, 4, 73, 4, 4, 135, + /* 180 */ 4, 4, 406, 4, 406, 406, 483, 1045, 452, 452, + /* 190 */ 452, 452, 452, 452, 1354, 413, 21, 75, 414, 414, + /* 200 */ 822, 125, 456, 711, 670, 186, 207, 723, 922, 922, + /* 210 */ 1336, 730, 972, 972, 972, 1211, 972, 1157, 853, 726, + /* 220 */ 1344, 1379, 949, 823, 1294, 1294, 1311, 1384, 1384, 1355, + /* 230 */ 1456, 213, 1294, 1115, 1506, 1763, 1807, 1809, 1601, 40, + /* 240 */ 1809, 40, 1625, 1807, 1839, 1815, 1839, 1815, 1681, 1807, + /* 250 */ 1839, 1807, 1815, 1681, 1681, 1681, 1770, 1785, 1807, 1807, + /* 260 */ 1797, 1807, 1807, 1807, 1893, 1864, 1893, 1864, 1809, 40, + /* 270 */ 40, 1914, 40, 1916, 1921, 40, 1916, 40, 1935, 40, + /* 280 */ 1946, 40, 40, 1807, 40, 1893, 4, 4, 4, 4, + /* 290 */ 4, 4, 4, 4, 4, 4, 4, 1807, 1045, 1045, + /* 300 */ 1893, 406, 406, 406, 1739, 1867, 1809, 749, 1986, 1793, + /* 310 */ 1798, 1914, 749, 1506, 1807, 406, 1709, 1714, 1709, 1714, + /* 320 */ 1707, 1828, 1709, 1713, 1717, 1740, 1506, 1742, 1745, 1718, + /* 330 */ 1728, 1731, 1839, 2045, 1940, 1759, 1916, 749, 749, 1714, + /* 340 */ 406, 406, 406, 406, 1714, 406, 1892, 749, 406, 1946, + /* 350 */ 749, 1979, 406, 1909, 1946, 749, 483, 749, 1839, 406, /* 360 */ 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, /* 370 */ 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, - /* 380 */ 1993, 406, 1795, 749, 2104, 2092, 2096, 2110, 1872, 3934, - /* 390 */ 3934, 3934, 3934, 3934, 3934, 3934, 3934, 3934, 3934, 3934, - /* 400 */ 3934, 39, 1238, 204, 302, 310, 84, 50, 197, 904, - /* 410 */ 1134, 1197, 1214, 1258, 1275, 1353, 1366, 1463, 589, 1055, - /* 420 */ 1280, 1307, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, - /* 430 */ 1065, 100, 65, 500, 1033, 3, 3, 653, 53, 387, - /* 440 */ 294, 284, 284, 604, 902, 284, 696, 977, 1246, 422, - /* 450 */ 47, 47, 1112, 1335, 1148, 1112, 1112, 1112, 1413, 94, - /* 460 */ 511, 1540, 1528, 1429, 1098, 576, 1423, 1460, 1462, 1500, - /* 470 */ 1545, 752, 971, 1602, 1628, 1629, 1630, 1340, 882, 1402, - /* 480 */ 1067, 1509, 1574, 1585, 1586, 1381, 1452, 1575, 1600, 1601, - /* 490 */ 1604, 1606, 1642, 1613, 1578, 1616, 1625, 1580, 1626, 1634, - /* 500 */ 1644, 1651, 1652, 1648, 1661, 1663, 1666, 1674, 1676, 1680, - /* 510 */ 1684, 1686, 1691, 1693, 1675, 1677, 1678, 1679, 1682, 1685, - /* 520 */ 1565, 1553, 1414, 1618, 1687, 1711, 1612, 1725, 2203, 2204, - /* 530 */ 2205, 2162, 2210, 2175, 1972, 2177, 2180, 2181, 1977, 2221, - /* 540 */ 2187, 2188, 1988, 2196, 2233, 2234, 1992, 2238, 2206, 2239, - /* 550 */ 2207, 2240, 2219, 2244, 2209, 2008, 2247, 2024, 2249, 2026, - /* 560 */ 2030, 2033, 2041, 2258, 2259, 2267, 2058, 2061, 2271, 2273, - /* 570 */ 2116, 2224, 2226, 2278, 2243, 2283, 2285, 2251, 2235, 2288, - /* 580 */ 2241, 2290, 2246, 2291, 2294, 2295, 2245, 2297, 2298, 2300, - /* 590 */ 2303, 2304, 2305, 2136, 2280, 2317, 2142, 2319, 2320, 2321, - /* 600 */ 2323, 2324, 2325, 2331, 2333, 2334, 2335, 2342, 2343, 2347, - /* 610 */ 2349, 2351, 2352, 2353, 2354, 2355, 2356, 2307, 2359, 2313, - /* 620 */ 2361, 2362, 2363, 2364, 2374, 2376, 2377, 2378, 2379, 2344, - /* 630 */ 2367, 2211, 2368, 2213, 2371, 2216, 2380, 2381, 2360, 2332, - /* 640 */ 2365, 2336, 2383, 2318, 2385, 2322, 2370, 2389, 2327, 2391, - /* 650 */ 2339, 2392, 2411, 2382, 2372, 2369, 2415, 2386, 2384, 2373, - /* 660 */ 2419, 2388, 2387, 2393, 2425, 2397, 2439, 2395, 2396, 2412, - /* 670 */ 2398, 2399, 2430, 2400, 2446, 2407, 2403, 2453, 2454, 2455, - /* 680 */ 2456, 2414, 2255, 2470, 2398, 2420, 2474, 2398, 2424, 2484, - /* 690 */ 2485, 2404, 2481, 2482, 2442, 2432, 2443, 2479, 2452, 2440, - /* 700 */ 2445, 2493, 2459, 2444, 2457, 2496, 2461, 2448, 2460, 2499, - /* 710 */ 2500, 2503, 2507, 2519, 2523, 2401, 2405, 2475, 2502, 2525, - /* 720 */ 2508, 2495, 2497, 2501, 2504, 2505, 2506, 2509, 2510, 2512, - /* 730 */ 2514, 2516, 2515, 2517, 2513, 2518, 2529, 2521, 2532, 2522, - /* 740 */ 2539, 2535, 2520, 2559, 2538, 2533, 2561, 2568, 2573, 2541, - /* 750 */ 2578, 2543, 2580, 2546, 2584, 2553, 2565, 2551, 2552, 2566, - /* 760 */ 2473, 2486, 2603, 2417, 2394, 2390, 2524, 2410, 2398, 2560, - /* 770 */ 2614, 2423, 2585, 2602, 2620, 2421, 2616, 2449, 2437, 2646, - /* 780 */ 2647, 2451, 2447, 2458, 2450, 2645, 2621, 2337, 2540, 2542, - /* 790 */ 2544, 2547, 2626, 2628, 2554, 2604, 2555, 2605, 2556, 2550, - /* 800 */ 2637, 2638, 2557, 2563, 2567, 2569, 2558, 2639, 2624, 2625, - /* 810 */ 2570, 2648, 2375, 2601, 2564, 2653, 2577, 2654, 2586, 2590, - /* 820 */ 2690, 2661, 2406, 2672, 2673, 2674, 2675, 2676, 2677, 2599, - /* 830 */ 2600, 2663, 2426, 2684, 2668, 2720, 2721, 2608, 2681, 2612, - /* 840 */ 2622, 2615, 2627, 2548, 2630, 2737, 2699, 2549, 2746, 2631, - /* 850 */ 2633, 2562, 2706, 2571, 2718, 2640, 2472, 2652, 2755, 2749, - /* 860 */ 2526, 2657, 2658, 2659, 2660, 2664, 2665, 2662, 2667, 2669, - /* 870 */ 2670, 2678, 2671, 2725, 2679, 2682, 2728, 2683, 2761, 2534, - /* 880 */ 2685, 2686, 2795, 2687, 2689, 2595, 2748, 2691, 2692, 2796, - /* 890 */ 2780, 2693, 2697, 2398, 2757, 2698, 2700, 2701, 2703, 2704, - /* 900 */ 2694, 2785, 2799, 2800, 2596, 2708, 2790, 2792, 2712, 2713, - /* 910 */ 2794, 2716, 2719, 2801, 2662, 2723, 2810, 2667, 2731, 2815, - /* 920 */ 2669, 2738, 2817, 2670, 2695, 2722, 2726, 2741, 2740, 2823, - /* 930 */ 2742, 2822, 2745, 2823, 2823, 2840, 2789, 2804, 2852, 2839, - /* 940 */ 2844, 2846, 2848, 2850, 2851, 2853, 2854, 2855, 2856, 2857, - /* 950 */ 2806, 2786, 2812, 2787, 2865, 2866, 2875, 2879, 2881, 2880, - /* 960 */ 2882, 2883, 2819, 2514, 2884, 2516, 2885, 2886, 2887, 2888, - /* 970 */ 2895, 2889, 2927, 2893, 2877, 2890, 2932, 2899, 2894, 2892, - /* 980 */ 2944, 2911, 2896, 2905, 2951, 2916, 2901, 2909, 2954, 2920, - /* 990 */ 2957, 2936, 2924, 2962, 2942, 2933, 2929, 2931, 2946, 2948, - /* 1000 */ 2949, 2950, 2953, 2955, + /* 380 */ 406, 2018, 406, 1807, 749, 2123, 2128, 2146, 2145, 1893, + /* 390 */ 3956, 3956, 3956, 3956, 3956, 3956, 3956, 3956, 3956, 3956, + /* 400 */ 3956, 3956, 39, 1238, 204, 302, 310, 84, 904, 50, + /* 410 */ 197, 1175, 1197, 1258, 1134, 1212, 1353, 1366, 1463, 1558, + /* 420 */ 589, 1611, 1280, 1307, 1281, 1281, 1281, 1281, 1281, 1281, + /* 430 */ 1281, 1281, 1281, 100, 65, 500, 837, 3, 3, 653, + /* 440 */ 53, 387, 294, 284, 284, 604, 902, 284, 696, 920, + /* 450 */ 1328, 422, 47, 47, 1154, 1221, 817, 1154, 1154, 1154, + /* 460 */ 1465, 94, 511, 1488, 1496, 1358, 1480, 1513, 1414, 1419, + /* 470 */ 1421, 1433, 1481, 752, 1532, 954, 1518, 1533, 1547, 1324, + /* 480 */ 1220, 1459, 970, 1528, 1544, 1550, 1551, 1428, 1254, 1412, + /* 490 */ 1574, 1576, 1585, 1597, 1600, 1604, 1559, 1606, 1610, 1525, + /* 500 */ 1613, 1616, 1617, 1618, 1642, 1619, 1640, 1645, 1665, 1667, + /* 510 */ 1675, 1682, 1685, 1691, 1693, 1706, 1669, 1694, 1697, 1715, + /* 520 */ 1735, 1738, 1546, 1526, 1612, 1646, 1663, 1737, 1736, 1806, + /* 530 */ 2216, 2220, 2221, 2176, 2224, 2195, 1989, 2197, 2198, 2199, + /* 540 */ 1995, 2239, 2204, 2205, 1999, 2207, 2244, 2245, 2003, 2247, + /* 550 */ 2212, 2249, 2214, 2251, 2233, 2267, 2222, 2019, 2259, 2046, + /* 560 */ 2271, 2048, 2049, 2055, 2059, 2277, 2278, 2279, 2071, 2074, + /* 570 */ 2287, 2288, 2131, 2240, 2241, 2290, 2258, 2293, 2295, 2260, + /* 580 */ 2243, 2298, 2252, 2300, 2265, 2304, 2305, 2316, 2266, 2318, + /* 590 */ 2319, 2320, 2321, 2323, 2324, 2149, 2297, 2331, 2158, 2335, + /* 600 */ 2342, 2343, 2347, 2349, 2351, 2352, 2353, 2354, 2355, 2356, + /* 610 */ 2358, 2359, 2360, 2361, 2362, 2363, 2364, 2374, 2376, 2315, + /* 620 */ 2367, 2322, 2368, 2370, 2371, 2372, 2377, 2378, 2379, 2380, + /* 630 */ 2381, 2365, 2382, 2225, 2384, 2228, 2386, 2232, 2389, 2390, + /* 640 */ 2369, 2340, 2373, 2385, 2406, 2339, 2411, 2345, 2383, 2415, + /* 650 */ 2350, 2417, 2357, 2422, 2424, 2397, 2387, 2394, 2426, 2403, + /* 660 */ 2388, 2396, 2444, 2409, 2392, 2405, 2447, 2414, 2451, 2407, + /* 670 */ 2408, 2420, 2404, 2419, 2442, 2423, 2458, 2430, 2418, 2471, + /* 680 */ 2484, 2485, 2486, 2434, 2272, 2480, 2404, 2431, 2481, 2404, + /* 690 */ 2436, 2488, 2489, 2421, 2493, 2494, 2459, 2445, 2452, 2498, + /* 700 */ 2464, 2448, 2457, 2503, 2468, 2453, 2467, 2519, 2487, 2470, + /* 710 */ 2482, 2525, 2528, 2529, 2530, 2531, 2532, 2410, 2416, 2497, + /* 720 */ 2515, 2540, 2520, 2505, 2507, 2508, 2509, 2510, 2512, 2513, + /* 730 */ 2517, 2518, 2514, 2524, 2523, 2533, 2538, 2537, 2551, 2539, + /* 740 */ 2568, 2553, 2576, 2555, 2522, 2579, 2558, 2546, 2584, 2585, + /* 750 */ 2587, 2552, 2589, 2565, 2602, 2567, 2610, 2592, 2591, 2580, + /* 760 */ 2582, 2588, 2504, 2506, 2623, 2440, 2413, 2412, 2521, 2425, + /* 770 */ 2404, 2593, 2638, 2446, 2611, 2624, 2649, 2432, 2628, 2460, + /* 780 */ 2455, 2655, 2656, 2465, 2454, 2466, 2461, 2659, 2630, 2375, + /* 790 */ 2554, 2550, 2556, 2557, 2631, 2632, 2559, 2618, 2562, 2620, + /* 800 */ 2569, 2560, 2645, 2648, 2566, 2570, 2571, 2577, 2574, 2650, + /* 810 */ 2633, 2642, 2594, 2661, 2398, 2626, 2586, 2673, 2596, 2676, + /* 820 */ 2597, 2599, 2711, 2683, 2433, 2681, 2682, 2684, 2685, 2686, + /* 830 */ 2687, 2603, 2609, 2677, 2441, 2697, 2691, 2738, 2743, 2629, + /* 840 */ 2702, 2634, 2635, 2641, 2643, 2561, 2651, 2747, 2704, 2549, + /* 850 */ 2753, 2657, 2660, 2578, 2721, 2581, 2739, 2658, 2495, 2664, + /* 860 */ 2776, 2758, 2535, 2667, 2668, 2669, 2670, 2665, 2675, 2674, + /* 870 */ 2678, 2679, 2688, 2689, 2690, 2741, 2692, 2693, 2744, 2695, + /* 880 */ 2774, 2542, 2694, 2698, 2799, 2696, 2700, 2604, 2757, 2703, + /* 890 */ 2705, 2802, 2795, 2701, 2706, 2404, 2769, 2709, 2710, 2712, + /* 900 */ 2714, 2715, 2713, 2804, 2805, 2809, 2606, 2716, 2801, 2810, + /* 910 */ 2732, 2735, 2817, 2740, 2742, 2818, 2674, 2746, 2820, 2678, + /* 920 */ 2749, 2821, 2679, 2759, 2823, 2688, 2723, 2726, 2737, 2745, + /* 930 */ 2763, 2846, 2765, 2848, 2771, 2846, 2846, 2865, 2812, 2814, + /* 940 */ 2868, 2855, 2856, 2857, 2858, 2859, 2860, 2862, 2866, 2875, + /* 950 */ 2879, 2880, 2819, 2794, 2835, 2811, 2885, 2883, 2886, 2887, + /* 960 */ 2899, 2888, 2889, 2891, 2847, 2514, 2895, 2524, 2896, 2898, + /* 970 */ 2908, 2911, 2913, 2912, 2936, 2914, 2897, 2905, 2952, 2917, + /* 980 */ 2901, 2910, 2956, 2921, 2904, 2916, 2962, 2928, 2915, 2922, + /* 990 */ 2965, 2931, 2968, 2948, 2935, 2972, 2953, 2941, 2943, 2951, + /* 1000 */ 2955, 2969, 2970, 2971, 2973, 2975, }; -#define YY_REDUCE_COUNT (400) +#define YY_REDUCE_COUNT (401) #define YY_REDUCE_MIN (-530) -#define YY_REDUCE_MAX (3422) +#define YY_REDUCE_MAX (3444) static const short yy_reduce_ofst[] = { /* 0 */ -142, -347, -206, 607, 1233, 1259, 1371, 1417, 1447, 1498, /* 10 */ 292, 1607, 1637, 1671, 1758, 1804, -101, 656, 1834, 1898, @@ -1579,144 +1584,144 @@ static const short yy_reduce_ofst[] = { /* 30 */ 2338, 2102, 2366, 2402, 2435, 2478, 2511, 2575, 2598, 2644, /* 40 */ 2666, 2680, 2756, 2791, 2824, 2867, 2900, 2964, 2987, 3033, /* 50 */ 3055, 3069, 3145, 3180, 3213, 3256, 3289, 3353, 3376, 3422, - /* 60 */ -342, 504, 467, -88, 315, 391, 476, 1007, 172, 255, - /* 70 */ 1009, 67, -530, -74, 308, -528, -203, 312, -273, -427, - /* 80 */ -120, -31, 371, -274, -70, -396, -394, 179, -402, -392, - /* 90 */ -19, -29, 524, -249, 317, 410, 521, 532, 554, -348, - /* 100 */ 355, 609, 619, 650, 694, 373, -374, 155, -293, 623, - /* 110 */ 767, 470, 615, 771, -20, 782, 564, 792, 798, 875, - /* 120 */ 891, 377, 896, 909, 974, 736, 981, 63, 953, 508, - /* 130 */ 407, -370, -370, 105, -424, 649, 81, -139, 272, 303, - /* 140 */ 344, 485, 652, 731, 847, 961, 1000, 1013, 1026, 1075, - /* 150 */ 1076, 1081, 1082, 1096, 1100, 1125, 1129, 1153, 142, 127, - /* 160 */ 277, 600, 679, 683, 892, 127, 718, 911, 502, 541, - /* 170 */ 685, -407, 982, 1021, 211, 774, 513, 550, 529, 748, - /* 180 */ 868, 991, 1087, 997, 1044, 962, 1140, 764, 788, 863, - /* 190 */ 1047, 1193, 1230, 1078, 1293, 1277, 1261, 1187, 1187, 1161, - /* 200 */ 1173, 1183, 1200, 1339, 1187, 1302, 1302, 1327, 1329, 1351, - /* 210 */ 1298, 1218, 1219, 1220, 1305, 1226, 1302, 1324, 1379, 1290, - /* 220 */ 1385, 1343, 1306, 1332, 1336, 1302, 1252, 1260, 1241, 1276, - /* 230 */ 1262, 1344, 1389, 1338, 1317, 1415, 1337, 1346, 1407, 1341, - /* 240 */ 1418, 1357, 1437, 1440, 1394, 1450, 1398, 1405, 1455, 1457, - /* 250 */ 1459, 1406, 1408, 1410, 1416, 1456, 1466, 1481, 1482, 1477, - /* 260 */ 1489, 1494, 1497, 1503, 1505, 1510, 1508, 1424, 1502, 1506, - /* 270 */ 1461, 1507, 1512, 1453, 1515, 1527, 1523, 1470, 1530, 1483, - /* 280 */ 1532, 1536, 1554, 1544, 1564, 1534, 1535, 1541, 1542, 1543, - /* 290 */ 1546, 1547, 1548, 1549, 1550, 1551, 1558, 1573, 1577, 1581, - /* 300 */ 1524, 1529, 1538, 1499, 1504, 1511, 1582, 1513, 1516, 1525, - /* 310 */ 1561, 1603, 1552, 1611, 1567, 1473, 1555, 1478, 1560, 1484, - /* 320 */ 1486, 1485, 1487, 1493, 1501, 1568, 1514, 1517, 1480, 1491, - /* 330 */ 1495, 1650, 1557, 1521, 1526, 1659, 1655, 1657, 1608, 1619, - /* 340 */ 1620, 1622, 1623, 1609, 1627, 1615, 1670, 1633, 1621, 1683, - /* 350 */ 1563, 1638, 1631, 1643, 1697, 1660, 1698, 1692, 1646, 1667, - /* 360 */ 1668, 1672, 1688, 1689, 1690, 1701, 1704, 1706, 1709, 1710, - /* 370 */ 1714, 1715, 1720, 1722, 1723, 1724, 1731, 1732, 1735, 1737, - /* 380 */ 1694, 1738, 1726, 1713, 1734, 1741, 1751, 1760, 1756, 1717, - /* 390 */ 1750, 1718, 1727, 1695, 1728, 1762, 1763, 1764, 1766, 1786, - /* 400 */ 1816, + /* 60 */ 3444, -342, 504, 467, -88, 315, 391, 1155, 1185, 172, + /* 70 */ 255, 1222, 67, -530, -74, 308, -528, -203, 312, -273, + /* 80 */ -427, -120, -31, 371, -274, -70, -396, -394, 179, -402, + /* 90 */ -392, -19, -29, 535, -249, 317, 410, 528, 634, 647, + /* 100 */ -348, 355, 650, 665, 668, 671, 373, -374, 155, -293, + /* 110 */ 623, 675, 470, 490, 678, -20, 713, 566, 719, 777, + /* 120 */ 866, 896, 377, 908, 974, 981, 687, 988, 63, 753, + /* 130 */ 517, 407, -370, -370, 105, -424, 482, 81, -139, 272, + /* 140 */ 303, 344, 571, 616, 643, 738, 829, 830, 860, 950, + /* 150 */ 961, 1012, 1013, 1026, 1033, 1035, 1075, 1076, 1082, 142, + /* 160 */ 127, 277, 591, 787, 793, 813, 127, 557, 595, 767, + /* 170 */ 768, 783, -407, 1018, 836, 211, 619, 513, 631, 795, + /* 180 */ 712, 1000, 991, 1016, 1037, 1083, 807, 1059, 743, 769, + /* 190 */ 867, 1008, 1131, 1149, 1109, 1208, 1244, 1194, 1123, 1123, + /* 200 */ 1108, 1132, 1142, 1167, 1302, 1123, 1278, 1278, 1305, 1306, + /* 210 */ 1320, 1326, 1214, 1227, 1234, 1315, 1236, 1278, 1330, 1385, + /* 220 */ 1296, 1390, 1347, 1314, 1337, 1338, 1278, 1262, 1263, 1243, + /* 230 */ 1272, 1274, 1342, 1394, 1348, 1327, 1426, 1350, 1346, 1429, + /* 240 */ 1357, 1442, 1378, 1455, 1457, 1406, 1467, 1411, 1418, 1478, + /* 250 */ 1482, 1489, 1427, 1436, 1438, 1444, 1484, 1486, 1502, 1503, + /* 260 */ 1495, 1507, 1508, 1509, 1519, 1527, 1522, 1530, 1432, 1514, + /* 270 */ 1520, 1487, 1536, 1534, 1468, 1545, 1539, 1548, 1493, 1553, + /* 280 */ 1511, 1556, 1557, 1571, 1563, 1580, 1549, 1552, 1554, 1560, + /* 290 */ 1562, 1565, 1566, 1567, 1575, 1579, 1581, 1577, 1586, 1587, + /* 300 */ 1588, 1540, 1541, 1542, 1490, 1524, 1505, 1609, 1537, 1543, + /* 310 */ 1555, 1573, 1615, 1568, 1626, 1582, 1483, 1564, 1499, 1578, + /* 320 */ 1485, 1494, 1501, 1504, 1510, 1515, 1589, 1516, 1521, 1497, + /* 330 */ 1517, 1512, 1659, 1569, 1535, 1529, 1674, 1666, 1670, 1614, + /* 340 */ 1632, 1633, 1635, 1636, 1620, 1638, 1627, 1684, 1647, 1643, + /* 350 */ 1698, 1594, 1658, 1654, 1661, 1711, 1696, 1719, 1723, 1679, + /* 360 */ 1683, 1686, 1688, 1689, 1690, 1701, 1702, 1703, 1704, 1708, + /* 370 */ 1710, 1716, 1720, 1722, 1724, 1732, 1743, 1747, 1748, 1749, + /* 380 */ 1750, 1725, 1754, 1752, 1762, 1767, 1786, 1790, 1791, 1808, + /* 390 */ 1726, 1771, 1692, 1727, 1741, 1744, 1784, 1787, 1777, 1792, + /* 400 */ 1778, 1829, }; static const YYACTIONTYPE yy_default[] = { - /* 0 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 10 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 20 */ 2842, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 30 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 40 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 50 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 60 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 70 */ 2632, 2257, 2257, 2588, 2257, 2257, 2257, 2257, 2257, 2257, - /* 80 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2595, 2595, - /* 90 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 100 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 110 */ 2257, 2363, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 120 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 130 */ 2361, 2899, 2257, 3025, 2673, 2257, 2257, 2928, 2257, 2257, - /* 140 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 150 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2911, - /* 160 */ 2257, 2257, 2334, 2334, 2257, 2911, 2911, 2911, 2871, 2871, - /* 170 */ 2361, 2257, 2257, 2363, 2257, 2675, 2257, 2257, 2257, 2257, - /* 180 */ 2257, 2257, 2257, 2257, 2257, 2504, 2287, 2257, 2257, 2257, - /* 190 */ 2257, 2257, 2257, 2658, 2257, 2257, 2957, 2903, 2904, 3019, - /* 200 */ 2257, 2960, 2922, 2257, 2917, 2257, 2257, 2257, 2257, 2257, - /* 210 */ 2947, 2257, 2257, 2257, 2257, 2257, 2257, 2600, 2257, 2701, - /* 220 */ 2257, 2449, 2652, 2257, 2257, 2257, 2257, 2257, 3003, 2901, - /* 230 */ 2941, 2257, 2257, 2951, 2257, 2257, 2257, 2689, 2363, 2257, - /* 240 */ 2363, 2645, 2583, 2257, 2593, 2257, 2593, 2590, 2257, 2257, - /* 250 */ 2257, 2593, 2590, 2590, 2590, 2437, 2433, 2257, 2257, 2431, - /* 260 */ 2257, 2257, 2257, 2257, 2317, 2257, 2317, 2257, 2363, 2363, - /* 270 */ 2257, 2363, 2257, 2257, 2363, 2257, 2363, 2257, 2363, 2257, - /* 280 */ 2363, 2363, 2257, 2363, 2257, 2257, 2257, 2257, 2257, 2257, - /* 290 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 300 */ 2257, 2257, 2257, 2687, 2668, 2257, 2361, 2257, 2656, 2654, - /* 310 */ 2257, 2361, 2951, 2257, 2257, 2973, 2968, 2973, 2968, 2987, - /* 320 */ 2983, 2973, 2992, 2989, 2953, 2951, 2934, 2930, 3022, 3009, - /* 330 */ 3005, 2257, 2257, 2939, 2937, 2257, 2361, 2361, 2968, 2257, - /* 340 */ 2257, 2257, 2257, 2968, 2257, 2257, 2361, 2257, 2257, 2361, - /* 350 */ 2257, 2257, 2257, 2257, 2361, 2257, 2361, 2257, 2257, 2257, - /* 360 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 370 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 380 */ 2466, 2257, 2257, 2361, 2257, 2289, 2291, 2301, 2257, 2647, - /* 390 */ 3025, 2673, 2678, 2628, 2628, 2507, 2507, 3025, 2507, 2364, - /* 400 */ 2262, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 410 */ 2257, 2257, 2257, 2806, 2257, 2257, 2257, 2257, 2257, 2257, - /* 420 */ 2257, 2257, 2986, 2985, 2807, 2257, 2875, 2874, 2873, 2864, - /* 430 */ 2806, 2462, 2257, 2257, 2257, 2805, 2804, 2257, 2257, 2257, - /* 440 */ 2257, 2453, 2450, 2257, 2257, 2475, 2257, 2257, 2257, 2257, - /* 450 */ 2619, 2618, 2798, 2257, 2257, 2799, 2797, 2796, 2257, 2257, - /* 460 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 470 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 480 */ 2257, 2257, 2257, 2257, 2257, 2257, 3006, 3010, 2257, 2257, - /* 490 */ 2257, 2257, 2900, 2257, 2257, 2257, 2257, 2777, 2257, 2257, - /* 500 */ 2257, 2257, 2257, 2745, 2740, 2731, 2722, 2737, 2728, 2716, - /* 510 */ 2734, 2725, 2713, 2710, 2257, 2257, 2257, 2257, 2257, 2257, - /* 520 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 530 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 540 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 550 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 560 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 570 */ 2589, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 580 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 590 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 600 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 610 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 620 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 630 */ 2257, 2257, 2257, 2257, 2257, 2604, 2257, 2257, 2257, 2257, - /* 640 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 650 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 660 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2306, - /* 670 */ 2784, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 680 */ 2257, 2257, 2257, 2257, 2787, 2257, 2257, 2788, 2257, 2257, - /* 690 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 700 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 710 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 720 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 730 */ 2408, 2407, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 740 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 750 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 760 */ 2789, 2257, 2257, 2257, 2257, 2672, 2257, 2257, 2779, 2257, - /* 770 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 780 */ 2257, 2257, 2257, 2257, 2257, 3002, 2954, 2257, 2257, 2257, - /* 790 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 800 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2777, - /* 810 */ 2257, 2984, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 3000, - /* 820 */ 2257, 3004, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2910, - /* 830 */ 2906, 2257, 2257, 2902, 2257, 2257, 2257, 2257, 2257, 2257, - /* 840 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 850 */ 2257, 2257, 2257, 2257, 2861, 2257, 2257, 2257, 2895, 2257, - /* 860 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2503, 2502, 2501, - /* 870 */ 2500, 2257, 2257, 2257, 2257, 2257, 2257, 2789, 2257, 2792, - /* 880 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 890 */ 2257, 2257, 2257, 2776, 2257, 2839, 2838, 2257, 2257, 2257, - /* 900 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2497, 2257, - /* 910 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 920 */ 2257, 2257, 2257, 2257, 2481, 2479, 2478, 2477, 2257, 2514, - /* 930 */ 2257, 2257, 2257, 2510, 2509, 2257, 2257, 2257, 2257, 2257, - /* 940 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 950 */ 2257, 2257, 2257, 2257, 2382, 2257, 2257, 2257, 2257, 2257, - /* 960 */ 2257, 2257, 2257, 2374, 2257, 2373, 2257, 2257, 2257, 2257, - /* 970 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 980 */ 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, 2257, - /* 990 */ 2257, 2257, 2257, 2257, 2257, 2286, 2257, 2257, 2257, 2257, - /* 1000 */ 2257, 2257, 2257, 2257, + /* 0 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 10 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 20 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 30 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 40 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 50 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 60 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 70 */ 2260, 2635, 2260, 2260, 2591, 2260, 2260, 2260, 2260, 2260, + /* 80 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2598, + /* 90 */ 2598, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 100 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 110 */ 2260, 2260, 2366, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 120 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 130 */ 2260, 2364, 2902, 2260, 3028, 2676, 2260, 2260, 2931, 2260, + /* 140 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 150 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 160 */ 2914, 2260, 2260, 2337, 2337, 2260, 2914, 2914, 2914, 2874, + /* 170 */ 2874, 2364, 2260, 2260, 2366, 2260, 2678, 2260, 2260, 2260, + /* 180 */ 2260, 2260, 2260, 2260, 2260, 2260, 2507, 2290, 2260, 2260, + /* 190 */ 2260, 2260, 2260, 2260, 2661, 2260, 2260, 2960, 2906, 2907, + /* 200 */ 3022, 2260, 2963, 2925, 2260, 2920, 2260, 2260, 2260, 2260, + /* 210 */ 2260, 2950, 2260, 2260, 2260, 2260, 2260, 2260, 2603, 2260, + /* 220 */ 2704, 2260, 2452, 2655, 2260, 2260, 2260, 2260, 2260, 3006, + /* 230 */ 2904, 2944, 2260, 2260, 2954, 2260, 2260, 2260, 2692, 2366, + /* 240 */ 2260, 2366, 2648, 2586, 2260, 2596, 2260, 2596, 2593, 2260, + /* 250 */ 2260, 2260, 2596, 2593, 2593, 2593, 2440, 2436, 2260, 2260, + /* 260 */ 2434, 2260, 2260, 2260, 2260, 2320, 2260, 2320, 2260, 2366, + /* 270 */ 2366, 2260, 2366, 2260, 2260, 2366, 2260, 2366, 2260, 2366, + /* 280 */ 2260, 2366, 2366, 2260, 2366, 2260, 2260, 2260, 2260, 2260, + /* 290 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 300 */ 2260, 2260, 2260, 2260, 2690, 2671, 2260, 2364, 2260, 2659, + /* 310 */ 2657, 2260, 2364, 2954, 2260, 2260, 2976, 2971, 2976, 2971, + /* 320 */ 2990, 2986, 2976, 2995, 2992, 2956, 2954, 2937, 2933, 3025, + /* 330 */ 3012, 3008, 2260, 2260, 2942, 2940, 2260, 2364, 2364, 2971, + /* 340 */ 2260, 2260, 2260, 2260, 2971, 2260, 2260, 2364, 2260, 2260, + /* 350 */ 2364, 2260, 2260, 2260, 2260, 2364, 2260, 2364, 2260, 2260, + /* 360 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 370 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 380 */ 2260, 2469, 2260, 2260, 2364, 2260, 2292, 2294, 2304, 2260, + /* 390 */ 2650, 3028, 2676, 2681, 2631, 2631, 2510, 2510, 3028, 2510, + /* 400 */ 2367, 2265, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 410 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2809, 2260, 2260, + /* 420 */ 2260, 2260, 2260, 2260, 2989, 2988, 2810, 2260, 2878, 2877, + /* 430 */ 2876, 2867, 2809, 2465, 2260, 2260, 2260, 2808, 2807, 2260, + /* 440 */ 2260, 2260, 2260, 2456, 2453, 2260, 2260, 2478, 2260, 2260, + /* 450 */ 2260, 2260, 2622, 2621, 2801, 2260, 2260, 2802, 2800, 2799, + /* 460 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 470 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 480 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 3009, 3013, + /* 490 */ 2260, 2260, 2260, 2260, 2903, 2260, 2260, 2260, 2260, 2780, + /* 500 */ 2260, 2260, 2260, 2260, 2260, 2748, 2743, 2734, 2725, 2740, + /* 510 */ 2731, 2719, 2737, 2728, 2716, 2713, 2260, 2260, 2260, 2260, + /* 520 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 530 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 540 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 550 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 560 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 570 */ 2260, 2260, 2592, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 580 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 590 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 600 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 610 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 620 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 630 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2607, 2260, 2260, + /* 640 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 650 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 660 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 670 */ 2260, 2309, 2787, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 680 */ 2260, 2260, 2260, 2260, 2260, 2260, 2790, 2260, 2260, 2791, + /* 690 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 700 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 710 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 720 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 730 */ 2260, 2260, 2411, 2410, 2260, 2260, 2260, 2260, 2260, 2260, + /* 740 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 750 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 760 */ 2260, 2260, 2792, 2260, 2260, 2260, 2260, 2675, 2260, 2260, + /* 770 */ 2782, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 780 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 3005, 2957, 2260, + /* 790 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 800 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 810 */ 2260, 2780, 2260, 2987, 2260, 2260, 2260, 2260, 2260, 2260, + /* 820 */ 2260, 3003, 2260, 3007, 2260, 2260, 2260, 2260, 2260, 2260, + /* 830 */ 2260, 2913, 2909, 2260, 2260, 2905, 2260, 2260, 2260, 2260, + /* 840 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 850 */ 2260, 2260, 2260, 2260, 2260, 2260, 2864, 2260, 2260, 2260, + /* 860 */ 2898, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2506, + /* 870 */ 2505, 2504, 2503, 2260, 2260, 2260, 2260, 2260, 2260, 2792, + /* 880 */ 2260, 2795, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 890 */ 2260, 2260, 2260, 2260, 2260, 2779, 2260, 2843, 2842, 2260, + /* 900 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 910 */ 2500, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 920 */ 2260, 2260, 2260, 2260, 2260, 2260, 2484, 2482, 2481, 2480, + /* 930 */ 2260, 2517, 2260, 2260, 2260, 2513, 2512, 2260, 2260, 2260, + /* 940 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 950 */ 2260, 2260, 2260, 2260, 2260, 2260, 2385, 2260, 2260, 2260, + /* 960 */ 2260, 2260, 2260, 2260, 2260, 2377, 2260, 2376, 2260, 2260, + /* 970 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 980 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2260, + /* 990 */ 2260, 2260, 2260, 2260, 2260, 2260, 2260, 2289, 2260, 2260, + /* 1000 */ 2260, 2260, 2260, 2260, 2260, 2260, }; /********** End of lemon-generated parsing tables *****************************/ @@ -3349,18 +3354,18 @@ static const char *const yyRuleName[] = { /* 568 */ "function_expression ::= POSITION NK_LP expr_or_subquery IN expr_or_subquery NK_RP", /* 569 */ "function_expression ::= TRIM NK_LP expr_or_subquery NK_RP", /* 570 */ "function_expression ::= TRIM NK_LP trim_specification_type FROM expr_or_subquery NK_RP", - /* 571 */ "function_expression ::= TRIM NK_LP trim_specification_type expr_or_subquery FROM expr_or_subquery NK_RP", - /* 572 */ "function_expression ::= substr_func NK_LP expression_list NK_RP", - /* 573 */ "function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery NK_RP", - /* 574 */ "function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery FOR expr_or_subquery NK_RP", - /* 575 */ "function_expression ::= REPLACE NK_LP expression_list NK_RP", - /* 576 */ "function_expression ::= literal_func", - /* 577 */ "literal_func ::= noarg_func NK_LP NK_RP", - /* 578 */ "literal_func ::= NOW", - /* 579 */ "literal_func ::= TODAY", - /* 580 */ "substr_func ::= SUBSTR", - /* 581 */ "substr_func ::= SUBSTRING", - /* 582 */ "trim_specification_type ::=", + /* 571 */ "function_expression ::= TRIM NK_LP expr_or_subquery FROM expr_or_subquery NK_RP", + /* 572 */ "function_expression ::= TRIM NK_LP trim_specification_type expr_or_subquery FROM expr_or_subquery NK_RP", + /* 573 */ "function_expression ::= substr_func NK_LP expression_list NK_RP", + /* 574 */ "function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery NK_RP", + /* 575 */ "function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery FOR expr_or_subquery NK_RP", + /* 576 */ "function_expression ::= REPLACE NK_LP expression_list NK_RP", + /* 577 */ "function_expression ::= literal_func", + /* 578 */ "literal_func ::= noarg_func NK_LP NK_RP", + /* 579 */ "literal_func ::= NOW", + /* 580 */ "literal_func ::= TODAY", + /* 581 */ "substr_func ::= SUBSTR", + /* 582 */ "substr_func ::= SUBSTRING", /* 583 */ "trim_specification_type ::= BOTH", /* 584 */ "trim_specification_type ::= TRAILING", /* 585 */ "trim_specification_type ::= LEADING", @@ -4802,18 +4807,18 @@ static const YYCODETYPE yyRuleInfoLhs[] = { 505, /* (568) function_expression ::= POSITION NK_LP expr_or_subquery IN expr_or_subquery NK_RP */ 505, /* (569) function_expression ::= TRIM NK_LP expr_or_subquery NK_RP */ 505, /* (570) function_expression ::= TRIM NK_LP trim_specification_type FROM expr_or_subquery NK_RP */ - 505, /* (571) function_expression ::= TRIM NK_LP trim_specification_type expr_or_subquery FROM expr_or_subquery NK_RP */ - 505, /* (572) function_expression ::= substr_func NK_LP expression_list NK_RP */ - 505, /* (573) function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery NK_RP */ - 505, /* (574) function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery FOR expr_or_subquery NK_RP */ - 505, /* (575) function_expression ::= REPLACE NK_LP expression_list NK_RP */ - 505, /* (576) function_expression ::= literal_func */ - 498, /* (577) literal_func ::= noarg_func NK_LP NK_RP */ - 498, /* (578) literal_func ::= NOW */ - 498, /* (579) literal_func ::= TODAY */ - 510, /* (580) substr_func ::= SUBSTR */ - 510, /* (581) substr_func ::= SUBSTRING */ - 509, /* (582) trim_specification_type ::= */ + 505, /* (571) function_expression ::= TRIM NK_LP expr_or_subquery FROM expr_or_subquery NK_RP */ + 505, /* (572) function_expression ::= TRIM NK_LP trim_specification_type expr_or_subquery FROM expr_or_subquery NK_RP */ + 505, /* (573) function_expression ::= substr_func NK_LP expression_list NK_RP */ + 505, /* (574) function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery NK_RP */ + 505, /* (575) function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery FOR expr_or_subquery NK_RP */ + 505, /* (576) function_expression ::= REPLACE NK_LP expression_list NK_RP */ + 505, /* (577) function_expression ::= literal_func */ + 498, /* (578) literal_func ::= noarg_func NK_LP NK_RP */ + 498, /* (579) literal_func ::= NOW */ + 498, /* (580) literal_func ::= TODAY */ + 510, /* (581) substr_func ::= SUBSTR */ + 510, /* (582) substr_func ::= SUBSTRING */ 509, /* (583) trim_specification_type ::= BOTH */ 509, /* (584) trim_specification_type ::= TRAILING */ 509, /* (585) trim_specification_type ::= LEADING */ @@ -5577,18 +5582,18 @@ static const signed char yyRuleInfoNRhs[] = { -6, /* (568) function_expression ::= POSITION NK_LP expr_or_subquery IN expr_or_subquery NK_RP */ -4, /* (569) function_expression ::= TRIM NK_LP expr_or_subquery NK_RP */ -6, /* (570) function_expression ::= TRIM NK_LP trim_specification_type FROM expr_or_subquery NK_RP */ - -7, /* (571) function_expression ::= TRIM NK_LP trim_specification_type expr_or_subquery FROM expr_or_subquery NK_RP */ - -4, /* (572) function_expression ::= substr_func NK_LP expression_list NK_RP */ - -6, /* (573) function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery NK_RP */ - -8, /* (574) function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery FOR expr_or_subquery NK_RP */ - -4, /* (575) function_expression ::= REPLACE NK_LP expression_list NK_RP */ - -1, /* (576) function_expression ::= literal_func */ - -3, /* (577) literal_func ::= noarg_func NK_LP NK_RP */ - -1, /* (578) literal_func ::= NOW */ - -1, /* (579) literal_func ::= TODAY */ - -1, /* (580) substr_func ::= SUBSTR */ - -1, /* (581) substr_func ::= SUBSTRING */ - 0, /* (582) trim_specification_type ::= */ + -6, /* (571) function_expression ::= TRIM NK_LP expr_or_subquery FROM expr_or_subquery NK_RP */ + -7, /* (572) function_expression ::= TRIM NK_LP trim_specification_type expr_or_subquery FROM expr_or_subquery NK_RP */ + -4, /* (573) function_expression ::= substr_func NK_LP expression_list NK_RP */ + -6, /* (574) function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery NK_RP */ + -8, /* (575) function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery FOR expr_or_subquery NK_RP */ + -4, /* (576) function_expression ::= REPLACE NK_LP expression_list NK_RP */ + -1, /* (577) function_expression ::= literal_func */ + -3, /* (578) literal_func ::= noarg_func NK_LP NK_RP */ + -1, /* (579) literal_func ::= NOW */ + -1, /* (580) literal_func ::= TODAY */ + -1, /* (581) substr_func ::= SUBSTR */ + -1, /* (582) substr_func ::= SUBSTRING */ -1, /* (583) trim_specification_type ::= BOTH */ -1, /* (584) trim_specification_type ::= TRAILING */ -1, /* (585) trim_specification_type ::= LEADING */ @@ -6065,8 +6070,8 @@ static YYACTIONTYPE yy_reduce( case 526: /* cgroup_name ::= NK_ID */ yytestcase(yyruleno==526); case 527: /* index_name ::= NK_ID */ yytestcase(yyruleno==527); case 528: /* tsma_name ::= NK_ID */ yytestcase(yyruleno==528); - case 580: /* substr_func ::= SUBSTR */ yytestcase(yyruleno==580); - case 581: /* substr_func ::= SUBSTRING */ yytestcase(yyruleno==581); + case 581: /* substr_func ::= SUBSTR */ yytestcase(yyruleno==581); + case 582: /* substr_func ::= SUBSTRING */ yytestcase(yyruleno==582); case 586: /* noarg_func ::= NOW */ yytestcase(yyruleno==586); case 587: /* noarg_func ::= TODAY */ yytestcase(yyruleno==587); case 588: /* noarg_func ::= TIMEZONE */ yytestcase(yyruleno==588); @@ -7317,7 +7322,7 @@ static YYACTIONTYPE yy_reduce( case 532: /* expression ::= column_reference */ yytestcase(yyruleno==532); case 533: /* expression ::= function_expression */ yytestcase(yyruleno==533); case 534: /* expression ::= case_when_expression */ yytestcase(yyruleno==534); - case 576: /* function_expression ::= literal_func */ yytestcase(yyruleno==576); + case 577: /* function_expression ::= literal_func */ yytestcase(yyruleno==577); case 633: /* boolean_value_expression ::= boolean_primary */ yytestcase(yyruleno==633); case 637: /* boolean_primary ::= predicate */ yytestcase(yyruleno==637); case 639: /* common_expression ::= expr_or_subquery */ yytestcase(yyruleno==639); @@ -7522,8 +7527,8 @@ static YYACTIONTYPE yy_reduce( case 561: /* pseudo_column ::= IROWTS */ yytestcase(yyruleno==561); case 562: /* pseudo_column ::= ISFILLED */ yytestcase(yyruleno==562); case 563: /* pseudo_column ::= QTAGS */ yytestcase(yyruleno==563); - case 578: /* literal_func ::= NOW */ yytestcase(yyruleno==578); - case 579: /* literal_func ::= TODAY */ yytestcase(yyruleno==579); + case 579: /* literal_func ::= NOW */ yytestcase(yyruleno==579); + case 580: /* literal_func ::= TODAY */ yytestcase(yyruleno==580); { yylhsminor.yy560 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[0].minor.yy0, NULL)); } yymsp[0].minor.yy560 = yylhsminor.yy560; break; @@ -7533,7 +7538,7 @@ static YYACTIONTYPE yy_reduce( break; case 564: /* function_expression ::= function_name NK_LP expression_list NK_RP */ case 565: /* function_expression ::= star_func NK_LP star_func_para_list NK_RP */ yytestcase(yyruleno==565); - case 572: /* function_expression ::= substr_func NK_LP expression_list NK_RP */ yytestcase(yyruleno==572); + case 573: /* function_expression ::= substr_func NK_LP expression_list NK_RP */ yytestcase(yyruleno==573); { yylhsminor.yy560 = createRawExprNodeExt(pCxt, &yymsp[-3].minor.yy533, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[-3].minor.yy533, yymsp[-1].minor.yy334)); } yymsp[-3].minor.yy560 = yylhsminor.yy560; break; @@ -7554,29 +7559,30 @@ static YYACTIONTYPE yy_reduce( { yylhsminor.yy560 = createRawExprNodeExt(pCxt, &yymsp[-5].minor.yy0, &yymsp[0].minor.yy0, createTrimFunctionNode(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy560), yymsp[-3].minor.yy672)); } yymsp[-5].minor.yy560 = yylhsminor.yy560; break; - case 571: /* function_expression ::= TRIM NK_LP trim_specification_type expr_or_subquery FROM expr_or_subquery NK_RP */ + case 571: /* function_expression ::= TRIM NK_LP expr_or_subquery FROM expr_or_subquery NK_RP */ + { yylhsminor.yy560 = createRawExprNodeExt(pCxt, &yymsp[-5].minor.yy0, &yymsp[0].minor.yy0, createTrimFunctionNodeExt(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy560), releaseRawExprNode(pCxt, yymsp[-1].minor.yy560), TRIM_TYPE_BOTH)); } + yymsp[-5].minor.yy560 = yylhsminor.yy560; + break; + case 572: /* function_expression ::= TRIM NK_LP trim_specification_type expr_or_subquery FROM expr_or_subquery NK_RP */ { yylhsminor.yy560 = createRawExprNodeExt(pCxt, &yymsp[-6].minor.yy0, &yymsp[0].minor.yy0, createTrimFunctionNodeExt(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy560), releaseRawExprNode(pCxt, yymsp[-1].minor.yy560), yymsp[-4].minor.yy672)); } yymsp[-6].minor.yy560 = yylhsminor.yy560; break; - case 573: /* function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery NK_RP */ + case 574: /* function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery NK_RP */ { yylhsminor.yy560 = createRawExprNodeExt(pCxt, &yymsp[-5].minor.yy533, &yymsp[0].minor.yy0, createSubstrFunctionNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy560), releaseRawExprNode(pCxt, yymsp[-1].minor.yy560))); } yymsp[-5].minor.yy560 = yylhsminor.yy560; break; - case 574: /* function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery FOR expr_or_subquery NK_RP */ + case 575: /* function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery FOR expr_or_subquery NK_RP */ { yylhsminor.yy560 = createRawExprNodeExt(pCxt, &yymsp[-7].minor.yy533, &yymsp[0].minor.yy0, createSubstrFunctionNodeExt(pCxt, releaseRawExprNode(pCxt, yymsp[-5].minor.yy560), releaseRawExprNode(pCxt, yymsp[-3].minor.yy560), releaseRawExprNode(pCxt, yymsp[-1].minor.yy560))); } yymsp[-7].minor.yy560 = yylhsminor.yy560; break; - case 575: /* function_expression ::= REPLACE NK_LP expression_list NK_RP */ + case 576: /* function_expression ::= REPLACE NK_LP expression_list NK_RP */ { yylhsminor.yy560 = createRawExprNodeExt(pCxt, &yymsp[-3].minor.yy0, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[-3].minor.yy0, yymsp[-1].minor.yy334)); } yymsp[-3].minor.yy560 = yylhsminor.yy560; break; - case 577: /* literal_func ::= noarg_func NK_LP NK_RP */ + case 578: /* literal_func ::= noarg_func NK_LP NK_RP */ { yylhsminor.yy560 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy533, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[-2].minor.yy533, NULL)); } yymsp[-2].minor.yy560 = yylhsminor.yy560; break; - case 582: /* trim_specification_type ::= */ - { yymsp[1].minor.yy672 = TRIM_TYPE_BOTH; } - break; case 583: /* trim_specification_type ::= BOTH */ { yymsp[0].minor.yy672 = TRIM_TYPE_BOTH; } break; From f2dcd3f16adb5aa1738d58a93014e17b97c01543 Mon Sep 17 00:00:00 2001 From: sima Date: Thu, 15 Aug 2024 15:20:10 +0800 Subject: [PATCH 24/46] fix:[TD-31470] Fix replace function wrong length. --- source/libs/function/src/builtins.c | 17 ++++++++++++++--- source/libs/scalar/src/sclfunc.c | 16 +++++++++++++++- 2 files changed, 29 insertions(+), 4 deletions(-) diff --git a/source/libs/function/src/builtins.c b/source/libs/function/src/builtins.c index e6724f9e0a..760a3c4a33 100644 --- a/source/libs/function/src/builtins.c +++ b/source/libs/function/src/builtins.c @@ -2327,13 +2327,24 @@ static int32_t translateReplace(SFunctionNode* pFunc, char* pErrBuf, int32_t len } } - uint8_t type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type; + uint8_t orgType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type; + uint8_t fromType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 1))->type; + uint8_t toType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 2))->type; int32_t orgLen = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->bytes; int32_t fromLen = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 1))->bytes; int32_t toLen = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 2))->bytes; - int32_t resLen = orgLen + orgLen / fromLen * (toLen - fromLen); - pFunc->node.resType = (SDataType){.bytes = resLen, .type = type}; + int32_t resLen; + // Since we don't know the accurate length of result, estimate the maximum length here. + // To make the resLen bigger, we should make fromLen smaller and toLen bigger. + if (orgType == TSDB_DATA_TYPE_VARBINARY && fromType != orgType) { + fromLen = fromLen / TSDB_NCHAR_SIZE; + } + if (orgType == TSDB_DATA_TYPE_NCHAR && toType != orgType) { + toLen = toLen * TSDB_NCHAR_SIZE; + } + resLen = TMAX(orgLen, orgLen + orgLen / fromLen * (toLen - fromLen)); + pFunc->node.resType = (SDataType){.bytes = resLen, .type = orgType}; return TSDB_CODE_SUCCESS; } diff --git a/source/libs/scalar/src/sclfunc.c b/source/libs/scalar/src/sclfunc.c index a666404838..fde87ba4f1 100644 --- a/source/libs/scalar/src/sclfunc.c +++ b/source/libs/scalar/src/sclfunc.c @@ -1458,7 +1458,21 @@ int32_t replaceFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pO numOfRows = TMAX(numOfRows, pInput[i].numOfRows); } - outputLen = pInputData[0]->info.bytes + pInputData[0]->info.bytes / pInputData[1]->info.bytes * (pInputData[2]->info.bytes - pInputData[1]->info.bytes); + int8_t orgType = pInputData[0]->info.type; + int8_t fromType = pInputData[1]->info.type; + int8_t toType = pInputData[2]->info.type; + int32_t orgLength = pInputData[0]->info.bytes; + int32_t fromLength = pInputData[1]->info.bytes; + int32_t toLength = pInputData[2]->info.bytes; + + if (orgType == TSDB_DATA_TYPE_VARBINARY && fromType != orgType) { + fromLength = fromLength / TSDB_NCHAR_SIZE; + } + if (orgType == TSDB_DATA_TYPE_NCHAR && toType != orgType) { + toLength = toLength * TSDB_NCHAR_SIZE; + } + outputLen = TMAX(orgLength, orgLength + orgLength / fromLength * (toLength - fromLength)); + if (GET_PARAM_TYPE(&pInput[0]) == TSDB_DATA_TYPE_NULL || GET_PARAM_TYPE(&pInput[1]) == TSDB_DATA_TYPE_NULL || GET_PARAM_TYPE(&pInput[2]) == TSDB_DATA_TYPE_NULL || From cbf5f231be10914f3afe0664dfaba3b93ab3b194 Mon Sep 17 00:00:00 2001 From: dapan1121 Date: Thu, 15 Aug 2024 15:38:09 +0800 Subject: [PATCH 25/46] fix: group cache log issue --- source/libs/executor/src/groupcacheoperator.c | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/source/libs/executor/src/groupcacheoperator.c b/source/libs/executor/src/groupcacheoperator.c index 00b8c3b9ae..3e9ac2b10a 100644 --- a/source/libs/executor/src/groupcacheoperator.c +++ b/source/libs/executor/src/groupcacheoperator.c @@ -73,6 +73,10 @@ static int32_t initGroupColsInfo(SGroupColsInfo* pCols, bool grpColsMayBeNull, S } static void logGroupCacheExecInfo(SGroupCacheOperatorInfo* pGrpCacheOperator) { + if (pGrpCacheOperator->downstreamNum <= 0 || NULL == pGrpCacheOperator->execInfo.pDownstreamBlkNum) { + return; + } + char* buf = taosMemoryMalloc(pGrpCacheOperator->downstreamNum * 32 + 100); if (NULL == buf) { return; From efff5e2bf9181cd483d3e76b29b3608640ca8c52 Mon Sep 17 00:00:00 2001 From: Shungang Li Date: Thu, 15 Aug 2024 16:06:12 +0800 Subject: [PATCH 26/46] feat: s3 use global s3BucketName --- source/common/src/cos.c | 24 ++++++++++++------------ source/common/src/tglobal.c | 20 ++++++++++++-------- 2 files changed, 24 insertions(+), 20 deletions(-) diff --git a/source/common/src/cos.c b/source/common/src/cos.c index aa587bf07b..8392b0564a 100644 --- a/source/common/src/cos.c +++ b/source/common/src/cos.c @@ -9,7 +9,7 @@ extern int8_t tsS3EpNum; extern char tsS3Endpoint[][TSDB_FQDN_LEN]; extern char tsS3AccessKeyId[][TSDB_FQDN_LEN]; extern char tsS3AccessKeySecret[][TSDB_FQDN_LEN]; -extern char tsS3BucketName[][TSDB_FQDN_LEN]; +extern char tsS3BucketName[TSDB_FQDN_LEN]; extern char tsS3AppId[][TSDB_FQDN_LEN]; extern char tsS3Hostname[][TSDB_FQDN_LEN]; extern int8_t tsS3Https; @@ -130,13 +130,13 @@ int32_t s3CheckCfg() { (void)fprintf(stderr, "put object %s: success.\n\n", objectname[0]); // list buckets - (void)fprintf(stderr, "start to list bucket %s by prefix s3.\n", tsS3BucketName[i]); - code = s3ListBucketByEp(tsS3BucketName[i], i); + (void)fprintf(stderr, "start to list bucket %s by prefix s3.\n", tsS3BucketName); + code = s3ListBucketByEp(tsS3BucketName, i); if (code != 0) { - (void)fprintf(stderr, "listing bucket %s : failed.\n", tsS3BucketName[i]); + (void)fprintf(stderr, "listing bucket %s : failed.\n", tsS3BucketName); TAOS_CHECK_GOTO(code, &lino, _exit); } - (void)fprintf(stderr, "listing bucket %s: success.\n\n", tsS3BucketName[i]); + (void)fprintf(stderr, "listing bucket %s: success.\n\n", tsS3BucketName); // test range get uint8_t *pBlock = NULL; @@ -975,7 +975,7 @@ int32_t s3PutObjectFromFile2ByEp(const char *file, const char *object_name, int8 contentLength; S3BucketContext bucketContext = {tsS3Hostname[epIndex], - tsS3BucketName[epIndex], + tsS3BucketName, protocolG, uriStyleG, tsS3AccessKeyId[epIndex], @@ -1058,7 +1058,7 @@ static int32_t s3PutObjectFromFileOffsetByEp(const char *file, const char *objec contentLength; S3BucketContext bucketContext = {tsS3Hostname[epIndex], - tsS3BucketName[epIndex], + tsS3BucketName, protocolG, uriStyleG, tsS3AccessKeyId[epIndex], @@ -1154,7 +1154,7 @@ static void s3FreeObjectKey(void *pItem) { static SArray *getListByPrefixByEp(const char *prefix, int8_t epIndex) { S3BucketContext bucketContext = {tsS3Hostname[epIndex], - tsS3BucketName[epIndex], + tsS3BucketName, protocolG, uriStyleG, tsS3AccessKeyId[epIndex], @@ -1222,7 +1222,7 @@ static int32_t s3DeleteObjectsByEp(const char *object_name[], int nobject, int8_ int32_t code = 0; S3BucketContext bucketContext = {tsS3Hostname[epIndex], - tsS3BucketName[epIndex], + tsS3BucketName, protocolG, uriStyleG, tsS3AccessKeyId[epIndex], @@ -1298,7 +1298,7 @@ static int32_t s3GetObjectBlockByEp(const char *object_name, int64_t offset, int const char *ifMatch = 0, *ifNotMatch = 0; S3BucketContext bucketContext = {tsS3Hostname[epIndex], - tsS3BucketName[epIndex], + tsS3BucketName, protocolG, uriStyleG, tsS3AccessKeyId[epIndex], @@ -1371,7 +1371,7 @@ static int32_t s3GetObjectToFileByEp(const char *object_name, const char *fileNa const char *ifMatch = 0, *ifNotMatch = 0; S3BucketContext bucketContext = {tsS3Hostname[epIndex], - tsS3BucketName[epIndex], + tsS3BucketName, protocolG, uriStyleG, tsS3AccessKeyId[epIndex], @@ -1448,7 +1448,7 @@ static long s3SizeByEp(const char *object_name, int8_t epIndex) { int status = 0; S3BucketContext bucketContext = {tsS3Hostname[epIndex], - tsS3BucketName[epIndex], + tsS3BucketName, protocolG, uriStyleG, tsS3AccessKeyId[epIndex], diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index 4e32288123..3373b56ad8 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -299,7 +299,7 @@ char tsS3Endpoint[TSDB_MAX_EP_NUM][TSDB_FQDN_LEN] = {""}; char tsS3AccessKey[TSDB_MAX_EP_NUM][TSDB_FQDN_LEN] = {""}; char tsS3AccessKeyId[TSDB_MAX_EP_NUM][TSDB_FQDN_LEN] = {""}; char tsS3AccessKeySecret[TSDB_MAX_EP_NUM][TSDB_FQDN_LEN] = {""}; -char tsS3BucketName[TSDB_MAX_EP_NUM][TSDB_FQDN_LEN] = {""}; +char tsS3BucketName[TSDB_FQDN_LEN] = ""; char tsS3AppId[TSDB_MAX_EP_NUM][TSDB_FQDN_LEN] = {""}; int8_t tsS3Enabled = false; int8_t tsS3EnabledCfg = false; @@ -404,10 +404,14 @@ int32_t taosSetS3Cfg(SConfig *pCfg) { } TAOS_CHECK_RETURN(taosSplitS3Cfg(pCfg, "s3Endpoint", tsS3Endpoint, &num)); - if (num != tsS3EpNum) TAOS_RETURN(TSDB_CODE_INVALID_CFG); + if (num != tsS3EpNum) { + uError("invalid s3 ep num:%d, expected:%d, ", num, tsS3EpNum); + TAOS_RETURN(TSDB_CODE_INVALID_CFG); + } - TAOS_CHECK_RETURN(taosSplitS3Cfg(pCfg, "s3BucketName", tsS3BucketName, &num)); - if (num != tsS3EpNum) TAOS_RETURN(TSDB_CODE_INVALID_CFG); + SConfigItem *pItem = NULL; + TAOS_CHECK_GET_CFG_ITEM(pCfg, pItem, "s3BucketName"); + tstrncpy(tsS3BucketName, pItem->str, TSDB_FQDN_LEN); for (int i = 0; i < tsS3EpNum; ++i) { char *proto = strstr(tsS3Endpoint[i], "https://"); @@ -419,9 +423,9 @@ int32_t taosSetS3Cfg(SConfig *pCfg) { char *cos = strstr(tsS3Endpoint[i], "cos."); if (cos) { - char *appid = strrchr(tsS3BucketName[i], '-'); + char *appid = strrchr(tsS3BucketName, '-'); if (!appid) { - uError("failed to locate appid in bucket:%s", tsS3BucketName[i]); + uError("failed to locate appid in bucket:%s", tsS3BucketName); TAOS_RETURN(TSDB_CODE_INVALID_CFG); } else { tstrncpy(tsS3AppId[i], appid + 1, TSDB_FQDN_LEN); @@ -432,7 +436,7 @@ int32_t taosSetS3Cfg(SConfig *pCfg) { tsS3Https = (strstr(tsS3Endpoint[0], "https://") != NULL); tsS3Oss = (strstr(tsS3Endpoint[0], "aliyuncs.") != NULL); - if (tsS3BucketName[0][0] != '<') { + if (tsS3BucketName[0] != '<') { #if defined(USE_COS) || defined(USE_S3) #ifdef TD_ENTERPRISE /*if (tsDiskCfgNum > 1) */ tsS3Enabled = true; @@ -818,7 +822,7 @@ static int32_t taosAddServerCfg(SConfig *pCfg) { TAOS_CHECK_RETURN(cfgAddString(pCfg, "s3Accesskey", tsS3AccessKey[0], CFG_SCOPE_SERVER, CFG_DYN_NONE)); TAOS_CHECK_RETURN(cfgAddString(pCfg, "s3Endpoint", tsS3Endpoint[0], CFG_SCOPE_SERVER, CFG_DYN_NONE)); - TAOS_CHECK_RETURN(cfgAddString(pCfg, "s3BucketName", tsS3BucketName[0], CFG_SCOPE_SERVER, CFG_DYN_NONE)); + TAOS_CHECK_RETURN(cfgAddString(pCfg, "s3BucketName", tsS3BucketName, CFG_SCOPE_SERVER, CFG_DYN_NONE)); TAOS_CHECK_RETURN(cfgAddInt32(pCfg, "s3PageCacheSize", tsS3PageCacheSize, 4, 1024 * 1024 * 1024, CFG_SCOPE_SERVER, CFG_DYN_ENT_SERVER)); TAOS_CHECK_RETURN(cfgAddInt32(pCfg, "s3UploadDelaySec", tsS3UploadDelaySec, 1, 60 * 60 * 24 * 30, CFG_SCOPE_SERVER, CFG_DYN_ENT_SERVER)); From f62e849222ab9f4f28177ebdcd159b32344bb088 Mon Sep 17 00:00:00 2001 From: kailixu Date: Thu, 15 Aug 2024 16:07:57 +0800 Subject: [PATCH 27/46] fix: support customized version --- source/dnode/mgmt/exe/dmMain.c | 6 +++--- source/os/src/osSysinfo.c | 6 +++++- tools/shell/src/shellArguments.c | 10 +++++++--- 3 files changed, 15 insertions(+), 7 deletions(-) diff --git a/source/dnode/mgmt/exe/dmMain.c b/source/dnode/mgmt/exe/dmMain.c index b3e5015706..e5c37e3d55 100644 --- a/source/dnode/mgmt/exe/dmMain.c +++ b/source/dnode/mgmt/exe/dmMain.c @@ -125,7 +125,7 @@ void dmLogCrash(int signum, void *sigInfo, void *context) { _return: - taosLogCrashInfo("taosd", pMsg, msgLen, signum, sigInfo); + taosLogCrashInfo(CUS_PROMPT "d", pMsg, msgLen, signum, sigInfo); #ifdef _TD_DARWIN_64 exit(signum); @@ -258,7 +258,7 @@ static void dmPrintArgs(int32_t argc, char const *argv[]) { static void dmGenerateGrant() { mndGenerateMachineCode(); } static void dmPrintVersion() { - printf("%s\ntaosd version: %s compatible_version: %s\n", TD_PRODUCT_NAME, version, compatible_version); + printf("%s\n%sd version: %s compatible_version: %s\n", TD_PRODUCT_NAME, CUS_PROMPT, version, compatible_version); printf("git: %s\n", gitinfo); #ifdef TD_ENTERPRISE printf("gitOfInternal: %s\n", gitinfoOfInternal); @@ -268,7 +268,7 @@ static void dmPrintVersion() { static void dmPrintHelp() { char indent[] = " "; - printf("Usage: taosd [OPTION...] \n\n"); + printf("Usage: %sd [OPTION...] \n\n", CUS_PROMPT); printf("%s%s%s%s\n", indent, "-a,", indent, DM_APOLLO_URL); printf("%s%s%s%s\n", indent, "-c,", indent, DM_CFG_DIR); printf("%s%s%s%s\n", indent, "-s,", indent, DM_SDB_INFO); diff --git a/source/os/src/osSysinfo.c b/source/os/src/osSysinfo.c index 67a2cd97c4..92e5967416 100644 --- a/source/os/src/osSysinfo.c +++ b/source/os/src/osSysinfo.c @@ -19,6 +19,10 @@ #if defined(CUS_NAME) || defined(CUS_PROMPT) || defined(CUS_EMAIL) #include "cus_name.h" +#else +#ifndef CUS_PROMPT +#define CUS_PROMPT "taos" +#endif #endif #define PROCESS_ITEM 12 @@ -987,7 +991,7 @@ void taosKillSystem() { exit(0); #else // SIGINT - (void)printf("taosd will shut down soon"); + (void)printf("%sd will shut down soon", CUS_PROMPT); (void)kill(tsProcId, 2); #endif } diff --git a/tools/shell/src/shellArguments.c b/tools/shell/src/shellArguments.c index 1eb61d2394..cf3c7824fa 100644 --- a/tools/shell/src/shellArguments.c +++ b/tools/shell/src/shellArguments.c @@ -22,6 +22,10 @@ #if defined(CUS_NAME) || defined(CUS_PROMPT) || defined(CUS_EMAIL) #include "cus_name.h" +#else +#ifndef CUS_PROMPT +#define CUS_PROMPT "taos" +#endif #endif #define TAOS_CONSOLE_PROMPT_CONTINUE " -> " @@ -435,11 +439,11 @@ int32_t shellParseArgs(int32_t argc, char *argv[]) { shell.info.promptSize = strlen(shell.info.promptHeader); #ifdef TD_ENTERPRISE snprintf(shell.info.programVersion, sizeof(shell.info.programVersion), - "%s\ntaos version: %s compatible_version: %s\ngit: %s\ngitOfInternal: %s\nbuild: %s", TD_PRODUCT_NAME, - version, compatible_version, gitinfo, gitinfoOfInternal, buildinfo); + "%s\n%s version: %s compatible_version: %s\ngit: %s\ngitOfInternal: %s\nbuild: %s", TD_PRODUCT_NAME, + CUS_PROMPT, version, compatible_version, gitinfo, gitinfoOfInternal, buildinfo); #else snprintf(shell.info.programVersion, sizeof(shell.info.programVersion), - "%s\ntaos version: %s compatible_version: %s\ngit: %s\nbuild: %s", TD_PRODUCT_NAME, version, + "%s\n%s version: %s compatible_version: %s\ngit: %s\nbuild: %s", TD_PRODUCT_NAME, CUS_PROMPT, version, compatible_version, gitinfo, buildinfo); #endif From 85ec91fb5837a5f6d803d42f7d4846f4e7351248 Mon Sep 17 00:00:00 2001 From: sima Date: Thu, 15 Aug 2024 16:11:38 +0800 Subject: [PATCH 28/46] fix:[TD-31473] Fix repeat function use tag as parameter. --- source/libs/scalar/src/sclfunc.c | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/source/libs/scalar/src/sclfunc.c b/source/libs/scalar/src/sclfunc.c index a666404838..a68dcb19b2 100644 --- a/source/libs/scalar/src/sclfunc.c +++ b/source/libs/scalar/src/sclfunc.c @@ -1639,7 +1639,9 @@ int32_t repeatFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOu int32_t maxCount = 0; for (int32_t i = 0; i < pInput[1].numOfRows; i++) { - maxCount = TMAX(maxCount, *(int32_t *)colDataGetData(pInput[1].columnData, i)); + int32_t tmpCount = 0; + GET_TYPED_DATA(tmpCount, int32_t, GET_PARAM_TYPE(&pInput[1]), colDataGetData(pInput[1].columnData, i)); + maxCount = TMAX(maxCount, tmpCount); } pInputData[0] = pInput[0].columnData; pInputData[1] = pInput[1].columnData; @@ -1663,7 +1665,8 @@ int32_t repeatFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOu colDataSetNULL(pOutputData, i); continue; } - int32_t count = *(int32_t *)colDataGetData(pInputData[1], i); + int32_t count = 0; + GET_TYPED_DATA(count, int32_t, GET_PARAM_TYPE(&pInput[1]), colDataGetData(pInput[1].columnData, i)); if (count <= 0) { varDataSetLen(output, 0); SCL_ERR_JRET(colDataSetVal(pOutputData, i, outputBuf, false)); @@ -1684,7 +1687,8 @@ int32_t repeatFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOu colDataSetNULL(pOutputData, i); continue; } - int32_t count = *(int32_t *)colDataGetData(pInputData[1], i); + int32_t count = 0; + GET_TYPED_DATA(count, int32_t, GET_PARAM_TYPE(&pInput[1]), colDataGetData(pInput[1].columnData, i)); if (count <= 0) { varDataSetLen(output, 0); SCL_ERR_JRET(colDataSetVal(pOutputData, i, outputBuf, false)); @@ -1706,7 +1710,8 @@ int32_t repeatFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOu colDataSetNULL(pOutputData, i); continue; } - int32_t count = *(int32_t *)colDataGetData(pInputData[1], 0); + int32_t count = 0; + GET_TYPED_DATA(count, int32_t, GET_PARAM_TYPE(&pInput[1]), colDataGetData(pInput[1].columnData, i)); if (count <= 0) { varDataSetLen(output, 0); SCL_ERR_JRET(colDataSetVal(pOutputData, i, outputBuf, false)); From b6629532bf3c8c1b8561ee949740cf1fc991de80 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 15 Aug 2024 16:20:49 +0800 Subject: [PATCH 29/46] fix(tsdb): fix memory leak. --- source/dnode/vnode/src/tsdb/tsdbReadUtil.c | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/tsdb/tsdbReadUtil.c b/source/dnode/vnode/src/tsdb/tsdbReadUtil.c index 4dabffc10a..a26c326b2a 100644 --- a/source/dnode/vnode/src/tsdb/tsdbReadUtil.c +++ b/source/dnode/vnode/src/tsdb/tsdbReadUtil.c @@ -337,13 +337,14 @@ int32_t createDataBlockScanInfo(STsdbReader* pTsdbReader, SBlockInfoBuf* pBuf, c int64_t st = taosGetTimestampUs(); code = initBlockScanInfoBuf(pBuf, numOfTables); if (code != TSDB_CODE_SUCCESS) { + tSimpleHashCleanup(pTableMap); return code; } pUidList->tableUidList = taosMemoryMalloc(numOfTables * sizeof(uint64_t)); if (pUidList->tableUidList == NULL) { tSimpleHashCleanup(pTableMap); - return TSDB_CODE_OUT_OF_MEMORY; + return terrno; } pUidList->currentIndex = 0; From cce4d1104f72a5c19411ef52a0231a7c943267d1 Mon Sep 17 00:00:00 2001 From: kailixu Date: Thu, 15 Aug 2024 16:29:20 +0800 Subject: [PATCH 30/46] fix: support customized version --- tools/shell/src/shellArguments.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/shell/src/shellArguments.c b/tools/shell/src/shellArguments.c index cf3c7824fa..4638f2ad74 100644 --- a/tools/shell/src/shellArguments.c +++ b/tools/shell/src/shellArguments.c @@ -61,7 +61,7 @@ static int32_t shellParseSingleOpt(int32_t key, char *arg); void shellPrintHelp() { char indent[] = " "; - printf("Usage: taos [OPTION...] \r\n\r\n"); + printf("Usage: %s [OPTION...] \r\n\r\n", CUS_PROMPT); printf("%s%s%s%s\r\n", indent, "-a,", indent, SHELL_AUTH); printf("%s%s%s%s\r\n", indent, "-A,", indent, SHELL_GEN_AUTH); printf("%s%s%s%s\r\n", indent, "-B,", indent, SHELL_BI_MODE); From 7cd086c8f97ad333038268423f83c5670f56ff03 Mon Sep 17 00:00:00 2001 From: Hongze Cheng Date: Thu, 15 Aug 2024 16:35:37 +0800 Subject: [PATCH 31/46] fix: --- source/client/src/clientTmq.c | 2 +- source/common/src/tmsg.c | 274 +++++++++++++++--------------- source/libs/tdb/src/db/tdbPager.c | 2 +- 3 files changed, 140 insertions(+), 138 deletions(-) diff --git a/source/client/src/clientTmq.c b/source/client/src/clientTmq.c index 0224a20109..1c0856b464 100644 --- a/source/client/src/clientTmq.c +++ b/source/client/src/clientTmq.c @@ -1441,7 +1441,7 @@ int32_t tmq_subscribe(tmq_t* tmq, const tmq_list_t* topic_list) { while ((code = syncAskEp(tmq)) != 0) { if (retryCnt++ > MAX_RETRY_COUNT || code == TSDB_CODE_MND_CONSUMER_NOT_EXIST) { tscError("consumer:0x%" PRIx64 ", mnd not ready for subscribe, retry more than 2 minutes, code:%s", - tmq->consumerId, strerror(code)); + tmq->consumerId, tstrerror(code)); if (code == TSDB_CODE_MND_CONSUMER_NOT_EXIST) { code = 0; } diff --git a/source/common/src/tmsg.c b/source/common/src/tmsg.c index 740e517e35..0258f4faa9 100644 --- a/source/common/src/tmsg.c +++ b/source/common/src/tmsg.c @@ -70,24 +70,24 @@ } while (0) static int32_t tSerializeSMonitorParas(SEncoder *encoder, const SMonitorParas *pMonitorParas) { - if (tEncodeI8(encoder, pMonitorParas->tsEnableMonitor) < 0) return -1; - if (tEncodeI32(encoder, pMonitorParas->tsMonitorInterval) < 0) return -1; - if (tEncodeI32(encoder, pMonitorParas->tsSlowLogScope) < 0) return -1; - if (tEncodeI32(encoder, pMonitorParas->tsSlowLogMaxLen) < 0) return -1; - if (tEncodeI32(encoder, pMonitorParas->tsSlowLogThreshold) < 0) return -1; - if (tEncodeI32(encoder, pMonitorParas->tsSlowLogThresholdTest) < 0) return -1; - if (tEncodeCStr(encoder, pMonitorParas->tsSlowLogExceptDb) < 0) return -1; + TAOS_CHECK_RETURN(tEncodeI8(encoder, pMonitorParas->tsEnableMonitor)); + TAOS_CHECK_RETURN(tEncodeI32(encoder, pMonitorParas->tsMonitorInterval)); + TAOS_CHECK_RETURN(tEncodeI32(encoder, pMonitorParas->tsSlowLogScope)); + TAOS_CHECK_RETURN(tEncodeI32(encoder, pMonitorParas->tsSlowLogMaxLen)); + TAOS_CHECK_RETURN(tEncodeI32(encoder, pMonitorParas->tsSlowLogThreshold)); + TAOS_CHECK_RETURN(tEncodeI32(encoder, pMonitorParas->tsSlowLogThresholdTest)); + TAOS_CHECK_RETURN(tEncodeCStr(encoder, pMonitorParas->tsSlowLogExceptDb)); return 0; } static int32_t tDeserializeSMonitorParas(SDecoder *decoder, SMonitorParas *pMonitorParas) { - if (tDecodeI8(decoder, (int8_t *)&pMonitorParas->tsEnableMonitor) < 0) return -1; - if (tDecodeI32(decoder, &pMonitorParas->tsMonitorInterval) < 0) return -1; - if (tDecodeI32(decoder, &pMonitorParas->tsSlowLogScope) < 0) return -1; - if (tDecodeI32(decoder, &pMonitorParas->tsSlowLogMaxLen) < 0) return -1; - if (tDecodeI32(decoder, &pMonitorParas->tsSlowLogThreshold) < 0) return -1; - if (tDecodeI32(decoder, &pMonitorParas->tsSlowLogThresholdTest) < 0) return -1; - if (tDecodeCStrTo(decoder, pMonitorParas->tsSlowLogExceptDb) < 0) return -1; + TAOS_CHECK_RETURN(tDecodeI8(decoder, (int8_t *)&pMonitorParas->tsEnableMonitor)); + TAOS_CHECK_RETURN(tDecodeI32(decoder, &pMonitorParas->tsMonitorInterval)); + TAOS_CHECK_RETURN(tDecodeI32(decoder, &pMonitorParas->tsSlowLogScope)); + TAOS_CHECK_RETURN(tDecodeI32(decoder, &pMonitorParas->tsSlowLogMaxLen)); + TAOS_CHECK_RETURN(tDecodeI32(decoder, &pMonitorParas->tsSlowLogThreshold)); + TAOS_CHECK_RETURN(tDecodeI32(decoder, &pMonitorParas->tsSlowLogThresholdTest)); + TAOS_CHECK_RETURN(tDecodeCStrTo(decoder, pMonitorParas->tsSlowLogExceptDb)); return 0; } @@ -98,8 +98,7 @@ static int32_t tDecodeTableTSMAInfoRsp(SDecoder *pDecoder, STableTSMAInfoRsp *pR int32_t tInitSubmitMsgIter(const SSubmitReq *pMsg, SSubmitMsgIter *pIter) { if (pMsg == NULL) { - terrno = TSDB_CODE_TDB_SUBMIT_MSG_MSSED_UP; - return -1; + return terrno = TSDB_CODE_TDB_SUBMIT_MSG_MSSED_UP; } pIter->totalLen = htonl(pMsg->length); @@ -108,8 +107,7 @@ int32_t tInitSubmitMsgIter(const SSubmitReq *pMsg, SSubmitMsgIter *pIter) { pIter->len = 0; pIter->pMsg = pMsg; if (pIter->totalLen <= sizeof(SSubmitReq)) { - terrno = TSDB_CODE_TDB_SUBMIT_MSG_MSSED_UP; - return -1; + return terrno = TSDB_CODE_TDB_SUBMIT_MSG_MSSED_UP; } return 0; @@ -130,9 +128,8 @@ int32_t tGetSubmitMsgNext(SSubmitMsgIter *pIter, SSubmitBlk **pPBlock) { } if (pIter->len > pIter->totalLen) { - terrno = TSDB_CODE_TDB_SUBMIT_MSG_MSSED_UP; *pPBlock = NULL; - return -1; + return terrno = TSDB_CODE_TDB_SUBMIT_MSG_MSSED_UP; } if (pIter->len == pIter->totalLen) { @@ -193,46 +190,46 @@ int32_t tPrintFixedSchemaSubmitReq(SSubmitReq *pReq, STSchema *pTschema) { #endif int32_t tEncodeSEpSet(SEncoder *pEncoder, const SEpSet *pEp) { - if (tEncodeI8(pEncoder, pEp->inUse) < 0) return -1; - if (tEncodeI8(pEncoder, pEp->numOfEps) < 0) return -1; + TAOS_CHECK_RETURN(tEncodeI8(pEncoder, pEp->inUse)); + TAOS_CHECK_RETURN(tEncodeI8(pEncoder, pEp->numOfEps)); for (int32_t i = 0; i < TSDB_MAX_REPLICA; i++) { - if (tEncodeU16(pEncoder, pEp->eps[i].port) < 0) return -1; - if (tEncodeCStrWithLen(pEncoder, pEp->eps[i].fqdn, TSDB_FQDN_LEN) < 0) return -1; + TAOS_CHECK_RETURN(tEncodeU16(pEncoder, pEp->eps[i].port)); + TAOS_CHECK_RETURN(tEncodeCStrWithLen(pEncoder, pEp->eps[i].fqdn, TSDB_FQDN_LEN)); } return 0; } int32_t tDecodeSEpSet(SDecoder *pDecoder, SEpSet *pEp) { - if (tDecodeI8(pDecoder, &pEp->inUse) < 0) return -1; - if (tDecodeI8(pDecoder, &pEp->numOfEps) < 0) return -1; + TAOS_CHECK_RETURN(tDecodeI8(pDecoder, &pEp->inUse)); + TAOS_CHECK_RETURN(tDecodeI8(pDecoder, &pEp->numOfEps)); for (int32_t i = 0; i < TSDB_MAX_REPLICA; i++) { - if (tDecodeU16(pDecoder, &pEp->eps[i].port) < 0) return -1; - if (tDecodeCStrTo(pDecoder, pEp->eps[i].fqdn) < 0) return -1; + TAOS_CHECK_RETURN(tDecodeU16(pDecoder, &pEp->eps[i].port)); + TAOS_CHECK_RETURN(tDecodeCStrTo(pDecoder, pEp->eps[i].fqdn)); } return 0; } int32_t tEncodeSQueryNodeAddr(SEncoder *pEncoder, SQueryNodeAddr *pAddr) { - if (tEncodeI32(pEncoder, pAddr->nodeId) < 0) return -1; - if (tEncodeSEpSet(pEncoder, &pAddr->epSet) < 0) return -1; + TAOS_CHECK_RETURN(tEncodeI32(pEncoder, pAddr->nodeId)); + TAOS_CHECK_RETURN(tEncodeSEpSet(pEncoder, &pAddr->epSet)); return 0; } int32_t tEncodeSQueryNodeLoad(SEncoder *pEncoder, SQueryNodeLoad *pLoad) { - if (tEncodeSQueryNodeAddr(pEncoder, &pLoad->addr) < 0) return -1; - if (tEncodeU64(pEncoder, pLoad->load) < 0) return -1; + TAOS_CHECK_RETURN(tEncodeSQueryNodeAddr(pEncoder, &pLoad->addr)); + TAOS_CHECK_RETURN(tEncodeU64(pEncoder, pLoad->load)); return 0; } int32_t tDecodeSQueryNodeAddr(SDecoder *pDecoder, SQueryNodeAddr *pAddr) { - if (tDecodeI32(pDecoder, &pAddr->nodeId) < 0) return -1; - if (tDecodeSEpSet(pDecoder, &pAddr->epSet) < 0) return -1; + TAOS_CHECK_RETURN(tDecodeI32(pDecoder, &pAddr->nodeId)); + TAOS_CHECK_RETURN(tDecodeSEpSet(pDecoder, &pAddr->epSet)); return 0; } int32_t tDecodeSQueryNodeLoad(SDecoder *pDecoder, SQueryNodeLoad *pLoad) { - if (tDecodeSQueryNodeAddr(pDecoder, &pLoad->addr) < 0) return -1; - if (tDecodeU64(pDecoder, &pLoad->load) < 0) return -1; + TAOS_CHECK_RETURN(tDecodeSQueryNodeAddr(pDecoder, &pLoad->addr)); + TAOS_CHECK_RETURN(tDecodeU64(pDecoder, &pLoad->load)); return 0; } @@ -258,63 +255,63 @@ void *taosDecodeSEpSet(const void *buf, SEpSet *pEp) { } static int32_t tSerializeSClientHbReq(SEncoder *pEncoder, const SClientHbReq *pReq) { - if (tEncodeSClientHbKey(pEncoder, &pReq->connKey) < 0) return -1; + TAOS_CHECK_RETURN(tEncodeSClientHbKey(pEncoder, &pReq->connKey)); if (pReq->connKey.connType == CONN_TYPE__QUERY) { - if (tEncodeI64(pEncoder, pReq->app.appId) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->app.pid) < 0) return -1; - if (tEncodeCStr(pEncoder, pReq->app.name) < 0) return -1; - if (tEncodeI64(pEncoder, pReq->app.startTime) < 0) return -1; - if (tEncodeU64(pEncoder, pReq->app.summary.numOfInsertsReq) < 0) return -1; - if (tEncodeU64(pEncoder, pReq->app.summary.numOfInsertRows) < 0) return -1; - if (tEncodeU64(pEncoder, pReq->app.summary.insertElapsedTime) < 0) return -1; - if (tEncodeU64(pEncoder, pReq->app.summary.insertBytes) < 0) return -1; - if (tEncodeU64(pEncoder, pReq->app.summary.fetchBytes) < 0) return -1; - if (tEncodeU64(pEncoder, pReq->app.summary.queryElapsedTime) < 0) return -1; - if (tEncodeU64(pEncoder, pReq->app.summary.numOfSlowQueries) < 0) return -1; - if (tEncodeU64(pEncoder, pReq->app.summary.totalRequests) < 0) return -1; - if (tEncodeU64(pEncoder, pReq->app.summary.currentRequests) < 0) return -1; + TAOS_CHECK_RETURN(tEncodeI64(pEncoder, pReq->app.appId)); + TAOS_CHECK_RETURN(tEncodeI32(pEncoder, pReq->app.pid)); + TAOS_CHECK_RETURN(tEncodeCStr(pEncoder, pReq->app.name)); + TAOS_CHECK_RETURN(tEncodeI64(pEncoder, pReq->app.startTime)); + TAOS_CHECK_RETURN(tEncodeU64(pEncoder, pReq->app.summary.numOfInsertsReq)); + TAOS_CHECK_RETURN(tEncodeU64(pEncoder, pReq->app.summary.numOfInsertRows)); + TAOS_CHECK_RETURN(tEncodeU64(pEncoder, pReq->app.summary.insertElapsedTime)); + TAOS_CHECK_RETURN(tEncodeU64(pEncoder, pReq->app.summary.insertBytes)); + TAOS_CHECK_RETURN(tEncodeU64(pEncoder, pReq->app.summary.fetchBytes)); + TAOS_CHECK_RETURN(tEncodeU64(pEncoder, pReq->app.summary.queryElapsedTime)); + TAOS_CHECK_RETURN(tEncodeU64(pEncoder, pReq->app.summary.numOfSlowQueries)); + TAOS_CHECK_RETURN(tEncodeU64(pEncoder, pReq->app.summary.totalRequests)); + TAOS_CHECK_RETURN(tEncodeU64(pEncoder, pReq->app.summary.currentRequests)); int32_t queryNum = 0; if (pReq->query) { queryNum = 1; - if (tEncodeI32(pEncoder, queryNum) < 0) return -1; - if (tEncodeU32(pEncoder, pReq->query->connId) < 0) return -1; + TAOS_CHECK_RETURN(tEncodeI32(pEncoder, queryNum)); + TAOS_CHECK_RETURN(tEncodeU32(pEncoder, pReq->query->connId)); int32_t num = taosArrayGetSize(pReq->query->queryDesc); - if (tEncodeI32(pEncoder, num) < 0) return -1; + TAOS_CHECK_RETURN(tEncodeI32(pEncoder, num)); for (int32_t i = 0; i < num; ++i) { SQueryDesc *desc = taosArrayGet(pReq->query->queryDesc, i); - if (tEncodeCStr(pEncoder, desc->sql) < 0) return -1; - if (tEncodeU64(pEncoder, desc->queryId) < 0) return -1; - if (tEncodeI64(pEncoder, desc->useconds) < 0) return -1; - if (tEncodeI64(pEncoder, desc->stime) < 0) return -1; - if (tEncodeI64(pEncoder, desc->reqRid) < 0) return -1; - if (tEncodeI8(pEncoder, desc->stableQuery) < 0) return -1; - if (tEncodeI8(pEncoder, desc->isSubQuery) < 0) return -1; - if (tEncodeCStr(pEncoder, desc->fqdn) < 0) return -1; - if (tEncodeI32(pEncoder, desc->subPlanNum) < 0) return -1; + TAOS_CHECK_RETURN(tEncodeCStr(pEncoder, desc->sql)); + TAOS_CHECK_RETURN(tEncodeU64(pEncoder, desc->queryId)); + TAOS_CHECK_RETURN(tEncodeI64(pEncoder, desc->useconds)); + TAOS_CHECK_RETURN(tEncodeI64(pEncoder, desc->stime)); + TAOS_CHECK_RETURN(tEncodeI64(pEncoder, desc->reqRid)); + TAOS_CHECK_RETURN(tEncodeI8(pEncoder, desc->stableQuery)); + TAOS_CHECK_RETURN(tEncodeI8(pEncoder, desc->isSubQuery)); + TAOS_CHECK_RETURN(tEncodeCStr(pEncoder, desc->fqdn)); + TAOS_CHECK_RETURN(tEncodeI32(pEncoder, desc->subPlanNum)); int32_t snum = desc->subDesc ? taosArrayGetSize(desc->subDesc) : 0; - if (tEncodeI32(pEncoder, snum) < 0) return -1; + TAOS_CHECK_RETURN(tEncodeI32(pEncoder, snum)); for (int32_t m = 0; m < snum; ++m) { SQuerySubDesc *sDesc = taosArrayGet(desc->subDesc, m); - if (tEncodeI64(pEncoder, sDesc->tid) < 0) return -1; - if (tEncodeCStr(pEncoder, sDesc->status) < 0) return -1; + TAOS_CHECK_RETURN(tEncodeI64(pEncoder, sDesc->tid)); + TAOS_CHECK_RETURN(tEncodeCStr(pEncoder, sDesc->status)); } } } else { - if (tEncodeI32(pEncoder, queryNum) < 0) return -1; + TAOS_CHECK_RETURN(tEncodeI32(pEncoder, queryNum)); } } int32_t kvNum = taosHashGetSize(pReq->info); - if (tEncodeI32(pEncoder, kvNum) < 0) return -1; + TAOS_CHECK_RETURN(tEncodeI32(pEncoder, kvNum)); void *pIter = taosHashIterate(pReq->info, NULL); while (pIter != NULL) { SKv *kv = pIter; - if (tEncodeSKv(pEncoder, kv) < 0) return -1; + TAOS_CHECK_RETURN(tEncodeSKv(pEncoder, kv)); pIter = taosHashIterate(pReq->info, pIter); } @@ -322,83 +319,88 @@ static int32_t tSerializeSClientHbReq(SEncoder *pEncoder, const SClientHbReq *pR } static int32_t tDeserializeSClientHbReq(SDecoder *pDecoder, SClientHbReq *pReq) { - if (tDecodeSClientHbKey(pDecoder, &pReq->connKey) < 0) return -1; + TAOS_CHECK_RETURN(tDecodeSClientHbKey(pDecoder, &pReq->connKey)); if (pReq->connKey.connType == CONN_TYPE__QUERY) { - if (tDecodeI64(pDecoder, &pReq->app.appId) < 0) return -1; - if (tDecodeI32(pDecoder, &pReq->app.pid) < 0) return -1; - if (tDecodeCStrTo(pDecoder, pReq->app.name) < 0) return -1; - if (tDecodeI64(pDecoder, &pReq->app.startTime) < 0) return -1; - if (tDecodeU64(pDecoder, &pReq->app.summary.numOfInsertsReq) < 0) return -1; - if (tDecodeU64(pDecoder, &pReq->app.summary.numOfInsertRows) < 0) return -1; - if (tDecodeU64(pDecoder, &pReq->app.summary.insertElapsedTime) < 0) return -1; - if (tDecodeU64(pDecoder, &pReq->app.summary.insertBytes) < 0) return -1; - if (tDecodeU64(pDecoder, &pReq->app.summary.fetchBytes) < 0) return -1; - if (tDecodeU64(pDecoder, &pReq->app.summary.queryElapsedTime) < 0) return -1; - if (tDecodeU64(pDecoder, &pReq->app.summary.numOfSlowQueries) < 0) return -1; - if (tDecodeU64(pDecoder, &pReq->app.summary.totalRequests) < 0) return -1; - if (tDecodeU64(pDecoder, &pReq->app.summary.currentRequests) < 0) return -1; + TAOS_CHECK_RETURN(tDecodeI64(pDecoder, &pReq->app.appId)); + TAOS_CHECK_RETURN(tDecodeI32(pDecoder, &pReq->app.pid)); + TAOS_CHECK_RETURN(tDecodeCStrTo(pDecoder, pReq->app.name)); + TAOS_CHECK_RETURN(tDecodeI64(pDecoder, &pReq->app.startTime)); + TAOS_CHECK_RETURN(tDecodeU64(pDecoder, &pReq->app.summary.numOfInsertsReq)); + TAOS_CHECK_RETURN(tDecodeU64(pDecoder, &pReq->app.summary.numOfInsertRows)); + TAOS_CHECK_RETURN(tDecodeU64(pDecoder, &pReq->app.summary.insertElapsedTime)); + TAOS_CHECK_RETURN(tDecodeU64(pDecoder, &pReq->app.summary.insertBytes)); + TAOS_CHECK_RETURN(tDecodeU64(pDecoder, &pReq->app.summary.fetchBytes)); + TAOS_CHECK_RETURN(tDecodeU64(pDecoder, &pReq->app.summary.queryElapsedTime)); + TAOS_CHECK_RETURN(tDecodeU64(pDecoder, &pReq->app.summary.numOfSlowQueries)); + TAOS_CHECK_RETURN(tDecodeU64(pDecoder, &pReq->app.summary.totalRequests)); + TAOS_CHECK_RETURN(tDecodeU64(pDecoder, &pReq->app.summary.currentRequests)); int32_t queryNum = 0; - if (tDecodeI32(pDecoder, &queryNum) < 0) return -1; + TAOS_CHECK_RETURN(tDecodeI32(pDecoder, &queryNum)); if (queryNum) { pReq->query = taosMemoryCalloc(1, sizeof(*pReq->query)); if (NULL == pReq->query) return -1; - if (tDecodeU32(pDecoder, &pReq->query->connId) < 0) return -1; + TAOS_CHECK_RETURN(tDecodeU32(pDecoder, &pReq->query->connId)); int32_t num = 0; - if (tDecodeI32(pDecoder, &num) < 0) return -1; + TAOS_CHECK_RETURN(tDecodeI32(pDecoder, &num)); if (num > 0) { pReq->query->queryDesc = taosArrayInit(num, sizeof(SQueryDesc)); if (NULL == pReq->query->queryDesc) return -1; for (int32_t i = 0; i < num; ++i) { SQueryDesc desc = {0}; - if (tDecodeCStrTo(pDecoder, desc.sql) < 0) return -1; - if (tDecodeU64(pDecoder, &desc.queryId) < 0) return -1; - if (tDecodeI64(pDecoder, &desc.useconds) < 0) return -1; - if (tDecodeI64(pDecoder, &desc.stime) < 0) return -1; - if (tDecodeI64(pDecoder, &desc.reqRid) < 0) return -1; - if (tDecodeI8(pDecoder, (int8_t *)&desc.stableQuery) < 0) return -1; - if (tDecodeI8(pDecoder, (int8_t *)&desc.isSubQuery) < 0) return -1; - if (tDecodeCStrTo(pDecoder, desc.fqdn) < 0) return -1; - if (tDecodeI32(pDecoder, &desc.subPlanNum) < 0) return -1; + TAOS_CHECK_RETURN(tDecodeCStrTo(pDecoder, desc.sql)); + TAOS_CHECK_RETURN(tDecodeU64(pDecoder, &desc.queryId)); + TAOS_CHECK_RETURN(tDecodeI64(pDecoder, &desc.useconds)); + TAOS_CHECK_RETURN(tDecodeI64(pDecoder, &desc.stime)); + TAOS_CHECK_RETURN(tDecodeI64(pDecoder, &desc.reqRid)); + TAOS_CHECK_RETURN(tDecodeI8(pDecoder, (int8_t *)&desc.stableQuery)); + TAOS_CHECK_RETURN(tDecodeI8(pDecoder, (int8_t *)&desc.isSubQuery)); + TAOS_CHECK_RETURN(tDecodeCStrTo(pDecoder, desc.fqdn)); + TAOS_CHECK_RETURN(tDecodeI32(pDecoder, &desc.subPlanNum)); int32_t snum = 0; - if (tDecodeI32(pDecoder, &snum) < 0) return -1; + TAOS_CHECK_RETURN(tDecodeI32(pDecoder, &snum)); if (snum > 0) { desc.subDesc = taosArrayInit(snum, sizeof(SQuerySubDesc)); if (NULL == desc.subDesc) return -1; for (int32_t m = 0; m < snum; ++m) { SQuerySubDesc sDesc = {0}; - if (tDecodeI64(pDecoder, &sDesc.tid) < 0) return -1; - if (tDecodeCStrTo(pDecoder, sDesc.status) < 0) return -1; - if (!taosArrayPush(desc.subDesc, &sDesc)) return -1; + TAOS_CHECK_RETURN(tDecodeI64(pDecoder, &sDesc.tid)); + TAOS_CHECK_RETURN(tDecodeCStrTo(pDecoder, sDesc.status)); + if (!taosArrayPush(desc.subDesc, &sDesc)) { + return terrno; + } } } ASSERT(desc.subPlanNum == taosArrayGetSize(desc.subDesc)); - if (!taosArrayPush(pReq->query->queryDesc, &desc)) return -1; + if (!taosArrayPush(pReq->query->queryDesc, &desc)) { + return terrno; + } } } } } int32_t kvNum = 0; - if (tDecodeI32(pDecoder, &kvNum) < 0) return -1; + TAOS_CHECK_RETURN(tDecodeI32(pDecoder, &kvNum)); if (pReq->info == NULL) { pReq->info = taosHashInit(kvNum, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), true, HASH_NO_LOCK); } - if (pReq->info == NULL) return -1; + if (pReq->info == NULL) { + return terrno; + } for (int32_t i = 0; i < kvNum; i++) { SKv kv = {0}; - if (tDecodeSKv(pDecoder, &kv) < 0) return -1; + TAOS_CHECK_RETURN(tDecodeSKv(pDecoder, &kv)); int32_t code = taosHashPut(pReq->info, &kv.key, sizeof(kv.key), &kv, sizeof(kv)); if (code) { - terrno = code; - return -1; + return terrno = code; } } @@ -406,75 +408,75 @@ static int32_t tDeserializeSClientHbReq(SDecoder *pDecoder, SClientHbReq *pReq) } static int32_t tSerializeSClientHbRsp(SEncoder *pEncoder, const SClientHbRsp *pRsp) { - if (tEncodeSClientHbKey(pEncoder, &pRsp->connKey) < 0) return -1; - if (tEncodeI32(pEncoder, pRsp->status) < 0) return -1; + TAOS_CHECK_RETURN(tEncodeSClientHbKey(pEncoder, &pRsp->connKey)); + TAOS_CHECK_RETURN(tEncodeI32(pEncoder, pRsp->status)); int32_t queryNum = 0; if (pRsp->query) { queryNum = 1; - if (tEncodeI32(pEncoder, queryNum) < 0) return -1; - if (tEncodeU32(pEncoder, pRsp->query->connId) < 0) return -1; - if (tEncodeU64(pEncoder, pRsp->query->killRid) < 0) return -1; - if (tEncodeI32(pEncoder, pRsp->query->totalDnodes) < 0) return -1; - if (tEncodeI32(pEncoder, pRsp->query->onlineDnodes) < 0) return -1; - if (tEncodeI8(pEncoder, pRsp->query->killConnection) < 0) return -1; - if (tEncodeSEpSet(pEncoder, &pRsp->query->epSet) < 0) return -1; + TAOS_CHECK_RETURN(tEncodeI32(pEncoder, queryNum)); + TAOS_CHECK_RETURN(tEncodeU32(pEncoder, pRsp->query->connId)); + TAOS_CHECK_RETURN(tEncodeU64(pEncoder, pRsp->query->killRid)); + TAOS_CHECK_RETURN(tEncodeI32(pEncoder, pRsp->query->totalDnodes)); + TAOS_CHECK_RETURN(tEncodeI32(pEncoder, pRsp->query->onlineDnodes)); + TAOS_CHECK_RETURN(tEncodeI8(pEncoder, pRsp->query->killConnection)); + TAOS_CHECK_RETURN(tEncodeSEpSet(pEncoder, &pRsp->query->epSet)); int32_t num = taosArrayGetSize(pRsp->query->pQnodeList); - if (tEncodeI32(pEncoder, num) < 0) return -1; + TAOS_CHECK_RETURN(tEncodeI32(pEncoder, num)); for (int32_t i = 0; i < num; ++i) { SQueryNodeLoad *pLoad = taosArrayGet(pRsp->query->pQnodeList, i); - if (tEncodeSQueryNodeLoad(pEncoder, pLoad) < 0) return -1; + TAOS_CHECK_RETURN(tEncodeSQueryNodeLoad(pEncoder, pLoad)); } } else { - if (tEncodeI32(pEncoder, queryNum) < 0) return -1; + TAOS_CHECK_RETURN(tEncodeI32(pEncoder, queryNum)); } int32_t kvNum = taosArrayGetSize(pRsp->info); - if (tEncodeI32(pEncoder, kvNum) < 0) return -1; + TAOS_CHECK_RETURN(tEncodeI32(pEncoder, kvNum)); for (int32_t i = 0; i < kvNum; i++) { SKv *kv = taosArrayGet(pRsp->info, i); - if (tEncodeSKv(pEncoder, kv) < 0) return -1; + TAOS_CHECK_RETURN(tEncodeSKv(pEncoder, kv)); } return 0; } static int32_t tDeserializeSClientHbRsp(SDecoder *pDecoder, SClientHbRsp *pRsp) { - if (tDecodeSClientHbKey(pDecoder, &pRsp->connKey) < 0) return -1; - if (tDecodeI32(pDecoder, &pRsp->status) < 0) return -1; + TAOS_CHECK_RETURN(tDecodeSClientHbKey(pDecoder, &pRsp->connKey)); + TAOS_CHECK_RETURN(tDecodeI32(pDecoder, &pRsp->status)); int32_t queryNum = 0; - if (tDecodeI32(pDecoder, &queryNum) < 0) return -1; + TAOS_CHECK_RETURN(tDecodeI32(pDecoder, &queryNum)); if (queryNum) { pRsp->query = taosMemoryCalloc(1, sizeof(*pRsp->query)); if (NULL == pRsp->query) return -1; - if (tDecodeU32(pDecoder, &pRsp->query->connId) < 0) return -1; - if (tDecodeU64(pDecoder, &pRsp->query->killRid) < 0) return -1; - if (tDecodeI32(pDecoder, &pRsp->query->totalDnodes) < 0) return -1; - if (tDecodeI32(pDecoder, &pRsp->query->onlineDnodes) < 0) return -1; - if (tDecodeI8(pDecoder, &pRsp->query->killConnection) < 0) return -1; - if (tDecodeSEpSet(pDecoder, &pRsp->query->epSet) < 0) return -1; + TAOS_CHECK_RETURN(tDecodeU32(pDecoder, &pRsp->query->connId)); + TAOS_CHECK_RETURN(tDecodeU64(pDecoder, &pRsp->query->killRid)); + TAOS_CHECK_RETURN(tDecodeI32(pDecoder, &pRsp->query->totalDnodes)); + TAOS_CHECK_RETURN(tDecodeI32(pDecoder, &pRsp->query->onlineDnodes)); + TAOS_CHECK_RETURN(tDecodeI8(pDecoder, &pRsp->query->killConnection)); + TAOS_CHECK_RETURN(tDecodeSEpSet(pDecoder, &pRsp->query->epSet)); int32_t pQnodeNum = 0; - if (tDecodeI32(pDecoder, &pQnodeNum) < 0) return -1; + TAOS_CHECK_RETURN(tDecodeI32(pDecoder, &pQnodeNum)); if (pQnodeNum > 0) { pRsp->query->pQnodeList = taosArrayInit(pQnodeNum, sizeof(SQueryNodeLoad)); - if (NULL == pRsp->query->pQnodeList) return -1; + if (NULL == pRsp->query->pQnodeList) return terrno; for (int32_t i = 0; i < pQnodeNum; ++i) { SQueryNodeLoad load = {0}; - if (tDecodeSQueryNodeLoad(pDecoder, &load) < 0) return -1; - if (!taosArrayPush(pRsp->query->pQnodeList, &load)) return -1; + TAOS_CHECK_RETURN(tDecodeSQueryNodeLoad(pDecoder, &load)); + if (!taosArrayPush(pRsp->query->pQnodeList, &load)) return terrno; } } } int32_t kvNum = 0; - if (tDecodeI32(pDecoder, &kvNum) < 0) return -1; + TAOS_CHECK_RETURN(tDecodeI32(pDecoder, &kvNum)); pRsp->info = taosArrayInit(kvNum, sizeof(SKv)); if (pRsp->info == NULL) return -1; for (int32_t i = 0; i < kvNum; i++) { SKv kv = {0}; - if (tDecodeSKv(pDecoder, &kv)) return -1; - if (!taosArrayPush(pRsp->info, &kv)) return -1; + TAOS_CHECK_RETURN(tDecodeSKv(pDecoder, &kv)); + if (!taosArrayPush(pRsp->info, &kv)) return terrno; } return 0; @@ -5308,7 +5310,7 @@ int32_t tSerializeSMTimerMsg(void *buf, int32_t bufLen, SMTimerReq *pReq) { // return 0; // } -int32_t tSerializeDropOrphanTaskMsg(void* buf, int32_t bufLen, SMStreamDropOrphanMsg* pMsg) { +int32_t tSerializeDropOrphanTaskMsg(void *buf, int32_t bufLen, SMStreamDropOrphanMsg *pMsg) { SEncoder encoder = {0}; tEncoderInit(&encoder, buf, bufLen); @@ -5331,7 +5333,7 @@ int32_t tSerializeDropOrphanTaskMsg(void* buf, int32_t bufLen, SMStreamDropOrpha return tlen; } -int32_t tDeserializeDropOrphanTaskMsg(void* buf, int32_t bufLen, SMStreamDropOrphanMsg* pMsg) { +int32_t tDeserializeDropOrphanTaskMsg(void *buf, int32_t bufLen, SMStreamDropOrphanMsg *pMsg) { SDecoder decoder = {0}; tDecoderInit(&decoder, buf, bufLen); diff --git a/source/libs/tdb/src/db/tdbPager.c b/source/libs/tdb/src/db/tdbPager.c index 01dd0ac766..a650847e1e 100644 --- a/source/libs/tdb/src/db/tdbPager.c +++ b/source/libs/tdb/src/db/tdbPager.c @@ -1180,7 +1180,7 @@ int tdbPagerRestoreJournals(SPager *pPager) { if (code) { taosArrayDestroy(pTxnList); (void)tdbCloseDir(&pDir); - tdbError("failed to restore file due to %s. jFileName:%s", strerror(code), jname); + tdbError("failed to restore file due to %s. jFileName:%s", tstrerror(code), jname); return code; } } From 59270dfd0dfe27c2b1f77bafdf8a2133e3b10af7 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 15 Aug 2024 16:56:30 +0800 Subject: [PATCH 32/46] fix(stream): check status before start timer. --- source/libs/stream/src/streamDispatch.c | 30 ++++++++++++++++--------- 1 file changed, 19 insertions(+), 11 deletions(-) diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 5a9a60db1d..7937402ccc 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -762,18 +762,26 @@ int32_t streamDispatchStreamBlock(SStreamTask* pTask) { code = sendDispatchMsg(pTask, pTask->msgInfo.pData); - streamMutexLock(&pTask->msgInfo.lock); - if (pTask->msgInfo.inMonitor == 0) { - int32_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1); - stDebug("s-task:%s start dispatch monitor tmr in %dms, ref:%d, dispatch code:%s", id, DISPATCH_RETRY_INTERVAL_MS, - ref, tstrerror(code)); - streamStartMonitorDispatchData(pTask, DISPATCH_RETRY_INTERVAL_MS); - pTask->msgInfo.inMonitor = 1; - } else { - stDebug("s-task:%s already in dispatch monitor tmr", id); - } + streamMutexLock(&pTask->lock); + bool shouldStop = streamTaskShouldStop(pTask); + streamMutexLock(&pTask->lock); - streamMutexUnlock(&pTask->msgInfo.lock); + if (shouldStop) { + stDebug("s-task:%s in stop/dropping status, not start dispatch monitor tmr", id); + } else { + streamMutexLock(&pTask->msgInfo.lock); + if (pTask->msgInfo.inMonitor == 0) { + int32_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1); + stDebug("s-task:%s start dispatch monitor tmr in %dms, ref:%d, dispatch code:%s", id, DISPATCH_RETRY_INTERVAL_MS, + ref, tstrerror(code)); + streamStartMonitorDispatchData(pTask, DISPATCH_RETRY_INTERVAL_MS); + pTask->msgInfo.inMonitor = 1; + } else { + stDebug("s-task:%s already in dispatch monitor tmr", id); + } + + streamMutexUnlock(&pTask->msgInfo.lock); + } // this block can not be deleted until it has been sent to downstream task successfully. return TSDB_CODE_SUCCESS; From 761ae2ab4b3767806ce8a44ef009044688eb090f Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 15 Aug 2024 16:59:43 +0800 Subject: [PATCH 33/46] refactor: do some internal refactor. --- source/libs/stream/src/streamCheckpoint.c | 22 ---------------------- 1 file changed, 22 deletions(-) diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 7e4d212457..4bf74d8d4f 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -938,28 +938,6 @@ void checkpointTriggerMonitorFn(void* param, void* tmrId) { return; } - if ((pTmrInfo->launchChkptId != pActiveInfo->activeId) || (pActiveInfo->activeId == 0)) { - streamMutexUnlock(&pActiveInfo->lock); - int32_t ref = streamCleanBeforeQuitTmr(pTmrInfo, pTask); - stWarn("s-task:%s vgId:%d checkpoint-trigger retrieve by previous checkpoint procedure, checkpointId:%" PRId64 - ", quit, ref:%d", - id, vgId, pTmrInfo->launchChkptId, ref); - - streamMetaReleaseTask(pTask->pMeta, pTask); - return; - } - - // active checkpoint info is cleared for now - if ((pActiveInfo->activeId == 0) || (pActiveInfo->transId == 0) || (pTask->chkInfo.startTs == 0)) { - streamMutexUnlock(&pActiveInfo->lock); - int32_t ref = streamCleanBeforeQuitTmr(pTmrInfo, pTask); - stWarn("s-task:%s vgId:%d active checkpoint may be cleared, quit from retrieve checkpoint-trigger send tmr, ref:%d", - id, vgId, ref); - - streamMetaReleaseTask(pTask->pMeta, pTask); - return; - } - for (int32_t i = 0; i < taosArrayGetSize(pList); ++i) { SStreamUpstreamEpInfo* pInfo = taosArrayGetP(pList, i); From cf796a45c1bb867330ac2447f49f9adc2ffd7d26 Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao> Date: Thu, 15 Aug 2024 17:03:32 +0800 Subject: [PATCH 34/46] fix issue --- source/libs/executor/src/aggregateoperator.c | 3 +++ source/libs/executor/src/countwindowoperator.c | 3 +++ source/libs/executor/src/eventwindowoperator.c | 3 +++ source/libs/executor/src/executil.c | 6 +++++- source/libs/executor/src/filloperator.c | 3 +++ source/libs/executor/src/groupcacheoperator.c | 3 +++ source/libs/executor/src/groupoperator.c | 9 +++++++++ source/libs/executor/src/projectoperator.c | 15 +++++++++++---- source/libs/executor/src/sortoperator.c | 6 ++++++ .../libs/executor/src/streamcountwindowoperator.c | 3 +++ .../libs/executor/src/streameventwindowoperator.c | 3 +++ source/libs/executor/src/streamfilloperator.c | 3 +++ .../libs/executor/src/streamtimewindowoperator.c | 15 +++++++++++++++ source/libs/executor/src/timesliceoperator.c | 3 +++ source/libs/executor/src/timewindowoperator.c | 15 +++++++++++++++ 15 files changed, 88 insertions(+), 5 deletions(-) diff --git a/source/libs/executor/src/aggregateoperator.c b/source/libs/executor/src/aggregateoperator.c index 093555c9c5..d7b60b2bcd 100644 --- a/source/libs/executor/src/aggregateoperator.c +++ b/source/libs/executor/src/aggregateoperator.c @@ -149,6 +149,9 @@ _error: } if (pOperator != NULL) { pOperator->info = NULL; + if (pOperator->pDownstream == NULL && downstream != NULL) { + destroyOperator(downstream); + } destroyOperator(pOperator); } pTaskInfo->code = code; diff --git a/source/libs/executor/src/countwindowoperator.c b/source/libs/executor/src/countwindowoperator.c index 9019fa0fef..a9858eeb96 100644 --- a/source/libs/executor/src/countwindowoperator.c +++ b/source/libs/executor/src/countwindowoperator.c @@ -344,6 +344,9 @@ _error: if (pOperator != NULL) { pOperator->info = NULL; + if (pOperator->pDownstream == NULL && downstream != NULL) { + destroyOperator(downstream); + } destroyOperator(pOperator); } pTaskInfo->code = code; diff --git a/source/libs/executor/src/eventwindowoperator.c b/source/libs/executor/src/eventwindowoperator.c index d4e5dedd20..b80ea74006 100644 --- a/source/libs/executor/src/eventwindowoperator.c +++ b/source/libs/executor/src/eventwindowoperator.c @@ -147,6 +147,9 @@ _error: if (pOperator != NULL) { pOperator->info = NULL; + if (pOperator->pDownstream == NULL && downstream != NULL) { + destroyOperator(downstream); + } destroyOperator(pOperator); } pTaskInfo->code = code; diff --git a/source/libs/executor/src/executil.c b/source/libs/executor/src/executil.c index b732fccd8e..210c073c6d 100644 --- a/source/libs/executor/src/executil.c +++ b/source/libs/executor/src/executil.c @@ -1814,6 +1814,10 @@ int32_t createExprFromOneNode(SExprInfo* pExp, SNode* pNode, int16_t slotId) { QUERY_CHECK_CODE(code, lino, _end); res->node.resType = (SDataType){.bytes = sizeof(int64_t), .type = TSDB_DATA_TYPE_BIGINT}; code = nodesListAppend(pFuncNode->pParameterList, (SNode*)res); + if (code != TSDB_CODE_SUCCESS) { + nodesDestroyNode((SNode*)res); + res = NULL; + } QUERY_CHECK_CODE(code, lino, _end); } #endif @@ -1945,7 +1949,7 @@ int32_t createExprInfo(SNodeList* pNodeList, SNodeList* pGroupKeys, SExprInfo** SExprInfo* pExp = &pExprs[i]; code = createExprFromTargetNode(pExp, pTargetNode); if (code != TSDB_CODE_SUCCESS) { - taosMemoryFreeClear(pExprs); + destroyExprInfo(pExprs, *numOfExprs); qError("%s failed at line %d since %s", __func__, __LINE__, tstrerror(code)); return code; } diff --git a/source/libs/executor/src/filloperator.c b/source/libs/executor/src/filloperator.c index 0b66834d45..5ece57cad1 100644 --- a/source/libs/executor/src/filloperator.c +++ b/source/libs/executor/src/filloperator.c @@ -579,6 +579,9 @@ _error: pTaskInfo->code = code; if (pOperator != NULL) { pOperator->info = NULL; + if (pOperator->pDownstream == NULL && downstream != NULL) { + destroyOperator(downstream); + } destroyOperator(pOperator); } return code; diff --git a/source/libs/executor/src/groupcacheoperator.c b/source/libs/executor/src/groupcacheoperator.c index 00b8c3b9ae..d5e6061a0f 100644 --- a/source/libs/executor/src/groupcacheoperator.c +++ b/source/libs/executor/src/groupcacheoperator.c @@ -1506,6 +1506,9 @@ _error: if (pOperator != NULL) { pOperator->info = NULL; + if (pOperator->pDownstream == NULL && pDownstream != NULL && (*pDownstream) != NULL) { + destroyOperator(*pDownstream); + } destroyOperator(pOperator); } pTaskInfo->code = code; diff --git a/source/libs/executor/src/groupoperator.c b/source/libs/executor/src/groupoperator.c index 69a9045004..064ce42840 100644 --- a/source/libs/executor/src/groupoperator.c +++ b/source/libs/executor/src/groupoperator.c @@ -618,6 +618,9 @@ _error: if (pOperator) { pOperator->info = NULL; + if (pOperator->pDownstream == NULL && downstream != NULL) { + destroyOperator(downstream); + } destroyOperator(pOperator); } @@ -1248,6 +1251,9 @@ _error: pTaskInfo->code = code; if (pOperator != NULL) { pOperator->info = NULL; + if (pOperator->pDownstream == NULL && downstream != NULL) { + destroyOperator(downstream); + } destroyOperator(pOperator); } TAOS_RETURN(code); @@ -1797,6 +1803,9 @@ _error: if (pInfo != NULL) destroyStreamPartitionOperatorInfo(pInfo); if (pOperator != NULL) { pOperator->info = NULL; + if (pOperator->pDownstream == NULL && downstream != NULL) { + destroyOperator(downstream); + } destroyOperator(pOperator); } qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); diff --git a/source/libs/executor/src/projectoperator.c b/source/libs/executor/src/projectoperator.c index 66a7408b13..8426cb73fe 100644 --- a/source/libs/executor/src/projectoperator.c +++ b/source/libs/executor/src/projectoperator.c @@ -107,10 +107,6 @@ int32_t createProjectOperatorInfo(SOperatorInfo* downstream, SProjectPhysiNode* pOperator->pTaskInfo = pTaskInfo; int32_t lino = 0; - int32_t numOfCols = 0; - SExprInfo* pExprInfo = NULL; - code = createExprInfo(pProjPhyNode->pProjections, NULL, &pExprInfo, &numOfCols); - TSDB_CHECK_CODE(code, lino, _error); SSDataBlock* pResBlock = createDataBlockFromDescNode(pProjPhyNode->node.pOutputDataBlockDesc); TSDB_CHECK_NULL(pResBlock, code, lino, _error, terrno); @@ -148,6 +144,11 @@ int32_t createProjectOperatorInfo(SOperatorInfo* downstream, SProjectPhysiNode* } initResultSizeInfo(&pOperator->resultInfo, numOfRows); + + int32_t numOfCols = 0; + SExprInfo* pExprInfo = NULL; + code = createExprInfo(pProjPhyNode->pProjections, NULL, &pExprInfo, &numOfCols); + TSDB_CHECK_CODE(code, lino, _error); code = initAggSup(&pOperator->exprSupp, &pInfo->aggSup, pExprInfo, numOfCols, keyBufSize, pTaskInfo->id.str, pTaskInfo->streamInfo.pState, &pTaskInfo->storageAPI.functionStore); TSDB_CHECK_CODE(code, lino, _error); @@ -182,6 +183,9 @@ _error: if (pInfo != NULL) destroyProjectOperatorInfo(pInfo); if (pOperator != NULL) { pOperator->info = NULL; + if (pOperator->pDownstream == NULL && downstream != NULL) { + destroyOperator(downstream); + } destroyOperator(pOperator); } pTaskInfo->code = code; @@ -534,6 +538,9 @@ _error: if (pInfo != NULL) destroyIndefinitOperatorInfo(pInfo); if (pOperator != NULL) { pOperator->info = NULL; + if (pOperator->pDownstream == NULL && downstream != NULL) { + destroyOperator(downstream); + } destroyOperator(pOperator); } pTaskInfo->code = code; diff --git a/source/libs/executor/src/sortoperator.c b/source/libs/executor/src/sortoperator.c index 59b4e1cbbb..36f9ac0954 100644 --- a/source/libs/executor/src/sortoperator.c +++ b/source/libs/executor/src/sortoperator.c @@ -166,6 +166,9 @@ _error: if (pOperator != NULL) { pOperator->info = NULL; + if (pOperator->pDownstream == NULL && downstream != NULL) { + destroyOperator(downstream); + } destroyOperator(pOperator); } pTaskInfo->code = code; @@ -841,6 +844,9 @@ _error: } if (pOperator != NULL) { pOperator->info = NULL; + if (pOperator->pDownstream == NULL && downstream != NULL) { + destroyOperator(downstream); + } destroyOperator(pOperator); } return code; diff --git a/source/libs/executor/src/streamcountwindowoperator.c b/source/libs/executor/src/streamcountwindowoperator.c index 62506858fc..fb4b9db05a 100644 --- a/source/libs/executor/src/streamcountwindowoperator.c +++ b/source/libs/executor/src/streamcountwindowoperator.c @@ -928,6 +928,9 @@ _error: if (pOperator != NULL) { pOperator->info = NULL; + if (pOperator->pDownstream == NULL && downstream != NULL) { + destroyOperator(downstream); + } destroyOperator(pOperator); } pTaskInfo->code = code; diff --git a/source/libs/executor/src/streameventwindowoperator.c b/source/libs/executor/src/streameventwindowoperator.c index 93f30ea899..67929678e5 100644 --- a/source/libs/executor/src/streameventwindowoperator.c +++ b/source/libs/executor/src/streameventwindowoperator.c @@ -986,6 +986,9 @@ _error: if (pInfo != NULL) destroyStreamEventOperatorInfo(pInfo); if (pOperator != NULL) { pOperator->info = NULL; + if (pOperator->pDownstream == NULL && downstream != NULL) { + destroyOperator(downstream); + } destroyOperator(pOperator); } pTaskInfo->code = code; diff --git a/source/libs/executor/src/streamfilloperator.c b/source/libs/executor/src/streamfilloperator.c index 39e602ee84..507ae724e0 100644 --- a/source/libs/executor/src/streamfilloperator.c +++ b/source/libs/executor/src/streamfilloperator.c @@ -1463,6 +1463,9 @@ _error: if (pInfo != NULL) destroyStreamFillOperatorInfo(pInfo); if (pOperator != NULL) { pOperator->info = NULL; + if (pOperator->pDownstream == NULL && downstream != NULL) { + destroyOperator(downstream); + } destroyOperator(pOperator); } pTaskInfo->code = code; diff --git a/source/libs/executor/src/streamtimewindowoperator.c b/source/libs/executor/src/streamtimewindowoperator.c index cf3b53bf02..823897eccd 100644 --- a/source/libs/executor/src/streamtimewindowoperator.c +++ b/source/libs/executor/src/streamtimewindowoperator.c @@ -2018,6 +2018,9 @@ _error: if (pInfo != NULL) destroyStreamFinalIntervalOperatorInfo(pInfo); if (pOperator != NULL) { pOperator->info = NULL; + if (pOperator->pDownstream == NULL && downstream != NULL) { + destroyOperator(downstream); + } destroyOperator(pOperator); } pTaskInfo->code = code; @@ -3843,6 +3846,9 @@ _error: if (pOperator != NULL) { pOperator->info = NULL; + if (pOperator->pDownstream == NULL && downstream != NULL) { + destroyOperator(downstream); + } destroyOperator(pOperator); } pTaskInfo->code = code; @@ -4102,6 +4108,9 @@ _error: } if (pOperator != NULL) { pOperator->info = NULL; + if (pOperator->pDownstream == NULL && downstream != NULL) { + destroyOperator(downstream); + } destroyOperator(pOperator); } pTaskInfo->code = code; @@ -4998,6 +5007,9 @@ _error: if (pInfo != NULL) destroyStreamStateOperatorInfo(pInfo); if (pOperator != NULL) { pOperator->info = NULL; + if (pOperator->pDownstream == NULL && downstream != NULL) { + destroyOperator(downstream); + } destroyOperator(pOperator); } pTaskInfo->code = code; @@ -5337,6 +5349,9 @@ _error: if (pInfo != NULL) destroyStreamFinalIntervalOperatorInfo(pInfo); if (pOperator != NULL) { pOperator->info = NULL; + if (pOperator->pDownstream == NULL && downstream != NULL) { + destroyOperator(downstream); + } destroyOperator(pOperator); } pTaskInfo->code = code; diff --git a/source/libs/executor/src/timesliceoperator.c b/source/libs/executor/src/timesliceoperator.c index b14f4f0266..258f886805 100644 --- a/source/libs/executor/src/timesliceoperator.c +++ b/source/libs/executor/src/timesliceoperator.c @@ -1212,6 +1212,9 @@ _error: if (pInfo != NULL) destroyTimeSliceOperatorInfo(pInfo); if (pOperator != NULL) { pOperator->info = NULL; + if (pOperator->pDownstream == NULL && downstream != NULL) { + destroyOperator(downstream); + } destroyOperator(pOperator); } pTaskInfo->code = code; diff --git a/source/libs/executor/src/timewindowoperator.c b/source/libs/executor/src/timewindowoperator.c index fa9dc79cc3..b3f060e213 100644 --- a/source/libs/executor/src/timewindowoperator.c +++ b/source/libs/executor/src/timewindowoperator.c @@ -1421,6 +1421,9 @@ _error: } if (pOperator != NULL) { pOperator->info = NULL; + if (pOperator->pDownstream == NULL && downstream != NULL) { + destroyOperator(downstream); + } destroyOperator(pOperator); } pTaskInfo->code = code; @@ -1700,6 +1703,9 @@ _error: if (pOperator != NULL) { pOperator->info = NULL; + if (pOperator->pDownstream == NULL && downstream != NULL) { + destroyOperator(downstream); + } destroyOperator(pOperator); } pTaskInfo->code = code; @@ -1796,6 +1802,9 @@ _error: if (pInfo != NULL) destroySWindowOperatorInfo(pInfo); if (pOperator != NULL) { pOperator->info = NULL; + if (pOperator->pDownstream == NULL && downstream != NULL) { + destroyOperator(downstream); + } destroyOperator(pOperator); } pTaskInfo->code = code; @@ -2113,6 +2122,9 @@ _error: if (miaInfo != NULL) destroyMAIOperatorInfo(miaInfo); if (pOperator != NULL) { pOperator->info = NULL; + if (pOperator->pDownstream == NULL && downstream != NULL) { + destroyOperator(downstream); + } destroyOperator(pOperator); } pTaskInfo->code = code; @@ -2450,6 +2462,9 @@ _error: if (pOperator != NULL) { pOperator->info = NULL; + if (pOperator->pDownstream == NULL && downstream != NULL) { + destroyOperator(downstream); + } destroyOperator(pOperator); } pTaskInfo->code = code; From 3274194d219d151fb0b75b6bc3158b9bbe267edb Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 15 Aug 2024 17:11:43 +0800 Subject: [PATCH 35/46] refactor: do some internal refactor. --- source/libs/stream/src/streamDispatch.c | 1 + 1 file changed, 1 insertion(+) diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 7937402ccc..86970f80fa 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -762,6 +762,7 @@ int32_t streamDispatchStreamBlock(SStreamTask* pTask) { code = sendDispatchMsg(pTask, pTask->msgInfo.pData); + // todo: secure the timerActive and start timer in after lock pTask->lock streamMutexLock(&pTask->lock); bool shouldStop = streamTaskShouldStop(pTask); streamMutexLock(&pTask->lock); From a82c220645b01760f1738250960cd11a5185629c Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao> Date: Thu, 15 Aug 2024 17:38:29 +0800 Subject: [PATCH 36/46] fix issue --- source/libs/executor/src/groupoperator.c | 12 +++---- source/libs/executor/src/projectoperator.c | 10 +++--- .../executor/src/streamcountwindowoperator.c | 7 ++-- .../executor/src/streameventwindowoperator.c | 6 ++-- source/libs/executor/src/streamfilloperator.c | 6 ++-- .../executor/src/streamtimewindowoperator.c | 32 +++++++++++-------- source/libs/executor/src/tfill.c | 1 + source/libs/executor/src/timewindowoperator.c | 18 +++++------ 8 files changed, 50 insertions(+), 42 deletions(-) diff --git a/source/libs/executor/src/groupoperator.c b/source/libs/executor/src/groupoperator.c index 064ce42840..e5289fa216 100644 --- a/source/libs/executor/src/groupoperator.c +++ b/source/libs/executor/src/groupoperator.c @@ -561,6 +561,10 @@ int32_t createGroupOperatorInfo(SOperatorInfo* downstream, SAggPhysiNode* pAggNo } initBasicInfo(&pInfo->binfo, pResBlock); + pInfo->pGroupCols = NULL; + code = extractColumnInfo(pAggNode->pGroupKeys, &pInfo->pGroupCols); + QUERY_CHECK_CODE(code, lino, _error); + int32_t numOfScalarExpr = 0; SExprInfo* pScalarExprInfo = NULL; if (pAggNode->pExprs != NULL) { @@ -568,10 +572,6 @@ int32_t createGroupOperatorInfo(SOperatorInfo* downstream, SAggPhysiNode* pAggNo QUERY_CHECK_CODE(code, lino, _error); } - pInfo->pGroupCols = NULL; - code = extractColumnInfo(pAggNode->pGroupKeys, &pInfo->pGroupCols); - QUERY_CHECK_CODE(code, lino, _error); - code = initExprSupp(&pInfo->scalarSup, pScalarExprInfo, numOfScalarExpr, &pTaskInfo->storageAPI.functionStore); QUERY_CHECK_CODE(code, lino, _error); @@ -1165,6 +1165,8 @@ int32_t createPartitionOperatorInfo(SOperatorInfo* downstream, SPartitionPhysiNo SExprInfo* pExprInfo = NULL; code = createExprInfo(pPartNode->pTargets, NULL, &pExprInfo, &numOfCols); QUERY_CHECK_CODE(code, lino, _error); + pOperator->exprSupp.numOfExprs = numOfCols; + pOperator->exprSupp.pExprInfo = pExprInfo; pInfo->pGroupCols = makeColumnArrayFromList(pPartNode->pPartitionKeys); @@ -1230,8 +1232,6 @@ int32_t createPartitionOperatorInfo(SOperatorInfo* downstream, SPartitionPhysiNo setOperatorInfo(pOperator, "PartitionOperator", QUERY_NODE_PHYSICAL_PLAN_PARTITION, false, OP_NOT_OPENED, pInfo, pTaskInfo); - pOperator->exprSupp.numOfExprs = numOfCols; - pOperator->exprSupp.pExprInfo = pExprInfo; pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, hashPartition, NULL, destroyPartitionOperatorInfo, optrDefaultBufFn, NULL, optrDefaultGetNextExtFn, NULL); diff --git a/source/libs/executor/src/projectoperator.c b/source/libs/executor/src/projectoperator.c index 8426cb73fe..4d2bdc62f8 100644 --- a/source/libs/executor/src/projectoperator.c +++ b/source/libs/executor/src/projectoperator.c @@ -474,11 +474,6 @@ int32_t createIndefinitOutputOperatorInfo(SOperatorInfo* downstream, SPhysiNode* SIndefRowsFuncPhysiNode* pPhyNode = (SIndefRowsFuncPhysiNode*)pNode; - int32_t numOfExpr = 0; - SExprInfo* pExprInfo = NULL; - code = createExprInfo(pPhyNode->pFuncs, NULL, &pExprInfo, &numOfExpr); - TSDB_CHECK_CODE(code, lino, _error); - if (pPhyNode->pExprs != NULL) { int32_t num = 0; SExprInfo* pSExpr = NULL; @@ -505,6 +500,11 @@ int32_t createIndefinitOutputOperatorInfo(SOperatorInfo* downstream, SPhysiNode* code = blockDataEnsureCapacity(pResBlock, numOfRows); TSDB_CHECK_CODE(code, lino, _error); + int32_t numOfExpr = 0; + SExprInfo* pExprInfo = NULL; + code = createExprInfo(pPhyNode->pFuncs, NULL, &pExprInfo, &numOfExpr); + TSDB_CHECK_CODE(code, lino, _error); + code = initAggSup(pSup, &pInfo->aggSup, pExprInfo, numOfExpr, keyBufSize, pTaskInfo->id.str, pTaskInfo->streamInfo.pState, &pTaskInfo->storageAPI.functionStore); TSDB_CHECK_CODE(code, lino, _error); diff --git a/source/libs/executor/src/streamcountwindowoperator.c b/source/libs/executor/src/streamcountwindowoperator.c index fb4b9db05a..44a383772d 100644 --- a/source/libs/executor/src/streamcountwindowoperator.c +++ b/source/libs/executor/src/streamcountwindowoperator.c @@ -834,12 +834,14 @@ int32_t createStreamCountAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode* } SExprSupp* pExpSup = &pOperator->exprSupp; + SSDataBlock* pResBlock = createDataBlockFromDescNode(pPhyNode->pOutputDataBlockDesc); + QUERY_CHECK_NULL(pResBlock, code, lino, _error, terrno); + pInfo->binfo.pRes = pResBlock; + SExprInfo* pExprInfo = NULL; code = createExprInfo(pCountNode->window.pFuncs, NULL, &pExprInfo, &numOfCols); QUERY_CHECK_CODE(code, lino, _error); - SSDataBlock* pResBlock = createDataBlockFromDescNode(pPhyNode->pOutputDataBlockDesc); - QUERY_CHECK_NULL(pResBlock, code, lino, _error, terrno); code = initBasicInfoEx(&pInfo->binfo, pExpSup, pExprInfo, numOfCols, pResBlock, &pTaskInfo->storageAPI.functionStore); QUERY_CHECK_CODE(code, lino, _error); @@ -863,7 +865,6 @@ int32_t createStreamCountAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode* code = initExecTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &pTaskInfo->window); QUERY_CHECK_CODE(code, lino, _error); - pInfo->binfo.pRes = pResBlock; _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); pInfo->pStDeleted = tSimpleHashInit(64, hashFn); QUERY_CHECK_NULL(pInfo->pStDeleted, code, lino, _error, terrno); diff --git a/source/libs/executor/src/streameventwindowoperator.c b/source/libs/executor/src/streameventwindowoperator.c index 67929678e5..ff1ff579fc 100644 --- a/source/libs/executor/src/streameventwindowoperator.c +++ b/source/libs/executor/src/streameventwindowoperator.c @@ -895,14 +895,16 @@ int32_t createStreamEventAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode* code = initExecTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &pTaskInfo->window); QUERY_CHECK_CODE(code, lino, _error); + SSDataBlock* pResBlock = createDataBlockFromDescNode(pPhyNode->pOutputDataBlockDesc); + QUERY_CHECK_NULL(pResBlock, code, lino, _error, terrno); + pInfo->binfo.pRes = pResBlock; + SExprSupp* pExpSup = &pOperator->exprSupp; int32_t numOfCols = 0; SExprInfo* pExprInfo = NULL; code = createExprInfo(pEventNode->window.pFuncs, NULL, &pExprInfo, &numOfCols); QUERY_CHECK_CODE(code, lino, _error); - SSDataBlock* pResBlock = createDataBlockFromDescNode(pPhyNode->pOutputDataBlockDesc); - QUERY_CHECK_NULL(pResBlock, code, lino, _error, terrno); code = initBasicInfoEx(&pInfo->binfo, pExpSup, pExprInfo, numOfCols, pResBlock, &pTaskInfo->storageAPI.functionStore); QUERY_CHECK_CODE(code, lino, _error); diff --git a/source/libs/executor/src/streamfilloperator.c b/source/libs/executor/src/streamfilloperator.c index 507ae724e0..75b15dbea4 100644 --- a/source/libs/executor/src/streamfilloperator.c +++ b/source/libs/executor/src/streamfilloperator.c @@ -1370,6 +1370,9 @@ int32_t createStreamFillOperatorInfo(SOperatorInfo* downstream, SStreamFillPhysi code = createExprInfo(pPhyFillNode->pFillExprs, NULL, &pFillExprInfo, &numOfFillCols); QUERY_CHECK_CODE(code, lino, _error); + code = initExprSupp(&pOperator->exprSupp, pFillExprInfo, numOfFillCols, &pTaskInfo->storageAPI.functionStore); + QUERY_CHECK_CODE(code, lino, _error); + pInfo->pFillSup = initStreamFillSup(pPhyFillNode, pInterval, pFillExprInfo, numOfFillCols, &pTaskInfo->storageAPI); if (!pInfo->pFillSup) { code = TSDB_CODE_FAILED; @@ -1440,9 +1443,6 @@ int32_t createStreamFillOperatorInfo(SOperatorInfo* downstream, SStreamFillPhysi code = filterInitFromNode((SNode*)pPhyFillNode->node.pConditions, &pOperator->exprSupp.pFilterInfo, 0); QUERY_CHECK_CODE(code, lino, _error); - code = initExprSupp(&pOperator->exprSupp, pFillExprInfo, numOfFillCols, &pTaskInfo->storageAPI.functionStore); - QUERY_CHECK_CODE(code, lino, _error); - pInfo->srcRowIndex = -1; setOperatorInfo(pOperator, "StreamFillOperator", QUERY_NODE_PHYSICAL_PLAN_STREAM_FILL, false, OP_NOT_OPENED, pInfo, pTaskInfo); diff --git a/source/libs/executor/src/streamtimewindowoperator.c b/source/libs/executor/src/streamtimewindowoperator.c index 823897eccd..756a6d71e1 100644 --- a/source/libs/executor/src/streamtimewindowoperator.c +++ b/source/libs/executor/src/streamtimewindowoperator.c @@ -1896,11 +1896,6 @@ int32_t createStreamFinalIntervalOperatorInfo(SOperatorInfo* downstream, SPhysiN QUERY_CHECK_CODE(code, lino, _error); } - int32_t numOfCols = 0; - SExprInfo* pExprInfo = NULL; - code = createExprInfo(pIntervalPhyNode->window.pFuncs, NULL, &pExprInfo, &numOfCols); - QUERY_CHECK_CODE(code, lino, _error); - SSDataBlock* pResBlock = createDataBlockFromDescNode(pPhyNode->pOutputDataBlockDesc); QUERY_CHECK_NULL(pResBlock, code, lino, _error, terrno); initBasicInfo(&pInfo->binfo, pResBlock); @@ -1914,6 +1909,12 @@ int32_t createStreamFinalIntervalOperatorInfo(SOperatorInfo* downstream, SPhysiN qInfo("copy state %p to %p", pTaskInfo->streamInfo.pState, pInfo->pState); pAPI->stateStore.streamStateSetNumber(pInfo->pState, -1, pInfo->primaryTsIndex); + + int32_t numOfCols = 0; + SExprInfo* pExprInfo = NULL; + code = createExprInfo(pIntervalPhyNode->window.pFuncs, NULL, &pExprInfo, &numOfCols); + QUERY_CHECK_CODE(code, lino, _error); + code = initAggSup(&pOperator->exprSupp, &pInfo->aggSup, pExprInfo, numOfCols, keyBufSize, pTaskInfo->id.str, pInfo->pState, &pTaskInfo->storageAPI.functionStore); QUERY_CHECK_CODE(code, lino, _error); @@ -3742,13 +3743,15 @@ int32_t createStreamSessionAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode } } SExprSupp* pExpSup = &pOperator->exprSupp; + + SSDataBlock* pResBlock = createDataBlockFromDescNode(pPhyNode->pOutputDataBlockDesc); + QUERY_CHECK_NULL(pResBlock, code, lino, _error, terrno); + pInfo->binfo.pRes = pResBlock; SExprInfo* pExprInfo = NULL; code = createExprInfo(pSessionNode->window.pFuncs, NULL, &pExprInfo, &numOfCols); QUERY_CHECK_CODE(code, lino, _error); - SSDataBlock* pResBlock = createDataBlockFromDescNode(pPhyNode->pOutputDataBlockDesc); - QUERY_CHECK_NULL(pResBlock, code, lino, _error, terrno); code = initBasicInfoEx(&pInfo->binfo, pExpSup, pExprInfo, numOfCols, pResBlock, &pTaskInfo->storageAPI.functionStore); QUERY_CHECK_CODE(code, lino, _error); @@ -3774,7 +3777,7 @@ int32_t createStreamSessionAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode if (pSessionNode->window.pTsEnd) { pInfo->endTsIndex = ((SColumnNode*)pSessionNode->window.pTsEnd)->slotId; } - pInfo->binfo.pRes = pResBlock; + pInfo->order = TSDB_ORDER_ASC; _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); pInfo->pStDeleted = tSimpleHashInit(64, hashFn); @@ -4924,14 +4927,16 @@ int32_t createStreamStateAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode* code = initExecTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &pTaskInfo->window); QUERY_CHECK_CODE(code, lino, _error); + SSDataBlock* pResBlock = createDataBlockFromDescNode(pPhyNode->pOutputDataBlockDesc); + QUERY_CHECK_NULL(pResBlock, code, lino, _error, terrno); + pInfo->binfo.pRes = pResBlock; + SExprSupp* pExpSup = &pOperator->exprSupp; int32_t numOfCols = 0; SExprInfo* pExprInfo = NULL; code = createExprInfo(pStateNode->window.pFuncs, NULL, &pExprInfo, &numOfCols); QUERY_CHECK_CODE(code, lino, _error); - SSDataBlock* pResBlock = createDataBlockFromDescNode(pPhyNode->pOutputDataBlockDesc); - QUERY_CHECK_NULL(pResBlock, code, lino, _error, terrno); code = initBasicInfoEx(&pInfo->binfo, pExpSup, pExprInfo, numOfCols, pResBlock, &pTaskInfo->storageAPI.functionStore); if (code != TSDB_CODE_SUCCESS) { goto _error; @@ -5218,10 +5223,6 @@ int32_t createStreamIntervalOperatorInfo(SOperatorInfo* downstream, SPhysiNode* SStreamIntervalPhysiNode* pIntervalPhyNode = (SStreamIntervalPhysiNode*)pPhyNode; - SExprInfo* pExprInfo = NULL; - code = createExprInfo(pIntervalPhyNode->window.pFuncs, NULL, &pExprInfo, &numOfCols); - QUERY_CHECK_CODE(code, lino, _error); - SSDataBlock* pResBlock = createDataBlockFromDescNode(pPhyNode->pOutputDataBlockDesc); QUERY_CHECK_NULL(pResBlock, code, lino, _error, terrno); initBasicInfo(&pInfo->binfo, pResBlock); @@ -5265,6 +5266,9 @@ int32_t createStreamIntervalOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pAPI->stateStore.streamStateSetNumber(pInfo->pState, -1, pInfo->primaryTsIndex); size_t keyBufSize = sizeof(int64_t) + sizeof(int64_t) + POINTER_BYTES; + SExprInfo* pExprInfo = NULL; + code = createExprInfo(pIntervalPhyNode->window.pFuncs, NULL, &pExprInfo, &numOfCols); + QUERY_CHECK_CODE(code, lino, _error); code = initAggSup(pSup, &pInfo->aggSup, pExprInfo, numOfCols, keyBufSize, pTaskInfo->id.str, pInfo->pState, &pTaskInfo->storageAPI.functionStore); QUERY_CHECK_CODE(code, lino, _error); diff --git a/source/libs/executor/src/tfill.c b/source/libs/executor/src/tfill.c index a7e2ea3429..59c19a706c 100644 --- a/source/libs/executor/src/tfill.c +++ b/source/libs/executor/src/tfill.c @@ -764,6 +764,7 @@ SFillColInfo* createFillColInfo(SExprInfo* pExpr, int32_t numOfFillExpr, SExprIn SValueNode* pv = (SValueNode*)nodesListGetNode(pValNode->pNodeList, index); QUERY_CHECK_NULL(pv, code, lino, _end, terrno); code = nodesValueNodeToVariant(pv, &pFillCol[i].fillVal); + QUERY_CHECK_CODE(code, lino, _end); } if (TSDB_CODE_SUCCESS != code) { goto _end; diff --git a/source/libs/executor/src/timewindowoperator.c b/source/libs/executor/src/timewindowoperator.c index b3f060e213..6a74c6a093 100644 --- a/source/libs/executor/src/timewindowoperator.c +++ b/source/libs/executor/src/timewindowoperator.c @@ -1746,15 +1746,15 @@ int32_t createSessionAggOperatorInfo(SOperatorInfo* downstream, SSessionWinodwPh size_t keyBufSize = sizeof(int64_t) + sizeof(int64_t) + POINTER_BYTES; initResultSizeInfo(&pOperator->resultInfo, 4096); + SSDataBlock* pResBlock = createDataBlockFromDescNode(pSessionNode->window.node.pOutputDataBlockDesc); + QUERY_CHECK_NULL(pResBlock, code, lino, _error, terrno); + initBasicInfo(&pInfo->binfo, pResBlock); + int32_t numOfCols = 0; SExprInfo* pExprInfo = NULL; code = createExprInfo(pSessionNode->window.pFuncs, NULL, &pExprInfo, &numOfCols); QUERY_CHECK_CODE(code, lino, _error); - SSDataBlock* pResBlock = createDataBlockFromDescNode(pSessionNode->window.node.pOutputDataBlockDesc); - QUERY_CHECK_NULL(pResBlock, code, lino, _error, terrno); - initBasicInfo(&pInfo->binfo, pResBlock); - code = initAggSup(&pOperator->exprSupp, &pInfo->aggSup, pExprInfo, numOfCols, keyBufSize, pTaskInfo->id.str, pTaskInfo->streamInfo.pState, &pTaskInfo->storageAPI.functionStore); QUERY_CHECK_CODE(code, lino, _error); @@ -2392,11 +2392,6 @@ int32_t createMergeIntervalOperatorInfo(SOperatorInfo* downstream, SMergeInterva goto _error; } - int32_t num = 0; - SExprInfo* pExprInfo = NULL; - code = createExprInfo(pIntervalPhyNode->window.pFuncs, NULL, &pExprInfo, &num); - QUERY_CHECK_CODE(code, lino, _error); - SInterval interval = {.interval = pIntervalPhyNode->interval, .sliding = pIntervalPhyNode->sliding, .intervalUnit = pIntervalPhyNode->intervalUnit, @@ -2420,6 +2415,11 @@ int32_t createMergeIntervalOperatorInfo(SOperatorInfo* downstream, SMergeInterva size_t keyBufSize = sizeof(int64_t) + sizeof(int64_t) + POINTER_BYTES; initResultSizeInfo(&pOperator->resultInfo, 4096); + int32_t num = 0; + SExprInfo* pExprInfo = NULL; + code = createExprInfo(pIntervalPhyNode->window.pFuncs, NULL, &pExprInfo, &num); + QUERY_CHECK_CODE(code, lino, _error); + code = initAggSup(pExprSupp, &pIntervalInfo->aggSup, pExprInfo, num, keyBufSize, pTaskInfo->id.str, pTaskInfo->streamInfo.pState, &pTaskInfo->storageAPI.functionStore); if (code != TSDB_CODE_SUCCESS) { From 75660b8f87f024c94f65400d3c2d48133e034e47 Mon Sep 17 00:00:00 2001 From: gccgdb1234 Date: Thu, 15 Aug 2024 17:47:13 +0800 Subject: [PATCH 37/46] docs: refine and correct errors according to comments from dong yan qiong --- docs/zh/07-operation/04-maintenance.md | 2 -- docs/zh/07-operation/18-dual.md | 2 +- docs/zh/14-reference/01-components/01-taosd.md | 4 ++-- docs/zh/14-reference/01-components/02-taosc.md | 2 +- docs/zh/14-reference/01-components/06-taoskeeper.md | 2 -- docs/zh/14-reference/01-components/09-taosdump.md | 2 -- docs/zh/14-reference/01-components/10-taosbenchmark.md | 2 -- docs/zh/14-reference/03-taos-sql/07-tag-index.md | 4 +--- docs/zh/14-reference/03-taos-sql/32-view.md | 2 -- docs/zh/14-reference/05-connector/10-cpp.mdx | 1 + docs/zh/14-reference/05-connector/45-php.mdx | 1 + docs/zh/14-reference/05-connector/50-odbc.mdx | 3 --- docs/zh/14-reference/05-connector/60-rest-api.mdx | 1 + docs/zh/14-reference/09-error-code.md | 4 ++-- docs/zh/27-train-faq/01-faq.md | 2 +- 15 files changed, 11 insertions(+), 23 deletions(-) diff --git a/docs/zh/07-operation/04-maintenance.md b/docs/zh/07-operation/04-maintenance.md index 3c02e4dd39..88122fed69 100644 --- a/docs/zh/07-operation/04-maintenance.md +++ b/docs/zh/07-operation/04-maintenance.md @@ -4,8 +4,6 @@ title: 集群维护 sidebar_label: 集群维护 --- -## 简介 - 本节介绍 TDengine Enterprise 中提供的高阶集群维护手段,能够使 TDengine 集群长期运行得更健壮和高效。 ## 节点管理 diff --git a/docs/zh/07-operation/18-dual.md b/docs/zh/07-operation/18-dual.md index 9de6a75b18..354e715602 100644 --- a/docs/zh/07-operation/18-dual.md +++ b/docs/zh/07-operation/18-dual.md @@ -4,7 +4,7 @@ sidebar_label: 双活系统 toc_max_heading_level: 4 --- -## 简介 +本节介绍 TDengine 双活系统的配置和使用。 1. 部分用户因为部署环境的特殊性只能部署两台服务器,同时希望实现一定的服务高可用和数据高可靠。本文主要描述基于数据复制和客户端 Failover 两项关键技术的 TDengine 双活系统的产品行为,包括双活系统的架构、配置、运维等。TDengine 双活既可以用于前面所述资源受限的环境,也可用于在两套 TDengine 集群(不限资源)之间的灾备场景。双活是 TDengine Enterprise 特有功能,在 3.3.0.0 版本中第一次发布,建议使用最新版本。 diff --git a/docs/zh/14-reference/01-components/01-taosd.md b/docs/zh/14-reference/01-components/01-taosd.md index 994f557a17..3746a16c54 100644 --- a/docs/zh/14-reference/01-components/01-taosd.md +++ b/docs/zh/14-reference/01-components/01-taosd.md @@ -143,7 +143,7 @@ charset 的有效值是 UTF-8。 | 参数名称 | 参数说明 | | :--------------: | :--------------------------------------------------------------------: | -| dataDir | 数据文件目录,所有的数据文件都将写入该目录,缺省值:/var/lib | +| dataDir | 数据文件目录,所有的数据文件都将写入该目录,缺省值:/var/lib/taos | | tempDir | 指定所有系统运行过程中的临时文件生成的目录,缺省值:/tmp | | minimalTmpDirGB | tempDir 所指定的临时文件目录所需要保留的最小空间,单位 GB,缺省值: 1 | | minimalDataDirGB | dataDir 指定的时序数据存储目录所需要保留的最小空间,单位 GB,缺省值: 2 | @@ -168,7 +168,7 @@ charset 的有效值是 UTF-8。 | minimalLogDirGB | 当日志文件夹所在磁盘可用空间大小小于该值时,停止写日志,单位GB,缺省值:1 | | numOfLogLines | 单个日志文件允许的最大行数,缺省值:10,000,000 | | asyncLog | 日志写入模式,0: 同步,1: 异步,缺省值: 1 | -| logKeepDays | 日志文件的最长保存时间 ,单位:天,缺省值:0,意味着无限保存;当设置为大于0 的值时,日志文件会被重命名为 taosdlog.xxx,其中 xxx 为日志文件最后修改的时间戳。 | +| logKeepDays | 日志文件的最长保存时间 ,单位:天,缺省值:0,意味着无限保存,日志文件不会被重命名,也不会有新的日志文件滚动产生,但日志文件的内容有可能会不断滚动,取决于日志文件大小的设置;当设置为大于0 的值时,当日志文件大小达到设置的上限时会被重命名为 taosdlog.xxx,其中 xxx 为日志文件最后修改的时间戳,并滚动产生新的日志文件 | | slowLogThreshold | 慢查询门限值,大于等于门限值认为是慢查询,单位秒,默认值: 3 | | slowLogScope | 定启动记录哪些类型的慢查询,可选值:ALL, QUERY, INSERT, OHTERS, NONE; 默认值:ALL | | debugFlag | 运行日志开关,131(输出错误和警告日志),135(输出错误、警告和调试日志),143(输出错误、警告、调试和跟踪日志); 默认值:131 或 135 (取决于不同模块) | diff --git a/docs/zh/14-reference/01-components/02-taosc.md b/docs/zh/14-reference/01-components/02-taosc.md index 96d108e8c8..d198890bd9 100644 --- a/docs/zh/14-reference/01-components/02-taosc.md +++ b/docs/zh/14-reference/01-components/02-taosc.md @@ -18,7 +18,7 @@ TDengine 客户端驱动提供了应用编程所需要的全部 API,并且在 |queryPolicy | 查询语句的执行策略,1: 只使用 vnode,不使用 qnode; 2: 没有扫描算子的子任务在 qnode 执行,带扫描算子的子任务在 vnode 执行; 3: vnode 只运行扫描算子,其余算子均在 qnode 执行 ;缺省值:1 | |querySmaOptimize | sma index 的优化策略,0: 表示不使用 sma index,永远从原始数据进行查询; 1: 表示使用 sma index,对符合的语句,直接从预计算的结果进行查询;缺省值:0 | |keepColumnName | Last、First、LastRow 函数查询且未指定别名时,自动设置别名为列名(不含函数名),因此 order by 子句如果引用了该列名将自动引用该列对应的函数; 1: 表示自动设置别名为列名(不包含函数名), 0: 表示不自动设置别名; 缺省值: 0 | -|countAlwaysReturnValue | ount/hyperloglog函数在输入数据为空或者NULL的情况下是否返回值; 0:返回空行,1:返回; 缺省值 1; 该参数设置为 1 时,如果查询中含有 INTERVAL 子句或者该查询使用了TSMA时, 且相应的组或窗口内数据为空或者NULL, 对应的组或窗口将不返回查询结果. 注意此参数客户端和服务端值应保持一致. | +|countAlwaysReturnValue | count/hyperloglog函数在输入数据为空或者NULL的情况下是否返回值; 0:返回空行,1:返回; 缺省值 1; 该参数设置为 1 时,如果查询中含有 INTERVAL 子句或者该查询使用了TSMA时, 且相应的组或窗口内数据为空或者NULL, 对应的组或窗口将不返回查询结果. 注意此参数客户端和服务端值应保持一致. | |multiResultFunctionStarReturnTags | 查询超级表时,last(\*)/last_row(\*)/first(\*) 是否返回标签列;查询普通表、子表时,不受该参数影响; 0:不返回标签列,1:返回标签列 ; 缺省值: 0; 该参数设置为 0 时,last(\*)/last_row(\*)/first(\*) 只返回超级表的普通列;为 1 时,返回超级表的普通列和标签列 | |maxTsmaCalcDelay| 查询时客户端可允许的tsma计算延迟, 若tsma的计算延迟大于配置值, 则该TSMA将不会被使用.; 取值范围: 600s - 86400s, 即10分钟-1小时 ; 缺省值:600 秒| |tsmaDataDeleteMark |TSMA计算的历史数据中间结果保存时间, 单位为毫秒; 取值范围:>= 3600000, 即大于等于1h; 缺省值: 86400000, 即1d | diff --git a/docs/zh/14-reference/01-components/06-taoskeeper.md b/docs/zh/14-reference/01-components/06-taoskeeper.md index 2877728077..f40b34ebeb 100644 --- a/docs/zh/14-reference/01-components/06-taoskeeper.md +++ b/docs/zh/14-reference/01-components/06-taoskeeper.md @@ -7,8 +7,6 @@ toc_max_heading_level: 4 import Tabs from "@theme/Tabs"; import TabItem from "@theme/TabItem"; -## 简介 - taosKeeper 是 TDengine 3.0 版本监控指标的导出工具,通过简单的几项配置即可获取 TDengine 的运行状态。taosKeeper 使用 TDengine RESTful 接口,所以不需要安装 TDengine 客户端即可使用。 ## 安装 diff --git a/docs/zh/14-reference/01-components/09-taosdump.md b/docs/zh/14-reference/01-components/09-taosdump.md index fd08da56d8..7afe8721ee 100644 --- a/docs/zh/14-reference/01-components/09-taosdump.md +++ b/docs/zh/14-reference/01-components/09-taosdump.md @@ -4,8 +4,6 @@ sidebar_label: taosdump toc_max_heading_level: 4 --- -## 简介 - taosdump 是一个支持从运行中的 TDengine 集群备份数据并将备份的数据恢复到相同或另一个运行中的 TDengine 集群中的工具应用程序。 taosdump 可以用数据库、超级表或普通表作为逻辑数据单元进行备份,也可以对数据库、超级 diff --git a/docs/zh/14-reference/01-components/10-taosbenchmark.md b/docs/zh/14-reference/01-components/10-taosbenchmark.md index 8540fa1cbb..3f15d6b8e3 100644 --- a/docs/zh/14-reference/01-components/10-taosbenchmark.md +++ b/docs/zh/14-reference/01-components/10-taosbenchmark.md @@ -4,8 +4,6 @@ sidebar_label: taosBenchmark toc_max_heading_level: 4 --- -## 简介 - taosBenchmark (曾用名 taosdemo ) 是一个用于测试 TDengine 产品性能的工具。taosBenchmark 可以测试 TDengine 的插入、查询和订阅等功能的性能,它可以模拟由大量设备产生的大量数据,还可以灵活地控制数据库、超级表、标签列的数量和类型、数据列的数量和类型、子表的数量、每张子表的数据量、插入数据的时间间隔、taosBenchmark 的工作线程数量、是否以及如何插入乱序数据等。为了兼容过往用户的使用习惯,安装包提供 了 taosdemo 作为 taosBenchmark 的软链接。 ## 安装 diff --git a/docs/zh/14-reference/03-taos-sql/07-tag-index.md b/docs/zh/14-reference/03-taos-sql/07-tag-index.md index c016a5b513..383c5b2a1f 100644 --- a/docs/zh/14-reference/03-taos-sql/07-tag-index.md +++ b/docs/zh/14-reference/03-taos-sql/07-tag-index.md @@ -4,9 +4,7 @@ title: 标签索引 description: 使用标签索引提升查询性能 --- -## 简介 - -在 TDengine 3.0.3.0 版本之前(不含),默认在第一列 TAG 上建立索引,但不支持给其它列动态添加索引。从 3.0.3.0 版本开始,可以动态地为其它 TAG 列添加索引。对于第一个 TAG 列上自动建立的索引,其在查询中默认生效,且用户无法对其进行任何干预。适当地使用索引能够有效地提升查询性能。 +本节说明 TDengine 的索引机制。在 TDengine 3.0.3.0 版本之前(不含),默认在第一列 TAG 上建立索引,但不支持给其它列动态添加索引。从 3.0.3.0 版本开始,可以动态地为其它 TAG 列添加索引。对于第一个 TAG 列上自动建立的索引,其在查询中默认生效,且用户无法对其进行任何干预。适当地使用索引能够有效地提升查询性能。 ## 语法 diff --git a/docs/zh/14-reference/03-taos-sql/32-view.md b/docs/zh/14-reference/03-taos-sql/32-view.md index 9b85ed906d..e8b08e05e8 100644 --- a/docs/zh/14-reference/03-taos-sql/32-view.md +++ b/docs/zh/14-reference/03-taos-sql/32-view.md @@ -4,8 +4,6 @@ title: "视图" sidebar_label: "视图" --- -## 简介 - 从 TDengine 3.2.1.0 开始,TDengine 企业版提供视图功能,便于用户简化操作,提升用户间的分享能力。 视图(View)本质上是一个存储在数据库中的查询语句。视图(非物化视图)本身不包含数据,只有在从视图读取数据时才动态执行视图所指定的查询语句。我们在创建视图时指定一个名称,然后可以像使用普通表一样对其进行查询等操作。视图的使用需遵循以下规则: diff --git a/docs/zh/14-reference/05-connector/10-cpp.mdx b/docs/zh/14-reference/05-connector/10-cpp.mdx index 727e33eb7c..be7e44812c 100644 --- a/docs/zh/14-reference/05-connector/10-cpp.mdx +++ b/docs/zh/14-reference/05-connector/10-cpp.mdx @@ -1,6 +1,7 @@ --- sidebar_label: C/C++ title: C/C++ Connector +toc_max_heading_level: 4 --- C/C++ 开发人员可以使用 TDengine 的客户端驱动,即 C/C++连接器 (以下都用 TDengine 客户端驱动表示),开发自己的应用来连接 TDengine 集群完成数据存储、查询以及其他功能。TDengine 客户端驱动的 API 类似于 MySQL 的 C API。应用程序使用时,需要包含 TDengine 头文件 _taos.h_,里面列出了提供的 API 的函数原型;应用程序还要链接到所在平台上对应的动态库。 diff --git a/docs/zh/14-reference/05-connector/45-php.mdx b/docs/zh/14-reference/05-connector/45-php.mdx index 9bc9662a72..0b453218f6 100644 --- a/docs/zh/14-reference/05-connector/45-php.mdx +++ b/docs/zh/14-reference/05-connector/45-php.mdx @@ -1,6 +1,7 @@ --- sidebar_label: PHP title: PHP Connector +toc_max_heading_level: 4 --- `php-tdengine` 是由社区贡献的 PHP 连接器扩展,还特别支持了 Swoole 协程化。 diff --git a/docs/zh/14-reference/05-connector/50-odbc.mdx b/docs/zh/14-reference/05-connector/50-odbc.mdx index 521b08c599..244a3f8d4e 100644 --- a/docs/zh/14-reference/05-connector/50-odbc.mdx +++ b/docs/zh/14-reference/05-connector/50-odbc.mdx @@ -3,9 +3,6 @@ sidebar_label: ODBC title: TDengine ODBC --- - -## 简介 - TDengine ODBC 是为 TDengine 实现的 ODBC 驱动程序,支持 Windows 系统的应用(如 [PowerBI](https://powerbi.microsoft.com/zh-cn/) 等)通过 ODBC 标准接口访问本地、远程和云服务的 TDengine 数据库。 TDengine ODBC 提供基于 WebSocket(推荐)和 原生连接两种方式连接 TDengine 数据库,使用时可以为 TDengine 数据源设置不同的连接方式。访问云服务时必须使用 WebSocket 连接方式。 diff --git a/docs/zh/14-reference/05-connector/60-rest-api.mdx b/docs/zh/14-reference/05-connector/60-rest-api.mdx index 4a3fcbd49e..b6d6ec3b4a 100644 --- a/docs/zh/14-reference/05-connector/60-rest-api.mdx +++ b/docs/zh/14-reference/05-connector/60-rest-api.mdx @@ -1,6 +1,7 @@ --- title: REST API sidebar_label: REST API +toc_max_heading_level: 4 description: 详细介绍 TDengine 提供的 RESTful API. --- diff --git a/docs/zh/14-reference/09-error-code.md b/docs/zh/14-reference/09-error-code.md index bb66ca7f95..fbd347b6af 100644 --- a/docs/zh/14-reference/09-error-code.md +++ b/docs/zh/14-reference/09-error-code.md @@ -122,7 +122,7 @@ description: TDengine 服务端的错误码列表和详细说明 | 0x80000335 | Cluster cfg inconsistent | 配置不一致 | 检查dnode节点与mnode节点配置是否一致。检查方式:1.节点启动时,在日志中输出 2.使用show variables | | 0x8000033B | Cluster id not match | 节点配置数据不一致 | 检查各节点data/dnode/dnodes.json文件中的clusterid | | 0x80000340 | Account already exists | (仅企业版)内部错误 | 上报issue | -| 0x80000342 | Invalid account options | (仅企业版)操作不zh | 确认操作是否正确 | +| 0x80000342 | Invalid account options | (仅企业版)该操作不支持 | 确认操作是否正确 | | 0x80000344 | Invalid account | 账户不存在 | 确认账户是否正确 | | 0x80000350 | User already exists | Create user, 重复创建 | 确认操作是否正确 | | 0x80000351 | Invalid user | 用户不存在 | 确认操作是否正确 | @@ -311,7 +311,7 @@ description: TDengine 服务端的错误码列表和详细说明 | ---------- | ---------------------------- | ----------------------------------------------------------------------------------------------------------- | ------------------------------------------------------------------------------------------ | | 0x80000903 | Sync timeout | 场景1:发生了切主;旧主节点上已经开始协商但尚未达成一致的请求将超时。 场景2:从节点响应超时,导致协商超时。 | 检查集群状态,例如:show vgroups;查看服务端日志,以及服务端节点之间的网络状况。 | | 0x8000090C | Sync leader is unreachable | 场景1:选主过程中 场景2:客户端请求路由到了从节点,且重定向失败 场景3:客户端或服务端网络配置错误 | 检查集群状态、网络配置、应用程序访问状态等。查看服务端日志,以及服务端节点之间的网络状况。 | -| 0x8000090F | Sync new config error | 成员变更新配置错误 | 预留 | +| 0x8000090F | Sync new config error | 成员变更配置错误 | 内部错误,用户无法干预 | | 0x80000911 | Sync not ready to propose | 场景1:恢复未完成 | 检查集群状态,例如:show vgroups。查看服务端日志,以及服务端节点之间的网络状况。 | | 0x80000914 | Sync leader is restoring | 场景1:发生了切主;选主后,日志重演中 | 检查集群状态,例如:show vgroups。查看服务端日志,观察恢复进度。 | | 0x80000915 | Sync invalid snapshot msg | 快照复制消息错误 | 服务端内部错误 | diff --git a/docs/zh/27-train-faq/01-faq.md b/docs/zh/27-train-faq/01-faq.md index 50470e92f1..01cb42213e 100644 --- a/docs/zh/27-train-faq/01-faq.md +++ b/docs/zh/27-train-faq/01-faq.md @@ -265,7 +265,7 @@ TDengine 在写入数据时如果有很严重的乱序写入问题,会严重 ### 26 遇到报错 “DND ERROR Version not compatible,cliver : 3000700swr wer : 3020300” 说明客户端和服务端版本不兼容,这里cliver的版本是3.0.7.0,server版本是 3.2.3.0。目前的兼容策略是前三位一致,client 和 sever才能兼容。 -### 27 修改database的root密码后,启动taos遇到报错 “failed to connect to server, reason: Authen tication failure” +### 27 修改database的root密码后,启动taos遇到报错 “failed to connect to server, reason: Authentication failure” 默认情况,启动taos服务会使用系统默认的用户名(root)和密码尝试连接taosd,在root密码修改后,启用taos连接就需要指明用户名和密码,例如: taos -h xxx.xxx.xxx.xxx -u root -p,然后输入新密码进行连接。 ### 28 修改database的root密码后,Grafana监控插件TDinsight无数据展示 From 54fdea3173b7132ea98b8c77f6a7ac611bce5de5 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 15 Aug 2024 18:01:28 +0800 Subject: [PATCH 38/46] fix(stream): perform node update when mnode leader/follower switches. --- include/common/tmsg.h | 1 + include/libs/stream/tstream.h | 8 +- source/dnode/mnode/impl/inc/mndStream.h | 3 + source/dnode/mnode/impl/src/mndStream.c | 117 ++++++++++++++++--- source/dnode/mnode/impl/src/mndStreamTrans.c | 2 +- source/dnode/mnode/impl/src/mndStreamUtil.c | 3 + source/dnode/mnode/impl/src/mndSync.c | 4 + source/dnode/vnode/src/tqCommon/tqCommon.c | 41 +++---- source/libs/stream/src/streamExec.c | 8 +- source/libs/stream/src/streamMeta.c | 59 ++++++++-- source/libs/stream/src/streamTask.c | 2 +- 11 files changed, 194 insertions(+), 54 deletions(-) diff --git a/include/common/tmsg.h b/include/common/tmsg.h index 70cf9c8b58..cfacb5cfd6 100644 --- a/include/common/tmsg.h +++ b/include/common/tmsg.h @@ -3696,6 +3696,7 @@ typedef struct { SMsgHead head; int64_t streamId; int32_t taskId; + int32_t transId; } SVPauseStreamTaskReq, SVResetStreamTaskReq; typedef struct { diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 9c59e3f3ec..5e7f2bf0a6 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -473,7 +473,9 @@ typedef struct STaskStartInfo { typedef struct STaskUpdateInfo { SHashObj* pTasks; - int32_t transId; + int32_t activeTransId; + int32_t completeTransId; + int64_t completeTs; } STaskUpdateInfo; typedef struct SScanWalInfo { @@ -753,8 +755,8 @@ int32_t streamMetaAddFailedTask(SStreamMeta* pMeta, int64_t streamId, int32_t ta void streamMetaAddFailedTaskSelf(SStreamTask* pTask, int64_t failedTs); void streamMetaAddIntoUpdateTaskList(SStreamMeta* pMeta, SStreamTask* pTask, SStreamTask* pHTask, int32_t transId, int64_t startTs); -void streamMetaClearUpdateTaskList(SStreamMeta* pMeta); -void streamMetaInitUpdateTaskList(SStreamMeta* pMeta, int32_t transId); +void streamMetaClearSetUpdateTaskListComplete(SStreamMeta* pMeta); +bool streamMetaInitUpdateTaskList(SStreamMeta* pMeta, int32_t transId); void streamMetaRLock(SStreamMeta* pMeta); void streamMetaRUnLock(SStreamMeta* pMeta); diff --git a/source/dnode/mnode/impl/inc/mndStream.h b/source/dnode/mnode/impl/inc/mndStream.h index 88b8e98afb..a87a01c5b6 100644 --- a/source/dnode/mnode/impl/inc/mndStream.h +++ b/source/dnode/mnode/impl/inc/mndStream.h @@ -64,6 +64,8 @@ typedef struct SChkptReportInfo { } SChkptReportInfo; typedef struct SStreamExecInfo { + int32_t role; + bool switchFromFollower; bool initTaskList; SArray *pNodeList; int64_t ts; // snapshot ts @@ -153,6 +155,7 @@ int32_t streamTaskIterGetCurrent(SStreamTaskIter *pIter, SStreamTask **pTask); int32_t mndInitExecInfo(); void mndInitStreamExecInfo(SMnode *pMnode, SStreamExecInfo *pExecInfo); void mndInitStreamExecInfoForLeader(SMnode *pMnode); +void mndInitStreamExecInfoUpdateRole(SMnode *pMnode, int32_t role); int32_t removeExpiredNodeEntryAndTaskInBuf(SArray *pNodeSnapshot); void removeStreamTasksInBuf(SStreamObj *pStream, SStreamExecInfo *pExecNode); diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index a01bc92a97..a85b5c733b 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -144,7 +144,7 @@ int32_t mndInitStream(SMnode *pMnode) { code = sdbSetTable(pMnode->pSdb, table); if (code) { - return terrno; + return code; } code = sdbSetTable(pMnode->pSdb, tableSeq); @@ -2024,7 +2024,7 @@ static SVgroupChangeInfo mndFindChangedNodeInfo(SMnode *pMnode, const SArray *pP return info; } -static int32_t mndProcessVgroupChange(SMnode *pMnode, SVgroupChangeInfo *pChangeInfo) { +static int32_t mndProcessVgroupChange(SMnode *pMnode, SVgroupChangeInfo *pChangeInfo, bool includeAllNodes) { SSdb *pSdb = pMnode->pSdb; SStreamObj *pStream = NULL; void *pIter = NULL; @@ -2069,12 +2069,14 @@ static int32_t mndProcessVgroupChange(SMnode *pMnode, SVgroupChangeInfo *pChange } } - void *p = taosHashGet(pChangeInfo->pDBMap, pStream->targetDb, strlen(pStream->targetDb)); - void *p1 = taosHashGet(pChangeInfo->pDBMap, pStream->sourceDb, strlen(pStream->sourceDb)); - if (p == NULL && p1 == NULL) { - mDebug("stream:0x%" PRIx64 " %s not involved nodeUpdate, ignore", pStream->uid, pStream->name); - sdbRelease(pSdb, pStream); - continue; + if (!includeAllNodes) { + void *p1 = taosHashGet(pChangeInfo->pDBMap, pStream->targetDb, strlen(pStream->targetDb)); + void *p2 = taosHashGet(pChangeInfo->pDBMap, pStream->sourceDb, strlen(pStream->sourceDb)); + if (p1 == NULL && p2 == NULL) { + mDebug("stream:0x%" PRIx64 " %s not involved nodeUpdate, ignore", pStream->uid, pStream->name); + sdbRelease(pSdb, pStream); + continue; + } } mDebug("stream:0x%" PRIx64 " %s involved node changed, create update trans, transId:%d", pStream->uid, @@ -2192,11 +2194,36 @@ static int32_t refreshNodeListFromExistedStreams(SMnode *pMnode, SArray *pNodeLi return code; } +static int32_t addAllDbsIntoHashmap(SHashObj *pDBMap, SSdb *pSdb) { + void *pIter = NULL; + int32_t code = 0; + while (1) { + SVgObj *pVgroup = NULL; + pIter = sdbFetch(pSdb, SDB_VGROUP, pIter, (void **)&pVgroup); + if (pIter == NULL) { + break; + } + + code = taosHashPut(pDBMap, pVgroup->dbName, strlen(pVgroup->dbName), NULL, 0); + sdbRelease(pSdb, pVgroup); + + if (code == 0) { + int32_t size = taosHashGetSize(pDBMap); + mDebug("add Db:%s into Dbs list (total:%d) for kill checkpoint trans", pVgroup->dbName, size); + } + } + + return code; +} + // this function runs by only one thread, so it is not multi-thread safe static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { int32_t code = 0; bool allReady = true; SArray *pNodeSnapshot = NULL; + SMnode *pMnode = pMsg->info.node; + int64_t ts = taosGetTimestampSec(); + bool updateAllVgroups = false; int32_t old = atomic_val_compare_exchange_32(&mndNodeCheckSentinel, 0, 1); if (old != 0) { @@ -2204,10 +2231,7 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { return 0; } - mDebug("start to do node change checking"); - int64_t ts = taosGetTimestampSec(); - - SMnode *pMnode = pMsg->info.node; + mDebug("start to do node changing check"); streamMutexLock(&execInfo.lock); int32_t numOfNodes = extractStreamNodeList(pMnode); @@ -2240,10 +2264,20 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { } SVgroupChangeInfo changeInfo = mndFindChangedNodeInfo(pMnode, execInfo.pNodeList, pNodeSnapshot); - if (taosArrayGetSize(changeInfo.pUpdateNodeList) > 0) { + + { + if (execInfo.role == NODE_ROLE_LEADER && execInfo.switchFromFollower) { + mInfo("rollback all stream due to mnode leader/follower switch by using nodeUpdate trans"); + updateAllVgroups = true; + execInfo.switchFromFollower = false; // reset the flag + (void) addAllDbsIntoHashmap(changeInfo.pDBMap, pMnode->pSdb); + } + } + + if (taosArrayGetSize(changeInfo.pUpdateNodeList) > 0 || updateAllVgroups) { // kill current active checkpoint transaction, since the transaction is vnode wide. killAllCheckpointTrans(pMnode, &changeInfo); - code = mndProcessVgroupChange(pMnode, &changeInfo); + code = mndProcessVgroupChange(pMnode, &changeInfo, updateAllVgroups); // keep the new vnode snapshot if success if (code == TSDB_CODE_SUCCESS || code == TSDB_CODE_ACTION_IN_PROGRESS) { @@ -2284,6 +2318,9 @@ static int32_t mndProcessNodeCheck(SRpcMsg *pReq) { int32_t size = sizeof(SMStreamNodeCheckMsg); SMStreamNodeCheckMsg *pMsg = rpcMallocCont(size); + if (pMsg == NULL) { + return terrno; + } SRpcMsg rpcMsg = {.msgType = TDMT_MND_STREAM_NODECHANGE_CHECK, .pCont = pMsg, .contLen = size}; return tmsgPutToQueue(&pMnode->msgCb, WRITE_QUEUE, &rpcMsg); @@ -2459,6 +2496,10 @@ int32_t mndProcessStreamReqCheckpoint(SRpcMsg *pReq) { { SRpcMsg rsp = {.code = 0, .info = pReq->info, .contLen = sizeof(SMStreamReqCheckpointRsp)}; rsp.pCont = rpcMallocCont(rsp.contLen); + if (rsp.pCont == NULL) { + return terrno; + } + SMsgHead *pHead = rsp.pCont; pHead->vgId = htonl(req.nodeId); @@ -2663,11 +2704,13 @@ static int64_t getConsensusId(int64_t streamId, int32_t numOfTasks, int32_t* pEx static void doSendQuickRsp(SRpcHandleInfo *pInfo, int32_t msgSize, int32_t vgId, int32_t code) { SRpcMsg rsp = {.code = code, .info = *pInfo, .contLen = msgSize}; rsp.pCont = rpcMallocCont(rsp.contLen); - SMsgHead *pHead = rsp.pCont; - pHead->vgId = htonl(vgId); + if (rsp.pCont != NULL) { + SMsgHead *pHead = rsp.pCont; + pHead->vgId = htonl(vgId); - tmsgSendRsp(&rsp); - pInfo->handle = NULL; // disable auto rsp + tmsgSendRsp(&rsp); + pInfo->handle = NULL; // disable auto rsp + } } int32_t mndProcessConsensusInTmr(SRpcMsg *pMsg) { @@ -2804,6 +2847,10 @@ static int32_t mndProcessCreateStreamReqFromMNode(SRpcMsg *pReq) { int32_t code = mndProcessCreateStreamReq(pReq); if (code != 0 && code != TSDB_CODE_ACTION_IN_PROGRESS) { pReq->info.rsp = rpcMallocCont(1); + if (pReq->info.rsp == NULL) { + return terrno; + } + pReq->info.rspLen = 1; pReq->info.noResp = false; pReq->code = code; @@ -2815,6 +2862,10 @@ static int32_t mndProcessDropStreamReqFromMNode(SRpcMsg *pReq) { int32_t code = mndProcessDropStreamReq(pReq); if (code != 0 && code != TSDB_CODE_ACTION_IN_PROGRESS) { pReq->info.rsp = rpcMallocCont(1); + if (pReq->info.rsp == NULL) { + return terrno; + } + pReq->info.rspLen = 1; pReq->info.noResp = false; pReq->code = code; @@ -2837,6 +2888,36 @@ void mndInitStreamExecInfoForLeader(SMnode* pMnode) { mndInitStreamExecInfo(pMnode, &execInfo); } +void mndInitStreamExecInfoUpdateRole(SMnode* pMnode, int32_t role) { + execInfo.switchFromFollower = false; + + if (execInfo.role == NODE_ROLE_UNINIT) { + execInfo.role = role; + if (role == NODE_ROLE_LEADER) { + mInfo("init mnode is set to leader"); + } else { + mInfo("init mnode is set to follower"); + } + } else { + if (role == NODE_ROLE_LEADER) { + if (execInfo.role == NODE_ROLE_FOLLOWER) { + execInfo.role = role; + execInfo.switchFromFollower = true; + mInfo("mnode switch to be leader from follower"); + } else { + mInfo("mnode remain to be leader, do nothing"); + } + } else { // follower's + if (execInfo.role == NODE_ROLE_LEADER) { + execInfo.role = role; + mInfo("mnode switch to be follower from leader"); + } else { + mInfo("mnode remain to be follower, do nothing"); + } + } + } +} + void addAllStreamTasksIntoBuf(SMnode *pMnode, SStreamExecInfo *pExecInfo) { SSdb *pSdb = pMnode->pSdb; SStreamObj *pStream = NULL; diff --git a/source/dnode/mnode/impl/src/mndStreamTrans.c b/source/dnode/mnode/impl/src/mndStreamTrans.c index 414cd402ec..494771e65e 100644 --- a/source/dnode/mnode/impl/src/mndStreamTrans.c +++ b/source/dnode/mnode/impl/src/mndStreamTrans.c @@ -334,5 +334,5 @@ void killAllCheckpointTrans(SMnode *pMnode, SVgroupChangeInfo *pChangeInfo) { taosMemoryFree(p); } - mDebug("complete clear checkpoints in Dbs"); + mDebug("complete clear checkpoints in all Dbs"); } diff --git a/source/dnode/mnode/impl/src/mndStreamUtil.c b/source/dnode/mnode/impl/src/mndStreamUtil.c index 383ffe16da..07bba4e1b3 100644 --- a/source/dnode/mnode/impl/src/mndStreamUtil.c +++ b/source/dnode/mnode/impl/src/mndStreamUtil.c @@ -785,6 +785,9 @@ int32_t mndInitExecInfo() { return terrno; } + execInfo.role = NODE_ROLE_UNINIT; + execInfo.switchFromFollower = false; + taosHashSetFreeFp(execInfo.pTransferStateStreams, freeTaskList); taosHashSetFreeFp(execInfo.pChkptStreams, freeTaskList); taosHashSetFreeFp(execInfo.pStreamConsensus, freeTaskList); diff --git a/source/dnode/mnode/impl/src/mndSync.c b/source/dnode/mnode/impl/src/mndSync.c index 1094a17f6b..f5704be371 100644 --- a/source/dnode/mnode/impl/src/mndSync.c +++ b/source/dnode/mnode/impl/src/mndSync.c @@ -360,6 +360,8 @@ static void mndBecomeFollower(const SSyncFSM *pFsm) { (void)tsem_post(&pMgmt->syncSem); } (void)taosThreadMutexUnlock(&pMgmt->lock); + + mndInitStreamExecInfoUpdateRole(pMnode, NODE_ROLE_FOLLOWER); } static void mndBecomeLearner(const SSyncFSM *pFsm) { @@ -382,6 +384,8 @@ static void mndBecomeLearner(const SSyncFSM *pFsm) { static void mndBecomeLeader(const SSyncFSM *pFsm) { mInfo("vgId:1, become leader"); SMnode *pMnode = pFsm->data; + + mndInitStreamExecInfoUpdateRole(pMnode, NODE_ROLE_LEADER); mndInitStreamExecInfoForLeader(pMnode); } diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index dc58bfd8c4..7037eb5199 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -193,28 +193,23 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM SStreamTask* pTask = *ppTask; const char* idstr = pTask->id.idStr; - if (pMeta->updateInfo.transId == -1) { // info needs to be kept till the new trans to update the nodeEp arrived. - streamMetaInitUpdateTaskList(pMeta, req.transId); + if (req.transId <= 0) { + tqError("vgId:%d invalid update nodeEp task, transId:%d, discard", vgId, req.taskId); + rsp.code = TSDB_CODE_SUCCESS; + streamMetaWUnLock(pMeta); + + taosArrayDestroy(req.pNodeList); + return rsp.code; } - if (pMeta->updateInfo.transId != req.transId) { - if (req.transId < pMeta->updateInfo.transId) { - tqError("s-task:%s vgId:%d disorder update nodeEp msg recv, discarded, newest transId:%d, recv:%d", idstr, vgId, - pMeta->updateInfo.transId, req.transId); - rsp.code = TSDB_CODE_SUCCESS; - streamMetaWUnLock(pMeta); + // info needs to be kept till the new trans to update the nodeEp arrived. + bool update = streamMetaInitUpdateTaskList(pMeta, req.transId); + if (!update) { + rsp.code = TSDB_CODE_SUCCESS; + streamMetaWUnLock(pMeta); - taosArrayDestroy(req.pNodeList); - return rsp.code; - } else { - tqInfo("s-task:%s vgId:%d receive new trans to update nodeEp msg from mnode, transId:%d, prev transId:%d", idstr, - vgId, req.transId, pMeta->updateInfo.transId); - // info needs to be kept till the new trans to update the nodeEp arrived. - streamMetaInitUpdateTaskList(pMeta, req.transId); - } - } else { - tqDebug("s-task:%s vgId:%d recv trans to update nodeEp from mnode, transId:%d, recorded update transId:%d", idstr, - vgId, req.transId, pMeta->updateInfo.transId); + taosArrayDestroy(req.pNodeList); + return rsp.code; } // duplicate update epset msg received, discard this redundant message @@ -311,7 +306,7 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM // persist to disk } - streamMetaClearUpdateTaskList(pMeta); + streamMetaClearSetUpdateTaskListComplete(pMeta); if (!restored) { tqDebug("vgId:%d vnode restore not completed, not start all tasks", vgId); @@ -775,8 +770,8 @@ static int32_t restartStreamTasks(SStreamMeta* pMeta, bool isLeader) { streamMetaWUnLock(pMeta); terrno = 0; - tqInfo("vgId:%d tasks are all updated and stopped, restart all tasks, triggered by transId:%d", vgId, - pMeta->updateInfo.transId); + tqInfo("vgId:%d tasks are all updated and stopped, restart all tasks, triggered by transId:%d, ts:%" PRId64, vgId, + pMeta->updateInfo.completeTransId, pMeta->updateInfo.completeTs); while (streamMetaTaskInTimer(pMeta)) { tqDebug("vgId:%d some tasks in timer, wait for 100ms and recheck", pMeta->vgId); @@ -902,7 +897,7 @@ int32_t tqStartTaskCompleteCallback(SStreamMeta* pMeta) { return restartStreamTasks(pMeta, (pMeta->role == NODE_ROLE_LEADER)); } else { if (pStartInfo->restartCount == 0) { - tqDebug("vgId:%d start all tasks completed in callbackFn, restartCount is 0", pMeta->vgId); + tqDebug("vgId:%d start all tasks completed in callbackFn, restartCounter is 0", pMeta->vgId); } else if (allReady) { pStartInfo->restartCount = 0; tqDebug("vgId:%d all tasks are ready, reset restartCounter 0, not restart tasks", vgId); diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index d222004fb7..cd69c9168c 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -631,7 +631,13 @@ void flushStateDataInExecutor(SStreamTask* pTask, SStreamQueueItem* pCheckpointB (void) streamTaskReloadState(pTask); stDebug("s-task:%s transfer state from fill-history task:%s, status:%s completed", id, pHTask->id.idStr, streamTaskGetStatus(pHTask).name); - + // todo execute qExecTask to fetch the reload-generated result, if this is stream is for session window query. + /* + * while(1) { + * qExecTask() + * } + * // put into the output queue. + */ streamMetaReleaseTask(pTask->pMeta, pHTask); } else { stError("s-task:%s related fill-history task:0x%x failed to acquire, transfer state failed", id, diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 5ed9f274a2..8379d904c2 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -431,7 +431,8 @@ int32_t streamMetaOpen(const char* path, void* ahandle, FTaskBuild buildTaskFn, pMeta->expandTaskFn = expandTaskFn; pMeta->stage = stage; pMeta->role = (vgId == SNODE_HANDLE) ? NODE_ROLE_LEADER : NODE_ROLE_UNINIT; - pMeta->updateInfo.transId = -1; + pMeta->updateInfo.activeTransId = -1; + pMeta->updateInfo.completeTransId = -1; pMeta->startInfo.completeFn = fn; pMeta->pTaskDbUnique = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), false, HASH_ENTRY_LOCK); @@ -1759,12 +1760,56 @@ void streamMetaAddIntoUpdateTaskList(SStreamMeta* pMeta, SStreamTask* pTask, SSt } } -void streamMetaClearUpdateTaskList(SStreamMeta* pMeta) { - taosHashClear(pMeta->updateInfo.pTasks); - pMeta->updateInfo.transId = -1; +void streamMetaClearSetUpdateTaskListComplete(SStreamMeta* pMeta) { + STaskUpdateInfo* pInfo = &pMeta->updateInfo; + + taosHashClear(pInfo->pTasks); + + int32_t prev = pInfo->completeTransId; + pInfo->completeTransId = pInfo->activeTransId; + pInfo->activeTransId = -1; + pInfo->completeTs = taosGetTimestampMs(); + + stDebug("vgId:%d set the nodeEp update complete, ts:%" PRId64 ", complete transId:%d->%d, reset active transId", + pMeta->vgId, pInfo->completeTs, prev, pInfo->completeTransId); } -void streamMetaInitUpdateTaskList(SStreamMeta* pMeta, int32_t transId) { - taosHashClear(pMeta->updateInfo.pTasks); - pMeta->updateInfo.transId = transId; +bool streamMetaInitUpdateTaskList(SStreamMeta* pMeta, int32_t transId) { + STaskUpdateInfo* pInfo = &pMeta->updateInfo; + + if (transId > pInfo->completeTransId) { + if (pInfo->activeTransId == -1) { + taosHashClear(pInfo->pTasks); + pInfo->activeTransId = transId; + + stInfo("vgId:%d set the active epset update transId:%d, prev complete transId:%d", pMeta->vgId, transId, + pInfo->completeTransId); + return true; + } else { + if (pInfo->activeTransId == transId) { + // do nothing + return true; + } else if (transId < pInfo->activeTransId) { + stError("vgId:%d invalid(out of order)epset update transId:%d, active transId:%d, complete transId:%d, discard", + pMeta->vgId, transId, pInfo->activeTransId, pInfo->completeTransId); + return false; + } else { // transId > pInfo->activeTransId + taosHashClear(pInfo->pTasks); + int32_t prev = pInfo->activeTransId; + pInfo->activeTransId = transId; + + stInfo("vgId:%d active epset update transId updated from:%d to %d, prev complete transId:%d", pMeta->vgId, + transId, prev, pInfo->completeTransId); + return true; + } + } + } else if (transId == pInfo->completeTransId) { + stError("vgId:%d already handled epset update transId:%d, completeTs:%" PRId64 " ignore", pMeta->vgId, transId, + pInfo->completeTs); + return false; + } else { // pInfo->completeTransId > transId + stError("vgId:%d disorder update nodeEp msg recv, prev completed epset update transId:%d, recv:%d, discard", + pMeta->vgId, pInfo->activeTransId, transId); + return false; + } } diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index c0b2b16d30..f190673430 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -637,7 +637,7 @@ bool streamTaskUpdateEpsetInfo(SStreamTask* pTask, SArray* pNodeList) { numOfNodes, p->updateCount, prevTs); bool updated = false; - for (int32_t i = 0; i < taosArrayGetSize(pNodeList); ++i) { + for (int32_t i = 0; i < numOfNodes; ++i) { SNodeUpdateInfo* pInfo = taosArrayGet(pNodeList, i); if (pInfo == NULL) { continue; From 5322b60a31086f37da347ce101e6243e8d5e4776 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 15 Aug 2024 18:08:16 +0800 Subject: [PATCH 39/46] refactor: remove unused attributes in msg. --- include/common/tmsg.h | 1 - 1 file changed, 1 deletion(-) diff --git a/include/common/tmsg.h b/include/common/tmsg.h index cfacb5cfd6..70cf9c8b58 100644 --- a/include/common/tmsg.h +++ b/include/common/tmsg.h @@ -3696,7 +3696,6 @@ typedef struct { SMsgHead head; int64_t streamId; int32_t taskId; - int32_t transId; } SVPauseStreamTaskReq, SVResetStreamTaskReq; typedef struct { From c94cd245931f713e3e980eb23f34b5e3013355ac Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 15 Aug 2024 18:29:36 +0800 Subject: [PATCH 40/46] refactor: do some internal refactor. --- source/libs/stream/src/streamCheckpoint.c | 14 +++++++++----- source/libs/stream/src/streamMeta.c | 20 ++++++++++++-------- 2 files changed, 21 insertions(+), 13 deletions(-) diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 4bf74d8d4f..9be8f5ffaa 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -561,12 +561,14 @@ int32_t streamTaskUpdateTaskCheckpointInfo(SStreamTask* pTask, bool restored, SV stDebug("s-task:%s vgId:%d related fill-history task:0x%x dropped in update checkpointInfo, remain tasks:%d", id, vgId, pReq->taskId, numOfTasks); } + streamMetaWLock(pMeta); - if (streamMetaCommit(pMeta) < 0) { - // persist to disk + if (pReq->dropRelHTask) { + code = streamMetaCommit(pMeta); } } + // always return true return TSDB_CODE_SUCCESS; } @@ -594,13 +596,15 @@ int32_t streamTaskUpdateTaskCheckpointInfo(SStreamTask* pTask, bool restored, SV ASSERT(pInfo->checkpointId <= pReq->checkpointId && pInfo->checkpointVer <= pReq->checkpointVer && pInfo->processedVer <= pReq->checkpointVer); - // update only it is in checkpoint status. - if (pStatus.state == TASK_STATUS__CK) { + // update only it is in checkpoint status, or during restore procedure. + if (pStatus.state == TASK_STATUS__CK || (!restored)) { pInfo->checkpointId = pReq->checkpointId; pInfo->checkpointVer = pReq->checkpointVer; pInfo->checkpointTime = pReq->checkpointTs; - code = streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_CHECKPOINT_DONE); + if (restored) { + code = streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_CHECKPOINT_DONE); + } } streamTaskClearCheckInfo(pTask, true); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 8379d904c2..7c6461b1c8 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -891,24 +891,28 @@ int32_t streamMetaBegin(SStreamMeta* pMeta) { } int32_t streamMetaCommit(SStreamMeta* pMeta) { - if (tdbCommit(pMeta->db, pMeta->txn) < 0) { + int32_t code = 0; + code = tdbCommit(pMeta->db, pMeta->txn); + if (code != 0) { stError("vgId:%d failed to commit stream meta", pMeta->vgId); - return -1; + return code; } - if (tdbPostCommit(pMeta->db, pMeta->txn) < 0) { + code = tdbPostCommit(pMeta->db, pMeta->txn); + if (code != 0) { stError("vgId:%d failed to do post-commit stream meta", pMeta->vgId); - return -1; + return code; } - if (tdbBegin(pMeta->db, &pMeta->txn, tdbDefaultMalloc, tdbDefaultFree, NULL, - TDB_TXN_WRITE | TDB_TXN_READ_UNCOMMITTED) < 0) { + code = tdbBegin(pMeta->db, &pMeta->txn, tdbDefaultMalloc, tdbDefaultFree, NULL, + TDB_TXN_WRITE | TDB_TXN_READ_UNCOMMITTED); + if (code != 0) { stError("vgId:%d failed to begin trans", pMeta->vgId); - return -1; + return code; } stDebug("vgId:%d stream meta file commit completed", pMeta->vgId); - return 0; + return code; } int64_t streamMetaGetLatestCheckpointId(SStreamMeta* pMeta) { From 6adc0543e86555517f36006f886ea13826a56baf Mon Sep 17 00:00:00 2001 From: kailixu Date: Thu, 15 Aug 2024 19:19:48 +0800 Subject: [PATCH 41/46] enh: support config randErrorChance dynamically --- source/common/src/tglobal.c | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index a013c98b73..6cd99d4443 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -592,7 +592,7 @@ static int32_t taosAddClientCfg(SConfig *pCfg) { CFG_SCOPE_CLIENT, CFG_DYN_NONE)); TAOS_CHECK_RETURN( cfgAddInt32(pCfg, "metaCacheMaxSize", tsMetaCacheMaxSize, -1, INT32_MAX, CFG_SCOPE_CLIENT, CFG_DYN_CLIENT)); - TAOS_CHECK_RETURN(cfgAddInt32(pCfg, "randErrorChance", tsRandErrChance, 0, 10000, CFG_SCOPE_BOTH, CFG_DYN_NONE)); + TAOS_CHECK_RETURN(cfgAddInt32(pCfg, "randErrorChance", tsRandErrChance, 0, 10000, CFG_SCOPE_BOTH, CFG_DYN_BOTH)); TAOS_CHECK_RETURN(cfgAddInt64(pCfg, "randErrorDivisor", tsRandErrDivisor, 1, INT64_MAX, CFG_SCOPE_BOTH, CFG_DYN_BOTH)); TAOS_CHECK_RETURN(cfgAddInt64(pCfg, "randErrorScope", tsRandErrScope, 0, INT64_MAX, CFG_SCOPE_BOTH, CFG_DYN_BOTH)); @@ -1930,6 +1930,7 @@ static int32_t taosCfgDynamicOptionsForServer(SConfig *pCfg, const char *name) { {"mndSdbWriteDelta", &tsMndSdbWriteDelta}, {"minDiskFreeSize", &tsMinDiskFreeSize}, + {"randErrorChance", &tsRandErrChance}, {"randErrorDivisor", &tsRandErrDivisor}, {"randErrorScope", &tsRandErrScope}, @@ -2210,6 +2211,7 @@ static int32_t taosCfgDynamicOptionsForClient(SConfig *pCfg, const char *name) { {"queryPlannerTrace", &tsQueryPlannerTrace}, {"queryNodeChunkSize", &tsQueryNodeChunkSize}, {"queryUseNodeAllocator", &tsQueryUseNodeAllocator}, + {"randErrorChance", &tsRandErrChance}, {"randErrorDivisor", &tsRandErrDivisor}, {"randErrorScope", &tsRandErrScope}, {"smlDot2Underline", &tsSmlDot2Underline}, From c1c59ce82b97c587b2472c18f096961300f21579 Mon Sep 17 00:00:00 2001 From: sima Date: Thu, 15 Aug 2024 17:31:18 +0800 Subject: [PATCH 42/46] fix:[TD-31479] Fix wrong ans when parameter is NULL in substring. --- source/libs/scalar/src/sclfunc.c | 96 +++++++++++++++++++++----------- 1 file changed, 64 insertions(+), 32 deletions(-) diff --git a/source/libs/scalar/src/sclfunc.c b/source/libs/scalar/src/sclfunc.c index 846837fd99..aa3f1dfd78 100644 --- a/source/libs/scalar/src/sclfunc.c +++ b/source/libs/scalar/src/sclfunc.c @@ -1173,49 +1173,79 @@ static int32_t findPosBytes(char *orgStr, char *delimStr, int32_t orgLen, int32_ int32_t substrFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOutput) { int32_t code = TSDB_CODE_SUCCESS; - int32_t subPos = 0; - GET_TYPED_DATA(subPos, int32_t, GET_PARAM_TYPE(&pInput[1]), pInput[1].columnData->pData); - - int32_t subLen = INT16_MAX; - if (inputNum == 3) { - GET_TYPED_DATA(subLen, int32_t, GET_PARAM_TYPE(&pInput[2]), pInput[2].columnData->pData); - } - - SColumnInfoData *pInputData = pInput->columnData; + SColumnInfoData *pInputData[3]; SColumnInfoData *pOutputData = pOutput->columnData; - int32_t outputLen = pInputData->info.bytes; + for (int32_t i = 0; i < inputNum; ++i) { + pInputData[i] = pInput[i].columnData; + } + + int32_t outputLen = pInputData[0]->info.bytes; char *outputBuf = taosMemoryMalloc(outputLen); if (outputBuf == NULL) { qError("substr function memory allocation failure. size: %d", outputLen); return TSDB_CODE_OUT_OF_MEMORY; } - for (int32_t i = 0; i < pInput->numOfRows; ++i) { - if (colDataIsNull_s(pInputData, i)) { + int32_t numOfRows = 0; + for (int32_t i = 0; i < inputNum; ++i) { + numOfRows = TMAX(pInput[i].numOfRows, numOfRows); + } + + bool hasNullType = (IS_NULL_TYPE(GET_PARAM_TYPE(&pInput[0])) || IS_NULL_TYPE(GET_PARAM_TYPE(&pInput[1])) || + (inputNum == 3 && IS_NULL_TYPE(GET_PARAM_TYPE(&pInput[2])))); + + if (hasNullType || + (pInput[0].numOfRows == 1 && colDataIsNull_s(pInputData[0], 0)) || + (pInput[1].numOfRows == 1 && colDataIsNull_s(pInputData[1], 0)) || + (inputNum == 3 && (pInput[2].numOfRows == 1 && colDataIsNull_s(pInputData[2], 0)))) { + colDataSetNNULL(pOutputData, 0, numOfRows); + pOutput->numOfRows = numOfRows; + goto _return; + } + + int32_t colIdx[3]; + for (int32_t i = 0; i < numOfRows; ++i) { + colIdx[0] = (pInput[0].numOfRows == 1) ? 0 : i; + colIdx[1] = (pInput[1].numOfRows == 1) ? 0 : i; + if (inputNum == 3) { + colIdx[2] = (pInput[2].numOfRows == 1) ? 0 : i; + } + + if (colDataIsNull_s(pInputData[0], colIdx[0]) || colDataIsNull_s(pInputData[1], colIdx[1]) || + (inputNum == 3 && colDataIsNull_s(pInputData[2], colIdx[2]))) { colDataSetNULL(pOutputData, i); continue; } + + int32_t subPos = 0; + int32_t subLen = INT16_MAX; + GET_TYPED_DATA(subPos, int32_t, GET_PARAM_TYPE(&pInput[1]), colDataGetData(pInputData[1], colIdx[1])); + if (inputNum == 3) { + GET_TYPED_DATA(subLen, int32_t, GET_PARAM_TYPE(&pInput[2]), colDataGetData(pInputData[2], colIdx[2])); + } + if (subPos == 0 || subLen < 1) { varDataSetLen(outputBuf, 0); SCL_ERR_JRET(colDataSetVal(pOutputData, i, outputBuf, false)); continue; } - char *input = colDataGetData(pInput[0].columnData, i); + + char *input = colDataGetData(pInputData[0], colIdx[0]); int32_t len = varDataLen(input); int32_t startPosBytes; int32_t endPosBytes = len; if (subPos > 0) { - startPosBytes = (GET_PARAM_TYPE(pInput) == TSDB_DATA_TYPE_VARCHAR) ? findPosBytes(varDataVal(input), NULL, varDataLen(input), -1, subPos) : (subPos - 1) * TSDB_NCHAR_SIZE; + startPosBytes = (GET_PARAM_TYPE(&pInput[0]) == TSDB_DATA_TYPE_VARCHAR) ? findPosBytes(varDataVal(input), NULL, varDataLen(input), -1, subPos) : (subPos - 1) * TSDB_NCHAR_SIZE; startPosBytes = TMIN(startPosBytes, len); } else { startPosBytes = - (GET_PARAM_TYPE(pInput) == TSDB_DATA_TYPE_VARCHAR) ? findPosBytes(varDataVal(input), NULL, varDataLen(input), -1, subPos) : len + subPos * TSDB_NCHAR_SIZE; + (GET_PARAM_TYPE(&pInput[0]) == TSDB_DATA_TYPE_VARCHAR) ? findPosBytes(varDataVal(input), NULL, varDataLen(input), -1, subPos) : len + subPos * TSDB_NCHAR_SIZE; startPosBytes = TMAX(startPosBytes, 0); } if (inputNum == 3) { endPosBytes = - (GET_PARAM_TYPE(pInput) == TSDB_DATA_TYPE_VARCHAR) + (GET_PARAM_TYPE(&pInput[0]) == TSDB_DATA_TYPE_VARCHAR) ? startPosBytes + findPosBytes(varDataVal(input) + startPosBytes, NULL, varDataLen(input) - startPosBytes, -1, subLen + 1) : startPosBytes + subLen * TSDB_NCHAR_SIZE; endPosBytes = TMIN(endPosBytes, len); @@ -1230,10 +1260,10 @@ int32_t substrFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOu varDataSetLen(output, 0); } - SCL_ERR_JRET(colDataSetVal(pOutputData, i, output, false)); + SCL_ERR_JRET(colDataSetVal(pOutputData, i, outputBuf, false)); } - pOutput->numOfRows = pInput->numOfRows; + pOutput->numOfRows = numOfRows; _return: taosMemoryFree(outputBuf); @@ -1510,13 +1540,13 @@ int32_t replaceFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pO bool needFreeTo = false; if (GET_PARAM_TYPE(&pInput[1]) != GET_PARAM_TYPE(&pInput[0])) { - SCL_ERR_RET(convBetweenNcharAndVarchar(varDataVal(colDataGetData(pInputData[1], colIdx2)), &fromStr, + SCL_ERR_JRET(convBetweenNcharAndVarchar(varDataVal(colDataGetData(pInputData[1], colIdx2)), &fromStr, varDataLen(colDataGetData(pInputData[1], colIdx2)), &fromLen, GET_PARAM_TYPE(&pInput[0]))); needFreeFrom = true; } if (GET_PARAM_TYPE(&pInput[2]) != GET_PARAM_TYPE(&pInput[0])) { - SCL_ERR_RET(convBetweenNcharAndVarchar(varDataVal(colDataGetData(pInputData[2], colIdx3)), &toStr, + SCL_ERR_JRET(convBetweenNcharAndVarchar(varDataVal(colDataGetData(pInputData[2], colIdx3)), &toStr, varDataLen(colDataGetData(pInputData[2], colIdx3)), &toLen, GET_PARAM_TYPE(&pInput[0]))); needFreeTo = true; @@ -1544,9 +1574,11 @@ int32_t replaceFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pO taosMemoryFree(fromStr); } varDataSetLen(outputBuf, totalLen); - SCL_ERR_RET(colDataSetVal(pOutputData, i, outputBuf, false)); + SCL_ERR_JRET(colDataSetVal(pOutputData, i, outputBuf, false)); } pOutput->numOfRows = numOfRows; +_return: + taosMemoryFree(outputBuf); return code; } @@ -1561,10 +1593,16 @@ int32_t substrIdxFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam * pInputData[1] = pInput[1].columnData; pInputData[2] = pInput[2].columnData; + for (int32_t i = 0; i < inputNum; ++i) { + if (pInput[i].numOfRows > numOfRows) { + numOfRows = pInput[i].numOfRows; + } + } + outputLen = pInputData[0]->info.bytes; if (GET_PARAM_TYPE(&pInput[0]) == TSDB_DATA_TYPE_NULL || GET_PARAM_TYPE(&pInput[1]) == TSDB_DATA_TYPE_NULL || GET_PARAM_TYPE(&pInput[2]) == TSDB_DATA_TYPE_NULL) { - colDataSetNNULL(pOutputData, 0, pInput[0].numOfRows); - pOutput->numOfRows = pInput[0].numOfRows; + colDataSetNNULL(pOutputData, 0, numOfRows); + pOutput->numOfRows = numOfRows; return TSDB_CODE_SUCCESS; } char *outputBuf = taosMemoryCalloc(outputLen + VARSTR_HEADER_SIZE, 1); @@ -1572,12 +1610,6 @@ int32_t substrIdxFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam * SCL_ERR_RET(TSDB_CODE_OUT_OF_MEMORY); } - for (int32_t i = 0; i < inputNum; ++i) { - if (pInput[i].numOfRows > numOfRows) { - numOfRows = pInput[i].numOfRows; - } - } - for (int32_t k = 0; k < numOfRows; ++k) { bool hasNull = false; for (int32_t i = 0; i < inputNum; ++i) { @@ -1600,9 +1632,9 @@ int32_t substrIdxFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam * int32_t startPosBytes; int32_t endPosBytes; if (GET_PARAM_TYPE(&pInput[0]) != GET_PARAM_TYPE(&pInput[1])) { - SCL_ERR_RET(convBetweenNcharAndVarchar(varDataVal(colDataGetData(pInputData[1], colIdx2)), &delimStr, - varDataLen(colDataGetData(pInputData[1], colIdx2)), &delimLen, - GET_PARAM_TYPE(&pInput[0]))); + SCL_ERR_JRET(convBetweenNcharAndVarchar(varDataVal(colDataGetData(pInputData[1], colIdx2)), &delimStr, + varDataLen(colDataGetData(pInputData[1], colIdx2)), &delimLen, + GET_PARAM_TYPE(&pInput[0]))); needFreeDelim = true; } From ebe5f183cbcd7a76969f75314a77c9f4e9ff39e5 Mon Sep 17 00:00:00 2001 From: sima Date: Thu, 15 Aug 2024 11:27:59 +0800 Subject: [PATCH 43/46] fix:[TD-31468] Fix char function wrong result. --- source/libs/scalar/src/sclfunc.c | 20 +++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/source/libs/scalar/src/sclfunc.c b/source/libs/scalar/src/sclfunc.c index 846837fd99..836dd89f0d 100644 --- a/source/libs/scalar/src/sclfunc.c +++ b/source/libs/scalar/src/sclfunc.c @@ -1295,30 +1295,40 @@ int32_t charFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOutp int32_t outputLen = inputNum * 4 + 2; char *outputBuf = taosMemoryCalloc(outputLen, 1); if (outputBuf == NULL) { - SCL_ERR_RET(TSDB_CODE_OUT_OF_MEMORY); + SCL_ERR_RET(terrno); } - for (int32_t i = 0; i < pInput[0].numOfRows; ++i) { + int32_t numOfRows = 0; + for (int32_t i = 0; i < inputNum; ++i) { + numOfRows = TMAX(numOfRows, pInput[i].numOfRows); + } + for (int32_t i = 0; i < numOfRows; ++i) { char *output = varDataVal(outputBuf); for (int32_t j = 0; j < inputNum; ++j) { + int32_t colIdx = (pInput[j].numOfRows == 1) ? 0 : i; int32_t num; if (colDataIsNull_s(pInput[j].columnData, i)) { continue; } else if (IS_NUMERIC_TYPE(GET_PARAM_TYPE(&pInput[j]))) { - GET_TYPED_DATA(num, int32_t, GET_PARAM_TYPE(&pInput[j]), pInput[j].columnData->pData); + GET_TYPED_DATA(num, int32_t, GET_PARAM_TYPE(&pInput[j]), colDataGetData(pInput[j].columnData, colIdx)); getAsciiChar(num, &output); } else if (TSDB_DATA_TYPE_BINARY == GET_PARAM_TYPE(&pInput[j])) { - num = taosStr2Int32(varDataVal(pInput[j].columnData->pData), NULL, 10); + num = taosStr2Int32(varDataVal(colDataGetData(pInput[j].columnData, colIdx)), NULL, 10); getAsciiChar(num, &output); } else if (TSDB_DATA_TYPE_NCHAR == GET_PARAM_TYPE(&pInput[j])) { char *convBuf = taosMemoryMalloc(GET_PARAM_BYTES(&pInput[j])); - int32_t len = taosUcs4ToMbs((TdUcs4 *)varDataVal(pInput[j].columnData->pData), varDataLen(pInput[j].columnData->pData), convBuf); + if (convBuf == NULL) { + SCL_ERR_RET(terrno); + } + int32_t len = taosUcs4ToMbs((TdUcs4 *)varDataVal(colDataGetData(pInput[j].columnData, colIdx)), varDataLen(colDataGetData(pInput[j].columnData, colIdx)), convBuf); if (len < 0) { + taosMemoryFree(convBuf); code = TSDB_CODE_SCALAR_CONVERT_ERROR; goto _return; } convBuf[len] = 0; num = taosStr2Int32(convBuf, NULL, 10); getAsciiChar(num, &output); + taosMemoryFree(convBuf); } else { code = TSDB_CODE_FUNC_FUNTION_PARA_TYPE; goto _return; From 44466a4bcdb3e0eb14c13c688b69c3e7b0c0e508 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 15 Aug 2024 22:28:13 +0800 Subject: [PATCH 44/46] fix(stream): fix a typo --- source/libs/stream/src/streamDispatch.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 86970f80fa..bf64af6558 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -765,7 +765,7 @@ int32_t streamDispatchStreamBlock(SStreamTask* pTask) { // todo: secure the timerActive and start timer in after lock pTask->lock streamMutexLock(&pTask->lock); bool shouldStop = streamTaskShouldStop(pTask); - streamMutexLock(&pTask->lock); + streamMutexUnlock(&pTask->lock); if (shouldStop) { stDebug("s-task:%s in stop/dropping status, not start dispatch monitor tmr", id); From 868898d11cf5f0f3f237ba3f7d0423d60783ba9d Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 16 Aug 2024 00:28:49 +0800 Subject: [PATCH 45/46] fix(stream): delay to load the stream task. --- source/dnode/mnode/impl/inc/mndStream.h | 4 ++-- source/dnode/mnode/impl/src/mndStream.c | 15 +++++++-------- source/dnode/mnode/impl/src/mndSync.c | 6 +++--- 3 files changed, 12 insertions(+), 13 deletions(-) diff --git a/source/dnode/mnode/impl/inc/mndStream.h b/source/dnode/mnode/impl/inc/mndStream.h index a87a01c5b6..75ba51e498 100644 --- a/source/dnode/mnode/impl/inc/mndStream.h +++ b/source/dnode/mnode/impl/inc/mndStream.h @@ -154,8 +154,8 @@ bool streamTaskIterNextTask(SStreamTaskIter *pIter); int32_t streamTaskIterGetCurrent(SStreamTaskIter *pIter, SStreamTask **pTask); int32_t mndInitExecInfo(); void mndInitStreamExecInfo(SMnode *pMnode, SStreamExecInfo *pExecInfo); -void mndInitStreamExecInfoForLeader(SMnode *pMnode); -void mndInitStreamExecInfoUpdateRole(SMnode *pMnode, int32_t role); +void mndStreamResetInitTaskListLoadFlag(); +void mndUpdateStreamExecInfoRole(SMnode *pMnode, int32_t role); int32_t removeExpiredNodeEntryAndTaskInBuf(SArray *pNodeSnapshot); void removeStreamTasksInBuf(SStreamObj *pStream, SStreamExecInfo *pExecNode); diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index a85b5c733b..9aa36c0c4e 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -2882,13 +2882,12 @@ void mndInitStreamExecInfo(SMnode *pMnode, SStreamExecInfo *pExecInfo) { pExecInfo->initTaskList = true; } -void mndInitStreamExecInfoForLeader(SMnode* pMnode) { +void mndStreamResetInitTaskListLoadFlag() { + mInfo("reset task list buffer init flag for leader"); execInfo.initTaskList = false; - mInfo("init stream execInfo for leader"); - mndInitStreamExecInfo(pMnode, &execInfo); } -void mndInitStreamExecInfoUpdateRole(SMnode* pMnode, int32_t role) { +void mndUpdateStreamExecInfoRole(SMnode* pMnode, int32_t role) { execInfo.switchFromFollower = false; if (execInfo.role == NODE_ROLE_UNINIT) { @@ -3013,7 +3012,8 @@ static int32_t mndProcessDropOrphanTaskReq(SRpcMsg *pReq) { // check if it is conflict with other trans in both sourceDb and targetDb. bool conflict = mndStreamTransConflictCheck(pMnode, pTask->streamId, MND_STREAM_DROP_NAME, false); if (conflict) { - TAOS_RETURN(TSDB_CODE_MND_TRANS_CONFLICT); + code = TSDB_CODE_MND_TRANS_CONFLICT; + goto _err; } SStreamObj dummyObj = {.uid = pTask->streamId, .sourceDb = "", .targetSTbName = ""}; @@ -3026,8 +3026,7 @@ static int32_t mndProcessDropOrphanTaskReq(SRpcMsg *pReq) { code = mndStreamRegisterTrans(pTrans, MND_STREAM_DROP_NAME, pTask->streamId); if (code) { - mndTransDrop(pTrans); - return code; + goto _err; } // drop all tasks @@ -3051,7 +3050,7 @@ _err: tDestroyDropOrphanTaskMsg(&msg); mndTransDrop(pTrans); - if (code == TSDB_CODE_SUCCESS) { + if (code == TSDB_CODE_SUCCESS || code == TSDB_CODE_ACTION_IN_PROGRESS) { mDebug("create drop %d orphan tasks trans succ", numOfTasks); } return code; diff --git a/source/dnode/mnode/impl/src/mndSync.c b/source/dnode/mnode/impl/src/mndSync.c index f5704be371..0f4e4f0363 100644 --- a/source/dnode/mnode/impl/src/mndSync.c +++ b/source/dnode/mnode/impl/src/mndSync.c @@ -361,7 +361,7 @@ static void mndBecomeFollower(const SSyncFSM *pFsm) { } (void)taosThreadMutexUnlock(&pMgmt->lock); - mndInitStreamExecInfoUpdateRole(pMnode, NODE_ROLE_FOLLOWER); + mndUpdateStreamExecInfoRole(pMnode, NODE_ROLE_FOLLOWER); } static void mndBecomeLearner(const SSyncFSM *pFsm) { @@ -385,8 +385,8 @@ static void mndBecomeLeader(const SSyncFSM *pFsm) { mInfo("vgId:1, become leader"); SMnode *pMnode = pFsm->data; - mndInitStreamExecInfoUpdateRole(pMnode, NODE_ROLE_LEADER); - mndInitStreamExecInfoForLeader(pMnode); + mndUpdateStreamExecInfoRole(pMnode, NODE_ROLE_LEADER); + mndStreamResetInitTaskListLoadFlag(); } static bool mndApplyQueueEmpty(const SSyncFSM *pFsm) { From 4e8d5e21a174d3032c97b25164feceaef8c43f86 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 16 Aug 2024 10:34:39 +0800 Subject: [PATCH 46/46] fix(tsdb): check tables number. --- source/dnode/vnode/src/tsdb/tsdbRead2.c | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/source/dnode/vnode/src/tsdb/tsdbRead2.c b/source/dnode/vnode/src/tsdb/tsdbRead2.c index 6783eb2cbd..7d633e2520 100644 --- a/source/dnode/vnode/src/tsdb/tsdbRead2.c +++ b/source/dnode/vnode/src/tsdb/tsdbRead2.c @@ -5182,7 +5182,10 @@ int32_t tsdbNextDataBlock2(STsdbReader* pReader, bool* hasNext) { if (pReader->step == EXTERNAL_ROWS_PREV) { // prepare for the main scan - code = doOpenReaderImpl(pReader); + if (tSimpleHashGetSize(pReader->status.pTableMap) > 0) { + code = doOpenReaderImpl(pReader); + } + int32_t step = 1; resetAllDataBlockScanInfo(pReader->status.pTableMap, pReader->innerReader[0]->info.window.ekey, step); @@ -5210,8 +5213,11 @@ int32_t tsdbNextDataBlock2(STsdbReader* pReader, bool* hasNext) { if (pReader->step == EXTERNAL_ROWS_MAIN && pReader->innerReader[1] != NULL) { // prepare for the next row scan + if (tSimpleHashGetSize(pReader->status.pTableMap) > 0) { + code = doOpenReaderImpl(pReader->innerReader[1]); + } + int32_t step = -1; - code = doOpenReaderImpl(pReader->innerReader[1]); resetAllDataBlockScanInfo(pReader->innerReader[1]->status.pTableMap, pReader->info.window.ekey, step); if (code != TSDB_CODE_SUCCESS) { (void) tsdbReleaseReader(pReader);