diff --git a/docs/en/12-taos-sql/14-stream.md b/docs/en/12-taos-sql/14-stream.md index 329e98dbe3..c8be1753a2 100644 --- a/docs/en/12-taos-sql/14-stream.md +++ b/docs/en/12-taos-sql/14-stream.md @@ -168,3 +168,11 @@ All [scalar functions](../function/#scalar-functions) are available in stream pr - [unique](../function/#unique) - [mode](../function/#mode) +## Pause\Resume stream +1.pause stream +PAUSE STREAM [IF EXISTS] stream_name; +If "IF EXISTS" is not specified and the stream does not exist, an error will be reported; If "IF EXISTS" is specified and the stream does not exist, success is returned; If the stream exists, paused all stream tasks. + +2.resume stream +RESUME STREAM [IF EXISTS] [IGNORE UNTREATED] stream_name; +If "IF EXISTS" is not specified and the stream does not exist, an error will be reported. If "IF EXISTS" is specified and the stream does not exist, success is returned; If the stream exists, all of the stream tasks will be resumed. If "IGNORE UntREATED" is specified, data written during the pause period of stream is ignored when resuming stream. diff --git a/docs/zh/12-taos-sql/14-stream.md b/docs/zh/12-taos-sql/14-stream.md index 634f50356d..43bdc9b04a 100644 --- a/docs/zh/12-taos-sql/14-stream.md +++ b/docs/zh/12-taos-sql/14-stream.md @@ -249,3 +249,12 @@ T = 最新事件时间 - DELETE_MARK - [unique](../function/#unique) - [mode](../function/#mode) +## 暂停、恢复流计算 +1.流计算暂停计算任务 +PAUSE STREAM [IF EXISTS] stream_name; +没有指定IF EXISTS,如果该stream不存在,则报错;如果存在,则暂停流计算。指定了IF EXISTS,如果该stream不存在,则返回成功;如果存在,则暂停流计算 + +2.流计算恢复计算任务 +RESUME STREAM [IF EXISTS] [IGNORE UNTREATED] stream_name; +没有指定IF EXISTS,如果该stream不存在,则报错,如果存在,则恢复流计算;指定了IF EXISTS,如果stream不存在,则返回成功;如果存在,则恢复流计算。如果指定IGNORE UNTREATED,则恢复流计算时,忽略流计算暂停期间写入的数据。 + diff --git a/include/common/tdatablock.h b/include/common/tdatablock.h index c0412d2617..dac723aa58 100644 --- a/include/common/tdatablock.h +++ b/include/common/tdatablock.h @@ -241,7 +241,7 @@ int32_t blockEncode(const SSDataBlock* pBlock, char* data, int32_t numOfCols); const char* blockDecode(SSDataBlock* pBlock, const char* pData); // for debug -char* dumpBlockData(SSDataBlock* pDataBlock, const char* flag, char** dumpBuf); +char* dumpBlockData(SSDataBlock* pDataBlock, const char* flag, char** dumpBuf, const char* taskIdStr); int32_t buildSubmitReqFromDataBlock(SSubmitReq2** pReq, const SSDataBlock* pDataBlocks, const STSchema* pTSchema, int64_t uid, int32_t vgId, tb_uid_t suid); diff --git a/include/common/tmsg.h b/include/common/tmsg.h index 01923d2b30..d7409da126 100644 --- a/include/common/tmsg.h +++ b/include/common/tmsg.h @@ -231,6 +231,7 @@ typedef struct SField { uint8_t type; int8_t flags; int32_t bytes; + char comment[TSDB_COL_COMMENT_LEN]; } SField; typedef struct SRetention { @@ -309,6 +310,7 @@ struct SSchema { col_id_t colId; int32_t bytes; char name[TSDB_COL_NAME_LEN]; + char comment[TSDB_COL_COMMENT_LEN]; }; struct SSchema2 { @@ -1163,6 +1165,9 @@ typedef struct { int32_t vgId; int8_t syncState; int8_t syncRestore; + int64_t syncTerm; + int64_t roleTimeMs; + int64_t startTimeMs; int8_t syncCanRead; int64_t cacheUsage; int64_t numOfTables; @@ -1179,9 +1184,9 @@ typedef struct { } SVnodeLoad; typedef struct { - int8_t syncState; - int8_t syncRestore; + int8_t syncState; int64_t syncTerm; + int8_t syncRestore; int64_t roleTimeMs; } SMnodeLoad; @@ -2387,6 +2392,9 @@ typedef struct { int8_t type; int8_t flags; int32_t bytes; + bool hasColComment; + char* colComment; + int32_t colCommentLen; // TSDB_ALTER_TABLE_DROP_COLUMN // TSDB_ALTER_TABLE_UPDATE_COLUMN_BYTES int8_t colModType; diff --git a/include/common/ttokendef.h b/include/common/ttokendef.h index 8a6b7b5020..0d21e91a08 100644 --- a/include/common/ttokendef.h +++ b/include/common/ttokendef.h @@ -130,25 +130,25 @@ #define TK_NK_EQ 112 #define TK_USING 113 #define TK_TAGS 114 -#define TK_BOOL 115 -#define TK_TINYINT 116 -#define TK_SMALLINT 117 -#define TK_INT 118 -#define TK_INTEGER 119 -#define TK_BIGINT 120 -#define TK_FLOAT 121 -#define TK_DOUBLE 122 -#define TK_BINARY 123 -#define TK_NCHAR 124 -#define TK_UNSIGNED 125 -#define TK_JSON 126 -#define TK_VARCHAR 127 -#define TK_MEDIUMBLOB 128 -#define TK_BLOB 129 -#define TK_VARBINARY 130 -#define TK_GEOMETRY 131 -#define TK_DECIMAL 132 -#define TK_COMMENT 133 +#define TK_COMMENT 115 +#define TK_BOOL 116 +#define TK_TINYINT 117 +#define TK_SMALLINT 118 +#define TK_INT 119 +#define TK_INTEGER 120 +#define TK_BIGINT 121 +#define TK_FLOAT 122 +#define TK_DOUBLE 123 +#define TK_BINARY 124 +#define TK_NCHAR 125 +#define TK_UNSIGNED 126 +#define TK_JSON 127 +#define TK_VARCHAR 128 +#define TK_MEDIUMBLOB 129 +#define TK_BLOB 130 +#define TK_VARBINARY 131 +#define TK_GEOMETRY 132 +#define TK_DECIMAL 133 #define TK_MAX_DELAY 134 #define TK_WATERMARK 135 #define TK_ROLLUP 136 @@ -354,7 +354,6 @@ #define TK_VIEW 336 #define TK_WAL 337 - #define TK_NK_SPACE 600 #define TK_NK_COMMENT 601 #define TK_NK_ILLEGAL 602 diff --git a/include/libs/executor/storageapi.h b/include/libs/executor/storageapi.h index d35ef52e87..0a240dd8f5 100644 --- a/include/libs/executor/storageapi.h +++ b/include/libs/executor/storageapi.h @@ -106,6 +106,8 @@ typedef struct SMCtbCursor { void *pVal; int kLen; int vLen; + int8_t paused; + int lock; } SMCtbCursor; typedef struct SRowBuffPos { @@ -295,7 +297,9 @@ int32_t vnodeGetCtbIdListByFilter(void *pVnode, int64_t suid, SArray *list, bool int32_t vnodeGetStbIdList(void *pVnode, int64_t suid, SArray *list); */ SMCtbCursor* (*openCtbCursor)(void *pVnode, tb_uid_t uid, int lock); - void (*closeCtbCursor)(SMCtbCursor *pCtbCur, int lock); + int32_t (*resumeCtbCursor)(SMCtbCursor* pCtbCur, int8_t first); + void (*pauseCtbCursor)(SMCtbCursor* pCtbCur); + void (*closeCtbCursor)(SMCtbCursor *pCtbCur); tb_uid_t (*ctbCursorNext)(SMCtbCursor* pCur); } SStoreMeta; diff --git a/include/libs/nodes/cmdnodes.h b/include/libs/nodes/cmdnodes.h index f0a715e651..ff02070882 100644 --- a/include/libs/nodes/cmdnodes.h +++ b/include/libs/nodes/cmdnodes.h @@ -23,10 +23,11 @@ extern "C" { #include "query.h" #include "querynodes.h" -#define DESCRIBE_RESULT_COLS 4 -#define DESCRIBE_RESULT_FIELD_LEN (TSDB_COL_NAME_LEN - 1 + VARSTR_HEADER_SIZE) -#define DESCRIBE_RESULT_TYPE_LEN (20 + VARSTR_HEADER_SIZE) -#define DESCRIBE_RESULT_NOTE_LEN (8 + VARSTR_HEADER_SIZE) +#define DESCRIBE_RESULT_COLS 5 +#define DESCRIBE_RESULT_FIELD_LEN (TSDB_COL_NAME_LEN - 1 + VARSTR_HEADER_SIZE) +#define DESCRIBE_RESULT_TYPE_LEN (20 + VARSTR_HEADER_SIZE) +#define DESCRIBE_RESULT_NOTE_LEN (8 + VARSTR_HEADER_SIZE) +#define DESCRIBE_RESULT_COL_COMMENT_LEN (TSDB_COL_COMMENT_LEN) #define SHOW_CREATE_DB_RESULT_COLS 2 #define SHOW_CREATE_DB_RESULT_FIELD1_LEN (TSDB_DB_NAME_LEN + VARSTR_HEADER_SIZE) @@ -155,7 +156,7 @@ typedef struct SColumnDefNode { ENodeType type; char colName[TSDB_COL_NAME_LEN]; SDataType dataType; - char comments[TSDB_TB_COMMENT_LEN]; + char comments[TSDB_COL_COMMENT_LEN]; bool sma; } SColumnDefNode; @@ -214,6 +215,7 @@ typedef struct SAlterTableStmt { char newColName[TSDB_COL_NAME_LEN]; STableOptions* pOptions; SDataType dataType; + char colComment[TSDB_COL_COMMENT_LEN]; SValueNode* pVal; } SAlterTableStmt; diff --git a/include/libs/sync/sync.h b/include/libs/sync/sync.h index 88ccf562c7..8844145652 100644 --- a/include/libs/sync/sync.h +++ b/include/libs/sync/sync.h @@ -241,6 +241,7 @@ typedef struct SSyncState { bool canRead; SyncTerm term; int64_t roleTimeMs; + int64_t startTimeMs; } SSyncState; int32_t syncInit(); diff --git a/include/util/tdef.h b/include/util/tdef.h index e4af88bf10..1e12f12087 100644 --- a/include/util/tdef.h +++ b/include/util/tdef.h @@ -230,6 +230,7 @@ typedef enum ELogicConditionType { #define TSDB_APP_NAME_LEN TSDB_UNI_LEN #define TSDB_TB_COMMENT_LEN 1025 +#define TSDB_COL_COMMENT_LEN 1025 #define TSDB_QUERY_ID_LEN 26 #define TSDB_TRANS_OPER_LEN 16 diff --git a/source/common/src/systable.c b/source/common/src/systable.c index eb8042099f..66a498d15c 100644 --- a/source/common/src/systable.c +++ b/source/common/src/systable.c @@ -284,7 +284,6 @@ static const SSysDbTableSchema topicSchema[] = { {.name = "type", .bytes = 8 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_BINARY, .sysInfo = false}, }; - 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}, @@ -295,12 +294,13 @@ static const SSysDbTableSchema subscriptionSchema[] = { }; static const SSysDbTableSchema vnodesSchema[] = { - {.name = "vgroup_id", .bytes = 4, .type = TSDB_DATA_TYPE_INT, .sysInfo = true}, - {.name = "replica", .bytes = 1, .type = TSDB_DATA_TYPE_TINYINT, .sysInfo = true}, - {.name = "status", .bytes = 9 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = true}, - {.name = "db_name", .bytes = SYSTABLE_SCH_DB_NAME_LEN, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = true}, {.name = "dnode_id", .bytes = 4, .type = TSDB_DATA_TYPE_INT, .sysInfo = true}, - {.name = "dnode_ep", .bytes = TSDB_EP_LEN + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = true}, + {.name = "vgroup_id", .bytes = 4, .type = TSDB_DATA_TYPE_INT, .sysInfo = true}, + {.name = "db_name", .bytes = SYSTABLE_SCH_DB_NAME_LEN, .type = TSDB_DATA_TYPE_BINARY, .sysInfo = false}, + {.name = "status", .bytes = 9 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = true}, + {.name = "role_time", .bytes = 8, .type = TSDB_DATA_TYPE_TIMESTAMP, .sysInfo = true}, + {.name = "start_time", .bytes = 8, .type = TSDB_DATA_TYPE_TIMESTAMP, .sysInfo = true}, + {.name = "restored", .bytes = 1, .type = TSDB_DATA_TYPE_BOOL, .sysInfo = true}, }; static const SSysDbTableSchema userUserPrivilegesSchema[] = { diff --git a/source/common/src/tdatablock.c b/source/common/src/tdatablock.c index 9f30d04b74..bbbb6c7178 100644 --- a/source/common/src/tdatablock.c +++ b/source/common/src/tdatablock.c @@ -1771,7 +1771,7 @@ static char* formatTimestamp(char* buf, int64_t val, int precision) { } // for debug -char* dumpBlockData(SSDataBlock* pDataBlock, const char* flag, char** pDataBuf) { +char* dumpBlockData(SSDataBlock* pDataBlock, const char* flag, char** pDataBuf, const char* taskIdStr) { int32_t size = 2048 * 1024; *pDataBuf = taosMemoryCalloc(size, 1); char* dumpBuf = *pDataBuf; @@ -1780,9 +1780,9 @@ char* dumpBlockData(SSDataBlock* pDataBlock, const char* flag, char** pDataBuf) int32_t rows = pDataBlock->info.rows; int32_t len = 0; len += snprintf(dumpBuf + len, size - len, - "===stream===%s|block type %d|child id %d|group id:%" PRIu64 "|uid:%" PRId64 "|rows:%" PRId64 - "|version:%" PRIu64 "|cal start:%" PRIu64 "|cal end:%" PRIu64 "|tbl:%s\n", - flag, (int32_t)pDataBlock->info.type, pDataBlock->info.childId, pDataBlock->info.id.groupId, + "%s===stream===%s|block type %d|child id %d|group id:%" PRIu64 "|uid:%" PRId64 + "|rows:%" PRId64 "|version:%" PRIu64 "|cal start:%" PRIu64 "|cal end:%" PRIu64 "|tbl:%s\n", + taskIdStr, flag, (int32_t)pDataBlock->info.type, pDataBlock->info.childId, pDataBlock->info.id.groupId, pDataBlock->info.id.uid, pDataBlock->info.rows, pDataBlock->info.version, pDataBlock->info.calWin.skey, pDataBlock->info.calWin.ekey, pDataBlock->info.parTbName); if (len >= size - 1) return dumpBuf; diff --git a/source/common/src/tmsg.c b/source/common/src/tmsg.c index ef0006e7ab..d6d9e954e8 100644 --- a/source/common/src/tmsg.c +++ b/source/common/src/tmsg.c @@ -534,6 +534,7 @@ int32_t tSerializeSMCreateStbReq(void *buf, int32_t bufLen, SMCreateStbReq *pReq if (tEncodeI8(&encoder, pField->flags) < 0) return -1; if (tEncodeI32(&encoder, pField->bytes) < 0) return -1; if (tEncodeCStr(&encoder, pField->name) < 0) return -1; + if (tEncodeCStr(&encoder, pField->comment) < 0) return -1; } for (int32_t i = 0; i < pReq->numOfTags; ++i) { @@ -542,6 +543,7 @@ int32_t tSerializeSMCreateStbReq(void *buf, int32_t bufLen, SMCreateStbReq *pReq if (tEncodeI8(&encoder, pField->flags) < 0) return -1; if (tEncodeI32(&encoder, pField->bytes) < 0) return -1; if (tEncodeCStr(&encoder, pField->name) < 0) return -1; + if (tEncodeCStr(&encoder, pField->comment) < 0) return -1; } for (int32_t i = 0; i < pReq->numOfFuncs; ++i) { @@ -608,6 +610,7 @@ int32_t tDeserializeSMCreateStbReq(void *buf, int32_t bufLen, SMCreateStbReq *pR if (tDecodeI8(&decoder, &field.flags) < 0) return -1; if (tDecodeI32(&decoder, &field.bytes) < 0) return -1; if (tDecodeCStrTo(&decoder, field.name) < 0) return -1; + if (tDecodeCStrTo(&decoder, field.comment) < 0) return -1; if (taosArrayPush(pReq->pColumns, &field) == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; @@ -620,6 +623,7 @@ int32_t tDeserializeSMCreateStbReq(void *buf, int32_t bufLen, SMCreateStbReq *pR if (tDecodeI8(&decoder, &field.flags) < 0) return -1; if (tDecodeI32(&decoder, &field.bytes) < 0) return -1; if (tDecodeCStrTo(&decoder, field.name) < 0) return -1; + if (tDecodeCStrTo(&decoder, field.comment) < 0) return -1; if (taosArrayPush(pReq->pTags, &field) == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; @@ -1079,8 +1083,8 @@ int32_t tSerializeSStatusReq(void *buf, int32_t bufLen, SStatusReq *pReq) { if (tEncodeI64(&encoder, pload->pointsWritten) < 0) return -1; if (tEncodeI32(&encoder, pload->numOfCachedTables) < 0) return -1; if (tEncodeI32(&encoder, reserved) < 0) return -1; - if (tEncodeI64(&encoder, reserved) < 0) return -1; - if (tEncodeI64(&encoder, reserved) < 0) return -1; + if (tEncodeI64(&encoder, pload->roleTimeMs) < 0) return -1; + if (tEncodeI64(&encoder, pload->startTimeMs) < 0) return -1; } // mnode loads @@ -1104,6 +1108,16 @@ int32_t tSerializeSStatusReq(void *buf, int32_t bufLen, SStatusReq *pReq) { if (tEncodeI64(&encoder, pReq->mload.syncTerm) < 0) return -1; if (tEncodeI64(&encoder, pReq->mload.roleTimeMs) < 0) return -1; if (tEncodeI8(&encoder, pReq->clusterCfg.ttlChangeOnWrite) < 0) return -1; + + // vnode extra + for (int32_t i = 0; i < vlen; ++i) { + SVnodeLoad *pload = taosArrayGet(pReq->pVloads, i); + int64_t reserved = 0; + if (tEncodeI64(&encoder, pload->syncTerm) < 0) return -1; + if (tEncodeI64(&encoder, reserved) < 0) return -1; + if (tEncodeI64(&encoder, reserved) < 0) return -1; + if (tEncodeI64(&encoder, reserved) < 0) return -1; + } tEndEncode(&encoder); int32_t tlen = encoder.pos; @@ -1148,7 +1162,7 @@ int32_t tDeserializeSStatusReq(void *buf, int32_t bufLen, SStatusReq *pReq) { for (int32_t i = 0; i < vlen; ++i) { SVnodeLoad vload = {0}; - int64_t reserved64 = 0; + vload.syncTerm = -1; int32_t reserved32 = 0; if (tDecodeI32(&decoder, &vload.vgId) < 0) return -1; if (tDecodeI8(&decoder, &vload.syncState) < 0) return -1; @@ -1162,14 +1176,15 @@ int32_t tDeserializeSStatusReq(void *buf, int32_t bufLen, SStatusReq *pReq) { if (tDecodeI64(&decoder, &vload.pointsWritten) < 0) return -1; if (tDecodeI32(&decoder, &vload.numOfCachedTables) < 0) return -1; if (tDecodeI32(&decoder, (int32_t *)&reserved32) < 0) return -1; - if (tDecodeI64(&decoder, &reserved64) < 0) return -1; - if (tDecodeI64(&decoder, &reserved64) < 0) return -1; + if (tDecodeI64(&decoder, &vload.roleTimeMs) < 0) return -1; + if (tDecodeI64(&decoder, &vload.startTimeMs) < 0) return -1; if (taosArrayPush(pReq->pVloads, &vload) == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } } + // mnode loads if (tDecodeI8(&decoder, &pReq->mload.syncState) < 0) return -1; if (tDecodeI8(&decoder, &pReq->mload.syncRestore) < 0) return -1; @@ -1200,6 +1215,17 @@ int32_t tDeserializeSStatusReq(void *buf, int32_t bufLen, SStatusReq *pReq) { if (tDecodeI8(&decoder, &pReq->clusterCfg.ttlChangeOnWrite) < 0) return -1; } + // vnode extra + if (!tDecodeIsEnd(&decoder)) { + for (int32_t i = 0; i < vlen; ++i) { + SVnodeLoad *pLoad = taosArrayGet(pReq->pVloads, i); + int64_t reserved = 0; + if (tDecodeI64(&decoder, &pLoad->syncTerm) < 0) return -1; + if (tDecodeI64(&decoder, &reserved) < 0) return -1; + if (tDecodeI64(&decoder, &reserved) < 0) return -1; + if (tDecodeI64(&decoder, &reserved) < 0) return -1; + } + } tEndDecode(&decoder); tDecoderClear(&decoder); return 0; @@ -2301,7 +2327,7 @@ int32_t tDeserializeSTableCfgRsp(void *buf, int32_t bufLen, STableCfgRsp *pRsp) } int32_t totalCols = pRsp->numOfTags + pRsp->numOfColumns; - pRsp->pSchemas = taosMemoryMalloc(sizeof(SSchema) * totalCols); + pRsp->pSchemas = taosMemoryCalloc(totalCols, sizeof(SSchema)); if (pRsp->pSchemas == NULL) return -1; for (int32_t i = 0; i < totalCols; ++i) { @@ -3684,7 +3710,7 @@ static int32_t tDecodeSTableMetaRsp(SDecoder *pDecoder, STableMetaRsp *pRsp) { int32_t totalCols = pRsp->numOfTags + pRsp->numOfColumns; if (totalCols > 0) { - pRsp->pSchemas = taosMemoryMalloc(sizeof(SSchema) * totalCols); + pRsp->pSchemas = taosMemoryCalloc(totalCols, sizeof(SSchema)); if (pRsp->pSchemas == NULL) return -1; for (int32_t i = 0; i < totalCols; ++i) { diff --git a/source/dnode/mnode/impl/inc/mndDef.h b/source/dnode/mnode/impl/inc/mndDef.h index 4fca835f0e..529cede453 100644 --- a/source/dnode/mnode/impl/inc/mndDef.h +++ b/source/dnode/mnode/impl/inc/mndDef.h @@ -347,8 +347,11 @@ typedef struct { typedef struct { int32_t dnodeId; ESyncState syncState; + int64_t syncTerm; bool syncRestore; bool syncCanRead; + int64_t roleTimeMs; + int64_t startTimeMs; ESyncRole nodeRole; } SVnodeGid; diff --git a/source/dnode/mnode/impl/src/mndDnode.c b/source/dnode/mnode/impl/src/mndDnode.c index 234e81a670..39285ced5d 100644 --- a/source/dnode/mnode/impl/src/mndDnode.c +++ b/source/dnode/mnode/impl/src/mndDnode.c @@ -424,6 +424,47 @@ static int32_t mndCheckClusterCfgPara(SMnode *pMnode, SDnodeObj *pDnode, const S return 0; } +static bool mndUpdateVnodeState(int32_t vgId, SVnodeGid *pGid, SVnodeLoad *pVload) { + bool stateChanged = false; + bool roleChanged = pGid->syncState != pVload->syncState || + (pVload->syncTerm != -1 && pGid->syncTerm != pVload->syncTerm) || + pGid->roleTimeMs != pVload->roleTimeMs; + if (roleChanged || pGid->syncRestore != pVload->syncRestore || pGid->syncCanRead != pVload->syncCanRead || + pGid->startTimeMs != pVload->startTimeMs) { + mInfo( + "vgId:%d, state changed by status msg, old state:%s restored:%d canRead:%d new state:%s restored:%d " + "canRead:%d, dnode:%d", + vgId, syncStr(pGid->syncState), pGid->syncRestore, pGid->syncCanRead, syncStr(pVload->syncState), + pVload->syncRestore, pVload->syncCanRead, pGid->dnodeId); + pGid->syncState = pVload->syncState; + pGid->syncTerm = pVload->syncTerm; + pGid->syncRestore = pVload->syncRestore; + pGid->syncCanRead = pVload->syncCanRead; + pGid->startTimeMs = pVload->startTimeMs; + pGid->roleTimeMs = pVload->roleTimeMs; + stateChanged = true; + } + return stateChanged; +} + +static bool mndUpdateMnodeState(SMnodeObj *pObj, SMnodeLoad *pMload) { + bool stateChanged = false; + bool roleChanged = pObj->syncState != pMload->syncState || + (pMload->syncTerm != -1 && pObj->syncTerm != pMload->syncTerm) || + pObj->roleTimeMs != pMload->roleTimeMs; + if (roleChanged || pObj->syncRestore != pMload->syncRestore) { + mInfo("dnode:%d, mnode syncState from %s to %s, restoreState from %d to %d, syncTerm from %" PRId64 " to %" PRId64, + pObj->id, syncStr(pObj->syncState), syncStr(pMload->syncState), pObj->syncRestore, pMload->syncRestore, + pObj->syncTerm, pMload->syncTerm); + pObj->syncState = pMload->syncState; + pObj->syncTerm = pMload->syncTerm; + pObj->syncRestore = pMload->syncRestore; + pObj->roleTimeMs = pMload->roleTimeMs; + stateChanged = true; + } + return stateChanged; +} + static int32_t mndProcessStatusReq(SRpcMsg *pReq) { SMnode *pMnode = pReq->info.node; SStatusReq statusReq = {0}; @@ -496,26 +537,21 @@ static int32_t mndProcessStatusReq(SRpcMsg *pReq) { pVgroup->compStorage = pVload->compStorage; pVgroup->pointsWritten = pVload->pointsWritten; } - bool roleChanged = false; + bool stateChanged = false; for (int32_t vg = 0; vg < pVgroup->replica; ++vg) { SVnodeGid *pGid = &pVgroup->vnodeGid[vg]; if (pGid->dnodeId == statusReq.dnodeId) { - if (pGid->syncState != pVload->syncState || pGid->syncRestore != pVload->syncRestore || - pGid->syncCanRead != pVload->syncCanRead) { - mInfo( - "vgId:%d, state changed by status msg, old state:%s restored:%d canRead:%d new state:%s restored:%d " - "canRead:%d, dnode:%d", - pVgroup->vgId, syncStr(pGid->syncState), pGid->syncRestore, pGid->syncCanRead, - syncStr(pVload->syncState), pVload->syncRestore, pVload->syncCanRead, pDnode->id); - pGid->syncState = pVload->syncState; - pGid->syncRestore = pVload->syncRestore; - pGid->syncCanRead = pVload->syncCanRead; - roleChanged = true; + if (pVload->startTimeMs == 0) { + pVload->startTimeMs = statusReq.rebootTime; } + if (pVload->roleTimeMs == 0) { + pVload->roleTimeMs = statusReq.rebootTime; + } + stateChanged = mndUpdateVnodeState(pVgroup->vgId, pGid, pVload); break; } } - if (roleChanged) { + if (stateChanged) { SDbObj *pDb = mndAcquireDb(pMnode, pVgroup->dbName); if (pDb != NULL && pDb->stateTs != curMs) { mInfo("db:%s, stateTs changed by status msg, old stateTs:%" PRId64 " new stateTs:%" PRId64, pDb->name, @@ -531,23 +567,10 @@ static int32_t mndProcessStatusReq(SRpcMsg *pReq) { SMnodeObj *pObj = mndAcquireMnode(pMnode, pDnode->id); if (pObj != NULL) { - bool roleChanged = pObj->syncState != statusReq.mload.syncState || - (statusReq.mload.syncTerm != -1 && pObj->syncTerm != statusReq.mload.syncTerm); - bool restoreChanged = pObj->syncRestore != statusReq.mload.syncRestore; - if (roleChanged || restoreChanged) { - mInfo("dnode:%d, mnode syncState from %s to %s, restoreState from %d to %d, syncTerm from %" PRId64 - " to %" PRId64, - pObj->id, syncStr(pObj->syncState), syncStr(statusReq.mload.syncState), pObj->syncRestore, - statusReq.mload.syncRestore, pObj->syncTerm, statusReq.mload.syncTerm); - pObj->syncState = statusReq.mload.syncState; - pObj->syncRestore = statusReq.mload.syncRestore; - pObj->syncTerm = statusReq.mload.syncTerm; + if (statusReq.mload.roleTimeMs == 0) { + statusReq.mload.roleTimeMs = statusReq.rebootTime; } - - if (roleChanged) { - pObj->roleTimeMs = (statusReq.mload.roleTimeMs != 0) ? statusReq.mload.roleTimeMs : taosGetTimestampMs(); - } - + mndUpdateMnodeState(pObj, &statusReq.mload); mndReleaseMnode(pMnode, pObj); } diff --git a/source/dnode/mnode/impl/src/mndMain.c b/source/dnode/mnode/impl/src/mndMain.c index c739636a84..78a9f4f508 100644 --- a/source/dnode/mnode/impl/src/mndMain.c +++ b/source/dnode/mnode/impl/src/mndMain.c @@ -187,7 +187,7 @@ static void mndSetVgroupOffline(SMnode *pMnode, int32_t dnodeId, int64_t curMs) pIter = sdbFetch(pSdb, SDB_VGROUP, pIter, (void **)&pVgroup); if (pIter == NULL) break; - bool roleChanged = false; + bool stateChanged = false; for (int32_t vg = 0; vg < pVgroup->replica; ++vg) { SVnodeGid *pGid = &pVgroup->vnodeGid[vg]; if (pGid->dnodeId == dnodeId) { @@ -199,13 +199,14 @@ static void mndSetVgroupOffline(SMnode *pMnode, int32_t dnodeId, int64_t curMs) pGid->syncState = TAOS_SYNC_STATE_OFFLINE; pGid->syncRestore = 0; pGid->syncCanRead = 0; - roleChanged = true; + pGid->startTimeMs = 0; + stateChanged = true; } break; } } - if (roleChanged) { + if (stateChanged) { SDbObj *pDb = mndAcquireDb(pMnode, pVgroup->dbName); if (pDb != NULL && pDb->stateTs != curMs) { mInfo("db:%s, stateTs changed by offline check, old newTs:%" PRId64 " newTs:%" PRId64, pDb->name, pDb->stateTs, diff --git a/source/dnode/mnode/impl/src/mndMnode.c b/source/dnode/mnode/impl/src/mndMnode.c index 4ee2bc159b..2757578d35 100644 --- a/source/dnode/mnode/impl/src/mndMnode.c +++ b/source/dnode/mnode/impl/src/mndMnode.c @@ -807,7 +807,6 @@ static int32_t mndRetrieveMnodes(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock *pB ESdbStatus objStatus = 0; char *pWrite; int64_t curMs = taosGetTimestampMs(); - int64_t dummyTimeMs = 0; pSelfObj = sdbAcquire(pSdb, SDB_MNODE, &pMnode->selfDnodeId); if (pSelfObj == NULL) { @@ -858,16 +857,9 @@ static int32_t mndRetrieveMnodes(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock *pB pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); colDataSetVal(pColInfo, numOfRows, (const char *)&pObj->createdTime, false); + int64_t roleTimeMs = (isDnodeOnline) ? pObj->roleTimeMs : 0; pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - if (pObj->syncTerm != pSelfObj->syncTerm || !isDnodeOnline) { - // state of old term / no status report => use dummyTimeMs - if (pObj->syncTerm > pSelfObj->syncTerm) { - mError("mnode:%d has a newer term:%" PRId64 " than me:%" PRId64, pObj->id, pObj->syncTerm, pSelfObj->syncTerm); - } - colDataSetVal(pColInfo, numOfRows, (const char *)&dummyTimeMs, false); - } else { - colDataSetVal(pColInfo, numOfRows, (const char *)&pObj->roleTimeMs, false); - } + colDataSetVal(pColInfo, numOfRows, (const char *)&roleTimeMs, false); numOfRows++; sdbRelease(pSdb, pObj); diff --git a/source/dnode/mnode/impl/src/mndStb.c b/source/dnode/mnode/impl/src/mndStb.c index c6dce0d578..a56ca552f6 100644 --- a/source/dnode/mnode/impl/src/mndStb.c +++ b/source/dnode/mnode/impl/src/mndStb.c @@ -835,6 +835,7 @@ int32_t mndBuildStbFromReq(SMnode *pMnode, SStbObj *pDst, SMCreateStbReq *pCreat pSchema->bytes = pField->bytes; pSchema->flags = pField->flags; memcpy(pSchema->name, pField->name, TSDB_COL_NAME_LEN); + memcpy(pSchema->comment, pField->comment, TSDB_COL_COMMENT_LEN); pSchema->colId = pDst->nextColId; pDst->nextColId++; } @@ -848,6 +849,7 @@ int32_t mndBuildStbFromReq(SMnode *pMnode, SStbObj *pDst, SMCreateStbReq *pCreat SSCHMEA_SET_IDX_ON(pSchema); } memcpy(pSchema->name, pField->name, TSDB_COL_NAME_LEN); + memcpy(pSchema->comment, pField->comment, TSDB_COL_COMMENT_LEN); pSchema->colId = pDst->nextColId; pDst->nextColId++; } diff --git a/source/dnode/mnode/impl/src/mndVgroup.c b/source/dnode/mnode/impl/src/mndVgroup.c index f3dded9c76..406392271c 100644 --- a/source/dnode/mnode/impl/src/mndVgroup.c +++ b/source/dnode/mnode/impl/src/mndVgroup.c @@ -961,27 +961,24 @@ static int32_t mndRetrieveVnodes(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock *pB int32_t numOfRows = 0; SVgObj *pVgroup = NULL; int32_t cols = 0; + int64_t curMs = taosGetTimestampMs(); while (numOfRows < rows) { pShow->pIter = sdbFetch(pSdb, SDB_VGROUP, pShow->pIter, (void **)&pVgroup); if (pShow->pIter == NULL) break; for (int32_t i = 0; i < pVgroup->replica && numOfRows < rows; ++i) { - SVnodeGid *pVgid = &pVgroup->vnodeGid[i]; + SVnodeGid *pGid = &pVgroup->vnodeGid[i]; SColumnInfoData *pColInfo = NULL; cols = 0; + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + colDataSetVal(pColInfo, numOfRows, (const char *)&pGid->dnodeId, false); + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); colDataSetVal(pColInfo, numOfRows, (const char *)&pVgroup->vgId, false); - pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)&pVgroup->replica, false); - - char buf[20] = {0}; - STR_TO_VARSTR(buf, syncStr(pVgid->syncState)); - pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)buf, false); - + // db_name const char *dbname = mndGetDbStr(pVgroup->dbName); char b1[TSDB_DB_NAME_LEN + VARSTR_HEADER_SIZE] = {0}; if (dbname != NULL) { @@ -992,20 +989,33 @@ static int32_t mndRetrieveVnodes(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock *pB pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); colDataSetVal(pColInfo, numOfRows, (const char *)b1, false); - pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)&pVgid->dnodeId, false); - - SDnodeObj *pDnode = mndAcquireDnode(pMnode, pVgid->dnodeId); - char b2[TSDB_EP_LEN + VARSTR_HEADER_SIZE] = {0}; - if (pDnode != NULL) { - STR_WITH_MAXSIZE_TO_VARSTR(b2, pDnode->ep, TSDB_EP_LEN + VARSTR_HEADER_SIZE); - } else { - STR_WITH_MAXSIZE_TO_VARSTR(b2, "NULL", TSDB_EP_LEN + VARSTR_HEADER_SIZE); + // dnode is online? + SDnodeObj *pDnode = mndAcquireDnode(pMnode, pGid->dnodeId); + if (pDnode == NULL) { + mError("failed to acquire dnode. dnodeId:%d", pGid->dnodeId); + break; } + bool isDnodeOnline = mndIsDnodeOnline(pDnode, curMs); + + char buf[20] = {0}; + ESyncState syncState = (isDnodeOnline) ? pGid->syncState : TAOS_SYNC_STATE_OFFLINE; + STR_TO_VARSTR(buf, syncStr(syncState)); pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)b2, false); + colDataSetVal(pColInfo, numOfRows, (const char *)buf, false); + + int64_t roleTimeMs = (isDnodeOnline) ? pGid->roleTimeMs : 0; + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + colDataSetVal(pColInfo, numOfRows, (const char *)&roleTimeMs, false); + + int64_t startTimeMs = (isDnodeOnline) ? pGid->startTimeMs : 0; + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + colDataSetVal(pColInfo, numOfRows, (const char *)&startTimeMs, false); + + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + colDataSetVal(pColInfo, numOfRows, (const char *)&pGid->syncRestore, false); numOfRows++; + sdbRelease(pSdb, pDnode); } sdbRelease(pSdb, pVgroup); diff --git a/source/dnode/vnode/src/inc/vnodeInt.h b/source/dnode/vnode/src/inc/vnodeInt.h index a791991d69..c85b790c07 100644 --- a/source/dnode/vnode/src/inc/vnodeInt.h +++ b/source/dnode/vnode/src/inc/vnodeInt.h @@ -169,7 +169,9 @@ int metaDropIndexFromSTable(SMeta* pMeta, int64_t version, SDropIndexReq* pReq); int64_t metaGetTimeSeriesNum(SMeta* pMeta); SMCtbCursor* metaOpenCtbCursor(void* pVnode, tb_uid_t uid, int lock); -void metaCloseCtbCursor(SMCtbCursor* pCtbCur, int lock); +int32_t metaResumeCtbCursor(SMCtbCursor* pCtbCur, int8_t first); +void metaPauseCtbCursor(SMCtbCursor* pCtbCur); +void metaCloseCtbCursor(SMCtbCursor* pCtbCur); tb_uid_t metaCtbCursorNext(SMCtbCursor* pCtbCur); SMStbCursor* metaOpenStbCursor(SMeta* pMeta, tb_uid_t uid); void metaCloseStbCursor(SMStbCursor* pStbCur); diff --git a/source/dnode/vnode/src/meta/metaQuery.c b/source/dnode/vnode/src/meta/metaQuery.c index d5e6a23d5c..ca5346eee3 100644 --- a/source/dnode/vnode/src/meta/metaQuery.c +++ b/source/dnode/vnode/src/meta/metaQuery.c @@ -423,40 +423,75 @@ SMCtbCursor *metaOpenCtbCursor(void* pVnode, tb_uid_t uid, int lock) { pCtbCur->pMeta = pMeta; pCtbCur->suid = uid; - if (lock) { - metaRLock(pMeta); - } + pCtbCur->lock = lock; + pCtbCur->paused = 1; - ret = tdbTbcOpen(pMeta->pCtbIdx, (TBC**)&pCtbCur->pCur, NULL); + ret = metaResumeCtbCursor(pCtbCur, 1); if (ret < 0) { - metaULock(pMeta); - taosMemoryFree(pCtbCur); return NULL; } - - // move to the suid - ctbIdxKey.suid = uid; - ctbIdxKey.uid = INT64_MIN; - tdbTbcMoveTo(pCtbCur->pCur, &ctbIdxKey, sizeof(ctbIdxKey), &c); - if (c > 0) { - tdbTbcMoveToNext(pCtbCur->pCur); - } - return pCtbCur; } -void metaCloseCtbCursor(SMCtbCursor *pCtbCur, int lock) { +void metaCloseCtbCursor(SMCtbCursor *pCtbCur) { if (pCtbCur) { - if (pCtbCur->pMeta && lock) metaULock(pCtbCur->pMeta); - if (pCtbCur->pCur) { - tdbTbcClose(pCtbCur->pCur); + if (!pCtbCur->paused) { + if (pCtbCur->pMeta && pCtbCur->lock) metaULock(pCtbCur->pMeta); + if (pCtbCur->pCur) { + tdbTbcClose(pCtbCur->pCur); + } + } + tdbFree(pCtbCur->pKey); + tdbFree(pCtbCur->pVal); + } + taosMemoryFree(pCtbCur); +} - tdbFree(pCtbCur->pKey); - tdbFree(pCtbCur->pVal); +void metaPauseCtbCursor(SMCtbCursor* pCtbCur) { + if (!pCtbCur->paused) { + tdbTbcClose((TBC*)pCtbCur->pCur); + if (pCtbCur->lock) { + metaULock(pCtbCur->pMeta); + } + pCtbCur->paused = 1; + } +} + +int32_t metaResumeCtbCursor(SMCtbCursor* pCtbCur, int8_t first) { + if (pCtbCur->paused) { + pCtbCur->paused = 0; + + if (pCtbCur->lock) { + metaRLock(pCtbCur->pMeta); + } + int ret = 0; + ret = tdbTbcOpen(pCtbCur->pMeta->pCtbIdx, (TBC**)&pCtbCur->pCur, NULL); + if (ret < 0) { + metaCloseCtbCursor(pCtbCur); + return -1; } - taosMemoryFree(pCtbCur); + if (first) { + SCtbIdxKey ctbIdxKey; + // move to the suid + ctbIdxKey.suid = pCtbCur->suid; + ctbIdxKey.uid = INT64_MIN; + int c = 0; + tdbTbcMoveTo(pCtbCur->pCur, &ctbIdxKey, sizeof(ctbIdxKey), &c); + if (c > 0) { + tdbTbcMoveToNext(pCtbCur->pCur); + } + } else { + int c = 0; + ret = tdbTbcMoveTo(pCtbCur->pCur, pCtbCur->pKey, pCtbCur->kLen, &c); + if (c < 0) { + tdbTbcMoveToPrev(pCtbCur->pCur); + } else { + tdbTbcMoveToNext(pCtbCur->pCur); + } + } } + return 0; } tb_uid_t metaCtbCursorNext(SMCtbCursor *pCtbCur) { @@ -1414,7 +1449,7 @@ int32_t metaGetTableTags(void *pVnode, uint64_t suid, SArray *pUidTagInfo) { } taosHashCleanup(pSepecifiedUidMap); - metaCloseCtbCursor(pCur, 1); + metaCloseCtbCursor(pCur); return TSDB_CODE_SUCCESS; } diff --git a/source/dnode/vnode/src/tq/tqSink.c b/source/dnode/vnode/src/tq/tqSink.c index 5e62f7f116..6acc5b7457 100644 --- a/source/dnode/vnode/src/tq/tqSink.c +++ b/source/dnode/vnode/src/tq/tqSink.c @@ -414,7 +414,7 @@ void tqSinkToTablePipeline(SStreamTask* pTask, void* vnode, int64_t ver, void* d if (k == 0) { SColumnInfoData* pColData = taosArrayGet(pDataBlock->pDataBlock, dataIndex); void* colData = colDataGetData(pColData, j); - tqTrace("tq sink pipe2, row %d, col %d ts %" PRId64, j, k, *(int64_t*)colData); + tqDebug("tq sink pipe2, row %d, col %d ts %" PRId64, j, k, *(int64_t*)colData); } if (IS_SET_NULL(pCol)) { SColVal cv = COL_VAL_NULL(pCol->colId, pCol->type); diff --git a/source/dnode/vnode/src/vnd/vnodeInitApi.c b/source/dnode/vnode/src/vnd/vnodeInitApi.c index dca8dd271c..c72ecd4824 100644 --- a/source/dnode/vnode/src/vnd/vnodeInitApi.c +++ b/source/dnode/vnode/src/vnd/vnodeInitApi.c @@ -98,6 +98,8 @@ void initMetadataAPI(SStoreMeta* pMeta) { pMeta->metaPutTbGroupToCache = metaPutTbGroupToCache; pMeta->openCtbCursor = metaOpenCtbCursor; + pMeta->resumeCtbCursor = metaResumeCtbCursor; + pMeta->pauseCtbCursor = metaPauseCtbCursor; pMeta->closeCtbCursor = metaCloseCtbCursor; pMeta->ctbCursorNext = metaCtbCursorNext; } diff --git a/source/dnode/vnode/src/vnd/vnodeQuery.c b/source/dnode/vnode/src/vnd/vnodeQuery.c index 48f8ec021d..70663a6375 100644 --- a/source/dnode/vnode/src/vnd/vnodeQuery.c +++ b/source/dnode/vnode/src/vnd/vnodeQuery.c @@ -216,7 +216,7 @@ int vnodeGetTableCfg(SVnode *pVnode, SRpcMsg *pMsg, bool direct) { cfgRsp.numOfTags = schemaTag.nCols; cfgRsp.numOfColumns = schema.nCols; - cfgRsp.pSchemas = (SSchema *)taosMemoryMalloc(sizeof(SSchema) * (cfgRsp.numOfColumns + cfgRsp.numOfTags)); + cfgRsp.pSchemas = (SSchema *)taosMemoryCalloc(cfgRsp.numOfColumns + cfgRsp.numOfTags, sizeof(SSchema)); memcpy(cfgRsp.pSchemas, schema.pSchema, sizeof(SSchema) * schema.nCols); if (schemaTag.nCols) { @@ -380,6 +380,9 @@ int32_t vnodeGetLoad(SVnode *pVnode, SVnodeLoad *pLoad) { pLoad->vgId = TD_VID(pVnode); pLoad->syncState = state.state; pLoad->syncRestore = state.restored; + pLoad->syncTerm = state.term; + pLoad->roleTimeMs = state.roleTimeMs; + pLoad->startTimeMs = state.startTimeMs; pLoad->syncCanRead = state.canRead; pLoad->cacheUsage = tsdbCacheGetUsage(pVnode); pLoad->numOfCachedTables = tsdbCacheGetElems(pVnode); @@ -452,7 +455,7 @@ int32_t vnodeGetAllTableList(SVnode *pVnode, uint64_t uid, SArray *list) { taosArrayPush(list, &info); } - metaCloseCtbCursor(pCur, 1); + metaCloseCtbCursor(pCur); return TSDB_CODE_SUCCESS; } @@ -473,7 +476,7 @@ int32_t vnodeGetCtbIdList(void *pVnode, int64_t suid, SArray *list) { taosArrayPush(list, &id); } - metaCloseCtbCursor(pCur, 1); + metaCloseCtbCursor(pCur); return TSDB_CODE_SUCCESS; } @@ -536,7 +539,7 @@ int32_t vnodeGetCtbNum(SVnode *pVnode, int64_t suid, int64_t *num) { ++(*num); } - metaCloseCtbCursor(pCur, 0); + metaCloseCtbCursor(pCur); return TSDB_CODE_SUCCESS; } diff --git a/source/libs/command/src/command.c b/source/libs/command/src/command.c index 921ec41021..1fa97904e4 100644 --- a/source/libs/command/src/command.c +++ b/source/libs/command/src/command.c @@ -78,6 +78,10 @@ static int32_t buildDescResultDataBlock(SSDataBlock** pOutput) { infoData = createColumnInfoData(TSDB_DATA_TYPE_VARCHAR, DESCRIBE_RESULT_NOTE_LEN, 4); code = blockDataAppendColInfo(pBlock, &infoData); } + if (TSDB_CODE_SUCCESS == code) { + infoData = createColumnInfoData(TSDB_DATA_TYPE_VARCHAR, DESCRIBE_RESULT_COL_COMMENT_LEN, 5); + code = blockDataAppendColInfo(pBlock, &infoData); + } if (TSDB_CODE_SUCCESS == code) { *pOutput = pBlock; @@ -99,7 +103,9 @@ static int32_t setDescResultIntoDataBlock(bool sysInfoUser, SSDataBlock* pBlock, SColumnInfoData* pCol3 = taosArrayGet(pBlock->pDataBlock, 2); // Note SColumnInfoData* pCol4 = taosArrayGet(pBlock->pDataBlock, 3); - char buf[DESCRIBE_RESULT_FIELD_LEN] = {0}; + // Comment + SColumnInfoData* pCol5 = taosArrayGet(pBlock->pDataBlock, 4); + char buf[DESCRIBE_RESULT_COL_COMMENT_LEN + VARSTR_HEADER_SIZE] = {0}; for (int32_t i = 0; i < numOfRows; ++i) { if (invisibleColumn(sysInfoUser, pMeta->tableType, pMeta->schema[i].flags)) { continue; @@ -112,6 +118,8 @@ static int32_t setDescResultIntoDataBlock(bool sysInfoUser, SSDataBlock* pBlock, colDataSetVal(pCol3, pBlock->info.rows, (const char*)&bytes, false); STR_TO_VARSTR(buf, i >= pMeta->tableInfo.numOfColumns ? "TAG" : ""); colDataSetVal(pCol4, pBlock->info.rows, buf, false); + STR_TO_VARSTR(buf, pMeta->schema[i].comment); + colDataSetVal(pCol5, pBlock->info.rows, buf, false); ++(pBlock->info.rows); } if (pBlock->info.rows <= 0) { @@ -456,14 +464,19 @@ void appendColumnFields(char* buf, int32_t* len, STableCfg* pCfg) { for (int32_t i = 0; i < pCfg->numOfColumns; ++i) { SSchema* pSchema = pCfg->pSchemas + i; char type[32]; + char comments[TSDB_COL_COMMENT_LEN + 16] = {0}; sprintf(type, "%s", tDataTypes[pSchema->type].name); if (TSDB_DATA_TYPE_VARCHAR == pSchema->type || TSDB_DATA_TYPE_GEOMETRY == pSchema->type) { sprintf(type + strlen(type), "(%d)", (int32_t)(pSchema->bytes - VARSTR_HEADER_SIZE)); } else if (TSDB_DATA_TYPE_NCHAR == pSchema->type) { sprintf(type + strlen(type), "(%d)", (int32_t)((pSchema->bytes - VARSTR_HEADER_SIZE) / TSDB_NCHAR_SIZE)); } + if (pSchema->comment[0]) { + sprintf(comments, " COMMENT '%s'", pSchema->comment); + } - *len += sprintf(buf + VARSTR_HEADER_SIZE + *len, "%s`%s` %s", ((i > 0) ? ", " : ""), pSchema->name, type); + *len += + sprintf(buf + VARSTR_HEADER_SIZE + *len, "%s`%s` %s%s", ((i > 0) ? ", " : ""), pSchema->name, type, comments); } } @@ -471,14 +484,18 @@ void appendTagFields(char* buf, int32_t* len, STableCfg* pCfg) { for (int32_t i = 0; i < pCfg->numOfTags; ++i) { SSchema* pSchema = pCfg->pSchemas + pCfg->numOfColumns + i; char type[32]; + char comments[TSDB_COL_COMMENT_LEN + 16] = {0}; sprintf(type, "%s", tDataTypes[pSchema->type].name); if (TSDB_DATA_TYPE_VARCHAR == pSchema->type || TSDB_DATA_TYPE_GEOMETRY == pSchema->type) { sprintf(type + strlen(type), "(%d)", (int32_t)(pSchema->bytes - VARSTR_HEADER_SIZE)); } else if (TSDB_DATA_TYPE_NCHAR == pSchema->type) { sprintf(type + strlen(type), "(%d)", (int32_t)((pSchema->bytes - VARSTR_HEADER_SIZE) / TSDB_NCHAR_SIZE)); } + if (pSchema->comment[0]) { + sprintf(comments, " COMMENT '%s'", pSchema->comment); + } - *len += sprintf(buf + VARSTR_HEADER_SIZE + *len, "%s`%s` %s", ((i > 0) ? ", " : ""), pSchema->name, type); + *len += sprintf(buf + VARSTR_HEADER_SIZE + *len, "%s`%s` %s%s", ((i > 0) ? ", " : ""), pSchema->name, type, comments); } } diff --git a/source/libs/executor/inc/executil.h b/source/libs/executor/inc/executil.h index f273f63770..2622903121 100644 --- a/source/libs/executor/inc/executil.h +++ b/source/libs/executor/inc/executil.h @@ -117,7 +117,7 @@ void* tableListDestroy(STableListInfo* pTableListInfo); void tableListClear(STableListInfo* pTableListInfo); int32_t tableListGetOutputGroups(const STableListInfo* pTableList); bool oneTableForEachGroup(const STableListInfo* pTableList); -uint64_t getTableGroupId(const STableListInfo* pTableList, uint64_t tableUid); +uint64_t tableListGetTableGroupId(const STableListInfo* pTableList, uint64_t tableUid); int32_t tableListAddTableInfo(STableListInfo* pTableList, uint64_t uid, uint64_t gid); int32_t tableListGetGroupList(const STableListInfo* pTableList, int32_t ordinalIndex, STableKeyInfo** pKeyInfo, int32_t* num); @@ -183,13 +183,17 @@ void cleanupQueryTableDataCond(SQueryTableDataCond* pCond); int32_t convertFillType(int32_t mode); int32_t resultrowComparAsc(const void* p1, const void* p2); -int32_t isQualifiedTable(STableKeyInfo* info, SNode* pTagCond, void* metaHandle, bool* pQualified, SStorageAPI *pAPI); - -void printDataBlock(SSDataBlock* pBlock, const char* flag); +int32_t isQualifiedTable(STableKeyInfo* info, SNode* pTagCond, void* metaHandle, bool* pQualified, SStorageAPI* pAPI); +char* getStreamOpName(uint16_t opType); +void printDataBlock(SSDataBlock* pBlock, const char* flag, const char* taskIdStr); +void printSpecDataBlock(SSDataBlock* pBlock, const char* flag, const char* opStr, const char* taskIdStr); void getNextTimeWindow(const SInterval* pInterval, STimeWindow* tw, int32_t order); void getInitialStartTimeWindow(SInterval* pInterval, TSKEY ts, STimeWindow* w, bool ascQuery); +TSKEY getStartTsKey(STimeWindow* win, const TSKEY* tsCols); +void updateTimeWindowInfo(SColumnInfoData* pColData, STimeWindow* pWin, int64_t delta); + SSDataBlock* createTagValBlockForFilter(SArray* pColList, int32_t numOfTables, SArray* pUidTagList, void* pVnode, SStorageAPI* pStorageAPI); #endif // TDENGINE_EXECUTIL_H diff --git a/source/libs/executor/inc/executorInt.h b/source/libs/executor/inc/executorInt.h index 43ccf3feb4..00b47e343f 100644 --- a/source/libs/executor/inc/executorInt.h +++ b/source/libs/executor/inc/executorInt.h @@ -21,11 +21,11 @@ extern "C" { #include "os.h" #include "tcommon.h" +#include "theap.h" #include "tlosertree.h" #include "tsort.h" #include "ttszip.h" #include "tvariant.h" -#include "theap.h" #include "dataSinkMgt.h" #include "executil.h" @@ -39,14 +39,14 @@ extern "C" { #include "tlockfree.h" #include "tmsg.h" #include "tpagedbuf.h" -//#include "tstream.h" -//#include "tstreamUpdate.h" +// #include "tstream.h" +// #include "tstreamUpdate.h" #include "tlrucache.h" typedef int32_t (*__block_search_fn_t)(char* data, int32_t num, int64_t key, int32_t order); typedef struct STsdbReader STsdbReader; -typedef struct STqReader STqReader; +typedef struct STqReader STqReader; #define IS_VALID_SESSION_WIN(winInfo) ((winInfo).sessionWin.win.skey > 0) #define SET_SESSION_WIN_INVALID(winInfo) ((winInfo).sessionWin.win.skey = INT64_MIN) @@ -208,7 +208,7 @@ typedef struct STableScanBase { SLimitInfo limitInfo; // there are more than one table list exists in one task, if only one vnode exists. STableListInfo* pTableListInfo; - TsdReader readerAPI; + TsdReader readerAPI; } STableScanBase; typedef struct STableScanInfo { @@ -224,7 +224,7 @@ typedef struct STableScanInfo { int8_t assignBlockUid; bool hasGroupByTag; bool countOnly; -// TsdReader readerAPI; + // TsdReader readerAPI; } STableScanInfo; typedef struct STableMergeScanInfo { @@ -258,21 +258,21 @@ typedef struct STagScanFilterContext { } STagScanFilterContext; typedef struct STagScanInfo { - SColumnInfo* pCols; - SSDataBlock* pRes; - SColMatchInfo matchInfo; - int32_t curPos; - SLimitNode* pSlimit; - SReadHandle readHandle; - STableListInfo* pTableListInfo; - uint64_t suid; - void* pCtbCursor; - SNode* pTagCond; - SNode* pTagIndexCond; + SColumnInfo* pCols; + SSDataBlock* pRes; + SColMatchInfo matchInfo; + int32_t curPos; + SLimitNode* pSlimit; + SReadHandle readHandle; + STableListInfo* pTableListInfo; + uint64_t suid; + void* pCtbCursor; + SNode* pTagCond; + SNode* pTagIndexCond; STagScanFilterContext filterCtx; - SArray* aUidTags; // SArray - SArray* aFilterIdxs; // SArray - SStorageAPI* pStorageAPI; + SArray* aUidTags; // SArray + SArray* aFilterIdxs; // SArray + SStorageAPI* pStorageAPI; } STagScanInfo; typedef enum EStreamScanMode { @@ -342,12 +342,12 @@ typedef struct SStreamScanInfo { SExprSupp tagCalSup; int32_t primaryTsIndex; // primary time stamp slot id SReadHandle readHandle; - SInterval interval; // if the upstream is an interval operator, the interval info is also kept here. + SInterval interval; // if the upstream is an interval operator, the interval info is also kept here. SColMatchInfo matchInfo; - SArray* pBlockLists; // multiple SSDatablock. - SSDataBlock* pRes; // result SSDataBlock - SSDataBlock* pUpdateRes; // update SSDataBlock + SArray* pBlockLists; // multiple SSDatablock. + SSDataBlock* pRes; // result SSDataBlock + SSDataBlock* pUpdateRes; // update SSDataBlock int32_t updateResIndex; int32_t blockType; // current block type int32_t validBlockIndex; // Is current data has returned? @@ -434,7 +434,7 @@ typedef struct SIntervalAggOperatorInfo { int64_t limit; bool slimited; int64_t slimit; - uint64_t curGroupId; // initialize to UINT64_MAX + uint64_t curGroupId; // initialize to UINT64_MAX uint64_t handledGroupNum; BoundedQueue* pBQ; } SIntervalAggOperatorInfo; @@ -450,7 +450,7 @@ typedef struct SMergeAlignedIntervalAggOperatorInfo { typedef struct SOpCheckPointInfo { uint16_t checkPointId; - SHashObj* children; // key:child id + SHashObj* children; // key:child id } SOpCheckPointInfo; typedef struct SStreamIntervalOperatorInfo { @@ -472,10 +472,9 @@ typedef struct SStreamIntervalOperatorInfo { SArray* pPullWins; // SPullWindowInfo int32_t pullIndex; SSDataBlock* pPullDataRes; - bool isFinal; SArray* pChildren; int32_t numOfChild; - SStreamState* pState; // void + SStreamState* pState; // void SWinKey delKey; uint64_t numOfDatapack; SArray* pUpdated; @@ -526,7 +525,6 @@ typedef struct SStreamSessionAggOperatorInfo { void* pDelIterator; SArray* pChildren; // cache for children's result; final stream operator SPhysiNode* pPhyNode; // create new child - bool isFinal; bool ignoreExpiredData; bool ignoreExpiredDataSaved; SArray* pUpdated; @@ -612,8 +610,9 @@ typedef struct SStreamFillOperatorInfo { #define OPTR_SET_OPENED(_optr) ((_optr)->status |= OP_OPENED) SSchemaWrapper* extractQueriedColumnSchema(SScanPhysiNode* pScanNode); -int32_t initQueriedTableSchemaInfo(SReadHandle* pHandle, SScanPhysiNode* pScanNode, const char* dbName, SExecTaskInfo* pTaskInfo); -void cleanupQueriedTableScanInfo(SSchemaInfo* pSchemaInfo); +int32_t initQueriedTableSchemaInfo(SReadHandle* pHandle, SScanPhysiNode* pScanNode, const char* dbName, + SExecTaskInfo* pTaskInfo); +void cleanupQueriedTableScanInfo(SSchemaInfo* pSchemaInfo); void initBasicInfo(SOptrBasicInfo* pInfo, SSDataBlock* pBlock); void cleanupBasicInfo(SOptrBasicInfo* pInfo); @@ -684,7 +683,8 @@ bool isInTimeWindow(STimeWindow* pWin, TSKEY ts, int64_t gap); bool functionNeedToExecute(SqlFunctionCtx* pCtx); bool isOverdue(TSKEY ts, STimeWindowAggSupp* pSup); bool isCloseWindow(STimeWindow* pWin, STimeWindowAggSupp* pSup); -bool isDeletedStreamWindow(STimeWindow* pWin, uint64_t groupId, void* pState, STimeWindowAggSupp* pTwSup, SStateStore* pStore); +bool isDeletedStreamWindow(STimeWindow* pWin, uint64_t groupId, void* pState, STimeWindowAggSupp* pTwSup, + SStateStore* pStore); void appendOneRowToStreamSpecialBlock(SSDataBlock* pBlock, TSKEY* pStartTs, TSKEY* pEndTs, uint64_t* pUid, uint64_t* pGp, void* pTbName); uint64_t calGroupIdByData(SPartitionBySupporter* pParSup, SExprSupp* pExprSup, SSDataBlock* pBlock, int32_t rowId); @@ -696,8 +696,8 @@ bool groupbyTbname(SNodeList* pGroupList); int32_t buildDataBlockFromGroupRes(struct SOperatorInfo* pOperator, void* pState, SSDataBlock* pBlock, SExprSupp* pSup, SGroupResInfo* pGroupResInfo); int32_t saveSessionDiscBuf(void* pState, SSessionKey* key, void* buf, int32_t size, SStateStore* pAPI); -int32_t buildSessionResultDataBlock(struct SOperatorInfo* pOperator, void* pState, SSDataBlock* pBlock, - SExprSupp* pSup, SGroupResInfo* pGroupResInfo); +int32_t buildSessionResultDataBlock(struct SOperatorInfo* pOperator, void* pState, SSDataBlock* pBlock, SExprSupp* pSup, + SGroupResInfo* pGroupResInfo); int32_t releaseOutputBuf(void* pState, SWinKey* pKey, SResultRow* pResult, SStateStore* pAPI); void getNextIntervalWindow(SInterval* pInterval, STimeWindow* tw, int32_t order); int32_t getForwardStepsInBlock(int32_t numOfRows, __block_search_fn_t searchFn, TSKEY ekey, int32_t pos, int32_t order, @@ -715,11 +715,17 @@ void doUpdateNumOfRows(SqlFunctionCtx* pCtx, SResultRow* pRow, int32_t numOfExpr void doClearBufferedBlocks(SStreamScanInfo* pInfo); uint64_t calcGroupId(char* pData, int32_t len); -void streamOpReleaseState(struct SOperatorInfo* pOperator); -void streamOpReloadState(struct SOperatorInfo* pOperator); +void streamOpReleaseState(struct SOperatorInfo* pOperator); +void streamOpReloadState(struct SOperatorInfo* pOperator); -int32_t encodeSTimeWindowAggSupp(void **buf, STimeWindowAggSupp* pTwAggSup); -void* decodeSTimeWindowAggSupp(void *buf, STimeWindowAggSupp* pTwAggSup); +int32_t encodeSTimeWindowAggSupp(void** buf, STimeWindowAggSupp* pTwAggSup); +void* decodeSTimeWindowAggSupp(void* buf, STimeWindowAggSupp* pTwAggSup); +bool inSlidingWindow(SInterval* pInterval, STimeWindow* pWin, SDataBlockInfo* pBlockInfo); +bool inCalSlidingWindow(SInterval* pInterval, STimeWindow* pWin, TSKEY calStart, TSKEY calEnd, EStreamType blockType); +bool compareVal(const char* v, const SStateKeys* pKey); + +int32_t getNextQualifiedWindow(SInterval* pInterval, STimeWindow* pNext, SDataBlockInfo* pDataBlockInfo, + TSKEY* primaryKeys, int32_t prevPosition, int32_t order); #ifdef __cplusplus } diff --git a/source/libs/executor/src/cachescanoperator.c b/source/libs/executor/src/cachescanoperator.c index ce39ebab59..a07a9d5dbd 100644 --- a/source/libs/executor/src/cachescanoperator.c +++ b/source/libs/executor/src/cachescanoperator.c @@ -218,7 +218,7 @@ SSDataBlock* doScanCache(SOperatorInfo* pOperator) { return NULL; } - pRes->info.id.groupId = getTableGroupId(pTableList, pRes->info.id.uid); + pRes->info.id.groupId = tableListGetTableGroupId(pTableList, pRes->info.id.uid); pInfo->indexOfBufferedRes += 1; return pRes; } else { diff --git a/source/libs/executor/src/eventwindowoperator.c b/source/libs/executor/src/eventwindowoperator.c index bbdc50183e..5c335c116f 100644 --- a/source/libs/executor/src/eventwindowoperator.c +++ b/source/libs/executor/src/eventwindowoperator.c @@ -58,16 +58,6 @@ static void doKeepTuple(SWindowRowsSup* pRowSup, int64_t ts, uint64_t groupId) { pRowSup->groupId = groupId; } -static void updateTimeWindowInfo(SColumnInfoData* pColData, STimeWindow* pWin, bool includeEndpoint) { - int64_t* ts = (int64_t*)pColData->pData; - int32_t delta = includeEndpoint ? 1 : 0; - - int64_t duration = pWin->ekey - pWin->skey + delta; - ts[2] = duration; // set the duration - ts[3] = pWin->skey; // window start key - ts[4] = pWin->ekey + delta; // window end key -} - SOperatorInfo* createEventwindowOperatorInfo(SOperatorInfo* downstream, SPhysiNode* physiNode, SExecTaskInfo* pTaskInfo) { SEventWindowOperatorInfo* pInfo = taosMemoryCalloc(1, sizeof(SEventWindowOperatorInfo)); @@ -250,7 +240,7 @@ static void doEventWindowAggImpl(SEventWindowOperatorInfo* pInfo, SExprSupp* pSu T_LONG_JMP(pTaskInfo->env, TSDB_CODE_APP_ERROR); } - updateTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &pRowSup->win, false); + updateTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &pRowSup->win, 0); applyAggFunctionOnPartialTuples(pTaskInfo, pSup->pCtx, &pInfo->twAggSup.timeWindowData, startIndex, numOfRows, pBlock->info.rows, numOfOutput); } diff --git a/source/libs/executor/src/executil.c b/source/libs/executor/src/executil.c index 4b3e223f38..c4f03c5779 100644 --- a/source/libs/executor/src/executil.c +++ b/source/libs/executor/src/executil.c @@ -1931,7 +1931,7 @@ void tableListGetSourceTableInfo(const STableListInfo* pTableList, uint64_t* psu *type = pTableList->idInfo.tableType; } -uint64_t getTableGroupId(const STableListInfo* pTableList, uint64_t tableUid) { +uint64_t tableListGetTableGroupId(const STableListInfo* pTableList, uint64_t tableUid) { int32_t* slot = taosHashGet(pTableList->map, &tableUid, sizeof(tableUid)); ASSERT(pTableList->map != NULL && slot != NULL); @@ -2177,12 +2177,67 @@ int32_t createScanTableListInfo(SScanPhysiNode* pScanNode, SNodeList* pGroupTags return TSDB_CODE_SUCCESS; } -void printDataBlock(SSDataBlock* pBlock, const char* flag) { +char* getStreamOpName(uint16_t opType) { + switch (opType) { + case QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN: + return "stream scan"; + case QUERY_NODE_PHYSICAL_PLAN_PROJECT: + return "project"; + case QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL: + return "interval single"; + case QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL: + return "interval final"; + case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL: + return "interval semi"; + case QUERY_NODE_PHYSICAL_PLAN_STREAM_FILL: + return "stream fill"; + case QUERY_NODE_PHYSICAL_PLAN_STREAM_SESSION: + return "session single"; + case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_SESSION: + return "session semi"; + case QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_SESSION: + return "session final"; + case QUERY_NODE_PHYSICAL_PLAN_STREAM_STATE: + return "state single"; + case QUERY_NODE_PHYSICAL_PLAN_STREAM_PARTITION: + return "stream partitionby"; + case QUERY_NODE_PHYSICAL_PLAN_STREAM_EVENT: + return "stream event"; + } + return ""; +} + +void printDataBlock(SSDataBlock* pBlock, const char* flag, const char* taskIdStr) { if (!pBlock || pBlock->info.rows == 0) { - qDebug("===stream===%s: Block is Null or Empty", flag); + qDebug("%s===stream===%s: Block is Null or Empty", taskIdStr, flag); return; } char* pBuf = NULL; - qDebug("%s", dumpBlockData(pBlock, flag, &pBuf)); + qDebug("%s", dumpBlockData(pBlock, flag, &pBuf, taskIdStr)); taosMemoryFree(pBuf); } + +void printSpecDataBlock(SSDataBlock* pBlock, const char* flag, const char* opStr, const char* taskIdStr) { + if (!pBlock || pBlock->info.rows == 0) { + qDebug("%s===stream===%s: Block is Null or Empty", taskIdStr, flag); + return; + } + if (qDebugFlag & DEBUG_DEBUG) { + char* pBuf = NULL; + char flagBuf[64]; + snprintf(flagBuf, sizeof(flagBuf), "%s %s", flag, opStr); + qDebug("%s", dumpBlockData(pBlock, flagBuf, &pBuf, taskIdStr)); + taosMemoryFree(pBuf); + } +} + +TSKEY getStartTsKey(STimeWindow* win, const TSKEY* tsCols) { return tsCols == NULL ? win->skey : tsCols[0]; } + +void updateTimeWindowInfo(SColumnInfoData* pColData, STimeWindow* pWin, int64_t delta) { + int64_t* ts = (int64_t*)pColData->pData; + + int64_t duration = pWin->ekey - pWin->skey + delta; + ts[2] = duration; // set the duration + ts[3] = pWin->skey; // window start key + ts[4] = pWin->ekey + delta; // window end key +} diff --git a/source/libs/executor/src/executorInt.c b/source/libs/executor/src/executorInt.c index ebec9aa94e..a7f0e02815 100644 --- a/source/libs/executor/src/executorInt.c +++ b/source/libs/executor/src/executorInt.c @@ -1070,3 +1070,15 @@ void streamOpReloadState(SOperatorInfo* pOperator) { downstream->fpSet.reloadStreamStateFn(downstream); } } + +bool compareVal(const char* v, const SStateKeys* pKey) { + if (IS_VAR_DATA_TYPE(pKey->type)) { + if (varDataLen(v) != varDataLen(pKey->pData)) { + return false; + } else { + return memcmp(varDataVal(v), varDataVal(pKey->pData), varDataLen(v)) == 0; + } + } else { + return memcmp(pKey->pData, v, pKey->bytes) == 0; + } +} diff --git a/source/libs/executor/src/filloperator.c b/source/libs/executor/src/filloperator.c index b7152de1fd..bf321b75b1 100644 --- a/source/libs/executor/src/filloperator.c +++ b/source/libs/executor/src/filloperator.c @@ -1292,14 +1292,14 @@ static SSDataBlock* doStreamFill(SOperatorInfo* pOperator) { (pInfo->pFillInfo->pos != FILL_POS_INVALID && pInfo->pFillInfo->needFill == true)) { doStreamFillRange(pInfo->pFillInfo, pInfo->pFillSup, pInfo->pRes); if (pInfo->pRes->info.rows > 0) { - printDataBlock(pInfo->pRes, "stream fill"); + printDataBlock(pInfo->pRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); return pInfo->pRes; } } if (pOperator->status == OP_RES_TO_RETURN) { doDeleteFillFinalize(pOperator); if (pInfo->pRes->info.rows > 0) { - printDataBlock(pInfo->pRes, "stream fill"); + printDataBlock(pInfo->pRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); return pInfo->pRes; } setOperatorCompleted(pOperator); @@ -1317,12 +1317,12 @@ static SSDataBlock* doStreamFill(SOperatorInfo* pOperator) { pOperator->status = OP_RES_TO_RETURN; pInfo->pFillInfo->preRowKey = INT64_MIN; if (pInfo->pRes->info.rows > 0) { - printDataBlock(pInfo->pRes, "stream fill"); + printDataBlock(pInfo->pRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); return pInfo->pRes; } break; } - printDataBlock(pBlock, "stream fill recv"); + printSpecDataBlock(pBlock, getStreamOpName(pOperator->operatorType), "recv", GET_TASKID(pTaskInfo)); if (pInfo->pFillInfo->curGroupId != pBlock->info.id.groupId) { pInfo->pFillInfo->curGroupId = pBlock->info.id.groupId; @@ -1339,7 +1339,7 @@ static SSDataBlock* doStreamFill(SOperatorInfo* pOperator) { pInfo->pFillSup->hasDelete = true; doDeleteFillResult(pOperator); if (pInfo->pDelRes->info.rows > 0) { - printDataBlock(pInfo->pDelRes, "stream fill delete"); + printDataBlock(pInfo->pDelRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); return pInfo->pDelRes; } continue; @@ -1379,7 +1379,7 @@ static SSDataBlock* doStreamFill(SOperatorInfo* pOperator) { } pOperator->resultInfo.totalRows += pInfo->pRes->info.rows; - printDataBlock(pInfo->pRes, "stream fill"); + printDataBlock(pInfo->pRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); return pInfo->pRes; } diff --git a/source/libs/executor/src/groupoperator.c b/source/libs/executor/src/groupoperator.c index 0eb20ad64e..0a14d6ea09 100644 --- a/source/libs/executor/src/groupoperator.c +++ b/source/libs/executor/src/groupoperator.c @@ -956,7 +956,8 @@ static bool hasRemainPartion(SStreamPartitionOperatorInfo* pInfo) { return pInfo static bool hasRemainTbName(SStreamPartitionOperatorInfo* pInfo) { return pInfo->pTbNameIte != NULL; } static SSDataBlock* buildStreamPartitionResult(SOperatorInfo* pOperator) { - SStorageAPI* pAPI = &pOperator->pTaskInfo->storageAPI; + SStorageAPI* pAPI = &pOperator->pTaskInfo->storageAPI; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; SStreamPartitionOperatorInfo* pInfo = pOperator->info; SSDataBlock* pDest = pInfo->binfo.pRes; @@ -994,7 +995,7 @@ static SSDataBlock* buildStreamPartitionResult(SOperatorInfo* pOperator) { pOperator->resultInfo.totalRows += pDest->info.rows; pInfo->parIte = taosHashIterate(pInfo->pPartitions, pInfo->parIte); ASSERT(pDest->info.rows > 0); - printDataBlock(pDest, "stream partitionby"); + printDataBlock(pDest, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); return pDest; } @@ -1115,7 +1116,7 @@ static SSDataBlock* doStreamHashPartition(SOperatorInfo* pOperator) { setOperatorCompleted(pOperator); return NULL; } - printDataBlock(pBlock, "stream partitionby recv"); + printSpecDataBlock(pBlock, getStreamOpName(pOperator->operatorType), "recv", GET_TASKID(pTaskInfo)); switch (pBlock->info.type) { case STREAM_NORMAL: case STREAM_PULL_DATA: @@ -1125,7 +1126,7 @@ static SSDataBlock* doStreamHashPartition(SOperatorInfo* pOperator) { case STREAM_DELETE_DATA: { copyDataBlock(pInfo->pDelRes, pBlock); pInfo->pDelRes->info.type = STREAM_DELETE_RESULT; - printDataBlock(pInfo->pDelRes, "stream partitionby delete"); + printDataBlock(pInfo->pDelRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); return pInfo->pDelRes; } break; case STREAM_CREATE_CHILD_TABLE: diff --git a/source/libs/executor/src/projectoperator.c b/source/libs/executor/src/projectoperator.c index 1cc377b3ee..8e31bc042f 100644 --- a/source/libs/executor/src/projectoperator.c +++ b/source/libs/executor/src/projectoperator.c @@ -372,6 +372,10 @@ SSDataBlock* doProjectOperation(SOperatorInfo* pOperator) { pOperator->cost.openCost = (taosGetTimestampUs() - st) / 1000.0; } + if (pTaskInfo->execModel == OPTR_EXEC_MODEL_STREAM) { + printDataBlock(p, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); + } + return (p->info.rows > 0) ? p : NULL; } diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index d346e0b402..5e9d047b47 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -694,7 +694,7 @@ static SSDataBlock* doTableScanImpl(SOperatorInfo* pOperator) { } if (pBlock->info.id.uid) { - pBlock->info.id.groupId = getTableGroupId(pTableScanInfo->base.pTableListInfo, pBlock->info.id.uid); + pBlock->info.id.groupId = tableListGetTableGroupId(pTableScanInfo->base.pTableListInfo, pBlock->info.id.uid); } uint32_t status = 0; @@ -1089,7 +1089,7 @@ static SSDataBlock* readPreVersionData(SOperatorInfo* pTableScanOp, uint64_t tbU if (hasNext) { /*SSDataBlock* p = */ pAPI->tsdReader.tsdReaderRetrieveDataBlock(pReader, NULL); doSetTagColumnData(&pTableScanInfo->base, pBlock, pTaskInfo, pBlock->info.rows); - pBlock->info.id.groupId = getTableGroupId(pTableScanInfo->base.pTableListInfo, pBlock->info.id.uid); + pBlock->info.id.groupId = tableListGetTableGroupId(pTableScanInfo->base.pTableListInfo, pBlock->info.id.uid); } pAPI->tsdReader.tsdReaderClose(pReader); @@ -1111,7 +1111,7 @@ static uint64_t getGroupIdByCol(SStreamScanInfo* pInfo, uint64_t uid, TSKEY ts, static uint64_t getGroupIdByUid(SStreamScanInfo* pInfo, uint64_t uid) { STableScanInfo* pTableScanInfo = pInfo->pTableScanOp->info; - return getTableGroupId(pTableScanInfo->base.pTableListInfo, uid); + return tableListGetTableGroupId(pTableScanInfo->base.pTableListInfo, uid); } static uint64_t getGroupIdByData(SStreamScanInfo* pInfo, uint64_t uid, TSKEY ts, int64_t maxVersion) { @@ -1344,7 +1344,7 @@ static int32_t generateIntervalScanRange(SStreamScanInfo* pInfo, SSDataBlock* pS if (rows == 0) { return TSDB_CODE_SUCCESS; } - + SExecTaskInfo* pTaskInfo = pInfo->pStreamScanOp->pTaskInfo; SColumnInfoData* pSrcStartTsCol = (SColumnInfoData*)taosArrayGet(pSrcBlock->pDataBlock, START_TS_COLUMN_INDEX); SColumnInfoData* pSrcEndTsCol = (SColumnInfoData*)taosArrayGet(pSrcBlock->pDataBlock, END_TS_COLUMN_INDEX); SColumnInfoData* pSrcUidCol = taosArrayGet(pSrcBlock->pDataBlock, UID_COLUMN_INDEX); @@ -1361,7 +1361,7 @@ static int32_t generateIntervalScanRange(SStreamScanInfo* pInfo, SSDataBlock* pS TSKEY startTs = srcStartTsCol[0]; TSKEY endTs = srcEndTsCol[0]; SSDataBlock* pPreRes = readPreVersionData(pInfo->pTableScanOp, srcUid, startTs, endTs, ver); - printDataBlock(pPreRes, "pre res"); + printDataBlock(pPreRes, "pre res", GET_TASKID(pTaskInfo)); blockDataCleanup(pSrcBlock); int32_t code = blockDataEnsureCapacity(pSrcBlock, pPreRes->info.rows); if (code != TSDB_CODE_SUCCESS) { @@ -1376,7 +1376,7 @@ static int32_t generateIntervalScanRange(SStreamScanInfo* pInfo, SSDataBlock* pS appendOneRowToStreamSpecialBlock(pSrcBlock, ((TSKEY*)pTsCol->pData) + i, ((TSKEY*)pTsCol->pData) + i, &srcUid, &groupId, NULL); } - printDataBlock(pSrcBlock, "new delete"); + printDataBlock(pSrcBlock, "new delete", GET_TASKID(pTaskInfo)); } uint64_t* srcGp = (uint64_t*)pSrcGpCol->pData; srcStartTsCol = (TSKEY*)pSrcStartTsCol->pData; @@ -1652,7 +1652,7 @@ static int32_t setBlockIntoRes(SStreamScanInfo* pInfo, const SSDataBlock* pBlock pBlockInfo->version = pBlock->info.version; STableScanInfo* pTableScanInfo = pInfo->pTableScanOp->info; - pBlockInfo->id.groupId = getTableGroupId(pTableScanInfo->base.pTableListInfo, pBlock->info.id.uid); + pBlockInfo->id.groupId = tableListGetTableGroupId(pTableScanInfo->base.pTableListInfo, pBlock->info.id.uid); // todo extract method for (int32_t i = 0; i < taosArrayGetSize(pInfo->matchInfo.pList); ++i) { @@ -1942,38 +1942,9 @@ static SSDataBlock* doStreamScan(SOperatorInfo* pOperator) { switch (pInfo->scanMode) { case STREAM_SCAN_FROM_RES: { pInfo->scanMode = STREAM_SCAN_FROM_READERHANDLE; - printDataBlock(pInfo->pRecoverRes, "scan recover"); + printSpecDataBlock(pInfo->pRecoverRes, getStreamOpName(pOperator->operatorType), "recover", GET_TASKID(pTaskInfo)); return pInfo->pRecoverRes; } break; - // case STREAM_SCAN_FROM_UPDATERES: { - // generateScanRange(pInfo, pInfo->pUpdateDataRes, pInfo->pUpdateRes); - // prepareRangeScan(pInfo, pInfo->pUpdateRes, &pInfo->updateResIndex); - // pInfo->scanMode = STREAM_SCAN_FROM_DATAREADER_RANGE; - // printDataBlock(pInfo->pUpdateRes, "recover update"); - // return pInfo->pUpdateRes; - // } break; - // case STREAM_SCAN_FROM_DELETE_DATA: { - // generateScanRange(pInfo, pInfo->pUpdateDataRes, pInfo->pUpdateRes); - // prepareRangeScan(pInfo, pInfo->pUpdateRes, &pInfo->updateResIndex); - // pInfo->scanMode = STREAM_SCAN_FROM_DATAREADER_RANGE; - // copyDataBlock(pInfo->pDeleteDataRes, pInfo->pUpdateRes); - // pInfo->pDeleteDataRes->info.type = STREAM_DELETE_DATA; - // printDataBlock(pInfo->pDeleteDataRes, "recover delete"); - // return pInfo->pDeleteDataRes; - // } break; - // case STREAM_SCAN_FROM_DATAREADER_RANGE: { - // SSDataBlock* pSDB = doRangeScan(pInfo, pInfo->pUpdateRes, pInfo->primaryTsIndex, &pInfo->updateResIndex); - // if (pSDB) { - // STableScanInfo* pTableScanInfo = pInfo->pTableScanOp->info; - // pSDB->info.type = pInfo->scanMode == STREAM_SCAN_FROM_DATAREADER_RANGE ? STREAM_NORMAL : STREAM_PULL_DATA; - // checkUpdateData(pInfo, true, pSDB, false); - // printDataBlock(pSDB, "scan recover update"); - // calBlockTbName(pInfo, pSDB); - // return pSDB; - // } - // blockDataCleanup(pInfo->pUpdateDataRes); - // pInfo->scanMode = STREAM_SCAN_FROM_READERHANDLE; - // } break; default: break; } @@ -1982,22 +1953,17 @@ static SSDataBlock* doStreamScan(SOperatorInfo* pOperator) { if (pInfo->pRecoverRes != NULL) { calBlockTbName(pInfo, pInfo->pRecoverRes); if (!pInfo->igCheckUpdate && pInfo->pUpdateInfo) { - // if (pStreamInfo->recoverStep == STREAM_RECOVER_STEP__SCAN1) { TSKEY maxTs = pAPI->stateStore.updateInfoFillBlockData(pInfo->pUpdateInfo, pInfo->pRecoverRes, pInfo->primaryTsIndex); pInfo->twAggSup.maxTs = TMAX(pInfo->twAggSup.maxTs, maxTs); - // } else { - // pInfo->pUpdateInfo->maxDataVersion = TMAX(pInfo->pUpdateInfo->maxDataVersion, pStreamInfo->fillHistoryVer.maxVer); - // doCheckUpdate(pInfo, pInfo->pRecoverRes->info.window.ekey, pInfo->pRecoverRes); - // } } if (pInfo->pCreateTbRes->info.rows > 0) { pInfo->scanMode = STREAM_SCAN_FROM_RES; - printDataBlock(pInfo->pCreateTbRes, "recover createTbl"); + printSpecDataBlock(pInfo->pCreateTbRes, getStreamOpName(pOperator->operatorType), "recover", GET_TASKID(pTaskInfo)); return pInfo->pCreateTbRes; } qDebug("stream recover scan get block, rows %" PRId64, pInfo->pRecoverRes->info.rows); - printDataBlock(pInfo->pRecoverRes, "scan recover"); + printSpecDataBlock(pInfo->pRecoverRes, getStreamOpName(pOperator->operatorType), "recover", GET_TASKID(pTaskInfo)); return pInfo->pRecoverRes; } pStreamInfo->recoverStep = STREAM_RECOVER_STEP__NONE; @@ -2053,7 +2019,7 @@ FETCH_NEXT_BLOCK: pAPI->stateStore.updateInfoAddCloseWindowSBF(pInfo->pUpdateInfo); } break; case STREAM_DELETE_DATA: { - printDataBlock(pBlock, "stream scan delete recv"); + printSpecDataBlock(pBlock, getStreamOpName(pOperator->operatorType), "delete recv", GET_TASKID(pTaskInfo)); SSDataBlock* pDelBlock = NULL; if (pInfo->tqReader) { pDelBlock = createSpecialDataBlock(STREAM_DELETE_DATA); @@ -2064,7 +2030,7 @@ FETCH_NEXT_BLOCK: setBlockGroupIdByUid(pInfo, pDelBlock); rebuildDeleteBlockData(pDelBlock, &pStreamInfo->fillHistoryWindow, id); - printDataBlock(pDelBlock, "stream scan delete recv filtered"); + printSpecDataBlock(pDelBlock, getStreamOpName(pOperator->operatorType), "delete recv filtered", GET_TASKID(pTaskInfo)); if (pDelBlock->info.rows == 0) { if (pInfo->tqReader) { blockDataDestroy(pDelBlock); @@ -2075,7 +2041,7 @@ FETCH_NEXT_BLOCK: if (!isIntervalWindow(pInfo) && !isSessionWindow(pInfo) && !isStateWindow(pInfo)) { generateDeleteResultBlock(pInfo, pDelBlock, pInfo->pDeleteDataRes); pInfo->pDeleteDataRes->info.type = STREAM_DELETE_RESULT; - printDataBlock(pDelBlock, "stream scan delete result"); + printSpecDataBlock(pDelBlock, getStreamOpName(pOperator->operatorType), "delete result", GET_TASKID(pTaskInfo)); blockDataDestroy(pDelBlock); if (pInfo->pDeleteDataRes->info.rows > 0) { @@ -2090,7 +2056,7 @@ FETCH_NEXT_BLOCK: prepareRangeScan(pInfo, pInfo->pUpdateRes, &pInfo->updateResIndex); copyDataBlock(pInfo->pDeleteDataRes, pInfo->pUpdateRes); pInfo->pDeleteDataRes->info.type = STREAM_DELETE_DATA; - printDataBlock(pDelBlock, "stream scan delete data"); + printSpecDataBlock(pDelBlock, getStreamOpName(pOperator->operatorType), "delete result", GET_TASKID(pTaskInfo)); if (pInfo->tqReader) { blockDataDestroy(pDelBlock); } @@ -2108,7 +2074,7 @@ FETCH_NEXT_BLOCK: default: break; } - // printDataBlock(pBlock, "stream scan recv"); + printDataBlock(pBlock, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); return pBlock; } else if (pInfo->blockType == STREAM_INPUT__DATA_SUBMIT) { qDebug("stream scan mode:%d, %s", pInfo->scanMode, id); @@ -2144,7 +2110,7 @@ FETCH_NEXT_BLOCK: STableScanInfo* pTableScanInfo = pInfo->pTableScanOp->info; pSDB->info.type = pInfo->scanMode == STREAM_SCAN_FROM_DATAREADER_RANGE ? STREAM_NORMAL : STREAM_PULL_DATA; checkUpdateData(pInfo, true, pSDB, false); - printDataBlock(pSDB, "stream scan update"); + printSpecDataBlock(pSDB, getStreamOpName(pOperator->operatorType), "update", GET_TASKID(pTaskInfo)); calBlockTbName(pInfo, pSDB); return pSDB; } @@ -2839,11 +2805,6 @@ static void tagScanFilterByTagCond(SArray* aUidTags, SNode* pTagCond, SArray* aF static void tagScanFillOneCellWithTag(const STUidTagInfo* pUidTagInfo, SExprInfo* pExprInfo, SColumnInfoData* pColInfo, int rowIndex, const SStorageAPI* pAPI, void* pVnode) { if (fmIsScanPseudoColumnFunc(pExprInfo->pExpr->_function.functionId)) { // tbname char str[TSDB_TABLE_FNAME_LEN + VARSTR_HEADER_SIZE] = {0}; -// if (pUidTagInfo->name != NULL) { -// STR_TO_VARSTR(str, pUidTagInfo->name); -// } else { // name is not retrieved during filter -// pAPI->metaFn.getTableNameByUid(pVnode, pUidTagInfo->uid, str); -// } STR_TO_VARSTR(str, "ctbidx"); colDataSetVal(pColInfo, rowIndex, str, false); @@ -2912,12 +2873,14 @@ static SSDataBlock* doTagScanFromCtbIdx(SOperatorInfo* pOperator) { if (pInfo->pCtbCursor == NULL) { pInfo->pCtbCursor = pAPI->metaFn.openCtbCursor(pInfo->readHandle.vnode, pInfo->suid, 1); + } else { + pAPI->metaFn.resumeCtbCursor(pInfo->pCtbCursor, 0); } SArray* aUidTags = pInfo->aUidTags; SArray* aFilterIdxs = pInfo->aFilterIdxs; int32_t count = 0; - + bool ctbCursorFinished = false; while (1) { taosArrayClearEx(aUidTags, tagScanFreeUidTag); taosArrayClear(aFilterIdxs); @@ -2927,6 +2890,7 @@ static SSDataBlock* doTagScanFromCtbIdx(SOperatorInfo* pOperator) { SMCtbCursor* pCur = pInfo->pCtbCursor; tb_uid_t uid = pAPI->metaFn.ctbCursorNext(pInfo->pCtbCursor); if (uid == 0) { + ctbCursorFinished = true; break; } STUidTagInfo info = {.uid = uid, .pTagVal = pCur->pVal}; @@ -2955,7 +2919,15 @@ static SSDataBlock* doTagScanFromCtbIdx(SOperatorInfo* pOperator) { break; } } - + + if (count > 0) { + pAPI->metaFn.pauseCtbCursor(pInfo->pCtbCursor); + } + if (count == 0 || ctbCursorFinished) { + pAPI->metaFn.closeCtbCursor(pInfo->pCtbCursor); + pInfo->pCtbCursor = NULL; + setOperatorCompleted(pOperator); + } pRes->info.rows = count; pOperator->resultInfo.totalRows += count; return (pRes->info.rows == 0) ? NULL : pInfo->pRes; @@ -3020,7 +2992,7 @@ static SSDataBlock* doTagScanFromMetaEntry(SOperatorInfo* pOperator) { static void destroyTagScanOperatorInfo(void* param) { STagScanInfo* pInfo = (STagScanInfo*)param; if (pInfo->pCtbCursor != NULL) { - pInfo->pStorageAPI->metaFn.closeCtbCursor(pInfo->pCtbCursor, 1); + pInfo->pStorageAPI->metaFn.closeCtbCursor(pInfo->pCtbCursor); } taosHashCleanup(pInfo->filterCtx.colHash); taosArrayDestroy(pInfo->filterCtx.cInfoList); @@ -3150,7 +3122,7 @@ static SSDataBlock* getBlockForTableMergeScan(void* param) { continue; } - pBlock->info.id.groupId = getTableGroupId(pInfo->base.pTableListInfo, pBlock->info.id.uid); + pBlock->info.id.groupId = tableListGetTableGroupId(pInfo->base.pTableListInfo, pBlock->info.id.uid); pOperator->resultInfo.totalRows += pBlock->info.rows; pInfo->base.readRecorder.elapsedTime += (taosGetTimestampUs() - st) / 1000.0; diff --git a/source/libs/executor/src/streamtimewindowoperator.c b/source/libs/executor/src/streamtimewindowoperator.c new file mode 100644 index 0000000000..a718373f60 --- /dev/null +++ b/source/libs/executor/src/streamtimewindowoperator.c @@ -0,0 +1,3195 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * This program is free software: you can use, redistribute, and/or modify + * it under the terms of the GNU Affero General Public License, version 3 + * or later ("AGPL"), as published by the Free Software Foundation. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +#include "executorInt.h" +#include "filter.h" +#include "function.h" +#include "functionMgt.h" +#include "operator.h" +#include "querytask.h" +#include "tcommon.h" +#include "tcompare.h" +#include "tdatablock.h" +#include "tfill.h" +#include "tglobal.h" +#include "tlog.h" +#include "ttime.h" + +#define IS_FINAL_INTERVAL_OP(op) ((op)->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL) +#define IS_FINAL_SESSION_OP(op) ((op)->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_SESSION) +#define DEAULT_DELETE_MARK (1000LL * 60LL * 60LL * 24LL * 365LL * 10LL); +#define STREAM_INTERVAL_OP_STATE_NAME "StreamIntervalHistoryState" +#define STREAM_SESSION_OP_STATE_NAME "StreamSessionHistoryState" +#define STREAM_STATE_OP_STATE_NAME "StreamStateHistoryState" + +typedef struct SStateWindowInfo { + SResultWindowInfo winInfo; + SStateKeys* pStateKey; +} SStateWindowInfo; + +typedef struct SPullWindowInfo { + STimeWindow window; + uint64_t groupId; + STimeWindow calWin; +} SPullWindowInfo; + +typedef int32_t (*__compare_fn_t)(void* pKey, void* data, int32_t index); + +static int32_t binarySearchCom(void* keyList, int num, void* pKey, int order, __compare_fn_t comparefn) { + int firstPos = 0, lastPos = num - 1, midPos = -1; + int numOfRows = 0; + + if (num <= 0) return -1; + if (order == TSDB_ORDER_DESC) { + // find the first position which is smaller or equal than the key + while (1) { + if (comparefn(pKey, keyList, lastPos) >= 0) return lastPos; + if (comparefn(pKey, keyList, firstPos) == 0) return firstPos; + if (comparefn(pKey, keyList, firstPos) < 0) return firstPos - 1; + + numOfRows = lastPos - firstPos + 1; + midPos = (numOfRows >> 1) + firstPos; + + if (comparefn(pKey, keyList, midPos) < 0) { + lastPos = midPos - 1; + } else if (comparefn(pKey, keyList, midPos) > 0) { + firstPos = midPos + 1; + } else { + break; + } + } + + } else { + // find the first position which is bigger or equal than the key + while (1) { + if (comparefn(pKey, keyList, firstPos) <= 0) return firstPos; + if (comparefn(pKey, keyList, lastPos) == 0) return lastPos; + + if (comparefn(pKey, keyList, lastPos) > 0) { + lastPos = lastPos + 1; + if (lastPos >= num) + return -1; + else + return lastPos; + } + + numOfRows = lastPos - firstPos + 1; + midPos = (numOfRows >> 1) + firstPos; + + if (comparefn(pKey, keyList, midPos) < 0) { + lastPos = midPos - 1; + } else if (comparefn(pKey, keyList, midPos) > 0) { + firstPos = midPos + 1; + } else { + break; + } + } + } + + return midPos; +} + +static int32_t comparePullWinKey(void* pKey, void* data, int32_t index) { + SArray* res = (SArray*)data; + SPullWindowInfo* pos = taosArrayGet(res, index); + SPullWindowInfo* pData = (SPullWindowInfo*)pKey; + if (pData->groupId > pos->groupId) { + return 1; + } else if (pData->groupId < pos->groupId) { + return -1; + } + + if (pData->window.skey > pos->window.ekey) { + return 1; + } else if (pData->window.ekey < pos->window.skey) { + return -1; + } + return 0; +} + +static int32_t savePullWindow(SPullWindowInfo* pPullInfo, SArray* pPullWins) { + int32_t size = taosArrayGetSize(pPullWins); + int32_t index = binarySearchCom(pPullWins, size, pPullInfo, TSDB_ORDER_DESC, comparePullWinKey); + if (index == -1) { + index = 0; + } else { + int32_t code = comparePullWinKey(pPullInfo, pPullWins, index); + if (code == 0) { + SPullWindowInfo* pos = taosArrayGet(pPullWins, index); + pos->window.skey = TMIN(pos->window.skey, pPullInfo->window.skey); + pos->window.ekey = TMAX(pos->window.ekey, pPullInfo->window.ekey); + pos->calWin.skey = TMIN(pos->calWin.skey, pPullInfo->calWin.skey); + pos->calWin.ekey = TMAX(pos->calWin.ekey, pPullInfo->calWin.ekey); + return TSDB_CODE_SUCCESS; + } else if (code > 0) { + index++; + } + } + if (taosArrayInsert(pPullWins, index, pPullInfo) == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + return TSDB_CODE_SUCCESS; +} + +static int32_t saveResult(SResultWindowInfo winInfo, SSHashObj* pStUpdated) { + winInfo.sessionWin.win.ekey = winInfo.sessionWin.win.skey; + return tSimpleHashPut(pStUpdated, &winInfo.sessionWin, sizeof(SSessionKey), &winInfo, sizeof(SResultWindowInfo)); +} + +static int32_t saveWinResult(SWinKey* pKey, SRowBuffPos* pPos, SSHashObj* pUpdatedMap) { + tSimpleHashPut(pUpdatedMap, pKey, sizeof(SWinKey), &pPos, POINTER_BYTES); + return TSDB_CODE_SUCCESS; +} + +static int32_t saveWinResultInfo(TSKEY ts, uint64_t groupId, SRowBuffPos* pPos, SSHashObj* pUpdatedMap) { + SWinKey key = {.ts = ts, .groupId = groupId}; + saveWinResult(&key, pPos, pUpdatedMap); + return TSDB_CODE_SUCCESS; +} + +static void removeResults(SArray* pWins, SSHashObj* pUpdatedMap) { + int32_t size = taosArrayGetSize(pWins); + for (int32_t i = 0; i < size; i++) { + SWinKey* pW = taosArrayGet(pWins, i); + void* tmp = tSimpleHashGet(pUpdatedMap, pW, sizeof(SWinKey)); + if (tmp) { + void* value = *(void**)tmp; + taosMemoryFree(value); + tSimpleHashRemove(pUpdatedMap, pW, sizeof(SWinKey)); + } + } +} + +static int32_t compareWinKey(void* pKey, void* data, int32_t index) { + void* pDataPos = taosArrayGet((SArray*)data, index); + return winKeyCmprImpl(pKey, pDataPos); +} + +static void removeDeleteResults(SSHashObj* pUpdatedMap, SArray* pDelWins) { + taosArraySort(pDelWins, winKeyCmprImpl); + taosArrayRemoveDuplicate(pDelWins, winKeyCmprImpl, NULL); + int32_t delSize = taosArrayGetSize(pDelWins); + if (tSimpleHashGetSize(pUpdatedMap) == 0 || delSize == 0) { + return; + } + void* pIte = NULL; + int32_t iter = 0; + while ((pIte = tSimpleHashIterate(pUpdatedMap, pIte, &iter)) != NULL) { + SWinKey* pResKey = tSimpleHashGetKey(pIte, NULL); + int32_t index = binarySearchCom(pDelWins, delSize, pResKey, TSDB_ORDER_DESC, compareWinKey); + if (index >= 0 && 0 == compareWinKey(pResKey, pDelWins, index)) { + taosArrayRemove(pDelWins, index); + delSize = taosArrayGetSize(pDelWins); + } + } +} + +bool isOverdue(TSKEY ekey, STimeWindowAggSupp* pTwSup) { + ASSERTS(pTwSup->maxTs == INT64_MIN || pTwSup->maxTs > 0, "maxts should greater than 0"); + return pTwSup->maxTs != INT64_MIN && ekey < pTwSup->maxTs - pTwSup->waterMark; +} + +bool isCloseWindow(STimeWindow* pWin, STimeWindowAggSupp* pTwSup) { return isOverdue(pWin->ekey, pTwSup); } + +static bool doDeleteWindow(SOperatorInfo* pOperator, TSKEY ts, uint64_t groupId) { + SStorageAPI* pAPI = &pOperator->pTaskInfo->storageAPI; + + SStreamIntervalOperatorInfo* pInfo = pOperator->info; + SWinKey key = {.ts = ts, .groupId = groupId}; + tSimpleHashRemove(pInfo->aggSup.pResultRowHashTable, &key, sizeof(SWinKey)); + pAPI->stateStore.streamStateDel(pInfo->pState, &key); + return true; +} + +static int32_t getChildIndex(SSDataBlock* pBlock) { return pBlock->info.childId; } + +static void doDeleteWindows(SOperatorInfo* pOperator, SInterval* pInterval, SSDataBlock* pBlock, SArray* pUpWins, + SSHashObj* pUpdatedMap) { + SStreamIntervalOperatorInfo* pInfo = pOperator->info; + SColumnInfoData* pStartTsCol = taosArrayGet(pBlock->pDataBlock, START_TS_COLUMN_INDEX); + TSKEY* startTsCols = (TSKEY*)pStartTsCol->pData; + SColumnInfoData* pEndTsCol = taosArrayGet(pBlock->pDataBlock, END_TS_COLUMN_INDEX); + TSKEY* endTsCols = (TSKEY*)pEndTsCol->pData; + SColumnInfoData* pCalStTsCol = taosArrayGet(pBlock->pDataBlock, CALCULATE_START_TS_COLUMN_INDEX); + TSKEY* calStTsCols = (TSKEY*)pCalStTsCol->pData; + SColumnInfoData* pCalEnTsCol = taosArrayGet(pBlock->pDataBlock, CALCULATE_END_TS_COLUMN_INDEX); + TSKEY* calEnTsCols = (TSKEY*)pCalEnTsCol->pData; + SColumnInfoData* pGpCol = taosArrayGet(pBlock->pDataBlock, GROUPID_COLUMN_INDEX); + uint64_t* pGpDatas = (uint64_t*)pGpCol->pData; + for (int32_t i = 0; i < pBlock->info.rows; i++) { + SResultRowInfo dumyInfo = {0}; + dumyInfo.cur.pageId = -1; + + STimeWindow win = {0}; + if (IS_FINAL_INTERVAL_OP(pOperator)) { + win.skey = startTsCols[i]; + win.ekey = endTsCols[i]; + } else { + win = getActiveTimeWindow(NULL, &dumyInfo, startTsCols[i], pInterval, TSDB_ORDER_ASC); + } + + do { + if (!inCalSlidingWindow(pInterval, &win, calStTsCols[i], calEnTsCols[i], pBlock->info.type)) { + getNextTimeWindow(pInterval, &win, TSDB_ORDER_ASC); + continue; + } + uint64_t winGpId = pGpDatas[i]; + SWinKey winRes = {.ts = win.skey, .groupId = winGpId}; + void* chIds = taosHashGet(pInfo->pPullDataMap, &winRes, sizeof(SWinKey)); + if (chIds) { + int32_t childId = getChildIndex(pBlock); + SArray* chArray = *(void**)chIds; + int32_t index = taosArraySearchIdx(chArray, &childId, compareInt32Val, TD_EQ); + if (index != -1) { + qDebug("===stream===try push delete window%" PRId64 "chId:%d ,continue", win.skey, childId); + getNextTimeWindow(pInterval, &win, TSDB_ORDER_ASC); + continue; + } + } + bool res = doDeleteWindow(pOperator, win.skey, winGpId); + if (pUpWins && res) { + taosArrayPush(pUpWins, &winRes); + } + if (pUpdatedMap) { + tSimpleHashRemove(pUpdatedMap, &winRes, sizeof(SWinKey)); + } + getNextTimeWindow(pInterval, &win, TSDB_ORDER_ASC); + } while (win.ekey <= endTsCols[i]); + } +} + +static int32_t getAllIntervalWindow(SSHashObj* pHashMap, SSHashObj* resWins) { + void* pIte = NULL; + int32_t iter = 0; + while ((pIte = tSimpleHashIterate(pHashMap, pIte, &iter)) != NULL) { + SWinKey* pKey = tSimpleHashGetKey(pIte, NULL); + uint64_t groupId = pKey->groupId; + TSKEY ts = pKey->ts; + int32_t code = saveWinResultInfo(ts, groupId, *(SRowBuffPos**)pIte, resWins); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + } + return TSDB_CODE_SUCCESS; +} + +static int32_t closeStreamIntervalWindow(SSHashObj* pHashMap, STimeWindowAggSupp* pTwSup, SInterval* pInterval, + SHashObj* pPullDataMap, SSHashObj* closeWins, SArray* pDelWins, + SOperatorInfo* pOperator) { + qDebug("===stream===close interval window"); + void* pIte = NULL; + int32_t iter = 0; + SStreamIntervalOperatorInfo* pInfo = pOperator->info; + int32_t delSize = taosArrayGetSize(pDelWins); + while ((pIte = tSimpleHashIterate(pHashMap, pIte, &iter)) != NULL) { + void* key = tSimpleHashGetKey(pIte, NULL); + SWinKey* pWinKey = (SWinKey*)key; + if (delSize > 0) { + int32_t index = binarySearchCom(pDelWins, delSize, pWinKey, TSDB_ORDER_DESC, compareWinKey); + if (index >= 0 && 0 == compareWinKey(pWinKey, pDelWins, index)) { + taosArrayRemove(pDelWins, index); + delSize = taosArrayGetSize(pDelWins); + } + } + + void* chIds = taosHashGet(pPullDataMap, pWinKey, sizeof(SWinKey)); + STimeWindow win = { + .skey = pWinKey->ts, + .ekey = taosTimeAdd(win.skey, pInterval->interval, pInterval->intervalUnit, pInterval->precision) - 1, + }; + if (isCloseWindow(&win, pTwSup)) { + if (chIds && pPullDataMap) { + SArray* chAy = *(SArray**)chIds; + int32_t size = taosArrayGetSize(chAy); + qDebug("===stream===window %" PRId64 " wait child size:%d", pWinKey->ts, size); + for (int32_t i = 0; i < size; i++) { + qDebug("===stream===window %" PRId64 " wait child id:%d", pWinKey->ts, *(int32_t*)taosArrayGet(chAy, i)); + } + continue; + } else if (pPullDataMap) { + qDebug("===stream===close window %" PRId64, pWinKey->ts); + } + + if (pTwSup->calTrigger == STREAM_TRIGGER_WINDOW_CLOSE) { + int32_t code = saveWinResult(pWinKey, *(SRowBuffPos**)pIte, closeWins); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + } + tSimpleHashIterateRemove(pHashMap, pWinKey, sizeof(SWinKey), &pIte, &iter); + } + } + return TSDB_CODE_SUCCESS; +} + +STimeWindow getFinalTimeWindow(int64_t ts, SInterval* pInterval) { + STimeWindow w = {.skey = ts, .ekey = INT64_MAX}; + w.ekey = taosTimeAdd(w.skey, pInterval->interval, pInterval->intervalUnit, pInterval->precision) - 1; + return w; +} + +static void doBuildDeleteResult(SStreamIntervalOperatorInfo* pInfo, SArray* pWins, int32_t* index, + SSDataBlock* pBlock) { + blockDataCleanup(pBlock); + int32_t size = taosArrayGetSize(pWins); + if (*index == size) { + *index = 0; + taosArrayClear(pWins); + return; + } + blockDataEnsureCapacity(pBlock, size - *index); + uint64_t uid = 0; + for (int32_t i = *index; i < size; i++) { + SWinKey* pWin = taosArrayGet(pWins, i); + void* tbname = NULL; + pInfo->statestore.streamStateGetParName(pInfo->pState, pWin->groupId, &tbname); + if (tbname == NULL) { + appendOneRowToStreamSpecialBlock(pBlock, &pWin->ts, &pWin->ts, &uid, &pWin->groupId, NULL); + } else { + char parTbName[VARSTR_HEADER_SIZE + TSDB_TABLE_NAME_LEN]; + STR_WITH_MAXSIZE_TO_VARSTR(parTbName, tbname, sizeof(parTbName)); + appendOneRowToStreamSpecialBlock(pBlock, &pWin->ts, &pWin->ts, &uid, &pWin->groupId, parTbName); + } + pInfo->statestore.streamStateFreeVal(tbname); + (*index)++; + } +} + +void destroyStreamFinalIntervalOperatorInfo(void* param) { + SStreamIntervalOperatorInfo* pInfo = (SStreamIntervalOperatorInfo*)param; + cleanupBasicInfo(&pInfo->binfo); + cleanupAggSup(&pInfo->aggSup); + // it should be empty. + void* pIte = NULL; + while ((pIte = taosHashIterate(pInfo->pPullDataMap, pIte)) != NULL) { + taosArrayDestroy(*(void**)pIte); + } + taosHashCleanup(pInfo->pPullDataMap); + taosHashCleanup(pInfo->pFinalPullDataMap); + taosArrayDestroy(pInfo->pPullWins); + blockDataDestroy(pInfo->pPullDataRes); + taosArrayDestroy(pInfo->pDelWins); + blockDataDestroy(pInfo->pDelRes); + pInfo->statestore.streamFileStateDestroy(pInfo->pState->pFileState); + taosMemoryFreeClear(pInfo->pState); + + nodesDestroyNode((SNode*)pInfo->pPhyNode); + colDataDestroy(&pInfo->twAggSup.timeWindowData); + pInfo->groupResInfo.pRows = taosArrayDestroy(pInfo->groupResInfo.pRows); + cleanupExprSupp(&pInfo->scalarSupp); + tSimpleHashCleanup(pInfo->pUpdatedMap); + pInfo->pUpdatedMap = NULL; + pInfo->pUpdated = taosArrayDestroy(pInfo->pUpdated); + + taosMemoryFreeClear(param); +} + +static bool allInvertible(SqlFunctionCtx* pFCtx, int32_t numOfCols) { + for (int32_t i = 0; i < numOfCols; i++) { + if (fmIsUserDefinedFunc(pFCtx[i].functionId) || !fmIsInvertible(pFCtx[i].functionId)) { + return false; + } + } + return true; +} + +void initIntervalDownStream(SOperatorInfo* downstream, uint16_t type, SStreamIntervalOperatorInfo* pInfo) { + SStateStore* pAPI = &downstream->pTaskInfo->storageAPI.stateStore; + + if (downstream->operatorType != QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN) { + initIntervalDownStream(downstream->pDownstream[0], type, pInfo); + return; + } + + SStreamScanInfo* pScanInfo = downstream->info; + pScanInfo->windowSup.parentType = type; + pScanInfo->windowSup.pIntervalAggSup = &pInfo->aggSup; + if (!pScanInfo->pUpdateInfo) { + pScanInfo->pUpdateInfo = pAPI->updateInfoInitP(&pInfo->interval, pInfo->twAggSup.waterMark); + } + + pScanInfo->interval = pInfo->interval; + pScanInfo->twAggSup = pInfo->twAggSup; + pScanInfo->pState = pInfo->pState; +} + +void compactFunctions(SqlFunctionCtx* pDestCtx, SqlFunctionCtx* pSourceCtx, int32_t numOfOutput, + SExecTaskInfo* pTaskInfo, SColumnInfoData* pTimeWindowData) { + for (int32_t k = 0; k < numOfOutput; ++k) { + if (fmIsWindowPseudoColumnFunc(pDestCtx[k].functionId)) { + if (!pTimeWindowData) { + continue; + } + + SResultRowEntryInfo* pEntryInfo = GET_RES_INFO(&pDestCtx[k]); + char* p = GET_ROWCELL_INTERBUF(pEntryInfo); + SColumnInfoData idata = {0}; + idata.info.type = TSDB_DATA_TYPE_BIGINT; + idata.info.bytes = tDataTypes[TSDB_DATA_TYPE_BIGINT].bytes; + idata.pData = p; + + SScalarParam out = {.columnData = &idata}; + SScalarParam tw = {.numOfRows = 5, .columnData = pTimeWindowData}; + pDestCtx[k].sfp.process(&tw, 1, &out); + pEntryInfo->numOfRes = 1; + } else if (functionNeedToExecute(&pDestCtx[k]) && pDestCtx[k].fpSet.combine != NULL) { + int32_t code = pDestCtx[k].fpSet.combine(&pDestCtx[k], &pSourceCtx[k]); + if (code != TSDB_CODE_SUCCESS) { + qError("%s apply combine functions error, code: %s", GET_TASKID(pTaskInfo), tstrerror(code)); + } + } else if (pDestCtx[k].fpSet.combine == NULL) { + char* funName = fmGetFuncName(pDestCtx[k].functionId); + qError("%s error, combine funcion for %s is not implemented", GET_TASKID(pTaskInfo), funName); + taosMemoryFreeClear(funName); + } + } +} + +bool hasIntervalWindow(void* pState, SWinKey* pKey, SStateStore* pStore) { + return pStore->streamStateCheck(pState, pKey); +} + +int32_t setIntervalOutputBuf(void* pState, STimeWindow* win, SRowBuffPos** pResult, int64_t groupId, + SqlFunctionCtx* pCtx, int32_t numOfOutput, int32_t* rowEntryInfoOffset, + SAggSupporter* pAggSup, SStateStore* pStore) { + SWinKey key = {.ts = win->skey, .groupId = groupId}; + char* value = NULL; + int32_t size = pAggSup->resultRowSize; + + if (pStore->streamStateAddIfNotExist(pState, &key, (void**)&value, &size) < 0) { + return TSDB_CODE_OUT_OF_MEMORY; + } + + *pResult = (SRowBuffPos*)value; + SResultRow* res = (SResultRow*)((*pResult)->pRowBuff); + + // set time window for current result + res->win = (*win); + setResultRowInitCtx(res, pCtx, numOfOutput, rowEntryInfoOffset); + return TSDB_CODE_SUCCESS; +} + +bool isDeletedStreamWindow(STimeWindow* pWin, uint64_t groupId, void* pState, STimeWindowAggSupp* pTwSup, + SStateStore* pStore) { + if (pTwSup->maxTs != INT64_MIN && pWin->ekey < pTwSup->maxTs - pTwSup->deleteMark) { + SWinKey key = {.ts = pWin->skey, .groupId = groupId}; + if (!hasIntervalWindow(pState, &key, pStore)) { + return true; + } + return false; + } + return false; +} + +int32_t getNexWindowPos(SInterval* pInterval, SDataBlockInfo* pBlockInfo, TSKEY* tsCols, int32_t startPos, TSKEY eKey, + STimeWindow* pNextWin) { + int32_t forwardRows = + getNumOfRowsInTimeWindow(pBlockInfo, tsCols, startPos, eKey, binarySearchForKey, NULL, TSDB_ORDER_ASC); + int32_t prevEndPos = forwardRows - 1 + startPos; + return getNextQualifiedWindow(pInterval, pNextWin, pBlockInfo, tsCols, prevEndPos, TSDB_ORDER_ASC); +} + +void addPullWindow(SHashObj* pMap, SWinKey* pWinRes, int32_t size) { + SArray* childIds = taosArrayInit(8, sizeof(int32_t)); + for (int32_t i = 0; i < size; i++) { + taosArrayPush(childIds, &i); + } + taosHashPut(pMap, pWinRes, sizeof(SWinKey), &childIds, sizeof(void*)); +} + +static void clearStreamIntervalOperator(SStreamIntervalOperatorInfo* pInfo) { + tSimpleHashClear(pInfo->aggSup.pResultRowHashTable); + clearDiskbasedBuf(pInfo->aggSup.pResultBuf); + initResultRowInfo(&pInfo->binfo.resultRowInfo); + pInfo->aggSup.currentPageId = -1; + pInfo->statestore.streamStateClear(pInfo->pState); +} + +static void clearSpecialDataBlock(SSDataBlock* pBlock) { + if (pBlock->info.rows <= 0) { + return; + } + blockDataCleanup(pBlock); +} + +static void doBuildPullDataBlock(SArray* array, int32_t* pIndex, SSDataBlock* pBlock) { + clearSpecialDataBlock(pBlock); + int32_t size = taosArrayGetSize(array); + if (size - (*pIndex) == 0) { + return; + } + blockDataEnsureCapacity(pBlock, size - (*pIndex)); + SColumnInfoData* pStartTs = (SColumnInfoData*)taosArrayGet(pBlock->pDataBlock, START_TS_COLUMN_INDEX); + SColumnInfoData* pEndTs = (SColumnInfoData*)taosArrayGet(pBlock->pDataBlock, END_TS_COLUMN_INDEX); + SColumnInfoData* pGroupId = (SColumnInfoData*)taosArrayGet(pBlock->pDataBlock, GROUPID_COLUMN_INDEX); + SColumnInfoData* pCalStartTs = (SColumnInfoData*)taosArrayGet(pBlock->pDataBlock, CALCULATE_START_TS_COLUMN_INDEX); + SColumnInfoData* pCalEndTs = (SColumnInfoData*)taosArrayGet(pBlock->pDataBlock, CALCULATE_END_TS_COLUMN_INDEX); + for (; (*pIndex) < size; (*pIndex)++) { + SPullWindowInfo* pWin = taosArrayGet(array, (*pIndex)); + colDataSetVal(pStartTs, pBlock->info.rows, (const char*)&pWin->window.skey, false); + colDataSetVal(pEndTs, pBlock->info.rows, (const char*)&pWin->window.ekey, false); + colDataSetVal(pGroupId, pBlock->info.rows, (const char*)&pWin->groupId, false); + colDataSetVal(pCalStartTs, pBlock->info.rows, (const char*)&pWin->calWin.skey, false); + colDataSetVal(pCalEndTs, pBlock->info.rows, (const char*)&pWin->calWin.ekey, false); + pBlock->info.rows++; + } + if ((*pIndex) == size) { + *pIndex = 0; + taosArrayClear(array); + } + blockDataUpdateTsWindow(pBlock, 0); +} + +void processPullOver(SSDataBlock* pBlock, SHashObj* pMap, SHashObj* pFinalMap, SInterval* pInterval, SArray* pPullWins, + int32_t numOfCh, SOperatorInfo* pOperator) { + SColumnInfoData* pStartCol = taosArrayGet(pBlock->pDataBlock, CALCULATE_START_TS_COLUMN_INDEX); + TSKEY* tsData = (TSKEY*)pStartCol->pData; + SColumnInfoData* pEndCol = taosArrayGet(pBlock->pDataBlock, CALCULATE_END_TS_COLUMN_INDEX); + TSKEY* tsEndData = (TSKEY*)pEndCol->pData; + SColumnInfoData* pGroupCol = taosArrayGet(pBlock->pDataBlock, GROUPID_COLUMN_INDEX); + uint64_t* groupIdData = (uint64_t*)pGroupCol->pData; + int32_t chId = getChildIndex(pBlock); + for (int32_t i = 0; i < pBlock->info.rows; i++) { + TSKEY winTs = tsData[i]; + while (winTs <= tsEndData[i]) { + SWinKey winRes = {.ts = winTs, .groupId = groupIdData[i]}; + void* chIds = taosHashGet(pMap, &winRes, sizeof(SWinKey)); + if (chIds) { + SArray* chArray = *(SArray**)chIds; + int32_t index = taosArraySearchIdx(chArray, &chId, compareInt32Val, TD_EQ); + if (index != -1) { + qDebug("===stream===retrive window %" PRId64 " delete child id %d", winRes.ts, chId); + taosArrayRemove(chArray, index); + if (taosArrayGetSize(chArray) == 0) { + // pull data is over + taosArrayDestroy(chArray); + taosHashRemove(pMap, &winRes, sizeof(SWinKey)); + qDebug("===stream===retrive pull data over.window %" PRId64, winRes.ts); + + void* pFinalCh = taosHashGet(pFinalMap, &winRes, sizeof(SWinKey)); + if (pFinalCh) { + taosHashRemove(pFinalMap, &winRes, sizeof(SWinKey)); + doDeleteWindow(pOperator, winRes.ts, winRes.groupId); + STimeWindow nextWin = getFinalTimeWindow(winRes.ts, pInterval); + SPullWindowInfo pull = {.window = nextWin, + .groupId = winRes.groupId, + .calWin.skey = nextWin.skey, + .calWin.ekey = nextWin.skey}; + // add pull data request + if (savePullWindow(&pull, pPullWins) == TSDB_CODE_SUCCESS) { + addPullWindow(pMap, &winRes, numOfCh); + qDebug("===stream===prepare final retrive for delete %" PRId64 ", size:%d", winRes.ts, numOfCh); + } + } + } + } + } + winTs = taosTimeAdd(winTs, pInterval->sliding, pInterval->slidingUnit, pInterval->precision); + } + } +} + +static void addRetriveWindow(SArray* wins, SStreamIntervalOperatorInfo* pInfo, int32_t childId) { + int32_t size = taosArrayGetSize(wins); + for (int32_t i = 0; i < size; i++) { + SWinKey* winKey = taosArrayGet(wins, i); + STimeWindow nextWin = getFinalTimeWindow(winKey->ts, &pInfo->interval); + if (isOverdue(nextWin.ekey, &pInfo->twAggSup) && pInfo->ignoreExpiredData) { + continue; + } + void* chIds = taosHashGet(pInfo->pPullDataMap, winKey, sizeof(SWinKey)); + if (!chIds) { + SPullWindowInfo pull = { + .window = nextWin, .groupId = winKey->groupId, .calWin.skey = nextWin.skey, .calWin.ekey = nextWin.skey}; + // add pull data request + if (savePullWindow(&pull, pInfo->pPullWins) == TSDB_CODE_SUCCESS) { + addPullWindow(pInfo->pPullDataMap, winKey, pInfo->numOfChild); + qDebug("===stream===prepare retrive for delete %" PRId64 ", size:%d", winKey->ts, pInfo->numOfChild); + } + } else { + SArray* chArray = *(void**)chIds; + int32_t index = taosArraySearchIdx(chArray, &childId, compareInt32Val, TD_EQ); + qDebug("===stream===check final retrive %" PRId64 ",chid:%d", winKey->ts, index); + if (index == -1) { + qDebug("===stream===add final retrive %" PRId64, winKey->ts); + taosHashPut(pInfo->pFinalPullDataMap, winKey, sizeof(SWinKey), NULL, 0); + } + } + } +} + +static void clearFunctionContext(SExprSupp* pSup) { + for (int32_t i = 0; i < pSup->numOfExprs; i++) { + pSup->pCtx[i].saveHandle.currentPage = -1; + } +} + +int32_t getOutputBuf(void* pState, SRowBuffPos* pPos, SResultRow** pResult, SStateStore* pStore) { + return pStore->streamStateGetByPos(pState, pPos, (void**)pResult); +} + +int32_t buildDataBlockFromGroupRes(SOperatorInfo* pOperator, void* pState, SSDataBlock* pBlock, SExprSupp* pSup, + SGroupResInfo* pGroupResInfo) { + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + SStorageAPI* pAPI = &pOperator->pTaskInfo->storageAPI; + + SExprInfo* pExprInfo = pSup->pExprInfo; + int32_t numOfExprs = pSup->numOfExprs; + int32_t* rowEntryOffset = pSup->rowEntryInfoOffset; + SqlFunctionCtx* pCtx = pSup->pCtx; + + int32_t numOfRows = getNumOfTotalRes(pGroupResInfo); + + for (int32_t i = pGroupResInfo->index; i < numOfRows; i += 1) { + SRowBuffPos* pPos = *(SRowBuffPos**)taosArrayGet(pGroupResInfo->pRows, i); + SResultRow* pRow = NULL; + int32_t code = getOutputBuf(pState, pPos, &pRow, &pAPI->stateStore); + uint64_t groupId = ((SWinKey*)pPos->pKey)->groupId; + ASSERT(code == 0); + doUpdateNumOfRows(pCtx, pRow, numOfExprs, rowEntryOffset); + // no results, continue to check the next one + if (pRow->numOfRows == 0) { + pGroupResInfo->index += 1; + continue; + } + if (pBlock->info.id.groupId == 0) { + pBlock->info.id.groupId = groupId; + void* tbname = NULL; + if (pAPI->stateStore.streamStateGetParName(pTaskInfo->streamInfo.pState, pBlock->info.id.groupId, &tbname) < 0) { + pBlock->info.parTbName[0] = 0; + } else { + memcpy(pBlock->info.parTbName, tbname, TSDB_TABLE_NAME_LEN); + } + pAPI->stateStore.streamStateFreeVal(tbname); + } else { + // current value belongs to different group, it can't be packed into one datablock + if (pBlock->info.id.groupId != groupId) { + break; + } + } + + if (pBlock->info.rows + pRow->numOfRows > pBlock->info.capacity) { + ASSERT(pBlock->info.rows > 0); + break; + } + pGroupResInfo->index += 1; + + for (int32_t j = 0; j < numOfExprs; ++j) { + int32_t slotId = pExprInfo[j].base.resSchema.slotId; + + pCtx[j].resultInfo = getResultEntryInfo(pRow, j, rowEntryOffset); + SResultRowEntryInfo* pEnryInfo = pCtx[j].resultInfo; + + if (pCtx[j].fpSet.finalize) { + int32_t code1 = pCtx[j].fpSet.finalize(&pCtx[j], pBlock); + if (TAOS_FAILED(code1)) { + qError("%s build result data block error, code %s", GET_TASKID(pTaskInfo), tstrerror(code1)); + T_LONG_JMP(pTaskInfo->env, code1); + } + } else if (strcmp(pCtx[j].pExpr->pExpr->_function.functionName, "_select_value") == 0) { + // do nothing, todo refactor + } else { + // expand the result into multiple rows. E.g., _wstart, top(k, 20) + // the _wstart needs to copy to 20 following rows, since the results of top-k expands to 20 different rows. + SColumnInfoData* pColInfoData = taosArrayGet(pBlock->pDataBlock, slotId); + char* in = GET_ROWCELL_INTERBUF(pCtx[j].resultInfo); + for (int32_t k = 0; k < pRow->numOfRows; ++k) { + colDataSetVal(pColInfoData, pBlock->info.rows + k, in, pCtx[j].resultInfo->isNullRes); + } + } + } + + pBlock->info.rows += pRow->numOfRows; + } + + pBlock->info.dataLoad = 1; + blockDataUpdateTsWindow(pBlock, 0); + return TSDB_CODE_SUCCESS; +} + +void doBuildStreamIntervalResult(SOperatorInfo* pOperator, void* pState, SSDataBlock* pBlock, + SGroupResInfo* pGroupResInfo) { + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + // set output datablock version + pBlock->info.version = pTaskInfo->version; + + blockDataCleanup(pBlock); + if (!hasRemainResults(pGroupResInfo)) { + return; + } + + // clear the existed group id + pBlock->info.id.groupId = 0; + buildDataBlockFromGroupRes(pOperator, pState, pBlock, &pOperator->exprSupp, pGroupResInfo); +} + +static int32_t getNextQualifiedFinalWindow(SInterval* pInterval, STimeWindow* pNext, SDataBlockInfo* pDataBlockInfo, + TSKEY* primaryKeys, int32_t prevPosition) { + int32_t startPos = prevPosition + 1; + if (startPos == pDataBlockInfo->rows) { + startPos = -1; + } else { + *pNext = getFinalTimeWindow(primaryKeys[startPos], pInterval); + } + return startPos; +} + +static void setStreamDataVersion(SExecTaskInfo* pTaskInfo, int64_t version, int64_t ckId) { + pTaskInfo->streamInfo.dataVersion = version; + pTaskInfo->streamInfo.checkPointId = ckId; +} + +static void doStreamIntervalAggImpl(SOperatorInfo* pOperator, SSDataBlock* pSDataBlock, uint64_t groupId, + SSHashObj* pUpdatedMap) { + SStreamIntervalOperatorInfo* pInfo = (SStreamIntervalOperatorInfo*)pOperator->info; + pInfo->dataVersion = TMAX(pInfo->dataVersion, pSDataBlock->info.version); + + SResultRowInfo* pResultRowInfo = &(pInfo->binfo.resultRowInfo); + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + SExprSupp* pSup = &pOperator->exprSupp; + int32_t numOfOutput = pSup->numOfExprs; + int32_t step = 1; + TSKEY* tsCols = NULL; + SRowBuffPos* pResPos = NULL; + SResultRow* pResult = NULL; + int32_t forwardRows = 0; + + SColumnInfoData* pColDataInfo = taosArrayGet(pSDataBlock->pDataBlock, pInfo->primaryTsIndex); + tsCols = (int64_t*)pColDataInfo->pData; + + int32_t startPos = 0; + TSKEY ts = getStartTsKey(&pSDataBlock->info.window, tsCols); + STimeWindow nextWin = {0}; + if (IS_FINAL_INTERVAL_OP(pOperator)) { + nextWin = getFinalTimeWindow(ts, &pInfo->interval); + } else { + nextWin = getActiveTimeWindow(pInfo->aggSup.pResultBuf, pResultRowInfo, ts, &pInfo->interval, TSDB_ORDER_ASC); + } + while (1) { + bool isClosed = isCloseWindow(&nextWin, &pInfo->twAggSup); + if ((pInfo->ignoreExpiredData && isClosed && !IS_FINAL_INTERVAL_OP(pOperator)) || + !inSlidingWindow(&pInfo->interval, &nextWin, &pSDataBlock->info)) { + startPos = getNexWindowPos(&pInfo->interval, &pSDataBlock->info, tsCols, startPos, nextWin.ekey, &nextWin); + if (startPos < 0) { + break; + } + continue; + } + + if (IS_FINAL_INTERVAL_OP(pOperator) && pInfo->numOfChild > 0) { + bool ignore = true; + SWinKey winRes = { + .ts = nextWin.skey, + .groupId = groupId, + }; + void* chIds = taosHashGet(pInfo->pPullDataMap, &winRes, sizeof(SWinKey)); + if (isDeletedStreamWindow(&nextWin, groupId, pInfo->pState, &pInfo->twAggSup, &pInfo->statestore) && isClosed && + !chIds) { + SPullWindowInfo pull = { + .window = nextWin, .groupId = groupId, .calWin.skey = nextWin.skey, .calWin.ekey = nextWin.skey}; + // add pull data request + if (savePullWindow(&pull, pInfo->pPullWins) == TSDB_CODE_SUCCESS) { + addPullWindow(pInfo->pPullDataMap, &winRes, pInfo->numOfChild); + } + } else { + int32_t index = -1; + SArray* chArray = NULL; + int32_t chId = 0; + if (chIds) { + chArray = *(void**)chIds; + chId = getChildIndex(pSDataBlock); + index = taosArraySearchIdx(chArray, &chId, compareInt32Val, TD_EQ); + } + if (index == -1 || pSDataBlock->info.type == STREAM_PULL_DATA) { + ignore = false; + } + } + + if (ignore) { + startPos = getNextQualifiedFinalWindow(&pInfo->interval, &nextWin, &pSDataBlock->info, tsCols, startPos); + if (startPos < 0) { + break; + } + continue; + } + } + + int32_t code = setIntervalOutputBuf(pInfo->pState, &nextWin, &pResPos, groupId, pSup->pCtx, numOfOutput, + pSup->rowEntryInfoOffset, &pInfo->aggSup, &pInfo->statestore); + pResult = (SResultRow*)pResPos->pRowBuff; + if (code != TSDB_CODE_SUCCESS || pResult == NULL) { + T_LONG_JMP(pTaskInfo->env, TSDB_CODE_OUT_OF_MEMORY); + } + if (IS_FINAL_INTERVAL_OP(pOperator)) { + forwardRows = 1; + } else { + forwardRows = getNumOfRowsInTimeWindow(&pSDataBlock->info, tsCols, startPos, nextWin.ekey, binarySearchForKey, + NULL, TSDB_ORDER_ASC); + } + + SWinKey key = { + .ts = pResult->win.skey, + .groupId = groupId, + }; + if (pInfo->twAggSup.calTrigger == STREAM_TRIGGER_AT_ONCE && pUpdatedMap) { + saveWinResult(&key, pResPos, pUpdatedMap); + } + + if (pInfo->twAggSup.calTrigger == STREAM_TRIGGER_WINDOW_CLOSE) { + tSimpleHashPut(pInfo->aggSup.pResultRowHashTable, &key, sizeof(SWinKey), &pResPos, POINTER_BYTES); + } + + updateTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &nextWin, 1); + applyAggFunctionOnPartialTuples(pTaskInfo, pSup->pCtx, &pInfo->twAggSup.timeWindowData, startPos, forwardRows, + pSDataBlock->info.rows, numOfOutput); + key.ts = nextWin.skey; + + if (pInfo->delKey.ts > key.ts) { + pInfo->delKey = key; + } + int32_t prevEndPos = (forwardRows - 1) * step + startPos; + if (pSDataBlock->info.window.skey <= 0 || pSDataBlock->info.window.ekey <= 0) { + qError("table uid %" PRIu64 " data block timestamp range may not be calculated! minKey %" PRId64 + ",maxKey %" PRId64, + pSDataBlock->info.id.uid, pSDataBlock->info.window.skey, pSDataBlock->info.window.ekey); + blockDataUpdateTsWindow(pSDataBlock, 0); + + // timestamp of the data is incorrect + if (pSDataBlock->info.window.skey <= 0 || pSDataBlock->info.window.ekey <= 0) { + qError("table uid %" PRIu64 " data block timestamp is out of range! minKey %" PRId64 ",maxKey %" PRId64, + pSDataBlock->info.id.uid, pSDataBlock->info.window.skey, pSDataBlock->info.window.ekey); + } + } + + if (IS_FINAL_INTERVAL_OP(pOperator)) { + startPos = getNextQualifiedFinalWindow(&pInfo->interval, &nextWin, &pSDataBlock->info, tsCols, prevEndPos); + } else { + startPos = + getNextQualifiedWindow(&pInfo->interval, &nextWin, &pSDataBlock->info, tsCols, prevEndPos, TSDB_ORDER_ASC); + } + if (startPos < 0) { + break; + } + } +} + +static inline int winPosCmprImpl(const void* pKey1, const void* pKey2) { + SRowBuffPos* pos1 = *(SRowBuffPos**)pKey1; + SRowBuffPos* pos2 = *(SRowBuffPos**)pKey2; + SWinKey* pWin1 = (SWinKey*)pos1->pKey; + SWinKey* pWin2 = (SWinKey*)pos2->pKey; + + if (pWin1->groupId > pWin2->groupId) { + return 1; + } else if (pWin1->groupId < pWin2->groupId) { + return -1; + } + + if (pWin1->ts > pWin2->ts) { + return 1; + } else if (pWin1->ts < pWin2->ts) { + return -1; + } + + return 0; +} + +static void resetUnCloseWinInfo(SSHashObj* winMap) { + void* pIte = NULL; + int32_t iter = 0; + while ((pIte = tSimpleHashIterate(winMap, pIte, &iter)) != NULL) { + SRowBuffPos* pPos = *(SRowBuffPos**)pIte; + pPos->beUsed = true; + } +} + + +static SSDataBlock* buildIntervalResult(SOperatorInfo* pOperator) { + SStreamIntervalOperatorInfo* pInfo = pOperator->info; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + uint16_t opType = pOperator->operatorType; + if (IS_FINAL_INTERVAL_OP(pOperator)) { + doBuildPullDataBlock(pInfo->pPullWins, &pInfo->pullIndex, pInfo->pPullDataRes); + if (pInfo->pPullDataRes->info.rows != 0) { + // process the rest of the data + printDataBlock(pInfo->pPullDataRes, getStreamOpName(opType), GET_TASKID(pTaskInfo)); + return pInfo->pPullDataRes; + } + } + + doBuildDeleteResult(pInfo, pInfo->pDelWins, &pInfo->delIndex, pInfo->pDelRes); + if (pInfo->pDelRes->info.rows != 0) { + // process the rest of the data + printDataBlock(pInfo->pDelRes, getStreamOpName(opType), GET_TASKID(pTaskInfo)); + return pInfo->pDelRes; + } + + doBuildStreamIntervalResult(pOperator, pInfo->pState, pInfo->binfo.pRes, &pInfo->groupResInfo); + if (pInfo->binfo.pRes->info.rows != 0) { + printDataBlock(pInfo->binfo.pRes, getStreamOpName(opType), GET_TASKID(pTaskInfo)); + return pInfo->binfo.pRes; + } + return NULL; +} + +static SSDataBlock* doStreamFinalIntervalAgg(SOperatorInfo* pOperator) { + SStreamIntervalOperatorInfo* pInfo = pOperator->info; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + SStorageAPI* pAPI = &pOperator->pTaskInfo->storageAPI; + + SOperatorInfo* downstream = pOperator->pDownstream[0]; + SExprSupp* pSup = &pOperator->exprSupp; + + qDebug("stask:%s %s status: %d", GET_TASKID(pTaskInfo), getStreamOpName(pOperator->operatorType), pOperator->status); + + if (pOperator->status == OP_EXEC_DONE) { + return NULL; + } else if (pOperator->status == OP_RES_TO_RETURN) { + SSDataBlock* resBlock = buildIntervalResult(pOperator); + if (resBlock != NULL) { + return resBlock; + } + + if (pInfo->recvGetAll) { + pInfo->recvGetAll = false; + resetUnCloseWinInfo(pInfo->aggSup.pResultRowHashTable); + } + + setOperatorCompleted(pOperator); + if (!IS_FINAL_INTERVAL_OP(pOperator)) { + clearFunctionContext(&pOperator->exprSupp); + // semi interval operator clear disk buffer + clearStreamIntervalOperator(pInfo); + setStreamDataVersion(pTaskInfo, pInfo->dataVersion, pInfo->pState->checkPointId); + qDebug("stask:%s ===stream===%s clear", GET_TASKID(pTaskInfo), getStreamOpName(pOperator->operatorType)); + } else { + if (pInfo->twAggSup.maxTs > 0 && + pInfo->twAggSup.maxTs - pInfo->twAggSup.checkPointInterval > pInfo->twAggSup.checkPointTs) { + pAPI->stateStore.streamStateCommit(pInfo->pState); + pAPI->stateStore.streamStateDeleteCheckPoint(pInfo->pState, pInfo->twAggSup.maxTs - pInfo->twAggSup.deleteMark); + pInfo->twAggSup.checkPointTs = pInfo->twAggSup.maxTs; + } + qDebug("stask:%s ===stream===%s close", GET_TASKID(pTaskInfo), getStreamOpName(pOperator->operatorType)); + } + return NULL; + } else { + if (!IS_FINAL_INTERVAL_OP(pOperator)) { + doBuildDeleteResult(pInfo, pInfo->pDelWins, &pInfo->delIndex, pInfo->pDelRes); + if (pInfo->pDelRes->info.rows != 0) { + // process the rest of the data + printDataBlock(pInfo->pDelRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); + return pInfo->pDelRes; + } + } + } + + if (!pInfo->pUpdated) { + pInfo->pUpdated = taosArrayInit(4096, POINTER_BYTES); + } + if (!pInfo->pUpdatedMap) { + _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); + pInfo->pUpdatedMap = tSimpleHashInit(4096, hashFn); + } + + while (1) { + if (isTaskKilled(pTaskInfo)) { + if (pInfo->pUpdated != NULL) { + pInfo->pUpdated = taosArrayDestroy(pInfo->pUpdated); + } + + if (pInfo->pUpdatedMap != NULL) { + tSimpleHashCleanup(pInfo->pUpdatedMap); + pInfo->pUpdatedMap = NULL; + } + + T_LONG_JMP(pTaskInfo->env, pTaskInfo->code); + } + + SSDataBlock* pBlock = downstream->fpSet.getNextFn(downstream); + if (pBlock == NULL) { + pOperator->status = OP_RES_TO_RETURN; + qDebug("===stream===return data:%s. recv datablock num:%" PRIu64, getStreamOpName(pOperator->operatorType), + pInfo->numOfDatapack); + pInfo->numOfDatapack = 0; + break; + } + pInfo->numOfDatapack++; + printSpecDataBlock(pBlock, getStreamOpName(pOperator->operatorType), "recv", GET_TASKID(pTaskInfo)); + + if (pBlock->info.type == STREAM_NORMAL || pBlock->info.type == STREAM_PULL_DATA) { + pInfo->binfo.pRes->info.type = pBlock->info.type; + } else if (pBlock->info.type == STREAM_DELETE_DATA || pBlock->info.type == STREAM_DELETE_RESULT || + pBlock->info.type == STREAM_CLEAR) { + SArray* delWins = taosArrayInit(8, sizeof(SWinKey)); + doDeleteWindows(pOperator, &pInfo->interval, pBlock, delWins, pInfo->pUpdatedMap); + if (IS_FINAL_INTERVAL_OP(pOperator)) { + int32_t chId = getChildIndex(pBlock); + addRetriveWindow(delWins, pInfo, chId); + if (pBlock->info.type != STREAM_CLEAR) { + taosArrayAddAll(pInfo->pDelWins, delWins); + } + taosArrayDestroy(delWins); + continue; + } + removeResults(delWins, pInfo->pUpdatedMap); + taosArrayAddAll(pInfo->pDelWins, delWins); + taosArrayDestroy(delWins); + + doBuildDeleteResult(pInfo, pInfo->pDelWins, &pInfo->delIndex, pInfo->pDelRes); + if (pInfo->pDelRes->info.rows != 0) { + // process the rest of the data + printDataBlock(pInfo->pDelRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); + if (pBlock->info.type == STREAM_CLEAR) { + pInfo->pDelRes->info.type = STREAM_CLEAR; + } else { + pInfo->pDelRes->info.type = STREAM_DELETE_RESULT; + } + return pInfo->pDelRes; + } + + break; + } else if (pBlock->info.type == STREAM_GET_ALL && IS_FINAL_INTERVAL_OP(pOperator)) { + pInfo->recvGetAll = true; + getAllIntervalWindow(pInfo->aggSup.pResultRowHashTable, pInfo->pUpdatedMap); + continue; + } else if (pBlock->info.type == STREAM_RETRIEVE && !IS_FINAL_INTERVAL_OP(pOperator)) { + doDeleteWindows(pOperator, &pInfo->interval, pBlock, NULL, pInfo->pUpdatedMap); + if (taosArrayGetSize(pInfo->pUpdated) > 0) { + break; + } + continue; + } else if (pBlock->info.type == STREAM_PULL_OVER && IS_FINAL_INTERVAL_OP(pOperator)) { + processPullOver(pBlock, pInfo->pPullDataMap, pInfo->pFinalPullDataMap, &pInfo->interval, pInfo->pPullWins, + pInfo->numOfChild, pOperator); + continue; + } else if (pBlock->info.type == STREAM_CREATE_CHILD_TABLE) { + return pBlock; + } else { + ASSERTS(pBlock->info.type == STREAM_INVALID, "invalid SSDataBlock type"); + } + + if (pInfo->scalarSupp.pExprInfo != NULL) { + SExprSupp* pExprSup = &pInfo->scalarSupp; + projectApplyFunctions(pExprSup->pExprInfo, pBlock, pBlock, pExprSup->pCtx, pExprSup->numOfExprs, NULL); + } + setInputDataBlock(pSup, pBlock, TSDB_ORDER_ASC, MAIN_SCAN, true); + doStreamIntervalAggImpl(pOperator, pBlock, pBlock->info.id.groupId, pInfo->pUpdatedMap); + pInfo->twAggSup.maxTs = TMAX(pInfo->twAggSup.maxTs, pBlock->info.window.ekey); + pInfo->twAggSup.maxTs = TMAX(pInfo->twAggSup.maxTs, pBlock->info.watermark); + pInfo->twAggSup.minTs = TMIN(pInfo->twAggSup.minTs, pBlock->info.window.skey); + } + + removeDeleteResults(pInfo->pUpdatedMap, pInfo->pDelWins); + if (IS_FINAL_INTERVAL_OP(pOperator)) { + closeStreamIntervalWindow(pInfo->aggSup.pResultRowHashTable, &pInfo->twAggSup, &pInfo->interval, + pInfo->pPullDataMap, pInfo->pUpdatedMap, pInfo->pDelWins, pOperator); + } + pInfo->binfo.pRes->info.watermark = pInfo->twAggSup.maxTs; + + void* pIte = NULL; + int32_t iter = 0; + while ((pIte = tSimpleHashIterate(pInfo->pUpdatedMap, pIte, &iter)) != NULL) { + taosArrayPush(pInfo->pUpdated, pIte); + } + + tSimpleHashCleanup(pInfo->pUpdatedMap); + pInfo->pUpdatedMap = NULL; + taosArraySort(pInfo->pUpdated, winPosCmprImpl); + + initMultiResInfoFromArrayList(&pInfo->groupResInfo, pInfo->pUpdated); + pInfo->pUpdated = NULL; + blockDataEnsureCapacity(pInfo->binfo.pRes, pOperator->resultInfo.capacity); + + return buildIntervalResult(pOperator); +} + +static int64_t getDeleteMark(SIntervalPhysiNode* pIntervalPhyNode) { + if (pIntervalPhyNode->window.deleteMark <= 0) { + return DEAULT_DELETE_MARK; + } + int64_t deleteMark = TMAX(pIntervalPhyNode->window.deleteMark, pIntervalPhyNode->window.watermark); + deleteMark = TMAX(deleteMark, pIntervalPhyNode->interval); + return deleteMark; +} + +static TSKEY compareTs(void* pKey) { + SWinKey* pWinKey = (SWinKey*)pKey; + return pWinKey->ts; +} + +static int32_t getSelectivityBufSize(SqlFunctionCtx* pCtx) { + if (pCtx->subsidiaries.rowLen == 0) { + int32_t rowLen = 0; + for (int32_t j = 0; j < pCtx->subsidiaries.num; ++j) { + SqlFunctionCtx* pc = pCtx->subsidiaries.pCtx[j]; + rowLen += pc->pExpr->base.resSchema.bytes; + } + + return rowLen + pCtx->subsidiaries.num * sizeof(bool); + } else { + return pCtx->subsidiaries.rowLen; + } +} + +static int32_t getMaxFunResSize(SExprSupp* pSup, int32_t numOfCols) { + int32_t size = 0; + for (int32_t i = 0; i < numOfCols; ++i) { + int32_t resSize = getSelectivityBufSize(pSup->pCtx + i); + size = TMAX(size, resSize); + } + return size; +} + +static void streamIntervalReleaseState(SOperatorInfo* pOperator) { + if (pOperator->operatorType != QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL) { + SStreamIntervalOperatorInfo* pInfo = pOperator->info; + int32_t resSize = sizeof(TSKEY); + pInfo->statestore.streamStateSaveInfo(pInfo->pState, STREAM_INTERVAL_OP_STATE_NAME, + strlen(STREAM_INTERVAL_OP_STATE_NAME), &pInfo->twAggSup.maxTs, resSize); + } + SStreamIntervalOperatorInfo* pInfo = pOperator->info; + SStorageAPI* pAPI = &pOperator->pTaskInfo->storageAPI; + pAPI->stateStore.streamStateCommit(pInfo->pState); + SOperatorInfo* downstream = pOperator->pDownstream[0]; + if (downstream->fpSet.releaseStreamStateFn) { + downstream->fpSet.releaseStreamStateFn(downstream); + } +} + +void streamIntervalReloadState(SOperatorInfo* pOperator) { + if (pOperator->operatorType != QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL) { + SStreamIntervalOperatorInfo* pInfo = pOperator->info; + int32_t size = 0; + void* pBuf = NULL; + int32_t code = pInfo->statestore.streamStateGetInfo(pInfo->pState, STREAM_INTERVAL_OP_STATE_NAME, + strlen(STREAM_INTERVAL_OP_STATE_NAME), &pBuf, &size); + TSKEY ts = *(TSKEY*)pBuf; + taosMemoryFree(pBuf); + pInfo->twAggSup.maxTs = TMAX(pInfo->twAggSup.maxTs, ts); + pInfo->statestore.streamStateReloadInfo(pInfo->pState, ts); + } + SOperatorInfo* downstream = pOperator->pDownstream[0]; + if (downstream->fpSet.reloadStreamStateFn) { + downstream->fpSet.reloadStreamStateFn(downstream); + } +} + +SOperatorInfo* createStreamFinalIntervalOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, + SExecTaskInfo* pTaskInfo, int32_t numOfChild) { + SIntervalPhysiNode* pIntervalPhyNode = (SIntervalPhysiNode*)pPhyNode; + SStreamIntervalOperatorInfo* pInfo = taosMemoryCalloc(1, sizeof(SStreamIntervalOperatorInfo)); + SOperatorInfo* pOperator = taosMemoryCalloc(1, sizeof(SOperatorInfo)); + if (pInfo == NULL || pOperator == NULL) { + goto _error; + } + + pOperator->pTaskInfo = pTaskInfo; + SStorageAPI* pAPI = &pTaskInfo->storageAPI; + + pInfo->interval = (SInterval){.interval = pIntervalPhyNode->interval, + .sliding = pIntervalPhyNode->sliding, + .intervalUnit = pIntervalPhyNode->intervalUnit, + .slidingUnit = pIntervalPhyNode->slidingUnit, + .offset = pIntervalPhyNode->offset, + .precision = ((SColumnNode*)pIntervalPhyNode->window.pTspk)->node.resType.precision}; + pInfo->twAggSup = (STimeWindowAggSupp){ + .waterMark = pIntervalPhyNode->window.watermark, + .calTrigger = pIntervalPhyNode->window.triggerType, + .maxTs = INT64_MIN, + .minTs = INT64_MAX, + .deleteMark = getDeleteMark(pIntervalPhyNode), + .deleteMarkSaved = 0, + .calTriggerSaved = 0, + .checkPointTs = 0, + .checkPointInterval = + convertTimePrecision(tsCheckpointInterval, TSDB_TIME_PRECISION_MILLI, pInfo->interval.precision), + }; + ASSERTS(pInfo->twAggSup.calTrigger != STREAM_TRIGGER_MAX_DELAY, "trigger type should not be max delay"); + pInfo->primaryTsIndex = ((SColumnNode*)pIntervalPhyNode->window.pTspk)->slotId; + size_t keyBufSize = sizeof(int64_t) + sizeof(int64_t) + POINTER_BYTES; + initResultSizeInfo(&pOperator->resultInfo, 4096); + if (pIntervalPhyNode->window.pExprs != NULL) { + int32_t numOfScalar = 0; + SExprInfo* pScalarExprInfo = createExprInfo(pIntervalPhyNode->window.pExprs, NULL, &numOfScalar); + int32_t code = initExprSupp(&pInfo->scalarSupp, pScalarExprInfo, numOfScalar, &pTaskInfo->storageAPI.functionStore); + if (code != TSDB_CODE_SUCCESS) { + goto _error; + } + } + + int32_t numOfCols = 0; + SExprInfo* pExprInfo = createExprInfo(pIntervalPhyNode->window.pFuncs, NULL, &numOfCols); + SSDataBlock* pResBlock = createDataBlockFromDescNode(pPhyNode->pOutputDataBlockDesc); + initBasicInfo(&pInfo->binfo, pResBlock); + + pInfo->pState = taosMemoryCalloc(1, sizeof(SStreamState)); + *(pInfo->pState) = *(pTaskInfo->streamInfo.pState); + + pAPI->stateStore.streamStateSetNumber(pInfo->pState, -1); + int32_t code = initAggSup(&pOperator->exprSupp, &pInfo->aggSup, pExprInfo, numOfCols, keyBufSize, pTaskInfo->id.str, + pInfo->pState, &pTaskInfo->storageAPI.functionStore); + if (code != TSDB_CODE_SUCCESS) { + goto _error; + } + + initExecTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &pTaskInfo->window); + initResultRowInfo(&pInfo->binfo.resultRowInfo); + + pInfo->numOfChild = numOfChild; + pInfo->pPhyNode = (SPhysiNode*)nodesCloneNode((SNode*)pPhyNode); + + pInfo->pPullWins = taosArrayInit(8, sizeof(SPullWindowInfo)); + pInfo->pullIndex = 0; + _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); + pInfo->pPullDataMap = taosHashInit(64, hashFn, false, HASH_NO_LOCK); + pInfo->pFinalPullDataMap = taosHashInit(64, hashFn, false, HASH_NO_LOCK); + pInfo->pPullDataRes = createSpecialDataBlock(STREAM_RETRIEVE); + pInfo->ignoreExpiredData = pIntervalPhyNode->window.igExpired; + pInfo->ignoreExpiredDataSaved = false; + pInfo->pDelRes = createSpecialDataBlock(STREAM_DELETE_RESULT); + pInfo->delIndex = 0; + pInfo->pDelWins = taosArrayInit(4, sizeof(SWinKey)); + pInfo->delKey.ts = INT64_MAX; + pInfo->delKey.groupId = 0; + pInfo->numOfDatapack = 0; + pInfo->pUpdated = NULL; + pInfo->pUpdatedMap = NULL; + int32_t funResSize = getMaxFunResSize(&pOperator->exprSupp, numOfCols); + pInfo->pState->pFileState = + pAPI->stateStore.streamFileStateInit(tsStreamBufferSize, sizeof(SWinKey), pInfo->aggSup.resultRowSize, funResSize, + compareTs, pInfo->pState, pInfo->twAggSup.deleteMark, GET_TASKID(pTaskInfo)); + pInfo->dataVersion = 0; + pInfo->statestore = pTaskInfo->storageAPI.stateStore; + pInfo->recvGetAll = false; + + pOperator->operatorType = pPhyNode->type; + if (!IS_FINAL_INTERVAL_OP(pOperator) || numOfChild == 0) { + pInfo->twAggSup.calTrigger = STREAM_TRIGGER_AT_ONCE; + } + pOperator->name = getStreamOpName(pOperator->operatorType); + pOperator->blocking = true; + pOperator->status = OP_NOT_OPENED; + pOperator->info = pInfo; + + pOperator->fpSet = createOperatorFpSet(NULL, doStreamFinalIntervalAgg, NULL, destroyStreamFinalIntervalOperatorInfo, + optrDefaultBufFn, NULL); + setOperatorStreamStateFn(pOperator, streamIntervalReleaseState, streamIntervalReloadState); + if (pPhyNode->type == QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL) { + initIntervalDownStream(downstream, pPhyNode->type, pInfo); + } + code = appendDownstream(pOperator, &downstream, 1); + if (code != TSDB_CODE_SUCCESS) { + goto _error; + } + + return pOperator; + +_error: + destroyStreamFinalIntervalOperatorInfo(pInfo); + taosMemoryFreeClear(pOperator); + pTaskInfo->code = code; + return NULL; +} + +void destroyStreamAggSupporter(SStreamAggSupporter* pSup) { + tSimpleHashCleanup(pSup->pResultRows); + destroyDiskbasedBuf(pSup->pResultBuf); + blockDataDestroy(pSup->pScanBlock); + taosMemoryFreeClear(pSup->pState); + taosMemoryFreeClear(pSup->pDummyCtx); +} + +void destroyStreamSessionAggOperatorInfo(void* param) { + SStreamSessionAggOperatorInfo* pInfo = (SStreamSessionAggOperatorInfo*)param; + cleanupBasicInfo(&pInfo->binfo); + destroyStreamAggSupporter(&pInfo->streamAggSup); + + if (pInfo->pChildren != NULL) { + int32_t size = taosArrayGetSize(pInfo->pChildren); + for (int32_t i = 0; i < size; i++) { + SOperatorInfo* pChild = taosArrayGetP(pInfo->pChildren, i); + destroyOperator(pChild); + } + taosArrayDestroy(pInfo->pChildren); + } + + colDataDestroy(&pInfo->twAggSup.timeWindowData); + blockDataDestroy(pInfo->pDelRes); + blockDataDestroy(pInfo->pWinBlock); + blockDataDestroy(pInfo->pUpdateRes); + tSimpleHashCleanup(pInfo->pStUpdated); + tSimpleHashCleanup(pInfo->pStDeleted); + + taosArrayDestroy(pInfo->historyWins); + taosMemoryFreeClear(param); +} + +int32_t initBasicInfoEx(SOptrBasicInfo* pBasicInfo, SExprSupp* pSup, SExprInfo* pExprInfo, int32_t numOfCols, + SSDataBlock* pResultBlock, SFunctionStateStore* pStore) { + initBasicInfo(pBasicInfo, pResultBlock); + int32_t code = initExprSupp(pSup, pExprInfo, numOfCols, pStore); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + + for (int32_t i = 0; i < numOfCols; ++i) { + pSup->pCtx[i].saveHandle.pBuf = NULL; + } + + ASSERT(numOfCols > 0); + return TSDB_CODE_SUCCESS; +} + +void initDummyFunction(SqlFunctionCtx* pDummy, SqlFunctionCtx* pCtx, int32_t nums) { + for (int i = 0; i < nums; i++) { + pDummy[i].functionId = pCtx[i].functionId; + pDummy[i].isNotNullFunc = pCtx[i].isNotNullFunc; + pDummy[i].isPseudoFunc = pCtx[i].isPseudoFunc; + } +} + +void initDownStream(SOperatorInfo* downstream, SStreamAggSupporter* pAggSup, uint16_t type, int32_t tsColIndex, + STimeWindowAggSupp* pTwSup) { + if (downstream->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_PARTITION) { + SStreamPartitionOperatorInfo* pScanInfo = downstream->info; + pScanInfo->tsColIndex = tsColIndex; + } + + if (downstream->operatorType != QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN) { + initDownStream(downstream->pDownstream[0], pAggSup, type, tsColIndex, pTwSup); + return; + } + SStreamScanInfo* pScanInfo = downstream->info; + pScanInfo->windowSup = (SWindowSupporter){.pStreamAggSup = pAggSup, .gap = pAggSup->gap, .parentType = type}; + pScanInfo->pState = pAggSup->pState; + if (!pScanInfo->pUpdateInfo) { + pScanInfo->pUpdateInfo = pAggSup->stateStore.updateInfoInit(60000, TSDB_TIME_PRECISION_MILLI, pTwSup->waterMark); + } + pScanInfo->twAggSup = *pTwSup; +} + +int32_t initStreamAggSupporter(SStreamAggSupporter* pSup, SqlFunctionCtx* pCtx, int32_t numOfOutput, int64_t gap, + SStreamState* pState, int32_t keySize, int16_t keyType, SStateStore* pStore, + SReadHandle* pHandle, SStorageAPI* pApi) { + pSup->resultRowSize = keySize + getResultRowSize(pCtx, numOfOutput); + pSup->pScanBlock = createSpecialDataBlock(STREAM_CLEAR); + pSup->gap = gap; + pSup->stateKeySize = keySize; + pSup->stateKeyType = keyType; + pSup->pDummyCtx = (SqlFunctionCtx*)taosMemoryCalloc(numOfOutput, sizeof(SqlFunctionCtx)); + if (pSup->pDummyCtx == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + + pSup->stateStore = *pStore; + + initDummyFunction(pSup->pDummyCtx, pCtx, numOfOutput); + pSup->pState = taosMemoryCalloc(1, sizeof(SStreamState)); + *(pSup->pState) = *pState; + pSup->stateStore.streamStateSetNumber(pSup->pState, -1); + + _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); + pSup->pResultRows = tSimpleHashInit(32, hashFn); + + int32_t pageSize = 4096; + while (pageSize < pSup->resultRowSize * 4) { + pageSize <<= 1u; + } + // at least four pages need to be in buffer + int32_t bufSize = 4096 * 256; + if (bufSize <= pageSize) { + bufSize = pageSize * 4; + } + + if (!osTempSpaceAvailable()) { + terrno = TSDB_CODE_NO_DISKSPACE; + qError("Init stream agg supporter failed since %s, tempDir:%s", terrstr(), tsTempDir); + return terrno; + } + + int32_t code = createDiskbasedBuf(&pSup->pResultBuf, pageSize, bufSize, "function", tsTempDir); + for (int32_t i = 0; i < numOfOutput; ++i) { + pCtx[i].saveHandle.pBuf = pSup->pResultBuf; + } + + pSup->pSessionAPI = pApi; + + return TSDB_CODE_SUCCESS; +} + +bool isInTimeWindow(STimeWindow* pWin, TSKEY ts, int64_t gap) { + if (ts + gap >= pWin->skey && ts - gap <= pWin->ekey) { + return true; + } + return false; +} + +bool isInWindow(SResultWindowInfo* pWinInfo, TSKEY ts, int64_t gap) { + return isInTimeWindow(&pWinInfo->sessionWin.win, ts, gap); +} + +void getCurSessionWindow(SStreamAggSupporter* pAggSup, TSKEY startTs, TSKEY endTs, uint64_t groupId, + SSessionKey* pKey) { + pKey->win.skey = startTs; + pKey->win.ekey = endTs; + pKey->groupId = groupId; + int32_t code = pAggSup->stateStore.streamStateSessionGetKeyByRange(pAggSup->pState, pKey, pKey); + if (code != TSDB_CODE_SUCCESS) { + SET_SESSION_WIN_KEY_INVALID(pKey); + } +} + +bool isInvalidSessionWin(SResultWindowInfo* pWinInfo) { return pWinInfo->sessionWin.win.skey == 0; } + +bool inWinRange(STimeWindow* range, STimeWindow* cur) { + if (cur->skey >= range->skey && cur->ekey <= range->ekey) { + return true; + } + return false; +} + +void setSessionOutputBuf(SStreamAggSupporter* pAggSup, TSKEY startTs, TSKEY endTs, uint64_t groupId, + SResultWindowInfo* pCurWin) { + pCurWin->sessionWin.groupId = groupId; + pCurWin->sessionWin.win.skey = startTs; + pCurWin->sessionWin.win.ekey = endTs; + int32_t size = pAggSup->resultRowSize; + int32_t code = pAggSup->stateStore.streamStateSessionAddIfNotExist(pAggSup->pState, &pCurWin->sessionWin, + pAggSup->gap, &pCurWin->pOutputBuf, &size); + if (code == TSDB_CODE_SUCCESS && !inWinRange(&pAggSup->winRange, &pCurWin->sessionWin.win)) { + code = TSDB_CODE_FAILED; + releaseOutputBuf(pAggSup->pState, NULL, (SResultRow*)pCurWin->pOutputBuf, &pAggSup->pSessionAPI->stateStore); + pCurWin->pOutputBuf = taosMemoryCalloc(1, size); + } + + if (code == TSDB_CODE_SUCCESS) { + pCurWin->isOutput = true; + pAggSup->stateStore.streamStateSessionDel(pAggSup->pState, &pCurWin->sessionWin); + } else { + pCurWin->sessionWin.win.skey = startTs; + pCurWin->sessionWin.win.ekey = endTs; + } +} + +int32_t getSessionWinBuf(SStreamAggSupporter* pAggSup, SStreamStateCur* pCur, SResultWindowInfo* pWinInfo) { + int32_t size = 0; + int32_t code = + pAggSup->stateStore.streamStateSessionGetKVByCur(pCur, &pWinInfo->sessionWin, &pWinInfo->pOutputBuf, &size); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + + pAggSup->stateStore.streamStateCurNext(pAggSup->pState, pCur); + return TSDB_CODE_SUCCESS; +} +void saveDeleteInfo(SArray* pWins, SSessionKey key) { + // key.win.ekey = key.win.skey; + taosArrayPush(pWins, &key); +} + +void saveDeleteRes(SSHashObj* pStDelete, SSessionKey key) { + key.win.ekey = key.win.skey; + tSimpleHashPut(pStDelete, &key, sizeof(SSessionKey), NULL, 0); +} + +static void removeSessionResult(SSHashObj* pHashMap, SSHashObj* pResMap, SSessionKey key) { + key.win.ekey = key.win.skey; + tSimpleHashRemove(pHashMap, &key, sizeof(SSessionKey)); + tSimpleHashRemove(pResMap, &key, sizeof(SSessionKey)); +} + +static void getSessionHashKey(const SSessionKey* pKey, SSessionKey* pHashKey) { + *pHashKey = *pKey; + pHashKey->win.ekey = pKey->win.skey; +} + +static void removeSessionResults(SSHashObj* pHashMap, SArray* pWins) { + if (tSimpleHashGetSize(pHashMap) == 0) { + return; + } + int32_t size = taosArrayGetSize(pWins); + for (int32_t i = 0; i < size; i++) { + SSessionKey* pWin = taosArrayGet(pWins, i); + if (!pWin) continue; + SSessionKey key = {0}; + getSessionHashKey(pWin, &key); + tSimpleHashRemove(pHashMap, &key, sizeof(SSessionKey)); + } +} + +int32_t updateSessionWindowInfo(SResultWindowInfo* pWinInfo, TSKEY* pStartTs, TSKEY* pEndTs, uint64_t groupId, + int32_t rows, int32_t start, int64_t gap, SSHashObj* pResultRows, SSHashObj* pStUpdated, + SSHashObj* pStDeleted) { + for (int32_t i = start; i < rows; ++i) { + if (!isInWindow(pWinInfo, pStartTs[i], gap) && (!pEndTs || !isInWindow(pWinInfo, pEndTs[i], gap))) { + return i - start; + } + if (pWinInfo->sessionWin.win.skey > pStartTs[i]) { + if (pStDeleted && pWinInfo->isOutput) { + saveDeleteRes(pStDeleted, pWinInfo->sessionWin); + } + removeSessionResult(pStUpdated, pResultRows, pWinInfo->sessionWin); + pWinInfo->sessionWin.win.skey = pStartTs[i]; + } + pWinInfo->sessionWin.win.ekey = TMAX(pWinInfo->sessionWin.win.ekey, pStartTs[i]); + if (pEndTs) { + pWinInfo->sessionWin.win.ekey = TMAX(pWinInfo->sessionWin.win.ekey, pEndTs[i]); + } + } + return rows - start; +} + +static int32_t initSessionOutputBuf(SResultWindowInfo* pWinInfo, SResultRow** pResult, SqlFunctionCtx* pCtx, + int32_t numOfOutput, int32_t* rowEntryInfoOffset) { + ASSERT(pWinInfo->sessionWin.win.skey <= pWinInfo->sessionWin.win.ekey); + *pResult = (SResultRow*)pWinInfo->pOutputBuf; + // set time window for current result + (*pResult)->win = pWinInfo->sessionWin.win; + setResultRowInitCtx(*pResult, pCtx, numOfOutput, rowEntryInfoOffset); + return TSDB_CODE_SUCCESS; +} + +static int32_t doOneWindowAggImpl(SColumnInfoData* pTimeWindowData, SResultWindowInfo* pCurWin, SResultRow** pResult, + int32_t startIndex, int32_t winRows, int32_t rows, int32_t numOutput, + SOperatorInfo* pOperator, int64_t winDelta) { + SExprSupp* pSup = &pOperator->exprSupp; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + int32_t code = initSessionOutputBuf(pCurWin, pResult, pSup->pCtx, numOutput, pSup->rowEntryInfoOffset); + if (code != TSDB_CODE_SUCCESS || (*pResult) == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + updateTimeWindowInfo(pTimeWindowData, &pCurWin->sessionWin.win, winDelta); + applyAggFunctionOnPartialTuples(pTaskInfo, pSup->pCtx, pTimeWindowData, startIndex, winRows, rows, numOutput); + return TSDB_CODE_SUCCESS; +} + +static bool doDeleteSessionWindow(SStreamAggSupporter* pAggSup, SSessionKey* pKey) { + pAggSup->stateStore.streamStateSessionDel(pAggSup->pState, pKey); + SSessionKey hashKey = {0}; + getSessionHashKey(pKey, &hashKey); + tSimpleHashRemove(pAggSup->pResultRows, &hashKey, sizeof(SSessionKey)); + return true; +} + +static int32_t setSessionWinOutputInfo(SSHashObj* pStUpdated, SResultWindowInfo* pWinInfo) { + void* pVal = tSimpleHashGet(pStUpdated, &pWinInfo->sessionWin, sizeof(SSessionKey)); + if (pVal) { + SResultWindowInfo* pWin = pVal; + pWinInfo->isOutput = pWin->isOutput; + } + return TSDB_CODE_SUCCESS; +} + +SStreamStateCur* getNextSessionWinInfo(SStreamAggSupporter* pAggSup, SSHashObj* pStUpdated, SResultWindowInfo* pCurWin, + SResultWindowInfo* pNextWin) { + SStreamStateCur* pCur = pAggSup->stateStore.streamStateSessionSeekKeyNext(pAggSup->pState, &pCurWin->sessionWin); + pNextWin->isOutput = true; + setSessionWinOutputInfo(pStUpdated, pNextWin); + int32_t size = 0; + pNextWin->sessionWin = pCurWin->sessionWin; + int32_t code = + pAggSup->stateStore.streamStateSessionGetKVByCur(pCur, &pNextWin->sessionWin, &pNextWin->pOutputBuf, &size); + if (code != TSDB_CODE_SUCCESS) { + taosMemoryFreeClear(pNextWin->pOutputBuf); + SET_SESSION_WIN_INVALID(*pNextWin); + } + return pCur; +} + +static int32_t compactSessionWindow(SOperatorInfo* pOperator, SResultWindowInfo* pCurWin, SSHashObj* pStUpdated, + SSHashObj* pStDeleted, bool addGap) { + SExprSupp* pSup = &pOperator->exprSupp; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + SStorageAPI* pAPI = &pOperator->pTaskInfo->storageAPI; + int32_t winNum = 0; + + SStreamSessionAggOperatorInfo* pInfo = pOperator->info; + SResultRow* pCurResult = NULL; + int32_t numOfOutput = pOperator->exprSupp.numOfExprs; + SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; + initSessionOutputBuf(pCurWin, &pCurResult, pSup->pCtx, numOfOutput, pSup->rowEntryInfoOffset); + // Just look for the window behind StartIndex + while (1) { + SResultWindowInfo winInfo = {0}; + SStreamStateCur* pCur = getNextSessionWinInfo(pAggSup, pStUpdated, pCurWin, &winInfo); + if (!IS_VALID_SESSION_WIN(winInfo) || !isInWindow(pCurWin, winInfo.sessionWin.win.skey, pAggSup->gap) || + !inWinRange(&pAggSup->winRange, &winInfo.sessionWin.win)) { + taosMemoryFree(winInfo.pOutputBuf); + pAPI->stateStore.streamStateFreeCur(pCur); + break; + } + SResultRow* pWinResult = NULL; + initSessionOutputBuf(&winInfo, &pWinResult, pAggSup->pDummyCtx, numOfOutput, pSup->rowEntryInfoOffset); + pCurWin->sessionWin.win.ekey = TMAX(pCurWin->sessionWin.win.ekey, winInfo.sessionWin.win.ekey); + int64_t winDelta = 0; + if (addGap) { + winDelta = pAggSup->gap; + } + updateTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &pCurWin->sessionWin.win, winDelta); + compactFunctions(pSup->pCtx, pAggSup->pDummyCtx, numOfOutput, pTaskInfo, &pInfo->twAggSup.timeWindowData); + tSimpleHashRemove(pStUpdated, &winInfo.sessionWin, sizeof(SSessionKey)); + if (winInfo.isOutput && pStDeleted) { + saveDeleteRes(pStDeleted, winInfo.sessionWin); + } + removeSessionResult(pStUpdated, pAggSup->pResultRows, winInfo.sessionWin); + doDeleteSessionWindow(pAggSup, &winInfo.sessionWin); + pAPI->stateStore.streamStateFreeCur(pCur); + taosMemoryFree(winInfo.pOutputBuf); + winNum++; + } + return winNum; +} + +int32_t saveSessionOutputBuf(SStreamAggSupporter* pAggSup, SResultWindowInfo* pWinInfo) { + saveSessionDiscBuf(pAggSup->pState, &pWinInfo->sessionWin, pWinInfo->pOutputBuf, pAggSup->resultRowSize, + &pAggSup->stateStore); + pWinInfo->pOutputBuf = NULL; + return TSDB_CODE_SUCCESS; +} + +static void doStreamSessionAggImpl(SOperatorInfo* pOperator, SSDataBlock* pSDataBlock, SSHashObj* pStUpdated, + SSHashObj* pStDeleted, bool hasEndTs, bool addGap) { + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + SStreamSessionAggOperatorInfo* pInfo = pOperator->info; + int32_t numOfOutput = pOperator->exprSupp.numOfExprs; + uint64_t groupId = pSDataBlock->info.id.groupId; + int64_t code = TSDB_CODE_SUCCESS; + SResultRow* pResult = NULL; + int32_t rows = pSDataBlock->info.rows; + int32_t winRows = 0; + SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; + + pInfo->dataVersion = TMAX(pInfo->dataVersion, pSDataBlock->info.version); + pAggSup->winRange = pTaskInfo->streamInfo.fillHistoryWindow; + if (pAggSup->winRange.ekey <= 0) { + pAggSup->winRange.ekey = INT64_MAX; + } + + SColumnInfoData* pStartTsCol = taosArrayGet(pSDataBlock->pDataBlock, pInfo->primaryTsIndex); + TSKEY* startTsCols = (int64_t*)pStartTsCol->pData; + SColumnInfoData* pEndTsCol = NULL; + if (hasEndTs) { + pEndTsCol = taosArrayGet(pSDataBlock->pDataBlock, pInfo->endTsIndex); + } else { + pEndTsCol = taosArrayGet(pSDataBlock->pDataBlock, pInfo->primaryTsIndex); + } + + TSKEY* endTsCols = (int64_t*)pEndTsCol->pData; + for (int32_t i = 0; i < rows;) { + if (pInfo->ignoreExpiredData && isOverdue(endTsCols[i], &pInfo->twAggSup)) { + i++; + continue; + } + SResultWindowInfo winInfo = {0}; + setSessionOutputBuf(pAggSup, startTsCols[i], endTsCols[i], groupId, &winInfo); + setSessionWinOutputInfo(pStUpdated, &winInfo); + winRows = updateSessionWindowInfo(&winInfo, startTsCols, endTsCols, groupId, rows, i, pAggSup->gap, + pAggSup->pResultRows, pStUpdated, pStDeleted); + // coverity scan error + if (!winInfo.pOutputBuf) { + T_LONG_JMP(pTaskInfo->env, TSDB_CODE_OUT_OF_MEMORY); + } + + int64_t winDelta = 0; + if (addGap) { + winDelta = pAggSup->gap; + } + code = doOneWindowAggImpl(&pInfo->twAggSup.timeWindowData, &winInfo, &pResult, i, winRows, rows, numOfOutput, + pOperator, winDelta); + if (code != TSDB_CODE_SUCCESS || pResult == NULL) { + T_LONG_JMP(pTaskInfo->env, TSDB_CODE_OUT_OF_MEMORY); + } + compactSessionWindow(pOperator, &winInfo, pStUpdated, pStDeleted, addGap); + saveSessionOutputBuf(pAggSup, &winInfo); + + if (pInfo->twAggSup.calTrigger == STREAM_TRIGGER_AT_ONCE && pStUpdated) { + code = saveResult(winInfo, pStUpdated); + if (code != TSDB_CODE_SUCCESS) { + T_LONG_JMP(pTaskInfo->env, TSDB_CODE_OUT_OF_MEMORY); + } + } + if (pInfo->twAggSup.calTrigger == STREAM_TRIGGER_WINDOW_CLOSE) { + SSessionKey key = {0}; + getSessionHashKey(&winInfo.sessionWin, &key); + tSimpleHashPut(pAggSup->pResultRows, &key, sizeof(SSessionKey), &winInfo, sizeof(SResultWindowInfo)); + } + + i += winRows; + } +} + +static void doDeleteTimeWindows(SStreamAggSupporter* pAggSup, SSDataBlock* pBlock, SArray* result) { + SColumnInfoData* pStartTsCol = taosArrayGet(pBlock->pDataBlock, START_TS_COLUMN_INDEX); + TSKEY* startDatas = (TSKEY*)pStartTsCol->pData; + SColumnInfoData* pEndTsCol = taosArrayGet(pBlock->pDataBlock, END_TS_COLUMN_INDEX); + TSKEY* endDatas = (TSKEY*)pEndTsCol->pData; + SColumnInfoData* pGroupCol = taosArrayGet(pBlock->pDataBlock, GROUPID_COLUMN_INDEX); + uint64_t* gpDatas = (uint64_t*)pGroupCol->pData; + for (int32_t i = 0; i < pBlock->info.rows; i++) { + while (1) { + SSessionKey curWin = {0}; + getCurSessionWindow(pAggSup, startDatas[i], endDatas[i], gpDatas[i], &curWin); + if (IS_INVALID_SESSION_WIN_KEY(curWin)) { + break; + } + doDeleteSessionWindow(pAggSup, &curWin); + if (result) { + saveDeleteInfo(result, curWin); + } + } + } +} + +static inline int32_t sessionKeyCompareAsc(const void* pKey1, const void* pKey2) { + SSessionKey* pWin1 = (SSessionKey*)pKey1; + SSessionKey* pWin2 = (SSessionKey*)pKey2; + + if (pWin1->groupId > pWin2->groupId) { + return 1; + } else if (pWin1->groupId < pWin2->groupId) { + return -1; + } + + if (pWin1->win.skey > pWin2->win.skey) { + return 1; + } else if (pWin1->win.skey < pWin2->win.skey) { + return -1; + } + + return 0; +} + +static int32_t copyUpdateResult(SSHashObj* pStUpdated, SArray* pUpdated) { + void* pIte = NULL; + int32_t iter = 0; + while ((pIte = tSimpleHashIterate(pStUpdated, pIte, &iter)) != NULL) { + void* key = tSimpleHashGetKey(pIte, NULL); + taosArrayPush(pUpdated, key); + } + taosArraySort(pUpdated, sessionKeyCompareAsc); + return TSDB_CODE_SUCCESS; +} + +void doBuildDeleteDataBlock(SOperatorInfo* pOp, SSHashObj* pStDeleted, SSDataBlock* pBlock, void** Ite) { + SStorageAPI* pAPI = &pOp->pTaskInfo->storageAPI; + + blockDataCleanup(pBlock); + int32_t size = tSimpleHashGetSize(pStDeleted); + if (size == 0) { + return; + } + blockDataEnsureCapacity(pBlock, size); + int32_t iter = 0; + while (((*Ite) = tSimpleHashIterate(pStDeleted, *Ite, &iter)) != NULL) { + if (pBlock->info.rows + 1 > pBlock->info.capacity) { + break; + } + SSessionKey* res = tSimpleHashGetKey(*Ite, NULL); + SColumnInfoData* pStartTsCol = taosArrayGet(pBlock->pDataBlock, START_TS_COLUMN_INDEX); + colDataSetVal(pStartTsCol, pBlock->info.rows, (const char*)&res->win.skey, false); + SColumnInfoData* pEndTsCol = taosArrayGet(pBlock->pDataBlock, END_TS_COLUMN_INDEX); + colDataSetVal(pEndTsCol, pBlock->info.rows, (const char*)&res->win.skey, false); + SColumnInfoData* pUidCol = taosArrayGet(pBlock->pDataBlock, UID_COLUMN_INDEX); + colDataSetNULL(pUidCol, pBlock->info.rows); + SColumnInfoData* pGpCol = taosArrayGet(pBlock->pDataBlock, GROUPID_COLUMN_INDEX); + colDataSetVal(pGpCol, pBlock->info.rows, (const char*)&res->groupId, false); + SColumnInfoData* pCalStCol = taosArrayGet(pBlock->pDataBlock, CALCULATE_START_TS_COLUMN_INDEX); + colDataSetNULL(pCalStCol, pBlock->info.rows); + SColumnInfoData* pCalEdCol = taosArrayGet(pBlock->pDataBlock, CALCULATE_END_TS_COLUMN_INDEX); + colDataSetNULL(pCalEdCol, pBlock->info.rows); + + SColumnInfoData* pTableCol = taosArrayGet(pBlock->pDataBlock, TABLE_NAME_COLUMN_INDEX); + + void* tbname = NULL; + pAPI->stateStore.streamStateGetParName(pOp->pTaskInfo->streamInfo.pState, res->groupId, &tbname); + if (tbname == NULL) { + colDataSetNULL(pTableCol, pBlock->info.rows); + } else { + char parTbName[VARSTR_HEADER_SIZE + TSDB_TABLE_NAME_LEN]; + STR_WITH_MAXSIZE_TO_VARSTR(parTbName, tbname, sizeof(parTbName)); + colDataSetVal(pTableCol, pBlock->info.rows, (const char*)parTbName, false); + pAPI->stateStore.streamStateFreeVal(tbname); + } + pBlock->info.rows += 1; + } + if ((*Ite) == NULL) { + tSimpleHashClear(pStDeleted); + } +} + +static void rebuildSessionWindow(SOperatorInfo* pOperator, SArray* pWinArray, SSHashObj* pStUpdated) { + SExprSupp* pSup = &pOperator->exprSupp; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + SStorageAPI* pAPI = &pOperator->pTaskInfo->storageAPI; + + int32_t size = taosArrayGetSize(pWinArray); + SStreamSessionAggOperatorInfo* pInfo = pOperator->info; + SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; + int32_t numOfOutput = pSup->numOfExprs; + int32_t numOfChild = taosArrayGetSize(pInfo->pChildren); + + for (int32_t i = 0; i < size; i++) { + SSessionKey* pWinKey = taosArrayGet(pWinArray, i); + int32_t num = 0; + SResultWindowInfo parentWin = {0}; + for (int32_t j = 0; j < numOfChild; j++) { + SOperatorInfo* pChild = taosArrayGetP(pInfo->pChildren, j); + SStreamSessionAggOperatorInfo* pChInfo = pChild->info; + SStreamAggSupporter* pChAggSup = &pChInfo->streamAggSup; + SSessionKey chWinKey = {0}; + getSessionHashKey(pWinKey, &chWinKey); + SStreamStateCur* pCur = pAggSup->stateStore.streamStateSessionSeekKeyCurrentNext(pChAggSup->pState, &chWinKey); + SResultRow* pResult = NULL; + SResultRow* pChResult = NULL; + while (1) { + SResultWindowInfo childWin = {0}; + childWin.sessionWin = *pWinKey; + int32_t code = getSessionWinBuf(pChAggSup, pCur, &childWin); + + if (code == TSDB_CODE_SUCCESS && !inWinRange(&pAggSup->winRange, &childWin.sessionWin.win)) { + continue; + } + + if (code == TSDB_CODE_SUCCESS && inWinRange(&pWinKey->win, &childWin.sessionWin.win)) { + if (num == 0) { + setSessionOutputBuf(pAggSup, pWinKey->win.skey, pWinKey->win.ekey, pWinKey->groupId, &parentWin); + code = initSessionOutputBuf(&parentWin, &pResult, pSup->pCtx, numOfOutput, pSup->rowEntryInfoOffset); + if (code != TSDB_CODE_SUCCESS || pResult == NULL) { + break; + } + } + num++; + updateTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &parentWin.sessionWin.win, pAggSup->gap); + initSessionOutputBuf(&childWin, &pChResult, pChild->exprSupp.pCtx, numOfOutput, + pChild->exprSupp.rowEntryInfoOffset); + compactFunctions(pSup->pCtx, pChild->exprSupp.pCtx, numOfOutput, pTaskInfo, &pInfo->twAggSup.timeWindowData); + compactSessionWindow(pOperator, &parentWin, pStUpdated, NULL, true); + saveResult(parentWin, pStUpdated); + } else { + break; + } + } + pAPI->stateStore.streamStateFreeCur(pCur); + } + if (num > 0) { + saveSessionOutputBuf(pAggSup, &parentWin); + } + } +} + +int32_t closeSessionWindow(SSHashObj* pHashMap, STimeWindowAggSupp* pTwSup, SSHashObj* pClosed) { + void* pIte = NULL; + int32_t iter = 0; + while ((pIte = tSimpleHashIterate(pHashMap, pIte, &iter)) != NULL) { + SResultWindowInfo* pWinInfo = pIte; + if (isCloseWindow(&pWinInfo->sessionWin.win, pTwSup)) { + if (pTwSup->calTrigger == STREAM_TRIGGER_WINDOW_CLOSE && pClosed) { + int32_t code = saveResult(*pWinInfo, pClosed); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + } + SSessionKey* pKey = tSimpleHashGetKey(pIte, NULL); + tSimpleHashIterateRemove(pHashMap, pKey, sizeof(SSessionKey), &pIte, &iter); + } + } + return TSDB_CODE_SUCCESS; +} + +static void closeChildSessionWindow(SArray* pChildren, TSKEY maxTs) { + int32_t size = taosArrayGetSize(pChildren); + for (int32_t i = 0; i < size; i++) { + SOperatorInfo* pChildOp = taosArrayGetP(pChildren, i); + SStreamSessionAggOperatorInfo* pChInfo = pChildOp->info; + pChInfo->twAggSup.maxTs = TMAX(pChInfo->twAggSup.maxTs, maxTs); + closeSessionWindow(pChInfo->streamAggSup.pResultRows, &pChInfo->twAggSup, NULL); + } +} + +int32_t getAllSessionWindow(SSHashObj* pHashMap, SSHashObj* pStUpdated) { + void* pIte = NULL; + int32_t iter = 0; + while ((pIte = tSimpleHashIterate(pHashMap, pIte, &iter)) != NULL) { + SResultWindowInfo* pWinInfo = pIte; + saveResult(*pWinInfo, pStUpdated); + } + return TSDB_CODE_SUCCESS; +} + +static void copyDeleteWindowInfo(SArray* pResWins, SSHashObj* pStDeleted) { + int32_t size = taosArrayGetSize(pResWins); + for (int32_t i = 0; i < size; i++) { + SSessionKey* pWinKey = taosArrayGet(pResWins, i); + if (!pWinKey) continue; + SSessionKey winInfo = {0}; + getSessionHashKey(pWinKey, &winInfo); + tSimpleHashPut(pStDeleted, &winInfo, sizeof(SSessionKey), NULL, 0); + } +} + +// the allocated memory comes from outer function. +void initGroupResInfoFromArrayList(SGroupResInfo* pGroupResInfo, SArray* pArrayList) { + pGroupResInfo->pRows = pArrayList; + pGroupResInfo->index = 0; + pGroupResInfo->pBuf = NULL; +} + +void doBuildSessionResult(SOperatorInfo* pOperator, void* pState, SGroupResInfo* pGroupResInfo, SSDataBlock* pBlock) { + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + // set output datablock version + pBlock->info.version = pTaskInfo->version; + + blockDataCleanup(pBlock); + if (!hasRemainResults(pGroupResInfo)) { + cleanupGroupResInfo(pGroupResInfo); + return; + } + + // clear the existed group id + pBlock->info.id.groupId = 0; + buildSessionResultDataBlock(pOperator, pState, pBlock, &pOperator->exprSupp, pGroupResInfo); + if (pBlock->info.rows == 0) { + cleanupGroupResInfo(pGroupResInfo); + } +} + +static SSDataBlock* buildSessionResult(SOperatorInfo* pOperator) { + SStreamSessionAggOperatorInfo* pInfo = pOperator->info; + SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; + SOptrBasicInfo* pBInfo = &pInfo->binfo; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + doBuildDeleteDataBlock(pOperator, pInfo->pStDeleted, pInfo->pDelRes, &pInfo->pDelIterator); + if (pInfo->pDelRes->info.rows > 0) { + printDataBlock(pInfo->pDelRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); + return pInfo->pDelRes; + } + doBuildSessionResult(pOperator, pAggSup->pState, &pInfo->groupResInfo, pBInfo->pRes); + if (pBInfo->pRes->info.rows > 0) { + printDataBlock(pBInfo->pRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); + return pBInfo->pRes; + } + return NULL; +} + +void getMaxTsWins(const SArray* pAllWins, SArray* pMaxWins) { + int32_t size = taosArrayGetSize(pAllWins); + if (size == 0) { + return; + } + + SSessionKey* pSeKey = taosArrayGet(pAllWins, size - 1); + taosArrayPush(pMaxWins, pSeKey); + if (pSeKey->groupId == 0) { + return; + } + uint64_t preGpId = pSeKey->groupId; + for (int32_t i = size - 2; i >= 0; i--) { + pSeKey = taosArrayGet(pAllWins, i); + if (preGpId != pSeKey->groupId) { + taosArrayPush(pMaxWins, pSeKey); + preGpId = pSeKey->groupId; + } + } +} + +static SSDataBlock* doStreamSessionAgg(SOperatorInfo* pOperator) { + SExprSupp* pSup = &pOperator->exprSupp; + SStreamSessionAggOperatorInfo* pInfo = pOperator->info; + SOptrBasicInfo* pBInfo = &pInfo->binfo; + SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + qDebug("stask:%s %s status: %d", GET_TASKID(pTaskInfo), getStreamOpName(pOperator->operatorType), pOperator->status); + if (pOperator->status == OP_EXEC_DONE) { + return NULL; + } else if (pOperator->status == OP_RES_TO_RETURN) { + SSDataBlock* opRes = buildSessionResult(pOperator); + if (opRes) { + return opRes; + } + setOperatorCompleted(pOperator); + return NULL; + } + + SOperatorInfo* downstream = pOperator->pDownstream[0]; + if (!pInfo->pUpdated) { + pInfo->pUpdated = taosArrayInit(16, sizeof(SSessionKey)); + } + if (!pInfo->pStUpdated) { + _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); + pInfo->pStUpdated = tSimpleHashInit(64, hashFn); + } + while (1) { + SSDataBlock* pBlock = downstream->fpSet.getNextFn(downstream); + if (pBlock == NULL) { + break; + } + printSpecDataBlock(pBlock, getStreamOpName(pOperator->operatorType), "recv", GET_TASKID(pTaskInfo)); + + if (pBlock->info.type == STREAM_DELETE_DATA || pBlock->info.type == STREAM_DELETE_RESULT || + pBlock->info.type == STREAM_CLEAR) { + SArray* pWins = taosArrayInit(16, sizeof(SSessionKey)); + // gap must be 0 + doDeleteTimeWindows(pAggSup, pBlock, pWins); + removeSessionResults(pInfo->pStUpdated, pWins); + if (IS_FINAL_SESSION_OP(pOperator)) { + int32_t childIndex = getChildIndex(pBlock); + SOperatorInfo* pChildOp = taosArrayGetP(pInfo->pChildren, childIndex); + SStreamSessionAggOperatorInfo* pChildInfo = pChildOp->info; + // gap must be 0 + doDeleteTimeWindows(&pChildInfo->streamAggSup, pBlock, NULL); + rebuildSessionWindow(pOperator, pWins, pInfo->pStUpdated); + } + copyDeleteWindowInfo(pWins, pInfo->pStDeleted); + taosArrayDestroy(pWins); + continue; + } else if (pBlock->info.type == STREAM_GET_ALL) { + getAllSessionWindow(pAggSup->pResultRows, pInfo->pStUpdated); + continue; + } else if (pBlock->info.type == STREAM_CREATE_CHILD_TABLE) { + return pBlock; + } else { + ASSERTS(pBlock->info.type == STREAM_NORMAL || pBlock->info.type == STREAM_INVALID, "invalid SSDataBlock type"); + } + + if (pInfo->scalarSupp.pExprInfo != NULL) { + SExprSupp* pExprSup = &pInfo->scalarSupp; + projectApplyFunctions(pExprSup->pExprInfo, pBlock, pBlock, pExprSup->pCtx, pExprSup->numOfExprs, NULL); + } + // the pDataBlock are always the same one, no need to call this again + setInputDataBlock(pSup, pBlock, TSDB_ORDER_ASC, MAIN_SCAN, true); + doStreamSessionAggImpl(pOperator, pBlock, pInfo->pStUpdated, pInfo->pStDeleted, IS_FINAL_SESSION_OP(pOperator), + true); + if (IS_FINAL_SESSION_OP(pOperator)) { + int32_t chIndex = getChildIndex(pBlock); + int32_t size = taosArrayGetSize(pInfo->pChildren); + // if chIndex + 1 - size > 0, add new child + for (int32_t i = 0; i < chIndex + 1 - size; i++) { + SOperatorInfo* pChildOp = + createStreamFinalSessionAggOperatorInfo(NULL, pInfo->pPhyNode, pOperator->pTaskInfo, 0, NULL); + if (!pChildOp) { + T_LONG_JMP(pOperator->pTaskInfo->env, TSDB_CODE_OUT_OF_MEMORY); + } + taosArrayPush(pInfo->pChildren, &pChildOp); + } + SOperatorInfo* pChildOp = taosArrayGetP(pInfo->pChildren, chIndex); + setInputDataBlock(&pChildOp->exprSupp, pBlock, TSDB_ORDER_ASC, MAIN_SCAN, true); + doStreamSessionAggImpl(pChildOp, pBlock, NULL, NULL, true, false); + } + pInfo->twAggSup.maxTs = TMAX(pInfo->twAggSup.maxTs, pBlock->info.window.ekey); + pInfo->twAggSup.maxTs = TMAX(pInfo->twAggSup.maxTs, pBlock->info.watermark); + } + // restore the value + pOperator->status = OP_RES_TO_RETURN; + + closeSessionWindow(pAggSup->pResultRows, &pInfo->twAggSup, pInfo->pStUpdated); + closeChildSessionWindow(pInfo->pChildren, pInfo->twAggSup.maxTs); + copyUpdateResult(pInfo->pStUpdated, pInfo->pUpdated); + removeSessionResults(pInfo->pStDeleted, pInfo->pUpdated); + tSimpleHashCleanup(pInfo->pStUpdated); + pInfo->pStUpdated = NULL; + if (pInfo->isHistoryOp) { + getMaxTsWins(pInfo->pUpdated, pInfo->historyWins); + } + initGroupResInfoFromArrayList(&pInfo->groupResInfo, pInfo->pUpdated); + pInfo->pUpdated = NULL; + blockDataEnsureCapacity(pInfo->binfo.pRes, pOperator->resultInfo.capacity); + + SSDataBlock* opRes = buildSessionResult(pOperator); + if (opRes) { + return opRes; + } + + setOperatorCompleted(pOperator); + return NULL; +} + +void streamSessionReleaseState(SOperatorInfo* pOperator) { + if (pOperator->operatorType != QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_SESSION) { + SStreamSessionAggOperatorInfo* pInfo = pOperator->info; + int32_t resSize = taosArrayGetSize(pInfo->historyWins) * sizeof(SSessionKey); + pInfo->streamAggSup.stateStore.streamStateSaveInfo(pInfo->streamAggSup.pState, STREAM_SESSION_OP_STATE_NAME, + strlen(STREAM_SESSION_OP_STATE_NAME), pInfo->historyWins->pData, + resSize); + } + SOperatorInfo* downstream = pOperator->pDownstream[0]; + if (downstream->fpSet.releaseStreamStateFn) { + downstream->fpSet.releaseStreamStateFn(downstream); + } +} + +void resetWinRange(STimeWindow* winRange) { + winRange->skey = INT64_MIN; + winRange->ekey = INT64_MAX; +} + +void streamSessionReloadState(SOperatorInfo* pOperator) { + SStreamSessionAggOperatorInfo* pInfo = pOperator->info; + SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; + resetWinRange(&pAggSup->winRange); + + SResultWindowInfo winInfo = {0}; + int32_t size = 0; + void* pBuf = NULL; + int32_t code = pAggSup->stateStore.streamStateGetInfo(pAggSup->pState, STREAM_SESSION_OP_STATE_NAME, + strlen(STREAM_SESSION_OP_STATE_NAME), &pBuf, &size); + int32_t num = size / sizeof(SSessionKey); + SSessionKey* pSeKeyBuf = (SSessionKey*)pBuf; + ASSERT(size == num * sizeof(SSessionKey)); + if (!pInfo->pStUpdated && num > 0) { + _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); + pInfo->pStUpdated = tSimpleHashInit(64, hashFn); + } + for (int32_t i = 0; i < num; i++) { + SResultWindowInfo winInfo = {0}; + setSessionOutputBuf(pAggSup, pSeKeyBuf[i].win.skey, pSeKeyBuf[i].win.ekey, pSeKeyBuf[i].groupId, &winInfo); + int32_t winNum = compactSessionWindow(pOperator, &winInfo, pInfo->pStUpdated, pInfo->pStDeleted, true); + if (winNum > 0) { + qDebug("===stream=== reload state. save result %" PRId64 ", %" PRIu64, winInfo.sessionWin.win.skey, + winInfo.sessionWin.groupId); + if (pInfo->twAggSup.calTrigger == STREAM_TRIGGER_AT_ONCE) { + saveResult(winInfo, pInfo->pStUpdated); + } else if (pInfo->twAggSup.calTrigger == STREAM_TRIGGER_WINDOW_CLOSE) { + if (!isCloseWindow(&winInfo.sessionWin.win, &pInfo->twAggSup)) { + saveDeleteRes(pInfo->pStDeleted, winInfo.sessionWin); + } + SSessionKey key = {0}; + getSessionHashKey(&winInfo.sessionWin, &key); + tSimpleHashPut(pAggSup->pResultRows, &key, sizeof(SSessionKey), &winInfo, sizeof(SResultWindowInfo)); + } + } + saveSessionOutputBuf(pAggSup, &winInfo); + } + taosMemoryFree(pBuf); + + SOperatorInfo* downstream = pOperator->pDownstream[0]; + if (downstream->fpSet.reloadStreamStateFn) { + downstream->fpSet.reloadStreamStateFn(downstream); + } +} + +SOperatorInfo* createStreamSessionAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, + SExecTaskInfo* pTaskInfo, SReadHandle* pHandle) { + SSessionWinodwPhysiNode* pSessionNode = (SSessionWinodwPhysiNode*)pPhyNode; + int32_t numOfCols = 0; + int32_t code = TSDB_CODE_OUT_OF_MEMORY; + SStreamSessionAggOperatorInfo* pInfo = taosMemoryCalloc(1, sizeof(SStreamSessionAggOperatorInfo)); + SOperatorInfo* pOperator = taosMemoryCalloc(1, sizeof(SOperatorInfo)); + if (pInfo == NULL || pOperator == NULL) { + goto _error; + } + + pOperator->pTaskInfo = pTaskInfo; + + initResultSizeInfo(&pOperator->resultInfo, 4096); + if (pSessionNode->window.pExprs != NULL) { + int32_t numOfScalar = 0; + SExprInfo* pScalarExprInfo = createExprInfo(pSessionNode->window.pExprs, NULL, &numOfScalar); + code = initExprSupp(&pInfo->scalarSupp, pScalarExprInfo, numOfScalar, &pTaskInfo->storageAPI.functionStore); + if (code != TSDB_CODE_SUCCESS) { + goto _error; + } + } + SExprSupp* pSup = &pOperator->exprSupp; + + SExprInfo* pExprInfo = createExprInfo(pSessionNode->window.pFuncs, NULL, &numOfCols); + SSDataBlock* pResBlock = createDataBlockFromDescNode(pPhyNode->pOutputDataBlockDesc); + code = initBasicInfoEx(&pInfo->binfo, pSup, pExprInfo, numOfCols, pResBlock, &pTaskInfo->storageAPI.functionStore); + if (code != TSDB_CODE_SUCCESS) { + goto _error; + } + + code = initStreamAggSupporter(&pInfo->streamAggSup, pSup->pCtx, numOfCols, pSessionNode->gap, + pTaskInfo->streamInfo.pState, 0, 0, &pTaskInfo->storageAPI.stateStore, pHandle, + &pTaskInfo->storageAPI); + if (code != TSDB_CODE_SUCCESS) { + goto _error; + } + + pInfo->twAggSup = (STimeWindowAggSupp){ + .waterMark = pSessionNode->window.watermark, + .calTrigger = pSessionNode->window.triggerType, + .maxTs = INT64_MIN, + .minTs = INT64_MAX, + }; + + initExecTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &pTaskInfo->window); + + pInfo->primaryTsIndex = ((SColumnNode*)pSessionNode->window.pTspk)->slotId; + 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); + pInfo->pDelIterator = NULL; + pInfo->pDelRes = createSpecialDataBlock(STREAM_DELETE_RESULT); + pInfo->pChildren = NULL; + pInfo->pPhyNode = pPhyNode; + pInfo->ignoreExpiredData = pSessionNode->window.igExpired; + pInfo->ignoreExpiredDataSaved = false; + pInfo->pUpdated = NULL; + pInfo->pStUpdated = NULL; + pInfo->dataVersion = 0; + pInfo->historyWins = taosArrayInit(4, sizeof(SSessionKey)); + if (!pInfo->historyWins) { + goto _error; + } + if (pHandle) { + pInfo->isHistoryOp = pHandle->fillHistory; + } + pOperator->operatorType = QUERY_NODE_PHYSICAL_PLAN_STREAM_SESSION; + setOperatorInfo(pOperator, getStreamOpName(pOperator->operatorType), QUERY_NODE_PHYSICAL_PLAN_STREAM_SESSION, true, + OP_NOT_OPENED, pInfo, pTaskInfo); + pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, doStreamSessionAgg, NULL, destroyStreamSessionAggOperatorInfo, + optrDefaultBufFn, NULL); + setOperatorStreamStateFn(pOperator, streamSessionReleaseState, streamSessionReloadState); + + if (downstream) { + initDownStream(downstream, &pInfo->streamAggSup, pOperator->operatorType, pInfo->primaryTsIndex, &pInfo->twAggSup); + code = appendDownstream(pOperator, &downstream, 1); + } + return pOperator; + +_error: + if (pInfo != NULL) { + destroyStreamSessionAggOperatorInfo(pInfo); + } + + taosMemoryFreeClear(pOperator); + pTaskInfo->code = code; + return NULL; +} + +static void clearStreamSessionOperator(SStreamSessionAggOperatorInfo* pInfo) { + tSimpleHashClear(pInfo->streamAggSup.pResultRows); + pInfo->streamAggSup.stateStore.streamStateSessionClear(pInfo->streamAggSup.pState); +} + +static SSDataBlock* doStreamSessionSemiAgg(SOperatorInfo* pOperator) { + SStreamSessionAggOperatorInfo* pInfo = pOperator->info; + SOptrBasicInfo* pBInfo = &pInfo->binfo; + TSKEY maxTs = INT64_MIN; + SExprSupp* pSup = &pOperator->exprSupp; + SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + + qDebug("stask:%s %s status: %d", GET_TASKID(pTaskInfo), getStreamOpName(pOperator->operatorType), pOperator->status); + if (pOperator->status == OP_EXEC_DONE) { + return NULL; + } + + { + SSDataBlock* opRes = buildSessionResult(pOperator); + if (opRes) { + return opRes; + } + + if (pOperator->status == OP_RES_TO_RETURN) { + clearFunctionContext(&pOperator->exprSupp); + // semi interval operator clear disk buffer + clearStreamSessionOperator(pInfo); + setOperatorCompleted(pOperator); + return NULL; + } + } + + SOperatorInfo* downstream = pOperator->pDownstream[0]; + if (!pInfo->pUpdated) { + pInfo->pUpdated = taosArrayInit(16, sizeof(SSessionKey)); + } + if (!pInfo->pStUpdated) { + _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); + pInfo->pStUpdated = tSimpleHashInit(64, hashFn); + } + while (1) { + SSDataBlock* pBlock = downstream->fpSet.getNextFn(downstream); + if (pBlock == NULL) { + clearSpecialDataBlock(pInfo->pUpdateRes); + pOperator->status = OP_RES_TO_RETURN; + break; + } + printSpecDataBlock(pBlock, getStreamOpName(pOperator->operatorType), "recv", GET_TASKID(pTaskInfo)); + + if (pBlock->info.type == STREAM_DELETE_DATA || pBlock->info.type == STREAM_DELETE_RESULT || + pBlock->info.type == STREAM_CLEAR) { + // gap must be 0 + SArray* pWins = taosArrayInit(16, sizeof(SSessionKey)); + doDeleteTimeWindows(&pInfo->streamAggSup, pBlock, pWins); + removeSessionResults(pInfo->pStUpdated, pWins); + copyDeleteWindowInfo(pWins, pInfo->pStDeleted); + taosArrayDestroy(pWins); + break; + } else if (pBlock->info.type == STREAM_GET_ALL) { + getAllSessionWindow(pInfo->streamAggSup.pResultRows, pInfo->pStUpdated); + continue; + } else if (pBlock->info.type == STREAM_CREATE_CHILD_TABLE) { + return pBlock; + } else { + ASSERTS(pBlock->info.type == STREAM_NORMAL || pBlock->info.type == STREAM_INVALID, "invalid SSDataBlock type"); + } + + if (pInfo->scalarSupp.pExprInfo != NULL) { + SExprSupp* pExprSup = &pInfo->scalarSupp; + projectApplyFunctions(pExprSup->pExprInfo, pBlock, pBlock, pExprSup->pCtx, pExprSup->numOfExprs, NULL); + } + // the pDataBlock are always the same one, no need to call this again + setInputDataBlock(pSup, pBlock, TSDB_ORDER_ASC, MAIN_SCAN, true); + doStreamSessionAggImpl(pOperator, pBlock, pInfo->pStUpdated, NULL, false, false); + maxTs = TMAX(pInfo->twAggSup.maxTs, pBlock->info.window.ekey); + } + + pInfo->twAggSup.maxTs = TMAX(pInfo->twAggSup.maxTs, maxTs); + pBInfo->pRes->info.watermark = pInfo->twAggSup.maxTs; + + copyUpdateResult(pInfo->pStUpdated, pInfo->pUpdated); + removeSessionResults(pInfo->pStDeleted, pInfo->pUpdated); + tSimpleHashCleanup(pInfo->pStUpdated); + pInfo->pStUpdated = NULL; + initGroupResInfoFromArrayList(&pInfo->groupResInfo, pInfo->pUpdated); + pInfo->pUpdated = NULL; + blockDataEnsureCapacity(pBInfo->pRes, pOperator->resultInfo.capacity); + + SSDataBlock* opRes = buildSessionResult(pOperator); + if (opRes) { + return opRes; + } + + clearFunctionContext(&pOperator->exprSupp); + // semi interval operator clear disk buffer + clearStreamSessionOperator(pInfo); + setOperatorCompleted(pOperator); + return NULL; +} + +SOperatorInfo* createStreamFinalSessionAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, + SExecTaskInfo* pTaskInfo, int32_t numOfChild, + SReadHandle* pHandle) { + int32_t code = TSDB_CODE_OUT_OF_MEMORY; + SOperatorInfo* pOperator = createStreamSessionAggOperatorInfo(downstream, pPhyNode, pTaskInfo, pHandle); + if (pOperator == NULL) { + goto _error; + } + + SStorageAPI* pAPI = &pTaskInfo->storageAPI; + SStreamSessionAggOperatorInfo* pInfo = pOperator->info; + pOperator->operatorType = pPhyNode->type; + + if (pPhyNode->type != QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_SESSION) { + pInfo->pUpdateRes = createSpecialDataBlock(STREAM_CLEAR); + blockDataEnsureCapacity(pInfo->pUpdateRes, 128); + pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, doStreamSessionSemiAgg, NULL, + destroyStreamSessionAggOperatorInfo, optrDefaultBufFn, NULL); + } + setOperatorInfo(pOperator, getStreamOpName(pOperator->operatorType), pPhyNode->type, false, OP_NOT_OPENED, pInfo, pTaskInfo); + + if (numOfChild > 0) { + pInfo->pChildren = taosArrayInit(numOfChild, sizeof(void*)); + for (int32_t i = 0; i < numOfChild; i++) { + SOperatorInfo* pChildOp = createStreamFinalSessionAggOperatorInfo(NULL, pPhyNode, pTaskInfo, 0, NULL); + if (pChildOp == NULL) { + goto _error; + } + SStreamSessionAggOperatorInfo* pChInfo = pChildOp->info; + pChInfo->twAggSup.calTrigger = STREAM_TRIGGER_AT_ONCE; + pAPI->stateStore.streamStateSetNumber(pChInfo->streamAggSup.pState, i); + taosArrayPush(pInfo->pChildren, &pChildOp); + } + } + + if (!IS_FINAL_SESSION_OP(pOperator) || numOfChild == 0) { + pInfo->twAggSup.calTrigger = STREAM_TRIGGER_AT_ONCE; + } + + return pOperator; + +_error: + if (pInfo != NULL) { + destroyStreamSessionAggOperatorInfo(pInfo); + } + taosMemoryFreeClear(pOperator); + pTaskInfo->code = code; + return NULL; +} + +void destroyStreamStateOperatorInfo(void* param) { + SStreamStateAggOperatorInfo* pInfo = (SStreamStateAggOperatorInfo*)param; + cleanupBasicInfo(&pInfo->binfo); + destroyStreamAggSupporter(&pInfo->streamAggSup); + cleanupGroupResInfo(&pInfo->groupResInfo); + if (pInfo->pChildren != NULL) { + int32_t size = taosArrayGetSize(pInfo->pChildren); + for (int32_t i = 0; i < size; i++) { + SOperatorInfo* pChild = taosArrayGetP(pInfo->pChildren, i); + destroyOperator(pChild); + } + taosArrayDestroy(pInfo->pChildren); + } + colDataDestroy(&pInfo->twAggSup.timeWindowData); + blockDataDestroy(pInfo->pDelRes); + taosArrayDestroy(pInfo->historyWins); + tSimpleHashCleanup(pInfo->pSeUpdated); + tSimpleHashCleanup(pInfo->pSeDeleted); + taosMemoryFreeClear(param); +} + +bool isTsInWindow(SStateWindowInfo* pWin, TSKEY ts) { + if (pWin->winInfo.sessionWin.win.skey <= ts && ts <= pWin->winInfo.sessionWin.win.ekey) { + return true; + } + return false; +} + +bool isEqualStateKey(SStateWindowInfo* pWin, char* pKeyData) { + return pKeyData && compareVal(pKeyData, pWin->pStateKey); +} + +bool compareStateKey(void* data, void* key) { + if (!data || !key) { + return true; + } + SStateKeys* stateKey = (SStateKeys*)key; + stateKey->pData = (char*)key + sizeof(SStateKeys); + return compareVal(data, stateKey); +} + +bool compareWinStateKey(SStateKeys* left, SStateKeys* right) { + if (!left || !right) { + return false; + } + return compareVal(left->pData, right); +} + +void setStateOutputBuf(SStreamAggSupporter* pAggSup, TSKEY ts, uint64_t groupId, char* pKeyData, + SStateWindowInfo* pCurWin, SStateWindowInfo* pNextWin) { + int32_t size = pAggSup->resultRowSize; + pCurWin->winInfo.sessionWin.groupId = groupId; + pCurWin->winInfo.sessionWin.win.skey = ts; + pCurWin->winInfo.sessionWin.win.ekey = ts; + int32_t code = pAggSup->stateStore.streamStateStateAddIfNotExist(pAggSup->pState, &pCurWin->winInfo.sessionWin, + pKeyData, pAggSup->stateKeySize, compareStateKey, + &pCurWin->winInfo.pOutputBuf, &size); + pCurWin->pStateKey = + (SStateKeys*)((char*)pCurWin->winInfo.pOutputBuf + (pAggSup->resultRowSize - pAggSup->stateKeySize)); + pCurWin->pStateKey->bytes = pAggSup->stateKeySize - sizeof(SStateKeys); + pCurWin->pStateKey->type = pAggSup->stateKeyType; + pCurWin->pStateKey->pData = (char*)pCurWin->pStateKey + sizeof(SStateKeys); + pCurWin->pStateKey->isNull = false; + + if (code == TSDB_CODE_SUCCESS && !inWinRange(&pAggSup->winRange, &pCurWin->winInfo.sessionWin.win)) { + code = TSDB_CODE_FAILED; + releaseOutputBuf(pAggSup->pState, NULL, (SResultRow*)pCurWin->winInfo.pOutputBuf, + &pAggSup->pSessionAPI->stateStore); + pCurWin->winInfo.pOutputBuf = taosMemoryCalloc(1, size); + pCurWin->pStateKey = + (SStateKeys*)((char*)pCurWin->winInfo.pOutputBuf + (pAggSup->resultRowSize - pAggSup->stateKeySize)); + pCurWin->pStateKey->bytes = pAggSup->stateKeySize - sizeof(SStateKeys); + pCurWin->pStateKey->type = pAggSup->stateKeyType; + pCurWin->pStateKey->pData = (char*)pCurWin->pStateKey + sizeof(SStateKeys); + pCurWin->pStateKey->isNull = false; + pCurWin->winInfo.sessionWin.groupId = groupId; + pCurWin->winInfo.sessionWin.win.skey = ts; + pCurWin->winInfo.sessionWin.win.ekey = ts; + qDebug("===stream===reset state win key. skey:%" PRId64 ", endkey:%" PRId64, pCurWin->winInfo.sessionWin.win.skey, + pCurWin->winInfo.sessionWin.win.ekey); + } + + if (code == TSDB_CODE_SUCCESS) { + pCurWin->winInfo.isOutput = true; + pAggSup->stateStore.streamStateSessionDel(pAggSup->pState, &pCurWin->winInfo.sessionWin); + } else if (pKeyData) { + if (IS_VAR_DATA_TYPE(pAggSup->stateKeyType)) { + varDataCopy(pCurWin->pStateKey->pData, pKeyData); + } else { + memcpy(pCurWin->pStateKey->pData, pKeyData, pCurWin->pStateKey->bytes); + } + } + + pNextWin->winInfo.sessionWin = pCurWin->winInfo.sessionWin; + SStreamStateCur* pCur = + pAggSup->stateStore.streamStateSessionSeekKeyNext(pAggSup->pState, &pNextWin->winInfo.sessionWin); + int32_t nextSize = pAggSup->resultRowSize; + code = pAggSup->stateStore.streamStateSessionGetKVByCur(pCur, &pNextWin->winInfo.sessionWin, + &pNextWin->winInfo.pOutputBuf, &nextSize); + if (code != TSDB_CODE_SUCCESS) { + SET_SESSION_WIN_INVALID(pNextWin->winInfo); + } else { + pNextWin->pStateKey = + (SStateKeys*)((char*)pNextWin->winInfo.pOutputBuf + (pAggSup->resultRowSize - pAggSup->stateKeySize)); + pNextWin->pStateKey->bytes = pAggSup->stateKeySize - sizeof(SStateKeys); + pNextWin->pStateKey->type = pAggSup->stateKeyType; + pNextWin->pStateKey->pData = (char*)pNextWin->pStateKey + sizeof(SStateKeys); + pNextWin->pStateKey->isNull = false; + pNextWin->winInfo.isOutput = true; + } + pAggSup->stateStore.streamStateFreeCur(pCur); +} + +int32_t updateStateWindowInfo(SStateWindowInfo* pWinInfo, SStateWindowInfo* pNextWin, TSKEY* pTs, uint64_t groupId, + SColumnInfoData* pKeyCol, int32_t rows, int32_t start, bool* allEqual, + SSHashObj* pResultRows, SSHashObj* pSeUpdated, SSHashObj* pSeDeleted) { + *allEqual = true; + for (int32_t i = start; i < rows; ++i) { + char* pKeyData = colDataGetData(pKeyCol, i); + if (!isTsInWindow(pWinInfo, pTs[i])) { + if (isEqualStateKey(pWinInfo, pKeyData)) { + if (IS_VALID_SESSION_WIN(pNextWin->winInfo)) { + // ts belongs to the next window + if (pTs[i] >= pNextWin->winInfo.sessionWin.win.skey) { + return i - start; + } + } + } else { + return i - start; + } + } + + if (pWinInfo->winInfo.sessionWin.win.skey > pTs[i]) { + if (pSeDeleted && pWinInfo->winInfo.isOutput) { + saveDeleteRes(pSeDeleted, pWinInfo->winInfo.sessionWin); + } + removeSessionResult(pSeUpdated, pResultRows, pWinInfo->winInfo.sessionWin); + pWinInfo->winInfo.sessionWin.win.skey = pTs[i]; + } + pWinInfo->winInfo.sessionWin.win.ekey = TMAX(pWinInfo->winInfo.sessionWin.win.ekey, pTs[i]); + if (!isEqualStateKey(pWinInfo, pKeyData)) { + *allEqual = false; + } + } + return rows - start; +} + +static void doStreamStateAggImpl(SOperatorInfo* pOperator, SSDataBlock* pSDataBlock, SSHashObj* pSeUpdated, + SSHashObj* pStDeleted) { + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + SStorageAPI* pAPI = &pOperator->pTaskInfo->storageAPI; + + SStreamStateAggOperatorInfo* pInfo = pOperator->info; + int32_t numOfOutput = pOperator->exprSupp.numOfExprs; + uint64_t groupId = pSDataBlock->info.id.groupId; + int64_t code = TSDB_CODE_SUCCESS; + TSKEY* tsCols = NULL; + SResultRow* pResult = NULL; + int32_t winRows = 0; + SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; + + pInfo->dataVersion = TMAX(pInfo->dataVersion, pSDataBlock->info.version); + pAggSup->winRange = pTaskInfo->streamInfo.fillHistoryWindow; + if (pAggSup->winRange.ekey <= 0) { + pAggSup->winRange.ekey = INT64_MAX; + } + + if (pSDataBlock->pDataBlock != NULL) { + SColumnInfoData* pColDataInfo = taosArrayGet(pSDataBlock->pDataBlock, pInfo->primaryTsIndex); + tsCols = (int64_t*)pColDataInfo->pData; + } else { + return; + } + + int32_t rows = pSDataBlock->info.rows; + blockDataEnsureCapacity(pAggSup->pScanBlock, rows); + SColumnInfoData* pKeyColInfo = taosArrayGet(pSDataBlock->pDataBlock, pInfo->stateCol.slotId); + for (int32_t i = 0; i < rows; i += winRows) { + if (pInfo->ignoreExpiredData && isOverdue(tsCols[i], &pInfo->twAggSup) || colDataIsNull_s(pKeyColInfo, i)) { + i++; + continue; + } + char* pKeyData = colDataGetData(pKeyColInfo, i); + int32_t winIndex = 0; + bool allEqual = true; + SStateWindowInfo curWin = {0}; + SStateWindowInfo nextWin = {0}; + setStateOutputBuf(pAggSup, tsCols[i], groupId, pKeyData, &curWin, &nextWin); + if (IS_VALID_SESSION_WIN(nextWin.winInfo)) { + releaseOutputBuf(pAggSup->pState, NULL, (SResultRow*)nextWin.winInfo.pOutputBuf, &pAPI->stateStore); + } + setSessionWinOutputInfo(pSeUpdated, &curWin.winInfo); + winRows = updateStateWindowInfo(&curWin, &nextWin, tsCols, groupId, pKeyColInfo, rows, i, &allEqual, + pAggSup->pResultRows, pSeUpdated, pStDeleted); + if (!allEqual) { + uint64_t uid = 0; + appendOneRowToStreamSpecialBlock(pAggSup->pScanBlock, &curWin.winInfo.sessionWin.win.skey, + &curWin.winInfo.sessionWin.win.ekey, &uid, &groupId, NULL); + tSimpleHashRemove(pSeUpdated, &curWin.winInfo.sessionWin, sizeof(SSessionKey)); + doDeleteSessionWindow(pAggSup, &curWin.winInfo.sessionWin); + releaseOutputBuf(pAggSup->pState, NULL, (SResultRow*)curWin.winInfo.pOutputBuf, &pAPI->stateStore); + continue; + } + code = doOneWindowAggImpl(&pInfo->twAggSup.timeWindowData, &curWin.winInfo, &pResult, i, winRows, rows, numOfOutput, + pOperator, 0); + if (code != TSDB_CODE_SUCCESS || pResult == NULL) { + T_LONG_JMP(pTaskInfo->env, TSDB_CODE_OUT_OF_MEMORY); + } + saveSessionOutputBuf(pAggSup, &curWin.winInfo); + + if (pInfo->twAggSup.calTrigger == STREAM_TRIGGER_AT_ONCE) { + code = saveResult(curWin.winInfo, pSeUpdated); + if (code != TSDB_CODE_SUCCESS) { + T_LONG_JMP(pTaskInfo->env, TSDB_CODE_OUT_OF_MEMORY); + } + } + + if (pInfo->twAggSup.calTrigger == STREAM_TRIGGER_WINDOW_CLOSE) { + SSessionKey key = {0}; + getSessionHashKey(&curWin.winInfo.sessionWin, &key); + tSimpleHashPut(pAggSup->pResultRows, &key, sizeof(SSessionKey), &curWin.winInfo, sizeof(SResultWindowInfo)); + } + } +} + +static SSDataBlock* buildStateResult(SOperatorInfo* pOperator) { + SStreamStateAggOperatorInfo* pInfo = pOperator->info; + SOptrBasicInfo* pBInfo = &pInfo->binfo; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + + doBuildDeleteDataBlock(pOperator, pInfo->pSeDeleted, pInfo->pDelRes, &pInfo->pDelIterator); + if (pInfo->pDelRes->info.rows > 0) { + printDataBlock(pInfo->pDelRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); + return pInfo->pDelRes; + } + + doBuildSessionResult(pOperator, pInfo->streamAggSup.pState, &pInfo->groupResInfo, pBInfo->pRes); + if (pBInfo->pRes->info.rows > 0) { + printDataBlock(pBInfo->pRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); + return pBInfo->pRes; + } + return NULL; +} + +static SSDataBlock* doStreamStateAgg(SOperatorInfo* pOperator) { + if (pOperator->status == OP_EXEC_DONE) { + return NULL; + } + + SExprSupp* pSup = &pOperator->exprSupp; + SStreamStateAggOperatorInfo* pInfo = pOperator->info; + SOptrBasicInfo* pBInfo = &pInfo->binfo; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + qDebug("===stream=== stream state agg"); + if (pOperator->status == OP_RES_TO_RETURN) { + SSDataBlock* resBlock = buildStateResult(pOperator); + if (resBlock != NULL) { + return resBlock; + } + + setOperatorCompleted(pOperator); + return NULL; + } + + SOperatorInfo* downstream = pOperator->pDownstream[0]; + if (!pInfo->pUpdated) { + pInfo->pUpdated = taosArrayInit(16, sizeof(SSessionKey)); + } + if (!pInfo->pSeUpdated) { + _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); + pInfo->pSeUpdated = tSimpleHashInit(64, hashFn); + } + while (1) { + SSDataBlock* pBlock = downstream->fpSet.getNextFn(downstream); + if (pBlock == NULL) { + break; + } + printDataBlock(pBlock, "single state recv", GET_TASKID(pTaskInfo)); + + if (pBlock->info.type == STREAM_DELETE_DATA || pBlock->info.type == STREAM_DELETE_RESULT || + pBlock->info.type == STREAM_CLEAR) { + SArray* pWins = taosArrayInit(16, sizeof(SSessionKey)); + doDeleteTimeWindows(&pInfo->streamAggSup, pBlock, pWins); + removeSessionResults(pInfo->pSeUpdated, pWins); + copyDeleteWindowInfo(pWins, pInfo->pSeDeleted); + taosArrayDestroy(pWins); + continue; + } else if (pBlock->info.type == STREAM_GET_ALL) { + getAllSessionWindow(pInfo->streamAggSup.pResultRows, pInfo->pSeUpdated); + continue; + } else if (pBlock->info.type == STREAM_CREATE_CHILD_TABLE) { + return pBlock; + } else { + ASSERTS(pBlock->info.type == STREAM_NORMAL || pBlock->info.type == STREAM_INVALID, "invalid SSDataBlock type"); + } + + if (pInfo->scalarSupp.pExprInfo != NULL) { + SExprSupp* pExprSup = &pInfo->scalarSupp; + projectApplyFunctions(pExprSup->pExprInfo, pBlock, pBlock, pExprSup->pCtx, pExprSup->numOfExprs, NULL); + } + // the pDataBlock are always the same one, no need to call this again + setInputDataBlock(pSup, pBlock, TSDB_ORDER_ASC, MAIN_SCAN, true); + doStreamStateAggImpl(pOperator, pBlock, pInfo->pSeUpdated, pInfo->pSeDeleted); + pInfo->twAggSup.maxTs = TMAX(pInfo->twAggSup.maxTs, pBlock->info.window.ekey); + } + // restore the value + pOperator->status = OP_RES_TO_RETURN; + + closeSessionWindow(pInfo->streamAggSup.pResultRows, &pInfo->twAggSup, pInfo->pSeUpdated); + copyUpdateResult(pInfo->pSeUpdated, pInfo->pUpdated); + removeSessionResults(pInfo->pSeDeleted, pInfo->pUpdated); + tSimpleHashCleanup(pInfo->pSeUpdated); + pInfo->pSeUpdated = NULL; + + if (pInfo->isHistoryOp) { + getMaxTsWins(pInfo->pUpdated, pInfo->historyWins); + } + + initGroupResInfoFromArrayList(&pInfo->groupResInfo, pInfo->pUpdated); + pInfo->pUpdated = NULL; + blockDataEnsureCapacity(pInfo->binfo.pRes, pOperator->resultInfo.capacity); + + SSDataBlock* resBlock = buildStateResult(pOperator); + if (resBlock != NULL) { + return resBlock; + } + setOperatorCompleted(pOperator); + return NULL; +} + +void streamStateReleaseState(SOperatorInfo* pOperator) { + SStreamStateAggOperatorInfo* pInfo = pOperator->info; + int32_t resSize = taosArrayGetSize(pInfo->historyWins) * sizeof(SSessionKey); + qDebug("===stream=== relase state. save result count:%d", (int32_t)taosArrayGetSize(pInfo->historyWins)); + pInfo->streamAggSup.stateStore.streamStateSaveInfo(pInfo->streamAggSup.pState, STREAM_STATE_OP_STATE_NAME, + strlen(STREAM_STATE_OP_STATE_NAME), pInfo->historyWins->pData, + resSize); + SOperatorInfo* downstream = pOperator->pDownstream[0]; + if (downstream->fpSet.releaseStreamStateFn) { + downstream->fpSet.releaseStreamStateFn(downstream); + } +} + +static void compactStateWindow(SOperatorInfo* pOperator, SResultWindowInfo* pCurWin, SResultWindowInfo* pNextWin, + SSHashObj* pStUpdated, SSHashObj* pStDeleted) { + SExprSupp* pSup = &pOperator->exprSupp; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + SStorageAPI* pAPI = &pOperator->pTaskInfo->storageAPI; + + SStreamStateAggOperatorInfo* pInfo = pOperator->info; + SResultRow* pCurResult = NULL; + int32_t numOfOutput = pOperator->exprSupp.numOfExprs; + SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; + initSessionOutputBuf(pCurWin, &pCurResult, pSup->pCtx, numOfOutput, pSup->rowEntryInfoOffset); + SResultRow* pWinResult = NULL; + initSessionOutputBuf(pNextWin, &pWinResult, pAggSup->pDummyCtx, numOfOutput, pSup->rowEntryInfoOffset); + pCurWin->sessionWin.win.ekey = TMAX(pCurWin->sessionWin.win.ekey, pNextWin->sessionWin.win.ekey); + + updateTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &pCurWin->sessionWin.win, 1); + compactFunctions(pSup->pCtx, pAggSup->pDummyCtx, numOfOutput, pTaskInfo, &pInfo->twAggSup.timeWindowData); + tSimpleHashRemove(pStUpdated, &pNextWin->sessionWin, sizeof(SSessionKey)); + if (pNextWin->isOutput && pStDeleted) { + qDebug("===stream=== save delete window info %" PRId64 ", %" PRIu64, pNextWin->sessionWin.win.skey, + pNextWin->sessionWin.groupId); + saveDeleteRes(pStDeleted, pNextWin->sessionWin); + } + removeSessionResult(pStUpdated, pAggSup->pResultRows, pNextWin->sessionWin); + doDeleteSessionWindow(pAggSup, &pNextWin->sessionWin); + taosMemoryFree(pNextWin->pOutputBuf); +} + +void streamStateReloadState(SOperatorInfo* pOperator) { + SStreamStateAggOperatorInfo* pInfo = pOperator->info; + SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; + resetWinRange(&pAggSup->winRange); + + SSessionKey seKey = {.win.skey = INT64_MIN, .win.ekey = INT64_MIN, .groupId = 0}; + int32_t size = 0; + void* pBuf = NULL; + int32_t code = pAggSup->stateStore.streamStateGetInfo(pAggSup->pState, STREAM_STATE_OP_STATE_NAME, + strlen(STREAM_STATE_OP_STATE_NAME), &pBuf, &size); + int32_t num = size / sizeof(SSessionKey); + qDebug("===stream=== reload state. get result count:%d", num); + SSessionKey* pSeKeyBuf = (SSessionKey*)pBuf; + ASSERT(size == num * sizeof(SSessionKey)); + if (!pInfo->pSeUpdated && num > 0) { + _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); + pInfo->pSeUpdated = tSimpleHashInit(64, hashFn); + } + if (!pInfo->pSeDeleted && num > 0) { + _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); + pInfo->pSeDeleted = tSimpleHashInit(64, hashFn); + } + for (int32_t i = 0; i < num; i++) { + SStateWindowInfo curInfo = {0}; + SStateWindowInfo nextInfo = {0}; + SStateWindowInfo dummy = {0}; + qDebug("===stream=== reload state. try process result %" PRId64 ", %" PRIu64 ", index:%d", pSeKeyBuf[i].win.skey, + pSeKeyBuf[i].groupId, i); + setStateOutputBuf(pAggSup, pSeKeyBuf[i].win.skey, pSeKeyBuf[i].groupId, NULL, &curInfo, &nextInfo); + bool cpRes = compareWinStateKey(curInfo.pStateKey, nextInfo.pStateKey); + qDebug("===stream=== reload state. next window info %" PRId64 ", %" PRIu64 ", compare:%d", + nextInfo.winInfo.sessionWin.win.skey, nextInfo.winInfo.sessionWin.groupId, cpRes); + if (cpRes) { + compactStateWindow(pOperator, &curInfo.winInfo, &nextInfo.winInfo, pInfo->pSeUpdated, pInfo->pSeDeleted); + qDebug("===stream=== reload state. save result %" PRId64 ", %" PRIu64, curInfo.winInfo.sessionWin.win.skey, + curInfo.winInfo.sessionWin.groupId); + if (pInfo->twAggSup.calTrigger == STREAM_TRIGGER_AT_ONCE) { + saveResult(curInfo.winInfo, pInfo->pSeUpdated); + } else if (pInfo->twAggSup.calTrigger == STREAM_TRIGGER_WINDOW_CLOSE) { + if (!isCloseWindow(&curInfo.winInfo.sessionWin.win, &pInfo->twAggSup)) { + saveDeleteRes(pInfo->pSeDeleted, curInfo.winInfo.sessionWin); + } + SSessionKey key = {0}; + getSessionHashKey(&curInfo.winInfo.sessionWin, &key); + tSimpleHashPut(pAggSup->pResultRows, &key, sizeof(SSessionKey), &curInfo.winInfo, sizeof(SResultWindowInfo)); + } + } else if (IS_VALID_SESSION_WIN(nextInfo.winInfo)) { + releaseOutputBuf(pAggSup->pState, NULL, (SResultRow*)nextInfo.winInfo.pOutputBuf, + &pAggSup->pSessionAPI->stateStore); + } + + if (IS_VALID_SESSION_WIN(curInfo.winInfo)) { + saveSessionOutputBuf(pAggSup, &curInfo.winInfo); + } + } + taosMemoryFree(pBuf); + + SOperatorInfo* downstream = pOperator->pDownstream[0]; + if (downstream->fpSet.reloadStreamStateFn) { + downstream->fpSet.reloadStreamStateFn(downstream); + } +} + +SOperatorInfo* createStreamStateAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, + SExecTaskInfo* pTaskInfo, SReadHandle* pHandle) { + SStreamStateWinodwPhysiNode* pStateNode = (SStreamStateWinodwPhysiNode*)pPhyNode; + int32_t tsSlotId = ((SColumnNode*)pStateNode->window.pTspk)->slotId; + SColumnNode* pColNode = (SColumnNode*)(pStateNode->pStateKey); + int32_t code = TSDB_CODE_SUCCESS; + + SStreamStateAggOperatorInfo* pInfo = taosMemoryCalloc(1, sizeof(SStreamStateAggOperatorInfo)); + SOperatorInfo* pOperator = taosMemoryCalloc(1, sizeof(SOperatorInfo)); + if (pInfo == NULL || pOperator == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + goto _error; + } + + pInfo->stateCol = extractColumnFromColumnNode(pColNode); + initResultSizeInfo(&pOperator->resultInfo, 4096); + if (pStateNode->window.pExprs != NULL) { + int32_t numOfScalar = 0; + SExprInfo* pScalarExprInfo = createExprInfo(pStateNode->window.pExprs, NULL, &numOfScalar); + code = initExprSupp(&pInfo->scalarSupp, pScalarExprInfo, numOfScalar, &pTaskInfo->storageAPI.functionStore); + if (code != TSDB_CODE_SUCCESS) { + goto _error; + } + } + + pInfo->twAggSup = (STimeWindowAggSupp){ + .waterMark = pStateNode->window.watermark, + .calTrigger = pStateNode->window.triggerType, + .maxTs = INT64_MIN, + .minTs = INT64_MAX, + }; + + initExecTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &pTaskInfo->window); + + SExprSupp* pSup = &pOperator->exprSupp; + int32_t numOfCols = 0; + SExprInfo* pExprInfo = createExprInfo(pStateNode->window.pFuncs, NULL, &numOfCols); + SSDataBlock* pResBlock = createDataBlockFromDescNode(pPhyNode->pOutputDataBlockDesc); + code = initBasicInfoEx(&pInfo->binfo, pSup, pExprInfo, numOfCols, pResBlock, &pTaskInfo->storageAPI.functionStore); + if (code != TSDB_CODE_SUCCESS) { + goto _error; + } + int32_t keySize = sizeof(SStateKeys) + pColNode->node.resType.bytes; + int16_t type = pColNode->node.resType.type; + code = initStreamAggSupporter(&pInfo->streamAggSup, pSup->pCtx, numOfCols, 0, pTaskInfo->streamInfo.pState, keySize, + type, &pTaskInfo->storageAPI.stateStore, pHandle, &pTaskInfo->storageAPI); + if (code != TSDB_CODE_SUCCESS) { + goto _error; + } + + pInfo->primaryTsIndex = tsSlotId; + _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); + pInfo->pSeDeleted = tSimpleHashInit(64, hashFn); + pInfo->pDelIterator = NULL; + pInfo->pDelRes = createSpecialDataBlock(STREAM_DELETE_RESULT); + pInfo->pChildren = NULL; + pInfo->ignoreExpiredData = pStateNode->window.igExpired; + pInfo->ignoreExpiredDataSaved = false; + pInfo->pUpdated = NULL; + pInfo->pSeUpdated = NULL; + pInfo->dataVersion = 0; + pInfo->historyWins = taosArrayInit(4, sizeof(SSessionKey)); + if (!pInfo->historyWins) { + goto _error; + } + if (pHandle) { + pInfo->isHistoryOp = pHandle->fillHistory; + } + + setOperatorInfo(pOperator, "StreamStateAggOperator", QUERY_NODE_PHYSICAL_PLAN_STREAM_STATE, true, OP_NOT_OPENED, + pInfo, pTaskInfo); + pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, doStreamStateAgg, NULL, destroyStreamStateOperatorInfo, + optrDefaultBufFn, NULL); + setOperatorStreamStateFn(pOperator, streamStateReleaseState, streamStateReloadState); + initDownStream(downstream, &pInfo->streamAggSup, pOperator->operatorType, pInfo->primaryTsIndex, &pInfo->twAggSup); + code = appendDownstream(pOperator, &downstream, 1); + if (code != TSDB_CODE_SUCCESS) { + goto _error; + } + return pOperator; + +_error: + destroyStreamStateOperatorInfo(pInfo); + taosMemoryFreeClear(pOperator); + pTaskInfo->code = code; + return NULL; +} + +static void cleanupAfterGroupResultGen(SMergeAlignedIntervalAggOperatorInfo* pMiaInfo, SSDataBlock* pRes) { + pRes->info.id.groupId = pMiaInfo->groupId; + pMiaInfo->curTs = INT64_MIN; + pMiaInfo->groupId = 0; +} + +static void setInverFunction(SqlFunctionCtx* pCtx, int32_t num, EStreamType type) { + for (int i = 0; i < num; i++) { + if (type == STREAM_INVERT) { + fmSetInvertFunc(pCtx[i].functionId, &(pCtx[i].fpSet)); + } else if (type == STREAM_NORMAL) { + fmSetNormalFunc(pCtx[i].functionId, &(pCtx[i].fpSet)); + } + } +} + +static SSDataBlock* doStreamIntervalAgg(SOperatorInfo* pOperator) { + SStreamIntervalOperatorInfo* pInfo = pOperator->info; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + SStorageAPI* pAPI = &pOperator->pTaskInfo->storageAPI; + SExprSupp* pSup = &pOperator->exprSupp; + + qDebug("stask:%s %s status: %d", GET_TASKID(pTaskInfo), getStreamOpName(pOperator->operatorType), pOperator->status); + + if (pOperator->status == OP_EXEC_DONE) { + return NULL; + } + + if (pOperator->status == OP_RES_TO_RETURN) { + SSDataBlock* resBlock = buildIntervalResult(pOperator); + if (resBlock != NULL) { + return resBlock; + } + + if (pInfo->recvGetAll) { + pInfo->recvGetAll = false; + resetUnCloseWinInfo(pInfo->aggSup.pResultRowHashTable); + } + + setOperatorCompleted(pOperator); + if (pInfo->twAggSup.maxTs > 0 && + pInfo->twAggSup.maxTs - pInfo->twAggSup.checkPointInterval > pInfo->twAggSup.checkPointTs) { + pAPI->stateStore.streamStateCommit(pInfo->pState); + pAPI->stateStore.streamStateDeleteCheckPoint(pInfo->pState, pInfo->twAggSup.maxTs - pInfo->twAggSup.deleteMark); + setStreamDataVersion(pTaskInfo, pInfo->dataVersion, pInfo->pState->checkPointId); + pInfo->twAggSup.checkPointTs = pInfo->twAggSup.maxTs; + } + return NULL; + } + + SOperatorInfo* downstream = pOperator->pDownstream[0]; + + if (!pInfo->pUpdated) { + pInfo->pUpdated = taosArrayInit(4096, POINTER_BYTES); + } + + if (!pInfo->pUpdatedMap) { + _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); + pInfo->pUpdatedMap = tSimpleHashInit(4096, hashFn); + } + + while (1) { + SSDataBlock* pBlock = downstream->fpSet.getNextFn(downstream); + if (pBlock == NULL) { + qDebug("===stream===return data:%s. recv datablock num:%" PRIu64, getStreamOpName(pOperator->operatorType), + pInfo->numOfDatapack); + pInfo->numOfDatapack = 0; + break; + } + + pInfo->numOfDatapack++; + printSpecDataBlock(pBlock, getStreamOpName(pOperator->operatorType), "recv", GET_TASKID(pTaskInfo)); + + if (pBlock->info.type == STREAM_DELETE_DATA || pBlock->info.type == STREAM_DELETE_RESULT || + pBlock->info.type == STREAM_CLEAR) { + doDeleteWindows(pOperator, &pInfo->interval, pBlock, pInfo->pDelWins, pInfo->pUpdatedMap); + continue; + } else if (pBlock->info.type == STREAM_GET_ALL) { + qDebug("===stream===%s recv|block type STREAM_GET_ALL", getStreamOpName(pOperator->operatorType)); + pInfo->recvGetAll = true; + getAllIntervalWindow(pInfo->aggSup.pResultRowHashTable, pInfo->pUpdatedMap); + continue; + } else if (pBlock->info.type == STREAM_CREATE_CHILD_TABLE) { + printDataBlock(pBlock, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); + return pBlock; + } else { + ASSERTS(pBlock->info.type == STREAM_NORMAL || pBlock->info.type == STREAM_INVALID, "invalid SSDataBlock type"); + } + + if (pBlock->info.type == STREAM_NORMAL && pBlock->info.version != 0) { + // set input version + pTaskInfo->version = pBlock->info.version; + } + + if (pInfo->scalarSupp.pExprInfo != NULL) { + SExprSupp* pExprSup = &pInfo->scalarSupp; + projectApplyFunctions(pExprSup->pExprInfo, pBlock, pBlock, pExprSup->pCtx, pExprSup->numOfExprs, NULL); + } + + // The timewindow that overlaps the timestamps of the input pBlock need to be recalculated and return to the + // caller. Note that all the time window are not close till now. + // the pDataBlock are always the same one, no need to call this again + setInputDataBlock(pSup, pBlock, TSDB_ORDER_ASC, MAIN_SCAN, true); + if (pInfo->invertible) { + setInverFunction(pSup->pCtx, pOperator->exprSupp.numOfExprs, pBlock->info.type); + } + + doStreamIntervalAggImpl(pOperator, pBlock, pBlock->info.id.groupId, pInfo->pUpdatedMap); + pInfo->twAggSup.maxTs = TMAX(pInfo->twAggSup.maxTs, pBlock->info.window.ekey); + pInfo->twAggSup.minTs = TMIN(pInfo->twAggSup.minTs, pBlock->info.window.skey); + } + pOperator->status = OP_RES_TO_RETURN; + removeDeleteResults(pInfo->pUpdatedMap, pInfo->pDelWins); + closeStreamIntervalWindow(pInfo->aggSup.pResultRowHashTable, &pInfo->twAggSup, &pInfo->interval, NULL, + pInfo->pUpdatedMap, pInfo->pDelWins, pOperator); + + void* pIte = NULL; + int32_t iter = 0; + while ((pIte = tSimpleHashIterate(pInfo->pUpdatedMap, pIte, &iter)) != NULL) { + taosArrayPush(pInfo->pUpdated, pIte); + } + taosArraySort(pInfo->pUpdated, winPosCmprImpl); + + initMultiResInfoFromArrayList(&pInfo->groupResInfo, pInfo->pUpdated); + pInfo->pUpdated = NULL; + blockDataEnsureCapacity(pInfo->binfo.pRes, pOperator->resultInfo.capacity); + tSimpleHashCleanup(pInfo->pUpdatedMap); + pInfo->pUpdatedMap = NULL; + + return buildIntervalResult(pOperator); +} + +SOperatorInfo* createStreamIntervalOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, + SExecTaskInfo* pTaskInfo) { + SStreamIntervalOperatorInfo* pInfo = taosMemoryCalloc(1, sizeof(SStreamIntervalOperatorInfo)); + SOperatorInfo* pOperator = taosMemoryCalloc(1, sizeof(SOperatorInfo)); + if (pInfo == NULL || pOperator == NULL) { + goto _error; + } + SStreamIntervalPhysiNode* pIntervalPhyNode = (SStreamIntervalPhysiNode*)pPhyNode; + + int32_t code = TSDB_CODE_SUCCESS; + int32_t numOfCols = 0; + SExprInfo* pExprInfo = createExprInfo(pIntervalPhyNode->window.pFuncs, NULL, &numOfCols); + + SSDataBlock* pResBlock = createDataBlockFromDescNode(pPhyNode->pOutputDataBlockDesc); + pInfo->interval = (SInterval){ + .interval = pIntervalPhyNode->interval, + .sliding = pIntervalPhyNode->sliding, + .intervalUnit = pIntervalPhyNode->intervalUnit, + .slidingUnit = pIntervalPhyNode->slidingUnit, + .offset = pIntervalPhyNode->offset, + .precision = ((SColumnNode*)pIntervalPhyNode->window.pTspk)->node.resType.precision, + }; + + pInfo->twAggSup = (STimeWindowAggSupp){ + .waterMark = pIntervalPhyNode->window.watermark, + .calTrigger = pIntervalPhyNode->window.triggerType, + .maxTs = INT64_MIN, + .minTs = INT64_MAX, + .deleteMark = getDeleteMark(pIntervalPhyNode), + .checkPointTs = 0, + .checkPointInterval = + convertTimePrecision(tsCheckpointInterval, TSDB_TIME_PRECISION_MILLI, pInfo->interval.precision), + }; + + ASSERTS(pInfo->twAggSup.calTrigger != STREAM_TRIGGER_MAX_DELAY, "trigger type should not be max delay"); + + pOperator->pTaskInfo = pTaskInfo; + SStorageAPI* pAPI = &pOperator->pTaskInfo->storageAPI; + + pInfo->ignoreExpiredData = pIntervalPhyNode->window.igExpired; + pInfo->ignoreExpiredDataSaved = false; + + SExprSupp* pSup = &pOperator->exprSupp; + initBasicInfo(&pInfo->binfo, pResBlock); + initExecTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &pTaskInfo->window); + + pInfo->primaryTsIndex = ((SColumnNode*)pIntervalPhyNode->window.pTspk)->slotId; + initResultSizeInfo(&pOperator->resultInfo, 4096); + + pInfo->pState = taosMemoryCalloc(1, sizeof(SStreamState)); + *(pInfo->pState) = *(pTaskInfo->streamInfo.pState); + pAPI->stateStore.streamStateSetNumber(pInfo->pState, -1); + + size_t keyBufSize = sizeof(int64_t) + sizeof(int64_t) + POINTER_BYTES; + code = initAggSup(pSup, &pInfo->aggSup, pExprInfo, numOfCols, keyBufSize, pTaskInfo->id.str, pInfo->pState, + &pTaskInfo->storageAPI.functionStore); + if (code != TSDB_CODE_SUCCESS) { + goto _error; + } + + if (pIntervalPhyNode->window.pExprs != NULL) { + int32_t numOfScalar = 0; + SExprInfo* pScalarExprInfo = createExprInfo(pIntervalPhyNode->window.pExprs, NULL, &numOfScalar); + code = initExprSupp(&pInfo->scalarSupp, pScalarExprInfo, numOfScalar, &pTaskInfo->storageAPI.functionStore); + if (code != TSDB_CODE_SUCCESS) { + goto _error; + } + } + + pInfo->invertible = false; + pInfo->pDelWins = taosArrayInit(4, sizeof(SWinKey)); + pInfo->delIndex = 0; + pInfo->pDelRes = createSpecialDataBlock(STREAM_DELETE_RESULT); + initResultRowInfo(&pInfo->binfo.resultRowInfo); + + pInfo->pPhyNode = NULL; // create new child + pInfo->pPullDataMap = NULL; + pInfo->pFinalPullDataMap = NULL; + pInfo->pPullWins = NULL; // SPullWindowInfo + pInfo->pullIndex = 0; + pInfo->pPullDataRes = NULL; + pInfo->numOfChild = 0; + pInfo->delKey.ts = INT64_MAX; + pInfo->delKey.groupId = 0; + pInfo->numOfDatapack = 0; + pInfo->pUpdated = NULL; + pInfo->pUpdatedMap = NULL; + int32_t funResSize = getMaxFunResSize(pSup, numOfCols); + + pInfo->pState->pFileState = pTaskInfo->storageAPI.stateStore.streamFileStateInit( + tsStreamBufferSize, sizeof(SWinKey), pInfo->aggSup.resultRowSize, funResSize, compareTs, pInfo->pState, + pInfo->twAggSup.deleteMark, GET_TASKID(pTaskInfo)); + + setOperatorInfo(pOperator, "StreamIntervalOperator", QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL, true, OP_NOT_OPENED, + pInfo, pTaskInfo); + pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, doStreamIntervalAgg, NULL, + destroyStreamFinalIntervalOperatorInfo, optrDefaultBufFn, NULL); + setOperatorStreamStateFn(pOperator, streamIntervalReleaseState, streamIntervalReloadState); + + pInfo->statestore = pTaskInfo->storageAPI.stateStore; + pInfo->recvGetAll = false; + + initIntervalDownStream(downstream, pPhyNode->type, pInfo); + code = appendDownstream(pOperator, &downstream, 1); + if (code != TSDB_CODE_SUCCESS) { + goto _error; + } + + return pOperator; + +_error: + destroyStreamFinalIntervalOperatorInfo(pInfo); + taosMemoryFreeClear(pOperator); + pTaskInfo->code = code; + return NULL; +} diff --git a/source/libs/executor/src/timewindowoperator.c b/source/libs/executor/src/timewindowoperator.c index 3eb0693344..ef0c206e7b 100644 --- a/source/libs/executor/src/timewindowoperator.c +++ b/source/libs/executor/src/timewindowoperator.c @@ -27,20 +27,6 @@ #include "tlog.h" #include "ttime.h" -#define IS_FINAL_OP(op) ((op)->isFinal) -#define DEAULT_DELETE_MARK (1000LL * 60LL * 60LL * 24LL * 365LL * 10LL); -#define STREAM_INTERVAL_OP_STATE_NAME "StreamIntervalHistoryState" -#define STREAM_SESSION_OP_STATE_NAME "StreamSessionHistoryState" -#define STREAM_STATE_OP_STATE_NAME "StreamStateHistoryState" -#define STREAM_INTERVAL_OP_CHECKPOINT_NAME "StreamIntervalOperator_Checkpoint" -#define STREAM_SESSION_OP_CHECKPOINT_NAME "StreamSessionOperator_Checkpoint" -#define STREAM_STATE_OP_CHECKPOINT_NAME "StreamStateOperator_Checkpoint" - -typedef struct SStateWindowInfo { - SResultWindowInfo winInfo; - SStateKeys* pStateKey; -} SStateWindowInfo; - typedef struct SSessionAggOperatorInfo { SOptrBasicInfo binfo; SAggSupporter aggSup; @@ -70,12 +56,6 @@ typedef enum SResultTsInterpType { RESULT_ROW_END_INTERP = 2, } SResultTsInterpType; -typedef struct SPullWindowInfo { - STimeWindow window; - uint64_t groupId; - STimeWindow calWin; -} SPullWindowInfo; - typedef struct SOpenWindowInfo { SResultRowPosition pos; uint64_t groupId; @@ -87,8 +67,6 @@ static SResultRowPosition addToOpenWindowList(SResultRowInfo* pResultRowInfo, co uint64_t groupId); static void doCloseWindow(SResultRowInfo* pResultRowInfo, const SIntervalAggOperatorInfo* pInfo, SResultRow* pResult); -static TSKEY getStartTsKey(STimeWindow* win, const TSKEY* tsCols) { return tsCols == NULL ? win->skey : tsCols[0]; } - static int32_t setTimeWindowOutputBuf(SResultRowInfo* pResultRowInfo, STimeWindow* win, bool masterscan, SResultRow** pResult, int64_t tableGroupId, SqlFunctionCtx* pCtx, int32_t numOfOutput, int32_t* rowEntryInfoOffset, SAggSupporter* pAggSup, @@ -110,15 +88,6 @@ static int32_t setTimeWindowOutputBuf(SResultRowInfo* pResultRowInfo, STimeWindo return TSDB_CODE_SUCCESS; } -static void updateTimeWindowInfo(SColumnInfoData* pColData, STimeWindow* pWin, int64_t delta) { - int64_t* ts = (int64_t*)pColData->pData; - - int64_t duration = pWin->ekey - pWin->skey + delta; - ts[2] = duration; // set the duration - ts[3] = pWin->skey; // window start key - ts[4] = pWin->ekey + delta; // window end key -} - static void doKeepTuple(SWindowRowsSup* pRowSup, int64_t ts, uint64_t groupId) { pRowSup->win.ekey = ts; pRowSup->prevTs = ts; @@ -430,8 +399,8 @@ bool inSlidingWindow(SInterval* pInterval, STimeWindow* pWin, SDataBlockInfo* pB return inCalSlidingWindow(pInterval, pWin, pBlockInfo->calWin.skey, pBlockInfo->calWin.ekey, pBlockInfo->type); } -static int32_t getNextQualifiedWindow(SInterval* pInterval, STimeWindow* pNext, SDataBlockInfo* pDataBlockInfo, - TSKEY* primaryKeys, int32_t prevPosition, int32_t order) { +int32_t getNextQualifiedWindow(SInterval* pInterval, STimeWindow* pNext, SDataBlockInfo* pDataBlockInfo, + TSKEY* primaryKeys, int32_t prevPosition, int32_t order) { bool ascQuery = (order == TSDB_ORDER_ASC); int32_t precision = pInterval->precision; @@ -662,227 +631,9 @@ static void doInterpUnclosedTimeWindow(SOperatorInfo* pOperatorInfo, int32_t num } } -typedef int32_t (*__compare_fn_t)(void* pKey, void* data, int32_t index); - -int32_t binarySearchCom(void* keyList, int num, void* pKey, int order, __compare_fn_t comparefn) { - int firstPos = 0, lastPos = num - 1, midPos = -1; - int numOfRows = 0; - - if (num <= 0) return -1; - if (order == TSDB_ORDER_DESC) { - // find the first position which is smaller or equal than the key - while (1) { - if (comparefn(pKey, keyList, lastPos) >= 0) return lastPos; - if (comparefn(pKey, keyList, firstPos) == 0) return firstPos; - if (comparefn(pKey, keyList, firstPos) < 0) return firstPos - 1; - - numOfRows = lastPos - firstPos + 1; - midPos = (numOfRows >> 1) + firstPos; - - if (comparefn(pKey, keyList, midPos) < 0) { - lastPos = midPos - 1; - } else if (comparefn(pKey, keyList, midPos) > 0) { - firstPos = midPos + 1; - } else { - break; - } - } - - } else { - // find the first position which is bigger or equal than the key - while (1) { - if (comparefn(pKey, keyList, firstPos) <= 0) return firstPos; - if (comparefn(pKey, keyList, lastPos) == 0) return lastPos; - - if (comparefn(pKey, keyList, lastPos) > 0) { - lastPos = lastPos + 1; - if (lastPos >= num) - return -1; - else - return lastPos; - } - - numOfRows = lastPos - firstPos + 1; - midPos = (numOfRows >> 1) + firstPos; - - if (comparefn(pKey, keyList, midPos) < 0) { - lastPos = midPos - 1; - } else if (comparefn(pKey, keyList, midPos) > 0) { - firstPos = midPos + 1; - } else { - break; - } - } - } - - return midPos; -} - -typedef int64_t (*__get_value_fn_t)(void* data, int32_t index); - -int32_t binarySearch(void* keyList, int num, TSKEY key, int order, __get_value_fn_t getValuefn) { - int firstPos = 0, lastPos = num - 1, midPos = -1; - int numOfRows = 0; - - if (num <= 0) return -1; - if (order == TSDB_ORDER_DESC) { - // find the first position which is smaller or equal than the key - while (1) { - if (key >= getValuefn(keyList, lastPos)) return lastPos; - if (key == getValuefn(keyList, firstPos)) return firstPos; - if (key < getValuefn(keyList, firstPos)) return firstPos - 1; - - numOfRows = lastPos - firstPos + 1; - midPos = (numOfRows >> 1) + firstPos; - - if (key < getValuefn(keyList, midPos)) { - lastPos = midPos - 1; - } else if (key > getValuefn(keyList, midPos)) { - firstPos = midPos + 1; - } else { - break; - } - } - - } else { - // find the first position which is bigger or equal than the key - while (1) { - if (key <= getValuefn(keyList, firstPos)) return firstPos; - if (key == getValuefn(keyList, lastPos)) return lastPos; - - if (key > getValuefn(keyList, lastPos)) { - lastPos = lastPos + 1; - if (lastPos >= num) - return -1; - else - return lastPos; - } - - numOfRows = lastPos - firstPos + 1; - midPos = (numOfRows >> 1) + firstPos; - - if (key < getValuefn(keyList, midPos)) { - lastPos = midPos - 1; - } else if (key > getValuefn(keyList, midPos)) { - firstPos = midPos + 1; - } else { - break; - } - } - } - - return midPos; -} - -int32_t comparePullWinKey(void* pKey, void* data, int32_t index) { - SArray* res = (SArray*)data; - SPullWindowInfo* pos = taosArrayGet(res, index); - SPullWindowInfo* pData = (SPullWindowInfo*)pKey; - if (pData->groupId > pos->groupId) { - return 1; - } else if (pData->groupId < pos->groupId) { - return -1; - } - - if (pData->window.skey > pos->window.ekey) { - return 1; - } else if (pData->window.ekey < pos->window.skey) { - return -1; - } - return 0; -} - -static int32_t savePullWindow(SPullWindowInfo* pPullInfo, SArray* pPullWins) { - int32_t size = taosArrayGetSize(pPullWins); - int32_t index = binarySearchCom(pPullWins, size, pPullInfo, TSDB_ORDER_DESC, comparePullWinKey); - if (index == -1) { - index = 0; - } else { - int32_t code = comparePullWinKey(pPullInfo, pPullWins, index); - if (code == 0) { - SPullWindowInfo* pos = taosArrayGet(pPullWins, index); - pos->window.skey = TMIN(pos->window.skey, pPullInfo->window.skey); - pos->window.ekey = TMAX(pos->window.ekey, pPullInfo->window.ekey); - pos->calWin.skey = TMIN(pos->calWin.skey, pPullInfo->calWin.skey); - pos->calWin.ekey = TMAX(pos->calWin.ekey, pPullInfo->calWin.ekey); - return TSDB_CODE_SUCCESS; - } else if (code > 0) { - index++; - } - } - if (taosArrayInsert(pPullWins, index, pPullInfo) == NULL) { - return TSDB_CODE_OUT_OF_MEMORY; - } - return TSDB_CODE_SUCCESS; -} - -static int32_t saveResult(SResultWindowInfo winInfo, SSHashObj* pStUpdated) { - winInfo.sessionWin.win.ekey = winInfo.sessionWin.win.skey; - return tSimpleHashPut(pStUpdated, &winInfo.sessionWin, sizeof(SSessionKey), &winInfo, sizeof(SResultWindowInfo)); -} - -static int32_t saveWinResult(SWinKey* pKey, SRowBuffPos* pPos, SSHashObj* pUpdatedMap) { - tSimpleHashPut(pUpdatedMap, pKey, sizeof(SWinKey), &pPos, POINTER_BYTES); - return TSDB_CODE_SUCCESS; -} - -static int32_t saveWinResultInfo(TSKEY ts, uint64_t groupId, SRowBuffPos* pPos, SSHashObj* pUpdatedMap) { - SWinKey key = {.ts = ts, .groupId = groupId}; - saveWinResult(&key, pPos, pUpdatedMap); - return TSDB_CODE_SUCCESS; -} - -static void removeResults(SArray* pWins, SSHashObj* pUpdatedMap) { - int32_t size = taosArrayGetSize(pWins); - for (int32_t i = 0; i < size; i++) { - SWinKey* pW = taosArrayGet(pWins, i); - void* tmp = tSimpleHashGet(pUpdatedMap, pW, sizeof(SWinKey)); - if (tmp) { - void* value = *(void**)tmp; - taosMemoryFree(value); - tSimpleHashRemove(pUpdatedMap, pW, sizeof(SWinKey)); - } - } -} - -int32_t compareWinKey(void* pKey, void* data, int32_t index) { - void* pDataPos = taosArrayGet((SArray*)data, index); - return winKeyCmprImpl(pKey, pDataPos); -} - -static void removeDeleteResults(SSHashObj* pUpdatedMap, SArray* pDelWins) { - taosArraySort(pDelWins, winKeyCmprImpl); - taosArrayRemoveDuplicate(pDelWins, winKeyCmprImpl, NULL); - int32_t delSize = taosArrayGetSize(pDelWins); - if (tSimpleHashGetSize(pUpdatedMap) == 0 || delSize == 0) { - return; - } - void* pIte = NULL; - int32_t iter = 0; - while ((pIte = tSimpleHashIterate(pUpdatedMap, pIte, &iter)) != NULL) { - SWinKey* pResKey = tSimpleHashGetKey(pIte, NULL); - int32_t index = binarySearchCom(pDelWins, delSize, pResKey, TSDB_ORDER_DESC, compareWinKey); - if (index >= 0 && 0 == compareWinKey(pResKey, pDelWins, index)) { - taosArrayRemove(pDelWins, index); - delSize = taosArrayGetSize(pDelWins); - } - } -} - -bool isOverdue(TSKEY ekey, STimeWindowAggSupp* pTwSup) { - ASSERTS(pTwSup->maxTs == INT64_MIN || pTwSup->maxTs > 0, "maxts should greater than 0"); - return pTwSup->maxTs != INT64_MIN && ekey < pTwSup->maxTs - pTwSup->waterMark; -} - -bool isCloseWindow(STimeWindow* pWin, STimeWindowAggSupp* pTwSup) { return isOverdue(pWin->ekey, pTwSup); } - -bool needDeleteWindowBuf(STimeWindow* pWin, STimeWindowAggSupp* pTwSup) { - return pTwSup->maxTs != INT64_MIN && pWin->ekey < pTwSup->maxTs - pTwSup->deleteMark; -} - static bool tsKeyCompFn(void* l, void* r, void* param) { - TSKEY* lTS = (TSKEY*)l; - TSKEY* rTS = (TSKEY*)r; + TSKEY* lTS = (TSKEY*)l; + TSKEY* rTS = (TSKEY*)r; SIntervalAggOperatorInfo* pInfo = param; return pInfo->binfo.outputTsOrder == ORDER_ASC ? *lTS < *rTS : *lTS > *rTS; } @@ -1005,7 +756,8 @@ static bool hashIntervalAgg(SOperatorInfo* pOperatorInfo, SResultRowInfo* pResul STimeWindow nextWin = win; while (1) { int32_t prevEndPos = forwardRows - 1 + startPos; - startPos = getNextQualifiedWindow(&pInfo->interval, &nextWin, &pBlock->info, tsCols, prevEndPos, pInfo->binfo.inputTsOrder); + startPos = getNextQualifiedWindow(&pInfo->interval, &nextWin, &pBlock->info, tsCols, prevEndPos, + pInfo->binfo.inputTsOrder); if (startPos < 0 || filterWindowWithLimit(pInfo, &nextWin, tableGroupId)) { break; } @@ -1131,18 +883,6 @@ static int32_t doOpenIntervalAgg(SOperatorInfo* pOperator) { return TSDB_CODE_SUCCESS; } -static bool compareVal(const char* v, const SStateKeys* pKey) { - if (IS_VAR_DATA_TYPE(pKey->type)) { - if (varDataLen(v) != varDataLen(pKey->pData)) { - return false; - } else { - return memcmp(varDataVal(v), varDataVal(pKey->pData), varDataLen(v)) == 0; - } - } else { - return memcmp(pKey->pData, v, pKey->bytes) == 0; - } -} - static void doStateWindowAggImpl(SOperatorInfo* pOperator, SStateWindowOperatorInfo* pInfo, SSDataBlock* pBlock) { SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; SExprSupp* pSup = &pOperator->exprSupp; @@ -1377,170 +1117,6 @@ static void doClearWindowImpl(SResultRowPosition* p1, SDiskbasedBuf* pResultBuf, releaseBufPage(pResultBuf, bufPage); } -static bool doDeleteWindow(SOperatorInfo* pOperator, TSKEY ts, uint64_t groupId) { - SStorageAPI* pAPI = &pOperator->pTaskInfo->storageAPI; - - SStreamIntervalOperatorInfo* pInfo = pOperator->info; - SWinKey key = {.ts = ts, .groupId = groupId}; - tSimpleHashRemove(pInfo->aggSup.pResultRowHashTable, &key, sizeof(SWinKey)); - pAPI->stateStore.streamStateDel(pInfo->pState, &key); - return true; -} - -static int32_t getChildIndex(SSDataBlock* pBlock) { return pBlock->info.childId; } - -static void doDeleteWindows(SOperatorInfo* pOperator, SInterval* pInterval, SSDataBlock* pBlock, SArray* pUpWins, - SSHashObj* pUpdatedMap) { - SStreamIntervalOperatorInfo* pInfo = pOperator->info; - SColumnInfoData* pStartTsCol = taosArrayGet(pBlock->pDataBlock, START_TS_COLUMN_INDEX); - TSKEY* startTsCols = (TSKEY*)pStartTsCol->pData; - SColumnInfoData* pEndTsCol = taosArrayGet(pBlock->pDataBlock, END_TS_COLUMN_INDEX); - TSKEY* endTsCols = (TSKEY*)pEndTsCol->pData; - SColumnInfoData* pCalStTsCol = taosArrayGet(pBlock->pDataBlock, CALCULATE_START_TS_COLUMN_INDEX); - TSKEY* calStTsCols = (TSKEY*)pCalStTsCol->pData; - SColumnInfoData* pCalEnTsCol = taosArrayGet(pBlock->pDataBlock, CALCULATE_END_TS_COLUMN_INDEX); - TSKEY* calEnTsCols = (TSKEY*)pCalEnTsCol->pData; - SColumnInfoData* pGpCol = taosArrayGet(pBlock->pDataBlock, GROUPID_COLUMN_INDEX); - uint64_t* pGpDatas = (uint64_t*)pGpCol->pData; - for (int32_t i = 0; i < pBlock->info.rows; i++) { - SResultRowInfo dumyInfo = {0}; - dumyInfo.cur.pageId = -1; - - STimeWindow win = {0}; - if (IS_FINAL_OP(pInfo)) { - win.skey = startTsCols[i]; - win.ekey = endTsCols[i]; - } else { - win = getActiveTimeWindow(NULL, &dumyInfo, startTsCols[i], pInterval, TSDB_ORDER_ASC); - } - - do { - if (!inCalSlidingWindow(pInterval, &win, calStTsCols[i], calEnTsCols[i], pBlock->info.type)) { - getNextTimeWindow(pInterval, &win, TSDB_ORDER_ASC); - continue; - } - uint64_t winGpId = pGpDatas[i]; - SWinKey winRes = {.ts = win.skey, .groupId = winGpId}; - void* chIds = taosHashGet(pInfo->pPullDataMap, &winRes, sizeof(SWinKey)); - if (chIds) { - int32_t childId = getChildIndex(pBlock); - SArray* chArray = *(void**)chIds; - int32_t index = taosArraySearchIdx(chArray, &childId, compareInt32Val, TD_EQ); - if (index != -1) { - qDebug("===stream===try push delete window%" PRId64 "chId:%d ,continue", win.skey, childId); - getNextTimeWindow(pInterval, &win, TSDB_ORDER_ASC); - continue; - } - } - bool res = doDeleteWindow(pOperator, win.skey, winGpId); - if (pUpWins && res) { - taosArrayPush(pUpWins, &winRes); - } - if (pUpdatedMap) { - tSimpleHashRemove(pUpdatedMap, &winRes, sizeof(SWinKey)); - } - getNextTimeWindow(pInterval, &win, TSDB_ORDER_ASC); - } while (win.ekey <= endTsCols[i]); - } -} - -static int32_t getAllIntervalWindow(SSHashObj* pHashMap, SSHashObj* resWins) { - void* pIte = NULL; - int32_t iter = 0; - while ((pIte = tSimpleHashIterate(pHashMap, pIte, &iter)) != NULL) { - SWinKey* pKey = tSimpleHashGetKey(pIte, NULL); - uint64_t groupId = pKey->groupId; - TSKEY ts = pKey->ts; - int32_t code = saveWinResultInfo(ts, groupId, *(SRowBuffPos**)pIte, resWins); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - } - return TSDB_CODE_SUCCESS; -} - -static int32_t closeStreamIntervalWindow(SSHashObj* pHashMap, STimeWindowAggSupp* pTwSup, SInterval* pInterval, - SHashObj* pPullDataMap, SSHashObj* closeWins, SArray* pDelWins, - SOperatorInfo* pOperator) { - qDebug("===stream===close interval window"); - void* pIte = NULL; - int32_t iter = 0; - SStreamIntervalOperatorInfo* pInfo = pOperator->info; - int32_t delSize = taosArrayGetSize(pDelWins); - while ((pIte = tSimpleHashIterate(pHashMap, pIte, &iter)) != NULL) { - void* key = tSimpleHashGetKey(pIte, NULL); - SWinKey* pWinKey = (SWinKey*)key; - if (delSize > 0) { - int32_t index = binarySearchCom(pDelWins, delSize, pWinKey, TSDB_ORDER_DESC, compareWinKey); - if (index >= 0 && 0 == compareWinKey(pWinKey, pDelWins, index)) { - taosArrayRemove(pDelWins, index); - delSize = taosArrayGetSize(pDelWins); - } - } - - void* chIds = taosHashGet(pPullDataMap, pWinKey, sizeof(SWinKey)); - STimeWindow win = { - .skey = pWinKey->ts, - .ekey = taosTimeAdd(win.skey, pInterval->interval, pInterval->intervalUnit, pInterval->precision) - 1, - }; - if (isCloseWindow(&win, pTwSup)) { - if (chIds && pPullDataMap) { - SArray* chAy = *(SArray**)chIds; - int32_t size = taosArrayGetSize(chAy); - qDebug("===stream===window %" PRId64 " wait child size:%d", pWinKey->ts, size); - for (int32_t i = 0; i < size; i++) { - qDebug("===stream===window %" PRId64 " wait child id:%d", pWinKey->ts, *(int32_t*)taosArrayGet(chAy, i)); - } - continue; - } else if (pPullDataMap) { - qDebug("===stream===close window %" PRId64, pWinKey->ts); - } - - if (pTwSup->calTrigger == STREAM_TRIGGER_WINDOW_CLOSE) { - int32_t code = saveWinResult(pWinKey, *(SRowBuffPos**)pIte, closeWins); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - } - tSimpleHashIterateRemove(pHashMap, pWinKey, sizeof(SWinKey), &pIte, &iter); - } - } - return TSDB_CODE_SUCCESS; -} - -STimeWindow getFinalTimeWindow(int64_t ts, SInterval* pInterval) { - STimeWindow w = {.skey = ts, .ekey = INT64_MAX}; - w.ekey = taosTimeAdd(w.skey, pInterval->interval, pInterval->intervalUnit, pInterval->precision) - 1; - return w; -} - -static void doBuildDeleteResult(SStreamIntervalOperatorInfo* pInfo, SArray* pWins, int32_t* index, - SSDataBlock* pBlock) { - blockDataCleanup(pBlock); - int32_t size = taosArrayGetSize(pWins); - if (*index == size) { - *index = 0; - taosArrayClear(pWins); - return; - } - blockDataEnsureCapacity(pBlock, size - *index); - uint64_t uid = 0; - for (int32_t i = *index; i < size; i++) { - SWinKey* pWin = taosArrayGet(pWins, i); - void* tbname = NULL; - pInfo->stateStore.streamStateGetParName(pInfo->pState, pWin->groupId, &tbname); - if (tbname == NULL) { - appendOneRowToStreamSpecialBlock(pBlock, &pWin->ts, &pWin->ts, &uid, &pWin->groupId, NULL); - } else { - char parTbName[VARSTR_HEADER_SIZE + TSDB_TABLE_NAME_LEN]; - STR_WITH_MAXSIZE_TO_VARSTR(parTbName, tbname, sizeof(parTbName)); - appendOneRowToStreamSpecialBlock(pBlock, &pWin->ts, &pWin->ts, &uid, &pWin->groupId, parTbName); - } - pInfo->stateStore.streamStateFreeVal(tbname); - (*index)++; - } -} - static void destroyStateWindowOperatorInfo(void* param) { SStateWindowOperatorInfo* pInfo = (SStateWindowOperatorInfo*)param; cleanupBasicInfo(&pInfo->binfo); @@ -1577,46 +1153,6 @@ void destroyIntervalOperatorInfo(void* param) { taosMemoryFreeClear(param); } -void destroyStreamFinalIntervalOperatorInfo(void* param) { - SStreamIntervalOperatorInfo* pInfo = (SStreamIntervalOperatorInfo*)param; - cleanupBasicInfo(&pInfo->binfo); - cleanupAggSup(&pInfo->aggSup); - // it should be empty. - void* pIte = NULL; - while ((pIte = taosHashIterate(pInfo->pPullDataMap, pIte)) != NULL) { - taosArrayDestroy(*(void**)pIte); - } - taosHashCleanup(pInfo->pPullDataMap); - taosHashCleanup(pInfo->pFinalPullDataMap); - taosArrayDestroy(pInfo->pPullWins); - blockDataDestroy(pInfo->pPullDataRes); - taosArrayDestroy(pInfo->pDelWins); - blockDataDestroy(pInfo->pDelRes); - pInfo->stateStore.streamFileStateDestroy(pInfo->pState->pFileState); - taosMemoryFreeClear(pInfo->pState); - - nodesDestroyNode((SNode*)pInfo->pPhyNode); - colDataDestroy(&pInfo->twAggSup.timeWindowData); - pInfo->groupResInfo.pRows = taosArrayDestroy(pInfo->groupResInfo.pRows); - cleanupExprSupp(&pInfo->scalarSupp); - tSimpleHashCleanup(pInfo->pUpdatedMap); - pInfo->pUpdatedMap = NULL; - pInfo->pUpdated = taosArrayDestroy(pInfo->pUpdated); - - blockDataDestroy(pInfo->pCheckpointRes); - - taosMemoryFreeClear(param); -} - -static bool allInvertible(SqlFunctionCtx* pFCtx, int32_t numOfCols) { - for (int32_t i = 0; i < numOfCols; i++) { - if (fmIsUserDefinedFunc(pFCtx[i].functionId) || !fmIsInvertible(pFCtx[i].functionId)) { - return false; - } - } - return true; -} - static bool timeWindowinterpNeeded(SqlFunctionCtx* pCtx, int32_t numOfCols, SIntervalAggOperatorInfo* pInfo) { // the primary timestamp column bool needed = false; @@ -1671,27 +1207,6 @@ static bool timeWindowinterpNeeded(SqlFunctionCtx* pCtx, int32_t numOfCols, SInt return needed; } -void initIntervalDownStream(SOperatorInfo* downstream, uint16_t type, SStreamIntervalOperatorInfo* pInfo) { - SStateStore* pAPI = &downstream->pTaskInfo->storageAPI.stateStore; - - if (downstream->operatorType != QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN) { - initIntervalDownStream(downstream->pDownstream[0], type, pInfo); - return; - } - - SStreamScanInfo* pScanInfo = downstream->info; - pScanInfo->windowSup.parentType = type; - pScanInfo->windowSup.pIntervalAggSup = &pInfo->aggSup; - if (!pScanInfo->pUpdateInfo) { - pScanInfo->pUpdateInfo = - pAPI->updateInfoInitP(&pInfo->interval, pInfo->twAggSup.waterMark, pScanInfo->igCheckUpdate); - } - - pScanInfo->interval = pInfo->interval; - pScanInfo->twAggSup = pInfo->twAggSup; - pScanInfo->pState = pInfo->pState; -} - SOperatorInfo* createIntervalOperatorInfo(SOperatorInfo* downstream, SIntervalPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo) { SIntervalAggOperatorInfo* pInfo = taosMemoryCalloc(1, sizeof(SIntervalAggOperatorInfo)); @@ -2093,3074 +1608,6 @@ _error: return NULL; } -void compactFunctions(SqlFunctionCtx* pDestCtx, SqlFunctionCtx* pSourceCtx, int32_t numOfOutput, - SExecTaskInfo* pTaskInfo, SColumnInfoData* pTimeWindowData) { - for (int32_t k = 0; k < numOfOutput; ++k) { - if (fmIsWindowPseudoColumnFunc(pDestCtx[k].functionId)) { - if (!pTimeWindowData) { - continue; - } - - SResultRowEntryInfo* pEntryInfo = GET_RES_INFO(&pDestCtx[k]); - char* p = GET_ROWCELL_INTERBUF(pEntryInfo); - SColumnInfoData idata = {0}; - idata.info.type = TSDB_DATA_TYPE_BIGINT; - idata.info.bytes = tDataTypes[TSDB_DATA_TYPE_BIGINT].bytes; - idata.pData = p; - - SScalarParam out = {.columnData = &idata}; - SScalarParam tw = {.numOfRows = 5, .columnData = pTimeWindowData}; - pDestCtx[k].sfp.process(&tw, 1, &out); - pEntryInfo->numOfRes = 1; - } else if (functionNeedToExecute(&pDestCtx[k]) && pDestCtx[k].fpSet.combine != NULL) { - int32_t code = pDestCtx[k].fpSet.combine(&pDestCtx[k], &pSourceCtx[k]); - if (code != TSDB_CODE_SUCCESS) { - qError("%s apply combine functions error, code: %s", GET_TASKID(pTaskInfo), tstrerror(code)); - } - } else if (pDestCtx[k].fpSet.combine == NULL) { - char* funName = fmGetFuncName(pDestCtx[k].functionId); - qError("%s error, combine funcion for %s is not implemented", GET_TASKID(pTaskInfo), funName); - taosMemoryFreeClear(funName); - } - } -} - -bool hasIntervalWindow(void* pState, SWinKey* pKey, SStateStore* pStore) { - return pStore->streamStateCheck(pState, pKey); -} - -int32_t setIntervalOutputBuf(void* pState, STimeWindow* win, SRowBuffPos** pResult, int64_t groupId, - SqlFunctionCtx* pCtx, int32_t numOfOutput, int32_t* rowEntryInfoOffset, - SAggSupporter* pAggSup, SStateStore* pStore) { - SWinKey key = {.ts = win->skey, .groupId = groupId}; - char* value = NULL; - int32_t size = pAggSup->resultRowSize; - - if (pStore->streamStateAddIfNotExist(pState, &key, (void**)&value, &size) < 0) { - return TSDB_CODE_OUT_OF_MEMORY; - } - - *pResult = (SRowBuffPos*)value; - SResultRow* res = (SResultRow*)((*pResult)->pRowBuff); - - // set time window for current result - res->win = (*win); - setResultRowInitCtx(res, pCtx, numOfOutput, rowEntryInfoOffset); - return TSDB_CODE_SUCCESS; -} - -bool isDeletedStreamWindow(STimeWindow* pWin, uint64_t groupId, void* pState, STimeWindowAggSupp* pTwSup, - SStateStore* pStore) { - if (pTwSup->maxTs != INT64_MIN && pWin->ekey < pTwSup->maxTs - pTwSup->deleteMark) { - SWinKey key = {.ts = pWin->skey, .groupId = groupId}; - if (!hasIntervalWindow(pState, &key, pStore)) { - return true; - } - return false; - } - return false; -} - -int32_t getNexWindowPos(SInterval* pInterval, SDataBlockInfo* pBlockInfo, TSKEY* tsCols, int32_t startPos, TSKEY eKey, - STimeWindow* pNextWin) { - int32_t forwardRows = - getNumOfRowsInTimeWindow(pBlockInfo, tsCols, startPos, eKey, binarySearchForKey, NULL, TSDB_ORDER_ASC); - int32_t prevEndPos = forwardRows - 1 + startPos; - return getNextQualifiedWindow(pInterval, pNextWin, pBlockInfo, tsCols, prevEndPos, TSDB_ORDER_ASC); -} - -void addPullWindow(SHashObj* pMap, SWinKey* pWinRes, int32_t size) { - SArray* childIds = taosArrayInit(8, sizeof(int32_t)); - for (int32_t i = 0; i < size; i++) { - taosArrayPush(childIds, &i); - } - taosHashPut(pMap, pWinRes, sizeof(SWinKey), &childIds, sizeof(void*)); -} - -static void clearStreamIntervalOperator(SStreamIntervalOperatorInfo* pInfo) { - tSimpleHashClear(pInfo->aggSup.pResultRowHashTable); - clearDiskbasedBuf(pInfo->aggSup.pResultBuf); - initResultRowInfo(&pInfo->binfo.resultRowInfo); - pInfo->aggSup.currentPageId = -1; - pInfo->stateStore.streamStateClear(pInfo->pState); -} - -static void clearSpecialDataBlock(SSDataBlock* pBlock) { - if (pBlock->info.rows <= 0) { - return; - } - blockDataCleanup(pBlock); -} - -static void doBuildPullDataBlock(SArray* array, int32_t* pIndex, SSDataBlock* pBlock) { - clearSpecialDataBlock(pBlock); - int32_t size = taosArrayGetSize(array); - if (size - (*pIndex) == 0) { - return; - } - blockDataEnsureCapacity(pBlock, size - (*pIndex)); - SColumnInfoData* pStartTs = (SColumnInfoData*)taosArrayGet(pBlock->pDataBlock, START_TS_COLUMN_INDEX); - SColumnInfoData* pEndTs = (SColumnInfoData*)taosArrayGet(pBlock->pDataBlock, END_TS_COLUMN_INDEX); - SColumnInfoData* pGroupId = (SColumnInfoData*)taosArrayGet(pBlock->pDataBlock, GROUPID_COLUMN_INDEX); - SColumnInfoData* pCalStartTs = (SColumnInfoData*)taosArrayGet(pBlock->pDataBlock, CALCULATE_START_TS_COLUMN_INDEX); - SColumnInfoData* pCalEndTs = (SColumnInfoData*)taosArrayGet(pBlock->pDataBlock, CALCULATE_END_TS_COLUMN_INDEX); - for (; (*pIndex) < size; (*pIndex)++) { - SPullWindowInfo* pWin = taosArrayGet(array, (*pIndex)); - colDataSetVal(pStartTs, pBlock->info.rows, (const char*)&pWin->window.skey, false); - colDataSetVal(pEndTs, pBlock->info.rows, (const char*)&pWin->window.ekey, false); - colDataSetVal(pGroupId, pBlock->info.rows, (const char*)&pWin->groupId, false); - colDataSetVal(pCalStartTs, pBlock->info.rows, (const char*)&pWin->calWin.skey, false); - colDataSetVal(pCalEndTs, pBlock->info.rows, (const char*)&pWin->calWin.ekey, false); - pBlock->info.rows++; - } - if ((*pIndex) == size) { - *pIndex = 0; - taosArrayClear(array); - } - blockDataUpdateTsWindow(pBlock, 0); -} - -void processPullOver(SSDataBlock* pBlock, SHashObj* pMap, SHashObj* pFinalMap, SInterval* pInterval, SArray* pPullWins, - int32_t numOfCh, SOperatorInfo* pOperator) { - SColumnInfoData* pStartCol = taosArrayGet(pBlock->pDataBlock, CALCULATE_START_TS_COLUMN_INDEX); - TSKEY* tsData = (TSKEY*)pStartCol->pData; - SColumnInfoData* pEndCol = taosArrayGet(pBlock->pDataBlock, CALCULATE_END_TS_COLUMN_INDEX); - TSKEY* tsEndData = (TSKEY*)pEndCol->pData; - SColumnInfoData* pGroupCol = taosArrayGet(pBlock->pDataBlock, GROUPID_COLUMN_INDEX); - uint64_t* groupIdData = (uint64_t*)pGroupCol->pData; - int32_t chId = getChildIndex(pBlock); - for (int32_t i = 0; i < pBlock->info.rows; i++) { - TSKEY winTs = tsData[i]; - while (winTs <= tsEndData[i]) { - SWinKey winRes = {.ts = winTs, .groupId = groupIdData[i]}; - void* chIds = taosHashGet(pMap, &winRes, sizeof(SWinKey)); - if (chIds) { - SArray* chArray = *(SArray**)chIds; - int32_t index = taosArraySearchIdx(chArray, &chId, compareInt32Val, TD_EQ); - if (index != -1) { - qDebug("===stream===retrive window %" PRId64 " delete child id %d", winRes.ts, chId); - taosArrayRemove(chArray, index); - if (taosArrayGetSize(chArray) == 0) { - // pull data is over - taosArrayDestroy(chArray); - taosHashRemove(pMap, &winRes, sizeof(SWinKey)); - qDebug("===stream===retrive pull data over.window %" PRId64, winRes.ts); - - void* pFinalCh = taosHashGet(pFinalMap, &winRes, sizeof(SWinKey)); - if (pFinalCh) { - taosHashRemove(pFinalMap, &winRes, sizeof(SWinKey)); - doDeleteWindow(pOperator, winRes.ts, winRes.groupId); - STimeWindow nextWin = getFinalTimeWindow(winRes.ts, pInterval); - SPullWindowInfo pull = {.window = nextWin, - .groupId = winRes.groupId, - .calWin.skey = nextWin.skey, - .calWin.ekey = nextWin.skey}; - // add pull data request - if (savePullWindow(&pull, pPullWins) == TSDB_CODE_SUCCESS) { - addPullWindow(pMap, &winRes, numOfCh); - qDebug("===stream===prepare final retrive for delete %" PRId64 ", size:%d", winRes.ts, numOfCh); - } - } - } - } - } - winTs = taosTimeAdd(winTs, pInterval->sliding, pInterval->slidingUnit, pInterval->precision); - } - } -} - -static void addRetriveWindow(SArray* wins, SStreamIntervalOperatorInfo* pInfo, int32_t childId) { - int32_t size = taosArrayGetSize(wins); - for (int32_t i = 0; i < size; i++) { - SWinKey* winKey = taosArrayGet(wins, i); - STimeWindow nextWin = getFinalTimeWindow(winKey->ts, &pInfo->interval); - if (isOverdue(nextWin.ekey, &pInfo->twAggSup) && pInfo->ignoreExpiredData) { - continue; - } - void* chIds = taosHashGet(pInfo->pPullDataMap, winKey, sizeof(SWinKey)); - if (!chIds) { - SPullWindowInfo pull = { - .window = nextWin, .groupId = winKey->groupId, .calWin.skey = nextWin.skey, .calWin.ekey = nextWin.skey}; - // add pull data request - if (savePullWindow(&pull, pInfo->pPullWins) == TSDB_CODE_SUCCESS) { - addPullWindow(pInfo->pPullDataMap, winKey, pInfo->numOfChild); - qDebug("===stream===prepare retrive for delete %" PRId64 ", size:%d", winKey->ts, pInfo->numOfChild); - } - } else { - SArray* chArray = *(void**)chIds; - int32_t index = taosArraySearchIdx(chArray, &childId, compareInt32Val, TD_EQ); - qDebug("===stream===check final retrive %" PRId64 ",chid:%d", winKey->ts, index); - if (index == -1) { - qDebug("===stream===add final retrive %" PRId64, winKey->ts); - taosHashPut(pInfo->pFinalPullDataMap, winKey, sizeof(SWinKey), NULL, 0); - } - } - } -} - -static void clearFunctionContext(SExprSupp* pSup) { - for (int32_t i = 0; i < pSup->numOfExprs; i++) { - pSup->pCtx[i].saveHandle.currentPage = -1; - } -} - -int32_t getOutputBuf(void* pState, SRowBuffPos* pPos, SResultRow** pResult, SStateStore* pStore) { - return pStore->streamStateGetByPos(pState, pPos, (void**)pResult); -} - -int32_t buildDataBlockFromGroupRes(SOperatorInfo* pOperator, void* pState, SSDataBlock* pBlock, SExprSupp* pSup, - SGroupResInfo* pGroupResInfo) { - SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; - SStorageAPI* pAPI = &pOperator->pTaskInfo->storageAPI; - - SExprInfo* pExprInfo = pSup->pExprInfo; - int32_t numOfExprs = pSup->numOfExprs; - int32_t* rowEntryOffset = pSup->rowEntryInfoOffset; - SqlFunctionCtx* pCtx = pSup->pCtx; - - int32_t numOfRows = getNumOfTotalRes(pGroupResInfo); - - for (int32_t i = pGroupResInfo->index; i < numOfRows; i += 1) { - SRowBuffPos* pPos = *(SRowBuffPos**)taosArrayGet(pGroupResInfo->pRows, i); - SResultRow* pRow = NULL; - int32_t code = getOutputBuf(pState, pPos, &pRow, &pAPI->stateStore); - uint64_t groupId = ((SWinKey*)pPos->pKey)->groupId; - ASSERT(code == 0); - doUpdateNumOfRows(pCtx, pRow, numOfExprs, rowEntryOffset); - // no results, continue to check the next one - if (pRow->numOfRows == 0) { - pGroupResInfo->index += 1; - continue; - } - if (pBlock->info.id.groupId == 0) { - pBlock->info.id.groupId = groupId; - void* tbname = NULL; - if (pAPI->stateStore.streamStateGetParName(pTaskInfo->streamInfo.pState, pBlock->info.id.groupId, &tbname) < 0) { - pBlock->info.parTbName[0] = 0; - } else { - memcpy(pBlock->info.parTbName, tbname, TSDB_TABLE_NAME_LEN); - } - pAPI->stateStore.streamStateFreeVal(tbname); - } else { - // current value belongs to different group, it can't be packed into one datablock - if (pBlock->info.id.groupId != groupId) { - break; - } - } - - if (pBlock->info.rows + pRow->numOfRows > pBlock->info.capacity) { - ASSERT(pBlock->info.rows > 0); - break; - } - pGroupResInfo->index += 1; - - for (int32_t j = 0; j < numOfExprs; ++j) { - int32_t slotId = pExprInfo[j].base.resSchema.slotId; - - pCtx[j].resultInfo = getResultEntryInfo(pRow, j, rowEntryOffset); - SResultRowEntryInfo* pEnryInfo = pCtx[j].resultInfo; - - if (pCtx[j].fpSet.finalize) { - int32_t code1 = pCtx[j].fpSet.finalize(&pCtx[j], pBlock); - if (TAOS_FAILED(code1)) { - qError("%s build result data block error, code %s", GET_TASKID(pTaskInfo), tstrerror(code1)); - T_LONG_JMP(pTaskInfo->env, code1); - } - } else if (strcmp(pCtx[j].pExpr->pExpr->_function.functionName, "_select_value") == 0) { - // do nothing, todo refactor - } else { - // expand the result into multiple rows. E.g., _wstart, top(k, 20) - // the _wstart needs to copy to 20 following rows, since the results of top-k expands to 20 different rows. - SColumnInfoData* pColInfoData = taosArrayGet(pBlock->pDataBlock, slotId); - char* in = GET_ROWCELL_INTERBUF(pCtx[j].resultInfo); - for (int32_t k = 0; k < pRow->numOfRows; ++k) { - colDataSetVal(pColInfoData, pBlock->info.rows + k, in, pCtx[j].resultInfo->isNullRes); - } - } - } - - pBlock->info.rows += pRow->numOfRows; - } - - pBlock->info.dataLoad = 1; - blockDataUpdateTsWindow(pBlock, 0); - return TSDB_CODE_SUCCESS; -} - -void doBuildStreamIntervalResult(SOperatorInfo* pOperator, void* pState, SSDataBlock* pBlock, - SGroupResInfo* pGroupResInfo) { - SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; - // set output datablock version - pBlock->info.version = pTaskInfo->version; - - blockDataCleanup(pBlock); - if (!hasRemainResults(pGroupResInfo)) { - return; - } - - // clear the existed group id - pBlock->info.id.groupId = 0; - buildDataBlockFromGroupRes(pOperator, pState, pBlock, &pOperator->exprSupp, pGroupResInfo); -} - -static int32_t getNextQualifiedFinalWindow(SInterval* pInterval, STimeWindow* pNext, SDataBlockInfo* pDataBlockInfo, - TSKEY* primaryKeys, int32_t prevPosition) { - int32_t startPos = prevPosition + 1; - if (startPos == pDataBlockInfo->rows) { - startPos = -1; - } else { - *pNext = getFinalTimeWindow(primaryKeys[startPos], pInterval); - } - return startPos; -} - -static void doStreamIntervalAggImpl(SOperatorInfo* pOperatorInfo, SSDataBlock* pSDataBlock, uint64_t groupId, - SSHashObj* pUpdatedMap) { - SStreamIntervalOperatorInfo* pInfo = (SStreamIntervalOperatorInfo*)pOperatorInfo->info; - pInfo->dataVersion = TMAX(pInfo->dataVersion, pSDataBlock->info.version); - - SResultRowInfo* pResultRowInfo = &(pInfo->binfo.resultRowInfo); - SExecTaskInfo* pTaskInfo = pOperatorInfo->pTaskInfo; - SExprSupp* pSup = &pOperatorInfo->exprSupp; - int32_t numOfOutput = pSup->numOfExprs; - int32_t step = 1; - TSKEY* tsCols = NULL; - SRowBuffPos* pResPos = NULL; - SResultRow* pResult = NULL; - int32_t forwardRows = 0; - - SColumnInfoData* pColDataInfo = taosArrayGet(pSDataBlock->pDataBlock, pInfo->primaryTsIndex); - tsCols = (int64_t*)pColDataInfo->pData; - - int32_t startPos = 0; - TSKEY ts = getStartTsKey(&pSDataBlock->info.window, tsCols); - STimeWindow nextWin = {0}; - if (IS_FINAL_OP(pInfo)) { - nextWin = getFinalTimeWindow(ts, &pInfo->interval); - } else { - nextWin = getActiveTimeWindow(pInfo->aggSup.pResultBuf, pResultRowInfo, ts, &pInfo->interval, TSDB_ORDER_ASC); - } - while (1) { - bool isClosed = isCloseWindow(&nextWin, &pInfo->twAggSup); - if ((pInfo->ignoreExpiredData && isClosed && !IS_FINAL_OP(pInfo)) || - !inSlidingWindow(&pInfo->interval, &nextWin, &pSDataBlock->info)) { - startPos = getNexWindowPos(&pInfo->interval, &pSDataBlock->info, tsCols, startPos, nextWin.ekey, &nextWin); - if (startPos < 0) { - break; - } - continue; - } - - if (IS_FINAL_OP(pInfo) && pInfo->numOfChild > 0) { - bool ignore = true; - SWinKey winRes = { - .ts = nextWin.skey, - .groupId = groupId, - }; - void* chIds = taosHashGet(pInfo->pPullDataMap, &winRes, sizeof(SWinKey)); - if (isDeletedStreamWindow(&nextWin, groupId, pInfo->pState, &pInfo->twAggSup, &pInfo->stateStore) && isClosed && - !chIds) { - SPullWindowInfo pull = { - .window = nextWin, .groupId = groupId, .calWin.skey = nextWin.skey, .calWin.ekey = nextWin.skey}; - // add pull data request - if (savePullWindow(&pull, pInfo->pPullWins) == TSDB_CODE_SUCCESS) { - addPullWindow(pInfo->pPullDataMap, &winRes, pInfo->numOfChild); - } - } else { - int32_t index = -1; - SArray* chArray = NULL; - int32_t chId = 0; - if (chIds) { - chArray = *(void**)chIds; - chId = getChildIndex(pSDataBlock); - index = taosArraySearchIdx(chArray, &chId, compareInt32Val, TD_EQ); - } - if (index == -1 || pSDataBlock->info.type == STREAM_PULL_DATA) { - ignore = false; - } - } - - if (ignore) { - startPos = getNextQualifiedFinalWindow(&pInfo->interval, &nextWin, &pSDataBlock->info, tsCols, startPos); - if (startPos < 0) { - break; - } - continue; - } - } - - int32_t code = setIntervalOutputBuf(pInfo->pState, &nextWin, &pResPos, groupId, pSup->pCtx, numOfOutput, - pSup->rowEntryInfoOffset, &pInfo->aggSup, &pInfo->stateStore); - pResult = (SResultRow*)pResPos->pRowBuff; - if (code != TSDB_CODE_SUCCESS || pResult == NULL) { - T_LONG_JMP(pTaskInfo->env, TSDB_CODE_OUT_OF_MEMORY); - } - if (IS_FINAL_OP(pInfo)) { - forwardRows = 1; - } else { - forwardRows = getNumOfRowsInTimeWindow(&pSDataBlock->info, tsCols, startPos, nextWin.ekey, binarySearchForKey, - NULL, TSDB_ORDER_ASC); - } - - SWinKey key = { - .ts = pResult->win.skey, - .groupId = groupId, - }; - if (pInfo->twAggSup.calTrigger == STREAM_TRIGGER_AT_ONCE && pUpdatedMap) { - saveWinResult(&key, pResPos, pUpdatedMap); - } - - if (pInfo->twAggSup.calTrigger == STREAM_TRIGGER_WINDOW_CLOSE) { - tSimpleHashPut(pInfo->aggSup.pResultRowHashTable, &key, sizeof(SWinKey), &pResPos, POINTER_BYTES); - } - - updateTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &nextWin, 1); - applyAggFunctionOnPartialTuples(pTaskInfo, pSup->pCtx, &pInfo->twAggSup.timeWindowData, startPos, forwardRows, - pSDataBlock->info.rows, numOfOutput); - key.ts = nextWin.skey; - - if (pInfo->delKey.ts > key.ts) { - pInfo->delKey = key; - } - int32_t prevEndPos = (forwardRows - 1) * step + startPos; - if (pSDataBlock->info.window.skey <= 0 || pSDataBlock->info.window.ekey <= 0) { - qError("table uid %" PRIu64 " data block timestamp range may not be calculated! minKey %" PRId64 - ",maxKey %" PRId64, - pSDataBlock->info.id.uid, pSDataBlock->info.window.skey, pSDataBlock->info.window.ekey); - blockDataUpdateTsWindow(pSDataBlock, 0); - - // timestamp of the data is incorrect - if (pSDataBlock->info.window.skey <= 0 || pSDataBlock->info.window.ekey <= 0) { - qError("table uid %" PRIu64 " data block timestamp is out of range! minKey %" PRId64 ",maxKey %" PRId64, - pSDataBlock->info.id.uid, pSDataBlock->info.window.skey, pSDataBlock->info.window.ekey); - } - } - - if (IS_FINAL_OP(pInfo)) { - startPos = getNextQualifiedFinalWindow(&pInfo->interval, &nextWin, &pSDataBlock->info, tsCols, prevEndPos); - } else { - startPos = - getNextQualifiedWindow(&pInfo->interval, &nextWin, &pSDataBlock->info, tsCols, prevEndPos, TSDB_ORDER_ASC); - } - if (startPos < 0) { - break; - } - } -} - -static inline int winPosCmprImpl(const void* pKey1, const void* pKey2) { - SRowBuffPos* pos1 = *(SRowBuffPos**)pKey1; - SRowBuffPos* pos2 = *(SRowBuffPos**)pKey2; - SWinKey* pWin1 = (SWinKey*)pos1->pKey; - SWinKey* pWin2 = (SWinKey*)pos2->pKey; - - if (pWin1->groupId > pWin2->groupId) { - return 1; - } else if (pWin1->groupId < pWin2->groupId) { - return -1; - } - - if (pWin1->ts > pWin2->ts) { - return 1; - } else if (pWin1->ts < pWin2->ts) { - return -1; - } - - return 0; -} - -static void resetUnCloseWinInfo(SSHashObj* winMap) { - void* pIte = NULL; - int32_t iter = 0; - while ((pIte = tSimpleHashIterate(winMap, pIte, &iter)) != NULL) { - SRowBuffPos* pPos = *(SRowBuffPos**)pIte; - pPos->beUsed = true; - } -} - -int32_t encodeSWinKey(void** buf, SWinKey* key) { - int32_t tlen = 0; - tlen += taosEncodeFixedI64(buf, key->ts); - tlen += taosEncodeFixedU64(buf, key->groupId); - return tlen; -} - -void* decodeSWinKey(void* buf, SWinKey* key) { - buf = taosDecodeFixedI64(buf, &key->ts); - buf = taosDecodeFixedU64(buf, &key->groupId); - return buf; -} - -int32_t encodeSRowBuffPos(void** buf, SRowBuffPos* pos) { - int32_t tlen = 0; - tlen += encodeSWinKey(buf, pos->pKey); - return tlen; -} - -void* decodeSRowBuffPos(void* buf, SRowBuffPos* pos) { - buf = decodeSWinKey(buf, pos->pKey); - return buf; -} - -int32_t encodeSTimeWindowAggSupp(void** buf, STimeWindowAggSupp* pTwAggSup) { - int32_t tlen = 0; - tlen += taosEncodeFixedI64(buf, pTwAggSup->minTs); - tlen += taosEncodeFixedI64(buf, pTwAggSup->maxTs); - return tlen; -} - -void* decodeSTimeWindowAggSupp(void* buf, STimeWindowAggSupp* pTwAggSup) { - buf = taosDecodeFixedI64(buf, &pTwAggSup->minTs); - buf = taosDecodeFixedI64(buf, &pTwAggSup->maxTs); - return buf; -} - -int32_t encodeSTimeWindow(void** buf, STimeWindow* pWin) { - int32_t tlen = 0; - tlen += taosEncodeFixedI64(buf, pWin->skey); - tlen += taosEncodeFixedI64(buf, pWin->ekey); - return tlen; -} - -void* decodeSTimeWindow(void* buf, STimeWindow* pWin) { - buf = taosDecodeFixedI64(buf, &pWin->skey); - buf = taosDecodeFixedI64(buf, &pWin->ekey); - return buf; -} - -int32_t encodeSPullWindowInfo(void** buf, SPullWindowInfo* pPullInfo) { - int32_t tlen = 0; - tlen += encodeSTimeWindow(buf, &pPullInfo->calWin); - tlen += taosEncodeFixedU64(buf, pPullInfo->groupId); - tlen += encodeSTimeWindow(buf, &pPullInfo->window); - return tlen; -} - -void* decodeSPullWindowInfo(void* buf, SPullWindowInfo* pPullInfo) { - buf = decodeSTimeWindow(buf, &pPullInfo->calWin); - buf = taosDecodeFixedU64(buf, &pPullInfo->groupId); - buf = decodeSTimeWindow(buf, &pPullInfo->window); - return buf; -} - -int32_t encodeSPullWindowInfoArray(void** buf, SArray* pPullInfos) { - int32_t tlen = 0; - int32_t size = taosArrayGetSize(pPullInfos); - tlen += taosEncodeFixedI32(buf, size); - for (int32_t i = 0; i < size; i++) { - void* pItem = taosArrayGet(pPullInfos, i); - tlen += encodeSPullWindowInfo(buf, pItem); - } - return tlen; -} - -void* decodeSPullWindowInfoArray(void* buf, SArray* pPullInfos) { - int32_t size = 0; - buf = taosDecodeFixedI32(buf, &size); - for (int32_t i = 0; i < size; i++) { - SPullWindowInfo item = {0}; - buf = decodeSPullWindowInfo(buf, &item); - taosArrayPush(pPullInfos, &item); - } - return buf; -} - -int32_t doStreamIntervalEncodeOpState(void** buf, int32_t len, SOperatorInfo* pOperator) { - SStreamIntervalOperatorInfo* pInfo = pOperator->info; - if (!pInfo) { - return 0; - } - - void* pData = (buf == NULL) ? NULL : *buf; - - // 1.pResultRowHashTable - int32_t tlen = 0; - int32_t mapSize = tSimpleHashGetSize(pInfo->aggSup.pResultRowHashTable); - tlen += taosEncodeFixedI32(buf, mapSize); - void* pIte = NULL; - size_t keyLen = 0; - int32_t iter = 0; - while ((pIte = tSimpleHashIterate(pInfo->aggSup.pResultRowHashTable, pIte, &iter)) != NULL) { - void* key = taosHashGetKey(pIte, &keyLen); - tlen += encodeSWinKey(buf, key); - } - - // 2.twAggSup - tlen += encodeSTimeWindowAggSupp(buf, &pInfo->twAggSup); - - // 3.pPullDataMap - int32_t size = taosHashGetSize(pInfo->pPullDataMap); - tlen += taosEncodeFixedI32(buf, size); - pIte = NULL; - keyLen = 0; - while ((pIte = taosHashIterate(pInfo->pPullDataMap, pIte)) != NULL) { - void* key = taosHashGetKey(pIte, &keyLen); - tlen += encodeSWinKey(buf, key); - SArray* pArray = (SArray*)pIte; - int32_t chSize = taosArrayGetSize(pArray); - tlen += taosEncodeFixedI32(buf, chSize); - for (int32_t i = 0; i < chSize; i++) { - void* pChItem = taosArrayGet(pArray, i); - tlen += taosEncodeFixedI32(buf, *(int32_t*)pChItem); - } - } - - // 4.pPullWins - tlen += encodeSPullWindowInfoArray(buf, pInfo->pPullWins); - - // 5.dataVersion - tlen += taosEncodeFixedI64(buf, pInfo->dataVersion); - - // 6.checksum - if (buf) { - uint32_t cksum = taosCalcChecksum(0, pData, len - sizeof(uint32_t)); - tlen += taosEncodeFixedU32(buf, cksum); - } else { - tlen += sizeof(uint32_t); - } - - return tlen; -} - -void doStreamIntervalDecodeOpState(void* buf, int32_t len, SOperatorInfo* pOperator) { - SStreamIntervalOperatorInfo* pInfo = pOperator->info; - if (!pInfo) { - return; - } - - // 6.checksum - int32_t dataLen = len - sizeof(uint32_t); - void* pCksum = POINTER_SHIFT(buf, dataLen); - if (taosCheckChecksum(buf, dataLen, *(uint32_t*)pCksum) != TSDB_CODE_SUCCESS) { - ASSERT(0); // debug - qError("stream interval state is invalid"); - return; - } - - // 1.pResultRowHashTable - int32_t mapSize = 0; - buf = taosDecodeFixedI32(buf, &mapSize); - for (int32_t i = 0; i < mapSize; i++) { - SWinKey key = {0}; - buf = decodeSWinKey(buf, &key); - SRowBuffPos* pPos = NULL; - int32_t resSize = pInfo->aggSup.resultRowSize; - pInfo->stateStore.streamStateAddIfNotExist(pInfo->pState, &key, (void**)&pPos, &resSize); - tSimpleHashPut(pInfo->aggSup.pResultRowHashTable, &key, sizeof(SWinKey), &pPos, POINTER_BYTES); - } - - // 2.twAggSup - buf = decodeSTimeWindowAggSupp(buf, &pInfo->twAggSup); - - // 3.pPullDataMap - int32_t size = 0; - buf = taosDecodeFixedI32(buf, &size); - for (int32_t i = 0; i < size; i++) { - SWinKey key = {0}; - SArray* pArray = taosArrayInit(0, sizeof(int32_t)); - buf = decodeSWinKey(buf, &key); - int32_t chSize = 0; - buf = taosDecodeFixedI32(buf, &chSize); - for (int32_t i = 0; i < chSize; i++) { - int32_t chId = 0; - buf = taosDecodeFixedI32(buf, &chId); - taosArrayPush(pArray, &chId); - } - taosHashPut(pInfo->pPullDataMap, &key, sizeof(SWinKey), &pArray, POINTER_BYTES); - } - - // 4.pPullWins - buf = decodeSPullWindowInfoArray(buf, pInfo->pPullWins); - - // 5.dataVersion - buf = taosDecodeFixedI64(buf, &pInfo->dataVersion); -} - -void doStreamIntervalSaveCheckpoint(SOperatorInfo* pOperator) { - SStreamIntervalOperatorInfo* pInfo = pOperator->info; - int32_t len = doStreamIntervalEncodeOpState(NULL, 0, pOperator); - void* buf = taosMemoryCalloc(1, len); - void* pBuf = buf; - len = doStreamIntervalEncodeOpState(&pBuf, len, pOperator); - pInfo->stateStore.streamStateSaveInfo(pInfo->pState, STREAM_INTERVAL_OP_CHECKPOINT_NAME, - strlen(STREAM_INTERVAL_OP_CHECKPOINT_NAME), buf, len); - taosMemoryFree(buf); -} - -static SSDataBlock* doStreamFinalIntervalAgg(SOperatorInfo* pOperator) { - SStreamIntervalOperatorInfo* pInfo = pOperator->info; - SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; - SStorageAPI* pAPI = &pOperator->pTaskInfo->storageAPI; - - SOperatorInfo* downstream = pOperator->pDownstream[0]; - SExprSupp* pSup = &pOperator->exprSupp; - - qDebug("interval status %d %s", pOperator->status, IS_FINAL_OP(pInfo) ? "interval final" : "interval semi"); - - if (pOperator->status == OP_EXEC_DONE) { - return NULL; - } else if (pOperator->status == OP_RES_TO_RETURN) { - doBuildPullDataBlock(pInfo->pPullWins, &pInfo->pullIndex, pInfo->pPullDataRes); - if (pInfo->pPullDataRes->info.rows != 0) { - // process the rest of the data - printDataBlock(pInfo->pPullDataRes, IS_FINAL_OP(pInfo) ? "interval final" : "interval semi"); - return pInfo->pPullDataRes; - } - - doBuildDeleteResult(pInfo, pInfo->pDelWins, &pInfo->delIndex, pInfo->pDelRes); - if (pInfo->pDelRes->info.rows != 0) { - // process the rest of the data - printDataBlock(pInfo->pDelRes, IS_FINAL_OP(pInfo) ? "interval final" : "interval semi"); - return pInfo->pDelRes; - } - - doBuildStreamIntervalResult(pOperator, pInfo->pState, pInfo->binfo.pRes, &pInfo->groupResInfo); - if (pInfo->binfo.pRes->info.rows != 0) { - printDataBlock(pInfo->binfo.pRes, IS_FINAL_OP(pInfo) ? "interval final" : "interval semi"); - return pInfo->binfo.pRes; - } - - if (pInfo->recvGetAll) { - pInfo->recvGetAll = false; - resetUnCloseWinInfo(pInfo->aggSup.pResultRowHashTable); - } - - if (pInfo->reCkBlock) { - pInfo->reCkBlock = false; - printDataBlock(pInfo->pCheckpointRes, IS_FINAL_OP(pInfo) ? "interval final ck" : "interval semi ck"); - return pInfo->pCheckpointRes; - } - - setOperatorCompleted(pOperator); - if (!IS_FINAL_OP(pInfo)) { - clearFunctionContext(&pOperator->exprSupp); - // semi interval operator clear disk buffer - clearStreamIntervalOperator(pInfo); - qDebug("===stream===clear semi operator"); - } - return NULL; - } else { - if (!IS_FINAL_OP(pInfo)) { - doBuildDeleteResult(pInfo, pInfo->pDelWins, &pInfo->delIndex, pInfo->pDelRes); - if (pInfo->pDelRes->info.rows != 0) { - // process the rest of the data - printDataBlock(pInfo->pDelRes, IS_FINAL_OP(pInfo) ? "interval final" : "interval semi"); - return pInfo->pDelRes; - } - } - } - - if (!pInfo->pUpdated) { - pInfo->pUpdated = taosArrayInit(4096, POINTER_BYTES); - } - if (!pInfo->pUpdatedMap) { - _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); - pInfo->pUpdatedMap = tSimpleHashInit(4096, hashFn); - } - - while (1) { - if (isTaskKilled(pTaskInfo)) { - if (pInfo->pUpdated != NULL) { - pInfo->pUpdated = taosArrayDestroy(pInfo->pUpdated); - } - - if (pInfo->pUpdatedMap != NULL) { - tSimpleHashCleanup(pInfo->pUpdatedMap); - pInfo->pUpdatedMap = NULL; - } - - T_LONG_JMP(pTaskInfo->env, pTaskInfo->code); - } - - SSDataBlock* pBlock = downstream->fpSet.getNextFn(downstream); - if (pBlock == NULL) { - pOperator->status = OP_RES_TO_RETURN; - qDebug("===stream===return data:%s. recv datablock num:%" PRIu64, - IS_FINAL_OP(pInfo) ? "interval final" : "interval semi", pInfo->numOfDatapack); - pInfo->numOfDatapack = 0; - break; - } - pInfo->numOfDatapack++; - printDataBlock(pBlock, IS_FINAL_OP(pInfo) ? "interval final recv" : "interval semi recv"); - - if (pBlock->info.type == STREAM_NORMAL || pBlock->info.type == STREAM_PULL_DATA) { - pInfo->binfo.pRes->info.type = pBlock->info.type; - } else if (pBlock->info.type == STREAM_DELETE_DATA || pBlock->info.type == STREAM_DELETE_RESULT || - pBlock->info.type == STREAM_CLEAR) { - SArray* delWins = taosArrayInit(8, sizeof(SWinKey)); - doDeleteWindows(pOperator, &pInfo->interval, pBlock, delWins, pInfo->pUpdatedMap); - if (IS_FINAL_OP(pInfo)) { - int32_t chId = getChildIndex(pBlock); - addRetriveWindow(delWins, pInfo, chId); - if (pBlock->info.type != STREAM_CLEAR) { - taosArrayAddAll(pInfo->pDelWins, delWins); - } - taosArrayDestroy(delWins); - continue; - } - removeResults(delWins, pInfo->pUpdatedMap); - taosArrayAddAll(pInfo->pDelWins, delWins); - taosArrayDestroy(delWins); - - doBuildDeleteResult(pInfo, pInfo->pDelWins, &pInfo->delIndex, pInfo->pDelRes); - if (pInfo->pDelRes->info.rows != 0) { - // process the rest of the data - printDataBlock(pInfo->pDelRes, IS_FINAL_OP(pInfo) ? "interval final" : "interval semi"); - if (pBlock->info.type == STREAM_CLEAR) { - pInfo->pDelRes->info.type = STREAM_CLEAR; - } else { - pInfo->pDelRes->info.type = STREAM_DELETE_RESULT; - } - return pInfo->pDelRes; - } - - break; - } else if (pBlock->info.type == STREAM_GET_ALL && IS_FINAL_OP(pInfo)) { - pInfo->recvGetAll = true; - getAllIntervalWindow(pInfo->aggSup.pResultRowHashTable, pInfo->pUpdatedMap); - continue; - } else if (pBlock->info.type == STREAM_RETRIEVE && !IS_FINAL_OP(pInfo)) { - doDeleteWindows(pOperator, &pInfo->interval, pBlock, NULL, pInfo->pUpdatedMap); - if (taosArrayGetSize(pInfo->pUpdated) > 0) { - break; - } - continue; - } else if (pBlock->info.type == STREAM_PULL_OVER && IS_FINAL_OP(pInfo)) { - processPullOver(pBlock, pInfo->pPullDataMap, pInfo->pFinalPullDataMap, &pInfo->interval, pInfo->pPullWins, - pInfo->numOfChild, pOperator); - continue; - } else if (pBlock->info.type == STREAM_CREATE_CHILD_TABLE) { - return pBlock; - } else if (pBlock->info.type == STREAM_CHECKPOINT) { - pAPI->stateStore.streamStateCommit(pInfo->pState); - doStreamIntervalSaveCheckpoint(pOperator); - copyDataBlock(pInfo->pCheckpointRes, pBlock); - continue; - } else { - ASSERTS(pBlock->info.type == STREAM_INVALID, "invalid SSDataBlock type"); - } - - if (pInfo->scalarSupp.pExprInfo != NULL) { - SExprSupp* pExprSup = &pInfo->scalarSupp; - projectApplyFunctions(pExprSup->pExprInfo, pBlock, pBlock, pExprSup->pCtx, pExprSup->numOfExprs, NULL); - } - setInputDataBlock(pSup, pBlock, TSDB_ORDER_ASC, MAIN_SCAN, true); - doStreamIntervalAggImpl(pOperator, pBlock, pBlock->info.id.groupId, pInfo->pUpdatedMap); - pInfo->twAggSup.maxTs = TMAX(pInfo->twAggSup.maxTs, pBlock->info.window.ekey); - pInfo->twAggSup.maxTs = TMAX(pInfo->twAggSup.maxTs, pBlock->info.watermark); - pInfo->twAggSup.minTs = TMIN(pInfo->twAggSup.minTs, pBlock->info.window.skey); - } - - removeDeleteResults(pInfo->pUpdatedMap, pInfo->pDelWins); - if (IS_FINAL_OP(pInfo)) { - closeStreamIntervalWindow(pInfo->aggSup.pResultRowHashTable, &pInfo->twAggSup, &pInfo->interval, - pInfo->pPullDataMap, pInfo->pUpdatedMap, pInfo->pDelWins, pOperator); - } - pInfo->binfo.pRes->info.watermark = pInfo->twAggSup.maxTs; - - void* pIte = NULL; - int32_t iter = 0; - while ((pIte = tSimpleHashIterate(pInfo->pUpdatedMap, pIte, &iter)) != NULL) { - taosArrayPush(pInfo->pUpdated, pIte); - } - - tSimpleHashCleanup(pInfo->pUpdatedMap); - pInfo->pUpdatedMap = NULL; - taosArraySort(pInfo->pUpdated, winPosCmprImpl); - - initMultiResInfoFromArrayList(&pInfo->groupResInfo, pInfo->pUpdated); - pInfo->pUpdated = NULL; - blockDataEnsureCapacity(pInfo->binfo.pRes, pOperator->resultInfo.capacity); - - doBuildPullDataBlock(pInfo->pPullWins, &pInfo->pullIndex, pInfo->pPullDataRes); - if (pInfo->pPullDataRes->info.rows != 0) { - // process the rest of the data - printDataBlock(pInfo->pPullDataRes, IS_FINAL_OP(pInfo) ? "interval final" : "interval semi"); - return pInfo->pPullDataRes; - } - - doBuildDeleteResult(pInfo, pInfo->pDelWins, &pInfo->delIndex, pInfo->pDelRes); - if (pInfo->pDelRes->info.rows != 0) { - // process the rest of the data - printDataBlock(pInfo->pDelRes, IS_FINAL_OP(pInfo) ? "interval final" : "interval semi"); - return pInfo->pDelRes; - } - - doBuildStreamIntervalResult(pOperator, pInfo->pState, pInfo->binfo.pRes, &pInfo->groupResInfo); - if (pInfo->binfo.pRes->info.rows != 0) { - printDataBlock(pInfo->binfo.pRes, IS_FINAL_OP(pInfo) ? "interval final" : "interval semi"); - return pInfo->binfo.pRes; - } - - if (pInfo->reCkBlock) { - pInfo->reCkBlock = false; - printDataBlock(pInfo->binfo.pRes, IS_FINAL_OP(pInfo) ? "interval final ck" : "interval semi ck"); - return pInfo->pCheckpointRes; - } - - return NULL; -} - -int64_t getDeleteMark(SIntervalPhysiNode* pIntervalPhyNode) { - if (pIntervalPhyNode->window.deleteMark <= 0) { - return DEAULT_DELETE_MARK; - } - int64_t deleteMark = TMAX(pIntervalPhyNode->window.deleteMark, pIntervalPhyNode->window.watermark); - deleteMark = TMAX(deleteMark, pIntervalPhyNode->interval); - return deleteMark; -} - -TSKEY compareTs(void* pKey) { - SWinKey* pWinKey = (SWinKey*)pKey; - return pWinKey->ts; -} - -int32_t getSelectivityBufSize(SqlFunctionCtx* pCtx) { - if (pCtx->subsidiaries.rowLen == 0) { - int32_t rowLen = 0; - for (int32_t j = 0; j < pCtx->subsidiaries.num; ++j) { - SqlFunctionCtx* pc = pCtx->subsidiaries.pCtx[j]; - rowLen += pc->pExpr->base.resSchema.bytes; - } - - return rowLen + pCtx->subsidiaries.num * sizeof(bool); - } else { - return pCtx->subsidiaries.rowLen; - } -} - -int32_t getMaxFunResSize(SExprSupp* pSup, int32_t numOfCols) { - int32_t size = 0; - for (int32_t i = 0; i < numOfCols; ++i) { - int32_t resSize = getSelectivityBufSize(pSup->pCtx + i); - size = TMAX(size, resSize); - } - return size; -} - -void streamIntervalReleaseState(SOperatorInfo* pOperator) { - if (pOperator->operatorType != QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL) { - SStreamIntervalOperatorInfo* pInfo = pOperator->info; - int32_t resSize = sizeof(TSKEY); - pInfo->stateStore.streamStateSaveInfo(pInfo->pState, STREAM_INTERVAL_OP_STATE_NAME, - strlen(STREAM_INTERVAL_OP_STATE_NAME), &pInfo->twAggSup.maxTs, resSize); - } - SStreamIntervalOperatorInfo* pInfo = pOperator->info; - SStorageAPI* pAPI = &pOperator->pTaskInfo->storageAPI; - pAPI->stateStore.streamStateCommit(pInfo->pState); - SOperatorInfo* downstream = pOperator->pDownstream[0]; - if (downstream->fpSet.releaseStreamStateFn) { - downstream->fpSet.releaseStreamStateFn(downstream); - } -} - -void streamIntervalReloadState(SOperatorInfo* pOperator) { - if (pOperator->operatorType != QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL) { - SStreamIntervalOperatorInfo* pInfo = pOperator->info; - int32_t size = 0; - void* pBuf = NULL; - int32_t code = pInfo->stateStore.streamStateGetInfo(pInfo->pState, STREAM_INTERVAL_OP_STATE_NAME, - strlen(STREAM_INTERVAL_OP_STATE_NAME), &pBuf, &size); - TSKEY ts = *(TSKEY*)pBuf; - taosMemoryFree(pBuf); - pInfo->twAggSup.maxTs = TMAX(pInfo->twAggSup.maxTs, ts); - pInfo->stateStore.streamStateReloadInfo(pInfo->pState, ts); - } - SOperatorInfo* downstream = pOperator->pDownstream[0]; - if (downstream->fpSet.reloadStreamStateFn) { - downstream->fpSet.reloadStreamStateFn(downstream); - } -} - -SOperatorInfo* createStreamFinalIntervalOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, - SExecTaskInfo* pTaskInfo, int32_t numOfChild, - SReadHandle* pHandle) { - SIntervalPhysiNode* pIntervalPhyNode = (SIntervalPhysiNode*)pPhyNode; - SStreamIntervalOperatorInfo* pInfo = taosMemoryCalloc(1, sizeof(SStreamIntervalOperatorInfo)); - SOperatorInfo* pOperator = taosMemoryCalloc(1, sizeof(SOperatorInfo)); - if (pInfo == NULL || pOperator == NULL) { - goto _error; - } - - pOperator->pTaskInfo = pTaskInfo; - SStorageAPI* pAPI = &pTaskInfo->storageAPI; - - pInfo->interval = (SInterval){.interval = pIntervalPhyNode->interval, - .sliding = pIntervalPhyNode->sliding, - .intervalUnit = pIntervalPhyNode->intervalUnit, - .slidingUnit = pIntervalPhyNode->slidingUnit, - .offset = pIntervalPhyNode->offset, - .precision = ((SColumnNode*)pIntervalPhyNode->window.pTspk)->node.resType.precision}; - pInfo->twAggSup = (STimeWindowAggSupp){ - .waterMark = pIntervalPhyNode->window.watermark, - .calTrigger = pIntervalPhyNode->window.triggerType, - .maxTs = INT64_MIN, - .minTs = INT64_MAX, - .deleteMark = getDeleteMark(pIntervalPhyNode), - .deleteMarkSaved = 0, - .calTriggerSaved = 0, - }; - ASSERTS(pInfo->twAggSup.calTrigger != STREAM_TRIGGER_MAX_DELAY, "trigger type should not be max delay"); - pInfo->primaryTsIndex = ((SColumnNode*)pIntervalPhyNode->window.pTspk)->slotId; - size_t keyBufSize = sizeof(int64_t) + sizeof(int64_t) + POINTER_BYTES; - initResultSizeInfo(&pOperator->resultInfo, 4096); - if (pIntervalPhyNode->window.pExprs != NULL) { - int32_t numOfScalar = 0; - SExprInfo* pScalarExprInfo = createExprInfo(pIntervalPhyNode->window.pExprs, NULL, &numOfScalar); - int32_t code = initExprSupp(&pInfo->scalarSupp, pScalarExprInfo, numOfScalar, &pTaskInfo->storageAPI.functionStore); - if (code != TSDB_CODE_SUCCESS) { - goto _error; - } - } - - int32_t numOfCols = 0; - SExprInfo* pExprInfo = createExprInfo(pIntervalPhyNode->window.pFuncs, NULL, &numOfCols); - SSDataBlock* pResBlock = createDataBlockFromDescNode(pPhyNode->pOutputDataBlockDesc); - initBasicInfo(&pInfo->binfo, pResBlock); - - pInfo->pState = taosMemoryCalloc(1, sizeof(SStreamState)); - *(pInfo->pState) = *(pTaskInfo->streamInfo.pState); - - pAPI->stateStore.streamStateSetNumber(pInfo->pState, -1); - int32_t code = initAggSup(&pOperator->exprSupp, &pInfo->aggSup, pExprInfo, numOfCols, keyBufSize, pTaskInfo->id.str, - pInfo->pState, &pTaskInfo->storageAPI.functionStore); - if (code != TSDB_CODE_SUCCESS) { - goto _error; - } - - initExecTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &pTaskInfo->window); - initResultRowInfo(&pInfo->binfo.resultRowInfo); - - pInfo->numOfChild = numOfChild; - - pInfo->pPhyNode = (SPhysiNode*)nodesCloneNode((SNode*)pPhyNode); - - if (pPhyNode->type == QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL) { - pInfo->isFinal = true; - pOperator->name = "StreamFinalIntervalOperator"; - } else { - // semi interval operator does not catch result - pInfo->isFinal = false; - pOperator->name = "StreamSemiIntervalOperator"; - } - - if (!IS_FINAL_OP(pInfo) || numOfChild == 0) { - pInfo->twAggSup.calTrigger = STREAM_TRIGGER_AT_ONCE; - } - - pInfo->pPullWins = taosArrayInit(8, sizeof(SPullWindowInfo)); - pInfo->pullIndex = 0; - _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); - pInfo->pPullDataMap = taosHashInit(64, hashFn, false, HASH_NO_LOCK); - pInfo->pFinalPullDataMap = taosHashInit(64, hashFn, false, HASH_NO_LOCK); - pInfo->pPullDataRes = createSpecialDataBlock(STREAM_RETRIEVE); - pInfo->ignoreExpiredData = pIntervalPhyNode->window.igExpired; - pInfo->ignoreExpiredDataSaved = false; - pInfo->pDelRes = createSpecialDataBlock(STREAM_DELETE_RESULT); - pInfo->delIndex = 0; - pInfo->pDelWins = taosArrayInit(4, sizeof(SWinKey)); - pInfo->delKey.ts = INT64_MAX; - pInfo->delKey.groupId = 0; - pInfo->numOfDatapack = 0; - pInfo->pUpdated = NULL; - pInfo->pUpdatedMap = NULL; - int32_t funResSize = getMaxFunResSize(&pOperator->exprSupp, numOfCols); - pInfo->pState->pFileState = pAPI->stateStore.streamFileStateInit( - tsStreamBufferSize, sizeof(SWinKey), pInfo->aggSup.resultRowSize, funResSize, compareTs, pInfo->pState, - pInfo->twAggSup.deleteMark, GET_TASKID(pTaskInfo), pHandle->checkpointId); - pInfo->dataVersion = 0; - pInfo->stateStore = pTaskInfo->storageAPI.stateStore; - pInfo->recvGetAll = false; - pInfo->pCheckpointRes = createSpecialDataBlock(STREAM_CHECKPOINT); - - pOperator->operatorType = pPhyNode->type; - pOperator->blocking = true; - pOperator->status = OP_NOT_OPENED; - pOperator->info = pInfo; - - pOperator->fpSet = createOperatorFpSet(NULL, doStreamFinalIntervalAgg, NULL, destroyStreamFinalIntervalOperatorInfo, - optrDefaultBufFn, NULL); - setOperatorStreamStateFn(pOperator, streamIntervalReleaseState, streamIntervalReloadState); - if (pPhyNode->type == QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL) { - initIntervalDownStream(downstream, pPhyNode->type, pInfo); - } - code = appendDownstream(pOperator, &downstream, 1); - if (code != TSDB_CODE_SUCCESS) { - goto _error; - } - - // for stream - void* buff = NULL; - int32_t len = 0; - int32_t res = pAPI->stateStore.streamStateGetInfo(pInfo->pState, STREAM_INTERVAL_OP_CHECKPOINT_NAME, - strlen(STREAM_INTERVAL_OP_CHECKPOINT_NAME), &buff, &len); - if (res == TSDB_CODE_SUCCESS) { - doStreamIntervalDecodeOpState(buff, len, pOperator); - taosMemoryFree(buff); - } - - return pOperator; - -_error: - destroyStreamFinalIntervalOperatorInfo(pInfo); - taosMemoryFreeClear(pOperator); - pTaskInfo->code = code; - return NULL; -} - -void destroyStreamAggSupporter(SStreamAggSupporter* pSup) { - tSimpleHashCleanup(pSup->pResultRows); - destroyDiskbasedBuf(pSup->pResultBuf); - blockDataDestroy(pSup->pScanBlock); - taosMemoryFreeClear(pSup->pState); - taosMemoryFreeClear(pSup->pDummyCtx); -} - -void destroyStreamSessionAggOperatorInfo(void* param) { - SStreamSessionAggOperatorInfo* pInfo = (SStreamSessionAggOperatorInfo*)param; - cleanupBasicInfo(&pInfo->binfo); - destroyStreamAggSupporter(&pInfo->streamAggSup); - - if (pInfo->pChildren != NULL) { - int32_t size = taosArrayGetSize(pInfo->pChildren); - for (int32_t i = 0; i < size; i++) { - SOperatorInfo* pChild = taosArrayGetP(pInfo->pChildren, i); - destroyOperator(pChild); - } - taosArrayDestroy(pInfo->pChildren); - } - - colDataDestroy(&pInfo->twAggSup.timeWindowData); - blockDataDestroy(pInfo->pDelRes); - blockDataDestroy(pInfo->pWinBlock); - tSimpleHashCleanup(pInfo->pStUpdated); - tSimpleHashCleanup(pInfo->pStDeleted); - - taosArrayDestroy(pInfo->historyWins); - blockDataDestroy(pInfo->pCheckpointRes); - - taosMemoryFreeClear(param); -} - -int32_t initBasicInfoEx(SOptrBasicInfo* pBasicInfo, SExprSupp* pSup, SExprInfo* pExprInfo, int32_t numOfCols, - SSDataBlock* pResultBlock, SFunctionStateStore* pStore) { - initBasicInfo(pBasicInfo, pResultBlock); - int32_t code = initExprSupp(pSup, pExprInfo, numOfCols, pStore); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - for (int32_t i = 0; i < numOfCols; ++i) { - pSup->pCtx[i].saveHandle.pBuf = NULL; - } - - ASSERT(numOfCols > 0); - return TSDB_CODE_SUCCESS; -} - -void initDummyFunction(SqlFunctionCtx* pDummy, SqlFunctionCtx* pCtx, int32_t nums) { - for (int i = 0; i < nums; i++) { - pDummy[i].functionId = pCtx[i].functionId; - pDummy[i].isNotNullFunc = pCtx[i].isNotNullFunc; - pDummy[i].isPseudoFunc = pCtx[i].isPseudoFunc; - } -} - -void initDownStream(SOperatorInfo* downstream, SStreamAggSupporter* pAggSup, uint16_t type, int32_t tsColIndex, - STimeWindowAggSupp* pTwSup) { - if (downstream->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_PARTITION) { - SStreamPartitionOperatorInfo* pScanInfo = downstream->info; - pScanInfo->tsColIndex = tsColIndex; - } - - if (downstream->operatorType != QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN) { - initDownStream(downstream->pDownstream[0], pAggSup, type, tsColIndex, pTwSup); - return; - } - SStreamScanInfo* pScanInfo = downstream->info; - pScanInfo->windowSup = (SWindowSupporter){.pStreamAggSup = pAggSup, .gap = pAggSup->gap, .parentType = type}; - pScanInfo->pState = pAggSup->pState; - if (!pScanInfo->pUpdateInfo) { - pScanInfo->pUpdateInfo = pAggSup->stateStore.updateInfoInit(60000, TSDB_TIME_PRECISION_MILLI, pTwSup->waterMark, - pScanInfo->igCheckUpdate); - } - pScanInfo->twAggSup = *pTwSup; -} - -int32_t initStreamAggSupporter(SStreamAggSupporter* pSup, SqlFunctionCtx* pCtx, int32_t numOfOutput, int64_t gap, - SStreamState* pState, int32_t keySize, int16_t keyType, SStateStore* pStore, - SReadHandle* pHandle, SStorageAPI* pApi) { - pSup->resultRowSize = keySize + getResultRowSize(pCtx, numOfOutput); - pSup->pScanBlock = createSpecialDataBlock(STREAM_CLEAR); - pSup->gap = gap; - pSup->stateKeySize = keySize; - pSup->stateKeyType = keyType; - pSup->pDummyCtx = (SqlFunctionCtx*)taosMemoryCalloc(numOfOutput, sizeof(SqlFunctionCtx)); - if (pSup->pDummyCtx == NULL) { - return TSDB_CODE_OUT_OF_MEMORY; - } - - pSup->stateStore = *pStore; - - initDummyFunction(pSup->pDummyCtx, pCtx, numOfOutput); - pSup->pState = taosMemoryCalloc(1, sizeof(SStreamState)); - *(pSup->pState) = *pState; - pSup->stateStore.streamStateSetNumber(pSup->pState, -1); - - _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); - pSup->pResultRows = tSimpleHashInit(32, hashFn); - - int32_t pageSize = 4096; - while (pageSize < pSup->resultRowSize * 4) { - pageSize <<= 1u; - } - // at least four pages need to be in buffer - int32_t bufSize = 4096 * 256; - if (bufSize <= pageSize) { - bufSize = pageSize * 4; - } - - if (!osTempSpaceAvailable()) { - terrno = TSDB_CODE_NO_DISKSPACE; - qError("Init stream agg supporter failed since %s, tempDir:%s", terrstr(), tsTempDir); - return terrno; - } - - int32_t code = createDiskbasedBuf(&pSup->pResultBuf, pageSize, bufSize, "function", tsTempDir); - for (int32_t i = 0; i < numOfOutput; ++i) { - pCtx[i].saveHandle.pBuf = pSup->pResultBuf; - } - - pSup->pSessionAPI = pApi; - - return TSDB_CODE_SUCCESS; -} - -bool isInTimeWindow(STimeWindow* pWin, TSKEY ts, int64_t gap) { - if (ts + gap >= pWin->skey && ts - gap <= pWin->ekey) { - return true; - } - return false; -} - -bool isInWindow(SResultWindowInfo* pWinInfo, TSKEY ts, int64_t gap) { - return isInTimeWindow(&pWinInfo->sessionWin.win, ts, gap); -} - -void getCurSessionWindow(SStreamAggSupporter* pAggSup, TSKEY startTs, TSKEY endTs, uint64_t groupId, - SSessionKey* pKey) { - pKey->win.skey = startTs; - pKey->win.ekey = endTs; - pKey->groupId = groupId; - int32_t code = pAggSup->stateStore.streamStateSessionGetKeyByRange(pAggSup->pState, pKey, pKey); - if (code != TSDB_CODE_SUCCESS) { - SET_SESSION_WIN_KEY_INVALID(pKey); - } -} - -bool isInvalidSessionWin(SResultWindowInfo* pWinInfo) { return pWinInfo->sessionWin.win.skey == 0; } - -bool inWinRange(STimeWindow* range, STimeWindow* cur) { - if (cur->skey >= range->skey && cur->ekey <= range->ekey) { - return true; - } - return false; -} - -void setSessionOutputBuf(SStreamAggSupporter* pAggSup, TSKEY startTs, TSKEY endTs, uint64_t groupId, - SResultWindowInfo* pCurWin) { - pCurWin->sessionWin.groupId = groupId; - pCurWin->sessionWin.win.skey = startTs; - pCurWin->sessionWin.win.ekey = endTs; - int32_t size = pAggSup->resultRowSize; - int32_t code = pAggSup->stateStore.streamStateSessionAddIfNotExist(pAggSup->pState, &pCurWin->sessionWin, - pAggSup->gap, &pCurWin->pOutputBuf, &size); - if (code == TSDB_CODE_SUCCESS && !inWinRange(&pAggSup->winRange, &pCurWin->sessionWin.win)) { - code = TSDB_CODE_FAILED; - releaseOutputBuf(pAggSup->pState, NULL, (SResultRow*)pCurWin->pOutputBuf, &pAggSup->pSessionAPI->stateStore); - pCurWin->pOutputBuf = taosMemoryCalloc(1, size); - } - - if (code == TSDB_CODE_SUCCESS) { - pCurWin->isOutput = true; - pAggSup->stateStore.streamStateSessionDel(pAggSup->pState, &pCurWin->sessionWin); - } else { - pCurWin->sessionWin.win.skey = startTs; - pCurWin->sessionWin.win.ekey = endTs; - } -} - -int32_t getSessionWinBuf(SStreamAggSupporter* pAggSup, SStreamStateCur* pCur, SResultWindowInfo* pWinInfo) { - int32_t size = 0; - int32_t code = - pAggSup->stateStore.streamStateSessionGetKVByCur(pCur, &pWinInfo->sessionWin, &pWinInfo->pOutputBuf, &size); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - pAggSup->stateStore.streamStateCurNext(pAggSup->pState, pCur); - return TSDB_CODE_SUCCESS; -} -void saveDeleteInfo(SArray* pWins, SSessionKey key) { - // key.win.ekey = key.win.skey; - taosArrayPush(pWins, &key); -} - -void saveDeleteRes(SSHashObj* pStDelete, SSessionKey key) { - key.win.ekey = key.win.skey; - tSimpleHashPut(pStDelete, &key, sizeof(SSessionKey), NULL, 0); -} - -static void removeSessionResult(SSHashObj* pHashMap, SSHashObj* pResMap, SSessionKey key) { - key.win.ekey = key.win.skey; - tSimpleHashRemove(pHashMap, &key, sizeof(SSessionKey)); - tSimpleHashRemove(pResMap, &key, sizeof(SSessionKey)); -} - -static void getSessionHashKey(const SSessionKey* pKey, SSessionKey* pHashKey) { - *pHashKey = *pKey; - pHashKey->win.ekey = pKey->win.skey; -} - -static void removeSessionResults(SSHashObj* pHashMap, SArray* pWins) { - if (tSimpleHashGetSize(pHashMap) == 0) { - return; - } - int32_t size = taosArrayGetSize(pWins); - for (int32_t i = 0; i < size; i++) { - SSessionKey* pWin = taosArrayGet(pWins, i); - if (!pWin) continue; - SSessionKey key = {0}; - getSessionHashKey(pWin, &key); - tSimpleHashRemove(pHashMap, &key, sizeof(SSessionKey)); - } -} - -int32_t updateSessionWindowInfo(SResultWindowInfo* pWinInfo, TSKEY* pStartTs, TSKEY* pEndTs, uint64_t groupId, - int32_t rows, int32_t start, int64_t gap, SSHashObj* pResultRows, SSHashObj* pStUpdated, - SSHashObj* pStDeleted) { - for (int32_t i = start; i < rows; ++i) { - if (!isInWindow(pWinInfo, pStartTs[i], gap) && (!pEndTs || !isInWindow(pWinInfo, pEndTs[i], gap))) { - return i - start; - } - if (pWinInfo->sessionWin.win.skey > pStartTs[i]) { - if (pStDeleted && pWinInfo->isOutput) { - saveDeleteRes(pStDeleted, pWinInfo->sessionWin); - } - removeSessionResult(pStUpdated, pResultRows, pWinInfo->sessionWin); - pWinInfo->sessionWin.win.skey = pStartTs[i]; - } - pWinInfo->sessionWin.win.ekey = TMAX(pWinInfo->sessionWin.win.ekey, pStartTs[i]); - if (pEndTs) { - pWinInfo->sessionWin.win.ekey = TMAX(pWinInfo->sessionWin.win.ekey, pEndTs[i]); - } - } - return rows - start; -} - -static int32_t initSessionOutputBuf(SResultWindowInfo* pWinInfo, SResultRow** pResult, SqlFunctionCtx* pCtx, - int32_t numOfOutput, int32_t* rowEntryInfoOffset) { - ASSERT(pWinInfo->sessionWin.win.skey <= pWinInfo->sessionWin.win.ekey); - *pResult = (SResultRow*)pWinInfo->pOutputBuf; - // set time window for current result - (*pResult)->win = pWinInfo->sessionWin.win; - setResultRowInitCtx(*pResult, pCtx, numOfOutput, rowEntryInfoOffset); - return TSDB_CODE_SUCCESS; -} - -static int32_t doOneWindowAggImpl(SColumnInfoData* pTimeWindowData, SResultWindowInfo* pCurWin, SResultRow** pResult, - int32_t startIndex, int32_t winRows, int32_t rows, int32_t numOutput, - SOperatorInfo* pOperator, int64_t winDelta) { - SExprSupp* pSup = &pOperator->exprSupp; - SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; - int32_t code = initSessionOutputBuf(pCurWin, pResult, pSup->pCtx, numOutput, pSup->rowEntryInfoOffset); - if (code != TSDB_CODE_SUCCESS || (*pResult) == NULL) { - return TSDB_CODE_OUT_OF_MEMORY; - } - updateTimeWindowInfo(pTimeWindowData, &pCurWin->sessionWin.win, winDelta); - applyAggFunctionOnPartialTuples(pTaskInfo, pSup->pCtx, pTimeWindowData, startIndex, winRows, rows, numOutput); - return TSDB_CODE_SUCCESS; -} - -static bool doDeleteSessionWindow(SStreamAggSupporter* pAggSup, SSessionKey* pKey) { - pAggSup->stateStore.streamStateSessionDel(pAggSup->pState, pKey); - SSessionKey hashKey = {0}; - getSessionHashKey(pKey, &hashKey); - tSimpleHashRemove(pAggSup->pResultRows, &hashKey, sizeof(SSessionKey)); - return true; -} - -static int32_t setSessionWinOutputInfo(SSHashObj* pStUpdated, SResultWindowInfo* pWinInfo) { - void* pVal = tSimpleHashGet(pStUpdated, &pWinInfo->sessionWin, sizeof(SSessionKey)); - if (pVal) { - SResultWindowInfo* pWin = pVal; - pWinInfo->isOutput = pWin->isOutput; - } - return TSDB_CODE_SUCCESS; -} - -SStreamStateCur* getNextSessionWinInfo(SStreamAggSupporter* pAggSup, SSHashObj* pStUpdated, SResultWindowInfo* pCurWin, - SResultWindowInfo* pNextWin) { - SStreamStateCur* pCur = pAggSup->stateStore.streamStateSessionSeekKeyNext(pAggSup->pState, &pCurWin->sessionWin); - pNextWin->isOutput = true; - setSessionWinOutputInfo(pStUpdated, pNextWin); - int32_t size = 0; - pNextWin->sessionWin = pCurWin->sessionWin; - int32_t code = - pAggSup->stateStore.streamStateSessionGetKVByCur(pCur, &pNextWin->sessionWin, &pNextWin->pOutputBuf, &size); - if (code != TSDB_CODE_SUCCESS) { - taosMemoryFreeClear(pNextWin->pOutputBuf); - SET_SESSION_WIN_INVALID(*pNextWin); - } - return pCur; -} - -static int32_t compactSessionWindow(SOperatorInfo* pOperator, SResultWindowInfo* pCurWin, SSHashObj* pStUpdated, - SSHashObj* pStDeleted, bool addGap) { - SExprSupp* pSup = &pOperator->exprSupp; - SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; - SStorageAPI* pAPI = &pOperator->pTaskInfo->storageAPI; - int32_t winNum = 0; - - SStreamSessionAggOperatorInfo* pInfo = pOperator->info; - SResultRow* pCurResult = NULL; - int32_t numOfOutput = pOperator->exprSupp.numOfExprs; - SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; - initSessionOutputBuf(pCurWin, &pCurResult, pSup->pCtx, numOfOutput, pSup->rowEntryInfoOffset); - // Just look for the window behind StartIndex - while (1) { - SResultWindowInfo winInfo = {0}; - SStreamStateCur* pCur = getNextSessionWinInfo(pAggSup, pStUpdated, pCurWin, &winInfo); - if (!IS_VALID_SESSION_WIN(winInfo) || !isInWindow(pCurWin, winInfo.sessionWin.win.skey, pAggSup->gap) || - !inWinRange(&pAggSup->winRange, &winInfo.sessionWin.win)) { - taosMemoryFree(winInfo.pOutputBuf); - pAPI->stateStore.streamStateFreeCur(pCur); - break; - } - SResultRow* pWinResult = NULL; - initSessionOutputBuf(&winInfo, &pWinResult, pAggSup->pDummyCtx, numOfOutput, pSup->rowEntryInfoOffset); - pCurWin->sessionWin.win.ekey = TMAX(pCurWin->sessionWin.win.ekey, winInfo.sessionWin.win.ekey); - int64_t winDelta = 0; - if (addGap) { - winDelta = pAggSup->gap; - } - updateTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &pCurWin->sessionWin.win, winDelta); - compactFunctions(pSup->pCtx, pAggSup->pDummyCtx, numOfOutput, pTaskInfo, &pInfo->twAggSup.timeWindowData); - tSimpleHashRemove(pStUpdated, &winInfo.sessionWin, sizeof(SSessionKey)); - if (winInfo.isOutput && pStDeleted) { - saveDeleteRes(pStDeleted, winInfo.sessionWin); - } - removeSessionResult(pStUpdated, pAggSup->pResultRows, winInfo.sessionWin); - doDeleteSessionWindow(pAggSup, &winInfo.sessionWin); - pAPI->stateStore.streamStateFreeCur(pCur); - taosMemoryFree(winInfo.pOutputBuf); - winNum++; - } - return winNum; -} - -int32_t saveSessionOutputBuf(SStreamAggSupporter* pAggSup, SResultWindowInfo* pWinInfo) { - saveSessionDiscBuf(pAggSup->pState, &pWinInfo->sessionWin, pWinInfo->pOutputBuf, pAggSup->resultRowSize, - &pAggSup->stateStore); - pWinInfo->pOutputBuf = NULL; - return TSDB_CODE_SUCCESS; -} - -static void doStreamSessionAggImpl(SOperatorInfo* pOperator, SSDataBlock* pSDataBlock, SSHashObj* pStUpdated, - SSHashObj* pStDeleted, bool hasEndTs, bool addGap) { - SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; - SStreamSessionAggOperatorInfo* pInfo = pOperator->info; - int32_t numOfOutput = pOperator->exprSupp.numOfExprs; - uint64_t groupId = pSDataBlock->info.id.groupId; - int64_t code = TSDB_CODE_SUCCESS; - SResultRow* pResult = NULL; - int32_t rows = pSDataBlock->info.rows; - int32_t winRows = 0; - SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; - - pInfo->dataVersion = TMAX(pInfo->dataVersion, pSDataBlock->info.version); - pAggSup->winRange = pTaskInfo->streamInfo.fillHistoryWindow; - if (pAggSup->winRange.ekey <= 0) { - pAggSup->winRange.ekey = INT64_MAX; - } - - SColumnInfoData* pStartTsCol = taosArrayGet(pSDataBlock->pDataBlock, pInfo->primaryTsIndex); - TSKEY* startTsCols = (int64_t*)pStartTsCol->pData; - SColumnInfoData* pEndTsCol = NULL; - if (hasEndTs) { - pEndTsCol = taosArrayGet(pSDataBlock->pDataBlock, pInfo->endTsIndex); - } else { - pEndTsCol = taosArrayGet(pSDataBlock->pDataBlock, pInfo->primaryTsIndex); - } - - TSKEY* endTsCols = (int64_t*)pEndTsCol->pData; - for (int32_t i = 0; i < rows;) { - if (pInfo->ignoreExpiredData && isOverdue(endTsCols[i], &pInfo->twAggSup)) { - i++; - continue; - } - SResultWindowInfo winInfo = {0}; - setSessionOutputBuf(pAggSup, startTsCols[i], endTsCols[i], groupId, &winInfo); - setSessionWinOutputInfo(pStUpdated, &winInfo); - winRows = updateSessionWindowInfo(&winInfo, startTsCols, endTsCols, groupId, rows, i, pAggSup->gap, - pAggSup->pResultRows, pStUpdated, pStDeleted); - // coverity scan error - if (!winInfo.pOutputBuf) { - T_LONG_JMP(pTaskInfo->env, TSDB_CODE_OUT_OF_MEMORY); - } - - int64_t winDelta = 0; - if (addGap) { - winDelta = pAggSup->gap; - } - code = doOneWindowAggImpl(&pInfo->twAggSup.timeWindowData, &winInfo, &pResult, i, winRows, rows, numOfOutput, - pOperator, winDelta); - if (code != TSDB_CODE_SUCCESS || pResult == NULL) { - T_LONG_JMP(pTaskInfo->env, TSDB_CODE_OUT_OF_MEMORY); - } - compactSessionWindow(pOperator, &winInfo, pStUpdated, pStDeleted, addGap); - saveSessionOutputBuf(pAggSup, &winInfo); - - if (pInfo->twAggSup.calTrigger == STREAM_TRIGGER_AT_ONCE && pStUpdated) { - code = saveResult(winInfo, pStUpdated); - if (code != TSDB_CODE_SUCCESS) { - T_LONG_JMP(pTaskInfo->env, TSDB_CODE_OUT_OF_MEMORY); - } - } - if (pInfo->twAggSup.calTrigger == STREAM_TRIGGER_WINDOW_CLOSE) { - SSessionKey key = {0}; - getSessionHashKey(&winInfo.sessionWin, &key); - tSimpleHashPut(pAggSup->pResultRows, &key, sizeof(SSessionKey), &winInfo, sizeof(SResultWindowInfo)); - } - - i += winRows; - } -} - -static void doDeleteTimeWindows(SStreamAggSupporter* pAggSup, SSDataBlock* pBlock, SArray* result) { - SColumnInfoData* pStartTsCol = taosArrayGet(pBlock->pDataBlock, START_TS_COLUMN_INDEX); - TSKEY* startDatas = (TSKEY*)pStartTsCol->pData; - SColumnInfoData* pEndTsCol = taosArrayGet(pBlock->pDataBlock, END_TS_COLUMN_INDEX); - TSKEY* endDatas = (TSKEY*)pEndTsCol->pData; - SColumnInfoData* pGroupCol = taosArrayGet(pBlock->pDataBlock, GROUPID_COLUMN_INDEX); - uint64_t* gpDatas = (uint64_t*)pGroupCol->pData; - for (int32_t i = 0; i < pBlock->info.rows; i++) { - while (1) { - SSessionKey curWin = {0}; - getCurSessionWindow(pAggSup, startDatas[i], endDatas[i], gpDatas[i], &curWin); - if (IS_INVALID_SESSION_WIN_KEY(curWin)) { - break; - } - doDeleteSessionWindow(pAggSup, &curWin); - if (result) { - saveDeleteInfo(result, curWin); - } - } - } -} - -static inline int32_t sessionKeyCompareAsc(const void* pKey1, const void* pKey2) { - SSessionKey* pWin1 = (SSessionKey*)pKey1; - SSessionKey* pWin2 = (SSessionKey*)pKey2; - - if (pWin1->groupId > pWin2->groupId) { - return 1; - } else if (pWin1->groupId < pWin2->groupId) { - return -1; - } - - if (pWin1->win.skey > pWin2->win.skey) { - return 1; - } else if (pWin1->win.skey < pWin2->win.skey) { - return -1; - } - - return 0; -} - -static int32_t copyUpdateResult(SSHashObj* pStUpdated, SArray* pUpdated) { - void* pIte = NULL; - int32_t iter = 0; - while ((pIte = tSimpleHashIterate(pStUpdated, pIte, &iter)) != NULL) { - void* key = tSimpleHashGetKey(pIte, NULL); - taosArrayPush(pUpdated, key); - } - taosArraySort(pUpdated, sessionKeyCompareAsc); - return TSDB_CODE_SUCCESS; -} - -void doBuildDeleteDataBlock(SOperatorInfo* pOp, SSHashObj* pStDeleted, SSDataBlock* pBlock, void** Ite) { - SStorageAPI* pAPI = &pOp->pTaskInfo->storageAPI; - - blockDataCleanup(pBlock); - int32_t size = tSimpleHashGetSize(pStDeleted); - if (size == 0) { - return; - } - blockDataEnsureCapacity(pBlock, size); - int32_t iter = 0; - while (((*Ite) = tSimpleHashIterate(pStDeleted, *Ite, &iter)) != NULL) { - if (pBlock->info.rows + 1 > pBlock->info.capacity) { - break; - } - SSessionKey* res = tSimpleHashGetKey(*Ite, NULL); - SColumnInfoData* pStartTsCol = taosArrayGet(pBlock->pDataBlock, START_TS_COLUMN_INDEX); - colDataSetVal(pStartTsCol, pBlock->info.rows, (const char*)&res->win.skey, false); - SColumnInfoData* pEndTsCol = taosArrayGet(pBlock->pDataBlock, END_TS_COLUMN_INDEX); - colDataSetVal(pEndTsCol, pBlock->info.rows, (const char*)&res->win.skey, false); - SColumnInfoData* pUidCol = taosArrayGet(pBlock->pDataBlock, UID_COLUMN_INDEX); - colDataSetNULL(pUidCol, pBlock->info.rows); - SColumnInfoData* pGpCol = taosArrayGet(pBlock->pDataBlock, GROUPID_COLUMN_INDEX); - colDataSetVal(pGpCol, pBlock->info.rows, (const char*)&res->groupId, false); - SColumnInfoData* pCalStCol = taosArrayGet(pBlock->pDataBlock, CALCULATE_START_TS_COLUMN_INDEX); - colDataSetNULL(pCalStCol, pBlock->info.rows); - SColumnInfoData* pCalEdCol = taosArrayGet(pBlock->pDataBlock, CALCULATE_END_TS_COLUMN_INDEX); - colDataSetNULL(pCalEdCol, pBlock->info.rows); - - SColumnInfoData* pTableCol = taosArrayGet(pBlock->pDataBlock, TABLE_NAME_COLUMN_INDEX); - - void* tbname = NULL; - pAPI->stateStore.streamStateGetParName(pOp->pTaskInfo->streamInfo.pState, res->groupId, &tbname); - if (tbname == NULL) { - colDataSetNULL(pTableCol, pBlock->info.rows); - } else { - char parTbName[VARSTR_HEADER_SIZE + TSDB_TABLE_NAME_LEN]; - STR_WITH_MAXSIZE_TO_VARSTR(parTbName, tbname, sizeof(parTbName)); - colDataSetVal(pTableCol, pBlock->info.rows, (const char*)parTbName, false); - pAPI->stateStore.streamStateFreeVal(tbname); - } - pBlock->info.rows += 1; - } - if ((*Ite) == NULL) { - tSimpleHashClear(pStDeleted); - } -} - -static void rebuildSessionWindow(SOperatorInfo* pOperator, SArray* pWinArray, SSHashObj* pStUpdated) { - SExprSupp* pSup = &pOperator->exprSupp; - SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; - SStorageAPI* pAPI = &pOperator->pTaskInfo->storageAPI; - - int32_t size = taosArrayGetSize(pWinArray); - SStreamSessionAggOperatorInfo* pInfo = pOperator->info; - SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; - int32_t numOfOutput = pSup->numOfExprs; - int32_t numOfChild = taosArrayGetSize(pInfo->pChildren); - - for (int32_t i = 0; i < size; i++) { - SSessionKey* pWinKey = taosArrayGet(pWinArray, i); - int32_t num = 0; - SResultWindowInfo parentWin = {0}; - for (int32_t j = 0; j < numOfChild; j++) { - SOperatorInfo* pChild = taosArrayGetP(pInfo->pChildren, j); - SStreamSessionAggOperatorInfo* pChInfo = pChild->info; - SStreamAggSupporter* pChAggSup = &pChInfo->streamAggSup; - SSessionKey chWinKey = {0}; - getSessionHashKey(pWinKey, &chWinKey); - SStreamStateCur* pCur = pAggSup->stateStore.streamStateSessionSeekKeyCurrentNext(pChAggSup->pState, &chWinKey); - SResultRow* pResult = NULL; - SResultRow* pChResult = NULL; - while (1) { - SResultWindowInfo childWin = {0}; - childWin.sessionWin = *pWinKey; - int32_t code = getSessionWinBuf(pChAggSup, pCur, &childWin); - - if (code == TSDB_CODE_SUCCESS && !inWinRange(&pAggSup->winRange, &childWin.sessionWin.win)) { - continue; - } - - if (code == TSDB_CODE_SUCCESS && inWinRange(&pWinKey->win, &childWin.sessionWin.win)) { - if (num == 0) { - setSessionOutputBuf(pAggSup, pWinKey->win.skey, pWinKey->win.ekey, pWinKey->groupId, &parentWin); - code = initSessionOutputBuf(&parentWin, &pResult, pSup->pCtx, numOfOutput, pSup->rowEntryInfoOffset); - if (code != TSDB_CODE_SUCCESS || pResult == NULL) { - break; - } - } - num++; - updateTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &parentWin.sessionWin.win, pAggSup->gap); - initSessionOutputBuf(&childWin, &pChResult, pChild->exprSupp.pCtx, numOfOutput, - pChild->exprSupp.rowEntryInfoOffset); - compactFunctions(pSup->pCtx, pChild->exprSupp.pCtx, numOfOutput, pTaskInfo, &pInfo->twAggSup.timeWindowData); - compactSessionWindow(pOperator, &parentWin, pStUpdated, NULL, true); - saveResult(parentWin, pStUpdated); - } else { - break; - } - } - pAPI->stateStore.streamStateFreeCur(pCur); - } - if (num > 0) { - saveSessionOutputBuf(pAggSup, &parentWin); - } - } -} - -int32_t closeSessionWindow(SSHashObj* pHashMap, STimeWindowAggSupp* pTwSup, SSHashObj* pClosed) { - void* pIte = NULL; - int32_t iter = 0; - while ((pIte = tSimpleHashIterate(pHashMap, pIte, &iter)) != NULL) { - SResultWindowInfo* pWinInfo = pIte; - if (isCloseWindow(&pWinInfo->sessionWin.win, pTwSup)) { - if (pTwSup->calTrigger == STREAM_TRIGGER_WINDOW_CLOSE && pClosed) { - int32_t code = saveResult(*pWinInfo, pClosed); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - } - SSessionKey* pKey = tSimpleHashGetKey(pIte, NULL); - tSimpleHashIterateRemove(pHashMap, pKey, sizeof(SSessionKey), &pIte, &iter); - } - } - return TSDB_CODE_SUCCESS; -} - -static void closeChildSessionWindow(SArray* pChildren, TSKEY maxTs) { - int32_t size = taosArrayGetSize(pChildren); - for (int32_t i = 0; i < size; i++) { - SOperatorInfo* pChildOp = taosArrayGetP(pChildren, i); - SStreamSessionAggOperatorInfo* pChInfo = pChildOp->info; - pChInfo->twAggSup.maxTs = TMAX(pChInfo->twAggSup.maxTs, maxTs); - closeSessionWindow(pChInfo->streamAggSup.pResultRows, &pChInfo->twAggSup, NULL); - } -} - -int32_t getAllSessionWindow(SSHashObj* pHashMap, SSHashObj* pStUpdated) { - void* pIte = NULL; - int32_t iter = 0; - while ((pIte = tSimpleHashIterate(pHashMap, pIte, &iter)) != NULL) { - SResultWindowInfo* pWinInfo = pIte; - saveResult(*pWinInfo, pStUpdated); - } - return TSDB_CODE_SUCCESS; -} - -static void copyDeleteWindowInfo(SArray* pResWins, SSHashObj* pStDeleted) { - int32_t size = taosArrayGetSize(pResWins); - for (int32_t i = 0; i < size; i++) { - SSessionKey* pWinKey = taosArrayGet(pResWins, i); - if (!pWinKey) continue; - SSessionKey winInfo = {0}; - getSessionHashKey(pWinKey, &winInfo); - tSimpleHashPut(pStDeleted, &winInfo, sizeof(SSessionKey), NULL, 0); - } -} - -// the allocated memory comes from outer function. -void initGroupResInfoFromArrayList(SGroupResInfo* pGroupResInfo, SArray* pArrayList) { - pGroupResInfo->pRows = pArrayList; - pGroupResInfo->index = 0; - pGroupResInfo->pBuf = NULL; -} - -void doBuildSessionResult(SOperatorInfo* pOperator, void* pState, SGroupResInfo* pGroupResInfo, SSDataBlock* pBlock) { - SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; - // set output datablock version - pBlock->info.version = pTaskInfo->version; - - blockDataCleanup(pBlock); - if (!hasRemainResults(pGroupResInfo)) { - cleanupGroupResInfo(pGroupResInfo); - return; - } - - // clear the existed group id - pBlock->info.id.groupId = 0; - buildSessionResultDataBlock(pOperator, pState, pBlock, &pOperator->exprSupp, pGroupResInfo); - if (pBlock->info.rows == 0) { - cleanupGroupResInfo(pGroupResInfo); - } -} -void getMaxTsWins(const SArray* pAllWins, SArray* pMaxWins) { - int32_t size = taosArrayGetSize(pAllWins); - if (size == 0) { - return; - } - - SSessionKey* pSeKey = taosArrayGet(pAllWins, size - 1); - taosArrayPush(pMaxWins, pSeKey); - if (pSeKey->groupId == 0) { - return; - } - uint64_t preGpId = pSeKey->groupId; - for (int32_t i = size - 2; i >= 0; i--) { - pSeKey = taosArrayGet(pAllWins, i); - if (preGpId != pSeKey->groupId) { - taosArrayPush(pMaxWins, pSeKey); - preGpId = pSeKey->groupId; - } - } -} - -int32_t encodeSSessionKey(void** buf, SSessionKey* key) { - int32_t tlen = 0; - tlen += encodeSTimeWindow(buf, &key->win); - tlen += taosEncodeFixedU64(buf, key->groupId); - return tlen; -} - -void* decodeSSessionKey(void* buf, SSessionKey* key) { - buf = decodeSTimeWindow(buf, &key->win); - buf = taosDecodeFixedU64(buf, &key->groupId); - return buf; -} - -int32_t encodeSResultWindowInfo(void** buf, SResultWindowInfo* key, int32_t outLen) { - int32_t tlen = 0; - tlen += taosEncodeFixedBool(buf, key->isOutput); - tlen += encodeSSessionKey(buf, &key->sessionWin); - return tlen; -} - -void* decodeSResultWindowInfo(void* buf, SResultWindowInfo* key, int32_t outLen) { - buf = taosDecodeFixedBool(buf, &key->isOutput); - key->pOutputBuf = NULL; - buf = decodeSSessionKey(buf, &key->sessionWin); - return buf; -} - -int32_t doStreamSessionEncodeOpState(void** buf, int32_t len, SOperatorInfo* pOperator, bool isParent) { - SStreamSessionAggOperatorInfo* pInfo = pOperator->info; - if (!pInfo) { - return 0; - } - - void* pData = (buf == NULL) ? NULL : *buf; - - // 1.streamAggSup.pResultRows - int32_t tlen = 0; - int32_t mapSize = tSimpleHashGetSize(pInfo->streamAggSup.pResultRows); - tlen += taosEncodeFixedI32(buf, mapSize); - void* pIte = NULL; - size_t keyLen = 0; - int32_t iter = 0; - while ((pIte = tSimpleHashIterate(pInfo->streamAggSup.pResultRows, pIte, &iter)) != NULL) { - void* key = taosHashGetKey(pIte, &keyLen); - tlen += encodeSSessionKey(buf, key); - tlen += encodeSResultWindowInfo(buf, pIte, pInfo->streamAggSup.resultRowSize); - } - - // 2.twAggSup - tlen += encodeSTimeWindowAggSupp(buf, &pInfo->twAggSup); - - // 3.pChildren - int32_t size = taosArrayGetSize(pInfo->pChildren); - tlen += taosEncodeFixedI32(buf, size); - for (int32_t i = 0; i < size; i++) { - SOperatorInfo* pChOp = taosArrayGetP(pInfo->pChildren, i); - tlen += doStreamSessionEncodeOpState(buf, 0, pChOp, false); - } - - // 4.dataVersion - tlen += taosEncodeFixedI32(buf, pInfo->dataVersion); - - // 5.checksum - if (isParent) { - if (buf) { - uint32_t cksum = taosCalcChecksum(0, pData, len - sizeof(uint32_t)); - tlen += taosEncodeFixedU32(buf, cksum); - } else { - tlen += sizeof(uint32_t); - } - } - - return tlen; -} - -void* doStreamSessionDecodeOpState(void* buf, int32_t len, SOperatorInfo* pOperator, bool isParent) { - SStreamSessionAggOperatorInfo* pInfo = pOperator->info; - if (!pInfo) { - return buf; - } - - // 5.checksum - if (isParent) { - int32_t dataLen = len - sizeof(uint32_t); - void* pCksum = POINTER_SHIFT(buf, dataLen); - if (taosCheckChecksum(buf, dataLen, *(uint32_t*)pCksum) != TSDB_CODE_SUCCESS) { - ASSERT(0); // debug - qError("stream interval state is invalid"); - return buf; - } - } - - // 1.streamAggSup.pResultRows - int32_t mapSize = 0; - buf = taosDecodeFixedI32(buf, &mapSize); - for (int32_t i = 0; i < mapSize; i++) { - SSessionKey key = {0}; - SResultWindowInfo winfo = {0}; - buf = decodeSSessionKey(buf, &key); - buf = decodeSResultWindowInfo(buf, &winfo, pInfo->streamAggSup.resultRowSize); - tSimpleHashPut(pInfo->streamAggSup.pResultRows, &key, sizeof(SSessionKey), &winfo, sizeof(SResultWindowInfo)); - } - - // 2.twAggSup - buf = decodeSTimeWindowAggSupp(buf, &pInfo->twAggSup); - - // 3.pChildren - int32_t size = 0; - buf = taosDecodeFixedI32(buf, &size); - ASSERT(size <= taosArrayGetSize(pInfo->pChildren)); - for (int32_t i = 0; i < size; i++) { - SOperatorInfo* pChOp = taosArrayGetP(pInfo->pChildren, i); - buf = doStreamSessionDecodeOpState(buf, 0, pChOp, false); - } - - // 4.dataVersion - buf = taosDecodeFixedI64(buf, &pInfo->dataVersion); - return buf; -} - -void doStreamSessionSaveCheckpoint(SOperatorInfo* pOperator) { - SStreamSessionAggOperatorInfo* pInfo = pOperator->info; - int32_t len = doStreamSessionEncodeOpState(NULL, 0, pOperator, true); - void* buf = taosMemoryCalloc(1, len); - void* pBuf = buf; - len = doStreamSessionEncodeOpState(&pBuf, len, pOperator, true); - pInfo->streamAggSup.stateStore.streamStateSaveInfo(pInfo->streamAggSup.pState, STREAM_SESSION_OP_CHECKPOINT_NAME, - strlen(STREAM_SESSION_OP_CHECKPOINT_NAME), buf, len); - taosMemoryFree(buf); -} - -static SSDataBlock* doStreamSessionAgg(SOperatorInfo* pOperator) { - SExprSupp* pSup = &pOperator->exprSupp; - SStreamSessionAggOperatorInfo* pInfo = pOperator->info; - SOptrBasicInfo* pBInfo = &pInfo->binfo; - SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; - qDebug("===stream=== stream session agg"); - if (pOperator->status == OP_EXEC_DONE) { - return NULL; - } else if (pOperator->status == OP_RES_TO_RETURN) { - doBuildDeleteDataBlock(pOperator, pInfo->pStDeleted, pInfo->pDelRes, &pInfo->pDelIterator); - if (pInfo->pDelRes->info.rows > 0) { - printDataBlock(pInfo->pDelRes, IS_FINAL_OP(pInfo) ? "final session" : "single session"); - return pInfo->pDelRes; - } - doBuildSessionResult(pOperator, pAggSup->pState, &pInfo->groupResInfo, pBInfo->pRes); - if (pBInfo->pRes->info.rows > 0) { - printDataBlock(pBInfo->pRes, IS_FINAL_OP(pInfo) ? "final session" : "single session"); - return pBInfo->pRes; - } - - if (pInfo->reCkBlock) { - pInfo->reCkBlock = false; - printDataBlock(pInfo->pCheckpointRes, IS_FINAL_OP(pInfo) ? "final session ck" : "single session ck"); - return pInfo->pCheckpointRes; - } - - setOperatorCompleted(pOperator); - return NULL; - } - - SOperatorInfo* downstream = pOperator->pDownstream[0]; - if (!pInfo->pUpdated) { - pInfo->pUpdated = taosArrayInit(16, sizeof(SSessionKey)); - } - if (!pInfo->pStUpdated) { - _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); - pInfo->pStUpdated = tSimpleHashInit(64, hashFn); - } - while (1) { - SSDataBlock* pBlock = downstream->fpSet.getNextFn(downstream); - if (pBlock == NULL) { - break; - } - printDataBlock(pBlock, IS_FINAL_OP(pInfo) ? "final session recv" : "single session recv"); - - if (pBlock->info.type == STREAM_DELETE_DATA || pBlock->info.type == STREAM_DELETE_RESULT || - pBlock->info.type == STREAM_CLEAR) { - SArray* pWins = taosArrayInit(16, sizeof(SSessionKey)); - // gap must be 0 - doDeleteTimeWindows(pAggSup, pBlock, pWins); - removeSessionResults(pInfo->pStUpdated, pWins); - if (IS_FINAL_OP(pInfo)) { - int32_t childIndex = getChildIndex(pBlock); - SOperatorInfo* pChildOp = taosArrayGetP(pInfo->pChildren, childIndex); - SStreamSessionAggOperatorInfo* pChildInfo = pChildOp->info; - // gap must be 0 - doDeleteTimeWindows(&pChildInfo->streamAggSup, pBlock, NULL); - rebuildSessionWindow(pOperator, pWins, pInfo->pStUpdated); - } - copyDeleteWindowInfo(pWins, pInfo->pStDeleted); - taosArrayDestroy(pWins); - continue; - } else if (pBlock->info.type == STREAM_GET_ALL) { - getAllSessionWindow(pAggSup->pResultRows, pInfo->pStUpdated); - continue; - } else if (pBlock->info.type == STREAM_CREATE_CHILD_TABLE) { - return pBlock; - } else if (pBlock->info.type == STREAM_CHECKPOINT) { - pAggSup->stateStore.streamStateCommit(pAggSup->pState); - doStreamSessionSaveCheckpoint(pOperator); - copyDataBlock(pInfo->pCheckpointRes, pBlock); - continue; - } else { - ASSERTS(pBlock->info.type == STREAM_NORMAL || pBlock->info.type == STREAM_INVALID, "invalid SSDataBlock type"); - } - - if (pInfo->scalarSupp.pExprInfo != NULL) { - SExprSupp* pExprSup = &pInfo->scalarSupp; - projectApplyFunctions(pExprSup->pExprInfo, pBlock, pBlock, pExprSup->pCtx, pExprSup->numOfExprs, NULL); - } - // the pDataBlock are always the same one, no need to call this again - setInputDataBlock(pSup, pBlock, TSDB_ORDER_ASC, MAIN_SCAN, true); - doStreamSessionAggImpl(pOperator, pBlock, pInfo->pStUpdated, pInfo->pStDeleted, IS_FINAL_OP(pInfo), true); - if (IS_FINAL_OP(pInfo)) { - int32_t chIndex = getChildIndex(pBlock); - int32_t size = taosArrayGetSize(pInfo->pChildren); - // if chIndex + 1 - size > 0, add new child - for (int32_t i = 0; i < chIndex + 1 - size; i++) { - SOperatorInfo* pChildOp = - createStreamFinalSessionAggOperatorInfo(NULL, pInfo->pPhyNode, pOperator->pTaskInfo, 0, NULL); - if (!pChildOp) { - T_LONG_JMP(pOperator->pTaskInfo->env, TSDB_CODE_OUT_OF_MEMORY); - } - taosArrayPush(pInfo->pChildren, &pChildOp); - } - SOperatorInfo* pChildOp = taosArrayGetP(pInfo->pChildren, chIndex); - setInputDataBlock(&pChildOp->exprSupp, pBlock, TSDB_ORDER_ASC, MAIN_SCAN, true); - doStreamSessionAggImpl(pChildOp, pBlock, NULL, NULL, true, false); - } - pInfo->twAggSup.maxTs = TMAX(pInfo->twAggSup.maxTs, pBlock->info.window.ekey); - pInfo->twAggSup.maxTs = TMAX(pInfo->twAggSup.maxTs, pBlock->info.watermark); - } - // restore the value - pOperator->status = OP_RES_TO_RETURN; - - closeSessionWindow(pAggSup->pResultRows, &pInfo->twAggSup, pInfo->pStUpdated); - closeChildSessionWindow(pInfo->pChildren, pInfo->twAggSup.maxTs); - copyUpdateResult(pInfo->pStUpdated, pInfo->pUpdated); - removeSessionResults(pInfo->pStDeleted, pInfo->pUpdated); - tSimpleHashCleanup(pInfo->pStUpdated); - pInfo->pStUpdated = NULL; - if (pInfo->isHistoryOp) { - getMaxTsWins(pInfo->pUpdated, pInfo->historyWins); - } - initGroupResInfoFromArrayList(&pInfo->groupResInfo, pInfo->pUpdated); - pInfo->pUpdated = NULL; - blockDataEnsureCapacity(pInfo->binfo.pRes, pOperator->resultInfo.capacity); - -#if 0 - char* pBuf = streamStateSessionDump(pAggSup->pState); - qDebug("===stream===final session%s", pBuf); - taosMemoryFree(pBuf); -#endif - - doBuildDeleteDataBlock(pOperator, pInfo->pStDeleted, pInfo->pDelRes, &pInfo->pDelIterator); - if (pInfo->pDelRes->info.rows > 0) { - printDataBlock(pInfo->pDelRes, IS_FINAL_OP(pInfo) ? "final session" : "single session"); - return pInfo->pDelRes; - } - - doBuildSessionResult(pOperator, pAggSup->pState, &pInfo->groupResInfo, pBInfo->pRes); - if (pBInfo->pRes->info.rows > 0) { - printDataBlock(pBInfo->pRes, IS_FINAL_OP(pInfo) ? "final session" : "single session"); - return pBInfo->pRes; - } - - if (pInfo->reCkBlock) { - pInfo->reCkBlock = false; - printDataBlock(pInfo->pCheckpointRes, IS_FINAL_OP(pInfo) ? "final session ck" : "single session ck"); - return pInfo->pCheckpointRes; - } - - setOperatorCompleted(pOperator); - return NULL; -} - -void streamSessionReleaseState(SOperatorInfo* pOperator) { - if (pOperator->operatorType != QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_SESSION) { - SStreamSessionAggOperatorInfo* pInfo = pOperator->info; - int32_t resSize = taosArrayGetSize(pInfo->historyWins) * sizeof(SSessionKey); - pInfo->streamAggSup.stateStore.streamStateSaveInfo(pInfo->streamAggSup.pState, STREAM_SESSION_OP_STATE_NAME, - strlen(STREAM_SESSION_OP_STATE_NAME), pInfo->historyWins->pData, - resSize); - } - SOperatorInfo* downstream = pOperator->pDownstream[0]; - if (downstream->fpSet.releaseStreamStateFn) { - downstream->fpSet.releaseStreamStateFn(downstream); - } -} - -void resetWinRange(STimeWindow* winRange) { - winRange->skey = INT64_MIN; - winRange->ekey = INT64_MAX; -} - -void streamSessionReloadState(SOperatorInfo* pOperator) { - SStreamSessionAggOperatorInfo* pInfo = pOperator->info; - SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; - resetWinRange(&pAggSup->winRange); - - SResultWindowInfo winInfo = {0}; - int32_t size = 0; - void* pBuf = NULL; - int32_t code = pAggSup->stateStore.streamStateGetInfo(pAggSup->pState, STREAM_SESSION_OP_STATE_NAME, - strlen(STREAM_SESSION_OP_STATE_NAME), &pBuf, &size); - int32_t num = size / sizeof(SSessionKey); - SSessionKey* pSeKeyBuf = (SSessionKey*)pBuf; - ASSERT(size == num * sizeof(SSessionKey)); - if (!pInfo->pStUpdated && num > 0) { - _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); - pInfo->pStUpdated = tSimpleHashInit(64, hashFn); - } - for (int32_t i = 0; i < num; i++) { - SResultWindowInfo winInfo = {0}; - setSessionOutputBuf(pAggSup, pSeKeyBuf[i].win.skey, pSeKeyBuf[i].win.ekey, pSeKeyBuf[i].groupId, &winInfo); - int32_t winNum = compactSessionWindow(pOperator, &winInfo, pInfo->pStUpdated, pInfo->pStDeleted, true); - if (winNum > 0) { - qDebug("===stream=== reload state. save result %" PRId64 ", %" PRIu64, winInfo.sessionWin.win.skey, winInfo.sessionWin.groupId); - if (pInfo->twAggSup.calTrigger == STREAM_TRIGGER_AT_ONCE) { - saveResult(winInfo, pInfo->pStUpdated); - } else if (pInfo->twAggSup.calTrigger == STREAM_TRIGGER_WINDOW_CLOSE) { - if (!isCloseWindow(&winInfo.sessionWin.win, &pInfo->twAggSup)) { - saveDeleteRes(pInfo->pStDeleted, winInfo.sessionWin); - } - SSessionKey key = {0}; - getSessionHashKey(&winInfo.sessionWin, &key); - tSimpleHashPut(pAggSup->pResultRows, &key, sizeof(SSessionKey), &winInfo, sizeof(SResultWindowInfo)); - } - } - saveSessionOutputBuf(pAggSup, &winInfo); - } - taosMemoryFree(pBuf); - - SOperatorInfo* downstream = pOperator->pDownstream[0]; - if (downstream->fpSet.reloadStreamStateFn) { - downstream->fpSet.reloadStreamStateFn(downstream); - } -} - -SOperatorInfo* createStreamSessionAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, - SExecTaskInfo* pTaskInfo, SReadHandle* pHandle) { - SSessionWinodwPhysiNode* pSessionNode = (SSessionWinodwPhysiNode*)pPhyNode; - int32_t numOfCols = 0; - int32_t code = TSDB_CODE_OUT_OF_MEMORY; - SStreamSessionAggOperatorInfo* pInfo = taosMemoryCalloc(1, sizeof(SStreamSessionAggOperatorInfo)); - SOperatorInfo* pOperator = taosMemoryCalloc(1, sizeof(SOperatorInfo)); - if (pInfo == NULL || pOperator == NULL) { - goto _error; - } - - pOperator->pTaskInfo = pTaskInfo; - - initResultSizeInfo(&pOperator->resultInfo, 4096); - if (pSessionNode->window.pExprs != NULL) { - int32_t numOfScalar = 0; - SExprInfo* pScalarExprInfo = createExprInfo(pSessionNode->window.pExprs, NULL, &numOfScalar); - code = initExprSupp(&pInfo->scalarSupp, pScalarExprInfo, numOfScalar, &pTaskInfo->storageAPI.functionStore); - if (code != TSDB_CODE_SUCCESS) { - goto _error; - } - } - SExprSupp* pSup = &pOperator->exprSupp; - - SExprInfo* pExprInfo = createExprInfo(pSessionNode->window.pFuncs, NULL, &numOfCols); - SSDataBlock* pResBlock = createDataBlockFromDescNode(pPhyNode->pOutputDataBlockDesc); - code = initBasicInfoEx(&pInfo->binfo, pSup, pExprInfo, numOfCols, pResBlock, &pTaskInfo->storageAPI.functionStore); - if (code != TSDB_CODE_SUCCESS) { - goto _error; - } - - code = initStreamAggSupporter(&pInfo->streamAggSup, pSup->pCtx, numOfCols, pSessionNode->gap, - pTaskInfo->streamInfo.pState, 0, 0, &pTaskInfo->storageAPI.stateStore, pHandle, - &pTaskInfo->storageAPI); - if (code != TSDB_CODE_SUCCESS) { - goto _error; - } - - pInfo->twAggSup = (STimeWindowAggSupp){ - .waterMark = pSessionNode->window.watermark, - .calTrigger = pSessionNode->window.triggerType, - .maxTs = INT64_MIN, - .minTs = INT64_MAX, - }; - - initExecTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &pTaskInfo->window); - - pInfo->primaryTsIndex = ((SColumnNode*)pSessionNode->window.pTspk)->slotId; - 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); - pInfo->pDelIterator = NULL; - pInfo->pDelRes = createSpecialDataBlock(STREAM_DELETE_RESULT); - pInfo->pChildren = NULL; - pInfo->isFinal = false; - pInfo->pPhyNode = pPhyNode; - pInfo->ignoreExpiredData = pSessionNode->window.igExpired; - pInfo->ignoreExpiredDataSaved = false; - pInfo->pUpdated = NULL; - pInfo->pStUpdated = NULL; - pInfo->dataVersion = 0; - pInfo->historyWins = taosArrayInit(4, sizeof(SSessionKey)); - if (!pInfo->historyWins) { - goto _error; - } - if (pHandle) { - pInfo->isHistoryOp = pHandle->fillHistory; - } - - pInfo->pCheckpointRes = createSpecialDataBlock(STREAM_CHECKPOINT); - - // for stream - void* buff = NULL; - int32_t len = 0; - int32_t res = - pInfo->streamAggSup.stateStore.streamStateGetInfo(pInfo->streamAggSup.pState, STREAM_SESSION_OP_CHECKPOINT_NAME, - strlen(STREAM_SESSION_OP_CHECKPOINT_NAME), &buff, &len); - if (res == TSDB_CODE_SUCCESS) { - doStreamSessionDecodeOpState(buff, len, pOperator, true); - taosMemoryFree(buff); - } - - setOperatorInfo(pOperator, "StreamSessionWindowAggOperator", QUERY_NODE_PHYSICAL_PLAN_STREAM_SESSION, true, - OP_NOT_OPENED, pInfo, pTaskInfo); - pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, doStreamSessionAgg, NULL, destroyStreamSessionAggOperatorInfo, - optrDefaultBufFn, NULL); - setOperatorStreamStateFn(pOperator, streamSessionReleaseState, streamSessionReloadState); - - if (downstream) { - initDownStream(downstream, &pInfo->streamAggSup, pOperator->operatorType, pInfo->primaryTsIndex, &pInfo->twAggSup); - code = appendDownstream(pOperator, &downstream, 1); - } - return pOperator; - -_error: - if (pInfo != NULL) { - destroyStreamSessionAggOperatorInfo(pInfo); - } - - taosMemoryFreeClear(pOperator); - pTaskInfo->code = code; - return NULL; -} - -static void clearStreamSessionOperator(SStreamSessionAggOperatorInfo* pInfo) { - tSimpleHashClear(pInfo->streamAggSup.pResultRows); - pInfo->streamAggSup.stateStore.streamStateSessionClear(pInfo->streamAggSup.pState); -} - -static SSDataBlock* doStreamSessionSemiAgg(SOperatorInfo* pOperator) { - SStreamSessionAggOperatorInfo* pInfo = pOperator->info; - SOptrBasicInfo* pBInfo = &pInfo->binfo; - TSKEY maxTs = INT64_MIN; - SExprSupp* pSup = &pOperator->exprSupp; - SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; - - qDebug("===stream=== stream session semi agg"); - if (pOperator->status == OP_EXEC_DONE) { - return NULL; - } - - { - doBuildSessionResult(pOperator, pAggSup->pState, &pInfo->groupResInfo, pBInfo->pRes); - if (pBInfo->pRes->info.rows > 0) { - printDataBlock(pBInfo->pRes, "semi session"); - return pBInfo->pRes; - } - - doBuildDeleteDataBlock(pOperator, pInfo->pStDeleted, pInfo->pDelRes, &pInfo->pDelIterator); - if (pInfo->pDelRes->info.rows > 0) { - printDataBlock(pInfo->pDelRes, "semi session delete"); - return pInfo->pDelRes; - } - - if (pInfo->reCkBlock) { - pInfo->reCkBlock = false; - printDataBlock(pInfo->pCheckpointRes, "semi session ck"); - return pInfo->pCheckpointRes; - } - - if (pOperator->status == OP_RES_TO_RETURN) { - clearFunctionContext(&pOperator->exprSupp); - // semi interval operator clear disk buffer - clearStreamSessionOperator(pInfo); - setOperatorCompleted(pOperator); - return NULL; - } - } - - SOperatorInfo* downstream = pOperator->pDownstream[0]; - if (!pInfo->pUpdated) { - pInfo->pUpdated = taosArrayInit(16, sizeof(SSessionKey)); - } - if (!pInfo->pStUpdated) { - _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); - pInfo->pStUpdated = tSimpleHashInit(64, hashFn); - } - while (1) { - SSDataBlock* pBlock = downstream->fpSet.getNextFn(downstream); - if (pBlock == NULL) { - pOperator->status = OP_RES_TO_RETURN; - break; - } - printDataBlock(pBlock, "semi session recv"); - - if (pBlock->info.type == STREAM_DELETE_DATA || pBlock->info.type == STREAM_DELETE_RESULT || - pBlock->info.type == STREAM_CLEAR) { - // gap must be 0 - SArray* pWins = taosArrayInit(16, sizeof(SSessionKey)); - doDeleteTimeWindows(&pInfo->streamAggSup, pBlock, pWins); - removeSessionResults(pInfo->pStUpdated, pWins); - copyDeleteWindowInfo(pWins, pInfo->pStDeleted); - taosArrayDestroy(pWins); - break; - } else if (pBlock->info.type == STREAM_GET_ALL) { - getAllSessionWindow(pInfo->streamAggSup.pResultRows, pInfo->pStUpdated); - continue; - } else if (pBlock->info.type == STREAM_CREATE_CHILD_TABLE) { - return pBlock; - } else if (pBlock->info.type == STREAM_CHECKPOINT) { - pAggSup->stateStore.streamStateCommit(pAggSup->pState); - doStreamSessionSaveCheckpoint(pOperator); - continue; - } else { - ASSERTS(pBlock->info.type == STREAM_NORMAL || pBlock->info.type == STREAM_INVALID, "invalid SSDataBlock type"); - } - - if (pInfo->scalarSupp.pExprInfo != NULL) { - SExprSupp* pExprSup = &pInfo->scalarSupp; - projectApplyFunctions(pExprSup->pExprInfo, pBlock, pBlock, pExprSup->pCtx, pExprSup->numOfExprs, NULL); - } - // the pDataBlock are always the same one, no need to call this again - setInputDataBlock(pSup, pBlock, TSDB_ORDER_ASC, MAIN_SCAN, true); - doStreamSessionAggImpl(pOperator, pBlock, pInfo->pStUpdated, NULL, false, false); - maxTs = TMAX(pInfo->twAggSup.maxTs, pBlock->info.window.ekey); - } - - pInfo->twAggSup.maxTs = TMAX(pInfo->twAggSup.maxTs, maxTs); - pBInfo->pRes->info.watermark = pInfo->twAggSup.maxTs; - - copyUpdateResult(pInfo->pStUpdated, pInfo->pUpdated); - removeSessionResults(pInfo->pStDeleted, pInfo->pUpdated); - tSimpleHashCleanup(pInfo->pStUpdated); - pInfo->pStUpdated = NULL; - initGroupResInfoFromArrayList(&pInfo->groupResInfo, pInfo->pUpdated); - pInfo->pUpdated = NULL; - blockDataEnsureCapacity(pBInfo->pRes, pOperator->resultInfo.capacity); - -#if 0 - char* pBuf = streamStateSessionDump(pAggSup->pState); - qDebug("===stream===semi session%s", pBuf); - taosMemoryFree(pBuf); -#endif - - doBuildSessionResult(pOperator, pAggSup->pState, &pInfo->groupResInfo, pBInfo->pRes); - if (pBInfo->pRes->info.rows > 0) { - printDataBlock(pBInfo->pRes, "semi session"); - return pBInfo->pRes; - } - - doBuildDeleteDataBlock(pOperator, pInfo->pStDeleted, pInfo->pDelRes, &pInfo->pDelIterator); - if (pInfo->pDelRes->info.rows > 0) { - printDataBlock(pInfo->pDelRes, "semi session delete"); - return pInfo->pDelRes; - } - - if (pInfo->reCkBlock) { - pInfo->reCkBlock = false; - printDataBlock(pInfo->pCheckpointRes, "semi session ck"); - return pInfo->pCheckpointRes; - } - - clearFunctionContext(&pOperator->exprSupp); - // semi interval operator clear disk buffer - clearStreamSessionOperator(pInfo); - setOperatorCompleted(pOperator); - return NULL; -} - -SOperatorInfo* createStreamFinalSessionAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, - SExecTaskInfo* pTaskInfo, int32_t numOfChild, - SReadHandle* pHandle) { - int32_t code = TSDB_CODE_OUT_OF_MEMORY; - SOperatorInfo* pOperator = createStreamSessionAggOperatorInfo(downstream, pPhyNode, pTaskInfo, pHandle); - if (pOperator == NULL) { - goto _error; - } - - SStorageAPI* pAPI = &pTaskInfo->storageAPI; - SStreamSessionAggOperatorInfo* pInfo = pOperator->info; - - pInfo->isFinal = (pPhyNode->type == QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_SESSION); - char* name = (pInfo->isFinal) ? "StreamSessionFinalAggOperator" : "StreamSessionSemiAggOperator"; - - if (pPhyNode->type != QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_SESSION) { - pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, doStreamSessionSemiAgg, NULL, - destroyStreamSessionAggOperatorInfo, optrDefaultBufFn, NULL); - } - setOperatorStreamStateFn(pOperator, streamSessionReleaseState, streamSessionReloadState); - setOperatorInfo(pOperator, name, pPhyNode->type, false, OP_NOT_OPENED, pInfo, pTaskInfo); - - pOperator->operatorType = pPhyNode->type; - if (numOfChild > 0) { - pInfo->pChildren = taosArrayInit(numOfChild, sizeof(void*)); - for (int32_t i = 0; i < numOfChild; i++) { - SOperatorInfo* pChildOp = createStreamFinalSessionAggOperatorInfo(NULL, pPhyNode, pTaskInfo, 0, NULL); - if (pChildOp == NULL) { - goto _error; - } - SStreamSessionAggOperatorInfo* pChInfo = pChildOp->info; - pChInfo->twAggSup.calTrigger = STREAM_TRIGGER_AT_ONCE; - pAPI->stateStore.streamStateSetNumber(pChInfo->streamAggSup.pState, i); - taosArrayPush(pInfo->pChildren, &pChildOp); - } - } - - if (!IS_FINAL_OP(pInfo) || numOfChild == 0) { - pInfo->twAggSup.calTrigger = STREAM_TRIGGER_AT_ONCE; - } - - return pOperator; - -_error: - if (pInfo != NULL) { - destroyStreamSessionAggOperatorInfo(pInfo); - } - taosMemoryFreeClear(pOperator); - pTaskInfo->code = code; - return NULL; -} - -void destroyStreamStateOperatorInfo(void* param) { - SStreamStateAggOperatorInfo* pInfo = (SStreamStateAggOperatorInfo*)param; - cleanupBasicInfo(&pInfo->binfo); - destroyStreamAggSupporter(&pInfo->streamAggSup); - cleanupGroupResInfo(&pInfo->groupResInfo); - if (pInfo->pChildren != NULL) { - int32_t size = taosArrayGetSize(pInfo->pChildren); - for (int32_t i = 0; i < size; i++) { - SOperatorInfo* pChild = taosArrayGetP(pInfo->pChildren, i); - destroyOperator(pChild); - } - taosArrayDestroy(pInfo->pChildren); - } - colDataDestroy(&pInfo->twAggSup.timeWindowData); - blockDataDestroy(pInfo->pDelRes); - taosArrayDestroy(pInfo->historyWins); - tSimpleHashCleanup(pInfo->pSeUpdated); - tSimpleHashCleanup(pInfo->pSeDeleted); - blockDataDestroy(pInfo->pCheckpointRes); - - taosMemoryFreeClear(param); -} - -bool isTsInWindow(SStateWindowInfo* pWin, TSKEY ts) { - if (pWin->winInfo.sessionWin.win.skey <= ts && ts <= pWin->winInfo.sessionWin.win.ekey) { - return true; - } - return false; -} - -bool isEqualStateKey(SStateWindowInfo* pWin, char* pKeyData) { - return pKeyData && compareVal(pKeyData, pWin->pStateKey); -} - -bool compareStateKey(void* data, void* key) { - if (!data || !key) { - return true; - } - SStateKeys* stateKey = (SStateKeys*)key; - stateKey->pData = (char*)key + sizeof(SStateKeys); - return compareVal(data, stateKey); -} - -bool compareWinStateKey(SStateKeys* left, SStateKeys* right) { - if (!left || !right) { - return false; - } - return compareVal(left->pData, right); -} - -void setStateOutputBuf(SStreamAggSupporter* pAggSup, TSKEY ts, uint64_t groupId, char* pKeyData, - SStateWindowInfo* pCurWin, SStateWindowInfo* pNextWin) { - int32_t size = pAggSup->resultRowSize; - pCurWin->winInfo.sessionWin.groupId = groupId; - pCurWin->winInfo.sessionWin.win.skey = ts; - pCurWin->winInfo.sessionWin.win.ekey = ts; - int32_t code = pAggSup->stateStore.streamStateStateAddIfNotExist(pAggSup->pState, &pCurWin->winInfo.sessionWin, - pKeyData, pAggSup->stateKeySize, compareStateKey, - &pCurWin->winInfo.pOutputBuf, &size); - pCurWin->pStateKey = - (SStateKeys*)((char*)pCurWin->winInfo.pOutputBuf + (pAggSup->resultRowSize - pAggSup->stateKeySize)); - pCurWin->pStateKey->bytes = pAggSup->stateKeySize - sizeof(SStateKeys); - pCurWin->pStateKey->type = pAggSup->stateKeyType; - pCurWin->pStateKey->pData = (char*)pCurWin->pStateKey + sizeof(SStateKeys); - pCurWin->pStateKey->isNull = false; - - if (code == TSDB_CODE_SUCCESS && !inWinRange(&pAggSup->winRange, &pCurWin->winInfo.sessionWin.win)) { - code = TSDB_CODE_FAILED; - releaseOutputBuf(pAggSup->pState, NULL, (SResultRow*)pCurWin->winInfo.pOutputBuf, &pAggSup->pSessionAPI->stateStore); - pCurWin->winInfo.pOutputBuf = taosMemoryCalloc(1, size); - pCurWin->pStateKey = - (SStateKeys*)((char*)pCurWin->winInfo.pOutputBuf + (pAggSup->resultRowSize - pAggSup->stateKeySize)); - pCurWin->pStateKey->bytes = pAggSup->stateKeySize - sizeof(SStateKeys); - pCurWin->pStateKey->type = pAggSup->stateKeyType; - pCurWin->pStateKey->pData = (char*)pCurWin->pStateKey + sizeof(SStateKeys); - pCurWin->pStateKey->isNull = false; - pCurWin->winInfo.sessionWin.groupId = groupId; - pCurWin->winInfo.sessionWin.win.skey = ts; - pCurWin->winInfo.sessionWin.win.ekey = ts; - qDebug("===stream===reset state win key. skey:%" PRId64 ", endkey:%" PRId64, pCurWin->winInfo.sessionWin.win.skey, pCurWin->winInfo.sessionWin.win.ekey); - } - - if (code == TSDB_CODE_SUCCESS) { - pCurWin->winInfo.isOutput = true; - pAggSup->stateStore.streamStateSessionDel(pAggSup->pState, &pCurWin->winInfo.sessionWin); - } else if (pKeyData) { - if (IS_VAR_DATA_TYPE(pAggSup->stateKeyType)) { - varDataCopy(pCurWin->pStateKey->pData, pKeyData); - } else { - memcpy(pCurWin->pStateKey->pData, pKeyData, pCurWin->pStateKey->bytes); - } - } - - pNextWin->winInfo.sessionWin = pCurWin->winInfo.sessionWin; - SStreamStateCur* pCur = pAggSup->stateStore.streamStateSessionSeekKeyNext(pAggSup->pState, &pNextWin->winInfo.sessionWin); - int32_t nextSize = pAggSup->resultRowSize; - code = pAggSup->stateStore.streamStateSessionGetKVByCur(pCur, &pNextWin->winInfo.sessionWin, &pNextWin->winInfo.pOutputBuf, &nextSize); - if (code != TSDB_CODE_SUCCESS) { - SET_SESSION_WIN_INVALID(pNextWin->winInfo); - } else { - pNextWin->pStateKey = - (SStateKeys*)((char*)pNextWin->winInfo.pOutputBuf + (pAggSup->resultRowSize - pAggSup->stateKeySize)); - pNextWin->pStateKey->bytes = pAggSup->stateKeySize - sizeof(SStateKeys); - pNextWin->pStateKey->type = pAggSup->stateKeyType; - pNextWin->pStateKey->pData = (char*)pNextWin->pStateKey + sizeof(SStateKeys); - pNextWin->pStateKey->isNull = false; - pNextWin->winInfo.isOutput = true; - } - pAggSup->stateStore.streamStateFreeCur(pCur); -} - -int32_t updateStateWindowInfo(SStateWindowInfo* pWinInfo, SStateWindowInfo* pNextWin, TSKEY* pTs, uint64_t groupId, - SColumnInfoData* pKeyCol, int32_t rows, int32_t start, bool* allEqual, - SSHashObj* pResultRows, SSHashObj* pSeUpdated, SSHashObj* pSeDeleted) { - *allEqual = true; - for (int32_t i = start; i < rows; ++i) { - char* pKeyData = colDataGetData(pKeyCol, i); - if (!isTsInWindow(pWinInfo, pTs[i])) { - if (isEqualStateKey(pWinInfo, pKeyData)) { - if (IS_VALID_SESSION_WIN(pNextWin->winInfo)) { - // ts belongs to the next window - if (pTs[i] >= pNextWin->winInfo.sessionWin.win.skey) { - return i - start; - } - } - } else { - return i - start; - } - } - - if (pWinInfo->winInfo.sessionWin.win.skey > pTs[i]) { - if (pSeDeleted && pWinInfo->winInfo.isOutput) { - saveDeleteRes(pSeDeleted, pWinInfo->winInfo.sessionWin); - } - removeSessionResult(pSeUpdated, pResultRows, pWinInfo->winInfo.sessionWin); - pWinInfo->winInfo.sessionWin.win.skey = pTs[i]; - } - pWinInfo->winInfo.sessionWin.win.ekey = TMAX(pWinInfo->winInfo.sessionWin.win.ekey, pTs[i]); - if (!isEqualStateKey(pWinInfo, pKeyData)) { - *allEqual = false; - } - } - return rows - start; -} - -static void doStreamStateAggImpl(SOperatorInfo* pOperator, SSDataBlock* pSDataBlock, SSHashObj* pSeUpdated, - SSHashObj* pStDeleted) { - SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; - SStorageAPI* pAPI = &pOperator->pTaskInfo->storageAPI; - - SStreamStateAggOperatorInfo* pInfo = pOperator->info; - int32_t numOfOutput = pOperator->exprSupp.numOfExprs; - uint64_t groupId = pSDataBlock->info.id.groupId; - int64_t code = TSDB_CODE_SUCCESS; - TSKEY* tsCols = NULL; - SResultRow* pResult = NULL; - int32_t winRows = 0; - SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; - - pInfo->dataVersion = TMAX(pInfo->dataVersion, pSDataBlock->info.version); - pAggSup->winRange = pTaskInfo->streamInfo.fillHistoryWindow; - if (pAggSup->winRange.ekey <= 0) { - pAggSup->winRange.ekey = INT64_MAX; - } - - if (pSDataBlock->pDataBlock != NULL) { - SColumnInfoData* pColDataInfo = taosArrayGet(pSDataBlock->pDataBlock, pInfo->primaryTsIndex); - tsCols = (int64_t*)pColDataInfo->pData; - } else { - return; - } - - int32_t rows = pSDataBlock->info.rows; - blockDataEnsureCapacity(pAggSup->pScanBlock, rows); - SColumnInfoData* pKeyColInfo = taosArrayGet(pSDataBlock->pDataBlock, pInfo->stateCol.slotId); - for (int32_t i = 0; i < rows; i += winRows) { - if (pInfo->ignoreExpiredData && isOverdue(tsCols[i], &pInfo->twAggSup) || colDataIsNull_s(pKeyColInfo, i)) { - i++; - continue; - } - char* pKeyData = colDataGetData(pKeyColInfo, i); - int32_t winIndex = 0; - bool allEqual = true; - SStateWindowInfo curWin = {0}; - SStateWindowInfo nextWin = {0}; - setStateOutputBuf(pAggSup, tsCols[i], groupId, pKeyData, &curWin, &nextWin); - if (IS_VALID_SESSION_WIN(nextWin.winInfo)) { - releaseOutputBuf(pAggSup->pState, NULL, (SResultRow*)nextWin.winInfo.pOutputBuf, &pAPI->stateStore); - } - setSessionWinOutputInfo(pSeUpdated, &curWin.winInfo); - winRows = updateStateWindowInfo(&curWin, &nextWin, tsCols, groupId, pKeyColInfo, rows, i, &allEqual, - pAggSup->pResultRows, pSeUpdated, pStDeleted); - if (!allEqual) { - uint64_t uid = 0; - appendOneRowToStreamSpecialBlock(pAggSup->pScanBlock, &curWin.winInfo.sessionWin.win.skey, - &curWin.winInfo.sessionWin.win.ekey, &uid, &groupId, NULL); - tSimpleHashRemove(pSeUpdated, &curWin.winInfo.sessionWin, sizeof(SSessionKey)); - doDeleteSessionWindow(pAggSup, &curWin.winInfo.sessionWin); - releaseOutputBuf(pAggSup->pState, NULL, (SResultRow*)curWin.winInfo.pOutputBuf, &pAPI->stateStore); - continue; - } - code = doOneWindowAggImpl(&pInfo->twAggSup.timeWindowData, &curWin.winInfo, &pResult, i, winRows, rows, numOfOutput, - pOperator, 0); - if (code != TSDB_CODE_SUCCESS || pResult == NULL) { - T_LONG_JMP(pTaskInfo->env, TSDB_CODE_OUT_OF_MEMORY); - } - saveSessionOutputBuf(pAggSup, &curWin.winInfo); - - if (pInfo->twAggSup.calTrigger == STREAM_TRIGGER_AT_ONCE) { - code = saveResult(curWin.winInfo, pSeUpdated); - if (code != TSDB_CODE_SUCCESS) { - T_LONG_JMP(pTaskInfo->env, TSDB_CODE_OUT_OF_MEMORY); - } - } - - if (pInfo->twAggSup.calTrigger == STREAM_TRIGGER_WINDOW_CLOSE) { - SSessionKey key = {0}; - getSessionHashKey(&curWin.winInfo.sessionWin, &key); - tSimpleHashPut(pAggSup->pResultRows, &key, sizeof(SSessionKey), &curWin.winInfo, sizeof(SResultWindowInfo)); - } - } -} - -int32_t doStreamStateEncodeOpState(void** buf, int32_t len, SOperatorInfo* pOperator, bool isParent) { - SStreamStateAggOperatorInfo* pInfo = pOperator->info; - if (!pInfo) { - return 0; - } - - void* pData = (buf == NULL) ? NULL : *buf; - - // 1.streamAggSup.pResultRows - int32_t tlen = 0; - int32_t mapSize = tSimpleHashGetSize(pInfo->streamAggSup.pResultRows); - tlen += taosEncodeFixedI32(buf, mapSize); - void* pIte = NULL; - size_t keyLen = 0; - int32_t iter = 0; - while ((pIte = tSimpleHashIterate(pInfo->streamAggSup.pResultRows, pIte, &iter)) != NULL) { - void* key = taosHashGetKey(pIte, &keyLen); - tlen += encodeSSessionKey(buf, key); - tlen += encodeSResultWindowInfo(buf, pIte, pInfo->streamAggSup.resultRowSize); - } - - // 2.twAggSup - tlen += encodeSTimeWindowAggSupp(buf, &pInfo->twAggSup); - - // 3.pChildren - int32_t size = taosArrayGetSize(pInfo->pChildren); - tlen += taosEncodeFixedI32(buf, size); - for (int32_t i = 0; i < size; i++) { - SOperatorInfo* pChOp = taosArrayGetP(pInfo->pChildren, i); - tlen += doStreamStateEncodeOpState(buf, 0, pChOp, false); - } - - // 4.dataVersion - tlen += taosEncodeFixedI32(buf, pInfo->dataVersion); - - // 5.checksum - if (isParent) { - if (buf) { - uint32_t cksum = taosCalcChecksum(0, pData, len - sizeof(uint32_t)); - tlen += taosEncodeFixedU32(buf, cksum); - } else { - tlen += sizeof(uint32_t); - } - } - - return tlen; -} - -void* doStreamStateDecodeOpState(void* buf, int32_t len, SOperatorInfo* pOperator, bool isParent) { - SStreamStateAggOperatorInfo* pInfo = pOperator->info; - if (!pInfo) { - return buf; - } - - // 5.checksum - if (isParent) { - int32_t dataLen = len - sizeof(uint32_t); - void* pCksum = POINTER_SHIFT(buf, dataLen); - if (taosCheckChecksum(buf, dataLen, *(uint32_t*)pCksum) != TSDB_CODE_SUCCESS) { - ASSERT(0); // debug - qError("stream interval state is invalid"); - return buf; - } - } - - // 1.streamAggSup.pResultRows - int32_t mapSize = 0; - buf = taosDecodeFixedI32(buf, &mapSize); - for (int32_t i = 0; i < mapSize; i++) { - SSessionKey key = {0}; - SResultWindowInfo winfo = {0}; - buf = decodeSSessionKey(buf, &key); - buf = decodeSResultWindowInfo(buf, &winfo, pInfo->streamAggSup.resultRowSize); - tSimpleHashPut(pInfo->streamAggSup.pResultRows, &key, sizeof(SSessionKey), &winfo, sizeof(SResultWindowInfo)); - } - - // 2.twAggSup - buf = decodeSTimeWindowAggSupp(buf, &pInfo->twAggSup); - - // 3.pChildren - int32_t size = 0; - buf = taosDecodeFixedI32(buf, &size); - ASSERT(size <= taosArrayGetSize(pInfo->pChildren)); - for (int32_t i = 0; i < size; i++) { - SOperatorInfo* pChOp = taosArrayGetP(pInfo->pChildren, i); - buf = doStreamStateDecodeOpState(buf, 0, pChOp, false); - } - - // 4.dataVersion - buf = taosDecodeFixedI64(buf, &pInfo->dataVersion); - return buf; -} - -void doStreamStateSaveCheckpoint(SOperatorInfo* pOperator) { - SStreamStateAggOperatorInfo* pInfo = pOperator->info; - int32_t len = doStreamStateEncodeOpState(NULL, 0, pOperator, true); - void* buf = taosMemoryCalloc(1, len); - void* pBuf = buf; - len = doStreamStateEncodeOpState(&pBuf, len, pOperator, true); - pInfo->streamAggSup.stateStore.streamStateSaveInfo(pInfo->streamAggSup.pState, STREAM_STATE_OP_CHECKPOINT_NAME, - strlen(STREAM_STATE_OP_CHECKPOINT_NAME), buf, len); -} - -static SSDataBlock* doStreamStateAgg(SOperatorInfo* pOperator) { - if (pOperator->status == OP_EXEC_DONE) { - return NULL; - } - - SExprSupp* pSup = &pOperator->exprSupp; - SStreamStateAggOperatorInfo* pInfo = pOperator->info; - SOptrBasicInfo* pBInfo = &pInfo->binfo; - qDebug("===stream=== stream state agg"); - if (pOperator->status == OP_RES_TO_RETURN) { - doBuildDeleteDataBlock(pOperator, pInfo->pSeDeleted, pInfo->pDelRes, &pInfo->pDelIterator); - if (pInfo->pDelRes->info.rows > 0) { - printDataBlock(pInfo->pDelRes, "single state delete"); - return pInfo->pDelRes; - } - - doBuildSessionResult(pOperator, pInfo->streamAggSup.pState, &pInfo->groupResInfo, pBInfo->pRes); - if (pBInfo->pRes->info.rows > 0) { - printDataBlock(pBInfo->pRes, "single state"); - return pBInfo->pRes; - } - - setOperatorCompleted(pOperator); - return NULL; - } - - SOperatorInfo* downstream = pOperator->pDownstream[0]; - if (!pInfo->pUpdated) { - pInfo->pUpdated = taosArrayInit(16, sizeof(SSessionKey)); - } - if (!pInfo->pSeUpdated) { - _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); - pInfo->pSeUpdated = tSimpleHashInit(64, hashFn); - } - while (1) { - SSDataBlock* pBlock = downstream->fpSet.getNextFn(downstream); - if (pBlock == NULL) { - break; - } - printDataBlock(pBlock, "single state recv"); - - if (pBlock->info.type == STREAM_DELETE_DATA || pBlock->info.type == STREAM_DELETE_RESULT || - pBlock->info.type == STREAM_CLEAR) { - SArray* pWins = taosArrayInit(16, sizeof(SSessionKey)); - doDeleteTimeWindows(&pInfo->streamAggSup, pBlock, pWins); - removeSessionResults(pInfo->pSeUpdated, pWins); - copyDeleteWindowInfo(pWins, pInfo->pSeDeleted); - taosArrayDestroy(pWins); - continue; - } else if (pBlock->info.type == STREAM_GET_ALL) { - getAllSessionWindow(pInfo->streamAggSup.pResultRows, pInfo->pSeUpdated); - continue; - } else if (pBlock->info.type == STREAM_CREATE_CHILD_TABLE) { - return pBlock; - } else if (pBlock->info.type == STREAM_CHECKPOINT) { - pInfo->streamAggSup.stateStore.streamStateCommit(pInfo->streamAggSup.pState); - doStreamSessionSaveCheckpoint(pOperator); - copyDataBlock(pInfo->pCheckpointRes, pBlock); - continue; - } else { - ASSERTS(pBlock->info.type == STREAM_NORMAL || pBlock->info.type == STREAM_INVALID, "invalid SSDataBlock type"); - } - - if (pInfo->scalarSupp.pExprInfo != NULL) { - SExprSupp* pExprSup = &pInfo->scalarSupp; - projectApplyFunctions(pExprSup->pExprInfo, pBlock, pBlock, pExprSup->pCtx, pExprSup->numOfExprs, NULL); - } - // the pDataBlock are always the same one, no need to call this again - setInputDataBlock(pSup, pBlock, TSDB_ORDER_ASC, MAIN_SCAN, true); - doStreamStateAggImpl(pOperator, pBlock, pInfo->pSeUpdated, pInfo->pSeDeleted); - pInfo->twAggSup.maxTs = TMAX(pInfo->twAggSup.maxTs, pBlock->info.window.ekey); - } - // restore the value - pOperator->status = OP_RES_TO_RETURN; - - closeSessionWindow(pInfo->streamAggSup.pResultRows, &pInfo->twAggSup, pInfo->pSeUpdated); - copyUpdateResult(pInfo->pSeUpdated, pInfo->pUpdated); - removeSessionResults(pInfo->pSeDeleted, pInfo->pUpdated); - tSimpleHashCleanup(pInfo->pSeUpdated); - pInfo->pSeUpdated = NULL; - - if (pInfo->isHistoryOp) { - getMaxTsWins(pInfo->pUpdated, pInfo->historyWins); - } - - initGroupResInfoFromArrayList(&pInfo->groupResInfo, pInfo->pUpdated); - pInfo->pUpdated = NULL; - blockDataEnsureCapacity(pInfo->binfo.pRes, pOperator->resultInfo.capacity); - -#if 0 - char* pBuf = streamStateSessionDump(pInfo->streamAggSup.pState); - qDebug("===stream===final session%s", pBuf); - taosMemoryFree(pBuf); -#endif - - doBuildDeleteDataBlock(pOperator, pInfo->pSeDeleted, pInfo->pDelRes, &pInfo->pDelIterator); - if (pInfo->pDelRes->info.rows > 0) { - printDataBlock(pInfo->pDelRes, "single state delete"); - return pInfo->pDelRes; - } - - doBuildSessionResult(pOperator, pInfo->streamAggSup.pState, &pInfo->groupResInfo, pBInfo->pRes); - if (pBInfo->pRes->info.rows > 0) { - printDataBlock(pBInfo->pRes, "single state"); - return pBInfo->pRes; - } - setOperatorCompleted(pOperator); - return NULL; -} - -void streamStateReleaseState(SOperatorInfo* pOperator) { - SStreamStateAggOperatorInfo* pInfo = pOperator->info; - int32_t resSize = taosArrayGetSize(pInfo->historyWins) * sizeof(SSessionKey); - qDebug("===stream=== relase state. save result count:%d", (int32_t)taosArrayGetSize(pInfo->historyWins)); - pInfo->streamAggSup.stateStore.streamStateSaveInfo(pInfo->streamAggSup.pState, STREAM_STATE_OP_STATE_NAME, strlen(STREAM_STATE_OP_STATE_NAME), pInfo->historyWins->pData, resSize); - SOperatorInfo* downstream = pOperator->pDownstream[0]; - if (downstream->fpSet.releaseStreamStateFn) { - downstream->fpSet.releaseStreamStateFn(downstream); - } -} - -static void compactStateWindow(SOperatorInfo* pOperator, SResultWindowInfo* pCurWin, SResultWindowInfo* pNextWin, - SSHashObj* pStUpdated, SSHashObj* pStDeleted) { - SExprSupp* pSup = &pOperator->exprSupp; - SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; - SStorageAPI* pAPI = &pOperator->pTaskInfo->storageAPI; - - SStreamStateAggOperatorInfo* pInfo = pOperator->info; - SResultRow* pCurResult = NULL; - int32_t numOfOutput = pOperator->exprSupp.numOfExprs; - SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; - initSessionOutputBuf(pCurWin, &pCurResult, pSup->pCtx, numOfOutput, pSup->rowEntryInfoOffset); - SResultRow* pWinResult = NULL; - initSessionOutputBuf(pNextWin, &pWinResult, pAggSup->pDummyCtx, numOfOutput, pSup->rowEntryInfoOffset); - pCurWin->sessionWin.win.ekey = TMAX(pCurWin->sessionWin.win.ekey, pNextWin->sessionWin.win.ekey); - - updateTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &pCurWin->sessionWin.win, 1); - compactFunctions(pSup->pCtx, pAggSup->pDummyCtx, numOfOutput, pTaskInfo, &pInfo->twAggSup.timeWindowData); - tSimpleHashRemove(pStUpdated, &pNextWin->sessionWin, sizeof(SSessionKey)); - if (pNextWin->isOutput && pStDeleted) { - qDebug("===stream=== save delete window info %" PRId64 ", %" PRIu64, pNextWin->sessionWin.win.skey, pNextWin->sessionWin.groupId); - saveDeleteRes(pStDeleted, pNextWin->sessionWin); - } - removeSessionResult(pStUpdated, pAggSup->pResultRows, pNextWin->sessionWin); - doDeleteSessionWindow(pAggSup, &pNextWin->sessionWin); - taosMemoryFree(pNextWin->pOutputBuf); -} - -void streamStateReloadState(SOperatorInfo* pOperator) { - SStreamStateAggOperatorInfo* pInfo = pOperator->info; - SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; - resetWinRange(&pAggSup->winRange); - - SSessionKey seKey = {.win.skey = INT64_MIN, .win.ekey = INT64_MIN, .groupId = 0}; - int32_t size = 0; - void* pBuf = NULL; - int32_t code = pAggSup->stateStore.streamStateGetInfo(pAggSup->pState, STREAM_STATE_OP_STATE_NAME, - strlen(STREAM_STATE_OP_STATE_NAME), &pBuf, &size); - int32_t num = size / sizeof(SSessionKey); - qDebug("===stream=== reload state. get result count:%d", num); - SSessionKey* pSeKeyBuf = (SSessionKey*) pBuf; - ASSERT(size == num * sizeof(SSessionKey)); - if (!pInfo->pSeUpdated && num > 0) { - _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); - pInfo->pSeUpdated = tSimpleHashInit(64, hashFn); - } - if (!pInfo->pSeDeleted && num > 0) { - _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); - pInfo->pSeDeleted = tSimpleHashInit(64, hashFn); - } - for (int32_t i = 0; i < num; i++) { - SStateWindowInfo curInfo = {0}; - SStateWindowInfo nextInfo = {0}; - SStateWindowInfo dummy = {0}; - qDebug("===stream=== reload state. try process result %" PRId64 ", %" PRIu64 ", index:%d", pSeKeyBuf[i].win.skey, pSeKeyBuf[i].groupId, i); - setStateOutputBuf(pAggSup, pSeKeyBuf[i].win.skey, pSeKeyBuf[i].groupId, NULL, &curInfo, &nextInfo); - bool cpRes = compareWinStateKey(curInfo.pStateKey,nextInfo.pStateKey); - qDebug("===stream=== reload state. next window info %" PRId64 ", %" PRIu64 ", compare:%d", nextInfo.winInfo.sessionWin.win.skey, nextInfo.winInfo.sessionWin.groupId, cpRes); - if (cpRes) { - compactStateWindow(pOperator, &curInfo.winInfo, &nextInfo.winInfo, pInfo->pSeUpdated, pInfo->pSeDeleted); - qDebug("===stream=== reload state. save result %" PRId64 ", %" PRIu64, curInfo.winInfo.sessionWin.win.skey, curInfo.winInfo.sessionWin.groupId); - if (pInfo->twAggSup.calTrigger == STREAM_TRIGGER_AT_ONCE) { - saveResult(curInfo.winInfo, pInfo->pSeUpdated); - } else if (pInfo->twAggSup.calTrigger == STREAM_TRIGGER_WINDOW_CLOSE) { - if (!isCloseWindow(&curInfo.winInfo.sessionWin.win, &pInfo->twAggSup)) { - saveDeleteRes(pInfo->pSeDeleted, curInfo.winInfo.sessionWin); - } - SSessionKey key = {0}; - getSessionHashKey(&curInfo.winInfo.sessionWin, &key); - tSimpleHashPut(pAggSup->pResultRows, &key, sizeof(SSessionKey), &curInfo.winInfo, sizeof(SResultWindowInfo)); - } - } else if (IS_VALID_SESSION_WIN(nextInfo.winInfo)) { - releaseOutputBuf(pAggSup->pState, NULL, (SResultRow*)nextInfo.winInfo.pOutputBuf, &pAggSup->pSessionAPI->stateStore); - } - - if (IS_VALID_SESSION_WIN(curInfo.winInfo)) { - saveSessionOutputBuf(pAggSup, &curInfo.winInfo); - } - } - taosMemoryFree(pBuf); - - SOperatorInfo* downstream = pOperator->pDownstream[0]; - if (downstream->fpSet.reloadStreamStateFn) { - downstream->fpSet.reloadStreamStateFn(downstream); - } -} - -SOperatorInfo* createStreamStateAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, - SExecTaskInfo* pTaskInfo, SReadHandle* pHandle) { - SStreamStateWinodwPhysiNode* pStateNode = (SStreamStateWinodwPhysiNode*)pPhyNode; - int32_t tsSlotId = ((SColumnNode*)pStateNode->window.pTspk)->slotId; - SColumnNode* pColNode = (SColumnNode*)(pStateNode->pStateKey); - int32_t code = TSDB_CODE_SUCCESS; - - SStreamStateAggOperatorInfo* pInfo = taosMemoryCalloc(1, sizeof(SStreamStateAggOperatorInfo)); - SOperatorInfo* pOperator = taosMemoryCalloc(1, sizeof(SOperatorInfo)); - if (pInfo == NULL || pOperator == NULL) { - code = TSDB_CODE_OUT_OF_MEMORY; - goto _error; - } - - pInfo->stateCol = extractColumnFromColumnNode(pColNode); - initResultSizeInfo(&pOperator->resultInfo, 4096); - if (pStateNode->window.pExprs != NULL) { - int32_t numOfScalar = 0; - SExprInfo* pScalarExprInfo = createExprInfo(pStateNode->window.pExprs, NULL, &numOfScalar); - code = initExprSupp(&pInfo->scalarSupp, pScalarExprInfo, numOfScalar, &pTaskInfo->storageAPI.functionStore); - if (code != TSDB_CODE_SUCCESS) { - goto _error; - } - } - - pInfo->twAggSup = (STimeWindowAggSupp){ - .waterMark = pStateNode->window.watermark, - .calTrigger = pStateNode->window.triggerType, - .maxTs = INT64_MIN, - .minTs = INT64_MAX, - }; - - initExecTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &pTaskInfo->window); - - SExprSupp* pSup = &pOperator->exprSupp; - int32_t numOfCols = 0; - SExprInfo* pExprInfo = createExprInfo(pStateNode->window.pFuncs, NULL, &numOfCols); - SSDataBlock* pResBlock = createDataBlockFromDescNode(pPhyNode->pOutputDataBlockDesc); - code = initBasicInfoEx(&pInfo->binfo, pSup, pExprInfo, numOfCols, pResBlock, &pTaskInfo->storageAPI.functionStore); - if (code != TSDB_CODE_SUCCESS) { - goto _error; - } - int32_t keySize = sizeof(SStateKeys) + pColNode->node.resType.bytes; - int16_t type = pColNode->node.resType.type; - code = initStreamAggSupporter(&pInfo->streamAggSup, pSup->pCtx, numOfCols, 0, pTaskInfo->streamInfo.pState, keySize, - type, &pTaskInfo->storageAPI.stateStore, pHandle, &pTaskInfo->storageAPI); - if (code != TSDB_CODE_SUCCESS) { - goto _error; - } - - pInfo->primaryTsIndex = tsSlotId; - _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); - pInfo->pSeDeleted = tSimpleHashInit(64, hashFn); - pInfo->pDelIterator = NULL; - pInfo->pDelRes = createSpecialDataBlock(STREAM_DELETE_RESULT); - pInfo->pChildren = NULL; - pInfo->ignoreExpiredData = pStateNode->window.igExpired; - pInfo->ignoreExpiredDataSaved = false; - pInfo->pUpdated = NULL; - pInfo->pSeUpdated = NULL; - pInfo->dataVersion = 0; - pInfo->historyWins = taosArrayInit(4, sizeof(SSessionKey)); - if (!pInfo->historyWins) { - goto _error; - } - if (pHandle) { - pInfo->isHistoryOp = pHandle->fillHistory; - } - - pInfo->pCheckpointRes = createSpecialDataBlock(STREAM_CHECKPOINT); - - // for stream - void* buff = NULL; - int32_t len = 0; - int32_t res = - pInfo->streamAggSup.stateStore.streamStateGetInfo(pInfo->streamAggSup.pState, STREAM_STATE_OP_CHECKPOINT_NAME, - strlen(STREAM_STATE_OP_CHECKPOINT_NAME), &buff, &len); - if (res == TSDB_CODE_SUCCESS) { - doStreamStateDecodeOpState(buff, len, pOperator, true); - taosMemoryFree(buff); - } - - setOperatorInfo(pOperator, "StreamStateAggOperator", QUERY_NODE_PHYSICAL_PLAN_STREAM_STATE, true, OP_NOT_OPENED, - pInfo, pTaskInfo); - pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, doStreamStateAgg, NULL, destroyStreamStateOperatorInfo, - optrDefaultBufFn, NULL); - setOperatorStreamStateFn(pOperator, streamStateReleaseState, streamStateReloadState); - initDownStream(downstream, &pInfo->streamAggSup, pOperator->operatorType, pInfo->primaryTsIndex, &pInfo->twAggSup); - code = appendDownstream(pOperator, &downstream, 1); - if (code != TSDB_CODE_SUCCESS) { - goto _error; - } - return pOperator; - -_error: - destroyStreamStateOperatorInfo(pInfo); - taosMemoryFreeClear(pOperator); - pTaskInfo->code = code; - return NULL; -} - void destroyMAIOperatorInfo(void* param) { SMergeAlignedIntervalAggOperatorInfo* miaInfo = (SMergeAlignedIntervalAggOperatorInfo*)param; destroyIntervalOperatorInfo(miaInfo->intervalAggOperatorInfo); @@ -5734,279 +2181,3 @@ _error: pTaskInfo->code = code; return NULL; } - -static SSDataBlock* doStreamIntervalAgg(SOperatorInfo* pOperator) { - SStreamIntervalOperatorInfo* pInfo = pOperator->info; - SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; - SStorageAPI* pAPI = &pOperator->pTaskInfo->storageAPI; - - SExprSupp* pSup = &pOperator->exprSupp; - - if (pOperator->status == OP_EXEC_DONE) { - return NULL; - } - - if (pOperator->status == OP_RES_TO_RETURN) { - doBuildDeleteResult(pInfo, pInfo->pDelWins, &pInfo->delIndex, pInfo->pDelRes); - if (pInfo->pDelRes->info.rows > 0) { - printDataBlock(pInfo->pDelRes, "single interval delete"); - return pInfo->pDelRes; - } - - doBuildStreamIntervalResult(pOperator, pInfo->pState, pInfo->binfo.pRes, &pInfo->groupResInfo); - if (pInfo->binfo.pRes->info.rows > 0) { - printDataBlock(pInfo->binfo.pRes, "single interval"); - return pInfo->binfo.pRes; - } - - if (pInfo->recvGetAll) { - pInfo->recvGetAll = false; - resetUnCloseWinInfo(pInfo->aggSup.pResultRowHashTable); - } - - if (pInfo->reCkBlock) { - pInfo->reCkBlock = false; - printDataBlock(pInfo->pCheckpointRes, "single interval ck"); - return pInfo->pCheckpointRes; - } - - setOperatorCompleted(pOperator); - return NULL; - } - - SOperatorInfo* downstream = pOperator->pDownstream[0]; - - if (!pInfo->pUpdated) { - pInfo->pUpdated = taosArrayInit(4096, POINTER_BYTES); - } - - if (!pInfo->pUpdatedMap) { - _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); - pInfo->pUpdatedMap = tSimpleHashInit(4096, hashFn); - } - - while (1) { - SSDataBlock* pBlock = downstream->fpSet.getNextFn(downstream); - if (pBlock == NULL) { - qDebug("===stream===return data:single interval. recv datablock num:%" PRIu64, pInfo->numOfDatapack); - pInfo->numOfDatapack = 0; - break; - } - - pInfo->numOfDatapack++; - printDataBlock(pBlock, "single interval recv"); - - if (pBlock->info.type == STREAM_DELETE_DATA || pBlock->info.type == STREAM_DELETE_RESULT || - pBlock->info.type == STREAM_CLEAR) { - doDeleteWindows(pOperator, &pInfo->interval, pBlock, pInfo->pDelWins, pInfo->pUpdatedMap); - continue; - } else if (pBlock->info.type == STREAM_GET_ALL) { - qDebug("===stream===single interval recv|block type STREAM_GET_ALL"); - pInfo->recvGetAll = true; - getAllIntervalWindow(pInfo->aggSup.pResultRowHashTable, pInfo->pUpdatedMap); - continue; - } else if (pBlock->info.type == STREAM_CREATE_CHILD_TABLE) { - printDataBlock(pBlock, "single interval"); - return pBlock; - } else if (pBlock->info.type == STREAM_CHECKPOINT) { - pAPI->stateStore.streamStateCommit(pInfo->pState); - doStreamIntervalSaveCheckpoint(pOperator); - pInfo->reCkBlock = true; - copyDataBlock(pInfo->pCheckpointRes, pBlock); - continue; - } else { - ASSERTS(pBlock->info.type == STREAM_NORMAL || pBlock->info.type == STREAM_INVALID, "invalid SSDataBlock type"); - } - - if (pBlock->info.type == STREAM_NORMAL && pBlock->info.version != 0) { - // set input version - pTaskInfo->version = pBlock->info.version; - } - - if (pInfo->scalarSupp.pExprInfo != NULL) { - SExprSupp* pExprSup = &pInfo->scalarSupp; - projectApplyFunctions(pExprSup->pExprInfo, pBlock, pBlock, pExprSup->pCtx, pExprSup->numOfExprs, NULL); - } - - // The timewindow that overlaps the timestamps of the input pBlock need to be recalculated and return to the - // caller. Note that all the time window are not close till now. - // the pDataBlock are always the same one, no need to call this again - setInputDataBlock(pSup, pBlock, TSDB_ORDER_ASC, MAIN_SCAN, true); - if (pInfo->invertible) { - setInverFunction(pSup->pCtx, pOperator->exprSupp.numOfExprs, pBlock->info.type); - } - - doStreamIntervalAggImpl(pOperator, pBlock, pBlock->info.id.groupId, pInfo->pUpdatedMap); - pInfo->twAggSup.maxTs = TMAX(pInfo->twAggSup.maxTs, pBlock->info.window.ekey); - pInfo->twAggSup.minTs = TMIN(pInfo->twAggSup.minTs, pBlock->info.window.skey); - } - pOperator->status = OP_RES_TO_RETURN; - removeDeleteResults(pInfo->pUpdatedMap, pInfo->pDelWins); - closeStreamIntervalWindow(pInfo->aggSup.pResultRowHashTable, &pInfo->twAggSup, &pInfo->interval, NULL, - pInfo->pUpdatedMap, pInfo->pDelWins, pOperator); - - void* pIte = NULL; - int32_t iter = 0; - while ((pIte = tSimpleHashIterate(pInfo->pUpdatedMap, pIte, &iter)) != NULL) { - taosArrayPush(pInfo->pUpdated, pIte); - } - taosArraySort(pInfo->pUpdated, winPosCmprImpl); - - initMultiResInfoFromArrayList(&pInfo->groupResInfo, pInfo->pUpdated); - pInfo->pUpdated = NULL; - blockDataEnsureCapacity(pInfo->binfo.pRes, pOperator->resultInfo.capacity); - tSimpleHashCleanup(pInfo->pUpdatedMap); - pInfo->pUpdatedMap = NULL; - -#if 0 - char* pBuf = streamStateIntervalDump(pInfo->pState); - qDebug("===stream===interval state%s", pBuf); - taosMemoryFree(pBuf); -#endif - - doBuildDeleteResult(pInfo, pInfo->pDelWins, &pInfo->delIndex, pInfo->pDelRes); - if (pInfo->pDelRes->info.rows > 0) { - printDataBlock(pInfo->pDelRes, "single interval delete"); - return pInfo->pDelRes; - } - - doBuildStreamIntervalResult(pOperator, pInfo->pState, pInfo->binfo.pRes, &pInfo->groupResInfo); - if (pInfo->binfo.pRes->info.rows > 0) { - printDataBlock(pInfo->binfo.pRes, "single interval"); - return pInfo->binfo.pRes; - } - - if (pInfo->reCkBlock) { - pInfo->reCkBlock = false; - printDataBlock(pInfo->pCheckpointRes, "single interval ck"); - return pInfo->pCheckpointRes; - } - - return NULL; -} - -SOperatorInfo* createStreamIntervalOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, - SExecTaskInfo* pTaskInfo, SReadHandle* pHandle) { - SStreamIntervalOperatorInfo* pInfo = taosMemoryCalloc(1, sizeof(SStreamIntervalOperatorInfo)); - SOperatorInfo* pOperator = taosMemoryCalloc(1, sizeof(SOperatorInfo)); - if (pInfo == NULL || pOperator == NULL) { - goto _error; - } - SStreamIntervalPhysiNode* pIntervalPhyNode = (SStreamIntervalPhysiNode*)pPhyNode; - - int32_t code = TSDB_CODE_SUCCESS; - int32_t numOfCols = 0; - SExprInfo* pExprInfo = createExprInfo(pIntervalPhyNode->window.pFuncs, NULL, &numOfCols); - - SSDataBlock* pResBlock = createDataBlockFromDescNode(pPhyNode->pOutputDataBlockDesc); - pInfo->interval = (SInterval){ - .interval = pIntervalPhyNode->interval, - .sliding = pIntervalPhyNode->sliding, - .intervalUnit = pIntervalPhyNode->intervalUnit, - .slidingUnit = pIntervalPhyNode->slidingUnit, - .offset = pIntervalPhyNode->offset, - .precision = ((SColumnNode*)pIntervalPhyNode->window.pTspk)->node.resType.precision, - }; - - pInfo->twAggSup = (STimeWindowAggSupp){.waterMark = pIntervalPhyNode->window.watermark, - .calTrigger = pIntervalPhyNode->window.triggerType, - .maxTs = INT64_MIN, - .minTs = INT64_MAX, - .deleteMark = getDeleteMark(pIntervalPhyNode)}; - - ASSERTS(pInfo->twAggSup.calTrigger != STREAM_TRIGGER_MAX_DELAY, "trigger type should not be max delay"); - - pOperator->pTaskInfo = pTaskInfo; - SStorageAPI* pAPI = &pOperator->pTaskInfo->storageAPI; - - pInfo->ignoreExpiredData = pIntervalPhyNode->window.igExpired; - pInfo->ignoreExpiredDataSaved = false; - pInfo->isFinal = false; - - SExprSupp* pSup = &pOperator->exprSupp; - initBasicInfo(&pInfo->binfo, pResBlock); - initExecTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &pTaskInfo->window); - - pInfo->primaryTsIndex = ((SColumnNode*)pIntervalPhyNode->window.pTspk)->slotId; - initResultSizeInfo(&pOperator->resultInfo, 4096); - - pInfo->pState = taosMemoryCalloc(1, sizeof(SStreamState)); - *(pInfo->pState) = *(pTaskInfo->streamInfo.pState); - pAPI->stateStore.streamStateSetNumber(pInfo->pState, -1); - - size_t keyBufSize = sizeof(int64_t) + sizeof(int64_t) + POINTER_BYTES; - code = initAggSup(pSup, &pInfo->aggSup, pExprInfo, numOfCols, keyBufSize, pTaskInfo->id.str, pInfo->pState, - &pTaskInfo->storageAPI.functionStore); - if (code != TSDB_CODE_SUCCESS) { - goto _error; - } - - if (pIntervalPhyNode->window.pExprs != NULL) { - int32_t numOfScalar = 0; - SExprInfo* pScalarExprInfo = createExprInfo(pIntervalPhyNode->window.pExprs, NULL, &numOfScalar); - code = initExprSupp(&pInfo->scalarSupp, pScalarExprInfo, numOfScalar, &pTaskInfo->storageAPI.functionStore); - if (code != TSDB_CODE_SUCCESS) { - goto _error; - } - } - - pInfo->invertible = allInvertible(pSup->pCtx, numOfCols); - pInfo->invertible = false; - pInfo->pDelWins = taosArrayInit(4, sizeof(SWinKey)); - pInfo->delIndex = 0; - pInfo->pDelRes = createSpecialDataBlock(STREAM_DELETE_RESULT); - initResultRowInfo(&pInfo->binfo.resultRowInfo); - - pInfo->pPhyNode = NULL; // create new child - pInfo->pPullDataMap = NULL; - pInfo->pFinalPullDataMap = NULL; - pInfo->pPullWins = NULL; // SPullWindowInfo - pInfo->pullIndex = 0; - pInfo->pPullDataRes = NULL; - pInfo->isFinal = false; - pInfo->numOfChild = 0; - pInfo->delKey.ts = INT64_MAX; - pInfo->delKey.groupId = 0; - pInfo->numOfDatapack = 0; - pInfo->pUpdated = NULL; - pInfo->pUpdatedMap = NULL; - int32_t funResSize = getMaxFunResSize(pSup, numOfCols); - - pInfo->pState->pFileState = pTaskInfo->storageAPI.stateStore.streamFileStateInit( - tsStreamBufferSize, sizeof(SWinKey), pInfo->aggSup.resultRowSize, funResSize, compareTs, pInfo->pState, - pInfo->twAggSup.deleteMark, GET_TASKID(pTaskInfo), pHandle->checkpointId); - - setOperatorInfo(pOperator, "StreamIntervalOperator", QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL, true, OP_NOT_OPENED, - pInfo, pTaskInfo); - pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, doStreamIntervalAgg, NULL, - destroyStreamFinalIntervalOperatorInfo, optrDefaultBufFn, NULL); - setOperatorStreamStateFn(pOperator, streamIntervalReleaseState, streamIntervalReloadState); - - pInfo->stateStore = pTaskInfo->storageAPI.stateStore; - pInfo->recvGetAll = false; - pInfo->pCheckpointRes = createSpecialDataBlock(STREAM_CHECKPOINT); - - // for stream - void* buff = NULL; - int32_t len = 0; - int32_t res = pAPI->stateStore.streamStateGetInfo(pInfo->pState, STREAM_INTERVAL_OP_CHECKPOINT_NAME, - strlen(STREAM_INTERVAL_OP_CHECKPOINT_NAME), &buff, &len); - if (res == TSDB_CODE_SUCCESS) { - doStreamIntervalDecodeOpState(buff, len, pOperator); - taosMemoryFree(buff); - } - - initIntervalDownStream(downstream, pPhyNode->type, pInfo); - code = appendDownstream(pOperator, &downstream, 1); - if (code != TSDB_CODE_SUCCESS) { - goto _error; - } - - return pOperator; - -_error: - destroyStreamFinalIntervalOperatorInfo(pInfo); - taosMemoryFreeClear(pOperator); - pTaskInfo->code = code; - return NULL; -} diff --git a/source/libs/nodes/src/nodesCloneFuncs.c b/source/libs/nodes/src/nodesCloneFuncs.c index d3cbaac5e1..afb5ce6c9a 100644 --- a/source/libs/nodes/src/nodesCloneFuncs.c +++ b/source/libs/nodes/src/nodesCloneFuncs.c @@ -399,6 +399,7 @@ static int32_t logicScanCopy(const SScanLogicNode* pSrc, SScanLogicNode* pDst) { CLONE_NODE_FIELD(pSubtable); COPY_SCALAR_FIELD(igLastNull); COPY_SCALAR_FIELD(groupOrderScan); + COPY_SCALAR_FIELD(onlyMetaCtbIdx); return TSDB_CODE_SUCCESS; } diff --git a/source/libs/parser/inc/parAst.h b/source/libs/parser/inc/parAst.h index 78422bf746..fd04b48086 100644 --- a/source/libs/parser/inc/parAst.h +++ b/source/libs/parser/inc/parAst.h @@ -143,6 +143,7 @@ SNode* addRangeClause(SAstCreateContext* pCxt, SNode* pStmt, SNode* pRange); SNode* addEveryClause(SAstCreateContext* pCxt, SNode* pStmt, SNode* pEvery); SNode* addFillClause(SAstCreateContext* pCxt, SNode* pStmt, SNode* pFill); SNode* createSelectStmt(SAstCreateContext* pCxt, bool isDistinct, SNodeList* pProjectionList, SNode* pTable); +SNode* setSelectStmtTagMode(SAstCreateContext* pCxt, SNode* pStmt, bool bSelectTags); SNode* createSetOperator(SAstCreateContext* pCxt, ESetOperatorType type, SNode* pLeft, SNode* pRight); SDataType createDataType(uint8_t type); @@ -171,8 +172,7 @@ SNode* createDropTableClause(SAstCreateContext* pCxt, bool ignoreNotExists, SNod SNode* createDropTableStmt(SAstCreateContext* pCxt, SNodeList* pTables); SNode* createDropSuperTableStmt(SAstCreateContext* pCxt, bool ignoreNotExists, SNode* pRealTable); SNode* createAlterTableModifyOptions(SAstCreateContext* pCxt, SNode* pRealTable, SNode* pOptions); -SNode* createAlterTableAddModifyCol(SAstCreateContext* pCxt, SNode* pRealTable, int8_t alterType, SToken* pColName, - SDataType dataType); +SNode* createAlterTableAddModifyCol(SAstCreateContext* pCxt, SNode* pRealTable, int8_t alterType, SNode* pColDefNode); SNode* createAlterTableDropCol(SAstCreateContext* pCxt, SNode* pRealTable, int8_t alterType, SToken* pColName); SNode* createAlterTableRenameCol(SAstCreateContext* pCxt, SNode* pRealTable, int8_t alterType, SToken* pOldColName, SToken* pNewColName); diff --git a/source/libs/parser/inc/sql.y b/source/libs/parser/inc/sql.y index 6c3f589159..c6387c2787 100755 --- a/source/libs/parser/inc/sql.y +++ b/source/libs/parser/inc/sql.y @@ -312,17 +312,17 @@ cmd ::= ALTER STABLE alter_table_clause(A). alter_table_clause(A) ::= full_table_name(B) alter_table_options(C). { A = createAlterTableModifyOptions(pCxt, B, C); } alter_table_clause(A) ::= - full_table_name(B) ADD COLUMN column_name(C) type_name(D). { A = createAlterTableAddModifyCol(pCxt, B, TSDB_ALTER_TABLE_ADD_COLUMN, &C, D); } + full_table_name(B) ADD COLUMN column_def(C). { A = createAlterTableAddModifyCol(pCxt, B, TSDB_ALTER_TABLE_ADD_COLUMN, C); } alter_table_clause(A) ::= full_table_name(B) DROP COLUMN column_name(C). { A = createAlterTableDropCol(pCxt, B, TSDB_ALTER_TABLE_DROP_COLUMN, &C); } alter_table_clause(A) ::= - full_table_name(B) MODIFY COLUMN column_name(C) type_name(D). { A = createAlterTableAddModifyCol(pCxt, B, TSDB_ALTER_TABLE_UPDATE_COLUMN_BYTES, &C, D); } + full_table_name(B) MODIFY COLUMN column_def(C). { A = createAlterTableAddModifyCol(pCxt, B, TSDB_ALTER_TABLE_UPDATE_COLUMN_BYTES, C); } alter_table_clause(A) ::= full_table_name(B) RENAME COLUMN column_name(C) column_name(D). { A = createAlterTableRenameCol(pCxt, B, TSDB_ALTER_TABLE_UPDATE_COLUMN_NAME, &C, &D); } alter_table_clause(A) ::= - full_table_name(B) ADD TAG column_name(C) type_name(D). { A = createAlterTableAddModifyCol(pCxt, B, TSDB_ALTER_TABLE_ADD_TAG, &C, D); } + full_table_name(B) ADD TAG column_def(C). { A = createAlterTableAddModifyCol(pCxt, B, TSDB_ALTER_TABLE_ADD_TAG, C); } alter_table_clause(A) ::= full_table_name(B) DROP TAG column_name(C). { A = createAlterTableDropCol(pCxt, B, TSDB_ALTER_TABLE_DROP_TAG, &C); } alter_table_clause(A) ::= - full_table_name(B) MODIFY TAG column_name(C) type_name(D). { A = createAlterTableAddModifyCol(pCxt, B, TSDB_ALTER_TABLE_UPDATE_TAG_BYTES, &C, D); } + full_table_name(B) MODIFY TAG column_def(C). { A = createAlterTableAddModifyCol(pCxt, B, TSDB_ALTER_TABLE_UPDATE_TAG_BYTES, C); } alter_table_clause(A) ::= full_table_name(B) RENAME TAG column_name(C) column_name(D). { A = createAlterTableRenameCol(pCxt, B, TSDB_ALTER_TABLE_UPDATE_TAG_NAME, &C, &D); } alter_table_clause(A) ::= @@ -358,7 +358,7 @@ column_def_list(A) ::= column_def(B). column_def_list(A) ::= column_def_list(B) NK_COMMA column_def(C). { A = addNodeToList(pCxt, B, C); } column_def(A) ::= column_name(B) type_name(C). { A = createColumnDefNode(pCxt, &B, C, NULL); } -//column_def(A) ::= column_name(B) type_name(C) COMMENT NK_STRING(D). { A = createColumnDefNode(pCxt, &B, C, &D); } +column_def(A) ::= column_name(B) type_name(C) COMMENT NK_STRING(D). { A = createColumnDefNode(pCxt, &B, C, &D); } %type type_name { SDataType } %destructor type_name { } @@ -475,8 +475,8 @@ cmd ::= SHOW TAGS FROM table_name_cond(A) from_db_opt(B). cmd ::= SHOW TAGS FROM db_name(B) NK_DOT table_name(A). { pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_TAGS_STMT, createIdentifierValueNode(pCxt, &B), createIdentifierValueNode(pCxt, &A), OP_TYPE_EQUAL); } cmd ::= SHOW TABLE TAGS tag_list_opt(C) FROM table_name_cond(A) from_db_opt(B). { pCxt->pRootNode = createShowTableTagsStmt(pCxt, A, B, C); } cmd ::= SHOW TABLE TAGS tag_list_opt(C) FROM db_name(B) NK_DOT table_name(A). { pCxt->pRootNode = createShowTableTagsStmt(pCxt, createIdentifierValueNode(pCxt, &A), createIdentifierValueNode(pCxt, &B), C); } -cmd ::= SHOW VNODES NK_INTEGER(A). { pCxt->pRootNode = createShowVnodesStmt(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &A), NULL); } -cmd ::= SHOW VNODES NK_STRING(A). { pCxt->pRootNode = createShowVnodesStmt(pCxt, NULL, createValueNode(pCxt, TSDB_DATA_TYPE_VARCHAR, &A)); } +cmd ::= SHOW VNODES ON DNODE NK_INTEGER(A). { pCxt->pRootNode = createShowVnodesStmt(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &A), NULL); } +cmd ::= SHOW VNODES. { pCxt->pRootNode = createShowVnodesStmt(pCxt, NULL, NULL); } // show alive cmd ::= SHOW db_name_cond_opt(A) ALIVE. { pCxt->pRootNode = createShowAliveStmt(pCxt, A, QUERY_NODE_SHOW_DB_ALIVE_STMT); } cmd ::= SHOW CLUSTER ALIVE. { pCxt->pRootNode = createShowAliveStmt(pCxt, NULL, QUERY_NODE_SHOW_CLUSTER_ALIVE_STMT); } @@ -1009,10 +1009,11 @@ join_type(A) ::= INNER. /************************************************ query_specification *************************************************/ query_specification(A) ::= - SELECT set_quantifier_opt(B) select_list(C) from_clause_opt(D) - where_clause_opt(E) partition_by_clause_opt(F) range_opt(J) every_opt(K) - fill_opt(L) twindow_clause_opt(G) group_by_clause_opt(H) having_clause_opt(I). { + SELECT tag_mode_opt(M) set_quantifier_opt(B) select_list(C) from_clause_opt(D) + where_clause_opt(E) partition_by_clause_opt(F) range_opt(J) every_opt(K) + fill_opt(L) twindow_clause_opt(G) group_by_clause_opt(H) having_clause_opt(I). { A = createSelectStmt(pCxt, B, C, D); + A = setSelectStmtTagMode(pCxt, A, M); A = addWhereClause(pCxt, A, E); A = addPartitionByClause(pCxt, A, F); A = addWindowClauseClause(pCxt, A, G); @@ -1023,6 +1024,11 @@ query_specification(A) ::= A = addFillClause(pCxt, A, L); } +%type tag_mode_opt { bool } +%destructor tag_mode_opt { } +tag_mode_opt(A) ::= . { A = false; } +tag_mode_opt(A) ::= TAGS. { A = true; } + %type set_quantifier_opt { bool } %destructor set_quantifier_opt { } set_quantifier_opt(A) ::= . { A = false; } diff --git a/source/libs/parser/src/parAstCreater.c b/source/libs/parser/src/parAstCreater.c index f85218c50a..3c31aae573 100644 --- a/source/libs/parser/src/parAstCreater.c +++ b/source/libs/parser/src/parAstCreater.c @@ -852,6 +852,13 @@ SNode* createSelectStmt(SAstCreateContext* pCxt, bool isDistinct, SNodeList* pPr return select; } +SNode* setSelectStmtTagMode(SAstCreateContext* pCxt, SNode* pStmt, bool bSelectTags) { + if (pStmt && QUERY_NODE_SELECT_STMT == nodeType(pStmt)) { + ((SSelectStmt*)pStmt)->tagScan = bSelectTags; + } + return pStmt; +} + static void setSubquery(SNode* pStmt) { if (QUERY_NODE_SELECT_STMT == nodeType(pStmt)) { ((SSelectStmt*)pStmt)->isSubquery = true; @@ -1333,17 +1340,15 @@ SNode* createAlterTableModifyOptions(SAstCreateContext* pCxt, SNode* pRealTable, return createAlterTableStmtFinalize(pRealTable, pStmt); } -SNode* createAlterTableAddModifyCol(SAstCreateContext* pCxt, SNode* pRealTable, int8_t alterType, SToken* pColName, - SDataType dataType) { +SNode* createAlterTableAddModifyCol(SAstCreateContext* pCxt, SNode* pRealTable, int8_t alterType, SNode* pColDefNode) { CHECK_PARSER_STATUS(pCxt); - if (!checkColumnName(pCxt, pColName)) { - return NULL; - } + SColumnDefNode* pCol = (SColumnDefNode*)pColDefNode; SAlterTableStmt* pStmt = (SAlterTableStmt*)nodesMakeNode(QUERY_NODE_ALTER_TABLE_STMT); CHECK_OUT_OF_MEM(pStmt); pStmt->alterType = alterType; - COPY_STRING_FORM_ID_TOKEN(pStmt->colName, pColName); - pStmt->dataType = dataType; + strcpy(pStmt->colName, pCol->colName); + strcpy(pStmt->colComment, pCol->comments); + pStmt->dataType = pCol->dataType; return createAlterTableStmtFinalize(pRealTable, pStmt); } diff --git a/source/libs/parser/src/parTranslater.c b/source/libs/parser/src/parTranslater.c index 3e8aeb0b9f..ea33103798 100644 --- a/source/libs/parser/src/parTranslater.c +++ b/source/libs/parser/src/parTranslater.c @@ -4697,6 +4697,7 @@ static int32_t columnDefNodeToField(SNodeList* pList, SArray** pArray) { SColumnDefNode* pCol = (SColumnDefNode*)pNode; SField field = {.type = pCol->dataType.type, .bytes = calcTypeBytes(pCol->dataType)}; strcpy(field.name, pCol->colName); + strcpy(field.comment, pCol->comments); if (pCol->sma) { field.flags |= COL_SMA_ON; } @@ -5044,6 +5045,7 @@ static void toSchema(const SColumnDefNode* pCol, col_id_t colId, SSchema* pSchem pSchema->bytes = calcTypeBytes(pCol->dataType); pSchema->flags = flags; strcpy(pSchema->name, pCol->colName); + strcpy(pSchema->comment, pCol->comments); } typedef struct SSampleAstInfo { @@ -7692,6 +7694,10 @@ static int32_t extractDescribeResultSchema(int32_t* numOfCols, SSchema** pSchema (*pSchema)[3].bytes = DESCRIBE_RESULT_NOTE_LEN; strcpy((*pSchema)[3].name, "note"); + (*pSchema)[4].type = TSDB_DATA_TYPE_BINARY; + (*pSchema)[4].bytes = DESCRIBE_RESULT_COL_COMMENT_LEN; + strcpy((*pSchema)[4].name, "comment"); + return TSDB_CODE_SUCCESS; } @@ -8067,8 +8073,6 @@ static int32_t rewriteShowVnodes(STranslateContext* pCxt, SQuery* pQuery) { if (TSDB_CODE_SUCCESS == code) { if (NULL != pShow->pDnodeId) { code = createOperatorNode(OP_TYPE_EQUAL, "dnode_id", pShow->pDnodeId, &pStmt->pWhere); - } else { - code = createOperatorNode(OP_TYPE_EQUAL, "dnode_ep", pShow->pDnodeEndpoint, &pStmt->pWhere); } } if (TSDB_CODE_SUCCESS == code) { @@ -8874,6 +8878,15 @@ static int32_t buildAddColReq(STranslateContext* pCxt, SAlterTableStmt* pStmt, S pReq->type = pStmt->dataType.type; pReq->flags = COL_SMA_ON; pReq->bytes = calcTypeBytes(pStmt->dataType); + if (pStmt->colComment[0]) { + pReq->colComment = taosStrdup(pStmt->colComment); + if (pReq->colComment == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + pReq->colCommentLen = strlen(pReq->colComment); + } else { + pReq->colCommentLen = -1; + } return TSDB_CODE_SUCCESS; } @@ -8924,6 +8937,15 @@ static int32_t buildUpdateColReq(STranslateContext* pCxt, SAlterTableStmt* pStmt return TSDB_CODE_OUT_OF_MEMORY; } pReq->colId = pSchema->colId; + if (pStmt->colComment[0]) { + pReq->colComment = taosStrdup(pStmt->colComment); + if (pReq->colComment == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + pReq->colCommentLen = strlen(pReq->colComment); + } else { + pReq->colCommentLen = -1; + } return TSDB_CODE_SUCCESS; } diff --git a/source/libs/parser/src/sql.c b/source/libs/parser/src/sql.c index a912fb4e71..d1fe6d9091 100644 --- a/source/libs/parser/src/sql.c +++ b/source/libs/parser/src/sql.c @@ -1,3 +1,5 @@ +/* This file is automatically generated by Lemon from input grammar +** source file "sql.y". */ /* ** 2000-05-29 ** @@ -22,9 +24,8 @@ ** The following is the concatenation of all %include directives from the ** input grammar file: */ -#include -#include /************ Begin %include sections from the grammar ************************/ +#line 11 "sql.y" #include #include @@ -41,12 +42,350 @@ #include "parAst.h" #define YYSTACKDEPTH 0 +#line 46 "sql.c" /**************** End of %include directives **********************************/ -/* These constants specify the various numeric values for terminal symbols -** in a format understandable to "makeheaders". This section is blank unless -** "lemon" is run with the "-m" command-line option. -***************** Begin makeheaders token definitions *************************/ -/**************** End makeheaders token definitions ***************************/ +/* These constants specify the various numeric values for terminal symbols. +***************** Begin token definitions *************************************/ +#ifndef TK_OR +#define TK_OR 1 +#define TK_AND 2 +#define TK_UNION 3 +#define TK_ALL 4 +#define TK_MINUS 5 +#define TK_EXCEPT 6 +#define TK_INTERSECT 7 +#define TK_NK_BITAND 8 +#define TK_NK_BITOR 9 +#define TK_NK_LSHIFT 10 +#define TK_NK_RSHIFT 11 +#define TK_NK_PLUS 12 +#define TK_NK_MINUS 13 +#define TK_NK_STAR 14 +#define TK_NK_SLASH 15 +#define TK_NK_REM 16 +#define TK_NK_CONCAT 17 +#define TK_CREATE 18 +#define TK_ACCOUNT 19 +#define TK_NK_ID 20 +#define TK_PASS 21 +#define TK_NK_STRING 22 +#define TK_ALTER 23 +#define TK_PPS 24 +#define TK_TSERIES 25 +#define TK_STORAGE 26 +#define TK_STREAMS 27 +#define TK_QTIME 28 +#define TK_DBS 29 +#define TK_USERS 30 +#define TK_CONNS 31 +#define TK_STATE 32 +#define TK_USER 33 +#define TK_ENABLE 34 +#define TK_NK_INTEGER 35 +#define TK_SYSINFO 36 +#define TK_DROP 37 +#define TK_GRANT 38 +#define TK_ON 39 +#define TK_TO 40 +#define TK_REVOKE 41 +#define TK_FROM 42 +#define TK_SUBSCRIBE 43 +#define TK_NK_COMMA 44 +#define TK_READ 45 +#define TK_WRITE 46 +#define TK_NK_DOT 47 +#define TK_WITH 48 +#define TK_DNODE 49 +#define TK_PORT 50 +#define TK_DNODES 51 +#define TK_RESTORE 52 +#define TK_NK_IPTOKEN 53 +#define TK_FORCE 54 +#define TK_UNSAFE 55 +#define TK_LOCAL 56 +#define TK_QNODE 57 +#define TK_BNODE 58 +#define TK_SNODE 59 +#define TK_MNODE 60 +#define TK_VNODE 61 +#define TK_DATABASE 62 +#define TK_USE 63 +#define TK_FLUSH 64 +#define TK_TRIM 65 +#define TK_COMPACT 66 +#define TK_IF 67 +#define TK_NOT 68 +#define TK_EXISTS 69 +#define TK_BUFFER 70 +#define TK_CACHEMODEL 71 +#define TK_CACHESIZE 72 +#define TK_COMP 73 +#define TK_DURATION 74 +#define TK_NK_VARIABLE 75 +#define TK_MAXROWS 76 +#define TK_MINROWS 77 +#define TK_KEEP 78 +#define TK_PAGES 79 +#define TK_PAGESIZE 80 +#define TK_TSDB_PAGESIZE 81 +#define TK_PRECISION 82 +#define TK_REPLICA 83 +#define TK_VGROUPS 84 +#define TK_SINGLE_STABLE 85 +#define TK_RETENTIONS 86 +#define TK_SCHEMALESS 87 +#define TK_WAL_LEVEL 88 +#define TK_WAL_FSYNC_PERIOD 89 +#define TK_WAL_RETENTION_PERIOD 90 +#define TK_WAL_RETENTION_SIZE 91 +#define TK_WAL_ROLL_PERIOD 92 +#define TK_WAL_SEGMENT_SIZE 93 +#define TK_STT_TRIGGER 94 +#define TK_TABLE_PREFIX 95 +#define TK_TABLE_SUFFIX 96 +#define TK_NK_COLON 97 +#define TK_MAX_SPEED 98 +#define TK_START 99 +#define TK_TIMESTAMP 100 +#define TK_END 101 +#define TK_TABLE 102 +#define TK_NK_LP 103 +#define TK_NK_RP 104 +#define TK_STABLE 105 +#define TK_ADD 106 +#define TK_COLUMN 107 +#define TK_MODIFY 108 +#define TK_RENAME 109 +#define TK_TAG 110 +#define TK_SET 111 +#define TK_NK_EQ 112 +#define TK_USING 113 +#define TK_TAGS 114 +#define TK_COMMENT 115 +#define TK_BOOL 116 +#define TK_TINYINT 117 +#define TK_SMALLINT 118 +#define TK_INT 119 +#define TK_INTEGER 120 +#define TK_BIGINT 121 +#define TK_FLOAT 122 +#define TK_DOUBLE 123 +#define TK_BINARY 124 +#define TK_NCHAR 125 +#define TK_UNSIGNED 126 +#define TK_JSON 127 +#define TK_VARCHAR 128 +#define TK_MEDIUMBLOB 129 +#define TK_BLOB 130 +#define TK_VARBINARY 131 +#define TK_GEOMETRY 132 +#define TK_DECIMAL 133 +#define TK_MAX_DELAY 134 +#define TK_WATERMARK 135 +#define TK_ROLLUP 136 +#define TK_TTL 137 +#define TK_SMA 138 +#define TK_DELETE_MARK 139 +#define TK_FIRST 140 +#define TK_LAST 141 +#define TK_SHOW 142 +#define TK_PRIVILEGES 143 +#define TK_DATABASES 144 +#define TK_TABLES 145 +#define TK_STABLES 146 +#define TK_MNODES 147 +#define TK_QNODES 148 +#define TK_FUNCTIONS 149 +#define TK_INDEXES 150 +#define TK_ACCOUNTS 151 +#define TK_APPS 152 +#define TK_CONNECTIONS 153 +#define TK_LICENCES 154 +#define TK_GRANTS 155 +#define TK_QUERIES 156 +#define TK_SCORES 157 +#define TK_TOPICS 158 +#define TK_VARIABLES 159 +#define TK_CLUSTER 160 +#define TK_BNODES 161 +#define TK_SNODES 162 +#define TK_TRANSACTIONS 163 +#define TK_DISTRIBUTED 164 +#define TK_CONSUMERS 165 +#define TK_SUBSCRIPTIONS 166 +#define TK_VNODES 167 +#define TK_ALIVE 168 +#define TK_LIKE 169 +#define TK_TBNAME 170 +#define TK_QTAGS 171 +#define TK_AS 172 +#define TK_INDEX 173 +#define TK_FUNCTION 174 +#define TK_INTERVAL 175 +#define TK_COUNT 176 +#define TK_LAST_ROW 177 +#define TK_META 178 +#define TK_ONLY 179 +#define TK_TOPIC 180 +#define TK_CONSUMER 181 +#define TK_GROUP 182 +#define TK_DESC 183 +#define TK_DESCRIBE 184 +#define TK_RESET 185 +#define TK_QUERY 186 +#define TK_CACHE 187 +#define TK_EXPLAIN 188 +#define TK_ANALYZE 189 +#define TK_VERBOSE 190 +#define TK_NK_BOOL 191 +#define TK_RATIO 192 +#define TK_NK_FLOAT 193 +#define TK_OUTPUTTYPE 194 +#define TK_AGGREGATE 195 +#define TK_BUFSIZE 196 +#define TK_LANGUAGE 197 +#define TK_REPLACE 198 +#define TK_STREAM 199 +#define TK_INTO 200 +#define TK_PAUSE 201 +#define TK_RESUME 202 +#define TK_TRIGGER 203 +#define TK_AT_ONCE 204 +#define TK_WINDOW_CLOSE 205 +#define TK_IGNORE 206 +#define TK_EXPIRED 207 +#define TK_FILL_HISTORY 208 +#define TK_UPDATE 209 +#define TK_SUBTABLE 210 +#define TK_UNTREATED 211 +#define TK_KILL 212 +#define TK_CONNECTION 213 +#define TK_TRANSACTION 214 +#define TK_BALANCE 215 +#define TK_VGROUP 216 +#define TK_LEADER 217 +#define TK_MERGE 218 +#define TK_REDISTRIBUTE 219 +#define TK_SPLIT 220 +#define TK_DELETE 221 +#define TK_INSERT 222 +#define TK_NULL 223 +#define TK_NK_QUESTION 224 +#define TK_NK_ARROW 225 +#define TK_ROWTS 226 +#define TK_QSTART 227 +#define TK_QEND 228 +#define TK_QDURATION 229 +#define TK_WSTART 230 +#define TK_WEND 231 +#define TK_WDURATION 232 +#define TK_IROWTS 233 +#define TK_ISFILLED 234 +#define TK_CAST 235 +#define TK_NOW 236 +#define TK_TODAY 237 +#define TK_TIMEZONE 238 +#define TK_CLIENT_VERSION 239 +#define TK_SERVER_VERSION 240 +#define TK_SERVER_STATUS 241 +#define TK_CURRENT_USER 242 +#define TK_CASE 243 +#define TK_WHEN 244 +#define TK_THEN 245 +#define TK_ELSE 246 +#define TK_BETWEEN 247 +#define TK_IS 248 +#define TK_NK_LT 249 +#define TK_NK_GT 250 +#define TK_NK_LE 251 +#define TK_NK_GE 252 +#define TK_NK_NE 253 +#define TK_MATCH 254 +#define TK_NMATCH 255 +#define TK_CONTAINS 256 +#define TK_IN 257 +#define TK_JOIN 258 +#define TK_INNER 259 +#define TK_SELECT 260 +#define TK_DISTINCT 261 +#define TK_WHERE 262 +#define TK_PARTITION 263 +#define TK_BY 264 +#define TK_SESSION 265 +#define TK_STATE_WINDOW 266 +#define TK_EVENT_WINDOW 267 +#define TK_SLIDING 268 +#define TK_FILL 269 +#define TK_VALUE 270 +#define TK_VALUE_F 271 +#define TK_NONE 272 +#define TK_PREV 273 +#define TK_NULL_F 274 +#define TK_LINEAR 275 +#define TK_NEXT 276 +#define TK_HAVING 277 +#define TK_RANGE 278 +#define TK_EVERY 279 +#define TK_ORDER 280 +#define TK_SLIMIT 281 +#define TK_SOFFSET 282 +#define TK_LIMIT 283 +#define TK_OFFSET 284 +#define TK_ASC 285 +#define TK_NULLS 286 +#define TK_ABORT 287 +#define TK_AFTER 288 +#define TK_ATTACH 289 +#define TK_BEFORE 290 +#define TK_BEGIN 291 +#define TK_BITAND 292 +#define TK_BITNOT 293 +#define TK_BITOR 294 +#define TK_BLOCKS 295 +#define TK_CHANGE 296 +#define TK_COMMA 297 +#define TK_CONCAT 298 +#define TK_CONFLICT 299 +#define TK_COPY 300 +#define TK_DEFERRED 301 +#define TK_DELIMITERS 302 +#define TK_DETACH 303 +#define TK_DIVIDE 304 +#define TK_DOT 305 +#define TK_EACH 306 +#define TK_FAIL 307 +#define TK_FILE 308 +#define TK_FOR 309 +#define TK_GLOB 310 +#define TK_ID 311 +#define TK_IMMEDIATE 312 +#define TK_IMPORT 313 +#define TK_INITIALLY 314 +#define TK_INSTEAD 315 +#define TK_ISNULL 316 +#define TK_KEY 317 +#define TK_MODULES 318 +#define TK_NK_BITNOT 319 +#define TK_NK_SEMI 320 +#define TK_NOTNULL 321 +#define TK_OF 322 +#define TK_PLUS 323 +#define TK_PRIVILEGE 324 +#define TK_RAISE 325 +#define TK_RESTRICT 326 +#define TK_ROW 327 +#define TK_SEMI 328 +#define TK_STAR 329 +#define TK_STATEMENT 330 +#define TK_STRICT 331 +#define TK_STRING 332 +#define TK_TIMES 333 +#define TK_VALUES 334 +#define TK_VARIABLE 335 +#define TK_VIEW 336 +#define TK_WAL 337 +#endif +/**************** End token definitions ***************************************/ /* The next sections is a series of control #defines. ** various aspects of the generated parser. @@ -104,27 +443,27 @@ #endif /************* Begin control #defines *****************************************/ #define YYCODETYPE unsigned short int -#define YYNOCODE 487 +#define YYNOCODE 488 #define YYACTIONTYPE unsigned short int #define ParseTOKENTYPE SToken typedef union { int yyinit; ParseTOKENTYPE yy0; - EJoinType yy140; - SDataType yy310; - STokenPair yy347; - EOperatorType yy354; - SAlterOption yy365; - SToken yy371; - ENullOrder yy399; - int32_t yy416; - SNode* yy452; - int8_t yy475; - bool yy667; - EOrder yy690; - int64_t yy729; - SNodeList* yy812; - EFillMode yy844; + SNodeList* yy56; + EOperatorType yy76; + EOrder yy82; + SToken yy129; + int8_t yy215; + bool yy313; + int32_t yy356; + SAlterOption yy461; + STokenPair yy505; + ENullOrder yy521; + EFillMode yy694; + EJoinType yy708; + int64_t yy717; + SDataType yy784; + SNode* yy840; } YYMINORTYPE; #ifndef YYSTACKDEPTH #define YYSTACKDEPTH 100 @@ -141,17 +480,17 @@ typedef union { #define ParseCTX_STORE #define YYFALLBACK 1 #define YYNSTATE 800 -#define YYNRULE 600 -#define YYNRULE_WITH_ACTION 600 +#define YYNRULE 603 +#define YYNRULE_WITH_ACTION 603 #define YYNTOKEN 338 #define YY_MAX_SHIFT 799 -#define YY_MIN_SHIFTREDUCE 1180 -#define YY_MAX_SHIFTREDUCE 1779 -#define YY_ERROR_ACTION 1780 -#define YY_ACCEPT_ACTION 1781 -#define YY_NO_ACTION 1782 -#define YY_MIN_REDUCE 1783 -#define YY_MAX_REDUCE 2382 +#define YY_MIN_SHIFTREDUCE 1181 +#define YY_MAX_SHIFTREDUCE 1783 +#define YY_ERROR_ACTION 1784 +#define YY_ACCEPT_ACTION 1785 +#define YY_NO_ACTION 1786 +#define YY_MIN_REDUCE 1787 +#define YY_MAX_REDUCE 2389 /************* End control #defines *******************************************/ #define YY_NLOOKAHEAD ((int)(sizeof(yy_lookahead)/sizeof(yy_lookahead[0]))) @@ -218,599 +557,616 @@ typedef union { ** yy_default[] Default action for each state. ** *********** Begin parsing tables **********************************************/ -#define YY_ACTTAB_COUNT (2858) +#define YY_ACTTAB_COUNT (3028) static const YYACTIONTYPE yy_action[] = { - /* 0 */ 2126, 2193, 2171, 2082, 221, 690, 1960, 2154, 537, 1949, - /* 10 */ 1826, 670, 48, 46, 1706, 394, 2179, 1213, 2079, 677, - /* 20 */ 401, 2358, 1555, 41, 40, 135, 2175, 47, 45, 44, - /* 30 */ 43, 42, 574, 1636, 453, 1553, 2171, 539, 1582, 2211, - /* 40 */ 41, 40, 1781, 536, 47, 45, 44, 43, 42, 254, - /* 50 */ 1951, 2161, 1945, 706, 630, 531, 1215, 2353, 1218, 1219, - /* 60 */ 2175, 181, 1631, 529, 2177, 398, 525, 521, 19, 1238, - /* 70 */ 66, 1237, 2359, 188, 700, 1561, 30, 2354, 656, 348, - /* 80 */ 689, 369, 2065, 361, 140, 690, 1960, 2192, 1580, 2228, - /* 90 */ 667, 144, 112, 2194, 710, 2196, 2197, 705, 2177, 700, - /* 100 */ 796, 168, 1239, 15, 185, 135, 2281, 103, 700, 1901, - /* 110 */ 397, 2277, 579, 497, 2082, 416, 48, 46, 690, 1960, - /* 120 */ 415, 689, 1769, 190, 401, 264, 1555, 1665, 1371, 2080, - /* 130 */ 677, 2307, 1953, 1580, 38, 306, 1746, 1636, 193, 1553, - /* 140 */ 1638, 1639, 1806, 1362, 735, 734, 733, 1366, 732, 1368, - /* 150 */ 1369, 731, 728, 1805, 1377, 725, 1379, 1380, 722, 719, - /* 160 */ 716, 184, 630, 51, 655, 2353, 1631, 2353, 94, 62, - /* 170 */ 1611, 1621, 19, 2000, 212, 211, 1637, 1640, 675, 1561, - /* 180 */ 2359, 188, 654, 188, 1666, 2354, 656, 2354, 656, 2296, - /* 190 */ 288, 1556, 2161, 1554, 286, 2289, 666, 496, 136, 665, - /* 200 */ 169, 2353, 1795, 2161, 796, 41, 40, 15, 2193, 47, - /* 210 */ 45, 44, 43, 42, 62, 2293, 654, 188, 707, 1315, - /* 220 */ 435, 2354, 656, 1559, 1560, 1783, 1610, 1613, 1614, 1615, - /* 230 */ 1616, 1617, 1618, 1619, 1620, 702, 698, 1629, 1630, 1632, - /* 240 */ 1633, 1634, 1635, 2, 1638, 1639, 2211, 437, 433, 134, - /* 250 */ 133, 132, 131, 130, 129, 128, 127, 126, 2161, 1317, - /* 260 */ 706, 1784, 37, 399, 1660, 1661, 1662, 1663, 1664, 1668, - /* 270 */ 1669, 1670, 1671, 534, 1611, 1621, 535, 1819, 551, 1580, - /* 280 */ 1637, 1640, 125, 1464, 1465, 124, 123, 122, 121, 120, - /* 290 */ 119, 118, 117, 116, 2192, 1556, 2228, 1554, 645, 112, - /* 300 */ 2194, 710, 2196, 2197, 705, 650, 700, 2044, 395, 147, - /* 310 */ 1580, 151, 2252, 2281, 1581, 2193, 166, 397, 2277, 1238, - /* 320 */ 191, 1237, 667, 144, 1962, 670, 191, 1559, 1560, 1703, - /* 330 */ 1610, 1613, 1614, 1615, 1616, 1617, 1618, 1619, 1620, 702, - /* 340 */ 698, 1629, 1630, 1632, 1633, 1634, 1635, 2, 12, 48, - /* 350 */ 46, 745, 1239, 2211, 2013, 410, 409, 401, 2182, 1555, - /* 360 */ 2358, 367, 62, 2353, 181, 2161, 1582, 706, 615, 2011, - /* 370 */ 1636, 191, 1553, 592, 591, 590, 690, 1960, 1562, 2357, - /* 380 */ 582, 141, 586, 2354, 2356, 2066, 585, 649, 651, 646, - /* 390 */ 639, 584, 589, 377, 376, 146, 56, 583, 2252, 1631, - /* 400 */ 252, 2192, 629, 2228, 251, 19, 112, 2194, 710, 2196, - /* 410 */ 2197, 705, 1561, 700, 2184, 2211, 297, 298, 185, 542, - /* 420 */ 2281, 296, 535, 1819, 397, 2277, 187, 2289, 2290, 2193, - /* 430 */ 142, 2294, 1221, 655, 1406, 1407, 2353, 796, 1579, 707, - /* 440 */ 15, 1828, 41, 40, 1272, 2308, 47, 45, 44, 43, - /* 450 */ 42, 654, 188, 48, 46, 1641, 2354, 656, 223, 2193, - /* 460 */ 1612, 401, 537, 1555, 1826, 93, 475, 2211, 356, 707, - /* 470 */ 648, 381, 166, 608, 1636, 474, 1553, 1638, 1639, 2161, - /* 480 */ 1963, 706, 2358, 125, 1273, 2353, 124, 123, 122, 121, - /* 490 */ 120, 119, 118, 117, 116, 2013, 62, 2211, 1804, 667, - /* 500 */ 144, 2357, 382, 1631, 12, 2354, 2355, 1611, 1621, 2161, - /* 510 */ 2011, 706, 109, 1637, 1640, 2192, 1561, 2228, 288, 191, - /* 520 */ 112, 2194, 710, 2196, 2197, 705, 60, 700, 1556, 145, - /* 530 */ 1554, 578, 2373, 627, 2281, 577, 1565, 1952, 397, 2277, - /* 540 */ 1710, 796, 690, 1960, 49, 2192, 1580, 2228, 2161, 2193, - /* 550 */ 170, 2194, 710, 2196, 2197, 705, 12, 700, 10, 707, - /* 560 */ 1559, 1560, 451, 1610, 1613, 1614, 1615, 1616, 1617, 1618, - /* 570 */ 1619, 1620, 702, 698, 1629, 1630, 1632, 1633, 1634, 1635, - /* 580 */ 2, 1638, 1639, 445, 1326, 444, 1702, 2211, 41, 40, - /* 590 */ 631, 2318, 47, 45, 44, 43, 42, 1325, 2296, 2161, - /* 600 */ 1583, 706, 669, 186, 2289, 2290, 165, 142, 2294, 1561, - /* 610 */ 1803, 1611, 1621, 690, 1960, 443, 406, 1637, 1640, 2006, - /* 620 */ 2008, 41, 40, 404, 2292, 47, 45, 44, 43, 42, - /* 630 */ 2296, 163, 1556, 452, 1554, 2192, 676, 2228, 383, 1962, - /* 640 */ 112, 2194, 710, 2196, 2197, 705, 2011, 700, 253, 689, - /* 650 */ 447, 2193, 2256, 191, 2281, 446, 2291, 2155, 397, 2277, - /* 660 */ 2161, 707, 506, 2315, 1559, 1560, 154, 1610, 1613, 1614, - /* 670 */ 1615, 1616, 1617, 1618, 1619, 1620, 702, 698, 1629, 1630, - /* 680 */ 1632, 1633, 1634, 1635, 2, 48, 46, 1937, 549, 2211, - /* 690 */ 2075, 457, 2061, 401, 755, 1555, 1612, 630, 667, 144, - /* 700 */ 2353, 2161, 2013, 706, 630, 1936, 1636, 2353, 1553, 391, - /* 710 */ 47, 45, 44, 43, 42, 2359, 188, 2011, 1526, 1527, - /* 720 */ 2354, 656, 2359, 188, 463, 2061, 55, 2354, 656, 14, - /* 730 */ 13, 1734, 51, 690, 1960, 1631, 1947, 2192, 202, 2228, - /* 740 */ 597, 676, 112, 2194, 710, 2196, 2197, 705, 1561, 700, - /* 750 */ 690, 1960, 266, 465, 2373, 607, 2281, 1862, 41, 40, - /* 760 */ 397, 2277, 47, 45, 44, 43, 42, 2193, 1802, 250, - /* 770 */ 480, 205, 745, 796, 690, 1960, 49, 704, 642, 641, - /* 780 */ 1732, 1733, 1735, 1736, 1737, 600, 490, 2061, 2357, 48, - /* 790 */ 46, 1935, 594, 674, 481, 2075, 1776, 401, 249, 1555, - /* 800 */ 404, 1581, 189, 2289, 2290, 2211, 142, 2294, 166, 692, - /* 810 */ 1636, 2253, 1553, 1638, 1639, 1846, 1962, 2161, 2161, 706, - /* 820 */ 743, 156, 155, 740, 739, 738, 153, 592, 591, 590, - /* 830 */ 2007, 2008, 1679, 210, 582, 141, 586, 593, 70, 1631, - /* 840 */ 585, 69, 52, 1611, 1621, 584, 589, 377, 376, 1637, - /* 850 */ 1640, 583, 1561, 2192, 1745, 2228, 1241, 1242, 342, 2194, - /* 860 */ 710, 2196, 2197, 705, 1556, 700, 1554, 2247, 41, 40, - /* 870 */ 1801, 1943, 47, 45, 44, 43, 42, 796, 570, 569, - /* 880 */ 15, 2193, 743, 156, 155, 740, 739, 738, 153, 1800, - /* 890 */ 613, 707, 207, 2328, 1799, 1964, 1559, 1560, 1775, 1610, - /* 900 */ 1613, 1614, 1615, 1616, 1617, 1618, 1619, 1620, 702, 698, - /* 910 */ 1629, 1630, 1632, 1633, 1634, 1635, 2, 1638, 1639, 2211, - /* 920 */ 2161, 1481, 1482, 606, 743, 156, 155, 740, 739, 738, - /* 930 */ 153, 2161, 86, 706, 2013, 85, 604, 630, 602, 2161, - /* 940 */ 2353, 396, 690, 1960, 2161, 690, 1960, 1611, 1621, 2011, - /* 950 */ 736, 690, 1960, 1637, 1640, 2359, 188, 1480, 1483, 259, - /* 960 */ 2354, 656, 550, 737, 1722, 1957, 2004, 2192, 1556, 2228, - /* 970 */ 1554, 255, 112, 2194, 710, 2196, 2197, 705, 9, 700, - /* 980 */ 588, 587, 34, 2147, 2373, 1330, 2281, 676, 41, 40, - /* 990 */ 397, 2277, 47, 45, 44, 43, 42, 1798, 1329, 191, - /* 1000 */ 1559, 1560, 1859, 1610, 1613, 1614, 1615, 1616, 1617, 1618, - /* 1010 */ 1619, 1620, 702, 698, 1629, 1630, 1632, 1633, 1634, 1635, - /* 1020 */ 2, 1646, 352, 167, 1578, 690, 1960, 1580, 327, 690, - /* 1030 */ 1960, 488, 423, 90, 504, 572, 571, 503, 2171, 685, - /* 1040 */ 407, 2075, 324, 73, 2144, 263, 72, 2161, 166, 673, - /* 1050 */ 371, 2013, 2180, 471, 671, 505, 1962, 349, 405, 1955, - /* 1060 */ 473, 701, 2175, 508, 1797, 658, 2011, 1794, 219, 516, - /* 1070 */ 514, 511, 773, 772, 771, 770, 413, 1938, 769, 768, - /* 1080 */ 148, 763, 762, 761, 760, 759, 758, 757, 158, 753, - /* 1090 */ 752, 751, 412, 411, 748, 747, 746, 176, 175, 1583, - /* 1100 */ 2177, 630, 690, 1960, 2353, 1583, 370, 237, 62, 374, - /* 1110 */ 700, 44, 43, 42, 2161, 690, 1960, 2161, 459, 2359, - /* 1120 */ 188, 149, 301, 173, 2354, 656, 690, 1960, 262, 690, - /* 1130 */ 1960, 568, 564, 560, 556, 687, 236, 1793, 662, 694, - /* 1140 */ 1667, 2253, 767, 765, 1792, 1791, 688, 111, 501, 307, - /* 1150 */ 1790, 495, 494, 493, 492, 487, 486, 485, 484, 483, - /* 1160 */ 479, 478, 477, 476, 351, 468, 467, 466, 659, 461, - /* 1170 */ 460, 368, 690, 1960, 2301, 1699, 91, 1612, 375, 234, - /* 1180 */ 373, 372, 1837, 576, 90, 1789, 1788, 2161, 81, 80, - /* 1190 */ 450, 2193, 408, 200, 2161, 2161, 1787, 2013, 1699, 741, - /* 1200 */ 2161, 707, 2004, 637, 595, 578, 442, 440, 1786, 577, - /* 1210 */ 1956, 742, 2012, 320, 2004, 1902, 1990, 350, 35, 54, - /* 1220 */ 431, 3, 2193, 429, 425, 421, 418, 443, 1672, 2211, - /* 1230 */ 756, 454, 707, 1922, 2346, 2161, 2161, 1218, 1219, 154, - /* 1240 */ 83, 2161, 137, 706, 455, 74, 2161, 233, 227, 580, - /* 1250 */ 242, 203, 2193, 240, 232, 547, 244, 265, 2161, 243, - /* 1260 */ 2211, 430, 707, 246, 2300, 191, 245, 248, 154, 581, - /* 1270 */ 247, 1313, 2161, 225, 706, 1835, 50, 2192, 610, 2228, - /* 1280 */ 609, 50, 112, 2194, 710, 2196, 2197, 705, 697, 700, - /* 1290 */ 2211, 1311, 1778, 1779, 2373, 84, 2281, 598, 1796, 1521, - /* 1300 */ 397, 2277, 2161, 643, 706, 1564, 270, 154, 2192, 108, - /* 1310 */ 2228, 2321, 283, 112, 2194, 710, 2196, 2197, 705, 105, - /* 1320 */ 700, 50, 294, 1563, 1555, 2373, 71, 2281, 1524, 152, - /* 1330 */ 154, 397, 2277, 14, 13, 64, 1731, 1553, 2192, 50, - /* 1340 */ 2228, 1730, 2193, 112, 2194, 710, 2196, 2197, 705, 50, - /* 1350 */ 700, 1829, 707, 410, 409, 2373, 277, 2281, 36, 714, - /* 1360 */ 139, 397, 2277, 1569, 41, 40, 272, 672, 47, 45, - /* 1370 */ 44, 43, 42, 2193, 1636, 1900, 1562, 1561, 663, 152, - /* 1380 */ 2211, 1478, 299, 707, 154, 749, 682, 1899, 2212, 303, - /* 1390 */ 1356, 750, 2161, 138, 706, 1673, 384, 152, 2070, 1622, - /* 1400 */ 414, 791, 796, 1631, 1820, 1825, 660, 1291, 2001, 319, - /* 1410 */ 2193, 2211, 2311, 1289, 668, 285, 1561, 282, 422, 1384, - /* 1420 */ 707, 1, 417, 2161, 5, 706, 365, 1586, 2192, 438, - /* 1430 */ 2228, 1502, 196, 112, 2194, 710, 2196, 2197, 705, 1388, - /* 1440 */ 700, 696, 439, 1657, 1395, 2254, 441, 2281, 2211, 195, - /* 1450 */ 198, 397, 2277, 1393, 314, 1579, 456, 157, 209, 2192, - /* 1460 */ 2161, 2228, 706, 1567, 112, 2194, 710, 2196, 2197, 705, - /* 1470 */ 1583, 700, 462, 458, 469, 499, 693, 2071, 2281, 464, - /* 1480 */ 1578, 1566, 397, 2277, 482, 491, 489, 2063, 498, 500, - /* 1490 */ 509, 510, 507, 1556, 213, 1554, 2192, 214, 2228, 2193, - /* 1500 */ 512, 113, 2194, 710, 2196, 2197, 705, 513, 700, 707, - /* 1510 */ 1584, 216, 532, 515, 517, 2281, 4, 533, 540, 2280, - /* 1520 */ 2277, 541, 543, 1581, 224, 1559, 1560, 226, 1585, 544, - /* 1530 */ 545, 1587, 1570, 546, 1565, 229, 548, 2211, 231, 88, - /* 1540 */ 89, 552, 235, 573, 355, 575, 1950, 114, 2193, 2161, - /* 1550 */ 239, 706, 1946, 612, 614, 92, 150, 618, 707, 315, - /* 1560 */ 256, 619, 617, 2135, 1573, 1575, 241, 159, 160, 1948, - /* 1570 */ 258, 260, 1944, 161, 2193, 162, 2132, 698, 1629, 1630, - /* 1580 */ 1632, 1633, 1634, 1635, 704, 2192, 2211, 2228, 625, 1509, - /* 1590 */ 113, 2194, 710, 2196, 2197, 705, 2131, 700, 2161, 644, - /* 1600 */ 706, 622, 634, 8, 2281, 2312, 2327, 680, 695, 2277, - /* 1610 */ 640, 2322, 2211, 387, 624, 2326, 647, 623, 268, 271, - /* 1620 */ 2303, 653, 276, 635, 2161, 2193, 706, 632, 633, 281, - /* 1630 */ 2376, 388, 1699, 661, 708, 707, 2228, 664, 2193, 113, - /* 1640 */ 2194, 710, 2196, 2197, 705, 143, 700, 1582, 707, 289, - /* 1650 */ 178, 1588, 278, 2281, 2076, 316, 98, 360, 2277, 2297, - /* 1660 */ 2192, 317, 2228, 2211, 678, 342, 2194, 710, 2196, 2197, - /* 1670 */ 705, 703, 700, 691, 2246, 2161, 2211, 706, 679, 683, - /* 1680 */ 2090, 279, 174, 684, 2089, 280, 100, 2193, 2161, 318, - /* 1690 */ 706, 2088, 393, 102, 61, 2262, 104, 707, 2352, 1961, - /* 1700 */ 712, 2005, 1923, 792, 2193, 284, 793, 321, 795, 2153, - /* 1710 */ 357, 2192, 310, 2228, 707, 358, 171, 2194, 710, 2196, - /* 1720 */ 2197, 705, 53, 700, 2192, 2211, 2228, 325, 323, 113, - /* 1730 */ 2194, 710, 2196, 2197, 705, 345, 700, 2161, 2152, 706, - /* 1740 */ 330, 344, 2211, 2281, 334, 2151, 78, 385, 2278, 2148, - /* 1750 */ 419, 420, 1546, 2193, 2161, 1547, 706, 194, 424, 2146, - /* 1760 */ 426, 427, 428, 707, 2145, 366, 2143, 657, 2374, 432, - /* 1770 */ 2142, 2141, 434, 2192, 436, 2228, 1537, 2193, 170, 2194, - /* 1780 */ 710, 2196, 2197, 705, 2122, 700, 197, 707, 2121, 199, - /* 1790 */ 2192, 2211, 2228, 1505, 79, 343, 2194, 710, 2196, 2197, - /* 1800 */ 705, 1504, 700, 2161, 2193, 706, 2103, 2102, 2101, 448, - /* 1810 */ 449, 2100, 2099, 2054, 707, 2211, 1455, 2053, 2050, 2319, - /* 1820 */ 386, 201, 2049, 82, 2193, 2048, 2047, 2161, 2052, 706, - /* 1830 */ 204, 2051, 2046, 2045, 707, 2043, 2042, 2041, 206, 2192, - /* 1840 */ 470, 2228, 2211, 2040, 336, 2194, 710, 2196, 2197, 705, - /* 1850 */ 472, 700, 2056, 2039, 2161, 2038, 706, 2037, 2036, 2035, - /* 1860 */ 2034, 2033, 2211, 2192, 2032, 2228, 2031, 392, 343, 2194, - /* 1870 */ 710, 2196, 2197, 705, 2161, 700, 706, 208, 2024, 2023, - /* 1880 */ 87, 2022, 2021, 2055, 2020, 2019, 215, 652, 2030, 2029, - /* 1890 */ 2192, 2193, 2228, 2028, 2027, 171, 2194, 710, 2196, 2197, - /* 1900 */ 705, 707, 700, 2026, 2025, 2018, 2017, 2016, 1457, 2015, - /* 1910 */ 2192, 502, 2228, 2014, 1327, 343, 2194, 710, 2196, 2197, - /* 1920 */ 705, 2193, 700, 353, 354, 1865, 1323, 1864, 1863, 2211, - /* 1930 */ 1331, 707, 217, 218, 400, 1861, 1858, 520, 1857, 519, - /* 1940 */ 524, 2161, 1850, 706, 523, 518, 522, 2375, 527, 526, - /* 1950 */ 1839, 528, 530, 1815, 1220, 76, 1814, 220, 2120, 2211, - /* 1960 */ 2110, 77, 182, 222, 402, 2098, 2181, 183, 538, 228, - /* 1970 */ 2097, 2161, 230, 706, 553, 554, 555, 2192, 2074, 2228, - /* 1980 */ 2193, 1939, 343, 2194, 710, 2196, 2197, 705, 616, 700, - /* 1990 */ 707, 1860, 1856, 1265, 1854, 558, 557, 1852, 559, 561, - /* 2000 */ 562, 563, 1849, 565, 566, 567, 799, 2192, 1834, 2228, - /* 2010 */ 1832, 1833, 343, 2194, 710, 2196, 2197, 705, 2211, 700, - /* 2020 */ 313, 1831, 1811, 1941, 1940, 1400, 1399, 764, 1314, 766, - /* 2030 */ 2161, 1312, 706, 1310, 1309, 1308, 180, 1847, 1307, 1301, - /* 2040 */ 1306, 63, 238, 1838, 789, 785, 781, 777, 1303, 311, - /* 2050 */ 1302, 1300, 378, 2193, 379, 1836, 380, 596, 1810, 1809, - /* 2060 */ 599, 601, 603, 707, 1808, 2119, 611, 605, 2228, 115, - /* 2070 */ 1531, 338, 2194, 710, 2196, 2197, 705, 1533, 700, 1530, - /* 2080 */ 1535, 1511, 29, 67, 2193, 1515, 2109, 2096, 1513, 110, - /* 2090 */ 164, 2211, 304, 620, 707, 2095, 2358, 20, 17, 1748, - /* 2100 */ 6, 21, 65, 2161, 31, 706, 57, 261, 7, 626, - /* 2110 */ 275, 638, 267, 621, 22, 1490, 1489, 274, 269, 2193, - /* 2120 */ 636, 172, 2211, 628, 1729, 686, 2182, 33, 24, 707, - /* 2130 */ 58, 273, 32, 23, 2161, 1721, 706, 1768, 18, 2192, - /* 2140 */ 1769, 2228, 95, 1763, 328, 2194, 710, 2196, 2197, 705, - /* 2150 */ 1762, 700, 389, 1767, 1766, 390, 2193, 2211, 287, 177, - /* 2160 */ 291, 2094, 2073, 292, 97, 1696, 707, 290, 59, 2161, - /* 2170 */ 2192, 706, 2228, 1695, 2072, 326, 2194, 710, 2196, 2197, - /* 2180 */ 705, 2193, 700, 96, 25, 295, 257, 99, 105, 293, - /* 2190 */ 305, 707, 1727, 300, 2211, 68, 26, 101, 1648, 11, - /* 2200 */ 13, 1647, 1571, 179, 2231, 2192, 2161, 2228, 706, 1658, - /* 2210 */ 329, 2194, 710, 2196, 2197, 705, 681, 700, 2193, 2211, - /* 2220 */ 302, 1603, 192, 711, 713, 1626, 1624, 403, 707, 699, - /* 2230 */ 39, 2161, 1623, 706, 16, 27, 717, 1595, 28, 720, - /* 2240 */ 1385, 715, 2192, 2193, 2228, 1382, 1381, 335, 2194, 710, - /* 2250 */ 2196, 2197, 705, 707, 700, 1378, 2211, 718, 721, 723, - /* 2260 */ 724, 726, 1372, 709, 1370, 729, 727, 2192, 2161, 2228, - /* 2270 */ 706, 730, 339, 2194, 710, 2196, 2197, 705, 1376, 700, - /* 2280 */ 106, 2211, 308, 1394, 1375, 1390, 107, 75, 1263, 1374, - /* 2290 */ 1373, 744, 1295, 2161, 1294, 706, 1293, 1292, 309, 1290, - /* 2300 */ 1288, 1287, 1286, 1321, 2192, 754, 2228, 1284, 1283, 331, - /* 2310 */ 2194, 710, 2196, 2197, 705, 2193, 700, 1282, 1281, 1280, - /* 2320 */ 1279, 1278, 1318, 1316, 1275, 707, 1274, 1271, 1270, 2192, - /* 2330 */ 2193, 2228, 1269, 1268, 340, 2194, 710, 2196, 2197, 705, - /* 2340 */ 707, 700, 1855, 774, 775, 776, 1853, 778, 2193, 779, - /* 2350 */ 1851, 782, 780, 2211, 783, 1848, 784, 786, 707, 788, - /* 2360 */ 1830, 790, 1210, 787, 1807, 2161, 312, 706, 2211, 794, - /* 2370 */ 1782, 1557, 798, 322, 797, 1782, 1782, 1782, 1782, 1782, - /* 2380 */ 2161, 1782, 706, 1782, 1782, 1782, 2211, 1782, 1782, 1782, - /* 2390 */ 1782, 1782, 1782, 1782, 1782, 2193, 1782, 1782, 2161, 1782, - /* 2400 */ 706, 2192, 1782, 2228, 1782, 707, 332, 2194, 710, 2196, - /* 2410 */ 2197, 705, 1782, 700, 1782, 1782, 2192, 2193, 2228, 1782, - /* 2420 */ 1782, 341, 2194, 710, 2196, 2197, 705, 707, 700, 1782, - /* 2430 */ 1782, 1782, 1782, 2211, 2192, 1782, 2228, 1782, 1782, 333, - /* 2440 */ 2194, 710, 2196, 2197, 705, 2161, 700, 706, 1782, 1782, - /* 2450 */ 1782, 1782, 1782, 1782, 1782, 2211, 1782, 1782, 1782, 1782, - /* 2460 */ 1782, 1782, 1782, 1782, 2193, 1782, 1782, 2161, 1782, 706, - /* 2470 */ 1782, 1782, 1782, 1782, 707, 1782, 1782, 1782, 1782, 1782, - /* 2480 */ 1782, 2192, 1782, 2228, 1782, 1782, 346, 2194, 710, 2196, - /* 2490 */ 2197, 705, 1782, 700, 1782, 1782, 1782, 1782, 1782, 1782, - /* 2500 */ 1782, 1782, 2211, 2192, 1782, 2228, 1782, 1782, 347, 2194, - /* 2510 */ 710, 2196, 2197, 705, 2161, 700, 706, 1782, 1782, 1782, - /* 2520 */ 1782, 1782, 1782, 2193, 1782, 1782, 1782, 1782, 1782, 1782, - /* 2530 */ 1782, 1782, 1782, 707, 1782, 1782, 2193, 1782, 1782, 1782, - /* 2540 */ 1782, 1782, 1782, 1782, 1782, 1782, 707, 1782, 1782, 1782, - /* 2550 */ 2192, 1782, 2228, 2193, 1782, 2205, 2194, 710, 2196, 2197, - /* 2560 */ 705, 2211, 700, 707, 1782, 1782, 1782, 1782, 1782, 1782, - /* 2570 */ 1782, 1782, 1782, 2161, 2211, 706, 1782, 1782, 1782, 1782, - /* 2580 */ 1782, 1782, 1782, 1782, 1782, 2193, 2161, 1782, 706, 1782, - /* 2590 */ 1782, 2211, 1782, 1782, 1782, 707, 1782, 1782, 1782, 1782, - /* 2600 */ 1782, 1782, 1782, 2161, 1782, 706, 1782, 1782, 1782, 2192, - /* 2610 */ 1782, 2228, 1782, 1782, 2204, 2194, 710, 2196, 2197, 705, - /* 2620 */ 1782, 700, 2192, 2211, 2228, 1782, 1782, 2203, 2194, 710, - /* 2630 */ 2196, 2197, 705, 1782, 700, 2161, 1782, 706, 1782, 2192, - /* 2640 */ 1782, 2228, 1782, 1782, 362, 2194, 710, 2196, 2197, 705, - /* 2650 */ 1782, 700, 2193, 1782, 1782, 1782, 1782, 1782, 1782, 1782, - /* 2660 */ 1782, 1782, 707, 1782, 1782, 1782, 1782, 1782, 1782, 1782, - /* 2670 */ 1782, 2192, 2193, 2228, 1782, 1782, 363, 2194, 710, 2196, - /* 2680 */ 2197, 705, 707, 700, 1782, 1782, 1782, 1782, 1782, 1782, - /* 2690 */ 2211, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, - /* 2700 */ 1782, 1782, 2161, 1782, 706, 1782, 1782, 1782, 1782, 1782, - /* 2710 */ 2211, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, - /* 2720 */ 1782, 1782, 2161, 1782, 706, 1782, 1782, 1782, 1782, 1782, - /* 2730 */ 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 2192, 1782, - /* 2740 */ 2228, 1782, 2193, 359, 2194, 710, 2196, 2197, 705, 1782, - /* 2750 */ 700, 1782, 707, 1782, 1782, 1782, 1782, 1782, 2192, 2193, - /* 2760 */ 2228, 1782, 1782, 364, 2194, 710, 2196, 2197, 705, 707, - /* 2770 */ 700, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, - /* 2780 */ 2211, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, - /* 2790 */ 1782, 1782, 2161, 1782, 706, 1782, 1782, 2211, 1782, 1782, - /* 2800 */ 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 2161, - /* 2810 */ 1782, 706, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, - /* 2820 */ 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 708, 1782, - /* 2830 */ 2228, 1782, 1782, 338, 2194, 710, 2196, 2197, 705, 1782, - /* 2840 */ 700, 1782, 1782, 1782, 1782, 2192, 1782, 2228, 1782, 1782, - /* 2850 */ 337, 2194, 710, 2196, 2197, 705, 1782, 700, + /* 0 */ 735, 2198, 2176, 532, 2018, 2159, 533, 1823, 90, 450, + /* 10 */ 364, 705, 48, 46, 1710, 169, 2184, 1799, 2016, 181, + /* 20 */ 398, 2087, 1557, 41, 40, 368, 2180, 47, 45, 44, + /* 30 */ 43, 42, 66, 1638, 1959, 1555, 2085, 675, 366, 2070, + /* 40 */ 2216, 146, 41, 40, 2257, 2087, 47, 45, 44, 43, + /* 50 */ 42, 2166, 628, 704, 345, 2360, 2176, 391, 401, 2198, + /* 60 */ 2084, 675, 1633, 1582, 2182, 395, 1966, 163, 19, 668, + /* 70 */ 1955, 2366, 188, 380, 698, 1563, 2361, 654, 30, 687, + /* 80 */ 2180, 2016, 653, 358, 537, 2360, 2059, 2197, 2365, 2233, + /* 90 */ 534, 2360, 170, 2199, 708, 2201, 2202, 703, 2216, 698, + /* 100 */ 796, 652, 188, 15, 38, 303, 2361, 654, 2364, 2166, + /* 110 */ 673, 704, 2361, 2363, 2012, 2013, 48, 46, 2182, 47, + /* 120 */ 45, 44, 43, 42, 398, 453, 1557, 1667, 698, 688, + /* 130 */ 1964, 62, 62, 629, 2325, 407, 406, 1638, 1750, 1555, + /* 140 */ 1640, 1641, 221, 1242, 1243, 2197, 535, 2233, 1830, 135, + /* 150 */ 112, 2199, 708, 2201, 2202, 703, 572, 698, 1564, 223, + /* 160 */ 688, 1964, 185, 535, 2286, 1830, 1633, 674, 394, 2282, + /* 170 */ 1613, 1623, 19, 1582, 688, 1964, 1639, 1642, 168, 1563, + /* 180 */ 135, 549, 190, 627, 1668, 1788, 1905, 577, 568, 567, + /* 190 */ 2314, 1558, 125, 1556, 193, 124, 123, 122, 121, 120, + /* 200 */ 119, 118, 117, 116, 796, 745, 125, 15, 2198, 124, + /* 210 */ 123, 122, 121, 120, 119, 118, 117, 116, 705, 547, + /* 220 */ 1669, 2080, 648, 1561, 1562, 1787, 1612, 1615, 1616, 1617, + /* 230 */ 1618, 1619, 1620, 1621, 1622, 700, 696, 1631, 1632, 1634, + /* 240 */ 1635, 1636, 1637, 2, 1640, 1641, 687, 2216, 1583, 134, + /* 250 */ 133, 132, 131, 130, 129, 128, 127, 126, 2166, 1239, + /* 260 */ 704, 1238, 37, 396, 1662, 1663, 1664, 1665, 1666, 1670, + /* 270 */ 1671, 1672, 1673, 540, 1613, 1623, 533, 1823, 41, 40, + /* 280 */ 1639, 1642, 47, 45, 44, 43, 42, 248, 285, 285, + /* 290 */ 1585, 247, 1240, 687, 2197, 1558, 2233, 1556, 35, 112, + /* 300 */ 2199, 708, 2201, 2202, 703, 643, 698, 60, 1674, 147, + /* 310 */ 184, 155, 2257, 2286, 625, 2198, 1567, 394, 2282, 44, + /* 320 */ 43, 42, 2005, 1466, 1467, 668, 103, 1561, 1562, 51, + /* 330 */ 1612, 1615, 1616, 1617, 1618, 1619, 1620, 1621, 1622, 700, + /* 340 */ 696, 1631, 1632, 1634, 1635, 1636, 1637, 2, 12, 48, + /* 350 */ 46, 1957, 93, 737, 2216, 353, 2009, 398, 378, 1557, + /* 360 */ 606, 1373, 1583, 529, 9, 2166, 1582, 704, 1408, 1409, + /* 370 */ 1638, 527, 1555, 647, 523, 519, 51, 1364, 733, 732, + /* 380 */ 731, 1368, 730, 1370, 1371, 729, 726, 2365, 1379, 723, + /* 390 */ 1381, 1382, 720, 717, 714, 649, 644, 637, 2160, 1633, + /* 400 */ 611, 2197, 2216, 2233, 1584, 19, 112, 2199, 708, 2201, + /* 410 */ 2202, 703, 1563, 698, 590, 589, 588, 207, 185, 576, + /* 420 */ 2286, 580, 141, 584, 394, 2282, 371, 583, 12, 2198, + /* 430 */ 570, 569, 582, 587, 374, 373, 2301, 796, 581, 705, + /* 440 */ 15, 575, 262, 392, 595, 628, 2315, 628, 2360, 62, + /* 450 */ 2360, 1966, 166, 48, 46, 1643, 646, 86, 674, 605, + /* 460 */ 85, 398, 2298, 1557, 2366, 188, 2366, 188, 2216, 2361, + /* 470 */ 654, 2361, 654, 246, 1638, 2301, 1555, 1640, 1641, 2166, + /* 480 */ 1785, 704, 1483, 1484, 455, 2066, 41, 40, 1773, 598, + /* 490 */ 47, 45, 44, 43, 42, 372, 592, 370, 369, 2198, + /* 500 */ 574, 2297, 245, 1633, 576, 461, 2066, 1613, 1623, 705, + /* 510 */ 672, 1832, 2080, 1639, 1642, 2197, 1563, 2233, 1482, 1485, + /* 520 */ 113, 2199, 708, 2201, 2202, 703, 575, 698, 1558, 2018, + /* 530 */ 1556, 202, 255, 285, 2286, 379, 665, 144, 2216, 2283, + /* 540 */ 1584, 796, 70, 2016, 49, 69, 12, 1214, 10, 2166, + /* 550 */ 1585, 704, 205, 413, 488, 2066, 284, 1810, 412, 1316, + /* 560 */ 1561, 1562, 674, 1612, 1615, 1616, 1617, 1618, 1619, 1620, + /* 570 */ 1621, 1622, 700, 696, 1631, 1632, 1634, 1635, 1636, 1637, + /* 580 */ 2, 1640, 1641, 604, 1701, 2197, 1216, 2233, 1219, 1220, + /* 590 */ 112, 2199, 708, 2201, 2202, 703, 602, 698, 600, 1318, + /* 600 */ 628, 210, 2380, 2360, 2286, 1809, 285, 2166, 394, 2282, + /* 610 */ 2049, 1613, 1623, 1239, 683, 1238, 2080, 1639, 1642, 2366, + /* 620 */ 188, 586, 585, 1327, 2361, 654, 62, 237, 140, 613, + /* 630 */ 688, 1964, 1558, 1866, 1556, 2176, 1326, 688, 1964, 667, + /* 640 */ 186, 2294, 2295, 173, 142, 2299, 1240, 249, 2198, 2185, + /* 650 */ 56, 566, 562, 558, 554, 2166, 236, 448, 705, 2180, + /* 660 */ 2322, 1780, 767, 765, 1561, 1562, 1563, 1612, 1615, 1616, + /* 670 */ 1617, 1618, 1619, 1620, 1621, 1622, 700, 696, 1631, 1632, + /* 680 */ 1634, 1635, 1636, 1637, 2, 48, 46, 2216, 495, 688, + /* 690 */ 1964, 736, 284, 398, 2301, 1557, 91, 2182, 2166, 234, + /* 700 */ 704, 504, 203, 590, 589, 588, 1638, 698, 1555, 449, + /* 710 */ 580, 141, 584, 442, 444, 441, 583, 1528, 1529, 443, + /* 720 */ 2296, 582, 587, 374, 373, 41, 40, 581, 52, 47, + /* 730 */ 45, 44, 43, 42, 2197, 1633, 2233, 669, 432, 112, + /* 740 */ 2199, 708, 2201, 2202, 703, 440, 698, 158, 1563, 212, + /* 750 */ 211, 2380, 403, 2286, 699, 2011, 2013, 394, 2282, 688, + /* 760 */ 1964, 628, 401, 1779, 2360, 434, 430, 233, 227, 473, + /* 770 */ 1966, 166, 494, 796, 232, 545, 49, 1738, 472, 463, + /* 780 */ 2366, 188, 2149, 285, 628, 2361, 654, 2360, 2018, 48, + /* 790 */ 46, 1953, 166, 225, 388, 1808, 2198, 398, 181, 1557, + /* 800 */ 688, 1964, 2016, 2366, 188, 2018, 705, 55, 2361, 654, + /* 810 */ 1638, 393, 1555, 1640, 1641, 665, 144, 1714, 2071, 2016, + /* 820 */ 478, 1749, 1941, 1582, 640, 639, 1736, 1737, 1739, 1740, + /* 830 */ 1741, 2365, 2131, 404, 2360, 2216, 1807, 665, 144, 1633, + /* 840 */ 2364, 1966, 166, 1613, 1623, 2166, 2166, 755, 704, 1639, + /* 850 */ 1642, 2364, 1563, 41, 40, 2361, 2362, 47, 45, 44, + /* 860 */ 43, 42, 34, 258, 1558, 1940, 1556, 741, 41, 40, + /* 870 */ 2009, 1906, 47, 45, 44, 43, 42, 796, 688, 1964, + /* 880 */ 15, 250, 2197, 2198, 2233, 285, 2166, 171, 2199, 708, + /* 890 */ 2201, 2202, 703, 705, 698, 2335, 1561, 1562, 479, 1612, + /* 900 */ 1615, 1616, 1617, 1618, 1619, 1620, 1621, 1622, 700, 696, + /* 910 */ 1631, 1632, 1634, 1635, 1636, 1637, 2, 1640, 1641, 187, + /* 920 */ 2294, 2295, 2216, 142, 2299, 90, 690, 109, 2258, 1707, + /* 930 */ 688, 1964, 745, 2166, 692, 704, 2258, 743, 1800, 655, + /* 940 */ 2381, 189, 2294, 2295, 145, 142, 2299, 1613, 1623, 1726, + /* 950 */ 548, 1960, 1956, 1639, 1642, 279, 160, 159, 740, 739, + /* 960 */ 738, 157, 688, 1964, 742, 688, 1964, 2009, 1558, 2197, + /* 970 */ 1556, 2233, 2328, 1614, 112, 2199, 708, 2201, 2202, 703, + /* 980 */ 2187, 698, 1961, 1648, 36, 685, 2380, 1939, 2286, 1582, + /* 990 */ 41, 40, 394, 2282, 47, 45, 44, 43, 42, 427, + /* 1000 */ 1561, 1562, 1863, 1612, 1615, 1616, 1617, 1618, 1619, 1620, + /* 1010 */ 1621, 1622, 700, 696, 1631, 1632, 1634, 1635, 1636, 1637, + /* 1020 */ 2, 2018, 349, 167, 1580, 688, 1964, 402, 324, 688, + /* 1030 */ 1964, 486, 688, 1964, 502, 2016, 2189, 501, 688, 1964, + /* 1040 */ 14, 13, 321, 73, 1582, 251, 72, 688, 1964, 259, + /* 1050 */ 1331, 273, 671, 469, 1806, 503, 317, 346, 298, 1995, + /* 1060 */ 471, 1219, 1220, 1330, 1805, 1804, 139, 686, 219, 514, + /* 1070 */ 512, 509, 773, 772, 771, 770, 410, 1803, 769, 768, + /* 1080 */ 148, 763, 762, 761, 760, 759, 758, 757, 162, 753, + /* 1090 */ 752, 751, 409, 408, 748, 747, 746, 176, 175, 1802, + /* 1100 */ 688, 1964, 743, 657, 2166, 1585, 367, 1222, 62, 688, + /* 1110 */ 1964, 294, 295, 1581, 2166, 2166, 293, 656, 457, 1557, + /* 1120 */ 304, 160, 159, 740, 739, 738, 157, 2166, 506, 405, + /* 1130 */ 41, 40, 1555, 2152, 47, 45, 44, 43, 42, 1614, + /* 1140 */ 665, 144, 54, 1681, 3, 1801, 1798, 111, 499, 2166, + /* 1150 */ 1797, 493, 492, 491, 490, 485, 484, 483, 482, 481, + /* 1160 */ 477, 476, 475, 474, 348, 466, 465, 464, 1796, 459, + /* 1170 */ 458, 365, 1563, 41, 40, 1949, 166, 47, 45, 44, + /* 1180 */ 43, 42, 420, 1795, 1794, 1793, 1706, 1792, 81, 80, + /* 1190 */ 447, 2198, 743, 200, 1614, 2166, 2166, 796, 1951, 166, + /* 1200 */ 2166, 705, 756, 635, 1791, 1926, 439, 437, 1947, 166, + /* 1210 */ 158, 160, 159, 740, 739, 738, 157, 347, 2166, 2198, + /* 1220 */ 428, 2306, 1701, 426, 422, 418, 415, 440, 1790, 705, + /* 1230 */ 2216, 2353, 1942, 2166, 2166, 2166, 653, 2166, 83, 2360, + /* 1240 */ 137, 2166, 74, 704, 282, 2294, 664, 242, 136, 663, + /* 1250 */ 240, 2360, 2198, 695, 2166, 652, 188, 261, 2216, 578, + /* 1260 */ 2361, 654, 705, 158, 2305, 285, 2018, 652, 188, 2166, + /* 1270 */ 1523, 704, 2361, 654, 1967, 166, 149, 2197, 2166, 2233, + /* 1280 */ 2017, 1314, 112, 2199, 708, 2201, 2202, 703, 1558, 698, + /* 1290 */ 1556, 2216, 84, 260, 2380, 151, 2286, 608, 150, 607, + /* 1300 */ 394, 2282, 2166, 579, 704, 2197, 244, 2233, 1850, 243, + /* 1310 */ 112, 2199, 708, 2201, 2202, 703, 153, 698, 50, 152, + /* 1320 */ 1561, 1562, 2380, 1526, 2286, 1312, 50, 1566, 394, 2282, + /* 1330 */ 591, 266, 1782, 1783, 1565, 660, 94, 165, 2197, 158, + /* 1340 */ 2233, 658, 2198, 112, 2199, 708, 2201, 2202, 703, 1841, + /* 1350 */ 698, 1839, 705, 407, 406, 2380, 641, 2286, 50, 291, + /* 1360 */ 1904, 394, 2282, 1571, 71, 156, 158, 14, 13, 64, + /* 1370 */ 2198, 593, 108, 596, 1638, 1903, 1564, 2217, 1735, 411, + /* 1380 */ 705, 2216, 105, 50, 50, 381, 1734, 1273, 749, 712, + /* 1390 */ 156, 268, 2166, 2075, 704, 750, 1833, 158, 1824, 670, + /* 1400 */ 138, 1829, 2006, 1633, 2318, 666, 281, 156, 1659, 2216, + /* 1410 */ 1292, 278, 1, 5, 419, 1588, 1563, 1290, 1480, 296, + /* 1420 */ 2166, 414, 704, 362, 680, 300, 1357, 1274, 2197, 1675, + /* 1430 */ 2233, 436, 2198, 112, 2199, 708, 2201, 2202, 703, 435, + /* 1440 */ 698, 694, 705, 1624, 316, 2261, 791, 2286, 195, 1386, + /* 1450 */ 1390, 394, 2282, 196, 438, 198, 2197, 1397, 2233, 1504, + /* 1460 */ 1395, 112, 2199, 708, 2201, 2202, 703, 161, 698, 311, + /* 1470 */ 1581, 2216, 454, 2259, 209, 2286, 456, 1585, 460, 394, + /* 1480 */ 2282, 2076, 2166, 467, 704, 1569, 462, 497, 1580, 480, + /* 1490 */ 489, 2068, 1568, 487, 496, 498, 214, 508, 507, 2198, + /* 1500 */ 505, 510, 216, 513, 515, 213, 511, 1586, 4, 705, + /* 1510 */ 530, 614, 538, 1583, 224, 531, 541, 539, 2197, 542, + /* 1520 */ 2233, 226, 1587, 112, 2199, 708, 2201, 2202, 703, 799, + /* 1530 */ 698, 543, 1572, 1589, 1567, 691, 544, 2286, 2216, 229, + /* 1540 */ 231, 394, 2282, 310, 550, 546, 88, 571, 89, 2166, + /* 1550 */ 235, 704, 573, 1954, 239, 1950, 241, 1952, 1948, 180, + /* 1560 */ 114, 2198, 2140, 612, 1575, 1577, 352, 789, 785, 781, + /* 1570 */ 777, 705, 308, 2137, 2136, 661, 92, 696, 1631, 1632, + /* 1580 */ 1634, 1635, 1636, 1637, 610, 2197, 154, 2233, 616, 615, + /* 1590 */ 113, 2199, 708, 2201, 2202, 703, 252, 698, 620, 256, + /* 1600 */ 2216, 1511, 622, 623, 2286, 312, 617, 2319, 2285, 2282, + /* 1610 */ 678, 2166, 110, 704, 254, 301, 621, 2329, 632, 642, + /* 1620 */ 264, 638, 651, 267, 2198, 2334, 384, 2333, 645, 276, + /* 1630 */ 272, 8, 2308, 631, 705, 630, 633, 277, 1701, 143, + /* 1640 */ 1584, 659, 662, 2383, 385, 1704, 2302, 2197, 684, 2233, + /* 1650 */ 178, 286, 113, 2199, 708, 2201, 2202, 703, 98, 698, + /* 1660 */ 1590, 2081, 313, 2216, 676, 174, 2286, 274, 275, 677, + /* 1670 */ 693, 2282, 2095, 191, 2166, 2094, 704, 2093, 314, 681, + /* 1680 */ 390, 100, 2359, 288, 315, 2198, 102, 280, 682, 1965, + /* 1690 */ 287, 61, 2267, 104, 710, 702, 2010, 1927, 318, 792, + /* 1700 */ 307, 342, 2158, 2198, 354, 793, 327, 795, 53, 253, + /* 1710 */ 706, 355, 2233, 705, 2157, 113, 2199, 708, 2201, 2202, + /* 1720 */ 703, 2156, 698, 322, 2216, 78, 2153, 320, 416, 2286, + /* 1730 */ 417, 1548, 1549, 357, 2282, 2166, 194, 704, 421, 2151, + /* 1740 */ 341, 331, 2216, 423, 424, 425, 2150, 2148, 363, 2147, + /* 1750 */ 429, 431, 2146, 2166, 433, 704, 1539, 2127, 197, 2126, + /* 1760 */ 199, 1507, 79, 1506, 2198, 2108, 2107, 2106, 445, 446, + /* 1770 */ 2105, 2197, 2104, 2233, 705, 1457, 339, 2199, 708, 2201, + /* 1780 */ 2202, 703, 701, 698, 689, 2251, 2058, 452, 451, 2197, + /* 1790 */ 2055, 2233, 201, 2054, 170, 2199, 708, 2201, 2202, 703, + /* 1800 */ 82, 698, 2053, 2216, 206, 2045, 468, 382, 2052, 2057, + /* 1810 */ 204, 2056, 2051, 2050, 2166, 2048, 704, 2047, 2046, 470, + /* 1820 */ 2061, 2044, 2198, 2043, 2042, 2041, 2040, 2039, 2038, 2037, + /* 1830 */ 2036, 2035, 705, 2034, 2033, 2032, 2326, 2031, 2030, 208, + /* 1840 */ 2029, 2198, 87, 2028, 2027, 2026, 2060, 2025, 2024, 2023, + /* 1850 */ 2197, 705, 2233, 2022, 1459, 340, 2199, 708, 2201, 2202, + /* 1860 */ 703, 2216, 698, 2021, 2020, 383, 500, 2019, 1869, 215, + /* 1870 */ 350, 1868, 2166, 1328, 704, 1867, 1332, 1324, 217, 1865, + /* 1880 */ 2216, 218, 1862, 1861, 351, 516, 1854, 520, 517, 1843, + /* 1890 */ 521, 2166, 524, 704, 518, 522, 528, 1819, 526, 1221, + /* 1900 */ 1818, 2125, 220, 2115, 76, 2103, 222, 525, 2197, 77, + /* 1910 */ 2233, 2102, 228, 340, 2199, 708, 2201, 2202, 703, 2186, + /* 1920 */ 698, 2198, 182, 183, 536, 230, 2079, 2197, 1943, 2233, + /* 1930 */ 1864, 705, 333, 2199, 708, 2201, 2202, 703, 1266, 698, + /* 1940 */ 1860, 551, 2198, 553, 552, 1858, 557, 555, 556, 1856, + /* 1950 */ 559, 560, 705, 561, 1853, 563, 565, 564, 1838, 1836, + /* 1960 */ 2216, 1837, 1835, 1815, 1945, 1401, 1402, 1944, 1315, 1313, + /* 1970 */ 1311, 2166, 1310, 704, 1309, 1302, 650, 1308, 1851, 1307, + /* 1980 */ 764, 2216, 766, 1304, 1842, 389, 63, 1303, 375, 1840, + /* 1990 */ 238, 1301, 2166, 376, 704, 377, 1814, 594, 1813, 1812, + /* 2000 */ 597, 603, 115, 599, 601, 1533, 2124, 2197, 2198, 2233, + /* 2010 */ 1535, 1532, 171, 2199, 708, 2201, 2202, 703, 702, 698, + /* 2020 */ 1537, 29, 67, 1513, 1515, 2114, 618, 2101, 2197, 2100, + /* 2030 */ 2233, 2365, 17, 340, 2199, 708, 2201, 2202, 703, 20, + /* 2040 */ 698, 21, 6, 23, 31, 2198, 65, 2216, 7, 636, + /* 2050 */ 271, 1752, 22, 57, 263, 705, 634, 270, 2166, 2187, + /* 2060 */ 704, 265, 619, 1492, 1733, 2382, 2198, 257, 1491, 283, + /* 2070 */ 624, 1767, 1766, 386, 33, 172, 705, 269, 24, 1771, + /* 2080 */ 32, 164, 1517, 95, 2216, 1725, 1770, 626, 397, 1772, + /* 2090 */ 1773, 387, 59, 177, 2197, 2166, 2233, 704, 97, 339, + /* 2100 */ 2199, 708, 2201, 2202, 703, 2216, 698, 2099, 2252, 399, + /* 2110 */ 2078, 1698, 289, 2077, 1697, 290, 2166, 96, 704, 25, + /* 2120 */ 99, 1731, 292, 297, 2198, 302, 68, 101, 105, 26, + /* 2130 */ 13, 2197, 1650, 2233, 705, 2236, 340, 2199, 708, 2201, + /* 2140 */ 2202, 703, 1649, 698, 1573, 2198, 58, 1660, 179, 1628, + /* 2150 */ 697, 192, 2197, 1626, 2233, 705, 299, 340, 2199, 708, + /* 2160 */ 2201, 2202, 703, 2216, 698, 679, 11, 39, 1605, 709, + /* 2170 */ 1625, 16, 27, 28, 2166, 1597, 704, 18, 1387, 711, + /* 2180 */ 400, 1384, 713, 715, 2216, 716, 718, 1383, 1380, 719, + /* 2190 */ 721, 722, 724, 1374, 725, 2166, 1372, 704, 727, 728, + /* 2200 */ 1363, 1378, 1377, 2198, 305, 734, 1376, 1396, 106, 107, + /* 2210 */ 609, 75, 2233, 705, 1392, 335, 2199, 708, 2201, 2202, + /* 2220 */ 703, 1375, 698, 1264, 1296, 1295, 707, 744, 1294, 1293, + /* 2230 */ 1291, 2197, 1289, 2233, 1288, 2198, 325, 2199, 708, 2201, + /* 2240 */ 2202, 703, 2216, 698, 1287, 705, 1322, 754, 306, 1282, + /* 2250 */ 1285, 1284, 1283, 2166, 1281, 704, 1280, 1279, 1319, 1317, + /* 2260 */ 1276, 1275, 1270, 2198, 1272, 1271, 1269, 1859, 774, 775, + /* 2270 */ 1857, 776, 778, 705, 2216, 1855, 782, 779, 780, 1852, + /* 2280 */ 786, 784, 788, 783, 787, 2166, 1834, 704, 790, 2197, + /* 2290 */ 1211, 2233, 1811, 309, 323, 2199, 708, 2201, 2202, 703, + /* 2300 */ 1559, 698, 2216, 794, 798, 319, 797, 1786, 1786, 1786, + /* 2310 */ 1786, 1786, 1786, 2166, 1786, 704, 1786, 1786, 1786, 1786, + /* 2320 */ 1786, 2197, 1786, 2233, 1786, 1786, 326, 2199, 708, 2201, + /* 2330 */ 2202, 703, 1786, 698, 2198, 1786, 1786, 1786, 1786, 1786, + /* 2340 */ 1786, 1786, 1786, 1786, 705, 1786, 1786, 1786, 1786, 2197, + /* 2350 */ 2198, 2233, 1786, 1786, 332, 2199, 708, 2201, 2202, 703, + /* 2360 */ 705, 698, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, + /* 2370 */ 1786, 1786, 1786, 2216, 1786, 1786, 1786, 1786, 1786, 1786, + /* 2380 */ 1786, 1786, 1786, 1786, 2166, 1786, 704, 1786, 1786, 2216, + /* 2390 */ 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, + /* 2400 */ 2166, 1786, 704, 1786, 1786, 1786, 1786, 1786, 2198, 1786, + /* 2410 */ 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 705, 1786, + /* 2420 */ 2197, 1786, 2233, 1786, 1786, 336, 2199, 708, 2201, 2202, + /* 2430 */ 703, 1786, 698, 1786, 1786, 1786, 2197, 1786, 2233, 2198, + /* 2440 */ 1786, 328, 2199, 708, 2201, 2202, 703, 2216, 698, 705, + /* 2450 */ 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 2166, 1786, + /* 2460 */ 704, 1786, 1786, 1786, 1786, 1786, 2198, 1786, 1786, 1786, + /* 2470 */ 1786, 1786, 1786, 1786, 1786, 1786, 705, 1786, 2216, 1786, + /* 2480 */ 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 2166, + /* 2490 */ 1786, 704, 1786, 1786, 2197, 1786, 2233, 2198, 1786, 337, + /* 2500 */ 2199, 708, 2201, 2202, 703, 2216, 698, 705, 1786, 1786, + /* 2510 */ 1786, 1786, 1786, 1786, 1786, 1786, 2166, 1786, 704, 1786, + /* 2520 */ 1786, 1786, 1786, 1786, 1786, 2197, 1786, 2233, 1786, 1786, + /* 2530 */ 329, 2199, 708, 2201, 2202, 703, 2216, 698, 1786, 1786, + /* 2540 */ 1786, 1786, 1786, 1786, 1786, 1786, 1786, 2166, 1786, 704, + /* 2550 */ 1786, 1786, 2197, 1786, 2233, 2198, 1786, 338, 2199, 708, + /* 2560 */ 2201, 2202, 703, 1786, 698, 705, 1786, 1786, 1786, 1786, + /* 2570 */ 1786, 1786, 1786, 1786, 1786, 1786, 1786, 2198, 1786, 1786, + /* 2580 */ 1786, 1786, 1786, 2197, 1786, 2233, 1786, 705, 330, 2199, + /* 2590 */ 708, 2201, 2202, 703, 2216, 698, 1786, 1786, 1786, 1786, + /* 2600 */ 1786, 1786, 1786, 1786, 1786, 2166, 1786, 704, 1786, 1786, + /* 2610 */ 1786, 1786, 1786, 1786, 1786, 1786, 2216, 1786, 1786, 1786, + /* 2620 */ 1786, 1786, 1786, 1786, 1786, 1786, 1786, 2166, 1786, 704, + /* 2630 */ 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, + /* 2640 */ 1786, 2197, 1786, 2233, 2198, 1786, 343, 2199, 708, 2201, + /* 2650 */ 2202, 703, 1786, 698, 705, 1786, 1786, 1786, 1786, 1786, + /* 2660 */ 1786, 1786, 1786, 2197, 1786, 2233, 1786, 1786, 344, 2199, + /* 2670 */ 708, 2201, 2202, 703, 1786, 698, 2198, 1786, 1786, 1786, + /* 2680 */ 1786, 1786, 1786, 2216, 1786, 1786, 705, 1786, 1786, 1786, + /* 2690 */ 1786, 1786, 1786, 1786, 2166, 1786, 704, 1786, 1786, 1786, + /* 2700 */ 1786, 1786, 2198, 1786, 1786, 1786, 1786, 1786, 1786, 1786, + /* 2710 */ 1786, 1786, 705, 1786, 1786, 2216, 1786, 1786, 1786, 1786, + /* 2720 */ 1786, 1786, 1786, 1786, 1786, 1786, 2166, 1786, 704, 1786, + /* 2730 */ 2197, 1786, 2233, 1786, 1786, 2210, 2199, 708, 2201, 2202, + /* 2740 */ 703, 2216, 698, 1786, 1786, 1786, 1786, 1786, 1786, 1786, + /* 2750 */ 1786, 1786, 2166, 1786, 704, 1786, 1786, 1786, 1786, 1786, + /* 2760 */ 1786, 1786, 2197, 1786, 2233, 1786, 1786, 2209, 2199, 708, + /* 2770 */ 2201, 2202, 703, 1786, 698, 1786, 1786, 1786, 1786, 1786, + /* 2780 */ 1786, 2198, 1786, 1786, 1786, 1786, 1786, 1786, 2197, 1786, + /* 2790 */ 2233, 705, 1786, 2208, 2199, 708, 2201, 2202, 703, 1786, + /* 2800 */ 698, 1786, 2198, 1786, 1786, 1786, 1786, 1786, 1786, 1786, + /* 2810 */ 1786, 1786, 705, 1786, 1786, 1786, 1786, 1786, 2198, 1786, + /* 2820 */ 2216, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 705, 1786, + /* 2830 */ 1786, 2166, 1786, 704, 1786, 1786, 1786, 1786, 1786, 2198, + /* 2840 */ 1786, 2216, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 705, + /* 2850 */ 1786, 1786, 2166, 1786, 704, 1786, 1786, 2216, 1786, 1786, + /* 2860 */ 1786, 1786, 1786, 1786, 1786, 1786, 1786, 2197, 2166, 2233, + /* 2870 */ 704, 1786, 359, 2199, 708, 2201, 2202, 703, 2216, 698, + /* 2880 */ 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 2197, 2166, + /* 2890 */ 2233, 704, 1786, 360, 2199, 708, 2201, 2202, 703, 1786, + /* 2900 */ 698, 1786, 1786, 1786, 2197, 1786, 2233, 2198, 1786, 356, + /* 2910 */ 2199, 708, 2201, 2202, 703, 1786, 698, 705, 1786, 1786, + /* 2920 */ 1786, 1786, 1786, 1786, 1786, 2197, 1786, 2233, 1786, 2198, + /* 2930 */ 361, 2199, 708, 2201, 2202, 703, 1786, 698, 1786, 705, + /* 2940 */ 1786, 1786, 1786, 1786, 1786, 1786, 2216, 1786, 1786, 1786, + /* 2950 */ 1786, 1786, 1786, 1786, 1786, 1786, 1786, 2166, 1786, 704, + /* 2960 */ 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 2216, 1786, + /* 2970 */ 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 2166, + /* 2980 */ 1786, 704, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, + /* 2990 */ 1786, 1786, 1786, 706, 1786, 2233, 1786, 1786, 335, 2199, + /* 3000 */ 708, 2201, 2202, 703, 1786, 698, 1786, 1786, 1786, 1786, + /* 3010 */ 1786, 1786, 1786, 1786, 1786, 2197, 1786, 2233, 1786, 1786, + /* 3020 */ 334, 2199, 708, 2201, 2202, 703, 1786, 698, }; static const YYCODETYPE yy_lookahead[] = { - /* 0 */ 375, 341, 367, 393, 346, 350, 351, 411, 350, 380, - /* 10 */ 352, 351, 12, 13, 14, 405, 381, 4, 408, 409, - /* 20 */ 20, 3, 22, 8, 9, 370, 391, 12, 13, 14, - /* 30 */ 15, 16, 377, 33, 350, 35, 367, 14, 20, 379, - /* 40 */ 8, 9, 338, 20, 12, 13, 14, 15, 16, 424, - /* 50 */ 381, 391, 380, 393, 458, 49, 43, 461, 45, 46, - /* 60 */ 391, 379, 62, 57, 429, 430, 60, 61, 68, 20, - /* 70 */ 4, 22, 476, 477, 439, 75, 44, 481, 482, 395, - /* 80 */ 20, 399, 400, 68, 35, 350, 351, 427, 20, 429, - /* 90 */ 350, 351, 432, 433, 434, 435, 436, 437, 429, 439, - /* 100 */ 100, 360, 53, 103, 444, 370, 446, 357, 439, 368, - /* 110 */ 450, 451, 377, 84, 393, 411, 12, 13, 350, 351, - /* 120 */ 416, 20, 104, 463, 20, 62, 22, 112, 100, 408, - /* 130 */ 409, 471, 382, 20, 447, 448, 104, 33, 370, 35, - /* 140 */ 140, 141, 341, 115, 116, 117, 118, 119, 120, 121, - /* 150 */ 122, 123, 124, 341, 126, 127, 128, 129, 130, 131, - /* 160 */ 132, 378, 458, 103, 458, 461, 62, 461, 105, 103, - /* 170 */ 170, 171, 68, 390, 145, 146, 176, 177, 20, 75, - /* 180 */ 476, 477, 476, 477, 169, 481, 482, 481, 482, 431, - /* 190 */ 172, 191, 391, 193, 454, 455, 456, 168, 458, 459, - /* 200 */ 340, 461, 342, 391, 100, 8, 9, 103, 341, 12, - /* 210 */ 13, 14, 15, 16, 103, 457, 476, 477, 351, 35, - /* 220 */ 186, 481, 482, 223, 224, 0, 226, 227, 228, 229, + /* 0 */ 387, 341, 367, 345, 380, 411, 348, 349, 359, 350, + /* 10 */ 386, 351, 12, 13, 14, 340, 381, 342, 394, 380, + /* 20 */ 20, 393, 22, 8, 9, 376, 391, 12, 13, 14, + /* 30 */ 15, 16, 4, 33, 385, 35, 408, 409, 399, 400, + /* 40 */ 380, 442, 8, 9, 445, 393, 12, 13, 14, 15, + /* 50 */ 16, 391, 458, 393, 395, 461, 367, 405, 371, 341, + /* 60 */ 408, 409, 62, 20, 429, 430, 379, 380, 68, 351, + /* 70 */ 381, 477, 478, 386, 439, 75, 482, 483, 44, 20, + /* 80 */ 391, 394, 458, 68, 14, 461, 0, 427, 458, 429, + /* 90 */ 20, 461, 432, 433, 434, 435, 436, 437, 380, 439, + /* 100 */ 100, 477, 478, 103, 447, 448, 482, 483, 478, 391, + /* 110 */ 20, 393, 482, 483, 392, 393, 12, 13, 429, 12, + /* 120 */ 13, 14, 15, 16, 20, 39, 22, 112, 439, 350, + /* 130 */ 351, 103, 103, 473, 474, 12, 13, 33, 104, 35, + /* 140 */ 140, 141, 346, 54, 55, 427, 350, 429, 352, 370, + /* 150 */ 432, 433, 434, 435, 436, 437, 377, 439, 35, 346, + /* 160 */ 350, 351, 444, 350, 446, 352, 62, 350, 450, 451, + /* 170 */ 170, 171, 68, 20, 350, 351, 176, 177, 360, 75, + /* 180 */ 370, 67, 464, 48, 169, 0, 368, 377, 355, 356, + /* 190 */ 472, 191, 21, 193, 370, 24, 25, 26, 27, 28, + /* 200 */ 29, 30, 31, 32, 100, 67, 21, 103, 341, 24, + /* 210 */ 25, 26, 27, 28, 29, 30, 31, 32, 351, 402, + /* 220 */ 169, 404, 20, 223, 224, 0, 226, 227, 228, 229, /* 230 */ 230, 231, 232, 233, 234, 235, 236, 237, 238, 239, - /* 240 */ 240, 241, 242, 243, 140, 141, 379, 213, 214, 24, - /* 250 */ 25, 26, 27, 28, 29, 30, 31, 32, 391, 75, - /* 260 */ 393, 0, 247, 248, 249, 250, 251, 252, 253, 254, - /* 270 */ 255, 256, 257, 345, 170, 171, 348, 349, 67, 20, - /* 280 */ 176, 177, 21, 170, 171, 24, 25, 26, 27, 28, - /* 290 */ 29, 30, 31, 32, 427, 191, 429, 193, 175, 432, - /* 300 */ 433, 434, 435, 436, 437, 20, 439, 0, 371, 442, - /* 310 */ 20, 444, 445, 446, 20, 341, 379, 450, 451, 20, - /* 320 */ 260, 22, 350, 351, 387, 351, 260, 223, 224, 4, + /* 240 */ 240, 241, 242, 243, 140, 141, 20, 380, 20, 24, + /* 250 */ 25, 26, 27, 28, 29, 30, 31, 32, 391, 20, + /* 260 */ 393, 22, 247, 248, 249, 250, 251, 252, 253, 254, + /* 270 */ 255, 256, 257, 345, 170, 171, 348, 349, 8, 9, + /* 280 */ 176, 177, 12, 13, 14, 15, 16, 135, 260, 260, + /* 290 */ 20, 139, 53, 20, 427, 191, 429, 193, 247, 432, + /* 300 */ 433, 434, 435, 436, 437, 175, 439, 172, 257, 442, + /* 310 */ 378, 444, 445, 446, 179, 341, 193, 450, 451, 14, + /* 320 */ 15, 16, 390, 170, 171, 351, 357, 223, 224, 103, /* 330 */ 226, 227, 228, 229, 230, 231, 232, 233, 234, 235, /* 340 */ 236, 237, 238, 239, 240, 241, 242, 243, 244, 12, - /* 350 */ 13, 67, 53, 379, 379, 12, 13, 20, 47, 22, - /* 360 */ 458, 386, 103, 461, 379, 391, 20, 393, 114, 394, - /* 370 */ 33, 260, 35, 70, 71, 72, 350, 351, 35, 477, - /* 380 */ 77, 78, 79, 481, 482, 400, 83, 351, 265, 266, - /* 390 */ 267, 88, 89, 90, 91, 442, 370, 94, 445, 62, - /* 400 */ 135, 427, 48, 429, 139, 68, 432, 433, 434, 435, - /* 410 */ 436, 437, 75, 439, 103, 379, 134, 135, 444, 345, - /* 420 */ 446, 139, 348, 349, 450, 451, 454, 455, 456, 341, - /* 430 */ 458, 459, 14, 458, 140, 141, 461, 100, 20, 351, - /* 440 */ 103, 353, 8, 9, 35, 471, 12, 13, 14, 15, - /* 450 */ 16, 476, 477, 12, 13, 14, 481, 482, 346, 341, - /* 460 */ 170, 20, 350, 22, 352, 200, 159, 379, 203, 351, - /* 470 */ 434, 206, 379, 208, 33, 168, 35, 140, 141, 391, - /* 480 */ 387, 393, 458, 21, 75, 461, 24, 25, 26, 27, - /* 490 */ 28, 29, 30, 31, 32, 379, 103, 379, 341, 350, - /* 500 */ 351, 477, 386, 62, 244, 481, 482, 170, 171, 391, - /* 510 */ 394, 393, 357, 176, 177, 427, 75, 429, 172, 260, - /* 520 */ 432, 433, 434, 435, 436, 437, 172, 439, 191, 374, - /* 530 */ 193, 133, 444, 179, 446, 137, 193, 382, 450, 451, - /* 540 */ 14, 100, 350, 351, 103, 427, 20, 429, 391, 341, - /* 550 */ 432, 433, 434, 435, 436, 437, 244, 439, 246, 351, - /* 560 */ 223, 224, 370, 226, 227, 228, 229, 230, 231, 232, + /* 350 */ 13, 382, 200, 388, 380, 203, 391, 20, 206, 22, + /* 360 */ 208, 100, 20, 49, 39, 391, 20, 393, 140, 141, + /* 370 */ 33, 57, 35, 351, 60, 61, 103, 116, 117, 118, + /* 380 */ 119, 120, 121, 122, 123, 124, 125, 3, 127, 128, + /* 390 */ 129, 130, 131, 132, 133, 265, 266, 267, 411, 62, + /* 400 */ 411, 427, 380, 429, 20, 68, 432, 433, 434, 435, + /* 410 */ 436, 437, 75, 439, 70, 71, 72, 62, 444, 115, + /* 420 */ 446, 77, 78, 79, 450, 451, 37, 83, 244, 341, + /* 430 */ 355, 356, 88, 89, 90, 91, 431, 100, 94, 351, + /* 440 */ 103, 137, 172, 371, 4, 458, 472, 458, 461, 103, + /* 450 */ 461, 379, 380, 12, 13, 14, 434, 102, 350, 19, + /* 460 */ 105, 20, 457, 22, 477, 478, 477, 478, 380, 482, + /* 470 */ 483, 482, 483, 33, 33, 431, 35, 140, 141, 391, + /* 480 */ 338, 393, 140, 141, 350, 351, 8, 9, 104, 49, + /* 490 */ 12, 13, 14, 15, 16, 106, 56, 108, 109, 341, + /* 500 */ 111, 457, 62, 62, 115, 350, 351, 170, 171, 351, + /* 510 */ 402, 353, 404, 176, 177, 427, 75, 429, 176, 177, + /* 520 */ 432, 433, 434, 435, 436, 437, 137, 439, 191, 380, + /* 530 */ 193, 397, 387, 260, 446, 386, 350, 351, 380, 451, + /* 540 */ 20, 100, 102, 394, 103, 105, 244, 4, 246, 391, + /* 550 */ 20, 393, 397, 411, 350, 351, 172, 341, 416, 35, + /* 560 */ 223, 224, 350, 226, 227, 228, 229, 230, 231, 232, /* 570 */ 233, 234, 235, 236, 237, 238, 239, 240, 241, 242, - /* 580 */ 243, 140, 141, 190, 22, 192, 261, 379, 8, 9, - /* 590 */ 472, 473, 12, 13, 14, 15, 16, 35, 431, 391, - /* 600 */ 20, 393, 453, 454, 455, 456, 172, 458, 459, 75, - /* 610 */ 341, 170, 171, 350, 351, 222, 389, 176, 177, 392, - /* 620 */ 393, 8, 9, 371, 457, 12, 13, 14, 15, 16, - /* 630 */ 431, 379, 191, 370, 193, 427, 350, 429, 386, 387, - /* 640 */ 432, 433, 434, 435, 436, 437, 394, 439, 134, 20, - /* 650 */ 411, 341, 444, 260, 446, 416, 457, 411, 450, 451, - /* 660 */ 391, 351, 100, 353, 223, 224, 44, 226, 227, 228, + /* 580 */ 243, 140, 141, 21, 259, 427, 43, 429, 45, 46, + /* 590 */ 432, 433, 434, 435, 436, 437, 34, 439, 36, 75, + /* 600 */ 458, 397, 444, 461, 446, 341, 260, 391, 450, 451, + /* 610 */ 0, 170, 171, 20, 402, 22, 404, 176, 177, 477, + /* 620 */ 478, 364, 365, 22, 482, 483, 103, 33, 35, 114, + /* 630 */ 350, 351, 191, 0, 193, 367, 35, 350, 351, 453, + /* 640 */ 454, 455, 456, 49, 458, 459, 53, 134, 341, 381, + /* 650 */ 370, 57, 58, 59, 60, 391, 62, 370, 351, 391, + /* 660 */ 353, 183, 364, 365, 223, 224, 75, 226, 227, 228, /* 670 */ 229, 230, 231, 232, 233, 234, 235, 236, 237, 238, - /* 680 */ 239, 240, 241, 242, 243, 12, 13, 0, 402, 379, - /* 690 */ 404, 350, 351, 20, 75, 22, 170, 458, 350, 351, - /* 700 */ 461, 391, 379, 393, 458, 0, 33, 461, 35, 386, - /* 710 */ 12, 13, 14, 15, 16, 476, 477, 394, 204, 205, - /* 720 */ 481, 482, 476, 477, 350, 351, 104, 481, 482, 1, - /* 730 */ 2, 223, 103, 350, 351, 62, 380, 427, 397, 429, - /* 740 */ 4, 350, 432, 433, 434, 435, 436, 437, 75, 439, - /* 750 */ 350, 351, 172, 370, 444, 19, 446, 0, 8, 9, - /* 760 */ 450, 451, 12, 13, 14, 15, 16, 341, 341, 33, - /* 770 */ 370, 397, 67, 100, 350, 351, 103, 351, 270, 271, - /* 780 */ 272, 273, 274, 275, 276, 49, 350, 351, 3, 12, - /* 790 */ 13, 0, 56, 402, 370, 404, 183, 20, 62, 22, - /* 800 */ 371, 20, 454, 455, 456, 379, 458, 459, 379, 443, - /* 810 */ 33, 445, 35, 140, 141, 0, 387, 391, 391, 393, - /* 820 */ 133, 134, 135, 136, 137, 138, 139, 70, 71, 72, - /* 830 */ 392, 393, 104, 397, 77, 78, 79, 22, 102, 62, - /* 840 */ 83, 105, 103, 170, 171, 88, 89, 90, 91, 176, - /* 850 */ 177, 94, 75, 427, 104, 429, 54, 55, 432, 433, - /* 860 */ 434, 435, 436, 437, 191, 439, 193, 441, 8, 9, - /* 870 */ 341, 380, 12, 13, 14, 15, 16, 100, 355, 356, - /* 880 */ 103, 341, 133, 134, 135, 136, 137, 138, 139, 341, - /* 890 */ 411, 351, 62, 353, 341, 380, 223, 224, 285, 226, + /* 680 */ 239, 240, 241, 242, 243, 12, 13, 380, 84, 350, + /* 690 */ 351, 114, 172, 20, 431, 22, 102, 429, 391, 105, + /* 700 */ 393, 100, 172, 70, 71, 72, 33, 439, 35, 370, + /* 710 */ 77, 78, 79, 190, 411, 192, 83, 204, 205, 416, + /* 720 */ 457, 88, 89, 90, 91, 8, 9, 94, 103, 12, + /* 730 */ 13, 14, 15, 16, 427, 62, 429, 411, 186, 432, + /* 740 */ 433, 434, 435, 436, 437, 222, 439, 44, 75, 145, + /* 750 */ 146, 444, 389, 446, 387, 392, 393, 450, 451, 350, + /* 760 */ 351, 458, 371, 285, 461, 213, 214, 173, 174, 159, + /* 770 */ 379, 380, 168, 100, 180, 181, 103, 223, 168, 370, + /* 780 */ 477, 478, 0, 260, 458, 482, 483, 461, 380, 12, + /* 790 */ 13, 379, 380, 199, 386, 341, 341, 20, 380, 22, + /* 800 */ 350, 351, 394, 477, 478, 380, 351, 104, 482, 483, + /* 810 */ 33, 386, 35, 140, 141, 350, 351, 14, 400, 394, + /* 820 */ 370, 104, 0, 20, 270, 271, 272, 273, 274, 275, + /* 830 */ 276, 458, 375, 371, 461, 380, 341, 350, 351, 62, + /* 840 */ 3, 379, 380, 170, 171, 391, 391, 75, 393, 176, + /* 850 */ 177, 478, 75, 8, 9, 482, 483, 12, 13, 14, + /* 860 */ 15, 16, 2, 417, 191, 0, 193, 388, 8, 9, + /* 870 */ 391, 368, 12, 13, 14, 15, 16, 100, 350, 351, + /* 880 */ 103, 424, 427, 341, 429, 260, 391, 432, 433, 434, + /* 890 */ 435, 436, 437, 351, 439, 353, 223, 224, 370, 226, /* 900 */ 227, 228, 229, 230, 231, 232, 233, 234, 235, 236, - /* 910 */ 237, 238, 239, 240, 241, 242, 243, 140, 141, 379, - /* 920 */ 391, 140, 141, 21, 133, 134, 135, 136, 137, 138, - /* 930 */ 139, 391, 102, 393, 379, 105, 34, 458, 36, 391, - /* 940 */ 461, 386, 350, 351, 391, 350, 351, 170, 171, 394, - /* 950 */ 114, 350, 351, 176, 177, 476, 477, 176, 177, 380, - /* 960 */ 481, 482, 370, 388, 104, 370, 391, 427, 191, 429, - /* 970 */ 193, 370, 432, 433, 434, 435, 436, 437, 39, 439, - /* 980 */ 364, 365, 2, 0, 444, 22, 446, 350, 8, 9, - /* 990 */ 450, 451, 12, 13, 14, 15, 16, 341, 35, 260, + /* 910 */ 237, 238, 239, 240, 241, 242, 243, 140, 141, 454, + /* 920 */ 455, 456, 380, 458, 459, 359, 443, 357, 445, 4, + /* 930 */ 350, 351, 67, 391, 443, 393, 445, 115, 342, 484, + /* 940 */ 485, 454, 455, 456, 374, 458, 459, 170, 171, 104, + /* 950 */ 370, 385, 382, 176, 177, 486, 134, 135, 136, 137, + /* 960 */ 138, 139, 350, 351, 388, 350, 351, 391, 191, 427, + /* 970 */ 193, 429, 401, 170, 432, 433, 434, 435, 436, 437, + /* 980 */ 47, 439, 370, 14, 2, 370, 444, 0, 446, 20, + /* 990 */ 8, 9, 450, 451, 12, 13, 14, 15, 16, 217, /* 1000 */ 223, 224, 0, 226, 227, 228, 229, 230, 231, 232, /* 1010 */ 233, 234, 235, 236, 237, 238, 239, 240, 241, 242, - /* 1020 */ 243, 14, 18, 18, 20, 350, 351, 20, 23, 350, - /* 1030 */ 351, 27, 49, 359, 30, 355, 356, 33, 367, 402, - /* 1040 */ 371, 404, 37, 38, 0, 370, 41, 391, 379, 370, - /* 1050 */ 376, 379, 381, 49, 411, 51, 387, 52, 386, 385, - /* 1060 */ 56, 380, 391, 100, 341, 280, 394, 341, 63, 64, - /* 1070 */ 65, 66, 70, 71, 72, 73, 74, 0, 76, 77, + /* 1020 */ 243, 380, 18, 18, 20, 350, 351, 386, 23, 350, + /* 1030 */ 351, 27, 350, 351, 30, 394, 103, 33, 350, 351, + /* 1040 */ 1, 2, 37, 38, 20, 370, 41, 350, 351, 370, + /* 1050 */ 22, 469, 370, 49, 341, 51, 372, 52, 370, 375, + /* 1060 */ 56, 45, 46, 35, 341, 341, 354, 370, 63, 64, + /* 1070 */ 65, 66, 70, 71, 72, 73, 74, 341, 76, 77, /* 1080 */ 78, 79, 80, 81, 82, 83, 84, 85, 86, 87, - /* 1090 */ 88, 89, 90, 91, 92, 93, 94, 95, 96, 20, - /* 1100 */ 429, 458, 350, 351, 461, 20, 102, 33, 103, 37, - /* 1110 */ 439, 14, 15, 16, 391, 350, 351, 391, 114, 476, - /* 1120 */ 477, 44, 370, 49, 481, 482, 350, 351, 417, 350, - /* 1130 */ 351, 57, 58, 59, 60, 370, 62, 341, 44, 443, - /* 1140 */ 169, 445, 364, 365, 341, 341, 370, 142, 144, 370, + /* 1090 */ 88, 89, 90, 91, 92, 93, 94, 95, 96, 341, + /* 1100 */ 350, 351, 115, 44, 391, 20, 102, 14, 103, 350, + /* 1110 */ 351, 134, 135, 20, 391, 391, 139, 280, 114, 22, + /* 1120 */ 370, 134, 135, 136, 137, 138, 139, 391, 100, 370, + /* 1130 */ 8, 9, 35, 0, 12, 13, 14, 15, 16, 170, + /* 1140 */ 350, 351, 42, 104, 44, 341, 341, 142, 144, 391, /* 1150 */ 341, 147, 148, 149, 150, 151, 152, 153, 154, 155, - /* 1160 */ 156, 157, 158, 159, 160, 161, 162, 163, 44, 165, - /* 1170 */ 166, 167, 350, 351, 258, 259, 102, 170, 106, 105, - /* 1180 */ 108, 109, 0, 111, 359, 341, 341, 391, 183, 184, - /* 1190 */ 185, 341, 370, 188, 391, 391, 341, 379, 259, 388, - /* 1200 */ 391, 351, 391, 353, 22, 133, 201, 202, 341, 137, - /* 1210 */ 385, 388, 394, 372, 391, 368, 375, 212, 247, 42, - /* 1220 */ 215, 44, 341, 218, 219, 220, 221, 222, 257, 379, - /* 1230 */ 366, 22, 351, 369, 353, 391, 391, 45, 46, 44, - /* 1240 */ 42, 391, 44, 393, 35, 114, 391, 173, 174, 13, - /* 1250 */ 107, 172, 341, 110, 180, 181, 107, 172, 391, 110, - /* 1260 */ 379, 217, 351, 107, 353, 260, 110, 107, 44, 13, - /* 1270 */ 110, 35, 391, 199, 393, 0, 44, 427, 207, 429, - /* 1280 */ 209, 44, 432, 433, 434, 435, 436, 437, 68, 439, - /* 1290 */ 379, 35, 140, 141, 444, 164, 446, 22, 342, 104, - /* 1300 */ 450, 451, 391, 474, 393, 35, 44, 44, 427, 103, - /* 1310 */ 429, 401, 485, 432, 433, 434, 435, 436, 437, 113, - /* 1320 */ 439, 44, 44, 35, 22, 444, 44, 446, 104, 44, - /* 1330 */ 44, 450, 451, 1, 2, 44, 104, 35, 427, 44, - /* 1340 */ 429, 104, 341, 432, 433, 434, 435, 436, 437, 44, - /* 1350 */ 439, 0, 351, 12, 13, 444, 468, 446, 2, 44, - /* 1360 */ 354, 450, 451, 22, 8, 9, 104, 104, 12, 13, - /* 1370 */ 14, 15, 16, 341, 33, 367, 35, 75, 284, 44, - /* 1380 */ 379, 104, 104, 351, 44, 13, 104, 367, 379, 104, - /* 1390 */ 104, 13, 391, 44, 393, 104, 410, 44, 401, 104, - /* 1400 */ 354, 50, 100, 62, 349, 351, 282, 35, 390, 104, - /* 1410 */ 341, 379, 401, 35, 460, 478, 75, 452, 49, 104, - /* 1420 */ 351, 462, 412, 391, 262, 393, 428, 20, 427, 206, - /* 1430 */ 429, 189, 359, 432, 433, 434, 435, 436, 437, 104, - /* 1440 */ 439, 100, 421, 223, 104, 444, 421, 446, 379, 426, - /* 1450 */ 359, 450, 451, 104, 414, 20, 351, 104, 42, 427, - /* 1460 */ 391, 429, 393, 193, 432, 433, 434, 435, 436, 437, - /* 1470 */ 20, 439, 351, 398, 396, 169, 444, 401, 446, 398, - /* 1480 */ 20, 193, 450, 451, 350, 398, 351, 350, 396, 396, - /* 1490 */ 101, 363, 99, 191, 362, 193, 427, 350, 429, 341, - /* 1500 */ 98, 432, 433, 434, 435, 436, 437, 361, 439, 351, - /* 1510 */ 20, 350, 343, 350, 350, 446, 48, 347, 343, 450, - /* 1520 */ 451, 347, 421, 20, 359, 223, 224, 359, 20, 393, - /* 1530 */ 352, 20, 191, 413, 193, 359, 352, 379, 359, 359, - /* 1540 */ 359, 350, 359, 343, 343, 379, 379, 350, 341, 391, - /* 1550 */ 379, 393, 379, 210, 425, 103, 423, 197, 351, 421, - /* 1560 */ 357, 420, 196, 391, 223, 224, 379, 379, 379, 379, - /* 1570 */ 419, 357, 379, 379, 341, 379, 391, 236, 237, 238, - /* 1580 */ 239, 240, 241, 242, 351, 427, 379, 429, 350, 195, - /* 1590 */ 432, 433, 434, 435, 436, 437, 391, 439, 391, 269, - /* 1600 */ 393, 393, 391, 277, 446, 401, 467, 268, 450, 451, - /* 1610 */ 391, 401, 379, 391, 412, 467, 391, 418, 406, 406, - /* 1620 */ 470, 182, 469, 279, 391, 341, 393, 263, 278, 412, - /* 1630 */ 486, 286, 259, 281, 427, 351, 429, 283, 341, 432, - /* 1640 */ 433, 434, 435, 436, 437, 351, 439, 20, 351, 357, - /* 1650 */ 352, 20, 466, 446, 404, 406, 357, 450, 451, 431, - /* 1660 */ 427, 406, 429, 379, 391, 432, 433, 434, 435, 436, - /* 1670 */ 437, 438, 439, 440, 441, 391, 379, 393, 391, 174, - /* 1680 */ 391, 465, 467, 403, 391, 464, 357, 341, 391, 375, - /* 1690 */ 393, 391, 391, 357, 103, 449, 103, 351, 480, 351, - /* 1700 */ 383, 391, 369, 36, 341, 479, 344, 350, 343, 0, - /* 1710 */ 407, 427, 357, 429, 351, 407, 432, 433, 434, 435, - /* 1720 */ 436, 437, 415, 439, 427, 379, 429, 339, 358, 432, - /* 1730 */ 433, 434, 435, 436, 437, 422, 439, 391, 0, 393, - /* 1740 */ 373, 373, 379, 446, 373, 0, 42, 384, 451, 0, - /* 1750 */ 35, 216, 35, 341, 391, 35, 393, 35, 216, 0, - /* 1760 */ 35, 35, 216, 351, 0, 216, 0, 483, 484, 35, - /* 1770 */ 0, 0, 22, 427, 35, 429, 211, 341, 432, 433, - /* 1780 */ 434, 435, 436, 437, 0, 439, 199, 351, 0, 199, - /* 1790 */ 427, 379, 429, 193, 200, 432, 433, 434, 435, 436, - /* 1800 */ 437, 191, 439, 391, 341, 393, 0, 0, 0, 187, - /* 1810 */ 186, 0, 0, 0, 351, 379, 47, 0, 0, 473, - /* 1820 */ 384, 47, 0, 42, 341, 0, 0, 391, 0, 393, - /* 1830 */ 47, 0, 0, 0, 351, 0, 0, 0, 159, 427, - /* 1840 */ 35, 429, 379, 0, 432, 433, 434, 435, 436, 437, - /* 1850 */ 159, 439, 0, 0, 391, 0, 393, 0, 0, 0, - /* 1860 */ 0, 0, 379, 427, 0, 429, 0, 384, 432, 433, - /* 1870 */ 434, 435, 436, 437, 391, 439, 393, 47, 0, 0, - /* 1880 */ 42, 0, 0, 0, 0, 0, 62, 475, 0, 0, - /* 1890 */ 427, 341, 429, 0, 0, 432, 433, 434, 435, 436, - /* 1900 */ 437, 351, 439, 0, 0, 0, 0, 0, 22, 0, - /* 1910 */ 427, 143, 429, 0, 22, 432, 433, 434, 435, 436, - /* 1920 */ 437, 341, 439, 48, 48, 0, 35, 0, 0, 379, - /* 1930 */ 22, 351, 62, 62, 384, 0, 0, 39, 0, 49, - /* 1940 */ 39, 391, 0, 393, 49, 35, 35, 484, 49, 35, - /* 1950 */ 0, 39, 35, 0, 14, 39, 0, 42, 0, 379, - /* 1960 */ 0, 39, 44, 40, 384, 0, 47, 47, 47, 39, - /* 1970 */ 0, 391, 182, 393, 35, 49, 39, 427, 0, 429, - /* 1980 */ 341, 0, 432, 433, 434, 435, 436, 437, 1, 439, - /* 1990 */ 351, 0, 0, 69, 0, 49, 35, 0, 39, 35, - /* 2000 */ 49, 39, 0, 35, 49, 39, 19, 427, 0, 429, - /* 2010 */ 0, 0, 432, 433, 434, 435, 436, 437, 379, 439, - /* 2020 */ 33, 0, 0, 0, 0, 35, 22, 44, 35, 44, - /* 2030 */ 391, 35, 393, 35, 35, 35, 49, 0, 35, 22, - /* 2040 */ 35, 112, 110, 0, 57, 58, 59, 60, 35, 62, - /* 2050 */ 35, 35, 22, 341, 22, 0, 22, 51, 0, 0, - /* 2060 */ 35, 35, 35, 351, 0, 0, 427, 22, 429, 20, - /* 2070 */ 35, 432, 433, 434, 435, 436, 437, 35, 439, 35, - /* 2080 */ 104, 35, 103, 103, 341, 198, 0, 0, 22, 102, - /* 2090 */ 194, 379, 105, 22, 351, 0, 3, 44, 264, 104, - /* 2100 */ 48, 44, 3, 391, 103, 393, 172, 174, 48, 178, - /* 2110 */ 47, 99, 103, 172, 44, 172, 172, 44, 104, 341, - /* 2120 */ 101, 103, 379, 178, 104, 138, 47, 44, 44, 351, - /* 2130 */ 258, 103, 103, 264, 391, 104, 393, 104, 264, 427, - /* 2140 */ 104, 429, 103, 35, 432, 433, 434, 435, 436, 437, - /* 2150 */ 35, 439, 35, 35, 35, 35, 341, 379, 47, 47, - /* 2160 */ 173, 0, 0, 47, 39, 104, 351, 180, 44, 391, - /* 2170 */ 427, 393, 429, 104, 0, 432, 433, 434, 435, 436, - /* 2180 */ 437, 341, 439, 103, 103, 103, 199, 39, 113, 104, - /* 2190 */ 47, 351, 104, 103, 379, 103, 44, 103, 101, 245, - /* 2200 */ 2, 101, 22, 47, 103, 427, 391, 429, 393, 223, - /* 2210 */ 432, 433, 434, 435, 436, 437, 175, 439, 341, 379, - /* 2220 */ 173, 22, 47, 114, 35, 104, 104, 35, 351, 103, - /* 2230 */ 103, 391, 104, 393, 103, 103, 35, 104, 103, 35, - /* 2240 */ 104, 103, 427, 341, 429, 104, 104, 432, 433, 434, - /* 2250 */ 435, 436, 437, 351, 439, 104, 379, 103, 103, 35, - /* 2260 */ 103, 35, 104, 225, 104, 35, 103, 427, 391, 429, - /* 2270 */ 393, 103, 432, 433, 434, 435, 436, 437, 125, 439, - /* 2280 */ 103, 379, 44, 35, 125, 22, 103, 103, 69, 125, - /* 2290 */ 125, 68, 35, 391, 35, 393, 35, 35, 44, 35, - /* 2300 */ 35, 35, 35, 75, 427, 97, 429, 35, 35, 432, - /* 2310 */ 433, 434, 435, 436, 437, 341, 439, 35, 22, 35, - /* 2320 */ 35, 35, 75, 35, 35, 351, 35, 35, 35, 427, - /* 2330 */ 341, 429, 22, 35, 432, 433, 434, 435, 436, 437, - /* 2340 */ 351, 439, 0, 35, 49, 39, 0, 35, 341, 49, - /* 2350 */ 0, 35, 39, 379, 49, 0, 39, 35, 351, 39, - /* 2360 */ 0, 35, 35, 49, 0, 391, 22, 393, 379, 21, - /* 2370 */ 487, 22, 20, 22, 21, 487, 487, 487, 487, 487, - /* 2380 */ 391, 487, 393, 487, 487, 487, 379, 487, 487, 487, - /* 2390 */ 487, 487, 487, 487, 487, 341, 487, 487, 391, 487, - /* 2400 */ 393, 427, 487, 429, 487, 351, 432, 433, 434, 435, - /* 2410 */ 436, 437, 487, 439, 487, 487, 427, 341, 429, 487, - /* 2420 */ 487, 432, 433, 434, 435, 436, 437, 351, 439, 487, - /* 2430 */ 487, 487, 487, 379, 427, 487, 429, 487, 487, 432, - /* 2440 */ 433, 434, 435, 436, 437, 391, 439, 393, 487, 487, - /* 2450 */ 487, 487, 487, 487, 487, 379, 487, 487, 487, 487, - /* 2460 */ 487, 487, 487, 487, 341, 487, 487, 391, 487, 393, - /* 2470 */ 487, 487, 487, 487, 351, 487, 487, 487, 487, 487, - /* 2480 */ 487, 427, 487, 429, 487, 487, 432, 433, 434, 435, - /* 2490 */ 436, 437, 487, 439, 487, 487, 487, 487, 487, 487, - /* 2500 */ 487, 487, 379, 427, 487, 429, 487, 487, 432, 433, - /* 2510 */ 434, 435, 436, 437, 391, 439, 393, 487, 487, 487, - /* 2520 */ 487, 487, 487, 341, 487, 487, 487, 487, 487, 487, - /* 2530 */ 487, 487, 487, 351, 487, 487, 341, 487, 487, 487, - /* 2540 */ 487, 487, 487, 487, 487, 487, 351, 487, 487, 487, - /* 2550 */ 427, 487, 429, 341, 487, 432, 433, 434, 435, 436, - /* 2560 */ 437, 379, 439, 351, 487, 487, 487, 487, 487, 487, - /* 2570 */ 487, 487, 487, 391, 379, 393, 487, 487, 487, 487, - /* 2580 */ 487, 487, 487, 487, 487, 341, 391, 487, 393, 487, - /* 2590 */ 487, 379, 487, 487, 487, 351, 487, 487, 487, 487, - /* 2600 */ 487, 487, 487, 391, 487, 393, 487, 487, 487, 427, - /* 2610 */ 487, 429, 487, 487, 432, 433, 434, 435, 436, 437, - /* 2620 */ 487, 439, 427, 379, 429, 487, 487, 432, 433, 434, - /* 2630 */ 435, 436, 437, 487, 439, 391, 487, 393, 487, 427, - /* 2640 */ 487, 429, 487, 487, 432, 433, 434, 435, 436, 437, - /* 2650 */ 487, 439, 341, 487, 487, 487, 487, 487, 487, 487, - /* 2660 */ 487, 487, 351, 487, 487, 487, 487, 487, 487, 487, - /* 2670 */ 487, 427, 341, 429, 487, 487, 432, 433, 434, 435, - /* 2680 */ 436, 437, 351, 439, 487, 487, 487, 487, 487, 487, - /* 2690 */ 379, 487, 487, 487, 487, 487, 487, 487, 487, 487, - /* 2700 */ 487, 487, 391, 487, 393, 487, 487, 487, 487, 487, - /* 2710 */ 379, 487, 487, 487, 487, 487, 487, 487, 487, 487, - /* 2720 */ 487, 487, 391, 487, 393, 487, 487, 487, 487, 487, - /* 2730 */ 487, 487, 487, 487, 487, 487, 487, 487, 427, 487, - /* 2740 */ 429, 487, 341, 432, 433, 434, 435, 436, 437, 487, - /* 2750 */ 439, 487, 351, 487, 487, 487, 487, 487, 427, 341, - /* 2760 */ 429, 487, 487, 432, 433, 434, 435, 436, 437, 351, - /* 2770 */ 439, 487, 487, 487, 487, 487, 487, 487, 487, 487, - /* 2780 */ 379, 487, 487, 487, 487, 487, 487, 487, 487, 487, - /* 2790 */ 487, 487, 391, 487, 393, 487, 487, 379, 487, 487, - /* 2800 */ 487, 487, 487, 487, 487, 487, 487, 487, 487, 391, - /* 2810 */ 487, 393, 487, 487, 487, 487, 487, 487, 487, 487, - /* 2820 */ 487, 487, 487, 487, 487, 487, 487, 487, 427, 487, - /* 2830 */ 429, 487, 487, 432, 433, 434, 435, 436, 437, 487, - /* 2840 */ 439, 487, 487, 487, 487, 427, 487, 429, 487, 487, - /* 2850 */ 432, 433, 434, 435, 436, 437, 487, 439, 338, 338, - /* 2860 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, - /* 2870 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, - /* 2880 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, - /* 2890 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, - /* 2900 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, - /* 2910 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, - /* 2920 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, - /* 2930 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, - /* 2940 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, - /* 2950 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, - /* 2960 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, - /* 2970 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, - /* 2980 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, - /* 2990 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, - /* 3000 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, - /* 3010 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, - /* 3020 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, + /* 1160 */ 156, 157, 158, 159, 160, 161, 162, 163, 341, 165, + /* 1170 */ 166, 167, 75, 8, 9, 379, 380, 12, 13, 14, + /* 1180 */ 15, 16, 49, 341, 341, 341, 261, 341, 183, 184, + /* 1190 */ 185, 341, 115, 188, 170, 391, 391, 100, 379, 380, + /* 1200 */ 391, 351, 366, 353, 341, 369, 201, 202, 379, 380, + /* 1210 */ 44, 134, 135, 136, 137, 138, 139, 212, 391, 341, + /* 1220 */ 215, 258, 259, 218, 219, 220, 221, 222, 341, 351, + /* 1230 */ 380, 353, 0, 391, 391, 391, 458, 391, 42, 461, + /* 1240 */ 44, 391, 114, 393, 454, 455, 456, 107, 458, 459, + /* 1250 */ 110, 461, 341, 68, 391, 477, 478, 172, 380, 13, + /* 1260 */ 482, 483, 351, 44, 353, 260, 380, 477, 478, 391, + /* 1270 */ 104, 393, 482, 483, 379, 380, 44, 427, 391, 429, + /* 1280 */ 394, 35, 432, 433, 434, 435, 436, 437, 191, 439, + /* 1290 */ 193, 380, 164, 62, 444, 107, 446, 207, 110, 209, + /* 1300 */ 450, 451, 391, 13, 393, 427, 107, 429, 0, 110, + /* 1310 */ 432, 433, 434, 435, 436, 437, 107, 439, 44, 110, + /* 1320 */ 223, 224, 444, 104, 446, 35, 44, 35, 450, 451, + /* 1330 */ 22, 44, 140, 141, 35, 44, 105, 172, 427, 44, + /* 1340 */ 429, 282, 341, 432, 433, 434, 435, 436, 437, 0, + /* 1350 */ 439, 0, 351, 12, 13, 444, 475, 446, 44, 44, + /* 1360 */ 367, 450, 451, 22, 44, 44, 44, 1, 2, 44, + /* 1370 */ 341, 22, 103, 22, 33, 367, 35, 380, 104, 354, + /* 1380 */ 351, 380, 113, 44, 44, 410, 104, 35, 13, 44, + /* 1390 */ 44, 104, 391, 401, 393, 13, 0, 44, 349, 104, + /* 1400 */ 44, 351, 390, 62, 401, 460, 479, 44, 223, 380, + /* 1410 */ 35, 452, 463, 262, 49, 20, 75, 35, 104, 104, + /* 1420 */ 391, 412, 393, 428, 104, 104, 104, 75, 427, 104, + /* 1430 */ 429, 421, 341, 432, 433, 434, 435, 436, 437, 206, + /* 1440 */ 439, 100, 351, 104, 104, 444, 50, 446, 426, 104, + /* 1450 */ 104, 450, 451, 359, 421, 359, 427, 104, 429, 189, + /* 1460 */ 104, 432, 433, 434, 435, 436, 437, 104, 439, 414, + /* 1470 */ 20, 380, 351, 444, 42, 446, 398, 20, 351, 450, + /* 1480 */ 451, 401, 391, 396, 393, 193, 398, 169, 20, 350, + /* 1490 */ 398, 350, 193, 351, 396, 396, 350, 363, 101, 341, + /* 1500 */ 99, 98, 350, 350, 350, 362, 361, 20, 48, 351, + /* 1510 */ 343, 1, 343, 20, 359, 347, 421, 347, 427, 393, + /* 1520 */ 429, 359, 20, 432, 433, 434, 435, 436, 437, 19, + /* 1530 */ 439, 352, 191, 20, 193, 444, 413, 446, 380, 359, + /* 1540 */ 359, 450, 451, 33, 350, 352, 359, 343, 359, 391, + /* 1550 */ 359, 393, 380, 380, 380, 380, 380, 380, 380, 49, + /* 1560 */ 350, 341, 391, 425, 223, 224, 343, 57, 58, 59, + /* 1570 */ 60, 351, 62, 391, 391, 284, 103, 236, 237, 238, + /* 1580 */ 239, 240, 241, 242, 210, 427, 423, 429, 197, 196, + /* 1590 */ 432, 433, 434, 435, 436, 437, 357, 439, 393, 357, + /* 1600 */ 380, 195, 412, 350, 446, 421, 420, 401, 450, 451, + /* 1610 */ 268, 391, 102, 393, 419, 105, 418, 401, 391, 269, + /* 1620 */ 406, 391, 182, 406, 341, 468, 391, 468, 391, 465, + /* 1630 */ 470, 277, 471, 278, 351, 263, 279, 412, 259, 351, + /* 1640 */ 20, 281, 283, 487, 286, 114, 431, 427, 138, 429, + /* 1650 */ 352, 357, 432, 433, 434, 435, 436, 437, 357, 439, + /* 1660 */ 20, 404, 406, 380, 391, 468, 446, 467, 466, 391, + /* 1670 */ 450, 451, 391, 462, 391, 391, 393, 391, 406, 174, + /* 1680 */ 391, 357, 481, 173, 375, 341, 357, 480, 403, 351, + /* 1690 */ 180, 103, 449, 103, 383, 351, 391, 369, 350, 36, + /* 1700 */ 357, 422, 0, 341, 407, 344, 373, 343, 415, 199, + /* 1710 */ 427, 407, 429, 351, 0, 432, 433, 434, 435, 436, + /* 1720 */ 437, 0, 439, 339, 380, 42, 0, 358, 35, 446, + /* 1730 */ 216, 35, 35, 450, 451, 391, 35, 393, 216, 0, + /* 1740 */ 373, 373, 380, 35, 35, 216, 0, 0, 216, 0, + /* 1750 */ 35, 22, 0, 391, 35, 393, 211, 0, 199, 0, + /* 1760 */ 199, 193, 200, 191, 341, 0, 0, 0, 187, 186, + /* 1770 */ 0, 427, 0, 429, 351, 47, 432, 433, 434, 435, + /* 1780 */ 436, 437, 438, 439, 440, 441, 0, 49, 35, 427, + /* 1790 */ 0, 429, 47, 0, 432, 433, 434, 435, 436, 437, + /* 1800 */ 42, 439, 0, 380, 159, 0, 35, 384, 0, 0, + /* 1810 */ 47, 0, 0, 0, 391, 0, 393, 0, 0, 159, + /* 1820 */ 0, 0, 341, 0, 0, 0, 0, 0, 0, 0, + /* 1830 */ 0, 0, 351, 0, 0, 0, 474, 0, 0, 47, + /* 1840 */ 0, 341, 42, 0, 0, 0, 0, 0, 0, 0, + /* 1850 */ 427, 351, 429, 0, 22, 432, 433, 434, 435, 436, + /* 1860 */ 437, 380, 439, 0, 0, 384, 143, 0, 0, 62, + /* 1870 */ 48, 0, 391, 22, 393, 0, 22, 35, 62, 0, + /* 1880 */ 380, 62, 0, 0, 48, 35, 0, 35, 49, 0, + /* 1890 */ 49, 391, 35, 393, 39, 39, 35, 0, 39, 14, + /* 1900 */ 0, 0, 42, 0, 39, 0, 40, 49, 427, 39, + /* 1910 */ 429, 0, 39, 432, 433, 434, 435, 436, 437, 47, + /* 1920 */ 439, 341, 44, 47, 47, 182, 0, 427, 0, 429, + /* 1930 */ 0, 351, 432, 433, 434, 435, 436, 437, 69, 439, + /* 1940 */ 0, 35, 341, 39, 49, 0, 39, 35, 49, 0, + /* 1950 */ 35, 49, 351, 39, 0, 35, 39, 49, 0, 0, + /* 1960 */ 380, 0, 0, 0, 0, 22, 35, 0, 35, 35, + /* 1970 */ 35, 391, 35, 393, 35, 22, 476, 35, 0, 35, + /* 1980 */ 44, 380, 44, 35, 0, 384, 112, 35, 22, 0, + /* 1990 */ 110, 35, 391, 22, 393, 22, 0, 51, 0, 0, + /* 2000 */ 35, 22, 20, 35, 35, 35, 0, 427, 341, 429, + /* 2010 */ 35, 35, 432, 433, 434, 435, 436, 437, 351, 439, + /* 2020 */ 104, 103, 103, 35, 22, 0, 22, 0, 427, 0, + /* 2030 */ 429, 3, 264, 432, 433, 434, 435, 436, 437, 44, + /* 2040 */ 439, 44, 48, 264, 103, 341, 3, 380, 48, 99, + /* 2050 */ 47, 104, 44, 172, 103, 351, 101, 44, 391, 47, + /* 2060 */ 393, 104, 172, 172, 104, 485, 341, 174, 172, 47, + /* 2070 */ 178, 35, 35, 35, 44, 103, 351, 103, 44, 35, + /* 2080 */ 103, 194, 198, 103, 380, 104, 35, 178, 384, 104, + /* 2090 */ 104, 35, 44, 47, 427, 391, 429, 393, 39, 432, + /* 2100 */ 433, 434, 435, 436, 437, 380, 439, 0, 441, 384, + /* 2110 */ 0, 104, 47, 0, 104, 104, 391, 103, 393, 103, + /* 2120 */ 39, 104, 103, 103, 341, 47, 103, 103, 113, 44, + /* 2130 */ 2, 427, 101, 429, 351, 103, 432, 433, 434, 435, + /* 2140 */ 436, 437, 101, 439, 22, 341, 258, 223, 47, 104, + /* 2150 */ 103, 47, 427, 104, 429, 351, 173, 432, 433, 434, + /* 2160 */ 435, 436, 437, 380, 439, 175, 245, 103, 22, 114, + /* 2170 */ 104, 103, 103, 103, 391, 104, 393, 264, 104, 35, + /* 2180 */ 35, 104, 103, 35, 380, 103, 35, 104, 104, 103, + /* 2190 */ 35, 103, 35, 104, 103, 391, 104, 393, 35, 103, + /* 2200 */ 22, 126, 126, 341, 44, 115, 126, 35, 103, 103, + /* 2210 */ 427, 103, 429, 351, 22, 432, 433, 434, 435, 436, + /* 2220 */ 437, 126, 439, 69, 35, 35, 225, 68, 35, 35, + /* 2230 */ 35, 427, 35, 429, 35, 341, 432, 433, 434, 435, + /* 2240 */ 436, 437, 380, 439, 35, 351, 75, 97, 44, 22, + /* 2250 */ 35, 35, 35, 391, 35, 393, 35, 35, 75, 35, + /* 2260 */ 35, 35, 22, 341, 35, 35, 35, 0, 35, 49, + /* 2270 */ 0, 39, 35, 351, 380, 0, 35, 49, 39, 0, + /* 2280 */ 35, 39, 39, 49, 49, 391, 0, 393, 35, 427, + /* 2290 */ 35, 429, 0, 22, 432, 433, 434, 435, 436, 437, + /* 2300 */ 22, 439, 380, 21, 20, 22, 21, 488, 488, 488, + /* 2310 */ 488, 488, 488, 391, 488, 393, 488, 488, 488, 488, + /* 2320 */ 488, 427, 488, 429, 488, 488, 432, 433, 434, 435, + /* 2330 */ 436, 437, 488, 439, 341, 488, 488, 488, 488, 488, + /* 2340 */ 488, 488, 488, 488, 351, 488, 488, 488, 488, 427, + /* 2350 */ 341, 429, 488, 488, 432, 433, 434, 435, 436, 437, + /* 2360 */ 351, 439, 488, 488, 488, 488, 488, 488, 488, 488, + /* 2370 */ 488, 488, 488, 380, 488, 488, 488, 488, 488, 488, + /* 2380 */ 488, 488, 488, 488, 391, 488, 393, 488, 488, 380, + /* 2390 */ 488, 488, 488, 488, 488, 488, 488, 488, 488, 488, + /* 2400 */ 391, 488, 393, 488, 488, 488, 488, 488, 341, 488, + /* 2410 */ 488, 488, 488, 488, 488, 488, 488, 488, 351, 488, + /* 2420 */ 427, 488, 429, 488, 488, 432, 433, 434, 435, 436, + /* 2430 */ 437, 488, 439, 488, 488, 488, 427, 488, 429, 341, + /* 2440 */ 488, 432, 433, 434, 435, 436, 437, 380, 439, 351, + /* 2450 */ 488, 488, 488, 488, 488, 488, 488, 488, 391, 488, + /* 2460 */ 393, 488, 488, 488, 488, 488, 341, 488, 488, 488, + /* 2470 */ 488, 488, 488, 488, 488, 488, 351, 488, 380, 488, + /* 2480 */ 488, 488, 488, 488, 488, 488, 488, 488, 488, 391, + /* 2490 */ 488, 393, 488, 488, 427, 488, 429, 341, 488, 432, + /* 2500 */ 433, 434, 435, 436, 437, 380, 439, 351, 488, 488, + /* 2510 */ 488, 488, 488, 488, 488, 488, 391, 488, 393, 488, + /* 2520 */ 488, 488, 488, 488, 488, 427, 488, 429, 488, 488, + /* 2530 */ 432, 433, 434, 435, 436, 437, 380, 439, 488, 488, + /* 2540 */ 488, 488, 488, 488, 488, 488, 488, 391, 488, 393, + /* 2550 */ 488, 488, 427, 488, 429, 341, 488, 432, 433, 434, + /* 2560 */ 435, 436, 437, 488, 439, 351, 488, 488, 488, 488, + /* 2570 */ 488, 488, 488, 488, 488, 488, 488, 341, 488, 488, + /* 2580 */ 488, 488, 488, 427, 488, 429, 488, 351, 432, 433, + /* 2590 */ 434, 435, 436, 437, 380, 439, 488, 488, 488, 488, + /* 2600 */ 488, 488, 488, 488, 488, 391, 488, 393, 488, 488, + /* 2610 */ 488, 488, 488, 488, 488, 488, 380, 488, 488, 488, + /* 2620 */ 488, 488, 488, 488, 488, 488, 488, 391, 488, 393, + /* 2630 */ 488, 488, 488, 488, 488, 488, 488, 488, 488, 488, + /* 2640 */ 488, 427, 488, 429, 341, 488, 432, 433, 434, 435, + /* 2650 */ 436, 437, 488, 439, 351, 488, 488, 488, 488, 488, + /* 2660 */ 488, 488, 488, 427, 488, 429, 488, 488, 432, 433, + /* 2670 */ 434, 435, 436, 437, 488, 439, 341, 488, 488, 488, + /* 2680 */ 488, 488, 488, 380, 488, 488, 351, 488, 488, 488, + /* 2690 */ 488, 488, 488, 488, 391, 488, 393, 488, 488, 488, + /* 2700 */ 488, 488, 341, 488, 488, 488, 488, 488, 488, 488, + /* 2710 */ 488, 488, 351, 488, 488, 380, 488, 488, 488, 488, + /* 2720 */ 488, 488, 488, 488, 488, 488, 391, 488, 393, 488, + /* 2730 */ 427, 488, 429, 488, 488, 432, 433, 434, 435, 436, + /* 2740 */ 437, 380, 439, 488, 488, 488, 488, 488, 488, 488, + /* 2750 */ 488, 488, 391, 488, 393, 488, 488, 488, 488, 488, + /* 2760 */ 488, 488, 427, 488, 429, 488, 488, 432, 433, 434, + /* 2770 */ 435, 436, 437, 488, 439, 488, 488, 488, 488, 488, + /* 2780 */ 488, 341, 488, 488, 488, 488, 488, 488, 427, 488, + /* 2790 */ 429, 351, 488, 432, 433, 434, 435, 436, 437, 488, + /* 2800 */ 439, 488, 341, 488, 488, 488, 488, 488, 488, 488, + /* 2810 */ 488, 488, 351, 488, 488, 488, 488, 488, 341, 488, + /* 2820 */ 380, 488, 488, 488, 488, 488, 488, 488, 351, 488, + /* 2830 */ 488, 391, 488, 393, 488, 488, 488, 488, 488, 341, + /* 2840 */ 488, 380, 488, 488, 488, 488, 488, 488, 488, 351, + /* 2850 */ 488, 488, 391, 488, 393, 488, 488, 380, 488, 488, + /* 2860 */ 488, 488, 488, 488, 488, 488, 488, 427, 391, 429, + /* 2870 */ 393, 488, 432, 433, 434, 435, 436, 437, 380, 439, + /* 2880 */ 488, 488, 488, 488, 488, 488, 488, 488, 427, 391, + /* 2890 */ 429, 393, 488, 432, 433, 434, 435, 436, 437, 488, + /* 2900 */ 439, 488, 488, 488, 427, 488, 429, 341, 488, 432, + /* 2910 */ 433, 434, 435, 436, 437, 488, 439, 351, 488, 488, + /* 2920 */ 488, 488, 488, 488, 488, 427, 488, 429, 488, 341, + /* 2930 */ 432, 433, 434, 435, 436, 437, 488, 439, 488, 351, + /* 2940 */ 488, 488, 488, 488, 488, 488, 380, 488, 488, 488, + /* 2950 */ 488, 488, 488, 488, 488, 488, 488, 391, 488, 393, + /* 2960 */ 488, 488, 488, 488, 488, 488, 488, 488, 380, 488, + /* 2970 */ 488, 488, 488, 488, 488, 488, 488, 488, 488, 391, + /* 2980 */ 488, 393, 488, 488, 488, 488, 488, 488, 488, 488, + /* 2990 */ 488, 488, 488, 427, 488, 429, 488, 488, 432, 433, + /* 3000 */ 434, 435, 436, 437, 488, 439, 488, 488, 488, 488, + /* 3010 */ 488, 488, 488, 488, 488, 427, 488, 429, 488, 488, + /* 3020 */ 432, 433, 434, 435, 436, 437, 488, 439, 338, 338, /* 3030 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, /* 3040 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, /* 3050 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, @@ -827,212 +1183,228 @@ static const YYCODETYPE yy_lookahead[] = { /* 3160 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, /* 3170 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, /* 3180 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, - /* 3190 */ 338, 338, 338, 338, 338, 338, + /* 3190 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, + /* 3200 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, + /* 3210 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, + /* 3220 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, + /* 3230 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, + /* 3240 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, + /* 3250 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, + /* 3260 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, + /* 3270 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, + /* 3280 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, + /* 3290 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, + /* 3300 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, + /* 3310 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, + /* 3320 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, + /* 3330 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, + /* 3340 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, + /* 3350 */ 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, + /* 3360 */ 338, 338, 338, 338, 338, 338, }; #define YY_SHIFT_COUNT (799) #define YY_SHIFT_MIN (0) -#define YY_SHIFT_MAX (2364) +#define YY_SHIFT_MAX (2292) static const unsigned short int yy_shift_ofst[] = { /* 0 */ 1005, 0, 104, 0, 337, 337, 337, 337, 337, 337, /* 10 */ 337, 337, 337, 337, 337, 337, 441, 673, 673, 777, /* 20 */ 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, /* 30 */ 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, /* 40 */ 673, 673, 673, 673, 673, 673, 673, 673, 673, 673, - /* 50 */ 673, 60, 259, 393, 629, 111, 739, 111, 629, 629, - /* 60 */ 111, 1341, 111, 1341, 1341, 66, 111, 68, 781, 101, - /* 70 */ 101, 781, 13, 13, 113, 294, 23, 23, 101, 101, - /* 80 */ 101, 101, 101, 101, 101, 101, 101, 101, 158, 101, - /* 90 */ 101, 211, 68, 101, 101, 285, 68, 101, 158, 101, - /* 100 */ 158, 68, 101, 101, 68, 101, 68, 68, 68, 101, - /* 110 */ 284, 1004, 15, 15, 303, 462, 1302, 1302, 1302, 1302, - /* 120 */ 1302, 1302, 1302, 1302, 1302, 1302, 1302, 1302, 1302, 1302, - /* 130 */ 1302, 1302, 1302, 1302, 1302, 1072, 18, 113, 294, 802, - /* 140 */ 802, 184, 346, 346, 346, 705, 312, 312, 184, 211, - /* 150 */ 254, 260, 68, 534, 68, 534, 534, 836, 619, 28, - /* 160 */ 28, 28, 28, 28, 28, 28, 28, 1987, 757, 261, - /* 170 */ 580, 613, 508, 49, 123, 343, 343, 526, 354, 1007, - /* 180 */ 299, 1079, 1192, 418, 398, 1085, 916, 939, 785, 916, - /* 190 */ 1177, 325, 290, 1162, 1369, 1407, 1223, 211, 1407, 211, - /* 200 */ 1242, 1435, 1416, 1450, 1435, 1416, 1306, 1460, 1435, 1460, - /* 210 */ 1416, 1306, 1306, 1389, 1393, 1460, 1402, 1460, 1460, 1460, - /* 220 */ 1490, 1468, 1490, 1468, 1407, 211, 1503, 211, 1508, 1511, - /* 230 */ 211, 1508, 211, 211, 211, 1460, 211, 1490, 68, 68, - /* 240 */ 68, 68, 68, 68, 68, 68, 68, 68, 68, 1460, - /* 250 */ 1490, 534, 534, 534, 1343, 1452, 1407, 284, 1360, 1366, - /* 260 */ 1503, 284, 1394, 1162, 1460, 1450, 1450, 534, 1330, 1339, - /* 270 */ 534, 1330, 1339, 534, 534, 68, 1326, 1439, 1330, 1344, - /* 280 */ 1350, 1364, 1162, 1345, 1354, 1352, 1373, 1435, 1627, 1508, - /* 290 */ 284, 284, 1631, 1339, 534, 534, 534, 534, 534, 1339, - /* 300 */ 534, 1505, 284, 836, 284, 1435, 1591, 1593, 534, 619, - /* 310 */ 1460, 284, 1667, 1490, 2858, 2858, 2858, 2858, 2858, 2858, - /* 320 */ 2858, 2858, 2858, 1002, 1074, 225, 32, 736, 750, 860, - /* 330 */ 687, 980, 1356, 434, 791, 197, 197, 197, 197, 197, - /* 340 */ 197, 197, 197, 197, 749, 265, 698, 698, 29, 6, - /* 350 */ 34, 307, 830, 562, 963, 902, 514, 282, 282, 1097, - /* 360 */ 728, 971, 1097, 1097, 1097, 983, 1044, 622, 1209, 1198, - /* 370 */ 1131, 1077, 1143, 1149, 1156, 1160, 1236, 1256, 815, 1182, - /* 380 */ 1275, 1071, 1195, 1224, 63, 1232, 1237, 1262, 1152, 1124, - /* 390 */ 1094, 1263, 1277, 1278, 1282, 1285, 1286, 1332, 1291, 1220, - /* 400 */ 1295, 311, 1305, 1315, 1335, 1340, 1349, 1353, 1206, 1270, - /* 410 */ 1288, 1372, 1378, 409, 1351, 1709, 1738, 1745, 1704, 1749, - /* 420 */ 1715, 1535, 1717, 1720, 1722, 1542, 1759, 1725, 1726, 1546, - /* 430 */ 1764, 1549, 1766, 1734, 1770, 1750, 1771, 1739, 1565, 1784, - /* 440 */ 1587, 1788, 1590, 1594, 1600, 1610, 1806, 1807, 1808, 1622, - /* 450 */ 1624, 1811, 1812, 1769, 1813, 1817, 1818, 1774, 1822, 1781, - /* 460 */ 1825, 1826, 1828, 1783, 1831, 1832, 1833, 1835, 1836, 1837, - /* 470 */ 1679, 1805, 1843, 1691, 1852, 1853, 1855, 1857, 1858, 1859, - /* 480 */ 1860, 1861, 1864, 1866, 1888, 1889, 1893, 1894, 1903, 1904, - /* 490 */ 1830, 1878, 1838, 1879, 1881, 1882, 1883, 1884, 1885, 1886, - /* 500 */ 1905, 1906, 1907, 1768, 1909, 1913, 1892, 1875, 1908, 1876, - /* 510 */ 1925, 1824, 1891, 1927, 1870, 1928, 1871, 1935, 1936, 1910, - /* 520 */ 1890, 1898, 1938, 1911, 1895, 1901, 1942, 1914, 1899, 1912, - /* 530 */ 1950, 1917, 1953, 1915, 1916, 1918, 1919, 1920, 1940, 1921, - /* 540 */ 1956, 1923, 1922, 1958, 1960, 1965, 1930, 1790, 1970, 1978, - /* 550 */ 1981, 1924, 1991, 1992, 1939, 1926, 1937, 1994, 1961, 1946, - /* 560 */ 1959, 1997, 1964, 1951, 1962, 2002, 1968, 1955, 1966, 2008, - /* 570 */ 2010, 2011, 2021, 2022, 2023, 1929, 1932, 1990, 2004, 2024, - /* 580 */ 1993, 1996, 1998, 1999, 2000, 2003, 2005, 1983, 1985, 2013, - /* 590 */ 2015, 2017, 2016, 2037, 2030, 2043, 2032, 2006, 2055, 2034, - /* 600 */ 2025, 2058, 2026, 2059, 2027, 2064, 2045, 2049, 2035, 2042, - /* 610 */ 2044, 1976, 1979, 2065, 1934, 1980, 1887, 2046, 2066, 2086, - /* 620 */ 1896, 2071, 1941, 1933, 2087, 2095, 1943, 1931, 1944, 1945, - /* 630 */ 2093, 2053, 1834, 2001, 1995, 2009, 2052, 2019, 2060, 2012, - /* 640 */ 2014, 2057, 2070, 2020, 2018, 2028, 2029, 2031, 2073, 2063, - /* 650 */ 2079, 2039, 2083, 1869, 2033, 2036, 2099, 2084, 1874, 2108, - /* 660 */ 2115, 2117, 2118, 2119, 2120, 2061, 2069, 2111, 1872, 2124, - /* 670 */ 2112, 2161, 2162, 2080, 2125, 1919, 2116, 2081, 2085, 2088, - /* 680 */ 2082, 2090, 2041, 2092, 2174, 2148, 2047, 2094, 2075, 1919, - /* 690 */ 2143, 2152, 2097, 1954, 2100, 2198, 2180, 1986, 2101, 2121, - /* 700 */ 2126, 2122, 2127, 2128, 2156, 2131, 2132, 2175, 2133, 2199, - /* 710 */ 2038, 2135, 2109, 2136, 2189, 2192, 2138, 2141, 2201, 2154, - /* 720 */ 2142, 2204, 2155, 2151, 2224, 2157, 2158, 2226, 2163, 2160, - /* 730 */ 2230, 2168, 2153, 2159, 2164, 2165, 2177, 2238, 2183, 2248, - /* 740 */ 2184, 2238, 2238, 2263, 2219, 2223, 2257, 2259, 2261, 2262, - /* 750 */ 2264, 2265, 2266, 2267, 2228, 2208, 2254, 2272, 2273, 2282, - /* 760 */ 2296, 2284, 2285, 2286, 2247, 1983, 2288, 1985, 2289, 2291, - /* 770 */ 2292, 2293, 2310, 2298, 2342, 2308, 2295, 2306, 2346, 2312, - /* 780 */ 2300, 2313, 2350, 2316, 2305, 2317, 2355, 2322, 2314, 2320, - /* 790 */ 2360, 2326, 2327, 2364, 2344, 2348, 2349, 2351, 2353, 2352, + /* 50 */ 673, 273, 346, 523, 226, 29, 625, 29, 226, 226, + /* 60 */ 29, 1341, 29, 1341, 1341, 28, 29, 43, 342, 59, + /* 70 */ 59, 342, 543, 543, 153, 228, 70, 70, 59, 59, + /* 80 */ 59, 59, 59, 59, 59, 59, 59, 59, 90, 59, + /* 90 */ 59, 114, 43, 59, 59, 202, 43, 59, 90, 59, + /* 100 */ 90, 43, 59, 59, 43, 59, 43, 43, 43, 59, + /* 110 */ 138, 1004, 15, 15, 344, 171, 1097, 1097, 1097, 1097, + /* 120 */ 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, + /* 130 */ 1097, 1097, 1097, 1097, 1097, 389, 384, 153, 228, 89, + /* 140 */ 89, 524, 520, 520, 520, 865, 302, 302, 524, 114, + /* 150 */ 43, 43, 43, 43, 515, 184, 43, 591, 43, 591, + /* 160 */ 591, 577, 772, 261, 261, 261, 261, 1510, 633, 185, + /* 170 */ 270, 478, 554, 593, 130, 123, 123, 803, 135, 969, + /* 180 */ 239, 530, 1016, 1093, 304, 1085, 963, 325, 837, 963, + /* 190 */ 1100, 925, 1024, 1151, 1365, 1395, 1233, 114, 1395, 114, + /* 200 */ 1270, 1450, 1432, 1457, 1450, 1432, 1318, 1468, 1450, 1468, + /* 210 */ 1432, 1318, 1318, 1397, 1401, 1468, 1403, 1468, 1468, 1468, + /* 220 */ 1487, 1460, 1487, 1460, 1395, 114, 1493, 114, 1502, 1513, + /* 230 */ 114, 1502, 114, 114, 114, 1468, 114, 1487, 43, 43, + /* 240 */ 43, 43, 43, 43, 43, 1468, 1487, 591, 591, 591, + /* 250 */ 1374, 1473, 1395, 138, 1391, 1393, 1493, 138, 1406, 1151, + /* 260 */ 1468, 1457, 1457, 591, 1350, 1342, 591, 1350, 1342, 591, + /* 270 */ 591, 43, 1354, 1440, 1350, 1357, 1355, 1372, 1151, 1358, + /* 280 */ 1359, 1360, 1379, 1450, 1620, 1531, 1502, 138, 138, 1640, + /* 290 */ 1342, 591, 591, 591, 591, 591, 1342, 591, 1505, 138, + /* 300 */ 577, 138, 1450, 1588, 1590, 591, 772, 1468, 138, 1663, + /* 310 */ 1487, 3028, 3028, 3028, 3028, 3028, 3028, 3028, 3028, 3028, + /* 320 */ 1002, 594, 225, 34, 440, 717, 845, 822, 860, 982, + /* 330 */ 1165, 987, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, + /* 340 */ 1122, 1077, 152, 107, 107, 604, 314, 552, 610, 355, + /* 350 */ 601, 1028, 562, 513, 977, 977, 305, 1039, 51, 305, + /* 360 */ 305, 305, 1133, 782, 703, 86, 1196, 1128, 1232, 1140, + /* 370 */ 1188, 1199, 1209, 1246, 1290, 1308, 1349, 1351, 1090, 1166, + /* 380 */ 1219, 1231, 1274, 1282, 1287, 1192, 1059, 1291, 1295, 1314, + /* 390 */ 1315, 1320, 1321, 1322, 1366, 1325, 1185, 1339, 933, 1340, + /* 400 */ 1345, 1346, 1353, 1356, 1363, 1269, 1292, 1299, 1375, 1382, + /* 410 */ 1352, 1396, 1702, 1714, 1721, 1683, 1726, 1693, 1514, 1696, + /* 420 */ 1697, 1701, 1522, 1739, 1708, 1709, 1529, 1746, 1532, 1747, + /* 430 */ 1715, 1749, 1729, 1752, 1719, 1545, 1757, 1559, 1759, 1561, + /* 440 */ 1562, 1568, 1572, 1765, 1766, 1767, 1581, 1583, 1770, 1772, + /* 450 */ 1728, 1786, 1753, 1738, 1790, 1745, 1793, 1758, 1802, 1808, + /* 460 */ 1809, 1763, 1811, 1812, 1813, 1815, 1817, 1818, 1645, 1771, + /* 470 */ 1805, 1660, 1820, 1821, 1823, 1824, 1825, 1826, 1827, 1828, + /* 480 */ 1829, 1830, 1831, 1833, 1834, 1835, 1837, 1838, 1792, 1840, + /* 490 */ 1800, 1843, 1844, 1845, 1846, 1847, 1848, 1832, 1849, 1853, + /* 500 */ 1863, 1723, 1864, 1867, 1851, 1822, 1854, 1836, 1868, 1807, + /* 510 */ 1842, 1871, 1816, 1875, 1819, 1879, 1882, 1850, 1839, 1855, + /* 520 */ 1883, 1852, 1841, 1856, 1886, 1857, 1858, 1859, 1889, 1861, + /* 530 */ 1897, 1860, 1865, 1878, 1872, 1876, 1885, 1877, 1900, 1866, + /* 540 */ 1870, 1901, 1903, 1905, 1873, 1743, 1911, 1926, 1928, 1869, + /* 550 */ 1930, 1940, 1906, 1895, 1904, 1945, 1912, 1899, 1907, 1949, + /* 560 */ 1915, 1902, 1914, 1954, 1920, 1908, 1917, 1958, 1959, 1961, + /* 570 */ 1962, 1963, 1964, 1874, 1880, 1931, 1943, 1967, 1933, 1934, + /* 580 */ 1935, 1937, 1939, 1942, 1944, 1936, 1938, 1948, 1952, 1953, + /* 590 */ 1956, 1978, 1966, 1984, 1971, 1946, 1989, 1973, 1965, 1996, + /* 600 */ 1968, 1998, 1969, 1999, 1979, 1982, 1970, 1975, 1976, 1916, + /* 610 */ 1918, 2006, 1881, 1919, 1884, 1988, 2002, 2025, 1887, 2004, + /* 620 */ 1890, 1893, 2027, 2029, 1891, 1892, 1896, 1909, 2028, 1995, + /* 630 */ 1768, 1941, 1947, 1951, 1994, 1955, 2000, 1950, 1957, 1997, + /* 640 */ 2008, 1960, 1972, 1974, 1977, 1981, 2013, 2003, 2012, 1980, + /* 650 */ 2030, 1779, 1985, 1986, 2043, 2034, 1913, 2036, 2037, 2038, + /* 660 */ 2044, 2051, 2056, 2007, 2010, 2022, 1888, 2048, 2046, 2107, + /* 670 */ 2110, 2014, 2059, 1872, 2065, 2016, 2011, 2017, 2019, 2020, + /* 680 */ 1990, 2023, 2113, 2081, 1983, 2024, 2015, 1872, 2078, 2085, + /* 690 */ 2031, 1921, 2041, 2128, 2122, 1924, 2032, 2045, 2047, 2049, + /* 700 */ 2064, 2066, 2101, 2068, 2069, 2104, 2071, 2146, 2001, 2070, + /* 710 */ 2055, 2074, 2144, 2145, 2079, 2077, 2148, 2082, 2083, 2151, + /* 720 */ 2086, 2084, 2155, 2088, 2089, 2157, 2091, 2092, 2163, 2096, + /* 730 */ 2075, 2076, 2080, 2095, 2178, 2090, 2105, 2160, 2106, 2172, + /* 740 */ 2108, 2160, 2160, 2192, 2154, 2159, 2189, 2190, 2193, 2194, + /* 750 */ 2195, 2197, 2199, 2209, 2171, 2150, 2204, 2215, 2216, 2217, + /* 760 */ 2227, 2219, 2221, 2222, 2183, 1936, 2224, 1938, 2225, 2226, + /* 770 */ 2229, 2230, 2240, 2231, 2267, 2233, 2220, 2232, 2270, 2237, + /* 780 */ 2228, 2239, 2275, 2241, 2234, 2242, 2279, 2245, 2235, 2243, + /* 790 */ 2286, 2253, 2255, 2292, 2271, 2282, 2278, 2283, 2285, 2284, }; -#define YY_REDUCE_COUNT (322) -#define YY_REDUCE_MIN (-404) -#define YY_REDUCE_MAX (2418) +#define YY_REDUCE_COUNT (319) +#define YY_REDUCE_MIN (-406) +#define YY_REDUCE_MAX (2588) static const short yy_reduce_ofst[] = { - /* 0 */ -296, -340, -133, -26, 88, 310, 540, 850, 881, 911, - /* 10 */ 208, 1001, 1032, 1069, 1158, 1207, 1233, 118, 1284, 1297, - /* 20 */ 1346, 1363, 1436, 1412, 1463, 1483, 426, 1550, 1580, 1639, - /* 30 */ 1712, 1743, 1778, 1815, 1840, 1877, 1902, 1974, 1989, 2007, - /* 40 */ 2054, 2076, 2123, 2182, 2195, 2212, 2244, 2311, 2331, 2401, - /* 50 */ 2418, -260, -25, 239, 149, -404, 246, 479, -28, 348, - /* 60 */ 643, -365, -294, -331, 671, -98, 24, 252, -390, -345, - /* 70 */ -265, -279, -72, 74, -318, 227, -342, 112, -232, 26, - /* 80 */ 192, 263, 341, 374, 383, 400, 424, 436, 286, 592, - /* 90 */ 595, 674, 116, 601, 675, 36, 323, 679, 391, 752, - /* 100 */ 637, -63, 765, 776, 555, 779, 429, 672, 669, 822, - /* 110 */ 155, -316, -313, -313, -259, -140, -199, -188, 157, 269, - /* 120 */ 427, 529, 548, 553, 656, 723, 726, 796, 803, 804, - /* 130 */ 809, 844, 845, 855, 867, -217, -242, -15, 438, 523, - /* 140 */ 680, 616, -242, 167, 199, -250, 366, 696, 778, 825, - /* 150 */ -375, -47, 93, 575, 818, 811, 823, 841, 864, -371, - /* 160 */ -328, 356, 491, 515, 579, 681, 515, 711, 847, 956, - /* 170 */ 910, 827, 829, 1006, 888, 1008, 1020, 1009, 986, 1009, - /* 180 */ 1046, 997, 1055, 1054, 1018, 1011, 954, 954, 937, 954, - /* 190 */ 965, 959, 1009, 1010, 998, 1021, 1023, 1073, 1025, 1091, - /* 200 */ 1040, 1105, 1075, 1076, 1121, 1081, 1078, 1134, 1135, 1137, - /* 210 */ 1087, 1092, 1093, 1128, 1132, 1147, 1146, 1161, 1163, 1164, - /* 220 */ 1169, 1170, 1175, 1174, 1101, 1165, 1136, 1168, 1178, 1120, - /* 230 */ 1176, 1184, 1179, 1180, 1181, 1191, 1183, 1200, 1166, 1167, - /* 240 */ 1171, 1173, 1187, 1188, 1189, 1190, 1193, 1194, 1196, 1197, - /* 250 */ 1201, 1172, 1185, 1205, 1129, 1133, 1138, 1203, 1141, 1151, - /* 260 */ 1208, 1214, 1199, 1202, 1238, 1204, 1210, 1211, 1139, 1212, - /* 270 */ 1219, 1148, 1213, 1222, 1225, 1009, 1150, 1153, 1215, 1186, - /* 280 */ 1216, 1221, 1217, 1144, 1218, 1226, 954, 1294, 1228, 1298, - /* 290 */ 1292, 1299, 1250, 1249, 1273, 1287, 1289, 1293, 1300, 1255, - /* 300 */ 1301, 1280, 1329, 1314, 1336, 1348, 1246, 1317, 1310, 1333, - /* 310 */ 1357, 1355, 1362, 1365, 1307, 1313, 1303, 1308, 1367, 1368, - /* 320 */ 1371, 1370, 1388, + /* 0 */ 142, -282, -133, -26, 158, 307, 542, 850, 878, 911, + /* 10 */ 1001, 1029, 1091, 1158, 1220, 1283, 1344, -340, 455, 88, + /* 20 */ 1362, 1423, 1481, 1500, 1580, 1601, 1667, 1704, 1725, 1783, + /* 30 */ 1804, 1862, 1894, 1922, 1993, 2009, 2067, 2098, 2125, 2156, + /* 40 */ 2214, 2236, 2303, 2335, 2361, 2440, 2461, 2477, 2498, 2566, + /* 50 */ 2588, 790, -376, 303, 186, -406, -13, -11, 465, 487, + /* 60 */ 326, -365, 778, -311, 268, -370, 373, -313, -348, -221, + /* 70 */ -190, -372, -342, -72, -361, 363, -204, -187, -176, 280, + /* 80 */ 287, 339, 134, 155, 409, 450, 528, 204, -183, 580, + /* 90 */ 612, -351, 149, 675, 679, 22, 408, 682, 108, 688, + /* 100 */ 212, 72, 615, 697, 425, 750, 391, 641, 462, 759, + /* 110 */ 570, -341, -343, -343, -182, -325, 216, 264, 454, 495, + /* 120 */ 713, 723, 724, 736, 758, 804, 805, 809, 827, 842, + /* 130 */ 843, 844, 846, 863, 887, -68, 5, 418, -278, -167, + /* 140 */ 75, 257, 5, 44, 263, -31, 483, 491, 298, 566, + /* 150 */ 412, 796, 819, 829, 457, -401, 895, -35, 886, 479, + /* 160 */ 576, 684, 836, -387, 145, 367, -387, 446, 503, 596, + /* 170 */ 571, 469, 881, 712, 582, 993, 1008, 997, 975, 997, + /* 180 */ 1025, 992, 1049, 1050, 1012, 1003, 945, 945, 927, 945, + /* 190 */ 959, 949, 997, 1009, 995, 1010, 1022, 1094, 1033, 1096, + /* 200 */ 1055, 1121, 1078, 1080, 1127, 1088, 1087, 1139, 1142, 1141, + /* 210 */ 1092, 1098, 1099, 1134, 1143, 1146, 1145, 1152, 1153, 1154, + /* 220 */ 1167, 1168, 1169, 1170, 1095, 1155, 1126, 1162, 1179, 1123, + /* 230 */ 1180, 1193, 1181, 1187, 1189, 1194, 1191, 1204, 1172, 1173, + /* 240 */ 1174, 1175, 1176, 1177, 1178, 1210, 1223, 1171, 1182, 1183, + /* 250 */ 1138, 1163, 1184, 1239, 1186, 1195, 1205, 1242, 1198, 1190, + /* 260 */ 1253, 1206, 1216, 1227, 1157, 1214, 1230, 1159, 1217, 1235, + /* 270 */ 1237, 997, 1161, 1160, 1197, 1200, 1202, 1164, 1225, 1156, + /* 280 */ 1201, 1207, 945, 1288, 1215, 1211, 1298, 1294, 1301, 1257, + /* 290 */ 1256, 1273, 1278, 1281, 1284, 1286, 1272, 1289, 1285, 1324, + /* 300 */ 1309, 1329, 1338, 1243, 1311, 1305, 1328, 1348, 1343, 1361, + /* 310 */ 1364, 1293, 1279, 1297, 1304, 1333, 1367, 1368, 1369, 1384, }; static const YYACTIONTYPE yy_default[] = { - /* 0 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 10 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 20 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 30 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 40 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 50 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 60 */ 2091, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 70 */ 1780, 1780, 1780, 1780, 2064, 1780, 1780, 1780, 1780, 1780, - /* 80 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 90 */ 1780, 1869, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 100 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 110 */ 1867, 2057, 2283, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 120 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 130 */ 1780, 1780, 1780, 1780, 1780, 1780, 2295, 1780, 1780, 1843, - /* 140 */ 1843, 1780, 2295, 2295, 2295, 1867, 2255, 2255, 1780, 1869, - /* 150 */ 2125, 1780, 1780, 1780, 1780, 1780, 1780, 1989, 1780, 1780, - /* 160 */ 1780, 1780, 1780, 2013, 1780, 1780, 1780, 2117, 1780, 1780, - /* 170 */ 2320, 2377, 1780, 1780, 2323, 1780, 1780, 1780, 1780, 1780, - /* 180 */ 1780, 2069, 1780, 1780, 1942, 2310, 2287, 2301, 2361, 2288, - /* 190 */ 2285, 2304, 1780, 2314, 1780, 1780, 2139, 1869, 1780, 1869, - /* 200 */ 2104, 1780, 2062, 1780, 1780, 2062, 2059, 1780, 1780, 1780, - /* 210 */ 2062, 2059, 2059, 1931, 1927, 1780, 1925, 1780, 1780, 1780, - /* 220 */ 1780, 1827, 1780, 1827, 1780, 1869, 1780, 1869, 1780, 1780, - /* 230 */ 1869, 1780, 1869, 1869, 1869, 1780, 1869, 1780, 1780, 1780, - /* 240 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 250 */ 1780, 1780, 1780, 1780, 2137, 2123, 1780, 1867, 2115, 2113, - /* 260 */ 1780, 1867, 2111, 2314, 1780, 1780, 1780, 1780, 2331, 2329, - /* 270 */ 1780, 2331, 2329, 1780, 1780, 1780, 2345, 2341, 2331, 2350, - /* 280 */ 2347, 2316, 2314, 2380, 2367, 2363, 2301, 1780, 1780, 1780, - /* 290 */ 1867, 1867, 1780, 2329, 1780, 1780, 1780, 1780, 1780, 2329, - /* 300 */ 1780, 1780, 1867, 1780, 1867, 1780, 1780, 1958, 1780, 1780, - /* 310 */ 1780, 1867, 1812, 1780, 2106, 2128, 2087, 2087, 1992, 1992, - /* 320 */ 1992, 1870, 1785, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 330 */ 1780, 1780, 1780, 1780, 1780, 2344, 2343, 2210, 1780, 2259, - /* 340 */ 2258, 2257, 2248, 2209, 1954, 1780, 2208, 2207, 1780, 1780, - /* 350 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 2078, 2077, 2201, - /* 360 */ 1780, 1780, 2202, 2200, 2199, 1780, 1780, 1780, 1780, 1780, - /* 370 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 380 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 2364, - /* 390 */ 2368, 1780, 1780, 1780, 1780, 1780, 1780, 2284, 1780, 1780, - /* 400 */ 1780, 2183, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 410 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 420 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 430 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 440 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 450 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 460 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 470 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 480 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 490 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 500 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 510 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 520 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 530 */ 1780, 1780, 1780, 1780, 1780, 1817, 2188, 1780, 1780, 1780, - /* 540 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 550 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 560 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 570 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 580 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1908, 1907, 1780, - /* 590 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 600 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 610 */ 1780, 2192, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 620 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 630 */ 2360, 2317, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 640 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 650 */ 2183, 1780, 2342, 1780, 1780, 2358, 1780, 2362, 1780, 1780, - /* 660 */ 1780, 1780, 1780, 1780, 1780, 2294, 2290, 1780, 1780, 2286, - /* 670 */ 1780, 1780, 1780, 1780, 1780, 2191, 1780, 1780, 1780, 1780, - /* 680 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 2182, - /* 690 */ 1780, 2245, 1780, 1780, 1780, 2279, 1780, 1780, 2230, 1780, - /* 700 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 2192, 1780, - /* 710 */ 2195, 1780, 1780, 1780, 1780, 1780, 1986, 1780, 1780, 1780, - /* 720 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 730 */ 1780, 1780, 1970, 1968, 1967, 1966, 1780, 1999, 1780, 1780, - /* 740 */ 1780, 1995, 1994, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 750 */ 1780, 1780, 1780, 1780, 1780, 1780, 1888, 1780, 1780, 1780, - /* 760 */ 1780, 1780, 1780, 1780, 1780, 1880, 1780, 1879, 1780, 1780, - /* 770 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 780 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, - /* 790 */ 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, + /* 0 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 10 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 20 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 30 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 40 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 50 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 60 */ 2096, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 70 */ 1784, 1784, 1784, 1784, 2069, 1784, 1784, 1784, 1784, 1784, + /* 80 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 90 */ 1784, 1873, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 100 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 110 */ 1871, 2062, 2288, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 120 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 130 */ 1784, 1784, 1784, 1784, 1784, 1784, 2300, 1784, 1784, 1847, + /* 140 */ 1847, 1784, 2300, 2300, 2300, 1871, 2260, 2260, 1784, 1873, + /* 150 */ 1784, 1784, 1784, 1784, 2130, 1784, 1784, 1784, 1784, 1784, + /* 160 */ 1784, 1994, 1784, 2018, 1784, 1784, 1784, 2122, 1784, 1784, + /* 170 */ 2327, 2384, 1784, 1784, 2330, 1784, 1784, 1784, 1784, 1784, + /* 180 */ 1784, 2074, 1784, 1784, 1946, 2317, 2292, 2306, 2368, 2293, + /* 190 */ 2290, 2311, 1784, 2321, 1784, 1784, 2144, 1873, 1784, 1873, + /* 200 */ 2109, 1784, 2067, 1784, 1784, 2067, 2064, 1784, 1784, 1784, + /* 210 */ 2067, 2064, 2064, 1935, 1931, 1784, 1929, 1784, 1784, 1784, + /* 220 */ 1784, 1831, 1784, 1831, 1784, 1873, 1784, 1873, 1784, 1784, + /* 230 */ 1873, 1784, 1873, 1873, 1873, 1784, 1873, 1784, 1784, 1784, + /* 240 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 250 */ 2142, 2128, 1784, 1871, 2120, 2118, 1784, 1871, 2116, 2321, + /* 260 */ 1784, 1784, 1784, 1784, 2338, 2336, 1784, 2338, 2336, 1784, + /* 270 */ 1784, 1784, 2352, 2348, 2338, 2357, 2354, 2323, 2321, 2387, + /* 280 */ 2374, 2370, 2306, 1784, 1784, 2309, 1784, 1871, 1871, 1784, + /* 290 */ 2336, 1784, 1784, 1784, 1784, 1784, 2336, 1784, 1784, 1871, + /* 300 */ 1784, 1871, 1784, 1784, 1962, 1784, 1784, 1784, 1871, 1816, + /* 310 */ 1784, 2111, 2133, 2092, 2092, 1997, 1997, 1997, 1874, 1789, + /* 320 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 330 */ 1784, 1784, 2351, 2350, 2215, 1784, 2264, 2263, 2262, 2253, + /* 340 */ 2214, 1958, 1784, 2213, 2212, 1784, 1784, 1784, 1784, 1784, + /* 350 */ 1784, 1784, 1784, 1784, 2083, 2082, 2206, 1784, 1784, 2207, + /* 360 */ 2205, 2204, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 370 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 380 */ 1784, 1784, 1784, 1784, 1784, 1784, 2371, 2375, 1784, 1784, + /* 390 */ 1784, 1784, 1784, 1784, 2289, 1784, 1784, 1784, 2188, 1784, + /* 400 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 410 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 420 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 430 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 440 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 450 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 460 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 470 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 480 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 490 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 500 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 510 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 520 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 530 */ 1784, 1784, 1784, 1821, 2193, 1784, 1784, 1784, 1784, 1784, + /* 540 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 550 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 560 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 570 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 580 */ 1784, 1784, 1784, 1784, 1784, 1912, 1911, 1784, 1784, 1784, + /* 590 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 600 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 2197, + /* 610 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 620 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 2367, 2324, + /* 630 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 640 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 2188, 1784, + /* 650 */ 2349, 1784, 1784, 2365, 1784, 2369, 1784, 1784, 1784, 1784, + /* 660 */ 1784, 1784, 1784, 2299, 2295, 1784, 1784, 2291, 1784, 1784, + /* 670 */ 1784, 1784, 1784, 2196, 1784, 1784, 1784, 1784, 1784, 1784, + /* 680 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 2187, 1784, 2250, + /* 690 */ 1784, 1784, 1784, 2284, 1784, 1784, 2235, 1784, 1784, 1784, + /* 700 */ 1784, 1784, 1784, 1784, 1784, 1784, 2197, 1784, 2200, 1784, + /* 710 */ 1784, 1784, 1784, 1784, 1991, 1784, 1784, 1784, 1784, 1784, + /* 720 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 730 */ 1975, 1973, 1972, 1971, 1784, 1968, 1784, 2004, 1784, 1784, + /* 740 */ 1784, 2000, 1999, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 750 */ 1784, 1784, 1784, 1784, 1784, 1784, 1892, 1784, 1784, 1784, + /* 760 */ 1784, 1784, 1784, 1784, 1784, 1884, 1784, 1883, 1784, 1784, + /* 770 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 780 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, + /* 790 */ 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, }; /********** End of lemon-generated parsing tables *****************************/ @@ -1167,6 +1539,7 @@ static const YYCODETYPE yyFallback[] = { 0, /* NK_EQ => nothing */ 0, /* USING => nothing */ 0, /* TAGS => nothing */ + 0, /* COMMENT => nothing */ 0, /* BOOL => nothing */ 0, /* TINYINT => nothing */ 0, /* SMALLINT => nothing */ @@ -1185,7 +1558,6 @@ static const YYCODETYPE yyFallback[] = { 0, /* VARBINARY => nothing */ 0, /* GEOMETRY => nothing */ 0, /* DECIMAL => nothing */ - 0, /* COMMENT => nothing */ 0, /* MAX_DELAY => nothing */ 0, /* WATERMARK => nothing */ 0, /* ROLLUP => nothing */ @@ -1441,6 +1813,7 @@ struct yyParser { }; typedef struct yyParser yyParser; +#include #ifndef NDEBUG #include static FILE *yyTraceFILE = 0; @@ -1592,25 +1965,25 @@ static const char *const yyTokenName[] = { /* 112 */ "NK_EQ", /* 113 */ "USING", /* 114 */ "TAGS", - /* 115 */ "BOOL", - /* 116 */ "TINYINT", - /* 117 */ "SMALLINT", - /* 118 */ "INT", - /* 119 */ "INTEGER", - /* 120 */ "BIGINT", - /* 121 */ "FLOAT", - /* 122 */ "DOUBLE", - /* 123 */ "BINARY", - /* 124 */ "NCHAR", - /* 125 */ "UNSIGNED", - /* 126 */ "JSON", - /* 127 */ "VARCHAR", - /* 128 */ "MEDIUMBLOB", - /* 129 */ "BLOB", - /* 130 */ "VARBINARY", - /* 131 */ "GEOMETRY", - /* 132 */ "DECIMAL", - /* 133 */ "COMMENT", + /* 115 */ "COMMENT", + /* 116 */ "BOOL", + /* 117 */ "TINYINT", + /* 118 */ "SMALLINT", + /* 119 */ "INT", + /* 120 */ "INTEGER", + /* 121 */ "BIGINT", + /* 122 */ "FLOAT", + /* 123 */ "DOUBLE", + /* 124 */ "BINARY", + /* 125 */ "NCHAR", + /* 126 */ "UNSIGNED", + /* 127 */ "JSON", + /* 128 */ "VARCHAR", + /* 129 */ "MEDIUMBLOB", + /* 130 */ "BLOB", + /* 131 */ "VARBINARY", + /* 132 */ "GEOMETRY", + /* 133 */ "DECIMAL", /* 134 */ "MAX_DELAY", /* 135 */ "WATERMARK", /* 136 */ "ROLLUP", @@ -1856,15 +2229,15 @@ static const char *const yyTokenName[] = { /* 376 */ "multi_drop_clause", /* 377 */ "alter_table_clause", /* 378 */ "alter_table_options", - /* 379 */ "column_name", - /* 380 */ "type_name", + /* 379 */ "column_def", + /* 380 */ "column_name", /* 381 */ "signed_literal", /* 382 */ "create_subtable_clause", /* 383 */ "specific_cols_opt", /* 384 */ "expression_list", /* 385 */ "drop_table_clause", /* 386 */ "col_name_list", - /* 387 */ "column_def", + /* 387 */ "type_name", /* 388 */ "duration_list", /* 389 */ "rollup_func_list", /* 390 */ "alter_table_option", @@ -1939,31 +2312,32 @@ static const char *const yyTokenName[] = { /* 459 */ "parenthesized_joined_table", /* 460 */ "join_type", /* 461 */ "query_specification", - /* 462 */ "set_quantifier_opt", - /* 463 */ "select_list", - /* 464 */ "partition_by_clause_opt", - /* 465 */ "range_opt", - /* 466 */ "every_opt", - /* 467 */ "fill_opt", - /* 468 */ "twindow_clause_opt", - /* 469 */ "group_by_clause_opt", - /* 470 */ "having_clause_opt", - /* 471 */ "select_item", - /* 472 */ "partition_list", - /* 473 */ "partition_item", - /* 474 */ "fill_mode", - /* 475 */ "group_by_list", - /* 476 */ "query_expression", - /* 477 */ "query_simple", - /* 478 */ "order_by_clause_opt", - /* 479 */ "slimit_clause_opt", - /* 480 */ "limit_clause_opt", - /* 481 */ "union_query_expression", - /* 482 */ "query_simple_or_subquery", - /* 483 */ "sort_specification_list", - /* 484 */ "sort_specification", - /* 485 */ "ordering_specification_opt", - /* 486 */ "null_ordering_opt", + /* 462 */ "tag_mode_opt", + /* 463 */ "set_quantifier_opt", + /* 464 */ "select_list", + /* 465 */ "partition_by_clause_opt", + /* 466 */ "range_opt", + /* 467 */ "every_opt", + /* 468 */ "fill_opt", + /* 469 */ "twindow_clause_opt", + /* 470 */ "group_by_clause_opt", + /* 471 */ "having_clause_opt", + /* 472 */ "select_item", + /* 473 */ "partition_list", + /* 474 */ "partition_item", + /* 475 */ "fill_mode", + /* 476 */ "group_by_list", + /* 477 */ "query_expression", + /* 478 */ "query_simple", + /* 479 */ "order_by_clause_opt", + /* 480 */ "slimit_clause_opt", + /* 481 */ "limit_clause_opt", + /* 482 */ "union_query_expression", + /* 483 */ "query_simple_or_subquery", + /* 484 */ "sort_specification_list", + /* 485 */ "sort_specification", + /* 486 */ "ordering_specification_opt", + /* 487 */ "null_ordering_opt", }; #endif /* defined(YYCOVERAGE) || !defined(NDEBUG) */ @@ -2131,13 +2505,13 @@ static const char *const yyRuleName[] = { /* 157 */ "cmd ::= ALTER TABLE alter_table_clause", /* 158 */ "cmd ::= ALTER STABLE alter_table_clause", /* 159 */ "alter_table_clause ::= full_table_name alter_table_options", - /* 160 */ "alter_table_clause ::= full_table_name ADD COLUMN column_name type_name", + /* 160 */ "alter_table_clause ::= full_table_name ADD COLUMN column_def", /* 161 */ "alter_table_clause ::= full_table_name DROP COLUMN column_name", - /* 162 */ "alter_table_clause ::= full_table_name MODIFY COLUMN column_name type_name", + /* 162 */ "alter_table_clause ::= full_table_name MODIFY COLUMN column_def", /* 163 */ "alter_table_clause ::= full_table_name RENAME COLUMN column_name column_name", - /* 164 */ "alter_table_clause ::= full_table_name ADD TAG column_name type_name", + /* 164 */ "alter_table_clause ::= full_table_name ADD TAG column_def", /* 165 */ "alter_table_clause ::= full_table_name DROP TAG column_name", - /* 166 */ "alter_table_clause ::= full_table_name MODIFY TAG column_name type_name", + /* 166 */ "alter_table_clause ::= full_table_name MODIFY TAG column_def", /* 167 */ "alter_table_clause ::= full_table_name RENAME TAG column_name column_name", /* 168 */ "alter_table_clause ::= full_table_name SET TAG column_name NK_EQ signed_literal", /* 169 */ "multi_create_clause ::= create_subtable_clause", @@ -2153,424 +2527,427 @@ static const char *const yyRuleName[] = { /* 179 */ "column_def_list ::= column_def", /* 180 */ "column_def_list ::= column_def_list NK_COMMA column_def", /* 181 */ "column_def ::= column_name type_name", - /* 182 */ "type_name ::= BOOL", - /* 183 */ "type_name ::= TINYINT", - /* 184 */ "type_name ::= SMALLINT", - /* 185 */ "type_name ::= INT", - /* 186 */ "type_name ::= INTEGER", - /* 187 */ "type_name ::= BIGINT", - /* 188 */ "type_name ::= FLOAT", - /* 189 */ "type_name ::= DOUBLE", - /* 190 */ "type_name ::= BINARY NK_LP NK_INTEGER NK_RP", - /* 191 */ "type_name ::= TIMESTAMP", - /* 192 */ "type_name ::= NCHAR NK_LP NK_INTEGER NK_RP", - /* 193 */ "type_name ::= TINYINT UNSIGNED", - /* 194 */ "type_name ::= SMALLINT UNSIGNED", - /* 195 */ "type_name ::= INT UNSIGNED", - /* 196 */ "type_name ::= BIGINT UNSIGNED", - /* 197 */ "type_name ::= JSON", - /* 198 */ "type_name ::= VARCHAR NK_LP NK_INTEGER NK_RP", - /* 199 */ "type_name ::= MEDIUMBLOB", - /* 200 */ "type_name ::= BLOB", - /* 201 */ "type_name ::= VARBINARY NK_LP NK_INTEGER NK_RP", - /* 202 */ "type_name ::= GEOMETRY NK_LP NK_INTEGER NK_RP", - /* 203 */ "type_name ::= DECIMAL", - /* 204 */ "type_name ::= DECIMAL NK_LP NK_INTEGER NK_RP", - /* 205 */ "type_name ::= DECIMAL NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP", - /* 206 */ "tags_def_opt ::=", - /* 207 */ "tags_def_opt ::= tags_def", - /* 208 */ "tags_def ::= TAGS NK_LP column_def_list NK_RP", - /* 209 */ "table_options ::=", - /* 210 */ "table_options ::= table_options COMMENT NK_STRING", - /* 211 */ "table_options ::= table_options MAX_DELAY duration_list", - /* 212 */ "table_options ::= table_options WATERMARK duration_list", - /* 213 */ "table_options ::= table_options ROLLUP NK_LP rollup_func_list NK_RP", - /* 214 */ "table_options ::= table_options TTL NK_INTEGER", - /* 215 */ "table_options ::= table_options SMA NK_LP col_name_list NK_RP", - /* 216 */ "table_options ::= table_options DELETE_MARK duration_list", - /* 217 */ "alter_table_options ::= alter_table_option", - /* 218 */ "alter_table_options ::= alter_table_options alter_table_option", - /* 219 */ "alter_table_option ::= COMMENT NK_STRING", - /* 220 */ "alter_table_option ::= TTL NK_INTEGER", - /* 221 */ "duration_list ::= duration_literal", - /* 222 */ "duration_list ::= duration_list NK_COMMA duration_literal", - /* 223 */ "rollup_func_list ::= rollup_func_name", - /* 224 */ "rollup_func_list ::= rollup_func_list NK_COMMA rollup_func_name", - /* 225 */ "rollup_func_name ::= function_name", - /* 226 */ "rollup_func_name ::= FIRST", - /* 227 */ "rollup_func_name ::= LAST", - /* 228 */ "col_name_list ::= col_name", - /* 229 */ "col_name_list ::= col_name_list NK_COMMA col_name", - /* 230 */ "col_name ::= column_name", - /* 231 */ "cmd ::= SHOW DNODES", - /* 232 */ "cmd ::= SHOW USERS", - /* 233 */ "cmd ::= SHOW USER PRIVILEGES", - /* 234 */ "cmd ::= SHOW DATABASES", - /* 235 */ "cmd ::= SHOW db_name_cond_opt TABLES like_pattern_opt", - /* 236 */ "cmd ::= SHOW db_name_cond_opt STABLES like_pattern_opt", - /* 237 */ "cmd ::= SHOW db_name_cond_opt VGROUPS", - /* 238 */ "cmd ::= SHOW MNODES", - /* 239 */ "cmd ::= SHOW QNODES", - /* 240 */ "cmd ::= SHOW FUNCTIONS", - /* 241 */ "cmd ::= SHOW INDEXES FROM table_name_cond from_db_opt", - /* 242 */ "cmd ::= SHOW INDEXES FROM db_name NK_DOT table_name", - /* 243 */ "cmd ::= SHOW STREAMS", - /* 244 */ "cmd ::= SHOW ACCOUNTS", - /* 245 */ "cmd ::= SHOW APPS", - /* 246 */ "cmd ::= SHOW CONNECTIONS", - /* 247 */ "cmd ::= SHOW LICENCES", - /* 248 */ "cmd ::= SHOW GRANTS", - /* 249 */ "cmd ::= SHOW CREATE DATABASE db_name", - /* 250 */ "cmd ::= SHOW CREATE TABLE full_table_name", - /* 251 */ "cmd ::= SHOW CREATE STABLE full_table_name", - /* 252 */ "cmd ::= SHOW QUERIES", - /* 253 */ "cmd ::= SHOW SCORES", - /* 254 */ "cmd ::= SHOW TOPICS", - /* 255 */ "cmd ::= SHOW VARIABLES", - /* 256 */ "cmd ::= SHOW CLUSTER VARIABLES", - /* 257 */ "cmd ::= SHOW LOCAL VARIABLES", - /* 258 */ "cmd ::= SHOW DNODE NK_INTEGER VARIABLES like_pattern_opt", - /* 259 */ "cmd ::= SHOW BNODES", - /* 260 */ "cmd ::= SHOW SNODES", - /* 261 */ "cmd ::= SHOW CLUSTER", - /* 262 */ "cmd ::= SHOW TRANSACTIONS", - /* 263 */ "cmd ::= SHOW TABLE DISTRIBUTED full_table_name", - /* 264 */ "cmd ::= SHOW CONSUMERS", - /* 265 */ "cmd ::= SHOW SUBSCRIPTIONS", - /* 266 */ "cmd ::= SHOW TAGS FROM table_name_cond from_db_opt", - /* 267 */ "cmd ::= SHOW TAGS FROM db_name NK_DOT table_name", - /* 268 */ "cmd ::= SHOW TABLE TAGS tag_list_opt FROM table_name_cond from_db_opt", - /* 269 */ "cmd ::= SHOW TABLE TAGS tag_list_opt FROM db_name NK_DOT table_name", - /* 270 */ "cmd ::= SHOW VNODES NK_INTEGER", - /* 271 */ "cmd ::= SHOW VNODES NK_STRING", - /* 272 */ "cmd ::= SHOW db_name_cond_opt ALIVE", - /* 273 */ "cmd ::= SHOW CLUSTER ALIVE", - /* 274 */ "db_name_cond_opt ::=", - /* 275 */ "db_name_cond_opt ::= db_name NK_DOT", - /* 276 */ "like_pattern_opt ::=", - /* 277 */ "like_pattern_opt ::= LIKE NK_STRING", - /* 278 */ "table_name_cond ::= table_name", - /* 279 */ "from_db_opt ::=", - /* 280 */ "from_db_opt ::= FROM db_name", - /* 281 */ "tag_list_opt ::=", - /* 282 */ "tag_list_opt ::= tag_item", - /* 283 */ "tag_list_opt ::= tag_list_opt NK_COMMA tag_item", - /* 284 */ "tag_item ::= TBNAME", - /* 285 */ "tag_item ::= QTAGS", - /* 286 */ "tag_item ::= column_name", - /* 287 */ "tag_item ::= column_name column_alias", - /* 288 */ "tag_item ::= column_name AS column_alias", - /* 289 */ "cmd ::= CREATE SMA INDEX not_exists_opt full_index_name ON full_table_name index_options", - /* 290 */ "cmd ::= CREATE INDEX not_exists_opt full_index_name ON full_table_name NK_LP col_name_list NK_RP", - /* 291 */ "cmd ::= DROP INDEX exists_opt full_index_name", - /* 292 */ "full_index_name ::= index_name", - /* 293 */ "full_index_name ::= db_name NK_DOT index_name", - /* 294 */ "index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_RP sliding_opt sma_stream_opt", - /* 295 */ "index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_COMMA duration_literal NK_RP sliding_opt sma_stream_opt", - /* 296 */ "func_list ::= func", - /* 297 */ "func_list ::= func_list NK_COMMA func", - /* 298 */ "func ::= sma_func_name NK_LP expression_list NK_RP", - /* 299 */ "sma_func_name ::= function_name", - /* 300 */ "sma_func_name ::= COUNT", - /* 301 */ "sma_func_name ::= FIRST", - /* 302 */ "sma_func_name ::= LAST", - /* 303 */ "sma_func_name ::= LAST_ROW", - /* 304 */ "sma_stream_opt ::=", - /* 305 */ "sma_stream_opt ::= sma_stream_opt WATERMARK duration_literal", - /* 306 */ "sma_stream_opt ::= sma_stream_opt MAX_DELAY duration_literal", - /* 307 */ "sma_stream_opt ::= sma_stream_opt DELETE_MARK duration_literal", - /* 308 */ "with_meta ::= AS", - /* 309 */ "with_meta ::= WITH META AS", - /* 310 */ "with_meta ::= ONLY META AS", - /* 311 */ "cmd ::= CREATE TOPIC not_exists_opt topic_name AS query_or_subquery", - /* 312 */ "cmd ::= CREATE TOPIC not_exists_opt topic_name with_meta DATABASE db_name", - /* 313 */ "cmd ::= CREATE TOPIC not_exists_opt topic_name with_meta STABLE full_table_name where_clause_opt", - /* 314 */ "cmd ::= DROP TOPIC exists_opt topic_name", - /* 315 */ "cmd ::= DROP CONSUMER GROUP exists_opt cgroup_name ON topic_name", - /* 316 */ "cmd ::= DESC full_table_name", - /* 317 */ "cmd ::= DESCRIBE full_table_name", - /* 318 */ "cmd ::= RESET QUERY CACHE", - /* 319 */ "cmd ::= EXPLAIN analyze_opt explain_options query_or_subquery", - /* 320 */ "cmd ::= EXPLAIN analyze_opt explain_options insert_query", - /* 321 */ "analyze_opt ::=", - /* 322 */ "analyze_opt ::= ANALYZE", - /* 323 */ "explain_options ::=", - /* 324 */ "explain_options ::= explain_options VERBOSE NK_BOOL", - /* 325 */ "explain_options ::= explain_options RATIO NK_FLOAT", - /* 326 */ "cmd ::= CREATE or_replace_opt agg_func_opt FUNCTION not_exists_opt function_name AS NK_STRING OUTPUTTYPE type_name bufsize_opt language_opt", - /* 327 */ "cmd ::= DROP FUNCTION exists_opt function_name", - /* 328 */ "agg_func_opt ::=", - /* 329 */ "agg_func_opt ::= AGGREGATE", - /* 330 */ "bufsize_opt ::=", - /* 331 */ "bufsize_opt ::= BUFSIZE NK_INTEGER", - /* 332 */ "language_opt ::=", - /* 333 */ "language_opt ::= LANGUAGE NK_STRING", - /* 334 */ "or_replace_opt ::=", - /* 335 */ "or_replace_opt ::= OR REPLACE", - /* 336 */ "cmd ::= CREATE STREAM not_exists_opt stream_name stream_options INTO full_table_name col_list_opt tag_def_or_ref_opt subtable_opt AS query_or_subquery", - /* 337 */ "cmd ::= DROP STREAM exists_opt stream_name", - /* 338 */ "cmd ::= PAUSE STREAM exists_opt stream_name", - /* 339 */ "cmd ::= RESUME STREAM exists_opt ignore_opt stream_name", - /* 340 */ "col_list_opt ::=", - /* 341 */ "col_list_opt ::= NK_LP col_name_list NK_RP", - /* 342 */ "tag_def_or_ref_opt ::=", - /* 343 */ "tag_def_or_ref_opt ::= tags_def", - /* 344 */ "tag_def_or_ref_opt ::= TAGS NK_LP col_name_list NK_RP", - /* 345 */ "stream_options ::=", - /* 346 */ "stream_options ::= stream_options TRIGGER AT_ONCE", - /* 347 */ "stream_options ::= stream_options TRIGGER WINDOW_CLOSE", - /* 348 */ "stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal", - /* 349 */ "stream_options ::= stream_options WATERMARK duration_literal", - /* 350 */ "stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER", - /* 351 */ "stream_options ::= stream_options FILL_HISTORY NK_INTEGER", - /* 352 */ "stream_options ::= stream_options DELETE_MARK duration_literal", - /* 353 */ "stream_options ::= stream_options IGNORE UPDATE NK_INTEGER", - /* 354 */ "subtable_opt ::=", - /* 355 */ "subtable_opt ::= SUBTABLE NK_LP expression NK_RP", - /* 356 */ "ignore_opt ::=", - /* 357 */ "ignore_opt ::= IGNORE UNTREATED", - /* 358 */ "cmd ::= KILL CONNECTION NK_INTEGER", - /* 359 */ "cmd ::= KILL QUERY NK_STRING", - /* 360 */ "cmd ::= KILL TRANSACTION NK_INTEGER", - /* 361 */ "cmd ::= BALANCE VGROUP", - /* 362 */ "cmd ::= BALANCE VGROUP LEADER", - /* 363 */ "cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER", - /* 364 */ "cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list", - /* 365 */ "cmd ::= SPLIT VGROUP NK_INTEGER", - /* 366 */ "dnode_list ::= DNODE NK_INTEGER", - /* 367 */ "dnode_list ::= dnode_list DNODE NK_INTEGER", - /* 368 */ "cmd ::= DELETE FROM full_table_name where_clause_opt", - /* 369 */ "cmd ::= query_or_subquery", - /* 370 */ "cmd ::= insert_query", - /* 371 */ "insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery", - /* 372 */ "insert_query ::= INSERT INTO full_table_name query_or_subquery", - /* 373 */ "literal ::= NK_INTEGER", - /* 374 */ "literal ::= NK_FLOAT", - /* 375 */ "literal ::= NK_STRING", - /* 376 */ "literal ::= NK_BOOL", - /* 377 */ "literal ::= TIMESTAMP NK_STRING", - /* 378 */ "literal ::= duration_literal", - /* 379 */ "literal ::= NULL", - /* 380 */ "literal ::= NK_QUESTION", - /* 381 */ "duration_literal ::= NK_VARIABLE", - /* 382 */ "signed ::= NK_INTEGER", - /* 383 */ "signed ::= NK_PLUS NK_INTEGER", - /* 384 */ "signed ::= NK_MINUS NK_INTEGER", - /* 385 */ "signed ::= NK_FLOAT", - /* 386 */ "signed ::= NK_PLUS NK_FLOAT", - /* 387 */ "signed ::= NK_MINUS NK_FLOAT", - /* 388 */ "signed_literal ::= signed", - /* 389 */ "signed_literal ::= NK_STRING", - /* 390 */ "signed_literal ::= NK_BOOL", - /* 391 */ "signed_literal ::= TIMESTAMP NK_STRING", - /* 392 */ "signed_literal ::= duration_literal", - /* 393 */ "signed_literal ::= NULL", - /* 394 */ "signed_literal ::= literal_func", - /* 395 */ "signed_literal ::= NK_QUESTION", - /* 396 */ "literal_list ::= signed_literal", - /* 397 */ "literal_list ::= literal_list NK_COMMA signed_literal", - /* 398 */ "db_name ::= NK_ID", - /* 399 */ "table_name ::= NK_ID", - /* 400 */ "column_name ::= NK_ID", - /* 401 */ "function_name ::= NK_ID", - /* 402 */ "table_alias ::= NK_ID", - /* 403 */ "column_alias ::= NK_ID", - /* 404 */ "user_name ::= NK_ID", - /* 405 */ "topic_name ::= NK_ID", - /* 406 */ "stream_name ::= NK_ID", - /* 407 */ "cgroup_name ::= NK_ID", - /* 408 */ "index_name ::= NK_ID", - /* 409 */ "expr_or_subquery ::= expression", - /* 410 */ "expression ::= literal", - /* 411 */ "expression ::= pseudo_column", - /* 412 */ "expression ::= column_reference", - /* 413 */ "expression ::= function_expression", - /* 414 */ "expression ::= case_when_expression", - /* 415 */ "expression ::= NK_LP expression NK_RP", - /* 416 */ "expression ::= NK_PLUS expr_or_subquery", - /* 417 */ "expression ::= NK_MINUS expr_or_subquery", - /* 418 */ "expression ::= expr_or_subquery NK_PLUS expr_or_subquery", - /* 419 */ "expression ::= expr_or_subquery NK_MINUS expr_or_subquery", - /* 420 */ "expression ::= expr_or_subquery NK_STAR expr_or_subquery", - /* 421 */ "expression ::= expr_or_subquery NK_SLASH expr_or_subquery", - /* 422 */ "expression ::= expr_or_subquery NK_REM expr_or_subquery", - /* 423 */ "expression ::= column_reference NK_ARROW NK_STRING", - /* 424 */ "expression ::= expr_or_subquery NK_BITAND expr_or_subquery", - /* 425 */ "expression ::= expr_or_subquery NK_BITOR expr_or_subquery", - /* 426 */ "expression_list ::= expr_or_subquery", - /* 427 */ "expression_list ::= expression_list NK_COMMA expr_or_subquery", - /* 428 */ "column_reference ::= column_name", - /* 429 */ "column_reference ::= table_name NK_DOT column_name", - /* 430 */ "pseudo_column ::= ROWTS", - /* 431 */ "pseudo_column ::= TBNAME", - /* 432 */ "pseudo_column ::= table_name NK_DOT TBNAME", - /* 433 */ "pseudo_column ::= QSTART", - /* 434 */ "pseudo_column ::= QEND", - /* 435 */ "pseudo_column ::= QDURATION", - /* 436 */ "pseudo_column ::= WSTART", - /* 437 */ "pseudo_column ::= WEND", - /* 438 */ "pseudo_column ::= WDURATION", - /* 439 */ "pseudo_column ::= IROWTS", - /* 440 */ "pseudo_column ::= ISFILLED", - /* 441 */ "pseudo_column ::= QTAGS", - /* 442 */ "function_expression ::= function_name NK_LP expression_list NK_RP", - /* 443 */ "function_expression ::= star_func NK_LP star_func_para_list NK_RP", - /* 444 */ "function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP", - /* 445 */ "function_expression ::= literal_func", - /* 446 */ "literal_func ::= noarg_func NK_LP NK_RP", - /* 447 */ "literal_func ::= NOW", - /* 448 */ "noarg_func ::= NOW", - /* 449 */ "noarg_func ::= TODAY", - /* 450 */ "noarg_func ::= TIMEZONE", - /* 451 */ "noarg_func ::= DATABASE", - /* 452 */ "noarg_func ::= CLIENT_VERSION", - /* 453 */ "noarg_func ::= SERVER_VERSION", - /* 454 */ "noarg_func ::= SERVER_STATUS", - /* 455 */ "noarg_func ::= CURRENT_USER", - /* 456 */ "noarg_func ::= USER", - /* 457 */ "star_func ::= COUNT", - /* 458 */ "star_func ::= FIRST", - /* 459 */ "star_func ::= LAST", - /* 460 */ "star_func ::= LAST_ROW", - /* 461 */ "star_func_para_list ::= NK_STAR", - /* 462 */ "star_func_para_list ::= other_para_list", - /* 463 */ "other_para_list ::= star_func_para", - /* 464 */ "other_para_list ::= other_para_list NK_COMMA star_func_para", - /* 465 */ "star_func_para ::= expr_or_subquery", - /* 466 */ "star_func_para ::= table_name NK_DOT NK_STAR", - /* 467 */ "case_when_expression ::= CASE when_then_list case_when_else_opt END", - /* 468 */ "case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END", - /* 469 */ "when_then_list ::= when_then_expr", - /* 470 */ "when_then_list ::= when_then_list when_then_expr", - /* 471 */ "when_then_expr ::= WHEN common_expression THEN common_expression", - /* 472 */ "case_when_else_opt ::=", - /* 473 */ "case_when_else_opt ::= ELSE common_expression", - /* 474 */ "predicate ::= expr_or_subquery compare_op expr_or_subquery", - /* 475 */ "predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery", - /* 476 */ "predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery", - /* 477 */ "predicate ::= expr_or_subquery IS NULL", - /* 478 */ "predicate ::= expr_or_subquery IS NOT NULL", - /* 479 */ "predicate ::= expr_or_subquery in_op in_predicate_value", - /* 480 */ "compare_op ::= NK_LT", - /* 481 */ "compare_op ::= NK_GT", - /* 482 */ "compare_op ::= NK_LE", - /* 483 */ "compare_op ::= NK_GE", - /* 484 */ "compare_op ::= NK_NE", - /* 485 */ "compare_op ::= NK_EQ", - /* 486 */ "compare_op ::= LIKE", - /* 487 */ "compare_op ::= NOT LIKE", - /* 488 */ "compare_op ::= MATCH", - /* 489 */ "compare_op ::= NMATCH", - /* 490 */ "compare_op ::= CONTAINS", - /* 491 */ "in_op ::= IN", - /* 492 */ "in_op ::= NOT IN", - /* 493 */ "in_predicate_value ::= NK_LP literal_list NK_RP", - /* 494 */ "boolean_value_expression ::= boolean_primary", - /* 495 */ "boolean_value_expression ::= NOT boolean_primary", - /* 496 */ "boolean_value_expression ::= boolean_value_expression OR boolean_value_expression", - /* 497 */ "boolean_value_expression ::= boolean_value_expression AND boolean_value_expression", - /* 498 */ "boolean_primary ::= predicate", - /* 499 */ "boolean_primary ::= NK_LP boolean_value_expression NK_RP", - /* 500 */ "common_expression ::= expr_or_subquery", - /* 501 */ "common_expression ::= boolean_value_expression", - /* 502 */ "from_clause_opt ::=", - /* 503 */ "from_clause_opt ::= FROM table_reference_list", - /* 504 */ "table_reference_list ::= table_reference", - /* 505 */ "table_reference_list ::= table_reference_list NK_COMMA table_reference", - /* 506 */ "table_reference ::= table_primary", - /* 507 */ "table_reference ::= joined_table", - /* 508 */ "table_primary ::= table_name alias_opt", - /* 509 */ "table_primary ::= db_name NK_DOT table_name alias_opt", - /* 510 */ "table_primary ::= subquery alias_opt", - /* 511 */ "table_primary ::= parenthesized_joined_table", - /* 512 */ "alias_opt ::=", - /* 513 */ "alias_opt ::= table_alias", - /* 514 */ "alias_opt ::= AS table_alias", - /* 515 */ "parenthesized_joined_table ::= NK_LP joined_table NK_RP", - /* 516 */ "parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP", - /* 517 */ "joined_table ::= table_reference join_type JOIN table_reference ON search_condition", - /* 518 */ "join_type ::=", - /* 519 */ "join_type ::= INNER", - /* 520 */ "query_specification ::= SELECT set_quantifier_opt select_list from_clause_opt where_clause_opt partition_by_clause_opt range_opt every_opt fill_opt twindow_clause_opt group_by_clause_opt having_clause_opt", - /* 521 */ "set_quantifier_opt ::=", - /* 522 */ "set_quantifier_opt ::= DISTINCT", - /* 523 */ "set_quantifier_opt ::= ALL", - /* 524 */ "select_list ::= select_item", - /* 525 */ "select_list ::= select_list NK_COMMA select_item", - /* 526 */ "select_item ::= NK_STAR", - /* 527 */ "select_item ::= common_expression", - /* 528 */ "select_item ::= common_expression column_alias", - /* 529 */ "select_item ::= common_expression AS column_alias", - /* 530 */ "select_item ::= table_name NK_DOT NK_STAR", - /* 531 */ "where_clause_opt ::=", - /* 532 */ "where_clause_opt ::= WHERE search_condition", - /* 533 */ "partition_by_clause_opt ::=", - /* 534 */ "partition_by_clause_opt ::= PARTITION BY partition_list", - /* 535 */ "partition_list ::= partition_item", - /* 536 */ "partition_list ::= partition_list NK_COMMA partition_item", - /* 537 */ "partition_item ::= expr_or_subquery", - /* 538 */ "partition_item ::= expr_or_subquery column_alias", - /* 539 */ "partition_item ::= expr_or_subquery AS column_alias", - /* 540 */ "twindow_clause_opt ::=", - /* 541 */ "twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA duration_literal NK_RP", - /* 542 */ "twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP", - /* 543 */ "twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_RP sliding_opt fill_opt", - /* 544 */ "twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_COMMA duration_literal NK_RP sliding_opt fill_opt", - /* 545 */ "twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition", - /* 546 */ "sliding_opt ::=", - /* 547 */ "sliding_opt ::= SLIDING NK_LP duration_literal NK_RP", - /* 548 */ "fill_opt ::=", - /* 549 */ "fill_opt ::= FILL NK_LP fill_mode NK_RP", - /* 550 */ "fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP", - /* 551 */ "fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP", - /* 552 */ "fill_mode ::= NONE", - /* 553 */ "fill_mode ::= PREV", - /* 554 */ "fill_mode ::= NULL", - /* 555 */ "fill_mode ::= NULL_F", - /* 556 */ "fill_mode ::= LINEAR", - /* 557 */ "fill_mode ::= NEXT", - /* 558 */ "group_by_clause_opt ::=", - /* 559 */ "group_by_clause_opt ::= GROUP BY group_by_list", - /* 560 */ "group_by_list ::= expr_or_subquery", - /* 561 */ "group_by_list ::= group_by_list NK_COMMA expr_or_subquery", - /* 562 */ "having_clause_opt ::=", - /* 563 */ "having_clause_opt ::= HAVING search_condition", - /* 564 */ "range_opt ::=", - /* 565 */ "range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP", - /* 566 */ "range_opt ::= RANGE NK_LP expr_or_subquery NK_RP", - /* 567 */ "every_opt ::=", - /* 568 */ "every_opt ::= EVERY NK_LP duration_literal NK_RP", - /* 569 */ "query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt", - /* 570 */ "query_simple ::= query_specification", - /* 571 */ "query_simple ::= union_query_expression", - /* 572 */ "union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery", - /* 573 */ "union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery", - /* 574 */ "query_simple_or_subquery ::= query_simple", - /* 575 */ "query_simple_or_subquery ::= subquery", - /* 576 */ "query_or_subquery ::= query_expression", - /* 577 */ "query_or_subquery ::= subquery", - /* 578 */ "order_by_clause_opt ::=", - /* 579 */ "order_by_clause_opt ::= ORDER BY sort_specification_list", - /* 580 */ "slimit_clause_opt ::=", - /* 581 */ "slimit_clause_opt ::= SLIMIT NK_INTEGER", - /* 582 */ "slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER", - /* 583 */ "slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER", - /* 584 */ "limit_clause_opt ::=", - /* 585 */ "limit_clause_opt ::= LIMIT NK_INTEGER", - /* 586 */ "limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER", - /* 587 */ "limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER", - /* 588 */ "subquery ::= NK_LP query_expression NK_RP", - /* 589 */ "subquery ::= NK_LP subquery NK_RP", - /* 590 */ "search_condition ::= common_expression", - /* 591 */ "sort_specification_list ::= sort_specification", - /* 592 */ "sort_specification_list ::= sort_specification_list NK_COMMA sort_specification", - /* 593 */ "sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt", - /* 594 */ "ordering_specification_opt ::=", - /* 595 */ "ordering_specification_opt ::= ASC", - /* 596 */ "ordering_specification_opt ::= DESC", - /* 597 */ "null_ordering_opt ::=", - /* 598 */ "null_ordering_opt ::= NULLS FIRST", - /* 599 */ "null_ordering_opt ::= NULLS LAST", + /* 182 */ "column_def ::= column_name type_name COMMENT NK_STRING", + /* 183 */ "type_name ::= BOOL", + /* 184 */ "type_name ::= TINYINT", + /* 185 */ "type_name ::= SMALLINT", + /* 186 */ "type_name ::= INT", + /* 187 */ "type_name ::= INTEGER", + /* 188 */ "type_name ::= BIGINT", + /* 189 */ "type_name ::= FLOAT", + /* 190 */ "type_name ::= DOUBLE", + /* 191 */ "type_name ::= BINARY NK_LP NK_INTEGER NK_RP", + /* 192 */ "type_name ::= TIMESTAMP", + /* 193 */ "type_name ::= NCHAR NK_LP NK_INTEGER NK_RP", + /* 194 */ "type_name ::= TINYINT UNSIGNED", + /* 195 */ "type_name ::= SMALLINT UNSIGNED", + /* 196 */ "type_name ::= INT UNSIGNED", + /* 197 */ "type_name ::= BIGINT UNSIGNED", + /* 198 */ "type_name ::= JSON", + /* 199 */ "type_name ::= VARCHAR NK_LP NK_INTEGER NK_RP", + /* 200 */ "type_name ::= MEDIUMBLOB", + /* 201 */ "type_name ::= BLOB", + /* 202 */ "type_name ::= VARBINARY NK_LP NK_INTEGER NK_RP", + /* 203 */ "type_name ::= GEOMETRY NK_LP NK_INTEGER NK_RP", + /* 204 */ "type_name ::= DECIMAL", + /* 205 */ "type_name ::= DECIMAL NK_LP NK_INTEGER NK_RP", + /* 206 */ "type_name ::= DECIMAL NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP", + /* 207 */ "tags_def_opt ::=", + /* 208 */ "tags_def_opt ::= tags_def", + /* 209 */ "tags_def ::= TAGS NK_LP column_def_list NK_RP", + /* 210 */ "table_options ::=", + /* 211 */ "table_options ::= table_options COMMENT NK_STRING", + /* 212 */ "table_options ::= table_options MAX_DELAY duration_list", + /* 213 */ "table_options ::= table_options WATERMARK duration_list", + /* 214 */ "table_options ::= table_options ROLLUP NK_LP rollup_func_list NK_RP", + /* 215 */ "table_options ::= table_options TTL NK_INTEGER", + /* 216 */ "table_options ::= table_options SMA NK_LP col_name_list NK_RP", + /* 217 */ "table_options ::= table_options DELETE_MARK duration_list", + /* 218 */ "alter_table_options ::= alter_table_option", + /* 219 */ "alter_table_options ::= alter_table_options alter_table_option", + /* 220 */ "alter_table_option ::= COMMENT NK_STRING", + /* 221 */ "alter_table_option ::= TTL NK_INTEGER", + /* 222 */ "duration_list ::= duration_literal", + /* 223 */ "duration_list ::= duration_list NK_COMMA duration_literal", + /* 224 */ "rollup_func_list ::= rollup_func_name", + /* 225 */ "rollup_func_list ::= rollup_func_list NK_COMMA rollup_func_name", + /* 226 */ "rollup_func_name ::= function_name", + /* 227 */ "rollup_func_name ::= FIRST", + /* 228 */ "rollup_func_name ::= LAST", + /* 229 */ "col_name_list ::= col_name", + /* 230 */ "col_name_list ::= col_name_list NK_COMMA col_name", + /* 231 */ "col_name ::= column_name", + /* 232 */ "cmd ::= SHOW DNODES", + /* 233 */ "cmd ::= SHOW USERS", + /* 234 */ "cmd ::= SHOW USER PRIVILEGES", + /* 235 */ "cmd ::= SHOW DATABASES", + /* 236 */ "cmd ::= SHOW db_name_cond_opt TABLES like_pattern_opt", + /* 237 */ "cmd ::= SHOW db_name_cond_opt STABLES like_pattern_opt", + /* 238 */ "cmd ::= SHOW db_name_cond_opt VGROUPS", + /* 239 */ "cmd ::= SHOW MNODES", + /* 240 */ "cmd ::= SHOW QNODES", + /* 241 */ "cmd ::= SHOW FUNCTIONS", + /* 242 */ "cmd ::= SHOW INDEXES FROM table_name_cond from_db_opt", + /* 243 */ "cmd ::= SHOW INDEXES FROM db_name NK_DOT table_name", + /* 244 */ "cmd ::= SHOW STREAMS", + /* 245 */ "cmd ::= SHOW ACCOUNTS", + /* 246 */ "cmd ::= SHOW APPS", + /* 247 */ "cmd ::= SHOW CONNECTIONS", + /* 248 */ "cmd ::= SHOW LICENCES", + /* 249 */ "cmd ::= SHOW GRANTS", + /* 250 */ "cmd ::= SHOW CREATE DATABASE db_name", + /* 251 */ "cmd ::= SHOW CREATE TABLE full_table_name", + /* 252 */ "cmd ::= SHOW CREATE STABLE full_table_name", + /* 253 */ "cmd ::= SHOW QUERIES", + /* 254 */ "cmd ::= SHOW SCORES", + /* 255 */ "cmd ::= SHOW TOPICS", + /* 256 */ "cmd ::= SHOW VARIABLES", + /* 257 */ "cmd ::= SHOW CLUSTER VARIABLES", + /* 258 */ "cmd ::= SHOW LOCAL VARIABLES", + /* 259 */ "cmd ::= SHOW DNODE NK_INTEGER VARIABLES like_pattern_opt", + /* 260 */ "cmd ::= SHOW BNODES", + /* 261 */ "cmd ::= SHOW SNODES", + /* 262 */ "cmd ::= SHOW CLUSTER", + /* 263 */ "cmd ::= SHOW TRANSACTIONS", + /* 264 */ "cmd ::= SHOW TABLE DISTRIBUTED full_table_name", + /* 265 */ "cmd ::= SHOW CONSUMERS", + /* 266 */ "cmd ::= SHOW SUBSCRIPTIONS", + /* 267 */ "cmd ::= SHOW TAGS FROM table_name_cond from_db_opt", + /* 268 */ "cmd ::= SHOW TAGS FROM db_name NK_DOT table_name", + /* 269 */ "cmd ::= SHOW TABLE TAGS tag_list_opt FROM table_name_cond from_db_opt", + /* 270 */ "cmd ::= SHOW TABLE TAGS tag_list_opt FROM db_name NK_DOT table_name", + /* 271 */ "cmd ::= SHOW VNODES ON DNODE NK_INTEGER", + /* 272 */ "cmd ::= SHOW VNODES", + /* 273 */ "cmd ::= SHOW db_name_cond_opt ALIVE", + /* 274 */ "cmd ::= SHOW CLUSTER ALIVE", + /* 275 */ "db_name_cond_opt ::=", + /* 276 */ "db_name_cond_opt ::= db_name NK_DOT", + /* 277 */ "like_pattern_opt ::=", + /* 278 */ "like_pattern_opt ::= LIKE NK_STRING", + /* 279 */ "table_name_cond ::= table_name", + /* 280 */ "from_db_opt ::=", + /* 281 */ "from_db_opt ::= FROM db_name", + /* 282 */ "tag_list_opt ::=", + /* 283 */ "tag_list_opt ::= tag_item", + /* 284 */ "tag_list_opt ::= tag_list_opt NK_COMMA tag_item", + /* 285 */ "tag_item ::= TBNAME", + /* 286 */ "tag_item ::= QTAGS", + /* 287 */ "tag_item ::= column_name", + /* 288 */ "tag_item ::= column_name column_alias", + /* 289 */ "tag_item ::= column_name AS column_alias", + /* 290 */ "cmd ::= CREATE SMA INDEX not_exists_opt full_index_name ON full_table_name index_options", + /* 291 */ "cmd ::= CREATE INDEX not_exists_opt full_index_name ON full_table_name NK_LP col_name_list NK_RP", + /* 292 */ "cmd ::= DROP INDEX exists_opt full_index_name", + /* 293 */ "full_index_name ::= index_name", + /* 294 */ "full_index_name ::= db_name NK_DOT index_name", + /* 295 */ "index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_RP sliding_opt sma_stream_opt", + /* 296 */ "index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_COMMA duration_literal NK_RP sliding_opt sma_stream_opt", + /* 297 */ "func_list ::= func", + /* 298 */ "func_list ::= func_list NK_COMMA func", + /* 299 */ "func ::= sma_func_name NK_LP expression_list NK_RP", + /* 300 */ "sma_func_name ::= function_name", + /* 301 */ "sma_func_name ::= COUNT", + /* 302 */ "sma_func_name ::= FIRST", + /* 303 */ "sma_func_name ::= LAST", + /* 304 */ "sma_func_name ::= LAST_ROW", + /* 305 */ "sma_stream_opt ::=", + /* 306 */ "sma_stream_opt ::= sma_stream_opt WATERMARK duration_literal", + /* 307 */ "sma_stream_opt ::= sma_stream_opt MAX_DELAY duration_literal", + /* 308 */ "sma_stream_opt ::= sma_stream_opt DELETE_MARK duration_literal", + /* 309 */ "with_meta ::= AS", + /* 310 */ "with_meta ::= WITH META AS", + /* 311 */ "with_meta ::= ONLY META AS", + /* 312 */ "cmd ::= CREATE TOPIC not_exists_opt topic_name AS query_or_subquery", + /* 313 */ "cmd ::= CREATE TOPIC not_exists_opt topic_name with_meta DATABASE db_name", + /* 314 */ "cmd ::= CREATE TOPIC not_exists_opt topic_name with_meta STABLE full_table_name where_clause_opt", + /* 315 */ "cmd ::= DROP TOPIC exists_opt topic_name", + /* 316 */ "cmd ::= DROP CONSUMER GROUP exists_opt cgroup_name ON topic_name", + /* 317 */ "cmd ::= DESC full_table_name", + /* 318 */ "cmd ::= DESCRIBE full_table_name", + /* 319 */ "cmd ::= RESET QUERY CACHE", + /* 320 */ "cmd ::= EXPLAIN analyze_opt explain_options query_or_subquery", + /* 321 */ "cmd ::= EXPLAIN analyze_opt explain_options insert_query", + /* 322 */ "analyze_opt ::=", + /* 323 */ "analyze_opt ::= ANALYZE", + /* 324 */ "explain_options ::=", + /* 325 */ "explain_options ::= explain_options VERBOSE NK_BOOL", + /* 326 */ "explain_options ::= explain_options RATIO NK_FLOAT", + /* 327 */ "cmd ::= CREATE or_replace_opt agg_func_opt FUNCTION not_exists_opt function_name AS NK_STRING OUTPUTTYPE type_name bufsize_opt language_opt", + /* 328 */ "cmd ::= DROP FUNCTION exists_opt function_name", + /* 329 */ "agg_func_opt ::=", + /* 330 */ "agg_func_opt ::= AGGREGATE", + /* 331 */ "bufsize_opt ::=", + /* 332 */ "bufsize_opt ::= BUFSIZE NK_INTEGER", + /* 333 */ "language_opt ::=", + /* 334 */ "language_opt ::= LANGUAGE NK_STRING", + /* 335 */ "or_replace_opt ::=", + /* 336 */ "or_replace_opt ::= OR REPLACE", + /* 337 */ "cmd ::= CREATE STREAM not_exists_opt stream_name stream_options INTO full_table_name col_list_opt tag_def_or_ref_opt subtable_opt AS query_or_subquery", + /* 338 */ "cmd ::= DROP STREAM exists_opt stream_name", + /* 339 */ "cmd ::= PAUSE STREAM exists_opt stream_name", + /* 340 */ "cmd ::= RESUME STREAM exists_opt ignore_opt stream_name", + /* 341 */ "col_list_opt ::=", + /* 342 */ "col_list_opt ::= NK_LP col_name_list NK_RP", + /* 343 */ "tag_def_or_ref_opt ::=", + /* 344 */ "tag_def_or_ref_opt ::= tags_def", + /* 345 */ "tag_def_or_ref_opt ::= TAGS NK_LP col_name_list NK_RP", + /* 346 */ "stream_options ::=", + /* 347 */ "stream_options ::= stream_options TRIGGER AT_ONCE", + /* 348 */ "stream_options ::= stream_options TRIGGER WINDOW_CLOSE", + /* 349 */ "stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal", + /* 350 */ "stream_options ::= stream_options WATERMARK duration_literal", + /* 351 */ "stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER", + /* 352 */ "stream_options ::= stream_options FILL_HISTORY NK_INTEGER", + /* 353 */ "stream_options ::= stream_options DELETE_MARK duration_literal", + /* 354 */ "stream_options ::= stream_options IGNORE UPDATE NK_INTEGER", + /* 355 */ "subtable_opt ::=", + /* 356 */ "subtable_opt ::= SUBTABLE NK_LP expression NK_RP", + /* 357 */ "ignore_opt ::=", + /* 358 */ "ignore_opt ::= IGNORE UNTREATED", + /* 359 */ "cmd ::= KILL CONNECTION NK_INTEGER", + /* 360 */ "cmd ::= KILL QUERY NK_STRING", + /* 361 */ "cmd ::= KILL TRANSACTION NK_INTEGER", + /* 362 */ "cmd ::= BALANCE VGROUP", + /* 363 */ "cmd ::= BALANCE VGROUP LEADER", + /* 364 */ "cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER", + /* 365 */ "cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list", + /* 366 */ "cmd ::= SPLIT VGROUP NK_INTEGER", + /* 367 */ "dnode_list ::= DNODE NK_INTEGER", + /* 368 */ "dnode_list ::= dnode_list DNODE NK_INTEGER", + /* 369 */ "cmd ::= DELETE FROM full_table_name where_clause_opt", + /* 370 */ "cmd ::= query_or_subquery", + /* 371 */ "cmd ::= insert_query", + /* 372 */ "insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery", + /* 373 */ "insert_query ::= INSERT INTO full_table_name query_or_subquery", + /* 374 */ "literal ::= NK_INTEGER", + /* 375 */ "literal ::= NK_FLOAT", + /* 376 */ "literal ::= NK_STRING", + /* 377 */ "literal ::= NK_BOOL", + /* 378 */ "literal ::= TIMESTAMP NK_STRING", + /* 379 */ "literal ::= duration_literal", + /* 380 */ "literal ::= NULL", + /* 381 */ "literal ::= NK_QUESTION", + /* 382 */ "duration_literal ::= NK_VARIABLE", + /* 383 */ "signed ::= NK_INTEGER", + /* 384 */ "signed ::= NK_PLUS NK_INTEGER", + /* 385 */ "signed ::= NK_MINUS NK_INTEGER", + /* 386 */ "signed ::= NK_FLOAT", + /* 387 */ "signed ::= NK_PLUS NK_FLOAT", + /* 388 */ "signed ::= NK_MINUS NK_FLOAT", + /* 389 */ "signed_literal ::= signed", + /* 390 */ "signed_literal ::= NK_STRING", + /* 391 */ "signed_literal ::= NK_BOOL", + /* 392 */ "signed_literal ::= TIMESTAMP NK_STRING", + /* 393 */ "signed_literal ::= duration_literal", + /* 394 */ "signed_literal ::= NULL", + /* 395 */ "signed_literal ::= literal_func", + /* 396 */ "signed_literal ::= NK_QUESTION", + /* 397 */ "literal_list ::= signed_literal", + /* 398 */ "literal_list ::= literal_list NK_COMMA signed_literal", + /* 399 */ "db_name ::= NK_ID", + /* 400 */ "table_name ::= NK_ID", + /* 401 */ "column_name ::= NK_ID", + /* 402 */ "function_name ::= NK_ID", + /* 403 */ "table_alias ::= NK_ID", + /* 404 */ "column_alias ::= NK_ID", + /* 405 */ "user_name ::= NK_ID", + /* 406 */ "topic_name ::= NK_ID", + /* 407 */ "stream_name ::= NK_ID", + /* 408 */ "cgroup_name ::= NK_ID", + /* 409 */ "index_name ::= NK_ID", + /* 410 */ "expr_or_subquery ::= expression", + /* 411 */ "expression ::= literal", + /* 412 */ "expression ::= pseudo_column", + /* 413 */ "expression ::= column_reference", + /* 414 */ "expression ::= function_expression", + /* 415 */ "expression ::= case_when_expression", + /* 416 */ "expression ::= NK_LP expression NK_RP", + /* 417 */ "expression ::= NK_PLUS expr_or_subquery", + /* 418 */ "expression ::= NK_MINUS expr_or_subquery", + /* 419 */ "expression ::= expr_or_subquery NK_PLUS expr_or_subquery", + /* 420 */ "expression ::= expr_or_subquery NK_MINUS expr_or_subquery", + /* 421 */ "expression ::= expr_or_subquery NK_STAR expr_or_subquery", + /* 422 */ "expression ::= expr_or_subquery NK_SLASH expr_or_subquery", + /* 423 */ "expression ::= expr_or_subquery NK_REM expr_or_subquery", + /* 424 */ "expression ::= column_reference NK_ARROW NK_STRING", + /* 425 */ "expression ::= expr_or_subquery NK_BITAND expr_or_subquery", + /* 426 */ "expression ::= expr_or_subquery NK_BITOR expr_or_subquery", + /* 427 */ "expression_list ::= expr_or_subquery", + /* 428 */ "expression_list ::= expression_list NK_COMMA expr_or_subquery", + /* 429 */ "column_reference ::= column_name", + /* 430 */ "column_reference ::= table_name NK_DOT column_name", + /* 431 */ "pseudo_column ::= ROWTS", + /* 432 */ "pseudo_column ::= TBNAME", + /* 433 */ "pseudo_column ::= table_name NK_DOT TBNAME", + /* 434 */ "pseudo_column ::= QSTART", + /* 435 */ "pseudo_column ::= QEND", + /* 436 */ "pseudo_column ::= QDURATION", + /* 437 */ "pseudo_column ::= WSTART", + /* 438 */ "pseudo_column ::= WEND", + /* 439 */ "pseudo_column ::= WDURATION", + /* 440 */ "pseudo_column ::= IROWTS", + /* 441 */ "pseudo_column ::= ISFILLED", + /* 442 */ "pseudo_column ::= QTAGS", + /* 443 */ "function_expression ::= function_name NK_LP expression_list NK_RP", + /* 444 */ "function_expression ::= star_func NK_LP star_func_para_list NK_RP", + /* 445 */ "function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP", + /* 446 */ "function_expression ::= literal_func", + /* 447 */ "literal_func ::= noarg_func NK_LP NK_RP", + /* 448 */ "literal_func ::= NOW", + /* 449 */ "noarg_func ::= NOW", + /* 450 */ "noarg_func ::= TODAY", + /* 451 */ "noarg_func ::= TIMEZONE", + /* 452 */ "noarg_func ::= DATABASE", + /* 453 */ "noarg_func ::= CLIENT_VERSION", + /* 454 */ "noarg_func ::= SERVER_VERSION", + /* 455 */ "noarg_func ::= SERVER_STATUS", + /* 456 */ "noarg_func ::= CURRENT_USER", + /* 457 */ "noarg_func ::= USER", + /* 458 */ "star_func ::= COUNT", + /* 459 */ "star_func ::= FIRST", + /* 460 */ "star_func ::= LAST", + /* 461 */ "star_func ::= LAST_ROW", + /* 462 */ "star_func_para_list ::= NK_STAR", + /* 463 */ "star_func_para_list ::= other_para_list", + /* 464 */ "other_para_list ::= star_func_para", + /* 465 */ "other_para_list ::= other_para_list NK_COMMA star_func_para", + /* 466 */ "star_func_para ::= expr_or_subquery", + /* 467 */ "star_func_para ::= table_name NK_DOT NK_STAR", + /* 468 */ "case_when_expression ::= CASE when_then_list case_when_else_opt END", + /* 469 */ "case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END", + /* 470 */ "when_then_list ::= when_then_expr", + /* 471 */ "when_then_list ::= when_then_list when_then_expr", + /* 472 */ "when_then_expr ::= WHEN common_expression THEN common_expression", + /* 473 */ "case_when_else_opt ::=", + /* 474 */ "case_when_else_opt ::= ELSE common_expression", + /* 475 */ "predicate ::= expr_or_subquery compare_op expr_or_subquery", + /* 476 */ "predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery", + /* 477 */ "predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery", + /* 478 */ "predicate ::= expr_or_subquery IS NULL", + /* 479 */ "predicate ::= expr_or_subquery IS NOT NULL", + /* 480 */ "predicate ::= expr_or_subquery in_op in_predicate_value", + /* 481 */ "compare_op ::= NK_LT", + /* 482 */ "compare_op ::= NK_GT", + /* 483 */ "compare_op ::= NK_LE", + /* 484 */ "compare_op ::= NK_GE", + /* 485 */ "compare_op ::= NK_NE", + /* 486 */ "compare_op ::= NK_EQ", + /* 487 */ "compare_op ::= LIKE", + /* 488 */ "compare_op ::= NOT LIKE", + /* 489 */ "compare_op ::= MATCH", + /* 490 */ "compare_op ::= NMATCH", + /* 491 */ "compare_op ::= CONTAINS", + /* 492 */ "in_op ::= IN", + /* 493 */ "in_op ::= NOT IN", + /* 494 */ "in_predicate_value ::= NK_LP literal_list NK_RP", + /* 495 */ "boolean_value_expression ::= boolean_primary", + /* 496 */ "boolean_value_expression ::= NOT boolean_primary", + /* 497 */ "boolean_value_expression ::= boolean_value_expression OR boolean_value_expression", + /* 498 */ "boolean_value_expression ::= boolean_value_expression AND boolean_value_expression", + /* 499 */ "boolean_primary ::= predicate", + /* 500 */ "boolean_primary ::= NK_LP boolean_value_expression NK_RP", + /* 501 */ "common_expression ::= expr_or_subquery", + /* 502 */ "common_expression ::= boolean_value_expression", + /* 503 */ "from_clause_opt ::=", + /* 504 */ "from_clause_opt ::= FROM table_reference_list", + /* 505 */ "table_reference_list ::= table_reference", + /* 506 */ "table_reference_list ::= table_reference_list NK_COMMA table_reference", + /* 507 */ "table_reference ::= table_primary", + /* 508 */ "table_reference ::= joined_table", + /* 509 */ "table_primary ::= table_name alias_opt", + /* 510 */ "table_primary ::= db_name NK_DOT table_name alias_opt", + /* 511 */ "table_primary ::= subquery alias_opt", + /* 512 */ "table_primary ::= parenthesized_joined_table", + /* 513 */ "alias_opt ::=", + /* 514 */ "alias_opt ::= table_alias", + /* 515 */ "alias_opt ::= AS table_alias", + /* 516 */ "parenthesized_joined_table ::= NK_LP joined_table NK_RP", + /* 517 */ "parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP", + /* 518 */ "joined_table ::= table_reference join_type JOIN table_reference ON search_condition", + /* 519 */ "join_type ::=", + /* 520 */ "join_type ::= INNER", + /* 521 */ "query_specification ::= SELECT tag_mode_opt set_quantifier_opt select_list from_clause_opt where_clause_opt partition_by_clause_opt range_opt every_opt fill_opt twindow_clause_opt group_by_clause_opt having_clause_opt", + /* 522 */ "tag_mode_opt ::=", + /* 523 */ "tag_mode_opt ::= TAGS", + /* 524 */ "set_quantifier_opt ::=", + /* 525 */ "set_quantifier_opt ::= DISTINCT", + /* 526 */ "set_quantifier_opt ::= ALL", + /* 527 */ "select_list ::= select_item", + /* 528 */ "select_list ::= select_list NK_COMMA select_item", + /* 529 */ "select_item ::= NK_STAR", + /* 530 */ "select_item ::= common_expression", + /* 531 */ "select_item ::= common_expression column_alias", + /* 532 */ "select_item ::= common_expression AS column_alias", + /* 533 */ "select_item ::= table_name NK_DOT NK_STAR", + /* 534 */ "where_clause_opt ::=", + /* 535 */ "where_clause_opt ::= WHERE search_condition", + /* 536 */ "partition_by_clause_opt ::=", + /* 537 */ "partition_by_clause_opt ::= PARTITION BY partition_list", + /* 538 */ "partition_list ::= partition_item", + /* 539 */ "partition_list ::= partition_list NK_COMMA partition_item", + /* 540 */ "partition_item ::= expr_or_subquery", + /* 541 */ "partition_item ::= expr_or_subquery column_alias", + /* 542 */ "partition_item ::= expr_or_subquery AS column_alias", + /* 543 */ "twindow_clause_opt ::=", + /* 544 */ "twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA duration_literal NK_RP", + /* 545 */ "twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP", + /* 546 */ "twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_RP sliding_opt fill_opt", + /* 547 */ "twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_COMMA duration_literal NK_RP sliding_opt fill_opt", + /* 548 */ "twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition", + /* 549 */ "sliding_opt ::=", + /* 550 */ "sliding_opt ::= SLIDING NK_LP duration_literal NK_RP", + /* 551 */ "fill_opt ::=", + /* 552 */ "fill_opt ::= FILL NK_LP fill_mode NK_RP", + /* 553 */ "fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP", + /* 554 */ "fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP", + /* 555 */ "fill_mode ::= NONE", + /* 556 */ "fill_mode ::= PREV", + /* 557 */ "fill_mode ::= NULL", + /* 558 */ "fill_mode ::= NULL_F", + /* 559 */ "fill_mode ::= LINEAR", + /* 560 */ "fill_mode ::= NEXT", + /* 561 */ "group_by_clause_opt ::=", + /* 562 */ "group_by_clause_opt ::= GROUP BY group_by_list", + /* 563 */ "group_by_list ::= expr_or_subquery", + /* 564 */ "group_by_list ::= group_by_list NK_COMMA expr_or_subquery", + /* 565 */ "having_clause_opt ::=", + /* 566 */ "having_clause_opt ::= HAVING search_condition", + /* 567 */ "range_opt ::=", + /* 568 */ "range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP", + /* 569 */ "range_opt ::= RANGE NK_LP expr_or_subquery NK_RP", + /* 570 */ "every_opt ::=", + /* 571 */ "every_opt ::= EVERY NK_LP duration_literal NK_RP", + /* 572 */ "query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt", + /* 573 */ "query_simple ::= query_specification", + /* 574 */ "query_simple ::= union_query_expression", + /* 575 */ "union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery", + /* 576 */ "union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery", + /* 577 */ "query_simple_or_subquery ::= query_simple", + /* 578 */ "query_simple_or_subquery ::= subquery", + /* 579 */ "query_or_subquery ::= query_expression", + /* 580 */ "query_or_subquery ::= subquery", + /* 581 */ "order_by_clause_opt ::=", + /* 582 */ "order_by_clause_opt ::= ORDER BY sort_specification_list", + /* 583 */ "slimit_clause_opt ::=", + /* 584 */ "slimit_clause_opt ::= SLIMIT NK_INTEGER", + /* 585 */ "slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER", + /* 586 */ "slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER", + /* 587 */ "limit_clause_opt ::=", + /* 588 */ "limit_clause_opt ::= LIMIT NK_INTEGER", + /* 589 */ "limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER", + /* 590 */ "limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER", + /* 591 */ "subquery ::= NK_LP query_expression NK_RP", + /* 592 */ "subquery ::= NK_LP subquery NK_RP", + /* 593 */ "search_condition ::= common_expression", + /* 594 */ "sort_specification_list ::= sort_specification", + /* 595 */ "sort_specification_list ::= sort_specification_list NK_COMMA sort_specification", + /* 596 */ "sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt", + /* 597 */ "ordering_specification_opt ::=", + /* 598 */ "ordering_specification_opt ::= ASC", + /* 599 */ "ordering_specification_opt ::= DESC", + /* 600 */ "null_ordering_opt ::=", + /* 601 */ "null_ordering_opt ::= NULLS FIRST", + /* 602 */ "null_ordering_opt ::= NULLS LAST", }; #endif /* NDEBUG */ @@ -2711,10 +3088,10 @@ static void yy_destructor( case 373: /* table_options */ case 377: /* alter_table_clause */ case 378: /* alter_table_options */ + case 379: /* column_def */ case 381: /* signed_literal */ case 382: /* create_subtable_clause */ case 385: /* drop_table_clause */ - case 387: /* column_def */ case 391: /* duration_literal */ case 392: /* rollup_func_name */ case 394: /* col_name */ @@ -2757,22 +3134,24 @@ static void yy_destructor( case 458: /* subquery */ case 459: /* parenthesized_joined_table */ case 461: /* query_specification */ - case 465: /* range_opt */ - case 466: /* every_opt */ - case 467: /* fill_opt */ - case 468: /* twindow_clause_opt */ - case 470: /* having_clause_opt */ - case 471: /* select_item */ - case 473: /* partition_item */ - case 476: /* query_expression */ - case 477: /* query_simple */ - case 479: /* slimit_clause_opt */ - case 480: /* limit_clause_opt */ - case 481: /* union_query_expression */ - case 482: /* query_simple_or_subquery */ - case 484: /* sort_specification */ + case 466: /* range_opt */ + case 467: /* every_opt */ + case 468: /* fill_opt */ + case 469: /* twindow_clause_opt */ + case 471: /* having_clause_opt */ + case 472: /* select_item */ + case 474: /* partition_item */ + case 477: /* query_expression */ + case 478: /* query_simple */ + case 480: /* slimit_clause_opt */ + case 481: /* limit_clause_opt */ + case 482: /* union_query_expression */ + case 483: /* query_simple_or_subquery */ + case 485: /* sort_specification */ { - nodesDestroyNode((yypminor->yy452)); +#line 7 "sql.y" + nodesDestroyNode((yypminor->yy840)); +#line 3154 "sql.c" } break; case 339: /* account_options */ @@ -2782,7 +3161,9 @@ static void yy_destructor( case 410: /* with_meta */ case 419: /* bufsize_opt */ { +#line 54 "sql.y" +#line 3166 "sql.c" } break; case 343: /* user_name */ @@ -2790,7 +3171,7 @@ static void yy_destructor( case 351: /* table_name */ case 352: /* topic_name */ case 354: /* dnode_endpoint */ - case 379: /* column_name */ + case 380: /* column_name */ case 393: /* function_name */ case 401: /* column_alias */ case 404: /* index_name */ @@ -2803,24 +3184,32 @@ static void yy_destructor( case 439: /* noarg_func */ case 457: /* alias_opt */ { +#line 742 "sql.y" +#line 3189 "sql.c" } break; case 344: /* sysinfo_opt */ { +#line 92 "sql.y" +#line 3196 "sql.c" } break; case 345: /* privileges */ case 348: /* priv_type_list */ case 349: /* priv_type */ { +#line 101 "sql.y" +#line 3205 "sql.c" } break; case 346: /* priv_level */ { +#line 117 "sql.y" +#line 3212 "sql.c" } break; case 355: /* force_opt */ @@ -2831,9 +3220,12 @@ static void yy_destructor( case 417: /* or_replace_opt */ case 418: /* agg_func_opt */ case 426: /* ignore_opt */ - case 462: /* set_quantifier_opt */ + case 462: /* tag_mode_opt */ + case 463: /* set_quantifier_opt */ { +#line 146 "sql.y" +#line 3228 "sql.c" } break; case 364: /* integer_list */ @@ -2858,52 +3250,68 @@ static void yy_destructor( case 438: /* star_func_para_list */ case 440: /* other_para_list */ case 442: /* when_then_list */ - case 463: /* select_list */ - case 464: /* partition_by_clause_opt */ - case 469: /* group_by_clause_opt */ - case 472: /* partition_list */ - case 475: /* group_by_list */ - case 478: /* order_by_clause_opt */ - case 483: /* sort_specification_list */ + case 464: /* select_list */ + case 465: /* partition_by_clause_opt */ + case 470: /* group_by_clause_opt */ + case 473: /* partition_list */ + case 476: /* group_by_list */ + case 479: /* order_by_clause_opt */ + case 484: /* sort_specification_list */ { - nodesDestroyList((yypminor->yy812)); +#line 270 "sql.y" + nodesDestroyList((yypminor->yy56)); +#line 3263 "sql.c" } break; case 368: /* alter_db_option */ case 390: /* alter_table_option */ { +#line 243 "sql.y" +#line 3271 "sql.c" } break; - case 380: /* type_name */ + case 387: /* type_name */ { +#line 364 "sql.y" +#line 3278 "sql.c" } break; case 447: /* compare_op */ case 448: /* in_op */ { +#line 930 "sql.y" +#line 3286 "sql.c" } break; case 460: /* join_type */ { +#line 1006 "sql.y" +#line 3293 "sql.c" } break; - case 474: /* fill_mode */ + case 475: /* fill_mode */ { +#line 1087 "sql.y" +#line 3300 "sql.c" } break; - case 485: /* ordering_specification_opt */ + case 486: /* ordering_specification_opt */ { +#line 1172 "sql.y" +#line 3307 "sql.c" } break; - case 486: /* null_ordering_opt */ + case 487: /* null_ordering_opt */ { +#line 1178 "sql.y" +#line 3314 "sql.c" } break; /********* End destructor definitions *****************************************/ @@ -3070,7 +3478,7 @@ static YYACTIONTYPE yy_find_shift_action( #endif /* YYWILDCARD */ return yy_default[stateno]; }else{ - assert( i>=0 && i=0 && i<(int)(sizeof(yy_action)/sizeof(yy_action[0])) ); return yy_action[i]; } }while(1); @@ -3352,13 +3760,13 @@ static const YYCODETYPE yyRuleInfoLhs[] = { 338, /* (157) cmd ::= ALTER TABLE alter_table_clause */ 338, /* (158) cmd ::= ALTER STABLE alter_table_clause */ 377, /* (159) alter_table_clause ::= full_table_name alter_table_options */ - 377, /* (160) alter_table_clause ::= full_table_name ADD COLUMN column_name type_name */ + 377, /* (160) alter_table_clause ::= full_table_name ADD COLUMN column_def */ 377, /* (161) alter_table_clause ::= full_table_name DROP COLUMN column_name */ - 377, /* (162) alter_table_clause ::= full_table_name MODIFY COLUMN column_name type_name */ + 377, /* (162) alter_table_clause ::= full_table_name MODIFY COLUMN column_def */ 377, /* (163) alter_table_clause ::= full_table_name RENAME COLUMN column_name column_name */ - 377, /* (164) alter_table_clause ::= full_table_name ADD TAG column_name type_name */ + 377, /* (164) alter_table_clause ::= full_table_name ADD TAG column_def */ 377, /* (165) alter_table_clause ::= full_table_name DROP TAG column_name */ - 377, /* (166) alter_table_clause ::= full_table_name MODIFY TAG column_name type_name */ + 377, /* (166) alter_table_clause ::= full_table_name MODIFY TAG column_def */ 377, /* (167) alter_table_clause ::= full_table_name RENAME TAG column_name column_name */ 377, /* (168) alter_table_clause ::= full_table_name SET TAG column_name NK_EQ signed_literal */ 374, /* (169) multi_create_clause ::= create_subtable_clause */ @@ -3373,425 +3781,428 @@ static const YYCODETYPE yyRuleInfoLhs[] = { 370, /* (178) full_table_name ::= db_name NK_DOT table_name */ 371, /* (179) column_def_list ::= column_def */ 371, /* (180) column_def_list ::= column_def_list NK_COMMA column_def */ - 387, /* (181) column_def ::= column_name type_name */ - 380, /* (182) type_name ::= BOOL */ - 380, /* (183) type_name ::= TINYINT */ - 380, /* (184) type_name ::= SMALLINT */ - 380, /* (185) type_name ::= INT */ - 380, /* (186) type_name ::= INTEGER */ - 380, /* (187) type_name ::= BIGINT */ - 380, /* (188) type_name ::= FLOAT */ - 380, /* (189) type_name ::= DOUBLE */ - 380, /* (190) type_name ::= BINARY NK_LP NK_INTEGER NK_RP */ - 380, /* (191) type_name ::= TIMESTAMP */ - 380, /* (192) type_name ::= NCHAR NK_LP NK_INTEGER NK_RP */ - 380, /* (193) type_name ::= TINYINT UNSIGNED */ - 380, /* (194) type_name ::= SMALLINT UNSIGNED */ - 380, /* (195) type_name ::= INT UNSIGNED */ - 380, /* (196) type_name ::= BIGINT UNSIGNED */ - 380, /* (197) type_name ::= JSON */ - 380, /* (198) type_name ::= VARCHAR NK_LP NK_INTEGER NK_RP */ - 380, /* (199) type_name ::= MEDIUMBLOB */ - 380, /* (200) type_name ::= BLOB */ - 380, /* (201) type_name ::= VARBINARY NK_LP NK_INTEGER NK_RP */ - 380, /* (202) type_name ::= GEOMETRY NK_LP NK_INTEGER NK_RP */ - 380, /* (203) type_name ::= DECIMAL */ - 380, /* (204) type_name ::= DECIMAL NK_LP NK_INTEGER NK_RP */ - 380, /* (205) type_name ::= DECIMAL NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP */ - 372, /* (206) tags_def_opt ::= */ - 372, /* (207) tags_def_opt ::= tags_def */ - 375, /* (208) tags_def ::= TAGS NK_LP column_def_list NK_RP */ - 373, /* (209) table_options ::= */ - 373, /* (210) table_options ::= table_options COMMENT NK_STRING */ - 373, /* (211) table_options ::= table_options MAX_DELAY duration_list */ - 373, /* (212) table_options ::= table_options WATERMARK duration_list */ - 373, /* (213) table_options ::= table_options ROLLUP NK_LP rollup_func_list NK_RP */ - 373, /* (214) table_options ::= table_options TTL NK_INTEGER */ - 373, /* (215) table_options ::= table_options SMA NK_LP col_name_list NK_RP */ - 373, /* (216) table_options ::= table_options DELETE_MARK duration_list */ - 378, /* (217) alter_table_options ::= alter_table_option */ - 378, /* (218) alter_table_options ::= alter_table_options alter_table_option */ - 390, /* (219) alter_table_option ::= COMMENT NK_STRING */ - 390, /* (220) alter_table_option ::= TTL NK_INTEGER */ - 388, /* (221) duration_list ::= duration_literal */ - 388, /* (222) duration_list ::= duration_list NK_COMMA duration_literal */ - 389, /* (223) rollup_func_list ::= rollup_func_name */ - 389, /* (224) rollup_func_list ::= rollup_func_list NK_COMMA rollup_func_name */ - 392, /* (225) rollup_func_name ::= function_name */ - 392, /* (226) rollup_func_name ::= FIRST */ - 392, /* (227) rollup_func_name ::= LAST */ - 386, /* (228) col_name_list ::= col_name */ - 386, /* (229) col_name_list ::= col_name_list NK_COMMA col_name */ - 394, /* (230) col_name ::= column_name */ - 338, /* (231) cmd ::= SHOW DNODES */ - 338, /* (232) cmd ::= SHOW USERS */ - 338, /* (233) cmd ::= SHOW USER PRIVILEGES */ - 338, /* (234) cmd ::= SHOW DATABASES */ - 338, /* (235) cmd ::= SHOW db_name_cond_opt TABLES like_pattern_opt */ - 338, /* (236) cmd ::= SHOW db_name_cond_opt STABLES like_pattern_opt */ - 338, /* (237) cmd ::= SHOW db_name_cond_opt VGROUPS */ - 338, /* (238) cmd ::= SHOW MNODES */ - 338, /* (239) cmd ::= SHOW QNODES */ - 338, /* (240) cmd ::= SHOW FUNCTIONS */ - 338, /* (241) cmd ::= SHOW INDEXES FROM table_name_cond from_db_opt */ - 338, /* (242) cmd ::= SHOW INDEXES FROM db_name NK_DOT table_name */ - 338, /* (243) cmd ::= SHOW STREAMS */ - 338, /* (244) cmd ::= SHOW ACCOUNTS */ - 338, /* (245) cmd ::= SHOW APPS */ - 338, /* (246) cmd ::= SHOW CONNECTIONS */ - 338, /* (247) cmd ::= SHOW LICENCES */ - 338, /* (248) cmd ::= SHOW GRANTS */ - 338, /* (249) cmd ::= SHOW CREATE DATABASE db_name */ - 338, /* (250) cmd ::= SHOW CREATE TABLE full_table_name */ - 338, /* (251) cmd ::= SHOW CREATE STABLE full_table_name */ - 338, /* (252) cmd ::= SHOW QUERIES */ - 338, /* (253) cmd ::= SHOW SCORES */ - 338, /* (254) cmd ::= SHOW TOPICS */ - 338, /* (255) cmd ::= SHOW VARIABLES */ - 338, /* (256) cmd ::= SHOW CLUSTER VARIABLES */ - 338, /* (257) cmd ::= SHOW LOCAL VARIABLES */ - 338, /* (258) cmd ::= SHOW DNODE NK_INTEGER VARIABLES like_pattern_opt */ - 338, /* (259) cmd ::= SHOW BNODES */ - 338, /* (260) cmd ::= SHOW SNODES */ - 338, /* (261) cmd ::= SHOW CLUSTER */ - 338, /* (262) cmd ::= SHOW TRANSACTIONS */ - 338, /* (263) cmd ::= SHOW TABLE DISTRIBUTED full_table_name */ - 338, /* (264) cmd ::= SHOW CONSUMERS */ - 338, /* (265) cmd ::= SHOW SUBSCRIPTIONS */ - 338, /* (266) cmd ::= SHOW TAGS FROM table_name_cond from_db_opt */ - 338, /* (267) cmd ::= SHOW TAGS FROM db_name NK_DOT table_name */ - 338, /* (268) cmd ::= SHOW TABLE TAGS tag_list_opt FROM table_name_cond from_db_opt */ - 338, /* (269) cmd ::= SHOW TABLE TAGS tag_list_opt FROM db_name NK_DOT table_name */ - 338, /* (270) cmd ::= SHOW VNODES NK_INTEGER */ - 338, /* (271) cmd ::= SHOW VNODES NK_STRING */ - 338, /* (272) cmd ::= SHOW db_name_cond_opt ALIVE */ - 338, /* (273) cmd ::= SHOW CLUSTER ALIVE */ - 395, /* (274) db_name_cond_opt ::= */ - 395, /* (275) db_name_cond_opt ::= db_name NK_DOT */ - 396, /* (276) like_pattern_opt ::= */ - 396, /* (277) like_pattern_opt ::= LIKE NK_STRING */ - 397, /* (278) table_name_cond ::= table_name */ - 398, /* (279) from_db_opt ::= */ - 398, /* (280) from_db_opt ::= FROM db_name */ - 399, /* (281) tag_list_opt ::= */ - 399, /* (282) tag_list_opt ::= tag_item */ - 399, /* (283) tag_list_opt ::= tag_list_opt NK_COMMA tag_item */ - 400, /* (284) tag_item ::= TBNAME */ - 400, /* (285) tag_item ::= QTAGS */ - 400, /* (286) tag_item ::= column_name */ - 400, /* (287) tag_item ::= column_name column_alias */ - 400, /* (288) tag_item ::= column_name AS column_alias */ - 338, /* (289) cmd ::= CREATE SMA INDEX not_exists_opt full_index_name ON full_table_name index_options */ - 338, /* (290) cmd ::= CREATE INDEX not_exists_opt full_index_name ON full_table_name NK_LP col_name_list NK_RP */ - 338, /* (291) cmd ::= DROP INDEX exists_opt full_index_name */ - 402, /* (292) full_index_name ::= index_name */ - 402, /* (293) full_index_name ::= db_name NK_DOT index_name */ - 403, /* (294) index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_RP sliding_opt sma_stream_opt */ - 403, /* (295) index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_COMMA duration_literal NK_RP sliding_opt sma_stream_opt */ - 405, /* (296) func_list ::= func */ - 405, /* (297) func_list ::= func_list NK_COMMA func */ - 408, /* (298) func ::= sma_func_name NK_LP expression_list NK_RP */ - 409, /* (299) sma_func_name ::= function_name */ - 409, /* (300) sma_func_name ::= COUNT */ - 409, /* (301) sma_func_name ::= FIRST */ - 409, /* (302) sma_func_name ::= LAST */ - 409, /* (303) sma_func_name ::= LAST_ROW */ - 407, /* (304) sma_stream_opt ::= */ - 407, /* (305) sma_stream_opt ::= sma_stream_opt WATERMARK duration_literal */ - 407, /* (306) sma_stream_opt ::= sma_stream_opt MAX_DELAY duration_literal */ - 407, /* (307) sma_stream_opt ::= sma_stream_opt DELETE_MARK duration_literal */ - 410, /* (308) with_meta ::= AS */ - 410, /* (309) with_meta ::= WITH META AS */ - 410, /* (310) with_meta ::= ONLY META AS */ - 338, /* (311) cmd ::= CREATE TOPIC not_exists_opt topic_name AS query_or_subquery */ - 338, /* (312) cmd ::= CREATE TOPIC not_exists_opt topic_name with_meta DATABASE db_name */ - 338, /* (313) cmd ::= CREATE TOPIC not_exists_opt topic_name with_meta STABLE full_table_name where_clause_opt */ - 338, /* (314) cmd ::= DROP TOPIC exists_opt topic_name */ - 338, /* (315) cmd ::= DROP CONSUMER GROUP exists_opt cgroup_name ON topic_name */ - 338, /* (316) cmd ::= DESC full_table_name */ - 338, /* (317) cmd ::= DESCRIBE full_table_name */ - 338, /* (318) cmd ::= RESET QUERY CACHE */ - 338, /* (319) cmd ::= EXPLAIN analyze_opt explain_options query_or_subquery */ - 338, /* (320) cmd ::= EXPLAIN analyze_opt explain_options insert_query */ - 414, /* (321) analyze_opt ::= */ - 414, /* (322) analyze_opt ::= ANALYZE */ - 415, /* (323) explain_options ::= */ - 415, /* (324) explain_options ::= explain_options VERBOSE NK_BOOL */ - 415, /* (325) explain_options ::= explain_options RATIO NK_FLOAT */ - 338, /* (326) cmd ::= CREATE or_replace_opt agg_func_opt FUNCTION not_exists_opt function_name AS NK_STRING OUTPUTTYPE type_name bufsize_opt language_opt */ - 338, /* (327) cmd ::= DROP FUNCTION exists_opt function_name */ - 418, /* (328) agg_func_opt ::= */ - 418, /* (329) agg_func_opt ::= AGGREGATE */ - 419, /* (330) bufsize_opt ::= */ - 419, /* (331) bufsize_opt ::= BUFSIZE NK_INTEGER */ - 420, /* (332) language_opt ::= */ - 420, /* (333) language_opt ::= LANGUAGE NK_STRING */ - 417, /* (334) or_replace_opt ::= */ - 417, /* (335) or_replace_opt ::= OR REPLACE */ - 338, /* (336) cmd ::= CREATE STREAM not_exists_opt stream_name stream_options INTO full_table_name col_list_opt tag_def_or_ref_opt subtable_opt AS query_or_subquery */ - 338, /* (337) cmd ::= DROP STREAM exists_opt stream_name */ - 338, /* (338) cmd ::= PAUSE STREAM exists_opt stream_name */ - 338, /* (339) cmd ::= RESUME STREAM exists_opt ignore_opt stream_name */ - 423, /* (340) col_list_opt ::= */ - 423, /* (341) col_list_opt ::= NK_LP col_name_list NK_RP */ - 424, /* (342) tag_def_or_ref_opt ::= */ - 424, /* (343) tag_def_or_ref_opt ::= tags_def */ - 424, /* (344) tag_def_or_ref_opt ::= TAGS NK_LP col_name_list NK_RP */ - 422, /* (345) stream_options ::= */ - 422, /* (346) stream_options ::= stream_options TRIGGER AT_ONCE */ - 422, /* (347) stream_options ::= stream_options TRIGGER WINDOW_CLOSE */ - 422, /* (348) stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal */ - 422, /* (349) stream_options ::= stream_options WATERMARK duration_literal */ - 422, /* (350) stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER */ - 422, /* (351) stream_options ::= stream_options FILL_HISTORY NK_INTEGER */ - 422, /* (352) stream_options ::= stream_options DELETE_MARK duration_literal */ - 422, /* (353) stream_options ::= stream_options IGNORE UPDATE NK_INTEGER */ - 425, /* (354) subtable_opt ::= */ - 425, /* (355) subtable_opt ::= SUBTABLE NK_LP expression NK_RP */ - 426, /* (356) ignore_opt ::= */ - 426, /* (357) ignore_opt ::= IGNORE UNTREATED */ - 338, /* (358) cmd ::= KILL CONNECTION NK_INTEGER */ - 338, /* (359) cmd ::= KILL QUERY NK_STRING */ - 338, /* (360) cmd ::= KILL TRANSACTION NK_INTEGER */ - 338, /* (361) cmd ::= BALANCE VGROUP */ - 338, /* (362) cmd ::= BALANCE VGROUP LEADER */ - 338, /* (363) cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER */ - 338, /* (364) cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list */ - 338, /* (365) cmd ::= SPLIT VGROUP NK_INTEGER */ - 428, /* (366) dnode_list ::= DNODE NK_INTEGER */ - 428, /* (367) dnode_list ::= dnode_list DNODE NK_INTEGER */ - 338, /* (368) cmd ::= DELETE FROM full_table_name where_clause_opt */ - 338, /* (369) cmd ::= query_or_subquery */ - 338, /* (370) cmd ::= insert_query */ - 416, /* (371) insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery */ - 416, /* (372) insert_query ::= INSERT INTO full_table_name query_or_subquery */ - 341, /* (373) literal ::= NK_INTEGER */ - 341, /* (374) literal ::= NK_FLOAT */ - 341, /* (375) literal ::= NK_STRING */ - 341, /* (376) literal ::= NK_BOOL */ - 341, /* (377) literal ::= TIMESTAMP NK_STRING */ - 341, /* (378) literal ::= duration_literal */ - 341, /* (379) literal ::= NULL */ - 341, /* (380) literal ::= NK_QUESTION */ - 391, /* (381) duration_literal ::= NK_VARIABLE */ - 367, /* (382) signed ::= NK_INTEGER */ - 367, /* (383) signed ::= NK_PLUS NK_INTEGER */ - 367, /* (384) signed ::= NK_MINUS NK_INTEGER */ - 367, /* (385) signed ::= NK_FLOAT */ - 367, /* (386) signed ::= NK_PLUS NK_FLOAT */ - 367, /* (387) signed ::= NK_MINUS NK_FLOAT */ - 381, /* (388) signed_literal ::= signed */ - 381, /* (389) signed_literal ::= NK_STRING */ - 381, /* (390) signed_literal ::= NK_BOOL */ - 381, /* (391) signed_literal ::= TIMESTAMP NK_STRING */ - 381, /* (392) signed_literal ::= duration_literal */ - 381, /* (393) signed_literal ::= NULL */ - 381, /* (394) signed_literal ::= literal_func */ - 381, /* (395) signed_literal ::= NK_QUESTION */ - 430, /* (396) literal_list ::= signed_literal */ - 430, /* (397) literal_list ::= literal_list NK_COMMA signed_literal */ - 350, /* (398) db_name ::= NK_ID */ - 351, /* (399) table_name ::= NK_ID */ - 379, /* (400) column_name ::= NK_ID */ - 393, /* (401) function_name ::= NK_ID */ - 431, /* (402) table_alias ::= NK_ID */ - 401, /* (403) column_alias ::= NK_ID */ - 343, /* (404) user_name ::= NK_ID */ - 352, /* (405) topic_name ::= NK_ID */ - 421, /* (406) stream_name ::= NK_ID */ - 413, /* (407) cgroup_name ::= NK_ID */ - 404, /* (408) index_name ::= NK_ID */ - 432, /* (409) expr_or_subquery ::= expression */ - 427, /* (410) expression ::= literal */ - 427, /* (411) expression ::= pseudo_column */ - 427, /* (412) expression ::= column_reference */ - 427, /* (413) expression ::= function_expression */ - 427, /* (414) expression ::= case_when_expression */ - 427, /* (415) expression ::= NK_LP expression NK_RP */ - 427, /* (416) expression ::= NK_PLUS expr_or_subquery */ - 427, /* (417) expression ::= NK_MINUS expr_or_subquery */ - 427, /* (418) expression ::= expr_or_subquery NK_PLUS expr_or_subquery */ - 427, /* (419) expression ::= expr_or_subquery NK_MINUS expr_or_subquery */ - 427, /* (420) expression ::= expr_or_subquery NK_STAR expr_or_subquery */ - 427, /* (421) expression ::= expr_or_subquery NK_SLASH expr_or_subquery */ - 427, /* (422) expression ::= expr_or_subquery NK_REM expr_or_subquery */ - 427, /* (423) expression ::= column_reference NK_ARROW NK_STRING */ - 427, /* (424) expression ::= expr_or_subquery NK_BITAND expr_or_subquery */ - 427, /* (425) expression ::= expr_or_subquery NK_BITOR expr_or_subquery */ - 384, /* (426) expression_list ::= expr_or_subquery */ - 384, /* (427) expression_list ::= expression_list NK_COMMA expr_or_subquery */ - 434, /* (428) column_reference ::= column_name */ - 434, /* (429) column_reference ::= table_name NK_DOT column_name */ - 433, /* (430) pseudo_column ::= ROWTS */ - 433, /* (431) pseudo_column ::= TBNAME */ - 433, /* (432) pseudo_column ::= table_name NK_DOT TBNAME */ - 433, /* (433) pseudo_column ::= QSTART */ - 433, /* (434) pseudo_column ::= QEND */ - 433, /* (435) pseudo_column ::= QDURATION */ - 433, /* (436) pseudo_column ::= WSTART */ - 433, /* (437) pseudo_column ::= WEND */ - 433, /* (438) pseudo_column ::= WDURATION */ - 433, /* (439) pseudo_column ::= IROWTS */ - 433, /* (440) pseudo_column ::= ISFILLED */ - 433, /* (441) pseudo_column ::= QTAGS */ - 435, /* (442) function_expression ::= function_name NK_LP expression_list NK_RP */ - 435, /* (443) function_expression ::= star_func NK_LP star_func_para_list NK_RP */ - 435, /* (444) function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP */ - 435, /* (445) function_expression ::= literal_func */ - 429, /* (446) literal_func ::= noarg_func NK_LP NK_RP */ - 429, /* (447) literal_func ::= NOW */ - 439, /* (448) noarg_func ::= NOW */ - 439, /* (449) noarg_func ::= TODAY */ - 439, /* (450) noarg_func ::= TIMEZONE */ - 439, /* (451) noarg_func ::= DATABASE */ - 439, /* (452) noarg_func ::= CLIENT_VERSION */ - 439, /* (453) noarg_func ::= SERVER_VERSION */ - 439, /* (454) noarg_func ::= SERVER_STATUS */ - 439, /* (455) noarg_func ::= CURRENT_USER */ - 439, /* (456) noarg_func ::= USER */ - 437, /* (457) star_func ::= COUNT */ - 437, /* (458) star_func ::= FIRST */ - 437, /* (459) star_func ::= LAST */ - 437, /* (460) star_func ::= LAST_ROW */ - 438, /* (461) star_func_para_list ::= NK_STAR */ - 438, /* (462) star_func_para_list ::= other_para_list */ - 440, /* (463) other_para_list ::= star_func_para */ - 440, /* (464) other_para_list ::= other_para_list NK_COMMA star_func_para */ - 441, /* (465) star_func_para ::= expr_or_subquery */ - 441, /* (466) star_func_para ::= table_name NK_DOT NK_STAR */ - 436, /* (467) case_when_expression ::= CASE when_then_list case_when_else_opt END */ - 436, /* (468) case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END */ - 442, /* (469) when_then_list ::= when_then_expr */ - 442, /* (470) when_then_list ::= when_then_list when_then_expr */ - 445, /* (471) when_then_expr ::= WHEN common_expression THEN common_expression */ - 443, /* (472) case_when_else_opt ::= */ - 443, /* (473) case_when_else_opt ::= ELSE common_expression */ - 446, /* (474) predicate ::= expr_or_subquery compare_op expr_or_subquery */ - 446, /* (475) predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery */ - 446, /* (476) predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery */ - 446, /* (477) predicate ::= expr_or_subquery IS NULL */ - 446, /* (478) predicate ::= expr_or_subquery IS NOT NULL */ - 446, /* (479) predicate ::= expr_or_subquery in_op in_predicate_value */ - 447, /* (480) compare_op ::= NK_LT */ - 447, /* (481) compare_op ::= NK_GT */ - 447, /* (482) compare_op ::= NK_LE */ - 447, /* (483) compare_op ::= NK_GE */ - 447, /* (484) compare_op ::= NK_NE */ - 447, /* (485) compare_op ::= NK_EQ */ - 447, /* (486) compare_op ::= LIKE */ - 447, /* (487) compare_op ::= NOT LIKE */ - 447, /* (488) compare_op ::= MATCH */ - 447, /* (489) compare_op ::= NMATCH */ - 447, /* (490) compare_op ::= CONTAINS */ - 448, /* (491) in_op ::= IN */ - 448, /* (492) in_op ::= NOT IN */ - 449, /* (493) in_predicate_value ::= NK_LP literal_list NK_RP */ - 450, /* (494) boolean_value_expression ::= boolean_primary */ - 450, /* (495) boolean_value_expression ::= NOT boolean_primary */ - 450, /* (496) boolean_value_expression ::= boolean_value_expression OR boolean_value_expression */ - 450, /* (497) boolean_value_expression ::= boolean_value_expression AND boolean_value_expression */ - 451, /* (498) boolean_primary ::= predicate */ - 451, /* (499) boolean_primary ::= NK_LP boolean_value_expression NK_RP */ - 444, /* (500) common_expression ::= expr_or_subquery */ - 444, /* (501) common_expression ::= boolean_value_expression */ - 452, /* (502) from_clause_opt ::= */ - 452, /* (503) from_clause_opt ::= FROM table_reference_list */ - 453, /* (504) table_reference_list ::= table_reference */ - 453, /* (505) table_reference_list ::= table_reference_list NK_COMMA table_reference */ - 454, /* (506) table_reference ::= table_primary */ - 454, /* (507) table_reference ::= joined_table */ - 455, /* (508) table_primary ::= table_name alias_opt */ - 455, /* (509) table_primary ::= db_name NK_DOT table_name alias_opt */ - 455, /* (510) table_primary ::= subquery alias_opt */ - 455, /* (511) table_primary ::= parenthesized_joined_table */ - 457, /* (512) alias_opt ::= */ - 457, /* (513) alias_opt ::= table_alias */ - 457, /* (514) alias_opt ::= AS table_alias */ - 459, /* (515) parenthesized_joined_table ::= NK_LP joined_table NK_RP */ - 459, /* (516) parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP */ - 456, /* (517) joined_table ::= table_reference join_type JOIN table_reference ON search_condition */ - 460, /* (518) join_type ::= */ - 460, /* (519) join_type ::= INNER */ - 461, /* (520) query_specification ::= SELECT set_quantifier_opt select_list from_clause_opt where_clause_opt partition_by_clause_opt range_opt every_opt fill_opt twindow_clause_opt group_by_clause_opt having_clause_opt */ - 462, /* (521) set_quantifier_opt ::= */ - 462, /* (522) set_quantifier_opt ::= DISTINCT */ - 462, /* (523) set_quantifier_opt ::= ALL */ - 463, /* (524) select_list ::= select_item */ - 463, /* (525) select_list ::= select_list NK_COMMA select_item */ - 471, /* (526) select_item ::= NK_STAR */ - 471, /* (527) select_item ::= common_expression */ - 471, /* (528) select_item ::= common_expression column_alias */ - 471, /* (529) select_item ::= common_expression AS column_alias */ - 471, /* (530) select_item ::= table_name NK_DOT NK_STAR */ - 412, /* (531) where_clause_opt ::= */ - 412, /* (532) where_clause_opt ::= WHERE search_condition */ - 464, /* (533) partition_by_clause_opt ::= */ - 464, /* (534) partition_by_clause_opt ::= PARTITION BY partition_list */ - 472, /* (535) partition_list ::= partition_item */ - 472, /* (536) partition_list ::= partition_list NK_COMMA partition_item */ - 473, /* (537) partition_item ::= expr_or_subquery */ - 473, /* (538) partition_item ::= expr_or_subquery column_alias */ - 473, /* (539) partition_item ::= expr_or_subquery AS column_alias */ - 468, /* (540) twindow_clause_opt ::= */ - 468, /* (541) twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA duration_literal NK_RP */ - 468, /* (542) twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP */ - 468, /* (543) twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_RP sliding_opt fill_opt */ - 468, /* (544) twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_COMMA duration_literal NK_RP sliding_opt fill_opt */ - 468, /* (545) twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition */ - 406, /* (546) sliding_opt ::= */ - 406, /* (547) sliding_opt ::= SLIDING NK_LP duration_literal NK_RP */ - 467, /* (548) fill_opt ::= */ - 467, /* (549) fill_opt ::= FILL NK_LP fill_mode NK_RP */ - 467, /* (550) fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP */ - 467, /* (551) fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP */ - 474, /* (552) fill_mode ::= NONE */ - 474, /* (553) fill_mode ::= PREV */ - 474, /* (554) fill_mode ::= NULL */ - 474, /* (555) fill_mode ::= NULL_F */ - 474, /* (556) fill_mode ::= LINEAR */ - 474, /* (557) fill_mode ::= NEXT */ - 469, /* (558) group_by_clause_opt ::= */ - 469, /* (559) group_by_clause_opt ::= GROUP BY group_by_list */ - 475, /* (560) group_by_list ::= expr_or_subquery */ - 475, /* (561) group_by_list ::= group_by_list NK_COMMA expr_or_subquery */ - 470, /* (562) having_clause_opt ::= */ - 470, /* (563) having_clause_opt ::= HAVING search_condition */ - 465, /* (564) range_opt ::= */ - 465, /* (565) range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP */ - 465, /* (566) range_opt ::= RANGE NK_LP expr_or_subquery NK_RP */ - 466, /* (567) every_opt ::= */ - 466, /* (568) every_opt ::= EVERY NK_LP duration_literal NK_RP */ - 476, /* (569) query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt */ - 477, /* (570) query_simple ::= query_specification */ - 477, /* (571) query_simple ::= union_query_expression */ - 481, /* (572) union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery */ - 481, /* (573) union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery */ - 482, /* (574) query_simple_or_subquery ::= query_simple */ - 482, /* (575) query_simple_or_subquery ::= subquery */ - 411, /* (576) query_or_subquery ::= query_expression */ - 411, /* (577) query_or_subquery ::= subquery */ - 478, /* (578) order_by_clause_opt ::= */ - 478, /* (579) order_by_clause_opt ::= ORDER BY sort_specification_list */ - 479, /* (580) slimit_clause_opt ::= */ - 479, /* (581) slimit_clause_opt ::= SLIMIT NK_INTEGER */ - 479, /* (582) slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER */ - 479, /* (583) slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER */ - 480, /* (584) limit_clause_opt ::= */ - 480, /* (585) limit_clause_opt ::= LIMIT NK_INTEGER */ - 480, /* (586) limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER */ - 480, /* (587) limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER */ - 458, /* (588) subquery ::= NK_LP query_expression NK_RP */ - 458, /* (589) subquery ::= NK_LP subquery NK_RP */ - 353, /* (590) search_condition ::= common_expression */ - 483, /* (591) sort_specification_list ::= sort_specification */ - 483, /* (592) sort_specification_list ::= sort_specification_list NK_COMMA sort_specification */ - 484, /* (593) sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt */ - 485, /* (594) ordering_specification_opt ::= */ - 485, /* (595) ordering_specification_opt ::= ASC */ - 485, /* (596) ordering_specification_opt ::= DESC */ - 486, /* (597) null_ordering_opt ::= */ - 486, /* (598) null_ordering_opt ::= NULLS FIRST */ - 486, /* (599) null_ordering_opt ::= NULLS LAST */ + 379, /* (181) column_def ::= column_name type_name */ + 379, /* (182) column_def ::= column_name type_name COMMENT NK_STRING */ + 387, /* (183) type_name ::= BOOL */ + 387, /* (184) type_name ::= TINYINT */ + 387, /* (185) type_name ::= SMALLINT */ + 387, /* (186) type_name ::= INT */ + 387, /* (187) type_name ::= INTEGER */ + 387, /* (188) type_name ::= BIGINT */ + 387, /* (189) type_name ::= FLOAT */ + 387, /* (190) type_name ::= DOUBLE */ + 387, /* (191) type_name ::= BINARY NK_LP NK_INTEGER NK_RP */ + 387, /* (192) type_name ::= TIMESTAMP */ + 387, /* (193) type_name ::= NCHAR NK_LP NK_INTEGER NK_RP */ + 387, /* (194) type_name ::= TINYINT UNSIGNED */ + 387, /* (195) type_name ::= SMALLINT UNSIGNED */ + 387, /* (196) type_name ::= INT UNSIGNED */ + 387, /* (197) type_name ::= BIGINT UNSIGNED */ + 387, /* (198) type_name ::= JSON */ + 387, /* (199) type_name ::= VARCHAR NK_LP NK_INTEGER NK_RP */ + 387, /* (200) type_name ::= MEDIUMBLOB */ + 387, /* (201) type_name ::= BLOB */ + 387, /* (202) type_name ::= VARBINARY NK_LP NK_INTEGER NK_RP */ + 387, /* (203) type_name ::= GEOMETRY NK_LP NK_INTEGER NK_RP */ + 387, /* (204) type_name ::= DECIMAL */ + 387, /* (205) type_name ::= DECIMAL NK_LP NK_INTEGER NK_RP */ + 387, /* (206) type_name ::= DECIMAL NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP */ + 372, /* (207) tags_def_opt ::= */ + 372, /* (208) tags_def_opt ::= tags_def */ + 375, /* (209) tags_def ::= TAGS NK_LP column_def_list NK_RP */ + 373, /* (210) table_options ::= */ + 373, /* (211) table_options ::= table_options COMMENT NK_STRING */ + 373, /* (212) table_options ::= table_options MAX_DELAY duration_list */ + 373, /* (213) table_options ::= table_options WATERMARK duration_list */ + 373, /* (214) table_options ::= table_options ROLLUP NK_LP rollup_func_list NK_RP */ + 373, /* (215) table_options ::= table_options TTL NK_INTEGER */ + 373, /* (216) table_options ::= table_options SMA NK_LP col_name_list NK_RP */ + 373, /* (217) table_options ::= table_options DELETE_MARK duration_list */ + 378, /* (218) alter_table_options ::= alter_table_option */ + 378, /* (219) alter_table_options ::= alter_table_options alter_table_option */ + 390, /* (220) alter_table_option ::= COMMENT NK_STRING */ + 390, /* (221) alter_table_option ::= TTL NK_INTEGER */ + 388, /* (222) duration_list ::= duration_literal */ + 388, /* (223) duration_list ::= duration_list NK_COMMA duration_literal */ + 389, /* (224) rollup_func_list ::= rollup_func_name */ + 389, /* (225) rollup_func_list ::= rollup_func_list NK_COMMA rollup_func_name */ + 392, /* (226) rollup_func_name ::= function_name */ + 392, /* (227) rollup_func_name ::= FIRST */ + 392, /* (228) rollup_func_name ::= LAST */ + 386, /* (229) col_name_list ::= col_name */ + 386, /* (230) col_name_list ::= col_name_list NK_COMMA col_name */ + 394, /* (231) col_name ::= column_name */ + 338, /* (232) cmd ::= SHOW DNODES */ + 338, /* (233) cmd ::= SHOW USERS */ + 338, /* (234) cmd ::= SHOW USER PRIVILEGES */ + 338, /* (235) cmd ::= SHOW DATABASES */ + 338, /* (236) cmd ::= SHOW db_name_cond_opt TABLES like_pattern_opt */ + 338, /* (237) cmd ::= SHOW db_name_cond_opt STABLES like_pattern_opt */ + 338, /* (238) cmd ::= SHOW db_name_cond_opt VGROUPS */ + 338, /* (239) cmd ::= SHOW MNODES */ + 338, /* (240) cmd ::= SHOW QNODES */ + 338, /* (241) cmd ::= SHOW FUNCTIONS */ + 338, /* (242) cmd ::= SHOW INDEXES FROM table_name_cond from_db_opt */ + 338, /* (243) cmd ::= SHOW INDEXES FROM db_name NK_DOT table_name */ + 338, /* (244) cmd ::= SHOW STREAMS */ + 338, /* (245) cmd ::= SHOW ACCOUNTS */ + 338, /* (246) cmd ::= SHOW APPS */ + 338, /* (247) cmd ::= SHOW CONNECTIONS */ + 338, /* (248) cmd ::= SHOW LICENCES */ + 338, /* (249) cmd ::= SHOW GRANTS */ + 338, /* (250) cmd ::= SHOW CREATE DATABASE db_name */ + 338, /* (251) cmd ::= SHOW CREATE TABLE full_table_name */ + 338, /* (252) cmd ::= SHOW CREATE STABLE full_table_name */ + 338, /* (253) cmd ::= SHOW QUERIES */ + 338, /* (254) cmd ::= SHOW SCORES */ + 338, /* (255) cmd ::= SHOW TOPICS */ + 338, /* (256) cmd ::= SHOW VARIABLES */ + 338, /* (257) cmd ::= SHOW CLUSTER VARIABLES */ + 338, /* (258) cmd ::= SHOW LOCAL VARIABLES */ + 338, /* (259) cmd ::= SHOW DNODE NK_INTEGER VARIABLES like_pattern_opt */ + 338, /* (260) cmd ::= SHOW BNODES */ + 338, /* (261) cmd ::= SHOW SNODES */ + 338, /* (262) cmd ::= SHOW CLUSTER */ + 338, /* (263) cmd ::= SHOW TRANSACTIONS */ + 338, /* (264) cmd ::= SHOW TABLE DISTRIBUTED full_table_name */ + 338, /* (265) cmd ::= SHOW CONSUMERS */ + 338, /* (266) cmd ::= SHOW SUBSCRIPTIONS */ + 338, /* (267) cmd ::= SHOW TAGS FROM table_name_cond from_db_opt */ + 338, /* (268) cmd ::= SHOW TAGS FROM db_name NK_DOT table_name */ + 338, /* (269) cmd ::= SHOW TABLE TAGS tag_list_opt FROM table_name_cond from_db_opt */ + 338, /* (270) cmd ::= SHOW TABLE TAGS tag_list_opt FROM db_name NK_DOT table_name */ + 338, /* (271) cmd ::= SHOW VNODES ON DNODE NK_INTEGER */ + 338, /* (272) cmd ::= SHOW VNODES */ + 338, /* (273) cmd ::= SHOW db_name_cond_opt ALIVE */ + 338, /* (274) cmd ::= SHOW CLUSTER ALIVE */ + 395, /* (275) db_name_cond_opt ::= */ + 395, /* (276) db_name_cond_opt ::= db_name NK_DOT */ + 396, /* (277) like_pattern_opt ::= */ + 396, /* (278) like_pattern_opt ::= LIKE NK_STRING */ + 397, /* (279) table_name_cond ::= table_name */ + 398, /* (280) from_db_opt ::= */ + 398, /* (281) from_db_opt ::= FROM db_name */ + 399, /* (282) tag_list_opt ::= */ + 399, /* (283) tag_list_opt ::= tag_item */ + 399, /* (284) tag_list_opt ::= tag_list_opt NK_COMMA tag_item */ + 400, /* (285) tag_item ::= TBNAME */ + 400, /* (286) tag_item ::= QTAGS */ + 400, /* (287) tag_item ::= column_name */ + 400, /* (288) tag_item ::= column_name column_alias */ + 400, /* (289) tag_item ::= column_name AS column_alias */ + 338, /* (290) cmd ::= CREATE SMA INDEX not_exists_opt full_index_name ON full_table_name index_options */ + 338, /* (291) cmd ::= CREATE INDEX not_exists_opt full_index_name ON full_table_name NK_LP col_name_list NK_RP */ + 338, /* (292) cmd ::= DROP INDEX exists_opt full_index_name */ + 402, /* (293) full_index_name ::= index_name */ + 402, /* (294) full_index_name ::= db_name NK_DOT index_name */ + 403, /* (295) index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_RP sliding_opt sma_stream_opt */ + 403, /* (296) index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_COMMA duration_literal NK_RP sliding_opt sma_stream_opt */ + 405, /* (297) func_list ::= func */ + 405, /* (298) func_list ::= func_list NK_COMMA func */ + 408, /* (299) func ::= sma_func_name NK_LP expression_list NK_RP */ + 409, /* (300) sma_func_name ::= function_name */ + 409, /* (301) sma_func_name ::= COUNT */ + 409, /* (302) sma_func_name ::= FIRST */ + 409, /* (303) sma_func_name ::= LAST */ + 409, /* (304) sma_func_name ::= LAST_ROW */ + 407, /* (305) sma_stream_opt ::= */ + 407, /* (306) sma_stream_opt ::= sma_stream_opt WATERMARK duration_literal */ + 407, /* (307) sma_stream_opt ::= sma_stream_opt MAX_DELAY duration_literal */ + 407, /* (308) sma_stream_opt ::= sma_stream_opt DELETE_MARK duration_literal */ + 410, /* (309) with_meta ::= AS */ + 410, /* (310) with_meta ::= WITH META AS */ + 410, /* (311) with_meta ::= ONLY META AS */ + 338, /* (312) cmd ::= CREATE TOPIC not_exists_opt topic_name AS query_or_subquery */ + 338, /* (313) cmd ::= CREATE TOPIC not_exists_opt topic_name with_meta DATABASE db_name */ + 338, /* (314) cmd ::= CREATE TOPIC not_exists_opt topic_name with_meta STABLE full_table_name where_clause_opt */ + 338, /* (315) cmd ::= DROP TOPIC exists_opt topic_name */ + 338, /* (316) cmd ::= DROP CONSUMER GROUP exists_opt cgroup_name ON topic_name */ + 338, /* (317) cmd ::= DESC full_table_name */ + 338, /* (318) cmd ::= DESCRIBE full_table_name */ + 338, /* (319) cmd ::= RESET QUERY CACHE */ + 338, /* (320) cmd ::= EXPLAIN analyze_opt explain_options query_or_subquery */ + 338, /* (321) cmd ::= EXPLAIN analyze_opt explain_options insert_query */ + 414, /* (322) analyze_opt ::= */ + 414, /* (323) analyze_opt ::= ANALYZE */ + 415, /* (324) explain_options ::= */ + 415, /* (325) explain_options ::= explain_options VERBOSE NK_BOOL */ + 415, /* (326) explain_options ::= explain_options RATIO NK_FLOAT */ + 338, /* (327) cmd ::= CREATE or_replace_opt agg_func_opt FUNCTION not_exists_opt function_name AS NK_STRING OUTPUTTYPE type_name bufsize_opt language_opt */ + 338, /* (328) cmd ::= DROP FUNCTION exists_opt function_name */ + 418, /* (329) agg_func_opt ::= */ + 418, /* (330) agg_func_opt ::= AGGREGATE */ + 419, /* (331) bufsize_opt ::= */ + 419, /* (332) bufsize_opt ::= BUFSIZE NK_INTEGER */ + 420, /* (333) language_opt ::= */ + 420, /* (334) language_opt ::= LANGUAGE NK_STRING */ + 417, /* (335) or_replace_opt ::= */ + 417, /* (336) or_replace_opt ::= OR REPLACE */ + 338, /* (337) cmd ::= CREATE STREAM not_exists_opt stream_name stream_options INTO full_table_name col_list_opt tag_def_or_ref_opt subtable_opt AS query_or_subquery */ + 338, /* (338) cmd ::= DROP STREAM exists_opt stream_name */ + 338, /* (339) cmd ::= PAUSE STREAM exists_opt stream_name */ + 338, /* (340) cmd ::= RESUME STREAM exists_opt ignore_opt stream_name */ + 423, /* (341) col_list_opt ::= */ + 423, /* (342) col_list_opt ::= NK_LP col_name_list NK_RP */ + 424, /* (343) tag_def_or_ref_opt ::= */ + 424, /* (344) tag_def_or_ref_opt ::= tags_def */ + 424, /* (345) tag_def_or_ref_opt ::= TAGS NK_LP col_name_list NK_RP */ + 422, /* (346) stream_options ::= */ + 422, /* (347) stream_options ::= stream_options TRIGGER AT_ONCE */ + 422, /* (348) stream_options ::= stream_options TRIGGER WINDOW_CLOSE */ + 422, /* (349) stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal */ + 422, /* (350) stream_options ::= stream_options WATERMARK duration_literal */ + 422, /* (351) stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER */ + 422, /* (352) stream_options ::= stream_options FILL_HISTORY NK_INTEGER */ + 422, /* (353) stream_options ::= stream_options DELETE_MARK duration_literal */ + 422, /* (354) stream_options ::= stream_options IGNORE UPDATE NK_INTEGER */ + 425, /* (355) subtable_opt ::= */ + 425, /* (356) subtable_opt ::= SUBTABLE NK_LP expression NK_RP */ + 426, /* (357) ignore_opt ::= */ + 426, /* (358) ignore_opt ::= IGNORE UNTREATED */ + 338, /* (359) cmd ::= KILL CONNECTION NK_INTEGER */ + 338, /* (360) cmd ::= KILL QUERY NK_STRING */ + 338, /* (361) cmd ::= KILL TRANSACTION NK_INTEGER */ + 338, /* (362) cmd ::= BALANCE VGROUP */ + 338, /* (363) cmd ::= BALANCE VGROUP LEADER */ + 338, /* (364) cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER */ + 338, /* (365) cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list */ + 338, /* (366) cmd ::= SPLIT VGROUP NK_INTEGER */ + 428, /* (367) dnode_list ::= DNODE NK_INTEGER */ + 428, /* (368) dnode_list ::= dnode_list DNODE NK_INTEGER */ + 338, /* (369) cmd ::= DELETE FROM full_table_name where_clause_opt */ + 338, /* (370) cmd ::= query_or_subquery */ + 338, /* (371) cmd ::= insert_query */ + 416, /* (372) insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery */ + 416, /* (373) insert_query ::= INSERT INTO full_table_name query_or_subquery */ + 341, /* (374) literal ::= NK_INTEGER */ + 341, /* (375) literal ::= NK_FLOAT */ + 341, /* (376) literal ::= NK_STRING */ + 341, /* (377) literal ::= NK_BOOL */ + 341, /* (378) literal ::= TIMESTAMP NK_STRING */ + 341, /* (379) literal ::= duration_literal */ + 341, /* (380) literal ::= NULL */ + 341, /* (381) literal ::= NK_QUESTION */ + 391, /* (382) duration_literal ::= NK_VARIABLE */ + 367, /* (383) signed ::= NK_INTEGER */ + 367, /* (384) signed ::= NK_PLUS NK_INTEGER */ + 367, /* (385) signed ::= NK_MINUS NK_INTEGER */ + 367, /* (386) signed ::= NK_FLOAT */ + 367, /* (387) signed ::= NK_PLUS NK_FLOAT */ + 367, /* (388) signed ::= NK_MINUS NK_FLOAT */ + 381, /* (389) signed_literal ::= signed */ + 381, /* (390) signed_literal ::= NK_STRING */ + 381, /* (391) signed_literal ::= NK_BOOL */ + 381, /* (392) signed_literal ::= TIMESTAMP NK_STRING */ + 381, /* (393) signed_literal ::= duration_literal */ + 381, /* (394) signed_literal ::= NULL */ + 381, /* (395) signed_literal ::= literal_func */ + 381, /* (396) signed_literal ::= NK_QUESTION */ + 430, /* (397) literal_list ::= signed_literal */ + 430, /* (398) literal_list ::= literal_list NK_COMMA signed_literal */ + 350, /* (399) db_name ::= NK_ID */ + 351, /* (400) table_name ::= NK_ID */ + 380, /* (401) column_name ::= NK_ID */ + 393, /* (402) function_name ::= NK_ID */ + 431, /* (403) table_alias ::= NK_ID */ + 401, /* (404) column_alias ::= NK_ID */ + 343, /* (405) user_name ::= NK_ID */ + 352, /* (406) topic_name ::= NK_ID */ + 421, /* (407) stream_name ::= NK_ID */ + 413, /* (408) cgroup_name ::= NK_ID */ + 404, /* (409) index_name ::= NK_ID */ + 432, /* (410) expr_or_subquery ::= expression */ + 427, /* (411) expression ::= literal */ + 427, /* (412) expression ::= pseudo_column */ + 427, /* (413) expression ::= column_reference */ + 427, /* (414) expression ::= function_expression */ + 427, /* (415) expression ::= case_when_expression */ + 427, /* (416) expression ::= NK_LP expression NK_RP */ + 427, /* (417) expression ::= NK_PLUS expr_or_subquery */ + 427, /* (418) expression ::= NK_MINUS expr_or_subquery */ + 427, /* (419) expression ::= expr_or_subquery NK_PLUS expr_or_subquery */ + 427, /* (420) expression ::= expr_or_subquery NK_MINUS expr_or_subquery */ + 427, /* (421) expression ::= expr_or_subquery NK_STAR expr_or_subquery */ + 427, /* (422) expression ::= expr_or_subquery NK_SLASH expr_or_subquery */ + 427, /* (423) expression ::= expr_or_subquery NK_REM expr_or_subquery */ + 427, /* (424) expression ::= column_reference NK_ARROW NK_STRING */ + 427, /* (425) expression ::= expr_or_subquery NK_BITAND expr_or_subquery */ + 427, /* (426) expression ::= expr_or_subquery NK_BITOR expr_or_subquery */ + 384, /* (427) expression_list ::= expr_or_subquery */ + 384, /* (428) expression_list ::= expression_list NK_COMMA expr_or_subquery */ + 434, /* (429) column_reference ::= column_name */ + 434, /* (430) column_reference ::= table_name NK_DOT column_name */ + 433, /* (431) pseudo_column ::= ROWTS */ + 433, /* (432) pseudo_column ::= TBNAME */ + 433, /* (433) pseudo_column ::= table_name NK_DOT TBNAME */ + 433, /* (434) pseudo_column ::= QSTART */ + 433, /* (435) pseudo_column ::= QEND */ + 433, /* (436) pseudo_column ::= QDURATION */ + 433, /* (437) pseudo_column ::= WSTART */ + 433, /* (438) pseudo_column ::= WEND */ + 433, /* (439) pseudo_column ::= WDURATION */ + 433, /* (440) pseudo_column ::= IROWTS */ + 433, /* (441) pseudo_column ::= ISFILLED */ + 433, /* (442) pseudo_column ::= QTAGS */ + 435, /* (443) function_expression ::= function_name NK_LP expression_list NK_RP */ + 435, /* (444) function_expression ::= star_func NK_LP star_func_para_list NK_RP */ + 435, /* (445) function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP */ + 435, /* (446) function_expression ::= literal_func */ + 429, /* (447) literal_func ::= noarg_func NK_LP NK_RP */ + 429, /* (448) literal_func ::= NOW */ + 439, /* (449) noarg_func ::= NOW */ + 439, /* (450) noarg_func ::= TODAY */ + 439, /* (451) noarg_func ::= TIMEZONE */ + 439, /* (452) noarg_func ::= DATABASE */ + 439, /* (453) noarg_func ::= CLIENT_VERSION */ + 439, /* (454) noarg_func ::= SERVER_VERSION */ + 439, /* (455) noarg_func ::= SERVER_STATUS */ + 439, /* (456) noarg_func ::= CURRENT_USER */ + 439, /* (457) noarg_func ::= USER */ + 437, /* (458) star_func ::= COUNT */ + 437, /* (459) star_func ::= FIRST */ + 437, /* (460) star_func ::= LAST */ + 437, /* (461) star_func ::= LAST_ROW */ + 438, /* (462) star_func_para_list ::= NK_STAR */ + 438, /* (463) star_func_para_list ::= other_para_list */ + 440, /* (464) other_para_list ::= star_func_para */ + 440, /* (465) other_para_list ::= other_para_list NK_COMMA star_func_para */ + 441, /* (466) star_func_para ::= expr_or_subquery */ + 441, /* (467) star_func_para ::= table_name NK_DOT NK_STAR */ + 436, /* (468) case_when_expression ::= CASE when_then_list case_when_else_opt END */ + 436, /* (469) case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END */ + 442, /* (470) when_then_list ::= when_then_expr */ + 442, /* (471) when_then_list ::= when_then_list when_then_expr */ + 445, /* (472) when_then_expr ::= WHEN common_expression THEN common_expression */ + 443, /* (473) case_when_else_opt ::= */ + 443, /* (474) case_when_else_opt ::= ELSE common_expression */ + 446, /* (475) predicate ::= expr_or_subquery compare_op expr_or_subquery */ + 446, /* (476) predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery */ + 446, /* (477) predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery */ + 446, /* (478) predicate ::= expr_or_subquery IS NULL */ + 446, /* (479) predicate ::= expr_or_subquery IS NOT NULL */ + 446, /* (480) predicate ::= expr_or_subquery in_op in_predicate_value */ + 447, /* (481) compare_op ::= NK_LT */ + 447, /* (482) compare_op ::= NK_GT */ + 447, /* (483) compare_op ::= NK_LE */ + 447, /* (484) compare_op ::= NK_GE */ + 447, /* (485) compare_op ::= NK_NE */ + 447, /* (486) compare_op ::= NK_EQ */ + 447, /* (487) compare_op ::= LIKE */ + 447, /* (488) compare_op ::= NOT LIKE */ + 447, /* (489) compare_op ::= MATCH */ + 447, /* (490) compare_op ::= NMATCH */ + 447, /* (491) compare_op ::= CONTAINS */ + 448, /* (492) in_op ::= IN */ + 448, /* (493) in_op ::= NOT IN */ + 449, /* (494) in_predicate_value ::= NK_LP literal_list NK_RP */ + 450, /* (495) boolean_value_expression ::= boolean_primary */ + 450, /* (496) boolean_value_expression ::= NOT boolean_primary */ + 450, /* (497) boolean_value_expression ::= boolean_value_expression OR boolean_value_expression */ + 450, /* (498) boolean_value_expression ::= boolean_value_expression AND boolean_value_expression */ + 451, /* (499) boolean_primary ::= predicate */ + 451, /* (500) boolean_primary ::= NK_LP boolean_value_expression NK_RP */ + 444, /* (501) common_expression ::= expr_or_subquery */ + 444, /* (502) common_expression ::= boolean_value_expression */ + 452, /* (503) from_clause_opt ::= */ + 452, /* (504) from_clause_opt ::= FROM table_reference_list */ + 453, /* (505) table_reference_list ::= table_reference */ + 453, /* (506) table_reference_list ::= table_reference_list NK_COMMA table_reference */ + 454, /* (507) table_reference ::= table_primary */ + 454, /* (508) table_reference ::= joined_table */ + 455, /* (509) table_primary ::= table_name alias_opt */ + 455, /* (510) table_primary ::= db_name NK_DOT table_name alias_opt */ + 455, /* (511) table_primary ::= subquery alias_opt */ + 455, /* (512) table_primary ::= parenthesized_joined_table */ + 457, /* (513) alias_opt ::= */ + 457, /* (514) alias_opt ::= table_alias */ + 457, /* (515) alias_opt ::= AS table_alias */ + 459, /* (516) parenthesized_joined_table ::= NK_LP joined_table NK_RP */ + 459, /* (517) parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP */ + 456, /* (518) joined_table ::= table_reference join_type JOIN table_reference ON search_condition */ + 460, /* (519) join_type ::= */ + 460, /* (520) join_type ::= INNER */ + 461, /* (521) query_specification ::= SELECT tag_mode_opt set_quantifier_opt select_list from_clause_opt where_clause_opt partition_by_clause_opt range_opt every_opt fill_opt twindow_clause_opt group_by_clause_opt having_clause_opt */ + 462, /* (522) tag_mode_opt ::= */ + 462, /* (523) tag_mode_opt ::= TAGS */ + 463, /* (524) set_quantifier_opt ::= */ + 463, /* (525) set_quantifier_opt ::= DISTINCT */ + 463, /* (526) set_quantifier_opt ::= ALL */ + 464, /* (527) select_list ::= select_item */ + 464, /* (528) select_list ::= select_list NK_COMMA select_item */ + 472, /* (529) select_item ::= NK_STAR */ + 472, /* (530) select_item ::= common_expression */ + 472, /* (531) select_item ::= common_expression column_alias */ + 472, /* (532) select_item ::= common_expression AS column_alias */ + 472, /* (533) select_item ::= table_name NK_DOT NK_STAR */ + 412, /* (534) where_clause_opt ::= */ + 412, /* (535) where_clause_opt ::= WHERE search_condition */ + 465, /* (536) partition_by_clause_opt ::= */ + 465, /* (537) partition_by_clause_opt ::= PARTITION BY partition_list */ + 473, /* (538) partition_list ::= partition_item */ + 473, /* (539) partition_list ::= partition_list NK_COMMA partition_item */ + 474, /* (540) partition_item ::= expr_or_subquery */ + 474, /* (541) partition_item ::= expr_or_subquery column_alias */ + 474, /* (542) partition_item ::= expr_or_subquery AS column_alias */ + 469, /* (543) twindow_clause_opt ::= */ + 469, /* (544) twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA duration_literal NK_RP */ + 469, /* (545) twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP */ + 469, /* (546) twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_RP sliding_opt fill_opt */ + 469, /* (547) twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_COMMA duration_literal NK_RP sliding_opt fill_opt */ + 469, /* (548) twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition */ + 406, /* (549) sliding_opt ::= */ + 406, /* (550) sliding_opt ::= SLIDING NK_LP duration_literal NK_RP */ + 468, /* (551) fill_opt ::= */ + 468, /* (552) fill_opt ::= FILL NK_LP fill_mode NK_RP */ + 468, /* (553) fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP */ + 468, /* (554) fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP */ + 475, /* (555) fill_mode ::= NONE */ + 475, /* (556) fill_mode ::= PREV */ + 475, /* (557) fill_mode ::= NULL */ + 475, /* (558) fill_mode ::= NULL_F */ + 475, /* (559) fill_mode ::= LINEAR */ + 475, /* (560) fill_mode ::= NEXT */ + 470, /* (561) group_by_clause_opt ::= */ + 470, /* (562) group_by_clause_opt ::= GROUP BY group_by_list */ + 476, /* (563) group_by_list ::= expr_or_subquery */ + 476, /* (564) group_by_list ::= group_by_list NK_COMMA expr_or_subquery */ + 471, /* (565) having_clause_opt ::= */ + 471, /* (566) having_clause_opt ::= HAVING search_condition */ + 466, /* (567) range_opt ::= */ + 466, /* (568) range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP */ + 466, /* (569) range_opt ::= RANGE NK_LP expr_or_subquery NK_RP */ + 467, /* (570) every_opt ::= */ + 467, /* (571) every_opt ::= EVERY NK_LP duration_literal NK_RP */ + 477, /* (572) query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt */ + 478, /* (573) query_simple ::= query_specification */ + 478, /* (574) query_simple ::= union_query_expression */ + 482, /* (575) union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery */ + 482, /* (576) union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery */ + 483, /* (577) query_simple_or_subquery ::= query_simple */ + 483, /* (578) query_simple_or_subquery ::= subquery */ + 411, /* (579) query_or_subquery ::= query_expression */ + 411, /* (580) query_or_subquery ::= subquery */ + 479, /* (581) order_by_clause_opt ::= */ + 479, /* (582) order_by_clause_opt ::= ORDER BY sort_specification_list */ + 480, /* (583) slimit_clause_opt ::= */ + 480, /* (584) slimit_clause_opt ::= SLIMIT NK_INTEGER */ + 480, /* (585) slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER */ + 480, /* (586) slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER */ + 481, /* (587) limit_clause_opt ::= */ + 481, /* (588) limit_clause_opt ::= LIMIT NK_INTEGER */ + 481, /* (589) limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER */ + 481, /* (590) limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER */ + 458, /* (591) subquery ::= NK_LP query_expression NK_RP */ + 458, /* (592) subquery ::= NK_LP subquery NK_RP */ + 353, /* (593) search_condition ::= common_expression */ + 484, /* (594) sort_specification_list ::= sort_specification */ + 484, /* (595) sort_specification_list ::= sort_specification_list NK_COMMA sort_specification */ + 485, /* (596) sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt */ + 486, /* (597) ordering_specification_opt ::= */ + 486, /* (598) ordering_specification_opt ::= ASC */ + 486, /* (599) ordering_specification_opt ::= DESC */ + 487, /* (600) null_ordering_opt ::= */ + 487, /* (601) null_ordering_opt ::= NULLS FIRST */ + 487, /* (602) null_ordering_opt ::= NULLS LAST */ }; /* For rule J, yyRuleInfoNRhs[J] contains the negative of the number @@ -3957,13 +4368,13 @@ static const signed char yyRuleInfoNRhs[] = { -3, /* (157) cmd ::= ALTER TABLE alter_table_clause */ -3, /* (158) cmd ::= ALTER STABLE alter_table_clause */ -2, /* (159) alter_table_clause ::= full_table_name alter_table_options */ - -5, /* (160) alter_table_clause ::= full_table_name ADD COLUMN column_name type_name */ + -4, /* (160) alter_table_clause ::= full_table_name ADD COLUMN column_def */ -4, /* (161) alter_table_clause ::= full_table_name DROP COLUMN column_name */ - -5, /* (162) alter_table_clause ::= full_table_name MODIFY COLUMN column_name type_name */ + -4, /* (162) alter_table_clause ::= full_table_name MODIFY COLUMN column_def */ -5, /* (163) alter_table_clause ::= full_table_name RENAME COLUMN column_name column_name */ - -5, /* (164) alter_table_clause ::= full_table_name ADD TAG column_name type_name */ + -4, /* (164) alter_table_clause ::= full_table_name ADD TAG column_def */ -4, /* (165) alter_table_clause ::= full_table_name DROP TAG column_name */ - -5, /* (166) alter_table_clause ::= full_table_name MODIFY TAG column_name type_name */ + -4, /* (166) alter_table_clause ::= full_table_name MODIFY TAG column_def */ -5, /* (167) alter_table_clause ::= full_table_name RENAME TAG column_name column_name */ -6, /* (168) alter_table_clause ::= full_table_name SET TAG column_name NK_EQ signed_literal */ -1, /* (169) multi_create_clause ::= create_subtable_clause */ @@ -3979,424 +4390,427 @@ static const signed char yyRuleInfoNRhs[] = { -1, /* (179) column_def_list ::= column_def */ -3, /* (180) column_def_list ::= column_def_list NK_COMMA column_def */ -2, /* (181) column_def ::= column_name type_name */ - -1, /* (182) type_name ::= BOOL */ - -1, /* (183) type_name ::= TINYINT */ - -1, /* (184) type_name ::= SMALLINT */ - -1, /* (185) type_name ::= INT */ - -1, /* (186) type_name ::= INTEGER */ - -1, /* (187) type_name ::= BIGINT */ - -1, /* (188) type_name ::= FLOAT */ - -1, /* (189) type_name ::= DOUBLE */ - -4, /* (190) type_name ::= BINARY NK_LP NK_INTEGER NK_RP */ - -1, /* (191) type_name ::= TIMESTAMP */ - -4, /* (192) type_name ::= NCHAR NK_LP NK_INTEGER NK_RP */ - -2, /* (193) type_name ::= TINYINT UNSIGNED */ - -2, /* (194) type_name ::= SMALLINT UNSIGNED */ - -2, /* (195) type_name ::= INT UNSIGNED */ - -2, /* (196) type_name ::= BIGINT UNSIGNED */ - -1, /* (197) type_name ::= JSON */ - -4, /* (198) type_name ::= VARCHAR NK_LP NK_INTEGER NK_RP */ - -1, /* (199) type_name ::= MEDIUMBLOB */ - -1, /* (200) type_name ::= BLOB */ - -4, /* (201) type_name ::= VARBINARY NK_LP NK_INTEGER NK_RP */ - -4, /* (202) type_name ::= GEOMETRY NK_LP NK_INTEGER NK_RP */ - -1, /* (203) type_name ::= DECIMAL */ - -4, /* (204) type_name ::= DECIMAL NK_LP NK_INTEGER NK_RP */ - -6, /* (205) type_name ::= DECIMAL NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP */ - 0, /* (206) tags_def_opt ::= */ - -1, /* (207) tags_def_opt ::= tags_def */ - -4, /* (208) tags_def ::= TAGS NK_LP column_def_list NK_RP */ - 0, /* (209) table_options ::= */ - -3, /* (210) table_options ::= table_options COMMENT NK_STRING */ - -3, /* (211) table_options ::= table_options MAX_DELAY duration_list */ - -3, /* (212) table_options ::= table_options WATERMARK duration_list */ - -5, /* (213) table_options ::= table_options ROLLUP NK_LP rollup_func_list NK_RP */ - -3, /* (214) table_options ::= table_options TTL NK_INTEGER */ - -5, /* (215) table_options ::= table_options SMA NK_LP col_name_list NK_RP */ - -3, /* (216) table_options ::= table_options DELETE_MARK duration_list */ - -1, /* (217) alter_table_options ::= alter_table_option */ - -2, /* (218) alter_table_options ::= alter_table_options alter_table_option */ - -2, /* (219) alter_table_option ::= COMMENT NK_STRING */ - -2, /* (220) alter_table_option ::= TTL NK_INTEGER */ - -1, /* (221) duration_list ::= duration_literal */ - -3, /* (222) duration_list ::= duration_list NK_COMMA duration_literal */ - -1, /* (223) rollup_func_list ::= rollup_func_name */ - -3, /* (224) rollup_func_list ::= rollup_func_list NK_COMMA rollup_func_name */ - -1, /* (225) rollup_func_name ::= function_name */ - -1, /* (226) rollup_func_name ::= FIRST */ - -1, /* (227) rollup_func_name ::= LAST */ - -1, /* (228) col_name_list ::= col_name */ - -3, /* (229) col_name_list ::= col_name_list NK_COMMA col_name */ - -1, /* (230) col_name ::= column_name */ - -2, /* (231) cmd ::= SHOW DNODES */ - -2, /* (232) cmd ::= SHOW USERS */ - -3, /* (233) cmd ::= SHOW USER PRIVILEGES */ - -2, /* (234) cmd ::= SHOW DATABASES */ - -4, /* (235) cmd ::= SHOW db_name_cond_opt TABLES like_pattern_opt */ - -4, /* (236) cmd ::= SHOW db_name_cond_opt STABLES like_pattern_opt */ - -3, /* (237) cmd ::= SHOW db_name_cond_opt VGROUPS */ - -2, /* (238) cmd ::= SHOW MNODES */ - -2, /* (239) cmd ::= SHOW QNODES */ - -2, /* (240) cmd ::= SHOW FUNCTIONS */ - -5, /* (241) cmd ::= SHOW INDEXES FROM table_name_cond from_db_opt */ - -6, /* (242) cmd ::= SHOW INDEXES FROM db_name NK_DOT table_name */ - -2, /* (243) cmd ::= SHOW STREAMS */ - -2, /* (244) cmd ::= SHOW ACCOUNTS */ - -2, /* (245) cmd ::= SHOW APPS */ - -2, /* (246) cmd ::= SHOW CONNECTIONS */ - -2, /* (247) cmd ::= SHOW LICENCES */ - -2, /* (248) cmd ::= SHOW GRANTS */ - -4, /* (249) cmd ::= SHOW CREATE DATABASE db_name */ - -4, /* (250) cmd ::= SHOW CREATE TABLE full_table_name */ - -4, /* (251) cmd ::= SHOW CREATE STABLE full_table_name */ - -2, /* (252) cmd ::= SHOW QUERIES */ - -2, /* (253) cmd ::= SHOW SCORES */ - -2, /* (254) cmd ::= SHOW TOPICS */ - -2, /* (255) cmd ::= SHOW VARIABLES */ - -3, /* (256) cmd ::= SHOW CLUSTER VARIABLES */ - -3, /* (257) cmd ::= SHOW LOCAL VARIABLES */ - -5, /* (258) cmd ::= SHOW DNODE NK_INTEGER VARIABLES like_pattern_opt */ - -2, /* (259) cmd ::= SHOW BNODES */ - -2, /* (260) cmd ::= SHOW SNODES */ - -2, /* (261) cmd ::= SHOW CLUSTER */ - -2, /* (262) cmd ::= SHOW TRANSACTIONS */ - -4, /* (263) cmd ::= SHOW TABLE DISTRIBUTED full_table_name */ - -2, /* (264) cmd ::= SHOW CONSUMERS */ - -2, /* (265) cmd ::= SHOW SUBSCRIPTIONS */ - -5, /* (266) cmd ::= SHOW TAGS FROM table_name_cond from_db_opt */ - -6, /* (267) cmd ::= SHOW TAGS FROM db_name NK_DOT table_name */ - -7, /* (268) cmd ::= SHOW TABLE TAGS tag_list_opt FROM table_name_cond from_db_opt */ - -8, /* (269) cmd ::= SHOW TABLE TAGS tag_list_opt FROM db_name NK_DOT table_name */ - -3, /* (270) cmd ::= SHOW VNODES NK_INTEGER */ - -3, /* (271) cmd ::= SHOW VNODES NK_STRING */ - -3, /* (272) cmd ::= SHOW db_name_cond_opt ALIVE */ - -3, /* (273) cmd ::= SHOW CLUSTER ALIVE */ - 0, /* (274) db_name_cond_opt ::= */ - -2, /* (275) db_name_cond_opt ::= db_name NK_DOT */ - 0, /* (276) like_pattern_opt ::= */ - -2, /* (277) like_pattern_opt ::= LIKE NK_STRING */ - -1, /* (278) table_name_cond ::= table_name */ - 0, /* (279) from_db_opt ::= */ - -2, /* (280) from_db_opt ::= FROM db_name */ - 0, /* (281) tag_list_opt ::= */ - -1, /* (282) tag_list_opt ::= tag_item */ - -3, /* (283) tag_list_opt ::= tag_list_opt NK_COMMA tag_item */ - -1, /* (284) tag_item ::= TBNAME */ - -1, /* (285) tag_item ::= QTAGS */ - -1, /* (286) tag_item ::= column_name */ - -2, /* (287) tag_item ::= column_name column_alias */ - -3, /* (288) tag_item ::= column_name AS column_alias */ - -8, /* (289) cmd ::= CREATE SMA INDEX not_exists_opt full_index_name ON full_table_name index_options */ - -9, /* (290) cmd ::= CREATE INDEX not_exists_opt full_index_name ON full_table_name NK_LP col_name_list NK_RP */ - -4, /* (291) cmd ::= DROP INDEX exists_opt full_index_name */ - -1, /* (292) full_index_name ::= index_name */ - -3, /* (293) full_index_name ::= db_name NK_DOT index_name */ - -10, /* (294) index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_RP sliding_opt sma_stream_opt */ - -12, /* (295) index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_COMMA duration_literal NK_RP sliding_opt sma_stream_opt */ - -1, /* (296) func_list ::= func */ - -3, /* (297) func_list ::= func_list NK_COMMA func */ - -4, /* (298) func ::= sma_func_name NK_LP expression_list NK_RP */ - -1, /* (299) sma_func_name ::= function_name */ - -1, /* (300) sma_func_name ::= COUNT */ - -1, /* (301) sma_func_name ::= FIRST */ - -1, /* (302) sma_func_name ::= LAST */ - -1, /* (303) sma_func_name ::= LAST_ROW */ - 0, /* (304) sma_stream_opt ::= */ - -3, /* (305) sma_stream_opt ::= sma_stream_opt WATERMARK duration_literal */ - -3, /* (306) sma_stream_opt ::= sma_stream_opt MAX_DELAY duration_literal */ - -3, /* (307) sma_stream_opt ::= sma_stream_opt DELETE_MARK duration_literal */ - -1, /* (308) with_meta ::= AS */ - -3, /* (309) with_meta ::= WITH META AS */ - -3, /* (310) with_meta ::= ONLY META AS */ - -6, /* (311) cmd ::= CREATE TOPIC not_exists_opt topic_name AS query_or_subquery */ - -7, /* (312) cmd ::= CREATE TOPIC not_exists_opt topic_name with_meta DATABASE db_name */ - -8, /* (313) cmd ::= CREATE TOPIC not_exists_opt topic_name with_meta STABLE full_table_name where_clause_opt */ - -4, /* (314) cmd ::= DROP TOPIC exists_opt topic_name */ - -7, /* (315) cmd ::= DROP CONSUMER GROUP exists_opt cgroup_name ON topic_name */ - -2, /* (316) cmd ::= DESC full_table_name */ - -2, /* (317) cmd ::= DESCRIBE full_table_name */ - -3, /* (318) cmd ::= RESET QUERY CACHE */ - -4, /* (319) cmd ::= EXPLAIN analyze_opt explain_options query_or_subquery */ - -4, /* (320) cmd ::= EXPLAIN analyze_opt explain_options insert_query */ - 0, /* (321) analyze_opt ::= */ - -1, /* (322) analyze_opt ::= ANALYZE */ - 0, /* (323) explain_options ::= */ - -3, /* (324) explain_options ::= explain_options VERBOSE NK_BOOL */ - -3, /* (325) explain_options ::= explain_options RATIO NK_FLOAT */ - -12, /* (326) cmd ::= CREATE or_replace_opt agg_func_opt FUNCTION not_exists_opt function_name AS NK_STRING OUTPUTTYPE type_name bufsize_opt language_opt */ - -4, /* (327) cmd ::= DROP FUNCTION exists_opt function_name */ - 0, /* (328) agg_func_opt ::= */ - -1, /* (329) agg_func_opt ::= AGGREGATE */ - 0, /* (330) bufsize_opt ::= */ - -2, /* (331) bufsize_opt ::= BUFSIZE NK_INTEGER */ - 0, /* (332) language_opt ::= */ - -2, /* (333) language_opt ::= LANGUAGE NK_STRING */ - 0, /* (334) or_replace_opt ::= */ - -2, /* (335) or_replace_opt ::= OR REPLACE */ - -12, /* (336) cmd ::= CREATE STREAM not_exists_opt stream_name stream_options INTO full_table_name col_list_opt tag_def_or_ref_opt subtable_opt AS query_or_subquery */ - -4, /* (337) cmd ::= DROP STREAM exists_opt stream_name */ - -4, /* (338) cmd ::= PAUSE STREAM exists_opt stream_name */ - -5, /* (339) cmd ::= RESUME STREAM exists_opt ignore_opt stream_name */ - 0, /* (340) col_list_opt ::= */ - -3, /* (341) col_list_opt ::= NK_LP col_name_list NK_RP */ - 0, /* (342) tag_def_or_ref_opt ::= */ - -1, /* (343) tag_def_or_ref_opt ::= tags_def */ - -4, /* (344) tag_def_or_ref_opt ::= TAGS NK_LP col_name_list NK_RP */ - 0, /* (345) stream_options ::= */ - -3, /* (346) stream_options ::= stream_options TRIGGER AT_ONCE */ - -3, /* (347) stream_options ::= stream_options TRIGGER WINDOW_CLOSE */ - -4, /* (348) stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal */ - -3, /* (349) stream_options ::= stream_options WATERMARK duration_literal */ - -4, /* (350) stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER */ - -3, /* (351) stream_options ::= stream_options FILL_HISTORY NK_INTEGER */ - -3, /* (352) stream_options ::= stream_options DELETE_MARK duration_literal */ - -4, /* (353) stream_options ::= stream_options IGNORE UPDATE NK_INTEGER */ - 0, /* (354) subtable_opt ::= */ - -4, /* (355) subtable_opt ::= SUBTABLE NK_LP expression NK_RP */ - 0, /* (356) ignore_opt ::= */ - -2, /* (357) ignore_opt ::= IGNORE UNTREATED */ - -3, /* (358) cmd ::= KILL CONNECTION NK_INTEGER */ - -3, /* (359) cmd ::= KILL QUERY NK_STRING */ - -3, /* (360) cmd ::= KILL TRANSACTION NK_INTEGER */ - -2, /* (361) cmd ::= BALANCE VGROUP */ - -3, /* (362) cmd ::= BALANCE VGROUP LEADER */ - -4, /* (363) cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER */ - -4, /* (364) cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list */ - -3, /* (365) cmd ::= SPLIT VGROUP NK_INTEGER */ - -2, /* (366) dnode_list ::= DNODE NK_INTEGER */ - -3, /* (367) dnode_list ::= dnode_list DNODE NK_INTEGER */ - -4, /* (368) cmd ::= DELETE FROM full_table_name where_clause_opt */ - -1, /* (369) cmd ::= query_or_subquery */ - -1, /* (370) cmd ::= insert_query */ - -7, /* (371) insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery */ - -4, /* (372) insert_query ::= INSERT INTO full_table_name query_or_subquery */ - -1, /* (373) literal ::= NK_INTEGER */ - -1, /* (374) literal ::= NK_FLOAT */ - -1, /* (375) literal ::= NK_STRING */ - -1, /* (376) literal ::= NK_BOOL */ - -2, /* (377) literal ::= TIMESTAMP NK_STRING */ - -1, /* (378) literal ::= duration_literal */ - -1, /* (379) literal ::= NULL */ - -1, /* (380) literal ::= NK_QUESTION */ - -1, /* (381) duration_literal ::= NK_VARIABLE */ - -1, /* (382) signed ::= NK_INTEGER */ - -2, /* (383) signed ::= NK_PLUS NK_INTEGER */ - -2, /* (384) signed ::= NK_MINUS NK_INTEGER */ - -1, /* (385) signed ::= NK_FLOAT */ - -2, /* (386) signed ::= NK_PLUS NK_FLOAT */ - -2, /* (387) signed ::= NK_MINUS NK_FLOAT */ - -1, /* (388) signed_literal ::= signed */ - -1, /* (389) signed_literal ::= NK_STRING */ - -1, /* (390) signed_literal ::= NK_BOOL */ - -2, /* (391) signed_literal ::= TIMESTAMP NK_STRING */ - -1, /* (392) signed_literal ::= duration_literal */ - -1, /* (393) signed_literal ::= NULL */ - -1, /* (394) signed_literal ::= literal_func */ - -1, /* (395) signed_literal ::= NK_QUESTION */ - -1, /* (396) literal_list ::= signed_literal */ - -3, /* (397) literal_list ::= literal_list NK_COMMA signed_literal */ - -1, /* (398) db_name ::= NK_ID */ - -1, /* (399) table_name ::= NK_ID */ - -1, /* (400) column_name ::= NK_ID */ - -1, /* (401) function_name ::= NK_ID */ - -1, /* (402) table_alias ::= NK_ID */ - -1, /* (403) column_alias ::= NK_ID */ - -1, /* (404) user_name ::= NK_ID */ - -1, /* (405) topic_name ::= NK_ID */ - -1, /* (406) stream_name ::= NK_ID */ - -1, /* (407) cgroup_name ::= NK_ID */ - -1, /* (408) index_name ::= NK_ID */ - -1, /* (409) expr_or_subquery ::= expression */ - -1, /* (410) expression ::= literal */ - -1, /* (411) expression ::= pseudo_column */ - -1, /* (412) expression ::= column_reference */ - -1, /* (413) expression ::= function_expression */ - -1, /* (414) expression ::= case_when_expression */ - -3, /* (415) expression ::= NK_LP expression NK_RP */ - -2, /* (416) expression ::= NK_PLUS expr_or_subquery */ - -2, /* (417) expression ::= NK_MINUS expr_or_subquery */ - -3, /* (418) expression ::= expr_or_subquery NK_PLUS expr_or_subquery */ - -3, /* (419) expression ::= expr_or_subquery NK_MINUS expr_or_subquery */ - -3, /* (420) expression ::= expr_or_subquery NK_STAR expr_or_subquery */ - -3, /* (421) expression ::= expr_or_subquery NK_SLASH expr_or_subquery */ - -3, /* (422) expression ::= expr_or_subquery NK_REM expr_or_subquery */ - -3, /* (423) expression ::= column_reference NK_ARROW NK_STRING */ - -3, /* (424) expression ::= expr_or_subquery NK_BITAND expr_or_subquery */ - -3, /* (425) expression ::= expr_or_subquery NK_BITOR expr_or_subquery */ - -1, /* (426) expression_list ::= expr_or_subquery */ - -3, /* (427) expression_list ::= expression_list NK_COMMA expr_or_subquery */ - -1, /* (428) column_reference ::= column_name */ - -3, /* (429) column_reference ::= table_name NK_DOT column_name */ - -1, /* (430) pseudo_column ::= ROWTS */ - -1, /* (431) pseudo_column ::= TBNAME */ - -3, /* (432) pseudo_column ::= table_name NK_DOT TBNAME */ - -1, /* (433) pseudo_column ::= QSTART */ - -1, /* (434) pseudo_column ::= QEND */ - -1, /* (435) pseudo_column ::= QDURATION */ - -1, /* (436) pseudo_column ::= WSTART */ - -1, /* (437) pseudo_column ::= WEND */ - -1, /* (438) pseudo_column ::= WDURATION */ - -1, /* (439) pseudo_column ::= IROWTS */ - -1, /* (440) pseudo_column ::= ISFILLED */ - -1, /* (441) pseudo_column ::= QTAGS */ - -4, /* (442) function_expression ::= function_name NK_LP expression_list NK_RP */ - -4, /* (443) function_expression ::= star_func NK_LP star_func_para_list NK_RP */ - -6, /* (444) function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP */ - -1, /* (445) function_expression ::= literal_func */ - -3, /* (446) literal_func ::= noarg_func NK_LP NK_RP */ - -1, /* (447) literal_func ::= NOW */ - -1, /* (448) noarg_func ::= NOW */ - -1, /* (449) noarg_func ::= TODAY */ - -1, /* (450) noarg_func ::= TIMEZONE */ - -1, /* (451) noarg_func ::= DATABASE */ - -1, /* (452) noarg_func ::= CLIENT_VERSION */ - -1, /* (453) noarg_func ::= SERVER_VERSION */ - -1, /* (454) noarg_func ::= SERVER_STATUS */ - -1, /* (455) noarg_func ::= CURRENT_USER */ - -1, /* (456) noarg_func ::= USER */ - -1, /* (457) star_func ::= COUNT */ - -1, /* (458) star_func ::= FIRST */ - -1, /* (459) star_func ::= LAST */ - -1, /* (460) star_func ::= LAST_ROW */ - -1, /* (461) star_func_para_list ::= NK_STAR */ - -1, /* (462) star_func_para_list ::= other_para_list */ - -1, /* (463) other_para_list ::= star_func_para */ - -3, /* (464) other_para_list ::= other_para_list NK_COMMA star_func_para */ - -1, /* (465) star_func_para ::= expr_or_subquery */ - -3, /* (466) star_func_para ::= table_name NK_DOT NK_STAR */ - -4, /* (467) case_when_expression ::= CASE when_then_list case_when_else_opt END */ - -5, /* (468) case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END */ - -1, /* (469) when_then_list ::= when_then_expr */ - -2, /* (470) when_then_list ::= when_then_list when_then_expr */ - -4, /* (471) when_then_expr ::= WHEN common_expression THEN common_expression */ - 0, /* (472) case_when_else_opt ::= */ - -2, /* (473) case_when_else_opt ::= ELSE common_expression */ - -3, /* (474) predicate ::= expr_or_subquery compare_op expr_or_subquery */ - -5, /* (475) predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery */ - -6, /* (476) predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery */ - -3, /* (477) predicate ::= expr_or_subquery IS NULL */ - -4, /* (478) predicate ::= expr_or_subquery IS NOT NULL */ - -3, /* (479) predicate ::= expr_or_subquery in_op in_predicate_value */ - -1, /* (480) compare_op ::= NK_LT */ - -1, /* (481) compare_op ::= NK_GT */ - -1, /* (482) compare_op ::= NK_LE */ - -1, /* (483) compare_op ::= NK_GE */ - -1, /* (484) compare_op ::= NK_NE */ - -1, /* (485) compare_op ::= NK_EQ */ - -1, /* (486) compare_op ::= LIKE */ - -2, /* (487) compare_op ::= NOT LIKE */ - -1, /* (488) compare_op ::= MATCH */ - -1, /* (489) compare_op ::= NMATCH */ - -1, /* (490) compare_op ::= CONTAINS */ - -1, /* (491) in_op ::= IN */ - -2, /* (492) in_op ::= NOT IN */ - -3, /* (493) in_predicate_value ::= NK_LP literal_list NK_RP */ - -1, /* (494) boolean_value_expression ::= boolean_primary */ - -2, /* (495) boolean_value_expression ::= NOT boolean_primary */ - -3, /* (496) boolean_value_expression ::= boolean_value_expression OR boolean_value_expression */ - -3, /* (497) boolean_value_expression ::= boolean_value_expression AND boolean_value_expression */ - -1, /* (498) boolean_primary ::= predicate */ - -3, /* (499) boolean_primary ::= NK_LP boolean_value_expression NK_RP */ - -1, /* (500) common_expression ::= expr_or_subquery */ - -1, /* (501) common_expression ::= boolean_value_expression */ - 0, /* (502) from_clause_opt ::= */ - -2, /* (503) from_clause_opt ::= FROM table_reference_list */ - -1, /* (504) table_reference_list ::= table_reference */ - -3, /* (505) table_reference_list ::= table_reference_list NK_COMMA table_reference */ - -1, /* (506) table_reference ::= table_primary */ - -1, /* (507) table_reference ::= joined_table */ - -2, /* (508) table_primary ::= table_name alias_opt */ - -4, /* (509) table_primary ::= db_name NK_DOT table_name alias_opt */ - -2, /* (510) table_primary ::= subquery alias_opt */ - -1, /* (511) table_primary ::= parenthesized_joined_table */ - 0, /* (512) alias_opt ::= */ - -1, /* (513) alias_opt ::= table_alias */ - -2, /* (514) alias_opt ::= AS table_alias */ - -3, /* (515) parenthesized_joined_table ::= NK_LP joined_table NK_RP */ - -3, /* (516) parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP */ - -6, /* (517) joined_table ::= table_reference join_type JOIN table_reference ON search_condition */ - 0, /* (518) join_type ::= */ - -1, /* (519) join_type ::= INNER */ - -12, /* (520) query_specification ::= SELECT set_quantifier_opt select_list from_clause_opt where_clause_opt partition_by_clause_opt range_opt every_opt fill_opt twindow_clause_opt group_by_clause_opt having_clause_opt */ - 0, /* (521) set_quantifier_opt ::= */ - -1, /* (522) set_quantifier_opt ::= DISTINCT */ - -1, /* (523) set_quantifier_opt ::= ALL */ - -1, /* (524) select_list ::= select_item */ - -3, /* (525) select_list ::= select_list NK_COMMA select_item */ - -1, /* (526) select_item ::= NK_STAR */ - -1, /* (527) select_item ::= common_expression */ - -2, /* (528) select_item ::= common_expression column_alias */ - -3, /* (529) select_item ::= common_expression AS column_alias */ - -3, /* (530) select_item ::= table_name NK_DOT NK_STAR */ - 0, /* (531) where_clause_opt ::= */ - -2, /* (532) where_clause_opt ::= WHERE search_condition */ - 0, /* (533) partition_by_clause_opt ::= */ - -3, /* (534) partition_by_clause_opt ::= PARTITION BY partition_list */ - -1, /* (535) partition_list ::= partition_item */ - -3, /* (536) partition_list ::= partition_list NK_COMMA partition_item */ - -1, /* (537) partition_item ::= expr_or_subquery */ - -2, /* (538) partition_item ::= expr_or_subquery column_alias */ - -3, /* (539) partition_item ::= expr_or_subquery AS column_alias */ - 0, /* (540) twindow_clause_opt ::= */ - -6, /* (541) twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA duration_literal NK_RP */ - -4, /* (542) twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP */ - -6, /* (543) twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_RP sliding_opt fill_opt */ - -8, /* (544) twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_COMMA duration_literal NK_RP sliding_opt fill_opt */ - -7, /* (545) twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition */ - 0, /* (546) sliding_opt ::= */ - -4, /* (547) sliding_opt ::= SLIDING NK_LP duration_literal NK_RP */ - 0, /* (548) fill_opt ::= */ - -4, /* (549) fill_opt ::= FILL NK_LP fill_mode NK_RP */ - -6, /* (550) fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP */ - -6, /* (551) fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP */ - -1, /* (552) fill_mode ::= NONE */ - -1, /* (553) fill_mode ::= PREV */ - -1, /* (554) fill_mode ::= NULL */ - -1, /* (555) fill_mode ::= NULL_F */ - -1, /* (556) fill_mode ::= LINEAR */ - -1, /* (557) fill_mode ::= NEXT */ - 0, /* (558) group_by_clause_opt ::= */ - -3, /* (559) group_by_clause_opt ::= GROUP BY group_by_list */ - -1, /* (560) group_by_list ::= expr_or_subquery */ - -3, /* (561) group_by_list ::= group_by_list NK_COMMA expr_or_subquery */ - 0, /* (562) having_clause_opt ::= */ - -2, /* (563) having_clause_opt ::= HAVING search_condition */ - 0, /* (564) range_opt ::= */ - -6, /* (565) range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP */ - -4, /* (566) range_opt ::= RANGE NK_LP expr_or_subquery NK_RP */ - 0, /* (567) every_opt ::= */ - -4, /* (568) every_opt ::= EVERY NK_LP duration_literal NK_RP */ - -4, /* (569) query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt */ - -1, /* (570) query_simple ::= query_specification */ - -1, /* (571) query_simple ::= union_query_expression */ - -4, /* (572) union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery */ - -3, /* (573) union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery */ - -1, /* (574) query_simple_or_subquery ::= query_simple */ - -1, /* (575) query_simple_or_subquery ::= subquery */ - -1, /* (576) query_or_subquery ::= query_expression */ - -1, /* (577) query_or_subquery ::= subquery */ - 0, /* (578) order_by_clause_opt ::= */ - -3, /* (579) order_by_clause_opt ::= ORDER BY sort_specification_list */ - 0, /* (580) slimit_clause_opt ::= */ - -2, /* (581) slimit_clause_opt ::= SLIMIT NK_INTEGER */ - -4, /* (582) slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER */ - -4, /* (583) slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER */ - 0, /* (584) limit_clause_opt ::= */ - -2, /* (585) limit_clause_opt ::= LIMIT NK_INTEGER */ - -4, /* (586) limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER */ - -4, /* (587) limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER */ - -3, /* (588) subquery ::= NK_LP query_expression NK_RP */ - -3, /* (589) subquery ::= NK_LP subquery NK_RP */ - -1, /* (590) search_condition ::= common_expression */ - -1, /* (591) sort_specification_list ::= sort_specification */ - -3, /* (592) sort_specification_list ::= sort_specification_list NK_COMMA sort_specification */ - -3, /* (593) sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt */ - 0, /* (594) ordering_specification_opt ::= */ - -1, /* (595) ordering_specification_opt ::= ASC */ - -1, /* (596) ordering_specification_opt ::= DESC */ - 0, /* (597) null_ordering_opt ::= */ - -2, /* (598) null_ordering_opt ::= NULLS FIRST */ - -2, /* (599) null_ordering_opt ::= NULLS LAST */ + -4, /* (182) column_def ::= column_name type_name COMMENT NK_STRING */ + -1, /* (183) type_name ::= BOOL */ + -1, /* (184) type_name ::= TINYINT */ + -1, /* (185) type_name ::= SMALLINT */ + -1, /* (186) type_name ::= INT */ + -1, /* (187) type_name ::= INTEGER */ + -1, /* (188) type_name ::= BIGINT */ + -1, /* (189) type_name ::= FLOAT */ + -1, /* (190) type_name ::= DOUBLE */ + -4, /* (191) type_name ::= BINARY NK_LP NK_INTEGER NK_RP */ + -1, /* (192) type_name ::= TIMESTAMP */ + -4, /* (193) type_name ::= NCHAR NK_LP NK_INTEGER NK_RP */ + -2, /* (194) type_name ::= TINYINT UNSIGNED */ + -2, /* (195) type_name ::= SMALLINT UNSIGNED */ + -2, /* (196) type_name ::= INT UNSIGNED */ + -2, /* (197) type_name ::= BIGINT UNSIGNED */ + -1, /* (198) type_name ::= JSON */ + -4, /* (199) type_name ::= VARCHAR NK_LP NK_INTEGER NK_RP */ + -1, /* (200) type_name ::= MEDIUMBLOB */ + -1, /* (201) type_name ::= BLOB */ + -4, /* (202) type_name ::= VARBINARY NK_LP NK_INTEGER NK_RP */ + -4, /* (203) type_name ::= GEOMETRY NK_LP NK_INTEGER NK_RP */ + -1, /* (204) type_name ::= DECIMAL */ + -4, /* (205) type_name ::= DECIMAL NK_LP NK_INTEGER NK_RP */ + -6, /* (206) type_name ::= DECIMAL NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP */ + 0, /* (207) tags_def_opt ::= */ + -1, /* (208) tags_def_opt ::= tags_def */ + -4, /* (209) tags_def ::= TAGS NK_LP column_def_list NK_RP */ + 0, /* (210) table_options ::= */ + -3, /* (211) table_options ::= table_options COMMENT NK_STRING */ + -3, /* (212) table_options ::= table_options MAX_DELAY duration_list */ + -3, /* (213) table_options ::= table_options WATERMARK duration_list */ + -5, /* (214) table_options ::= table_options ROLLUP NK_LP rollup_func_list NK_RP */ + -3, /* (215) table_options ::= table_options TTL NK_INTEGER */ + -5, /* (216) table_options ::= table_options SMA NK_LP col_name_list NK_RP */ + -3, /* (217) table_options ::= table_options DELETE_MARK duration_list */ + -1, /* (218) alter_table_options ::= alter_table_option */ + -2, /* (219) alter_table_options ::= alter_table_options alter_table_option */ + -2, /* (220) alter_table_option ::= COMMENT NK_STRING */ + -2, /* (221) alter_table_option ::= TTL NK_INTEGER */ + -1, /* (222) duration_list ::= duration_literal */ + -3, /* (223) duration_list ::= duration_list NK_COMMA duration_literal */ + -1, /* (224) rollup_func_list ::= rollup_func_name */ + -3, /* (225) rollup_func_list ::= rollup_func_list NK_COMMA rollup_func_name */ + -1, /* (226) rollup_func_name ::= function_name */ + -1, /* (227) rollup_func_name ::= FIRST */ + -1, /* (228) rollup_func_name ::= LAST */ + -1, /* (229) col_name_list ::= col_name */ + -3, /* (230) col_name_list ::= col_name_list NK_COMMA col_name */ + -1, /* (231) col_name ::= column_name */ + -2, /* (232) cmd ::= SHOW DNODES */ + -2, /* (233) cmd ::= SHOW USERS */ + -3, /* (234) cmd ::= SHOW USER PRIVILEGES */ + -2, /* (235) cmd ::= SHOW DATABASES */ + -4, /* (236) cmd ::= SHOW db_name_cond_opt TABLES like_pattern_opt */ + -4, /* (237) cmd ::= SHOW db_name_cond_opt STABLES like_pattern_opt */ + -3, /* (238) cmd ::= SHOW db_name_cond_opt VGROUPS */ + -2, /* (239) cmd ::= SHOW MNODES */ + -2, /* (240) cmd ::= SHOW QNODES */ + -2, /* (241) cmd ::= SHOW FUNCTIONS */ + -5, /* (242) cmd ::= SHOW INDEXES FROM table_name_cond from_db_opt */ + -6, /* (243) cmd ::= SHOW INDEXES FROM db_name NK_DOT table_name */ + -2, /* (244) cmd ::= SHOW STREAMS */ + -2, /* (245) cmd ::= SHOW ACCOUNTS */ + -2, /* (246) cmd ::= SHOW APPS */ + -2, /* (247) cmd ::= SHOW CONNECTIONS */ + -2, /* (248) cmd ::= SHOW LICENCES */ + -2, /* (249) cmd ::= SHOW GRANTS */ + -4, /* (250) cmd ::= SHOW CREATE DATABASE db_name */ + -4, /* (251) cmd ::= SHOW CREATE TABLE full_table_name */ + -4, /* (252) cmd ::= SHOW CREATE STABLE full_table_name */ + -2, /* (253) cmd ::= SHOW QUERIES */ + -2, /* (254) cmd ::= SHOW SCORES */ + -2, /* (255) cmd ::= SHOW TOPICS */ + -2, /* (256) cmd ::= SHOW VARIABLES */ + -3, /* (257) cmd ::= SHOW CLUSTER VARIABLES */ + -3, /* (258) cmd ::= SHOW LOCAL VARIABLES */ + -5, /* (259) cmd ::= SHOW DNODE NK_INTEGER VARIABLES like_pattern_opt */ + -2, /* (260) cmd ::= SHOW BNODES */ + -2, /* (261) cmd ::= SHOW SNODES */ + -2, /* (262) cmd ::= SHOW CLUSTER */ + -2, /* (263) cmd ::= SHOW TRANSACTIONS */ + -4, /* (264) cmd ::= SHOW TABLE DISTRIBUTED full_table_name */ + -2, /* (265) cmd ::= SHOW CONSUMERS */ + -2, /* (266) cmd ::= SHOW SUBSCRIPTIONS */ + -5, /* (267) cmd ::= SHOW TAGS FROM table_name_cond from_db_opt */ + -6, /* (268) cmd ::= SHOW TAGS FROM db_name NK_DOT table_name */ + -7, /* (269) cmd ::= SHOW TABLE TAGS tag_list_opt FROM table_name_cond from_db_opt */ + -8, /* (270) cmd ::= SHOW TABLE TAGS tag_list_opt FROM db_name NK_DOT table_name */ + -5, /* (271) cmd ::= SHOW VNODES ON DNODE NK_INTEGER */ + -2, /* (272) cmd ::= SHOW VNODES */ + -3, /* (273) cmd ::= SHOW db_name_cond_opt ALIVE */ + -3, /* (274) cmd ::= SHOW CLUSTER ALIVE */ + 0, /* (275) db_name_cond_opt ::= */ + -2, /* (276) db_name_cond_opt ::= db_name NK_DOT */ + 0, /* (277) like_pattern_opt ::= */ + -2, /* (278) like_pattern_opt ::= LIKE NK_STRING */ + -1, /* (279) table_name_cond ::= table_name */ + 0, /* (280) from_db_opt ::= */ + -2, /* (281) from_db_opt ::= FROM db_name */ + 0, /* (282) tag_list_opt ::= */ + -1, /* (283) tag_list_opt ::= tag_item */ + -3, /* (284) tag_list_opt ::= tag_list_opt NK_COMMA tag_item */ + -1, /* (285) tag_item ::= TBNAME */ + -1, /* (286) tag_item ::= QTAGS */ + -1, /* (287) tag_item ::= column_name */ + -2, /* (288) tag_item ::= column_name column_alias */ + -3, /* (289) tag_item ::= column_name AS column_alias */ + -8, /* (290) cmd ::= CREATE SMA INDEX not_exists_opt full_index_name ON full_table_name index_options */ + -9, /* (291) cmd ::= CREATE INDEX not_exists_opt full_index_name ON full_table_name NK_LP col_name_list NK_RP */ + -4, /* (292) cmd ::= DROP INDEX exists_opt full_index_name */ + -1, /* (293) full_index_name ::= index_name */ + -3, /* (294) full_index_name ::= db_name NK_DOT index_name */ + -10, /* (295) index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_RP sliding_opt sma_stream_opt */ + -12, /* (296) index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_COMMA duration_literal NK_RP sliding_opt sma_stream_opt */ + -1, /* (297) func_list ::= func */ + -3, /* (298) func_list ::= func_list NK_COMMA func */ + -4, /* (299) func ::= sma_func_name NK_LP expression_list NK_RP */ + -1, /* (300) sma_func_name ::= function_name */ + -1, /* (301) sma_func_name ::= COUNT */ + -1, /* (302) sma_func_name ::= FIRST */ + -1, /* (303) sma_func_name ::= LAST */ + -1, /* (304) sma_func_name ::= LAST_ROW */ + 0, /* (305) sma_stream_opt ::= */ + -3, /* (306) sma_stream_opt ::= sma_stream_opt WATERMARK duration_literal */ + -3, /* (307) sma_stream_opt ::= sma_stream_opt MAX_DELAY duration_literal */ + -3, /* (308) sma_stream_opt ::= sma_stream_opt DELETE_MARK duration_literal */ + -1, /* (309) with_meta ::= AS */ + -3, /* (310) with_meta ::= WITH META AS */ + -3, /* (311) with_meta ::= ONLY META AS */ + -6, /* (312) cmd ::= CREATE TOPIC not_exists_opt topic_name AS query_or_subquery */ + -7, /* (313) cmd ::= CREATE TOPIC not_exists_opt topic_name with_meta DATABASE db_name */ + -8, /* (314) cmd ::= CREATE TOPIC not_exists_opt topic_name with_meta STABLE full_table_name where_clause_opt */ + -4, /* (315) cmd ::= DROP TOPIC exists_opt topic_name */ + -7, /* (316) cmd ::= DROP CONSUMER GROUP exists_opt cgroup_name ON topic_name */ + -2, /* (317) cmd ::= DESC full_table_name */ + -2, /* (318) cmd ::= DESCRIBE full_table_name */ + -3, /* (319) cmd ::= RESET QUERY CACHE */ + -4, /* (320) cmd ::= EXPLAIN analyze_opt explain_options query_or_subquery */ + -4, /* (321) cmd ::= EXPLAIN analyze_opt explain_options insert_query */ + 0, /* (322) analyze_opt ::= */ + -1, /* (323) analyze_opt ::= ANALYZE */ + 0, /* (324) explain_options ::= */ + -3, /* (325) explain_options ::= explain_options VERBOSE NK_BOOL */ + -3, /* (326) explain_options ::= explain_options RATIO NK_FLOAT */ + -12, /* (327) cmd ::= CREATE or_replace_opt agg_func_opt FUNCTION not_exists_opt function_name AS NK_STRING OUTPUTTYPE type_name bufsize_opt language_opt */ + -4, /* (328) cmd ::= DROP FUNCTION exists_opt function_name */ + 0, /* (329) agg_func_opt ::= */ + -1, /* (330) agg_func_opt ::= AGGREGATE */ + 0, /* (331) bufsize_opt ::= */ + -2, /* (332) bufsize_opt ::= BUFSIZE NK_INTEGER */ + 0, /* (333) language_opt ::= */ + -2, /* (334) language_opt ::= LANGUAGE NK_STRING */ + 0, /* (335) or_replace_opt ::= */ + -2, /* (336) or_replace_opt ::= OR REPLACE */ + -12, /* (337) cmd ::= CREATE STREAM not_exists_opt stream_name stream_options INTO full_table_name col_list_opt tag_def_or_ref_opt subtable_opt AS query_or_subquery */ + -4, /* (338) cmd ::= DROP STREAM exists_opt stream_name */ + -4, /* (339) cmd ::= PAUSE STREAM exists_opt stream_name */ + -5, /* (340) cmd ::= RESUME STREAM exists_opt ignore_opt stream_name */ + 0, /* (341) col_list_opt ::= */ + -3, /* (342) col_list_opt ::= NK_LP col_name_list NK_RP */ + 0, /* (343) tag_def_or_ref_opt ::= */ + -1, /* (344) tag_def_or_ref_opt ::= tags_def */ + -4, /* (345) tag_def_or_ref_opt ::= TAGS NK_LP col_name_list NK_RP */ + 0, /* (346) stream_options ::= */ + -3, /* (347) stream_options ::= stream_options TRIGGER AT_ONCE */ + -3, /* (348) stream_options ::= stream_options TRIGGER WINDOW_CLOSE */ + -4, /* (349) stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal */ + -3, /* (350) stream_options ::= stream_options WATERMARK duration_literal */ + -4, /* (351) stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER */ + -3, /* (352) stream_options ::= stream_options FILL_HISTORY NK_INTEGER */ + -3, /* (353) stream_options ::= stream_options DELETE_MARK duration_literal */ + -4, /* (354) stream_options ::= stream_options IGNORE UPDATE NK_INTEGER */ + 0, /* (355) subtable_opt ::= */ + -4, /* (356) subtable_opt ::= SUBTABLE NK_LP expression NK_RP */ + 0, /* (357) ignore_opt ::= */ + -2, /* (358) ignore_opt ::= IGNORE UNTREATED */ + -3, /* (359) cmd ::= KILL CONNECTION NK_INTEGER */ + -3, /* (360) cmd ::= KILL QUERY NK_STRING */ + -3, /* (361) cmd ::= KILL TRANSACTION NK_INTEGER */ + -2, /* (362) cmd ::= BALANCE VGROUP */ + -3, /* (363) cmd ::= BALANCE VGROUP LEADER */ + -4, /* (364) cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER */ + -4, /* (365) cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list */ + -3, /* (366) cmd ::= SPLIT VGROUP NK_INTEGER */ + -2, /* (367) dnode_list ::= DNODE NK_INTEGER */ + -3, /* (368) dnode_list ::= dnode_list DNODE NK_INTEGER */ + -4, /* (369) cmd ::= DELETE FROM full_table_name where_clause_opt */ + -1, /* (370) cmd ::= query_or_subquery */ + -1, /* (371) cmd ::= insert_query */ + -7, /* (372) insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery */ + -4, /* (373) insert_query ::= INSERT INTO full_table_name query_or_subquery */ + -1, /* (374) literal ::= NK_INTEGER */ + -1, /* (375) literal ::= NK_FLOAT */ + -1, /* (376) literal ::= NK_STRING */ + -1, /* (377) literal ::= NK_BOOL */ + -2, /* (378) literal ::= TIMESTAMP NK_STRING */ + -1, /* (379) literal ::= duration_literal */ + -1, /* (380) literal ::= NULL */ + -1, /* (381) literal ::= NK_QUESTION */ + -1, /* (382) duration_literal ::= NK_VARIABLE */ + -1, /* (383) signed ::= NK_INTEGER */ + -2, /* (384) signed ::= NK_PLUS NK_INTEGER */ + -2, /* (385) signed ::= NK_MINUS NK_INTEGER */ + -1, /* (386) signed ::= NK_FLOAT */ + -2, /* (387) signed ::= NK_PLUS NK_FLOAT */ + -2, /* (388) signed ::= NK_MINUS NK_FLOAT */ + -1, /* (389) signed_literal ::= signed */ + -1, /* (390) signed_literal ::= NK_STRING */ + -1, /* (391) signed_literal ::= NK_BOOL */ + -2, /* (392) signed_literal ::= TIMESTAMP NK_STRING */ + -1, /* (393) signed_literal ::= duration_literal */ + -1, /* (394) signed_literal ::= NULL */ + -1, /* (395) signed_literal ::= literal_func */ + -1, /* (396) signed_literal ::= NK_QUESTION */ + -1, /* (397) literal_list ::= signed_literal */ + -3, /* (398) literal_list ::= literal_list NK_COMMA signed_literal */ + -1, /* (399) db_name ::= NK_ID */ + -1, /* (400) table_name ::= NK_ID */ + -1, /* (401) column_name ::= NK_ID */ + -1, /* (402) function_name ::= NK_ID */ + -1, /* (403) table_alias ::= NK_ID */ + -1, /* (404) column_alias ::= NK_ID */ + -1, /* (405) user_name ::= NK_ID */ + -1, /* (406) topic_name ::= NK_ID */ + -1, /* (407) stream_name ::= NK_ID */ + -1, /* (408) cgroup_name ::= NK_ID */ + -1, /* (409) index_name ::= NK_ID */ + -1, /* (410) expr_or_subquery ::= expression */ + -1, /* (411) expression ::= literal */ + -1, /* (412) expression ::= pseudo_column */ + -1, /* (413) expression ::= column_reference */ + -1, /* (414) expression ::= function_expression */ + -1, /* (415) expression ::= case_when_expression */ + -3, /* (416) expression ::= NK_LP expression NK_RP */ + -2, /* (417) expression ::= NK_PLUS expr_or_subquery */ + -2, /* (418) expression ::= NK_MINUS expr_or_subquery */ + -3, /* (419) expression ::= expr_or_subquery NK_PLUS expr_or_subquery */ + -3, /* (420) expression ::= expr_or_subquery NK_MINUS expr_or_subquery */ + -3, /* (421) expression ::= expr_or_subquery NK_STAR expr_or_subquery */ + -3, /* (422) expression ::= expr_or_subquery NK_SLASH expr_or_subquery */ + -3, /* (423) expression ::= expr_or_subquery NK_REM expr_or_subquery */ + -3, /* (424) expression ::= column_reference NK_ARROW NK_STRING */ + -3, /* (425) expression ::= expr_or_subquery NK_BITAND expr_or_subquery */ + -3, /* (426) expression ::= expr_or_subquery NK_BITOR expr_or_subquery */ + -1, /* (427) expression_list ::= expr_or_subquery */ + -3, /* (428) expression_list ::= expression_list NK_COMMA expr_or_subquery */ + -1, /* (429) column_reference ::= column_name */ + -3, /* (430) column_reference ::= table_name NK_DOT column_name */ + -1, /* (431) pseudo_column ::= ROWTS */ + -1, /* (432) pseudo_column ::= TBNAME */ + -3, /* (433) pseudo_column ::= table_name NK_DOT TBNAME */ + -1, /* (434) pseudo_column ::= QSTART */ + -1, /* (435) pseudo_column ::= QEND */ + -1, /* (436) pseudo_column ::= QDURATION */ + -1, /* (437) pseudo_column ::= WSTART */ + -1, /* (438) pseudo_column ::= WEND */ + -1, /* (439) pseudo_column ::= WDURATION */ + -1, /* (440) pseudo_column ::= IROWTS */ + -1, /* (441) pseudo_column ::= ISFILLED */ + -1, /* (442) pseudo_column ::= QTAGS */ + -4, /* (443) function_expression ::= function_name NK_LP expression_list NK_RP */ + -4, /* (444) function_expression ::= star_func NK_LP star_func_para_list NK_RP */ + -6, /* (445) function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP */ + -1, /* (446) function_expression ::= literal_func */ + -3, /* (447) literal_func ::= noarg_func NK_LP NK_RP */ + -1, /* (448) literal_func ::= NOW */ + -1, /* (449) noarg_func ::= NOW */ + -1, /* (450) noarg_func ::= TODAY */ + -1, /* (451) noarg_func ::= TIMEZONE */ + -1, /* (452) noarg_func ::= DATABASE */ + -1, /* (453) noarg_func ::= CLIENT_VERSION */ + -1, /* (454) noarg_func ::= SERVER_VERSION */ + -1, /* (455) noarg_func ::= SERVER_STATUS */ + -1, /* (456) noarg_func ::= CURRENT_USER */ + -1, /* (457) noarg_func ::= USER */ + -1, /* (458) star_func ::= COUNT */ + -1, /* (459) star_func ::= FIRST */ + -1, /* (460) star_func ::= LAST */ + -1, /* (461) star_func ::= LAST_ROW */ + -1, /* (462) star_func_para_list ::= NK_STAR */ + -1, /* (463) star_func_para_list ::= other_para_list */ + -1, /* (464) other_para_list ::= star_func_para */ + -3, /* (465) other_para_list ::= other_para_list NK_COMMA star_func_para */ + -1, /* (466) star_func_para ::= expr_or_subquery */ + -3, /* (467) star_func_para ::= table_name NK_DOT NK_STAR */ + -4, /* (468) case_when_expression ::= CASE when_then_list case_when_else_opt END */ + -5, /* (469) case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END */ + -1, /* (470) when_then_list ::= when_then_expr */ + -2, /* (471) when_then_list ::= when_then_list when_then_expr */ + -4, /* (472) when_then_expr ::= WHEN common_expression THEN common_expression */ + 0, /* (473) case_when_else_opt ::= */ + -2, /* (474) case_when_else_opt ::= ELSE common_expression */ + -3, /* (475) predicate ::= expr_or_subquery compare_op expr_or_subquery */ + -5, /* (476) predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery */ + -6, /* (477) predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery */ + -3, /* (478) predicate ::= expr_or_subquery IS NULL */ + -4, /* (479) predicate ::= expr_or_subquery IS NOT NULL */ + -3, /* (480) predicate ::= expr_or_subquery in_op in_predicate_value */ + -1, /* (481) compare_op ::= NK_LT */ + -1, /* (482) compare_op ::= NK_GT */ + -1, /* (483) compare_op ::= NK_LE */ + -1, /* (484) compare_op ::= NK_GE */ + -1, /* (485) compare_op ::= NK_NE */ + -1, /* (486) compare_op ::= NK_EQ */ + -1, /* (487) compare_op ::= LIKE */ + -2, /* (488) compare_op ::= NOT LIKE */ + -1, /* (489) compare_op ::= MATCH */ + -1, /* (490) compare_op ::= NMATCH */ + -1, /* (491) compare_op ::= CONTAINS */ + -1, /* (492) in_op ::= IN */ + -2, /* (493) in_op ::= NOT IN */ + -3, /* (494) in_predicate_value ::= NK_LP literal_list NK_RP */ + -1, /* (495) boolean_value_expression ::= boolean_primary */ + -2, /* (496) boolean_value_expression ::= NOT boolean_primary */ + -3, /* (497) boolean_value_expression ::= boolean_value_expression OR boolean_value_expression */ + -3, /* (498) boolean_value_expression ::= boolean_value_expression AND boolean_value_expression */ + -1, /* (499) boolean_primary ::= predicate */ + -3, /* (500) boolean_primary ::= NK_LP boolean_value_expression NK_RP */ + -1, /* (501) common_expression ::= expr_or_subquery */ + -1, /* (502) common_expression ::= boolean_value_expression */ + 0, /* (503) from_clause_opt ::= */ + -2, /* (504) from_clause_opt ::= FROM table_reference_list */ + -1, /* (505) table_reference_list ::= table_reference */ + -3, /* (506) table_reference_list ::= table_reference_list NK_COMMA table_reference */ + -1, /* (507) table_reference ::= table_primary */ + -1, /* (508) table_reference ::= joined_table */ + -2, /* (509) table_primary ::= table_name alias_opt */ + -4, /* (510) table_primary ::= db_name NK_DOT table_name alias_opt */ + -2, /* (511) table_primary ::= subquery alias_opt */ + -1, /* (512) table_primary ::= parenthesized_joined_table */ + 0, /* (513) alias_opt ::= */ + -1, /* (514) alias_opt ::= table_alias */ + -2, /* (515) alias_opt ::= AS table_alias */ + -3, /* (516) parenthesized_joined_table ::= NK_LP joined_table NK_RP */ + -3, /* (517) parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP */ + -6, /* (518) joined_table ::= table_reference join_type JOIN table_reference ON search_condition */ + 0, /* (519) join_type ::= */ + -1, /* (520) join_type ::= INNER */ + -13, /* (521) query_specification ::= SELECT tag_mode_opt set_quantifier_opt select_list from_clause_opt where_clause_opt partition_by_clause_opt range_opt every_opt fill_opt twindow_clause_opt group_by_clause_opt having_clause_opt */ + 0, /* (522) tag_mode_opt ::= */ + -1, /* (523) tag_mode_opt ::= TAGS */ + 0, /* (524) set_quantifier_opt ::= */ + -1, /* (525) set_quantifier_opt ::= DISTINCT */ + -1, /* (526) set_quantifier_opt ::= ALL */ + -1, /* (527) select_list ::= select_item */ + -3, /* (528) select_list ::= select_list NK_COMMA select_item */ + -1, /* (529) select_item ::= NK_STAR */ + -1, /* (530) select_item ::= common_expression */ + -2, /* (531) select_item ::= common_expression column_alias */ + -3, /* (532) select_item ::= common_expression AS column_alias */ + -3, /* (533) select_item ::= table_name NK_DOT NK_STAR */ + 0, /* (534) where_clause_opt ::= */ + -2, /* (535) where_clause_opt ::= WHERE search_condition */ + 0, /* (536) partition_by_clause_opt ::= */ + -3, /* (537) partition_by_clause_opt ::= PARTITION BY partition_list */ + -1, /* (538) partition_list ::= partition_item */ + -3, /* (539) partition_list ::= partition_list NK_COMMA partition_item */ + -1, /* (540) partition_item ::= expr_or_subquery */ + -2, /* (541) partition_item ::= expr_or_subquery column_alias */ + -3, /* (542) partition_item ::= expr_or_subquery AS column_alias */ + 0, /* (543) twindow_clause_opt ::= */ + -6, /* (544) twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA duration_literal NK_RP */ + -4, /* (545) twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP */ + -6, /* (546) twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_RP sliding_opt fill_opt */ + -8, /* (547) twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_COMMA duration_literal NK_RP sliding_opt fill_opt */ + -7, /* (548) twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition */ + 0, /* (549) sliding_opt ::= */ + -4, /* (550) sliding_opt ::= SLIDING NK_LP duration_literal NK_RP */ + 0, /* (551) fill_opt ::= */ + -4, /* (552) fill_opt ::= FILL NK_LP fill_mode NK_RP */ + -6, /* (553) fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP */ + -6, /* (554) fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP */ + -1, /* (555) fill_mode ::= NONE */ + -1, /* (556) fill_mode ::= PREV */ + -1, /* (557) fill_mode ::= NULL */ + -1, /* (558) fill_mode ::= NULL_F */ + -1, /* (559) fill_mode ::= LINEAR */ + -1, /* (560) fill_mode ::= NEXT */ + 0, /* (561) group_by_clause_opt ::= */ + -3, /* (562) group_by_clause_opt ::= GROUP BY group_by_list */ + -1, /* (563) group_by_list ::= expr_or_subquery */ + -3, /* (564) group_by_list ::= group_by_list NK_COMMA expr_or_subquery */ + 0, /* (565) having_clause_opt ::= */ + -2, /* (566) having_clause_opt ::= HAVING search_condition */ + 0, /* (567) range_opt ::= */ + -6, /* (568) range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP */ + -4, /* (569) range_opt ::= RANGE NK_LP expr_or_subquery NK_RP */ + 0, /* (570) every_opt ::= */ + -4, /* (571) every_opt ::= EVERY NK_LP duration_literal NK_RP */ + -4, /* (572) query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt */ + -1, /* (573) query_simple ::= query_specification */ + -1, /* (574) query_simple ::= union_query_expression */ + -4, /* (575) union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery */ + -3, /* (576) union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery */ + -1, /* (577) query_simple_or_subquery ::= query_simple */ + -1, /* (578) query_simple_or_subquery ::= subquery */ + -1, /* (579) query_or_subquery ::= query_expression */ + -1, /* (580) query_or_subquery ::= subquery */ + 0, /* (581) order_by_clause_opt ::= */ + -3, /* (582) order_by_clause_opt ::= ORDER BY sort_specification_list */ + 0, /* (583) slimit_clause_opt ::= */ + -2, /* (584) slimit_clause_opt ::= SLIMIT NK_INTEGER */ + -4, /* (585) slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER */ + -4, /* (586) slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER */ + 0, /* (587) limit_clause_opt ::= */ + -2, /* (588) limit_clause_opt ::= LIMIT NK_INTEGER */ + -4, /* (589) limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER */ + -4, /* (590) limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER */ + -3, /* (591) subquery ::= NK_LP query_expression NK_RP */ + -3, /* (592) subquery ::= NK_LP subquery NK_RP */ + -1, /* (593) search_condition ::= common_expression */ + -1, /* (594) sort_specification_list ::= sort_specification */ + -3, /* (595) sort_specification_list ::= sort_specification_list NK_COMMA sort_specification */ + -3, /* (596) sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt */ + 0, /* (597) ordering_specification_opt ::= */ + -1, /* (598) ordering_specification_opt ::= ASC */ + -1, /* (599) ordering_specification_opt ::= DESC */ + 0, /* (600) null_ordering_opt ::= */ + -2, /* (601) null_ordering_opt ::= NULLS FIRST */ + -2, /* (602) null_ordering_opt ::= NULLS LAST */ }; static void yy_accept(yyParser*); /* Forward Declaration */ @@ -4426,54 +4840,6 @@ static YYACTIONTYPE yy_reduce( (void)yyLookahead; (void)yyLookaheadToken; yymsp = yypParser->yytos; -#ifndef NDEBUG - if( yyTraceFILE && yyruleno<(int)(sizeof(yyRuleName)/sizeof(yyRuleName[0])) ){ - yysize = yyRuleInfoNRhs[yyruleno]; - if( yysize ){ - fprintf(yyTraceFILE, "%sReduce %d [%s]%s, pop back to state %d.\n", - yyTracePrompt, - yyruleno, yyRuleName[yyruleno], - yyrulenoyytos - yypParser->yystack)>yypParser->yyhwm ){ - yypParser->yyhwm++; - assert( yypParser->yyhwm == (int)(yypParser->yytos - yypParser->yystack)); - } -#endif -#if YYSTACKDEPTH>0 - if( yypParser->yytos>=yypParser->yystackEnd ){ - yyStackOverflow(yypParser); - /* The call to yyStackOverflow() above pops the stack until it is - ** empty, causing the main parser loop to exit. So the return value - ** is never used and does not matter. */ - return 0; - } -#else - if( yypParser->yytos>=&yypParser->yystack[yypParser->yystksz-1] ){ - if( yyGrowStack(yypParser) ){ - yyStackOverflow(yypParser); - /* The call to yyStackOverflow() above pops the stack until it is - ** empty, causing the main parser loop to exit. So the return value - ** is never used and does not matter. */ - return 0; - } - yymsp = yypParser->yytos; - } -#endif - } switch( yyruleno ){ /* Beginning here are the reduction cases. A typical example @@ -4487,15 +4853,21 @@ static YYACTIONTYPE yy_reduce( /********** Begin reduce actions **********************************************/ YYMINORTYPE yylhsminor; case 0: /* cmd ::= CREATE ACCOUNT NK_ID PASS NK_STRING account_options */ +#line 50 "sql.y" { pCxt->errCode = generateSyntaxErrMsg(&pCxt->msgBuf, TSDB_CODE_PAR_EXPRIE_STATEMENT); } +#line 4858 "sql.c" yy_destructor(yypParser,339,&yymsp[0].minor); break; case 1: /* cmd ::= ALTER ACCOUNT NK_ID alter_account_options */ +#line 51 "sql.y" { pCxt->errCode = generateSyntaxErrMsg(&pCxt->msgBuf, TSDB_CODE_PAR_EXPRIE_STATEMENT); } +#line 4864 "sql.c" yy_destructor(yypParser,340,&yymsp[0].minor); break; case 2: /* account_options ::= */ +#line 55 "sql.y" { } +#line 4870 "sql.c" break; case 3: /* account_options ::= account_options PPS literal */ case 4: /* account_options ::= account_options TSERIES literal */ yytestcase(yyruleno==4); @@ -4507,18 +4879,24 @@ static YYACTIONTYPE yy_reduce( case 10: /* account_options ::= account_options CONNS literal */ yytestcase(yyruleno==10); case 11: /* account_options ::= account_options STATE literal */ yytestcase(yyruleno==11); { yy_destructor(yypParser,339,&yymsp[-2].minor); +#line 56 "sql.y" { } +#line 4884 "sql.c" yy_destructor(yypParser,341,&yymsp[0].minor); } break; case 12: /* alter_account_options ::= alter_account_option */ { yy_destructor(yypParser,342,&yymsp[0].minor); +#line 68 "sql.y" { } +#line 4892 "sql.c" } break; case 13: /* alter_account_options ::= alter_account_options alter_account_option */ { yy_destructor(yypParser,340,&yymsp[-1].minor); +#line 69 "sql.y" { } +#line 4899 "sql.c" yy_destructor(yypParser,342,&yymsp[0].minor); } break; @@ -4532,1646 +4910,2455 @@ static YYACTIONTYPE yy_reduce( case 21: /* alter_account_option ::= USERS literal */ yytestcase(yyruleno==21); case 22: /* alter_account_option ::= CONNS literal */ yytestcase(yyruleno==22); case 23: /* alter_account_option ::= STATE literal */ yytestcase(yyruleno==23); +#line 73 "sql.y" { } +#line 4915 "sql.c" yy_destructor(yypParser,341,&yymsp[0].minor); break; case 24: /* cmd ::= CREATE USER user_name PASS NK_STRING sysinfo_opt */ -{ pCxt->pRootNode = createCreateUserStmt(pCxt, &yymsp[-3].minor.yy371, &yymsp[-1].minor.yy0, yymsp[0].minor.yy475); } +#line 85 "sql.y" +{ pCxt->pRootNode = createCreateUserStmt(pCxt, &yymsp[-3].minor.yy129, &yymsp[-1].minor.yy0, yymsp[0].minor.yy215); } +#line 4921 "sql.c" break; case 25: /* cmd ::= ALTER USER user_name PASS NK_STRING */ -{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy371, TSDB_ALTER_USER_PASSWD, &yymsp[0].minor.yy0); } +#line 86 "sql.y" +{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy129, TSDB_ALTER_USER_PASSWD, &yymsp[0].minor.yy0); } +#line 4926 "sql.c" break; case 26: /* cmd ::= ALTER USER user_name ENABLE NK_INTEGER */ -{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy371, TSDB_ALTER_USER_ENABLE, &yymsp[0].minor.yy0); } +#line 87 "sql.y" +{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy129, TSDB_ALTER_USER_ENABLE, &yymsp[0].minor.yy0); } +#line 4931 "sql.c" break; case 27: /* cmd ::= ALTER USER user_name SYSINFO NK_INTEGER */ -{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy371, TSDB_ALTER_USER_SYSINFO, &yymsp[0].minor.yy0); } +#line 88 "sql.y" +{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy129, TSDB_ALTER_USER_SYSINFO, &yymsp[0].minor.yy0); } +#line 4936 "sql.c" break; case 28: /* cmd ::= DROP USER user_name */ -{ pCxt->pRootNode = createDropUserStmt(pCxt, &yymsp[0].minor.yy371); } +#line 89 "sql.y" +{ pCxt->pRootNode = createDropUserStmt(pCxt, &yymsp[0].minor.yy129); } +#line 4941 "sql.c" break; case 29: /* sysinfo_opt ::= */ -{ yymsp[1].minor.yy475 = 1; } +#line 93 "sql.y" +{ yymsp[1].minor.yy215 = 1; } +#line 4946 "sql.c" break; case 30: /* sysinfo_opt ::= SYSINFO NK_INTEGER */ -{ yymsp[-1].minor.yy475 = taosStr2Int8(yymsp[0].minor.yy0.z, NULL, 10); } +#line 94 "sql.y" +{ yymsp[-1].minor.yy215 = taosStr2Int8(yymsp[0].minor.yy0.z, NULL, 10); } +#line 4951 "sql.c" break; case 31: /* cmd ::= GRANT privileges ON priv_level with_opt TO user_name */ -{ pCxt->pRootNode = createGrantStmt(pCxt, yymsp[-5].minor.yy729, &yymsp[-3].minor.yy347, &yymsp[0].minor.yy371, yymsp[-2].minor.yy452); } +#line 97 "sql.y" +{ pCxt->pRootNode = createGrantStmt(pCxt, yymsp[-5].minor.yy717, &yymsp[-3].minor.yy505, &yymsp[0].minor.yy129, yymsp[-2].minor.yy840); } +#line 4956 "sql.c" break; case 32: /* cmd ::= REVOKE privileges ON priv_level with_opt FROM user_name */ -{ pCxt->pRootNode = createRevokeStmt(pCxt, yymsp[-5].minor.yy729, &yymsp[-3].minor.yy347, &yymsp[0].minor.yy371, yymsp[-2].minor.yy452); } +#line 98 "sql.y" +{ pCxt->pRootNode = createRevokeStmt(pCxt, yymsp[-5].minor.yy717, &yymsp[-3].minor.yy505, &yymsp[0].minor.yy129, yymsp[-2].minor.yy840); } +#line 4961 "sql.c" break; case 33: /* privileges ::= ALL */ -{ yymsp[0].minor.yy729 = PRIVILEGE_TYPE_ALL; } +#line 102 "sql.y" +{ yymsp[0].minor.yy717 = PRIVILEGE_TYPE_ALL; } +#line 4966 "sql.c" break; case 34: /* privileges ::= priv_type_list */ case 36: /* priv_type_list ::= priv_type */ yytestcase(yyruleno==36); -{ yylhsminor.yy729 = yymsp[0].minor.yy729; } - yymsp[0].minor.yy729 = yylhsminor.yy729; +#line 103 "sql.y" +{ yylhsminor.yy717 = yymsp[0].minor.yy717; } +#line 4972 "sql.c" + yymsp[0].minor.yy717 = yylhsminor.yy717; break; case 35: /* privileges ::= SUBSCRIBE */ -{ yymsp[0].minor.yy729 = PRIVILEGE_TYPE_SUBSCRIBE; } +#line 104 "sql.y" +{ yymsp[0].minor.yy717 = PRIVILEGE_TYPE_SUBSCRIBE; } +#line 4978 "sql.c" break; case 37: /* priv_type_list ::= priv_type_list NK_COMMA priv_type */ -{ yylhsminor.yy729 = yymsp[-2].minor.yy729 | yymsp[0].minor.yy729; } - yymsp[-2].minor.yy729 = yylhsminor.yy729; +#line 109 "sql.y" +{ yylhsminor.yy717 = yymsp[-2].minor.yy717 | yymsp[0].minor.yy717; } +#line 4983 "sql.c" + yymsp[-2].minor.yy717 = yylhsminor.yy717; break; case 38: /* priv_type ::= READ */ -{ yymsp[0].minor.yy729 = PRIVILEGE_TYPE_READ; } +#line 113 "sql.y" +{ yymsp[0].minor.yy717 = PRIVILEGE_TYPE_READ; } +#line 4989 "sql.c" break; case 39: /* priv_type ::= WRITE */ -{ yymsp[0].minor.yy729 = PRIVILEGE_TYPE_WRITE; } +#line 114 "sql.y" +{ yymsp[0].minor.yy717 = PRIVILEGE_TYPE_WRITE; } +#line 4994 "sql.c" break; case 40: /* priv_level ::= NK_STAR NK_DOT NK_STAR */ -{ yylhsminor.yy347.first = yymsp[-2].minor.yy0; yylhsminor.yy347.second = yymsp[0].minor.yy0; } - yymsp[-2].minor.yy347 = yylhsminor.yy347; +#line 118 "sql.y" +{ yylhsminor.yy505.first = yymsp[-2].minor.yy0; yylhsminor.yy505.second = yymsp[0].minor.yy0; } +#line 4999 "sql.c" + yymsp[-2].minor.yy505 = yylhsminor.yy505; break; case 41: /* priv_level ::= db_name NK_DOT NK_STAR */ -{ yylhsminor.yy347.first = yymsp[-2].minor.yy371; yylhsminor.yy347.second = yymsp[0].minor.yy0; } - yymsp[-2].minor.yy347 = yylhsminor.yy347; +#line 119 "sql.y" +{ yylhsminor.yy505.first = yymsp[-2].minor.yy129; yylhsminor.yy505.second = yymsp[0].minor.yy0; } +#line 5005 "sql.c" + yymsp[-2].minor.yy505 = yylhsminor.yy505; break; case 42: /* priv_level ::= db_name NK_DOT table_name */ -{ yylhsminor.yy347.first = yymsp[-2].minor.yy371; yylhsminor.yy347.second = yymsp[0].minor.yy371; } - yymsp[-2].minor.yy347 = yylhsminor.yy347; +#line 120 "sql.y" +{ yylhsminor.yy505.first = yymsp[-2].minor.yy129; yylhsminor.yy505.second = yymsp[0].minor.yy129; } +#line 5011 "sql.c" + yymsp[-2].minor.yy505 = yylhsminor.yy505; break; case 43: /* priv_level ::= topic_name */ -{ yylhsminor.yy347.first = yymsp[0].minor.yy371; yylhsminor.yy347.second = nil_token; } - yymsp[0].minor.yy347 = yylhsminor.yy347; +#line 121 "sql.y" +{ yylhsminor.yy505.first = yymsp[0].minor.yy129; yylhsminor.yy505.second = nil_token; } +#line 5017 "sql.c" + yymsp[0].minor.yy505 = yylhsminor.yy505; break; case 44: /* with_opt ::= */ case 144: /* start_opt ::= */ yytestcase(yyruleno==144); case 148: /* end_opt ::= */ yytestcase(yyruleno==148); - case 276: /* like_pattern_opt ::= */ yytestcase(yyruleno==276); - case 354: /* subtable_opt ::= */ yytestcase(yyruleno==354); - case 472: /* case_when_else_opt ::= */ yytestcase(yyruleno==472); - case 502: /* from_clause_opt ::= */ yytestcase(yyruleno==502); - case 531: /* where_clause_opt ::= */ yytestcase(yyruleno==531); - case 540: /* twindow_clause_opt ::= */ yytestcase(yyruleno==540); - case 546: /* sliding_opt ::= */ yytestcase(yyruleno==546); - case 548: /* fill_opt ::= */ yytestcase(yyruleno==548); - case 562: /* having_clause_opt ::= */ yytestcase(yyruleno==562); - case 564: /* range_opt ::= */ yytestcase(yyruleno==564); - case 567: /* every_opt ::= */ yytestcase(yyruleno==567); - case 580: /* slimit_clause_opt ::= */ yytestcase(yyruleno==580); - case 584: /* limit_clause_opt ::= */ yytestcase(yyruleno==584); -{ yymsp[1].minor.yy452 = NULL; } + case 277: /* like_pattern_opt ::= */ yytestcase(yyruleno==277); + case 355: /* subtable_opt ::= */ yytestcase(yyruleno==355); + case 473: /* case_when_else_opt ::= */ yytestcase(yyruleno==473); + case 503: /* from_clause_opt ::= */ yytestcase(yyruleno==503); + case 534: /* where_clause_opt ::= */ yytestcase(yyruleno==534); + case 543: /* twindow_clause_opt ::= */ yytestcase(yyruleno==543); + case 549: /* sliding_opt ::= */ yytestcase(yyruleno==549); + case 551: /* fill_opt ::= */ yytestcase(yyruleno==551); + case 565: /* having_clause_opt ::= */ yytestcase(yyruleno==565); + case 567: /* range_opt ::= */ yytestcase(yyruleno==567); + case 570: /* every_opt ::= */ yytestcase(yyruleno==570); + case 583: /* slimit_clause_opt ::= */ yytestcase(yyruleno==583); + case 587: /* limit_clause_opt ::= */ yytestcase(yyruleno==587); +#line 123 "sql.y" +{ yymsp[1].minor.yy840 = NULL; } +#line 5038 "sql.c" break; case 45: /* with_opt ::= WITH search_condition */ - case 503: /* from_clause_opt ::= FROM table_reference_list */ yytestcase(yyruleno==503); - case 532: /* where_clause_opt ::= WHERE search_condition */ yytestcase(yyruleno==532); - case 563: /* having_clause_opt ::= HAVING search_condition */ yytestcase(yyruleno==563); -{ yymsp[-1].minor.yy452 = yymsp[0].minor.yy452; } + case 504: /* from_clause_opt ::= FROM table_reference_list */ yytestcase(yyruleno==504); + case 535: /* where_clause_opt ::= WHERE search_condition */ yytestcase(yyruleno==535); + case 566: /* having_clause_opt ::= HAVING search_condition */ yytestcase(yyruleno==566); +#line 124 "sql.y" +{ yymsp[-1].minor.yy840 = yymsp[0].minor.yy840; } +#line 5046 "sql.c" break; case 46: /* cmd ::= CREATE DNODE dnode_endpoint */ -{ pCxt->pRootNode = createCreateDnodeStmt(pCxt, &yymsp[0].minor.yy371, NULL); } +#line 127 "sql.y" +{ pCxt->pRootNode = createCreateDnodeStmt(pCxt, &yymsp[0].minor.yy129, NULL); } +#line 5051 "sql.c" break; case 47: /* cmd ::= CREATE DNODE dnode_endpoint PORT NK_INTEGER */ -{ pCxt->pRootNode = createCreateDnodeStmt(pCxt, &yymsp[-2].minor.yy371, &yymsp[0].minor.yy0); } +#line 128 "sql.y" +{ pCxt->pRootNode = createCreateDnodeStmt(pCxt, &yymsp[-2].minor.yy129, &yymsp[0].minor.yy0); } +#line 5056 "sql.c" break; case 48: /* cmd ::= DROP DNODE NK_INTEGER force_opt */ -{ pCxt->pRootNode = createDropDnodeStmt(pCxt, &yymsp[-1].minor.yy0, yymsp[0].minor.yy667, false); } +#line 129 "sql.y" +{ pCxt->pRootNode = createDropDnodeStmt(pCxt, &yymsp[-1].minor.yy0, yymsp[0].minor.yy313, false); } +#line 5061 "sql.c" break; case 49: /* cmd ::= DROP DNODE dnode_endpoint force_opt */ -{ pCxt->pRootNode = createDropDnodeStmt(pCxt, &yymsp[-1].minor.yy371, yymsp[0].minor.yy667, false); } +#line 130 "sql.y" +{ pCxt->pRootNode = createDropDnodeStmt(pCxt, &yymsp[-1].minor.yy129, yymsp[0].minor.yy313, false); } +#line 5066 "sql.c" break; case 50: /* cmd ::= DROP DNODE NK_INTEGER unsafe_opt */ -{ pCxt->pRootNode = createDropDnodeStmt(pCxt, &yymsp[-1].minor.yy0, false, yymsp[0].minor.yy667); } +#line 131 "sql.y" +{ pCxt->pRootNode = createDropDnodeStmt(pCxt, &yymsp[-1].minor.yy0, false, yymsp[0].minor.yy313); } +#line 5071 "sql.c" break; case 51: /* cmd ::= DROP DNODE dnode_endpoint unsafe_opt */ -{ pCxt->pRootNode = createDropDnodeStmt(pCxt, &yymsp[-1].minor.yy371, false, yymsp[0].minor.yy667); } +#line 132 "sql.y" +{ pCxt->pRootNode = createDropDnodeStmt(pCxt, &yymsp[-1].minor.yy129, false, yymsp[0].minor.yy313); } +#line 5076 "sql.c" break; case 52: /* cmd ::= ALTER DNODE NK_INTEGER NK_STRING */ +#line 133 "sql.y" { pCxt->pRootNode = createAlterDnodeStmt(pCxt, &yymsp[-1].minor.yy0, &yymsp[0].minor.yy0, NULL); } +#line 5081 "sql.c" break; case 53: /* cmd ::= ALTER DNODE NK_INTEGER NK_STRING NK_STRING */ +#line 134 "sql.y" { pCxt->pRootNode = createAlterDnodeStmt(pCxt, &yymsp[-2].minor.yy0, &yymsp[-1].minor.yy0, &yymsp[0].minor.yy0); } +#line 5086 "sql.c" break; case 54: /* cmd ::= ALTER ALL DNODES NK_STRING */ +#line 135 "sql.y" { pCxt->pRootNode = createAlterDnodeStmt(pCxt, NULL, &yymsp[0].minor.yy0, NULL); } +#line 5091 "sql.c" break; case 55: /* cmd ::= ALTER ALL DNODES NK_STRING NK_STRING */ +#line 136 "sql.y" { pCxt->pRootNode = createAlterDnodeStmt(pCxt, NULL, &yymsp[-1].minor.yy0, &yymsp[0].minor.yy0); } +#line 5096 "sql.c" break; case 56: /* cmd ::= RESTORE DNODE NK_INTEGER */ +#line 137 "sql.y" { pCxt->pRootNode = createRestoreComponentNodeStmt(pCxt, QUERY_NODE_RESTORE_DNODE_STMT, &yymsp[0].minor.yy0); } +#line 5101 "sql.c" break; case 57: /* dnode_endpoint ::= NK_STRING */ case 58: /* dnode_endpoint ::= NK_ID */ yytestcase(yyruleno==58); case 59: /* dnode_endpoint ::= NK_IPTOKEN */ yytestcase(yyruleno==59); - case 300: /* sma_func_name ::= COUNT */ yytestcase(yyruleno==300); - case 301: /* sma_func_name ::= FIRST */ yytestcase(yyruleno==301); - case 302: /* sma_func_name ::= LAST */ yytestcase(yyruleno==302); - case 303: /* sma_func_name ::= LAST_ROW */ yytestcase(yyruleno==303); - case 398: /* db_name ::= NK_ID */ yytestcase(yyruleno==398); - case 399: /* table_name ::= NK_ID */ yytestcase(yyruleno==399); - case 400: /* column_name ::= NK_ID */ yytestcase(yyruleno==400); - case 401: /* function_name ::= NK_ID */ yytestcase(yyruleno==401); - case 402: /* table_alias ::= NK_ID */ yytestcase(yyruleno==402); - case 403: /* column_alias ::= NK_ID */ yytestcase(yyruleno==403); - case 404: /* user_name ::= NK_ID */ yytestcase(yyruleno==404); - case 405: /* topic_name ::= NK_ID */ yytestcase(yyruleno==405); - case 406: /* stream_name ::= NK_ID */ yytestcase(yyruleno==406); - case 407: /* cgroup_name ::= NK_ID */ yytestcase(yyruleno==407); - case 408: /* index_name ::= NK_ID */ yytestcase(yyruleno==408); - case 448: /* noarg_func ::= NOW */ yytestcase(yyruleno==448); - case 449: /* noarg_func ::= TODAY */ yytestcase(yyruleno==449); - case 450: /* noarg_func ::= TIMEZONE */ yytestcase(yyruleno==450); - case 451: /* noarg_func ::= DATABASE */ yytestcase(yyruleno==451); - case 452: /* noarg_func ::= CLIENT_VERSION */ yytestcase(yyruleno==452); - case 453: /* noarg_func ::= SERVER_VERSION */ yytestcase(yyruleno==453); - case 454: /* noarg_func ::= SERVER_STATUS */ yytestcase(yyruleno==454); - case 455: /* noarg_func ::= CURRENT_USER */ yytestcase(yyruleno==455); - case 456: /* noarg_func ::= USER */ yytestcase(yyruleno==456); - case 457: /* star_func ::= COUNT */ yytestcase(yyruleno==457); - case 458: /* star_func ::= FIRST */ yytestcase(yyruleno==458); - case 459: /* star_func ::= LAST */ yytestcase(yyruleno==459); - case 460: /* star_func ::= LAST_ROW */ yytestcase(yyruleno==460); -{ yylhsminor.yy371 = yymsp[0].minor.yy0; } - yymsp[0].minor.yy371 = yylhsminor.yy371; + case 301: /* sma_func_name ::= COUNT */ yytestcase(yyruleno==301); + case 302: /* sma_func_name ::= FIRST */ yytestcase(yyruleno==302); + case 303: /* sma_func_name ::= LAST */ yytestcase(yyruleno==303); + case 304: /* sma_func_name ::= LAST_ROW */ yytestcase(yyruleno==304); + case 399: /* db_name ::= NK_ID */ yytestcase(yyruleno==399); + case 400: /* table_name ::= NK_ID */ yytestcase(yyruleno==400); + case 401: /* column_name ::= NK_ID */ yytestcase(yyruleno==401); + case 402: /* function_name ::= NK_ID */ yytestcase(yyruleno==402); + case 403: /* table_alias ::= NK_ID */ yytestcase(yyruleno==403); + case 404: /* column_alias ::= NK_ID */ yytestcase(yyruleno==404); + case 405: /* user_name ::= NK_ID */ yytestcase(yyruleno==405); + case 406: /* topic_name ::= NK_ID */ yytestcase(yyruleno==406); + case 407: /* stream_name ::= NK_ID */ yytestcase(yyruleno==407); + case 408: /* cgroup_name ::= NK_ID */ yytestcase(yyruleno==408); + case 409: /* index_name ::= NK_ID */ yytestcase(yyruleno==409); + case 449: /* noarg_func ::= NOW */ yytestcase(yyruleno==449); + case 450: /* noarg_func ::= TODAY */ yytestcase(yyruleno==450); + case 451: /* noarg_func ::= TIMEZONE */ yytestcase(yyruleno==451); + case 452: /* noarg_func ::= DATABASE */ yytestcase(yyruleno==452); + case 453: /* noarg_func ::= CLIENT_VERSION */ yytestcase(yyruleno==453); + case 454: /* noarg_func ::= SERVER_VERSION */ yytestcase(yyruleno==454); + case 455: /* noarg_func ::= SERVER_STATUS */ yytestcase(yyruleno==455); + case 456: /* noarg_func ::= CURRENT_USER */ yytestcase(yyruleno==456); + case 457: /* noarg_func ::= USER */ yytestcase(yyruleno==457); + case 458: /* star_func ::= COUNT */ yytestcase(yyruleno==458); + case 459: /* star_func ::= FIRST */ yytestcase(yyruleno==459); + case 460: /* star_func ::= LAST */ yytestcase(yyruleno==460); + case 461: /* star_func ::= LAST_ROW */ yytestcase(yyruleno==461); +#line 141 "sql.y" +{ yylhsminor.yy129 = yymsp[0].minor.yy0; } +#line 5136 "sql.c" + yymsp[0].minor.yy129 = yylhsminor.yy129; break; case 60: /* force_opt ::= */ case 84: /* not_exists_opt ::= */ yytestcase(yyruleno==84); case 86: /* exists_opt ::= */ yytestcase(yyruleno==86); - case 321: /* analyze_opt ::= */ yytestcase(yyruleno==321); - case 328: /* agg_func_opt ::= */ yytestcase(yyruleno==328); - case 334: /* or_replace_opt ::= */ yytestcase(yyruleno==334); - case 356: /* ignore_opt ::= */ yytestcase(yyruleno==356); - case 521: /* set_quantifier_opt ::= */ yytestcase(yyruleno==521); -{ yymsp[1].minor.yy667 = false; } + case 322: /* analyze_opt ::= */ yytestcase(yyruleno==322); + case 329: /* agg_func_opt ::= */ yytestcase(yyruleno==329); + case 335: /* or_replace_opt ::= */ yytestcase(yyruleno==335); + case 357: /* ignore_opt ::= */ yytestcase(yyruleno==357); + case 522: /* tag_mode_opt ::= */ yytestcase(yyruleno==522); + case 524: /* set_quantifier_opt ::= */ yytestcase(yyruleno==524); +#line 147 "sql.y" +{ yymsp[1].minor.yy313 = false; } +#line 5150 "sql.c" break; case 61: /* force_opt ::= FORCE */ case 62: /* unsafe_opt ::= UNSAFE */ yytestcase(yyruleno==62); - case 322: /* analyze_opt ::= ANALYZE */ yytestcase(yyruleno==322); - case 329: /* agg_func_opt ::= AGGREGATE */ yytestcase(yyruleno==329); - case 522: /* set_quantifier_opt ::= DISTINCT */ yytestcase(yyruleno==522); -{ yymsp[0].minor.yy667 = true; } + case 323: /* analyze_opt ::= ANALYZE */ yytestcase(yyruleno==323); + case 330: /* agg_func_opt ::= AGGREGATE */ yytestcase(yyruleno==330); + case 523: /* tag_mode_opt ::= TAGS */ yytestcase(yyruleno==523); + case 525: /* set_quantifier_opt ::= DISTINCT */ yytestcase(yyruleno==525); +#line 148 "sql.y" +{ yymsp[0].minor.yy313 = true; } +#line 5160 "sql.c" break; case 63: /* cmd ::= ALTER LOCAL NK_STRING */ +#line 155 "sql.y" { pCxt->pRootNode = createAlterLocalStmt(pCxt, &yymsp[0].minor.yy0, NULL); } +#line 5165 "sql.c" break; case 64: /* cmd ::= ALTER LOCAL NK_STRING NK_STRING */ +#line 156 "sql.y" { pCxt->pRootNode = createAlterLocalStmt(pCxt, &yymsp[-1].minor.yy0, &yymsp[0].minor.yy0); } +#line 5170 "sql.c" break; case 65: /* cmd ::= CREATE QNODE ON DNODE NK_INTEGER */ +#line 159 "sql.y" { pCxt->pRootNode = createCreateComponentNodeStmt(pCxt, QUERY_NODE_CREATE_QNODE_STMT, &yymsp[0].minor.yy0); } +#line 5175 "sql.c" break; case 66: /* cmd ::= DROP QNODE ON DNODE NK_INTEGER */ +#line 160 "sql.y" { pCxt->pRootNode = createDropComponentNodeStmt(pCxt, QUERY_NODE_DROP_QNODE_STMT, &yymsp[0].minor.yy0); } +#line 5180 "sql.c" break; case 67: /* cmd ::= RESTORE QNODE ON DNODE NK_INTEGER */ +#line 161 "sql.y" { pCxt->pRootNode = createRestoreComponentNodeStmt(pCxt, QUERY_NODE_RESTORE_QNODE_STMT, &yymsp[0].minor.yy0); } +#line 5185 "sql.c" break; case 68: /* cmd ::= CREATE BNODE ON DNODE NK_INTEGER */ +#line 164 "sql.y" { pCxt->pRootNode = createCreateComponentNodeStmt(pCxt, QUERY_NODE_CREATE_BNODE_STMT, &yymsp[0].minor.yy0); } +#line 5190 "sql.c" break; case 69: /* cmd ::= DROP BNODE ON DNODE NK_INTEGER */ +#line 165 "sql.y" { pCxt->pRootNode = createDropComponentNodeStmt(pCxt, QUERY_NODE_DROP_BNODE_STMT, &yymsp[0].minor.yy0); } +#line 5195 "sql.c" break; case 70: /* cmd ::= CREATE SNODE ON DNODE NK_INTEGER */ +#line 168 "sql.y" { pCxt->pRootNode = createCreateComponentNodeStmt(pCxt, QUERY_NODE_CREATE_SNODE_STMT, &yymsp[0].minor.yy0); } +#line 5200 "sql.c" break; case 71: /* cmd ::= DROP SNODE ON DNODE NK_INTEGER */ +#line 169 "sql.y" { pCxt->pRootNode = createDropComponentNodeStmt(pCxt, QUERY_NODE_DROP_SNODE_STMT, &yymsp[0].minor.yy0); } +#line 5205 "sql.c" break; case 72: /* cmd ::= CREATE MNODE ON DNODE NK_INTEGER */ +#line 172 "sql.y" { pCxt->pRootNode = createCreateComponentNodeStmt(pCxt, QUERY_NODE_CREATE_MNODE_STMT, &yymsp[0].minor.yy0); } +#line 5210 "sql.c" break; case 73: /* cmd ::= DROP MNODE ON DNODE NK_INTEGER */ +#line 173 "sql.y" { pCxt->pRootNode = createDropComponentNodeStmt(pCxt, QUERY_NODE_DROP_MNODE_STMT, &yymsp[0].minor.yy0); } +#line 5215 "sql.c" break; case 74: /* cmd ::= RESTORE MNODE ON DNODE NK_INTEGER */ +#line 174 "sql.y" { pCxt->pRootNode = createRestoreComponentNodeStmt(pCxt, QUERY_NODE_RESTORE_MNODE_STMT, &yymsp[0].minor.yy0); } +#line 5220 "sql.c" break; case 75: /* cmd ::= RESTORE VNODE ON DNODE NK_INTEGER */ +#line 177 "sql.y" { pCxt->pRootNode = createRestoreComponentNodeStmt(pCxt, QUERY_NODE_RESTORE_VNODE_STMT, &yymsp[0].minor.yy0); } +#line 5225 "sql.c" break; case 76: /* cmd ::= CREATE DATABASE not_exists_opt db_name db_options */ -{ pCxt->pRootNode = createCreateDatabaseStmt(pCxt, yymsp[-2].minor.yy667, &yymsp[-1].minor.yy371, yymsp[0].minor.yy452); } +#line 180 "sql.y" +{ pCxt->pRootNode = createCreateDatabaseStmt(pCxt, yymsp[-2].minor.yy313, &yymsp[-1].minor.yy129, yymsp[0].minor.yy840); } +#line 5230 "sql.c" break; case 77: /* cmd ::= DROP DATABASE exists_opt db_name */ -{ pCxt->pRootNode = createDropDatabaseStmt(pCxt, yymsp[-1].minor.yy667, &yymsp[0].minor.yy371); } +#line 181 "sql.y" +{ pCxt->pRootNode = createDropDatabaseStmt(pCxt, yymsp[-1].minor.yy313, &yymsp[0].minor.yy129); } +#line 5235 "sql.c" break; case 78: /* cmd ::= USE db_name */ -{ pCxt->pRootNode = createUseDatabaseStmt(pCxt, &yymsp[0].minor.yy371); } +#line 182 "sql.y" +{ pCxt->pRootNode = createUseDatabaseStmt(pCxt, &yymsp[0].minor.yy129); } +#line 5240 "sql.c" break; case 79: /* cmd ::= ALTER DATABASE db_name alter_db_options */ -{ pCxt->pRootNode = createAlterDatabaseStmt(pCxt, &yymsp[-1].minor.yy371, yymsp[0].minor.yy452); } +#line 183 "sql.y" +{ pCxt->pRootNode = createAlterDatabaseStmt(pCxt, &yymsp[-1].minor.yy129, yymsp[0].minor.yy840); } +#line 5245 "sql.c" break; case 80: /* cmd ::= FLUSH DATABASE db_name */ -{ pCxt->pRootNode = createFlushDatabaseStmt(pCxt, &yymsp[0].minor.yy371); } +#line 184 "sql.y" +{ pCxt->pRootNode = createFlushDatabaseStmt(pCxt, &yymsp[0].minor.yy129); } +#line 5250 "sql.c" break; case 81: /* cmd ::= TRIM DATABASE db_name speed_opt */ -{ pCxt->pRootNode = createTrimDatabaseStmt(pCxt, &yymsp[-1].minor.yy371, yymsp[0].minor.yy416); } +#line 185 "sql.y" +{ pCxt->pRootNode = createTrimDatabaseStmt(pCxt, &yymsp[-1].minor.yy129, yymsp[0].minor.yy356); } +#line 5255 "sql.c" break; case 82: /* cmd ::= COMPACT DATABASE db_name start_opt end_opt */ -{ pCxt->pRootNode = createCompactStmt(pCxt, &yymsp[-2].minor.yy371, yymsp[-1].minor.yy452, yymsp[0].minor.yy452); } +#line 186 "sql.y" +{ pCxt->pRootNode = createCompactStmt(pCxt, &yymsp[-2].minor.yy129, yymsp[-1].minor.yy840, yymsp[0].minor.yy840); } +#line 5260 "sql.c" break; case 83: /* not_exists_opt ::= IF NOT EXISTS */ -{ yymsp[-2].minor.yy667 = true; } +#line 190 "sql.y" +{ yymsp[-2].minor.yy313 = true; } +#line 5265 "sql.c" break; case 85: /* exists_opt ::= IF EXISTS */ - case 335: /* or_replace_opt ::= OR REPLACE */ yytestcase(yyruleno==335); - case 357: /* ignore_opt ::= IGNORE UNTREATED */ yytestcase(yyruleno==357); -{ yymsp[-1].minor.yy667 = true; } + case 336: /* or_replace_opt ::= OR REPLACE */ yytestcase(yyruleno==336); + case 358: /* ignore_opt ::= IGNORE UNTREATED */ yytestcase(yyruleno==358); +#line 195 "sql.y" +{ yymsp[-1].minor.yy313 = true; } +#line 5272 "sql.c" break; case 87: /* db_options ::= */ -{ yymsp[1].minor.yy452 = createDefaultDatabaseOptions(pCxt); } +#line 198 "sql.y" +{ yymsp[1].minor.yy840 = createDefaultDatabaseOptions(pCxt); } +#line 5277 "sql.c" break; case 88: /* db_options ::= db_options BUFFER NK_INTEGER */ -{ yylhsminor.yy452 = setDatabaseOption(pCxt, yymsp[-2].minor.yy452, DB_OPTION_BUFFER, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 199 "sql.y" +{ yylhsminor.yy840 = setDatabaseOption(pCxt, yymsp[-2].minor.yy840, DB_OPTION_BUFFER, &yymsp[0].minor.yy0); } +#line 5282 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; case 89: /* db_options ::= db_options CACHEMODEL NK_STRING */ -{ yylhsminor.yy452 = setDatabaseOption(pCxt, yymsp[-2].minor.yy452, DB_OPTION_CACHEMODEL, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 200 "sql.y" +{ yylhsminor.yy840 = setDatabaseOption(pCxt, yymsp[-2].minor.yy840, DB_OPTION_CACHEMODEL, &yymsp[0].minor.yy0); } +#line 5288 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; case 90: /* db_options ::= db_options CACHESIZE NK_INTEGER */ -{ yylhsminor.yy452 = setDatabaseOption(pCxt, yymsp[-2].minor.yy452, DB_OPTION_CACHESIZE, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 201 "sql.y" +{ yylhsminor.yy840 = setDatabaseOption(pCxt, yymsp[-2].minor.yy840, DB_OPTION_CACHESIZE, &yymsp[0].minor.yy0); } +#line 5294 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; case 91: /* db_options ::= db_options COMP NK_INTEGER */ -{ yylhsminor.yy452 = setDatabaseOption(pCxt, yymsp[-2].minor.yy452, DB_OPTION_COMP, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 202 "sql.y" +{ yylhsminor.yy840 = setDatabaseOption(pCxt, yymsp[-2].minor.yy840, DB_OPTION_COMP, &yymsp[0].minor.yy0); } +#line 5300 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; case 92: /* db_options ::= db_options DURATION NK_INTEGER */ case 93: /* db_options ::= db_options DURATION NK_VARIABLE */ yytestcase(yyruleno==93); -{ yylhsminor.yy452 = setDatabaseOption(pCxt, yymsp[-2].minor.yy452, DB_OPTION_DAYS, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 203 "sql.y" +{ yylhsminor.yy840 = setDatabaseOption(pCxt, yymsp[-2].minor.yy840, DB_OPTION_DAYS, &yymsp[0].minor.yy0); } +#line 5307 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; case 94: /* db_options ::= db_options MAXROWS NK_INTEGER */ -{ yylhsminor.yy452 = setDatabaseOption(pCxt, yymsp[-2].minor.yy452, DB_OPTION_MAXROWS, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 205 "sql.y" +{ yylhsminor.yy840 = setDatabaseOption(pCxt, yymsp[-2].minor.yy840, DB_OPTION_MAXROWS, &yymsp[0].minor.yy0); } +#line 5313 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; case 95: /* db_options ::= db_options MINROWS NK_INTEGER */ -{ yylhsminor.yy452 = setDatabaseOption(pCxt, yymsp[-2].minor.yy452, DB_OPTION_MINROWS, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 206 "sql.y" +{ yylhsminor.yy840 = setDatabaseOption(pCxt, yymsp[-2].minor.yy840, DB_OPTION_MINROWS, &yymsp[0].minor.yy0); } +#line 5319 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; case 96: /* db_options ::= db_options KEEP integer_list */ case 97: /* db_options ::= db_options KEEP variable_list */ yytestcase(yyruleno==97); -{ yylhsminor.yy452 = setDatabaseOption(pCxt, yymsp[-2].minor.yy452, DB_OPTION_KEEP, yymsp[0].minor.yy812); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 207 "sql.y" +{ yylhsminor.yy840 = setDatabaseOption(pCxt, yymsp[-2].minor.yy840, DB_OPTION_KEEP, yymsp[0].minor.yy56); } +#line 5326 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; case 98: /* db_options ::= db_options PAGES NK_INTEGER */ -{ yylhsminor.yy452 = setDatabaseOption(pCxt, yymsp[-2].minor.yy452, DB_OPTION_PAGES, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 209 "sql.y" +{ yylhsminor.yy840 = setDatabaseOption(pCxt, yymsp[-2].minor.yy840, DB_OPTION_PAGES, &yymsp[0].minor.yy0); } +#line 5332 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; case 99: /* db_options ::= db_options PAGESIZE NK_INTEGER */ -{ yylhsminor.yy452 = setDatabaseOption(pCxt, yymsp[-2].minor.yy452, DB_OPTION_PAGESIZE, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 210 "sql.y" +{ yylhsminor.yy840 = setDatabaseOption(pCxt, yymsp[-2].minor.yy840, DB_OPTION_PAGESIZE, &yymsp[0].minor.yy0); } +#line 5338 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; case 100: /* db_options ::= db_options TSDB_PAGESIZE NK_INTEGER */ -{ yylhsminor.yy452 = setDatabaseOption(pCxt, yymsp[-2].minor.yy452, DB_OPTION_TSDB_PAGESIZE, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 211 "sql.y" +{ yylhsminor.yy840 = setDatabaseOption(pCxt, yymsp[-2].minor.yy840, DB_OPTION_TSDB_PAGESIZE, &yymsp[0].minor.yy0); } +#line 5344 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; case 101: /* db_options ::= db_options PRECISION NK_STRING */ -{ yylhsminor.yy452 = setDatabaseOption(pCxt, yymsp[-2].minor.yy452, DB_OPTION_PRECISION, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 212 "sql.y" +{ yylhsminor.yy840 = setDatabaseOption(pCxt, yymsp[-2].minor.yy840, DB_OPTION_PRECISION, &yymsp[0].minor.yy0); } +#line 5350 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; case 102: /* db_options ::= db_options REPLICA NK_INTEGER */ -{ yylhsminor.yy452 = setDatabaseOption(pCxt, yymsp[-2].minor.yy452, DB_OPTION_REPLICA, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 213 "sql.y" +{ yylhsminor.yy840 = setDatabaseOption(pCxt, yymsp[-2].minor.yy840, DB_OPTION_REPLICA, &yymsp[0].minor.yy0); } +#line 5356 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; case 103: /* db_options ::= db_options VGROUPS NK_INTEGER */ -{ yylhsminor.yy452 = setDatabaseOption(pCxt, yymsp[-2].minor.yy452, DB_OPTION_VGROUPS, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 215 "sql.y" +{ yylhsminor.yy840 = setDatabaseOption(pCxt, yymsp[-2].minor.yy840, DB_OPTION_VGROUPS, &yymsp[0].minor.yy0); } +#line 5362 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; case 104: /* db_options ::= db_options SINGLE_STABLE NK_INTEGER */ -{ yylhsminor.yy452 = setDatabaseOption(pCxt, yymsp[-2].minor.yy452, DB_OPTION_SINGLE_STABLE, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 216 "sql.y" +{ yylhsminor.yy840 = setDatabaseOption(pCxt, yymsp[-2].minor.yy840, DB_OPTION_SINGLE_STABLE, &yymsp[0].minor.yy0); } +#line 5368 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; case 105: /* db_options ::= db_options RETENTIONS retention_list */ -{ yylhsminor.yy452 = setDatabaseOption(pCxt, yymsp[-2].minor.yy452, DB_OPTION_RETENTIONS, yymsp[0].minor.yy812); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 217 "sql.y" +{ yylhsminor.yy840 = setDatabaseOption(pCxt, yymsp[-2].minor.yy840, DB_OPTION_RETENTIONS, yymsp[0].minor.yy56); } +#line 5374 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; case 106: /* db_options ::= db_options SCHEMALESS NK_INTEGER */ -{ yylhsminor.yy452 = setDatabaseOption(pCxt, yymsp[-2].minor.yy452, DB_OPTION_SCHEMALESS, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 218 "sql.y" +{ yylhsminor.yy840 = setDatabaseOption(pCxt, yymsp[-2].minor.yy840, DB_OPTION_SCHEMALESS, &yymsp[0].minor.yy0); } +#line 5380 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; case 107: /* db_options ::= db_options WAL_LEVEL NK_INTEGER */ -{ yylhsminor.yy452 = setDatabaseOption(pCxt, yymsp[-2].minor.yy452, DB_OPTION_WAL, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 219 "sql.y" +{ yylhsminor.yy840 = setDatabaseOption(pCxt, yymsp[-2].minor.yy840, DB_OPTION_WAL, &yymsp[0].minor.yy0); } +#line 5386 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; case 108: /* db_options ::= db_options WAL_FSYNC_PERIOD NK_INTEGER */ -{ yylhsminor.yy452 = setDatabaseOption(pCxt, yymsp[-2].minor.yy452, DB_OPTION_FSYNC, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 220 "sql.y" +{ yylhsminor.yy840 = setDatabaseOption(pCxt, yymsp[-2].minor.yy840, DB_OPTION_FSYNC, &yymsp[0].minor.yy0); } +#line 5392 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; case 109: /* db_options ::= db_options WAL_RETENTION_PERIOD NK_INTEGER */ -{ yylhsminor.yy452 = setDatabaseOption(pCxt, yymsp[-2].minor.yy452, DB_OPTION_WAL_RETENTION_PERIOD, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 221 "sql.y" +{ yylhsminor.yy840 = setDatabaseOption(pCxt, yymsp[-2].minor.yy840, DB_OPTION_WAL_RETENTION_PERIOD, &yymsp[0].minor.yy0); } +#line 5398 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; case 110: /* db_options ::= db_options WAL_RETENTION_PERIOD NK_MINUS NK_INTEGER */ +#line 222 "sql.y" { SToken t = yymsp[-1].minor.yy0; t.n = (yymsp[0].minor.yy0.z + yymsp[0].minor.yy0.n) - yymsp[-1].minor.yy0.z; - yylhsminor.yy452 = setDatabaseOption(pCxt, yymsp[-3].minor.yy452, DB_OPTION_WAL_RETENTION_PERIOD, &t); + yylhsminor.yy840 = setDatabaseOption(pCxt, yymsp[-3].minor.yy840, DB_OPTION_WAL_RETENTION_PERIOD, &t); } - yymsp[-3].minor.yy452 = yylhsminor.yy452; +#line 5408 "sql.c" + yymsp[-3].minor.yy840 = yylhsminor.yy840; break; case 111: /* db_options ::= db_options WAL_RETENTION_SIZE NK_INTEGER */ -{ yylhsminor.yy452 = setDatabaseOption(pCxt, yymsp[-2].minor.yy452, DB_OPTION_WAL_RETENTION_SIZE, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 227 "sql.y" +{ yylhsminor.yy840 = setDatabaseOption(pCxt, yymsp[-2].minor.yy840, DB_OPTION_WAL_RETENTION_SIZE, &yymsp[0].minor.yy0); } +#line 5414 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; case 112: /* db_options ::= db_options WAL_RETENTION_SIZE NK_MINUS NK_INTEGER */ +#line 228 "sql.y" { SToken t = yymsp[-1].minor.yy0; t.n = (yymsp[0].minor.yy0.z + yymsp[0].minor.yy0.n) - yymsp[-1].minor.yy0.z; - yylhsminor.yy452 = setDatabaseOption(pCxt, yymsp[-3].minor.yy452, DB_OPTION_WAL_RETENTION_SIZE, &t); + yylhsminor.yy840 = setDatabaseOption(pCxt, yymsp[-3].minor.yy840, DB_OPTION_WAL_RETENTION_SIZE, &t); } - yymsp[-3].minor.yy452 = yylhsminor.yy452; +#line 5424 "sql.c" + yymsp[-3].minor.yy840 = yylhsminor.yy840; break; case 113: /* db_options ::= db_options WAL_ROLL_PERIOD NK_INTEGER */ -{ yylhsminor.yy452 = setDatabaseOption(pCxt, yymsp[-2].minor.yy452, DB_OPTION_WAL_ROLL_PERIOD, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 233 "sql.y" +{ yylhsminor.yy840 = setDatabaseOption(pCxt, yymsp[-2].minor.yy840, DB_OPTION_WAL_ROLL_PERIOD, &yymsp[0].minor.yy0); } +#line 5430 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; case 114: /* db_options ::= db_options WAL_SEGMENT_SIZE NK_INTEGER */ -{ yylhsminor.yy452 = setDatabaseOption(pCxt, yymsp[-2].minor.yy452, DB_OPTION_WAL_SEGMENT_SIZE, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 234 "sql.y" +{ yylhsminor.yy840 = setDatabaseOption(pCxt, yymsp[-2].minor.yy840, DB_OPTION_WAL_SEGMENT_SIZE, &yymsp[0].minor.yy0); } +#line 5436 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; case 115: /* db_options ::= db_options STT_TRIGGER NK_INTEGER */ -{ yylhsminor.yy452 = setDatabaseOption(pCxt, yymsp[-2].minor.yy452, DB_OPTION_STT_TRIGGER, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 235 "sql.y" +{ yylhsminor.yy840 = setDatabaseOption(pCxt, yymsp[-2].minor.yy840, DB_OPTION_STT_TRIGGER, &yymsp[0].minor.yy0); } +#line 5442 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; case 116: /* db_options ::= db_options TABLE_PREFIX signed */ -{ yylhsminor.yy452 = setDatabaseOption(pCxt, yymsp[-2].minor.yy452, DB_OPTION_TABLE_PREFIX, yymsp[0].minor.yy452); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 236 "sql.y" +{ yylhsminor.yy840 = setDatabaseOption(pCxt, yymsp[-2].minor.yy840, DB_OPTION_TABLE_PREFIX, yymsp[0].minor.yy840); } +#line 5448 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; case 117: /* db_options ::= db_options TABLE_SUFFIX signed */ -{ yylhsminor.yy452 = setDatabaseOption(pCxt, yymsp[-2].minor.yy452, DB_OPTION_TABLE_SUFFIX, yymsp[0].minor.yy452); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 237 "sql.y" +{ yylhsminor.yy840 = setDatabaseOption(pCxt, yymsp[-2].minor.yy840, DB_OPTION_TABLE_SUFFIX, yymsp[0].minor.yy840); } +#line 5454 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; case 118: /* alter_db_options ::= alter_db_option */ -{ yylhsminor.yy452 = createAlterDatabaseOptions(pCxt); yylhsminor.yy452 = setAlterDatabaseOption(pCxt, yylhsminor.yy452, &yymsp[0].minor.yy365); } - yymsp[0].minor.yy452 = yylhsminor.yy452; +#line 239 "sql.y" +{ yylhsminor.yy840 = createAlterDatabaseOptions(pCxt); yylhsminor.yy840 = setAlterDatabaseOption(pCxt, yylhsminor.yy840, &yymsp[0].minor.yy461); } +#line 5460 "sql.c" + yymsp[0].minor.yy840 = yylhsminor.yy840; break; case 119: /* alter_db_options ::= alter_db_options alter_db_option */ -{ yylhsminor.yy452 = setAlterDatabaseOption(pCxt, yymsp[-1].minor.yy452, &yymsp[0].minor.yy365); } - yymsp[-1].minor.yy452 = yylhsminor.yy452; +#line 240 "sql.y" +{ yylhsminor.yy840 = setAlterDatabaseOption(pCxt, yymsp[-1].minor.yy840, &yymsp[0].minor.yy461); } +#line 5466 "sql.c" + yymsp[-1].minor.yy840 = yylhsminor.yy840; break; case 120: /* alter_db_option ::= BUFFER NK_INTEGER */ -{ yymsp[-1].minor.yy365.type = DB_OPTION_BUFFER; yymsp[-1].minor.yy365.val = yymsp[0].minor.yy0; } +#line 244 "sql.y" +{ yymsp[-1].minor.yy461.type = DB_OPTION_BUFFER; yymsp[-1].minor.yy461.val = yymsp[0].minor.yy0; } +#line 5472 "sql.c" break; case 121: /* alter_db_option ::= CACHEMODEL NK_STRING */ -{ yymsp[-1].minor.yy365.type = DB_OPTION_CACHEMODEL; yymsp[-1].minor.yy365.val = yymsp[0].minor.yy0; } +#line 245 "sql.y" +{ yymsp[-1].minor.yy461.type = DB_OPTION_CACHEMODEL; yymsp[-1].minor.yy461.val = yymsp[0].minor.yy0; } +#line 5477 "sql.c" break; case 122: /* alter_db_option ::= CACHESIZE NK_INTEGER */ -{ yymsp[-1].minor.yy365.type = DB_OPTION_CACHESIZE; yymsp[-1].minor.yy365.val = yymsp[0].minor.yy0; } +#line 246 "sql.y" +{ yymsp[-1].minor.yy461.type = DB_OPTION_CACHESIZE; yymsp[-1].minor.yy461.val = yymsp[0].minor.yy0; } +#line 5482 "sql.c" break; case 123: /* alter_db_option ::= WAL_FSYNC_PERIOD NK_INTEGER */ -{ yymsp[-1].minor.yy365.type = DB_OPTION_FSYNC; yymsp[-1].minor.yy365.val = yymsp[0].minor.yy0; } +#line 247 "sql.y" +{ yymsp[-1].minor.yy461.type = DB_OPTION_FSYNC; yymsp[-1].minor.yy461.val = yymsp[0].minor.yy0; } +#line 5487 "sql.c" break; case 124: /* alter_db_option ::= KEEP integer_list */ case 125: /* alter_db_option ::= KEEP variable_list */ yytestcase(yyruleno==125); -{ yymsp[-1].minor.yy365.type = DB_OPTION_KEEP; yymsp[-1].minor.yy365.pList = yymsp[0].minor.yy812; } +#line 248 "sql.y" +{ yymsp[-1].minor.yy461.type = DB_OPTION_KEEP; yymsp[-1].minor.yy461.pList = yymsp[0].minor.yy56; } +#line 5493 "sql.c" break; case 126: /* alter_db_option ::= PAGES NK_INTEGER */ -{ yymsp[-1].minor.yy365.type = DB_OPTION_PAGES; yymsp[-1].minor.yy365.val = yymsp[0].minor.yy0; } +#line 250 "sql.y" +{ yymsp[-1].minor.yy461.type = DB_OPTION_PAGES; yymsp[-1].minor.yy461.val = yymsp[0].minor.yy0; } +#line 5498 "sql.c" break; case 127: /* alter_db_option ::= REPLICA NK_INTEGER */ -{ yymsp[-1].minor.yy365.type = DB_OPTION_REPLICA; yymsp[-1].minor.yy365.val = yymsp[0].minor.yy0; } +#line 251 "sql.y" +{ yymsp[-1].minor.yy461.type = DB_OPTION_REPLICA; yymsp[-1].minor.yy461.val = yymsp[0].minor.yy0; } +#line 5503 "sql.c" break; case 128: /* alter_db_option ::= WAL_LEVEL NK_INTEGER */ -{ yymsp[-1].minor.yy365.type = DB_OPTION_WAL; yymsp[-1].minor.yy365.val = yymsp[0].minor.yy0; } +#line 253 "sql.y" +{ yymsp[-1].minor.yy461.type = DB_OPTION_WAL; yymsp[-1].minor.yy461.val = yymsp[0].minor.yy0; } +#line 5508 "sql.c" break; case 129: /* alter_db_option ::= STT_TRIGGER NK_INTEGER */ -{ yymsp[-1].minor.yy365.type = DB_OPTION_STT_TRIGGER; yymsp[-1].minor.yy365.val = yymsp[0].minor.yy0; } +#line 254 "sql.y" +{ yymsp[-1].minor.yy461.type = DB_OPTION_STT_TRIGGER; yymsp[-1].minor.yy461.val = yymsp[0].minor.yy0; } +#line 5513 "sql.c" break; case 130: /* alter_db_option ::= MINROWS NK_INTEGER */ -{ yymsp[-1].minor.yy365.type = DB_OPTION_MINROWS; yymsp[-1].minor.yy365.val = yymsp[0].minor.yy0; } +#line 255 "sql.y" +{ yymsp[-1].minor.yy461.type = DB_OPTION_MINROWS; yymsp[-1].minor.yy461.val = yymsp[0].minor.yy0; } +#line 5518 "sql.c" break; case 131: /* alter_db_option ::= WAL_RETENTION_PERIOD NK_INTEGER */ -{ yymsp[-1].minor.yy365.type = DB_OPTION_WAL_RETENTION_PERIOD; yymsp[-1].minor.yy365.val = yymsp[0].minor.yy0; } +#line 256 "sql.y" +{ yymsp[-1].minor.yy461.type = DB_OPTION_WAL_RETENTION_PERIOD; yymsp[-1].minor.yy461.val = yymsp[0].minor.yy0; } +#line 5523 "sql.c" break; case 132: /* alter_db_option ::= WAL_RETENTION_PERIOD NK_MINUS NK_INTEGER */ +#line 257 "sql.y" { SToken t = yymsp[-1].minor.yy0; t.n = (yymsp[0].minor.yy0.z + yymsp[0].minor.yy0.n) - yymsp[-1].minor.yy0.z; - yymsp[-2].minor.yy365.type = DB_OPTION_WAL_RETENTION_PERIOD; yymsp[-2].minor.yy365.val = t; + yymsp[-2].minor.yy461.type = DB_OPTION_WAL_RETENTION_PERIOD; yymsp[-2].minor.yy461.val = t; } +#line 5532 "sql.c" break; case 133: /* alter_db_option ::= WAL_RETENTION_SIZE NK_INTEGER */ -{ yymsp[-1].minor.yy365.type = DB_OPTION_WAL_RETENTION_SIZE; yymsp[-1].minor.yy365.val = yymsp[0].minor.yy0; } +#line 262 "sql.y" +{ yymsp[-1].minor.yy461.type = DB_OPTION_WAL_RETENTION_SIZE; yymsp[-1].minor.yy461.val = yymsp[0].minor.yy0; } +#line 5537 "sql.c" break; case 134: /* alter_db_option ::= WAL_RETENTION_SIZE NK_MINUS NK_INTEGER */ +#line 263 "sql.y" { SToken t = yymsp[-1].minor.yy0; t.n = (yymsp[0].minor.yy0.z + yymsp[0].minor.yy0.n) - yymsp[-1].minor.yy0.z; - yymsp[-2].minor.yy365.type = DB_OPTION_WAL_RETENTION_SIZE; yymsp[-2].minor.yy365.val = t; + yymsp[-2].minor.yy461.type = DB_OPTION_WAL_RETENTION_SIZE; yymsp[-2].minor.yy461.val = t; } +#line 5546 "sql.c" break; case 135: /* integer_list ::= NK_INTEGER */ -{ yylhsminor.yy812 = createNodeList(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0)); } - yymsp[0].minor.yy812 = yylhsminor.yy812; +#line 271 "sql.y" +{ yylhsminor.yy56 = createNodeList(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0)); } +#line 5551 "sql.c" + yymsp[0].minor.yy56 = yylhsminor.yy56; break; case 136: /* integer_list ::= integer_list NK_COMMA NK_INTEGER */ - case 367: /* dnode_list ::= dnode_list DNODE NK_INTEGER */ yytestcase(yyruleno==367); -{ yylhsminor.yy812 = addNodeToList(pCxt, yymsp[-2].minor.yy812, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0)); } - yymsp[-2].minor.yy812 = yylhsminor.yy812; + case 368: /* dnode_list ::= dnode_list DNODE NK_INTEGER */ yytestcase(yyruleno==368); +#line 272 "sql.y" +{ yylhsminor.yy56 = addNodeToList(pCxt, yymsp[-2].minor.yy56, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0)); } +#line 5558 "sql.c" + yymsp[-2].minor.yy56 = yylhsminor.yy56; break; case 137: /* variable_list ::= NK_VARIABLE */ -{ yylhsminor.yy812 = createNodeList(pCxt, createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } - yymsp[0].minor.yy812 = yylhsminor.yy812; +#line 276 "sql.y" +{ yylhsminor.yy56 = createNodeList(pCxt, createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } +#line 5564 "sql.c" + yymsp[0].minor.yy56 = yylhsminor.yy56; break; case 138: /* variable_list ::= variable_list NK_COMMA NK_VARIABLE */ -{ yylhsminor.yy812 = addNodeToList(pCxt, yymsp[-2].minor.yy812, createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } - yymsp[-2].minor.yy812 = yylhsminor.yy812; +#line 277 "sql.y" +{ yylhsminor.yy56 = addNodeToList(pCxt, yymsp[-2].minor.yy56, createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } +#line 5570 "sql.c" + yymsp[-2].minor.yy56 = yylhsminor.yy56; break; case 139: /* retention_list ::= retention */ case 169: /* multi_create_clause ::= create_subtable_clause */ yytestcase(yyruleno==169); case 172: /* multi_drop_clause ::= drop_table_clause */ yytestcase(yyruleno==172); case 179: /* column_def_list ::= column_def */ yytestcase(yyruleno==179); - case 223: /* rollup_func_list ::= rollup_func_name */ yytestcase(yyruleno==223); - case 228: /* col_name_list ::= col_name */ yytestcase(yyruleno==228); - case 282: /* tag_list_opt ::= tag_item */ yytestcase(yyruleno==282); - case 296: /* func_list ::= func */ yytestcase(yyruleno==296); - case 396: /* literal_list ::= signed_literal */ yytestcase(yyruleno==396); - case 463: /* other_para_list ::= star_func_para */ yytestcase(yyruleno==463); - case 469: /* when_then_list ::= when_then_expr */ yytestcase(yyruleno==469); - case 524: /* select_list ::= select_item */ yytestcase(yyruleno==524); - case 535: /* partition_list ::= partition_item */ yytestcase(yyruleno==535); - case 591: /* sort_specification_list ::= sort_specification */ yytestcase(yyruleno==591); -{ yylhsminor.yy812 = createNodeList(pCxt, yymsp[0].minor.yy452); } - yymsp[0].minor.yy812 = yylhsminor.yy812; + case 224: /* rollup_func_list ::= rollup_func_name */ yytestcase(yyruleno==224); + case 229: /* col_name_list ::= col_name */ yytestcase(yyruleno==229); + case 283: /* tag_list_opt ::= tag_item */ yytestcase(yyruleno==283); + case 297: /* func_list ::= func */ yytestcase(yyruleno==297); + case 397: /* literal_list ::= signed_literal */ yytestcase(yyruleno==397); + case 464: /* other_para_list ::= star_func_para */ yytestcase(yyruleno==464); + case 470: /* when_then_list ::= when_then_expr */ yytestcase(yyruleno==470); + case 527: /* select_list ::= select_item */ yytestcase(yyruleno==527); + case 538: /* partition_list ::= partition_item */ yytestcase(yyruleno==538); + case 594: /* sort_specification_list ::= sort_specification */ yytestcase(yyruleno==594); +#line 281 "sql.y" +{ yylhsminor.yy56 = createNodeList(pCxt, yymsp[0].minor.yy840); } +#line 5589 "sql.c" + yymsp[0].minor.yy56 = yylhsminor.yy56; break; case 140: /* retention_list ::= retention_list NK_COMMA retention */ case 173: /* multi_drop_clause ::= multi_drop_clause NK_COMMA drop_table_clause */ yytestcase(yyruleno==173); case 180: /* column_def_list ::= column_def_list NK_COMMA column_def */ yytestcase(yyruleno==180); - case 224: /* rollup_func_list ::= rollup_func_list NK_COMMA rollup_func_name */ yytestcase(yyruleno==224); - case 229: /* col_name_list ::= col_name_list NK_COMMA col_name */ yytestcase(yyruleno==229); - case 283: /* tag_list_opt ::= tag_list_opt NK_COMMA tag_item */ yytestcase(yyruleno==283); - case 297: /* func_list ::= func_list NK_COMMA func */ yytestcase(yyruleno==297); - case 397: /* literal_list ::= literal_list NK_COMMA signed_literal */ yytestcase(yyruleno==397); - case 464: /* other_para_list ::= other_para_list NK_COMMA star_func_para */ yytestcase(yyruleno==464); - case 525: /* select_list ::= select_list NK_COMMA select_item */ yytestcase(yyruleno==525); - case 536: /* partition_list ::= partition_list NK_COMMA partition_item */ yytestcase(yyruleno==536); - case 592: /* sort_specification_list ::= sort_specification_list NK_COMMA sort_specification */ yytestcase(yyruleno==592); -{ yylhsminor.yy812 = addNodeToList(pCxt, yymsp[-2].minor.yy812, yymsp[0].minor.yy452); } - yymsp[-2].minor.yy812 = yylhsminor.yy812; + case 225: /* rollup_func_list ::= rollup_func_list NK_COMMA rollup_func_name */ yytestcase(yyruleno==225); + case 230: /* col_name_list ::= col_name_list NK_COMMA col_name */ yytestcase(yyruleno==230); + case 284: /* tag_list_opt ::= tag_list_opt NK_COMMA tag_item */ yytestcase(yyruleno==284); + case 298: /* func_list ::= func_list NK_COMMA func */ yytestcase(yyruleno==298); + case 398: /* literal_list ::= literal_list NK_COMMA signed_literal */ yytestcase(yyruleno==398); + case 465: /* other_para_list ::= other_para_list NK_COMMA star_func_para */ yytestcase(yyruleno==465); + case 528: /* select_list ::= select_list NK_COMMA select_item */ yytestcase(yyruleno==528); + case 539: /* partition_list ::= partition_list NK_COMMA partition_item */ yytestcase(yyruleno==539); + case 595: /* sort_specification_list ::= sort_specification_list NK_COMMA sort_specification */ yytestcase(yyruleno==595); +#line 282 "sql.y" +{ yylhsminor.yy56 = addNodeToList(pCxt, yymsp[-2].minor.yy56, yymsp[0].minor.yy840); } +#line 5606 "sql.c" + yymsp[-2].minor.yy56 = yylhsminor.yy56; break; case 141: /* retention ::= NK_VARIABLE NK_COLON NK_VARIABLE */ -{ yylhsminor.yy452 = createNodeListNodeEx(pCxt, createDurationValueNode(pCxt, &yymsp[-2].minor.yy0), createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 284 "sql.y" +{ yylhsminor.yy840 = createNodeListNodeEx(pCxt, createDurationValueNode(pCxt, &yymsp[-2].minor.yy0), createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } +#line 5612 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; case 142: /* speed_opt ::= */ - case 330: /* bufsize_opt ::= */ yytestcase(yyruleno==330); -{ yymsp[1].minor.yy416 = 0; } + case 331: /* bufsize_opt ::= */ yytestcase(yyruleno==331); +#line 288 "sql.y" +{ yymsp[1].minor.yy356 = 0; } +#line 5619 "sql.c" break; case 143: /* speed_opt ::= MAX_SPEED NK_INTEGER */ - case 331: /* bufsize_opt ::= BUFSIZE NK_INTEGER */ yytestcase(yyruleno==331); -{ yymsp[-1].minor.yy416 = taosStr2Int32(yymsp[0].minor.yy0.z, NULL, 10); } + case 332: /* bufsize_opt ::= BUFSIZE NK_INTEGER */ yytestcase(yyruleno==332); +#line 289 "sql.y" +{ yymsp[-1].minor.yy356 = taosStr2Int32(yymsp[0].minor.yy0.z, NULL, 10); } +#line 5625 "sql.c" break; case 145: /* start_opt ::= START WITH NK_INTEGER */ case 149: /* end_opt ::= END WITH NK_INTEGER */ yytestcase(yyruleno==149); -{ yymsp[-2].minor.yy452 = createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0); } +#line 292 "sql.y" +{ yymsp[-2].minor.yy840 = createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0); } +#line 5631 "sql.c" break; case 146: /* start_opt ::= START WITH NK_STRING */ case 150: /* end_opt ::= END WITH NK_STRING */ yytestcase(yyruleno==150); -{ yymsp[-2].minor.yy452 = createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0); } +#line 293 "sql.y" +{ yymsp[-2].minor.yy840 = createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0); } +#line 5637 "sql.c" break; case 147: /* start_opt ::= START WITH TIMESTAMP NK_STRING */ case 151: /* end_opt ::= END WITH TIMESTAMP NK_STRING */ yytestcase(yyruleno==151); -{ yymsp[-3].minor.yy452 = createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0); } +#line 294 "sql.y" +{ yymsp[-3].minor.yy840 = createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0); } +#line 5643 "sql.c" break; case 152: /* cmd ::= CREATE TABLE not_exists_opt full_table_name NK_LP column_def_list NK_RP tags_def_opt table_options */ case 154: /* cmd ::= CREATE STABLE not_exists_opt full_table_name NK_LP column_def_list NK_RP tags_def table_options */ yytestcase(yyruleno==154); -{ pCxt->pRootNode = createCreateTableStmt(pCxt, yymsp[-6].minor.yy667, yymsp[-5].minor.yy452, yymsp[-3].minor.yy812, yymsp[-1].minor.yy812, yymsp[0].minor.yy452); } +#line 303 "sql.y" +{ pCxt->pRootNode = createCreateTableStmt(pCxt, yymsp[-6].minor.yy313, yymsp[-5].minor.yy840, yymsp[-3].minor.yy56, yymsp[-1].minor.yy56, yymsp[0].minor.yy840); } +#line 5649 "sql.c" break; case 153: /* cmd ::= CREATE TABLE multi_create_clause */ -{ pCxt->pRootNode = createCreateMultiTableStmt(pCxt, yymsp[0].minor.yy812); } +#line 304 "sql.y" +{ pCxt->pRootNode = createCreateMultiTableStmt(pCxt, yymsp[0].minor.yy56); } +#line 5654 "sql.c" break; case 155: /* cmd ::= DROP TABLE multi_drop_clause */ -{ pCxt->pRootNode = createDropTableStmt(pCxt, yymsp[0].minor.yy812); } +#line 307 "sql.y" +{ pCxt->pRootNode = createDropTableStmt(pCxt, yymsp[0].minor.yy56); } +#line 5659 "sql.c" break; case 156: /* cmd ::= DROP STABLE exists_opt full_table_name */ -{ pCxt->pRootNode = createDropSuperTableStmt(pCxt, yymsp[-1].minor.yy667, yymsp[0].minor.yy452); } +#line 308 "sql.y" +{ pCxt->pRootNode = createDropSuperTableStmt(pCxt, yymsp[-1].minor.yy313, yymsp[0].minor.yy840); } +#line 5664 "sql.c" break; case 157: /* cmd ::= ALTER TABLE alter_table_clause */ - case 369: /* cmd ::= query_or_subquery */ yytestcase(yyruleno==369); - case 370: /* cmd ::= insert_query */ yytestcase(yyruleno==370); -{ pCxt->pRootNode = yymsp[0].minor.yy452; } + case 370: /* cmd ::= query_or_subquery */ yytestcase(yyruleno==370); + case 371: /* cmd ::= insert_query */ yytestcase(yyruleno==371); +#line 310 "sql.y" +{ pCxt->pRootNode = yymsp[0].minor.yy840; } +#line 5671 "sql.c" break; case 158: /* cmd ::= ALTER STABLE alter_table_clause */ -{ pCxt->pRootNode = setAlterSuperTableType(yymsp[0].minor.yy452); } +#line 311 "sql.y" +{ pCxt->pRootNode = setAlterSuperTableType(yymsp[0].minor.yy840); } +#line 5676 "sql.c" break; case 159: /* alter_table_clause ::= full_table_name alter_table_options */ -{ yylhsminor.yy452 = createAlterTableModifyOptions(pCxt, yymsp[-1].minor.yy452, yymsp[0].minor.yy452); } - yymsp[-1].minor.yy452 = yylhsminor.yy452; +#line 313 "sql.y" +{ yylhsminor.yy840 = createAlterTableModifyOptions(pCxt, yymsp[-1].minor.yy840, yymsp[0].minor.yy840); } +#line 5681 "sql.c" + yymsp[-1].minor.yy840 = yylhsminor.yy840; break; - case 160: /* alter_table_clause ::= full_table_name ADD COLUMN column_name type_name */ -{ yylhsminor.yy452 = createAlterTableAddModifyCol(pCxt, yymsp[-4].minor.yy452, TSDB_ALTER_TABLE_ADD_COLUMN, &yymsp[-1].minor.yy371, yymsp[0].minor.yy310); } - yymsp[-4].minor.yy452 = yylhsminor.yy452; + case 160: /* alter_table_clause ::= full_table_name ADD COLUMN column_def */ +#line 315 "sql.y" +{ yylhsminor.yy840 = createAlterTableAddModifyCol(pCxt, yymsp[-3].minor.yy840, TSDB_ALTER_TABLE_ADD_COLUMN, yymsp[0].minor.yy840); } +#line 5687 "sql.c" + yymsp[-3].minor.yy840 = yylhsminor.yy840; break; case 161: /* alter_table_clause ::= full_table_name DROP COLUMN column_name */ -{ yylhsminor.yy452 = createAlterTableDropCol(pCxt, yymsp[-3].minor.yy452, TSDB_ALTER_TABLE_DROP_COLUMN, &yymsp[0].minor.yy371); } - yymsp[-3].minor.yy452 = yylhsminor.yy452; +#line 316 "sql.y" +{ yylhsminor.yy840 = createAlterTableDropCol(pCxt, yymsp[-3].minor.yy840, TSDB_ALTER_TABLE_DROP_COLUMN, &yymsp[0].minor.yy129); } +#line 5693 "sql.c" + yymsp[-3].minor.yy840 = yylhsminor.yy840; break; - case 162: /* alter_table_clause ::= full_table_name MODIFY COLUMN column_name type_name */ -{ yylhsminor.yy452 = createAlterTableAddModifyCol(pCxt, yymsp[-4].minor.yy452, TSDB_ALTER_TABLE_UPDATE_COLUMN_BYTES, &yymsp[-1].minor.yy371, yymsp[0].minor.yy310); } - yymsp[-4].minor.yy452 = yylhsminor.yy452; + case 162: /* alter_table_clause ::= full_table_name MODIFY COLUMN column_def */ +#line 318 "sql.y" +{ yylhsminor.yy840 = createAlterTableAddModifyCol(pCxt, yymsp[-3].minor.yy840, TSDB_ALTER_TABLE_UPDATE_COLUMN_BYTES, yymsp[0].minor.yy840); } +#line 5699 "sql.c" + yymsp[-3].minor.yy840 = yylhsminor.yy840; break; case 163: /* alter_table_clause ::= full_table_name RENAME COLUMN column_name column_name */ -{ yylhsminor.yy452 = createAlterTableRenameCol(pCxt, yymsp[-4].minor.yy452, TSDB_ALTER_TABLE_UPDATE_COLUMN_NAME, &yymsp[-1].minor.yy371, &yymsp[0].minor.yy371); } - yymsp[-4].minor.yy452 = yylhsminor.yy452; +#line 320 "sql.y" +{ yylhsminor.yy840 = createAlterTableRenameCol(pCxt, yymsp[-4].minor.yy840, TSDB_ALTER_TABLE_UPDATE_COLUMN_NAME, &yymsp[-1].minor.yy129, &yymsp[0].minor.yy129); } +#line 5705 "sql.c" + yymsp[-4].minor.yy840 = yylhsminor.yy840; break; - case 164: /* alter_table_clause ::= full_table_name ADD TAG column_name type_name */ -{ yylhsminor.yy452 = createAlterTableAddModifyCol(pCxt, yymsp[-4].minor.yy452, TSDB_ALTER_TABLE_ADD_TAG, &yymsp[-1].minor.yy371, yymsp[0].minor.yy310); } - yymsp[-4].minor.yy452 = yylhsminor.yy452; + case 164: /* alter_table_clause ::= full_table_name ADD TAG column_def */ +#line 322 "sql.y" +{ yylhsminor.yy840 = createAlterTableAddModifyCol(pCxt, yymsp[-3].minor.yy840, TSDB_ALTER_TABLE_ADD_TAG, yymsp[0].minor.yy840); } +#line 5711 "sql.c" + yymsp[-3].minor.yy840 = yylhsminor.yy840; break; case 165: /* alter_table_clause ::= full_table_name DROP TAG column_name */ -{ yylhsminor.yy452 = createAlterTableDropCol(pCxt, yymsp[-3].minor.yy452, TSDB_ALTER_TABLE_DROP_TAG, &yymsp[0].minor.yy371); } - yymsp[-3].minor.yy452 = yylhsminor.yy452; +#line 323 "sql.y" +{ yylhsminor.yy840 = createAlterTableDropCol(pCxt, yymsp[-3].minor.yy840, TSDB_ALTER_TABLE_DROP_TAG, &yymsp[0].minor.yy129); } +#line 5717 "sql.c" + yymsp[-3].minor.yy840 = yylhsminor.yy840; break; - case 166: /* alter_table_clause ::= full_table_name MODIFY TAG column_name type_name */ -{ yylhsminor.yy452 = createAlterTableAddModifyCol(pCxt, yymsp[-4].minor.yy452, TSDB_ALTER_TABLE_UPDATE_TAG_BYTES, &yymsp[-1].minor.yy371, yymsp[0].minor.yy310); } - yymsp[-4].minor.yy452 = yylhsminor.yy452; + case 166: /* alter_table_clause ::= full_table_name MODIFY TAG column_def */ +#line 325 "sql.y" +{ yylhsminor.yy840 = createAlterTableAddModifyCol(pCxt, yymsp[-3].minor.yy840, TSDB_ALTER_TABLE_UPDATE_TAG_BYTES, yymsp[0].minor.yy840); } +#line 5723 "sql.c" + yymsp[-3].minor.yy840 = yylhsminor.yy840; break; case 167: /* alter_table_clause ::= full_table_name RENAME TAG column_name column_name */ -{ yylhsminor.yy452 = createAlterTableRenameCol(pCxt, yymsp[-4].minor.yy452, TSDB_ALTER_TABLE_UPDATE_TAG_NAME, &yymsp[-1].minor.yy371, &yymsp[0].minor.yy371); } - yymsp[-4].minor.yy452 = yylhsminor.yy452; +#line 327 "sql.y" +{ yylhsminor.yy840 = createAlterTableRenameCol(pCxt, yymsp[-4].minor.yy840, TSDB_ALTER_TABLE_UPDATE_TAG_NAME, &yymsp[-1].minor.yy129, &yymsp[0].minor.yy129); } +#line 5729 "sql.c" + yymsp[-4].minor.yy840 = yylhsminor.yy840; break; case 168: /* alter_table_clause ::= full_table_name SET TAG column_name NK_EQ signed_literal */ -{ yylhsminor.yy452 = createAlterTableSetTag(pCxt, yymsp[-5].minor.yy452, &yymsp[-2].minor.yy371, yymsp[0].minor.yy452); } - yymsp[-5].minor.yy452 = yylhsminor.yy452; +#line 329 "sql.y" +{ yylhsminor.yy840 = createAlterTableSetTag(pCxt, yymsp[-5].minor.yy840, &yymsp[-2].minor.yy129, yymsp[0].minor.yy840); } +#line 5735 "sql.c" + yymsp[-5].minor.yy840 = yylhsminor.yy840; break; case 170: /* multi_create_clause ::= multi_create_clause create_subtable_clause */ - case 470: /* when_then_list ::= when_then_list when_then_expr */ yytestcase(yyruleno==470); -{ yylhsminor.yy812 = addNodeToList(pCxt, yymsp[-1].minor.yy812, yymsp[0].minor.yy452); } - yymsp[-1].minor.yy812 = yylhsminor.yy812; + case 471: /* when_then_list ::= when_then_list when_then_expr */ yytestcase(yyruleno==471); +#line 334 "sql.y" +{ yylhsminor.yy56 = addNodeToList(pCxt, yymsp[-1].minor.yy56, yymsp[0].minor.yy840); } +#line 5742 "sql.c" + yymsp[-1].minor.yy56 = yylhsminor.yy56; break; case 171: /* create_subtable_clause ::= not_exists_opt full_table_name USING full_table_name specific_cols_opt TAGS NK_LP expression_list NK_RP table_options */ -{ yylhsminor.yy452 = createCreateSubTableClause(pCxt, yymsp[-9].minor.yy667, yymsp[-8].minor.yy452, yymsp[-6].minor.yy452, yymsp[-5].minor.yy812, yymsp[-2].minor.yy812, yymsp[0].minor.yy452); } - yymsp[-9].minor.yy452 = yylhsminor.yy452; +#line 338 "sql.y" +{ yylhsminor.yy840 = createCreateSubTableClause(pCxt, yymsp[-9].minor.yy313, yymsp[-8].minor.yy840, yymsp[-6].minor.yy840, yymsp[-5].minor.yy56, yymsp[-2].minor.yy56, yymsp[0].minor.yy840); } +#line 5748 "sql.c" + yymsp[-9].minor.yy840 = yylhsminor.yy840; break; case 174: /* drop_table_clause ::= exists_opt full_table_name */ -{ yylhsminor.yy452 = createDropTableClause(pCxt, yymsp[-1].minor.yy667, yymsp[0].minor.yy452); } - yymsp[-1].minor.yy452 = yylhsminor.yy452; +#line 345 "sql.y" +{ yylhsminor.yy840 = createDropTableClause(pCxt, yymsp[-1].minor.yy313, yymsp[0].minor.yy840); } +#line 5754 "sql.c" + yymsp[-1].minor.yy840 = yylhsminor.yy840; break; case 175: /* specific_cols_opt ::= */ - case 206: /* tags_def_opt ::= */ yytestcase(yyruleno==206); - case 281: /* tag_list_opt ::= */ yytestcase(yyruleno==281); - case 340: /* col_list_opt ::= */ yytestcase(yyruleno==340); - case 342: /* tag_def_or_ref_opt ::= */ yytestcase(yyruleno==342); - case 533: /* partition_by_clause_opt ::= */ yytestcase(yyruleno==533); - case 558: /* group_by_clause_opt ::= */ yytestcase(yyruleno==558); - case 578: /* order_by_clause_opt ::= */ yytestcase(yyruleno==578); -{ yymsp[1].minor.yy812 = NULL; } + case 207: /* tags_def_opt ::= */ yytestcase(yyruleno==207); + case 282: /* tag_list_opt ::= */ yytestcase(yyruleno==282); + case 341: /* col_list_opt ::= */ yytestcase(yyruleno==341); + case 343: /* tag_def_or_ref_opt ::= */ yytestcase(yyruleno==343); + case 536: /* partition_by_clause_opt ::= */ yytestcase(yyruleno==536); + case 561: /* group_by_clause_opt ::= */ yytestcase(yyruleno==561); + case 581: /* order_by_clause_opt ::= */ yytestcase(yyruleno==581); +#line 349 "sql.y" +{ yymsp[1].minor.yy56 = NULL; } +#line 5767 "sql.c" break; case 176: /* specific_cols_opt ::= NK_LP col_name_list NK_RP */ - case 341: /* col_list_opt ::= NK_LP col_name_list NK_RP */ yytestcase(yyruleno==341); -{ yymsp[-2].minor.yy812 = yymsp[-1].minor.yy812; } + case 342: /* col_list_opt ::= NK_LP col_name_list NK_RP */ yytestcase(yyruleno==342); +#line 350 "sql.y" +{ yymsp[-2].minor.yy56 = yymsp[-1].minor.yy56; } +#line 5773 "sql.c" break; case 177: /* full_table_name ::= table_name */ -{ yylhsminor.yy452 = createRealTableNode(pCxt, NULL, &yymsp[0].minor.yy371, NULL); } - yymsp[0].minor.yy452 = yylhsminor.yy452; +#line 352 "sql.y" +{ yylhsminor.yy840 = createRealTableNode(pCxt, NULL, &yymsp[0].minor.yy129, NULL); } +#line 5778 "sql.c" + yymsp[0].minor.yy840 = yylhsminor.yy840; break; case 178: /* full_table_name ::= db_name NK_DOT table_name */ -{ yylhsminor.yy452 = createRealTableNode(pCxt, &yymsp[-2].minor.yy371, &yymsp[0].minor.yy371, NULL); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 353 "sql.y" +{ yylhsminor.yy840 = createRealTableNode(pCxt, &yymsp[-2].minor.yy129, &yymsp[0].minor.yy129, NULL); } +#line 5784 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; case 181: /* column_def ::= column_name type_name */ -{ yylhsminor.yy452 = createColumnDefNode(pCxt, &yymsp[-1].minor.yy371, yymsp[0].minor.yy310, NULL); } - yymsp[-1].minor.yy452 = yylhsminor.yy452; +#line 360 "sql.y" +{ yylhsminor.yy840 = createColumnDefNode(pCxt, &yymsp[-1].minor.yy129, yymsp[0].minor.yy784, NULL); } +#line 5790 "sql.c" + yymsp[-1].minor.yy840 = yylhsminor.yy840; break; - case 182: /* type_name ::= BOOL */ -{ yymsp[0].minor.yy310 = createDataType(TSDB_DATA_TYPE_BOOL); } + case 182: /* column_def ::= column_name type_name COMMENT NK_STRING */ +#line 361 "sql.y" +{ yylhsminor.yy840 = createColumnDefNode(pCxt, &yymsp[-3].minor.yy129, yymsp[-2].minor.yy784, &yymsp[0].minor.yy0); } +#line 5796 "sql.c" + yymsp[-3].minor.yy840 = yylhsminor.yy840; break; - case 183: /* type_name ::= TINYINT */ -{ yymsp[0].minor.yy310 = createDataType(TSDB_DATA_TYPE_TINYINT); } + case 183: /* type_name ::= BOOL */ +#line 365 "sql.y" +{ yymsp[0].minor.yy784 = createDataType(TSDB_DATA_TYPE_BOOL); } +#line 5802 "sql.c" break; - case 184: /* type_name ::= SMALLINT */ -{ yymsp[0].minor.yy310 = createDataType(TSDB_DATA_TYPE_SMALLINT); } + case 184: /* type_name ::= TINYINT */ +#line 366 "sql.y" +{ yymsp[0].minor.yy784 = createDataType(TSDB_DATA_TYPE_TINYINT); } +#line 5807 "sql.c" break; - case 185: /* type_name ::= INT */ - case 186: /* type_name ::= INTEGER */ yytestcase(yyruleno==186); -{ yymsp[0].minor.yy310 = createDataType(TSDB_DATA_TYPE_INT); } + case 185: /* type_name ::= SMALLINT */ +#line 367 "sql.y" +{ yymsp[0].minor.yy784 = createDataType(TSDB_DATA_TYPE_SMALLINT); } +#line 5812 "sql.c" break; - case 187: /* type_name ::= BIGINT */ -{ yymsp[0].minor.yy310 = createDataType(TSDB_DATA_TYPE_BIGINT); } + case 186: /* type_name ::= INT */ + case 187: /* type_name ::= INTEGER */ yytestcase(yyruleno==187); +#line 368 "sql.y" +{ yymsp[0].minor.yy784 = createDataType(TSDB_DATA_TYPE_INT); } +#line 5818 "sql.c" break; - case 188: /* type_name ::= FLOAT */ -{ yymsp[0].minor.yy310 = createDataType(TSDB_DATA_TYPE_FLOAT); } + case 188: /* type_name ::= BIGINT */ +#line 370 "sql.y" +{ yymsp[0].minor.yy784 = createDataType(TSDB_DATA_TYPE_BIGINT); } +#line 5823 "sql.c" break; - case 189: /* type_name ::= DOUBLE */ -{ yymsp[0].minor.yy310 = createDataType(TSDB_DATA_TYPE_DOUBLE); } + case 189: /* type_name ::= FLOAT */ +#line 371 "sql.y" +{ yymsp[0].minor.yy784 = createDataType(TSDB_DATA_TYPE_FLOAT); } +#line 5828 "sql.c" break; - case 190: /* type_name ::= BINARY NK_LP NK_INTEGER NK_RP */ -{ yymsp[-3].minor.yy310 = createVarLenDataType(TSDB_DATA_TYPE_BINARY, &yymsp[-1].minor.yy0); } + case 190: /* type_name ::= DOUBLE */ +#line 372 "sql.y" +{ yymsp[0].minor.yy784 = createDataType(TSDB_DATA_TYPE_DOUBLE); } +#line 5833 "sql.c" break; - case 191: /* type_name ::= TIMESTAMP */ -{ yymsp[0].minor.yy310 = createDataType(TSDB_DATA_TYPE_TIMESTAMP); } + case 191: /* type_name ::= BINARY NK_LP NK_INTEGER NK_RP */ +#line 373 "sql.y" +{ yymsp[-3].minor.yy784 = createVarLenDataType(TSDB_DATA_TYPE_BINARY, &yymsp[-1].minor.yy0); } +#line 5838 "sql.c" break; - case 192: /* type_name ::= NCHAR NK_LP NK_INTEGER NK_RP */ -{ yymsp[-3].minor.yy310 = createVarLenDataType(TSDB_DATA_TYPE_NCHAR, &yymsp[-1].minor.yy0); } + case 192: /* type_name ::= TIMESTAMP */ +#line 374 "sql.y" +{ yymsp[0].minor.yy784 = createDataType(TSDB_DATA_TYPE_TIMESTAMP); } +#line 5843 "sql.c" break; - case 193: /* type_name ::= TINYINT UNSIGNED */ -{ yymsp[-1].minor.yy310 = createDataType(TSDB_DATA_TYPE_UTINYINT); } + case 193: /* type_name ::= NCHAR NK_LP NK_INTEGER NK_RP */ +#line 375 "sql.y" +{ yymsp[-3].minor.yy784 = createVarLenDataType(TSDB_DATA_TYPE_NCHAR, &yymsp[-1].minor.yy0); } +#line 5848 "sql.c" break; - case 194: /* type_name ::= SMALLINT UNSIGNED */ -{ yymsp[-1].minor.yy310 = createDataType(TSDB_DATA_TYPE_USMALLINT); } + case 194: /* type_name ::= TINYINT UNSIGNED */ +#line 376 "sql.y" +{ yymsp[-1].minor.yy784 = createDataType(TSDB_DATA_TYPE_UTINYINT); } +#line 5853 "sql.c" break; - case 195: /* type_name ::= INT UNSIGNED */ -{ yymsp[-1].minor.yy310 = createDataType(TSDB_DATA_TYPE_UINT); } + case 195: /* type_name ::= SMALLINT UNSIGNED */ +#line 377 "sql.y" +{ yymsp[-1].minor.yy784 = createDataType(TSDB_DATA_TYPE_USMALLINT); } +#line 5858 "sql.c" break; - case 196: /* type_name ::= BIGINT UNSIGNED */ -{ yymsp[-1].minor.yy310 = createDataType(TSDB_DATA_TYPE_UBIGINT); } + case 196: /* type_name ::= INT UNSIGNED */ +#line 378 "sql.y" +{ yymsp[-1].minor.yy784 = createDataType(TSDB_DATA_TYPE_UINT); } +#line 5863 "sql.c" break; - case 197: /* type_name ::= JSON */ -{ yymsp[0].minor.yy310 = createDataType(TSDB_DATA_TYPE_JSON); } + case 197: /* type_name ::= BIGINT UNSIGNED */ +#line 379 "sql.y" +{ yymsp[-1].minor.yy784 = createDataType(TSDB_DATA_TYPE_UBIGINT); } +#line 5868 "sql.c" break; - case 198: /* type_name ::= VARCHAR NK_LP NK_INTEGER NK_RP */ -{ yymsp[-3].minor.yy310 = createVarLenDataType(TSDB_DATA_TYPE_VARCHAR, &yymsp[-1].minor.yy0); } + case 198: /* type_name ::= JSON */ +#line 380 "sql.y" +{ yymsp[0].minor.yy784 = createDataType(TSDB_DATA_TYPE_JSON); } +#line 5873 "sql.c" break; - case 199: /* type_name ::= MEDIUMBLOB */ -{ yymsp[0].minor.yy310 = createDataType(TSDB_DATA_TYPE_MEDIUMBLOB); } + case 199: /* type_name ::= VARCHAR NK_LP NK_INTEGER NK_RP */ +#line 381 "sql.y" +{ yymsp[-3].minor.yy784 = createVarLenDataType(TSDB_DATA_TYPE_VARCHAR, &yymsp[-1].minor.yy0); } +#line 5878 "sql.c" break; - case 200: /* type_name ::= BLOB */ -{ yymsp[0].minor.yy310 = createDataType(TSDB_DATA_TYPE_BLOB); } + case 200: /* type_name ::= MEDIUMBLOB */ +#line 382 "sql.y" +{ yymsp[0].minor.yy784 = createDataType(TSDB_DATA_TYPE_MEDIUMBLOB); } +#line 5883 "sql.c" break; - case 201: /* type_name ::= VARBINARY NK_LP NK_INTEGER NK_RP */ -{ yymsp[-3].minor.yy310 = createVarLenDataType(TSDB_DATA_TYPE_VARBINARY, &yymsp[-1].minor.yy0); } + case 201: /* type_name ::= BLOB */ +#line 383 "sql.y" +{ yymsp[0].minor.yy784 = createDataType(TSDB_DATA_TYPE_BLOB); } +#line 5888 "sql.c" break; - case 202: /* type_name ::= GEOMETRY NK_LP NK_INTEGER NK_RP */ -{ yymsp[-3].minor.yy310 = createVarLenDataType(TSDB_DATA_TYPE_GEOMETRY, &yymsp[-1].minor.yy0); } + case 202: /* type_name ::= VARBINARY NK_LP NK_INTEGER NK_RP */ +#line 384 "sql.y" +{ yymsp[-3].minor.yy784 = createVarLenDataType(TSDB_DATA_TYPE_VARBINARY, &yymsp[-1].minor.yy0); } +#line 5893 "sql.c" break; - case 203: /* type_name ::= DECIMAL */ -{ yymsp[0].minor.yy310 = createDataType(TSDB_DATA_TYPE_DECIMAL); } + case 203: /* type_name ::= GEOMETRY NK_LP NK_INTEGER NK_RP */ +#line 385 "sql.y" +{ yymsp[-3].minor.yy784 = createVarLenDataType(TSDB_DATA_TYPE_GEOMETRY, &yymsp[-1].minor.yy0); } +#line 5898 "sql.c" break; - case 204: /* type_name ::= DECIMAL NK_LP NK_INTEGER NK_RP */ -{ yymsp[-3].minor.yy310 = createDataType(TSDB_DATA_TYPE_DECIMAL); } + case 204: /* type_name ::= DECIMAL */ +#line 386 "sql.y" +{ yymsp[0].minor.yy784 = createDataType(TSDB_DATA_TYPE_DECIMAL); } +#line 5903 "sql.c" break; - case 205: /* type_name ::= DECIMAL NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP */ -{ yymsp[-5].minor.yy310 = createDataType(TSDB_DATA_TYPE_DECIMAL); } + case 205: /* type_name ::= DECIMAL NK_LP NK_INTEGER NK_RP */ +#line 387 "sql.y" +{ yymsp[-3].minor.yy784 = createDataType(TSDB_DATA_TYPE_DECIMAL); } +#line 5908 "sql.c" break; - case 207: /* tags_def_opt ::= tags_def */ - case 343: /* tag_def_or_ref_opt ::= tags_def */ yytestcase(yyruleno==343); - case 462: /* star_func_para_list ::= other_para_list */ yytestcase(yyruleno==462); -{ yylhsminor.yy812 = yymsp[0].minor.yy812; } - yymsp[0].minor.yy812 = yylhsminor.yy812; + case 206: /* type_name ::= DECIMAL NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP */ +#line 388 "sql.y" +{ yymsp[-5].minor.yy784 = createDataType(TSDB_DATA_TYPE_DECIMAL); } +#line 5913 "sql.c" break; - case 208: /* tags_def ::= TAGS NK_LP column_def_list NK_RP */ - case 344: /* tag_def_or_ref_opt ::= TAGS NK_LP col_name_list NK_RP */ yytestcase(yyruleno==344); -{ yymsp[-3].minor.yy812 = yymsp[-1].minor.yy812; } + case 208: /* tags_def_opt ::= tags_def */ + case 344: /* tag_def_or_ref_opt ::= tags_def */ yytestcase(yyruleno==344); + case 463: /* star_func_para_list ::= other_para_list */ yytestcase(yyruleno==463); +#line 393 "sql.y" +{ yylhsminor.yy56 = yymsp[0].minor.yy56; } +#line 5920 "sql.c" + yymsp[0].minor.yy56 = yylhsminor.yy56; break; - case 209: /* table_options ::= */ -{ yymsp[1].minor.yy452 = createDefaultTableOptions(pCxt); } + case 209: /* tags_def ::= TAGS NK_LP column_def_list NK_RP */ + case 345: /* tag_def_or_ref_opt ::= TAGS NK_LP col_name_list NK_RP */ yytestcase(yyruleno==345); +#line 397 "sql.y" +{ yymsp[-3].minor.yy56 = yymsp[-1].minor.yy56; } +#line 5927 "sql.c" break; - case 210: /* table_options ::= table_options COMMENT NK_STRING */ -{ yylhsminor.yy452 = setTableOption(pCxt, yymsp[-2].minor.yy452, TABLE_OPTION_COMMENT, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; + case 210: /* table_options ::= */ +#line 399 "sql.y" +{ yymsp[1].minor.yy840 = createDefaultTableOptions(pCxt); } +#line 5932 "sql.c" break; - case 211: /* table_options ::= table_options MAX_DELAY duration_list */ -{ yylhsminor.yy452 = setTableOption(pCxt, yymsp[-2].minor.yy452, TABLE_OPTION_MAXDELAY, yymsp[0].minor.yy812); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; + case 211: /* table_options ::= table_options COMMENT NK_STRING */ +#line 400 "sql.y" +{ yylhsminor.yy840 = setTableOption(pCxt, yymsp[-2].minor.yy840, TABLE_OPTION_COMMENT, &yymsp[0].minor.yy0); } +#line 5937 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 212: /* table_options ::= table_options WATERMARK duration_list */ -{ yylhsminor.yy452 = setTableOption(pCxt, yymsp[-2].minor.yy452, TABLE_OPTION_WATERMARK, yymsp[0].minor.yy812); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; + case 212: /* table_options ::= table_options MAX_DELAY duration_list */ +#line 401 "sql.y" +{ yylhsminor.yy840 = setTableOption(pCxt, yymsp[-2].minor.yy840, TABLE_OPTION_MAXDELAY, yymsp[0].minor.yy56); } +#line 5943 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 213: /* table_options ::= table_options ROLLUP NK_LP rollup_func_list NK_RP */ -{ yylhsminor.yy452 = setTableOption(pCxt, yymsp[-4].minor.yy452, TABLE_OPTION_ROLLUP, yymsp[-1].minor.yy812); } - yymsp[-4].minor.yy452 = yylhsminor.yy452; + case 213: /* table_options ::= table_options WATERMARK duration_list */ +#line 402 "sql.y" +{ yylhsminor.yy840 = setTableOption(pCxt, yymsp[-2].minor.yy840, TABLE_OPTION_WATERMARK, yymsp[0].minor.yy56); } +#line 5949 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 214: /* table_options ::= table_options TTL NK_INTEGER */ -{ yylhsminor.yy452 = setTableOption(pCxt, yymsp[-2].minor.yy452, TABLE_OPTION_TTL, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; + case 214: /* table_options ::= table_options ROLLUP NK_LP rollup_func_list NK_RP */ +#line 403 "sql.y" +{ yylhsminor.yy840 = setTableOption(pCxt, yymsp[-4].minor.yy840, TABLE_OPTION_ROLLUP, yymsp[-1].minor.yy56); } +#line 5955 "sql.c" + yymsp[-4].minor.yy840 = yylhsminor.yy840; break; - case 215: /* table_options ::= table_options SMA NK_LP col_name_list NK_RP */ -{ yylhsminor.yy452 = setTableOption(pCxt, yymsp[-4].minor.yy452, TABLE_OPTION_SMA, yymsp[-1].minor.yy812); } - yymsp[-4].minor.yy452 = yylhsminor.yy452; + case 215: /* table_options ::= table_options TTL NK_INTEGER */ +#line 404 "sql.y" +{ yylhsminor.yy840 = setTableOption(pCxt, yymsp[-2].minor.yy840, TABLE_OPTION_TTL, &yymsp[0].minor.yy0); } +#line 5961 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 216: /* table_options ::= table_options DELETE_MARK duration_list */ -{ yylhsminor.yy452 = setTableOption(pCxt, yymsp[-2].minor.yy452, TABLE_OPTION_DELETE_MARK, yymsp[0].minor.yy812); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; + case 216: /* table_options ::= table_options SMA NK_LP col_name_list NK_RP */ +#line 405 "sql.y" +{ yylhsminor.yy840 = setTableOption(pCxt, yymsp[-4].minor.yy840, TABLE_OPTION_SMA, yymsp[-1].minor.yy56); } +#line 5967 "sql.c" + yymsp[-4].minor.yy840 = yylhsminor.yy840; break; - case 217: /* alter_table_options ::= alter_table_option */ -{ yylhsminor.yy452 = createAlterTableOptions(pCxt); yylhsminor.yy452 = setTableOption(pCxt, yylhsminor.yy452, yymsp[0].minor.yy365.type, &yymsp[0].minor.yy365.val); } - yymsp[0].minor.yy452 = yylhsminor.yy452; + case 217: /* table_options ::= table_options DELETE_MARK duration_list */ +#line 406 "sql.y" +{ yylhsminor.yy840 = setTableOption(pCxt, yymsp[-2].minor.yy840, TABLE_OPTION_DELETE_MARK, yymsp[0].minor.yy56); } +#line 5973 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 218: /* alter_table_options ::= alter_table_options alter_table_option */ -{ yylhsminor.yy452 = setTableOption(pCxt, yymsp[-1].minor.yy452, yymsp[0].minor.yy365.type, &yymsp[0].minor.yy365.val); } - yymsp[-1].minor.yy452 = yylhsminor.yy452; + case 218: /* alter_table_options ::= alter_table_option */ +#line 408 "sql.y" +{ yylhsminor.yy840 = createAlterTableOptions(pCxt); yylhsminor.yy840 = setTableOption(pCxt, yylhsminor.yy840, yymsp[0].minor.yy461.type, &yymsp[0].minor.yy461.val); } +#line 5979 "sql.c" + yymsp[0].minor.yy840 = yylhsminor.yy840; break; - case 219: /* alter_table_option ::= COMMENT NK_STRING */ -{ yymsp[-1].minor.yy365.type = TABLE_OPTION_COMMENT; yymsp[-1].minor.yy365.val = yymsp[0].minor.yy0; } + case 219: /* alter_table_options ::= alter_table_options alter_table_option */ +#line 409 "sql.y" +{ yylhsminor.yy840 = setTableOption(pCxt, yymsp[-1].minor.yy840, yymsp[0].minor.yy461.type, &yymsp[0].minor.yy461.val); } +#line 5985 "sql.c" + yymsp[-1].minor.yy840 = yylhsminor.yy840; break; - case 220: /* alter_table_option ::= TTL NK_INTEGER */ -{ yymsp[-1].minor.yy365.type = TABLE_OPTION_TTL; yymsp[-1].minor.yy365.val = yymsp[0].minor.yy0; } + case 220: /* alter_table_option ::= COMMENT NK_STRING */ +#line 413 "sql.y" +{ yymsp[-1].minor.yy461.type = TABLE_OPTION_COMMENT; yymsp[-1].minor.yy461.val = yymsp[0].minor.yy0; } +#line 5991 "sql.c" break; - case 221: /* duration_list ::= duration_literal */ - case 426: /* expression_list ::= expr_or_subquery */ yytestcase(yyruleno==426); -{ yylhsminor.yy812 = createNodeList(pCxt, releaseRawExprNode(pCxt, yymsp[0].minor.yy452)); } - yymsp[0].minor.yy812 = yylhsminor.yy812; + case 221: /* alter_table_option ::= TTL NK_INTEGER */ +#line 414 "sql.y" +{ yymsp[-1].minor.yy461.type = TABLE_OPTION_TTL; yymsp[-1].minor.yy461.val = yymsp[0].minor.yy0; } +#line 5996 "sql.c" break; - case 222: /* duration_list ::= duration_list NK_COMMA duration_literal */ - case 427: /* expression_list ::= expression_list NK_COMMA expr_or_subquery */ yytestcase(yyruleno==427); -{ yylhsminor.yy812 = addNodeToList(pCxt, yymsp[-2].minor.yy812, releaseRawExprNode(pCxt, yymsp[0].minor.yy452)); } - yymsp[-2].minor.yy812 = yylhsminor.yy812; + case 222: /* duration_list ::= duration_literal */ + case 427: /* expression_list ::= expr_or_subquery */ yytestcase(yyruleno==427); +#line 418 "sql.y" +{ yylhsminor.yy56 = createNodeList(pCxt, releaseRawExprNode(pCxt, yymsp[0].minor.yy840)); } +#line 6002 "sql.c" + yymsp[0].minor.yy56 = yylhsminor.yy56; break; - case 225: /* rollup_func_name ::= function_name */ -{ yylhsminor.yy452 = createFunctionNode(pCxt, &yymsp[0].minor.yy371, NULL); } - yymsp[0].minor.yy452 = yylhsminor.yy452; + case 223: /* duration_list ::= duration_list NK_COMMA duration_literal */ + case 428: /* expression_list ::= expression_list NK_COMMA expr_or_subquery */ yytestcase(yyruleno==428); +#line 419 "sql.y" +{ yylhsminor.yy56 = addNodeToList(pCxt, yymsp[-2].minor.yy56, releaseRawExprNode(pCxt, yymsp[0].minor.yy840)); } +#line 6009 "sql.c" + yymsp[-2].minor.yy56 = yylhsminor.yy56; break; - case 226: /* rollup_func_name ::= FIRST */ - case 227: /* rollup_func_name ::= LAST */ yytestcase(yyruleno==227); - case 285: /* tag_item ::= QTAGS */ yytestcase(yyruleno==285); -{ yylhsminor.yy452 = createFunctionNode(pCxt, &yymsp[0].minor.yy0, NULL); } - yymsp[0].minor.yy452 = yylhsminor.yy452; + case 226: /* rollup_func_name ::= function_name */ +#line 426 "sql.y" +{ yylhsminor.yy840 = createFunctionNode(pCxt, &yymsp[0].minor.yy129, NULL); } +#line 6015 "sql.c" + yymsp[0].minor.yy840 = yylhsminor.yy840; break; - case 230: /* col_name ::= column_name */ - case 286: /* tag_item ::= column_name */ yytestcase(yyruleno==286); -{ yylhsminor.yy452 = createColumnNode(pCxt, NULL, &yymsp[0].minor.yy371); } - yymsp[0].minor.yy452 = yylhsminor.yy452; + case 227: /* rollup_func_name ::= FIRST */ + case 228: /* rollup_func_name ::= LAST */ yytestcase(yyruleno==228); + case 286: /* tag_item ::= QTAGS */ yytestcase(yyruleno==286); +#line 427 "sql.y" +{ yylhsminor.yy840 = createFunctionNode(pCxt, &yymsp[0].minor.yy0, NULL); } +#line 6023 "sql.c" + yymsp[0].minor.yy840 = yylhsminor.yy840; break; - case 231: /* cmd ::= SHOW DNODES */ + case 231: /* col_name ::= column_name */ + case 287: /* tag_item ::= column_name */ yytestcase(yyruleno==287); +#line 435 "sql.y" +{ yylhsminor.yy840 = createColumnNode(pCxt, NULL, &yymsp[0].minor.yy129); } +#line 6030 "sql.c" + yymsp[0].minor.yy840 = yylhsminor.yy840; + break; + case 232: /* cmd ::= SHOW DNODES */ +#line 438 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_DNODES_STMT); } +#line 6036 "sql.c" break; - case 232: /* cmd ::= SHOW USERS */ + case 233: /* cmd ::= SHOW USERS */ +#line 439 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_USERS_STMT); } +#line 6041 "sql.c" break; - case 233: /* cmd ::= SHOW USER PRIVILEGES */ + case 234: /* cmd ::= SHOW USER PRIVILEGES */ +#line 440 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_USER_PRIVILEGES_STMT); } +#line 6046 "sql.c" break; - case 234: /* cmd ::= SHOW DATABASES */ + case 235: /* cmd ::= SHOW DATABASES */ +#line 441 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_DATABASES_STMT); } +#line 6051 "sql.c" break; - case 235: /* cmd ::= SHOW db_name_cond_opt TABLES like_pattern_opt */ -{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_TABLES_STMT, yymsp[-2].minor.yy452, yymsp[0].minor.yy452, OP_TYPE_LIKE); } + case 236: /* cmd ::= SHOW db_name_cond_opt TABLES like_pattern_opt */ +#line 442 "sql.y" +{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_TABLES_STMT, yymsp[-2].minor.yy840, yymsp[0].minor.yy840, OP_TYPE_LIKE); } +#line 6056 "sql.c" break; - case 236: /* cmd ::= SHOW db_name_cond_opt STABLES like_pattern_opt */ -{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_STABLES_STMT, yymsp[-2].minor.yy452, yymsp[0].minor.yy452, OP_TYPE_LIKE); } + case 237: /* cmd ::= SHOW db_name_cond_opt STABLES like_pattern_opt */ +#line 443 "sql.y" +{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_STABLES_STMT, yymsp[-2].minor.yy840, yymsp[0].minor.yy840, OP_TYPE_LIKE); } +#line 6061 "sql.c" break; - case 237: /* cmd ::= SHOW db_name_cond_opt VGROUPS */ -{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_VGROUPS_STMT, yymsp[-1].minor.yy452, NULL, OP_TYPE_LIKE); } + case 238: /* cmd ::= SHOW db_name_cond_opt VGROUPS */ +#line 444 "sql.y" +{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_VGROUPS_STMT, yymsp[-1].minor.yy840, NULL, OP_TYPE_LIKE); } +#line 6066 "sql.c" break; - case 238: /* cmd ::= SHOW MNODES */ + case 239: /* cmd ::= SHOW MNODES */ +#line 445 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_MNODES_STMT); } +#line 6071 "sql.c" break; - case 239: /* cmd ::= SHOW QNODES */ + case 240: /* cmd ::= SHOW QNODES */ +#line 447 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_QNODES_STMT); } +#line 6076 "sql.c" break; - case 240: /* cmd ::= SHOW FUNCTIONS */ + case 241: /* cmd ::= SHOW FUNCTIONS */ +#line 448 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_FUNCTIONS_STMT); } +#line 6081 "sql.c" break; - case 241: /* cmd ::= SHOW INDEXES FROM table_name_cond from_db_opt */ -{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_INDEXES_STMT, yymsp[0].minor.yy452, yymsp[-1].minor.yy452, OP_TYPE_EQUAL); } + case 242: /* cmd ::= SHOW INDEXES FROM table_name_cond from_db_opt */ +#line 449 "sql.y" +{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_INDEXES_STMT, yymsp[0].minor.yy840, yymsp[-1].minor.yy840, OP_TYPE_EQUAL); } +#line 6086 "sql.c" break; - case 242: /* cmd ::= SHOW INDEXES FROM db_name NK_DOT table_name */ -{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_INDEXES_STMT, createIdentifierValueNode(pCxt, &yymsp[-2].minor.yy371), createIdentifierValueNode(pCxt, &yymsp[0].minor.yy371), OP_TYPE_EQUAL); } + case 243: /* cmd ::= SHOW INDEXES FROM db_name NK_DOT table_name */ +#line 450 "sql.y" +{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_INDEXES_STMT, createIdentifierValueNode(pCxt, &yymsp[-2].minor.yy129), createIdentifierValueNode(pCxt, &yymsp[0].minor.yy129), OP_TYPE_EQUAL); } +#line 6091 "sql.c" break; - case 243: /* cmd ::= SHOW STREAMS */ + case 244: /* cmd ::= SHOW STREAMS */ +#line 451 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_STREAMS_STMT); } +#line 6096 "sql.c" break; - case 244: /* cmd ::= SHOW ACCOUNTS */ + case 245: /* cmd ::= SHOW ACCOUNTS */ +#line 452 "sql.y" { pCxt->errCode = generateSyntaxErrMsg(&pCxt->msgBuf, TSDB_CODE_PAR_EXPRIE_STATEMENT); } +#line 6101 "sql.c" break; - case 245: /* cmd ::= SHOW APPS */ + case 246: /* cmd ::= SHOW APPS */ +#line 453 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_APPS_STMT); } +#line 6106 "sql.c" break; - case 246: /* cmd ::= SHOW CONNECTIONS */ + case 247: /* cmd ::= SHOW CONNECTIONS */ +#line 454 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_CONNECTIONS_STMT); } +#line 6111 "sql.c" break; - case 247: /* cmd ::= SHOW LICENCES */ - case 248: /* cmd ::= SHOW GRANTS */ yytestcase(yyruleno==248); + case 248: /* cmd ::= SHOW LICENCES */ + case 249: /* cmd ::= SHOW GRANTS */ yytestcase(yyruleno==249); +#line 455 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_LICENCES_STMT); } +#line 6117 "sql.c" break; - case 249: /* cmd ::= SHOW CREATE DATABASE db_name */ -{ pCxt->pRootNode = createShowCreateDatabaseStmt(pCxt, &yymsp[0].minor.yy371); } + case 250: /* cmd ::= SHOW CREATE DATABASE db_name */ +#line 457 "sql.y" +{ pCxt->pRootNode = createShowCreateDatabaseStmt(pCxt, &yymsp[0].minor.yy129); } +#line 6122 "sql.c" break; - case 250: /* cmd ::= SHOW CREATE TABLE full_table_name */ -{ pCxt->pRootNode = createShowCreateTableStmt(pCxt, QUERY_NODE_SHOW_CREATE_TABLE_STMT, yymsp[0].minor.yy452); } + case 251: /* cmd ::= SHOW CREATE TABLE full_table_name */ +#line 458 "sql.y" +{ pCxt->pRootNode = createShowCreateTableStmt(pCxt, QUERY_NODE_SHOW_CREATE_TABLE_STMT, yymsp[0].minor.yy840); } +#line 6127 "sql.c" break; - case 251: /* cmd ::= SHOW CREATE STABLE full_table_name */ -{ pCxt->pRootNode = createShowCreateTableStmt(pCxt, QUERY_NODE_SHOW_CREATE_STABLE_STMT, yymsp[0].minor.yy452); } + case 252: /* cmd ::= SHOW CREATE STABLE full_table_name */ +#line 459 "sql.y" +{ pCxt->pRootNode = createShowCreateTableStmt(pCxt, QUERY_NODE_SHOW_CREATE_STABLE_STMT, yymsp[0].minor.yy840); } +#line 6132 "sql.c" break; - case 252: /* cmd ::= SHOW QUERIES */ + case 253: /* cmd ::= SHOW QUERIES */ +#line 460 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_QUERIES_STMT); } +#line 6137 "sql.c" break; - case 253: /* cmd ::= SHOW SCORES */ + case 254: /* cmd ::= SHOW SCORES */ +#line 461 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_SCORES_STMT); } +#line 6142 "sql.c" break; - case 254: /* cmd ::= SHOW TOPICS */ + case 255: /* cmd ::= SHOW TOPICS */ +#line 462 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_TOPICS_STMT); } +#line 6147 "sql.c" break; - case 255: /* cmd ::= SHOW VARIABLES */ - case 256: /* cmd ::= SHOW CLUSTER VARIABLES */ yytestcase(yyruleno==256); + case 256: /* cmd ::= SHOW VARIABLES */ + case 257: /* cmd ::= SHOW CLUSTER VARIABLES */ yytestcase(yyruleno==257); +#line 463 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_VARIABLES_STMT); } +#line 6153 "sql.c" break; - case 257: /* cmd ::= SHOW LOCAL VARIABLES */ + case 258: /* cmd ::= SHOW LOCAL VARIABLES */ +#line 465 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_LOCAL_VARIABLES_STMT); } +#line 6158 "sql.c" break; - case 258: /* cmd ::= SHOW DNODE NK_INTEGER VARIABLES like_pattern_opt */ -{ pCxt->pRootNode = createShowDnodeVariablesStmt(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[-2].minor.yy0), yymsp[0].minor.yy452); } + case 259: /* cmd ::= SHOW DNODE NK_INTEGER VARIABLES like_pattern_opt */ +#line 466 "sql.y" +{ pCxt->pRootNode = createShowDnodeVariablesStmt(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[-2].minor.yy0), yymsp[0].minor.yy840); } +#line 6163 "sql.c" break; - case 259: /* cmd ::= SHOW BNODES */ + case 260: /* cmd ::= SHOW BNODES */ +#line 467 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_BNODES_STMT); } +#line 6168 "sql.c" break; - case 260: /* cmd ::= SHOW SNODES */ + case 261: /* cmd ::= SHOW SNODES */ +#line 468 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_SNODES_STMT); } +#line 6173 "sql.c" break; - case 261: /* cmd ::= SHOW CLUSTER */ + case 262: /* cmd ::= SHOW CLUSTER */ +#line 469 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_CLUSTER_STMT); } +#line 6178 "sql.c" break; - case 262: /* cmd ::= SHOW TRANSACTIONS */ + case 263: /* cmd ::= SHOW TRANSACTIONS */ +#line 470 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_TRANSACTIONS_STMT); } +#line 6183 "sql.c" break; - case 263: /* cmd ::= SHOW TABLE DISTRIBUTED full_table_name */ -{ pCxt->pRootNode = createShowTableDistributedStmt(pCxt, yymsp[0].minor.yy452); } + case 264: /* cmd ::= SHOW TABLE DISTRIBUTED full_table_name */ +#line 471 "sql.y" +{ pCxt->pRootNode = createShowTableDistributedStmt(pCxt, yymsp[0].minor.yy840); } +#line 6188 "sql.c" break; - case 264: /* cmd ::= SHOW CONSUMERS */ + case 265: /* cmd ::= SHOW CONSUMERS */ +#line 472 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_CONSUMERS_STMT); } +#line 6193 "sql.c" break; - case 265: /* cmd ::= SHOW SUBSCRIPTIONS */ + case 266: /* cmd ::= SHOW SUBSCRIPTIONS */ +#line 473 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_SUBSCRIPTIONS_STMT); } +#line 6198 "sql.c" break; - case 266: /* cmd ::= SHOW TAGS FROM table_name_cond from_db_opt */ -{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_TAGS_STMT, yymsp[0].minor.yy452, yymsp[-1].minor.yy452, OP_TYPE_EQUAL); } + case 267: /* cmd ::= SHOW TAGS FROM table_name_cond from_db_opt */ +#line 474 "sql.y" +{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_TAGS_STMT, yymsp[0].minor.yy840, yymsp[-1].minor.yy840, OP_TYPE_EQUAL); } +#line 6203 "sql.c" break; - case 267: /* cmd ::= SHOW TAGS FROM db_name NK_DOT table_name */ -{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_TAGS_STMT, createIdentifierValueNode(pCxt, &yymsp[-2].minor.yy371), createIdentifierValueNode(pCxt, &yymsp[0].minor.yy371), OP_TYPE_EQUAL); } + case 268: /* cmd ::= SHOW TAGS FROM db_name NK_DOT table_name */ +#line 475 "sql.y" +{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_TAGS_STMT, createIdentifierValueNode(pCxt, &yymsp[-2].minor.yy129), createIdentifierValueNode(pCxt, &yymsp[0].minor.yy129), OP_TYPE_EQUAL); } +#line 6208 "sql.c" break; - case 268: /* cmd ::= SHOW TABLE TAGS tag_list_opt FROM table_name_cond from_db_opt */ -{ pCxt->pRootNode = createShowTableTagsStmt(pCxt, yymsp[-1].minor.yy452, yymsp[0].minor.yy452, yymsp[-3].minor.yy812); } + case 269: /* cmd ::= SHOW TABLE TAGS tag_list_opt FROM table_name_cond from_db_opt */ +#line 476 "sql.y" +{ pCxt->pRootNode = createShowTableTagsStmt(pCxt, yymsp[-1].minor.yy840, yymsp[0].minor.yy840, yymsp[-3].minor.yy56); } +#line 6213 "sql.c" break; - case 269: /* cmd ::= SHOW TABLE TAGS tag_list_opt FROM db_name NK_DOT table_name */ -{ pCxt->pRootNode = createShowTableTagsStmt(pCxt, createIdentifierValueNode(pCxt, &yymsp[0].minor.yy371), createIdentifierValueNode(pCxt, &yymsp[-2].minor.yy371), yymsp[-4].minor.yy812); } + case 270: /* cmd ::= SHOW TABLE TAGS tag_list_opt FROM db_name NK_DOT table_name */ +#line 477 "sql.y" +{ pCxt->pRootNode = createShowTableTagsStmt(pCxt, createIdentifierValueNode(pCxt, &yymsp[0].minor.yy129), createIdentifierValueNode(pCxt, &yymsp[-2].minor.yy129), yymsp[-4].minor.yy56); } +#line 6218 "sql.c" break; - case 270: /* cmd ::= SHOW VNODES NK_INTEGER */ + case 271: /* cmd ::= SHOW VNODES ON DNODE NK_INTEGER */ +#line 478 "sql.y" { pCxt->pRootNode = createShowVnodesStmt(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0), NULL); } +#line 6223 "sql.c" break; - case 271: /* cmd ::= SHOW VNODES NK_STRING */ -{ pCxt->pRootNode = createShowVnodesStmt(pCxt, NULL, createValueNode(pCxt, TSDB_DATA_TYPE_VARCHAR, &yymsp[0].minor.yy0)); } + case 272: /* cmd ::= SHOW VNODES */ +#line 479 "sql.y" +{ pCxt->pRootNode = createShowVnodesStmt(pCxt, NULL, NULL); } +#line 6228 "sql.c" break; - case 272: /* cmd ::= SHOW db_name_cond_opt ALIVE */ -{ pCxt->pRootNode = createShowAliveStmt(pCxt, yymsp[-1].minor.yy452, QUERY_NODE_SHOW_DB_ALIVE_STMT); } + case 273: /* cmd ::= SHOW db_name_cond_opt ALIVE */ +#line 481 "sql.y" +{ pCxt->pRootNode = createShowAliveStmt(pCxt, yymsp[-1].minor.yy840, QUERY_NODE_SHOW_DB_ALIVE_STMT); } +#line 6233 "sql.c" break; - case 273: /* cmd ::= SHOW CLUSTER ALIVE */ + case 274: /* cmd ::= SHOW CLUSTER ALIVE */ +#line 482 "sql.y" { pCxt->pRootNode = createShowAliveStmt(pCxt, NULL, QUERY_NODE_SHOW_CLUSTER_ALIVE_STMT); } +#line 6238 "sql.c" break; - case 274: /* db_name_cond_opt ::= */ - case 279: /* from_db_opt ::= */ yytestcase(yyruleno==279); -{ yymsp[1].minor.yy452 = createDefaultDatabaseCondValue(pCxt); } + case 275: /* db_name_cond_opt ::= */ + case 280: /* from_db_opt ::= */ yytestcase(yyruleno==280); +#line 484 "sql.y" +{ yymsp[1].minor.yy840 = createDefaultDatabaseCondValue(pCxt); } +#line 6244 "sql.c" break; - case 275: /* db_name_cond_opt ::= db_name NK_DOT */ -{ yylhsminor.yy452 = createIdentifierValueNode(pCxt, &yymsp[-1].minor.yy371); } - yymsp[-1].minor.yy452 = yylhsminor.yy452; + case 276: /* db_name_cond_opt ::= db_name NK_DOT */ +#line 485 "sql.y" +{ yylhsminor.yy840 = createIdentifierValueNode(pCxt, &yymsp[-1].minor.yy129); } +#line 6249 "sql.c" + yymsp[-1].minor.yy840 = yylhsminor.yy840; break; - case 277: /* like_pattern_opt ::= LIKE NK_STRING */ -{ yymsp[-1].minor.yy452 = createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0); } + case 278: /* like_pattern_opt ::= LIKE NK_STRING */ +#line 488 "sql.y" +{ yymsp[-1].minor.yy840 = createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0); } +#line 6255 "sql.c" break; - case 278: /* table_name_cond ::= table_name */ -{ yylhsminor.yy452 = createIdentifierValueNode(pCxt, &yymsp[0].minor.yy371); } - yymsp[0].minor.yy452 = yylhsminor.yy452; + case 279: /* table_name_cond ::= table_name */ +#line 490 "sql.y" +{ yylhsminor.yy840 = createIdentifierValueNode(pCxt, &yymsp[0].minor.yy129); } +#line 6260 "sql.c" + yymsp[0].minor.yy840 = yylhsminor.yy840; break; - case 280: /* from_db_opt ::= FROM db_name */ -{ yymsp[-1].minor.yy452 = createIdentifierValueNode(pCxt, &yymsp[0].minor.yy371); } + case 281: /* from_db_opt ::= FROM db_name */ +#line 493 "sql.y" +{ yymsp[-1].minor.yy840 = createIdentifierValueNode(pCxt, &yymsp[0].minor.yy129); } +#line 6266 "sql.c" break; - case 284: /* tag_item ::= TBNAME */ -{ yylhsminor.yy452 = setProjectionAlias(pCxt, createFunctionNode(pCxt, &yymsp[0].minor.yy0, NULL), &yymsp[0].minor.yy0); } - yymsp[0].minor.yy452 = yylhsminor.yy452; + case 285: /* tag_item ::= TBNAME */ +#line 501 "sql.y" +{ yylhsminor.yy840 = setProjectionAlias(pCxt, createFunctionNode(pCxt, &yymsp[0].minor.yy0, NULL), &yymsp[0].minor.yy0); } +#line 6271 "sql.c" + yymsp[0].minor.yy840 = yylhsminor.yy840; break; - case 287: /* tag_item ::= column_name column_alias */ -{ yylhsminor.yy452 = setProjectionAlias(pCxt, createColumnNode(pCxt, NULL, &yymsp[-1].minor.yy371), &yymsp[0].minor.yy371); } - yymsp[-1].minor.yy452 = yylhsminor.yy452; + case 288: /* tag_item ::= column_name column_alias */ +#line 504 "sql.y" +{ yylhsminor.yy840 = setProjectionAlias(pCxt, createColumnNode(pCxt, NULL, &yymsp[-1].minor.yy129), &yymsp[0].minor.yy129); } +#line 6277 "sql.c" + yymsp[-1].minor.yy840 = yylhsminor.yy840; break; - case 288: /* tag_item ::= column_name AS column_alias */ -{ yylhsminor.yy452 = setProjectionAlias(pCxt, createColumnNode(pCxt, NULL, &yymsp[-2].minor.yy371), &yymsp[0].minor.yy371); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; + case 289: /* tag_item ::= column_name AS column_alias */ +#line 505 "sql.y" +{ yylhsminor.yy840 = setProjectionAlias(pCxt, createColumnNode(pCxt, NULL, &yymsp[-2].minor.yy129), &yymsp[0].minor.yy129); } +#line 6283 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 289: /* cmd ::= CREATE SMA INDEX not_exists_opt full_index_name ON full_table_name index_options */ -{ pCxt->pRootNode = createCreateIndexStmt(pCxt, INDEX_TYPE_SMA, yymsp[-4].minor.yy667, yymsp[-3].minor.yy452, yymsp[-1].minor.yy452, NULL, yymsp[0].minor.yy452); } + case 290: /* cmd ::= CREATE SMA INDEX not_exists_opt full_index_name ON full_table_name index_options */ +#line 509 "sql.y" +{ pCxt->pRootNode = createCreateIndexStmt(pCxt, INDEX_TYPE_SMA, yymsp[-4].minor.yy313, yymsp[-3].minor.yy840, yymsp[-1].minor.yy840, NULL, yymsp[0].minor.yy840); } +#line 6289 "sql.c" break; - case 290: /* cmd ::= CREATE INDEX not_exists_opt full_index_name ON full_table_name NK_LP col_name_list NK_RP */ -{ pCxt->pRootNode = createCreateIndexStmt(pCxt, INDEX_TYPE_NORMAL, yymsp[-6].minor.yy667, yymsp[-5].minor.yy452, yymsp[-3].minor.yy452, yymsp[-1].minor.yy812, NULL); } + case 291: /* cmd ::= CREATE INDEX not_exists_opt full_index_name ON full_table_name NK_LP col_name_list NK_RP */ +#line 511 "sql.y" +{ pCxt->pRootNode = createCreateIndexStmt(pCxt, INDEX_TYPE_NORMAL, yymsp[-6].minor.yy313, yymsp[-5].minor.yy840, yymsp[-3].minor.yy840, yymsp[-1].minor.yy56, NULL); } +#line 6294 "sql.c" break; - case 291: /* cmd ::= DROP INDEX exists_opt full_index_name */ -{ pCxt->pRootNode = createDropIndexStmt(pCxt, yymsp[-1].minor.yy667, yymsp[0].minor.yy452); } + case 292: /* cmd ::= DROP INDEX exists_opt full_index_name */ +#line 512 "sql.y" +{ pCxt->pRootNode = createDropIndexStmt(pCxt, yymsp[-1].minor.yy313, yymsp[0].minor.yy840); } +#line 6299 "sql.c" break; - case 292: /* full_index_name ::= index_name */ -{ yylhsminor.yy452 = createRealTableNodeForIndexName(pCxt, NULL, &yymsp[0].minor.yy371); } - yymsp[0].minor.yy452 = yylhsminor.yy452; + case 293: /* full_index_name ::= index_name */ +#line 514 "sql.y" +{ yylhsminor.yy840 = createRealTableNodeForIndexName(pCxt, NULL, &yymsp[0].minor.yy129); } +#line 6304 "sql.c" + yymsp[0].minor.yy840 = yylhsminor.yy840; break; - case 293: /* full_index_name ::= db_name NK_DOT index_name */ -{ yylhsminor.yy452 = createRealTableNodeForIndexName(pCxt, &yymsp[-2].minor.yy371, &yymsp[0].minor.yy371); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; + case 294: /* full_index_name ::= db_name NK_DOT index_name */ +#line 515 "sql.y" +{ yylhsminor.yy840 = createRealTableNodeForIndexName(pCxt, &yymsp[-2].minor.yy129, &yymsp[0].minor.yy129); } +#line 6310 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 294: /* index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_RP sliding_opt sma_stream_opt */ -{ yymsp[-9].minor.yy452 = createIndexOption(pCxt, yymsp[-7].minor.yy812, releaseRawExprNode(pCxt, yymsp[-3].minor.yy452), NULL, yymsp[-1].minor.yy452, yymsp[0].minor.yy452); } + case 295: /* index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_RP sliding_opt sma_stream_opt */ +#line 518 "sql.y" +{ yymsp[-9].minor.yy840 = createIndexOption(pCxt, yymsp[-7].minor.yy56, releaseRawExprNode(pCxt, yymsp[-3].minor.yy840), NULL, yymsp[-1].minor.yy840, yymsp[0].minor.yy840); } +#line 6316 "sql.c" break; - case 295: /* index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_COMMA duration_literal NK_RP sliding_opt sma_stream_opt */ -{ yymsp[-11].minor.yy452 = createIndexOption(pCxt, yymsp[-9].minor.yy812, releaseRawExprNode(pCxt, yymsp[-5].minor.yy452), releaseRawExprNode(pCxt, yymsp[-3].minor.yy452), yymsp[-1].minor.yy452, yymsp[0].minor.yy452); } + case 296: /* index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_COMMA duration_literal NK_RP sliding_opt sma_stream_opt */ +#line 521 "sql.y" +{ yymsp[-11].minor.yy840 = createIndexOption(pCxt, yymsp[-9].minor.yy56, releaseRawExprNode(pCxt, yymsp[-5].minor.yy840), releaseRawExprNode(pCxt, yymsp[-3].minor.yy840), yymsp[-1].minor.yy840, yymsp[0].minor.yy840); } +#line 6321 "sql.c" break; - case 298: /* func ::= sma_func_name NK_LP expression_list NK_RP */ -{ yylhsminor.yy452 = createFunctionNode(pCxt, &yymsp[-3].minor.yy371, yymsp[-1].minor.yy812); } - yymsp[-3].minor.yy452 = yylhsminor.yy452; + case 299: /* func ::= sma_func_name NK_LP expression_list NK_RP */ +#line 528 "sql.y" +{ yylhsminor.yy840 = createFunctionNode(pCxt, &yymsp[-3].minor.yy129, yymsp[-1].minor.yy56); } +#line 6326 "sql.c" + yymsp[-3].minor.yy840 = yylhsminor.yy840; break; - case 299: /* sma_func_name ::= function_name */ - case 513: /* alias_opt ::= table_alias */ yytestcase(yyruleno==513); -{ yylhsminor.yy371 = yymsp[0].minor.yy371; } - yymsp[0].minor.yy371 = yylhsminor.yy371; + case 300: /* sma_func_name ::= function_name */ + case 514: /* alias_opt ::= table_alias */ yytestcase(yyruleno==514); +#line 532 "sql.y" +{ yylhsminor.yy129 = yymsp[0].minor.yy129; } +#line 6333 "sql.c" + yymsp[0].minor.yy129 = yylhsminor.yy129; break; - case 304: /* sma_stream_opt ::= */ - case 345: /* stream_options ::= */ yytestcase(yyruleno==345); -{ yymsp[1].minor.yy452 = createStreamOptions(pCxt); } + case 305: /* sma_stream_opt ::= */ + case 346: /* stream_options ::= */ yytestcase(yyruleno==346); +#line 538 "sql.y" +{ yymsp[1].minor.yy840 = createStreamOptions(pCxt); } +#line 6340 "sql.c" break; - case 305: /* sma_stream_opt ::= sma_stream_opt WATERMARK duration_literal */ -{ ((SStreamOptions*)yymsp[-2].minor.yy452)->pWatermark = releaseRawExprNode(pCxt, yymsp[0].minor.yy452); yylhsminor.yy452 = yymsp[-2].minor.yy452; } - yymsp[-2].minor.yy452 = yylhsminor.yy452; + case 306: /* sma_stream_opt ::= sma_stream_opt WATERMARK duration_literal */ +#line 539 "sql.y" +{ ((SStreamOptions*)yymsp[-2].minor.yy840)->pWatermark = releaseRawExprNode(pCxt, yymsp[0].minor.yy840); yylhsminor.yy840 = yymsp[-2].minor.yy840; } +#line 6345 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 306: /* sma_stream_opt ::= sma_stream_opt MAX_DELAY duration_literal */ -{ ((SStreamOptions*)yymsp[-2].minor.yy452)->pDelay = releaseRawExprNode(pCxt, yymsp[0].minor.yy452); yylhsminor.yy452 = yymsp[-2].minor.yy452; } - yymsp[-2].minor.yy452 = yylhsminor.yy452; + case 307: /* sma_stream_opt ::= sma_stream_opt MAX_DELAY duration_literal */ +#line 540 "sql.y" +{ ((SStreamOptions*)yymsp[-2].minor.yy840)->pDelay = releaseRawExprNode(pCxt, yymsp[0].minor.yy840); yylhsminor.yy840 = yymsp[-2].minor.yy840; } +#line 6351 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 307: /* sma_stream_opt ::= sma_stream_opt DELETE_MARK duration_literal */ -{ ((SStreamOptions*)yymsp[-2].minor.yy452)->pDeleteMark = releaseRawExprNode(pCxt, yymsp[0].minor.yy452); yylhsminor.yy452 = yymsp[-2].minor.yy452; } - yymsp[-2].minor.yy452 = yylhsminor.yy452; + case 308: /* sma_stream_opt ::= sma_stream_opt DELETE_MARK duration_literal */ +#line 541 "sql.y" +{ ((SStreamOptions*)yymsp[-2].minor.yy840)->pDeleteMark = releaseRawExprNode(pCxt, yymsp[0].minor.yy840); yylhsminor.yy840 = yymsp[-2].minor.yy840; } +#line 6357 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 308: /* with_meta ::= AS */ -{ yymsp[0].minor.yy416 = 0; } + case 309: /* with_meta ::= AS */ +#line 546 "sql.y" +{ yymsp[0].minor.yy356 = 0; } +#line 6363 "sql.c" break; - case 309: /* with_meta ::= WITH META AS */ -{ yymsp[-2].minor.yy416 = 1; } + case 310: /* with_meta ::= WITH META AS */ +#line 547 "sql.y" +{ yymsp[-2].minor.yy356 = 1; } +#line 6368 "sql.c" break; - case 310: /* with_meta ::= ONLY META AS */ -{ yymsp[-2].minor.yy416 = 2; } + case 311: /* with_meta ::= ONLY META AS */ +#line 548 "sql.y" +{ yymsp[-2].minor.yy356 = 2; } +#line 6373 "sql.c" break; - case 311: /* cmd ::= CREATE TOPIC not_exists_opt topic_name AS query_or_subquery */ -{ pCxt->pRootNode = createCreateTopicStmtUseQuery(pCxt, yymsp[-3].minor.yy667, &yymsp[-2].minor.yy371, yymsp[0].minor.yy452); } + case 312: /* cmd ::= CREATE TOPIC not_exists_opt topic_name AS query_or_subquery */ +#line 550 "sql.y" +{ pCxt->pRootNode = createCreateTopicStmtUseQuery(pCxt, yymsp[-3].minor.yy313, &yymsp[-2].minor.yy129, yymsp[0].minor.yy840); } +#line 6378 "sql.c" break; - case 312: /* cmd ::= CREATE TOPIC not_exists_opt topic_name with_meta DATABASE db_name */ -{ pCxt->pRootNode = createCreateTopicStmtUseDb(pCxt, yymsp[-4].minor.yy667, &yymsp[-3].minor.yy371, &yymsp[0].minor.yy371, yymsp[-2].minor.yy416); } + case 313: /* cmd ::= CREATE TOPIC not_exists_opt topic_name with_meta DATABASE db_name */ +#line 552 "sql.y" +{ pCxt->pRootNode = createCreateTopicStmtUseDb(pCxt, yymsp[-4].minor.yy313, &yymsp[-3].minor.yy129, &yymsp[0].minor.yy129, yymsp[-2].minor.yy356); } +#line 6383 "sql.c" break; - case 313: /* cmd ::= CREATE TOPIC not_exists_opt topic_name with_meta STABLE full_table_name where_clause_opt */ -{ pCxt->pRootNode = createCreateTopicStmtUseTable(pCxt, yymsp[-5].minor.yy667, &yymsp[-4].minor.yy371, yymsp[-1].minor.yy452, yymsp[-3].minor.yy416, yymsp[0].minor.yy452); } + case 314: /* cmd ::= CREATE TOPIC not_exists_opt topic_name with_meta STABLE full_table_name where_clause_opt */ +#line 554 "sql.y" +{ pCxt->pRootNode = createCreateTopicStmtUseTable(pCxt, yymsp[-5].minor.yy313, &yymsp[-4].minor.yy129, yymsp[-1].minor.yy840, yymsp[-3].minor.yy356, yymsp[0].minor.yy840); } +#line 6388 "sql.c" break; - case 314: /* cmd ::= DROP TOPIC exists_opt topic_name */ -{ pCxt->pRootNode = createDropTopicStmt(pCxt, yymsp[-1].minor.yy667, &yymsp[0].minor.yy371); } + case 315: /* cmd ::= DROP TOPIC exists_opt topic_name */ +#line 556 "sql.y" +{ pCxt->pRootNode = createDropTopicStmt(pCxt, yymsp[-1].minor.yy313, &yymsp[0].minor.yy129); } +#line 6393 "sql.c" break; - case 315: /* cmd ::= DROP CONSUMER GROUP exists_opt cgroup_name ON topic_name */ -{ pCxt->pRootNode = createDropCGroupStmt(pCxt, yymsp[-3].minor.yy667, &yymsp[-2].minor.yy371, &yymsp[0].minor.yy371); } + case 316: /* cmd ::= DROP CONSUMER GROUP exists_opt cgroup_name ON topic_name */ +#line 557 "sql.y" +{ pCxt->pRootNode = createDropCGroupStmt(pCxt, yymsp[-3].minor.yy313, &yymsp[-2].minor.yy129, &yymsp[0].minor.yy129); } +#line 6398 "sql.c" break; - case 316: /* cmd ::= DESC full_table_name */ - case 317: /* cmd ::= DESCRIBE full_table_name */ yytestcase(yyruleno==317); -{ pCxt->pRootNode = createDescribeStmt(pCxt, yymsp[0].minor.yy452); } + case 317: /* cmd ::= DESC full_table_name */ + case 318: /* cmd ::= DESCRIBE full_table_name */ yytestcase(yyruleno==318); +#line 560 "sql.y" +{ pCxt->pRootNode = createDescribeStmt(pCxt, yymsp[0].minor.yy840); } +#line 6404 "sql.c" break; - case 318: /* cmd ::= RESET QUERY CACHE */ + case 319: /* cmd ::= RESET QUERY CACHE */ +#line 564 "sql.y" { pCxt->pRootNode = createResetQueryCacheStmt(pCxt); } +#line 6409 "sql.c" break; - case 319: /* cmd ::= EXPLAIN analyze_opt explain_options query_or_subquery */ - case 320: /* cmd ::= EXPLAIN analyze_opt explain_options insert_query */ yytestcase(yyruleno==320); -{ pCxt->pRootNode = createExplainStmt(pCxt, yymsp[-2].minor.yy667, yymsp[-1].minor.yy452, yymsp[0].minor.yy452); } + case 320: /* cmd ::= EXPLAIN analyze_opt explain_options query_or_subquery */ + case 321: /* cmd ::= EXPLAIN analyze_opt explain_options insert_query */ yytestcase(yyruleno==321); +#line 567 "sql.y" +{ pCxt->pRootNode = createExplainStmt(pCxt, yymsp[-2].minor.yy313, yymsp[-1].minor.yy840, yymsp[0].minor.yy840); } +#line 6415 "sql.c" break; - case 323: /* explain_options ::= */ -{ yymsp[1].minor.yy452 = createDefaultExplainOptions(pCxt); } + case 324: /* explain_options ::= */ +#line 575 "sql.y" +{ yymsp[1].minor.yy840 = createDefaultExplainOptions(pCxt); } +#line 6420 "sql.c" break; - case 324: /* explain_options ::= explain_options VERBOSE NK_BOOL */ -{ yylhsminor.yy452 = setExplainVerbose(pCxt, yymsp[-2].minor.yy452, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; + case 325: /* explain_options ::= explain_options VERBOSE NK_BOOL */ +#line 576 "sql.y" +{ yylhsminor.yy840 = setExplainVerbose(pCxt, yymsp[-2].minor.yy840, &yymsp[0].minor.yy0); } +#line 6425 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 325: /* explain_options ::= explain_options RATIO NK_FLOAT */ -{ yylhsminor.yy452 = setExplainRatio(pCxt, yymsp[-2].minor.yy452, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; + case 326: /* explain_options ::= explain_options RATIO NK_FLOAT */ +#line 577 "sql.y" +{ yylhsminor.yy840 = setExplainRatio(pCxt, yymsp[-2].minor.yy840, &yymsp[0].minor.yy0); } +#line 6431 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 326: /* cmd ::= CREATE or_replace_opt agg_func_opt FUNCTION not_exists_opt function_name AS NK_STRING OUTPUTTYPE type_name bufsize_opt language_opt */ -{ pCxt->pRootNode = createCreateFunctionStmt(pCxt, yymsp[-7].minor.yy667, yymsp[-9].minor.yy667, &yymsp[-6].minor.yy371, &yymsp[-4].minor.yy0, yymsp[-2].minor.yy310, yymsp[-1].minor.yy416, &yymsp[0].minor.yy371, yymsp[-10].minor.yy667); } + case 327: /* cmd ::= CREATE or_replace_opt agg_func_opt FUNCTION not_exists_opt function_name AS NK_STRING OUTPUTTYPE type_name bufsize_opt language_opt */ +#line 582 "sql.y" +{ pCxt->pRootNode = createCreateFunctionStmt(pCxt, yymsp[-7].minor.yy313, yymsp[-9].minor.yy313, &yymsp[-6].minor.yy129, &yymsp[-4].minor.yy0, yymsp[-2].minor.yy784, yymsp[-1].minor.yy356, &yymsp[0].minor.yy129, yymsp[-10].minor.yy313); } +#line 6437 "sql.c" break; - case 327: /* cmd ::= DROP FUNCTION exists_opt function_name */ -{ pCxt->pRootNode = createDropFunctionStmt(pCxt, yymsp[-1].minor.yy667, &yymsp[0].minor.yy371); } + case 328: /* cmd ::= DROP FUNCTION exists_opt function_name */ +#line 583 "sql.y" +{ pCxt->pRootNode = createDropFunctionStmt(pCxt, yymsp[-1].minor.yy313, &yymsp[0].minor.yy129); } +#line 6442 "sql.c" break; - case 332: /* language_opt ::= */ -{ yymsp[1].minor.yy371 = nil_token; } + case 333: /* language_opt ::= */ +#line 597 "sql.y" +{ yymsp[1].minor.yy129 = nil_token; } +#line 6447 "sql.c" break; - case 333: /* language_opt ::= LANGUAGE NK_STRING */ -{ yymsp[-1].minor.yy371 = yymsp[0].minor.yy0; } + case 334: /* language_opt ::= LANGUAGE NK_STRING */ +#line 598 "sql.y" +{ yymsp[-1].minor.yy129 = yymsp[0].minor.yy0; } +#line 6452 "sql.c" break; - case 336: /* cmd ::= CREATE STREAM not_exists_opt stream_name stream_options INTO full_table_name col_list_opt tag_def_or_ref_opt subtable_opt AS query_or_subquery */ -{ pCxt->pRootNode = createCreateStreamStmt(pCxt, yymsp[-9].minor.yy667, &yymsp[-8].minor.yy371, yymsp[-5].minor.yy452, yymsp[-7].minor.yy452, yymsp[-3].minor.yy812, yymsp[-2].minor.yy452, yymsp[0].minor.yy452, yymsp[-4].minor.yy812); } + case 337: /* cmd ::= CREATE STREAM not_exists_opt stream_name stream_options INTO full_table_name col_list_opt tag_def_or_ref_opt subtable_opt AS query_or_subquery */ +#line 608 "sql.y" +{ pCxt->pRootNode = createCreateStreamStmt(pCxt, yymsp[-9].minor.yy313, &yymsp[-8].minor.yy129, yymsp[-5].minor.yy840, yymsp[-7].minor.yy840, yymsp[-3].minor.yy56, yymsp[-2].minor.yy840, yymsp[0].minor.yy840, yymsp[-4].minor.yy56); } +#line 6457 "sql.c" break; - case 337: /* cmd ::= DROP STREAM exists_opt stream_name */ -{ pCxt->pRootNode = createDropStreamStmt(pCxt, yymsp[-1].minor.yy667, &yymsp[0].minor.yy371); } + case 338: /* cmd ::= DROP STREAM exists_opt stream_name */ +#line 609 "sql.y" +{ pCxt->pRootNode = createDropStreamStmt(pCxt, yymsp[-1].minor.yy313, &yymsp[0].minor.yy129); } +#line 6462 "sql.c" break; - case 338: /* cmd ::= PAUSE STREAM exists_opt stream_name */ -{ pCxt->pRootNode = createPauseStreamStmt(pCxt, yymsp[-1].minor.yy667, &yymsp[0].minor.yy371); } + case 339: /* cmd ::= PAUSE STREAM exists_opt stream_name */ +#line 610 "sql.y" +{ pCxt->pRootNode = createPauseStreamStmt(pCxt, yymsp[-1].minor.yy313, &yymsp[0].minor.yy129); } +#line 6467 "sql.c" break; - case 339: /* cmd ::= RESUME STREAM exists_opt ignore_opt stream_name */ -{ pCxt->pRootNode = createResumeStreamStmt(pCxt, yymsp[-2].minor.yy667, yymsp[-1].minor.yy667, &yymsp[0].minor.yy371); } + case 340: /* cmd ::= RESUME STREAM exists_opt ignore_opt stream_name */ +#line 611 "sql.y" +{ pCxt->pRootNode = createResumeStreamStmt(pCxt, yymsp[-2].minor.yy313, yymsp[-1].minor.yy313, &yymsp[0].minor.yy129); } +#line 6472 "sql.c" break; - case 346: /* stream_options ::= stream_options TRIGGER AT_ONCE */ - case 347: /* stream_options ::= stream_options TRIGGER WINDOW_CLOSE */ yytestcase(yyruleno==347); -{ yylhsminor.yy452 = setStreamOptions(pCxt, yymsp[-2].minor.yy452, SOPT_TRIGGER_TYPE_SET, &yymsp[0].minor.yy0, NULL); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; + case 347: /* stream_options ::= stream_options TRIGGER AT_ONCE */ + case 348: /* stream_options ::= stream_options TRIGGER WINDOW_CLOSE */ yytestcase(yyruleno==348); +#line 625 "sql.y" +{ yylhsminor.yy840 = setStreamOptions(pCxt, yymsp[-2].minor.yy840, SOPT_TRIGGER_TYPE_SET, &yymsp[0].minor.yy0, NULL); } +#line 6478 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 348: /* stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal */ -{ yylhsminor.yy452 = setStreamOptions(pCxt, yymsp[-3].minor.yy452, SOPT_TRIGGER_TYPE_SET, &yymsp[-1].minor.yy0, releaseRawExprNode(pCxt, yymsp[0].minor.yy452)); } - yymsp[-3].minor.yy452 = yylhsminor.yy452; + case 349: /* stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal */ +#line 627 "sql.y" +{ yylhsminor.yy840 = setStreamOptions(pCxt, yymsp[-3].minor.yy840, SOPT_TRIGGER_TYPE_SET, &yymsp[-1].minor.yy0, releaseRawExprNode(pCxt, yymsp[0].minor.yy840)); } +#line 6484 "sql.c" + yymsp[-3].minor.yy840 = yylhsminor.yy840; break; - case 349: /* stream_options ::= stream_options WATERMARK duration_literal */ -{ yylhsminor.yy452 = setStreamOptions(pCxt, yymsp[-2].minor.yy452, SOPT_WATERMARK_SET, NULL, releaseRawExprNode(pCxt, yymsp[0].minor.yy452)); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; + case 350: /* stream_options ::= stream_options WATERMARK duration_literal */ +#line 628 "sql.y" +{ yylhsminor.yy840 = setStreamOptions(pCxt, yymsp[-2].minor.yy840, SOPT_WATERMARK_SET, NULL, releaseRawExprNode(pCxt, yymsp[0].minor.yy840)); } +#line 6490 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 350: /* stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER */ -{ yylhsminor.yy452 = setStreamOptions(pCxt, yymsp[-3].minor.yy452, SOPT_IGNORE_EXPIRED_SET, &yymsp[0].minor.yy0, NULL); } - yymsp[-3].minor.yy452 = yylhsminor.yy452; + case 351: /* stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER */ +#line 629 "sql.y" +{ yylhsminor.yy840 = setStreamOptions(pCxt, yymsp[-3].minor.yy840, SOPT_IGNORE_EXPIRED_SET, &yymsp[0].minor.yy0, NULL); } +#line 6496 "sql.c" + yymsp[-3].minor.yy840 = yylhsminor.yy840; break; - case 351: /* stream_options ::= stream_options FILL_HISTORY NK_INTEGER */ -{ yylhsminor.yy452 = setStreamOptions(pCxt, yymsp[-2].minor.yy452, SOPT_FILL_HISTORY_SET, &yymsp[0].minor.yy0, NULL); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; + case 352: /* stream_options ::= stream_options FILL_HISTORY NK_INTEGER */ +#line 630 "sql.y" +{ yylhsminor.yy840 = setStreamOptions(pCxt, yymsp[-2].minor.yy840, SOPT_FILL_HISTORY_SET, &yymsp[0].minor.yy0, NULL); } +#line 6502 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 352: /* stream_options ::= stream_options DELETE_MARK duration_literal */ -{ yylhsminor.yy452 = setStreamOptions(pCxt, yymsp[-2].minor.yy452, SOPT_DELETE_MARK_SET, NULL, releaseRawExprNode(pCxt, yymsp[0].minor.yy452)); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; + case 353: /* stream_options ::= stream_options DELETE_MARK duration_literal */ +#line 631 "sql.y" +{ yylhsminor.yy840 = setStreamOptions(pCxt, yymsp[-2].minor.yy840, SOPT_DELETE_MARK_SET, NULL, releaseRawExprNode(pCxt, yymsp[0].minor.yy840)); } +#line 6508 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 353: /* stream_options ::= stream_options IGNORE UPDATE NK_INTEGER */ -{ yylhsminor.yy452 = setStreamOptions(pCxt, yymsp[-3].minor.yy452, SOPT_IGNORE_UPDATE_SET, &yymsp[0].minor.yy0, NULL); } - yymsp[-3].minor.yy452 = yylhsminor.yy452; + case 354: /* stream_options ::= stream_options IGNORE UPDATE NK_INTEGER */ +#line 632 "sql.y" +{ yylhsminor.yy840 = setStreamOptions(pCxt, yymsp[-3].minor.yy840, SOPT_IGNORE_UPDATE_SET, &yymsp[0].minor.yy0, NULL); } +#line 6514 "sql.c" + yymsp[-3].minor.yy840 = yylhsminor.yy840; break; - case 355: /* subtable_opt ::= SUBTABLE NK_LP expression NK_RP */ - case 547: /* sliding_opt ::= SLIDING NK_LP duration_literal NK_RP */ yytestcase(yyruleno==547); - case 568: /* every_opt ::= EVERY NK_LP duration_literal NK_RP */ yytestcase(yyruleno==568); -{ yymsp[-3].minor.yy452 = releaseRawExprNode(pCxt, yymsp[-1].minor.yy452); } + case 356: /* subtable_opt ::= SUBTABLE NK_LP expression NK_RP */ + case 550: /* sliding_opt ::= SLIDING NK_LP duration_literal NK_RP */ yytestcase(yyruleno==550); + case 571: /* every_opt ::= EVERY NK_LP duration_literal NK_RP */ yytestcase(yyruleno==571); +#line 635 "sql.y" +{ yymsp[-3].minor.yy840 = releaseRawExprNode(pCxt, yymsp[-1].minor.yy840); } +#line 6522 "sql.c" break; - case 358: /* cmd ::= KILL CONNECTION NK_INTEGER */ + case 359: /* cmd ::= KILL CONNECTION NK_INTEGER */ +#line 643 "sql.y" { pCxt->pRootNode = createKillStmt(pCxt, QUERY_NODE_KILL_CONNECTION_STMT, &yymsp[0].minor.yy0); } +#line 6527 "sql.c" break; - case 359: /* cmd ::= KILL QUERY NK_STRING */ + case 360: /* cmd ::= KILL QUERY NK_STRING */ +#line 644 "sql.y" { pCxt->pRootNode = createKillQueryStmt(pCxt, &yymsp[0].minor.yy0); } +#line 6532 "sql.c" break; - case 360: /* cmd ::= KILL TRANSACTION NK_INTEGER */ + case 361: /* cmd ::= KILL TRANSACTION NK_INTEGER */ +#line 645 "sql.y" { pCxt->pRootNode = createKillStmt(pCxt, QUERY_NODE_KILL_TRANSACTION_STMT, &yymsp[0].minor.yy0); } +#line 6537 "sql.c" break; - case 361: /* cmd ::= BALANCE VGROUP */ + case 362: /* cmd ::= BALANCE VGROUP */ +#line 648 "sql.y" { pCxt->pRootNode = createBalanceVgroupStmt(pCxt); } +#line 6542 "sql.c" break; - case 362: /* cmd ::= BALANCE VGROUP LEADER */ + case 363: /* cmd ::= BALANCE VGROUP LEADER */ +#line 649 "sql.y" { pCxt->pRootNode = createBalanceVgroupLeaderStmt(pCxt); } +#line 6547 "sql.c" break; - case 363: /* cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER */ + case 364: /* cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER */ +#line 650 "sql.y" { pCxt->pRootNode = createMergeVgroupStmt(pCxt, &yymsp[-1].minor.yy0, &yymsp[0].minor.yy0); } +#line 6552 "sql.c" break; - case 364: /* cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list */ -{ pCxt->pRootNode = createRedistributeVgroupStmt(pCxt, &yymsp[-1].minor.yy0, yymsp[0].minor.yy812); } + case 365: /* cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list */ +#line 651 "sql.y" +{ pCxt->pRootNode = createRedistributeVgroupStmt(pCxt, &yymsp[-1].minor.yy0, yymsp[0].minor.yy56); } +#line 6557 "sql.c" break; - case 365: /* cmd ::= SPLIT VGROUP NK_INTEGER */ + case 366: /* cmd ::= SPLIT VGROUP NK_INTEGER */ +#line 652 "sql.y" { pCxt->pRootNode = createSplitVgroupStmt(pCxt, &yymsp[0].minor.yy0); } +#line 6562 "sql.c" break; - case 366: /* dnode_list ::= DNODE NK_INTEGER */ -{ yymsp[-1].minor.yy812 = createNodeList(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0)); } + case 367: /* dnode_list ::= DNODE NK_INTEGER */ +#line 656 "sql.y" +{ yymsp[-1].minor.yy56 = createNodeList(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0)); } +#line 6567 "sql.c" break; - case 368: /* cmd ::= DELETE FROM full_table_name where_clause_opt */ -{ pCxt->pRootNode = createDeleteStmt(pCxt, yymsp[-1].minor.yy452, yymsp[0].minor.yy452); } + case 369: /* cmd ::= DELETE FROM full_table_name where_clause_opt */ +#line 663 "sql.y" +{ pCxt->pRootNode = createDeleteStmt(pCxt, yymsp[-1].minor.yy840, yymsp[0].minor.yy840); } +#line 6572 "sql.c" break; - case 371: /* insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery */ -{ yymsp[-6].minor.yy452 = createInsertStmt(pCxt, yymsp[-4].minor.yy452, yymsp[-2].minor.yy812, yymsp[0].minor.yy452); } + case 372: /* insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery */ +#line 672 "sql.y" +{ yymsp[-6].minor.yy840 = createInsertStmt(pCxt, yymsp[-4].minor.yy840, yymsp[-2].minor.yy56, yymsp[0].minor.yy840); } +#line 6577 "sql.c" break; - case 372: /* insert_query ::= INSERT INTO full_table_name query_or_subquery */ -{ yymsp[-3].minor.yy452 = createInsertStmt(pCxt, yymsp[-1].minor.yy452, NULL, yymsp[0].minor.yy452); } + case 373: /* insert_query ::= INSERT INTO full_table_name query_or_subquery */ +#line 673 "sql.y" +{ yymsp[-3].minor.yy840 = createInsertStmt(pCxt, yymsp[-1].minor.yy840, NULL, yymsp[0].minor.yy840); } +#line 6582 "sql.c" break; - case 373: /* literal ::= NK_INTEGER */ -{ yylhsminor.yy452 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &yymsp[0].minor.yy0)); } - yymsp[0].minor.yy452 = yylhsminor.yy452; + case 374: /* literal ::= NK_INTEGER */ +#line 676 "sql.y" +{ yylhsminor.yy840 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &yymsp[0].minor.yy0)); } +#line 6587 "sql.c" + yymsp[0].minor.yy840 = yylhsminor.yy840; break; - case 374: /* literal ::= NK_FLOAT */ -{ yylhsminor.yy452 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &yymsp[0].minor.yy0)); } - yymsp[0].minor.yy452 = yylhsminor.yy452; + case 375: /* literal ::= NK_FLOAT */ +#line 677 "sql.y" +{ yylhsminor.yy840 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &yymsp[0].minor.yy0)); } +#line 6593 "sql.c" + yymsp[0].minor.yy840 = yylhsminor.yy840; break; - case 375: /* literal ::= NK_STRING */ -{ yylhsminor.yy452 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0)); } - yymsp[0].minor.yy452 = yylhsminor.yy452; + case 376: /* literal ::= NK_STRING */ +#line 678 "sql.y" +{ yylhsminor.yy840 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0)); } +#line 6599 "sql.c" + yymsp[0].minor.yy840 = yylhsminor.yy840; break; - case 376: /* literal ::= NK_BOOL */ -{ yylhsminor.yy452 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_BOOL, &yymsp[0].minor.yy0)); } - yymsp[0].minor.yy452 = yylhsminor.yy452; + case 377: /* literal ::= NK_BOOL */ +#line 679 "sql.y" +{ yylhsminor.yy840 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_BOOL, &yymsp[0].minor.yy0)); } +#line 6605 "sql.c" + yymsp[0].minor.yy840 = yylhsminor.yy840; break; - case 377: /* literal ::= TIMESTAMP NK_STRING */ -{ yylhsminor.yy452 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0)); } - yymsp[-1].minor.yy452 = yylhsminor.yy452; + case 378: /* literal ::= TIMESTAMP NK_STRING */ +#line 680 "sql.y" +{ yylhsminor.yy840 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0)); } +#line 6611 "sql.c" + yymsp[-1].minor.yy840 = yylhsminor.yy840; break; - case 378: /* literal ::= duration_literal */ - case 388: /* signed_literal ::= signed */ yytestcase(yyruleno==388); - case 409: /* expr_or_subquery ::= expression */ yytestcase(yyruleno==409); - case 410: /* expression ::= literal */ yytestcase(yyruleno==410); - case 411: /* expression ::= pseudo_column */ yytestcase(yyruleno==411); - case 412: /* expression ::= column_reference */ yytestcase(yyruleno==412); - case 413: /* expression ::= function_expression */ yytestcase(yyruleno==413); - case 414: /* expression ::= case_when_expression */ yytestcase(yyruleno==414); - case 445: /* function_expression ::= literal_func */ yytestcase(yyruleno==445); - case 494: /* boolean_value_expression ::= boolean_primary */ yytestcase(yyruleno==494); - case 498: /* boolean_primary ::= predicate */ yytestcase(yyruleno==498); - case 500: /* common_expression ::= expr_or_subquery */ yytestcase(yyruleno==500); - case 501: /* common_expression ::= boolean_value_expression */ yytestcase(yyruleno==501); - case 504: /* table_reference_list ::= table_reference */ yytestcase(yyruleno==504); - case 506: /* table_reference ::= table_primary */ yytestcase(yyruleno==506); - case 507: /* table_reference ::= joined_table */ yytestcase(yyruleno==507); - case 511: /* table_primary ::= parenthesized_joined_table */ yytestcase(yyruleno==511); - case 570: /* query_simple ::= query_specification */ yytestcase(yyruleno==570); - case 571: /* query_simple ::= union_query_expression */ yytestcase(yyruleno==571); - case 574: /* query_simple_or_subquery ::= query_simple */ yytestcase(yyruleno==574); - case 576: /* query_or_subquery ::= query_expression */ yytestcase(yyruleno==576); -{ yylhsminor.yy452 = yymsp[0].minor.yy452; } - yymsp[0].minor.yy452 = yylhsminor.yy452; + case 379: /* literal ::= duration_literal */ + case 389: /* signed_literal ::= signed */ yytestcase(yyruleno==389); + case 410: /* expr_or_subquery ::= expression */ yytestcase(yyruleno==410); + case 411: /* expression ::= literal */ yytestcase(yyruleno==411); + case 412: /* expression ::= pseudo_column */ yytestcase(yyruleno==412); + case 413: /* expression ::= column_reference */ yytestcase(yyruleno==413); + case 414: /* expression ::= function_expression */ yytestcase(yyruleno==414); + case 415: /* expression ::= case_when_expression */ yytestcase(yyruleno==415); + case 446: /* function_expression ::= literal_func */ yytestcase(yyruleno==446); + case 495: /* boolean_value_expression ::= boolean_primary */ yytestcase(yyruleno==495); + case 499: /* boolean_primary ::= predicate */ yytestcase(yyruleno==499); + case 501: /* common_expression ::= expr_or_subquery */ yytestcase(yyruleno==501); + case 502: /* common_expression ::= boolean_value_expression */ yytestcase(yyruleno==502); + case 505: /* table_reference_list ::= table_reference */ yytestcase(yyruleno==505); + case 507: /* table_reference ::= table_primary */ yytestcase(yyruleno==507); + case 508: /* table_reference ::= joined_table */ yytestcase(yyruleno==508); + case 512: /* table_primary ::= parenthesized_joined_table */ yytestcase(yyruleno==512); + case 573: /* query_simple ::= query_specification */ yytestcase(yyruleno==573); + case 574: /* query_simple ::= union_query_expression */ yytestcase(yyruleno==574); + case 577: /* query_simple_or_subquery ::= query_simple */ yytestcase(yyruleno==577); + case 579: /* query_or_subquery ::= query_expression */ yytestcase(yyruleno==579); +#line 681 "sql.y" +{ yylhsminor.yy840 = yymsp[0].minor.yy840; } +#line 6637 "sql.c" + yymsp[0].minor.yy840 = yylhsminor.yy840; break; - case 379: /* literal ::= NULL */ -{ yylhsminor.yy452 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_NULL, &yymsp[0].minor.yy0)); } - yymsp[0].minor.yy452 = yylhsminor.yy452; + case 380: /* literal ::= NULL */ +#line 682 "sql.y" +{ yylhsminor.yy840 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_NULL, &yymsp[0].minor.yy0)); } +#line 6643 "sql.c" + yymsp[0].minor.yy840 = yylhsminor.yy840; break; - case 380: /* literal ::= NK_QUESTION */ -{ yylhsminor.yy452 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createPlaceholderValueNode(pCxt, &yymsp[0].minor.yy0)); } - yymsp[0].minor.yy452 = yylhsminor.yy452; + case 381: /* literal ::= NK_QUESTION */ +#line 683 "sql.y" +{ yylhsminor.yy840 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createPlaceholderValueNode(pCxt, &yymsp[0].minor.yy0)); } +#line 6649 "sql.c" + yymsp[0].minor.yy840 = yylhsminor.yy840; break; - case 381: /* duration_literal ::= NK_VARIABLE */ -{ yylhsminor.yy452 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } - yymsp[0].minor.yy452 = yylhsminor.yy452; + case 382: /* duration_literal ::= NK_VARIABLE */ +#line 685 "sql.y" +{ yylhsminor.yy840 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } +#line 6655 "sql.c" + yymsp[0].minor.yy840 = yylhsminor.yy840; break; - case 382: /* signed ::= NK_INTEGER */ -{ yylhsminor.yy452 = createValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &yymsp[0].minor.yy0); } - yymsp[0].minor.yy452 = yylhsminor.yy452; + case 383: /* signed ::= NK_INTEGER */ +#line 687 "sql.y" +{ yylhsminor.yy840 = createValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &yymsp[0].minor.yy0); } +#line 6661 "sql.c" + yymsp[0].minor.yy840 = yylhsminor.yy840; break; - case 383: /* signed ::= NK_PLUS NK_INTEGER */ -{ yymsp[-1].minor.yy452 = createValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &yymsp[0].minor.yy0); } + case 384: /* signed ::= NK_PLUS NK_INTEGER */ +#line 688 "sql.y" +{ yymsp[-1].minor.yy840 = createValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &yymsp[0].minor.yy0); } +#line 6667 "sql.c" break; - case 384: /* signed ::= NK_MINUS NK_INTEGER */ + case 385: /* signed ::= NK_MINUS NK_INTEGER */ +#line 689 "sql.y" { SToken t = yymsp[-1].minor.yy0; t.n = (yymsp[0].minor.yy0.z + yymsp[0].minor.yy0.n) - yymsp[-1].minor.yy0.z; - yylhsminor.yy452 = createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &t); + yylhsminor.yy840 = createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &t); } - yymsp[-1].minor.yy452 = yylhsminor.yy452; +#line 6676 "sql.c" + yymsp[-1].minor.yy840 = yylhsminor.yy840; break; - case 385: /* signed ::= NK_FLOAT */ -{ yylhsminor.yy452 = createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &yymsp[0].minor.yy0); } - yymsp[0].minor.yy452 = yylhsminor.yy452; + case 386: /* signed ::= NK_FLOAT */ +#line 694 "sql.y" +{ yylhsminor.yy840 = createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &yymsp[0].minor.yy0); } +#line 6682 "sql.c" + yymsp[0].minor.yy840 = yylhsminor.yy840; break; - case 386: /* signed ::= NK_PLUS NK_FLOAT */ -{ yymsp[-1].minor.yy452 = createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &yymsp[0].minor.yy0); } + case 387: /* signed ::= NK_PLUS NK_FLOAT */ +#line 695 "sql.y" +{ yymsp[-1].minor.yy840 = createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &yymsp[0].minor.yy0); } +#line 6688 "sql.c" break; - case 387: /* signed ::= NK_MINUS NK_FLOAT */ + case 388: /* signed ::= NK_MINUS NK_FLOAT */ +#line 696 "sql.y" { SToken t = yymsp[-1].minor.yy0; t.n = (yymsp[0].minor.yy0.z + yymsp[0].minor.yy0.n) - yymsp[-1].minor.yy0.z; - yylhsminor.yy452 = createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &t); + yylhsminor.yy840 = createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &t); } - yymsp[-1].minor.yy452 = yylhsminor.yy452; +#line 6697 "sql.c" + yymsp[-1].minor.yy840 = yylhsminor.yy840; break; - case 389: /* signed_literal ::= NK_STRING */ -{ yylhsminor.yy452 = createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0); } - yymsp[0].minor.yy452 = yylhsminor.yy452; + case 390: /* signed_literal ::= NK_STRING */ +#line 703 "sql.y" +{ yylhsminor.yy840 = createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0); } +#line 6703 "sql.c" + yymsp[0].minor.yy840 = yylhsminor.yy840; break; - case 390: /* signed_literal ::= NK_BOOL */ -{ yylhsminor.yy452 = createValueNode(pCxt, TSDB_DATA_TYPE_BOOL, &yymsp[0].minor.yy0); } - yymsp[0].minor.yy452 = yylhsminor.yy452; + case 391: /* signed_literal ::= NK_BOOL */ +#line 704 "sql.y" +{ yylhsminor.yy840 = createValueNode(pCxt, TSDB_DATA_TYPE_BOOL, &yymsp[0].minor.yy0); } +#line 6709 "sql.c" + yymsp[0].minor.yy840 = yylhsminor.yy840; break; - case 391: /* signed_literal ::= TIMESTAMP NK_STRING */ -{ yymsp[-1].minor.yy452 = createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0); } + case 392: /* signed_literal ::= TIMESTAMP NK_STRING */ +#line 705 "sql.y" +{ yymsp[-1].minor.yy840 = createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0); } +#line 6715 "sql.c" break; - case 392: /* signed_literal ::= duration_literal */ - case 394: /* signed_literal ::= literal_func */ yytestcase(yyruleno==394); - case 465: /* star_func_para ::= expr_or_subquery */ yytestcase(yyruleno==465); - case 527: /* select_item ::= common_expression */ yytestcase(yyruleno==527); - case 537: /* partition_item ::= expr_or_subquery */ yytestcase(yyruleno==537); - case 575: /* query_simple_or_subquery ::= subquery */ yytestcase(yyruleno==575); - case 577: /* query_or_subquery ::= subquery */ yytestcase(yyruleno==577); - case 590: /* search_condition ::= common_expression */ yytestcase(yyruleno==590); -{ yylhsminor.yy452 = releaseRawExprNode(pCxt, yymsp[0].minor.yy452); } - yymsp[0].minor.yy452 = yylhsminor.yy452; + case 393: /* signed_literal ::= duration_literal */ + case 395: /* signed_literal ::= literal_func */ yytestcase(yyruleno==395); + case 466: /* star_func_para ::= expr_or_subquery */ yytestcase(yyruleno==466); + case 530: /* select_item ::= common_expression */ yytestcase(yyruleno==530); + case 540: /* partition_item ::= expr_or_subquery */ yytestcase(yyruleno==540); + case 578: /* query_simple_or_subquery ::= subquery */ yytestcase(yyruleno==578); + case 580: /* query_or_subquery ::= subquery */ yytestcase(yyruleno==580); + case 593: /* search_condition ::= common_expression */ yytestcase(yyruleno==593); +#line 706 "sql.y" +{ yylhsminor.yy840 = releaseRawExprNode(pCxt, yymsp[0].minor.yy840); } +#line 6727 "sql.c" + yymsp[0].minor.yy840 = yylhsminor.yy840; break; - case 393: /* signed_literal ::= NULL */ -{ yylhsminor.yy452 = createValueNode(pCxt, TSDB_DATA_TYPE_NULL, &yymsp[0].minor.yy0); } - yymsp[0].minor.yy452 = yylhsminor.yy452; + case 394: /* signed_literal ::= NULL */ +#line 707 "sql.y" +{ yylhsminor.yy840 = createValueNode(pCxt, TSDB_DATA_TYPE_NULL, &yymsp[0].minor.yy0); } +#line 6733 "sql.c" + yymsp[0].minor.yy840 = yylhsminor.yy840; break; - case 395: /* signed_literal ::= NK_QUESTION */ -{ yylhsminor.yy452 = createPlaceholderValueNode(pCxt, &yymsp[0].minor.yy0); } - yymsp[0].minor.yy452 = yylhsminor.yy452; + case 396: /* signed_literal ::= NK_QUESTION */ +#line 709 "sql.y" +{ yylhsminor.yy840 = createPlaceholderValueNode(pCxt, &yymsp[0].minor.yy0); } +#line 6739 "sql.c" + yymsp[0].minor.yy840 = yylhsminor.yy840; break; - case 415: /* expression ::= NK_LP expression NK_RP */ - case 499: /* boolean_primary ::= NK_LP boolean_value_expression NK_RP */ yytestcase(yyruleno==499); - case 589: /* subquery ::= NK_LP subquery NK_RP */ yytestcase(yyruleno==589); -{ yylhsminor.yy452 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0, releaseRawExprNode(pCxt, yymsp[-1].minor.yy452)); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; + case 416: /* expression ::= NK_LP expression NK_RP */ + case 500: /* boolean_primary ::= NK_LP boolean_value_expression NK_RP */ yytestcase(yyruleno==500); + case 592: /* subquery ::= NK_LP subquery NK_RP */ yytestcase(yyruleno==592); +#line 770 "sql.y" +{ yylhsminor.yy840 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0, releaseRawExprNode(pCxt, yymsp[-1].minor.yy840)); } +#line 6747 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 416: /* expression ::= NK_PLUS expr_or_subquery */ + case 417: /* expression ::= NK_PLUS expr_or_subquery */ +#line 771 "sql.y" { - SToken t = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy452); - yylhsminor.yy452 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &t, releaseRawExprNode(pCxt, yymsp[0].minor.yy452)); + SToken t = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy840); + yylhsminor.yy840 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &t, releaseRawExprNode(pCxt, yymsp[0].minor.yy840)); } - yymsp[-1].minor.yy452 = yylhsminor.yy452; +#line 6756 "sql.c" + yymsp[-1].minor.yy840 = yylhsminor.yy840; break; - case 417: /* expression ::= NK_MINUS expr_or_subquery */ + case 418: /* expression ::= NK_MINUS expr_or_subquery */ +#line 775 "sql.y" { - SToken t = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy452); - yylhsminor.yy452 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &t, createOperatorNode(pCxt, OP_TYPE_MINUS, releaseRawExprNode(pCxt, yymsp[0].minor.yy452), NULL)); + SToken t = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy840); + yylhsminor.yy840 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &t, createOperatorNode(pCxt, OP_TYPE_MINUS, releaseRawExprNode(pCxt, yymsp[0].minor.yy840), NULL)); } - yymsp[-1].minor.yy452 = yylhsminor.yy452; +#line 6765 "sql.c" + yymsp[-1].minor.yy840 = yylhsminor.yy840; break; - case 418: /* expression ::= expr_or_subquery NK_PLUS expr_or_subquery */ + case 419: /* expression ::= expr_or_subquery NK_PLUS expr_or_subquery */ +#line 779 "sql.y" { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy452); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy452); - yylhsminor.yy452 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_ADD, releaseRawExprNode(pCxt, yymsp[-2].minor.yy452), releaseRawExprNode(pCxt, yymsp[0].minor.yy452))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy840); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy840); + yylhsminor.yy840 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_ADD, releaseRawExprNode(pCxt, yymsp[-2].minor.yy840), releaseRawExprNode(pCxt, yymsp[0].minor.yy840))); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 6775 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 419: /* expression ::= expr_or_subquery NK_MINUS expr_or_subquery */ + case 420: /* expression ::= expr_or_subquery NK_MINUS expr_or_subquery */ +#line 784 "sql.y" { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy452); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy452); - yylhsminor.yy452 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_SUB, releaseRawExprNode(pCxt, yymsp[-2].minor.yy452), releaseRawExprNode(pCxt, yymsp[0].minor.yy452))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy840); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy840); + yylhsminor.yy840 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_SUB, releaseRawExprNode(pCxt, yymsp[-2].minor.yy840), releaseRawExprNode(pCxt, yymsp[0].minor.yy840))); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 6785 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 420: /* expression ::= expr_or_subquery NK_STAR expr_or_subquery */ + case 421: /* expression ::= expr_or_subquery NK_STAR expr_or_subquery */ +#line 789 "sql.y" { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy452); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy452); - yylhsminor.yy452 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_MULTI, releaseRawExprNode(pCxt, yymsp[-2].minor.yy452), releaseRawExprNode(pCxt, yymsp[0].minor.yy452))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy840); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy840); + yylhsminor.yy840 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_MULTI, releaseRawExprNode(pCxt, yymsp[-2].minor.yy840), releaseRawExprNode(pCxt, yymsp[0].minor.yy840))); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 6795 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 421: /* expression ::= expr_or_subquery NK_SLASH expr_or_subquery */ + case 422: /* expression ::= expr_or_subquery NK_SLASH expr_or_subquery */ +#line 794 "sql.y" { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy452); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy452); - yylhsminor.yy452 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_DIV, releaseRawExprNode(pCxt, yymsp[-2].minor.yy452), releaseRawExprNode(pCxt, yymsp[0].minor.yy452))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy840); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy840); + yylhsminor.yy840 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_DIV, releaseRawExprNode(pCxt, yymsp[-2].minor.yy840), releaseRawExprNode(pCxt, yymsp[0].minor.yy840))); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 6805 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 422: /* expression ::= expr_or_subquery NK_REM expr_or_subquery */ + case 423: /* expression ::= expr_or_subquery NK_REM expr_or_subquery */ +#line 799 "sql.y" { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy452); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy452); - yylhsminor.yy452 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_REM, releaseRawExprNode(pCxt, yymsp[-2].minor.yy452), releaseRawExprNode(pCxt, yymsp[0].minor.yy452))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy840); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy840); + yylhsminor.yy840 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_REM, releaseRawExprNode(pCxt, yymsp[-2].minor.yy840), releaseRawExprNode(pCxt, yymsp[0].minor.yy840))); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 6815 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 423: /* expression ::= column_reference NK_ARROW NK_STRING */ + case 424: /* expression ::= column_reference NK_ARROW NK_STRING */ +#line 804 "sql.y" { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy452); - yylhsminor.yy452 = createRawExprNodeExt(pCxt, &s, &yymsp[0].minor.yy0, createOperatorNode(pCxt, OP_TYPE_JSON_GET_VALUE, releaseRawExprNode(pCxt, yymsp[-2].minor.yy452), createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy840); + yylhsminor.yy840 = createRawExprNodeExt(pCxt, &s, &yymsp[0].minor.yy0, createOperatorNode(pCxt, OP_TYPE_JSON_GET_VALUE, releaseRawExprNode(pCxt, yymsp[-2].minor.yy840), createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0))); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 6824 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 424: /* expression ::= expr_or_subquery NK_BITAND expr_or_subquery */ + case 425: /* expression ::= expr_or_subquery NK_BITAND expr_or_subquery */ +#line 808 "sql.y" { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy452); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy452); - yylhsminor.yy452 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_BIT_AND, releaseRawExprNode(pCxt, yymsp[-2].minor.yy452), releaseRawExprNode(pCxt, yymsp[0].minor.yy452))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy840); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy840); + yylhsminor.yy840 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_BIT_AND, releaseRawExprNode(pCxt, yymsp[-2].minor.yy840), releaseRawExprNode(pCxt, yymsp[0].minor.yy840))); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 6834 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 425: /* expression ::= expr_or_subquery NK_BITOR expr_or_subquery */ + case 426: /* expression ::= expr_or_subquery NK_BITOR expr_or_subquery */ +#line 813 "sql.y" { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy452); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy452); - yylhsminor.yy452 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_BIT_OR, releaseRawExprNode(pCxt, yymsp[-2].minor.yy452), releaseRawExprNode(pCxt, yymsp[0].minor.yy452))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy840); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy840); + yylhsminor.yy840 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_BIT_OR, releaseRawExprNode(pCxt, yymsp[-2].minor.yy840), releaseRawExprNode(pCxt, yymsp[0].minor.yy840))); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 6844 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 428: /* column_reference ::= column_name */ -{ yylhsminor.yy452 = createRawExprNode(pCxt, &yymsp[0].minor.yy371, createColumnNode(pCxt, NULL, &yymsp[0].minor.yy371)); } - yymsp[0].minor.yy452 = yylhsminor.yy452; + case 429: /* column_reference ::= column_name */ +#line 824 "sql.y" +{ yylhsminor.yy840 = createRawExprNode(pCxt, &yymsp[0].minor.yy129, createColumnNode(pCxt, NULL, &yymsp[0].minor.yy129)); } +#line 6850 "sql.c" + yymsp[0].minor.yy840 = yylhsminor.yy840; break; - case 429: /* column_reference ::= table_name NK_DOT column_name */ -{ yylhsminor.yy452 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy371, &yymsp[0].minor.yy371, createColumnNode(pCxt, &yymsp[-2].minor.yy371, &yymsp[0].minor.yy371)); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; + case 430: /* column_reference ::= table_name NK_DOT column_name */ +#line 825 "sql.y" +{ yylhsminor.yy840 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy129, &yymsp[0].minor.yy129, createColumnNode(pCxt, &yymsp[-2].minor.yy129, &yymsp[0].minor.yy129)); } +#line 6856 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 430: /* pseudo_column ::= ROWTS */ - case 431: /* pseudo_column ::= TBNAME */ yytestcase(yyruleno==431); - case 433: /* pseudo_column ::= QSTART */ yytestcase(yyruleno==433); - case 434: /* pseudo_column ::= QEND */ yytestcase(yyruleno==434); - case 435: /* pseudo_column ::= QDURATION */ yytestcase(yyruleno==435); - case 436: /* pseudo_column ::= WSTART */ yytestcase(yyruleno==436); - case 437: /* pseudo_column ::= WEND */ yytestcase(yyruleno==437); - case 438: /* pseudo_column ::= WDURATION */ yytestcase(yyruleno==438); - case 439: /* pseudo_column ::= IROWTS */ yytestcase(yyruleno==439); - case 440: /* pseudo_column ::= ISFILLED */ yytestcase(yyruleno==440); - case 441: /* pseudo_column ::= QTAGS */ yytestcase(yyruleno==441); - case 447: /* literal_func ::= NOW */ yytestcase(yyruleno==447); -{ yylhsminor.yy452 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[0].minor.yy0, NULL)); } - yymsp[0].minor.yy452 = yylhsminor.yy452; + case 431: /* pseudo_column ::= ROWTS */ + case 432: /* pseudo_column ::= TBNAME */ yytestcase(yyruleno==432); + case 434: /* pseudo_column ::= QSTART */ yytestcase(yyruleno==434); + case 435: /* pseudo_column ::= QEND */ yytestcase(yyruleno==435); + case 436: /* pseudo_column ::= QDURATION */ yytestcase(yyruleno==436); + case 437: /* pseudo_column ::= WSTART */ yytestcase(yyruleno==437); + case 438: /* pseudo_column ::= WEND */ yytestcase(yyruleno==438); + case 439: /* pseudo_column ::= WDURATION */ yytestcase(yyruleno==439); + case 440: /* pseudo_column ::= IROWTS */ yytestcase(yyruleno==440); + case 441: /* pseudo_column ::= ISFILLED */ yytestcase(yyruleno==441); + case 442: /* pseudo_column ::= QTAGS */ yytestcase(yyruleno==442); + case 448: /* literal_func ::= NOW */ yytestcase(yyruleno==448); +#line 827 "sql.y" +{ yylhsminor.yy840 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[0].minor.yy0, NULL)); } +#line 6873 "sql.c" + yymsp[0].minor.yy840 = yylhsminor.yy840; break; - case 432: /* pseudo_column ::= table_name NK_DOT TBNAME */ -{ yylhsminor.yy452 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy371, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[0].minor.yy0, createNodeList(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[-2].minor.yy371)))); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; + case 433: /* pseudo_column ::= table_name NK_DOT TBNAME */ +#line 829 "sql.y" +{ yylhsminor.yy840 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy129, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[0].minor.yy0, createNodeList(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[-2].minor.yy129)))); } +#line 6879 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 442: /* function_expression ::= function_name NK_LP expression_list NK_RP */ - case 443: /* function_expression ::= star_func NK_LP star_func_para_list NK_RP */ yytestcase(yyruleno==443); -{ yylhsminor.yy452 = createRawExprNodeExt(pCxt, &yymsp[-3].minor.yy371, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[-3].minor.yy371, yymsp[-1].minor.yy812)); } - yymsp[-3].minor.yy452 = yylhsminor.yy452; + case 443: /* function_expression ::= function_name NK_LP expression_list NK_RP */ + case 444: /* function_expression ::= star_func NK_LP star_func_para_list NK_RP */ yytestcase(yyruleno==444); +#line 840 "sql.y" +{ yylhsminor.yy840 = createRawExprNodeExt(pCxt, &yymsp[-3].minor.yy129, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[-3].minor.yy129, yymsp[-1].minor.yy56)); } +#line 6886 "sql.c" + yymsp[-3].minor.yy840 = yylhsminor.yy840; break; - case 444: /* function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP */ -{ yylhsminor.yy452 = createRawExprNodeExt(pCxt, &yymsp[-5].minor.yy0, &yymsp[0].minor.yy0, createCastFunctionNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy452), yymsp[-1].minor.yy310)); } - yymsp[-5].minor.yy452 = yylhsminor.yy452; + case 445: /* function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP */ +#line 843 "sql.y" +{ yylhsminor.yy840 = createRawExprNodeExt(pCxt, &yymsp[-5].minor.yy0, &yymsp[0].minor.yy0, createCastFunctionNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy840), yymsp[-1].minor.yy784)); } +#line 6892 "sql.c" + yymsp[-5].minor.yy840 = yylhsminor.yy840; break; - case 446: /* literal_func ::= noarg_func NK_LP NK_RP */ -{ yylhsminor.yy452 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy371, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[-2].minor.yy371, NULL)); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; + case 447: /* literal_func ::= noarg_func NK_LP NK_RP */ +#line 846 "sql.y" +{ yylhsminor.yy840 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy129, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[-2].minor.yy129, NULL)); } +#line 6898 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 461: /* star_func_para_list ::= NK_STAR */ -{ yylhsminor.yy812 = createNodeList(pCxt, createColumnNode(pCxt, NULL, &yymsp[0].minor.yy0)); } - yymsp[0].minor.yy812 = yylhsminor.yy812; + case 462: /* star_func_para_list ::= NK_STAR */ +#line 870 "sql.y" +{ yylhsminor.yy56 = createNodeList(pCxt, createColumnNode(pCxt, NULL, &yymsp[0].minor.yy0)); } +#line 6904 "sql.c" + yymsp[0].minor.yy56 = yylhsminor.yy56; break; - case 466: /* star_func_para ::= table_name NK_DOT NK_STAR */ - case 530: /* select_item ::= table_name NK_DOT NK_STAR */ yytestcase(yyruleno==530); -{ yylhsminor.yy452 = createColumnNode(pCxt, &yymsp[-2].minor.yy371, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; + case 467: /* star_func_para ::= table_name NK_DOT NK_STAR */ + case 533: /* select_item ::= table_name NK_DOT NK_STAR */ yytestcase(yyruleno==533); +#line 879 "sql.y" +{ yylhsminor.yy840 = createColumnNode(pCxt, &yymsp[-2].minor.yy129, &yymsp[0].minor.yy0); } +#line 6911 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 467: /* case_when_expression ::= CASE when_then_list case_when_else_opt END */ -{ yylhsminor.yy452 = createRawExprNodeExt(pCxt, &yymsp[-3].minor.yy0, &yymsp[0].minor.yy0, createCaseWhenNode(pCxt, NULL, yymsp[-2].minor.yy812, yymsp[-1].minor.yy452)); } - yymsp[-3].minor.yy452 = yylhsminor.yy452; + case 468: /* case_when_expression ::= CASE when_then_list case_when_else_opt END */ +#line 882 "sql.y" +{ yylhsminor.yy840 = createRawExprNodeExt(pCxt, &yymsp[-3].minor.yy0, &yymsp[0].minor.yy0, createCaseWhenNode(pCxt, NULL, yymsp[-2].minor.yy56, yymsp[-1].minor.yy840)); } +#line 6917 "sql.c" + yymsp[-3].minor.yy840 = yylhsminor.yy840; break; - case 468: /* case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END */ -{ yylhsminor.yy452 = createRawExprNodeExt(pCxt, &yymsp[-4].minor.yy0, &yymsp[0].minor.yy0, createCaseWhenNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy452), yymsp[-2].minor.yy812, yymsp[-1].minor.yy452)); } - yymsp[-4].minor.yy452 = yylhsminor.yy452; + case 469: /* case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END */ +#line 884 "sql.y" +{ yylhsminor.yy840 = createRawExprNodeExt(pCxt, &yymsp[-4].minor.yy0, &yymsp[0].minor.yy0, createCaseWhenNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy840), yymsp[-2].minor.yy56, yymsp[-1].minor.yy840)); } +#line 6923 "sql.c" + yymsp[-4].minor.yy840 = yylhsminor.yy840; break; - case 471: /* when_then_expr ::= WHEN common_expression THEN common_expression */ -{ yymsp[-3].minor.yy452 = createWhenThenNode(pCxt, releaseRawExprNode(pCxt, yymsp[-2].minor.yy452), releaseRawExprNode(pCxt, yymsp[0].minor.yy452)); } + case 472: /* when_then_expr ::= WHEN common_expression THEN common_expression */ +#line 891 "sql.y" +{ yymsp[-3].minor.yy840 = createWhenThenNode(pCxt, releaseRawExprNode(pCxt, yymsp[-2].minor.yy840), releaseRawExprNode(pCxt, yymsp[0].minor.yy840)); } +#line 6929 "sql.c" break; - case 473: /* case_when_else_opt ::= ELSE common_expression */ -{ yymsp[-1].minor.yy452 = releaseRawExprNode(pCxt, yymsp[0].minor.yy452); } + case 474: /* case_when_else_opt ::= ELSE common_expression */ +#line 894 "sql.y" +{ yymsp[-1].minor.yy840 = releaseRawExprNode(pCxt, yymsp[0].minor.yy840); } +#line 6934 "sql.c" break; - case 474: /* predicate ::= expr_or_subquery compare_op expr_or_subquery */ - case 479: /* predicate ::= expr_or_subquery in_op in_predicate_value */ yytestcase(yyruleno==479); + case 475: /* predicate ::= expr_or_subquery compare_op expr_or_subquery */ + case 480: /* predicate ::= expr_or_subquery in_op in_predicate_value */ yytestcase(yyruleno==480); +#line 897 "sql.y" { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy452); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy452); - yylhsminor.yy452 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, yymsp[-1].minor.yy354, releaseRawExprNode(pCxt, yymsp[-2].minor.yy452), releaseRawExprNode(pCxt, yymsp[0].minor.yy452))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy840); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy840); + yylhsminor.yy840 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, yymsp[-1].minor.yy76, releaseRawExprNode(pCxt, yymsp[-2].minor.yy840), releaseRawExprNode(pCxt, yymsp[0].minor.yy840))); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 6944 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 475: /* predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery */ + case 476: /* predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery */ +#line 904 "sql.y" { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-4].minor.yy452); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy452); - yylhsminor.yy452 = createRawExprNodeExt(pCxt, &s, &e, createBetweenAnd(pCxt, releaseRawExprNode(pCxt, yymsp[-4].minor.yy452), releaseRawExprNode(pCxt, yymsp[-2].minor.yy452), releaseRawExprNode(pCxt, yymsp[0].minor.yy452))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-4].minor.yy840); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy840); + yylhsminor.yy840 = createRawExprNodeExt(pCxt, &s, &e, createBetweenAnd(pCxt, releaseRawExprNode(pCxt, yymsp[-4].minor.yy840), releaseRawExprNode(pCxt, yymsp[-2].minor.yy840), releaseRawExprNode(pCxt, yymsp[0].minor.yy840))); } - yymsp[-4].minor.yy452 = yylhsminor.yy452; +#line 6954 "sql.c" + yymsp[-4].minor.yy840 = yylhsminor.yy840; break; - case 476: /* predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery */ + case 477: /* predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery */ +#line 910 "sql.y" { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-5].minor.yy452); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy452); - yylhsminor.yy452 = createRawExprNodeExt(pCxt, &s, &e, createNotBetweenAnd(pCxt, releaseRawExprNode(pCxt, yymsp[-5].minor.yy452), releaseRawExprNode(pCxt, yymsp[-2].minor.yy452), releaseRawExprNode(pCxt, yymsp[0].minor.yy452))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-5].minor.yy840); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy840); + yylhsminor.yy840 = createRawExprNodeExt(pCxt, &s, &e, createNotBetweenAnd(pCxt, releaseRawExprNode(pCxt, yymsp[-5].minor.yy840), releaseRawExprNode(pCxt, yymsp[-2].minor.yy840), releaseRawExprNode(pCxt, yymsp[0].minor.yy840))); } - yymsp[-5].minor.yy452 = yylhsminor.yy452; +#line 6964 "sql.c" + yymsp[-5].minor.yy840 = yylhsminor.yy840; break; - case 477: /* predicate ::= expr_or_subquery IS NULL */ + case 478: /* predicate ::= expr_or_subquery IS NULL */ +#line 915 "sql.y" { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy452); - yylhsminor.yy452 = createRawExprNodeExt(pCxt, &s, &yymsp[0].minor.yy0, createOperatorNode(pCxt, OP_TYPE_IS_NULL, releaseRawExprNode(pCxt, yymsp[-2].minor.yy452), NULL)); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy840); + yylhsminor.yy840 = createRawExprNodeExt(pCxt, &s, &yymsp[0].minor.yy0, createOperatorNode(pCxt, OP_TYPE_IS_NULL, releaseRawExprNode(pCxt, yymsp[-2].minor.yy840), NULL)); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 6973 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 478: /* predicate ::= expr_or_subquery IS NOT NULL */ + case 479: /* predicate ::= expr_or_subquery IS NOT NULL */ +#line 919 "sql.y" { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-3].minor.yy452); - yylhsminor.yy452 = createRawExprNodeExt(pCxt, &s, &yymsp[0].minor.yy0, createOperatorNode(pCxt, OP_TYPE_IS_NOT_NULL, releaseRawExprNode(pCxt, yymsp[-3].minor.yy452), NULL)); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-3].minor.yy840); + yylhsminor.yy840 = createRawExprNodeExt(pCxt, &s, &yymsp[0].minor.yy0, createOperatorNode(pCxt, OP_TYPE_IS_NOT_NULL, releaseRawExprNode(pCxt, yymsp[-3].minor.yy840), NULL)); } - yymsp[-3].minor.yy452 = yylhsminor.yy452; +#line 6982 "sql.c" + yymsp[-3].minor.yy840 = yylhsminor.yy840; break; - case 480: /* compare_op ::= NK_LT */ -{ yymsp[0].minor.yy354 = OP_TYPE_LOWER_THAN; } + case 481: /* compare_op ::= NK_LT */ +#line 931 "sql.y" +{ yymsp[0].minor.yy76 = OP_TYPE_LOWER_THAN; } +#line 6988 "sql.c" break; - case 481: /* compare_op ::= NK_GT */ -{ yymsp[0].minor.yy354 = OP_TYPE_GREATER_THAN; } + case 482: /* compare_op ::= NK_GT */ +#line 932 "sql.y" +{ yymsp[0].minor.yy76 = OP_TYPE_GREATER_THAN; } +#line 6993 "sql.c" break; - case 482: /* compare_op ::= NK_LE */ -{ yymsp[0].minor.yy354 = OP_TYPE_LOWER_EQUAL; } + case 483: /* compare_op ::= NK_LE */ +#line 933 "sql.y" +{ yymsp[0].minor.yy76 = OP_TYPE_LOWER_EQUAL; } +#line 6998 "sql.c" break; - case 483: /* compare_op ::= NK_GE */ -{ yymsp[0].minor.yy354 = OP_TYPE_GREATER_EQUAL; } + case 484: /* compare_op ::= NK_GE */ +#line 934 "sql.y" +{ yymsp[0].minor.yy76 = OP_TYPE_GREATER_EQUAL; } +#line 7003 "sql.c" break; - case 484: /* compare_op ::= NK_NE */ -{ yymsp[0].minor.yy354 = OP_TYPE_NOT_EQUAL; } + case 485: /* compare_op ::= NK_NE */ +#line 935 "sql.y" +{ yymsp[0].minor.yy76 = OP_TYPE_NOT_EQUAL; } +#line 7008 "sql.c" break; - case 485: /* compare_op ::= NK_EQ */ -{ yymsp[0].minor.yy354 = OP_TYPE_EQUAL; } + case 486: /* compare_op ::= NK_EQ */ +#line 936 "sql.y" +{ yymsp[0].minor.yy76 = OP_TYPE_EQUAL; } +#line 7013 "sql.c" break; - case 486: /* compare_op ::= LIKE */ -{ yymsp[0].minor.yy354 = OP_TYPE_LIKE; } + case 487: /* compare_op ::= LIKE */ +#line 937 "sql.y" +{ yymsp[0].minor.yy76 = OP_TYPE_LIKE; } +#line 7018 "sql.c" break; - case 487: /* compare_op ::= NOT LIKE */ -{ yymsp[-1].minor.yy354 = OP_TYPE_NOT_LIKE; } + case 488: /* compare_op ::= NOT LIKE */ +#line 938 "sql.y" +{ yymsp[-1].minor.yy76 = OP_TYPE_NOT_LIKE; } +#line 7023 "sql.c" break; - case 488: /* compare_op ::= MATCH */ -{ yymsp[0].minor.yy354 = OP_TYPE_MATCH; } + case 489: /* compare_op ::= MATCH */ +#line 939 "sql.y" +{ yymsp[0].minor.yy76 = OP_TYPE_MATCH; } +#line 7028 "sql.c" break; - case 489: /* compare_op ::= NMATCH */ -{ yymsp[0].minor.yy354 = OP_TYPE_NMATCH; } + case 490: /* compare_op ::= NMATCH */ +#line 940 "sql.y" +{ yymsp[0].minor.yy76 = OP_TYPE_NMATCH; } +#line 7033 "sql.c" break; - case 490: /* compare_op ::= CONTAINS */ -{ yymsp[0].minor.yy354 = OP_TYPE_JSON_CONTAINS; } + case 491: /* compare_op ::= CONTAINS */ +#line 941 "sql.y" +{ yymsp[0].minor.yy76 = OP_TYPE_JSON_CONTAINS; } +#line 7038 "sql.c" break; - case 491: /* in_op ::= IN */ -{ yymsp[0].minor.yy354 = OP_TYPE_IN; } + case 492: /* in_op ::= IN */ +#line 945 "sql.y" +{ yymsp[0].minor.yy76 = OP_TYPE_IN; } +#line 7043 "sql.c" break; - case 492: /* in_op ::= NOT IN */ -{ yymsp[-1].minor.yy354 = OP_TYPE_NOT_IN; } + case 493: /* in_op ::= NOT IN */ +#line 946 "sql.y" +{ yymsp[-1].minor.yy76 = OP_TYPE_NOT_IN; } +#line 7048 "sql.c" break; - case 493: /* in_predicate_value ::= NK_LP literal_list NK_RP */ -{ yylhsminor.yy452 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0, createNodeListNode(pCxt, yymsp[-1].minor.yy812)); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; + case 494: /* in_predicate_value ::= NK_LP literal_list NK_RP */ +#line 948 "sql.y" +{ yylhsminor.yy840 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0, createNodeListNode(pCxt, yymsp[-1].minor.yy56)); } +#line 7053 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 495: /* boolean_value_expression ::= NOT boolean_primary */ + case 496: /* boolean_value_expression ::= NOT boolean_primary */ +#line 952 "sql.y" { - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy452); - yylhsminor.yy452 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &e, createLogicConditionNode(pCxt, LOGIC_COND_TYPE_NOT, releaseRawExprNode(pCxt, yymsp[0].minor.yy452), NULL)); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy840); + yylhsminor.yy840 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &e, createLogicConditionNode(pCxt, LOGIC_COND_TYPE_NOT, releaseRawExprNode(pCxt, yymsp[0].minor.yy840), NULL)); } - yymsp[-1].minor.yy452 = yylhsminor.yy452; +#line 7062 "sql.c" + yymsp[-1].minor.yy840 = yylhsminor.yy840; break; - case 496: /* boolean_value_expression ::= boolean_value_expression OR boolean_value_expression */ + case 497: /* boolean_value_expression ::= boolean_value_expression OR boolean_value_expression */ +#line 957 "sql.y" { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy452); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy452); - yylhsminor.yy452 = createRawExprNodeExt(pCxt, &s, &e, createLogicConditionNode(pCxt, LOGIC_COND_TYPE_OR, releaseRawExprNode(pCxt, yymsp[-2].minor.yy452), releaseRawExprNode(pCxt, yymsp[0].minor.yy452))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy840); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy840); + yylhsminor.yy840 = createRawExprNodeExt(pCxt, &s, &e, createLogicConditionNode(pCxt, LOGIC_COND_TYPE_OR, releaseRawExprNode(pCxt, yymsp[-2].minor.yy840), releaseRawExprNode(pCxt, yymsp[0].minor.yy840))); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 7072 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 497: /* boolean_value_expression ::= boolean_value_expression AND boolean_value_expression */ + case 498: /* boolean_value_expression ::= boolean_value_expression AND boolean_value_expression */ +#line 963 "sql.y" { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy452); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy452); - yylhsminor.yy452 = createRawExprNodeExt(pCxt, &s, &e, createLogicConditionNode(pCxt, LOGIC_COND_TYPE_AND, releaseRawExprNode(pCxt, yymsp[-2].minor.yy452), releaseRawExprNode(pCxt, yymsp[0].minor.yy452))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy840); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy840); + yylhsminor.yy840 = createRawExprNodeExt(pCxt, &s, &e, createLogicConditionNode(pCxt, LOGIC_COND_TYPE_AND, releaseRawExprNode(pCxt, yymsp[-2].minor.yy840), releaseRawExprNode(pCxt, yymsp[0].minor.yy840))); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; +#line 7082 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 505: /* table_reference_list ::= table_reference_list NK_COMMA table_reference */ -{ yylhsminor.yy452 = createJoinTableNode(pCxt, JOIN_TYPE_INNER, yymsp[-2].minor.yy452, yymsp[0].minor.yy452, NULL); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; + case 506: /* table_reference_list ::= table_reference_list NK_COMMA table_reference */ +#line 981 "sql.y" +{ yylhsminor.yy840 = createJoinTableNode(pCxt, JOIN_TYPE_INNER, yymsp[-2].minor.yy840, yymsp[0].minor.yy840, NULL); } +#line 7088 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 508: /* table_primary ::= table_name alias_opt */ -{ yylhsminor.yy452 = createRealTableNode(pCxt, NULL, &yymsp[-1].minor.yy371, &yymsp[0].minor.yy371); } - yymsp[-1].minor.yy452 = yylhsminor.yy452; + case 509: /* table_primary ::= table_name alias_opt */ +#line 987 "sql.y" +{ yylhsminor.yy840 = createRealTableNode(pCxt, NULL, &yymsp[-1].minor.yy129, &yymsp[0].minor.yy129); } +#line 7094 "sql.c" + yymsp[-1].minor.yy840 = yylhsminor.yy840; break; - case 509: /* table_primary ::= db_name NK_DOT table_name alias_opt */ -{ yylhsminor.yy452 = createRealTableNode(pCxt, &yymsp[-3].minor.yy371, &yymsp[-1].minor.yy371, &yymsp[0].minor.yy371); } - yymsp[-3].minor.yy452 = yylhsminor.yy452; + case 510: /* table_primary ::= db_name NK_DOT table_name alias_opt */ +#line 988 "sql.y" +{ yylhsminor.yy840 = createRealTableNode(pCxt, &yymsp[-3].minor.yy129, &yymsp[-1].minor.yy129, &yymsp[0].minor.yy129); } +#line 7100 "sql.c" + yymsp[-3].minor.yy840 = yylhsminor.yy840; break; - case 510: /* table_primary ::= subquery alias_opt */ -{ yylhsminor.yy452 = createTempTableNode(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy452), &yymsp[0].minor.yy371); } - yymsp[-1].minor.yy452 = yylhsminor.yy452; + case 511: /* table_primary ::= subquery alias_opt */ +#line 989 "sql.y" +{ yylhsminor.yy840 = createTempTableNode(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy840), &yymsp[0].minor.yy129); } +#line 7106 "sql.c" + yymsp[-1].minor.yy840 = yylhsminor.yy840; break; - case 512: /* alias_opt ::= */ -{ yymsp[1].minor.yy371 = nil_token; } + case 513: /* alias_opt ::= */ +#line 994 "sql.y" +{ yymsp[1].minor.yy129 = nil_token; } +#line 7112 "sql.c" break; - case 514: /* alias_opt ::= AS table_alias */ -{ yymsp[-1].minor.yy371 = yymsp[0].minor.yy371; } + case 515: /* alias_opt ::= AS table_alias */ +#line 996 "sql.y" +{ yymsp[-1].minor.yy129 = yymsp[0].minor.yy129; } +#line 7117 "sql.c" break; - case 515: /* parenthesized_joined_table ::= NK_LP joined_table NK_RP */ - case 516: /* parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP */ yytestcase(yyruleno==516); -{ yymsp[-2].minor.yy452 = yymsp[-1].minor.yy452; } + case 516: /* parenthesized_joined_table ::= NK_LP joined_table NK_RP */ + case 517: /* parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP */ yytestcase(yyruleno==517); +#line 998 "sql.y" +{ yymsp[-2].minor.yy840 = yymsp[-1].minor.yy840; } +#line 7123 "sql.c" break; - case 517: /* joined_table ::= table_reference join_type JOIN table_reference ON search_condition */ -{ yylhsminor.yy452 = createJoinTableNode(pCxt, yymsp[-4].minor.yy140, yymsp[-5].minor.yy452, yymsp[-2].minor.yy452, yymsp[0].minor.yy452); } - yymsp[-5].minor.yy452 = yylhsminor.yy452; + case 518: /* joined_table ::= table_reference join_type JOIN table_reference ON search_condition */ +#line 1003 "sql.y" +{ yylhsminor.yy840 = createJoinTableNode(pCxt, yymsp[-4].minor.yy708, yymsp[-5].minor.yy840, yymsp[-2].minor.yy840, yymsp[0].minor.yy840); } +#line 7128 "sql.c" + yymsp[-5].minor.yy840 = yylhsminor.yy840; break; - case 518: /* join_type ::= */ -{ yymsp[1].minor.yy140 = JOIN_TYPE_INNER; } + case 519: /* join_type ::= */ +#line 1007 "sql.y" +{ yymsp[1].minor.yy708 = JOIN_TYPE_INNER; } +#line 7134 "sql.c" break; - case 519: /* join_type ::= INNER */ -{ yymsp[0].minor.yy140 = JOIN_TYPE_INNER; } + case 520: /* join_type ::= INNER */ +#line 1008 "sql.y" +{ yymsp[0].minor.yy708 = JOIN_TYPE_INNER; } +#line 7139 "sql.c" break; - case 520: /* query_specification ::= SELECT set_quantifier_opt select_list from_clause_opt where_clause_opt partition_by_clause_opt range_opt every_opt fill_opt twindow_clause_opt group_by_clause_opt having_clause_opt */ -{ - yymsp[-11].minor.yy452 = createSelectStmt(pCxt, yymsp[-10].minor.yy667, yymsp[-9].minor.yy812, yymsp[-8].minor.yy452); - yymsp[-11].minor.yy452 = addWhereClause(pCxt, yymsp[-11].minor.yy452, yymsp[-7].minor.yy452); - yymsp[-11].minor.yy452 = addPartitionByClause(pCxt, yymsp[-11].minor.yy452, yymsp[-6].minor.yy812); - yymsp[-11].minor.yy452 = addWindowClauseClause(pCxt, yymsp[-11].minor.yy452, yymsp[-2].minor.yy452); - yymsp[-11].minor.yy452 = addGroupByClause(pCxt, yymsp[-11].minor.yy452, yymsp[-1].minor.yy812); - yymsp[-11].minor.yy452 = addHavingClause(pCxt, yymsp[-11].minor.yy452, yymsp[0].minor.yy452); - yymsp[-11].minor.yy452 = addRangeClause(pCxt, yymsp[-11].minor.yy452, yymsp[-5].minor.yy452); - yymsp[-11].minor.yy452 = addEveryClause(pCxt, yymsp[-11].minor.yy452, yymsp[-4].minor.yy452); - yymsp[-11].minor.yy452 = addFillClause(pCxt, yymsp[-11].minor.yy452, yymsp[-3].minor.yy452); - } - break; - case 523: /* set_quantifier_opt ::= ALL */ -{ yymsp[0].minor.yy667 = false; } - break; - case 526: /* select_item ::= NK_STAR */ -{ yylhsminor.yy452 = createColumnNode(pCxt, NULL, &yymsp[0].minor.yy0); } - yymsp[0].minor.yy452 = yylhsminor.yy452; - break; - case 528: /* select_item ::= common_expression column_alias */ - case 538: /* partition_item ::= expr_or_subquery column_alias */ yytestcase(yyruleno==538); -{ yylhsminor.yy452 = setProjectionAlias(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy452), &yymsp[0].minor.yy371); } - yymsp[-1].minor.yy452 = yylhsminor.yy452; - break; - case 529: /* select_item ::= common_expression AS column_alias */ - case 539: /* partition_item ::= expr_or_subquery AS column_alias */ yytestcase(yyruleno==539); -{ yylhsminor.yy452 = setProjectionAlias(pCxt, releaseRawExprNode(pCxt, yymsp[-2].minor.yy452), &yymsp[0].minor.yy371); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; - break; - case 534: /* partition_by_clause_opt ::= PARTITION BY partition_list */ - case 559: /* group_by_clause_opt ::= GROUP BY group_by_list */ yytestcase(yyruleno==559); - case 579: /* order_by_clause_opt ::= ORDER BY sort_specification_list */ yytestcase(yyruleno==579); -{ yymsp[-2].minor.yy812 = yymsp[0].minor.yy812; } - break; - case 541: /* twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA duration_literal NK_RP */ -{ yymsp[-5].minor.yy452 = createSessionWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy452), releaseRawExprNode(pCxt, yymsp[-1].minor.yy452)); } - break; - case 542: /* twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP */ -{ yymsp[-3].minor.yy452 = createStateWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy452)); } - break; - case 543: /* twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_RP sliding_opt fill_opt */ -{ yymsp[-5].minor.yy452 = createIntervalWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy452), NULL, yymsp[-1].minor.yy452, yymsp[0].minor.yy452); } - break; - case 544: /* twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_COMMA duration_literal NK_RP sliding_opt fill_opt */ -{ yymsp[-7].minor.yy452 = createIntervalWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-5].minor.yy452), releaseRawExprNode(pCxt, yymsp[-3].minor.yy452), yymsp[-1].minor.yy452, yymsp[0].minor.yy452); } - break; - case 545: /* twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition */ -{ yymsp[-6].minor.yy452 = createEventWindowNode(pCxt, yymsp[-3].minor.yy452, yymsp[0].minor.yy452); } - break; - case 549: /* fill_opt ::= FILL NK_LP fill_mode NK_RP */ -{ yymsp[-3].minor.yy452 = createFillNode(pCxt, yymsp[-1].minor.yy844, NULL); } - break; - case 550: /* fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP */ -{ yymsp[-5].minor.yy452 = createFillNode(pCxt, FILL_MODE_VALUE, createNodeListNode(pCxt, yymsp[-1].minor.yy812)); } - break; - case 551: /* fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP */ -{ yymsp[-5].minor.yy452 = createFillNode(pCxt, FILL_MODE_VALUE_F, createNodeListNode(pCxt, yymsp[-1].minor.yy812)); } - break; - case 552: /* fill_mode ::= NONE */ -{ yymsp[0].minor.yy844 = FILL_MODE_NONE; } - break; - case 553: /* fill_mode ::= PREV */ -{ yymsp[0].minor.yy844 = FILL_MODE_PREV; } - break; - case 554: /* fill_mode ::= NULL */ -{ yymsp[0].minor.yy844 = FILL_MODE_NULL; } - break; - case 555: /* fill_mode ::= NULL_F */ -{ yymsp[0].minor.yy844 = FILL_MODE_NULL_F; } - break; - case 556: /* fill_mode ::= LINEAR */ -{ yymsp[0].minor.yy844 = FILL_MODE_LINEAR; } - break; - case 557: /* fill_mode ::= NEXT */ -{ yymsp[0].minor.yy844 = FILL_MODE_NEXT; } - break; - case 560: /* group_by_list ::= expr_or_subquery */ -{ yylhsminor.yy812 = createNodeList(pCxt, createGroupingSetNode(pCxt, releaseRawExprNode(pCxt, yymsp[0].minor.yy452))); } - yymsp[0].minor.yy812 = yylhsminor.yy812; - break; - case 561: /* group_by_list ::= group_by_list NK_COMMA expr_or_subquery */ -{ yylhsminor.yy812 = addNodeToList(pCxt, yymsp[-2].minor.yy812, createGroupingSetNode(pCxt, releaseRawExprNode(pCxt, yymsp[0].minor.yy452))); } - yymsp[-2].minor.yy812 = yylhsminor.yy812; - break; - case 565: /* range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP */ -{ yymsp[-5].minor.yy452 = createInterpTimeRange(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy452), releaseRawExprNode(pCxt, yymsp[-1].minor.yy452)); } - break; - case 566: /* range_opt ::= RANGE NK_LP expr_or_subquery NK_RP */ -{ yymsp[-3].minor.yy452 = createInterpTimePoint(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy452)); } - break; - case 569: /* query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt */ + case 521: /* query_specification ::= SELECT tag_mode_opt set_quantifier_opt select_list from_clause_opt where_clause_opt partition_by_clause_opt range_opt every_opt fill_opt twindow_clause_opt group_by_clause_opt having_clause_opt */ +#line 1014 "sql.y" { - yylhsminor.yy452 = addOrderByClause(pCxt, yymsp[-3].minor.yy452, yymsp[-2].minor.yy812); - yylhsminor.yy452 = addSlimitClause(pCxt, yylhsminor.yy452, yymsp[-1].minor.yy452); - yylhsminor.yy452 = addLimitClause(pCxt, yylhsminor.yy452, yymsp[0].minor.yy452); + yymsp[-12].minor.yy840 = createSelectStmt(pCxt, yymsp[-10].minor.yy313, yymsp[-9].minor.yy56, yymsp[-8].minor.yy840); + yymsp[-12].minor.yy840 = setSelectStmtTagMode(pCxt, yymsp[-12].minor.yy840, yymsp[-11].minor.yy313); + yymsp[-12].minor.yy840 = addWhereClause(pCxt, yymsp[-12].minor.yy840, yymsp[-7].minor.yy840); + yymsp[-12].minor.yy840 = addPartitionByClause(pCxt, yymsp[-12].minor.yy840, yymsp[-6].minor.yy56); + yymsp[-12].minor.yy840 = addWindowClauseClause(pCxt, yymsp[-12].minor.yy840, yymsp[-2].minor.yy840); + yymsp[-12].minor.yy840 = addGroupByClause(pCxt, yymsp[-12].minor.yy840, yymsp[-1].minor.yy56); + yymsp[-12].minor.yy840 = addHavingClause(pCxt, yymsp[-12].minor.yy840, yymsp[0].minor.yy840); + yymsp[-12].minor.yy840 = addRangeClause(pCxt, yymsp[-12].minor.yy840, yymsp[-5].minor.yy840); + yymsp[-12].minor.yy840 = addEveryClause(pCxt, yymsp[-12].minor.yy840, yymsp[-4].minor.yy840); + yymsp[-12].minor.yy840 = addFillClause(pCxt, yymsp[-12].minor.yy840, yymsp[-3].minor.yy840); } - yymsp[-3].minor.yy452 = yylhsminor.yy452; +#line 7155 "sql.c" break; - case 572: /* union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery */ -{ yylhsminor.yy452 = createSetOperator(pCxt, SET_OP_TYPE_UNION_ALL, yymsp[-3].minor.yy452, yymsp[0].minor.yy452); } - yymsp[-3].minor.yy452 = yylhsminor.yy452; + case 526: /* set_quantifier_opt ::= ALL */ +#line 1036 "sql.y" +{ yymsp[0].minor.yy313 = false; } +#line 7160 "sql.c" break; - case 573: /* union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery */ -{ yylhsminor.yy452 = createSetOperator(pCxt, SET_OP_TYPE_UNION, yymsp[-2].minor.yy452, yymsp[0].minor.yy452); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; + case 529: /* select_item ::= NK_STAR */ +#line 1043 "sql.y" +{ yylhsminor.yy840 = createColumnNode(pCxt, NULL, &yymsp[0].minor.yy0); } +#line 7165 "sql.c" + yymsp[0].minor.yy840 = yylhsminor.yy840; break; - case 581: /* slimit_clause_opt ::= SLIMIT NK_INTEGER */ - case 585: /* limit_clause_opt ::= LIMIT NK_INTEGER */ yytestcase(yyruleno==585); -{ yymsp[-1].minor.yy452 = createLimitNode(pCxt, &yymsp[0].minor.yy0, NULL); } + case 531: /* select_item ::= common_expression column_alias */ + case 541: /* partition_item ::= expr_or_subquery column_alias */ yytestcase(yyruleno==541); +#line 1045 "sql.y" +{ yylhsminor.yy840 = setProjectionAlias(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy840), &yymsp[0].minor.yy129); } +#line 7172 "sql.c" + yymsp[-1].minor.yy840 = yylhsminor.yy840; break; - case 582: /* slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER */ - case 586: /* limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER */ yytestcase(yyruleno==586); -{ yymsp[-3].minor.yy452 = createLimitNode(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0); } + case 532: /* select_item ::= common_expression AS column_alias */ + case 542: /* partition_item ::= expr_or_subquery AS column_alias */ yytestcase(yyruleno==542); +#line 1046 "sql.y" +{ yylhsminor.yy840 = setProjectionAlias(pCxt, releaseRawExprNode(pCxt, yymsp[-2].minor.yy840), &yymsp[0].minor.yy129); } +#line 7179 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; break; - case 583: /* slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER */ - case 587: /* limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER */ yytestcase(yyruleno==587); -{ yymsp[-3].minor.yy452 = createLimitNode(pCxt, &yymsp[0].minor.yy0, &yymsp[-2].minor.yy0); } + case 537: /* partition_by_clause_opt ::= PARTITION BY partition_list */ + case 562: /* group_by_clause_opt ::= GROUP BY group_by_list */ yytestcase(yyruleno==562); + case 582: /* order_by_clause_opt ::= ORDER BY sort_specification_list */ yytestcase(yyruleno==582); +#line 1055 "sql.y" +{ yymsp[-2].minor.yy56 = yymsp[0].minor.yy56; } +#line 7187 "sql.c" break; - case 588: /* subquery ::= NK_LP query_expression NK_RP */ -{ yylhsminor.yy452 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0, yymsp[-1].minor.yy452); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; + case 544: /* twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA duration_literal NK_RP */ +#line 1068 "sql.y" +{ yymsp[-5].minor.yy840 = createSessionWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy840), releaseRawExprNode(pCxt, yymsp[-1].minor.yy840)); } +#line 7192 "sql.c" break; - case 593: /* sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt */ -{ yylhsminor.yy452 = createOrderByExprNode(pCxt, releaseRawExprNode(pCxt, yymsp[-2].minor.yy452), yymsp[-1].minor.yy690, yymsp[0].minor.yy399); } - yymsp[-2].minor.yy452 = yylhsminor.yy452; + case 545: /* twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP */ +#line 1069 "sql.y" +{ yymsp[-3].minor.yy840 = createStateWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy840)); } +#line 7197 "sql.c" break; - case 594: /* ordering_specification_opt ::= */ -{ yymsp[1].minor.yy690 = ORDER_ASC; } + case 546: /* twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_RP sliding_opt fill_opt */ +#line 1071 "sql.y" +{ yymsp[-5].minor.yy840 = createIntervalWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy840), NULL, yymsp[-1].minor.yy840, yymsp[0].minor.yy840); } +#line 7202 "sql.c" break; - case 595: /* ordering_specification_opt ::= ASC */ -{ yymsp[0].minor.yy690 = ORDER_ASC; } + case 547: /* twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_COMMA duration_literal NK_RP sliding_opt fill_opt */ +#line 1074 "sql.y" +{ yymsp[-7].minor.yy840 = createIntervalWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-5].minor.yy840), releaseRawExprNode(pCxt, yymsp[-3].minor.yy840), yymsp[-1].minor.yy840, yymsp[0].minor.yy840); } +#line 7207 "sql.c" break; - case 596: /* ordering_specification_opt ::= DESC */ -{ yymsp[0].minor.yy690 = ORDER_DESC; } + case 548: /* twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition */ +#line 1076 "sql.y" +{ yymsp[-6].minor.yy840 = createEventWindowNode(pCxt, yymsp[-3].minor.yy840, yymsp[0].minor.yy840); } +#line 7212 "sql.c" break; - case 597: /* null_ordering_opt ::= */ -{ yymsp[1].minor.yy399 = NULL_ORDER_DEFAULT; } + case 552: /* fill_opt ::= FILL NK_LP fill_mode NK_RP */ +#line 1082 "sql.y" +{ yymsp[-3].minor.yy840 = createFillNode(pCxt, yymsp[-1].minor.yy694, NULL); } +#line 7217 "sql.c" break; - case 598: /* null_ordering_opt ::= NULLS FIRST */ -{ yymsp[-1].minor.yy399 = NULL_ORDER_FIRST; } + case 553: /* fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP */ +#line 1083 "sql.y" +{ yymsp[-5].minor.yy840 = createFillNode(pCxt, FILL_MODE_VALUE, createNodeListNode(pCxt, yymsp[-1].minor.yy56)); } +#line 7222 "sql.c" break; - case 599: /* null_ordering_opt ::= NULLS LAST */ -{ yymsp[-1].minor.yy399 = NULL_ORDER_LAST; } + case 554: /* fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP */ +#line 1084 "sql.y" +{ yymsp[-5].minor.yy840 = createFillNode(pCxt, FILL_MODE_VALUE_F, createNodeListNode(pCxt, yymsp[-1].minor.yy56)); } +#line 7227 "sql.c" + break; + case 555: /* fill_mode ::= NONE */ +#line 1088 "sql.y" +{ yymsp[0].minor.yy694 = FILL_MODE_NONE; } +#line 7232 "sql.c" + break; + case 556: /* fill_mode ::= PREV */ +#line 1089 "sql.y" +{ yymsp[0].minor.yy694 = FILL_MODE_PREV; } +#line 7237 "sql.c" + break; + case 557: /* fill_mode ::= NULL */ +#line 1090 "sql.y" +{ yymsp[0].minor.yy694 = FILL_MODE_NULL; } +#line 7242 "sql.c" + break; + case 558: /* fill_mode ::= NULL_F */ +#line 1091 "sql.y" +{ yymsp[0].minor.yy694 = FILL_MODE_NULL_F; } +#line 7247 "sql.c" + break; + case 559: /* fill_mode ::= LINEAR */ +#line 1092 "sql.y" +{ yymsp[0].minor.yy694 = FILL_MODE_LINEAR; } +#line 7252 "sql.c" + break; + case 560: /* fill_mode ::= NEXT */ +#line 1093 "sql.y" +{ yymsp[0].minor.yy694 = FILL_MODE_NEXT; } +#line 7257 "sql.c" + break; + case 563: /* group_by_list ::= expr_or_subquery */ +#line 1102 "sql.y" +{ yylhsminor.yy56 = createNodeList(pCxt, createGroupingSetNode(pCxt, releaseRawExprNode(pCxt, yymsp[0].minor.yy840))); } +#line 7262 "sql.c" + yymsp[0].minor.yy56 = yylhsminor.yy56; + break; + case 564: /* group_by_list ::= group_by_list NK_COMMA expr_or_subquery */ +#line 1103 "sql.y" +{ yylhsminor.yy56 = addNodeToList(pCxt, yymsp[-2].minor.yy56, createGroupingSetNode(pCxt, releaseRawExprNode(pCxt, yymsp[0].minor.yy840))); } +#line 7268 "sql.c" + yymsp[-2].minor.yy56 = yylhsminor.yy56; + break; + case 568: /* range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP */ +#line 1110 "sql.y" +{ yymsp[-5].minor.yy840 = createInterpTimeRange(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy840), releaseRawExprNode(pCxt, yymsp[-1].minor.yy840)); } +#line 7274 "sql.c" + break; + case 569: /* range_opt ::= RANGE NK_LP expr_or_subquery NK_RP */ +#line 1112 "sql.y" +{ yymsp[-3].minor.yy840 = createInterpTimePoint(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy840)); } +#line 7279 "sql.c" + break; + case 572: /* query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt */ +#line 1119 "sql.y" +{ + yylhsminor.yy840 = addOrderByClause(pCxt, yymsp[-3].minor.yy840, yymsp[-2].minor.yy56); + yylhsminor.yy840 = addSlimitClause(pCxt, yylhsminor.yy840, yymsp[-1].minor.yy840); + yylhsminor.yy840 = addLimitClause(pCxt, yylhsminor.yy840, yymsp[0].minor.yy840); + } +#line 7288 "sql.c" + yymsp[-3].minor.yy840 = yylhsminor.yy840; + break; + case 575: /* union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery */ +#line 1129 "sql.y" +{ yylhsminor.yy840 = createSetOperator(pCxt, SET_OP_TYPE_UNION_ALL, yymsp[-3].minor.yy840, yymsp[0].minor.yy840); } +#line 7294 "sql.c" + yymsp[-3].minor.yy840 = yylhsminor.yy840; + break; + case 576: /* union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery */ +#line 1131 "sql.y" +{ yylhsminor.yy840 = createSetOperator(pCxt, SET_OP_TYPE_UNION, yymsp[-2].minor.yy840, yymsp[0].minor.yy840); } +#line 7300 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; + break; + case 584: /* slimit_clause_opt ::= SLIMIT NK_INTEGER */ + case 588: /* limit_clause_opt ::= LIMIT NK_INTEGER */ yytestcase(yyruleno==588); +#line 1145 "sql.y" +{ yymsp[-1].minor.yy840 = createLimitNode(pCxt, &yymsp[0].minor.yy0, NULL); } +#line 7307 "sql.c" + break; + case 585: /* slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER */ + case 589: /* limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER */ yytestcase(yyruleno==589); +#line 1146 "sql.y" +{ yymsp[-3].minor.yy840 = createLimitNode(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0); } +#line 7313 "sql.c" + break; + case 586: /* slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER */ + case 590: /* limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER */ yytestcase(yyruleno==590); +#line 1147 "sql.y" +{ yymsp[-3].minor.yy840 = createLimitNode(pCxt, &yymsp[0].minor.yy0, &yymsp[-2].minor.yy0); } +#line 7319 "sql.c" + break; + case 591: /* subquery ::= NK_LP query_expression NK_RP */ +#line 1155 "sql.y" +{ yylhsminor.yy840 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0, yymsp[-1].minor.yy840); } +#line 7324 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; + break; + case 596: /* sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt */ +#line 1169 "sql.y" +{ yylhsminor.yy840 = createOrderByExprNode(pCxt, releaseRawExprNode(pCxt, yymsp[-2].minor.yy840), yymsp[-1].minor.yy82, yymsp[0].minor.yy521); } +#line 7330 "sql.c" + yymsp[-2].minor.yy840 = yylhsminor.yy840; + break; + case 597: /* ordering_specification_opt ::= */ +#line 1173 "sql.y" +{ yymsp[1].minor.yy82 = ORDER_ASC; } +#line 7336 "sql.c" + break; + case 598: /* ordering_specification_opt ::= ASC */ +#line 1174 "sql.y" +{ yymsp[0].minor.yy82 = ORDER_ASC; } +#line 7341 "sql.c" + break; + case 599: /* ordering_specification_opt ::= DESC */ +#line 1175 "sql.y" +{ yymsp[0].minor.yy82 = ORDER_DESC; } +#line 7346 "sql.c" + break; + case 600: /* null_ordering_opt ::= */ +#line 1179 "sql.y" +{ yymsp[1].minor.yy521 = NULL_ORDER_DEFAULT; } +#line 7351 "sql.c" + break; + case 601: /* null_ordering_opt ::= NULLS FIRST */ +#line 1180 "sql.y" +{ yymsp[-1].minor.yy521 = NULL_ORDER_FIRST; } +#line 7356 "sql.c" + break; + case 602: /* null_ordering_opt ::= NULLS LAST */ +#line 1181 "sql.y" +{ yymsp[-1].minor.yy521 = NULL_ORDER_LAST; } +#line 7361 "sql.c" break; default: break; @@ -6233,6 +7420,7 @@ static void yy_syntax_error( ParseCTX_FETCH #define TOKEN yyminor /************ Begin %syntax_error code ****************************************/ +#line 29 "sql.y" if (TSDB_CODE_SUCCESS == pCxt->errCode) { if(TOKEN.z) { @@ -6243,6 +7431,7 @@ static void yy_syntax_error( } else if (TSDB_CODE_PAR_DB_NOT_SPECIFIED == pCxt->errCode && TK_NK_FLOAT == TOKEN.type) { pCxt->errCode = generateSyntaxErrMsg(&pCxt->msgBuf, TSDB_CODE_PAR_SYNTAX_ERROR, TOKEN.z); } +#line 7434 "sql.c" /************ End %syntax_error code ******************************************/ ParseARG_STORE /* Suppress warning about unused %extra_argument variable */ ParseCTX_STORE @@ -6328,12 +7517,56 @@ void Parse( } #endif - do{ + while(1){ /* Exit by "break" */ + assert( yypParser->yytos>=yypParser->yystack ); assert( yyact==yypParser->yytos->stateno ); yyact = yy_find_shift_action((YYCODETYPE)yymajor,yyact); if( yyact >= YY_MIN_REDUCE ){ - yyact = yy_reduce(yypParser,yyact-YY_MIN_REDUCE,yymajor, - yyminor ParseCTX_PARAM); + unsigned int yyruleno = yyact - YY_MIN_REDUCE; /* Reduce by this rule */ +#ifndef NDEBUG + assert( yyruleno<(int)(sizeof(yyRuleName)/sizeof(yyRuleName[0])) ); + if( yyTraceFILE ){ + int yysize = yyRuleInfoNRhs[yyruleno]; + if( yysize ){ + fprintf(yyTraceFILE, "%sReduce %d [%s]%s, pop back to state %d.\n", + yyTracePrompt, + yyruleno, yyRuleName[yyruleno], + yyrulenoyytos[yysize].stateno); + }else{ + fprintf(yyTraceFILE, "%sReduce %d [%s]%s.\n", + yyTracePrompt, yyruleno, yyRuleName[yyruleno], + yyrulenoyytos - yypParser->yystack)>yypParser->yyhwm ){ + yypParser->yyhwm++; + assert( yypParser->yyhwm == + (int)(yypParser->yytos - yypParser->yystack)); + } +#endif +#if YYSTACKDEPTH>0 + if( yypParser->yytos>=yypParser->yystackEnd ){ + yyStackOverflow(yypParser); + break; + } +#else + if( yypParser->yytos>=&yypParser->yystack[yypParser->yystksz-1] ){ + if( yyGrowStack(yypParser) ){ + yyStackOverflow(yypParser); + break; + } + } +#endif + } + yyact = yy_reduce(yypParser,yyruleno,yymajor,yyminor ParseCTX_PARAM); }else if( yyact <= YY_MAX_SHIFTREDUCE ){ yy_shift(yypParser,yyact,(YYCODETYPE)yymajor,yyminor); #ifndef YYNOERRORRECOVERY @@ -6389,14 +7622,13 @@ void Parse( yy_destructor(yypParser, (YYCODETYPE)yymajor, &yyminorunion); yymajor = YYNOCODE; }else{ - while( yypParser->yytos >= yypParser->yystack - && (yyact = yy_find_reduce_action( - yypParser->yytos->stateno, - YYERRORSYMBOL)) > YY_MAX_SHIFTREDUCE - ){ + while( yypParser->yytos > yypParser->yystack ){ + yyact = yy_find_reduce_action(yypParser->yytos->stateno, + YYERRORSYMBOL); + if( yyact<=YY_MAX_SHIFTREDUCE ) break; yy_pop_parser_stack(yypParser); } - if( yypParser->yytos < yypParser->yystack || yymajor==0 ){ + if( yypParser->yytos <= yypParser->yystack || yymajor==0 ){ yy_destructor(yypParser,(YYCODETYPE)yymajor,&yyminorunion); yy_parse_failed(yypParser); #ifndef YYNOERRORRECOVERY @@ -6446,7 +7678,7 @@ void Parse( break; #endif } - }while( yypParser->yytos>yypParser->yystack ); + } #ifndef NDEBUG if( yyTraceFILE ){ yyStackEntry *i; diff --git a/source/libs/parser/test/parShowToUse.cpp b/source/libs/parser/test/parShowToUse.cpp index b7bd0e802c..3e1482d420 100644 --- a/source/libs/parser/test/parShowToUse.cpp +++ b/source/libs/parser/test/parShowToUse.cpp @@ -239,9 +239,9 @@ TEST_F(ParserShowToUseTest, showVgroups) { TEST_F(ParserShowToUseTest, showVnodes) { useDb("root", "test"); - run("SHOW VNODES 1"); + run("SHOW VNODES ON DNODE 1"); - run("SHOW VNODES 'node1:7030'"); + run("SHOW VNODES"); } TEST_F(ParserShowToUseTest, splitVgroup) { diff --git a/source/libs/planner/src/planLogicCreater.c b/source/libs/planner/src/planLogicCreater.c index 9345834ec0..a822c7f9fe 100644 --- a/source/libs/planner/src/planLogicCreater.c +++ b/source/libs/planner/src/planLogicCreater.c @@ -14,7 +14,7 @@ */ #include "planInt.h" - +#include "filter.h" #include "functionMgt.h" typedef struct SLogicPlanContext { @@ -253,7 +253,7 @@ static EScanType getScanType(SLogicPlanContext* pCxt, SNodeList* pScanPseudoCols return SCAN_TYPE_SYSTEM_TABLE; } - if (tagScan) { + if (tagScan && 0 == LIST_LENGTH(pScanCols) && 0 != LIST_LENGTH(pScanPseudoCols)) { return SCAN_TYPE_TAG; } @@ -344,6 +344,55 @@ static int32_t makeScanLogicNode(SLogicPlanContext* pCxt, SRealTableNode* pRealT static bool needScanDefaultCol(EScanType scanType) { return SCAN_TYPE_TABLE_COUNT != scanType; } +static EDealRes tagScanNodeHasTbnameFunc(SNode* pNode, void* pContext) { + if (QUERY_NODE_FUNCTION == nodeType(pNode) && FUNCTION_TYPE_TBNAME == ((SFunctionNode*)pNode)->funcType || + (QUERY_NODE_COLUMN == nodeType(pNode) && COLUMN_TYPE_TBNAME == ((SColumnNode*)pNode)->colType)) { + *(bool*)pContext = true; + return DEAL_RES_END; + } + return DEAL_RES_CONTINUE; +} + +static bool tagScanNodeListHasTbname(SNodeList* pCols) { + bool hasTbname = false; + nodesWalkExprs(pCols, tagScanNodeHasTbnameFunc, &hasTbname); + return hasTbname; +} + +static bool tagScanNodeHasTbname(SNode* pKeys) { + bool hasTbname = false; + nodesWalkExpr(pKeys, tagScanNodeHasTbnameFunc, &hasTbname); + return hasTbname; +} + +static int32_t tagScanSetExecutionMode(SScanLogicNode* pScan) { + pScan->onlyMetaCtbIdx = false; + + if (tagScanNodeListHasTbname(pScan->pScanPseudoCols)) { + pScan->onlyMetaCtbIdx = false; + return TSDB_CODE_SUCCESS; + } + + if (pScan->node.pConditions == NULL) { + pScan->onlyMetaCtbIdx = true; + return TSDB_CODE_SUCCESS; + } + + SNode* pCond = nodesCloneNode(pScan->node.pConditions); + SNode* pTagCond = NULL; + SNode* pTagIndexCond = NULL; + filterPartitionCond(&pCond, NULL, &pTagIndexCond, &pTagCond, NULL); + if (pTagIndexCond || tagScanNodeHasTbname(pTagCond)) { + pScan->onlyMetaCtbIdx = false; + } else { + pScan->onlyMetaCtbIdx = true; + } + nodesDestroyNode(pCond); + nodesDestroyNode(pTagIndexCond); + nodesDestroyNode(pTagCond); + return TSDB_CODE_SUCCESS; +} + static int32_t createScanLogicNode(SLogicPlanContext* pCxt, SSelectStmt* pSelect, SRealTableNode* pRealTable, SLogicNode** pLogicNode) { SScanLogicNode* pScan = NULL; @@ -411,6 +460,10 @@ static int32_t createScanLogicNode(SLogicPlanContext* pCxt, SSelectStmt* pSelect code = createColumnByRewriteExprs(pScan->pScanPseudoCols, &pScan->node.pTargets); } + if (pScan->scanType == SCAN_TYPE_TAG) { + code = tagScanSetExecutionMode(pScan); + } + if (TSDB_CODE_SUCCESS == code) { *pLogicNode = (SLogicNode*)pScan; } else { diff --git a/source/libs/planner/src/planOptimizer.c b/source/libs/planner/src/planOptimizer.c index 69a7b0cf87..2d21359c62 100644 --- a/source/libs/planner/src/planOptimizer.c +++ b/source/libs/planner/src/planOptimizer.c @@ -1563,7 +1563,8 @@ static bool planOptNodeListHasTbname(SNodeList* pKeys) { static bool partTagsIsOptimizableNode(SLogicNode* pNode) { bool ret = 1 == LIST_LENGTH(pNode->pChildren) && - QUERY_NODE_LOGIC_PLAN_SCAN == nodeType(nodesListGetNode(pNode->pChildren, 0)); + QUERY_NODE_LOGIC_PLAN_SCAN == nodeType(nodesListGetNode(pNode->pChildren, 0)) && + SCAN_TYPE_TAG != ((SScanLogicNode*)nodesListGetNode(pNode->pChildren, 0))->scanType; if (!ret) return ret; switch (nodeType(pNode)) { case QUERY_NODE_LOGIC_PLAN_PARTITION: { diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index cab6c240ab..e92327ffb0 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -16,9 +16,9 @@ #include "streamInt.h" // maximum allowed processed block batches. One block may include several submit blocks -#define MAX_STREAM_EXEC_BATCH_NUM 32 -#define MIN_STREAM_EXEC_BATCH_NUM 4 -#define MAX_STREAM_RESULT_DUMP_THRESHOLD 100 +#define MAX_STREAM_EXEC_BATCH_NUM 32 +#define MIN_STREAM_EXEC_BATCH_NUM 4 +#define MAX_STREAM_RESULT_DUMP_THRESHOLD 100 static int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask); @@ -61,7 +61,8 @@ static int32_t doDumpResult(SStreamTask* pTask, SStreamQueueItem* pItem, SArray* return TSDB_CODE_SUCCESS; } -static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, int64_t* totalSize, int32_t* totalBlocks) { +static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, int64_t* totalSize, + int32_t* totalBlocks) { int32_t code = TSDB_CODE_SUCCESS; void* pExecutor = pTask->exec.pExecutor; @@ -118,7 +119,7 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, i } continue; } else if (output->info.type == STREAM_CHECKPOINT) { - continue; // checkpoint block not dispatch to downstream tasks + continue; // checkpoint block not dispatch to downstream tasks } SSDataBlock block = {0}; @@ -354,7 +355,8 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { // 7. pause allowed. streamTaskEnablePause(pStreamTask); if (taosQueueEmpty(pStreamTask->inputQueue->queue)) { - SStreamRefDataBlock* pItem = taosAllocateQitem(sizeof(SStreamRefDataBlock), DEF_QITEM, 0);; + SStreamRefDataBlock* pItem = taosAllocateQitem(sizeof(SStreamRefDataBlock), DEF_QITEM, 0); + ; SSDataBlock* pDelBlock = createSpecialDataBlock(STREAM_DELETE_DATA); pDelBlock->info.rows = 0; pDelBlock->info.version = 0; @@ -473,7 +475,7 @@ int32_t streamProcessTranstateBlock(SStreamTask* pTask, SStreamDataBlock* pBlock } else { // level == TASK_LEVEL__SINK streamFreeQitem((SStreamQueueItem*)pBlock); } - } else { // non-dispatch task, do task state transfer directly + } else { // non-dispatch task, do task state transfer directly streamFreeQitem((SStreamQueueItem*)pBlock); if (level != TASK_LEVEL__SINK) { qDebug("s-task:%s non-dispatch task, start to transfer state directly", id); @@ -495,7 +497,7 @@ int32_t streamProcessTranstateBlock(SStreamTask* pTask, SStreamDataBlock* pBlock // * todo: the batch of blocks should be tuned dynamic, according to the total elapsed time of each batch of blocks, the // * appropriate batch of blocks should be handled in 5 to 10 sec. // */ -//int32_t streamExecForAll(SStreamTask* pTask) { +// int32_t streamExecForAll(SStreamTask* pTask) { // const char* id = pTask->id.idStr; // // while (1) { @@ -554,7 +556,7 @@ int32_t streamExecForAll(SStreamTask* pTask) { // merge multiple input data if possible in the input queue. qDebug("s-task:%s start to extract data block from inputQ", id); - /*int32_t code = */extractBlocksFromInputQ(pTask, &pInput, &numOfBlocks); + /*int32_t code = */ extractBlocksFromInputQ(pTask, &pInput, &numOfBlocks); if (pInput == NULL) { ASSERT(numOfBlocks == 0); return 0; diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 9a33e7a1e1..6cfd646f06 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -136,11 +136,11 @@ SStreamQueueItem* doReadMultiBlocksFromQueue(SQueueReader* pReader, const char* if (pReader->taskLevel == TASK_LEVEL__SOURCE && numOfBlocks < MIN_STREAM_EXEC_BATCH_NUM && tryCount < pReader->waitDuration) { tryCount++; taosMsleep(1); - qDebug("===stream===try again batchSize:%d", numOfBlocks); + qDebug("try again batchSize:%d", numOfBlocks); continue; } - qDebug("===stream===break batchSize:%d", numOfBlocks); + qDebug("break batchSize:%d", numOfBlocks); break; } diff --git a/source/libs/sync/src/syncMain.c b/source/libs/sync/src/syncMain.c index f1e3c35a49..9108090b87 100644 --- a/source/libs/sync/src/syncMain.c +++ b/source/libs/sync/src/syncMain.c @@ -509,6 +509,7 @@ SSyncState syncGetState(int64_t rid) { if (pSyncNode != NULL) { state.state = pSyncNode->state; state.roleTimeMs = pSyncNode->roleTimeMs; + state.startTimeMs = pSyncNode->startTime; state.restored = pSyncNode->restoreFinish; if (pSyncNode->vgId != 1) { state.canRead = syncNodeIsReadyForRead(pSyncNode); diff --git a/tests/script/tsim/db/basic1.sim b/tests/script/tsim/db/basic1.sim index 5d7d0bd9e9..8eb6dce759 100644 --- a/tests/script/tsim/db/basic1.sim +++ b/tests/script/tsim/db/basic1.sim @@ -107,35 +107,39 @@ if $data30 != 12 then return -1 endi -print =============== show vnodes -sql show vnodes 1 +print =============== show vnodes on dnode 1 +sql show vnodes on dnode 1 if $rows != 9 then return -1 endi -if $data(4)[1] != 1 then - return -1 +if $data10 != 1 then + return -1 endi -if $data(4)[2] != leader then - return -1 +if $data11 != 5 then + return -1 endi -if $data(4)[3] != d2 then - return -1 +if $data12 != d2 then + return -1 endi -if $data(4)[4] != 1 then - return -1 +if $data13 != leader then + return -1 endi -if $data(4)[5] != localhost:7100 then - return -1 +print $data14 +print $data15 + +if $data16 != 1 then + return -1 endi -sql show vnodes 'localhost:7100' +print ================ show vnodes +sql show vnodes if $rows != 9 then - return -1 + return -1 endi print =============== drop database diff --git a/tests/script/tsim/query/tag_scan.sim b/tests/script/tsim/query/tag_scan.sim index 03e3a20632..3a64cf1a1c 100644 --- a/tests/script/tsim/query/tag_scan.sim +++ b/tests/script/tsim/query/tag_scan.sim @@ -45,4 +45,86 @@ if $rows != 4 then return -1 endi +sql create table stt1(ts timestamp, f int) tags (t int, b varchar(10)); +sql insert into ctt11 using stt1 tags(1, '1aa') values(now, 1); +sql insert into ctt12 using stt1 tags(2, '1bb') values(now, 2); +sql insert into ctt13 using stt1 tags(3, '1cc') values(now, 3); +sql insert into ctt14 using stt1 tags(4, '1dd') values(now, 4); +sql insert into ctt14 values(now, 5); + +sql create table stt2(ts timestamp, f int) tags (t int, b varchar(10)); +sql insert into ctt21 using stt2 tags(1, '2aa') values(now, 1); +sql insert into ctt22 using stt2 tags(2, '2bb') values(now, 2); +sql insert into ctt23 using stt2 tags(3, '2cc') values(now, 3); +sql insert into ctt24 using stt2 tags(4, '2dd') values(now, 4); + +sql select tags t, b from stt1 order by t +print $rows +print $data00 $data01 $data10 $data11 $data20 $data21 $data30 $data31 +if $rows != 4 then + return -1 +endi +if $data31 != @1dd@ then + return -1 +endi + +sql select tags t, b from stt2 order by t +print $rows +print $data00 $data01 $data10 $data11 $data20 $data21 $data30 $data31 +if $rows != 4 then + return -1 +endi +if $data31 != @2dd@ then + return -1 +endi + +sql select tags t,b,f from stt1 order by t +print $rows +print $data00 $data01 $data02 $data10 $data11 $data12 $data20 $data21 $data22 $data30 $data31 $data32 $data40 $data41 $data42 +if $rows != 5 then + return -1 +endi +if $data42 != 5 then + return -1 +endi + +sql select tags tbname,t,b from stt1 order by t +print $rows +print $data00 $data01 $data02 $data10 $data11 $data12 $data20 $data21 $data22 $data30 $data31 $data32 +if $rows != 4 then + return -1 +endi +if $data30 != @ctt14@ then + return -1 +endi +if $data32 != @1dd@ then + return -1 +endi + +sql select tags t,b from stt1 where t=1 +print $rows +print $data00 $data01 +if $rows != 1 then + return -1 +endi +if $data00 != @1@ then + return -1 +endi +if $data01 != @1aa@ then + return -1 +endi + +sql select tags t,b from stt1 where tbname='ctt11' +print $rows +print $data00 $data01 +if $rows != 1 then + return -1 +endi +if $data00 != @1@ then + return -1 +endi +if $data01 != @1aa@ then + return -1 +endi + system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/system-test/0-others/show.py b/tests/system-test/0-others/show.py index 9d26b3a2ae..d9cc4895b1 100644 --- a/tests/system-test/0-others/show.py +++ b/tests/system-test/0-others/show.py @@ -128,7 +128,7 @@ class TDTestCase: continue else: tdLog.exit(f"show create database check failed with {key} {value}") - tdSql.query('show vnodes 1') + tdSql.query('show vnodes on dnode 1') tdSql.checkRows(self.vgroups) tdSql.execute(f'use {self.dbname}') @@ -210,6 +210,66 @@ class TDTestCase: licences_info = tdSql.queryResult tdSql.checkEqual(grants_info,licences_info) + def show_create_table_with_col_comment(self): + tdSql.execute("create database comment_test_db") + tdSql.execute("use comment_test_db") + tdSql.execute("create table normal_table(ts timestamp, c2 int comment 'c2 comment')") + tdSql.execute("create stable super_table(ts timestamp comment 'ts', c2 int comment 'c2 comment') tags(tg int comment 'tg comment')") + + create_sql = "create table `normal_table` (`ts` timestamp, `c2` int)" + tdSql.query('show create table normal_table') + tdSql.checkEqual(tdSql.queryResult[0][1].lower(), create_sql) + tdSql.query('show create table super_table') + create_sql = "create stable `super_table` (`ts` timestamp, `c2` int) tags (`tg` int)" + tdSql.checkEqual(tdSql.queryResult[0][1].lower(), create_sql) + + tdSql.query("desc normal_table") + tdSql.checkCols(5) + tdSql.checkData(0, 4, "") + + tdSql.query("desc super_table") + tdSql.checkCols(5) + tdSql.checkData(0, 4, "") + + tdSql.execute("drop database comment_test_db") + + def alter_table_with_col_comment(self): + tdSql.execute("create database comment_test_db") + tdSql.execute("use comment_test_db") + tdSql.execute("create table normal_table(ts timestamp, c2 int comment 'c2 comment')") + tdSql.execute("create stable super_table(ts timestamp comment 'ts', c2 int comment 'c2 comment') tags(tg int comment 'tg comment')") + + create_sql = "create table `normal_table` (`ts` timestamp, `c2` int, `c3` int)" + tdSql.execute("alter table normal_table add column c3 int comment 'c3 comment'", queryTimes=1) + tdSql.query("show create table normal_table") + tdSql.checkEqual(tdSql.queryResult[0][1].lower(), create_sql) + + create_sql = "create table `normal_table` (`ts` timestamp, `c2` int, `c3` int, `c4` varchar(255))" + tdSql.execute("alter table normal_table add column c4 varchar(255) comment 'c4 comment'", queryTimes=1) + tdSql.query("show create table normal_table") + tdSql.checkEqual(tdSql.queryResult[0][1].lower(), create_sql) + + create_sql = "create table `normal_table` (`ts` timestamp, `c2` int, `c3` int, `c4` varchar(255), `c5` varchar(255))" + tdSql.execute("alter table normal_table add column c5 varchar(255)", queryTimes=1) + tdSql.query("show create table normal_table") + tdSql.checkEqual(tdSql.queryResult[0][1].lower(), create_sql) + + create_sql = "create stable `super_table` (`ts` timestamp, `c2` int, `c3` int) tags (`tg` int) sma(`ts`,`c2`)" + tdSql.execute("alter table super_table add column c3 int comment 'c3 comment'", queryTimes=1) + tdSql.query("show create table super_table") + tdSql.checkEqual(tdSql.queryResult[0][1].lower(), create_sql) + + create_sql = "create stable `super_table` (`ts` timestamp, `c2` int, `c3` int, `c4` varchar(255)) tags (`tg` int) sma(`ts`,`c2`)" + tdSql.execute("alter table super_table add column c4 varchar(255) comment 'c4 comment'", queryTimes=1) + tdSql.query("show create table super_table") + tdSql.checkEqual(tdSql.queryResult[0][1].lower(), create_sql) + + create_sql = "create stable `super_table` (`ts` timestamp, `c2` int, `c3` int, `c4` varchar(256)) tags (`tg` int) sma(`ts`,`c2`)" + tdSql.execute("alter table super_table modify column c4 varchar(256) comment 'c4 256 comment'", queryTimes=1) + tdSql.query("show create table super_table") + tdSql.checkEqual(tdSql.queryResult[0][1].lower(), create_sql) + tdSql.execute("drop database comment_test_db") + def run(self): self.check_gitinfo() self.show_base() @@ -218,6 +278,8 @@ class TDTestCase: self.show_create_sql() self.show_create_sysdb_sql() self.show_create_systb_sql() + self.show_create_table_with_col_comment() + self.alter_table_with_col_comment() def stop(self): tdSql.close() diff --git a/tests/system-test/7-tmq/tmqCommon.py b/tests/system-test/7-tmq/tmqCommon.py index 3ea8273e7f..7f972d857e 100644 --- a/tests/system-test/7-tmq/tmqCommon.py +++ b/tests/system-test/7-tmq/tmqCommon.py @@ -578,18 +578,40 @@ class TMQCom: tdLog.info("wait subscriptions exit for %d s"%wait_cnt) def killProcesser(self, processerName): - killCmd = ( - "ps -ef|grep -w %s| grep -v grep | awk '{print $2}' | xargs kill -TERM > /dev/null 2>&1" - % processerName - ) - - psCmd = "ps -ef|grep -w %s| grep -v grep | awk '{print $2}'" % processerName - processID = subprocess.check_output(psCmd, shell=True) + if platform.system().lower() == 'windows': + killCmd = ("wmic process where name=\"%s.exe\" call terminate > NUL 2>&1" % processerName) + psCmd = ("wmic process where name=\"%s.exe\" | findstr \"%s.exe\"" % (processerName, processerName)) + else: + killCmd = ( + "ps -ef|grep -w %s| grep -v grep | awk '{print $2}' | xargs kill -TERM > /dev/null 2>&1" + % processerName + ) + psCmd = ("ps -ef|grep -w %s| grep -v grep | awk '{print $2}'" % processerName) + processID = "" + + try: + processID = subprocess.check_output(psCmd, shell=True) + except Exception as err: + processID = "" + print('**** warn: ', err) + while processID: os.system(killCmd) time.sleep(1) - processID = subprocess.check_output(psCmd, shell=True) + try: + processID = subprocess.check_output(psCmd, shell=True) + except Exception as err: + processID = "" + print('**** warn: ', err) + + def startProcess(self, processName, param): + if platform.system().lower() == 'windows': + cmd = f"mintty -h never %s %s > NUL 2>&1" % (processName, param) + else: + cmd = f"nohup %s %s > /dev/null 2>&1 &" % (processName, param) + tdLog.info("%s"%(cmd)) + os.system(cmd) def close(self): self.cursor.close() diff --git a/tests/system-test/7-tmq/tmqDropConsumer.py b/tests/system-test/7-tmq/tmqDropConsumer.py index 06ce4c0fd7..137b5c6584 100644 --- a/tests/system-test/7-tmq/tmqDropConsumer.py +++ b/tests/system-test/7-tmq/tmqDropConsumer.py @@ -176,9 +176,7 @@ class TDTestCase: # use taosBenchmark to subscribe binPath = self.getPath() - cmd = "nohup %s -f ./7-tmq/tmqDropConsumer.json > /dev/null 2>&1 & " % binPath - tdLog.info("%s"%(cmd)) - os.system(cmd) + tmqCom.startProcess(binPath, "-f ./7-tmq/tmqDropConsumer.json") expectTopicNum = len(topicNameList) consumerThreadNum = 2 diff --git a/tests/system-test/7-tmq/tmqMaxGroupIds.py b/tests/system-test/7-tmq/tmqMaxGroupIds.py index d22b79a44c..75e2993a5b 100644 --- a/tests/system-test/7-tmq/tmqMaxGroupIds.py +++ b/tests/system-test/7-tmq/tmqMaxGroupIds.py @@ -176,9 +176,7 @@ class TDTestCase: # use taosBenchmark to subscribe binPath = self.getPath() - cmd = "nohup %s -f ./7-tmq/tmqMaxGroupIds.json > /dev/null 2>&1 & " % binPath - tdLog.info("%s"%(cmd)) - os.system(cmd) + tmqCom.startProcess(binPath, "-f ./7-tmq/tmqMaxGroupIds.json") expectTopicNum = 1 expectConsumerNUm = 99