From bdaf0d419ef13f36a0c2e9a9dd2e8f7611f2e9f7 Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao> Date: Tue, 6 Aug 2024 16:14:15 +0800 Subject: [PATCH] stream interp force window close --- include/common/tcommon.h | 5 + include/common/tmsg.h | 12 +- include/common/ttokendef.h | 292 +- include/libs/executor/storageapi.h | 10 + include/libs/stream/streamState.h | 16 +- include/libs/stream/tstreamFileState.h | 8 + source/dnode/mnode/impl/src/mndStreamUtil.c | 2 + source/dnode/snode/src/snodeInitApi.c | 14 +- source/dnode/vnode/src/vnd/vnodeInitApi.c | 6 + source/libs/executor/inc/executorInt.h | 1 + source/libs/executor/src/executil.c | 21 +- source/libs/executor/src/scanoperator.c | 40 +- .../executor/src/streamtimesliceoperator.c | 110 +- source/libs/parser/inc/sql.y | 1 + source/libs/parser/src/parAstCreater.c | 2 + source/libs/parser/src/parTokenizer.c | 1 + source/libs/parser/src/parTranslater.c | 2 +- source/libs/parser/src/sql.c | 9198 ++++++++--------- source/libs/stream/inc/streamBackendRocksdb.h | 2 + source/libs/stream/src/streamBackendRocksdb.c | 55 +- source/libs/stream/src/streamSessionState.c | 8 +- source/libs/stream/src/streamSliceState.c | 2 +- source/libs/stream/src/streamState.c | 57 +- source/libs/stream/src/tstreamFileState.c | 186 +- .../script/tsim/stream/checkpointSession1.sim | 2 + 25 files changed, 4683 insertions(+), 5370 deletions(-) diff --git a/include/common/tcommon.h b/include/common/tcommon.h index 5714990dd5..b706e13c4a 100644 --- a/include/common/tcommon.h +++ b/include/common/tcommon.h @@ -178,6 +178,7 @@ typedef enum EStreamType { STREAM_TRANS_STATE, STREAM_MID_RETRIEVE, STREAM_PARTITION_DELETE_DATA, + STREAM_GET_RESULT, } EStreamType; #pragma pack(push, 1) @@ -407,6 +408,10 @@ typedef struct STUidTagInfo { #define TABLE_NAME_COLUMN_INDEX 6 #define PRIMARY_KEY_COLUMN_INDEX 7 +//steam get result block column +#define DATA_TS_COLUMN_INDEX 0 +#define DATA_VERSION_COLUMN_INDEX 1 + // stream create table block column #define UD_TABLE_NAME_COLUMN_INDEX 0 #define UD_GROUPID_COLUMN_INDEX 1 diff --git a/include/common/tmsg.h b/include/common/tmsg.h index d0be5b43f6..9c681e937e 100644 --- a/include/common/tmsg.h +++ b/include/common/tmsg.h @@ -2747,9 +2747,11 @@ typedef struct { int32_t code; } STaskDropRsp; -#define STREAM_TRIGGER_AT_ONCE 1 -#define STREAM_TRIGGER_WINDOW_CLOSE 2 -#define STREAM_TRIGGER_MAX_DELAY 3 +#define STREAM_TRIGGER_AT_ONCE 1 +#define STREAM_TRIGGER_WINDOW_CLOSE 2 +#define STREAM_TRIGGER_MAX_DELAY 3 +#define STREAM_TRIGGER_FORCE_WINDOW_CLOSE 4 + #define STREAM_DEFAULT_IGNORE_EXPIRED 1 #define STREAM_FILL_HISTORY_ON 1 #define STREAM_FILL_HISTORY_OFF 0 @@ -2812,8 +2814,8 @@ enum { TOPIC_SUB_TYPE__COLUMN, }; -#define DEFAULT_MAX_POLL_INTERVAL 3000000 -#define DEFAULT_SESSION_TIMEOUT 12000 +#define DEFAULT_MAX_POLL_INTERVAL 3000000 +#define DEFAULT_SESSION_TIMEOUT 12000 typedef struct { char name[TSDB_TOPIC_FNAME_LEN]; // accout.topic diff --git a/include/common/ttokendef.h b/include/common/ttokendef.h index 41802d5400..921ecf356d 100644 --- a/include/common/ttokendef.h +++ b/include/common/ttokendef.h @@ -248,151 +248,153 @@ #define TK_TRIGGER 230 #define TK_AT_ONCE 231 #define TK_WINDOW_CLOSE 232 -#define TK_IGNORE 233 -#define TK_EXPIRED 234 -#define TK_FILL_HISTORY 235 -#define TK_UPDATE 236 -#define TK_SUBTABLE 237 -#define TK_UNTREATED 238 -#define TK_KILL 239 -#define TK_CONNECTION 240 -#define TK_TRANSACTION 241 -#define TK_BALANCE 242 -#define TK_VGROUP 243 -#define TK_LEADER 244 -#define TK_MERGE 245 -#define TK_REDISTRIBUTE 246 -#define TK_SPLIT 247 -#define TK_DELETE 248 -#define TK_INSERT 249 -#define TK_NK_BIN 250 -#define TK_NK_HEX 251 -#define TK_NULL 252 -#define TK_NK_QUESTION 253 -#define TK_NK_ALIAS 254 -#define TK_NK_ARROW 255 -#define TK_ROWTS 256 -#define TK_QSTART 257 -#define TK_QEND 258 -#define TK_QDURATION 259 -#define TK_WSTART 260 -#define TK_WEND 261 -#define TK_WDURATION 262 -#define TK_IROWTS 263 -#define TK_ISFILLED 264 -#define TK_CAST 265 -#define TK_NOW 266 -#define TK_TODAY 267 -#define TK_TIMEZONE 268 -#define TK_CLIENT_VERSION 269 -#define TK_SERVER_VERSION 270 -#define TK_SERVER_STATUS 271 -#define TK_CURRENT_USER 272 -#define TK_CASE 273 -#define TK_WHEN 274 -#define TK_THEN 275 -#define TK_ELSE 276 -#define TK_BETWEEN 277 -#define TK_IS 278 -#define TK_NK_LT 279 -#define TK_NK_GT 280 -#define TK_NK_LE 281 -#define TK_NK_GE 282 -#define TK_NK_NE 283 -#define TK_MATCH 284 -#define TK_NMATCH 285 -#define TK_CONTAINS 286 -#define TK_IN 287 -#define TK_JOIN 288 -#define TK_INNER 289 -#define TK_LEFT 290 -#define TK_RIGHT 291 -#define TK_OUTER 292 -#define TK_SEMI 293 -#define TK_ANTI 294 -#define TK_ASOF 295 -#define TK_WINDOW 296 -#define TK_WINDOW_OFFSET 297 -#define TK_JLIMIT 298 -#define TK_SELECT 299 -#define TK_NK_HINT 300 -#define TK_DISTINCT 301 -#define TK_WHERE 302 -#define TK_PARTITION 303 -#define TK_BY 304 -#define TK_SESSION 305 -#define TK_STATE_WINDOW 306 -#define TK_EVENT_WINDOW 307 -#define TK_COUNT_WINDOW 308 -#define TK_SLIDING 309 -#define TK_FILL 310 -#define TK_VALUE 311 -#define TK_VALUE_F 312 -#define TK_NONE 313 -#define TK_PREV 314 -#define TK_NULL_F 315 -#define TK_LINEAR 316 -#define TK_NEXT 317 -#define TK_HAVING 318 -#define TK_RANGE 319 -#define TK_EVERY 320 -#define TK_ORDER 321 -#define TK_SLIMIT 322 -#define TK_SOFFSET 323 -#define TK_LIMIT 324 -#define TK_OFFSET 325 -#define TK_ASC 326 -#define TK_NULLS 327 -#define TK_ABORT 328 -#define TK_AFTER 329 -#define TK_ATTACH 330 -#define TK_BEFORE 331 -#define TK_BEGIN 332 -#define TK_BITAND 333 -#define TK_BITNOT 334 -#define TK_BITOR 335 -#define TK_BLOCKS 336 -#define TK_CHANGE 337 -#define TK_COMMA 338 -#define TK_CONCAT 339 -#define TK_CONFLICT 340 -#define TK_COPY 341 -#define TK_DEFERRED 342 -#define TK_DELIMITERS 343 -#define TK_DETACH 344 -#define TK_DIVIDE 345 -#define TK_DOT 346 -#define TK_EACH 347 -#define TK_FAIL 348 -#define TK_FOR 349 -#define TK_GLOB 350 -#define TK_ID 351 -#define TK_IMMEDIATE 352 -#define TK_IMPORT 353 -#define TK_INITIALLY 354 -#define TK_INSTEAD 355 -#define TK_ISNULL 356 -#define TK_MODULES 357 -#define TK_NK_BITNOT 358 -#define TK_NK_SEMI 359 -#define TK_NOTNULL 360 -#define TK_OF 361 -#define TK_PLUS 362 -#define TK_PRIVILEGE 363 -#define TK_RAISE 364 -#define TK_RESTRICT 365 -#define TK_ROW 366 -#define TK_STAR 367 -#define TK_STATEMENT 368 -#define TK_STRICT 369 -#define TK_STRING 370 -#define TK_TIMES 371 -#define TK_VALUES 372 -#define TK_VARIABLE 373 -#define TK_WAL 374 -#define TK_ENCODE 375 -#define TK_COMPRESS 376 -#define TK_LEVEL 377 +#define TK_FORCE_WINDOW_CLOSE 233 +#define TK_IGNORE 234 +#define TK_EXPIRED 235 +#define TK_FILL_HISTORY 236 +#define TK_UPDATE 237 +#define TK_SUBTABLE 238 +#define TK_UNTREATED 239 +#define TK_KILL 240 +#define TK_CONNECTION 241 +#define TK_TRANSACTION 242 +#define TK_BALANCE 243 +#define TK_VGROUP 244 +#define TK_LEADER 245 +#define TK_MERGE 246 +#define TK_REDISTRIBUTE 247 +#define TK_SPLIT 248 +#define TK_DELETE 249 +#define TK_INSERT 250 +#define TK_NK_BIN 251 +#define TK_NK_HEX 252 +#define TK_NULL 253 +#define TK_NK_QUESTION 254 +#define TK_NK_ALIAS 255 +#define TK_NK_ARROW 256 +#define TK_ROWTS 257 +#define TK_QSTART 258 +#define TK_QEND 259 +#define TK_QDURATION 260 +#define TK_WSTART 261 +#define TK_WEND 262 +#define TK_WDURATION 263 +#define TK_IROWTS 264 +#define TK_ISFILLED 265 +#define TK_CAST 266 +#define TK_NOW 267 +#define TK_TODAY 268 +#define TK_TIMEZONE 269 +#define TK_CLIENT_VERSION 270 +#define TK_SERVER_VERSION 271 +#define TK_SERVER_STATUS 272 +#define TK_CURRENT_USER 273 +#define TK_CASE 274 +#define TK_WHEN 275 +#define TK_THEN 276 +#define TK_ELSE 277 +#define TK_BETWEEN 278 +#define TK_IS 279 +#define TK_NK_LT 280 +#define TK_NK_GT 281 +#define TK_NK_LE 282 +#define TK_NK_GE 283 +#define TK_NK_NE 284 +#define TK_MATCH 285 +#define TK_NMATCH 286 +#define TK_CONTAINS 287 +#define TK_IN 288 +#define TK_JOIN 289 +#define TK_INNER 290 +#define TK_LEFT 291 +#define TK_RIGHT 292 +#define TK_OUTER 293 +#define TK_SEMI 294 +#define TK_ANTI 295 +#define TK_ASOF 296 +#define TK_WINDOW 297 +#define TK_WINDOW_OFFSET 298 +#define TK_JLIMIT 299 +#define TK_SELECT 300 +#define TK_NK_HINT 301 +#define TK_DISTINCT 302 +#define TK_WHERE 303 +#define TK_PARTITION 304 +#define TK_BY 305 +#define TK_SESSION 306 +#define TK_STATE_WINDOW 307 +#define TK_EVENT_WINDOW 308 +#define TK_COUNT_WINDOW 309 +#define TK_SLIDING 310 +#define TK_FILL 311 +#define TK_VALUE 312 +#define TK_VALUE_F 313 +#define TK_NONE 314 +#define TK_PREV 315 +#define TK_NULL_F 316 +#define TK_LINEAR 317 +#define TK_NEXT 318 +#define TK_HAVING 319 +#define TK_RANGE 320 +#define TK_EVERY 321 +#define TK_ORDER 322 +#define TK_SLIMIT 323 +#define TK_SOFFSET 324 +#define TK_LIMIT 325 +#define TK_OFFSET 326 +#define TK_ASC 327 +#define TK_NULLS 328 +#define TK_ABORT 329 +#define TK_AFTER 330 +#define TK_ATTACH 331 +#define TK_BEFORE 332 +#define TK_BEGIN 333 +#define TK_BITAND 334 +#define TK_BITNOT 335 +#define TK_BITOR 336 +#define TK_BLOCKS 337 +#define TK_CHANGE 338 +#define TK_COMMA 339 +#define TK_CONCAT 340 +#define TK_CONFLICT 341 +#define TK_COPY 342 +#define TK_DEFERRED 343 +#define TK_DELIMITERS 344 +#define TK_DETACH 345 +#define TK_DIVIDE 346 +#define TK_DOT 347 +#define TK_EACH 348 +#define TK_FAIL 349 +#define TK_FOR 350 +#define TK_GLOB 351 +#define TK_ID 352 +#define TK_IMMEDIATE 353 +#define TK_IMPORT 354 +#define TK_INITIALLY 355 +#define TK_INSTEAD 356 +#define TK_ISNULL 357 +#define TK_MODULES 358 +#define TK_NK_BITNOT 359 +#define TK_NK_SEMI 360 +#define TK_NOTNULL 361 +#define TK_OF 362 +#define TK_PLUS 363 +#define TK_PRIVILEGE 364 +#define TK_RAISE 365 +#define TK_RESTRICT 366 +#define TK_ROW 367 +#define TK_STAR 368 +#define TK_STATEMENT 369 +#define TK_STRICT 370 +#define TK_STRING 371 +#define TK_TIMES 372 +#define TK_VALUES 373 +#define TK_VARIABLE 374 +#define TK_WAL 375 +#define TK_ENCODE 376 +#define TK_COMPRESS 377 +#define TK_LEVEL 378 + #define TK_NK_SPACE 600 #define TK_NK_COMMENT 601 diff --git a/include/libs/executor/storageapi.h b/include/libs/executor/storageapi.h index de04288b79..06a0a0348f 100644 --- a/include/libs/executor/storageapi.h +++ b/include/libs/executor/storageapi.h @@ -326,6 +326,9 @@ typedef struct { int64_t number; void* pStreamFileState; int32_t buffIndex; + int32_t hashIter; + void* pHashData; + int64_t minGpId; } SStreamStateCur; typedef struct SStateStore { @@ -352,6 +355,8 @@ typedef struct SStateStore { int32_t (*streamStateFillPut)(SStreamState* pState, const SWinKey* key, const void* value, int32_t vLen); int32_t (*streamStateFillGet)(SStreamState* pState, const SWinKey* key, void** pVal, int32_t* pVLen, int32_t* pWinCode); + int32_t (*streamStateFillAddIfNotExist)(SStreamState* pState, const SWinKey* key, void** pVal, int32_t* pVLen, + int32_t* pWinCode); void (*streamStateFillDel)(SStreamState* pState, const SWinKey* key); int32_t (*streamStateFillGetNext)(SStreamState* pState, const SWinKey* pKey, SWinKey* pResKey, void** pVal, int32_t* pVLen, int32_t* pWinCode); @@ -417,6 +422,11 @@ typedef struct SStateStore { struct SStreamFileState* (*streamFileStateInit)(int64_t memSize, uint32_t keySize, uint32_t rowSize, uint32_t selectRowSize, GetTsFun fp, void* pFile, TSKEY delMark, const char* id, int64_t ckId, int8_t type); + + int32_t (*streamStateGroupPut)(SStreamState* pState, int64_t groupId, void* value, int32_t vLen); + SStreamStateCur* (*streamStateGroupGetCur)(SStreamState* pState); + void (*streamStateGroupCurNext)(SStreamStateCur* pCur); + int32_t (*streamStateGroupGetKVByCur)(SStreamStateCur* pCur, int64_t* pKey, void** pVal, int32_t* pVLen); void (*streamFileStateDestroy)(struct SStreamFileState* pFileState); void (*streamFileStateClear)(struct SStreamFileState* pFileState); diff --git a/include/libs/stream/streamState.h b/include/libs/stream/streamState.h index 046812f396..415a33a0a3 100644 --- a/include/libs/stream/streamState.h +++ b/include/libs/stream/streamState.h @@ -76,11 +76,13 @@ int32_t streamStateStateAddIfNotExist(SStreamState* pState, SSessionKey* key, ch // fill int32_t streamStateFillPut(SStreamState* pState, const SWinKey* key, const void* value, int32_t vLen); int32_t streamStateFillGet(SStreamState* pState, const SWinKey* key, void** pVal, int32_t* pVLen, int32_t* pWinCode); +int32_t streamStateFillAddIfNotExist(SStreamState* pState, const SWinKey* key, void** pVal, int32_t* pVLen, + int32_t* pWinCode); void streamStateFillDel(SStreamState* pState, const SWinKey* key); -int32_t streamStateFillGetNext(SStreamState* pState, const SWinKey* pKey, SWinKey* pResKey, void** pVal, - int32_t* pVLen, int32_t* pWinCode); -int32_t streamStateFillGetPrev(SStreamState* pState, const SWinKey* pKey, SWinKey* pResKey, void** pVal, - int32_t* pVLen, int32_t* pWinCode); +int32_t streamStateFillGetNext(SStreamState* pState, const SWinKey* pKey, SWinKey* pResKey, void** pVal, int32_t* pVLen, + int32_t* pWinCode); +int32_t streamStateFillGetPrev(SStreamState* pState, const SWinKey* pKey, SWinKey* pResKey, void** pVal, int32_t* pVLen, + int32_t* pWinCode); int32_t streamStateAddIfNotExist(SStreamState* pState, const SWinKey* key, void** pVal, int32_t* pVLen, int32_t* pWinCode); @@ -109,6 +111,12 @@ void streamStateCurPrev(SStreamState* pState, SStreamStateCur* pCur); int32_t streamStatePutParName(SStreamState* pState, int64_t groupId, const char* tbname); int32_t streamStateGetParName(SStreamState* pState, int64_t groupId, void** pVal, bool onlyCache, int32_t* pWinCode); +// group id +int32_t streamStateGroupPut(SStreamState* pState, int64_t groupId, void* value, int32_t vLen); +SStreamStateCur* streamStateGroupGetCur(SStreamState* pState); +void streamStateGroupCurNext(SStreamStateCur* pCur); +int32_t streamStateGroupGetKVByCur(SStreamStateCur* pCur, int64_t* pKey, void** pVal, int32_t* pVLen); + void streamStateReloadInfo(SStreamState* pState, TSKEY ts); void streamStateCopyBackend(SStreamState* src, SStreamState* dst); diff --git a/include/libs/stream/tstreamFileState.h b/include/libs/stream/tstreamFileState.h index bb0d49f1f1..f6f3943240 100644 --- a/include/libs/stream/tstreamFileState.h +++ b/include/libs/stream/tstreamFileState.h @@ -56,6 +56,8 @@ bool needClearDiskBuff(SStreamFileState* pFileState); void streamFileStateReleaseBuff(SStreamFileState* pFileState, SRowBuffPos* pPos, bool used); void streamFileStateClearBuff(SStreamFileState* pFileState, SRowBuffPos* pPos); +int32_t addRowBuffIfNotExist(SStreamFileState* pFileState, void* pKey, int32_t keyLen, void** pVal, int32_t* pVLen, + int32_t* pWinCode); int32_t getRowBuff(SStreamFileState* pFileState, void* pKey, int32_t keyLen, void** pVal, int32_t* pVLen, int32_t* pWinCode); void deleteRowBuff(SStreamFileState* pFileState, const void* pKey, int32_t keyLen); @@ -98,6 +100,7 @@ void recoverSesssion(SStreamFileState* pFileState, int64_t ckId); void sessionWinStateClear(SStreamFileState* pFileState); void sessionWinStateCleanup(void* pBuff); +SStreamStateCur* createStateCursor(SStreamFileState* pFileState); SStreamStateCur* sessionWinStateSeekKeyCurrentPrev(SStreamFileState* pFileState, const SSessionKey* pWinKey); SStreamStateCur* sessionWinStateSeekKeyCurrentNext(SStreamFileState* pFileState, const SSessionKey* pWinKey); SStreamStateCur* sessionWinStateSeekKeyNext(SStreamFileState* pFileState, const SSessionKey* pWinKey); @@ -136,6 +139,11 @@ int32_t getHashSortPrevRow(SStreamFileState* pFileState, const SWinKey* pKey, SW int32_t recoverFillSnapshot(SStreamFileState* pFileState, int64_t ckId); void deleteHashSortRowBuff(SStreamFileState* pFileState, const SWinKey* pKey); +//group +int32_t streamFileStateGroupPut(SStreamFileState* pFileState, int64_t groupId, void* value, int32_t vLen); +void streamFileStateGroupCurNext(SStreamStateCur* pCur); +int32_t streamFileStateGroupGetKVByCur(SStreamStateCur* pCur, int64_t* pKey, void** pVal, int32_t* pVLen); + #ifdef __cplusplus } #endif diff --git a/source/dnode/mnode/impl/src/mndStreamUtil.c b/source/dnode/mnode/impl/src/mndStreamUtil.c index 38fddd8bf0..71858be72f 100644 --- a/source/dnode/mnode/impl/src/mndStreamUtil.c +++ b/source/dnode/mnode/impl/src/mndStreamUtil.c @@ -1349,6 +1349,8 @@ static void mndShowStreamTrigger(char *dst, SStreamObj *pStream) { strcpy(dst, "window close"); } else if (trigger == STREAM_TRIGGER_MAX_DELAY) { strcpy(dst, "max delay"); + } else if (trigger == STREAM_TRIGGER_FORCE_WINDOW_CLOSE) { + strcpy(dst, "force window close"); } } diff --git a/source/dnode/snode/src/snodeInitApi.c b/source/dnode/snode/src/snodeInitApi.c index 9a353a0b7c..5485c000d1 100644 --- a/source/dnode/snode/src/snodeInitApi.c +++ b/source/dnode/snode/src/snodeInitApi.c @@ -49,6 +49,7 @@ void initStateStoreAPI(SStateStore* pStore) { pStore->streamStateFillPut = streamStateFillPut; pStore->streamStateFillGet = streamStateFillGet; + pStore->streamStateFillAddIfNotExist = streamStateFillAddIfNotExist; pStore->streamStateFillDel = streamStateFillDel; pStore->streamStateFillGetNext = streamStateFillGetNext; pStore->streamStateFillGetPrev = streamStateFillGetPrev; @@ -77,10 +78,10 @@ void initStateStoreAPI(SStateStore* pStore) { pStore->streamStateCountGetKeyByRange = streamStateCountGetKeyByRange; pStore->streamStateSessionAllocWinBuffByNextPosition = streamStateSessionAllocWinBuffByNextPosition; -//void initStreamStateAPI(SStorageAPI* pAPI) { -// initStateStoreAPI(&pAPI->stateStore); -// initFunctionStateStore(&pAPI->functionStore); -//} + // void initStreamStateAPI(SStorageAPI* pAPI) { + // initStateStoreAPI(&pAPI->stateStore); + // initFunctionStateStore(&pAPI->functionStore); + // } pStore->updateInfoInit = updateInfoInit; pStore->updateInfoFillBlockData = updateInfoFillBlockData; @@ -102,6 +103,11 @@ void initStateStoreAPI(SStateStore* pStore) { pStore->streamStateSessionSeekKeyCurrentPrev = streamStateSessionSeekKeyCurrentPrev; pStore->streamStateSessionSeekKeyCurrentNext = streamStateSessionSeekKeyCurrentNext; + pStore->streamStateGroupPut = streamStateGroupPut; + pStore->streamStateGroupGetCur = streamStateGroupGetCur; + pStore->streamStateGroupCurNext = streamStateGroupCurNext; + pStore->streamStateGroupGetKVByCur = streamStateGroupGetKVByCur; + pStore->streamFileStateDestroy = streamFileStateDestroy; pStore->streamFileStateClear = streamFileStateClear; pStore->needClearDiskBuff = needClearDiskBuff; diff --git a/source/dnode/vnode/src/vnd/vnodeInitApi.c b/source/dnode/vnode/src/vnd/vnodeInitApi.c index b96f7ba0a7..ed1632cb97 100644 --- a/source/dnode/vnode/src/vnd/vnodeInitApi.c +++ b/source/dnode/vnode/src/vnd/vnodeInitApi.c @@ -165,6 +165,7 @@ void initStateStoreAPI(SStateStore* pStore) { pStore->streamStateFillPut = streamStateFillPut; pStore->streamStateFillGet = streamStateFillGet; + pStore->streamStateFillAddIfNotExist = streamStateFillAddIfNotExist; pStore->streamStateFillDel = streamStateFillDel; pStore->streamStateFillGetNext = streamStateFillGetNext; pStore->streamStateFillGetPrev = streamStateFillGetPrev; @@ -216,6 +217,11 @@ void initStateStoreAPI(SStateStore* pStore) { pStore->streamStateSessionSeekKeyCurrentPrev = streamStateSessionSeekKeyCurrentPrev; pStore->streamStateSessionSeekKeyCurrentNext = streamStateSessionSeekKeyCurrentNext; + pStore->streamStateGroupPut = streamStateGroupPut; + pStore->streamStateGroupGetCur = streamStateGroupGetCur; + pStore->streamStateGroupCurNext = streamStateGroupCurNext; + pStore->streamStateGroupGetKVByCur = streamStateGroupGetKVByCur; + pStore->streamFileStateDestroy = streamFileStateDestroy; pStore->streamFileStateClear = streamFileStateClear; pStore->needClearDiskBuff = needClearDiskBuff; diff --git a/source/libs/executor/inc/executorInt.h b/source/libs/executor/inc/executorInt.h index 3ae7a968ff..f09545a8db 100644 --- a/source/libs/executor/inc/executorInt.h +++ b/source/libs/executor/inc/executorInt.h @@ -476,6 +476,7 @@ typedef struct SStreamScanInfo { STqReader* tqReader; uint64_t groupId; + bool igCheckGroupId; struct SUpdateInfo* pUpdateInfo; EStreamScanMode scanMode; diff --git a/source/libs/executor/src/executil.c b/source/libs/executor/src/executil.c index 573cc0425d..2a9f952477 100644 --- a/source/libs/executor/src/executil.c +++ b/source/libs/executor/src/executil.c @@ -321,8 +321,8 @@ int32_t prepareDataBlockBuf(SSDataBlock* pDataBlock, SColMatchInfo* pMatchInfo) EDealRes doTranslateTagExpr(SNode** pNode, void* pContext) { STransTagExprCtx* pCtx = pContext; - SMetaReader* mr = pCtx->pReader; - bool isTagCol = false, isTbname = false; + SMetaReader* mr = pCtx->pReader; + bool isTagCol = false, isTbname = false; if (nodeType(*pNode) == QUERY_NODE_COLUMN) { SColumnNode* pCol = (SColumnNode*)*pNode; if (pCol->colType == COLUMN_TYPE_TBNAME) @@ -434,7 +434,7 @@ int32_t isQualifiedTable(STableKeyInfo* info, SNode* pTagCond, void* metaHandle, static EDealRes getColumn(SNode** pNode, void* pContext) { tagFilterAssist* pData = (tagFilterAssist*)pContext; - SColumnNode* pSColumnNode = NULL; + SColumnNode* pSColumnNode = NULL; if (QUERY_NODE_COLUMN == nodeType((*pNode))) { pSColumnNode = *(SColumnNode**)pNode; } else if (QUERY_NODE_FUNCTION == nodeType((*pNode))) { @@ -457,7 +457,7 @@ static EDealRes getColumn(SNode** pNode, void* pContext) { return DEAL_RES_CONTINUE; } - void* data = taosHashGet(pData->colHash, &pSColumnNode->colId, sizeof(pSColumnNode->colId)); + void* data = taosHashGet(pData->colHash, &pSColumnNode->colId, sizeof(pSColumnNode->colId)); if (!data) { int32_t tempRes = taosHashPut(pData->colHash, &pSColumnNode->colId, sizeof(pSColumnNode->colId), pNode, sizeof((*pNode))); @@ -1458,7 +1458,7 @@ int32_t getGroupIdFromTagsVal(void* pVnode, uint64_t uid, SNodeList* pGroupNode, } SNodeList* groupNew = NULL; - int32_t code = nodesCloneList(pGroupNode, &groupNew); + int32_t code = nodesCloneList(pGroupNode, &groupNew); if (TSDB_CODE_SUCCESS != code) { pAPI->metaReaderFn.clearReader(&mr); return code; @@ -1722,7 +1722,7 @@ int32_t createExprFromOneNode(SExprInfo* pExp, SNode* pNode, int16_t slotId) { if (!pFuncNode->pParameterList && (memcmp(pExprNode->_function.functionName, name, len) == 0) && pExprNode->_function.functionName[len] == 0) { pFuncNode->pParameterList = NULL; - int32_t code = nodesMakeList(&pFuncNode->pParameterList); + int32_t code = nodesMakeList(&pFuncNode->pParameterList); SValueNode* res = NULL; if (TSDB_CODE_SUCCESS == code) { code = nodesMakeNode(QUERY_NODE_VALUE, (SNode**)&res); @@ -1787,7 +1787,8 @@ int32_t createExprFromOneNode(SExprInfo* pExp, SNode* pNode, int16_t slotId) { if (TSDB_CODE_SUCCESS == code) { pExp->base.numOfParams = 1; SDataType* pType = &pCond->node.resType; - pExp->base.resSchema = createResSchema(pType->type, pType->bytes, slotId, pType->scale, pType->precision, pCond->node.aliasName); + pExp->base.resSchema = + createResSchema(pType->type, pType->bytes, slotId, pType->scale, pType->precision, pCond->node.aliasName); pExp->pExpr->_optrRoot.pRootNode = pNode; } } else { @@ -2325,7 +2326,7 @@ int32_t tableListAddTableInfo(STableListInfo* pTableList, uint64_t uid, uint64_t int32_t slot = (int32_t)taosArrayGetSize(pTableList->pTableList) - 1; code = taosHashPut(pTableList->map, &uid, sizeof(uid), &slot, sizeof(slot)); if (code != TSDB_CODE_SUCCESS) { - ASSERT(code != TSDB_CODE_DUP_KEY); // we have checked the existence of uid in hash map above + ASSERT(code != TSDB_CODE_DUP_KEY); // we have checked the existence of uid in hash map above taosArrayPopTailBatch(pTableList->pTableList, 1); // let's pop the last element in the array list } @@ -2664,7 +2665,9 @@ void printSpecDataBlock(SSDataBlock* pBlock, const char* flag, const char* opStr qDebug("%s===stream===%s %s: Block is Null", taskIdStr, flag, opStr); return; } else if (pBlock->info.rows == 0) { - qDebug("%s===stream===%s %s: Block is Empty. block type %d", taskIdStr, flag, opStr, pBlock->info.type); + qDebug("%s===stream===%s %s: Block is Empty. block type %d.skey:%" PRId64 ",ekey:%" PRId64 ",version%" PRId64, + taskIdStr, flag, opStr, pBlock->info.type, pBlock->info.window.skey, pBlock->info.window.ekey, + pBlock->info.version); return; } if (qDebugFlag & DEBUG_DEBUG) { diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index bd0b9057e0..ff2cdcd027 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -1483,7 +1483,11 @@ static void setGroupId(SStreamScanInfo* pInfo, SSDataBlock* pBlock, int32_t grou SColumnInfoData* pColInfo = taosArrayGet(pBlock->pDataBlock, groupColIndex); uint64_t* groupCol = (uint64_t*)pColInfo->pData; ASSERT(rowIndex < pBlock->info.rows); - pInfo->groupId = groupCol[rowIndex]; + if (colDataIsNull_s(pColInfo, rowIndex)) { + pInfo->igCheckGroupId = true; + } else { + pInfo->groupId = groupCol[rowIndex]; + } } void resetTableScanInfo(STableScanInfo* pTableScanInfo, STimeWindow* pWin, uint64_t ver) { @@ -1751,6 +1755,12 @@ static int32_t doRangeScan(SStreamScanInfo* pInfo, SSDataBlock* pSDB, int32_t ts continue; } + if (pInfo->igCheckGroupId == true) { + pResult->info.calWin = pInfo->updateWin; + (*ppRes) = pResult; + goto _end; + } + if (pInfo->partitionSup.needCalc) { SSDataBlock* tmpBlock = NULL; code = createOneDataBlock(pResult, true, &tmpBlock); @@ -1825,10 +1835,10 @@ int32_t appendOneRowToSpecialBlockImpl(SSDataBlock* pBlock, TSKEY* pStartTs, TSK code = colDataSetVal(pEndTsCol, pBlock->info.rows, (const char*)pEndTs, false); QUERY_CHECK_CODE(code, lino, _end); - code = colDataSetVal(pUidCol, pBlock->info.rows, (const char*)pUid, false); + code = colDataSetVal(pUidCol, pBlock->info.rows, (const char*)pUid, pUid == NULL); QUERY_CHECK_CODE(code, lino, _end); - code = colDataSetVal(pGpCol, pBlock->info.rows, (const char*)pGp, false); + code = colDataSetVal(pGpCol, pBlock->info.rows, (const char*)pGp, pGp == NULL); QUERY_CHECK_CODE(code, lino, _end); code = colDataSetVal(pCalStartCol, pBlock->info.rows, (const char*)pCalStartTs, false); @@ -2065,8 +2075,8 @@ _end: return code; } -int32_t getTimeSliceWinRange(SStreamAggSupporter* pAggSup, SInterval* pInterval, TSKEY start, TSKEY end, int64_t groupId, - STimeWindow* pScanRange, STimeWindow* pDelRange) { +int32_t getTimeSliceWinRange(SStreamAggSupporter* pAggSup, SInterval* pInterval, TSKEY start, TSKEY end, + int64_t groupId, STimeWindow* pScanRange, STimeWindow* pDelRange) { int32_t code = TSDB_CODE_SUCCESS; int32_t lino = 0; int32_t winCode = TSDB_CODE_SUCCESS; @@ -2161,7 +2171,7 @@ static int32_t generateTimeSliceScanRange(SStreamScanInfo* pInfo, SSDataBlock* p STimeWindow delRange = {0}; ASSERT(mode == STREAM_DELETE_RESULT || mode == STREAM_DELETE_DATA); code = getTimeSliceWinRange(pInfo->windowSup.pStreamAggSup, &pInfo->interval, startData[i], endData[i], groupId, - &scanRange, &delRange); + &scanRange, &delRange); QUERY_CHECK_CODE(code, lino, _end); code = colDataSetVal(pDestStartCol, i, (const char*)&scanRange.skey, false); @@ -3157,6 +3167,12 @@ static bool isStreamWindow(SStreamScanInfo* pInfo) { isTimeSlice(pInfo); } +static int32_t copyGetResultBlock(SSDataBlock* dest, const SSDataBlock* src) { + TSKEY start = src->info.window.skey; + TSKEY end = src->info.window.ekey; + return appendDataToSpecialBlock(dest, &start, &end, NULL, NULL, NULL); +} + static int32_t doStreamScanNext(SOperatorInfo* pOperator, SSDataBlock** ppRes) { // NOTE: this operator does never check if current status is done or not int32_t code = TSDB_CODE_SUCCESS; @@ -3380,13 +3396,22 @@ FETCH_NEXT_BLOCK: } } } break; + case STREAM_GET_RESULT: { + pInfo->blockType = STREAM_INPUT__DATA_SUBMIT; + pInfo->updateResIndex = 0; + code = copyGetResultBlock(pInfo->pUpdateRes, pBlock); + QUERY_CHECK_CODE(code, lino, _end); + pInfo->pUpdateInfo->maxDataVersion = pBlock->info.version; + prepareRangeScan(pInfo, pInfo->pUpdateRes, &pInfo->updateResIndex, NULL); + pInfo->scanMode = STREAM_SCAN_FROM_DATAREADER_RANGE; + } break; case STREAM_CHECKPOINT: { qError("stream check point error. msg type: STREAM_INPUT__DATA_BLOCK"); } break; default: break; } - printDataBlock(pBlock, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); + printSpecDataBlock(pBlock, getStreamOpName(pOperator->operatorType), "block recv", GET_TASKID(pTaskInfo)); setStreamOperatorState(&pInfo->basic, pBlock->info.type); (*ppRes) = pBlock; return code; @@ -4129,6 +4154,7 @@ int32_t createStreamScanOperatorInfo(SReadHandle* pHandle, STableScanPhysiNode* pInfo->scanMode = STREAM_SCAN_FROM_READERHANDLE; pInfo->windowSup = (SWindowSupporter){.pStreamAggSup = NULL, .gap = -1, .parentType = QUERY_NODE_PHYSICAL_PLAN}; pInfo->groupId = 0; + pInfo->igCheckGroupId = false; pInfo->pStreamScanOp = pOperator; pInfo->deleteDataIndex = 0; code = createSpecialDataBlock(STREAM_DELETE_DATA, &pInfo->pDeleteDataRes); diff --git a/source/libs/executor/src/streamtimesliceoperator.c b/source/libs/executor/src/streamtimesliceoperator.c index a58c80d986..98991a850a 100644 --- a/source/libs/executor/src/streamtimesliceoperator.c +++ b/source/libs/executor/src/streamtimesliceoperator.c @@ -591,7 +591,7 @@ static TSKEY adustEndTsKey(TSKEY pointTs, TSKEY rowTs, SInterval* pInterval) { } static int32_t getLinearResultInfoFromState(SStreamAggSupporter* pAggSup, SStreamFillSupporter* pFillSup, TSKEY ts, int64_t groupId, SSlicePoint* pCurPoint, SSlicePoint* pPrevPoint, - SSlicePoint* pNextPoint, int32_t* pWinCode) { + SSlicePoint* pNextPoint) { int32_t code = TSDB_CODE_SUCCESS; int32_t lino = 0; int32_t tmpRes = TSDB_CODE_SUCCESS; @@ -605,7 +605,7 @@ static int32_t getLinearResultInfoFromState(SStreamAggSupporter* pAggSup, SStrea pCurPoint->key.ts = ts; int32_t curVLen = 0; code = - pAggSup->stateStore.streamStateFillGet(pState, &pCurPoint->key, (void**)&pCurPoint->pResPos, &curVLen, pWinCode); + pAggSup->stateStore.streamStateFillGet(pState, &pCurPoint->key, (void**)&pCurPoint->pResPos, &curVLen, &tmpRes); QUERY_CHECK_CODE(code, lino, _end); setPointBuff(pCurPoint, pFillSup); @@ -674,7 +674,7 @@ _end: static int32_t getResultInfoFromState(SStreamAggSupporter* pAggSup, SStreamFillSupporter* pFillSup, TSKEY ts, int64_t groupId, SSlicePoint* pCurPoint, SSlicePoint* pPrevPoint, - SSlicePoint* pNextPoint, int32_t* pWinCode) { + SSlicePoint* pNextPoint) { int32_t code = TSDB_CODE_SUCCESS; int32_t lino = 0; int32_t tmpRes = TSDB_CODE_SUCCESS; @@ -688,13 +688,16 @@ static int32_t getResultInfoFromState(SStreamAggSupporter* pAggSup, SStreamFillS pCurPoint->key.ts = ts; int32_t curVLen = 0; code = - pAggSup->stateStore.streamStateFillGet(pState, &pCurPoint->key, (void**)&pCurPoint->pResPos, &curVLen, pWinCode); + pAggSup->stateStore.streamStateFillGet(pState, &pCurPoint->key, (void**)&pCurPoint->pResPos, &curVLen, &tmpRes); QUERY_CHECK_CODE(code, lino, _end); - setPointBuff(pCurPoint, pFillSup); - - pFillSup->cur.key = pCurPoint->pRightRow->key; - pFillSup->cur.pRowVal = pCurPoint->pRightRow->pRowVal; + if (tmpRes == TSDB_CODE_SUCCESS) { + setPointBuff(pCurPoint, pFillSup); + pFillSup->cur.key = pCurPoint->pRightRow->key; + pFillSup->cur.pRowVal = pCurPoint->pRightRow->pRowVal; + } else { + pFillSup->cur.key = pCurPoint->key.ts + 1; + } pPrevPoint->key.groupId = groupId; int32_t preVLen = 0; @@ -755,8 +758,8 @@ static int32_t getPointInfoFromStateRight(SStreamAggSupporter* pAggSup, SStreamF pNextPoint->key.ts = stw.skey; int32_t curVLen = 0; - code = pAggSup->stateStore.streamStateFillGet(pState, &pNextPoint->key, (void**)&pNextPoint->pResPos, &curVLen, - pWinCode); + code = pAggSup->stateStore.streamStateFillAddIfNotExist(pState, &pNextPoint->key, (void**)&pNextPoint->pResPos, + &curVLen, pWinCode); QUERY_CHECK_CODE(code, lino, _end); setPointBuff(pNextPoint, pFillSup); @@ -800,8 +803,8 @@ static int32_t getPointInfoFromState(SStreamAggSupporter* pAggSup, SStreamFillSu pCurPoint->key.ts = ts; int32_t curVLen = 0; - code = - pAggSup->stateStore.streamStateFillGet(pState, &pCurPoint->key, (void**)&pCurPoint->pResPos, &curVLen, pWinCode); + code = pAggSup->stateStore.streamStateFillAddIfNotExist(pState, &pCurPoint->key, (void**)&pCurPoint->pResPos, + &curVLen, pWinCode); QUERY_CHECK_CODE(code, lino, _end); setPointBuff(pCurPoint, pFillSup); @@ -828,8 +831,8 @@ static int32_t getPointInfoFromState(SStreamAggSupporter* pAggSup, SStreamFillSu } else { pNextPoint->key.ts = taosTimeAdd(pCurPoint->key.ts, pFillSup->interval.sliding, pFillSup->interval.slidingUnit, pFillSup->interval.precision); - code = pAggSup->stateStore.streamStateFillGet(pState, &pNextPoint->key, (void**)&pNextPoint->pResPos, &nextVLen, - &tmpRes); + code = pAggSup->stateStore.streamStateFillAddIfNotExist(pState, &pNextPoint->key, (void**)&pNextPoint->pResPos, + &nextVLen, &tmpRes); QUERY_CHECK_CODE(code, lino, _end); setPointBuff(pNextPoint, pFillSup); if (tmpRes != TSDB_CODE_SUCCESS) { @@ -1021,6 +1024,30 @@ static void transBlockToResultRow(const SSDataBlock* pBlock, int32_t rowId, TSKE pRowVal->key = ts; } +static int32_t saveTimeSliceWinResultInfo(SStreamAggSupporter* pAggSup, int8_t trigger, SWinKey* pKey, + SSHashObj* pUpdatedMap, bool needDel, SSHashObj* pDeletedMap) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + + if (trigger == STREAM_TRIGGER_AT_ONCE) { + code = saveTimeSliceWinResult(pKey, pUpdatedMap); + QUERY_CHECK_CODE(code, lino, _end); + if (needDel) { + code = saveTimeSliceWinResult(pKey, pDeletedMap); + QUERY_CHECK_CODE(code, lino, _end); + } + } else if (trigger == STREAM_TRIGGER_FORCE_WINDOW_CLOSE) { + code = pAggSup->stateStore.streamStateGroupPut(pAggSup->pState, pKey->groupId, NULL, 0); + QUERY_CHECK_CODE(code, lino, _end); + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + static void doStreamTimeSliceImpl(SOperatorInfo* pOperator, SSDataBlock* pBlock) { int32_t code = TSDB_CODE_SUCCESS; int32_t lino = 0; @@ -1088,10 +1115,9 @@ static void doStreamTimeSliceImpl(SOperatorInfo* pOperator, SSDataBlock* pBlock) right = needAdjustValue(&curPoint, tsCols[startPos], false, pFillSup->type); if (right) { transBlockToResultRow(pBlock, startPos, tsCols[startPos], curPoint.pRightRow); - saveTimeSliceWinResult(&curPoint.key, pInfo->pUpdatedMap); - if (pInfo->destHasPrimaryKey && winCode == TSDB_CODE_SUCCESS) { - saveTimeSliceWinResult(&curPoint.key, pInfo->pDeletedMap); - } + bool needDel = pInfo->destHasPrimaryKey && winCode == TSDB_CODE_SUCCESS; + saveTimeSliceWinResultInfo(pAggSup, pInfo->twAggSup.calTrigger, &curPoint.key, pInfo->pUpdatedMap, needDel, + pInfo->pDeletedMap); } releaseOutputBuf(pAggSup->pState, curPoint.pResPos, &pAggSup->stateStore); @@ -1103,10 +1129,9 @@ static void doStreamTimeSliceImpl(SOperatorInfo* pOperator, SSDataBlock* pBlock) left = needAdjustValue(&nextPoint, tsCols[leftRowId], true, pFillSup->type); if (left) { transBlockToResultRow(pBlock, leftRowId, tsCols[leftRowId], nextPoint.pLeftRow); - saveTimeSliceWinResult(&nextPoint.key, pInfo->pUpdatedMap); - if (pInfo->destHasPrimaryKey && winCode == TSDB_CODE_SUCCESS) { - saveTimeSliceWinResult(&curPoint.key, pInfo->pDeletedMap); - } + bool needDel = pInfo->destHasPrimaryKey && winCode == TSDB_CODE_SUCCESS; + saveTimeSliceWinResultInfo(pAggSup, pInfo->twAggSup.calTrigger, &nextPoint.key, pInfo->pUpdatedMap, needDel, + pInfo->pDeletedMap); } releaseOutputBuf(pAggSup->pState, nextPoint.pResPos, &pAggSup->stateStore); @@ -1125,10 +1150,9 @@ static void doStreamTimeSliceImpl(SOperatorInfo* pOperator, SSDataBlock* pBlock) right = needAdjustValue(&curPoint, tsCols[startPos], false, pFillSup->type); if (right) { transBlockToResultRow(pBlock, startPos, tsCols[startPos], curPoint.pRightRow); - saveTimeSliceWinResult(&curPoint.key, pInfo->pUpdatedMap); - if (pInfo->destHasPrimaryKey && winCode == TSDB_CODE_SUCCESS) { - saveTimeSliceWinResult(&curPoint.key, pInfo->pDeletedMap); - } + bool needDel = pInfo->destHasPrimaryKey && winCode == TSDB_CODE_SUCCESS; + saveTimeSliceWinResultInfo(pAggSup, pInfo->twAggSup.calTrigger, &curPoint.key, pInfo->pUpdatedMap, needDel, + pInfo->pDeletedMap); } releaseOutputBuf(pAggSup->pState, curPoint.pResPos, &pAggSup->stateStore); } @@ -1162,13 +1186,11 @@ void doBuildTimeSlicePointResult(SStreamAggSupporter* pAggSup, SStreamFillSuppor SSlicePoint curPoint = {.key.ts = pKey->ts, .key.groupId = pKey->groupId}; SSlicePoint prevPoint = {0}; SSlicePoint nextPoint = {0}; - int32_t winCode = TSDB_CODE_SUCCESS; if (pFillSup->type != TSDB_FILL_LINEAR) { - code = getResultInfoFromState(pAggSup, pFillSup, pKey->ts, pKey->groupId, &curPoint, &prevPoint, &nextPoint, - &winCode); + code = getResultInfoFromState(pAggSup, pFillSup, pKey->ts, pKey->groupId, &curPoint, &prevPoint, &nextPoint); } else { - code = getLinearResultInfoFromState(pAggSup, pFillSup, pKey->ts, pKey->groupId, &curPoint, &prevPoint, &nextPoint, - &winCode); + code = + getLinearResultInfoFromState(pAggSup, pFillSup, pKey->ts, pKey->groupId, &curPoint, &prevPoint, &nextPoint); } QUERY_CHECK_CODE(code, lino, _end); @@ -1291,6 +1313,28 @@ _end: return code; } +static int32_t setAllResultKey(SStreamAggSupporter* pAggSup, TSKEY ts, SSHashObj* pUpdatedMap) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + int64_t groupId = 0; + SStreamStateCur* pCur = pAggSup->stateStore.streamStateGroupGetCur(pAggSup->pState); + int32_t winCode = pAggSup->stateStore.streamStateGroupGetKVByCur(pCur, &groupId, NULL, NULL); + if (winCode != TSDB_CODE_SUCCESS) { + goto _end; + } + SWinKey key = {.ts = ts, .groupId = groupId}; + code = saveTimeSliceWinResult(&key, pUpdatedMap); + QUERY_CHECK_CODE(code, lino, _end); + + pAggSup->stateStore.streamStateGroupCurNext(pCur); + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + static int32_t doStreamTimeSliceNext(SOperatorInfo* pOperator, SSDataBlock** ppRes) { int32_t code = TSDB_CODE_SUCCESS; int32_t lino = 0; @@ -1376,6 +1420,10 @@ static int32_t doStreamTimeSliceNext(SOperatorInfo* pOperator, SSDataBlock** ppR (*ppRes) = pBlock; goto _end; } break; + case STREAM_GET_RESULT: { + setAllResultKey(pAggSup, pBlock->info.window.skey, pInfo->pUpdatedMap); + continue; + } default: ASSERTS(false, "invalid SSDataBlock type"); } diff --git a/source/libs/parser/inc/sql.y b/source/libs/parser/inc/sql.y index f2d804df27..5b8e7b33a2 100644 --- a/source/libs/parser/inc/sql.y +++ b/source/libs/parser/inc/sql.y @@ -762,6 +762,7 @@ tag_def_or_ref_opt(A) ::= TAGS NK_LP column_stream_def_list(B) NK_RP. stream_options(A) ::= . { A = createStreamOptions(pCxt); } stream_options(A) ::= stream_options(B) TRIGGER AT_ONCE(C). { A = setStreamOptions(pCxt, B, SOPT_TRIGGER_TYPE_SET, &C, NULL); } stream_options(A) ::= stream_options(B) TRIGGER WINDOW_CLOSE(C). { A = setStreamOptions(pCxt, B, SOPT_TRIGGER_TYPE_SET, &C, NULL); } +stream_options(A) ::= stream_options(B) TRIGGER FORCE_WINDOW_CLOSE(C). { A = setStreamOptions(pCxt, B, SOPT_TRIGGER_TYPE_SET, &C, NULL); } stream_options(A) ::= stream_options(B) TRIGGER MAX_DELAY(C) duration_literal(D). { A = setStreamOptions(pCxt, B, SOPT_TRIGGER_TYPE_SET, &C, releaseRawExprNode(pCxt, D)); } stream_options(A) ::= stream_options(B) WATERMARK duration_literal(C). { A = setStreamOptions(pCxt, B, SOPT_WATERMARK_SET, NULL, releaseRawExprNode(pCxt, C)); } stream_options(A) ::= stream_options(B) IGNORE EXPIRED NK_INTEGER(C). { A = setStreamOptions(pCxt, B, SOPT_IGNORE_EXPIRED_SET, &C, NULL); } diff --git a/source/libs/parser/src/parAstCreater.c b/source/libs/parser/src/parAstCreater.c index cd7cda01e0..a196fcfb56 100644 --- a/source/libs/parser/src/parAstCreater.c +++ b/source/libs/parser/src/parAstCreater.c @@ -2953,6 +2953,8 @@ static int8_t getTriggerType(uint32_t tokenType) { return STREAM_TRIGGER_WINDOW_CLOSE; case TK_MAX_DELAY: return STREAM_TRIGGER_MAX_DELAY; + case TK_FORCE_WINDOW_CLOSE: + return STREAM_TRIGGER_FORCE_WINDOW_CLOSE; default: break; } diff --git a/source/libs/parser/src/parTokenizer.c b/source/libs/parser/src/parTokenizer.c index deec310862..9fe191208c 100644 --- a/source/libs/parser/src/parTokenizer.c +++ b/source/libs/parser/src/parTokenizer.c @@ -335,6 +335,7 @@ static SKeyword keywordTable[] = { {"LEVEL", TK_LEVEL}, {"ARBGROUPS", TK_ARBGROUPS}, {"IS_IMPORT", TK_IS_IMPORT}, + {"FORCE_WINDOW_CLOSE", TK_FORCE_WINDOW_CLOSE}, }; // clang-format on diff --git a/source/libs/parser/src/parTranslater.c b/source/libs/parser/src/parTranslater.c index 0cc8f39f33..ef89131564 100755 --- a/source/libs/parser/src/parTranslater.c +++ b/source/libs/parser/src/parTranslater.c @@ -10291,7 +10291,7 @@ static int32_t checkStreamQuery(STranslateContext* pCxt, SCreateStreamStmt* pStm "SUBTABLE expression must not has column when no partition by clause"); } - if (NULL == pSelect->pWindow && STREAM_TRIGGER_AT_ONCE != pStmt->pOptions->triggerType) { + if (NULL == pSelect->pWindow && !pSelect->hasInterpFunc && STREAM_TRIGGER_AT_ONCE != pStmt->pOptions->triggerType) { return generateSyntaxErrMsgExt(&pCxt->msgBuf, TSDB_CODE_PAR_INVALID_STREAM_QUERY, "The trigger mode of non window query can only be AT_ONCE"); } diff --git a/source/libs/parser/src/sql.c b/source/libs/parser/src/sql.c index 26c5465b8c..dabd25a412 100644 --- a/source/libs/parser/src/sql.c +++ b/source/libs/parser/src/sql.c @@ -25,8 +25,6 @@ ** input grammar file: */ /************ Begin %include sections from the grammar ************************/ -#line 11 "sql.y" - #include #include #include @@ -42,7 +40,6 @@ #include "parAst.h" #define YYSTACKDEPTH 0 -#line 46 "sql.c" /**************** End of %include directives **********************************/ /* These constants specify the various numeric values for terminal symbols. ***************** Begin token definitions *************************************/ @@ -279,151 +276,152 @@ #define TK_TRIGGER 230 #define TK_AT_ONCE 231 #define TK_WINDOW_CLOSE 232 -#define TK_IGNORE 233 -#define TK_EXPIRED 234 -#define TK_FILL_HISTORY 235 -#define TK_UPDATE 236 -#define TK_SUBTABLE 237 -#define TK_UNTREATED 238 -#define TK_KILL 239 -#define TK_CONNECTION 240 -#define TK_TRANSACTION 241 -#define TK_BALANCE 242 -#define TK_VGROUP 243 -#define TK_LEADER 244 -#define TK_MERGE 245 -#define TK_REDISTRIBUTE 246 -#define TK_SPLIT 247 -#define TK_DELETE 248 -#define TK_INSERT 249 -#define TK_NK_BIN 250 -#define TK_NK_HEX 251 -#define TK_NULL 252 -#define TK_NK_QUESTION 253 -#define TK_NK_ALIAS 254 -#define TK_NK_ARROW 255 -#define TK_ROWTS 256 -#define TK_QSTART 257 -#define TK_QEND 258 -#define TK_QDURATION 259 -#define TK_WSTART 260 -#define TK_WEND 261 -#define TK_WDURATION 262 -#define TK_IROWTS 263 -#define TK_ISFILLED 264 -#define TK_CAST 265 -#define TK_NOW 266 -#define TK_TODAY 267 -#define TK_TIMEZONE 268 -#define TK_CLIENT_VERSION 269 -#define TK_SERVER_VERSION 270 -#define TK_SERVER_STATUS 271 -#define TK_CURRENT_USER 272 -#define TK_CASE 273 -#define TK_WHEN 274 -#define TK_THEN 275 -#define TK_ELSE 276 -#define TK_BETWEEN 277 -#define TK_IS 278 -#define TK_NK_LT 279 -#define TK_NK_GT 280 -#define TK_NK_LE 281 -#define TK_NK_GE 282 -#define TK_NK_NE 283 -#define TK_MATCH 284 -#define TK_NMATCH 285 -#define TK_CONTAINS 286 -#define TK_IN 287 -#define TK_JOIN 288 -#define TK_INNER 289 -#define TK_LEFT 290 -#define TK_RIGHT 291 -#define TK_OUTER 292 -#define TK_SEMI 293 -#define TK_ANTI 294 -#define TK_ASOF 295 -#define TK_WINDOW 296 -#define TK_WINDOW_OFFSET 297 -#define TK_JLIMIT 298 -#define TK_SELECT 299 -#define TK_NK_HINT 300 -#define TK_DISTINCT 301 -#define TK_WHERE 302 -#define TK_PARTITION 303 -#define TK_BY 304 -#define TK_SESSION 305 -#define TK_STATE_WINDOW 306 -#define TK_EVENT_WINDOW 307 -#define TK_COUNT_WINDOW 308 -#define TK_SLIDING 309 -#define TK_FILL 310 -#define TK_VALUE 311 -#define TK_VALUE_F 312 -#define TK_NONE 313 -#define TK_PREV 314 -#define TK_NULL_F 315 -#define TK_LINEAR 316 -#define TK_NEXT 317 -#define TK_HAVING 318 -#define TK_RANGE 319 -#define TK_EVERY 320 -#define TK_ORDER 321 -#define TK_SLIMIT 322 -#define TK_SOFFSET 323 -#define TK_LIMIT 324 -#define TK_OFFSET 325 -#define TK_ASC 326 -#define TK_NULLS 327 -#define TK_ABORT 328 -#define TK_AFTER 329 -#define TK_ATTACH 330 -#define TK_BEFORE 331 -#define TK_BEGIN 332 -#define TK_BITAND 333 -#define TK_BITNOT 334 -#define TK_BITOR 335 -#define TK_BLOCKS 336 -#define TK_CHANGE 337 -#define TK_COMMA 338 -#define TK_CONCAT 339 -#define TK_CONFLICT 340 -#define TK_COPY 341 -#define TK_DEFERRED 342 -#define TK_DELIMITERS 343 -#define TK_DETACH 344 -#define TK_DIVIDE 345 -#define TK_DOT 346 -#define TK_EACH 347 -#define TK_FAIL 348 -#define TK_FOR 349 -#define TK_GLOB 350 -#define TK_ID 351 -#define TK_IMMEDIATE 352 -#define TK_IMPORT 353 -#define TK_INITIALLY 354 -#define TK_INSTEAD 355 -#define TK_ISNULL 356 -#define TK_MODULES 357 -#define TK_NK_BITNOT 358 -#define TK_NK_SEMI 359 -#define TK_NOTNULL 360 -#define TK_OF 361 -#define TK_PLUS 362 -#define TK_PRIVILEGE 363 -#define TK_RAISE 364 -#define TK_RESTRICT 365 -#define TK_ROW 366 -#define TK_STAR 367 -#define TK_STATEMENT 368 -#define TK_STRICT 369 -#define TK_STRING 370 -#define TK_TIMES 371 -#define TK_VALUES 372 -#define TK_VARIABLE 373 -#define TK_WAL 374 -#define TK_ENCODE 375 -#define TK_COMPRESS 376 -#define TK_LEVEL 377 +#define TK_FORCE_WINDOW_CLOSE 233 +#define TK_IGNORE 234 +#define TK_EXPIRED 235 +#define TK_FILL_HISTORY 236 +#define TK_UPDATE 237 +#define TK_SUBTABLE 238 +#define TK_UNTREATED 239 +#define TK_KILL 240 +#define TK_CONNECTION 241 +#define TK_TRANSACTION 242 +#define TK_BALANCE 243 +#define TK_VGROUP 244 +#define TK_LEADER 245 +#define TK_MERGE 246 +#define TK_REDISTRIBUTE 247 +#define TK_SPLIT 248 +#define TK_DELETE 249 +#define TK_INSERT 250 +#define TK_NK_BIN 251 +#define TK_NK_HEX 252 +#define TK_NULL 253 +#define TK_NK_QUESTION 254 +#define TK_NK_ALIAS 255 +#define TK_NK_ARROW 256 +#define TK_ROWTS 257 +#define TK_QSTART 258 +#define TK_QEND 259 +#define TK_QDURATION 260 +#define TK_WSTART 261 +#define TK_WEND 262 +#define TK_WDURATION 263 +#define TK_IROWTS 264 +#define TK_ISFILLED 265 +#define TK_CAST 266 +#define TK_NOW 267 +#define TK_TODAY 268 +#define TK_TIMEZONE 269 +#define TK_CLIENT_VERSION 270 +#define TK_SERVER_VERSION 271 +#define TK_SERVER_STATUS 272 +#define TK_CURRENT_USER 273 +#define TK_CASE 274 +#define TK_WHEN 275 +#define TK_THEN 276 +#define TK_ELSE 277 +#define TK_BETWEEN 278 +#define TK_IS 279 +#define TK_NK_LT 280 +#define TK_NK_GT 281 +#define TK_NK_LE 282 +#define TK_NK_GE 283 +#define TK_NK_NE 284 +#define TK_MATCH 285 +#define TK_NMATCH 286 +#define TK_CONTAINS 287 +#define TK_IN 288 +#define TK_JOIN 289 +#define TK_INNER 290 +#define TK_LEFT 291 +#define TK_RIGHT 292 +#define TK_OUTER 293 +#define TK_SEMI 294 +#define TK_ANTI 295 +#define TK_ASOF 296 +#define TK_WINDOW 297 +#define TK_WINDOW_OFFSET 298 +#define TK_JLIMIT 299 +#define TK_SELECT 300 +#define TK_NK_HINT 301 +#define TK_DISTINCT 302 +#define TK_WHERE 303 +#define TK_PARTITION 304 +#define TK_BY 305 +#define TK_SESSION 306 +#define TK_STATE_WINDOW 307 +#define TK_EVENT_WINDOW 308 +#define TK_COUNT_WINDOW 309 +#define TK_SLIDING 310 +#define TK_FILL 311 +#define TK_VALUE 312 +#define TK_VALUE_F 313 +#define TK_NONE 314 +#define TK_PREV 315 +#define TK_NULL_F 316 +#define TK_LINEAR 317 +#define TK_NEXT 318 +#define TK_HAVING 319 +#define TK_RANGE 320 +#define TK_EVERY 321 +#define TK_ORDER 322 +#define TK_SLIMIT 323 +#define TK_SOFFSET 324 +#define TK_LIMIT 325 +#define TK_OFFSET 326 +#define TK_ASC 327 +#define TK_NULLS 328 +#define TK_ABORT 329 +#define TK_AFTER 330 +#define TK_ATTACH 331 +#define TK_BEFORE 332 +#define TK_BEGIN 333 +#define TK_BITAND 334 +#define TK_BITNOT 335 +#define TK_BITOR 336 +#define TK_BLOCKS 337 +#define TK_CHANGE 338 +#define TK_COMMA 339 +#define TK_CONCAT 340 +#define TK_CONFLICT 341 +#define TK_COPY 342 +#define TK_DEFERRED 343 +#define TK_DELIMITERS 344 +#define TK_DETACH 345 +#define TK_DIVIDE 346 +#define TK_DOT 347 +#define TK_EACH 348 +#define TK_FAIL 349 +#define TK_FOR 350 +#define TK_GLOB 351 +#define TK_ID 352 +#define TK_IMMEDIATE 353 +#define TK_IMPORT 354 +#define TK_INITIALLY 355 +#define TK_INSTEAD 356 +#define TK_ISNULL 357 +#define TK_MODULES 358 +#define TK_NK_BITNOT 359 +#define TK_NK_SEMI 360 +#define TK_NOTNULL 361 +#define TK_OF 362 +#define TK_PLUS 363 +#define TK_PRIVILEGE 364 +#define TK_RAISE 365 +#define TK_RESTRICT 366 +#define TK_ROW 367 +#define TK_STAR 368 +#define TK_STATEMENT 369 +#define TK_STRICT 370 +#define TK_STRING 371 +#define TK_TIMES 372 +#define TK_VALUES 373 +#define TK_VARIABLE 374 +#define TK_WAL 375 +#define TK_ENCODE 376 +#define TK_COMPRESS 377 +#define TK_LEVEL 378 #endif /**************** End token definitions ***************************************/ @@ -483,30 +481,30 @@ #endif /************* Begin control #defines *****************************************/ #define YYCODETYPE unsigned short int -#define YYNOCODE 558 +#define YYNOCODE 559 #define YYACTIONTYPE unsigned short int #define ParseTOKENTYPE SToken typedef union { int yyinit; ParseTOKENTYPE yy0; - SAlterOption yy101; - bool yy209; - SNodeList* yy316; - SNode* yy416; - EOrder yy506; - EJoinSubType yy630; - EShowKind yy681; - int32_t yy820; - EOperatorType yy848; - STokenPair yy849; - EFillMode yy882; - SShowTablesOption yy925; - SDataType yy952; - EJoinType yy972; - int8_t yy1043; - ENullOrder yy1045; - int64_t yy1089; - SToken yy1109; + SNode* yy54; + SDataType yy84; + EJoinType yy230; + bool yy325; + int32_t yy332; + EFillMode yy478; + STokenPair yy495; + ENullOrder yy503; + int8_t yy535; + SShowTablesOption yy579; + SNodeList* yy652; + SAlterOption yy663; + EShowKind yy719; + EOrder yy760; + SToken yy837; + int64_t yy909; + EOperatorType yy922; + EJoinSubType yy948; } YYMINORTYPE; #ifndef YYSTACKDEPTH #define YYSTACKDEPTH 100 @@ -523,17 +521,17 @@ typedef union { #define ParseCTX_STORE #define YYFALLBACK 1 #define YYNSTATE 979 -#define YYNRULE 755 -#define YYNRULE_WITH_ACTION 755 -#define YYNTOKEN 378 +#define YYNRULE 756 +#define YYNRULE_WITH_ACTION 756 +#define YYNTOKEN 379 #define YY_MAX_SHIFT 978 -#define YY_MIN_SHIFTREDUCE 1448 -#define YY_MAX_SHIFTREDUCE 2202 -#define YY_ERROR_ACTION 2203 -#define YY_ACCEPT_ACTION 2204 -#define YY_NO_ACTION 2205 -#define YY_MIN_REDUCE 2206 -#define YY_MAX_REDUCE 2960 +#define YY_MIN_SHIFTREDUCE 1449 +#define YY_MAX_SHIFTREDUCE 2204 +#define YY_ERROR_ACTION 2205 +#define YY_ACCEPT_ACTION 2206 +#define YY_NO_ACTION 2207 +#define YY_MIN_REDUCE 2208 +#define YY_MAX_REDUCE 2963 /************* End control #defines *******************************************/ #define YY_NLOOKAHEAD ((int)(sizeof(yy_lookahead)/sizeof(yy_lookahead[0]))) @@ -600,900 +598,936 @@ typedef union { ** yy_default[] Default action for each state. ** *********** Begin parsing tables **********************************************/ -#define YY_ACTTAB_COUNT (3018) +#define YY_ACTTAB_COUNT (3192) static const YYACTIONTYPE yy_action[] = { - /* 0 */ 2711, 2403, 647, 543, 2569, 648, 2254, 2569, 542, 37, - /* 10 */ 340, 550, 47, 45, 2119, 2401, 2736, 652, 798, 211, - /* 20 */ 476, 471, 1940, 649, 2567, 855, 2926, 2566, 855, 2715, - /* 30 */ 2711, 813, 124, 810, 157, 2461, 1938, 787, 2028, 2405, - /* 40 */ 1965, 495, 2416, 2694, 797, 218, 493, 40, 39, 2927, - /* 50 */ 799, 46, 44, 43, 42, 41, 2736, 764, 748, 2715, - /* 60 */ 2754, 612, 610, 2123, 415, 2926, 2754, 232, 2023, 1965, - /* 70 */ 788, 851, 868, 502, 501, 19, 2701, 527, 850, 2717, - /* 80 */ 2719, 473, 1946, 2932, 218, 810, 157, 2569, 2927, 799, - /* 90 */ 873, 2645, 460, 2616, 1966, 47, 45, 1947, 810, 157, - /* 100 */ 2754, 478, 482, 476, 2695, 1940, 482, 2566, 855, 2717, - /* 110 */ 2720, 670, 975, 873, 435, 15, 2701, 873, 850, 1938, - /* 120 */ 873, 2028, 868, 2735, 810, 157, 2774, 43, 42, 41, - /* 130 */ 121, 2737, 854, 2739, 2740, 849, 1965, 873, 786, 2204, - /* 140 */ 657, 2608, 200, 655, 2828, 515, 648, 2254, 472, 2824, - /* 150 */ 2229, 2023, 2030, 2031, 812, 187, 2836, 2837, 19, 155, - /* 160 */ 2841, 14, 13, 2735, 2057, 1946, 2774, 482, 219, 114, - /* 170 */ 121, 2737, 854, 2739, 2740, 849, 2875, 873, 873, 2736, - /* 180 */ 159, 488, 168, 2799, 2828, 869, 2414, 929, 472, 2824, - /* 190 */ 2373, 2001, 2011, 254, 851, 975, 2264, 650, 15, 2262, - /* 200 */ 256, 2029, 2032, 841, 650, 148, 2262, 194, 2836, 809, - /* 210 */ 2701, 149, 808, 693, 840, 531, 1941, 50, 1939, 2926, - /* 220 */ 127, 2836, 2837, 2754, 155, 2841, 1712, 1713, 508, 2058, - /* 230 */ 2474, 62, 868, 507, 868, 2030, 2031, 797, 218, 2701, - /* 240 */ 2002, 850, 2927, 799, 533, 529, 188, 2836, 2837, 817, - /* 250 */ 155, 2841, 1944, 1945, 1998, 172, 2000, 2003, 2004, 2005, - /* 260 */ 2006, 2007, 2008, 2009, 2010, 846, 871, 870, 2022, 2024, - /* 270 */ 2025, 2026, 2027, 2, 2001, 2011, 2071, 2094, 443, 1950, - /* 280 */ 9, 1968, 764, 207, 2029, 2032, 2735, 331, 332, 2774, - /* 290 */ 2926, 491, 330, 121, 2737, 854, 2739, 2740, 849, 1941, - /* 300 */ 873, 1939, 2301, 1999, 1998, 2946, 747, 2828, 2932, 218, - /* 310 */ 2542, 472, 2824, 2927, 799, 2418, 36, 474, 2052, 2053, - /* 320 */ 2054, 2055, 2056, 2060, 2061, 2062, 2063, 50, 910, 174, - /* 330 */ 173, 907, 906, 905, 171, 1944, 1945, 1998, 54, 2000, - /* 340 */ 2003, 2004, 2005, 2006, 2007, 2008, 2009, 2010, 846, 871, - /* 350 */ 870, 2022, 2024, 2025, 2026, 2027, 2, 12, 47, 45, - /* 360 */ 2398, 422, 2736, 1909, 2474, 183, 476, 291, 1940, 388, - /* 370 */ 394, 290, 441, 2451, 869, 2414, 744, 813, 2228, 714, - /* 380 */ 713, 712, 1938, 2472, 2028, 667, 704, 154, 708, 392, - /* 390 */ 76, 2094, 707, 75, 148, 487, 486, 706, 711, 453, - /* 400 */ 452, 324, 698, 705, 421, 2207, 2754, 451, 701, 700, - /* 410 */ 699, 2091, 2092, 2093, 2023, 324, 252, 629, 627, 624, - /* 420 */ 622, 19, 2701, 192, 850, 317, 138, 663, 1946, 137, - /* 430 */ 136, 135, 134, 133, 132, 131, 130, 129, 2701, 2931, - /* 440 */ 2474, 47, 45, 2033, 2556, 103, 668, 2926, 469, 476, - /* 450 */ 430, 1940, 1966, 458, 322, 736, 2474, 1965, 975, 2472, - /* 460 */ 798, 15, 62, 1493, 481, 1938, 2930, 2028, 2926, 2735, - /* 470 */ 2927, 2929, 2774, 1908, 926, 2472, 121, 2737, 854, 2739, - /* 480 */ 2740, 849, 1500, 873, 869, 2414, 797, 218, 200, 381, - /* 490 */ 2828, 2927, 799, 666, 472, 2824, 2388, 2023, 2030, 2031, - /* 500 */ 185, 63, 2218, 2094, 499, 490, 489, 1495, 1498, 1499, - /* 510 */ 138, 1946, 207, 137, 136, 135, 134, 133, 132, 131, - /* 520 */ 130, 129, 2876, 62, 2848, 2091, 2092, 2093, 2848, 2848, - /* 530 */ 2848, 2848, 2848, 734, 2931, 480, 182, 2001, 2011, 2543, - /* 540 */ 207, 975, 2926, 928, 48, 495, 2416, 2029, 2032, 728, - /* 550 */ 2617, 732, 730, 288, 287, 88, 87, 546, 2285, 783, - /* 560 */ 231, 2930, 1941, 1965, 1939, 2927, 2928, 2542, 869, 2414, - /* 570 */ 912, 40, 39, 538, 536, 46, 44, 43, 42, 41, - /* 580 */ 715, 2030, 2031, 1969, 1813, 1814, 418, 497, 223, 525, - /* 590 */ 2467, 2469, 521, 517, 513, 510, 539, 697, 1944, 1945, - /* 600 */ 1998, 696, 2000, 2003, 2004, 2005, 2006, 2007, 2008, 2009, - /* 610 */ 2010, 846, 871, 870, 2022, 2024, 2025, 2026, 2027, 2, - /* 620 */ 2001, 2011, 2096, 2097, 2098, 2099, 2100, 66, 1786, 1787, - /* 630 */ 2029, 2032, 100, 1812, 1815, 1673, 2848, 2091, 2092, 2093, - /* 640 */ 2848, 2848, 2848, 2848, 2848, 1941, 324, 1939, 2206, 62, - /* 650 */ 1664, 902, 901, 900, 1668, 899, 1670, 1671, 898, 895, - /* 660 */ 2410, 1679, 892, 1681, 1682, 889, 886, 883, 789, 784, - /* 670 */ 777, 773, 147, 146, 145, 144, 143, 142, 141, 140, - /* 680 */ 139, 1944, 1945, 1998, 238, 2000, 2003, 2004, 2005, 2006, - /* 690 */ 2007, 2008, 2009, 2010, 846, 871, 870, 2022, 2024, 2025, - /* 700 */ 2026, 2027, 2, 47, 45, 1623, 2736, 324, 869, 2414, - /* 710 */ 912, 476, 1969, 1940, 46, 44, 43, 42, 41, 1622, - /* 720 */ 1520, 851, 1519, 2883, 714, 713, 712, 1938, 55, 2028, - /* 730 */ 94, 704, 154, 708, 2002, 93, 153, 707, 62, 745, - /* 740 */ 869, 2414, 706, 711, 453, 452, 2931, 2736, 705, 541, - /* 750 */ 2754, 540, 451, 701, 700, 699, 214, 12, 1521, 2023, - /* 760 */ 547, 2038, 851, 1968, 2896, 479, 2701, 1965, 850, 2494, - /* 770 */ 879, 40, 39, 1946, 181, 46, 44, 43, 42, 41, - /* 780 */ 644, 498, 664, 539, 606, 2419, 47, 45, 748, 642, - /* 790 */ 181, 2754, 638, 634, 476, 617, 1940, 1999, 869, 2414, - /* 800 */ 2474, 2419, 92, 975, 559, 2538, 48, 2701, 496, 850, - /* 810 */ 1938, 2511, 2028, 2735, 2468, 2469, 2774, 1970, 548, 2472, - /* 820 */ 121, 2737, 854, 2739, 2740, 849, 1611, 873, 869, 2414, - /* 830 */ 565, 2538, 2946, 324, 2828, 869, 2414, 381, 472, 2824, - /* 840 */ 869, 2414, 2023, 2030, 2031, 839, 665, 2562, 567, 869, - /* 850 */ 2414, 2152, 243, 3, 2735, 581, 1946, 2774, 234, 2187, - /* 860 */ 669, 121, 2737, 854, 2739, 2740, 849, 53, 873, 582, - /* 870 */ 750, 2608, 1613, 2946, 1946, 2828, 100, 605, 242, 472, - /* 880 */ 2824, 51, 2001, 2011, 236, 213, 975, 596, 2538, 15, - /* 890 */ 903, 603, 2029, 2032, 446, 40, 39, 459, 2616, 46, - /* 900 */ 44, 43, 42, 41, 2409, 274, 12, 1941, 10, 1939, - /* 910 */ 780, 779, 2150, 2151, 2153, 2154, 2155, 878, 877, 876, - /* 920 */ 29, 193, 324, 2144, 589, 2521, 2030, 2031, 2059, 2227, - /* 930 */ 687, 683, 679, 675, 2390, 273, 322, 2145, 2002, 588, - /* 940 */ 834, 241, 2800, 1944, 1945, 1998, 1970, 2000, 2003, 2004, - /* 950 */ 2005, 2006, 2007, 2008, 2009, 2010, 846, 871, 870, 2022, - /* 960 */ 2024, 2025, 2026, 2027, 2, 2001, 2011, 84, 419, 557, - /* 970 */ 1963, 2194, 2474, 869, 2414, 2029, 2032, 594, 869, 2414, - /* 980 */ 445, 101, 587, 2143, 160, 2116, 271, 586, 614, 2701, - /* 990 */ 1941, 825, 1939, 583, 2406, 585, 869, 2414, 2411, 763, - /* 1000 */ 184, 1999, 292, 2164, 573, 836, 616, 2800, 2347, 869, - /* 1010 */ 2414, 420, 575, 40, 39, 34, 294, 46, 44, 43, - /* 1020 */ 42, 41, 2049, 553, 2843, 2064, 1944, 1945, 1998, 302, - /* 1030 */ 2000, 2003, 2004, 2005, 2006, 2007, 2008, 2009, 2010, 846, - /* 1040 */ 871, 870, 2022, 2024, 2025, 2026, 2027, 2, 869, 2414, - /* 1050 */ 2840, 40, 39, 259, 522, 46, 44, 43, 42, 41, - /* 1060 */ 1940, 1969, 270, 1502, 444, 324, 261, 268, 816, 1964, - /* 1070 */ 869, 2414, 266, 661, 1938, 113, 561, 2298, 225, 910, - /* 1080 */ 174, 173, 907, 906, 905, 171, 2736, 1867, 1868, 2193, - /* 1090 */ 335, 258, 40, 39, 1523, 1524, 46, 44, 43, 42, - /* 1100 */ 41, 851, 2407, 775, 689, 688, 2930, 2549, 2528, 1500, - /* 1110 */ 602, 601, 600, 599, 598, 593, 592, 591, 590, 427, - /* 1120 */ 1946, 2163, 580, 579, 578, 577, 576, 570, 569, 568, - /* 1130 */ 2754, 563, 562, 442, 2226, 1498, 1499, 554, 1774, 1775, - /* 1140 */ 60, 691, 690, 182, 1793, 78, 2701, 805, 850, 158, - /* 1150 */ 975, 761, 2799, 2417, 946, 945, 944, 943, 505, 2135, - /* 1160 */ 942, 941, 162, 936, 935, 934, 933, 932, 931, 930, - /* 1170 */ 161, 924, 923, 922, 504, 503, 919, 918, 917, 198, - /* 1180 */ 197, 916, 500, 915, 914, 913, 1969, 502, 501, 869, - /* 1190 */ 2414, 2736, 1965, 2735, 2701, 2225, 2774, 1954, 2224, 91, - /* 1200 */ 121, 2737, 854, 2739, 2740, 849, 851, 873, 2919, 830, - /* 1210 */ 721, 1947, 2946, 2028, 2828, 2223, 33, 2222, 472, 2824, - /* 1220 */ 2221, 2736, 40, 39, 2220, 735, 46, 44, 43, 42, - /* 1230 */ 41, 1627, 869, 2414, 221, 2754, 851, 104, 2860, 869, - /* 1240 */ 2414, 2641, 2217, 2023, 289, 1626, 869, 2414, 1520, 2476, - /* 1250 */ 1519, 2701, 342, 850, 1941, 2701, 1939, 1946, 2701, 862, - /* 1260 */ 724, 374, 2216, 450, 449, 2754, 863, 718, 716, 869, - /* 1270 */ 2414, 1841, 869, 2414, 286, 2701, 303, 2701, 710, 709, - /* 1280 */ 2701, 2701, 2115, 850, 2701, 2215, 1521, 838, 181, 867, - /* 1290 */ 1944, 1945, 370, 940, 938, 1970, 2474, 2214, 2735, 2420, - /* 1300 */ 2843, 2774, 2701, 2234, 968, 121, 2737, 854, 2739, 2740, - /* 1310 */ 849, 802, 873, 2213, 2622, 2473, 164, 2946, 2212, 2828, - /* 1320 */ 72, 619, 2701, 472, 2824, 71, 2839, 105, 2735, 2211, - /* 1330 */ 2210, 2774, 2654, 792, 150, 121, 2737, 854, 2739, 2740, - /* 1340 */ 849, 2736, 873, 448, 447, 2701, 695, 2946, 90, 2828, - /* 1350 */ 425, 424, 35, 472, 2824, 2209, 851, 2701, 40, 39, - /* 1360 */ 483, 380, 46, 44, 43, 42, 41, 697, 2196, 2197, - /* 1370 */ 2843, 696, 2736, 2701, 492, 293, 2028, 904, 2701, 908, - /* 1380 */ 2465, 909, 2465, 1606, 2465, 2754, 764, 851, 2391, 2701, - /* 1390 */ 2701, 1955, 279, 1950, 2926, 277, 2838, 2283, 738, 1857, - /* 1400 */ 737, 2701, 2105, 850, 615, 1949, 2023, 281, 2736, 164, - /* 1410 */ 280, 283, 2932, 218, 282, 2701, 2754, 2927, 799, 717, - /* 1420 */ 1970, 163, 2655, 851, 801, 299, 1999, 1958, 1960, 1607, - /* 1430 */ 285, 702, 2701, 284, 850, 387, 771, 49, 703, 806, - /* 1440 */ 2266, 871, 870, 2022, 2024, 2025, 2026, 2027, 2735, 2274, - /* 1450 */ 2272, 2774, 2754, 2722, 1604, 121, 2737, 854, 2739, 2740, - /* 1460 */ 849, 1602, 873, 49, 201, 172, 2418, 2803, 2701, 2828, - /* 1470 */ 850, 719, 722, 472, 2824, 2387, 764, 14, 13, 2735, - /* 1480 */ 212, 119, 2774, 116, 2926, 209, 121, 2737, 854, 2739, - /* 1490 */ 2740, 849, 1865, 873, 1948, 964, 329, 77, 2801, 64, - /* 1500 */ 2828, 49, 2932, 218, 472, 2824, 2348, 2927, 799, 1584, - /* 1510 */ 49, 77, 169, 349, 348, 2735, 2219, 2724, 2774, 2139, - /* 1520 */ 2149, 920, 121, 2737, 854, 2739, 2740, 849, 2889, 873, - /* 1530 */ 150, 172, 351, 350, 835, 192, 2828, 2736, 353, 352, - /* 1540 */ 472, 2824, 355, 354, 1576, 1673, 2148, 308, 815, 318, - /* 1550 */ 357, 356, 851, 781, 1931, 1585, 1907, 811, 359, 358, - /* 1560 */ 1664, 902, 901, 900, 1668, 899, 1670, 1671, 845, 844, - /* 1570 */ 310, 1679, 843, 1681, 1682, 842, 886, 883, 746, 333, - /* 1580 */ 822, 2754, 2065, 1557, 2012, 361, 360, 1952, 485, 484, - /* 1590 */ 1932, 363, 362, 1810, 1800, 345, 978, 2701, 74, 850, - /* 1600 */ 881, 803, 365, 364, 871, 870, 2022, 2024, 2025, 2026, - /* 1610 */ 2027, 921, 741, 866, 1655, 378, 367, 366, 2736, 170, - /* 1620 */ 910, 174, 173, 907, 906, 905, 171, 369, 368, 1558, - /* 1630 */ 966, 208, 749, 851, 1574, 152, 172, 2755, 2340, 2255, - /* 1640 */ 962, 958, 954, 950, 2735, 373, 151, 2774, 2879, 778, - /* 1650 */ 2339, 122, 2737, 854, 2739, 2740, 849, 169, 873, 465, - /* 1660 */ 785, 819, 2754, 461, 2547, 2828, 764, 506, 814, 2827, - /* 1670 */ 2824, 524, 2261, 2462, 2926, 793, 1951, 757, 2701, 794, - /* 1680 */ 850, 386, 2880, 1686, 2890, 320, 764, 2548, 315, 323, - /* 1690 */ 514, 120, 2932, 218, 2926, 2374, 346, 2927, 799, 2736, - /* 1700 */ 40, 39, 1694, 5, 46, 44, 43, 42, 41, 439, - /* 1710 */ 509, 1963, 2932, 218, 851, 1973, 523, 2927, 799, 1701, - /* 1720 */ 534, 227, 764, 2736, 226, 2735, 535, 826, 2774, 1699, - /* 1730 */ 2926, 537, 122, 2737, 854, 2739, 2740, 849, 848, 873, - /* 1740 */ 175, 229, 2736, 2754, 379, 1834, 2828, 551, 2932, 218, - /* 1750 */ 837, 2824, 1964, 2927, 799, 558, 240, 851, 560, 2701, - /* 1760 */ 564, 850, 566, 571, 608, 584, 2540, 2754, 595, 597, - /* 1770 */ 604, 607, 344, 609, 832, 620, 621, 327, 618, 246, - /* 1780 */ 245, 623, 326, 2701, 625, 850, 2754, 626, 249, 628, - /* 1790 */ 630, 1971, 4, 645, 656, 653, 257, 1966, 646, 658, - /* 1800 */ 654, 296, 2701, 96, 850, 1972, 852, 659, 1974, 2774, - /* 1810 */ 2736, 662, 660, 122, 2737, 854, 2739, 2740, 849, 260, - /* 1820 */ 873, 263, 1975, 265, 97, 851, 1976, 2828, 98, 2557, - /* 1830 */ 2735, 434, 2824, 2774, 2563, 692, 671, 410, 2737, 854, - /* 1840 */ 2739, 2740, 849, 847, 873, 833, 2793, 99, 272, 2735, - /* 1850 */ 2736, 694, 2774, 2404, 2754, 126, 186, 2737, 854, 2739, - /* 1860 */ 2740, 849, 276, 873, 2400, 851, 725, 278, 726, 413, - /* 1870 */ 2701, 177, 850, 123, 740, 2736, 2402, 2397, 178, 179, - /* 1880 */ 102, 742, 295, 1967, 751, 125, 2631, 752, 382, 758, - /* 1890 */ 851, 2609, 165, 300, 2754, 759, 2628, 2627, 753, 782, - /* 1900 */ 756, 820, 298, 765, 2886, 8, 768, 2867, 791, 766, - /* 1910 */ 2701, 769, 850, 767, 2895, 2894, 191, 2735, 2736, 2754, - /* 1920 */ 2774, 309, 305, 307, 189, 2737, 854, 2739, 2740, 849, - /* 1930 */ 796, 873, 311, 851, 313, 2701, 795, 850, 314, 466, - /* 1940 */ 2949, 807, 2925, 2847, 2736, 804, 156, 312, 2113, 316, - /* 1950 */ 1968, 2111, 204, 325, 166, 818, 383, 2735, 2577, 851, - /* 1960 */ 2774, 2844, 2754, 2576, 186, 2737, 854, 2739, 2740, 849, - /* 1970 */ 1, 873, 384, 2575, 470, 823, 824, 167, 2701, 831, - /* 1980 */ 850, 828, 2735, 800, 2947, 2774, 2736, 338, 2754, 122, - /* 1990 */ 2737, 854, 2739, 2740, 849, 61, 873, 2809, 858, 856, - /* 2000 */ 860, 851, 463, 2828, 2701, 385, 850, 861, 2825, 319, - /* 2010 */ 343, 112, 2887, 220, 2415, 115, 389, 2693, 1472, 2692, - /* 2020 */ 971, 2688, 2687, 2736, 2679, 2735, 875, 970, 2774, 2678, - /* 2030 */ 2754, 372, 411, 2737, 854, 2739, 2740, 849, 851, 873, - /* 2040 */ 2670, 2669, 2685, 2684, 176, 972, 2701, 375, 850, 967, - /* 2050 */ 376, 2735, 2676, 52, 2774, 2675, 974, 2664, 404, 2737, - /* 2060 */ 854, 2739, 2740, 849, 393, 873, 2663, 2754, 2682, 2681, - /* 2070 */ 464, 2673, 2672, 423, 2661, 2660, 2658, 414, 391, 2657, - /* 2080 */ 744, 2466, 431, 2701, 432, 850, 426, 401, 412, 402, - /* 2090 */ 2653, 2652, 2651, 2735, 85, 2646, 2774, 511, 512, 1891, - /* 2100 */ 411, 2737, 854, 2739, 2740, 849, 1892, 873, 224, 790, - /* 2110 */ 2736, 516, 2644, 518, 519, 520, 1890, 2643, 2642, 440, - /* 2120 */ 2640, 526, 2639, 528, 2638, 848, 530, 2637, 1878, 532, - /* 2130 */ 2735, 2613, 2736, 2774, 228, 2612, 230, 189, 2737, 854, - /* 2140 */ 2739, 2740, 849, 86, 873, 1837, 2590, 851, 1836, 2589, - /* 2150 */ 2588, 544, 545, 2587, 2754, 2586, 2530, 549, 1773, 2736, - /* 2160 */ 2527, 552, 2526, 2520, 555, 556, 2517, 233, 2516, 2515, - /* 2170 */ 2701, 2514, 850, 89, 851, 2519, 2754, 2518, 2513, 2512, - /* 2180 */ 235, 2510, 2509, 2508, 572, 2507, 237, 574, 2505, 2504, - /* 2190 */ 2503, 2502, 2701, 2501, 850, 2525, 2736, 2948, 2500, 2499, - /* 2200 */ 2498, 2523, 2506, 2754, 2497, 2496, 2495, 2493, 2492, 2491, - /* 2210 */ 2490, 851, 2489, 2488, 2487, 239, 475, 2735, 2486, 2701, - /* 2220 */ 2774, 850, 2485, 2736, 410, 2737, 854, 2739, 2740, 849, - /* 2230 */ 2484, 873, 2483, 2794, 95, 2555, 2524, 2522, 851, 2735, - /* 2240 */ 2754, 2482, 2774, 477, 2481, 2736, 411, 2737, 854, 2739, - /* 2250 */ 2740, 849, 2480, 873, 244, 1779, 2701, 2479, 850, 611, - /* 2260 */ 851, 2478, 2477, 2475, 613, 1624, 2735, 2754, 428, 2774, - /* 2270 */ 1628, 429, 2736, 411, 2737, 854, 2739, 2740, 849, 2305, - /* 2280 */ 873, 2304, 2303, 2701, 2302, 850, 1620, 851, 247, 2754, - /* 2290 */ 2300, 248, 250, 251, 2297, 631, 632, 633, 2296, 636, - /* 2300 */ 635, 2289, 637, 739, 639, 2701, 2774, 850, 640, 2736, - /* 2310 */ 406, 2737, 854, 2739, 2740, 849, 2754, 873, 641, 2276, - /* 2320 */ 2250, 643, 253, 2721, 851, 199, 1501, 81, 210, 651, - /* 2330 */ 2735, 2249, 2701, 2774, 850, 2611, 255, 396, 2737, 854, - /* 2340 */ 2739, 2740, 849, 82, 873, 2607, 2597, 2585, 262, 264, - /* 2350 */ 2736, 267, 2735, 2754, 2584, 2774, 2561, 269, 2554, 395, - /* 2360 */ 2737, 854, 2739, 2740, 849, 851, 873, 2392, 2299, 2701, - /* 2370 */ 2295, 850, 1550, 672, 674, 673, 2293, 676, 677, 2735, - /* 2380 */ 678, 2291, 2774, 681, 680, 2736, 397, 2737, 854, 2739, - /* 2390 */ 2740, 849, 2288, 873, 2754, 684, 682, 685, 2271, 686, - /* 2400 */ 851, 2269, 2270, 2268, 2246, 2394, 1705, 1706, 2393, 1610, - /* 2410 */ 2701, 1609, 850, 73, 1608, 1605, 2735, 2736, 275, 2774, - /* 2420 */ 2286, 1603, 1601, 403, 2737, 854, 2739, 2740, 849, 2754, - /* 2430 */ 873, 1600, 851, 1599, 2284, 1598, 2736, 1592, 454, 455, - /* 2440 */ 937, 2275, 1597, 939, 456, 2701, 720, 850, 1594, 1593, - /* 2450 */ 1591, 851, 2273, 457, 2245, 723, 2244, 2735, 2243, 727, - /* 2460 */ 2774, 2754, 2242, 729, 407, 2737, 854, 2739, 2740, 849, - /* 2470 */ 2241, 873, 731, 2240, 1872, 733, 1874, 2701, 1871, 850, - /* 2480 */ 2754, 128, 2736, 2610, 1862, 1876, 56, 28, 743, 67, - /* 2490 */ 1847, 2606, 2735, 1843, 297, 2774, 2701, 851, 850, 398, - /* 2500 */ 2737, 854, 2739, 2740, 849, 2596, 873, 57, 1845, 754, - /* 2510 */ 755, 2583, 301, 2582, 2736, 1822, 180, 2931, 760, 1821, - /* 2520 */ 770, 762, 20, 6, 2735, 2166, 2754, 2774, 462, 851, - /* 2530 */ 17, 408, 2737, 854, 2739, 2740, 849, 2140, 873, 30, - /* 2540 */ 304, 772, 2701, 2735, 850, 774, 2774, 7, 215, 306, - /* 2550 */ 399, 2737, 854, 2739, 2740, 849, 776, 873, 2754, 21, - /* 2560 */ 22, 203, 2147, 190, 202, 31, 32, 2134, 2722, 23, - /* 2570 */ 83, 216, 2106, 2108, 2701, 2104, 850, 65, 2736, 217, - /* 2580 */ 24, 2186, 2187, 2088, 18, 2181, 2180, 467, 2185, 2735, - /* 2590 */ 2184, 468, 2774, 851, 2087, 321, 409, 2737, 854, 2739, - /* 2600 */ 2740, 849, 2736, 873, 58, 59, 2581, 195, 2560, 106, - /* 2610 */ 107, 2559, 328, 108, 2142, 205, 334, 851, 69, 2553, - /* 2620 */ 827, 2735, 2754, 337, 2774, 821, 829, 2736, 400, 2737, - /* 2630 */ 854, 2739, 2740, 849, 109, 873, 25, 2040, 2701, 336, - /* 2640 */ 850, 11, 851, 2039, 13, 1956, 2754, 2015, 2050, 196, - /* 2650 */ 2014, 888, 206, 1991, 891, 339, 894, 897, 38, 2552, - /* 2660 */ 110, 2013, 2701, 16, 850, 859, 853, 2736, 26, 2389, - /* 2670 */ 864, 2754, 347, 1983, 27, 70, 865, 857, 341, 111, - /* 2680 */ 79, 880, 851, 2779, 116, 2735, 2778, 2701, 2774, 850, - /* 2690 */ 872, 2736, 416, 2737, 854, 2739, 2740, 849, 2017, 873, - /* 2700 */ 68, 2202, 874, 2201, 2199, 2200, 851, 1687, 494, 2735, - /* 2710 */ 882, 2754, 2774, 1684, 884, 885, 417, 2737, 854, 2739, - /* 2720 */ 2740, 849, 887, 873, 890, 1683, 1680, 2701, 1674, 850, - /* 2730 */ 893, 1672, 2736, 896, 2735, 2754, 1678, 2774, 371, 1677, - /* 2740 */ 117, 2748, 2737, 854, 2739, 2740, 849, 851, 873, 118, - /* 2750 */ 1676, 2701, 1700, 850, 1675, 80, 1696, 1588, 1548, 911, - /* 2760 */ 1587, 1586, 1583, 1580, 1579, 1578, 1577, 1575, 1573, 1572, - /* 2770 */ 925, 1618, 2736, 1571, 2735, 1617, 2754, 2774, 222, 927, - /* 2780 */ 1569, 2747, 2737, 854, 2739, 2740, 849, 851, 873, 1568, - /* 2790 */ 1567, 1566, 2701, 1565, 850, 1564, 1563, 1614, 2735, 2736, - /* 2800 */ 1612, 2774, 1560, 1559, 1556, 2746, 2737, 854, 2739, 2740, - /* 2810 */ 849, 1555, 873, 1554, 851, 1553, 2754, 2294, 2736, 947, - /* 2820 */ 949, 2292, 951, 2290, 948, 952, 953, 956, 955, 957, - /* 2830 */ 2287, 959, 2701, 851, 850, 960, 2267, 961, 963, 2735, - /* 2840 */ 2265, 965, 2774, 2754, 1490, 2239, 436, 2737, 854, 2739, - /* 2850 */ 2740, 849, 1478, 873, 1473, 969, 1480, 377, 973, 2701, - /* 2860 */ 1942, 850, 2754, 390, 976, 2205, 977, 2205, 2205, 2205, - /* 2870 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2701, 2735, - /* 2880 */ 850, 2205, 2774, 2736, 2205, 2205, 437, 2737, 854, 2739, - /* 2890 */ 2740, 849, 2205, 873, 2205, 2205, 2736, 2205, 851, 2205, - /* 2900 */ 2205, 2205, 2205, 2205, 2205, 2205, 2735, 2205, 2205, 2774, - /* 2910 */ 2205, 851, 2205, 433, 2737, 854, 2739, 2740, 849, 2205, - /* 2920 */ 873, 2205, 2205, 2205, 2205, 2735, 2205, 2754, 2774, 2205, - /* 2930 */ 2205, 2205, 438, 2737, 854, 2739, 2740, 849, 2205, 873, - /* 2940 */ 2754, 2205, 2205, 2701, 2205, 850, 2205, 2205, 2205, 2205, - /* 2950 */ 2205, 2205, 2205, 2205, 2205, 2205, 2701, 2205, 850, 2205, - /* 2960 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, - /* 2970 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, - /* 2980 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, - /* 2990 */ 852, 2205, 2205, 2774, 2205, 2205, 2205, 406, 2737, 854, - /* 3000 */ 2739, 2740, 849, 2735, 873, 2205, 2774, 2205, 2205, 2205, - /* 3010 */ 405, 2737, 854, 2739, 2740, 849, 2205, 873, + /* 0 */ 543, 559, 2540, 184, 2697, 542, 647, 493, 841, 648, + /* 10 */ 2256, 2349, 47, 45, 2121, 869, 2416, 869, 2416, 840, + /* 20 */ 476, 2209, 1942, 40, 39, 565, 2540, 46, 44, 43, + /* 30 */ 42, 41, 37, 340, 1501, 148, 1940, 499, 2030, 2300, + /* 40 */ 2407, 374, 138, 693, 2739, 137, 136, 135, 134, 133, + /* 50 */ 132, 131, 130, 129, 764, 234, 810, 157, 9, 813, + /* 60 */ 1499, 1500, 2929, 869, 2416, 40, 39, 482, 2025, 46, + /* 70 */ 44, 43, 42, 41, 868, 19, 2714, 1971, 873, 236, + /* 80 */ 2935, 218, 1948, 148, 1967, 2930, 799, 2571, 2757, 929, + /* 90 */ 138, 698, 2375, 137, 136, 135, 134, 133, 132, 131, + /* 100 */ 130, 129, 2476, 482, 2704, 2718, 850, 2569, 855, 868, + /* 110 */ 441, 2698, 975, 2231, 873, 15, 946, 945, 944, 943, + /* 120 */ 505, 2474, 942, 941, 162, 936, 935, 934, 933, 932, + /* 130 */ 931, 930, 161, 924, 923, 922, 504, 503, 919, 918, + /* 140 */ 917, 198, 197, 916, 500, 915, 914, 913, 211, 596, + /* 150 */ 2540, 2738, 2032, 2033, 2777, 2720, 2722, 473, 121, 2740, + /* 160 */ 854, 2742, 2743, 849, 2463, 873, 873, 66, 734, 2096, + /* 170 */ 200, 114, 2831, 2704, 482, 1494, 472, 2827, 194, 2839, + /* 180 */ 809, 62, 149, 808, 728, 873, 732, 730, 288, 287, + /* 190 */ 2929, 2003, 2013, 254, 1501, 182, 219, 650, 798, 2264, + /* 200 */ 2846, 2031, 2034, 241, 2878, 2419, 2929, 655, 797, 218, + /* 210 */ 648, 2256, 51, 2930, 799, 2208, 1943, 443, 1941, 1496, + /* 220 */ 1499, 1500, 207, 2196, 797, 218, 2843, 40, 39, 2930, + /* 230 */ 799, 46, 44, 43, 42, 41, 2096, 550, 1970, 147, + /* 240 */ 146, 145, 144, 143, 142, 141, 140, 139, 62, 2544, + /* 250 */ 214, 331, 332, 1946, 1947, 2000, 330, 2002, 2005, 2006, + /* 260 */ 2007, 2008, 2009, 2010, 2011, 2012, 846, 871, 870, 2024, + /* 270 */ 2026, 2027, 2028, 2029, 2, 47, 45, 2571, 62, 2657, + /* 280 */ 419, 670, 1965, 476, 2303, 1942, 124, 612, 610, 594, + /* 290 */ 415, 471, 445, 232, 435, 495, 2418, 2568, 855, 1940, + /* 300 */ 614, 2030, 100, 2851, 2093, 2094, 2095, 2851, 2851, 2851, + /* 310 */ 2851, 2851, 1972, 502, 501, 33, 573, 747, 616, 2934, + /* 320 */ 446, 40, 39, 420, 575, 46, 44, 43, 42, 41, + /* 330 */ 2411, 2025, 2476, 764, 2420, 553, 1970, 1949, 19, 1967, + /* 340 */ 469, 2929, 2195, 721, 2059, 1948, 460, 2618, 541, 667, + /* 350 */ 540, 2474, 2739, 2098, 2099, 2100, 2101, 2102, 735, 2935, + /* 360 */ 218, 714, 713, 712, 2930, 799, 324, 813, 704, 154, + /* 370 */ 708, 2093, 2094, 2095, 707, 975, 444, 289, 15, 706, + /* 380 */ 711, 453, 452, 539, 788, 705, 2934, 2268, 561, 451, + /* 390 */ 701, 700, 699, 724, 2929, 744, 2757, 324, 40, 39, + /* 400 */ 718, 716, 46, 44, 43, 42, 41, 286, 2558, 2060, + /* 410 */ 668, 322, 2704, 2933, 850, 2032, 2033, 2930, 2932, 2551, + /* 420 */ 2530, 2513, 602, 601, 600, 599, 598, 593, 592, 591, + /* 430 */ 590, 428, 2189, 324, 580, 579, 578, 577, 576, 570, + /* 440 */ 569, 568, 964, 563, 562, 442, 652, 868, 644, 554, + /* 450 */ 1775, 1776, 649, 72, 2003, 2013, 1794, 642, 71, 2738, + /* 460 */ 638, 634, 2777, 324, 2031, 2034, 121, 2740, 854, 2742, + /* 470 */ 2743, 849, 497, 873, 158, 2469, 2471, 2802, 200, 1943, + /* 480 */ 2831, 1941, 879, 904, 472, 2827, 2467, 2096, 40, 39, + /* 490 */ 810, 157, 46, 44, 43, 42, 41, 36, 474, 2054, + /* 500 */ 2055, 2056, 2057, 2058, 2062, 2063, 2064, 2065, 663, 322, + /* 510 */ 1787, 1788, 2879, 29, 2714, 926, 1946, 1947, 2000, 1952, + /* 520 */ 2002, 2005, 2006, 2007, 2008, 2009, 2010, 2011, 2012, 846, + /* 530 */ 871, 870, 2024, 2026, 2027, 2028, 2029, 2, 12, 47, + /* 540 */ 45, 2739, 50, 2718, 256, 666, 2934, 476, 650, 1942, + /* 550 */ 2264, 714, 713, 712, 2929, 868, 851, 291, 704, 154, + /* 560 */ 708, 290, 100, 1940, 707, 2030, 527, 2400, 422, 706, + /* 570 */ 711, 453, 452, 2933, 2739, 705, 792, 2930, 2931, 451, + /* 580 */ 701, 700, 699, 125, 928, 2757, 2571, 238, 2206, 851, + /* 590 */ 2412, 2266, 587, 2720, 2723, 2025, 2166, 586, 2470, 2471, + /* 600 */ 478, 2704, 19, 850, 873, 585, 2568, 855, 1968, 1948, + /* 610 */ 381, 812, 187, 2839, 2840, 2846, 155, 2844, 2757, 2000, + /* 620 */ 2393, 2851, 2093, 2094, 2095, 2851, 2851, 2851, 2851, 2851, + /* 630 */ 878, 877, 876, 94, 2704, 103, 850, 84, 93, 975, + /* 640 */ 424, 2842, 15, 606, 458, 2107, 736, 1521, 2738, 1520, + /* 650 */ 50, 2777, 1942, 163, 160, 121, 2740, 854, 2742, 2743, + /* 660 */ 849, 2154, 873, 153, 2408, 159, 1940, 168, 2802, 2831, + /* 670 */ 459, 2618, 2146, 472, 2827, 488, 2648, 508, 479, 2032, + /* 680 */ 2033, 2738, 507, 1968, 2777, 1522, 2147, 181, 121, 2740, + /* 690 */ 854, 2742, 2743, 849, 2392, 873, 14, 13, 2421, 192, + /* 700 */ 2949, 317, 2831, 491, 531, 92, 472, 2827, 2230, 40, + /* 710 */ 39, 243, 1948, 46, 44, 43, 42, 41, 2003, 2013, + /* 720 */ 780, 779, 2152, 2153, 2155, 2156, 2157, 2125, 2031, 2034, + /* 730 */ 515, 764, 2145, 1967, 533, 529, 605, 242, 2040, 2929, + /* 740 */ 1713, 1714, 975, 1943, 1967, 1941, 787, 292, 40, 39, + /* 750 */ 603, 2390, 46, 44, 43, 42, 41, 2935, 218, 35, + /* 760 */ 2229, 2739, 2930, 799, 1971, 40, 39, 839, 2704, 46, + /* 770 */ 44, 43, 42, 41, 480, 2757, 851, 748, 2886, 207, + /* 780 */ 1946, 1947, 2000, 1612, 2002, 2005, 2006, 2007, 2008, 2009, + /* 790 */ 2010, 2011, 2012, 846, 871, 870, 2024, 2026, 2027, 2028, + /* 800 */ 2029, 2, 47, 45, 2035, 2757, 2544, 2739, 912, 664, + /* 810 */ 476, 2073, 1942, 1967, 745, 1814, 1815, 2165, 869, 2416, + /* 820 */ 2704, 2704, 851, 850, 2899, 912, 1940, 2389, 2030, 1614, + /* 830 */ 450, 449, 1868, 1869, 1870, 324, 113, 2739, 223, 910, + /* 840 */ 174, 173, 907, 906, 905, 171, 1943, 786, 1941, 783, + /* 850 */ 1971, 2757, 851, 185, 775, 2220, 2137, 1911, 2025, 657, + /* 860 */ 2610, 810, 157, 2409, 1813, 1816, 2846, 2704, 2738, 850, + /* 870 */ 12, 2777, 1948, 665, 2564, 121, 2740, 854, 2742, 2743, + /* 880 */ 849, 2757, 873, 1946, 1947, 1910, 2739, 2949, 2476, 2831, + /* 890 */ 487, 486, 2841, 472, 2827, 1971, 481, 2704, 1967, 850, + /* 900 */ 1948, 851, 975, 2922, 2004, 48, 908, 2474, 62, 2467, + /* 910 */ 448, 447, 2476, 695, 2738, 2004, 181, 2777, 490, 489, + /* 920 */ 496, 121, 2740, 854, 2742, 2743, 849, 2422, 873, 2228, + /* 930 */ 2757, 2474, 903, 2949, 697, 2831, 2658, 213, 696, 472, + /* 940 */ 2827, 748, 2032, 2033, 2738, 2051, 2704, 2777, 850, 2227, + /* 950 */ 2405, 121, 2740, 854, 2742, 2743, 849, 3, 873, 789, + /* 960 */ 784, 777, 773, 2949, 2933, 2831, 869, 2416, 2001, 472, + /* 970 */ 2827, 53, 910, 174, 173, 907, 906, 905, 171, 2001, + /* 980 */ 1951, 2003, 2013, 127, 2839, 2840, 55, 155, 2844, 2704, + /* 990 */ 764, 2031, 2034, 2738, 2496, 763, 2777, 2350, 2929, 1972, + /* 1000 */ 121, 2740, 854, 2742, 2743, 849, 1943, 873, 1941, 2704, + /* 1010 */ 1674, 2226, 2949, 12, 2831, 10, 2935, 218, 472, 2827, + /* 1020 */ 381, 2930, 799, 750, 2610, 1665, 902, 901, 900, 1669, + /* 1030 */ 899, 1671, 1672, 898, 895, 2403, 1680, 892, 1682, 1683, + /* 1040 */ 889, 886, 883, 1946, 1947, 2000, 2523, 2002, 2005, 2006, + /* 1050 */ 2007, 2008, 2009, 2010, 2011, 2012, 846, 871, 870, 2024, + /* 1060 */ 2026, 2027, 2028, 2029, 2, 47, 45, 2739, 221, 2004, + /* 1070 */ 741, 2704, 749, 476, 498, 1942, 46, 44, 43, 42, + /* 1080 */ 41, 2619, 851, 181, 2863, 1972, 814, 869, 2416, 1940, + /* 1090 */ 557, 2030, 207, 324, 2421, 869, 2416, 909, 40, 39, + /* 1100 */ 2467, 2739, 46, 44, 43, 42, 41, 547, 869, 2416, + /* 1110 */ 380, 2757, 798, 810, 157, 862, 851, 869, 2416, 2545, + /* 1120 */ 2929, 2025, 869, 2416, 764, 2118, 764, 2704, 548, 850, + /* 1130 */ 1972, 299, 2929, 2001, 2929, 1948, 60, 567, 797, 218, + /* 1140 */ 764, 387, 581, 2930, 799, 2757, 212, 761, 2929, 589, + /* 1150 */ 2935, 218, 2935, 218, 2476, 2930, 799, 2930, 799, 869, + /* 1160 */ 2416, 2704, 1954, 850, 588, 975, 2935, 218, 48, 869, + /* 1170 */ 2416, 2930, 799, 817, 2738, 2644, 1521, 2777, 1520, 582, + /* 1180 */ 1624, 121, 2740, 854, 2742, 2743, 849, 1967, 873, 583, + /* 1190 */ 869, 2416, 182, 2949, 1623, 2831, 2624, 502, 501, 472, + /* 1200 */ 2827, 495, 2418, 1524, 1525, 2032, 2033, 1956, 2738, 2476, + /* 1210 */ 669, 2777, 869, 2416, 1522, 121, 2740, 854, 2742, 2743, + /* 1220 */ 849, 1949, 873, 2030, 689, 688, 1607, 2806, 825, 2831, + /* 1230 */ 869, 2416, 2413, 472, 2827, 188, 2839, 2840, 2225, 155, + /* 1240 */ 2844, 2224, 1628, 2223, 2003, 2013, 869, 2416, 869, 2416, + /* 1250 */ 294, 710, 709, 2025, 2031, 2034, 1627, 293, 1674, 910, + /* 1260 */ 174, 173, 907, 906, 905, 171, 302, 1948, 816, 1943, + /* 1270 */ 617, 1941, 1608, 1665, 902, 901, 900, 1669, 899, 1671, + /* 1280 */ 1672, 845, 844, 801, 1680, 843, 1682, 1683, 842, 886, + /* 1290 */ 883, 43, 42, 41, 2061, 691, 690, 838, 2704, 940, + /* 1300 */ 938, 2704, 2222, 2704, 2236, 968, 1946, 1947, 2000, 2219, + /* 1310 */ 2002, 2005, 2006, 2007, 2008, 2009, 2010, 2011, 2012, 846, + /* 1320 */ 871, 870, 2024, 2026, 2027, 2028, 2029, 2, 47, 45, + /* 1330 */ 869, 2416, 619, 869, 2416, 183, 476, 2218, 1942, 2217, + /* 1340 */ 394, 869, 2416, 388, 869, 2416, 2216, 2453, 869, 2416, + /* 1350 */ 335, 2215, 1940, 830, 2030, 869, 2416, 2214, 2213, 392, + /* 1360 */ 76, 342, 2704, 75, 863, 2739, 2212, 522, 867, 2704, + /* 1370 */ 2211, 2198, 2199, 104, 421, 370, 834, 836, 2803, 2803, + /* 1380 */ 851, 78, 34, 2287, 2025, 2739, 252, 629, 627, 624, + /* 1390 */ 622, 225, 2066, 2476, 2478, 172, 802, 2704, 1948, 2704, + /* 1400 */ 851, 1957, 1503, 1952, 164, 715, 2704, 1842, 1966, 2757, + /* 1410 */ 697, 2704, 2475, 279, 696, 805, 277, 2704, 2704, 150, + /* 1420 */ 209, 281, 2001, 2117, 280, 2704, 2704, 850, 975, 2757, + /* 1430 */ 2704, 15, 62, 90, 283, 91, 702, 282, 1960, 1962, + /* 1440 */ 426, 425, 703, 14, 13, 2704, 285, 850, 2285, 284, + /* 1450 */ 483, 2276, 871, 870, 2024, 2026, 2027, 2028, 2029, 1605, + /* 1460 */ 738, 303, 737, 2420, 492, 1603, 2030, 2274, 2032, 2033, + /* 1470 */ 717, 63, 2738, 719, 2221, 2777, 164, 2725, 54, 121, + /* 1480 */ 2740, 854, 2742, 2743, 849, 771, 873, 1858, 119, 722, + /* 1490 */ 116, 2804, 2738, 2831, 2739, 2777, 2025, 472, 2827, 186, + /* 1500 */ 2740, 854, 2742, 2743, 849, 2892, 873, 2003, 2013, 851, + /* 1510 */ 49, 49, 105, 49, 201, 172, 192, 2031, 2034, 329, + /* 1520 */ 349, 348, 318, 77, 1950, 88, 87, 546, 351, 350, + /* 1530 */ 231, 64, 1943, 781, 1941, 49, 77, 310, 2757, 811, + /* 1540 */ 169, 2727, 150, 538, 536, 172, 765, 2889, 152, 615, + /* 1550 */ 2758, 353, 352, 2342, 2704, 2341, 850, 418, 2257, 1866, + /* 1560 */ 525, 2882, 74, 521, 517, 513, 510, 539, 2141, 1946, + /* 1570 */ 1947, 2000, 778, 2002, 2005, 2006, 2007, 2008, 2009, 2010, + /* 1580 */ 2011, 2012, 846, 871, 870, 2024, 2026, 2027, 2028, 2029, + /* 1590 */ 2, 355, 354, 2151, 1811, 465, 2150, 308, 815, 357, + /* 1600 */ 356, 2738, 333, 746, 2777, 881, 822, 170, 121, 2740, + /* 1610 */ 854, 2742, 2743, 849, 2067, 873, 2739, 324, 2014, 1801, + /* 1620 */ 835, 978, 2831, 345, 172, 866, 472, 2827, 1656, 359, + /* 1630 */ 358, 851, 361, 360, 363, 362, 365, 364, 367, 366, + /* 1640 */ 378, 369, 368, 1585, 1933, 386, 1909, 1558, 920, 921, + /* 1650 */ 785, 461, 2549, 819, 506, 966, 208, 2263, 2739, 524, + /* 1660 */ 2757, 757, 2464, 151, 169, 962, 958, 954, 950, 2883, + /* 1670 */ 373, 1577, 1575, 851, 793, 320, 2704, 2893, 850, 485, + /* 1680 */ 484, 1934, 794, 323, 315, 2550, 2376, 803, 1687, 1586, + /* 1690 */ 1695, 509, 5, 1559, 2739, 871, 870, 2024, 2026, 2027, + /* 1700 */ 2028, 2029, 2757, 514, 439, 1965, 1953, 1702, 806, 848, + /* 1710 */ 1975, 523, 535, 534, 226, 227, 120, 537, 2704, 1835, + /* 1720 */ 850, 346, 379, 2738, 229, 551, 2777, 1966, 558, 240, + /* 1730 */ 122, 2740, 854, 2742, 2743, 849, 560, 873, 2757, 564, + /* 1740 */ 566, 608, 571, 2739, 2831, 584, 1700, 175, 2830, 2827, + /* 1750 */ 595, 2542, 826, 597, 2704, 604, 850, 607, 851, 609, + /* 1760 */ 621, 620, 618, 246, 245, 2738, 623, 625, 2777, 1973, + /* 1770 */ 249, 626, 122, 2740, 854, 2742, 2743, 849, 4, 873, + /* 1780 */ 628, 2739, 257, 630, 645, 646, 2831, 2757, 656, 653, + /* 1790 */ 837, 2827, 1968, 1974, 654, 96, 851, 344, 658, 832, + /* 1800 */ 659, 2738, 327, 2704, 2777, 850, 1976, 326, 410, 2740, + /* 1810 */ 854, 2742, 2743, 849, 847, 873, 833, 2796, 260, 263, + /* 1820 */ 660, 265, 662, 2739, 1977, 2757, 296, 2565, 671, 1978, + /* 1830 */ 97, 2559, 98, 692, 694, 99, 272, 2406, 851, 725, + /* 1840 */ 726, 2704, 740, 850, 276, 2402, 278, 177, 123, 126, + /* 1850 */ 852, 413, 2634, 2777, 2631, 102, 742, 122, 2740, 854, + /* 1860 */ 2742, 2743, 849, 2404, 873, 2739, 2399, 2757, 295, 1969, + /* 1870 */ 2611, 2831, 178, 179, 752, 434, 2827, 2630, 165, 751, + /* 1880 */ 851, 759, 753, 2704, 382, 850, 782, 300, 2738, 274, + /* 1890 */ 2898, 2777, 820, 298, 756, 189, 2740, 854, 2742, 2743, + /* 1900 */ 849, 768, 873, 758, 8, 193, 2897, 305, 791, 2757, + /* 1910 */ 2870, 307, 767, 191, 687, 683, 679, 675, 309, 273, + /* 1920 */ 769, 795, 311, 766, 796, 2704, 314, 850, 466, 804, + /* 1930 */ 2738, 2850, 313, 2777, 312, 807, 316, 122, 2740, 854, + /* 1940 */ 2742, 2743, 849, 156, 873, 1970, 2115, 2113, 2739, 383, + /* 1950 */ 204, 2831, 325, 2847, 800, 2950, 2828, 166, 384, 2928, + /* 1960 */ 818, 2579, 2578, 851, 2952, 101, 319, 2577, 470, 1, + /* 1970 */ 271, 167, 2738, 823, 824, 2777, 828, 831, 61, 186, + /* 1980 */ 2740, 854, 2742, 2743, 849, 2812, 873, 2739, 856, 338, + /* 1990 */ 858, 220, 2757, 860, 2417, 861, 343, 112, 385, 2696, + /* 2000 */ 2695, 2691, 851, 2690, 2682, 2681, 2673, 2672, 2704, 2688, + /* 2010 */ 850, 2687, 2679, 2678, 115, 2667, 372, 389, 1473, 2666, + /* 2020 */ 2685, 875, 2684, 2676, 970, 2675, 2664, 2890, 971, 2739, + /* 2030 */ 2663, 2757, 463, 375, 972, 967, 423, 259, 2661, 974, + /* 2040 */ 2660, 176, 391, 2468, 851, 376, 270, 2704, 744, 850, + /* 2050 */ 261, 268, 401, 412, 393, 2738, 266, 661, 2777, 2656, + /* 2060 */ 427, 52, 411, 2740, 854, 2742, 2743, 849, 2655, 873, + /* 2070 */ 2654, 464, 431, 2757, 85, 258, 414, 432, 402, 2649, + /* 2080 */ 511, 512, 1893, 1894, 224, 516, 2647, 518, 519, 2704, + /* 2090 */ 520, 850, 1892, 2646, 2738, 2645, 440, 2777, 2643, 526, + /* 2100 */ 2642, 411, 2740, 854, 2742, 2743, 849, 528, 873, 2739, + /* 2110 */ 2641, 530, 2640, 532, 1880, 2615, 228, 2614, 230, 1838, + /* 2120 */ 86, 1837, 2592, 2591, 851, 2590, 2739, 544, 545, 2589, + /* 2130 */ 2588, 2532, 549, 1774, 2529, 552, 2738, 2528, 2522, 2777, + /* 2140 */ 555, 848, 556, 404, 2740, 854, 2742, 2743, 849, 2519, + /* 2150 */ 873, 2739, 233, 2757, 2518, 89, 2517, 2516, 2521, 2520, + /* 2160 */ 235, 2515, 2514, 2512, 2511, 2510, 851, 237, 572, 2704, + /* 2170 */ 2757, 850, 2509, 2507, 574, 2506, 2505, 2504, 2503, 2527, + /* 2180 */ 2502, 2501, 2500, 2525, 2508, 2499, 2704, 2498, 850, 95, + /* 2190 */ 2739, 2497, 2495, 2494, 790, 2757, 2493, 2492, 2491, 2490, + /* 2200 */ 2489, 239, 2488, 2487, 2486, 851, 2485, 2557, 2526, 2524, + /* 2210 */ 2484, 2704, 2483, 850, 2482, 244, 2738, 1780, 2481, 2777, + /* 2220 */ 2480, 611, 613, 189, 2740, 854, 2742, 2743, 849, 2479, + /* 2230 */ 873, 2477, 2307, 2738, 2757, 475, 2777, 1625, 429, 1629, + /* 2240 */ 410, 2740, 854, 2742, 2743, 849, 430, 873, 2306, 2797, + /* 2250 */ 2704, 2305, 850, 1502, 2251, 1621, 247, 2304, 2738, 2302, + /* 2260 */ 2299, 2777, 631, 2298, 635, 411, 2740, 854, 2742, 2743, + /* 2270 */ 849, 2739, 873, 248, 477, 632, 636, 633, 637, 250, + /* 2280 */ 2291, 251, 639, 2951, 640, 641, 851, 2278, 2739, 2252, + /* 2290 */ 253, 81, 643, 2724, 199, 210, 651, 2738, 2613, 82, + /* 2300 */ 2777, 2609, 2599, 851, 411, 2740, 854, 2742, 2743, 849, + /* 2310 */ 2739, 873, 255, 2587, 262, 2757, 2586, 267, 264, 2563, + /* 2320 */ 269, 2556, 2394, 2301, 2297, 851, 1551, 672, 673, 674, + /* 2330 */ 2295, 2704, 2757, 850, 678, 676, 677, 2293, 681, 680, + /* 2340 */ 682, 2290, 684, 2273, 686, 2271, 685, 2272, 2704, 2270, + /* 2350 */ 850, 1707, 73, 2739, 2757, 2248, 2396, 1706, 275, 2395, + /* 2360 */ 1611, 1610, 1609, 1606, 1604, 1602, 2288, 1601, 851, 1600, + /* 2370 */ 2704, 2286, 850, 720, 1593, 1599, 1598, 937, 739, 939, + /* 2380 */ 2277, 2777, 2275, 454, 1595, 406, 2740, 854, 2742, 2743, + /* 2390 */ 849, 455, 873, 2739, 1594, 2738, 1592, 2757, 2777, 456, + /* 2400 */ 457, 723, 396, 2740, 854, 2742, 2743, 849, 851, 873, + /* 2410 */ 2247, 2246, 2245, 2704, 2244, 850, 733, 2738, 2243, 727, + /* 2420 */ 2777, 729, 2242, 731, 395, 2740, 854, 2742, 2743, 849, + /* 2430 */ 1878, 873, 2739, 128, 1874, 1876, 1873, 2757, 2612, 297, + /* 2440 */ 2608, 1844, 56, 1846, 2598, 1863, 754, 851, 1848, 28, + /* 2450 */ 2585, 743, 2584, 2704, 2934, 850, 180, 67, 2739, 20, + /* 2460 */ 2738, 57, 21, 2777, 755, 1823, 1822, 397, 2740, 854, + /* 2470 */ 2742, 2743, 849, 851, 873, 301, 2757, 30, 304, 17, + /* 2480 */ 760, 762, 770, 2168, 2142, 462, 6, 772, 774, 7, + /* 2490 */ 776, 22, 2704, 203, 850, 32, 215, 306, 2725, 190, + /* 2500 */ 2738, 2108, 2757, 2777, 216, 2149, 202, 403, 2740, 854, + /* 2510 */ 2742, 2743, 849, 2136, 873, 2110, 31, 83, 2704, 2106, + /* 2520 */ 850, 217, 2739, 2188, 2189, 65, 24, 2183, 2182, 23, + /* 2530 */ 467, 2090, 2187, 2186, 468, 321, 2089, 851, 59, 2738, + /* 2540 */ 195, 18, 2777, 2583, 2562, 106, 407, 2740, 854, 2742, + /* 2550 */ 2743, 849, 2739, 873, 58, 107, 2561, 328, 2144, 108, + /* 2560 */ 821, 2555, 109, 205, 25, 2738, 2757, 851, 2777, 334, + /* 2570 */ 69, 2739, 398, 2740, 854, 2742, 2743, 849, 337, 873, + /* 2580 */ 827, 2042, 2704, 11, 850, 2041, 851, 13, 829, 1958, + /* 2590 */ 336, 339, 2017, 2016, 2015, 888, 2757, 891, 894, 196, + /* 2600 */ 897, 38, 16, 2052, 206, 26, 1993, 2554, 110, 853, + /* 2610 */ 2391, 864, 2704, 859, 850, 2757, 1985, 27, 70, 865, + /* 2620 */ 857, 341, 111, 347, 79, 116, 2019, 2204, 2782, 2738, + /* 2630 */ 2203, 2704, 2777, 850, 2781, 2739, 408, 2740, 854, 2742, + /* 2640 */ 2743, 849, 872, 873, 68, 2202, 874, 880, 1688, 494, + /* 2650 */ 851, 882, 2201, 1685, 884, 885, 887, 2739, 1684, 2738, + /* 2660 */ 890, 1681, 2777, 1675, 893, 1673, 399, 2740, 854, 2742, + /* 2670 */ 2743, 849, 851, 873, 896, 117, 1679, 371, 2738, 2757, + /* 2680 */ 118, 2777, 1701, 1678, 80, 409, 2740, 854, 2742, 2743, + /* 2690 */ 849, 1677, 873, 1697, 1676, 2704, 911, 850, 1549, 1589, + /* 2700 */ 1588, 2757, 1587, 1584, 1581, 1580, 1579, 1619, 1578, 1576, + /* 2710 */ 1574, 1573, 1572, 925, 1618, 927, 222, 2704, 1570, 850, + /* 2720 */ 1569, 1568, 1567, 1566, 1615, 1565, 1564, 1613, 1561, 1560, + /* 2730 */ 1557, 1556, 1555, 2739, 1554, 2296, 2294, 947, 951, 2292, + /* 2740 */ 948, 949, 2738, 953, 955, 2777, 2289, 959, 851, 400, + /* 2750 */ 2740, 854, 2742, 2743, 849, 2739, 873, 952, 956, 957, + /* 2760 */ 960, 961, 2269, 963, 2738, 2267, 965, 2777, 2739, 1491, + /* 2770 */ 851, 416, 2740, 854, 2742, 2743, 849, 2757, 873, 2241, + /* 2780 */ 1474, 969, 377, 851, 973, 1479, 1481, 2207, 1944, 390, + /* 2790 */ 976, 977, 2207, 2704, 2207, 850, 2207, 2207, 2739, 2757, + /* 2800 */ 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, + /* 2810 */ 2207, 2207, 2757, 851, 2207, 2704, 2207, 850, 2207, 2207, + /* 2820 */ 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2704, 2207, + /* 2830 */ 850, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, + /* 2840 */ 2738, 2207, 2757, 2777, 2207, 2207, 2207, 417, 2740, 854, + /* 2850 */ 2742, 2743, 849, 2207, 873, 2207, 2207, 2207, 2704, 2207, + /* 2860 */ 850, 2207, 2738, 2207, 2207, 2777, 2207, 2207, 2207, 2751, + /* 2870 */ 2740, 854, 2742, 2743, 849, 2738, 873, 2739, 2777, 2207, + /* 2880 */ 2207, 2207, 2750, 2740, 854, 2742, 2743, 849, 2207, 873, + /* 2890 */ 2207, 2207, 851, 2207, 2739, 2207, 2207, 2207, 2207, 2207, + /* 2900 */ 2207, 2207, 2207, 2207, 2207, 2738, 2207, 2207, 2777, 851, + /* 2910 */ 2207, 2207, 2749, 2740, 854, 2742, 2743, 849, 2207, 873, + /* 2920 */ 2739, 2757, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, + /* 2930 */ 2207, 2207, 2207, 2207, 2207, 851, 2207, 2704, 2757, 850, + /* 2940 */ 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, + /* 2950 */ 2207, 2207, 2207, 2207, 2704, 2207, 850, 2207, 2739, 2207, + /* 2960 */ 2207, 2207, 2207, 2207, 2757, 2207, 2207, 2207, 2207, 2207, + /* 2970 */ 2207, 2207, 2207, 851, 2207, 2207, 2207, 2207, 2207, 2207, + /* 2980 */ 2704, 2207, 850, 2207, 2738, 2207, 2207, 2777, 2207, 2207, + /* 2990 */ 2207, 436, 2740, 854, 2742, 2743, 849, 2207, 873, 2207, + /* 3000 */ 2207, 2738, 2757, 2207, 2777, 2207, 2207, 2207, 437, 2740, + /* 3010 */ 854, 2742, 2743, 849, 2207, 873, 2207, 2207, 2704, 2207, + /* 3020 */ 850, 2207, 2207, 2207, 2207, 2207, 2207, 2738, 2207, 2207, + /* 3030 */ 2777, 2207, 2207, 2207, 433, 2740, 854, 2742, 2743, 849, + /* 3040 */ 2739, 873, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, + /* 3050 */ 2207, 2207, 2207, 2207, 2207, 851, 2207, 2207, 2207, 2207, + /* 3060 */ 2207, 2207, 2207, 2207, 2207, 2738, 2207, 2207, 2777, 2207, + /* 3070 */ 2739, 2207, 438, 2740, 854, 2742, 2743, 849, 2207, 873, + /* 3080 */ 2207, 2207, 2207, 2207, 2757, 851, 2207, 2207, 2207, 2207, + /* 3090 */ 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, + /* 3100 */ 2704, 2207, 850, 2207, 2207, 2207, 2207, 2207, 2207, 2207, + /* 3110 */ 2207, 2207, 2207, 2207, 2757, 2207, 2207, 2207, 2207, 2207, + /* 3120 */ 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, + /* 3130 */ 2704, 2207, 850, 2207, 2207, 2207, 2207, 2207, 2207, 2207, + /* 3140 */ 2207, 2207, 2207, 2207, 2207, 2207, 2207, 852, 2207, 2207, + /* 3150 */ 2777, 2207, 2207, 2207, 406, 2740, 854, 2742, 2743, 849, + /* 3160 */ 2207, 873, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, + /* 3170 */ 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2738, 2207, 2207, + /* 3180 */ 2777, 2207, 2207, 2207, 405, 2740, 854, 2742, 2743, 849, + /* 3190 */ 2207, 873, }; static const YYCODETYPE yy_lookahead[] = { - /* 0 */ 412, 426, 390, 467, 443, 393, 394, 443, 472, 510, - /* 10 */ 511, 395, 12, 13, 14, 426, 381, 14, 521, 424, - /* 20 */ 20, 457, 22, 20, 463, 464, 529, 463, 464, 441, - /* 30 */ 412, 396, 425, 395, 396, 440, 36, 396, 38, 428, - /* 40 */ 20, 434, 435, 428, 547, 548, 431, 8, 9, 552, - /* 50 */ 553, 12, 13, 14, 15, 16, 381, 521, 395, 441, - /* 60 */ 425, 445, 446, 14, 448, 529, 425, 451, 68, 20, - /* 70 */ 20, 396, 20, 12, 13, 75, 441, 73, 443, 491, - /* 80 */ 492, 493, 82, 547, 548, 395, 396, 443, 552, 553, - /* 90 */ 502, 0, 485, 486, 20, 12, 13, 36, 395, 396, - /* 100 */ 425, 457, 491, 20, 428, 22, 491, 463, 464, 491, - /* 110 */ 492, 74, 112, 502, 75, 115, 441, 502, 443, 36, - /* 120 */ 502, 38, 20, 488, 395, 396, 491, 14, 15, 16, - /* 130 */ 495, 496, 497, 498, 499, 500, 20, 502, 497, 378, - /* 140 */ 477, 478, 507, 390, 509, 54, 393, 394, 513, 514, - /* 150 */ 381, 68, 152, 153, 516, 517, 518, 519, 75, 521, - /* 160 */ 522, 1, 2, 488, 125, 82, 491, 491, 533, 117, - /* 170 */ 495, 496, 497, 498, 499, 500, 541, 502, 502, 381, - /* 180 */ 505, 36, 507, 508, 509, 395, 396, 411, 513, 514, - /* 190 */ 414, 191, 192, 391, 396, 112, 398, 395, 115, 397, - /* 200 */ 391, 201, 202, 426, 395, 415, 397, 517, 518, 519, - /* 210 */ 441, 521, 522, 423, 437, 211, 216, 115, 218, 529, - /* 220 */ 517, 518, 519, 425, 521, 522, 152, 153, 467, 190, - /* 230 */ 425, 115, 20, 472, 20, 152, 153, 547, 548, 441, - /* 240 */ 191, 443, 552, 553, 240, 241, 517, 518, 519, 444, - /* 250 */ 521, 522, 252, 253, 254, 33, 256, 257, 258, 259, + /* 0 */ 468, 396, 397, 406, 429, 473, 391, 432, 427, 394, + /* 10 */ 395, 414, 12, 13, 14, 396, 397, 396, 397, 438, + /* 20 */ 20, 0, 22, 8, 9, 396, 397, 12, 13, 14, + /* 30 */ 15, 16, 511, 512, 23, 416, 36, 416, 38, 0, + /* 40 */ 429, 34, 21, 424, 382, 24, 25, 26, 27, 28, + /* 50 */ 29, 30, 31, 32, 522, 450, 396, 397, 44, 397, + /* 60 */ 49, 50, 530, 396, 397, 8, 9, 492, 68, 12, + /* 70 */ 13, 14, 15, 16, 20, 75, 413, 20, 503, 450, + /* 80 */ 548, 549, 82, 416, 20, 553, 554, 444, 426, 412, + /* 90 */ 21, 424, 415, 24, 25, 26, 27, 28, 29, 30, + /* 100 */ 31, 32, 426, 492, 442, 442, 444, 464, 465, 20, + /* 110 */ 434, 429, 112, 382, 503, 115, 77, 78, 79, 80, + /* 120 */ 81, 445, 83, 84, 85, 86, 87, 88, 89, 90, + /* 130 */ 91, 92, 93, 94, 95, 96, 97, 98, 99, 100, + /* 140 */ 101, 102, 103, 104, 105, 106, 107, 108, 425, 396, + /* 150 */ 397, 489, 152, 153, 492, 492, 493, 494, 496, 497, + /* 160 */ 498, 499, 500, 501, 441, 503, 503, 4, 21, 155, + /* 170 */ 508, 117, 510, 442, 492, 4, 514, 515, 518, 519, + /* 180 */ 520, 115, 522, 523, 37, 503, 39, 40, 41, 42, + /* 190 */ 530, 191, 192, 392, 23, 426, 534, 396, 522, 398, + /* 200 */ 495, 201, 202, 450, 542, 436, 530, 391, 548, 549, + /* 210 */ 394, 395, 115, 553, 554, 0, 216, 421, 218, 48, + /* 220 */ 49, 50, 426, 208, 548, 549, 521, 8, 9, 553, + /* 230 */ 554, 12, 13, 14, 15, 16, 155, 396, 20, 24, + /* 240 */ 25, 26, 27, 28, 29, 30, 31, 32, 115, 453, + /* 250 */ 193, 146, 147, 253, 254, 255, 151, 257, 258, 259, /* 260 */ 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, - /* 270 */ 270, 271, 272, 273, 191, 192, 116, 155, 420, 218, - /* 280 */ 44, 20, 521, 425, 201, 202, 488, 146, 147, 491, - /* 290 */ 529, 36, 151, 495, 496, 497, 498, 499, 500, 216, - /* 300 */ 502, 218, 0, 254, 254, 507, 20, 509, 547, 548, - /* 310 */ 452, 513, 514, 552, 553, 426, 277, 278, 279, 280, - /* 320 */ 281, 282, 283, 284, 285, 286, 287, 115, 145, 146, - /* 330 */ 147, 148, 149, 150, 151, 252, 253, 254, 116, 256, - /* 340 */ 257, 258, 259, 260, 261, 262, 263, 264, 265, 266, - /* 350 */ 267, 268, 269, 270, 271, 272, 273, 274, 12, 13, - /* 360 */ 426, 427, 381, 218, 425, 18, 20, 147, 22, 417, - /* 370 */ 23, 151, 433, 421, 395, 396, 487, 396, 381, 77, - /* 380 */ 78, 79, 36, 444, 38, 395, 84, 85, 86, 42, - /* 390 */ 43, 155, 90, 46, 415, 250, 251, 95, 96, 97, - /* 400 */ 98, 299, 423, 101, 57, 0, 425, 105, 106, 107, - /* 410 */ 108, 289, 290, 291, 68, 299, 69, 70, 71, 72, - /* 420 */ 73, 75, 441, 523, 443, 525, 21, 20, 82, 24, - /* 430 */ 25, 26, 27, 28, 29, 30, 31, 32, 441, 521, - /* 440 */ 425, 12, 13, 14, 454, 225, 456, 529, 433, 20, - /* 450 */ 230, 22, 20, 233, 193, 235, 425, 20, 112, 444, - /* 460 */ 521, 115, 115, 4, 433, 36, 548, 38, 529, 488, - /* 470 */ 552, 553, 491, 218, 13, 444, 495, 496, 497, 498, - /* 480 */ 499, 500, 23, 502, 395, 396, 547, 548, 507, 425, - /* 490 */ 509, 552, 553, 20, 513, 514, 0, 68, 152, 153, - /* 500 */ 380, 154, 382, 155, 415, 250, 251, 48, 49, 50, - /* 510 */ 21, 82, 425, 24, 25, 26, 27, 28, 29, 30, - /* 520 */ 31, 32, 541, 115, 288, 289, 290, 291, 292, 293, - /* 530 */ 294, 295, 296, 21, 521, 420, 425, 191, 192, 452, - /* 540 */ 425, 112, 529, 82, 115, 434, 435, 201, 202, 37, - /* 550 */ 486, 39, 40, 41, 42, 208, 209, 210, 0, 196, - /* 560 */ 213, 548, 216, 20, 218, 552, 553, 452, 395, 396, - /* 570 */ 74, 8, 9, 226, 227, 12, 13, 14, 15, 16, - /* 580 */ 22, 152, 153, 20, 152, 153, 239, 439, 415, 242, - /* 590 */ 442, 443, 245, 246, 247, 248, 249, 145, 252, 253, - /* 600 */ 254, 149, 256, 257, 258, 259, 260, 261, 262, 263, - /* 610 */ 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, - /* 620 */ 191, 192, 292, 293, 294, 295, 296, 4, 191, 192, - /* 630 */ 201, 202, 404, 201, 202, 112, 288, 289, 290, 291, - /* 640 */ 292, 293, 294, 295, 296, 216, 299, 218, 0, 115, - /* 650 */ 127, 128, 129, 130, 131, 132, 133, 134, 135, 136, - /* 660 */ 432, 138, 139, 140, 141, 142, 143, 144, 305, 306, - /* 670 */ 307, 308, 24, 25, 26, 27, 28, 29, 30, 31, - /* 680 */ 32, 252, 253, 254, 68, 256, 257, 258, 259, 260, - /* 690 */ 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, - /* 700 */ 271, 272, 273, 12, 13, 22, 381, 299, 395, 396, - /* 710 */ 74, 20, 20, 22, 12, 13, 14, 15, 16, 36, - /* 720 */ 20, 396, 22, 398, 77, 78, 79, 36, 415, 38, - /* 730 */ 114, 84, 85, 86, 191, 119, 36, 90, 115, 126, - /* 740 */ 395, 396, 95, 96, 97, 98, 3, 381, 101, 215, - /* 750 */ 425, 217, 105, 106, 107, 108, 193, 274, 58, 68, - /* 760 */ 415, 14, 396, 20, 398, 416, 441, 20, 443, 0, - /* 770 */ 228, 8, 9, 82, 425, 12, 13, 14, 15, 16, - /* 780 */ 54, 416, 395, 249, 91, 436, 12, 13, 395, 63, - /* 790 */ 425, 425, 66, 67, 20, 112, 22, 254, 395, 396, - /* 800 */ 425, 436, 186, 112, 395, 396, 115, 441, 433, 443, - /* 810 */ 36, 0, 38, 488, 442, 443, 491, 254, 415, 444, - /* 820 */ 495, 496, 497, 498, 499, 500, 36, 502, 395, 396, - /* 830 */ 395, 396, 507, 299, 509, 395, 396, 425, 513, 514, - /* 840 */ 395, 396, 68, 152, 153, 75, 459, 460, 415, 395, - /* 850 */ 396, 252, 159, 33, 488, 415, 82, 491, 449, 116, - /* 860 */ 415, 495, 496, 497, 498, 499, 500, 47, 502, 415, - /* 870 */ 477, 478, 82, 507, 82, 509, 404, 184, 185, 513, - /* 880 */ 514, 115, 191, 192, 449, 193, 112, 395, 396, 115, - /* 890 */ 126, 198, 201, 202, 422, 8, 9, 485, 486, 12, - /* 900 */ 13, 14, 15, 16, 432, 38, 274, 216, 276, 218, - /* 910 */ 311, 312, 313, 314, 315, 316, 317, 375, 376, 377, - /* 920 */ 33, 54, 299, 22, 155, 0, 152, 153, 190, 381, - /* 930 */ 63, 64, 65, 66, 0, 68, 193, 36, 191, 170, - /* 940 */ 506, 449, 508, 252, 253, 254, 254, 256, 257, 258, - /* 950 */ 259, 260, 261, 262, 263, 264, 265, 266, 267, 268, - /* 960 */ 269, 270, 271, 272, 273, 191, 192, 402, 18, 44, - /* 970 */ 20, 208, 425, 395, 396, 201, 202, 27, 395, 396, - /* 980 */ 30, 114, 171, 82, 419, 4, 119, 176, 38, 441, - /* 990 */ 216, 444, 218, 415, 429, 184, 395, 396, 415, 52, - /* 1000 */ 405, 254, 146, 116, 54, 506, 56, 508, 413, 395, - /* 1010 */ 396, 61, 62, 8, 9, 277, 415, 12, 13, 14, - /* 1020 */ 15, 16, 252, 73, 494, 287, 252, 253, 254, 415, - /* 1030 */ 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, - /* 1040 */ 266, 267, 268, 269, 270, 271, 272, 273, 395, 396, - /* 1050 */ 520, 8, 9, 186, 44, 12, 13, 14, 15, 16, - /* 1060 */ 22, 20, 195, 14, 114, 299, 199, 200, 415, 20, - /* 1070 */ 395, 396, 205, 206, 36, 402, 126, 0, 68, 145, - /* 1080 */ 146, 147, 148, 149, 150, 151, 381, 231, 232, 326, - /* 1090 */ 415, 224, 8, 9, 59, 60, 12, 13, 14, 15, - /* 1100 */ 16, 396, 429, 398, 400, 401, 3, 157, 158, 23, - /* 1110 */ 160, 161, 162, 163, 164, 165, 166, 167, 168, 169, - /* 1120 */ 82, 116, 172, 173, 174, 175, 176, 177, 178, 179, - /* 1130 */ 425, 181, 182, 183, 381, 49, 50, 187, 188, 189, - /* 1140 */ 193, 400, 401, 425, 194, 126, 441, 33, 443, 505, - /* 1150 */ 112, 204, 508, 435, 77, 78, 79, 80, 81, 116, - /* 1160 */ 83, 84, 85, 86, 87, 88, 89, 90, 91, 92, - /* 1170 */ 93, 94, 95, 96, 97, 98, 99, 100, 101, 102, - /* 1180 */ 103, 104, 105, 106, 107, 108, 20, 12, 13, 395, - /* 1190 */ 396, 381, 20, 488, 441, 381, 491, 22, 381, 180, - /* 1200 */ 495, 496, 497, 498, 499, 500, 396, 502, 398, 415, - /* 1210 */ 4, 36, 507, 38, 509, 381, 2, 381, 513, 514, - /* 1220 */ 381, 381, 8, 9, 381, 19, 12, 13, 14, 15, - /* 1230 */ 16, 22, 395, 396, 193, 425, 396, 186, 398, 395, - /* 1240 */ 396, 0, 381, 68, 38, 36, 395, 396, 20, 0, - /* 1250 */ 22, 441, 415, 443, 216, 441, 218, 82, 441, 415, - /* 1260 */ 54, 34, 381, 41, 42, 425, 415, 61, 62, 395, - /* 1270 */ 396, 220, 395, 396, 68, 441, 68, 441, 409, 410, - /* 1280 */ 441, 441, 301, 443, 441, 381, 58, 112, 425, 415, - /* 1290 */ 252, 253, 415, 409, 410, 254, 425, 381, 488, 436, - /* 1300 */ 494, 491, 441, 384, 385, 495, 496, 497, 498, 499, - /* 1310 */ 500, 33, 502, 381, 421, 444, 33, 507, 381, 509, - /* 1320 */ 114, 112, 441, 513, 514, 119, 520, 119, 488, 381, - /* 1330 */ 381, 491, 467, 13, 33, 495, 496, 497, 498, 499, - /* 1340 */ 500, 381, 502, 121, 122, 441, 124, 507, 47, 509, - /* 1350 */ 12, 13, 2, 513, 514, 381, 396, 441, 8, 9, - /* 1360 */ 22, 426, 12, 13, 14, 15, 16, 145, 152, 153, - /* 1370 */ 494, 149, 381, 441, 36, 482, 38, 438, 441, 438, - /* 1380 */ 441, 438, 441, 36, 441, 425, 521, 396, 0, 441, - /* 1390 */ 441, 216, 120, 218, 529, 123, 520, 0, 234, 116, - /* 1400 */ 236, 441, 82, 443, 155, 36, 68, 120, 381, 33, - /* 1410 */ 123, 120, 547, 548, 123, 441, 425, 552, 553, 22, - /* 1420 */ 254, 33, 467, 396, 321, 426, 254, 252, 253, 82, - /* 1430 */ 120, 13, 441, 123, 443, 426, 33, 33, 13, 325, - /* 1440 */ 0, 266, 267, 268, 269, 270, 271, 272, 488, 0, - /* 1450 */ 0, 491, 425, 51, 36, 495, 496, 497, 498, 499, - /* 1460 */ 500, 36, 502, 33, 33, 33, 426, 507, 441, 509, - /* 1470 */ 443, 22, 22, 513, 514, 0, 521, 1, 2, 488, - /* 1480 */ 473, 115, 491, 117, 529, 244, 495, 496, 497, 498, - /* 1490 */ 499, 500, 116, 502, 36, 55, 33, 33, 507, 33, - /* 1500 */ 509, 33, 547, 548, 513, 514, 413, 552, 553, 36, - /* 1510 */ 33, 33, 33, 12, 13, 488, 382, 115, 491, 116, - /* 1520 */ 116, 13, 495, 496, 497, 498, 499, 500, 453, 502, - /* 1530 */ 33, 33, 12, 13, 507, 523, 509, 381, 12, 13, - /* 1540 */ 513, 514, 12, 13, 36, 112, 116, 116, 116, 556, - /* 1550 */ 12, 13, 396, 545, 216, 82, 218, 524, 12, 13, - /* 1560 */ 127, 128, 129, 130, 131, 132, 133, 134, 135, 136, - /* 1570 */ 538, 138, 139, 140, 141, 142, 143, 144, 1, 116, - /* 1580 */ 116, 425, 116, 36, 116, 12, 13, 218, 250, 251, - /* 1590 */ 252, 12, 13, 116, 116, 116, 19, 441, 33, 443, - /* 1600 */ 33, 323, 12, 13, 266, 267, 268, 269, 270, 271, - /* 1610 */ 272, 13, 467, 116, 116, 38, 12, 13, 381, 33, - /* 1620 */ 145, 146, 147, 148, 149, 150, 151, 12, 13, 82, - /* 1630 */ 53, 54, 467, 396, 36, 399, 33, 425, 412, 394, - /* 1640 */ 63, 64, 65, 66, 488, 68, 33, 491, 453, 544, - /* 1650 */ 412, 495, 496, 497, 498, 499, 500, 33, 502, 544, - /* 1660 */ 544, 544, 425, 466, 453, 509, 521, 399, 467, 513, - /* 1670 */ 514, 489, 396, 440, 529, 528, 218, 474, 441, 528, - /* 1680 */ 443, 116, 453, 116, 453, 549, 521, 453, 515, 531, - /* 1690 */ 54, 114, 547, 548, 529, 414, 119, 552, 553, 381, - /* 1700 */ 8, 9, 116, 302, 12, 13, 14, 15, 16, 490, - /* 1710 */ 468, 20, 547, 548, 396, 20, 395, 552, 553, 116, - /* 1720 */ 233, 404, 521, 381, 484, 488, 479, 150, 491, 116, - /* 1730 */ 529, 479, 495, 496, 497, 498, 499, 500, 396, 502, - /* 1740 */ 116, 404, 381, 425, 470, 214, 509, 395, 547, 548, - /* 1750 */ 513, 514, 20, 552, 553, 396, 47, 396, 450, 441, - /* 1760 */ 396, 443, 450, 447, 190, 395, 395, 425, 396, 450, - /* 1770 */ 447, 447, 195, 447, 197, 113, 408, 200, 111, 395, - /* 1780 */ 407, 395, 205, 441, 110, 443, 425, 406, 395, 395, - /* 1790 */ 395, 20, 52, 388, 479, 388, 404, 20, 392, 443, - /* 1800 */ 392, 224, 441, 404, 443, 20, 488, 397, 20, 491, - /* 1810 */ 381, 397, 469, 495, 496, 497, 498, 499, 500, 404, - /* 1820 */ 502, 404, 20, 404, 404, 396, 20, 509, 404, 454, - /* 1830 */ 488, 513, 514, 491, 460, 388, 395, 495, 496, 497, - /* 1840 */ 498, 499, 500, 501, 502, 503, 504, 404, 404, 488, - /* 1850 */ 381, 425, 491, 425, 425, 395, 495, 496, 497, 498, - /* 1860 */ 499, 500, 425, 502, 425, 396, 384, 425, 384, 388, - /* 1870 */ 441, 425, 443, 425, 237, 381, 425, 425, 425, 425, - /* 1880 */ 115, 483, 402, 20, 221, 193, 441, 222, 479, 468, - /* 1890 */ 396, 478, 481, 402, 425, 395, 441, 441, 476, 310, - /* 1900 */ 443, 309, 475, 542, 543, 318, 441, 540, 207, 303, - /* 1910 */ 441, 320, 443, 319, 537, 537, 537, 488, 381, 425, - /* 1920 */ 491, 539, 461, 461, 495, 496, 497, 498, 499, 500, - /* 1930 */ 298, 502, 536, 396, 534, 441, 297, 443, 468, 327, - /* 1940 */ 557, 324, 551, 527, 381, 322, 396, 535, 126, 526, - /* 1950 */ 20, 300, 397, 402, 402, 441, 461, 488, 441, 396, - /* 1960 */ 491, 494, 425, 441, 495, 496, 497, 498, 499, 500, - /* 1970 */ 532, 502, 461, 441, 441, 199, 458, 402, 441, 454, - /* 1980 */ 443, 441, 488, 554, 555, 491, 381, 402, 425, 495, - /* 1990 */ 496, 497, 498, 499, 500, 115, 502, 512, 441, 199, - /* 2000 */ 455, 396, 465, 509, 441, 421, 443, 454, 514, 550, - /* 2010 */ 402, 402, 543, 530, 396, 115, 395, 441, 22, 441, - /* 2020 */ 35, 441, 441, 381, 441, 488, 430, 383, 491, 441, - /* 2030 */ 425, 402, 495, 496, 497, 498, 499, 500, 396, 502, - /* 2040 */ 441, 441, 441, 441, 386, 37, 441, 387, 443, 40, - /* 2050 */ 389, 488, 441, 471, 491, 441, 388, 441, 495, 496, - /* 2060 */ 497, 498, 499, 500, 379, 502, 441, 425, 441, 441, - /* 2070 */ 465, 441, 441, 427, 441, 441, 441, 480, 403, 441, - /* 2080 */ 487, 441, 462, 441, 462, 443, 427, 418, 418, 418, - /* 2090 */ 0, 0, 0, 488, 47, 0, 491, 36, 243, 36, - /* 2100 */ 495, 496, 497, 498, 499, 500, 36, 502, 36, 546, - /* 2110 */ 381, 243, 0, 36, 36, 243, 36, 0, 0, 243, - /* 2120 */ 0, 36, 0, 36, 0, 396, 22, 0, 238, 36, - /* 2130 */ 488, 0, 381, 491, 224, 0, 224, 495, 496, 497, - /* 2140 */ 498, 499, 500, 225, 502, 218, 0, 396, 216, 0, - /* 2150 */ 0, 212, 211, 0, 425, 0, 158, 51, 51, 381, - /* 2160 */ 0, 36, 0, 0, 36, 54, 0, 51, 0, 0, - /* 2170 */ 441, 0, 443, 47, 396, 0, 425, 0, 0, 0, - /* 2180 */ 51, 0, 0, 0, 36, 0, 176, 176, 0, 0, - /* 2190 */ 0, 0, 441, 0, 443, 0, 381, 555, 0, 0, - /* 2200 */ 0, 0, 0, 425, 0, 0, 0, 0, 0, 0, - /* 2210 */ 0, 396, 0, 0, 0, 51, 465, 488, 0, 441, - /* 2220 */ 491, 443, 0, 381, 495, 496, 497, 498, 499, 500, - /* 2230 */ 0, 502, 0, 504, 47, 0, 0, 0, 396, 488, - /* 2240 */ 425, 0, 491, 465, 0, 381, 495, 496, 497, 498, - /* 2250 */ 499, 500, 0, 502, 158, 22, 441, 0, 443, 157, - /* 2260 */ 396, 0, 0, 0, 156, 22, 488, 425, 52, 491, - /* 2270 */ 22, 52, 381, 495, 496, 497, 498, 499, 500, 0, - /* 2280 */ 502, 0, 0, 441, 0, 443, 36, 396, 68, 425, - /* 2290 */ 0, 68, 68, 68, 0, 36, 54, 44, 0, 54, - /* 2300 */ 36, 0, 44, 488, 36, 441, 491, 443, 54, 381, - /* 2310 */ 495, 496, 497, 498, 499, 500, 425, 502, 44, 0, - /* 2320 */ 0, 36, 47, 51, 396, 33, 14, 44, 51, 51, - /* 2330 */ 488, 0, 441, 491, 443, 0, 45, 495, 496, 497, - /* 2340 */ 498, 499, 500, 44, 502, 0, 0, 0, 44, 207, - /* 2350 */ 381, 51, 488, 425, 0, 491, 0, 51, 0, 495, - /* 2360 */ 496, 497, 498, 499, 500, 396, 502, 0, 0, 441, - /* 2370 */ 0, 443, 76, 36, 44, 54, 0, 36, 54, 488, - /* 2380 */ 44, 0, 491, 54, 36, 381, 495, 496, 497, 498, - /* 2390 */ 499, 500, 0, 502, 425, 36, 44, 54, 0, 44, - /* 2400 */ 396, 0, 0, 0, 0, 0, 22, 36, 0, 22, - /* 2410 */ 441, 36, 443, 125, 36, 36, 488, 381, 123, 491, - /* 2420 */ 0, 36, 36, 495, 496, 497, 498, 499, 500, 425, - /* 2430 */ 502, 36, 396, 36, 0, 36, 381, 22, 22, 22, - /* 2440 */ 33, 0, 36, 33, 22, 441, 56, 443, 36, 36, - /* 2450 */ 36, 396, 0, 22, 0, 36, 0, 488, 0, 36, - /* 2460 */ 491, 425, 0, 36, 495, 496, 497, 498, 499, 500, - /* 2470 */ 0, 502, 36, 0, 36, 22, 36, 441, 36, 443, - /* 2480 */ 425, 20, 381, 0, 229, 116, 193, 115, 228, 115, - /* 2490 */ 223, 0, 488, 36, 51, 491, 441, 396, 443, 495, - /* 2500 */ 496, 497, 498, 499, 500, 0, 502, 193, 22, 22, - /* 2510 */ 193, 0, 199, 0, 381, 193, 219, 3, 203, 193, - /* 2520 */ 36, 203, 33, 52, 488, 116, 425, 491, 36, 396, - /* 2530 */ 304, 495, 496, 497, 498, 499, 500, 116, 502, 115, - /* 2540 */ 115, 115, 441, 488, 443, 113, 491, 52, 51, 116, - /* 2550 */ 495, 496, 497, 498, 499, 500, 111, 502, 425, 33, - /* 2560 */ 33, 33, 116, 115, 115, 115, 33, 116, 51, 304, - /* 2570 */ 115, 33, 82, 36, 441, 116, 443, 3, 381, 115, - /* 2580 */ 33, 116, 116, 116, 304, 36, 36, 36, 36, 488, - /* 2590 */ 36, 36, 491, 396, 116, 51, 495, 496, 497, 498, - /* 2600 */ 499, 500, 381, 502, 288, 33, 0, 51, 0, 115, - /* 2610 */ 44, 0, 116, 44, 116, 115, 115, 396, 115, 0, - /* 2620 */ 116, 488, 425, 115, 491, 196, 196, 381, 495, 496, - /* 2630 */ 497, 498, 499, 500, 44, 502, 33, 113, 441, 200, - /* 2640 */ 443, 275, 396, 113, 2, 22, 425, 116, 252, 51, - /* 2650 */ 116, 115, 51, 22, 115, 195, 115, 115, 115, 0, - /* 2660 */ 44, 116, 441, 115, 443, 196, 255, 381, 115, 0, - /* 2670 */ 22, 425, 51, 116, 115, 115, 118, 116, 115, 115, - /* 2680 */ 115, 36, 396, 115, 117, 488, 115, 441, 491, 443, - /* 2690 */ 115, 381, 495, 496, 497, 498, 499, 500, 116, 502, - /* 2700 */ 115, 22, 126, 22, 229, 22, 396, 116, 36, 488, - /* 2710 */ 115, 425, 491, 116, 36, 115, 495, 496, 497, 498, - /* 2720 */ 499, 500, 36, 502, 36, 116, 116, 441, 116, 443, - /* 2730 */ 36, 116, 381, 36, 488, 425, 137, 491, 33, 137, - /* 2740 */ 115, 495, 496, 497, 498, 499, 500, 396, 502, 115, - /* 2750 */ 137, 441, 36, 443, 137, 115, 22, 22, 76, 75, - /* 2760 */ 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, - /* 2770 */ 109, 82, 381, 36, 488, 82, 425, 491, 33, 109, - /* 2780 */ 36, 495, 496, 497, 498, 499, 500, 396, 502, 36, - /* 2790 */ 36, 22, 441, 36, 443, 36, 36, 82, 488, 381, - /* 2800 */ 36, 491, 36, 36, 36, 495, 496, 497, 498, 499, - /* 2810 */ 500, 36, 502, 22, 396, 36, 425, 0, 381, 36, - /* 2820 */ 44, 0, 36, 0, 54, 54, 44, 54, 36, 44, - /* 2830 */ 0, 36, 441, 396, 443, 54, 0, 44, 36, 488, - /* 2840 */ 0, 22, 491, 425, 36, 0, 495, 496, 497, 498, - /* 2850 */ 499, 500, 36, 502, 22, 33, 36, 22, 21, 441, - /* 2860 */ 22, 443, 425, 22, 21, 558, 20, 558, 558, 558, - /* 2870 */ 558, 558, 558, 558, 558, 558, 558, 558, 441, 488, - /* 2880 */ 443, 558, 491, 381, 558, 558, 495, 496, 497, 498, - /* 2890 */ 499, 500, 558, 502, 558, 558, 381, 558, 396, 558, - /* 2900 */ 558, 558, 558, 558, 558, 558, 488, 558, 558, 491, - /* 2910 */ 558, 396, 558, 495, 496, 497, 498, 499, 500, 558, - /* 2920 */ 502, 558, 558, 558, 558, 488, 558, 425, 491, 558, - /* 2930 */ 558, 558, 495, 496, 497, 498, 499, 500, 558, 502, - /* 2940 */ 425, 558, 558, 441, 558, 443, 558, 558, 558, 558, - /* 2950 */ 558, 558, 558, 558, 558, 558, 441, 558, 443, 558, - /* 2960 */ 558, 558, 558, 558, 558, 558, 558, 558, 558, 558, - /* 2970 */ 558, 558, 558, 558, 558, 558, 558, 558, 558, 558, - /* 2980 */ 558, 558, 558, 558, 558, 558, 558, 558, 558, 558, - /* 2990 */ 488, 558, 558, 491, 558, 558, 558, 495, 496, 497, - /* 3000 */ 498, 499, 500, 488, 502, 558, 491, 558, 558, 558, - /* 3010 */ 495, 496, 497, 498, 499, 500, 558, 502, 558, 558, - /* 3020 */ 558, 558, 558, 558, 558, 558, 558, 558, 558, 558, - /* 3030 */ 558, 558, 558, 558, 558, 558, 558, 558, 558, 558, - /* 3040 */ 558, 558, 558, 558, 558, 558, 558, 558, 558, 558, - /* 3050 */ 558, 558, 558, 558, 558, 558, 558, 558, 558, 558, - /* 3060 */ 558, 558, 558, 558, 558, 558, 558, 558, 558, 558, - /* 3070 */ 558, 558, 558, 558, 558, 558, 558, 558, 558, 558, - /* 3080 */ 558, 558, 558, 558, 558, 558, 558, 558, 558, 558, - /* 3090 */ 558, 558, 558, 558, 558, 558, 558, 558, 558, 558, - /* 3100 */ 558, 558, 558, 558, 558, 558, 558, 558, 558, 558, - /* 3110 */ 558, 558, 558, 558, 558, 558, 558, 558, 558, 558, - /* 3120 */ 558, 558, 558, 558, 558, 558, 558, 558, 558, 558, - /* 3130 */ 558, 558, 558, 558, 558, 558, 558, 558, 558, 558, - /* 3140 */ 558, 558, 558, 558, 558, 558, 558, 558, 558, 558, - /* 3150 */ 558, 558, 558, 558, 558, 558, 558, 558, 558, 558, - /* 3160 */ 558, 558, 558, 558, 558, 558, 558, 558, 558, 558, - /* 3170 */ 558, 558, 558, 558, 558, 558, 558, 558, 558, 558, - /* 3180 */ 558, 558, 558, 558, 558, 558, 558, 558, 558, 558, - /* 3190 */ 558, 558, 558, 558, 558, 558, 558, 558, 558, 558, - /* 3200 */ 558, 558, 558, 558, 558, 558, 558, 558, 558, 558, - /* 3210 */ 558, 558, 558, 558, 558, 558, 558, 558, 558, 558, - /* 3220 */ 558, 558, 558, 558, 558, 558, 558, 558, 558, 558, - /* 3230 */ 558, 558, 558, 558, 558, 558, 558, 558, 558, 558, - /* 3240 */ 558, 558, 558, 558, 558, 378, 378, 378, 378, 378, - /* 3250 */ 378, 378, 378, 378, 378, 378, 378, 378, 378, 378, - /* 3260 */ 378, 378, 378, 378, 378, 378, 378, 378, 378, 378, - /* 3270 */ 378, 378, 378, 378, 378, 378, 378, 378, 378, 378, - /* 3280 */ 378, 378, 378, 378, 378, 378, 378, 378, 378, 378, - /* 3290 */ 378, 378, 378, 378, 378, 378, 378, 378, 378, 378, - /* 3300 */ 378, 378, 378, 378, 378, 378, 378, 378, 378, 378, - /* 3310 */ 378, 378, 378, 378, 378, 378, 378, 378, 378, 378, - /* 3320 */ 378, 378, 378, 378, 378, 378, 378, 378, 378, 378, - /* 3330 */ 378, 378, 378, 378, 378, 378, 378, 378, 378, 378, - /* 3340 */ 378, 378, 378, 378, 378, 378, 378, 378, 378, 378, - /* 3350 */ 378, 378, 378, 378, 378, 378, 378, 378, 378, 378, - /* 3360 */ 378, 378, 378, 378, 378, 378, 378, 378, 378, 378, - /* 3370 */ 378, 378, 378, 378, 378, 378, 378, 378, 378, 378, - /* 3380 */ 378, 378, 378, 378, 378, 378, 378, 378, 378, 378, - /* 3390 */ 378, 378, 378, 378, 378, 378, + /* 270 */ 270, 271, 272, 273, 274, 12, 13, 444, 115, 468, + /* 280 */ 18, 74, 20, 20, 0, 22, 426, 446, 447, 27, + /* 290 */ 449, 458, 30, 452, 75, 435, 436, 464, 465, 36, + /* 300 */ 38, 38, 405, 289, 290, 291, 292, 293, 294, 295, + /* 310 */ 296, 297, 255, 12, 13, 2, 54, 20, 56, 3, + /* 320 */ 423, 8, 9, 61, 62, 12, 13, 14, 15, 16, + /* 330 */ 433, 68, 426, 522, 427, 73, 20, 36, 75, 20, + /* 340 */ 434, 530, 327, 4, 125, 82, 486, 487, 215, 396, + /* 350 */ 217, 445, 382, 293, 294, 295, 296, 297, 19, 548, + /* 360 */ 549, 77, 78, 79, 553, 554, 300, 397, 84, 85, + /* 370 */ 86, 290, 291, 292, 90, 112, 114, 38, 115, 95, + /* 380 */ 96, 97, 98, 250, 20, 101, 522, 0, 126, 105, + /* 390 */ 106, 107, 108, 54, 530, 488, 426, 300, 8, 9, + /* 400 */ 61, 62, 12, 13, 14, 15, 16, 68, 455, 190, + /* 410 */ 457, 193, 442, 549, 444, 152, 153, 553, 554, 157, + /* 420 */ 158, 0, 160, 161, 162, 163, 164, 165, 166, 167, + /* 430 */ 168, 169, 116, 300, 172, 173, 174, 175, 176, 177, + /* 440 */ 178, 179, 55, 181, 182, 183, 14, 20, 54, 187, + /* 450 */ 188, 189, 20, 114, 191, 192, 194, 63, 119, 489, + /* 460 */ 66, 67, 492, 300, 201, 202, 496, 497, 498, 499, + /* 470 */ 500, 501, 440, 503, 506, 443, 444, 509, 508, 216, + /* 480 */ 510, 218, 228, 439, 514, 515, 442, 155, 8, 9, + /* 490 */ 396, 397, 12, 13, 14, 15, 16, 278, 279, 280, + /* 500 */ 281, 282, 283, 284, 285, 286, 287, 288, 20, 193, + /* 510 */ 191, 192, 542, 33, 413, 13, 253, 254, 255, 218, + /* 520 */ 257, 258, 259, 260, 261, 262, 263, 264, 265, 266, + /* 530 */ 267, 268, 269, 270, 271, 272, 273, 274, 275, 12, + /* 540 */ 13, 382, 115, 442, 392, 20, 522, 20, 396, 22, + /* 550 */ 398, 77, 78, 79, 530, 20, 397, 147, 84, 85, + /* 560 */ 86, 151, 405, 36, 90, 38, 73, 427, 428, 95, + /* 570 */ 96, 97, 98, 549, 382, 101, 13, 553, 554, 105, + /* 580 */ 106, 107, 108, 193, 82, 426, 444, 68, 379, 397, + /* 590 */ 433, 399, 171, 492, 493, 68, 116, 176, 443, 444, + /* 600 */ 458, 442, 75, 444, 503, 184, 464, 465, 20, 82, + /* 610 */ 426, 517, 518, 519, 520, 495, 522, 523, 426, 255, + /* 620 */ 0, 289, 290, 291, 292, 293, 294, 295, 296, 297, + /* 630 */ 376, 377, 378, 114, 442, 225, 444, 403, 119, 112, + /* 640 */ 230, 521, 115, 91, 234, 82, 236, 20, 489, 22, + /* 650 */ 115, 492, 22, 33, 420, 496, 497, 498, 499, 500, + /* 660 */ 501, 253, 503, 36, 430, 506, 36, 508, 509, 510, + /* 670 */ 486, 487, 22, 514, 515, 36, 0, 468, 417, 152, + /* 680 */ 153, 489, 473, 20, 492, 58, 36, 426, 496, 497, + /* 690 */ 498, 499, 500, 501, 0, 503, 1, 2, 437, 524, + /* 700 */ 508, 526, 510, 36, 211, 186, 514, 515, 382, 8, + /* 710 */ 9, 159, 82, 12, 13, 14, 15, 16, 191, 192, + /* 720 */ 312, 313, 314, 315, 316, 317, 318, 14, 201, 202, + /* 730 */ 54, 522, 82, 20, 241, 242, 184, 185, 14, 530, + /* 740 */ 152, 153, 112, 216, 20, 218, 397, 146, 8, 9, + /* 750 */ 198, 0, 12, 13, 14, 15, 16, 548, 549, 2, + /* 760 */ 382, 382, 553, 554, 20, 8, 9, 75, 442, 12, + /* 770 */ 13, 14, 15, 16, 421, 426, 397, 396, 399, 426, + /* 780 */ 253, 254, 255, 36, 257, 258, 259, 260, 261, 262, + /* 790 */ 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, + /* 800 */ 273, 274, 12, 13, 14, 426, 453, 382, 74, 396, + /* 810 */ 20, 116, 22, 20, 126, 152, 153, 116, 396, 397, + /* 820 */ 442, 442, 397, 444, 399, 74, 36, 0, 38, 82, + /* 830 */ 41, 42, 231, 232, 233, 300, 403, 382, 416, 145, + /* 840 */ 146, 147, 148, 149, 150, 151, 216, 498, 218, 196, + /* 850 */ 20, 426, 397, 381, 399, 383, 116, 218, 68, 478, + /* 860 */ 479, 396, 397, 430, 201, 202, 495, 442, 489, 444, + /* 870 */ 275, 492, 82, 460, 461, 496, 497, 498, 499, 500, + /* 880 */ 501, 426, 503, 253, 254, 218, 382, 508, 426, 510, + /* 890 */ 251, 252, 521, 514, 515, 20, 434, 442, 20, 444, + /* 900 */ 82, 397, 112, 399, 191, 115, 439, 445, 115, 442, + /* 910 */ 121, 122, 426, 124, 489, 191, 426, 492, 251, 252, + /* 920 */ 434, 496, 497, 498, 499, 500, 501, 437, 503, 382, + /* 930 */ 426, 445, 126, 508, 145, 510, 468, 193, 149, 514, + /* 940 */ 515, 396, 152, 153, 489, 253, 442, 492, 444, 382, + /* 950 */ 427, 496, 497, 498, 499, 500, 501, 33, 503, 306, + /* 960 */ 307, 308, 309, 508, 3, 510, 396, 397, 255, 514, + /* 970 */ 515, 47, 145, 146, 147, 148, 149, 150, 151, 255, + /* 980 */ 36, 191, 192, 518, 519, 520, 416, 522, 523, 442, + /* 990 */ 522, 201, 202, 489, 0, 52, 492, 414, 530, 255, + /* 1000 */ 496, 497, 498, 499, 500, 501, 216, 503, 218, 442, + /* 1010 */ 112, 382, 508, 275, 510, 277, 548, 549, 514, 515, + /* 1020 */ 426, 553, 554, 478, 479, 127, 128, 129, 130, 131, + /* 1030 */ 132, 133, 134, 135, 136, 427, 138, 139, 140, 141, + /* 1040 */ 142, 143, 144, 253, 254, 255, 0, 257, 258, 259, + /* 1050 */ 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, + /* 1060 */ 270, 271, 272, 273, 274, 12, 13, 382, 193, 191, + /* 1070 */ 468, 442, 468, 20, 417, 22, 12, 13, 14, 15, + /* 1080 */ 16, 487, 397, 426, 399, 255, 468, 396, 397, 36, + /* 1090 */ 44, 38, 426, 300, 437, 396, 397, 439, 8, 9, + /* 1100 */ 442, 382, 12, 13, 14, 15, 16, 416, 396, 397, + /* 1110 */ 427, 426, 522, 396, 397, 416, 397, 396, 397, 453, + /* 1120 */ 530, 68, 396, 397, 522, 4, 522, 442, 416, 444, + /* 1130 */ 255, 427, 530, 255, 530, 82, 193, 416, 548, 549, + /* 1140 */ 522, 427, 416, 553, 554, 426, 474, 204, 530, 155, + /* 1150 */ 548, 549, 548, 549, 426, 553, 554, 553, 554, 396, + /* 1160 */ 397, 442, 218, 444, 170, 112, 548, 549, 115, 396, + /* 1170 */ 397, 553, 554, 445, 489, 0, 20, 492, 22, 416, + /* 1180 */ 22, 496, 497, 498, 499, 500, 501, 20, 503, 416, + /* 1190 */ 396, 397, 426, 508, 36, 510, 422, 12, 13, 514, + /* 1200 */ 515, 435, 436, 59, 60, 152, 153, 22, 489, 426, + /* 1210 */ 416, 492, 396, 397, 58, 496, 497, 498, 499, 500, + /* 1220 */ 501, 36, 503, 38, 401, 402, 36, 508, 445, 510, + /* 1230 */ 396, 397, 416, 514, 515, 518, 519, 520, 382, 522, + /* 1240 */ 523, 382, 22, 382, 191, 192, 396, 397, 396, 397, + /* 1250 */ 416, 410, 411, 68, 201, 202, 36, 483, 112, 145, + /* 1260 */ 146, 147, 148, 149, 150, 151, 416, 82, 416, 216, + /* 1270 */ 112, 218, 82, 127, 128, 129, 130, 131, 132, 133, + /* 1280 */ 134, 135, 136, 322, 138, 139, 140, 141, 142, 143, + /* 1290 */ 144, 14, 15, 16, 190, 401, 402, 112, 442, 410, + /* 1300 */ 411, 442, 382, 442, 385, 386, 253, 254, 255, 382, + /* 1310 */ 257, 258, 259, 260, 261, 262, 263, 264, 265, 266, + /* 1320 */ 267, 268, 269, 270, 271, 272, 273, 274, 12, 13, + /* 1330 */ 396, 397, 112, 396, 397, 18, 20, 382, 22, 382, + /* 1340 */ 23, 396, 397, 418, 396, 397, 382, 422, 396, 397, + /* 1350 */ 416, 382, 36, 416, 38, 396, 397, 382, 382, 42, + /* 1360 */ 43, 416, 442, 46, 416, 382, 382, 44, 416, 442, + /* 1370 */ 382, 152, 153, 186, 57, 416, 507, 507, 509, 509, + /* 1380 */ 397, 126, 278, 0, 68, 382, 69, 70, 71, 72, + /* 1390 */ 73, 68, 288, 426, 0, 33, 33, 442, 82, 442, + /* 1400 */ 397, 216, 14, 218, 33, 22, 442, 220, 20, 426, + /* 1410 */ 145, 442, 445, 120, 149, 33, 123, 442, 442, 33, + /* 1420 */ 245, 120, 255, 302, 123, 442, 442, 444, 112, 426, + /* 1430 */ 442, 115, 115, 47, 120, 180, 13, 123, 253, 254, + /* 1440 */ 12, 13, 13, 1, 2, 442, 120, 444, 0, 123, + /* 1450 */ 22, 0, 267, 268, 269, 270, 271, 272, 273, 36, + /* 1460 */ 235, 68, 237, 427, 36, 36, 38, 0, 152, 153, + /* 1470 */ 22, 154, 489, 22, 383, 492, 33, 51, 116, 496, + /* 1480 */ 497, 498, 499, 500, 501, 33, 503, 116, 115, 22, + /* 1490 */ 117, 508, 489, 510, 382, 492, 68, 514, 515, 496, + /* 1500 */ 497, 498, 499, 500, 501, 454, 503, 191, 192, 397, + /* 1510 */ 33, 33, 119, 33, 33, 33, 524, 201, 202, 33, + /* 1520 */ 12, 13, 557, 33, 36, 208, 209, 210, 12, 13, + /* 1530 */ 213, 33, 216, 546, 218, 33, 33, 539, 426, 525, + /* 1540 */ 33, 115, 33, 226, 227, 33, 543, 544, 400, 155, + /* 1550 */ 426, 12, 13, 413, 442, 413, 444, 240, 395, 116, + /* 1560 */ 243, 454, 33, 246, 247, 248, 249, 250, 116, 253, + /* 1570 */ 254, 255, 545, 257, 258, 259, 260, 261, 262, 263, + /* 1580 */ 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, + /* 1590 */ 274, 12, 13, 116, 116, 545, 116, 116, 116, 12, + /* 1600 */ 13, 489, 116, 1, 492, 33, 116, 33, 496, 497, + /* 1610 */ 498, 499, 500, 501, 116, 503, 382, 300, 116, 116, + /* 1620 */ 508, 19, 510, 116, 33, 116, 514, 515, 116, 12, + /* 1630 */ 13, 397, 12, 13, 12, 13, 12, 13, 12, 13, + /* 1640 */ 38, 12, 13, 36, 216, 116, 218, 36, 13, 13, + /* 1650 */ 545, 467, 454, 545, 400, 53, 54, 397, 382, 490, + /* 1660 */ 426, 475, 441, 33, 33, 63, 64, 65, 66, 454, + /* 1670 */ 68, 36, 36, 397, 529, 550, 442, 454, 444, 251, + /* 1680 */ 252, 253, 529, 532, 516, 454, 415, 324, 116, 82, + /* 1690 */ 116, 469, 303, 82, 382, 267, 268, 269, 270, 271, + /* 1700 */ 272, 273, 426, 54, 491, 20, 218, 116, 326, 397, + /* 1710 */ 20, 396, 480, 234, 485, 405, 114, 480, 442, 214, + /* 1720 */ 444, 119, 471, 489, 405, 396, 492, 20, 397, 47, + /* 1730 */ 496, 497, 498, 499, 500, 501, 451, 503, 426, 397, + /* 1740 */ 451, 190, 448, 382, 510, 396, 116, 116, 514, 515, + /* 1750 */ 397, 396, 150, 451, 442, 448, 444, 448, 397, 448, + /* 1760 */ 409, 113, 111, 396, 408, 489, 396, 110, 492, 20, + /* 1770 */ 396, 407, 496, 497, 498, 499, 500, 501, 52, 503, + /* 1780 */ 396, 382, 405, 396, 389, 393, 510, 426, 480, 389, + /* 1790 */ 514, 515, 20, 20, 393, 405, 397, 195, 444, 197, + /* 1800 */ 398, 489, 200, 442, 492, 444, 20, 205, 496, 497, + /* 1810 */ 498, 499, 500, 501, 502, 503, 504, 505, 405, 405, + /* 1820 */ 470, 405, 398, 382, 20, 426, 224, 461, 396, 20, + /* 1830 */ 405, 455, 405, 389, 426, 405, 405, 426, 397, 385, + /* 1840 */ 385, 442, 238, 444, 426, 426, 426, 426, 426, 396, + /* 1850 */ 489, 389, 442, 492, 442, 115, 484, 496, 497, 498, + /* 1860 */ 499, 500, 501, 426, 503, 382, 426, 426, 403, 20, + /* 1870 */ 479, 510, 426, 426, 222, 514, 515, 442, 482, 221, + /* 1880 */ 397, 396, 477, 442, 480, 444, 311, 403, 489, 38, + /* 1890 */ 538, 492, 310, 476, 444, 496, 497, 498, 499, 500, + /* 1900 */ 501, 442, 503, 469, 319, 54, 538, 462, 207, 426, + /* 1910 */ 541, 462, 320, 538, 63, 64, 65, 66, 540, 68, + /* 1920 */ 321, 298, 537, 304, 299, 442, 469, 444, 328, 323, + /* 1930 */ 489, 528, 535, 492, 536, 325, 527, 496, 497, 498, + /* 1940 */ 499, 500, 501, 397, 503, 20, 126, 301, 382, 462, + /* 1950 */ 398, 510, 403, 495, 555, 556, 515, 403, 462, 552, + /* 1960 */ 442, 442, 442, 397, 558, 114, 551, 442, 442, 533, + /* 1970 */ 119, 403, 489, 199, 459, 492, 442, 455, 115, 496, + /* 1980 */ 497, 498, 499, 500, 501, 513, 503, 382, 199, 403, + /* 1990 */ 442, 531, 426, 456, 397, 455, 403, 403, 422, 442, + /* 2000 */ 442, 442, 397, 442, 442, 442, 442, 442, 442, 442, + /* 2010 */ 444, 442, 442, 442, 115, 442, 403, 396, 22, 442, + /* 2020 */ 442, 431, 442, 442, 384, 442, 442, 544, 35, 382, + /* 2030 */ 442, 426, 466, 388, 37, 40, 428, 186, 442, 389, + /* 2040 */ 442, 387, 404, 442, 397, 390, 195, 442, 488, 444, + /* 2050 */ 199, 200, 419, 419, 380, 489, 205, 206, 492, 0, + /* 2060 */ 428, 472, 496, 497, 498, 499, 500, 501, 0, 503, + /* 2070 */ 0, 466, 463, 426, 47, 224, 481, 463, 419, 0, + /* 2080 */ 36, 244, 36, 36, 36, 244, 0, 36, 36, 442, + /* 2090 */ 244, 444, 36, 0, 489, 0, 244, 492, 0, 36, + /* 2100 */ 0, 496, 497, 498, 499, 500, 501, 36, 503, 382, + /* 2110 */ 0, 22, 0, 36, 239, 0, 224, 0, 224, 218, + /* 2120 */ 225, 216, 0, 0, 397, 0, 382, 212, 211, 0, + /* 2130 */ 0, 158, 51, 51, 0, 36, 489, 0, 0, 492, + /* 2140 */ 36, 397, 54, 496, 497, 498, 499, 500, 501, 0, + /* 2150 */ 503, 382, 51, 426, 0, 47, 0, 0, 0, 0, + /* 2160 */ 51, 0, 0, 0, 0, 0, 397, 176, 36, 442, + /* 2170 */ 426, 444, 0, 0, 176, 0, 0, 0, 0, 0, + /* 2180 */ 0, 0, 0, 0, 0, 0, 442, 0, 444, 47, + /* 2190 */ 382, 0, 0, 0, 547, 426, 0, 0, 0, 0, + /* 2200 */ 0, 51, 0, 0, 0, 397, 0, 0, 0, 0, + /* 2210 */ 0, 442, 0, 444, 0, 158, 489, 22, 0, 492, + /* 2220 */ 0, 157, 156, 496, 497, 498, 499, 500, 501, 0, + /* 2230 */ 503, 0, 0, 489, 426, 466, 492, 22, 52, 22, + /* 2240 */ 496, 497, 498, 499, 500, 501, 52, 503, 0, 505, + /* 2250 */ 442, 0, 444, 14, 0, 36, 68, 0, 489, 0, + /* 2260 */ 0, 492, 36, 0, 36, 496, 497, 498, 499, 500, + /* 2270 */ 501, 382, 503, 68, 466, 54, 54, 44, 44, 68, + /* 2280 */ 0, 68, 36, 556, 54, 44, 397, 0, 382, 0, + /* 2290 */ 47, 44, 36, 51, 33, 51, 51, 489, 0, 44, + /* 2300 */ 492, 0, 0, 397, 496, 497, 498, 499, 500, 501, + /* 2310 */ 382, 503, 45, 0, 44, 426, 0, 51, 207, 0, + /* 2320 */ 51, 0, 0, 0, 0, 397, 76, 36, 54, 44, + /* 2330 */ 0, 442, 426, 444, 44, 36, 54, 0, 54, 36, + /* 2340 */ 44, 0, 36, 0, 44, 0, 54, 0, 442, 0, + /* 2350 */ 444, 36, 125, 382, 426, 0, 0, 22, 123, 0, + /* 2360 */ 22, 36, 36, 36, 36, 36, 0, 36, 397, 36, + /* 2370 */ 442, 0, 444, 56, 22, 36, 36, 33, 489, 33, + /* 2380 */ 0, 492, 0, 22, 36, 496, 497, 498, 499, 500, + /* 2390 */ 501, 22, 503, 382, 36, 489, 36, 426, 492, 22, + /* 2400 */ 22, 36, 496, 497, 498, 499, 500, 501, 397, 503, + /* 2410 */ 0, 0, 0, 442, 0, 444, 22, 489, 0, 36, + /* 2420 */ 492, 36, 0, 36, 496, 497, 498, 499, 500, 501, + /* 2430 */ 116, 503, 382, 20, 36, 36, 36, 426, 0, 51, + /* 2440 */ 0, 36, 193, 22, 0, 229, 22, 397, 223, 115, + /* 2450 */ 0, 228, 0, 442, 3, 444, 219, 115, 382, 33, + /* 2460 */ 489, 193, 33, 492, 193, 193, 193, 496, 497, 498, + /* 2470 */ 499, 500, 501, 397, 503, 199, 426, 115, 115, 305, + /* 2480 */ 203, 203, 36, 116, 116, 36, 52, 115, 113, 52, + /* 2490 */ 111, 33, 442, 33, 444, 33, 51, 116, 51, 115, + /* 2500 */ 489, 82, 426, 492, 33, 116, 115, 496, 497, 498, + /* 2510 */ 499, 500, 501, 116, 503, 36, 115, 115, 442, 116, + /* 2520 */ 444, 115, 382, 116, 116, 3, 33, 36, 36, 305, + /* 2530 */ 36, 116, 36, 36, 36, 51, 116, 397, 33, 489, + /* 2540 */ 51, 305, 492, 0, 0, 115, 496, 497, 498, 499, + /* 2550 */ 500, 501, 382, 503, 289, 44, 0, 116, 116, 44, + /* 2560 */ 196, 0, 44, 115, 33, 489, 426, 397, 492, 115, + /* 2570 */ 115, 382, 496, 497, 498, 499, 500, 501, 115, 503, + /* 2580 */ 116, 113, 442, 276, 444, 113, 397, 2, 196, 22, + /* 2590 */ 200, 195, 116, 116, 116, 115, 426, 115, 115, 51, + /* 2600 */ 115, 115, 115, 253, 51, 115, 22, 0, 44, 256, + /* 2610 */ 0, 22, 442, 196, 444, 426, 116, 115, 115, 118, + /* 2620 */ 116, 115, 115, 51, 115, 117, 116, 22, 115, 489, + /* 2630 */ 22, 442, 492, 444, 115, 382, 496, 497, 498, 499, + /* 2640 */ 500, 501, 115, 503, 115, 22, 126, 36, 116, 36, + /* 2650 */ 397, 115, 229, 116, 36, 115, 36, 382, 116, 489, + /* 2660 */ 36, 116, 492, 116, 36, 116, 496, 497, 498, 499, + /* 2670 */ 500, 501, 397, 503, 36, 115, 137, 33, 489, 426, + /* 2680 */ 115, 492, 36, 137, 115, 496, 497, 498, 499, 500, + /* 2690 */ 501, 137, 503, 22, 137, 442, 75, 444, 76, 22, + /* 2700 */ 36, 426, 36, 36, 36, 36, 36, 82, 36, 36, + /* 2710 */ 36, 36, 36, 109, 82, 109, 33, 442, 36, 444, + /* 2720 */ 36, 36, 22, 36, 82, 36, 36, 36, 36, 36, + /* 2730 */ 36, 36, 22, 382, 36, 0, 0, 36, 36, 0, + /* 2740 */ 54, 44, 489, 44, 36, 492, 0, 36, 397, 496, + /* 2750 */ 497, 498, 499, 500, 501, 382, 503, 54, 54, 44, + /* 2760 */ 54, 44, 0, 36, 489, 0, 22, 492, 382, 36, + /* 2770 */ 397, 496, 497, 498, 499, 500, 501, 426, 503, 0, + /* 2780 */ 22, 33, 22, 397, 21, 36, 36, 559, 22, 22, + /* 2790 */ 21, 20, 559, 442, 559, 444, 559, 559, 382, 426, + /* 2800 */ 559, 559, 559, 559, 559, 559, 559, 559, 559, 559, + /* 2810 */ 559, 559, 426, 397, 559, 442, 559, 444, 559, 559, + /* 2820 */ 559, 559, 559, 559, 559, 559, 559, 559, 442, 559, + /* 2830 */ 444, 559, 559, 559, 559, 559, 559, 559, 559, 559, + /* 2840 */ 489, 559, 426, 492, 559, 559, 559, 496, 497, 498, + /* 2850 */ 499, 500, 501, 559, 503, 559, 559, 559, 442, 559, + /* 2860 */ 444, 559, 489, 559, 559, 492, 559, 559, 559, 496, + /* 2870 */ 497, 498, 499, 500, 501, 489, 503, 382, 492, 559, + /* 2880 */ 559, 559, 496, 497, 498, 499, 500, 501, 559, 503, + /* 2890 */ 559, 559, 397, 559, 382, 559, 559, 559, 559, 559, + /* 2900 */ 559, 559, 559, 559, 559, 489, 559, 559, 492, 397, + /* 2910 */ 559, 559, 496, 497, 498, 499, 500, 501, 559, 503, + /* 2920 */ 382, 426, 559, 559, 559, 559, 559, 559, 559, 559, + /* 2930 */ 559, 559, 559, 559, 559, 397, 559, 442, 426, 444, + /* 2940 */ 559, 559, 559, 559, 559, 559, 559, 559, 559, 559, + /* 2950 */ 559, 559, 559, 559, 442, 559, 444, 559, 382, 559, + /* 2960 */ 559, 559, 559, 559, 426, 559, 559, 559, 559, 559, + /* 2970 */ 559, 559, 559, 397, 559, 559, 559, 559, 559, 559, + /* 2980 */ 442, 559, 444, 559, 489, 559, 559, 492, 559, 559, + /* 2990 */ 559, 496, 497, 498, 499, 500, 501, 559, 503, 559, + /* 3000 */ 559, 489, 426, 559, 492, 559, 559, 559, 496, 497, + /* 3010 */ 498, 499, 500, 501, 559, 503, 559, 559, 442, 559, + /* 3020 */ 444, 559, 559, 559, 559, 559, 559, 489, 559, 559, + /* 3030 */ 492, 559, 559, 559, 496, 497, 498, 499, 500, 501, + /* 3040 */ 382, 503, 559, 559, 559, 559, 559, 559, 559, 559, + /* 3050 */ 559, 559, 559, 559, 559, 397, 559, 559, 559, 559, + /* 3060 */ 559, 559, 559, 559, 559, 489, 559, 559, 492, 559, + /* 3070 */ 382, 559, 496, 497, 498, 499, 500, 501, 559, 503, + /* 3080 */ 559, 559, 559, 559, 426, 397, 559, 559, 559, 559, + /* 3090 */ 559, 559, 559, 559, 559, 559, 559, 559, 559, 559, + /* 3100 */ 442, 559, 444, 559, 559, 559, 559, 559, 559, 559, + /* 3110 */ 559, 559, 559, 559, 426, 559, 559, 559, 559, 559, + /* 3120 */ 559, 559, 559, 559, 559, 559, 559, 559, 559, 559, + /* 3130 */ 442, 559, 444, 559, 559, 559, 559, 559, 559, 559, + /* 3140 */ 559, 559, 559, 559, 559, 559, 559, 489, 559, 559, + /* 3150 */ 492, 559, 559, 559, 496, 497, 498, 499, 500, 501, + /* 3160 */ 559, 503, 559, 559, 559, 559, 559, 559, 559, 559, + /* 3170 */ 559, 559, 559, 559, 559, 559, 559, 489, 559, 559, + /* 3180 */ 492, 559, 559, 559, 496, 497, 498, 499, 500, 501, + /* 3190 */ 559, 503, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3200 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3210 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3220 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3230 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3240 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3250 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3260 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3270 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3280 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3290 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3300 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3310 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3320 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3330 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3340 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3350 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3360 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3370 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3380 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3390 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3400 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3410 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3420 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3430 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3440 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3450 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3460 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3470 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3480 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3490 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3500 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3510 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3520 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3530 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3540 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3550 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3560 */ 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, + /* 3570 */ 379, }; #define YY_SHIFT_COUNT (978) #define YY_SHIFT_MIN (0) -#define YY_SHIFT_MAX (2846) +#define YY_SHIFT_MAX (2779) static const unsigned short int yy_shift_ofst[] = { - /* 0 */ 347, 0, 83, 0, 346, 346, 346, 346, 346, 346, - /* 10 */ 346, 346, 346, 346, 346, 346, 429, 691, 691, 774, - /* 20 */ 691, 691, 691, 691, 691, 691, 691, 691, 691, 691, - /* 30 */ 691, 691, 691, 691, 691, 691, 691, 691, 691, 691, - /* 40 */ 691, 691, 691, 691, 691, 691, 691, 691, 691, 691, - /* 50 */ 102, 116, 534, 212, 408, 766, 408, 408, 212, 212, - /* 60 */ 408, 1175, 408, 950, 1175, 623, 408, 20, 1338, 432, - /* 70 */ 432, 214, 214, 1338, 1338, 459, 459, 432, 437, 437, - /* 80 */ 74, 3, 3, 50, 52, 214, 214, 214, 214, 214, - /* 90 */ 214, 214, 214, 214, 214, 214, 286, 407, 473, 214, - /* 100 */ 214, 37, 20, 214, 286, 214, 20, 214, 214, 214, - /* 110 */ 214, 20, 214, 214, 214, 20, 214, 20, 20, 20, - /* 120 */ 636, 39, 39, 523, 523, 1433, 647, 236, 489, 1038, - /* 130 */ 1038, 1038, 1038, 1038, 1038, 1038, 1038, 1038, 1038, 1038, - /* 140 */ 1038, 1038, 1038, 1038, 1038, 1038, 1038, 1038, 1222, 743, - /* 150 */ 437, 74, 1035, 1035, 790, 261, 261, 261, 632, 632, - /* 160 */ 496, 461, 790, 37, 20, 613, 20, 20, 483, 20, - /* 170 */ 20, 792, 20, 792, 792, 764, 1227, 523, 523, 523, - /* 180 */ 523, 523, 523, 1577, 302, 405, 563, 348, 348, 763, - /* 190 */ 599, 363, 330, 700, 122, 49, 747, 61, 61, 1086, - /* 200 */ 692, 901, 901, 901, 947, 901, 543, 1041, 1228, 1010, - /* 210 */ 1049, 452, 1051, 1166, 1166, 1172, 1320, 1320, 1103, 820, - /* 220 */ 981, 1166, 461, 1401, 1636, 1691, 1695, 1487, 37, 1695, - /* 230 */ 37, 1531, 1691, 1732, 1709, 1732, 1709, 1574, 1691, 1732, - /* 240 */ 1691, 1709, 1574, 1574, 1574, 1662, 1667, 1691, 1691, 1674, - /* 250 */ 1691, 1691, 1691, 1771, 1740, 1771, 1740, 1695, 37, 37, - /* 260 */ 1777, 37, 1785, 1788, 37, 1785, 37, 1802, 37, 1806, - /* 270 */ 37, 37, 1691, 37, 1771, 20, 20, 20, 20, 20, - /* 280 */ 20, 20, 20, 20, 20, 20, 1691, 1227, 1227, 1771, - /* 290 */ 792, 792, 792, 1637, 1765, 1695, 636, 1863, 1665, 1663, - /* 300 */ 1777, 636, 1401, 1691, 792, 1589, 1592, 1589, 1592, 1587, - /* 310 */ 1701, 1589, 1591, 1594, 1606, 1401, 1632, 1639, 1612, 1617, - /* 320 */ 1623, 1732, 1930, 1822, 1651, 1785, 636, 636, 1592, 792, - /* 330 */ 792, 792, 792, 1592, 792, 1776, 636, 792, 1806, 636, - /* 340 */ 1880, 792, 1800, 1806, 636, 764, 636, 1732, 792, 792, - /* 350 */ 792, 792, 792, 792, 792, 792, 792, 792, 792, 792, - /* 360 */ 792, 792, 792, 792, 792, 792, 792, 792, 792, 792, - /* 370 */ 1900, 792, 1691, 636, 1996, 1985, 2008, 2009, 1771, 3018, - /* 380 */ 3018, 3018, 3018, 3018, 3018, 3018, 3018, 3018, 3018, 3018, - /* 390 */ 3018, 1077, 867, 648, 1206, 887, 1005, 1043, 1214, 1350, - /* 400 */ 1692, 934, 1475, 1084, 1084, 1084, 1084, 1084, 1084, 1084, - /* 410 */ 1084, 1084, 183, 512, 220, 693, 702, 702, 4, 616, - /* 420 */ 811, 726, 542, 542, 145, 255, 542, 769, 683, 1209, - /* 430 */ 856, 141, 141, 113, 160, 738, 113, 113, 113, 91, - /* 440 */ 1241, 222, 925, 1301, 1019, 1249, 1388, 1272, 1287, 1291, - /* 450 */ 1310, 1347, 1418, 1425, 558, 1397, 1449, 1450, 1164, 1283, - /* 460 */ 1376, 1208, 1403, 1404, 1430, 1431, 1216, 1278, 1114, 1432, - /* 470 */ 1463, 1464, 1476, 1466, 770, 1468, 1402, 1477, 1478, 1479, - /* 480 */ 1497, 1498, 1501, 1520, 1526, 1530, 1538, 1546, 1573, 1579, - /* 490 */ 1590, 1604, 1615, 1565, 1567, 1586, 1603, 1613, 1624, 1366, - /* 500 */ 1473, 1369, 1458, 1508, 1598, 1547, 1440, 2090, 2091, 2092, - /* 510 */ 2047, 2095, 2061, 1855, 2063, 2070, 2072, 1868, 2112, 2077, - /* 520 */ 2078, 1872, 2080, 2117, 2118, 1876, 2120, 2085, 2122, 2087, - /* 530 */ 2124, 2104, 2127, 2093, 1890, 2131, 1910, 2135, 1912, 1918, - /* 540 */ 1927, 1932, 2146, 2149, 2150, 1939, 1941, 2153, 2155, 1998, - /* 550 */ 2106, 2107, 2160, 2125, 2162, 2163, 2128, 2111, 2166, 2116, - /* 560 */ 2168, 2126, 2169, 2171, 2175, 2129, 2177, 2178, 2179, 2181, - /* 570 */ 2182, 2183, 2010, 2148, 2185, 2011, 2188, 2189, 2190, 2191, - /* 580 */ 2193, 2195, 2198, 2199, 2200, 2201, 2202, 2204, 2205, 2206, - /* 590 */ 2207, 2208, 2209, 2210, 2212, 2213, 2164, 2214, 2187, 2218, - /* 600 */ 2222, 2230, 2232, 2235, 2236, 2237, 2241, 2244, 2233, 2252, - /* 610 */ 2096, 2257, 2102, 2261, 2108, 2262, 2263, 2243, 2216, 2248, - /* 620 */ 2219, 2279, 2220, 2281, 2223, 2250, 2282, 2224, 2284, 2225, - /* 630 */ 2290, 2294, 2259, 2242, 2253, 2298, 2264, 2245, 2258, 2301, - /* 640 */ 2268, 2254, 2274, 2319, 2285, 2320, 2275, 2283, 2292, 2272, - /* 650 */ 2277, 2312, 2278, 2331, 2291, 2299, 2335, 2345, 2346, 2347, - /* 660 */ 2304, 2142, 2354, 2272, 2300, 2356, 2272, 2306, 2358, 2367, - /* 670 */ 2296, 2368, 2370, 2337, 2321, 2330, 2376, 2341, 2324, 2336, - /* 680 */ 2381, 2348, 2329, 2352, 2392, 2359, 2343, 2355, 2398, 2401, - /* 690 */ 2402, 2403, 2404, 2405, 2288, 2295, 2371, 2384, 2408, 2387, - /* 700 */ 2375, 2378, 2379, 2385, 2386, 2395, 2397, 2399, 2406, 2407, - /* 710 */ 2410, 2412, 2413, 2415, 2414, 2420, 2416, 2434, 2417, 2441, - /* 720 */ 2422, 2390, 2452, 2431, 2419, 2454, 2456, 2458, 2423, 2462, - /* 730 */ 2427, 2470, 2436, 2473, 2453, 2461, 2438, 2440, 2442, 2369, - /* 740 */ 2372, 2483, 2293, 2255, 2260, 2374, 2267, 2272, 2443, 2491, - /* 750 */ 2314, 2457, 2486, 2505, 2297, 2487, 2317, 2313, 2511, 2513, - /* 760 */ 2322, 2315, 2326, 2318, 2514, 2489, 2226, 2424, 2409, 2425, - /* 770 */ 2421, 2484, 2492, 2426, 2471, 2432, 2495, 2445, 2433, 2526, - /* 780 */ 2527, 2446, 2448, 2449, 2450, 2451, 2528, 2497, 2517, 2455, - /* 790 */ 2533, 2265, 2490, 2459, 2538, 2464, 2537, 2465, 2466, 2574, - /* 800 */ 2547, 2280, 2549, 2550, 2551, 2552, 2554, 2555, 2467, 2478, - /* 810 */ 2544, 2316, 2572, 2556, 2606, 2608, 2494, 2566, 2496, 2498, - /* 820 */ 2500, 2501, 2429, 2503, 2611, 2569, 2439, 2619, 2504, 2508, - /* 830 */ 2430, 2590, 2460, 2603, 2524, 2366, 2530, 2642, 2623, 2396, - /* 840 */ 2531, 2534, 2536, 2539, 2541, 2542, 2543, 2545, 2598, 2548, - /* 850 */ 2553, 2601, 2557, 2631, 2411, 2559, 2560, 2659, 2561, 2563, - /* 860 */ 2469, 2616, 2564, 2567, 2669, 2648, 2558, 2565, 2272, 2621, - /* 870 */ 2568, 2571, 2582, 2575, 2585, 2576, 2679, 2681, 2683, 2475, - /* 880 */ 2591, 2645, 2672, 2595, 2597, 2678, 2600, 2609, 2686, 2536, - /* 890 */ 2610, 2688, 2539, 2612, 2694, 2541, 2615, 2697, 2542, 2599, - /* 900 */ 2602, 2613, 2617, 2625, 2705, 2634, 2716, 2640, 2705, 2705, - /* 910 */ 2734, 2682, 2684, 2735, 2724, 2725, 2726, 2727, 2728, 2729, - /* 920 */ 2730, 2731, 2732, 2733, 2737, 2689, 2661, 2693, 2670, 2745, - /* 930 */ 2744, 2753, 2754, 2769, 2757, 2759, 2760, 2715, 2407, 2764, - /* 940 */ 2410, 2766, 2767, 2768, 2775, 2791, 2779, 2817, 2783, 2770, - /* 950 */ 2776, 2821, 2786, 2771, 2782, 2823, 2792, 2773, 2785, 2830, - /* 960 */ 2795, 2781, 2793, 2836, 2802, 2840, 2819, 2808, 2845, 2832, - /* 970 */ 2822, 2816, 2820, 2835, 2837, 2838, 2841, 2843, 2846, + /* 0 */ 1317, 0, 263, 0, 527, 527, 527, 527, 527, 527, + /* 10 */ 527, 527, 527, 527, 527, 527, 790, 1053, 1053, 1316, + /* 20 */ 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, + /* 30 */ 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, + /* 40 */ 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, + /* 50 */ 535, 793, 133, 427, 66, 97, 66, 66, 427, 427, + /* 60 */ 66, 1185, 66, 262, 1185, 163, 66, 64, 1428, 663, + /* 70 */ 663, 89, 89, 1428, 1428, 171, 171, 663, 319, 319, + /* 80 */ 588, 432, 432, 364, 54, 89, 89, 89, 89, 89, + /* 90 */ 89, 89, 89, 89, 89, 89, 297, 488, 525, 89, + /* 100 */ 89, 207, 64, 89, 297, 89, 64, 89, 89, 89, + /* 110 */ 89, 64, 89, 89, 89, 64, 89, 64, 64, 64, + /* 120 */ 734, 219, 219, 898, 898, 1146, 474, 14, 69, 630, + /* 130 */ 630, 630, 630, 630, 630, 630, 630, 630, 630, 630, + /* 140 */ 630, 630, 630, 630, 630, 630, 630, 630, 789, 316, + /* 150 */ 319, 588, 1144, 1144, 747, 218, 218, 218, 738, 738, + /* 160 */ 751, 502, 747, 207, 64, 688, 64, 64, 595, 64, + /* 170 */ 64, 818, 64, 818, 818, 806, 7, 898, 898, 898, + /* 180 */ 898, 898, 898, 1602, 284, 21, 57, 332, 332, 15, + /* 190 */ 408, 653, 60, 627, 81, 713, 724, 301, 301, 11, + /* 200 */ 744, 650, 650, 650, 943, 650, 878, 875, 1156, 1323, + /* 210 */ 1388, 1265, 1187, 830, 830, 1167, 563, 563, 961, 924, + /* 220 */ 1121, 830, 502, 1389, 1649, 1685, 1690, 1479, 207, 1690, + /* 230 */ 207, 1505, 1685, 1707, 1682, 1707, 1682, 1551, 1685, 1707, + /* 240 */ 1685, 1682, 1551, 1551, 1551, 1648, 1651, 1685, 1685, 1657, + /* 250 */ 1685, 1685, 1685, 1749, 1726, 1749, 1726, 1690, 207, 207, + /* 260 */ 1772, 207, 1773, 1786, 207, 1773, 207, 1804, 207, 1809, + /* 270 */ 207, 207, 1685, 207, 1749, 64, 64, 64, 64, 64, + /* 280 */ 64, 64, 64, 64, 64, 64, 1685, 7, 7, 1749, + /* 290 */ 818, 818, 818, 1604, 1740, 1690, 734, 1849, 1652, 1658, + /* 300 */ 1772, 734, 1389, 1685, 818, 1575, 1582, 1575, 1582, 1585, + /* 310 */ 1701, 1575, 1599, 1592, 1619, 1389, 1625, 1623, 1600, 1610, + /* 320 */ 1606, 1707, 1925, 1820, 1646, 1773, 734, 734, 1582, 818, + /* 330 */ 818, 818, 818, 1582, 818, 1774, 734, 818, 1809, 734, + /* 340 */ 1863, 818, 1789, 1809, 734, 806, 734, 1707, 818, 818, + /* 350 */ 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, + /* 360 */ 818, 818, 818, 818, 818, 818, 818, 818, 818, 818, + /* 370 */ 1899, 818, 1685, 734, 1996, 1993, 1997, 1995, 1749, 3192, + /* 380 */ 3192, 3192, 3192, 3192, 3192, 3192, 3192, 3192, 3192, 3192, + /* 390 */ 3192, 39, 1851, 215, 339, 480, 701, 740, 313, 757, + /* 400 */ 390, 694, 827, 1090, 1090, 1090, 1090, 1090, 1090, 1090, + /* 410 */ 1090, 1090, 1114, 147, 410, 552, 1064, 1064, 493, 519, + /* 420 */ 421, 394, 254, 254, 601, 639, 667, 254, 994, 1158, + /* 430 */ 1220, 105, 105, 1277, 695, 1104, 1277, 1277, 1277, 676, + /* 440 */ 1175, 1362, 1046, 1386, 1255, 1394, 620, 1293, 1301, 1314, + /* 450 */ 1326, 1190, 1423, 1429, 1383, 1448, 1451, 1467, 1225, 1371, + /* 460 */ 1443, 1393, 1452, 1477, 1480, 1481, 1219, 1363, 1382, 1482, + /* 470 */ 1486, 1490, 1442, 1498, 692, 1502, 1426, 1478, 1503, 1507, + /* 480 */ 1509, 1512, 1508, 1516, 1539, 1579, 1587, 1617, 1620, 1622, + /* 490 */ 1624, 1626, 1629, 1529, 1572, 1574, 1591, 1630, 1631, 1373, + /* 500 */ 1607, 944, 1488, 1635, 1636, 1611, 387, 2059, 2068, 2070, + /* 510 */ 2027, 2079, 2044, 1837, 2046, 2047, 2048, 1841, 2086, 2051, + /* 520 */ 2052, 1846, 2056, 2093, 2095, 1852, 2098, 2063, 2100, 2071, + /* 530 */ 2110, 2089, 2112, 2077, 1875, 2115, 1892, 2117, 1894, 1895, + /* 540 */ 1901, 1905, 2122, 2123, 2125, 1915, 1917, 2129, 2130, 1973, + /* 550 */ 2081, 2082, 2134, 2099, 2137, 2138, 2104, 2088, 2149, 2101, + /* 560 */ 2154, 2108, 2156, 2157, 2158, 2109, 2159, 2161, 2162, 2163, + /* 570 */ 2164, 2165, 1991, 2132, 2172, 1998, 2173, 2175, 2176, 2177, + /* 580 */ 2178, 2179, 2180, 2181, 2182, 2183, 2184, 2185, 2187, 2191, + /* 590 */ 2192, 2193, 2196, 2197, 2198, 2199, 2150, 2200, 2142, 2202, + /* 600 */ 2203, 2204, 2206, 2207, 2208, 2209, 2210, 2212, 2195, 2214, + /* 610 */ 2057, 2218, 2064, 2220, 2066, 2229, 2231, 2215, 2186, 2217, + /* 620 */ 2194, 2232, 2188, 2248, 2205, 2219, 2251, 2211, 2257, 2213, + /* 630 */ 2259, 2260, 2226, 2221, 2233, 2263, 2228, 2222, 2234, 2280, + /* 640 */ 2246, 2230, 2241, 2287, 2256, 2289, 2243, 2247, 2261, 2242, + /* 650 */ 2244, 2239, 2245, 2254, 2267, 2255, 2298, 2301, 2302, 2313, + /* 660 */ 2270, 2111, 2316, 2242, 2266, 2319, 2242, 2269, 2321, 2322, + /* 670 */ 2250, 2323, 2324, 2291, 2274, 2285, 2330, 2299, 2282, 2290, + /* 680 */ 2337, 2303, 2284, 2296, 2341, 2306, 2292, 2300, 2343, 2345, + /* 690 */ 2347, 2349, 2355, 2356, 2227, 2235, 2315, 2335, 2359, 2338, + /* 700 */ 2325, 2326, 2327, 2328, 2329, 2331, 2333, 2339, 2340, 2344, + /* 710 */ 2346, 2348, 2358, 2352, 2360, 2366, 2361, 2371, 2369, 2380, + /* 720 */ 2377, 2317, 2382, 2378, 2365, 2410, 2411, 2412, 2383, 2414, + /* 730 */ 2385, 2418, 2387, 2422, 2394, 2413, 2398, 2399, 2400, 2314, + /* 740 */ 2334, 2438, 2249, 2216, 2223, 2342, 2225, 2242, 2388, 2440, + /* 750 */ 2268, 2405, 2421, 2444, 2237, 2424, 2271, 2276, 2450, 2452, + /* 760 */ 2272, 2277, 2273, 2278, 2451, 2426, 2174, 2362, 2367, 2363, + /* 770 */ 2368, 2446, 2449, 2372, 2434, 2375, 2437, 2379, 2381, 2429, + /* 780 */ 2458, 2389, 2384, 2391, 2401, 2397, 2460, 2445, 2447, 2402, + /* 790 */ 2462, 2224, 2419, 2403, 2471, 2406, 2479, 2407, 2408, 2522, + /* 800 */ 2493, 2236, 2491, 2492, 2494, 2496, 2497, 2498, 2415, 2420, + /* 810 */ 2484, 2265, 2505, 2489, 2543, 2544, 2430, 2511, 2441, 2442, + /* 820 */ 2448, 2454, 2364, 2455, 2556, 2515, 2390, 2561, 2464, 2463, + /* 830 */ 2392, 2518, 2396, 2531, 2468, 2307, 2472, 2585, 2567, 2350, + /* 840 */ 2476, 2477, 2480, 2482, 2483, 2485, 2486, 2478, 2548, 2487, + /* 850 */ 2490, 2553, 2500, 2584, 2353, 2502, 2503, 2607, 2504, 2506, + /* 860 */ 2417, 2564, 2507, 2508, 2610, 2589, 2501, 2509, 2242, 2572, + /* 870 */ 2513, 2519, 2510, 2527, 2529, 2520, 2605, 2608, 2623, 2423, + /* 880 */ 2532, 2611, 2613, 2536, 2537, 2618, 2540, 2542, 2620, 2480, + /* 890 */ 2545, 2624, 2482, 2547, 2628, 2483, 2549, 2638, 2485, 2539, + /* 900 */ 2546, 2554, 2557, 2560, 2644, 2565, 2646, 2569, 2644, 2644, + /* 910 */ 2671, 2622, 2621, 2677, 2664, 2666, 2667, 2668, 2669, 2670, + /* 920 */ 2672, 2673, 2674, 2675, 2676, 2625, 2604, 2632, 2606, 2683, + /* 930 */ 2682, 2684, 2685, 2700, 2687, 2689, 2690, 2642, 2344, 2691, + /* 940 */ 2346, 2692, 2693, 2694, 2695, 2710, 2698, 2735, 2701, 2686, + /* 950 */ 2697, 2736, 2702, 2703, 2699, 2739, 2708, 2704, 2715, 2746, + /* 960 */ 2711, 2706, 2717, 2762, 2727, 2765, 2744, 2733, 2779, 2758, + /* 970 */ 2748, 2749, 2750, 2760, 2763, 2766, 2767, 2769, 2771, }; #define YY_REDUCE_COUNT (390) -#define YY_REDUCE_MIN (-503) -#define YY_REDUCE_MAX (2515) +#define YY_REDUCE_MIN (-479) +#define YY_REDUCE_MAX (2688) static const short yy_reduce_ofst[] = { - /* 0 */ -239, -365, -325, -19, -202, 325, 366, 705, 810, 840, - /* 10 */ 960, 991, 1027, 1156, 1237, 1318, 1342, 1361, 1429, 1494, - /* 20 */ 1469, 1537, 1605, 1563, 1642, 1729, 1751, 1778, 1815, 1842, - /* 30 */ 1864, 1891, 1928, 1969, 2004, 2036, 2055, 2101, 2133, 2197, - /* 40 */ 2221, 2246, 2286, 2310, 2351, 2391, 2418, 2437, 2502, 2515, - /* 50 */ -310, -61, -464, -362, 865, 955, 1145, 1165, -297, -271, - /* 60 */ 1201, -412, -503, -384, -382, -82, 13, -393, -385, -436, - /* 70 */ -356, -210, -21, -389, -324, -388, -247, -439, -142, 115, - /* 80 */ 148, -198, -191, -359, 89, 173, 313, 345, 403, 409, - /* 90 */ 435, 433, 440, 454, 578, 492, -337, 387, -10, 445, - /* 100 */ 583, 472, 412, 601, 393, 614, 15, 653, 675, 794, - /* 110 */ 837, 349, 844, 851, 874, 31, 877, 111, 375, 365, - /* 120 */ 565, -501, -501, -66, -111, -223, 595, -100, 120, -231, - /* 130 */ -3, 548, 753, 814, 817, 834, 836, 839, 843, 861, - /* 140 */ 881, 904, 916, 932, 937, 948, 949, 974, -405, 530, - /* 150 */ 87, 372, 704, 741, 869, 530, 806, 876, 434, 499, - /* 160 */ 673, -224, 884, 228, 64, 893, -195, 547, 644, 863, - /* 170 */ 718, 939, 871, 941, 943, -48, 919, -425, -411, 935, - /* 180 */ 999, 1009, 1040, 1007, 1093, 1134, 1075, 1012, 1012, 993, - /* 190 */ 1008, 1032, 1033, 1236, 1012, 1212, 1212, 1226, 1238, 1245, - /* 200 */ 1195, 1105, 1115, 1116, 1197, 1117, 1212, 1211, 1268, 1182, - /* 210 */ 1276, 1233, 1203, 1229, 1231, 1212, 1147, 1151, 1136, 1173, - /* 220 */ 1158, 1234, 1281, 1242, 1219, 1321, 1247, 1240, 1317, 1252, - /* 230 */ 1337, 1274, 1352, 1359, 1308, 1364, 1312, 1316, 1370, 1372, - /* 240 */ 1371, 1319, 1323, 1324, 1326, 1368, 1373, 1384, 1386, 1381, - /* 250 */ 1393, 1394, 1395, 1405, 1406, 1407, 1408, 1315, 1392, 1399, - /* 260 */ 1356, 1415, 1410, 1343, 1417, 1414, 1419, 1374, 1420, 1375, - /* 270 */ 1424, 1443, 1441, 1444, 1447, 1426, 1428, 1437, 1439, 1442, - /* 280 */ 1446, 1448, 1451, 1452, 1453, 1454, 1460, 1482, 1484, 1481, - /* 290 */ 1445, 1455, 1456, 1398, 1411, 1409, 1480, 1413, 1422, 1427, - /* 300 */ 1457, 1491, 1421, 1500, 1465, 1377, 1461, 1378, 1462, 1367, - /* 310 */ 1382, 1379, 1396, 1412, 1400, 1470, 1416, 1423, 1383, 1391, - /* 320 */ 1459, 1550, 1467, 1438, 1483, 1555, 1551, 1552, 1495, 1514, - /* 330 */ 1517, 1522, 1532, 1511, 1533, 1518, 1575, 1540, 1525, 1585, - /* 340 */ 1485, 1557, 1545, 1553, 1608, 1584, 1609, 1618, 1576, 1578, - /* 350 */ 1580, 1581, 1583, 1588, 1599, 1600, 1601, 1602, 1611, 1614, - /* 360 */ 1616, 1625, 1627, 1628, 1630, 1631, 1633, 1634, 1635, 1638, - /* 370 */ 1596, 1640, 1621, 1629, 1644, 1658, 1660, 1661, 1668, 1582, - /* 380 */ 1646, 1593, 1597, 1620, 1622, 1669, 1670, 1659, 1671, 1675, - /* 390 */ 1685, + /* 0 */ 209, -338, 159, -30, 192, 379, 425, 455, 504, 685, + /* 10 */ 719, 983, 1112, 1234, 1276, 1361, 1312, 1003, 1399, 1441, + /* 20 */ 1483, 1566, 1605, 1647, 1727, 1744, 1769, 1808, 1889, 1906, + /* 30 */ 1928, 1971, 2011, 2050, 2076, 2140, 2170, 2189, 2253, 2275, + /* 40 */ 2351, 2373, 2386, 2416, 2495, 2512, 2538, 2576, 2658, 2688, + /* 50 */ -340, -324, -468, 94, -189, 468, 602, 604, 465, 717, + /* 60 */ 618, -337, 590, -159, 101, -136, 24, -140, -425, -167, + /* 70 */ 142, -381, -333, -389, -318, -385, -184, -357, -204, 353, + /* 80 */ 32, -199, 152, 349, -379, 422, 570, 691, 712, -395, + /* 90 */ -371, 721, 726, 763, 773, -247, 381, 413, -47, 794, + /* 100 */ 816, -103, 184, 834, 545, 850, -94, 852, 934, 937, + /* 110 */ 945, 261, 699, 948, 952, 462, 959, 766, 486, 657, + /* 120 */ 234, -479, -479, 140, -93, -419, -403, 175, 472, -269, + /* 130 */ 326, 378, 547, 567, 629, 856, 859, 861, 920, 927, + /* 140 */ 955, 957, 964, 969, 975, 976, 984, 988, -277, -295, + /* 150 */ 666, 155, 823, 894, 841, -295, 120, 371, 869, 870, + /* 160 */ 433, -323, 889, 157, 594, 774, 728, 783, -32, 490, + /* 170 */ -231, 44, 967, 467, 658, 925, 919, 523, 608, 683, + /* 180 */ 704, 714, 1036, 672, 583, 1091, 1051, 992, 992, 965, + /* 190 */ 987, 998, 1014, 1148, 992, 1124, 1124, 1140, 1142, 1163, + /* 200 */ 1107, 1027, 1050, 1105, 1184, 1108, 1124, 1198, 1254, 1169, + /* 210 */ 1260, 1221, 1186, 1215, 1223, 1124, 1145, 1153, 1125, 1168, + /* 220 */ 1151, 1231, 1271, 1222, 1213, 1315, 1232, 1229, 1310, 1237, + /* 230 */ 1319, 1251, 1329, 1331, 1285, 1342, 1289, 1294, 1349, 1353, + /* 240 */ 1355, 1302, 1307, 1309, 1311, 1351, 1356, 1367, 1370, 1364, + /* 250 */ 1374, 1384, 1387, 1395, 1392, 1400, 1401, 1308, 1377, 1390, + /* 260 */ 1354, 1413, 1402, 1350, 1414, 1424, 1416, 1366, 1425, 1376, + /* 270 */ 1427, 1430, 1432, 1431, 1444, 1408, 1411, 1418, 1419, 1420, + /* 280 */ 1421, 1422, 1437, 1440, 1446, 1447, 1453, 1454, 1455, 1462, + /* 290 */ 1410, 1412, 1435, 1372, 1396, 1404, 1465, 1391, 1405, 1417, + /* 300 */ 1450, 1484, 1434, 1485, 1459, 1352, 1445, 1368, 1449, 1369, + /* 310 */ 1378, 1375, 1385, 1398, 1397, 1457, 1403, 1409, 1406, 1407, + /* 320 */ 1415, 1546, 1458, 1436, 1460, 1552, 1549, 1554, 1487, 1518, + /* 330 */ 1519, 1520, 1525, 1496, 1526, 1515, 1568, 1534, 1522, 1586, + /* 340 */ 1472, 1548, 1537, 1540, 1593, 1576, 1594, 1597, 1557, 1558, + /* 350 */ 1559, 1561, 1562, 1563, 1564, 1565, 1567, 1569, 1570, 1571, + /* 360 */ 1573, 1577, 1578, 1580, 1581, 1583, 1584, 1588, 1596, 1598, + /* 370 */ 1590, 1601, 1621, 1613, 1640, 1654, 1645, 1655, 1650, 1589, + /* 380 */ 1608, 1560, 1595, 1609, 1614, 1633, 1634, 1632, 1659, 1638, + /* 390 */ 1674, }; static const YYACTIONTYPE yy_default[] = { - /* 0 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 10 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 20 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 30 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 40 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 50 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 60 */ 2578, 2203, 2203, 2534, 2203, 2203, 2203, 2203, 2203, 2203, - /* 70 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2541, 2541, - /* 80 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 90 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 100 */ 2203, 2309, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 110 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 120 */ 2307, 2830, 2203, 2956, 2619, 2203, 2203, 2859, 2203, 2203, - /* 130 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 140 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2842, - /* 150 */ 2203, 2203, 2280, 2280, 2203, 2842, 2842, 2842, 2802, 2802, - /* 160 */ 2307, 2203, 2203, 2309, 2203, 2621, 2203, 2203, 2203, 2203, - /* 170 */ 2203, 2203, 2203, 2203, 2203, 2450, 2233, 2203, 2203, 2203, - /* 180 */ 2203, 2203, 2203, 2604, 2203, 2203, 2888, 2834, 2835, 2950, - /* 190 */ 2203, 2891, 2853, 2203, 2848, 2203, 2203, 2203, 2203, 2203, - /* 200 */ 2878, 2203, 2203, 2203, 2203, 2203, 2203, 2546, 2203, 2647, - /* 210 */ 2203, 2395, 2598, 2203, 2203, 2203, 2203, 2203, 2934, 2832, - /* 220 */ 2872, 2203, 2203, 2882, 2203, 2203, 2203, 2635, 2309, 2203, - /* 230 */ 2309, 2591, 2529, 2203, 2539, 2203, 2539, 2536, 2203, 2203, - /* 240 */ 2203, 2539, 2536, 2536, 2536, 2383, 2379, 2203, 2203, 2377, - /* 250 */ 2203, 2203, 2203, 2203, 2263, 2203, 2263, 2203, 2309, 2309, - /* 260 */ 2203, 2309, 2203, 2203, 2309, 2203, 2309, 2203, 2309, 2203, - /* 270 */ 2309, 2309, 2203, 2309, 2203, 2203, 2203, 2203, 2203, 2203, - /* 280 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 290 */ 2203, 2203, 2203, 2633, 2614, 2203, 2307, 2203, 2602, 2600, - /* 300 */ 2203, 2307, 2882, 2203, 2203, 2904, 2899, 2904, 2899, 2918, - /* 310 */ 2914, 2904, 2923, 2920, 2884, 2882, 2865, 2861, 2953, 2940, - /* 320 */ 2936, 2203, 2203, 2870, 2868, 2203, 2307, 2307, 2899, 2203, - /* 330 */ 2203, 2203, 2203, 2899, 2203, 2203, 2307, 2203, 2203, 2307, - /* 340 */ 2203, 2203, 2203, 2203, 2307, 2203, 2307, 2203, 2203, 2203, - /* 350 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 360 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 370 */ 2412, 2203, 2203, 2307, 2203, 2235, 2237, 2247, 2203, 2593, - /* 380 */ 2956, 2619, 2624, 2574, 2574, 2453, 2453, 2956, 2453, 2310, - /* 390 */ 2208, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 400 */ 2203, 2203, 2203, 2917, 2916, 2753, 2203, 2806, 2805, 2804, - /* 410 */ 2795, 2752, 2408, 2203, 2203, 2203, 2751, 2750, 2203, 2203, - /* 420 */ 2203, 2203, 2399, 2396, 2203, 2203, 2421, 2203, 2203, 2203, - /* 430 */ 2203, 2565, 2564, 2744, 2203, 2203, 2745, 2743, 2742, 2203, - /* 440 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 450 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 460 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2937, 2941, 2203, - /* 470 */ 2203, 2203, 2831, 2203, 2203, 2203, 2723, 2203, 2203, 2203, - /* 480 */ 2203, 2203, 2691, 2686, 2677, 2668, 2683, 2674, 2662, 2680, - /* 490 */ 2671, 2659, 2656, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 500 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 510 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 520 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 530 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 540 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2535, - /* 550 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 560 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 570 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 580 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 590 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 600 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 610 */ 2203, 2203, 2203, 2203, 2550, 2203, 2203, 2203, 2203, 2203, - /* 620 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 630 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 640 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2252, 2730, - /* 650 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 660 */ 2203, 2203, 2203, 2733, 2203, 2203, 2734, 2203, 2203, 2203, - /* 670 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 680 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 690 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 700 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2354, - /* 710 */ 2353, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 720 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 730 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2735, - /* 740 */ 2203, 2203, 2203, 2203, 2618, 2203, 2203, 2725, 2203, 2203, - /* 750 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 760 */ 2203, 2203, 2203, 2203, 2933, 2885, 2203, 2203, 2203, 2203, - /* 770 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 780 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2723, 2203, - /* 790 */ 2915, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2931, 2203, - /* 800 */ 2935, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2841, 2837, - /* 810 */ 2203, 2203, 2833, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 820 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 830 */ 2203, 2203, 2203, 2792, 2203, 2203, 2203, 2826, 2203, 2203, - /* 840 */ 2203, 2203, 2449, 2448, 2447, 2446, 2203, 2203, 2203, 2203, - /* 850 */ 2203, 2203, 2735, 2203, 2738, 2203, 2203, 2203, 2203, 2203, - /* 860 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2722, 2203, - /* 870 */ 2777, 2776, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 880 */ 2203, 2203, 2203, 2443, 2203, 2203, 2203, 2203, 2203, 2203, - /* 890 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2427, - /* 900 */ 2425, 2424, 2423, 2203, 2460, 2203, 2203, 2203, 2456, 2455, - /* 910 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 920 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2328, - /* 930 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2320, 2203, - /* 940 */ 2319, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 950 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 960 */ 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, - /* 970 */ 2232, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, + /* 0 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 10 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 20 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 30 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 40 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 50 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 60 */ 2580, 2205, 2205, 2536, 2205, 2205, 2205, 2205, 2205, 2205, + /* 70 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2543, 2543, + /* 80 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 90 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 100 */ 2205, 2311, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 110 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 120 */ 2309, 2833, 2205, 2959, 2621, 2205, 2205, 2862, 2205, 2205, + /* 130 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 140 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2845, + /* 150 */ 2205, 2205, 2282, 2282, 2205, 2845, 2845, 2845, 2805, 2805, + /* 160 */ 2309, 2205, 2205, 2311, 2205, 2623, 2205, 2205, 2205, 2205, + /* 170 */ 2205, 2205, 2205, 2205, 2205, 2452, 2235, 2205, 2205, 2205, + /* 180 */ 2205, 2205, 2205, 2606, 2205, 2205, 2891, 2837, 2838, 2953, + /* 190 */ 2205, 2894, 2856, 2205, 2851, 2205, 2205, 2205, 2205, 2205, + /* 200 */ 2881, 2205, 2205, 2205, 2205, 2205, 2205, 2548, 2205, 2650, + /* 210 */ 2205, 2397, 2600, 2205, 2205, 2205, 2205, 2205, 2937, 2835, + /* 220 */ 2875, 2205, 2205, 2885, 2205, 2205, 2205, 2638, 2311, 2205, + /* 230 */ 2311, 2593, 2531, 2205, 2541, 2205, 2541, 2538, 2205, 2205, + /* 240 */ 2205, 2541, 2538, 2538, 2538, 2385, 2381, 2205, 2205, 2379, + /* 250 */ 2205, 2205, 2205, 2205, 2265, 2205, 2265, 2205, 2311, 2311, + /* 260 */ 2205, 2311, 2205, 2205, 2311, 2205, 2311, 2205, 2311, 2205, + /* 270 */ 2311, 2311, 2205, 2311, 2205, 2205, 2205, 2205, 2205, 2205, + /* 280 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 290 */ 2205, 2205, 2205, 2636, 2616, 2205, 2309, 2205, 2604, 2602, + /* 300 */ 2205, 2309, 2885, 2205, 2205, 2907, 2902, 2907, 2902, 2921, + /* 310 */ 2917, 2907, 2926, 2923, 2887, 2885, 2868, 2864, 2956, 2943, + /* 320 */ 2939, 2205, 2205, 2873, 2871, 2205, 2309, 2309, 2902, 2205, + /* 330 */ 2205, 2205, 2205, 2902, 2205, 2205, 2309, 2205, 2205, 2309, + /* 340 */ 2205, 2205, 2205, 2205, 2309, 2205, 2309, 2205, 2205, 2205, + /* 350 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 360 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 370 */ 2414, 2205, 2205, 2309, 2205, 2237, 2239, 2249, 2205, 2595, + /* 380 */ 2959, 2621, 2626, 2576, 2576, 2455, 2455, 2959, 2455, 2312, + /* 390 */ 2210, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 400 */ 2205, 2205, 2205, 2920, 2919, 2756, 2205, 2809, 2808, 2807, + /* 410 */ 2798, 2755, 2410, 2205, 2205, 2205, 2754, 2753, 2205, 2205, + /* 420 */ 2205, 2205, 2401, 2398, 2205, 2205, 2205, 2423, 2205, 2205, + /* 430 */ 2205, 2567, 2566, 2747, 2205, 2205, 2748, 2746, 2745, 2205, + /* 440 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 450 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 460 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2940, 2944, 2205, + /* 470 */ 2205, 2205, 2834, 2205, 2205, 2205, 2726, 2205, 2205, 2205, + /* 480 */ 2205, 2205, 2694, 2689, 2680, 2671, 2686, 2677, 2665, 2683, + /* 490 */ 2674, 2662, 2659, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 500 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 510 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 520 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 530 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 540 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2537, + /* 550 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 560 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 570 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 580 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 590 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 600 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 610 */ 2205, 2205, 2205, 2205, 2552, 2205, 2205, 2205, 2205, 2205, + /* 620 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 630 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 640 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2254, 2733, + /* 650 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 660 */ 2205, 2205, 2205, 2736, 2205, 2205, 2737, 2205, 2205, 2205, + /* 670 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 680 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 690 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 700 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2356, + /* 710 */ 2355, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 720 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 730 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2738, + /* 740 */ 2205, 2205, 2205, 2205, 2620, 2205, 2205, 2728, 2205, 2205, + /* 750 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 760 */ 2205, 2205, 2205, 2205, 2936, 2888, 2205, 2205, 2205, 2205, + /* 770 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 780 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2726, 2205, + /* 790 */ 2918, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2934, 2205, + /* 800 */ 2938, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2844, 2840, + /* 810 */ 2205, 2205, 2836, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 820 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 830 */ 2205, 2205, 2205, 2795, 2205, 2205, 2205, 2829, 2205, 2205, + /* 840 */ 2205, 2205, 2451, 2450, 2449, 2448, 2205, 2205, 2205, 2205, + /* 850 */ 2205, 2205, 2738, 2205, 2741, 2205, 2205, 2205, 2205, 2205, + /* 860 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2725, 2205, + /* 870 */ 2780, 2779, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 880 */ 2205, 2205, 2205, 2445, 2205, 2205, 2205, 2205, 2205, 2205, + /* 890 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2429, + /* 900 */ 2427, 2426, 2425, 2205, 2462, 2205, 2205, 2205, 2458, 2457, + /* 910 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 920 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2330, + /* 930 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2322, 2205, + /* 940 */ 2321, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 950 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 960 */ 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, + /* 970 */ 2234, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, }; /********** End of lemon-generated parsing tables *****************************/ @@ -1626,12 +1660,12 @@ static const YYCODETYPE yyFallback[] = { 0, /* BWLIMIT => nothing */ 0, /* START => nothing */ 0, /* TIMESTAMP => nothing */ - 328, /* END => ABORT */ + 329, /* END => ABORT */ 0, /* TABLE => nothing */ 0, /* NK_LP => nothing */ 0, /* NK_RP => nothing */ 0, /* USING => nothing */ - 328, /* FILE => ABORT */ + 329, /* FILE => ABORT */ 0, /* STABLE => nothing */ 0, /* COLUMN => nothing */ 0, /* MODIFY => nothing */ @@ -1699,7 +1733,7 @@ static const YYCODETYPE yyFallback[] = { 0, /* VNODES => nothing */ 0, /* ALIVE => nothing */ 0, /* VIEWS => nothing */ - 328, /* VIEW => ABORT */ + 329, /* VIEW => ABORT */ 0, /* COMPACTS => nothing */ 0, /* NORMAL => nothing */ 0, /* CHILD => nothing */ @@ -1742,10 +1776,11 @@ static const YYCODETYPE yyFallback[] = { 0, /* PAUSE => nothing */ 0, /* RESUME => nothing */ 0, /* PRIMARY => nothing */ - 328, /* KEY => ABORT */ + 329, /* KEY => ABORT */ 0, /* TRIGGER => nothing */ 0, /* AT_ONCE => nothing */ 0, /* WINDOW_CLOSE => nothing */ + 0, /* FORCE_WINDOW_CLOSE => nothing */ 0, /* IGNORE => nothing */ 0, /* EXPIRED => nothing */ 0, /* FILL_HISTORY => nothing */ @@ -1806,7 +1841,7 @@ static const YYCODETYPE yyFallback[] = { 0, /* LEFT => nothing */ 0, /* RIGHT => nothing */ 0, /* OUTER => nothing */ - 328, /* SEMI => ABORT */ + 329, /* SEMI => ABORT */ 0, /* ANTI => nothing */ 0, /* ASOF => nothing */ 0, /* WINDOW => nothing */ @@ -1842,52 +1877,52 @@ static const YYCODETYPE yyFallback[] = { 0, /* ASC => nothing */ 0, /* NULLS => nothing */ 0, /* ABORT => nothing */ - 328, /* AFTER => ABORT */ - 328, /* ATTACH => ABORT */ - 328, /* BEFORE => ABORT */ - 328, /* BEGIN => ABORT */ - 328, /* BITAND => ABORT */ - 328, /* BITNOT => ABORT */ - 328, /* BITOR => ABORT */ - 328, /* BLOCKS => ABORT */ - 328, /* CHANGE => ABORT */ - 328, /* COMMA => ABORT */ - 328, /* CONCAT => ABORT */ - 328, /* CONFLICT => ABORT */ - 328, /* COPY => ABORT */ - 328, /* DEFERRED => ABORT */ - 328, /* DELIMITERS => ABORT */ - 328, /* DETACH => ABORT */ - 328, /* DIVIDE => ABORT */ - 328, /* DOT => ABORT */ - 328, /* EACH => ABORT */ - 328, /* FAIL => ABORT */ - 328, /* FOR => ABORT */ - 328, /* GLOB => ABORT */ - 328, /* ID => ABORT */ - 328, /* IMMEDIATE => ABORT */ - 328, /* IMPORT => ABORT */ - 328, /* INITIALLY => ABORT */ - 328, /* INSTEAD => ABORT */ - 328, /* ISNULL => ABORT */ - 328, /* MODULES => ABORT */ - 328, /* NK_BITNOT => ABORT */ - 328, /* NK_SEMI => ABORT */ - 328, /* NOTNULL => ABORT */ - 328, /* OF => ABORT */ - 328, /* PLUS => ABORT */ - 328, /* PRIVILEGE => ABORT */ - 328, /* RAISE => ABORT */ - 328, /* RESTRICT => ABORT */ - 328, /* ROW => ABORT */ - 328, /* STAR => ABORT */ - 328, /* STATEMENT => ABORT */ - 328, /* STRICT => ABORT */ - 328, /* STRING => ABORT */ - 328, /* TIMES => ABORT */ - 328, /* VALUES => ABORT */ - 328, /* VARIABLE => ABORT */ - 328, /* WAL => ABORT */ + 329, /* AFTER => ABORT */ + 329, /* ATTACH => ABORT */ + 329, /* BEFORE => ABORT */ + 329, /* BEGIN => ABORT */ + 329, /* BITAND => ABORT */ + 329, /* BITNOT => ABORT */ + 329, /* BITOR => ABORT */ + 329, /* BLOCKS => ABORT */ + 329, /* CHANGE => ABORT */ + 329, /* COMMA => ABORT */ + 329, /* CONCAT => ABORT */ + 329, /* CONFLICT => ABORT */ + 329, /* COPY => ABORT */ + 329, /* DEFERRED => ABORT */ + 329, /* DELIMITERS => ABORT */ + 329, /* DETACH => ABORT */ + 329, /* DIVIDE => ABORT */ + 329, /* DOT => ABORT */ + 329, /* EACH => ABORT */ + 329, /* FAIL => ABORT */ + 329, /* FOR => ABORT */ + 329, /* GLOB => ABORT */ + 329, /* ID => ABORT */ + 329, /* IMMEDIATE => ABORT */ + 329, /* IMPORT => ABORT */ + 329, /* INITIALLY => ABORT */ + 329, /* INSTEAD => ABORT */ + 329, /* ISNULL => ABORT */ + 329, /* MODULES => ABORT */ + 329, /* NK_BITNOT => ABORT */ + 329, /* NK_SEMI => ABORT */ + 329, /* NOTNULL => ABORT */ + 329, /* OF => ABORT */ + 329, /* PLUS => ABORT */ + 329, /* PRIVILEGE => ABORT */ + 329, /* RAISE => ABORT */ + 329, /* RESTRICT => ABORT */ + 329, /* ROW => ABORT */ + 329, /* STAR => ABORT */ + 329, /* STATEMENT => ABORT */ + 329, /* STRICT => ABORT */ + 329, /* STRING => ABORT */ + 329, /* TIMES => ABORT */ + 329, /* VALUES => ABORT */ + 329, /* VARIABLE => ABORT */ + 329, /* WAL => ABORT */ 0, /* ENCODE => nothing */ 0, /* COMPRESS => nothing */ 0, /* LEVEL => nothing */ @@ -2212,331 +2247,332 @@ static const char *const yyTokenName[] = { /* 230 */ "TRIGGER", /* 231 */ "AT_ONCE", /* 232 */ "WINDOW_CLOSE", - /* 233 */ "IGNORE", - /* 234 */ "EXPIRED", - /* 235 */ "FILL_HISTORY", - /* 236 */ "UPDATE", - /* 237 */ "SUBTABLE", - /* 238 */ "UNTREATED", - /* 239 */ "KILL", - /* 240 */ "CONNECTION", - /* 241 */ "TRANSACTION", - /* 242 */ "BALANCE", - /* 243 */ "VGROUP", - /* 244 */ "LEADER", - /* 245 */ "MERGE", - /* 246 */ "REDISTRIBUTE", - /* 247 */ "SPLIT", - /* 248 */ "DELETE", - /* 249 */ "INSERT", - /* 250 */ "NK_BIN", - /* 251 */ "NK_HEX", - /* 252 */ "NULL", - /* 253 */ "NK_QUESTION", - /* 254 */ "NK_ALIAS", - /* 255 */ "NK_ARROW", - /* 256 */ "ROWTS", - /* 257 */ "QSTART", - /* 258 */ "QEND", - /* 259 */ "QDURATION", - /* 260 */ "WSTART", - /* 261 */ "WEND", - /* 262 */ "WDURATION", - /* 263 */ "IROWTS", - /* 264 */ "ISFILLED", - /* 265 */ "CAST", - /* 266 */ "NOW", - /* 267 */ "TODAY", - /* 268 */ "TIMEZONE", - /* 269 */ "CLIENT_VERSION", - /* 270 */ "SERVER_VERSION", - /* 271 */ "SERVER_STATUS", - /* 272 */ "CURRENT_USER", - /* 273 */ "CASE", - /* 274 */ "WHEN", - /* 275 */ "THEN", - /* 276 */ "ELSE", - /* 277 */ "BETWEEN", - /* 278 */ "IS", - /* 279 */ "NK_LT", - /* 280 */ "NK_GT", - /* 281 */ "NK_LE", - /* 282 */ "NK_GE", - /* 283 */ "NK_NE", - /* 284 */ "MATCH", - /* 285 */ "NMATCH", - /* 286 */ "CONTAINS", - /* 287 */ "IN", - /* 288 */ "JOIN", - /* 289 */ "INNER", - /* 290 */ "LEFT", - /* 291 */ "RIGHT", - /* 292 */ "OUTER", - /* 293 */ "SEMI", - /* 294 */ "ANTI", - /* 295 */ "ASOF", - /* 296 */ "WINDOW", - /* 297 */ "WINDOW_OFFSET", - /* 298 */ "JLIMIT", - /* 299 */ "SELECT", - /* 300 */ "NK_HINT", - /* 301 */ "DISTINCT", - /* 302 */ "WHERE", - /* 303 */ "PARTITION", - /* 304 */ "BY", - /* 305 */ "SESSION", - /* 306 */ "STATE_WINDOW", - /* 307 */ "EVENT_WINDOW", - /* 308 */ "COUNT_WINDOW", - /* 309 */ "SLIDING", - /* 310 */ "FILL", - /* 311 */ "VALUE", - /* 312 */ "VALUE_F", - /* 313 */ "NONE", - /* 314 */ "PREV", - /* 315 */ "NULL_F", - /* 316 */ "LINEAR", - /* 317 */ "NEXT", - /* 318 */ "HAVING", - /* 319 */ "RANGE", - /* 320 */ "EVERY", - /* 321 */ "ORDER", - /* 322 */ "SLIMIT", - /* 323 */ "SOFFSET", - /* 324 */ "LIMIT", - /* 325 */ "OFFSET", - /* 326 */ "ASC", - /* 327 */ "NULLS", - /* 328 */ "ABORT", - /* 329 */ "AFTER", - /* 330 */ "ATTACH", - /* 331 */ "BEFORE", - /* 332 */ "BEGIN", - /* 333 */ "BITAND", - /* 334 */ "BITNOT", - /* 335 */ "BITOR", - /* 336 */ "BLOCKS", - /* 337 */ "CHANGE", - /* 338 */ "COMMA", - /* 339 */ "CONCAT", - /* 340 */ "CONFLICT", - /* 341 */ "COPY", - /* 342 */ "DEFERRED", - /* 343 */ "DELIMITERS", - /* 344 */ "DETACH", - /* 345 */ "DIVIDE", - /* 346 */ "DOT", - /* 347 */ "EACH", - /* 348 */ "FAIL", - /* 349 */ "FOR", - /* 350 */ "GLOB", - /* 351 */ "ID", - /* 352 */ "IMMEDIATE", - /* 353 */ "IMPORT", - /* 354 */ "INITIALLY", - /* 355 */ "INSTEAD", - /* 356 */ "ISNULL", - /* 357 */ "MODULES", - /* 358 */ "NK_BITNOT", - /* 359 */ "NK_SEMI", - /* 360 */ "NOTNULL", - /* 361 */ "OF", - /* 362 */ "PLUS", - /* 363 */ "PRIVILEGE", - /* 364 */ "RAISE", - /* 365 */ "RESTRICT", - /* 366 */ "ROW", - /* 367 */ "STAR", - /* 368 */ "STATEMENT", - /* 369 */ "STRICT", - /* 370 */ "STRING", - /* 371 */ "TIMES", - /* 372 */ "VALUES", - /* 373 */ "VARIABLE", - /* 374 */ "WAL", - /* 375 */ "ENCODE", - /* 376 */ "COMPRESS", - /* 377 */ "LEVEL", - /* 378 */ "cmd", - /* 379 */ "account_options", - /* 380 */ "alter_account_options", - /* 381 */ "literal", - /* 382 */ "alter_account_option", - /* 383 */ "ip_range_list", - /* 384 */ "white_list", - /* 385 */ "white_list_opt", - /* 386 */ "is_import_opt", - /* 387 */ "is_createdb_opt", - /* 388 */ "user_name", - /* 389 */ "sysinfo_opt", - /* 390 */ "privileges", - /* 391 */ "priv_level", - /* 392 */ "with_opt", - /* 393 */ "priv_type_list", - /* 394 */ "priv_type", - /* 395 */ "db_name", - /* 396 */ "table_name", - /* 397 */ "topic_name", - /* 398 */ "search_condition", - /* 399 */ "dnode_endpoint", - /* 400 */ "force_opt", - /* 401 */ "unsafe_opt", - /* 402 */ "not_exists_opt", - /* 403 */ "db_options", - /* 404 */ "exists_opt", - /* 405 */ "alter_db_options", - /* 406 */ "speed_opt", - /* 407 */ "start_opt", - /* 408 */ "end_opt", - /* 409 */ "integer_list", - /* 410 */ "variable_list", - /* 411 */ "retention_list", - /* 412 */ "signed", - /* 413 */ "alter_db_option", - /* 414 */ "retention", - /* 415 */ "full_table_name", - /* 416 */ "column_def_list", - /* 417 */ "tags_def_opt", - /* 418 */ "table_options", - /* 419 */ "multi_create_clause", - /* 420 */ "tag_list_opt", - /* 421 */ "tags_def", - /* 422 */ "multi_drop_clause", - /* 423 */ "alter_table_clause", - /* 424 */ "alter_table_options", - /* 425 */ "column_name", - /* 426 */ "type_name", - /* 427 */ "column_options", - /* 428 */ "tags_literal", - /* 429 */ "create_subtable_clause", - /* 430 */ "specific_cols_opt", - /* 431 */ "tags_literal_list", - /* 432 */ "drop_table_clause", - /* 433 */ "col_name_list", - /* 434 */ "tag_def_list", - /* 435 */ "tag_def", - /* 436 */ "column_def", - /* 437 */ "type_name_default_len", - /* 438 */ "duration_list", - /* 439 */ "rollup_func_list", - /* 440 */ "alter_table_option", - /* 441 */ "duration_literal", - /* 442 */ "rollup_func_name", - /* 443 */ "function_name", - /* 444 */ "col_name", - /* 445 */ "db_kind_opt", - /* 446 */ "table_kind_db_name_cond_opt", - /* 447 */ "like_pattern_opt", - /* 448 */ "db_name_cond_opt", - /* 449 */ "table_name_cond", - /* 450 */ "from_db_opt", - /* 451 */ "table_kind", - /* 452 */ "tag_item", - /* 453 */ "column_alias", - /* 454 */ "tsma_name", - /* 455 */ "tsma_func_list", - /* 456 */ "full_tsma_name", - /* 457 */ "func_list", - /* 458 */ "index_options", - /* 459 */ "full_index_name", - /* 460 */ "index_name", - /* 461 */ "sliding_opt", - /* 462 */ "sma_stream_opt", - /* 463 */ "func", - /* 464 */ "sma_func_name", - /* 465 */ "expression_list", - /* 466 */ "with_meta", - /* 467 */ "query_or_subquery", - /* 468 */ "where_clause_opt", - /* 469 */ "cgroup_name", - /* 470 */ "analyze_opt", - /* 471 */ "explain_options", - /* 472 */ "insert_query", - /* 473 */ "or_replace_opt", - /* 474 */ "agg_func_opt", - /* 475 */ "bufsize_opt", - /* 476 */ "language_opt", - /* 477 */ "full_view_name", - /* 478 */ "view_name", - /* 479 */ "stream_name", - /* 480 */ "stream_options", - /* 481 */ "col_list_opt", - /* 482 */ "tag_def_or_ref_opt", - /* 483 */ "subtable_opt", - /* 484 */ "ignore_opt", - /* 485 */ "column_stream_def_list", - /* 486 */ "column_stream_def", - /* 487 */ "stream_col_options", - /* 488 */ "expression", - /* 489 */ "on_vgroup_id", - /* 490 */ "dnode_list", - /* 491 */ "literal_func", - /* 492 */ "signed_literal", - /* 493 */ "literal_list", - /* 494 */ "table_alias", - /* 495 */ "expr_or_subquery", - /* 496 */ "pseudo_column", - /* 497 */ "column_reference", - /* 498 */ "function_expression", - /* 499 */ "case_when_expression", - /* 500 */ "star_func", - /* 501 */ "star_func_para_list", - /* 502 */ "noarg_func", - /* 503 */ "other_para_list", - /* 504 */ "star_func_para", - /* 505 */ "when_then_list", - /* 506 */ "case_when_else_opt", - /* 507 */ "common_expression", - /* 508 */ "when_then_expr", - /* 509 */ "predicate", - /* 510 */ "compare_op", - /* 511 */ "in_op", - /* 512 */ "in_predicate_value", - /* 513 */ "boolean_value_expression", - /* 514 */ "boolean_primary", - /* 515 */ "from_clause_opt", - /* 516 */ "table_reference_list", - /* 517 */ "table_reference", - /* 518 */ "table_primary", - /* 519 */ "joined_table", - /* 520 */ "alias_opt", - /* 521 */ "subquery", - /* 522 */ "parenthesized_joined_table", - /* 523 */ "join_type", - /* 524 */ "join_subtype", - /* 525 */ "join_on_clause_opt", - /* 526 */ "window_offset_clause_opt", - /* 527 */ "jlimit_clause_opt", - /* 528 */ "window_offset_literal", - /* 529 */ "query_specification", - /* 530 */ "hint_list", - /* 531 */ "set_quantifier_opt", - /* 532 */ "tag_mode_opt", - /* 533 */ "select_list", - /* 534 */ "partition_by_clause_opt", - /* 535 */ "range_opt", - /* 536 */ "every_opt", - /* 537 */ "fill_opt", - /* 538 */ "twindow_clause_opt", - /* 539 */ "group_by_clause_opt", - /* 540 */ "having_clause_opt", - /* 541 */ "select_item", - /* 542 */ "partition_list", - /* 543 */ "partition_item", - /* 544 */ "interval_sliding_duration_literal", - /* 545 */ "fill_mode", - /* 546 */ "group_by_list", - /* 547 */ "query_expression", - /* 548 */ "query_simple", - /* 549 */ "order_by_clause_opt", - /* 550 */ "slimit_clause_opt", - /* 551 */ "limit_clause_opt", - /* 552 */ "union_query_expression", - /* 553 */ "query_simple_or_subquery", - /* 554 */ "sort_specification_list", - /* 555 */ "sort_specification", - /* 556 */ "ordering_specification_opt", - /* 557 */ "null_ordering_opt", + /* 233 */ "FORCE_WINDOW_CLOSE", + /* 234 */ "IGNORE", + /* 235 */ "EXPIRED", + /* 236 */ "FILL_HISTORY", + /* 237 */ "UPDATE", + /* 238 */ "SUBTABLE", + /* 239 */ "UNTREATED", + /* 240 */ "KILL", + /* 241 */ "CONNECTION", + /* 242 */ "TRANSACTION", + /* 243 */ "BALANCE", + /* 244 */ "VGROUP", + /* 245 */ "LEADER", + /* 246 */ "MERGE", + /* 247 */ "REDISTRIBUTE", + /* 248 */ "SPLIT", + /* 249 */ "DELETE", + /* 250 */ "INSERT", + /* 251 */ "NK_BIN", + /* 252 */ "NK_HEX", + /* 253 */ "NULL", + /* 254 */ "NK_QUESTION", + /* 255 */ "NK_ALIAS", + /* 256 */ "NK_ARROW", + /* 257 */ "ROWTS", + /* 258 */ "QSTART", + /* 259 */ "QEND", + /* 260 */ "QDURATION", + /* 261 */ "WSTART", + /* 262 */ "WEND", + /* 263 */ "WDURATION", + /* 264 */ "IROWTS", + /* 265 */ "ISFILLED", + /* 266 */ "CAST", + /* 267 */ "NOW", + /* 268 */ "TODAY", + /* 269 */ "TIMEZONE", + /* 270 */ "CLIENT_VERSION", + /* 271 */ "SERVER_VERSION", + /* 272 */ "SERVER_STATUS", + /* 273 */ "CURRENT_USER", + /* 274 */ "CASE", + /* 275 */ "WHEN", + /* 276 */ "THEN", + /* 277 */ "ELSE", + /* 278 */ "BETWEEN", + /* 279 */ "IS", + /* 280 */ "NK_LT", + /* 281 */ "NK_GT", + /* 282 */ "NK_LE", + /* 283 */ "NK_GE", + /* 284 */ "NK_NE", + /* 285 */ "MATCH", + /* 286 */ "NMATCH", + /* 287 */ "CONTAINS", + /* 288 */ "IN", + /* 289 */ "JOIN", + /* 290 */ "INNER", + /* 291 */ "LEFT", + /* 292 */ "RIGHT", + /* 293 */ "OUTER", + /* 294 */ "SEMI", + /* 295 */ "ANTI", + /* 296 */ "ASOF", + /* 297 */ "WINDOW", + /* 298 */ "WINDOW_OFFSET", + /* 299 */ "JLIMIT", + /* 300 */ "SELECT", + /* 301 */ "NK_HINT", + /* 302 */ "DISTINCT", + /* 303 */ "WHERE", + /* 304 */ "PARTITION", + /* 305 */ "BY", + /* 306 */ "SESSION", + /* 307 */ "STATE_WINDOW", + /* 308 */ "EVENT_WINDOW", + /* 309 */ "COUNT_WINDOW", + /* 310 */ "SLIDING", + /* 311 */ "FILL", + /* 312 */ "VALUE", + /* 313 */ "VALUE_F", + /* 314 */ "NONE", + /* 315 */ "PREV", + /* 316 */ "NULL_F", + /* 317 */ "LINEAR", + /* 318 */ "NEXT", + /* 319 */ "HAVING", + /* 320 */ "RANGE", + /* 321 */ "EVERY", + /* 322 */ "ORDER", + /* 323 */ "SLIMIT", + /* 324 */ "SOFFSET", + /* 325 */ "LIMIT", + /* 326 */ "OFFSET", + /* 327 */ "ASC", + /* 328 */ "NULLS", + /* 329 */ "ABORT", + /* 330 */ "AFTER", + /* 331 */ "ATTACH", + /* 332 */ "BEFORE", + /* 333 */ "BEGIN", + /* 334 */ "BITAND", + /* 335 */ "BITNOT", + /* 336 */ "BITOR", + /* 337 */ "BLOCKS", + /* 338 */ "CHANGE", + /* 339 */ "COMMA", + /* 340 */ "CONCAT", + /* 341 */ "CONFLICT", + /* 342 */ "COPY", + /* 343 */ "DEFERRED", + /* 344 */ "DELIMITERS", + /* 345 */ "DETACH", + /* 346 */ "DIVIDE", + /* 347 */ "DOT", + /* 348 */ "EACH", + /* 349 */ "FAIL", + /* 350 */ "FOR", + /* 351 */ "GLOB", + /* 352 */ "ID", + /* 353 */ "IMMEDIATE", + /* 354 */ "IMPORT", + /* 355 */ "INITIALLY", + /* 356 */ "INSTEAD", + /* 357 */ "ISNULL", + /* 358 */ "MODULES", + /* 359 */ "NK_BITNOT", + /* 360 */ "NK_SEMI", + /* 361 */ "NOTNULL", + /* 362 */ "OF", + /* 363 */ "PLUS", + /* 364 */ "PRIVILEGE", + /* 365 */ "RAISE", + /* 366 */ "RESTRICT", + /* 367 */ "ROW", + /* 368 */ "STAR", + /* 369 */ "STATEMENT", + /* 370 */ "STRICT", + /* 371 */ "STRING", + /* 372 */ "TIMES", + /* 373 */ "VALUES", + /* 374 */ "VARIABLE", + /* 375 */ "WAL", + /* 376 */ "ENCODE", + /* 377 */ "COMPRESS", + /* 378 */ "LEVEL", + /* 379 */ "cmd", + /* 380 */ "account_options", + /* 381 */ "alter_account_options", + /* 382 */ "literal", + /* 383 */ "alter_account_option", + /* 384 */ "ip_range_list", + /* 385 */ "white_list", + /* 386 */ "white_list_opt", + /* 387 */ "is_import_opt", + /* 388 */ "is_createdb_opt", + /* 389 */ "user_name", + /* 390 */ "sysinfo_opt", + /* 391 */ "privileges", + /* 392 */ "priv_level", + /* 393 */ "with_opt", + /* 394 */ "priv_type_list", + /* 395 */ "priv_type", + /* 396 */ "db_name", + /* 397 */ "table_name", + /* 398 */ "topic_name", + /* 399 */ "search_condition", + /* 400 */ "dnode_endpoint", + /* 401 */ "force_opt", + /* 402 */ "unsafe_opt", + /* 403 */ "not_exists_opt", + /* 404 */ "db_options", + /* 405 */ "exists_opt", + /* 406 */ "alter_db_options", + /* 407 */ "speed_opt", + /* 408 */ "start_opt", + /* 409 */ "end_opt", + /* 410 */ "integer_list", + /* 411 */ "variable_list", + /* 412 */ "retention_list", + /* 413 */ "signed", + /* 414 */ "alter_db_option", + /* 415 */ "retention", + /* 416 */ "full_table_name", + /* 417 */ "column_def_list", + /* 418 */ "tags_def_opt", + /* 419 */ "table_options", + /* 420 */ "multi_create_clause", + /* 421 */ "tag_list_opt", + /* 422 */ "tags_def", + /* 423 */ "multi_drop_clause", + /* 424 */ "alter_table_clause", + /* 425 */ "alter_table_options", + /* 426 */ "column_name", + /* 427 */ "type_name", + /* 428 */ "column_options", + /* 429 */ "tags_literal", + /* 430 */ "create_subtable_clause", + /* 431 */ "specific_cols_opt", + /* 432 */ "tags_literal_list", + /* 433 */ "drop_table_clause", + /* 434 */ "col_name_list", + /* 435 */ "tag_def_list", + /* 436 */ "tag_def", + /* 437 */ "column_def", + /* 438 */ "type_name_default_len", + /* 439 */ "duration_list", + /* 440 */ "rollup_func_list", + /* 441 */ "alter_table_option", + /* 442 */ "duration_literal", + /* 443 */ "rollup_func_name", + /* 444 */ "function_name", + /* 445 */ "col_name", + /* 446 */ "db_kind_opt", + /* 447 */ "table_kind_db_name_cond_opt", + /* 448 */ "like_pattern_opt", + /* 449 */ "db_name_cond_opt", + /* 450 */ "table_name_cond", + /* 451 */ "from_db_opt", + /* 452 */ "table_kind", + /* 453 */ "tag_item", + /* 454 */ "column_alias", + /* 455 */ "tsma_name", + /* 456 */ "tsma_func_list", + /* 457 */ "full_tsma_name", + /* 458 */ "func_list", + /* 459 */ "index_options", + /* 460 */ "full_index_name", + /* 461 */ "index_name", + /* 462 */ "sliding_opt", + /* 463 */ "sma_stream_opt", + /* 464 */ "func", + /* 465 */ "sma_func_name", + /* 466 */ "expression_list", + /* 467 */ "with_meta", + /* 468 */ "query_or_subquery", + /* 469 */ "where_clause_opt", + /* 470 */ "cgroup_name", + /* 471 */ "analyze_opt", + /* 472 */ "explain_options", + /* 473 */ "insert_query", + /* 474 */ "or_replace_opt", + /* 475 */ "agg_func_opt", + /* 476 */ "bufsize_opt", + /* 477 */ "language_opt", + /* 478 */ "full_view_name", + /* 479 */ "view_name", + /* 480 */ "stream_name", + /* 481 */ "stream_options", + /* 482 */ "col_list_opt", + /* 483 */ "tag_def_or_ref_opt", + /* 484 */ "subtable_opt", + /* 485 */ "ignore_opt", + /* 486 */ "column_stream_def_list", + /* 487 */ "column_stream_def", + /* 488 */ "stream_col_options", + /* 489 */ "expression", + /* 490 */ "on_vgroup_id", + /* 491 */ "dnode_list", + /* 492 */ "literal_func", + /* 493 */ "signed_literal", + /* 494 */ "literal_list", + /* 495 */ "table_alias", + /* 496 */ "expr_or_subquery", + /* 497 */ "pseudo_column", + /* 498 */ "column_reference", + /* 499 */ "function_expression", + /* 500 */ "case_when_expression", + /* 501 */ "star_func", + /* 502 */ "star_func_para_list", + /* 503 */ "noarg_func", + /* 504 */ "other_para_list", + /* 505 */ "star_func_para", + /* 506 */ "when_then_list", + /* 507 */ "case_when_else_opt", + /* 508 */ "common_expression", + /* 509 */ "when_then_expr", + /* 510 */ "predicate", + /* 511 */ "compare_op", + /* 512 */ "in_op", + /* 513 */ "in_predicate_value", + /* 514 */ "boolean_value_expression", + /* 515 */ "boolean_primary", + /* 516 */ "from_clause_opt", + /* 517 */ "table_reference_list", + /* 518 */ "table_reference", + /* 519 */ "table_primary", + /* 520 */ "joined_table", + /* 521 */ "alias_opt", + /* 522 */ "subquery", + /* 523 */ "parenthesized_joined_table", + /* 524 */ "join_type", + /* 525 */ "join_subtype", + /* 526 */ "join_on_clause_opt", + /* 527 */ "window_offset_clause_opt", + /* 528 */ "jlimit_clause_opt", + /* 529 */ "window_offset_literal", + /* 530 */ "query_specification", + /* 531 */ "hint_list", + /* 532 */ "set_quantifier_opt", + /* 533 */ "tag_mode_opt", + /* 534 */ "select_list", + /* 535 */ "partition_by_clause_opt", + /* 536 */ "range_opt", + /* 537 */ "every_opt", + /* 538 */ "fill_opt", + /* 539 */ "twindow_clause_opt", + /* 540 */ "group_by_clause_opt", + /* 541 */ "having_clause_opt", + /* 542 */ "select_item", + /* 543 */ "partition_list", + /* 544 */ "partition_item", + /* 545 */ "interval_sliding_duration_literal", + /* 546 */ "fill_mode", + /* 547 */ "group_by_list", + /* 548 */ "query_expression", + /* 549 */ "query_simple", + /* 550 */ "order_by_clause_opt", + /* 551 */ "slimit_clause_opt", + /* 552 */ "limit_clause_opt", + /* 553 */ "union_query_expression", + /* 554 */ "query_simple_or_subquery", + /* 555 */ "sort_specification_list", + /* 556 */ "sort_specification", + /* 557 */ "ordering_specification_opt", + /* 558 */ "null_ordering_opt", }; #endif /* defined(YYCOVERAGE) || !defined(NDEBUG) */ @@ -2965,340 +3001,341 @@ static const char *const yyRuleName[] = { /* 418 */ "stream_options ::=", /* 419 */ "stream_options ::= stream_options TRIGGER AT_ONCE", /* 420 */ "stream_options ::= stream_options TRIGGER WINDOW_CLOSE", - /* 421 */ "stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal", - /* 422 */ "stream_options ::= stream_options WATERMARK duration_literal", - /* 423 */ "stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER", - /* 424 */ "stream_options ::= stream_options FILL_HISTORY NK_INTEGER", - /* 425 */ "stream_options ::= stream_options DELETE_MARK duration_literal", - /* 426 */ "stream_options ::= stream_options IGNORE UPDATE NK_INTEGER", - /* 427 */ "subtable_opt ::=", - /* 428 */ "subtable_opt ::= SUBTABLE NK_LP expression NK_RP", - /* 429 */ "ignore_opt ::=", - /* 430 */ "ignore_opt ::= IGNORE UNTREATED", - /* 431 */ "cmd ::= KILL CONNECTION NK_INTEGER", - /* 432 */ "cmd ::= KILL QUERY NK_STRING", - /* 433 */ "cmd ::= KILL TRANSACTION NK_INTEGER", - /* 434 */ "cmd ::= KILL COMPACT NK_INTEGER", - /* 435 */ "cmd ::= BALANCE VGROUP", - /* 436 */ "cmd ::= BALANCE VGROUP LEADER on_vgroup_id", - /* 437 */ "cmd ::= BALANCE VGROUP LEADER DATABASE db_name", - /* 438 */ "cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER", - /* 439 */ "cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list", - /* 440 */ "cmd ::= SPLIT VGROUP NK_INTEGER", - /* 441 */ "on_vgroup_id ::=", - /* 442 */ "on_vgroup_id ::= ON NK_INTEGER", - /* 443 */ "dnode_list ::= DNODE NK_INTEGER", - /* 444 */ "dnode_list ::= dnode_list DNODE NK_INTEGER", - /* 445 */ "cmd ::= DELETE FROM full_table_name where_clause_opt", - /* 446 */ "cmd ::= query_or_subquery", - /* 447 */ "cmd ::= insert_query", - /* 448 */ "insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery", - /* 449 */ "insert_query ::= INSERT INTO full_table_name query_or_subquery", - /* 450 */ "tags_literal ::= NK_INTEGER", - /* 451 */ "tags_literal ::= NK_INTEGER NK_PLUS duration_literal", - /* 452 */ "tags_literal ::= NK_INTEGER NK_MINUS duration_literal", - /* 453 */ "tags_literal ::= NK_PLUS NK_INTEGER", - /* 454 */ "tags_literal ::= NK_PLUS NK_INTEGER NK_PLUS duration_literal", - /* 455 */ "tags_literal ::= NK_PLUS NK_INTEGER NK_MINUS duration_literal", - /* 456 */ "tags_literal ::= NK_MINUS NK_INTEGER", - /* 457 */ "tags_literal ::= NK_MINUS NK_INTEGER NK_PLUS duration_literal", - /* 458 */ "tags_literal ::= NK_MINUS NK_INTEGER NK_MINUS duration_literal", - /* 459 */ "tags_literal ::= NK_FLOAT", - /* 460 */ "tags_literal ::= NK_PLUS NK_FLOAT", - /* 461 */ "tags_literal ::= NK_MINUS NK_FLOAT", - /* 462 */ "tags_literal ::= NK_BIN", - /* 463 */ "tags_literal ::= NK_BIN NK_PLUS duration_literal", - /* 464 */ "tags_literal ::= NK_BIN NK_MINUS duration_literal", - /* 465 */ "tags_literal ::= NK_PLUS NK_BIN", - /* 466 */ "tags_literal ::= NK_PLUS NK_BIN NK_PLUS duration_literal", - /* 467 */ "tags_literal ::= NK_PLUS NK_BIN NK_MINUS duration_literal", - /* 468 */ "tags_literal ::= NK_MINUS NK_BIN", - /* 469 */ "tags_literal ::= NK_MINUS NK_BIN NK_PLUS duration_literal", - /* 470 */ "tags_literal ::= NK_MINUS NK_BIN NK_MINUS duration_literal", - /* 471 */ "tags_literal ::= NK_HEX", - /* 472 */ "tags_literal ::= NK_HEX NK_PLUS duration_literal", - /* 473 */ "tags_literal ::= NK_HEX NK_MINUS duration_literal", - /* 474 */ "tags_literal ::= NK_PLUS NK_HEX", - /* 475 */ "tags_literal ::= NK_PLUS NK_HEX NK_PLUS duration_literal", - /* 476 */ "tags_literal ::= NK_PLUS NK_HEX NK_MINUS duration_literal", - /* 477 */ "tags_literal ::= NK_MINUS NK_HEX", - /* 478 */ "tags_literal ::= NK_MINUS NK_HEX NK_PLUS duration_literal", - /* 479 */ "tags_literal ::= NK_MINUS NK_HEX NK_MINUS duration_literal", - /* 480 */ "tags_literal ::= NK_STRING", - /* 481 */ "tags_literal ::= NK_STRING NK_PLUS duration_literal", - /* 482 */ "tags_literal ::= NK_STRING NK_MINUS duration_literal", - /* 483 */ "tags_literal ::= NK_BOOL", - /* 484 */ "tags_literal ::= NULL", - /* 485 */ "tags_literal ::= literal_func", - /* 486 */ "tags_literal ::= literal_func NK_PLUS duration_literal", - /* 487 */ "tags_literal ::= literal_func NK_MINUS duration_literal", - /* 488 */ "tags_literal_list ::= tags_literal", - /* 489 */ "tags_literal_list ::= tags_literal_list NK_COMMA tags_literal", - /* 490 */ "literal ::= NK_INTEGER", - /* 491 */ "literal ::= NK_FLOAT", - /* 492 */ "literal ::= NK_STRING", - /* 493 */ "literal ::= NK_BOOL", - /* 494 */ "literal ::= TIMESTAMP NK_STRING", - /* 495 */ "literal ::= duration_literal", - /* 496 */ "literal ::= NULL", - /* 497 */ "literal ::= NK_QUESTION", - /* 498 */ "duration_literal ::= NK_VARIABLE", - /* 499 */ "signed ::= NK_INTEGER", - /* 500 */ "signed ::= NK_PLUS NK_INTEGER", - /* 501 */ "signed ::= NK_MINUS NK_INTEGER", - /* 502 */ "signed ::= NK_FLOAT", - /* 503 */ "signed ::= NK_PLUS NK_FLOAT", - /* 504 */ "signed ::= NK_MINUS NK_FLOAT", - /* 505 */ "signed_literal ::= signed", - /* 506 */ "signed_literal ::= NK_STRING", - /* 507 */ "signed_literal ::= NK_BOOL", - /* 508 */ "signed_literal ::= TIMESTAMP NK_STRING", - /* 509 */ "signed_literal ::= duration_literal", - /* 510 */ "signed_literal ::= NULL", - /* 511 */ "signed_literal ::= literal_func", - /* 512 */ "signed_literal ::= NK_QUESTION", - /* 513 */ "literal_list ::= signed_literal", - /* 514 */ "literal_list ::= literal_list NK_COMMA signed_literal", - /* 515 */ "db_name ::= NK_ID", - /* 516 */ "table_name ::= NK_ID", - /* 517 */ "column_name ::= NK_ID", - /* 518 */ "function_name ::= NK_ID", - /* 519 */ "view_name ::= NK_ID", - /* 520 */ "table_alias ::= NK_ID", - /* 521 */ "column_alias ::= NK_ID", - /* 522 */ "column_alias ::= NK_ALIAS", - /* 523 */ "user_name ::= NK_ID", - /* 524 */ "topic_name ::= NK_ID", - /* 525 */ "stream_name ::= NK_ID", - /* 526 */ "cgroup_name ::= NK_ID", - /* 527 */ "index_name ::= NK_ID", - /* 528 */ "tsma_name ::= NK_ID", - /* 529 */ "expr_or_subquery ::= expression", - /* 530 */ "expression ::= literal", - /* 531 */ "expression ::= pseudo_column", - /* 532 */ "expression ::= column_reference", - /* 533 */ "expression ::= function_expression", - /* 534 */ "expression ::= case_when_expression", - /* 535 */ "expression ::= NK_LP expression NK_RP", - /* 536 */ "expression ::= NK_PLUS expr_or_subquery", - /* 537 */ "expression ::= NK_MINUS expr_or_subquery", - /* 538 */ "expression ::= expr_or_subquery NK_PLUS expr_or_subquery", - /* 539 */ "expression ::= expr_or_subquery NK_MINUS expr_or_subquery", - /* 540 */ "expression ::= expr_or_subquery NK_STAR expr_or_subquery", - /* 541 */ "expression ::= expr_or_subquery NK_SLASH expr_or_subquery", - /* 542 */ "expression ::= expr_or_subquery NK_REM expr_or_subquery", - /* 543 */ "expression ::= column_reference NK_ARROW NK_STRING", - /* 544 */ "expression ::= expr_or_subquery NK_BITAND expr_or_subquery", - /* 545 */ "expression ::= expr_or_subquery NK_BITOR expr_or_subquery", - /* 546 */ "expression_list ::= expr_or_subquery", - /* 547 */ "expression_list ::= expression_list NK_COMMA expr_or_subquery", - /* 548 */ "column_reference ::= column_name", - /* 549 */ "column_reference ::= table_name NK_DOT column_name", - /* 550 */ "column_reference ::= NK_ALIAS", - /* 551 */ "column_reference ::= table_name NK_DOT NK_ALIAS", - /* 552 */ "pseudo_column ::= ROWTS", - /* 553 */ "pseudo_column ::= TBNAME", - /* 554 */ "pseudo_column ::= table_name NK_DOT TBNAME", - /* 555 */ "pseudo_column ::= QSTART", - /* 556 */ "pseudo_column ::= QEND", - /* 557 */ "pseudo_column ::= QDURATION", - /* 558 */ "pseudo_column ::= WSTART", - /* 559 */ "pseudo_column ::= WEND", - /* 560 */ "pseudo_column ::= WDURATION", - /* 561 */ "pseudo_column ::= IROWTS", - /* 562 */ "pseudo_column ::= ISFILLED", - /* 563 */ "pseudo_column ::= QTAGS", - /* 564 */ "function_expression ::= function_name NK_LP expression_list NK_RP", - /* 565 */ "function_expression ::= star_func NK_LP star_func_para_list NK_RP", - /* 566 */ "function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP", - /* 567 */ "function_expression ::= CAST NK_LP expr_or_subquery AS type_name_default_len NK_RP", - /* 568 */ "function_expression ::= literal_func", - /* 569 */ "literal_func ::= noarg_func NK_LP NK_RP", - /* 570 */ "literal_func ::= NOW", - /* 571 */ "literal_func ::= TODAY", - /* 572 */ "noarg_func ::= NOW", - /* 573 */ "noarg_func ::= TODAY", - /* 574 */ "noarg_func ::= TIMEZONE", - /* 575 */ "noarg_func ::= DATABASE", - /* 576 */ "noarg_func ::= CLIENT_VERSION", - /* 577 */ "noarg_func ::= SERVER_VERSION", - /* 578 */ "noarg_func ::= SERVER_STATUS", - /* 579 */ "noarg_func ::= CURRENT_USER", - /* 580 */ "noarg_func ::= USER", - /* 581 */ "star_func ::= COUNT", - /* 582 */ "star_func ::= FIRST", - /* 583 */ "star_func ::= LAST", - /* 584 */ "star_func ::= LAST_ROW", - /* 585 */ "star_func_para_list ::= NK_STAR", - /* 586 */ "star_func_para_list ::= other_para_list", - /* 587 */ "other_para_list ::= star_func_para", - /* 588 */ "other_para_list ::= other_para_list NK_COMMA star_func_para", - /* 589 */ "star_func_para ::= expr_or_subquery", - /* 590 */ "star_func_para ::= table_name NK_DOT NK_STAR", - /* 591 */ "case_when_expression ::= CASE when_then_list case_when_else_opt END", - /* 592 */ "case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END", - /* 593 */ "when_then_list ::= when_then_expr", - /* 594 */ "when_then_list ::= when_then_list when_then_expr", - /* 595 */ "when_then_expr ::= WHEN common_expression THEN common_expression", - /* 596 */ "case_when_else_opt ::=", - /* 597 */ "case_when_else_opt ::= ELSE common_expression", - /* 598 */ "predicate ::= expr_or_subquery compare_op expr_or_subquery", - /* 599 */ "predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery", - /* 600 */ "predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery", - /* 601 */ "predicate ::= expr_or_subquery IS NULL", - /* 602 */ "predicate ::= expr_or_subquery IS NOT NULL", - /* 603 */ "predicate ::= expr_or_subquery in_op in_predicate_value", - /* 604 */ "compare_op ::= NK_LT", - /* 605 */ "compare_op ::= NK_GT", - /* 606 */ "compare_op ::= NK_LE", - /* 607 */ "compare_op ::= NK_GE", - /* 608 */ "compare_op ::= NK_NE", - /* 609 */ "compare_op ::= NK_EQ", - /* 610 */ "compare_op ::= LIKE", - /* 611 */ "compare_op ::= NOT LIKE", - /* 612 */ "compare_op ::= MATCH", - /* 613 */ "compare_op ::= NMATCH", - /* 614 */ "compare_op ::= CONTAINS", - /* 615 */ "in_op ::= IN", - /* 616 */ "in_op ::= NOT IN", - /* 617 */ "in_predicate_value ::= NK_LP literal_list NK_RP", - /* 618 */ "boolean_value_expression ::= boolean_primary", - /* 619 */ "boolean_value_expression ::= NOT boolean_primary", - /* 620 */ "boolean_value_expression ::= boolean_value_expression OR boolean_value_expression", - /* 621 */ "boolean_value_expression ::= boolean_value_expression AND boolean_value_expression", - /* 622 */ "boolean_primary ::= predicate", - /* 623 */ "boolean_primary ::= NK_LP boolean_value_expression NK_RP", - /* 624 */ "common_expression ::= expr_or_subquery", - /* 625 */ "common_expression ::= boolean_value_expression", - /* 626 */ "from_clause_opt ::=", - /* 627 */ "from_clause_opt ::= FROM table_reference_list", - /* 628 */ "table_reference_list ::= table_reference", - /* 629 */ "table_reference_list ::= table_reference_list NK_COMMA table_reference", - /* 630 */ "table_reference ::= table_primary", - /* 631 */ "table_reference ::= joined_table", - /* 632 */ "table_primary ::= table_name alias_opt", - /* 633 */ "table_primary ::= db_name NK_DOT table_name alias_opt", - /* 634 */ "table_primary ::= subquery alias_opt", - /* 635 */ "table_primary ::= parenthesized_joined_table", - /* 636 */ "alias_opt ::=", - /* 637 */ "alias_opt ::= table_alias", - /* 638 */ "alias_opt ::= AS table_alias", - /* 639 */ "parenthesized_joined_table ::= NK_LP joined_table NK_RP", - /* 640 */ "parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP", - /* 641 */ "joined_table ::= table_reference join_type join_subtype JOIN table_reference join_on_clause_opt window_offset_clause_opt jlimit_clause_opt", - /* 642 */ "join_type ::=", - /* 643 */ "join_type ::= INNER", - /* 644 */ "join_type ::= LEFT", - /* 645 */ "join_type ::= RIGHT", - /* 646 */ "join_type ::= FULL", - /* 647 */ "join_subtype ::=", - /* 648 */ "join_subtype ::= OUTER", - /* 649 */ "join_subtype ::= SEMI", - /* 650 */ "join_subtype ::= ANTI", - /* 651 */ "join_subtype ::= ASOF", - /* 652 */ "join_subtype ::= WINDOW", - /* 653 */ "join_on_clause_opt ::=", - /* 654 */ "join_on_clause_opt ::= ON search_condition", - /* 655 */ "window_offset_clause_opt ::=", - /* 656 */ "window_offset_clause_opt ::= WINDOW_OFFSET NK_LP window_offset_literal NK_COMMA window_offset_literal NK_RP", - /* 657 */ "window_offset_literal ::= NK_VARIABLE", - /* 658 */ "window_offset_literal ::= NK_MINUS NK_VARIABLE", - /* 659 */ "jlimit_clause_opt ::=", - /* 660 */ "jlimit_clause_opt ::= JLIMIT NK_INTEGER", - /* 661 */ "query_specification ::= SELECT hint_list set_quantifier_opt tag_mode_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", - /* 662 */ "hint_list ::=", - /* 663 */ "hint_list ::= NK_HINT", - /* 664 */ "tag_mode_opt ::=", - /* 665 */ "tag_mode_opt ::= TAGS", - /* 666 */ "set_quantifier_opt ::=", - /* 667 */ "set_quantifier_opt ::= DISTINCT", - /* 668 */ "set_quantifier_opt ::= ALL", - /* 669 */ "select_list ::= select_item", - /* 670 */ "select_list ::= select_list NK_COMMA select_item", - /* 671 */ "select_item ::= NK_STAR", - /* 672 */ "select_item ::= common_expression", - /* 673 */ "select_item ::= common_expression column_alias", - /* 674 */ "select_item ::= common_expression AS column_alias", - /* 675 */ "select_item ::= table_name NK_DOT NK_STAR", - /* 676 */ "where_clause_opt ::=", - /* 677 */ "where_clause_opt ::= WHERE search_condition", - /* 678 */ "partition_by_clause_opt ::=", - /* 679 */ "partition_by_clause_opt ::= PARTITION BY partition_list", - /* 680 */ "partition_list ::= partition_item", - /* 681 */ "partition_list ::= partition_list NK_COMMA partition_item", - /* 682 */ "partition_item ::= expr_or_subquery", - /* 683 */ "partition_item ::= expr_or_subquery column_alias", - /* 684 */ "partition_item ::= expr_or_subquery AS column_alias", - /* 685 */ "twindow_clause_opt ::=", - /* 686 */ "twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA interval_sliding_duration_literal NK_RP", - /* 687 */ "twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP", - /* 688 */ "twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_RP sliding_opt fill_opt", - /* 689 */ "twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_COMMA interval_sliding_duration_literal NK_RP sliding_opt fill_opt", - /* 690 */ "twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition", - /* 691 */ "twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_RP", - /* 692 */ "twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP", - /* 693 */ "sliding_opt ::=", - /* 694 */ "sliding_opt ::= SLIDING NK_LP interval_sliding_duration_literal NK_RP", - /* 695 */ "interval_sliding_duration_literal ::= NK_VARIABLE", - /* 696 */ "interval_sliding_duration_literal ::= NK_STRING", - /* 697 */ "interval_sliding_duration_literal ::= NK_INTEGER", - /* 698 */ "fill_opt ::=", - /* 699 */ "fill_opt ::= FILL NK_LP fill_mode NK_RP", - /* 700 */ "fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP", - /* 701 */ "fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP", - /* 702 */ "fill_mode ::= NONE", - /* 703 */ "fill_mode ::= PREV", - /* 704 */ "fill_mode ::= NULL", - /* 705 */ "fill_mode ::= NULL_F", - /* 706 */ "fill_mode ::= LINEAR", - /* 707 */ "fill_mode ::= NEXT", - /* 708 */ "group_by_clause_opt ::=", - /* 709 */ "group_by_clause_opt ::= GROUP BY group_by_list", - /* 710 */ "group_by_list ::= expr_or_subquery", - /* 711 */ "group_by_list ::= group_by_list NK_COMMA expr_or_subquery", - /* 712 */ "having_clause_opt ::=", - /* 713 */ "having_clause_opt ::= HAVING search_condition", - /* 714 */ "range_opt ::=", - /* 715 */ "range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP", - /* 716 */ "range_opt ::= RANGE NK_LP expr_or_subquery NK_RP", - /* 717 */ "every_opt ::=", - /* 718 */ "every_opt ::= EVERY NK_LP duration_literal NK_RP", - /* 719 */ "query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt", - /* 720 */ "query_simple ::= query_specification", - /* 721 */ "query_simple ::= union_query_expression", - /* 722 */ "union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery", - /* 723 */ "union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery", - /* 724 */ "query_simple_or_subquery ::= query_simple", - /* 725 */ "query_simple_or_subquery ::= subquery", - /* 726 */ "query_or_subquery ::= query_expression", - /* 727 */ "query_or_subquery ::= subquery", - /* 728 */ "order_by_clause_opt ::=", - /* 729 */ "order_by_clause_opt ::= ORDER BY sort_specification_list", - /* 730 */ "slimit_clause_opt ::=", - /* 731 */ "slimit_clause_opt ::= SLIMIT NK_INTEGER", - /* 732 */ "slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER", - /* 733 */ "slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER", - /* 734 */ "limit_clause_opt ::=", - /* 735 */ "limit_clause_opt ::= LIMIT NK_INTEGER", - /* 736 */ "limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER", - /* 737 */ "limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER", - /* 738 */ "subquery ::= NK_LP query_expression NK_RP", - /* 739 */ "subquery ::= NK_LP subquery NK_RP", - /* 740 */ "search_condition ::= common_expression", - /* 741 */ "sort_specification_list ::= sort_specification", - /* 742 */ "sort_specification_list ::= sort_specification_list NK_COMMA sort_specification", - /* 743 */ "sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt", - /* 744 */ "ordering_specification_opt ::=", - /* 745 */ "ordering_specification_opt ::= ASC", - /* 746 */ "ordering_specification_opt ::= DESC", - /* 747 */ "null_ordering_opt ::=", - /* 748 */ "null_ordering_opt ::= NULLS FIRST", - /* 749 */ "null_ordering_opt ::= NULLS LAST", - /* 750 */ "column_options ::=", - /* 751 */ "column_options ::= column_options PRIMARY KEY", - /* 752 */ "column_options ::= column_options ENCODE NK_STRING", - /* 753 */ "column_options ::= column_options COMPRESS NK_STRING", - /* 754 */ "column_options ::= column_options LEVEL NK_STRING", + /* 421 */ "stream_options ::= stream_options TRIGGER FORCE_WINDOW_CLOSE", + /* 422 */ "stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal", + /* 423 */ "stream_options ::= stream_options WATERMARK duration_literal", + /* 424 */ "stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER", + /* 425 */ "stream_options ::= stream_options FILL_HISTORY NK_INTEGER", + /* 426 */ "stream_options ::= stream_options DELETE_MARK duration_literal", + /* 427 */ "stream_options ::= stream_options IGNORE UPDATE NK_INTEGER", + /* 428 */ "subtable_opt ::=", + /* 429 */ "subtable_opt ::= SUBTABLE NK_LP expression NK_RP", + /* 430 */ "ignore_opt ::=", + /* 431 */ "ignore_opt ::= IGNORE UNTREATED", + /* 432 */ "cmd ::= KILL CONNECTION NK_INTEGER", + /* 433 */ "cmd ::= KILL QUERY NK_STRING", + /* 434 */ "cmd ::= KILL TRANSACTION NK_INTEGER", + /* 435 */ "cmd ::= KILL COMPACT NK_INTEGER", + /* 436 */ "cmd ::= BALANCE VGROUP", + /* 437 */ "cmd ::= BALANCE VGROUP LEADER on_vgroup_id", + /* 438 */ "cmd ::= BALANCE VGROUP LEADER DATABASE db_name", + /* 439 */ "cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER", + /* 440 */ "cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list", + /* 441 */ "cmd ::= SPLIT VGROUP NK_INTEGER", + /* 442 */ "on_vgroup_id ::=", + /* 443 */ "on_vgroup_id ::= ON NK_INTEGER", + /* 444 */ "dnode_list ::= DNODE NK_INTEGER", + /* 445 */ "dnode_list ::= dnode_list DNODE NK_INTEGER", + /* 446 */ "cmd ::= DELETE FROM full_table_name where_clause_opt", + /* 447 */ "cmd ::= query_or_subquery", + /* 448 */ "cmd ::= insert_query", + /* 449 */ "insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery", + /* 450 */ "insert_query ::= INSERT INTO full_table_name query_or_subquery", + /* 451 */ "tags_literal ::= NK_INTEGER", + /* 452 */ "tags_literal ::= NK_INTEGER NK_PLUS duration_literal", + /* 453 */ "tags_literal ::= NK_INTEGER NK_MINUS duration_literal", + /* 454 */ "tags_literal ::= NK_PLUS NK_INTEGER", + /* 455 */ "tags_literal ::= NK_PLUS NK_INTEGER NK_PLUS duration_literal", + /* 456 */ "tags_literal ::= NK_PLUS NK_INTEGER NK_MINUS duration_literal", + /* 457 */ "tags_literal ::= NK_MINUS NK_INTEGER", + /* 458 */ "tags_literal ::= NK_MINUS NK_INTEGER NK_PLUS duration_literal", + /* 459 */ "tags_literal ::= NK_MINUS NK_INTEGER NK_MINUS duration_literal", + /* 460 */ "tags_literal ::= NK_FLOAT", + /* 461 */ "tags_literal ::= NK_PLUS NK_FLOAT", + /* 462 */ "tags_literal ::= NK_MINUS NK_FLOAT", + /* 463 */ "tags_literal ::= NK_BIN", + /* 464 */ "tags_literal ::= NK_BIN NK_PLUS duration_literal", + /* 465 */ "tags_literal ::= NK_BIN NK_MINUS duration_literal", + /* 466 */ "tags_literal ::= NK_PLUS NK_BIN", + /* 467 */ "tags_literal ::= NK_PLUS NK_BIN NK_PLUS duration_literal", + /* 468 */ "tags_literal ::= NK_PLUS NK_BIN NK_MINUS duration_literal", + /* 469 */ "tags_literal ::= NK_MINUS NK_BIN", + /* 470 */ "tags_literal ::= NK_MINUS NK_BIN NK_PLUS duration_literal", + /* 471 */ "tags_literal ::= NK_MINUS NK_BIN NK_MINUS duration_literal", + /* 472 */ "tags_literal ::= NK_HEX", + /* 473 */ "tags_literal ::= NK_HEX NK_PLUS duration_literal", + /* 474 */ "tags_literal ::= NK_HEX NK_MINUS duration_literal", + /* 475 */ "tags_literal ::= NK_PLUS NK_HEX", + /* 476 */ "tags_literal ::= NK_PLUS NK_HEX NK_PLUS duration_literal", + /* 477 */ "tags_literal ::= NK_PLUS NK_HEX NK_MINUS duration_literal", + /* 478 */ "tags_literal ::= NK_MINUS NK_HEX", + /* 479 */ "tags_literal ::= NK_MINUS NK_HEX NK_PLUS duration_literal", + /* 480 */ "tags_literal ::= NK_MINUS NK_HEX NK_MINUS duration_literal", + /* 481 */ "tags_literal ::= NK_STRING", + /* 482 */ "tags_literal ::= NK_STRING NK_PLUS duration_literal", + /* 483 */ "tags_literal ::= NK_STRING NK_MINUS duration_literal", + /* 484 */ "tags_literal ::= NK_BOOL", + /* 485 */ "tags_literal ::= NULL", + /* 486 */ "tags_literal ::= literal_func", + /* 487 */ "tags_literal ::= literal_func NK_PLUS duration_literal", + /* 488 */ "tags_literal ::= literal_func NK_MINUS duration_literal", + /* 489 */ "tags_literal_list ::= tags_literal", + /* 490 */ "tags_literal_list ::= tags_literal_list NK_COMMA tags_literal", + /* 491 */ "literal ::= NK_INTEGER", + /* 492 */ "literal ::= NK_FLOAT", + /* 493 */ "literal ::= NK_STRING", + /* 494 */ "literal ::= NK_BOOL", + /* 495 */ "literal ::= TIMESTAMP NK_STRING", + /* 496 */ "literal ::= duration_literal", + /* 497 */ "literal ::= NULL", + /* 498 */ "literal ::= NK_QUESTION", + /* 499 */ "duration_literal ::= NK_VARIABLE", + /* 500 */ "signed ::= NK_INTEGER", + /* 501 */ "signed ::= NK_PLUS NK_INTEGER", + /* 502 */ "signed ::= NK_MINUS NK_INTEGER", + /* 503 */ "signed ::= NK_FLOAT", + /* 504 */ "signed ::= NK_PLUS NK_FLOAT", + /* 505 */ "signed ::= NK_MINUS NK_FLOAT", + /* 506 */ "signed_literal ::= signed", + /* 507 */ "signed_literal ::= NK_STRING", + /* 508 */ "signed_literal ::= NK_BOOL", + /* 509 */ "signed_literal ::= TIMESTAMP NK_STRING", + /* 510 */ "signed_literal ::= duration_literal", + /* 511 */ "signed_literal ::= NULL", + /* 512 */ "signed_literal ::= literal_func", + /* 513 */ "signed_literal ::= NK_QUESTION", + /* 514 */ "literal_list ::= signed_literal", + /* 515 */ "literal_list ::= literal_list NK_COMMA signed_literal", + /* 516 */ "db_name ::= NK_ID", + /* 517 */ "table_name ::= NK_ID", + /* 518 */ "column_name ::= NK_ID", + /* 519 */ "function_name ::= NK_ID", + /* 520 */ "view_name ::= NK_ID", + /* 521 */ "table_alias ::= NK_ID", + /* 522 */ "column_alias ::= NK_ID", + /* 523 */ "column_alias ::= NK_ALIAS", + /* 524 */ "user_name ::= NK_ID", + /* 525 */ "topic_name ::= NK_ID", + /* 526 */ "stream_name ::= NK_ID", + /* 527 */ "cgroup_name ::= NK_ID", + /* 528 */ "index_name ::= NK_ID", + /* 529 */ "tsma_name ::= NK_ID", + /* 530 */ "expr_or_subquery ::= expression", + /* 531 */ "expression ::= literal", + /* 532 */ "expression ::= pseudo_column", + /* 533 */ "expression ::= column_reference", + /* 534 */ "expression ::= function_expression", + /* 535 */ "expression ::= case_when_expression", + /* 536 */ "expression ::= NK_LP expression NK_RP", + /* 537 */ "expression ::= NK_PLUS expr_or_subquery", + /* 538 */ "expression ::= NK_MINUS expr_or_subquery", + /* 539 */ "expression ::= expr_or_subquery NK_PLUS expr_or_subquery", + /* 540 */ "expression ::= expr_or_subquery NK_MINUS expr_or_subquery", + /* 541 */ "expression ::= expr_or_subquery NK_STAR expr_or_subquery", + /* 542 */ "expression ::= expr_or_subquery NK_SLASH expr_or_subquery", + /* 543 */ "expression ::= expr_or_subquery NK_REM expr_or_subquery", + /* 544 */ "expression ::= column_reference NK_ARROW NK_STRING", + /* 545 */ "expression ::= expr_or_subquery NK_BITAND expr_or_subquery", + /* 546 */ "expression ::= expr_or_subquery NK_BITOR expr_or_subquery", + /* 547 */ "expression_list ::= expr_or_subquery", + /* 548 */ "expression_list ::= expression_list NK_COMMA expr_or_subquery", + /* 549 */ "column_reference ::= column_name", + /* 550 */ "column_reference ::= table_name NK_DOT column_name", + /* 551 */ "column_reference ::= NK_ALIAS", + /* 552 */ "column_reference ::= table_name NK_DOT NK_ALIAS", + /* 553 */ "pseudo_column ::= ROWTS", + /* 554 */ "pseudo_column ::= TBNAME", + /* 555 */ "pseudo_column ::= table_name NK_DOT TBNAME", + /* 556 */ "pseudo_column ::= QSTART", + /* 557 */ "pseudo_column ::= QEND", + /* 558 */ "pseudo_column ::= QDURATION", + /* 559 */ "pseudo_column ::= WSTART", + /* 560 */ "pseudo_column ::= WEND", + /* 561 */ "pseudo_column ::= WDURATION", + /* 562 */ "pseudo_column ::= IROWTS", + /* 563 */ "pseudo_column ::= ISFILLED", + /* 564 */ "pseudo_column ::= QTAGS", + /* 565 */ "function_expression ::= function_name NK_LP expression_list NK_RP", + /* 566 */ "function_expression ::= star_func NK_LP star_func_para_list NK_RP", + /* 567 */ "function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP", + /* 568 */ "function_expression ::= CAST NK_LP expr_or_subquery AS type_name_default_len NK_RP", + /* 569 */ "function_expression ::= literal_func", + /* 570 */ "literal_func ::= noarg_func NK_LP NK_RP", + /* 571 */ "literal_func ::= NOW", + /* 572 */ "literal_func ::= TODAY", + /* 573 */ "noarg_func ::= NOW", + /* 574 */ "noarg_func ::= TODAY", + /* 575 */ "noarg_func ::= TIMEZONE", + /* 576 */ "noarg_func ::= DATABASE", + /* 577 */ "noarg_func ::= CLIENT_VERSION", + /* 578 */ "noarg_func ::= SERVER_VERSION", + /* 579 */ "noarg_func ::= SERVER_STATUS", + /* 580 */ "noarg_func ::= CURRENT_USER", + /* 581 */ "noarg_func ::= USER", + /* 582 */ "star_func ::= COUNT", + /* 583 */ "star_func ::= FIRST", + /* 584 */ "star_func ::= LAST", + /* 585 */ "star_func ::= LAST_ROW", + /* 586 */ "star_func_para_list ::= NK_STAR", + /* 587 */ "star_func_para_list ::= other_para_list", + /* 588 */ "other_para_list ::= star_func_para", + /* 589 */ "other_para_list ::= other_para_list NK_COMMA star_func_para", + /* 590 */ "star_func_para ::= expr_or_subquery", + /* 591 */ "star_func_para ::= table_name NK_DOT NK_STAR", + /* 592 */ "case_when_expression ::= CASE when_then_list case_when_else_opt END", + /* 593 */ "case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END", + /* 594 */ "when_then_list ::= when_then_expr", + /* 595 */ "when_then_list ::= when_then_list when_then_expr", + /* 596 */ "when_then_expr ::= WHEN common_expression THEN common_expression", + /* 597 */ "case_when_else_opt ::=", + /* 598 */ "case_when_else_opt ::= ELSE common_expression", + /* 599 */ "predicate ::= expr_or_subquery compare_op expr_or_subquery", + /* 600 */ "predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery", + /* 601 */ "predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery", + /* 602 */ "predicate ::= expr_or_subquery IS NULL", + /* 603 */ "predicate ::= expr_or_subquery IS NOT NULL", + /* 604 */ "predicate ::= expr_or_subquery in_op in_predicate_value", + /* 605 */ "compare_op ::= NK_LT", + /* 606 */ "compare_op ::= NK_GT", + /* 607 */ "compare_op ::= NK_LE", + /* 608 */ "compare_op ::= NK_GE", + /* 609 */ "compare_op ::= NK_NE", + /* 610 */ "compare_op ::= NK_EQ", + /* 611 */ "compare_op ::= LIKE", + /* 612 */ "compare_op ::= NOT LIKE", + /* 613 */ "compare_op ::= MATCH", + /* 614 */ "compare_op ::= NMATCH", + /* 615 */ "compare_op ::= CONTAINS", + /* 616 */ "in_op ::= IN", + /* 617 */ "in_op ::= NOT IN", + /* 618 */ "in_predicate_value ::= NK_LP literal_list NK_RP", + /* 619 */ "boolean_value_expression ::= boolean_primary", + /* 620 */ "boolean_value_expression ::= NOT boolean_primary", + /* 621 */ "boolean_value_expression ::= boolean_value_expression OR boolean_value_expression", + /* 622 */ "boolean_value_expression ::= boolean_value_expression AND boolean_value_expression", + /* 623 */ "boolean_primary ::= predicate", + /* 624 */ "boolean_primary ::= NK_LP boolean_value_expression NK_RP", + /* 625 */ "common_expression ::= expr_or_subquery", + /* 626 */ "common_expression ::= boolean_value_expression", + /* 627 */ "from_clause_opt ::=", + /* 628 */ "from_clause_opt ::= FROM table_reference_list", + /* 629 */ "table_reference_list ::= table_reference", + /* 630 */ "table_reference_list ::= table_reference_list NK_COMMA table_reference", + /* 631 */ "table_reference ::= table_primary", + /* 632 */ "table_reference ::= joined_table", + /* 633 */ "table_primary ::= table_name alias_opt", + /* 634 */ "table_primary ::= db_name NK_DOT table_name alias_opt", + /* 635 */ "table_primary ::= subquery alias_opt", + /* 636 */ "table_primary ::= parenthesized_joined_table", + /* 637 */ "alias_opt ::=", + /* 638 */ "alias_opt ::= table_alias", + /* 639 */ "alias_opt ::= AS table_alias", + /* 640 */ "parenthesized_joined_table ::= NK_LP joined_table NK_RP", + /* 641 */ "parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP", + /* 642 */ "joined_table ::= table_reference join_type join_subtype JOIN table_reference join_on_clause_opt window_offset_clause_opt jlimit_clause_opt", + /* 643 */ "join_type ::=", + /* 644 */ "join_type ::= INNER", + /* 645 */ "join_type ::= LEFT", + /* 646 */ "join_type ::= RIGHT", + /* 647 */ "join_type ::= FULL", + /* 648 */ "join_subtype ::=", + /* 649 */ "join_subtype ::= OUTER", + /* 650 */ "join_subtype ::= SEMI", + /* 651 */ "join_subtype ::= ANTI", + /* 652 */ "join_subtype ::= ASOF", + /* 653 */ "join_subtype ::= WINDOW", + /* 654 */ "join_on_clause_opt ::=", + /* 655 */ "join_on_clause_opt ::= ON search_condition", + /* 656 */ "window_offset_clause_opt ::=", + /* 657 */ "window_offset_clause_opt ::= WINDOW_OFFSET NK_LP window_offset_literal NK_COMMA window_offset_literal NK_RP", + /* 658 */ "window_offset_literal ::= NK_VARIABLE", + /* 659 */ "window_offset_literal ::= NK_MINUS NK_VARIABLE", + /* 660 */ "jlimit_clause_opt ::=", + /* 661 */ "jlimit_clause_opt ::= JLIMIT NK_INTEGER", + /* 662 */ "query_specification ::= SELECT hint_list set_quantifier_opt tag_mode_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", + /* 663 */ "hint_list ::=", + /* 664 */ "hint_list ::= NK_HINT", + /* 665 */ "tag_mode_opt ::=", + /* 666 */ "tag_mode_opt ::= TAGS", + /* 667 */ "set_quantifier_opt ::=", + /* 668 */ "set_quantifier_opt ::= DISTINCT", + /* 669 */ "set_quantifier_opt ::= ALL", + /* 670 */ "select_list ::= select_item", + /* 671 */ "select_list ::= select_list NK_COMMA select_item", + /* 672 */ "select_item ::= NK_STAR", + /* 673 */ "select_item ::= common_expression", + /* 674 */ "select_item ::= common_expression column_alias", + /* 675 */ "select_item ::= common_expression AS column_alias", + /* 676 */ "select_item ::= table_name NK_DOT NK_STAR", + /* 677 */ "where_clause_opt ::=", + /* 678 */ "where_clause_opt ::= WHERE search_condition", + /* 679 */ "partition_by_clause_opt ::=", + /* 680 */ "partition_by_clause_opt ::= PARTITION BY partition_list", + /* 681 */ "partition_list ::= partition_item", + /* 682 */ "partition_list ::= partition_list NK_COMMA partition_item", + /* 683 */ "partition_item ::= expr_or_subquery", + /* 684 */ "partition_item ::= expr_or_subquery column_alias", + /* 685 */ "partition_item ::= expr_or_subquery AS column_alias", + /* 686 */ "twindow_clause_opt ::=", + /* 687 */ "twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA interval_sliding_duration_literal NK_RP", + /* 688 */ "twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP", + /* 689 */ "twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_RP sliding_opt fill_opt", + /* 690 */ "twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_COMMA interval_sliding_duration_literal NK_RP sliding_opt fill_opt", + /* 691 */ "twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition", + /* 692 */ "twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_RP", + /* 693 */ "twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP", + /* 694 */ "sliding_opt ::=", + /* 695 */ "sliding_opt ::= SLIDING NK_LP interval_sliding_duration_literal NK_RP", + /* 696 */ "interval_sliding_duration_literal ::= NK_VARIABLE", + /* 697 */ "interval_sliding_duration_literal ::= NK_STRING", + /* 698 */ "interval_sliding_duration_literal ::= NK_INTEGER", + /* 699 */ "fill_opt ::=", + /* 700 */ "fill_opt ::= FILL NK_LP fill_mode NK_RP", + /* 701 */ "fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP", + /* 702 */ "fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP", + /* 703 */ "fill_mode ::= NONE", + /* 704 */ "fill_mode ::= PREV", + /* 705 */ "fill_mode ::= NULL", + /* 706 */ "fill_mode ::= NULL_F", + /* 707 */ "fill_mode ::= LINEAR", + /* 708 */ "fill_mode ::= NEXT", + /* 709 */ "group_by_clause_opt ::=", + /* 710 */ "group_by_clause_opt ::= GROUP BY group_by_list", + /* 711 */ "group_by_list ::= expr_or_subquery", + /* 712 */ "group_by_list ::= group_by_list NK_COMMA expr_or_subquery", + /* 713 */ "having_clause_opt ::=", + /* 714 */ "having_clause_opt ::= HAVING search_condition", + /* 715 */ "range_opt ::=", + /* 716 */ "range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP", + /* 717 */ "range_opt ::= RANGE NK_LP expr_or_subquery NK_RP", + /* 718 */ "every_opt ::=", + /* 719 */ "every_opt ::= EVERY NK_LP duration_literal NK_RP", + /* 720 */ "query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt", + /* 721 */ "query_simple ::= query_specification", + /* 722 */ "query_simple ::= union_query_expression", + /* 723 */ "union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery", + /* 724 */ "union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery", + /* 725 */ "query_simple_or_subquery ::= query_simple", + /* 726 */ "query_simple_or_subquery ::= subquery", + /* 727 */ "query_or_subquery ::= query_expression", + /* 728 */ "query_or_subquery ::= subquery", + /* 729 */ "order_by_clause_opt ::=", + /* 730 */ "order_by_clause_opt ::= ORDER BY sort_specification_list", + /* 731 */ "slimit_clause_opt ::=", + /* 732 */ "slimit_clause_opt ::= SLIMIT NK_INTEGER", + /* 733 */ "slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER", + /* 734 */ "slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER", + /* 735 */ "limit_clause_opt ::=", + /* 736 */ "limit_clause_opt ::= LIMIT NK_INTEGER", + /* 737 */ "limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER", + /* 738 */ "limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER", + /* 739 */ "subquery ::= NK_LP query_expression NK_RP", + /* 740 */ "subquery ::= NK_LP subquery NK_RP", + /* 741 */ "search_condition ::= common_expression", + /* 742 */ "sort_specification_list ::= sort_specification", + /* 743 */ "sort_specification_list ::= sort_specification_list NK_COMMA sort_specification", + /* 744 */ "sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt", + /* 745 */ "ordering_specification_opt ::=", + /* 746 */ "ordering_specification_opt ::= ASC", + /* 747 */ "ordering_specification_opt ::= DESC", + /* 748 */ "null_ordering_opt ::=", + /* 749 */ "null_ordering_opt ::= NULLS FIRST", + /* 750 */ "null_ordering_opt ::= NULLS LAST", + /* 751 */ "column_options ::=", + /* 752 */ "column_options ::= column_options PRIMARY KEY", + /* 753 */ "column_options ::= column_options ENCODE NK_STRING", + /* 754 */ "column_options ::= column_options COMPRESS NK_STRING", + /* 755 */ "column_options ::= column_options LEVEL NK_STRING", }; #endif /* NDEBUG */ @@ -3425,298 +3462,260 @@ static void yy_destructor( */ /********* Begin destructor definitions ***************************************/ /* Default NON-TERMINAL Destructor */ - case 378: /* cmd */ - case 381: /* literal */ - case 392: /* with_opt */ - case 398: /* search_condition */ - case 403: /* db_options */ - case 405: /* alter_db_options */ - case 407: /* start_opt */ - case 408: /* end_opt */ - case 412: /* signed */ - case 414: /* retention */ - case 415: /* full_table_name */ - case 418: /* table_options */ - case 423: /* alter_table_clause */ - case 424: /* alter_table_options */ - case 427: /* column_options */ - case 428: /* tags_literal */ - case 429: /* create_subtable_clause */ - case 432: /* drop_table_clause */ - case 435: /* tag_def */ - case 436: /* column_def */ - case 441: /* duration_literal */ - case 442: /* rollup_func_name */ - case 444: /* col_name */ - case 447: /* like_pattern_opt */ - case 448: /* db_name_cond_opt */ - case 449: /* table_name_cond */ - case 450: /* from_db_opt */ - case 452: /* tag_item */ - case 456: /* full_tsma_name */ - case 458: /* index_options */ - case 459: /* full_index_name */ - case 461: /* sliding_opt */ - case 462: /* sma_stream_opt */ - case 463: /* func */ - case 467: /* query_or_subquery */ - case 468: /* where_clause_opt */ - case 471: /* explain_options */ - case 472: /* insert_query */ - case 477: /* full_view_name */ - case 480: /* stream_options */ - case 483: /* subtable_opt */ - case 486: /* column_stream_def */ - case 487: /* stream_col_options */ - case 488: /* expression */ - case 491: /* literal_func */ - case 492: /* signed_literal */ - case 495: /* expr_or_subquery */ - case 496: /* pseudo_column */ - case 497: /* column_reference */ - case 498: /* function_expression */ - case 499: /* case_when_expression */ - case 504: /* star_func_para */ - case 506: /* case_when_else_opt */ - case 507: /* common_expression */ - case 508: /* when_then_expr */ - case 509: /* predicate */ - case 512: /* in_predicate_value */ - case 513: /* boolean_value_expression */ - case 514: /* boolean_primary */ - case 515: /* from_clause_opt */ - case 516: /* table_reference_list */ - case 517: /* table_reference */ - case 518: /* table_primary */ - case 519: /* joined_table */ - case 521: /* subquery */ - case 522: /* parenthesized_joined_table */ - case 525: /* join_on_clause_opt */ - case 526: /* window_offset_clause_opt */ - case 527: /* jlimit_clause_opt */ - case 528: /* window_offset_literal */ - case 529: /* query_specification */ - case 535: /* range_opt */ - case 536: /* every_opt */ - case 537: /* fill_opt */ - case 538: /* twindow_clause_opt */ - case 540: /* having_clause_opt */ - case 541: /* select_item */ - case 543: /* partition_item */ - case 544: /* interval_sliding_duration_literal */ - case 547: /* query_expression */ - case 548: /* query_simple */ - case 550: /* slimit_clause_opt */ - case 551: /* limit_clause_opt */ - case 552: /* union_query_expression */ - case 553: /* query_simple_or_subquery */ - case 555: /* sort_specification */ + case 379: /* cmd */ + case 382: /* literal */ + case 393: /* with_opt */ + case 399: /* search_condition */ + case 404: /* db_options */ + case 406: /* alter_db_options */ + case 408: /* start_opt */ + case 409: /* end_opt */ + case 413: /* signed */ + case 415: /* retention */ + case 416: /* full_table_name */ + case 419: /* table_options */ + case 424: /* alter_table_clause */ + case 425: /* alter_table_options */ + case 428: /* column_options */ + case 429: /* tags_literal */ + case 430: /* create_subtable_clause */ + case 433: /* drop_table_clause */ + case 436: /* tag_def */ + case 437: /* column_def */ + case 442: /* duration_literal */ + case 443: /* rollup_func_name */ + case 445: /* col_name */ + case 448: /* like_pattern_opt */ + case 449: /* db_name_cond_opt */ + case 450: /* table_name_cond */ + case 451: /* from_db_opt */ + case 453: /* tag_item */ + case 457: /* full_tsma_name */ + case 459: /* index_options */ + case 460: /* full_index_name */ + case 462: /* sliding_opt */ + case 463: /* sma_stream_opt */ + case 464: /* func */ + case 468: /* query_or_subquery */ + case 469: /* where_clause_opt */ + case 472: /* explain_options */ + case 473: /* insert_query */ + case 478: /* full_view_name */ + case 481: /* stream_options */ + case 484: /* subtable_opt */ + case 487: /* column_stream_def */ + case 488: /* stream_col_options */ + case 489: /* expression */ + case 492: /* literal_func */ + case 493: /* signed_literal */ + case 496: /* expr_or_subquery */ + case 497: /* pseudo_column */ + case 498: /* column_reference */ + case 499: /* function_expression */ + case 500: /* case_when_expression */ + case 505: /* star_func_para */ + case 507: /* case_when_else_opt */ + case 508: /* common_expression */ + case 509: /* when_then_expr */ + case 510: /* predicate */ + case 513: /* in_predicate_value */ + case 514: /* boolean_value_expression */ + case 515: /* boolean_primary */ + case 516: /* from_clause_opt */ + case 517: /* table_reference_list */ + case 518: /* table_reference */ + case 519: /* table_primary */ + case 520: /* joined_table */ + case 522: /* subquery */ + case 523: /* parenthesized_joined_table */ + case 526: /* join_on_clause_opt */ + case 527: /* window_offset_clause_opt */ + case 528: /* jlimit_clause_opt */ + case 529: /* window_offset_literal */ + case 530: /* query_specification */ + case 536: /* range_opt */ + case 537: /* every_opt */ + case 538: /* fill_opt */ + case 539: /* twindow_clause_opt */ + case 541: /* having_clause_opt */ + case 542: /* select_item */ + case 544: /* partition_item */ + case 545: /* interval_sliding_duration_literal */ + case 548: /* query_expression */ + case 549: /* query_simple */ + case 551: /* slimit_clause_opt */ + case 552: /* limit_clause_opt */ + case 553: /* union_query_expression */ + case 554: /* query_simple_or_subquery */ + case 556: /* sort_specification */ { -#line 7 "sql.y" - nodesDestroyNode((yypminor->yy416)); -#line 3517 "sql.c" + nodesDestroyNode((yypminor->yy54)); } break; - case 379: /* account_options */ - case 380: /* alter_account_options */ - case 382: /* alter_account_option */ - case 406: /* speed_opt */ - case 466: /* with_meta */ - case 475: /* bufsize_opt */ + case 380: /* account_options */ + case 381: /* alter_account_options */ + case 383: /* alter_account_option */ + case 407: /* speed_opt */ + case 467: /* with_meta */ + case 476: /* bufsize_opt */ { -#line 54 "sql.y" -#line 3529 "sql.c" } break; - case 383: /* ip_range_list */ - case 384: /* white_list */ - case 385: /* white_list_opt */ - case 409: /* integer_list */ - case 410: /* variable_list */ - case 411: /* retention_list */ - case 416: /* column_def_list */ - case 417: /* tags_def_opt */ - case 419: /* multi_create_clause */ - case 420: /* tag_list_opt */ - case 421: /* tags_def */ - case 422: /* multi_drop_clause */ - case 430: /* specific_cols_opt */ - case 431: /* tags_literal_list */ - case 433: /* col_name_list */ - case 434: /* tag_def_list */ - case 438: /* duration_list */ - case 439: /* rollup_func_list */ - case 457: /* func_list */ - case 465: /* expression_list */ - case 481: /* col_list_opt */ - case 482: /* tag_def_or_ref_opt */ - case 485: /* column_stream_def_list */ - case 490: /* dnode_list */ - case 493: /* literal_list */ - case 501: /* star_func_para_list */ - case 503: /* other_para_list */ - case 505: /* when_then_list */ - case 530: /* hint_list */ - case 533: /* select_list */ - case 534: /* partition_by_clause_opt */ - case 539: /* group_by_clause_opt */ - case 542: /* partition_list */ - case 546: /* group_by_list */ - case 549: /* order_by_clause_opt */ - case 554: /* sort_specification_list */ + case 384: /* ip_range_list */ + case 385: /* white_list */ + case 386: /* white_list_opt */ + case 410: /* integer_list */ + case 411: /* variable_list */ + case 412: /* retention_list */ + case 417: /* column_def_list */ + case 418: /* tags_def_opt */ + case 420: /* multi_create_clause */ + case 421: /* tag_list_opt */ + case 422: /* tags_def */ + case 423: /* multi_drop_clause */ + case 431: /* specific_cols_opt */ + case 432: /* tags_literal_list */ + case 434: /* col_name_list */ + case 435: /* tag_def_list */ + case 439: /* duration_list */ + case 440: /* rollup_func_list */ + case 458: /* func_list */ + case 466: /* expression_list */ + case 482: /* col_list_opt */ + case 483: /* tag_def_or_ref_opt */ + case 486: /* column_stream_def_list */ + case 491: /* dnode_list */ + case 494: /* literal_list */ + case 502: /* star_func_para_list */ + case 504: /* other_para_list */ + case 506: /* when_then_list */ + case 531: /* hint_list */ + case 534: /* select_list */ + case 535: /* partition_by_clause_opt */ + case 540: /* group_by_clause_opt */ + case 543: /* partition_list */ + case 547: /* group_by_list */ + case 550: /* order_by_clause_opt */ + case 555: /* sort_specification_list */ { -#line 85 "sql.y" - nodesDestroyList((yypminor->yy316)); -#line 3571 "sql.c" + nodesDestroyList((yypminor->yy652)); } break; - case 386: /* is_import_opt */ - case 387: /* is_createdb_opt */ - case 389: /* sysinfo_opt */ + case 387: /* is_import_opt */ + case 388: /* is_createdb_opt */ + case 390: /* sysinfo_opt */ { -#line 99 "sql.y" -#line 3580 "sql.c" } break; - case 388: /* user_name */ - case 395: /* db_name */ - case 396: /* table_name */ - case 397: /* topic_name */ - case 399: /* dnode_endpoint */ - case 425: /* column_name */ - case 443: /* function_name */ - case 453: /* column_alias */ - case 454: /* tsma_name */ - case 460: /* index_name */ - case 464: /* sma_func_name */ - case 469: /* cgroup_name */ - case 476: /* language_opt */ - case 478: /* view_name */ - case 479: /* stream_name */ - case 489: /* on_vgroup_id */ - case 494: /* table_alias */ - case 500: /* star_func */ - case 502: /* noarg_func */ - case 520: /* alias_opt */ + case 389: /* user_name */ + case 396: /* db_name */ + case 397: /* table_name */ + case 398: /* topic_name */ + case 400: /* dnode_endpoint */ + case 426: /* column_name */ + case 444: /* function_name */ + case 454: /* column_alias */ + case 455: /* tsma_name */ + case 461: /* index_name */ + case 465: /* sma_func_name */ + case 470: /* cgroup_name */ + case 477: /* language_opt */ + case 479: /* view_name */ + case 480: /* stream_name */ + case 490: /* on_vgroup_id */ + case 495: /* table_alias */ + case 501: /* star_func */ + case 503: /* noarg_func */ + case 521: /* alias_opt */ { -#line 1082 "sql.y" -#line 3606 "sql.c" } break; - case 390: /* privileges */ - case 393: /* priv_type_list */ - case 394: /* priv_type */ + case 391: /* privileges */ + case 394: /* priv_type_list */ + case 395: /* priv_type */ { -#line 131 "sql.y" -#line 3615 "sql.c" } break; - case 391: /* priv_level */ + case 392: /* priv_level */ { -#line 148 "sql.y" -#line 3622 "sql.c" } break; - case 400: /* force_opt */ - case 401: /* unsafe_opt */ - case 402: /* not_exists_opt */ - case 404: /* exists_opt */ - case 470: /* analyze_opt */ - case 473: /* or_replace_opt */ - case 474: /* agg_func_opt */ - case 484: /* ignore_opt */ - case 531: /* set_quantifier_opt */ - case 532: /* tag_mode_opt */ + case 401: /* force_opt */ + case 402: /* unsafe_opt */ + case 403: /* not_exists_opt */ + case 405: /* exists_opt */ + case 471: /* analyze_opt */ + case 474: /* or_replace_opt */ + case 475: /* agg_func_opt */ + case 485: /* ignore_opt */ + case 532: /* set_quantifier_opt */ + case 533: /* tag_mode_opt */ { -#line 180 "sql.y" -#line 3638 "sql.c" } break; - case 413: /* alter_db_option */ - case 440: /* alter_table_option */ + case 414: /* alter_db_option */ + case 441: /* alter_table_option */ { -#line 288 "sql.y" -#line 3646 "sql.c" } break; - case 426: /* type_name */ - case 437: /* type_name_default_len */ + case 427: /* type_name */ + case 438: /* type_name_default_len */ { -#line 425 "sql.y" -#line 3654 "sql.c" } break; - case 445: /* db_kind_opt */ - case 451: /* table_kind */ + case 446: /* db_kind_opt */ + case 452: /* table_kind */ { -#line 604 "sql.y" -#line 3662 "sql.c" } break; - case 446: /* table_kind_db_name_cond_opt */ + case 447: /* table_kind_db_name_cond_opt */ { -#line 569 "sql.y" -#line 3669 "sql.c" } break; - case 455: /* tsma_func_list */ + case 456: /* tsma_func_list */ { -#line 623 "sql.y" - nodesDestroyNode((yypminor->yy416)); -#line 3676 "sql.c" + nodesDestroyNode((yypminor->yy54)); } break; - case 510: /* compare_op */ - case 511: /* in_op */ + case 511: /* compare_op */ + case 512: /* in_op */ { -#line 1280 "sql.y" -#line 3684 "sql.c" } break; - case 523: /* join_type */ + case 524: /* join_type */ { -#line 1361 "sql.y" -#line 3691 "sql.c" } break; - case 524: /* join_subtype */ + case 525: /* join_subtype */ { -#line 1369 "sql.y" -#line 3698 "sql.c" } break; - case 545: /* fill_mode */ + case 546: /* fill_mode */ { -#line 1485 "sql.y" -#line 3705 "sql.c" } break; - case 556: /* ordering_specification_opt */ + case 557: /* ordering_specification_opt */ { -#line 1570 "sql.y" -#line 3712 "sql.c" } break; - case 557: /* null_ordering_opt */ + case 558: /* null_ordering_opt */ { -#line 1576 "sql.y" -#line 3719 "sql.c" } break; /********* End destructor definitions *****************************************/ @@ -4005,761 +4004,762 @@ static void yy_shift( /* For rule J, yyRuleInfoLhs[J] contains the symbol on the left-hand side ** of that rule */ static const YYCODETYPE yyRuleInfoLhs[] = { - 378, /* (0) cmd ::= CREATE ACCOUNT NK_ID PASS NK_STRING account_options */ - 378, /* (1) cmd ::= ALTER ACCOUNT NK_ID alter_account_options */ - 379, /* (2) account_options ::= */ - 379, /* (3) account_options ::= account_options PPS literal */ - 379, /* (4) account_options ::= account_options TSERIES literal */ - 379, /* (5) account_options ::= account_options STORAGE literal */ - 379, /* (6) account_options ::= account_options STREAMS literal */ - 379, /* (7) account_options ::= account_options QTIME literal */ - 379, /* (8) account_options ::= account_options DBS literal */ - 379, /* (9) account_options ::= account_options USERS literal */ - 379, /* (10) account_options ::= account_options CONNS literal */ - 379, /* (11) account_options ::= account_options STATE literal */ - 380, /* (12) alter_account_options ::= alter_account_option */ - 380, /* (13) alter_account_options ::= alter_account_options alter_account_option */ - 382, /* (14) alter_account_option ::= PASS literal */ - 382, /* (15) alter_account_option ::= PPS literal */ - 382, /* (16) alter_account_option ::= TSERIES literal */ - 382, /* (17) alter_account_option ::= STORAGE literal */ - 382, /* (18) alter_account_option ::= STREAMS literal */ - 382, /* (19) alter_account_option ::= QTIME literal */ - 382, /* (20) alter_account_option ::= DBS literal */ - 382, /* (21) alter_account_option ::= USERS literal */ - 382, /* (22) alter_account_option ::= CONNS literal */ - 382, /* (23) alter_account_option ::= STATE literal */ - 383, /* (24) ip_range_list ::= NK_STRING */ - 383, /* (25) ip_range_list ::= ip_range_list NK_COMMA NK_STRING */ - 384, /* (26) white_list ::= HOST ip_range_list */ - 385, /* (27) white_list_opt ::= */ - 385, /* (28) white_list_opt ::= white_list */ - 386, /* (29) is_import_opt ::= */ - 386, /* (30) is_import_opt ::= IS_IMPORT NK_INTEGER */ - 387, /* (31) is_createdb_opt ::= */ - 387, /* (32) is_createdb_opt ::= CREATEDB NK_INTEGER */ - 378, /* (33) cmd ::= CREATE USER user_name PASS NK_STRING sysinfo_opt is_createdb_opt is_import_opt white_list_opt */ - 378, /* (34) cmd ::= ALTER USER user_name PASS NK_STRING */ - 378, /* (35) cmd ::= ALTER USER user_name ENABLE NK_INTEGER */ - 378, /* (36) cmd ::= ALTER USER user_name SYSINFO NK_INTEGER */ - 378, /* (37) cmd ::= ALTER USER user_name CREATEDB NK_INTEGER */ - 378, /* (38) cmd ::= ALTER USER user_name ADD white_list */ - 378, /* (39) cmd ::= ALTER USER user_name DROP white_list */ - 378, /* (40) cmd ::= DROP USER user_name */ - 389, /* (41) sysinfo_opt ::= */ - 389, /* (42) sysinfo_opt ::= SYSINFO NK_INTEGER */ - 378, /* (43) cmd ::= GRANT privileges ON priv_level with_opt TO user_name */ - 378, /* (44) cmd ::= REVOKE privileges ON priv_level with_opt FROM user_name */ - 390, /* (45) privileges ::= ALL */ - 390, /* (46) privileges ::= priv_type_list */ - 390, /* (47) privileges ::= SUBSCRIBE */ - 393, /* (48) priv_type_list ::= priv_type */ - 393, /* (49) priv_type_list ::= priv_type_list NK_COMMA priv_type */ - 394, /* (50) priv_type ::= READ */ - 394, /* (51) priv_type ::= WRITE */ - 394, /* (52) priv_type ::= ALTER */ - 391, /* (53) priv_level ::= NK_STAR NK_DOT NK_STAR */ - 391, /* (54) priv_level ::= db_name NK_DOT NK_STAR */ - 391, /* (55) priv_level ::= db_name NK_DOT table_name */ - 391, /* (56) priv_level ::= topic_name */ - 392, /* (57) with_opt ::= */ - 392, /* (58) with_opt ::= WITH search_condition */ - 378, /* (59) cmd ::= CREATE ENCRYPT_KEY NK_STRING */ - 378, /* (60) cmd ::= CREATE DNODE dnode_endpoint */ - 378, /* (61) cmd ::= CREATE DNODE dnode_endpoint PORT NK_INTEGER */ - 378, /* (62) cmd ::= DROP DNODE NK_INTEGER force_opt */ - 378, /* (63) cmd ::= DROP DNODE dnode_endpoint force_opt */ - 378, /* (64) cmd ::= DROP DNODE NK_INTEGER unsafe_opt */ - 378, /* (65) cmd ::= DROP DNODE dnode_endpoint unsafe_opt */ - 378, /* (66) cmd ::= ALTER DNODE NK_INTEGER NK_STRING */ - 378, /* (67) cmd ::= ALTER DNODE NK_INTEGER NK_STRING NK_STRING */ - 378, /* (68) cmd ::= ALTER ALL DNODES NK_STRING */ - 378, /* (69) cmd ::= ALTER ALL DNODES NK_STRING NK_STRING */ - 378, /* (70) cmd ::= RESTORE DNODE NK_INTEGER */ - 399, /* (71) dnode_endpoint ::= NK_STRING */ - 399, /* (72) dnode_endpoint ::= NK_ID */ - 399, /* (73) dnode_endpoint ::= NK_IPTOKEN */ - 400, /* (74) force_opt ::= */ - 400, /* (75) force_opt ::= FORCE */ - 401, /* (76) unsafe_opt ::= UNSAFE */ - 378, /* (77) cmd ::= ALTER CLUSTER NK_STRING */ - 378, /* (78) cmd ::= ALTER CLUSTER NK_STRING NK_STRING */ - 378, /* (79) cmd ::= ALTER LOCAL NK_STRING */ - 378, /* (80) cmd ::= ALTER LOCAL NK_STRING NK_STRING */ - 378, /* (81) cmd ::= CREATE QNODE ON DNODE NK_INTEGER */ - 378, /* (82) cmd ::= DROP QNODE ON DNODE NK_INTEGER */ - 378, /* (83) cmd ::= RESTORE QNODE ON DNODE NK_INTEGER */ - 378, /* (84) cmd ::= CREATE BNODE ON DNODE NK_INTEGER */ - 378, /* (85) cmd ::= DROP BNODE ON DNODE NK_INTEGER */ - 378, /* (86) cmd ::= CREATE SNODE ON DNODE NK_INTEGER */ - 378, /* (87) cmd ::= DROP SNODE ON DNODE NK_INTEGER */ - 378, /* (88) cmd ::= CREATE MNODE ON DNODE NK_INTEGER */ - 378, /* (89) cmd ::= DROP MNODE ON DNODE NK_INTEGER */ - 378, /* (90) cmd ::= RESTORE MNODE ON DNODE NK_INTEGER */ - 378, /* (91) cmd ::= RESTORE VNODE ON DNODE NK_INTEGER */ - 378, /* (92) cmd ::= CREATE DATABASE not_exists_opt db_name db_options */ - 378, /* (93) cmd ::= DROP DATABASE exists_opt db_name */ - 378, /* (94) cmd ::= USE db_name */ - 378, /* (95) cmd ::= ALTER DATABASE db_name alter_db_options */ - 378, /* (96) cmd ::= FLUSH DATABASE db_name */ - 378, /* (97) cmd ::= TRIM DATABASE db_name speed_opt */ - 378, /* (98) cmd ::= S3MIGRATE DATABASE db_name */ - 378, /* (99) cmd ::= COMPACT DATABASE db_name start_opt end_opt */ - 402, /* (100) not_exists_opt ::= IF NOT EXISTS */ - 402, /* (101) not_exists_opt ::= */ - 404, /* (102) exists_opt ::= IF EXISTS */ - 404, /* (103) exists_opt ::= */ - 403, /* (104) db_options ::= */ - 403, /* (105) db_options ::= db_options BUFFER NK_INTEGER */ - 403, /* (106) db_options ::= db_options CACHEMODEL NK_STRING */ - 403, /* (107) db_options ::= db_options CACHESIZE NK_INTEGER */ - 403, /* (108) db_options ::= db_options COMP NK_INTEGER */ - 403, /* (109) db_options ::= db_options DURATION NK_INTEGER */ - 403, /* (110) db_options ::= db_options DURATION NK_VARIABLE */ - 403, /* (111) db_options ::= db_options MAXROWS NK_INTEGER */ - 403, /* (112) db_options ::= db_options MINROWS NK_INTEGER */ - 403, /* (113) db_options ::= db_options KEEP integer_list */ - 403, /* (114) db_options ::= db_options KEEP variable_list */ - 403, /* (115) db_options ::= db_options PAGES NK_INTEGER */ - 403, /* (116) db_options ::= db_options PAGESIZE NK_INTEGER */ - 403, /* (117) db_options ::= db_options TSDB_PAGESIZE NK_INTEGER */ - 403, /* (118) db_options ::= db_options PRECISION NK_STRING */ - 403, /* (119) db_options ::= db_options REPLICA NK_INTEGER */ - 403, /* (120) db_options ::= db_options VGROUPS NK_INTEGER */ - 403, /* (121) db_options ::= db_options SINGLE_STABLE NK_INTEGER */ - 403, /* (122) db_options ::= db_options RETENTIONS retention_list */ - 403, /* (123) db_options ::= db_options SCHEMALESS NK_INTEGER */ - 403, /* (124) db_options ::= db_options WAL_LEVEL NK_INTEGER */ - 403, /* (125) db_options ::= db_options WAL_FSYNC_PERIOD NK_INTEGER */ - 403, /* (126) db_options ::= db_options WAL_RETENTION_PERIOD NK_INTEGER */ - 403, /* (127) db_options ::= db_options WAL_RETENTION_PERIOD NK_MINUS NK_INTEGER */ - 403, /* (128) db_options ::= db_options WAL_RETENTION_SIZE NK_INTEGER */ - 403, /* (129) db_options ::= db_options WAL_RETENTION_SIZE NK_MINUS NK_INTEGER */ - 403, /* (130) db_options ::= db_options WAL_ROLL_PERIOD NK_INTEGER */ - 403, /* (131) db_options ::= db_options WAL_SEGMENT_SIZE NK_INTEGER */ - 403, /* (132) db_options ::= db_options STT_TRIGGER NK_INTEGER */ - 403, /* (133) db_options ::= db_options TABLE_PREFIX signed */ - 403, /* (134) db_options ::= db_options TABLE_SUFFIX signed */ - 403, /* (135) db_options ::= db_options S3_CHUNKSIZE NK_INTEGER */ - 403, /* (136) db_options ::= db_options S3_KEEPLOCAL NK_INTEGER */ - 403, /* (137) db_options ::= db_options S3_KEEPLOCAL NK_VARIABLE */ - 403, /* (138) db_options ::= db_options S3_COMPACT NK_INTEGER */ - 403, /* (139) db_options ::= db_options KEEP_TIME_OFFSET NK_INTEGER */ - 403, /* (140) db_options ::= db_options ENCRYPT_ALGORITHM NK_STRING */ - 405, /* (141) alter_db_options ::= alter_db_option */ - 405, /* (142) alter_db_options ::= alter_db_options alter_db_option */ - 413, /* (143) alter_db_option ::= BUFFER NK_INTEGER */ - 413, /* (144) alter_db_option ::= CACHEMODEL NK_STRING */ - 413, /* (145) alter_db_option ::= CACHESIZE NK_INTEGER */ - 413, /* (146) alter_db_option ::= WAL_FSYNC_PERIOD NK_INTEGER */ - 413, /* (147) alter_db_option ::= KEEP integer_list */ - 413, /* (148) alter_db_option ::= KEEP variable_list */ - 413, /* (149) alter_db_option ::= PAGES NK_INTEGER */ - 413, /* (150) alter_db_option ::= REPLICA NK_INTEGER */ - 413, /* (151) alter_db_option ::= WAL_LEVEL NK_INTEGER */ - 413, /* (152) alter_db_option ::= STT_TRIGGER NK_INTEGER */ - 413, /* (153) alter_db_option ::= MINROWS NK_INTEGER */ - 413, /* (154) alter_db_option ::= WAL_RETENTION_PERIOD NK_INTEGER */ - 413, /* (155) alter_db_option ::= WAL_RETENTION_PERIOD NK_MINUS NK_INTEGER */ - 413, /* (156) alter_db_option ::= WAL_RETENTION_SIZE NK_INTEGER */ - 413, /* (157) alter_db_option ::= WAL_RETENTION_SIZE NK_MINUS NK_INTEGER */ - 413, /* (158) alter_db_option ::= S3_KEEPLOCAL NK_INTEGER */ - 413, /* (159) alter_db_option ::= S3_KEEPLOCAL NK_VARIABLE */ - 413, /* (160) alter_db_option ::= S3_COMPACT NK_INTEGER */ - 413, /* (161) alter_db_option ::= KEEP_TIME_OFFSET NK_INTEGER */ - 413, /* (162) alter_db_option ::= ENCRYPT_ALGORITHM NK_STRING */ - 409, /* (163) integer_list ::= NK_INTEGER */ - 409, /* (164) integer_list ::= integer_list NK_COMMA NK_INTEGER */ - 410, /* (165) variable_list ::= NK_VARIABLE */ - 410, /* (166) variable_list ::= variable_list NK_COMMA NK_VARIABLE */ - 411, /* (167) retention_list ::= retention */ - 411, /* (168) retention_list ::= retention_list NK_COMMA retention */ - 414, /* (169) retention ::= NK_VARIABLE NK_COLON NK_VARIABLE */ - 414, /* (170) retention ::= NK_MINUS NK_COLON NK_VARIABLE */ - 406, /* (171) speed_opt ::= */ - 406, /* (172) speed_opt ::= BWLIMIT NK_INTEGER */ - 407, /* (173) start_opt ::= */ - 407, /* (174) start_opt ::= START WITH NK_INTEGER */ - 407, /* (175) start_opt ::= START WITH NK_STRING */ - 407, /* (176) start_opt ::= START WITH TIMESTAMP NK_STRING */ - 408, /* (177) end_opt ::= */ - 408, /* (178) end_opt ::= END WITH NK_INTEGER */ - 408, /* (179) end_opt ::= END WITH NK_STRING */ - 408, /* (180) end_opt ::= END WITH TIMESTAMP NK_STRING */ - 378, /* (181) cmd ::= CREATE TABLE not_exists_opt full_table_name NK_LP column_def_list NK_RP tags_def_opt table_options */ - 378, /* (182) cmd ::= CREATE TABLE multi_create_clause */ - 378, /* (183) cmd ::= CREATE TABLE not_exists_opt USING full_table_name NK_LP tag_list_opt NK_RP FILE NK_STRING */ - 378, /* (184) cmd ::= CREATE STABLE not_exists_opt full_table_name NK_LP column_def_list NK_RP tags_def table_options */ - 378, /* (185) cmd ::= DROP TABLE multi_drop_clause */ - 378, /* (186) cmd ::= DROP STABLE exists_opt full_table_name */ - 378, /* (187) cmd ::= ALTER TABLE alter_table_clause */ - 378, /* (188) cmd ::= ALTER STABLE alter_table_clause */ - 423, /* (189) alter_table_clause ::= full_table_name alter_table_options */ - 423, /* (190) alter_table_clause ::= full_table_name ADD COLUMN column_name type_name column_options */ - 423, /* (191) alter_table_clause ::= full_table_name DROP COLUMN column_name */ - 423, /* (192) alter_table_clause ::= full_table_name MODIFY COLUMN column_name type_name */ - 423, /* (193) alter_table_clause ::= full_table_name MODIFY COLUMN column_name column_options */ - 423, /* (194) alter_table_clause ::= full_table_name RENAME COLUMN column_name column_name */ - 423, /* (195) alter_table_clause ::= full_table_name ADD TAG column_name type_name */ - 423, /* (196) alter_table_clause ::= full_table_name DROP TAG column_name */ - 423, /* (197) alter_table_clause ::= full_table_name MODIFY TAG column_name type_name */ - 423, /* (198) alter_table_clause ::= full_table_name RENAME TAG column_name column_name */ - 423, /* (199) alter_table_clause ::= full_table_name SET TAG column_name NK_EQ tags_literal */ - 419, /* (200) multi_create_clause ::= create_subtable_clause */ - 419, /* (201) multi_create_clause ::= multi_create_clause create_subtable_clause */ - 429, /* (202) create_subtable_clause ::= not_exists_opt full_table_name USING full_table_name specific_cols_opt TAGS NK_LP tags_literal_list NK_RP table_options */ - 422, /* (203) multi_drop_clause ::= drop_table_clause */ - 422, /* (204) multi_drop_clause ::= multi_drop_clause NK_COMMA drop_table_clause */ - 432, /* (205) drop_table_clause ::= exists_opt full_table_name */ - 430, /* (206) specific_cols_opt ::= */ - 430, /* (207) specific_cols_opt ::= NK_LP col_name_list NK_RP */ - 415, /* (208) full_table_name ::= table_name */ - 415, /* (209) full_table_name ::= db_name NK_DOT table_name */ - 434, /* (210) tag_def_list ::= tag_def */ - 434, /* (211) tag_def_list ::= tag_def_list NK_COMMA tag_def */ - 435, /* (212) tag_def ::= column_name type_name */ - 416, /* (213) column_def_list ::= column_def */ - 416, /* (214) column_def_list ::= column_def_list NK_COMMA column_def */ - 436, /* (215) column_def ::= column_name type_name column_options */ - 426, /* (216) type_name ::= BOOL */ - 426, /* (217) type_name ::= TINYINT */ - 426, /* (218) type_name ::= SMALLINT */ - 426, /* (219) type_name ::= INT */ - 426, /* (220) type_name ::= INTEGER */ - 426, /* (221) type_name ::= BIGINT */ - 426, /* (222) type_name ::= FLOAT */ - 426, /* (223) type_name ::= DOUBLE */ - 426, /* (224) type_name ::= BINARY NK_LP NK_INTEGER NK_RP */ - 426, /* (225) type_name ::= TIMESTAMP */ - 426, /* (226) type_name ::= NCHAR NK_LP NK_INTEGER NK_RP */ - 426, /* (227) type_name ::= TINYINT UNSIGNED */ - 426, /* (228) type_name ::= SMALLINT UNSIGNED */ - 426, /* (229) type_name ::= INT UNSIGNED */ - 426, /* (230) type_name ::= BIGINT UNSIGNED */ - 426, /* (231) type_name ::= JSON */ - 426, /* (232) type_name ::= VARCHAR NK_LP NK_INTEGER NK_RP */ - 426, /* (233) type_name ::= MEDIUMBLOB */ - 426, /* (234) type_name ::= BLOB */ - 426, /* (235) type_name ::= VARBINARY NK_LP NK_INTEGER NK_RP */ - 426, /* (236) type_name ::= GEOMETRY NK_LP NK_INTEGER NK_RP */ - 426, /* (237) type_name ::= DECIMAL */ - 426, /* (238) type_name ::= DECIMAL NK_LP NK_INTEGER NK_RP */ - 426, /* (239) type_name ::= DECIMAL NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP */ - 437, /* (240) type_name_default_len ::= BINARY */ - 437, /* (241) type_name_default_len ::= NCHAR */ - 437, /* (242) type_name_default_len ::= VARCHAR */ - 437, /* (243) type_name_default_len ::= VARBINARY */ - 417, /* (244) tags_def_opt ::= */ - 417, /* (245) tags_def_opt ::= tags_def */ - 421, /* (246) tags_def ::= TAGS NK_LP tag_def_list NK_RP */ - 418, /* (247) table_options ::= */ - 418, /* (248) table_options ::= table_options COMMENT NK_STRING */ - 418, /* (249) table_options ::= table_options MAX_DELAY duration_list */ - 418, /* (250) table_options ::= table_options WATERMARK duration_list */ - 418, /* (251) table_options ::= table_options ROLLUP NK_LP rollup_func_list NK_RP */ - 418, /* (252) table_options ::= table_options TTL NK_INTEGER */ - 418, /* (253) table_options ::= table_options SMA NK_LP col_name_list NK_RP */ - 418, /* (254) table_options ::= table_options DELETE_MARK duration_list */ - 424, /* (255) alter_table_options ::= alter_table_option */ - 424, /* (256) alter_table_options ::= alter_table_options alter_table_option */ - 440, /* (257) alter_table_option ::= COMMENT NK_STRING */ - 440, /* (258) alter_table_option ::= TTL NK_INTEGER */ - 438, /* (259) duration_list ::= duration_literal */ - 438, /* (260) duration_list ::= duration_list NK_COMMA duration_literal */ - 439, /* (261) rollup_func_list ::= rollup_func_name */ - 439, /* (262) rollup_func_list ::= rollup_func_list NK_COMMA rollup_func_name */ - 442, /* (263) rollup_func_name ::= function_name */ - 442, /* (264) rollup_func_name ::= FIRST */ - 442, /* (265) rollup_func_name ::= LAST */ - 433, /* (266) col_name_list ::= col_name */ - 433, /* (267) col_name_list ::= col_name_list NK_COMMA col_name */ - 444, /* (268) col_name ::= column_name */ - 378, /* (269) cmd ::= SHOW DNODES */ - 378, /* (270) cmd ::= SHOW USERS */ - 378, /* (271) cmd ::= SHOW USERS FULL */ - 378, /* (272) cmd ::= SHOW USER PRIVILEGES */ - 378, /* (273) cmd ::= SHOW db_kind_opt DATABASES */ - 378, /* (274) cmd ::= SHOW table_kind_db_name_cond_opt TABLES like_pattern_opt */ - 378, /* (275) cmd ::= SHOW db_name_cond_opt STABLES like_pattern_opt */ - 378, /* (276) cmd ::= SHOW db_name_cond_opt VGROUPS */ - 378, /* (277) cmd ::= SHOW MNODES */ - 378, /* (278) cmd ::= SHOW QNODES */ - 378, /* (279) cmd ::= SHOW ARBGROUPS */ - 378, /* (280) cmd ::= SHOW FUNCTIONS */ - 378, /* (281) cmd ::= SHOW INDEXES FROM table_name_cond from_db_opt */ - 378, /* (282) cmd ::= SHOW INDEXES FROM db_name NK_DOT table_name */ - 378, /* (283) cmd ::= SHOW STREAMS */ - 378, /* (284) cmd ::= SHOW ACCOUNTS */ - 378, /* (285) cmd ::= SHOW APPS */ - 378, /* (286) cmd ::= SHOW CONNECTIONS */ - 378, /* (287) cmd ::= SHOW LICENCES */ - 378, /* (288) cmd ::= SHOW GRANTS */ - 378, /* (289) cmd ::= SHOW GRANTS FULL */ - 378, /* (290) cmd ::= SHOW GRANTS LOGS */ - 378, /* (291) cmd ::= SHOW CLUSTER MACHINES */ - 378, /* (292) cmd ::= SHOW CREATE DATABASE db_name */ - 378, /* (293) cmd ::= SHOW CREATE TABLE full_table_name */ - 378, /* (294) cmd ::= SHOW CREATE STABLE full_table_name */ - 378, /* (295) cmd ::= SHOW ENCRYPTIONS */ - 378, /* (296) cmd ::= SHOW QUERIES */ - 378, /* (297) cmd ::= SHOW SCORES */ - 378, /* (298) cmd ::= SHOW TOPICS */ - 378, /* (299) cmd ::= SHOW VARIABLES */ - 378, /* (300) cmd ::= SHOW CLUSTER VARIABLES */ - 378, /* (301) cmd ::= SHOW LOCAL VARIABLES */ - 378, /* (302) cmd ::= SHOW DNODE NK_INTEGER VARIABLES like_pattern_opt */ - 378, /* (303) cmd ::= SHOW BNODES */ - 378, /* (304) cmd ::= SHOW SNODES */ - 378, /* (305) cmd ::= SHOW CLUSTER */ - 378, /* (306) cmd ::= SHOW TRANSACTIONS */ - 378, /* (307) cmd ::= SHOW TABLE DISTRIBUTED full_table_name */ - 378, /* (308) cmd ::= SHOW CONSUMERS */ - 378, /* (309) cmd ::= SHOW SUBSCRIPTIONS */ - 378, /* (310) cmd ::= SHOW TAGS FROM table_name_cond from_db_opt */ - 378, /* (311) cmd ::= SHOW TAGS FROM db_name NK_DOT table_name */ - 378, /* (312) cmd ::= SHOW TABLE TAGS tag_list_opt FROM table_name_cond from_db_opt */ - 378, /* (313) cmd ::= SHOW TABLE TAGS tag_list_opt FROM db_name NK_DOT table_name */ - 378, /* (314) cmd ::= SHOW VNODES ON DNODE NK_INTEGER */ - 378, /* (315) cmd ::= SHOW VNODES */ - 378, /* (316) cmd ::= SHOW db_name_cond_opt ALIVE */ - 378, /* (317) cmd ::= SHOW CLUSTER ALIVE */ - 378, /* (318) cmd ::= SHOW db_name_cond_opt VIEWS like_pattern_opt */ - 378, /* (319) cmd ::= SHOW CREATE VIEW full_table_name */ - 378, /* (320) cmd ::= SHOW COMPACTS */ - 378, /* (321) cmd ::= SHOW COMPACT NK_INTEGER */ - 446, /* (322) table_kind_db_name_cond_opt ::= */ - 446, /* (323) table_kind_db_name_cond_opt ::= table_kind */ - 446, /* (324) table_kind_db_name_cond_opt ::= db_name NK_DOT */ - 446, /* (325) table_kind_db_name_cond_opt ::= table_kind db_name NK_DOT */ - 451, /* (326) table_kind ::= NORMAL */ - 451, /* (327) table_kind ::= CHILD */ - 448, /* (328) db_name_cond_opt ::= */ - 448, /* (329) db_name_cond_opt ::= db_name NK_DOT */ - 447, /* (330) like_pattern_opt ::= */ - 447, /* (331) like_pattern_opt ::= LIKE NK_STRING */ - 449, /* (332) table_name_cond ::= table_name */ - 450, /* (333) from_db_opt ::= */ - 450, /* (334) from_db_opt ::= FROM db_name */ - 420, /* (335) tag_list_opt ::= */ - 420, /* (336) tag_list_opt ::= tag_item */ - 420, /* (337) tag_list_opt ::= tag_list_opt NK_COMMA tag_item */ - 452, /* (338) tag_item ::= TBNAME */ - 452, /* (339) tag_item ::= QTAGS */ - 452, /* (340) tag_item ::= column_name */ - 452, /* (341) tag_item ::= column_name column_alias */ - 452, /* (342) tag_item ::= column_name AS column_alias */ - 445, /* (343) db_kind_opt ::= */ - 445, /* (344) db_kind_opt ::= USER */ - 445, /* (345) db_kind_opt ::= SYSTEM */ - 378, /* (346) cmd ::= CREATE TSMA not_exists_opt tsma_name ON full_table_name tsma_func_list INTERVAL NK_LP duration_literal NK_RP */ - 378, /* (347) cmd ::= CREATE RECURSIVE TSMA not_exists_opt tsma_name ON full_table_name INTERVAL NK_LP duration_literal NK_RP */ - 378, /* (348) cmd ::= DROP TSMA exists_opt full_tsma_name */ - 378, /* (349) cmd ::= SHOW db_name_cond_opt TSMAS */ - 456, /* (350) full_tsma_name ::= tsma_name */ - 456, /* (351) full_tsma_name ::= db_name NK_DOT tsma_name */ - 455, /* (352) tsma_func_list ::= FUNCTION NK_LP func_list NK_RP */ - 378, /* (353) cmd ::= CREATE SMA INDEX not_exists_opt col_name ON full_table_name index_options */ - 378, /* (354) cmd ::= CREATE INDEX not_exists_opt col_name ON full_table_name NK_LP col_name_list NK_RP */ - 378, /* (355) cmd ::= DROP INDEX exists_opt full_index_name */ - 459, /* (356) full_index_name ::= index_name */ - 459, /* (357) full_index_name ::= db_name NK_DOT index_name */ - 458, /* (358) index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_RP sliding_opt sma_stream_opt */ - 458, /* (359) 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 */ - 457, /* (360) func_list ::= func */ - 457, /* (361) func_list ::= func_list NK_COMMA func */ - 463, /* (362) func ::= sma_func_name NK_LP expression_list NK_RP */ - 464, /* (363) sma_func_name ::= function_name */ - 464, /* (364) sma_func_name ::= COUNT */ - 464, /* (365) sma_func_name ::= FIRST */ - 464, /* (366) sma_func_name ::= LAST */ - 464, /* (367) sma_func_name ::= LAST_ROW */ - 462, /* (368) sma_stream_opt ::= */ - 462, /* (369) sma_stream_opt ::= sma_stream_opt WATERMARK duration_literal */ - 462, /* (370) sma_stream_opt ::= sma_stream_opt MAX_DELAY duration_literal */ - 462, /* (371) sma_stream_opt ::= sma_stream_opt DELETE_MARK duration_literal */ - 466, /* (372) with_meta ::= AS */ - 466, /* (373) with_meta ::= WITH META AS */ - 466, /* (374) with_meta ::= ONLY META AS */ - 378, /* (375) cmd ::= CREATE TOPIC not_exists_opt topic_name AS query_or_subquery */ - 378, /* (376) cmd ::= CREATE TOPIC not_exists_opt topic_name with_meta DATABASE db_name */ - 378, /* (377) cmd ::= CREATE TOPIC not_exists_opt topic_name with_meta STABLE full_table_name where_clause_opt */ - 378, /* (378) cmd ::= DROP TOPIC exists_opt topic_name */ - 378, /* (379) cmd ::= DROP CONSUMER GROUP exists_opt cgroup_name ON topic_name */ - 378, /* (380) cmd ::= DESC full_table_name */ - 378, /* (381) cmd ::= DESCRIBE full_table_name */ - 378, /* (382) cmd ::= RESET QUERY CACHE */ - 378, /* (383) cmd ::= EXPLAIN analyze_opt explain_options query_or_subquery */ - 378, /* (384) cmd ::= EXPLAIN analyze_opt explain_options insert_query */ - 470, /* (385) analyze_opt ::= */ - 470, /* (386) analyze_opt ::= ANALYZE */ - 471, /* (387) explain_options ::= */ - 471, /* (388) explain_options ::= explain_options VERBOSE NK_BOOL */ - 471, /* (389) explain_options ::= explain_options RATIO NK_FLOAT */ - 378, /* (390) cmd ::= CREATE or_replace_opt agg_func_opt FUNCTION not_exists_opt function_name AS NK_STRING OUTPUTTYPE type_name bufsize_opt language_opt */ - 378, /* (391) cmd ::= DROP FUNCTION exists_opt function_name */ - 474, /* (392) agg_func_opt ::= */ - 474, /* (393) agg_func_opt ::= AGGREGATE */ - 475, /* (394) bufsize_opt ::= */ - 475, /* (395) bufsize_opt ::= BUFSIZE NK_INTEGER */ - 476, /* (396) language_opt ::= */ - 476, /* (397) language_opt ::= LANGUAGE NK_STRING */ - 473, /* (398) or_replace_opt ::= */ - 473, /* (399) or_replace_opt ::= OR REPLACE */ - 378, /* (400) cmd ::= CREATE or_replace_opt VIEW full_view_name AS query_or_subquery */ - 378, /* (401) cmd ::= DROP VIEW exists_opt full_view_name */ - 477, /* (402) full_view_name ::= view_name */ - 477, /* (403) full_view_name ::= db_name NK_DOT view_name */ - 378, /* (404) 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 */ - 378, /* (405) cmd ::= DROP STREAM exists_opt stream_name */ - 378, /* (406) cmd ::= PAUSE STREAM exists_opt stream_name */ - 378, /* (407) cmd ::= RESUME STREAM exists_opt ignore_opt stream_name */ - 481, /* (408) col_list_opt ::= */ - 481, /* (409) col_list_opt ::= NK_LP column_stream_def_list NK_RP */ - 485, /* (410) column_stream_def_list ::= column_stream_def */ - 485, /* (411) column_stream_def_list ::= column_stream_def_list NK_COMMA column_stream_def */ - 486, /* (412) column_stream_def ::= column_name stream_col_options */ - 487, /* (413) stream_col_options ::= */ - 487, /* (414) stream_col_options ::= stream_col_options PRIMARY KEY */ - 482, /* (415) tag_def_or_ref_opt ::= */ - 482, /* (416) tag_def_or_ref_opt ::= tags_def */ - 482, /* (417) tag_def_or_ref_opt ::= TAGS NK_LP column_stream_def_list NK_RP */ - 480, /* (418) stream_options ::= */ - 480, /* (419) stream_options ::= stream_options TRIGGER AT_ONCE */ - 480, /* (420) stream_options ::= stream_options TRIGGER WINDOW_CLOSE */ - 480, /* (421) stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal */ - 480, /* (422) stream_options ::= stream_options WATERMARK duration_literal */ - 480, /* (423) stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER */ - 480, /* (424) stream_options ::= stream_options FILL_HISTORY NK_INTEGER */ - 480, /* (425) stream_options ::= stream_options DELETE_MARK duration_literal */ - 480, /* (426) stream_options ::= stream_options IGNORE UPDATE NK_INTEGER */ - 483, /* (427) subtable_opt ::= */ - 483, /* (428) subtable_opt ::= SUBTABLE NK_LP expression NK_RP */ - 484, /* (429) ignore_opt ::= */ - 484, /* (430) ignore_opt ::= IGNORE UNTREATED */ - 378, /* (431) cmd ::= KILL CONNECTION NK_INTEGER */ - 378, /* (432) cmd ::= KILL QUERY NK_STRING */ - 378, /* (433) cmd ::= KILL TRANSACTION NK_INTEGER */ - 378, /* (434) cmd ::= KILL COMPACT NK_INTEGER */ - 378, /* (435) cmd ::= BALANCE VGROUP */ - 378, /* (436) cmd ::= BALANCE VGROUP LEADER on_vgroup_id */ - 378, /* (437) cmd ::= BALANCE VGROUP LEADER DATABASE db_name */ - 378, /* (438) cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER */ - 378, /* (439) cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list */ - 378, /* (440) cmd ::= SPLIT VGROUP NK_INTEGER */ - 489, /* (441) on_vgroup_id ::= */ - 489, /* (442) on_vgroup_id ::= ON NK_INTEGER */ - 490, /* (443) dnode_list ::= DNODE NK_INTEGER */ - 490, /* (444) dnode_list ::= dnode_list DNODE NK_INTEGER */ - 378, /* (445) cmd ::= DELETE FROM full_table_name where_clause_opt */ - 378, /* (446) cmd ::= query_or_subquery */ - 378, /* (447) cmd ::= insert_query */ - 472, /* (448) insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery */ - 472, /* (449) insert_query ::= INSERT INTO full_table_name query_or_subquery */ - 428, /* (450) tags_literal ::= NK_INTEGER */ - 428, /* (451) tags_literal ::= NK_INTEGER NK_PLUS duration_literal */ - 428, /* (452) tags_literal ::= NK_INTEGER NK_MINUS duration_literal */ - 428, /* (453) tags_literal ::= NK_PLUS NK_INTEGER */ - 428, /* (454) tags_literal ::= NK_PLUS NK_INTEGER NK_PLUS duration_literal */ - 428, /* (455) tags_literal ::= NK_PLUS NK_INTEGER NK_MINUS duration_literal */ - 428, /* (456) tags_literal ::= NK_MINUS NK_INTEGER */ - 428, /* (457) tags_literal ::= NK_MINUS NK_INTEGER NK_PLUS duration_literal */ - 428, /* (458) tags_literal ::= NK_MINUS NK_INTEGER NK_MINUS duration_literal */ - 428, /* (459) tags_literal ::= NK_FLOAT */ - 428, /* (460) tags_literal ::= NK_PLUS NK_FLOAT */ - 428, /* (461) tags_literal ::= NK_MINUS NK_FLOAT */ - 428, /* (462) tags_literal ::= NK_BIN */ - 428, /* (463) tags_literal ::= NK_BIN NK_PLUS duration_literal */ - 428, /* (464) tags_literal ::= NK_BIN NK_MINUS duration_literal */ - 428, /* (465) tags_literal ::= NK_PLUS NK_BIN */ - 428, /* (466) tags_literal ::= NK_PLUS NK_BIN NK_PLUS duration_literal */ - 428, /* (467) tags_literal ::= NK_PLUS NK_BIN NK_MINUS duration_literal */ - 428, /* (468) tags_literal ::= NK_MINUS NK_BIN */ - 428, /* (469) tags_literal ::= NK_MINUS NK_BIN NK_PLUS duration_literal */ - 428, /* (470) tags_literal ::= NK_MINUS NK_BIN NK_MINUS duration_literal */ - 428, /* (471) tags_literal ::= NK_HEX */ - 428, /* (472) tags_literal ::= NK_HEX NK_PLUS duration_literal */ - 428, /* (473) tags_literal ::= NK_HEX NK_MINUS duration_literal */ - 428, /* (474) tags_literal ::= NK_PLUS NK_HEX */ - 428, /* (475) tags_literal ::= NK_PLUS NK_HEX NK_PLUS duration_literal */ - 428, /* (476) tags_literal ::= NK_PLUS NK_HEX NK_MINUS duration_literal */ - 428, /* (477) tags_literal ::= NK_MINUS NK_HEX */ - 428, /* (478) tags_literal ::= NK_MINUS NK_HEX NK_PLUS duration_literal */ - 428, /* (479) tags_literal ::= NK_MINUS NK_HEX NK_MINUS duration_literal */ - 428, /* (480) tags_literal ::= NK_STRING */ - 428, /* (481) tags_literal ::= NK_STRING NK_PLUS duration_literal */ - 428, /* (482) tags_literal ::= NK_STRING NK_MINUS duration_literal */ - 428, /* (483) tags_literal ::= NK_BOOL */ - 428, /* (484) tags_literal ::= NULL */ - 428, /* (485) tags_literal ::= literal_func */ - 428, /* (486) tags_literal ::= literal_func NK_PLUS duration_literal */ - 428, /* (487) tags_literal ::= literal_func NK_MINUS duration_literal */ - 431, /* (488) tags_literal_list ::= tags_literal */ - 431, /* (489) tags_literal_list ::= tags_literal_list NK_COMMA tags_literal */ - 381, /* (490) literal ::= NK_INTEGER */ - 381, /* (491) literal ::= NK_FLOAT */ - 381, /* (492) literal ::= NK_STRING */ - 381, /* (493) literal ::= NK_BOOL */ - 381, /* (494) literal ::= TIMESTAMP NK_STRING */ - 381, /* (495) literal ::= duration_literal */ - 381, /* (496) literal ::= NULL */ - 381, /* (497) literal ::= NK_QUESTION */ - 441, /* (498) duration_literal ::= NK_VARIABLE */ - 412, /* (499) signed ::= NK_INTEGER */ - 412, /* (500) signed ::= NK_PLUS NK_INTEGER */ - 412, /* (501) signed ::= NK_MINUS NK_INTEGER */ - 412, /* (502) signed ::= NK_FLOAT */ - 412, /* (503) signed ::= NK_PLUS NK_FLOAT */ - 412, /* (504) signed ::= NK_MINUS NK_FLOAT */ - 492, /* (505) signed_literal ::= signed */ - 492, /* (506) signed_literal ::= NK_STRING */ - 492, /* (507) signed_literal ::= NK_BOOL */ - 492, /* (508) signed_literal ::= TIMESTAMP NK_STRING */ - 492, /* (509) signed_literal ::= duration_literal */ - 492, /* (510) signed_literal ::= NULL */ - 492, /* (511) signed_literal ::= literal_func */ - 492, /* (512) signed_literal ::= NK_QUESTION */ - 493, /* (513) literal_list ::= signed_literal */ - 493, /* (514) literal_list ::= literal_list NK_COMMA signed_literal */ - 395, /* (515) db_name ::= NK_ID */ - 396, /* (516) table_name ::= NK_ID */ - 425, /* (517) column_name ::= NK_ID */ - 443, /* (518) function_name ::= NK_ID */ - 478, /* (519) view_name ::= NK_ID */ - 494, /* (520) table_alias ::= NK_ID */ - 453, /* (521) column_alias ::= NK_ID */ - 453, /* (522) column_alias ::= NK_ALIAS */ - 388, /* (523) user_name ::= NK_ID */ - 397, /* (524) topic_name ::= NK_ID */ - 479, /* (525) stream_name ::= NK_ID */ - 469, /* (526) cgroup_name ::= NK_ID */ - 460, /* (527) index_name ::= NK_ID */ - 454, /* (528) tsma_name ::= NK_ID */ - 495, /* (529) expr_or_subquery ::= expression */ - 488, /* (530) expression ::= literal */ - 488, /* (531) expression ::= pseudo_column */ - 488, /* (532) expression ::= column_reference */ - 488, /* (533) expression ::= function_expression */ - 488, /* (534) expression ::= case_when_expression */ - 488, /* (535) expression ::= NK_LP expression NK_RP */ - 488, /* (536) expression ::= NK_PLUS expr_or_subquery */ - 488, /* (537) expression ::= NK_MINUS expr_or_subquery */ - 488, /* (538) expression ::= expr_or_subquery NK_PLUS expr_or_subquery */ - 488, /* (539) expression ::= expr_or_subquery NK_MINUS expr_or_subquery */ - 488, /* (540) expression ::= expr_or_subquery NK_STAR expr_or_subquery */ - 488, /* (541) expression ::= expr_or_subquery NK_SLASH expr_or_subquery */ - 488, /* (542) expression ::= expr_or_subquery NK_REM expr_or_subquery */ - 488, /* (543) expression ::= column_reference NK_ARROW NK_STRING */ - 488, /* (544) expression ::= expr_or_subquery NK_BITAND expr_or_subquery */ - 488, /* (545) expression ::= expr_or_subquery NK_BITOR expr_or_subquery */ - 465, /* (546) expression_list ::= expr_or_subquery */ - 465, /* (547) expression_list ::= expression_list NK_COMMA expr_or_subquery */ - 497, /* (548) column_reference ::= column_name */ - 497, /* (549) column_reference ::= table_name NK_DOT column_name */ - 497, /* (550) column_reference ::= NK_ALIAS */ - 497, /* (551) column_reference ::= table_name NK_DOT NK_ALIAS */ - 496, /* (552) pseudo_column ::= ROWTS */ - 496, /* (553) pseudo_column ::= TBNAME */ - 496, /* (554) pseudo_column ::= table_name NK_DOT TBNAME */ - 496, /* (555) pseudo_column ::= QSTART */ - 496, /* (556) pseudo_column ::= QEND */ - 496, /* (557) pseudo_column ::= QDURATION */ - 496, /* (558) pseudo_column ::= WSTART */ - 496, /* (559) pseudo_column ::= WEND */ - 496, /* (560) pseudo_column ::= WDURATION */ - 496, /* (561) pseudo_column ::= IROWTS */ - 496, /* (562) pseudo_column ::= ISFILLED */ - 496, /* (563) pseudo_column ::= QTAGS */ - 498, /* (564) function_expression ::= function_name NK_LP expression_list NK_RP */ - 498, /* (565) function_expression ::= star_func NK_LP star_func_para_list NK_RP */ - 498, /* (566) function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP */ - 498, /* (567) function_expression ::= CAST NK_LP expr_or_subquery AS type_name_default_len NK_RP */ - 498, /* (568) function_expression ::= literal_func */ - 491, /* (569) literal_func ::= noarg_func NK_LP NK_RP */ - 491, /* (570) literal_func ::= NOW */ - 491, /* (571) literal_func ::= TODAY */ - 502, /* (572) noarg_func ::= NOW */ - 502, /* (573) noarg_func ::= TODAY */ - 502, /* (574) noarg_func ::= TIMEZONE */ - 502, /* (575) noarg_func ::= DATABASE */ - 502, /* (576) noarg_func ::= CLIENT_VERSION */ - 502, /* (577) noarg_func ::= SERVER_VERSION */ - 502, /* (578) noarg_func ::= SERVER_STATUS */ - 502, /* (579) noarg_func ::= CURRENT_USER */ - 502, /* (580) noarg_func ::= USER */ - 500, /* (581) star_func ::= COUNT */ - 500, /* (582) star_func ::= FIRST */ - 500, /* (583) star_func ::= LAST */ - 500, /* (584) star_func ::= LAST_ROW */ - 501, /* (585) star_func_para_list ::= NK_STAR */ - 501, /* (586) star_func_para_list ::= other_para_list */ - 503, /* (587) other_para_list ::= star_func_para */ - 503, /* (588) other_para_list ::= other_para_list NK_COMMA star_func_para */ - 504, /* (589) star_func_para ::= expr_or_subquery */ - 504, /* (590) star_func_para ::= table_name NK_DOT NK_STAR */ - 499, /* (591) case_when_expression ::= CASE when_then_list case_when_else_opt END */ - 499, /* (592) case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END */ - 505, /* (593) when_then_list ::= when_then_expr */ - 505, /* (594) when_then_list ::= when_then_list when_then_expr */ - 508, /* (595) when_then_expr ::= WHEN common_expression THEN common_expression */ - 506, /* (596) case_when_else_opt ::= */ - 506, /* (597) case_when_else_opt ::= ELSE common_expression */ - 509, /* (598) predicate ::= expr_or_subquery compare_op expr_or_subquery */ - 509, /* (599) predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery */ - 509, /* (600) predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery */ - 509, /* (601) predicate ::= expr_or_subquery IS NULL */ - 509, /* (602) predicate ::= expr_or_subquery IS NOT NULL */ - 509, /* (603) predicate ::= expr_or_subquery in_op in_predicate_value */ - 510, /* (604) compare_op ::= NK_LT */ - 510, /* (605) compare_op ::= NK_GT */ - 510, /* (606) compare_op ::= NK_LE */ - 510, /* (607) compare_op ::= NK_GE */ - 510, /* (608) compare_op ::= NK_NE */ - 510, /* (609) compare_op ::= NK_EQ */ - 510, /* (610) compare_op ::= LIKE */ - 510, /* (611) compare_op ::= NOT LIKE */ - 510, /* (612) compare_op ::= MATCH */ - 510, /* (613) compare_op ::= NMATCH */ - 510, /* (614) compare_op ::= CONTAINS */ - 511, /* (615) in_op ::= IN */ - 511, /* (616) in_op ::= NOT IN */ - 512, /* (617) in_predicate_value ::= NK_LP literal_list NK_RP */ - 513, /* (618) boolean_value_expression ::= boolean_primary */ - 513, /* (619) boolean_value_expression ::= NOT boolean_primary */ - 513, /* (620) boolean_value_expression ::= boolean_value_expression OR boolean_value_expression */ - 513, /* (621) boolean_value_expression ::= boolean_value_expression AND boolean_value_expression */ - 514, /* (622) boolean_primary ::= predicate */ - 514, /* (623) boolean_primary ::= NK_LP boolean_value_expression NK_RP */ - 507, /* (624) common_expression ::= expr_or_subquery */ - 507, /* (625) common_expression ::= boolean_value_expression */ - 515, /* (626) from_clause_opt ::= */ - 515, /* (627) from_clause_opt ::= FROM table_reference_list */ - 516, /* (628) table_reference_list ::= table_reference */ - 516, /* (629) table_reference_list ::= table_reference_list NK_COMMA table_reference */ - 517, /* (630) table_reference ::= table_primary */ - 517, /* (631) table_reference ::= joined_table */ - 518, /* (632) table_primary ::= table_name alias_opt */ - 518, /* (633) table_primary ::= db_name NK_DOT table_name alias_opt */ - 518, /* (634) table_primary ::= subquery alias_opt */ - 518, /* (635) table_primary ::= parenthesized_joined_table */ - 520, /* (636) alias_opt ::= */ - 520, /* (637) alias_opt ::= table_alias */ - 520, /* (638) alias_opt ::= AS table_alias */ - 522, /* (639) parenthesized_joined_table ::= NK_LP joined_table NK_RP */ - 522, /* (640) parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP */ - 519, /* (641) joined_table ::= table_reference join_type join_subtype JOIN table_reference join_on_clause_opt window_offset_clause_opt jlimit_clause_opt */ - 523, /* (642) join_type ::= */ - 523, /* (643) join_type ::= INNER */ - 523, /* (644) join_type ::= LEFT */ - 523, /* (645) join_type ::= RIGHT */ - 523, /* (646) join_type ::= FULL */ - 524, /* (647) join_subtype ::= */ - 524, /* (648) join_subtype ::= OUTER */ - 524, /* (649) join_subtype ::= SEMI */ - 524, /* (650) join_subtype ::= ANTI */ - 524, /* (651) join_subtype ::= ASOF */ - 524, /* (652) join_subtype ::= WINDOW */ - 525, /* (653) join_on_clause_opt ::= */ - 525, /* (654) join_on_clause_opt ::= ON search_condition */ - 526, /* (655) window_offset_clause_opt ::= */ - 526, /* (656) window_offset_clause_opt ::= WINDOW_OFFSET NK_LP window_offset_literal NK_COMMA window_offset_literal NK_RP */ - 528, /* (657) window_offset_literal ::= NK_VARIABLE */ - 528, /* (658) window_offset_literal ::= NK_MINUS NK_VARIABLE */ - 527, /* (659) jlimit_clause_opt ::= */ - 527, /* (660) jlimit_clause_opt ::= JLIMIT NK_INTEGER */ - 529, /* (661) query_specification ::= SELECT hint_list set_quantifier_opt tag_mode_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 */ - 530, /* (662) hint_list ::= */ - 530, /* (663) hint_list ::= NK_HINT */ - 532, /* (664) tag_mode_opt ::= */ - 532, /* (665) tag_mode_opt ::= TAGS */ - 531, /* (666) set_quantifier_opt ::= */ - 531, /* (667) set_quantifier_opt ::= DISTINCT */ - 531, /* (668) set_quantifier_opt ::= ALL */ - 533, /* (669) select_list ::= select_item */ - 533, /* (670) select_list ::= select_list NK_COMMA select_item */ - 541, /* (671) select_item ::= NK_STAR */ - 541, /* (672) select_item ::= common_expression */ - 541, /* (673) select_item ::= common_expression column_alias */ - 541, /* (674) select_item ::= common_expression AS column_alias */ - 541, /* (675) select_item ::= table_name NK_DOT NK_STAR */ - 468, /* (676) where_clause_opt ::= */ - 468, /* (677) where_clause_opt ::= WHERE search_condition */ - 534, /* (678) partition_by_clause_opt ::= */ - 534, /* (679) partition_by_clause_opt ::= PARTITION BY partition_list */ - 542, /* (680) partition_list ::= partition_item */ - 542, /* (681) partition_list ::= partition_list NK_COMMA partition_item */ - 543, /* (682) partition_item ::= expr_or_subquery */ - 543, /* (683) partition_item ::= expr_or_subquery column_alias */ - 543, /* (684) partition_item ::= expr_or_subquery AS column_alias */ - 538, /* (685) twindow_clause_opt ::= */ - 538, /* (686) twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA interval_sliding_duration_literal NK_RP */ - 538, /* (687) twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP */ - 538, /* (688) twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_RP sliding_opt fill_opt */ - 538, /* (689) twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_COMMA interval_sliding_duration_literal NK_RP sliding_opt fill_opt */ - 538, /* (690) twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition */ - 538, /* (691) twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_RP */ - 538, /* (692) twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP */ - 461, /* (693) sliding_opt ::= */ - 461, /* (694) sliding_opt ::= SLIDING NK_LP interval_sliding_duration_literal NK_RP */ - 544, /* (695) interval_sliding_duration_literal ::= NK_VARIABLE */ - 544, /* (696) interval_sliding_duration_literal ::= NK_STRING */ - 544, /* (697) interval_sliding_duration_literal ::= NK_INTEGER */ - 537, /* (698) fill_opt ::= */ - 537, /* (699) fill_opt ::= FILL NK_LP fill_mode NK_RP */ - 537, /* (700) fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP */ - 537, /* (701) fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP */ - 545, /* (702) fill_mode ::= NONE */ - 545, /* (703) fill_mode ::= PREV */ - 545, /* (704) fill_mode ::= NULL */ - 545, /* (705) fill_mode ::= NULL_F */ - 545, /* (706) fill_mode ::= LINEAR */ - 545, /* (707) fill_mode ::= NEXT */ - 539, /* (708) group_by_clause_opt ::= */ - 539, /* (709) group_by_clause_opt ::= GROUP BY group_by_list */ - 546, /* (710) group_by_list ::= expr_or_subquery */ - 546, /* (711) group_by_list ::= group_by_list NK_COMMA expr_or_subquery */ - 540, /* (712) having_clause_opt ::= */ - 540, /* (713) having_clause_opt ::= HAVING search_condition */ - 535, /* (714) range_opt ::= */ - 535, /* (715) range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP */ - 535, /* (716) range_opt ::= RANGE NK_LP expr_or_subquery NK_RP */ - 536, /* (717) every_opt ::= */ - 536, /* (718) every_opt ::= EVERY NK_LP duration_literal NK_RP */ - 547, /* (719) query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt */ - 548, /* (720) query_simple ::= query_specification */ - 548, /* (721) query_simple ::= union_query_expression */ - 552, /* (722) union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery */ - 552, /* (723) union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery */ - 553, /* (724) query_simple_or_subquery ::= query_simple */ - 553, /* (725) query_simple_or_subquery ::= subquery */ - 467, /* (726) query_or_subquery ::= query_expression */ - 467, /* (727) query_or_subquery ::= subquery */ - 549, /* (728) order_by_clause_opt ::= */ - 549, /* (729) order_by_clause_opt ::= ORDER BY sort_specification_list */ - 550, /* (730) slimit_clause_opt ::= */ - 550, /* (731) slimit_clause_opt ::= SLIMIT NK_INTEGER */ - 550, /* (732) slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER */ - 550, /* (733) slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER */ - 551, /* (734) limit_clause_opt ::= */ - 551, /* (735) limit_clause_opt ::= LIMIT NK_INTEGER */ - 551, /* (736) limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER */ - 551, /* (737) limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER */ - 521, /* (738) subquery ::= NK_LP query_expression NK_RP */ - 521, /* (739) subquery ::= NK_LP subquery NK_RP */ - 398, /* (740) search_condition ::= common_expression */ - 554, /* (741) sort_specification_list ::= sort_specification */ - 554, /* (742) sort_specification_list ::= sort_specification_list NK_COMMA sort_specification */ - 555, /* (743) sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt */ - 556, /* (744) ordering_specification_opt ::= */ - 556, /* (745) ordering_specification_opt ::= ASC */ - 556, /* (746) ordering_specification_opt ::= DESC */ - 557, /* (747) null_ordering_opt ::= */ - 557, /* (748) null_ordering_opt ::= NULLS FIRST */ - 557, /* (749) null_ordering_opt ::= NULLS LAST */ - 427, /* (750) column_options ::= */ - 427, /* (751) column_options ::= column_options PRIMARY KEY */ - 427, /* (752) column_options ::= column_options ENCODE NK_STRING */ - 427, /* (753) column_options ::= column_options COMPRESS NK_STRING */ - 427, /* (754) column_options ::= column_options LEVEL NK_STRING */ + 379, /* (0) cmd ::= CREATE ACCOUNT NK_ID PASS NK_STRING account_options */ + 379, /* (1) cmd ::= ALTER ACCOUNT NK_ID alter_account_options */ + 380, /* (2) account_options ::= */ + 380, /* (3) account_options ::= account_options PPS literal */ + 380, /* (4) account_options ::= account_options TSERIES literal */ + 380, /* (5) account_options ::= account_options STORAGE literal */ + 380, /* (6) account_options ::= account_options STREAMS literal */ + 380, /* (7) account_options ::= account_options QTIME literal */ + 380, /* (8) account_options ::= account_options DBS literal */ + 380, /* (9) account_options ::= account_options USERS literal */ + 380, /* (10) account_options ::= account_options CONNS literal */ + 380, /* (11) account_options ::= account_options STATE literal */ + 381, /* (12) alter_account_options ::= alter_account_option */ + 381, /* (13) alter_account_options ::= alter_account_options alter_account_option */ + 383, /* (14) alter_account_option ::= PASS literal */ + 383, /* (15) alter_account_option ::= PPS literal */ + 383, /* (16) alter_account_option ::= TSERIES literal */ + 383, /* (17) alter_account_option ::= STORAGE literal */ + 383, /* (18) alter_account_option ::= STREAMS literal */ + 383, /* (19) alter_account_option ::= QTIME literal */ + 383, /* (20) alter_account_option ::= DBS literal */ + 383, /* (21) alter_account_option ::= USERS literal */ + 383, /* (22) alter_account_option ::= CONNS literal */ + 383, /* (23) alter_account_option ::= STATE literal */ + 384, /* (24) ip_range_list ::= NK_STRING */ + 384, /* (25) ip_range_list ::= ip_range_list NK_COMMA NK_STRING */ + 385, /* (26) white_list ::= HOST ip_range_list */ + 386, /* (27) white_list_opt ::= */ + 386, /* (28) white_list_opt ::= white_list */ + 387, /* (29) is_import_opt ::= */ + 387, /* (30) is_import_opt ::= IS_IMPORT NK_INTEGER */ + 388, /* (31) is_createdb_opt ::= */ + 388, /* (32) is_createdb_opt ::= CREATEDB NK_INTEGER */ + 379, /* (33) cmd ::= CREATE USER user_name PASS NK_STRING sysinfo_opt is_createdb_opt is_import_opt white_list_opt */ + 379, /* (34) cmd ::= ALTER USER user_name PASS NK_STRING */ + 379, /* (35) cmd ::= ALTER USER user_name ENABLE NK_INTEGER */ + 379, /* (36) cmd ::= ALTER USER user_name SYSINFO NK_INTEGER */ + 379, /* (37) cmd ::= ALTER USER user_name CREATEDB NK_INTEGER */ + 379, /* (38) cmd ::= ALTER USER user_name ADD white_list */ + 379, /* (39) cmd ::= ALTER USER user_name DROP white_list */ + 379, /* (40) cmd ::= DROP USER user_name */ + 390, /* (41) sysinfo_opt ::= */ + 390, /* (42) sysinfo_opt ::= SYSINFO NK_INTEGER */ + 379, /* (43) cmd ::= GRANT privileges ON priv_level with_opt TO user_name */ + 379, /* (44) cmd ::= REVOKE privileges ON priv_level with_opt FROM user_name */ + 391, /* (45) privileges ::= ALL */ + 391, /* (46) privileges ::= priv_type_list */ + 391, /* (47) privileges ::= SUBSCRIBE */ + 394, /* (48) priv_type_list ::= priv_type */ + 394, /* (49) priv_type_list ::= priv_type_list NK_COMMA priv_type */ + 395, /* (50) priv_type ::= READ */ + 395, /* (51) priv_type ::= WRITE */ + 395, /* (52) priv_type ::= ALTER */ + 392, /* (53) priv_level ::= NK_STAR NK_DOT NK_STAR */ + 392, /* (54) priv_level ::= db_name NK_DOT NK_STAR */ + 392, /* (55) priv_level ::= db_name NK_DOT table_name */ + 392, /* (56) priv_level ::= topic_name */ + 393, /* (57) with_opt ::= */ + 393, /* (58) with_opt ::= WITH search_condition */ + 379, /* (59) cmd ::= CREATE ENCRYPT_KEY NK_STRING */ + 379, /* (60) cmd ::= CREATE DNODE dnode_endpoint */ + 379, /* (61) cmd ::= CREATE DNODE dnode_endpoint PORT NK_INTEGER */ + 379, /* (62) cmd ::= DROP DNODE NK_INTEGER force_opt */ + 379, /* (63) cmd ::= DROP DNODE dnode_endpoint force_opt */ + 379, /* (64) cmd ::= DROP DNODE NK_INTEGER unsafe_opt */ + 379, /* (65) cmd ::= DROP DNODE dnode_endpoint unsafe_opt */ + 379, /* (66) cmd ::= ALTER DNODE NK_INTEGER NK_STRING */ + 379, /* (67) cmd ::= ALTER DNODE NK_INTEGER NK_STRING NK_STRING */ + 379, /* (68) cmd ::= ALTER ALL DNODES NK_STRING */ + 379, /* (69) cmd ::= ALTER ALL DNODES NK_STRING NK_STRING */ + 379, /* (70) cmd ::= RESTORE DNODE NK_INTEGER */ + 400, /* (71) dnode_endpoint ::= NK_STRING */ + 400, /* (72) dnode_endpoint ::= NK_ID */ + 400, /* (73) dnode_endpoint ::= NK_IPTOKEN */ + 401, /* (74) force_opt ::= */ + 401, /* (75) force_opt ::= FORCE */ + 402, /* (76) unsafe_opt ::= UNSAFE */ + 379, /* (77) cmd ::= ALTER CLUSTER NK_STRING */ + 379, /* (78) cmd ::= ALTER CLUSTER NK_STRING NK_STRING */ + 379, /* (79) cmd ::= ALTER LOCAL NK_STRING */ + 379, /* (80) cmd ::= ALTER LOCAL NK_STRING NK_STRING */ + 379, /* (81) cmd ::= CREATE QNODE ON DNODE NK_INTEGER */ + 379, /* (82) cmd ::= DROP QNODE ON DNODE NK_INTEGER */ + 379, /* (83) cmd ::= RESTORE QNODE ON DNODE NK_INTEGER */ + 379, /* (84) cmd ::= CREATE BNODE ON DNODE NK_INTEGER */ + 379, /* (85) cmd ::= DROP BNODE ON DNODE NK_INTEGER */ + 379, /* (86) cmd ::= CREATE SNODE ON DNODE NK_INTEGER */ + 379, /* (87) cmd ::= DROP SNODE ON DNODE NK_INTEGER */ + 379, /* (88) cmd ::= CREATE MNODE ON DNODE NK_INTEGER */ + 379, /* (89) cmd ::= DROP MNODE ON DNODE NK_INTEGER */ + 379, /* (90) cmd ::= RESTORE MNODE ON DNODE NK_INTEGER */ + 379, /* (91) cmd ::= RESTORE VNODE ON DNODE NK_INTEGER */ + 379, /* (92) cmd ::= CREATE DATABASE not_exists_opt db_name db_options */ + 379, /* (93) cmd ::= DROP DATABASE exists_opt db_name */ + 379, /* (94) cmd ::= USE db_name */ + 379, /* (95) cmd ::= ALTER DATABASE db_name alter_db_options */ + 379, /* (96) cmd ::= FLUSH DATABASE db_name */ + 379, /* (97) cmd ::= TRIM DATABASE db_name speed_opt */ + 379, /* (98) cmd ::= S3MIGRATE DATABASE db_name */ + 379, /* (99) cmd ::= COMPACT DATABASE db_name start_opt end_opt */ + 403, /* (100) not_exists_opt ::= IF NOT EXISTS */ + 403, /* (101) not_exists_opt ::= */ + 405, /* (102) exists_opt ::= IF EXISTS */ + 405, /* (103) exists_opt ::= */ + 404, /* (104) db_options ::= */ + 404, /* (105) db_options ::= db_options BUFFER NK_INTEGER */ + 404, /* (106) db_options ::= db_options CACHEMODEL NK_STRING */ + 404, /* (107) db_options ::= db_options CACHESIZE NK_INTEGER */ + 404, /* (108) db_options ::= db_options COMP NK_INTEGER */ + 404, /* (109) db_options ::= db_options DURATION NK_INTEGER */ + 404, /* (110) db_options ::= db_options DURATION NK_VARIABLE */ + 404, /* (111) db_options ::= db_options MAXROWS NK_INTEGER */ + 404, /* (112) db_options ::= db_options MINROWS NK_INTEGER */ + 404, /* (113) db_options ::= db_options KEEP integer_list */ + 404, /* (114) db_options ::= db_options KEEP variable_list */ + 404, /* (115) db_options ::= db_options PAGES NK_INTEGER */ + 404, /* (116) db_options ::= db_options PAGESIZE NK_INTEGER */ + 404, /* (117) db_options ::= db_options TSDB_PAGESIZE NK_INTEGER */ + 404, /* (118) db_options ::= db_options PRECISION NK_STRING */ + 404, /* (119) db_options ::= db_options REPLICA NK_INTEGER */ + 404, /* (120) db_options ::= db_options VGROUPS NK_INTEGER */ + 404, /* (121) db_options ::= db_options SINGLE_STABLE NK_INTEGER */ + 404, /* (122) db_options ::= db_options RETENTIONS retention_list */ + 404, /* (123) db_options ::= db_options SCHEMALESS NK_INTEGER */ + 404, /* (124) db_options ::= db_options WAL_LEVEL NK_INTEGER */ + 404, /* (125) db_options ::= db_options WAL_FSYNC_PERIOD NK_INTEGER */ + 404, /* (126) db_options ::= db_options WAL_RETENTION_PERIOD NK_INTEGER */ + 404, /* (127) db_options ::= db_options WAL_RETENTION_PERIOD NK_MINUS NK_INTEGER */ + 404, /* (128) db_options ::= db_options WAL_RETENTION_SIZE NK_INTEGER */ + 404, /* (129) db_options ::= db_options WAL_RETENTION_SIZE NK_MINUS NK_INTEGER */ + 404, /* (130) db_options ::= db_options WAL_ROLL_PERIOD NK_INTEGER */ + 404, /* (131) db_options ::= db_options WAL_SEGMENT_SIZE NK_INTEGER */ + 404, /* (132) db_options ::= db_options STT_TRIGGER NK_INTEGER */ + 404, /* (133) db_options ::= db_options TABLE_PREFIX signed */ + 404, /* (134) db_options ::= db_options TABLE_SUFFIX signed */ + 404, /* (135) db_options ::= db_options S3_CHUNKSIZE NK_INTEGER */ + 404, /* (136) db_options ::= db_options S3_KEEPLOCAL NK_INTEGER */ + 404, /* (137) db_options ::= db_options S3_KEEPLOCAL NK_VARIABLE */ + 404, /* (138) db_options ::= db_options S3_COMPACT NK_INTEGER */ + 404, /* (139) db_options ::= db_options KEEP_TIME_OFFSET NK_INTEGER */ + 404, /* (140) db_options ::= db_options ENCRYPT_ALGORITHM NK_STRING */ + 406, /* (141) alter_db_options ::= alter_db_option */ + 406, /* (142) alter_db_options ::= alter_db_options alter_db_option */ + 414, /* (143) alter_db_option ::= BUFFER NK_INTEGER */ + 414, /* (144) alter_db_option ::= CACHEMODEL NK_STRING */ + 414, /* (145) alter_db_option ::= CACHESIZE NK_INTEGER */ + 414, /* (146) alter_db_option ::= WAL_FSYNC_PERIOD NK_INTEGER */ + 414, /* (147) alter_db_option ::= KEEP integer_list */ + 414, /* (148) alter_db_option ::= KEEP variable_list */ + 414, /* (149) alter_db_option ::= PAGES NK_INTEGER */ + 414, /* (150) alter_db_option ::= REPLICA NK_INTEGER */ + 414, /* (151) alter_db_option ::= WAL_LEVEL NK_INTEGER */ + 414, /* (152) alter_db_option ::= STT_TRIGGER NK_INTEGER */ + 414, /* (153) alter_db_option ::= MINROWS NK_INTEGER */ + 414, /* (154) alter_db_option ::= WAL_RETENTION_PERIOD NK_INTEGER */ + 414, /* (155) alter_db_option ::= WAL_RETENTION_PERIOD NK_MINUS NK_INTEGER */ + 414, /* (156) alter_db_option ::= WAL_RETENTION_SIZE NK_INTEGER */ + 414, /* (157) alter_db_option ::= WAL_RETENTION_SIZE NK_MINUS NK_INTEGER */ + 414, /* (158) alter_db_option ::= S3_KEEPLOCAL NK_INTEGER */ + 414, /* (159) alter_db_option ::= S3_KEEPLOCAL NK_VARIABLE */ + 414, /* (160) alter_db_option ::= S3_COMPACT NK_INTEGER */ + 414, /* (161) alter_db_option ::= KEEP_TIME_OFFSET NK_INTEGER */ + 414, /* (162) alter_db_option ::= ENCRYPT_ALGORITHM NK_STRING */ + 410, /* (163) integer_list ::= NK_INTEGER */ + 410, /* (164) integer_list ::= integer_list NK_COMMA NK_INTEGER */ + 411, /* (165) variable_list ::= NK_VARIABLE */ + 411, /* (166) variable_list ::= variable_list NK_COMMA NK_VARIABLE */ + 412, /* (167) retention_list ::= retention */ + 412, /* (168) retention_list ::= retention_list NK_COMMA retention */ + 415, /* (169) retention ::= NK_VARIABLE NK_COLON NK_VARIABLE */ + 415, /* (170) retention ::= NK_MINUS NK_COLON NK_VARIABLE */ + 407, /* (171) speed_opt ::= */ + 407, /* (172) speed_opt ::= BWLIMIT NK_INTEGER */ + 408, /* (173) start_opt ::= */ + 408, /* (174) start_opt ::= START WITH NK_INTEGER */ + 408, /* (175) start_opt ::= START WITH NK_STRING */ + 408, /* (176) start_opt ::= START WITH TIMESTAMP NK_STRING */ + 409, /* (177) end_opt ::= */ + 409, /* (178) end_opt ::= END WITH NK_INTEGER */ + 409, /* (179) end_opt ::= END WITH NK_STRING */ + 409, /* (180) end_opt ::= END WITH TIMESTAMP NK_STRING */ + 379, /* (181) cmd ::= CREATE TABLE not_exists_opt full_table_name NK_LP column_def_list NK_RP tags_def_opt table_options */ + 379, /* (182) cmd ::= CREATE TABLE multi_create_clause */ + 379, /* (183) cmd ::= CREATE TABLE not_exists_opt USING full_table_name NK_LP tag_list_opt NK_RP FILE NK_STRING */ + 379, /* (184) cmd ::= CREATE STABLE not_exists_opt full_table_name NK_LP column_def_list NK_RP tags_def table_options */ + 379, /* (185) cmd ::= DROP TABLE multi_drop_clause */ + 379, /* (186) cmd ::= DROP STABLE exists_opt full_table_name */ + 379, /* (187) cmd ::= ALTER TABLE alter_table_clause */ + 379, /* (188) cmd ::= ALTER STABLE alter_table_clause */ + 424, /* (189) alter_table_clause ::= full_table_name alter_table_options */ + 424, /* (190) alter_table_clause ::= full_table_name ADD COLUMN column_name type_name column_options */ + 424, /* (191) alter_table_clause ::= full_table_name DROP COLUMN column_name */ + 424, /* (192) alter_table_clause ::= full_table_name MODIFY COLUMN column_name type_name */ + 424, /* (193) alter_table_clause ::= full_table_name MODIFY COLUMN column_name column_options */ + 424, /* (194) alter_table_clause ::= full_table_name RENAME COLUMN column_name column_name */ + 424, /* (195) alter_table_clause ::= full_table_name ADD TAG column_name type_name */ + 424, /* (196) alter_table_clause ::= full_table_name DROP TAG column_name */ + 424, /* (197) alter_table_clause ::= full_table_name MODIFY TAG column_name type_name */ + 424, /* (198) alter_table_clause ::= full_table_name RENAME TAG column_name column_name */ + 424, /* (199) alter_table_clause ::= full_table_name SET TAG column_name NK_EQ tags_literal */ + 420, /* (200) multi_create_clause ::= create_subtable_clause */ + 420, /* (201) multi_create_clause ::= multi_create_clause create_subtable_clause */ + 430, /* (202) create_subtable_clause ::= not_exists_opt full_table_name USING full_table_name specific_cols_opt TAGS NK_LP tags_literal_list NK_RP table_options */ + 423, /* (203) multi_drop_clause ::= drop_table_clause */ + 423, /* (204) multi_drop_clause ::= multi_drop_clause NK_COMMA drop_table_clause */ + 433, /* (205) drop_table_clause ::= exists_opt full_table_name */ + 431, /* (206) specific_cols_opt ::= */ + 431, /* (207) specific_cols_opt ::= NK_LP col_name_list NK_RP */ + 416, /* (208) full_table_name ::= table_name */ + 416, /* (209) full_table_name ::= db_name NK_DOT table_name */ + 435, /* (210) tag_def_list ::= tag_def */ + 435, /* (211) tag_def_list ::= tag_def_list NK_COMMA tag_def */ + 436, /* (212) tag_def ::= column_name type_name */ + 417, /* (213) column_def_list ::= column_def */ + 417, /* (214) column_def_list ::= column_def_list NK_COMMA column_def */ + 437, /* (215) column_def ::= column_name type_name column_options */ + 427, /* (216) type_name ::= BOOL */ + 427, /* (217) type_name ::= TINYINT */ + 427, /* (218) type_name ::= SMALLINT */ + 427, /* (219) type_name ::= INT */ + 427, /* (220) type_name ::= INTEGER */ + 427, /* (221) type_name ::= BIGINT */ + 427, /* (222) type_name ::= FLOAT */ + 427, /* (223) type_name ::= DOUBLE */ + 427, /* (224) type_name ::= BINARY NK_LP NK_INTEGER NK_RP */ + 427, /* (225) type_name ::= TIMESTAMP */ + 427, /* (226) type_name ::= NCHAR NK_LP NK_INTEGER NK_RP */ + 427, /* (227) type_name ::= TINYINT UNSIGNED */ + 427, /* (228) type_name ::= SMALLINT UNSIGNED */ + 427, /* (229) type_name ::= INT UNSIGNED */ + 427, /* (230) type_name ::= BIGINT UNSIGNED */ + 427, /* (231) type_name ::= JSON */ + 427, /* (232) type_name ::= VARCHAR NK_LP NK_INTEGER NK_RP */ + 427, /* (233) type_name ::= MEDIUMBLOB */ + 427, /* (234) type_name ::= BLOB */ + 427, /* (235) type_name ::= VARBINARY NK_LP NK_INTEGER NK_RP */ + 427, /* (236) type_name ::= GEOMETRY NK_LP NK_INTEGER NK_RP */ + 427, /* (237) type_name ::= DECIMAL */ + 427, /* (238) type_name ::= DECIMAL NK_LP NK_INTEGER NK_RP */ + 427, /* (239) type_name ::= DECIMAL NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP */ + 438, /* (240) type_name_default_len ::= BINARY */ + 438, /* (241) type_name_default_len ::= NCHAR */ + 438, /* (242) type_name_default_len ::= VARCHAR */ + 438, /* (243) type_name_default_len ::= VARBINARY */ + 418, /* (244) tags_def_opt ::= */ + 418, /* (245) tags_def_opt ::= tags_def */ + 422, /* (246) tags_def ::= TAGS NK_LP tag_def_list NK_RP */ + 419, /* (247) table_options ::= */ + 419, /* (248) table_options ::= table_options COMMENT NK_STRING */ + 419, /* (249) table_options ::= table_options MAX_DELAY duration_list */ + 419, /* (250) table_options ::= table_options WATERMARK duration_list */ + 419, /* (251) table_options ::= table_options ROLLUP NK_LP rollup_func_list NK_RP */ + 419, /* (252) table_options ::= table_options TTL NK_INTEGER */ + 419, /* (253) table_options ::= table_options SMA NK_LP col_name_list NK_RP */ + 419, /* (254) table_options ::= table_options DELETE_MARK duration_list */ + 425, /* (255) alter_table_options ::= alter_table_option */ + 425, /* (256) alter_table_options ::= alter_table_options alter_table_option */ + 441, /* (257) alter_table_option ::= COMMENT NK_STRING */ + 441, /* (258) alter_table_option ::= TTL NK_INTEGER */ + 439, /* (259) duration_list ::= duration_literal */ + 439, /* (260) duration_list ::= duration_list NK_COMMA duration_literal */ + 440, /* (261) rollup_func_list ::= rollup_func_name */ + 440, /* (262) rollup_func_list ::= rollup_func_list NK_COMMA rollup_func_name */ + 443, /* (263) rollup_func_name ::= function_name */ + 443, /* (264) rollup_func_name ::= FIRST */ + 443, /* (265) rollup_func_name ::= LAST */ + 434, /* (266) col_name_list ::= col_name */ + 434, /* (267) col_name_list ::= col_name_list NK_COMMA col_name */ + 445, /* (268) col_name ::= column_name */ + 379, /* (269) cmd ::= SHOW DNODES */ + 379, /* (270) cmd ::= SHOW USERS */ + 379, /* (271) cmd ::= SHOW USERS FULL */ + 379, /* (272) cmd ::= SHOW USER PRIVILEGES */ + 379, /* (273) cmd ::= SHOW db_kind_opt DATABASES */ + 379, /* (274) cmd ::= SHOW table_kind_db_name_cond_opt TABLES like_pattern_opt */ + 379, /* (275) cmd ::= SHOW db_name_cond_opt STABLES like_pattern_opt */ + 379, /* (276) cmd ::= SHOW db_name_cond_opt VGROUPS */ + 379, /* (277) cmd ::= SHOW MNODES */ + 379, /* (278) cmd ::= SHOW QNODES */ + 379, /* (279) cmd ::= SHOW ARBGROUPS */ + 379, /* (280) cmd ::= SHOW FUNCTIONS */ + 379, /* (281) cmd ::= SHOW INDEXES FROM table_name_cond from_db_opt */ + 379, /* (282) cmd ::= SHOW INDEXES FROM db_name NK_DOT table_name */ + 379, /* (283) cmd ::= SHOW STREAMS */ + 379, /* (284) cmd ::= SHOW ACCOUNTS */ + 379, /* (285) cmd ::= SHOW APPS */ + 379, /* (286) cmd ::= SHOW CONNECTIONS */ + 379, /* (287) cmd ::= SHOW LICENCES */ + 379, /* (288) cmd ::= SHOW GRANTS */ + 379, /* (289) cmd ::= SHOW GRANTS FULL */ + 379, /* (290) cmd ::= SHOW GRANTS LOGS */ + 379, /* (291) cmd ::= SHOW CLUSTER MACHINES */ + 379, /* (292) cmd ::= SHOW CREATE DATABASE db_name */ + 379, /* (293) cmd ::= SHOW CREATE TABLE full_table_name */ + 379, /* (294) cmd ::= SHOW CREATE STABLE full_table_name */ + 379, /* (295) cmd ::= SHOW ENCRYPTIONS */ + 379, /* (296) cmd ::= SHOW QUERIES */ + 379, /* (297) cmd ::= SHOW SCORES */ + 379, /* (298) cmd ::= SHOW TOPICS */ + 379, /* (299) cmd ::= SHOW VARIABLES */ + 379, /* (300) cmd ::= SHOW CLUSTER VARIABLES */ + 379, /* (301) cmd ::= SHOW LOCAL VARIABLES */ + 379, /* (302) cmd ::= SHOW DNODE NK_INTEGER VARIABLES like_pattern_opt */ + 379, /* (303) cmd ::= SHOW BNODES */ + 379, /* (304) cmd ::= SHOW SNODES */ + 379, /* (305) cmd ::= SHOW CLUSTER */ + 379, /* (306) cmd ::= SHOW TRANSACTIONS */ + 379, /* (307) cmd ::= SHOW TABLE DISTRIBUTED full_table_name */ + 379, /* (308) cmd ::= SHOW CONSUMERS */ + 379, /* (309) cmd ::= SHOW SUBSCRIPTIONS */ + 379, /* (310) cmd ::= SHOW TAGS FROM table_name_cond from_db_opt */ + 379, /* (311) cmd ::= SHOW TAGS FROM db_name NK_DOT table_name */ + 379, /* (312) cmd ::= SHOW TABLE TAGS tag_list_opt FROM table_name_cond from_db_opt */ + 379, /* (313) cmd ::= SHOW TABLE TAGS tag_list_opt FROM db_name NK_DOT table_name */ + 379, /* (314) cmd ::= SHOW VNODES ON DNODE NK_INTEGER */ + 379, /* (315) cmd ::= SHOW VNODES */ + 379, /* (316) cmd ::= SHOW db_name_cond_opt ALIVE */ + 379, /* (317) cmd ::= SHOW CLUSTER ALIVE */ + 379, /* (318) cmd ::= SHOW db_name_cond_opt VIEWS like_pattern_opt */ + 379, /* (319) cmd ::= SHOW CREATE VIEW full_table_name */ + 379, /* (320) cmd ::= SHOW COMPACTS */ + 379, /* (321) cmd ::= SHOW COMPACT NK_INTEGER */ + 447, /* (322) table_kind_db_name_cond_opt ::= */ + 447, /* (323) table_kind_db_name_cond_opt ::= table_kind */ + 447, /* (324) table_kind_db_name_cond_opt ::= db_name NK_DOT */ + 447, /* (325) table_kind_db_name_cond_opt ::= table_kind db_name NK_DOT */ + 452, /* (326) table_kind ::= NORMAL */ + 452, /* (327) table_kind ::= CHILD */ + 449, /* (328) db_name_cond_opt ::= */ + 449, /* (329) db_name_cond_opt ::= db_name NK_DOT */ + 448, /* (330) like_pattern_opt ::= */ + 448, /* (331) like_pattern_opt ::= LIKE NK_STRING */ + 450, /* (332) table_name_cond ::= table_name */ + 451, /* (333) from_db_opt ::= */ + 451, /* (334) from_db_opt ::= FROM db_name */ + 421, /* (335) tag_list_opt ::= */ + 421, /* (336) tag_list_opt ::= tag_item */ + 421, /* (337) tag_list_opt ::= tag_list_opt NK_COMMA tag_item */ + 453, /* (338) tag_item ::= TBNAME */ + 453, /* (339) tag_item ::= QTAGS */ + 453, /* (340) tag_item ::= column_name */ + 453, /* (341) tag_item ::= column_name column_alias */ + 453, /* (342) tag_item ::= column_name AS column_alias */ + 446, /* (343) db_kind_opt ::= */ + 446, /* (344) db_kind_opt ::= USER */ + 446, /* (345) db_kind_opt ::= SYSTEM */ + 379, /* (346) cmd ::= CREATE TSMA not_exists_opt tsma_name ON full_table_name tsma_func_list INTERVAL NK_LP duration_literal NK_RP */ + 379, /* (347) cmd ::= CREATE RECURSIVE TSMA not_exists_opt tsma_name ON full_table_name INTERVAL NK_LP duration_literal NK_RP */ + 379, /* (348) cmd ::= DROP TSMA exists_opt full_tsma_name */ + 379, /* (349) cmd ::= SHOW db_name_cond_opt TSMAS */ + 457, /* (350) full_tsma_name ::= tsma_name */ + 457, /* (351) full_tsma_name ::= db_name NK_DOT tsma_name */ + 456, /* (352) tsma_func_list ::= FUNCTION NK_LP func_list NK_RP */ + 379, /* (353) cmd ::= CREATE SMA INDEX not_exists_opt col_name ON full_table_name index_options */ + 379, /* (354) cmd ::= CREATE INDEX not_exists_opt col_name ON full_table_name NK_LP col_name_list NK_RP */ + 379, /* (355) cmd ::= DROP INDEX exists_opt full_index_name */ + 460, /* (356) full_index_name ::= index_name */ + 460, /* (357) full_index_name ::= db_name NK_DOT index_name */ + 459, /* (358) index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_RP sliding_opt sma_stream_opt */ + 459, /* (359) 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 */ + 458, /* (360) func_list ::= func */ + 458, /* (361) func_list ::= func_list NK_COMMA func */ + 464, /* (362) func ::= sma_func_name NK_LP expression_list NK_RP */ + 465, /* (363) sma_func_name ::= function_name */ + 465, /* (364) sma_func_name ::= COUNT */ + 465, /* (365) sma_func_name ::= FIRST */ + 465, /* (366) sma_func_name ::= LAST */ + 465, /* (367) sma_func_name ::= LAST_ROW */ + 463, /* (368) sma_stream_opt ::= */ + 463, /* (369) sma_stream_opt ::= sma_stream_opt WATERMARK duration_literal */ + 463, /* (370) sma_stream_opt ::= sma_stream_opt MAX_DELAY duration_literal */ + 463, /* (371) sma_stream_opt ::= sma_stream_opt DELETE_MARK duration_literal */ + 467, /* (372) with_meta ::= AS */ + 467, /* (373) with_meta ::= WITH META AS */ + 467, /* (374) with_meta ::= ONLY META AS */ + 379, /* (375) cmd ::= CREATE TOPIC not_exists_opt topic_name AS query_or_subquery */ + 379, /* (376) cmd ::= CREATE TOPIC not_exists_opt topic_name with_meta DATABASE db_name */ + 379, /* (377) cmd ::= CREATE TOPIC not_exists_opt topic_name with_meta STABLE full_table_name where_clause_opt */ + 379, /* (378) cmd ::= DROP TOPIC exists_opt topic_name */ + 379, /* (379) cmd ::= DROP CONSUMER GROUP exists_opt cgroup_name ON topic_name */ + 379, /* (380) cmd ::= DESC full_table_name */ + 379, /* (381) cmd ::= DESCRIBE full_table_name */ + 379, /* (382) cmd ::= RESET QUERY CACHE */ + 379, /* (383) cmd ::= EXPLAIN analyze_opt explain_options query_or_subquery */ + 379, /* (384) cmd ::= EXPLAIN analyze_opt explain_options insert_query */ + 471, /* (385) analyze_opt ::= */ + 471, /* (386) analyze_opt ::= ANALYZE */ + 472, /* (387) explain_options ::= */ + 472, /* (388) explain_options ::= explain_options VERBOSE NK_BOOL */ + 472, /* (389) explain_options ::= explain_options RATIO NK_FLOAT */ + 379, /* (390) cmd ::= CREATE or_replace_opt agg_func_opt FUNCTION not_exists_opt function_name AS NK_STRING OUTPUTTYPE type_name bufsize_opt language_opt */ + 379, /* (391) cmd ::= DROP FUNCTION exists_opt function_name */ + 475, /* (392) agg_func_opt ::= */ + 475, /* (393) agg_func_opt ::= AGGREGATE */ + 476, /* (394) bufsize_opt ::= */ + 476, /* (395) bufsize_opt ::= BUFSIZE NK_INTEGER */ + 477, /* (396) language_opt ::= */ + 477, /* (397) language_opt ::= LANGUAGE NK_STRING */ + 474, /* (398) or_replace_opt ::= */ + 474, /* (399) or_replace_opt ::= OR REPLACE */ + 379, /* (400) cmd ::= CREATE or_replace_opt VIEW full_view_name AS query_or_subquery */ + 379, /* (401) cmd ::= DROP VIEW exists_opt full_view_name */ + 478, /* (402) full_view_name ::= view_name */ + 478, /* (403) full_view_name ::= db_name NK_DOT view_name */ + 379, /* (404) 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 */ + 379, /* (405) cmd ::= DROP STREAM exists_opt stream_name */ + 379, /* (406) cmd ::= PAUSE STREAM exists_opt stream_name */ + 379, /* (407) cmd ::= RESUME STREAM exists_opt ignore_opt stream_name */ + 482, /* (408) col_list_opt ::= */ + 482, /* (409) col_list_opt ::= NK_LP column_stream_def_list NK_RP */ + 486, /* (410) column_stream_def_list ::= column_stream_def */ + 486, /* (411) column_stream_def_list ::= column_stream_def_list NK_COMMA column_stream_def */ + 487, /* (412) column_stream_def ::= column_name stream_col_options */ + 488, /* (413) stream_col_options ::= */ + 488, /* (414) stream_col_options ::= stream_col_options PRIMARY KEY */ + 483, /* (415) tag_def_or_ref_opt ::= */ + 483, /* (416) tag_def_or_ref_opt ::= tags_def */ + 483, /* (417) tag_def_or_ref_opt ::= TAGS NK_LP column_stream_def_list NK_RP */ + 481, /* (418) stream_options ::= */ + 481, /* (419) stream_options ::= stream_options TRIGGER AT_ONCE */ + 481, /* (420) stream_options ::= stream_options TRIGGER WINDOW_CLOSE */ + 481, /* (421) stream_options ::= stream_options TRIGGER FORCE_WINDOW_CLOSE */ + 481, /* (422) stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal */ + 481, /* (423) stream_options ::= stream_options WATERMARK duration_literal */ + 481, /* (424) stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER */ + 481, /* (425) stream_options ::= stream_options FILL_HISTORY NK_INTEGER */ + 481, /* (426) stream_options ::= stream_options DELETE_MARK duration_literal */ + 481, /* (427) stream_options ::= stream_options IGNORE UPDATE NK_INTEGER */ + 484, /* (428) subtable_opt ::= */ + 484, /* (429) subtable_opt ::= SUBTABLE NK_LP expression NK_RP */ + 485, /* (430) ignore_opt ::= */ + 485, /* (431) ignore_opt ::= IGNORE UNTREATED */ + 379, /* (432) cmd ::= KILL CONNECTION NK_INTEGER */ + 379, /* (433) cmd ::= KILL QUERY NK_STRING */ + 379, /* (434) cmd ::= KILL TRANSACTION NK_INTEGER */ + 379, /* (435) cmd ::= KILL COMPACT NK_INTEGER */ + 379, /* (436) cmd ::= BALANCE VGROUP */ + 379, /* (437) cmd ::= BALANCE VGROUP LEADER on_vgroup_id */ + 379, /* (438) cmd ::= BALANCE VGROUP LEADER DATABASE db_name */ + 379, /* (439) cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER */ + 379, /* (440) cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list */ + 379, /* (441) cmd ::= SPLIT VGROUP NK_INTEGER */ + 490, /* (442) on_vgroup_id ::= */ + 490, /* (443) on_vgroup_id ::= ON NK_INTEGER */ + 491, /* (444) dnode_list ::= DNODE NK_INTEGER */ + 491, /* (445) dnode_list ::= dnode_list DNODE NK_INTEGER */ + 379, /* (446) cmd ::= DELETE FROM full_table_name where_clause_opt */ + 379, /* (447) cmd ::= query_or_subquery */ + 379, /* (448) cmd ::= insert_query */ + 473, /* (449) insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery */ + 473, /* (450) insert_query ::= INSERT INTO full_table_name query_or_subquery */ + 429, /* (451) tags_literal ::= NK_INTEGER */ + 429, /* (452) tags_literal ::= NK_INTEGER NK_PLUS duration_literal */ + 429, /* (453) tags_literal ::= NK_INTEGER NK_MINUS duration_literal */ + 429, /* (454) tags_literal ::= NK_PLUS NK_INTEGER */ + 429, /* (455) tags_literal ::= NK_PLUS NK_INTEGER NK_PLUS duration_literal */ + 429, /* (456) tags_literal ::= NK_PLUS NK_INTEGER NK_MINUS duration_literal */ + 429, /* (457) tags_literal ::= NK_MINUS NK_INTEGER */ + 429, /* (458) tags_literal ::= NK_MINUS NK_INTEGER NK_PLUS duration_literal */ + 429, /* (459) tags_literal ::= NK_MINUS NK_INTEGER NK_MINUS duration_literal */ + 429, /* (460) tags_literal ::= NK_FLOAT */ + 429, /* (461) tags_literal ::= NK_PLUS NK_FLOAT */ + 429, /* (462) tags_literal ::= NK_MINUS NK_FLOAT */ + 429, /* (463) tags_literal ::= NK_BIN */ + 429, /* (464) tags_literal ::= NK_BIN NK_PLUS duration_literal */ + 429, /* (465) tags_literal ::= NK_BIN NK_MINUS duration_literal */ + 429, /* (466) tags_literal ::= NK_PLUS NK_BIN */ + 429, /* (467) tags_literal ::= NK_PLUS NK_BIN NK_PLUS duration_literal */ + 429, /* (468) tags_literal ::= NK_PLUS NK_BIN NK_MINUS duration_literal */ + 429, /* (469) tags_literal ::= NK_MINUS NK_BIN */ + 429, /* (470) tags_literal ::= NK_MINUS NK_BIN NK_PLUS duration_literal */ + 429, /* (471) tags_literal ::= NK_MINUS NK_BIN NK_MINUS duration_literal */ + 429, /* (472) tags_literal ::= NK_HEX */ + 429, /* (473) tags_literal ::= NK_HEX NK_PLUS duration_literal */ + 429, /* (474) tags_literal ::= NK_HEX NK_MINUS duration_literal */ + 429, /* (475) tags_literal ::= NK_PLUS NK_HEX */ + 429, /* (476) tags_literal ::= NK_PLUS NK_HEX NK_PLUS duration_literal */ + 429, /* (477) tags_literal ::= NK_PLUS NK_HEX NK_MINUS duration_literal */ + 429, /* (478) tags_literal ::= NK_MINUS NK_HEX */ + 429, /* (479) tags_literal ::= NK_MINUS NK_HEX NK_PLUS duration_literal */ + 429, /* (480) tags_literal ::= NK_MINUS NK_HEX NK_MINUS duration_literal */ + 429, /* (481) tags_literal ::= NK_STRING */ + 429, /* (482) tags_literal ::= NK_STRING NK_PLUS duration_literal */ + 429, /* (483) tags_literal ::= NK_STRING NK_MINUS duration_literal */ + 429, /* (484) tags_literal ::= NK_BOOL */ + 429, /* (485) tags_literal ::= NULL */ + 429, /* (486) tags_literal ::= literal_func */ + 429, /* (487) tags_literal ::= literal_func NK_PLUS duration_literal */ + 429, /* (488) tags_literal ::= literal_func NK_MINUS duration_literal */ + 432, /* (489) tags_literal_list ::= tags_literal */ + 432, /* (490) tags_literal_list ::= tags_literal_list NK_COMMA tags_literal */ + 382, /* (491) literal ::= NK_INTEGER */ + 382, /* (492) literal ::= NK_FLOAT */ + 382, /* (493) literal ::= NK_STRING */ + 382, /* (494) literal ::= NK_BOOL */ + 382, /* (495) literal ::= TIMESTAMP NK_STRING */ + 382, /* (496) literal ::= duration_literal */ + 382, /* (497) literal ::= NULL */ + 382, /* (498) literal ::= NK_QUESTION */ + 442, /* (499) duration_literal ::= NK_VARIABLE */ + 413, /* (500) signed ::= NK_INTEGER */ + 413, /* (501) signed ::= NK_PLUS NK_INTEGER */ + 413, /* (502) signed ::= NK_MINUS NK_INTEGER */ + 413, /* (503) signed ::= NK_FLOAT */ + 413, /* (504) signed ::= NK_PLUS NK_FLOAT */ + 413, /* (505) signed ::= NK_MINUS NK_FLOAT */ + 493, /* (506) signed_literal ::= signed */ + 493, /* (507) signed_literal ::= NK_STRING */ + 493, /* (508) signed_literal ::= NK_BOOL */ + 493, /* (509) signed_literal ::= TIMESTAMP NK_STRING */ + 493, /* (510) signed_literal ::= duration_literal */ + 493, /* (511) signed_literal ::= NULL */ + 493, /* (512) signed_literal ::= literal_func */ + 493, /* (513) signed_literal ::= NK_QUESTION */ + 494, /* (514) literal_list ::= signed_literal */ + 494, /* (515) literal_list ::= literal_list NK_COMMA signed_literal */ + 396, /* (516) db_name ::= NK_ID */ + 397, /* (517) table_name ::= NK_ID */ + 426, /* (518) column_name ::= NK_ID */ + 444, /* (519) function_name ::= NK_ID */ + 479, /* (520) view_name ::= NK_ID */ + 495, /* (521) table_alias ::= NK_ID */ + 454, /* (522) column_alias ::= NK_ID */ + 454, /* (523) column_alias ::= NK_ALIAS */ + 389, /* (524) user_name ::= NK_ID */ + 398, /* (525) topic_name ::= NK_ID */ + 480, /* (526) stream_name ::= NK_ID */ + 470, /* (527) cgroup_name ::= NK_ID */ + 461, /* (528) index_name ::= NK_ID */ + 455, /* (529) tsma_name ::= NK_ID */ + 496, /* (530) expr_or_subquery ::= expression */ + 489, /* (531) expression ::= literal */ + 489, /* (532) expression ::= pseudo_column */ + 489, /* (533) expression ::= column_reference */ + 489, /* (534) expression ::= function_expression */ + 489, /* (535) expression ::= case_when_expression */ + 489, /* (536) expression ::= NK_LP expression NK_RP */ + 489, /* (537) expression ::= NK_PLUS expr_or_subquery */ + 489, /* (538) expression ::= NK_MINUS expr_or_subquery */ + 489, /* (539) expression ::= expr_or_subquery NK_PLUS expr_or_subquery */ + 489, /* (540) expression ::= expr_or_subquery NK_MINUS expr_or_subquery */ + 489, /* (541) expression ::= expr_or_subquery NK_STAR expr_or_subquery */ + 489, /* (542) expression ::= expr_or_subquery NK_SLASH expr_or_subquery */ + 489, /* (543) expression ::= expr_or_subquery NK_REM expr_or_subquery */ + 489, /* (544) expression ::= column_reference NK_ARROW NK_STRING */ + 489, /* (545) expression ::= expr_or_subquery NK_BITAND expr_or_subquery */ + 489, /* (546) expression ::= expr_or_subquery NK_BITOR expr_or_subquery */ + 466, /* (547) expression_list ::= expr_or_subquery */ + 466, /* (548) expression_list ::= expression_list NK_COMMA expr_or_subquery */ + 498, /* (549) column_reference ::= column_name */ + 498, /* (550) column_reference ::= table_name NK_DOT column_name */ + 498, /* (551) column_reference ::= NK_ALIAS */ + 498, /* (552) column_reference ::= table_name NK_DOT NK_ALIAS */ + 497, /* (553) pseudo_column ::= ROWTS */ + 497, /* (554) pseudo_column ::= TBNAME */ + 497, /* (555) pseudo_column ::= table_name NK_DOT TBNAME */ + 497, /* (556) pseudo_column ::= QSTART */ + 497, /* (557) pseudo_column ::= QEND */ + 497, /* (558) pseudo_column ::= QDURATION */ + 497, /* (559) pseudo_column ::= WSTART */ + 497, /* (560) pseudo_column ::= WEND */ + 497, /* (561) pseudo_column ::= WDURATION */ + 497, /* (562) pseudo_column ::= IROWTS */ + 497, /* (563) pseudo_column ::= ISFILLED */ + 497, /* (564) pseudo_column ::= QTAGS */ + 499, /* (565) function_expression ::= function_name NK_LP expression_list NK_RP */ + 499, /* (566) function_expression ::= star_func NK_LP star_func_para_list NK_RP */ + 499, /* (567) function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP */ + 499, /* (568) function_expression ::= CAST NK_LP expr_or_subquery AS type_name_default_len NK_RP */ + 499, /* (569) function_expression ::= literal_func */ + 492, /* (570) literal_func ::= noarg_func NK_LP NK_RP */ + 492, /* (571) literal_func ::= NOW */ + 492, /* (572) literal_func ::= TODAY */ + 503, /* (573) noarg_func ::= NOW */ + 503, /* (574) noarg_func ::= TODAY */ + 503, /* (575) noarg_func ::= TIMEZONE */ + 503, /* (576) noarg_func ::= DATABASE */ + 503, /* (577) noarg_func ::= CLIENT_VERSION */ + 503, /* (578) noarg_func ::= SERVER_VERSION */ + 503, /* (579) noarg_func ::= SERVER_STATUS */ + 503, /* (580) noarg_func ::= CURRENT_USER */ + 503, /* (581) noarg_func ::= USER */ + 501, /* (582) star_func ::= COUNT */ + 501, /* (583) star_func ::= FIRST */ + 501, /* (584) star_func ::= LAST */ + 501, /* (585) star_func ::= LAST_ROW */ + 502, /* (586) star_func_para_list ::= NK_STAR */ + 502, /* (587) star_func_para_list ::= other_para_list */ + 504, /* (588) other_para_list ::= star_func_para */ + 504, /* (589) other_para_list ::= other_para_list NK_COMMA star_func_para */ + 505, /* (590) star_func_para ::= expr_or_subquery */ + 505, /* (591) star_func_para ::= table_name NK_DOT NK_STAR */ + 500, /* (592) case_when_expression ::= CASE when_then_list case_when_else_opt END */ + 500, /* (593) case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END */ + 506, /* (594) when_then_list ::= when_then_expr */ + 506, /* (595) when_then_list ::= when_then_list when_then_expr */ + 509, /* (596) when_then_expr ::= WHEN common_expression THEN common_expression */ + 507, /* (597) case_when_else_opt ::= */ + 507, /* (598) case_when_else_opt ::= ELSE common_expression */ + 510, /* (599) predicate ::= expr_or_subquery compare_op expr_or_subquery */ + 510, /* (600) predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery */ + 510, /* (601) predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery */ + 510, /* (602) predicate ::= expr_or_subquery IS NULL */ + 510, /* (603) predicate ::= expr_or_subquery IS NOT NULL */ + 510, /* (604) predicate ::= expr_or_subquery in_op in_predicate_value */ + 511, /* (605) compare_op ::= NK_LT */ + 511, /* (606) compare_op ::= NK_GT */ + 511, /* (607) compare_op ::= NK_LE */ + 511, /* (608) compare_op ::= NK_GE */ + 511, /* (609) compare_op ::= NK_NE */ + 511, /* (610) compare_op ::= NK_EQ */ + 511, /* (611) compare_op ::= LIKE */ + 511, /* (612) compare_op ::= NOT LIKE */ + 511, /* (613) compare_op ::= MATCH */ + 511, /* (614) compare_op ::= NMATCH */ + 511, /* (615) compare_op ::= CONTAINS */ + 512, /* (616) in_op ::= IN */ + 512, /* (617) in_op ::= NOT IN */ + 513, /* (618) in_predicate_value ::= NK_LP literal_list NK_RP */ + 514, /* (619) boolean_value_expression ::= boolean_primary */ + 514, /* (620) boolean_value_expression ::= NOT boolean_primary */ + 514, /* (621) boolean_value_expression ::= boolean_value_expression OR boolean_value_expression */ + 514, /* (622) boolean_value_expression ::= boolean_value_expression AND boolean_value_expression */ + 515, /* (623) boolean_primary ::= predicate */ + 515, /* (624) boolean_primary ::= NK_LP boolean_value_expression NK_RP */ + 508, /* (625) common_expression ::= expr_or_subquery */ + 508, /* (626) common_expression ::= boolean_value_expression */ + 516, /* (627) from_clause_opt ::= */ + 516, /* (628) from_clause_opt ::= FROM table_reference_list */ + 517, /* (629) table_reference_list ::= table_reference */ + 517, /* (630) table_reference_list ::= table_reference_list NK_COMMA table_reference */ + 518, /* (631) table_reference ::= table_primary */ + 518, /* (632) table_reference ::= joined_table */ + 519, /* (633) table_primary ::= table_name alias_opt */ + 519, /* (634) table_primary ::= db_name NK_DOT table_name alias_opt */ + 519, /* (635) table_primary ::= subquery alias_opt */ + 519, /* (636) table_primary ::= parenthesized_joined_table */ + 521, /* (637) alias_opt ::= */ + 521, /* (638) alias_opt ::= table_alias */ + 521, /* (639) alias_opt ::= AS table_alias */ + 523, /* (640) parenthesized_joined_table ::= NK_LP joined_table NK_RP */ + 523, /* (641) parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP */ + 520, /* (642) joined_table ::= table_reference join_type join_subtype JOIN table_reference join_on_clause_opt window_offset_clause_opt jlimit_clause_opt */ + 524, /* (643) join_type ::= */ + 524, /* (644) join_type ::= INNER */ + 524, /* (645) join_type ::= LEFT */ + 524, /* (646) join_type ::= RIGHT */ + 524, /* (647) join_type ::= FULL */ + 525, /* (648) join_subtype ::= */ + 525, /* (649) join_subtype ::= OUTER */ + 525, /* (650) join_subtype ::= SEMI */ + 525, /* (651) join_subtype ::= ANTI */ + 525, /* (652) join_subtype ::= ASOF */ + 525, /* (653) join_subtype ::= WINDOW */ + 526, /* (654) join_on_clause_opt ::= */ + 526, /* (655) join_on_clause_opt ::= ON search_condition */ + 527, /* (656) window_offset_clause_opt ::= */ + 527, /* (657) window_offset_clause_opt ::= WINDOW_OFFSET NK_LP window_offset_literal NK_COMMA window_offset_literal NK_RP */ + 529, /* (658) window_offset_literal ::= NK_VARIABLE */ + 529, /* (659) window_offset_literal ::= NK_MINUS NK_VARIABLE */ + 528, /* (660) jlimit_clause_opt ::= */ + 528, /* (661) jlimit_clause_opt ::= JLIMIT NK_INTEGER */ + 530, /* (662) query_specification ::= SELECT hint_list set_quantifier_opt tag_mode_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 */ + 531, /* (663) hint_list ::= */ + 531, /* (664) hint_list ::= NK_HINT */ + 533, /* (665) tag_mode_opt ::= */ + 533, /* (666) tag_mode_opt ::= TAGS */ + 532, /* (667) set_quantifier_opt ::= */ + 532, /* (668) set_quantifier_opt ::= DISTINCT */ + 532, /* (669) set_quantifier_opt ::= ALL */ + 534, /* (670) select_list ::= select_item */ + 534, /* (671) select_list ::= select_list NK_COMMA select_item */ + 542, /* (672) select_item ::= NK_STAR */ + 542, /* (673) select_item ::= common_expression */ + 542, /* (674) select_item ::= common_expression column_alias */ + 542, /* (675) select_item ::= common_expression AS column_alias */ + 542, /* (676) select_item ::= table_name NK_DOT NK_STAR */ + 469, /* (677) where_clause_opt ::= */ + 469, /* (678) where_clause_opt ::= WHERE search_condition */ + 535, /* (679) partition_by_clause_opt ::= */ + 535, /* (680) partition_by_clause_opt ::= PARTITION BY partition_list */ + 543, /* (681) partition_list ::= partition_item */ + 543, /* (682) partition_list ::= partition_list NK_COMMA partition_item */ + 544, /* (683) partition_item ::= expr_or_subquery */ + 544, /* (684) partition_item ::= expr_or_subquery column_alias */ + 544, /* (685) partition_item ::= expr_or_subquery AS column_alias */ + 539, /* (686) twindow_clause_opt ::= */ + 539, /* (687) twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA interval_sliding_duration_literal NK_RP */ + 539, /* (688) twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP */ + 539, /* (689) twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_RP sliding_opt fill_opt */ + 539, /* (690) twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_COMMA interval_sliding_duration_literal NK_RP sliding_opt fill_opt */ + 539, /* (691) twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition */ + 539, /* (692) twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_RP */ + 539, /* (693) twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP */ + 462, /* (694) sliding_opt ::= */ + 462, /* (695) sliding_opt ::= SLIDING NK_LP interval_sliding_duration_literal NK_RP */ + 545, /* (696) interval_sliding_duration_literal ::= NK_VARIABLE */ + 545, /* (697) interval_sliding_duration_literal ::= NK_STRING */ + 545, /* (698) interval_sliding_duration_literal ::= NK_INTEGER */ + 538, /* (699) fill_opt ::= */ + 538, /* (700) fill_opt ::= FILL NK_LP fill_mode NK_RP */ + 538, /* (701) fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP */ + 538, /* (702) fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP */ + 546, /* (703) fill_mode ::= NONE */ + 546, /* (704) fill_mode ::= PREV */ + 546, /* (705) fill_mode ::= NULL */ + 546, /* (706) fill_mode ::= NULL_F */ + 546, /* (707) fill_mode ::= LINEAR */ + 546, /* (708) fill_mode ::= NEXT */ + 540, /* (709) group_by_clause_opt ::= */ + 540, /* (710) group_by_clause_opt ::= GROUP BY group_by_list */ + 547, /* (711) group_by_list ::= expr_or_subquery */ + 547, /* (712) group_by_list ::= group_by_list NK_COMMA expr_or_subquery */ + 541, /* (713) having_clause_opt ::= */ + 541, /* (714) having_clause_opt ::= HAVING search_condition */ + 536, /* (715) range_opt ::= */ + 536, /* (716) range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP */ + 536, /* (717) range_opt ::= RANGE NK_LP expr_or_subquery NK_RP */ + 537, /* (718) every_opt ::= */ + 537, /* (719) every_opt ::= EVERY NK_LP duration_literal NK_RP */ + 548, /* (720) query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt */ + 549, /* (721) query_simple ::= query_specification */ + 549, /* (722) query_simple ::= union_query_expression */ + 553, /* (723) union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery */ + 553, /* (724) union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery */ + 554, /* (725) query_simple_or_subquery ::= query_simple */ + 554, /* (726) query_simple_or_subquery ::= subquery */ + 468, /* (727) query_or_subquery ::= query_expression */ + 468, /* (728) query_or_subquery ::= subquery */ + 550, /* (729) order_by_clause_opt ::= */ + 550, /* (730) order_by_clause_opt ::= ORDER BY sort_specification_list */ + 551, /* (731) slimit_clause_opt ::= */ + 551, /* (732) slimit_clause_opt ::= SLIMIT NK_INTEGER */ + 551, /* (733) slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER */ + 551, /* (734) slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER */ + 552, /* (735) limit_clause_opt ::= */ + 552, /* (736) limit_clause_opt ::= LIMIT NK_INTEGER */ + 552, /* (737) limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER */ + 552, /* (738) limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER */ + 522, /* (739) subquery ::= NK_LP query_expression NK_RP */ + 522, /* (740) subquery ::= NK_LP subquery NK_RP */ + 399, /* (741) search_condition ::= common_expression */ + 555, /* (742) sort_specification_list ::= sort_specification */ + 555, /* (743) sort_specification_list ::= sort_specification_list NK_COMMA sort_specification */ + 556, /* (744) sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt */ + 557, /* (745) ordering_specification_opt ::= */ + 557, /* (746) ordering_specification_opt ::= ASC */ + 557, /* (747) ordering_specification_opt ::= DESC */ + 558, /* (748) null_ordering_opt ::= */ + 558, /* (749) null_ordering_opt ::= NULLS FIRST */ + 558, /* (750) null_ordering_opt ::= NULLS LAST */ + 428, /* (751) column_options ::= */ + 428, /* (752) column_options ::= column_options PRIMARY KEY */ + 428, /* (753) column_options ::= column_options ENCODE NK_STRING */ + 428, /* (754) column_options ::= column_options COMPRESS NK_STRING */ + 428, /* (755) column_options ::= column_options LEVEL NK_STRING */ }; /* For rule J, yyRuleInfoNRhs[J] contains the negative of the number @@ -5186,340 +5186,341 @@ static const signed char yyRuleInfoNRhs[] = { 0, /* (418) stream_options ::= */ -3, /* (419) stream_options ::= stream_options TRIGGER AT_ONCE */ -3, /* (420) stream_options ::= stream_options TRIGGER WINDOW_CLOSE */ - -4, /* (421) stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal */ - -3, /* (422) stream_options ::= stream_options WATERMARK duration_literal */ - -4, /* (423) stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER */ - -3, /* (424) stream_options ::= stream_options FILL_HISTORY NK_INTEGER */ - -3, /* (425) stream_options ::= stream_options DELETE_MARK duration_literal */ - -4, /* (426) stream_options ::= stream_options IGNORE UPDATE NK_INTEGER */ - 0, /* (427) subtable_opt ::= */ - -4, /* (428) subtable_opt ::= SUBTABLE NK_LP expression NK_RP */ - 0, /* (429) ignore_opt ::= */ - -2, /* (430) ignore_opt ::= IGNORE UNTREATED */ - -3, /* (431) cmd ::= KILL CONNECTION NK_INTEGER */ - -3, /* (432) cmd ::= KILL QUERY NK_STRING */ - -3, /* (433) cmd ::= KILL TRANSACTION NK_INTEGER */ - -3, /* (434) cmd ::= KILL COMPACT NK_INTEGER */ - -2, /* (435) cmd ::= BALANCE VGROUP */ - -4, /* (436) cmd ::= BALANCE VGROUP LEADER on_vgroup_id */ - -5, /* (437) cmd ::= BALANCE VGROUP LEADER DATABASE db_name */ - -4, /* (438) cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER */ - -4, /* (439) cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list */ - -3, /* (440) cmd ::= SPLIT VGROUP NK_INTEGER */ - 0, /* (441) on_vgroup_id ::= */ - -2, /* (442) on_vgroup_id ::= ON NK_INTEGER */ - -2, /* (443) dnode_list ::= DNODE NK_INTEGER */ - -3, /* (444) dnode_list ::= dnode_list DNODE NK_INTEGER */ - -4, /* (445) cmd ::= DELETE FROM full_table_name where_clause_opt */ - -1, /* (446) cmd ::= query_or_subquery */ - -1, /* (447) cmd ::= insert_query */ - -7, /* (448) insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery */ - -4, /* (449) insert_query ::= INSERT INTO full_table_name query_or_subquery */ - -1, /* (450) tags_literal ::= NK_INTEGER */ - -3, /* (451) tags_literal ::= NK_INTEGER NK_PLUS duration_literal */ - -3, /* (452) tags_literal ::= NK_INTEGER NK_MINUS duration_literal */ - -2, /* (453) tags_literal ::= NK_PLUS NK_INTEGER */ - -4, /* (454) tags_literal ::= NK_PLUS NK_INTEGER NK_PLUS duration_literal */ - -4, /* (455) tags_literal ::= NK_PLUS NK_INTEGER NK_MINUS duration_literal */ - -2, /* (456) tags_literal ::= NK_MINUS NK_INTEGER */ - -4, /* (457) tags_literal ::= NK_MINUS NK_INTEGER NK_PLUS duration_literal */ - -4, /* (458) tags_literal ::= NK_MINUS NK_INTEGER NK_MINUS duration_literal */ - -1, /* (459) tags_literal ::= NK_FLOAT */ - -2, /* (460) tags_literal ::= NK_PLUS NK_FLOAT */ - -2, /* (461) tags_literal ::= NK_MINUS NK_FLOAT */ - -1, /* (462) tags_literal ::= NK_BIN */ - -3, /* (463) tags_literal ::= NK_BIN NK_PLUS duration_literal */ - -3, /* (464) tags_literal ::= NK_BIN NK_MINUS duration_literal */ - -2, /* (465) tags_literal ::= NK_PLUS NK_BIN */ - -4, /* (466) tags_literal ::= NK_PLUS NK_BIN NK_PLUS duration_literal */ - -4, /* (467) tags_literal ::= NK_PLUS NK_BIN NK_MINUS duration_literal */ - -2, /* (468) tags_literal ::= NK_MINUS NK_BIN */ - -4, /* (469) tags_literal ::= NK_MINUS NK_BIN NK_PLUS duration_literal */ - -4, /* (470) tags_literal ::= NK_MINUS NK_BIN NK_MINUS duration_literal */ - -1, /* (471) tags_literal ::= NK_HEX */ - -3, /* (472) tags_literal ::= NK_HEX NK_PLUS duration_literal */ - -3, /* (473) tags_literal ::= NK_HEX NK_MINUS duration_literal */ - -2, /* (474) tags_literal ::= NK_PLUS NK_HEX */ - -4, /* (475) tags_literal ::= NK_PLUS NK_HEX NK_PLUS duration_literal */ - -4, /* (476) tags_literal ::= NK_PLUS NK_HEX NK_MINUS duration_literal */ - -2, /* (477) tags_literal ::= NK_MINUS NK_HEX */ - -4, /* (478) tags_literal ::= NK_MINUS NK_HEX NK_PLUS duration_literal */ - -4, /* (479) tags_literal ::= NK_MINUS NK_HEX NK_MINUS duration_literal */ - -1, /* (480) tags_literal ::= NK_STRING */ - -3, /* (481) tags_literal ::= NK_STRING NK_PLUS duration_literal */ - -3, /* (482) tags_literal ::= NK_STRING NK_MINUS duration_literal */ - -1, /* (483) tags_literal ::= NK_BOOL */ - -1, /* (484) tags_literal ::= NULL */ - -1, /* (485) tags_literal ::= literal_func */ - -3, /* (486) tags_literal ::= literal_func NK_PLUS duration_literal */ - -3, /* (487) tags_literal ::= literal_func NK_MINUS duration_literal */ - -1, /* (488) tags_literal_list ::= tags_literal */ - -3, /* (489) tags_literal_list ::= tags_literal_list NK_COMMA tags_literal */ - -1, /* (490) literal ::= NK_INTEGER */ - -1, /* (491) literal ::= NK_FLOAT */ - -1, /* (492) literal ::= NK_STRING */ - -1, /* (493) literal ::= NK_BOOL */ - -2, /* (494) literal ::= TIMESTAMP NK_STRING */ - -1, /* (495) literal ::= duration_literal */ - -1, /* (496) literal ::= NULL */ - -1, /* (497) literal ::= NK_QUESTION */ - -1, /* (498) duration_literal ::= NK_VARIABLE */ - -1, /* (499) signed ::= NK_INTEGER */ - -2, /* (500) signed ::= NK_PLUS NK_INTEGER */ - -2, /* (501) signed ::= NK_MINUS NK_INTEGER */ - -1, /* (502) signed ::= NK_FLOAT */ - -2, /* (503) signed ::= NK_PLUS NK_FLOAT */ - -2, /* (504) signed ::= NK_MINUS NK_FLOAT */ - -1, /* (505) signed_literal ::= signed */ - -1, /* (506) signed_literal ::= NK_STRING */ - -1, /* (507) signed_literal ::= NK_BOOL */ - -2, /* (508) signed_literal ::= TIMESTAMP NK_STRING */ - -1, /* (509) signed_literal ::= duration_literal */ - -1, /* (510) signed_literal ::= NULL */ - -1, /* (511) signed_literal ::= literal_func */ - -1, /* (512) signed_literal ::= NK_QUESTION */ - -1, /* (513) literal_list ::= signed_literal */ - -3, /* (514) literal_list ::= literal_list NK_COMMA signed_literal */ - -1, /* (515) db_name ::= NK_ID */ - -1, /* (516) table_name ::= NK_ID */ - -1, /* (517) column_name ::= NK_ID */ - -1, /* (518) function_name ::= NK_ID */ - -1, /* (519) view_name ::= NK_ID */ - -1, /* (520) table_alias ::= NK_ID */ - -1, /* (521) column_alias ::= NK_ID */ - -1, /* (522) column_alias ::= NK_ALIAS */ - -1, /* (523) user_name ::= NK_ID */ - -1, /* (524) topic_name ::= NK_ID */ - -1, /* (525) stream_name ::= NK_ID */ - -1, /* (526) cgroup_name ::= NK_ID */ - -1, /* (527) index_name ::= NK_ID */ - -1, /* (528) tsma_name ::= NK_ID */ - -1, /* (529) expr_or_subquery ::= expression */ - -1, /* (530) expression ::= literal */ - -1, /* (531) expression ::= pseudo_column */ - -1, /* (532) expression ::= column_reference */ - -1, /* (533) expression ::= function_expression */ - -1, /* (534) expression ::= case_when_expression */ - -3, /* (535) expression ::= NK_LP expression NK_RP */ - -2, /* (536) expression ::= NK_PLUS expr_or_subquery */ - -2, /* (537) expression ::= NK_MINUS expr_or_subquery */ - -3, /* (538) expression ::= expr_or_subquery NK_PLUS expr_or_subquery */ - -3, /* (539) expression ::= expr_or_subquery NK_MINUS expr_or_subquery */ - -3, /* (540) expression ::= expr_or_subquery NK_STAR expr_or_subquery */ - -3, /* (541) expression ::= expr_or_subquery NK_SLASH expr_or_subquery */ - -3, /* (542) expression ::= expr_or_subquery NK_REM expr_or_subquery */ - -3, /* (543) expression ::= column_reference NK_ARROW NK_STRING */ - -3, /* (544) expression ::= expr_or_subquery NK_BITAND expr_or_subquery */ - -3, /* (545) expression ::= expr_or_subquery NK_BITOR expr_or_subquery */ - -1, /* (546) expression_list ::= expr_or_subquery */ - -3, /* (547) expression_list ::= expression_list NK_COMMA expr_or_subquery */ - -1, /* (548) column_reference ::= column_name */ - -3, /* (549) column_reference ::= table_name NK_DOT column_name */ - -1, /* (550) column_reference ::= NK_ALIAS */ - -3, /* (551) column_reference ::= table_name NK_DOT NK_ALIAS */ - -1, /* (552) pseudo_column ::= ROWTS */ - -1, /* (553) pseudo_column ::= TBNAME */ - -3, /* (554) pseudo_column ::= table_name NK_DOT TBNAME */ - -1, /* (555) pseudo_column ::= QSTART */ - -1, /* (556) pseudo_column ::= QEND */ - -1, /* (557) pseudo_column ::= QDURATION */ - -1, /* (558) pseudo_column ::= WSTART */ - -1, /* (559) pseudo_column ::= WEND */ - -1, /* (560) pseudo_column ::= WDURATION */ - -1, /* (561) pseudo_column ::= IROWTS */ - -1, /* (562) pseudo_column ::= ISFILLED */ - -1, /* (563) pseudo_column ::= QTAGS */ - -4, /* (564) function_expression ::= function_name NK_LP expression_list NK_RP */ - -4, /* (565) function_expression ::= star_func NK_LP star_func_para_list NK_RP */ - -6, /* (566) function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP */ - -6, /* (567) function_expression ::= CAST NK_LP expr_or_subquery AS type_name_default_len NK_RP */ - -1, /* (568) function_expression ::= literal_func */ - -3, /* (569) literal_func ::= noarg_func NK_LP NK_RP */ - -1, /* (570) literal_func ::= NOW */ - -1, /* (571) literal_func ::= TODAY */ - -1, /* (572) noarg_func ::= NOW */ - -1, /* (573) noarg_func ::= TODAY */ - -1, /* (574) noarg_func ::= TIMEZONE */ - -1, /* (575) noarg_func ::= DATABASE */ - -1, /* (576) noarg_func ::= CLIENT_VERSION */ - -1, /* (577) noarg_func ::= SERVER_VERSION */ - -1, /* (578) noarg_func ::= SERVER_STATUS */ - -1, /* (579) noarg_func ::= CURRENT_USER */ - -1, /* (580) noarg_func ::= USER */ - -1, /* (581) star_func ::= COUNT */ - -1, /* (582) star_func ::= FIRST */ - -1, /* (583) star_func ::= LAST */ - -1, /* (584) star_func ::= LAST_ROW */ - -1, /* (585) star_func_para_list ::= NK_STAR */ - -1, /* (586) star_func_para_list ::= other_para_list */ - -1, /* (587) other_para_list ::= star_func_para */ - -3, /* (588) other_para_list ::= other_para_list NK_COMMA star_func_para */ - -1, /* (589) star_func_para ::= expr_or_subquery */ - -3, /* (590) star_func_para ::= table_name NK_DOT NK_STAR */ - -4, /* (591) case_when_expression ::= CASE when_then_list case_when_else_opt END */ - -5, /* (592) case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END */ - -1, /* (593) when_then_list ::= when_then_expr */ - -2, /* (594) when_then_list ::= when_then_list when_then_expr */ - -4, /* (595) when_then_expr ::= WHEN common_expression THEN common_expression */ - 0, /* (596) case_when_else_opt ::= */ - -2, /* (597) case_when_else_opt ::= ELSE common_expression */ - -3, /* (598) predicate ::= expr_or_subquery compare_op expr_or_subquery */ - -5, /* (599) predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery */ - -6, /* (600) predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery */ - -3, /* (601) predicate ::= expr_or_subquery IS NULL */ - -4, /* (602) predicate ::= expr_or_subquery IS NOT NULL */ - -3, /* (603) predicate ::= expr_or_subquery in_op in_predicate_value */ - -1, /* (604) compare_op ::= NK_LT */ - -1, /* (605) compare_op ::= NK_GT */ - -1, /* (606) compare_op ::= NK_LE */ - -1, /* (607) compare_op ::= NK_GE */ - -1, /* (608) compare_op ::= NK_NE */ - -1, /* (609) compare_op ::= NK_EQ */ - -1, /* (610) compare_op ::= LIKE */ - -2, /* (611) compare_op ::= NOT LIKE */ - -1, /* (612) compare_op ::= MATCH */ - -1, /* (613) compare_op ::= NMATCH */ - -1, /* (614) compare_op ::= CONTAINS */ - -1, /* (615) in_op ::= IN */ - -2, /* (616) in_op ::= NOT IN */ - -3, /* (617) in_predicate_value ::= NK_LP literal_list NK_RP */ - -1, /* (618) boolean_value_expression ::= boolean_primary */ - -2, /* (619) boolean_value_expression ::= NOT boolean_primary */ - -3, /* (620) boolean_value_expression ::= boolean_value_expression OR boolean_value_expression */ - -3, /* (621) boolean_value_expression ::= boolean_value_expression AND boolean_value_expression */ - -1, /* (622) boolean_primary ::= predicate */ - -3, /* (623) boolean_primary ::= NK_LP boolean_value_expression NK_RP */ - -1, /* (624) common_expression ::= expr_or_subquery */ - -1, /* (625) common_expression ::= boolean_value_expression */ - 0, /* (626) from_clause_opt ::= */ - -2, /* (627) from_clause_opt ::= FROM table_reference_list */ - -1, /* (628) table_reference_list ::= table_reference */ - -3, /* (629) table_reference_list ::= table_reference_list NK_COMMA table_reference */ - -1, /* (630) table_reference ::= table_primary */ - -1, /* (631) table_reference ::= joined_table */ - -2, /* (632) table_primary ::= table_name alias_opt */ - -4, /* (633) table_primary ::= db_name NK_DOT table_name alias_opt */ - -2, /* (634) table_primary ::= subquery alias_opt */ - -1, /* (635) table_primary ::= parenthesized_joined_table */ - 0, /* (636) alias_opt ::= */ - -1, /* (637) alias_opt ::= table_alias */ - -2, /* (638) alias_opt ::= AS table_alias */ - -3, /* (639) parenthesized_joined_table ::= NK_LP joined_table NK_RP */ - -3, /* (640) parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP */ - -8, /* (641) joined_table ::= table_reference join_type join_subtype JOIN table_reference join_on_clause_opt window_offset_clause_opt jlimit_clause_opt */ - 0, /* (642) join_type ::= */ - -1, /* (643) join_type ::= INNER */ - -1, /* (644) join_type ::= LEFT */ - -1, /* (645) join_type ::= RIGHT */ - -1, /* (646) join_type ::= FULL */ - 0, /* (647) join_subtype ::= */ - -1, /* (648) join_subtype ::= OUTER */ - -1, /* (649) join_subtype ::= SEMI */ - -1, /* (650) join_subtype ::= ANTI */ - -1, /* (651) join_subtype ::= ASOF */ - -1, /* (652) join_subtype ::= WINDOW */ - 0, /* (653) join_on_clause_opt ::= */ - -2, /* (654) join_on_clause_opt ::= ON search_condition */ - 0, /* (655) window_offset_clause_opt ::= */ - -6, /* (656) window_offset_clause_opt ::= WINDOW_OFFSET NK_LP window_offset_literal NK_COMMA window_offset_literal NK_RP */ - -1, /* (657) window_offset_literal ::= NK_VARIABLE */ - -2, /* (658) window_offset_literal ::= NK_MINUS NK_VARIABLE */ - 0, /* (659) jlimit_clause_opt ::= */ - -2, /* (660) jlimit_clause_opt ::= JLIMIT NK_INTEGER */ - -14, /* (661) query_specification ::= SELECT hint_list set_quantifier_opt tag_mode_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, /* (662) hint_list ::= */ - -1, /* (663) hint_list ::= NK_HINT */ - 0, /* (664) tag_mode_opt ::= */ - -1, /* (665) tag_mode_opt ::= TAGS */ - 0, /* (666) set_quantifier_opt ::= */ - -1, /* (667) set_quantifier_opt ::= DISTINCT */ - -1, /* (668) set_quantifier_opt ::= ALL */ - -1, /* (669) select_list ::= select_item */ - -3, /* (670) select_list ::= select_list NK_COMMA select_item */ - -1, /* (671) select_item ::= NK_STAR */ - -1, /* (672) select_item ::= common_expression */ - -2, /* (673) select_item ::= common_expression column_alias */ - -3, /* (674) select_item ::= common_expression AS column_alias */ - -3, /* (675) select_item ::= table_name NK_DOT NK_STAR */ - 0, /* (676) where_clause_opt ::= */ - -2, /* (677) where_clause_opt ::= WHERE search_condition */ - 0, /* (678) partition_by_clause_opt ::= */ - -3, /* (679) partition_by_clause_opt ::= PARTITION BY partition_list */ - -1, /* (680) partition_list ::= partition_item */ - -3, /* (681) partition_list ::= partition_list NK_COMMA partition_item */ - -1, /* (682) partition_item ::= expr_or_subquery */ - -2, /* (683) partition_item ::= expr_or_subquery column_alias */ - -3, /* (684) partition_item ::= expr_or_subquery AS column_alias */ - 0, /* (685) twindow_clause_opt ::= */ - -6, /* (686) twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA interval_sliding_duration_literal NK_RP */ - -4, /* (687) twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP */ - -6, /* (688) twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_RP sliding_opt fill_opt */ - -8, /* (689) twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_COMMA interval_sliding_duration_literal NK_RP sliding_opt fill_opt */ - -7, /* (690) twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition */ - -4, /* (691) twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_RP */ - -6, /* (692) twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP */ - 0, /* (693) sliding_opt ::= */ - -4, /* (694) sliding_opt ::= SLIDING NK_LP interval_sliding_duration_literal NK_RP */ - -1, /* (695) interval_sliding_duration_literal ::= NK_VARIABLE */ - -1, /* (696) interval_sliding_duration_literal ::= NK_STRING */ - -1, /* (697) interval_sliding_duration_literal ::= NK_INTEGER */ - 0, /* (698) fill_opt ::= */ - -4, /* (699) fill_opt ::= FILL NK_LP fill_mode NK_RP */ - -6, /* (700) fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP */ - -6, /* (701) fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP */ - -1, /* (702) fill_mode ::= NONE */ - -1, /* (703) fill_mode ::= PREV */ - -1, /* (704) fill_mode ::= NULL */ - -1, /* (705) fill_mode ::= NULL_F */ - -1, /* (706) fill_mode ::= LINEAR */ - -1, /* (707) fill_mode ::= NEXT */ - 0, /* (708) group_by_clause_opt ::= */ - -3, /* (709) group_by_clause_opt ::= GROUP BY group_by_list */ - -1, /* (710) group_by_list ::= expr_or_subquery */ - -3, /* (711) group_by_list ::= group_by_list NK_COMMA expr_or_subquery */ - 0, /* (712) having_clause_opt ::= */ - -2, /* (713) having_clause_opt ::= HAVING search_condition */ - 0, /* (714) range_opt ::= */ - -6, /* (715) range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP */ - -4, /* (716) range_opt ::= RANGE NK_LP expr_or_subquery NK_RP */ - 0, /* (717) every_opt ::= */ - -4, /* (718) every_opt ::= EVERY NK_LP duration_literal NK_RP */ - -4, /* (719) query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt */ - -1, /* (720) query_simple ::= query_specification */ - -1, /* (721) query_simple ::= union_query_expression */ - -4, /* (722) union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery */ - -3, /* (723) union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery */ - -1, /* (724) query_simple_or_subquery ::= query_simple */ - -1, /* (725) query_simple_or_subquery ::= subquery */ - -1, /* (726) query_or_subquery ::= query_expression */ - -1, /* (727) query_or_subquery ::= subquery */ - 0, /* (728) order_by_clause_opt ::= */ - -3, /* (729) order_by_clause_opt ::= ORDER BY sort_specification_list */ - 0, /* (730) slimit_clause_opt ::= */ - -2, /* (731) slimit_clause_opt ::= SLIMIT NK_INTEGER */ - -4, /* (732) slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER */ - -4, /* (733) slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER */ - 0, /* (734) limit_clause_opt ::= */ - -2, /* (735) limit_clause_opt ::= LIMIT NK_INTEGER */ - -4, /* (736) limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER */ - -4, /* (737) limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER */ - -3, /* (738) subquery ::= NK_LP query_expression NK_RP */ - -3, /* (739) subquery ::= NK_LP subquery NK_RP */ - -1, /* (740) search_condition ::= common_expression */ - -1, /* (741) sort_specification_list ::= sort_specification */ - -3, /* (742) sort_specification_list ::= sort_specification_list NK_COMMA sort_specification */ - -3, /* (743) sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt */ - 0, /* (744) ordering_specification_opt ::= */ - -1, /* (745) ordering_specification_opt ::= ASC */ - -1, /* (746) ordering_specification_opt ::= DESC */ - 0, /* (747) null_ordering_opt ::= */ - -2, /* (748) null_ordering_opt ::= NULLS FIRST */ - -2, /* (749) null_ordering_opt ::= NULLS LAST */ - 0, /* (750) column_options ::= */ - -3, /* (751) column_options ::= column_options PRIMARY KEY */ - -3, /* (752) column_options ::= column_options ENCODE NK_STRING */ - -3, /* (753) column_options ::= column_options COMPRESS NK_STRING */ - -3, /* (754) column_options ::= column_options LEVEL NK_STRING */ + -3, /* (421) stream_options ::= stream_options TRIGGER FORCE_WINDOW_CLOSE */ + -4, /* (422) stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal */ + -3, /* (423) stream_options ::= stream_options WATERMARK duration_literal */ + -4, /* (424) stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER */ + -3, /* (425) stream_options ::= stream_options FILL_HISTORY NK_INTEGER */ + -3, /* (426) stream_options ::= stream_options DELETE_MARK duration_literal */ + -4, /* (427) stream_options ::= stream_options IGNORE UPDATE NK_INTEGER */ + 0, /* (428) subtable_opt ::= */ + -4, /* (429) subtable_opt ::= SUBTABLE NK_LP expression NK_RP */ + 0, /* (430) ignore_opt ::= */ + -2, /* (431) ignore_opt ::= IGNORE UNTREATED */ + -3, /* (432) cmd ::= KILL CONNECTION NK_INTEGER */ + -3, /* (433) cmd ::= KILL QUERY NK_STRING */ + -3, /* (434) cmd ::= KILL TRANSACTION NK_INTEGER */ + -3, /* (435) cmd ::= KILL COMPACT NK_INTEGER */ + -2, /* (436) cmd ::= BALANCE VGROUP */ + -4, /* (437) cmd ::= BALANCE VGROUP LEADER on_vgroup_id */ + -5, /* (438) cmd ::= BALANCE VGROUP LEADER DATABASE db_name */ + -4, /* (439) cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER */ + -4, /* (440) cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list */ + -3, /* (441) cmd ::= SPLIT VGROUP NK_INTEGER */ + 0, /* (442) on_vgroup_id ::= */ + -2, /* (443) on_vgroup_id ::= ON NK_INTEGER */ + -2, /* (444) dnode_list ::= DNODE NK_INTEGER */ + -3, /* (445) dnode_list ::= dnode_list DNODE NK_INTEGER */ + -4, /* (446) cmd ::= DELETE FROM full_table_name where_clause_opt */ + -1, /* (447) cmd ::= query_or_subquery */ + -1, /* (448) cmd ::= insert_query */ + -7, /* (449) insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery */ + -4, /* (450) insert_query ::= INSERT INTO full_table_name query_or_subquery */ + -1, /* (451) tags_literal ::= NK_INTEGER */ + -3, /* (452) tags_literal ::= NK_INTEGER NK_PLUS duration_literal */ + -3, /* (453) tags_literal ::= NK_INTEGER NK_MINUS duration_literal */ + -2, /* (454) tags_literal ::= NK_PLUS NK_INTEGER */ + -4, /* (455) tags_literal ::= NK_PLUS NK_INTEGER NK_PLUS duration_literal */ + -4, /* (456) tags_literal ::= NK_PLUS NK_INTEGER NK_MINUS duration_literal */ + -2, /* (457) tags_literal ::= NK_MINUS NK_INTEGER */ + -4, /* (458) tags_literal ::= NK_MINUS NK_INTEGER NK_PLUS duration_literal */ + -4, /* (459) tags_literal ::= NK_MINUS NK_INTEGER NK_MINUS duration_literal */ + -1, /* (460) tags_literal ::= NK_FLOAT */ + -2, /* (461) tags_literal ::= NK_PLUS NK_FLOAT */ + -2, /* (462) tags_literal ::= NK_MINUS NK_FLOAT */ + -1, /* (463) tags_literal ::= NK_BIN */ + -3, /* (464) tags_literal ::= NK_BIN NK_PLUS duration_literal */ + -3, /* (465) tags_literal ::= NK_BIN NK_MINUS duration_literal */ + -2, /* (466) tags_literal ::= NK_PLUS NK_BIN */ + -4, /* (467) tags_literal ::= NK_PLUS NK_BIN NK_PLUS duration_literal */ + -4, /* (468) tags_literal ::= NK_PLUS NK_BIN NK_MINUS duration_literal */ + -2, /* (469) tags_literal ::= NK_MINUS NK_BIN */ + -4, /* (470) tags_literal ::= NK_MINUS NK_BIN NK_PLUS duration_literal */ + -4, /* (471) tags_literal ::= NK_MINUS NK_BIN NK_MINUS duration_literal */ + -1, /* (472) tags_literal ::= NK_HEX */ + -3, /* (473) tags_literal ::= NK_HEX NK_PLUS duration_literal */ + -3, /* (474) tags_literal ::= NK_HEX NK_MINUS duration_literal */ + -2, /* (475) tags_literal ::= NK_PLUS NK_HEX */ + -4, /* (476) tags_literal ::= NK_PLUS NK_HEX NK_PLUS duration_literal */ + -4, /* (477) tags_literal ::= NK_PLUS NK_HEX NK_MINUS duration_literal */ + -2, /* (478) tags_literal ::= NK_MINUS NK_HEX */ + -4, /* (479) tags_literal ::= NK_MINUS NK_HEX NK_PLUS duration_literal */ + -4, /* (480) tags_literal ::= NK_MINUS NK_HEX NK_MINUS duration_literal */ + -1, /* (481) tags_literal ::= NK_STRING */ + -3, /* (482) tags_literal ::= NK_STRING NK_PLUS duration_literal */ + -3, /* (483) tags_literal ::= NK_STRING NK_MINUS duration_literal */ + -1, /* (484) tags_literal ::= NK_BOOL */ + -1, /* (485) tags_literal ::= NULL */ + -1, /* (486) tags_literal ::= literal_func */ + -3, /* (487) tags_literal ::= literal_func NK_PLUS duration_literal */ + -3, /* (488) tags_literal ::= literal_func NK_MINUS duration_literal */ + -1, /* (489) tags_literal_list ::= tags_literal */ + -3, /* (490) tags_literal_list ::= tags_literal_list NK_COMMA tags_literal */ + -1, /* (491) literal ::= NK_INTEGER */ + -1, /* (492) literal ::= NK_FLOAT */ + -1, /* (493) literal ::= NK_STRING */ + -1, /* (494) literal ::= NK_BOOL */ + -2, /* (495) literal ::= TIMESTAMP NK_STRING */ + -1, /* (496) literal ::= duration_literal */ + -1, /* (497) literal ::= NULL */ + -1, /* (498) literal ::= NK_QUESTION */ + -1, /* (499) duration_literal ::= NK_VARIABLE */ + -1, /* (500) signed ::= NK_INTEGER */ + -2, /* (501) signed ::= NK_PLUS NK_INTEGER */ + -2, /* (502) signed ::= NK_MINUS NK_INTEGER */ + -1, /* (503) signed ::= NK_FLOAT */ + -2, /* (504) signed ::= NK_PLUS NK_FLOAT */ + -2, /* (505) signed ::= NK_MINUS NK_FLOAT */ + -1, /* (506) signed_literal ::= signed */ + -1, /* (507) signed_literal ::= NK_STRING */ + -1, /* (508) signed_literal ::= NK_BOOL */ + -2, /* (509) signed_literal ::= TIMESTAMP NK_STRING */ + -1, /* (510) signed_literal ::= duration_literal */ + -1, /* (511) signed_literal ::= NULL */ + -1, /* (512) signed_literal ::= literal_func */ + -1, /* (513) signed_literal ::= NK_QUESTION */ + -1, /* (514) literal_list ::= signed_literal */ + -3, /* (515) literal_list ::= literal_list NK_COMMA signed_literal */ + -1, /* (516) db_name ::= NK_ID */ + -1, /* (517) table_name ::= NK_ID */ + -1, /* (518) column_name ::= NK_ID */ + -1, /* (519) function_name ::= NK_ID */ + -1, /* (520) view_name ::= NK_ID */ + -1, /* (521) table_alias ::= NK_ID */ + -1, /* (522) column_alias ::= NK_ID */ + -1, /* (523) column_alias ::= NK_ALIAS */ + -1, /* (524) user_name ::= NK_ID */ + -1, /* (525) topic_name ::= NK_ID */ + -1, /* (526) stream_name ::= NK_ID */ + -1, /* (527) cgroup_name ::= NK_ID */ + -1, /* (528) index_name ::= NK_ID */ + -1, /* (529) tsma_name ::= NK_ID */ + -1, /* (530) expr_or_subquery ::= expression */ + -1, /* (531) expression ::= literal */ + -1, /* (532) expression ::= pseudo_column */ + -1, /* (533) expression ::= column_reference */ + -1, /* (534) expression ::= function_expression */ + -1, /* (535) expression ::= case_when_expression */ + -3, /* (536) expression ::= NK_LP expression NK_RP */ + -2, /* (537) expression ::= NK_PLUS expr_or_subquery */ + -2, /* (538) expression ::= NK_MINUS expr_or_subquery */ + -3, /* (539) expression ::= expr_or_subquery NK_PLUS expr_or_subquery */ + -3, /* (540) expression ::= expr_or_subquery NK_MINUS expr_or_subquery */ + -3, /* (541) expression ::= expr_or_subquery NK_STAR expr_or_subquery */ + -3, /* (542) expression ::= expr_or_subquery NK_SLASH expr_or_subquery */ + -3, /* (543) expression ::= expr_or_subquery NK_REM expr_or_subquery */ + -3, /* (544) expression ::= column_reference NK_ARROW NK_STRING */ + -3, /* (545) expression ::= expr_or_subquery NK_BITAND expr_or_subquery */ + -3, /* (546) expression ::= expr_or_subquery NK_BITOR expr_or_subquery */ + -1, /* (547) expression_list ::= expr_or_subquery */ + -3, /* (548) expression_list ::= expression_list NK_COMMA expr_or_subquery */ + -1, /* (549) column_reference ::= column_name */ + -3, /* (550) column_reference ::= table_name NK_DOT column_name */ + -1, /* (551) column_reference ::= NK_ALIAS */ + -3, /* (552) column_reference ::= table_name NK_DOT NK_ALIAS */ + -1, /* (553) pseudo_column ::= ROWTS */ + -1, /* (554) pseudo_column ::= TBNAME */ + -3, /* (555) pseudo_column ::= table_name NK_DOT TBNAME */ + -1, /* (556) pseudo_column ::= QSTART */ + -1, /* (557) pseudo_column ::= QEND */ + -1, /* (558) pseudo_column ::= QDURATION */ + -1, /* (559) pseudo_column ::= WSTART */ + -1, /* (560) pseudo_column ::= WEND */ + -1, /* (561) pseudo_column ::= WDURATION */ + -1, /* (562) pseudo_column ::= IROWTS */ + -1, /* (563) pseudo_column ::= ISFILLED */ + -1, /* (564) pseudo_column ::= QTAGS */ + -4, /* (565) function_expression ::= function_name NK_LP expression_list NK_RP */ + -4, /* (566) function_expression ::= star_func NK_LP star_func_para_list NK_RP */ + -6, /* (567) function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP */ + -6, /* (568) function_expression ::= CAST NK_LP expr_or_subquery AS type_name_default_len NK_RP */ + -1, /* (569) function_expression ::= literal_func */ + -3, /* (570) literal_func ::= noarg_func NK_LP NK_RP */ + -1, /* (571) literal_func ::= NOW */ + -1, /* (572) literal_func ::= TODAY */ + -1, /* (573) noarg_func ::= NOW */ + -1, /* (574) noarg_func ::= TODAY */ + -1, /* (575) noarg_func ::= TIMEZONE */ + -1, /* (576) noarg_func ::= DATABASE */ + -1, /* (577) noarg_func ::= CLIENT_VERSION */ + -1, /* (578) noarg_func ::= SERVER_VERSION */ + -1, /* (579) noarg_func ::= SERVER_STATUS */ + -1, /* (580) noarg_func ::= CURRENT_USER */ + -1, /* (581) noarg_func ::= USER */ + -1, /* (582) star_func ::= COUNT */ + -1, /* (583) star_func ::= FIRST */ + -1, /* (584) star_func ::= LAST */ + -1, /* (585) star_func ::= LAST_ROW */ + -1, /* (586) star_func_para_list ::= NK_STAR */ + -1, /* (587) star_func_para_list ::= other_para_list */ + -1, /* (588) other_para_list ::= star_func_para */ + -3, /* (589) other_para_list ::= other_para_list NK_COMMA star_func_para */ + -1, /* (590) star_func_para ::= expr_or_subquery */ + -3, /* (591) star_func_para ::= table_name NK_DOT NK_STAR */ + -4, /* (592) case_when_expression ::= CASE when_then_list case_when_else_opt END */ + -5, /* (593) case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END */ + -1, /* (594) when_then_list ::= when_then_expr */ + -2, /* (595) when_then_list ::= when_then_list when_then_expr */ + -4, /* (596) when_then_expr ::= WHEN common_expression THEN common_expression */ + 0, /* (597) case_when_else_opt ::= */ + -2, /* (598) case_when_else_opt ::= ELSE common_expression */ + -3, /* (599) predicate ::= expr_or_subquery compare_op expr_or_subquery */ + -5, /* (600) predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery */ + -6, /* (601) predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery */ + -3, /* (602) predicate ::= expr_or_subquery IS NULL */ + -4, /* (603) predicate ::= expr_or_subquery IS NOT NULL */ + -3, /* (604) predicate ::= expr_or_subquery in_op in_predicate_value */ + -1, /* (605) compare_op ::= NK_LT */ + -1, /* (606) compare_op ::= NK_GT */ + -1, /* (607) compare_op ::= NK_LE */ + -1, /* (608) compare_op ::= NK_GE */ + -1, /* (609) compare_op ::= NK_NE */ + -1, /* (610) compare_op ::= NK_EQ */ + -1, /* (611) compare_op ::= LIKE */ + -2, /* (612) compare_op ::= NOT LIKE */ + -1, /* (613) compare_op ::= MATCH */ + -1, /* (614) compare_op ::= NMATCH */ + -1, /* (615) compare_op ::= CONTAINS */ + -1, /* (616) in_op ::= IN */ + -2, /* (617) in_op ::= NOT IN */ + -3, /* (618) in_predicate_value ::= NK_LP literal_list NK_RP */ + -1, /* (619) boolean_value_expression ::= boolean_primary */ + -2, /* (620) boolean_value_expression ::= NOT boolean_primary */ + -3, /* (621) boolean_value_expression ::= boolean_value_expression OR boolean_value_expression */ + -3, /* (622) boolean_value_expression ::= boolean_value_expression AND boolean_value_expression */ + -1, /* (623) boolean_primary ::= predicate */ + -3, /* (624) boolean_primary ::= NK_LP boolean_value_expression NK_RP */ + -1, /* (625) common_expression ::= expr_or_subquery */ + -1, /* (626) common_expression ::= boolean_value_expression */ + 0, /* (627) from_clause_opt ::= */ + -2, /* (628) from_clause_opt ::= FROM table_reference_list */ + -1, /* (629) table_reference_list ::= table_reference */ + -3, /* (630) table_reference_list ::= table_reference_list NK_COMMA table_reference */ + -1, /* (631) table_reference ::= table_primary */ + -1, /* (632) table_reference ::= joined_table */ + -2, /* (633) table_primary ::= table_name alias_opt */ + -4, /* (634) table_primary ::= db_name NK_DOT table_name alias_opt */ + -2, /* (635) table_primary ::= subquery alias_opt */ + -1, /* (636) table_primary ::= parenthesized_joined_table */ + 0, /* (637) alias_opt ::= */ + -1, /* (638) alias_opt ::= table_alias */ + -2, /* (639) alias_opt ::= AS table_alias */ + -3, /* (640) parenthesized_joined_table ::= NK_LP joined_table NK_RP */ + -3, /* (641) parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP */ + -8, /* (642) joined_table ::= table_reference join_type join_subtype JOIN table_reference join_on_clause_opt window_offset_clause_opt jlimit_clause_opt */ + 0, /* (643) join_type ::= */ + -1, /* (644) join_type ::= INNER */ + -1, /* (645) join_type ::= LEFT */ + -1, /* (646) join_type ::= RIGHT */ + -1, /* (647) join_type ::= FULL */ + 0, /* (648) join_subtype ::= */ + -1, /* (649) join_subtype ::= OUTER */ + -1, /* (650) join_subtype ::= SEMI */ + -1, /* (651) join_subtype ::= ANTI */ + -1, /* (652) join_subtype ::= ASOF */ + -1, /* (653) join_subtype ::= WINDOW */ + 0, /* (654) join_on_clause_opt ::= */ + -2, /* (655) join_on_clause_opt ::= ON search_condition */ + 0, /* (656) window_offset_clause_opt ::= */ + -6, /* (657) window_offset_clause_opt ::= WINDOW_OFFSET NK_LP window_offset_literal NK_COMMA window_offset_literal NK_RP */ + -1, /* (658) window_offset_literal ::= NK_VARIABLE */ + -2, /* (659) window_offset_literal ::= NK_MINUS NK_VARIABLE */ + 0, /* (660) jlimit_clause_opt ::= */ + -2, /* (661) jlimit_clause_opt ::= JLIMIT NK_INTEGER */ + -14, /* (662) query_specification ::= SELECT hint_list set_quantifier_opt tag_mode_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, /* (663) hint_list ::= */ + -1, /* (664) hint_list ::= NK_HINT */ + 0, /* (665) tag_mode_opt ::= */ + -1, /* (666) tag_mode_opt ::= TAGS */ + 0, /* (667) set_quantifier_opt ::= */ + -1, /* (668) set_quantifier_opt ::= DISTINCT */ + -1, /* (669) set_quantifier_opt ::= ALL */ + -1, /* (670) select_list ::= select_item */ + -3, /* (671) select_list ::= select_list NK_COMMA select_item */ + -1, /* (672) select_item ::= NK_STAR */ + -1, /* (673) select_item ::= common_expression */ + -2, /* (674) select_item ::= common_expression column_alias */ + -3, /* (675) select_item ::= common_expression AS column_alias */ + -3, /* (676) select_item ::= table_name NK_DOT NK_STAR */ + 0, /* (677) where_clause_opt ::= */ + -2, /* (678) where_clause_opt ::= WHERE search_condition */ + 0, /* (679) partition_by_clause_opt ::= */ + -3, /* (680) partition_by_clause_opt ::= PARTITION BY partition_list */ + -1, /* (681) partition_list ::= partition_item */ + -3, /* (682) partition_list ::= partition_list NK_COMMA partition_item */ + -1, /* (683) partition_item ::= expr_or_subquery */ + -2, /* (684) partition_item ::= expr_or_subquery column_alias */ + -3, /* (685) partition_item ::= expr_or_subquery AS column_alias */ + 0, /* (686) twindow_clause_opt ::= */ + -6, /* (687) twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA interval_sliding_duration_literal NK_RP */ + -4, /* (688) twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP */ + -6, /* (689) twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_RP sliding_opt fill_opt */ + -8, /* (690) twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_COMMA interval_sliding_duration_literal NK_RP sliding_opt fill_opt */ + -7, /* (691) twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition */ + -4, /* (692) twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_RP */ + -6, /* (693) twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP */ + 0, /* (694) sliding_opt ::= */ + -4, /* (695) sliding_opt ::= SLIDING NK_LP interval_sliding_duration_literal NK_RP */ + -1, /* (696) interval_sliding_duration_literal ::= NK_VARIABLE */ + -1, /* (697) interval_sliding_duration_literal ::= NK_STRING */ + -1, /* (698) interval_sliding_duration_literal ::= NK_INTEGER */ + 0, /* (699) fill_opt ::= */ + -4, /* (700) fill_opt ::= FILL NK_LP fill_mode NK_RP */ + -6, /* (701) fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP */ + -6, /* (702) fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP */ + -1, /* (703) fill_mode ::= NONE */ + -1, /* (704) fill_mode ::= PREV */ + -1, /* (705) fill_mode ::= NULL */ + -1, /* (706) fill_mode ::= NULL_F */ + -1, /* (707) fill_mode ::= LINEAR */ + -1, /* (708) fill_mode ::= NEXT */ + 0, /* (709) group_by_clause_opt ::= */ + -3, /* (710) group_by_clause_opt ::= GROUP BY group_by_list */ + -1, /* (711) group_by_list ::= expr_or_subquery */ + -3, /* (712) group_by_list ::= group_by_list NK_COMMA expr_or_subquery */ + 0, /* (713) having_clause_opt ::= */ + -2, /* (714) having_clause_opt ::= HAVING search_condition */ + 0, /* (715) range_opt ::= */ + -6, /* (716) range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP */ + -4, /* (717) range_opt ::= RANGE NK_LP expr_or_subquery NK_RP */ + 0, /* (718) every_opt ::= */ + -4, /* (719) every_opt ::= EVERY NK_LP duration_literal NK_RP */ + -4, /* (720) query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt */ + -1, /* (721) query_simple ::= query_specification */ + -1, /* (722) query_simple ::= union_query_expression */ + -4, /* (723) union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery */ + -3, /* (724) union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery */ + -1, /* (725) query_simple_or_subquery ::= query_simple */ + -1, /* (726) query_simple_or_subquery ::= subquery */ + -1, /* (727) query_or_subquery ::= query_expression */ + -1, /* (728) query_or_subquery ::= subquery */ + 0, /* (729) order_by_clause_opt ::= */ + -3, /* (730) order_by_clause_opt ::= ORDER BY sort_specification_list */ + 0, /* (731) slimit_clause_opt ::= */ + -2, /* (732) slimit_clause_opt ::= SLIMIT NK_INTEGER */ + -4, /* (733) slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER */ + -4, /* (734) slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER */ + 0, /* (735) limit_clause_opt ::= */ + -2, /* (736) limit_clause_opt ::= LIMIT NK_INTEGER */ + -4, /* (737) limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER */ + -4, /* (738) limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER */ + -3, /* (739) subquery ::= NK_LP query_expression NK_RP */ + -3, /* (740) subquery ::= NK_LP subquery NK_RP */ + -1, /* (741) search_condition ::= common_expression */ + -1, /* (742) sort_specification_list ::= sort_specification */ + -3, /* (743) sort_specification_list ::= sort_specification_list NK_COMMA sort_specification */ + -3, /* (744) sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt */ + 0, /* (745) ordering_specification_opt ::= */ + -1, /* (746) ordering_specification_opt ::= ASC */ + -1, /* (747) ordering_specification_opt ::= DESC */ + 0, /* (748) null_ordering_opt ::= */ + -2, /* (749) null_ordering_opt ::= NULLS FIRST */ + -2, /* (750) null_ordering_opt ::= NULLS LAST */ + 0, /* (751) column_options ::= */ + -3, /* (752) column_options ::= column_options PRIMARY KEY */ + -3, /* (753) column_options ::= column_options ENCODE NK_STRING */ + -3, /* (754) column_options ::= column_options COMPRESS NK_STRING */ + -3, /* (755) column_options ::= column_options LEVEL NK_STRING */ }; static void yy_accept(yyParser*); /* Forward Declaration */ @@ -5562,21 +5563,15 @@ 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 5567 "sql.c" - yy_destructor(yypParser,379,&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 5573 "sql.c" yy_destructor(yypParser,380,&yymsp[0].minor); break; + case 1: /* cmd ::= ALTER ACCOUNT NK_ID alter_account_options */ +{ pCxt->errCode = generateSyntaxErrMsg(&pCxt->msgBuf, TSDB_CODE_PAR_EXPRIE_STATEMENT); } + yy_destructor(yypParser,381,&yymsp[0].minor); + break; case 2: /* account_options ::= */ -#line 55 "sql.y" { } -#line 5579 "sql.c" break; case 3: /* account_options ::= account_options PPS literal */ case 4: /* account_options ::= account_options TSERIES literal */ yytestcase(yyruleno==4); @@ -5587,26 +5582,20 @@ static YYACTIONTYPE yy_reduce( case 9: /* account_options ::= account_options USERS literal */ yytestcase(yyruleno==9); 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,379,&yymsp[-2].minor); -#line 56 "sql.y" +{ yy_destructor(yypParser,380,&yymsp[-2].minor); { } -#line 5593 "sql.c" - yy_destructor(yypParser,381,&yymsp[0].minor); + yy_destructor(yypParser,382,&yymsp[0].minor); } break; case 12: /* alter_account_options ::= alter_account_option */ -{ yy_destructor(yypParser,382,&yymsp[0].minor); -#line 68 "sql.y" +{ yy_destructor(yypParser,383,&yymsp[0].minor); { } -#line 5601 "sql.c" } break; case 13: /* alter_account_options ::= alter_account_options alter_account_option */ -{ yy_destructor(yypParser,380,&yymsp[-1].minor); -#line 69 "sql.y" +{ yy_destructor(yypParser,381,&yymsp[-1].minor); { } -#line 5608 "sql.c" - yy_destructor(yypParser,382,&yymsp[0].minor); + yy_destructor(yypParser,383,&yymsp[0].minor); } break; case 14: /* alter_account_option ::= PASS literal */ @@ -5619,27 +5608,19 @@ 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 5624 "sql.c" - yy_destructor(yypParser,381,&yymsp[0].minor); + yy_destructor(yypParser,382,&yymsp[0].minor); break; case 24: /* ip_range_list ::= NK_STRING */ -#line 86 "sql.y" -{ yylhsminor.yy316 = createNodeList(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0)); } -#line 5630 "sql.c" - yymsp[0].minor.yy316 = yylhsminor.yy316; +{ yylhsminor.yy652 = createNodeList(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy652 = yylhsminor.yy652; break; case 25: /* ip_range_list ::= ip_range_list NK_COMMA NK_STRING */ -#line 87 "sql.y" -{ yylhsminor.yy316 = addNodeToList(pCxt, yymsp[-2].minor.yy316, createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0)); } -#line 5636 "sql.c" - yymsp[-2].minor.yy316 = yylhsminor.yy316; +{ yylhsminor.yy652 = addNodeToList(pCxt, yymsp[-2].minor.yy652, createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0)); } + yymsp[-2].minor.yy652 = yylhsminor.yy652; break; case 26: /* white_list ::= HOST ip_range_list */ -#line 91 "sql.y" -{ yymsp[-1].minor.yy316 = yymsp[0].minor.yy316; } -#line 5642 "sql.c" +{ yymsp[-1].minor.yy652 = yymsp[0].minor.yy652; } break; case 27: /* white_list_opt ::= */ case 206: /* specific_cols_opt ::= */ yytestcase(yyruleno==206); @@ -5647,246 +5628,166 @@ static YYACTIONTYPE yy_reduce( case 335: /* tag_list_opt ::= */ yytestcase(yyruleno==335); case 408: /* col_list_opt ::= */ yytestcase(yyruleno==408); case 415: /* tag_def_or_ref_opt ::= */ yytestcase(yyruleno==415); - case 678: /* partition_by_clause_opt ::= */ yytestcase(yyruleno==678); - case 708: /* group_by_clause_opt ::= */ yytestcase(yyruleno==708); - case 728: /* order_by_clause_opt ::= */ yytestcase(yyruleno==728); -#line 95 "sql.y" -{ yymsp[1].minor.yy316 = NULL; } -#line 5655 "sql.c" + case 679: /* partition_by_clause_opt ::= */ yytestcase(yyruleno==679); + case 709: /* group_by_clause_opt ::= */ yytestcase(yyruleno==709); + case 729: /* order_by_clause_opt ::= */ yytestcase(yyruleno==729); +{ yymsp[1].minor.yy652 = NULL; } break; case 28: /* white_list_opt ::= white_list */ case 245: /* tags_def_opt ::= tags_def */ yytestcase(yyruleno==245); case 416: /* tag_def_or_ref_opt ::= tags_def */ yytestcase(yyruleno==416); - case 586: /* star_func_para_list ::= other_para_list */ yytestcase(yyruleno==586); -#line 96 "sql.y" -{ yylhsminor.yy316 = yymsp[0].minor.yy316; } -#line 5663 "sql.c" - yymsp[0].minor.yy316 = yylhsminor.yy316; + case 587: /* star_func_para_list ::= other_para_list */ yytestcase(yyruleno==587); +{ yylhsminor.yy652 = yymsp[0].minor.yy652; } + yymsp[0].minor.yy652 = yylhsminor.yy652; break; case 29: /* is_import_opt ::= */ case 31: /* is_createdb_opt ::= */ yytestcase(yyruleno==31); -#line 100 "sql.y" -{ yymsp[1].minor.yy1043 = 0; } -#line 5670 "sql.c" +{ yymsp[1].minor.yy535 = 0; } break; case 30: /* is_import_opt ::= IS_IMPORT NK_INTEGER */ case 32: /* is_createdb_opt ::= CREATEDB NK_INTEGER */ yytestcase(yyruleno==32); case 42: /* sysinfo_opt ::= SYSINFO NK_INTEGER */ yytestcase(yyruleno==42); -#line 101 "sql.y" -{ yymsp[-1].minor.yy1043 = taosStr2Int8(yymsp[0].minor.yy0.z, NULL, 10); } -#line 5677 "sql.c" +{ yymsp[-1].minor.yy535 = taosStr2Int8(yymsp[0].minor.yy0.z, NULL, 10); } break; case 33: /* cmd ::= CREATE USER user_name PASS NK_STRING sysinfo_opt is_createdb_opt is_import_opt white_list_opt */ -#line 109 "sql.y" { - pCxt->pRootNode = createCreateUserStmt(pCxt, &yymsp[-6].minor.yy1109, &yymsp[-4].minor.yy0, yymsp[-3].minor.yy1043, yymsp[-1].minor.yy1043, yymsp[-2].minor.yy1043); - pCxt->pRootNode = addCreateUserStmtWhiteList(pCxt, pCxt->pRootNode, yymsp[0].minor.yy316); + pCxt->pRootNode = createCreateUserStmt(pCxt, &yymsp[-6].minor.yy837, &yymsp[-4].minor.yy0, yymsp[-3].minor.yy535, yymsp[-1].minor.yy535, yymsp[-2].minor.yy535); + pCxt->pRootNode = addCreateUserStmtWhiteList(pCxt, pCxt->pRootNode, yymsp[0].minor.yy652); } -#line 5685 "sql.c" break; case 34: /* cmd ::= ALTER USER user_name PASS NK_STRING */ -#line 113 "sql.y" -{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy1109, TSDB_ALTER_USER_PASSWD, &yymsp[0].minor.yy0); } -#line 5690 "sql.c" +{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy837, TSDB_ALTER_USER_PASSWD, &yymsp[0].minor.yy0); } break; case 35: /* cmd ::= ALTER USER user_name ENABLE NK_INTEGER */ -#line 114 "sql.y" -{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy1109, TSDB_ALTER_USER_ENABLE, &yymsp[0].minor.yy0); } -#line 5695 "sql.c" +{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy837, TSDB_ALTER_USER_ENABLE, &yymsp[0].minor.yy0); } break; case 36: /* cmd ::= ALTER USER user_name SYSINFO NK_INTEGER */ -#line 115 "sql.y" -{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy1109, TSDB_ALTER_USER_SYSINFO, &yymsp[0].minor.yy0); } -#line 5700 "sql.c" +{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy837, TSDB_ALTER_USER_SYSINFO, &yymsp[0].minor.yy0); } break; case 37: /* cmd ::= ALTER USER user_name CREATEDB NK_INTEGER */ -#line 116 "sql.y" -{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy1109, TSDB_ALTER_USER_CREATEDB, &yymsp[0].minor.yy0); } -#line 5705 "sql.c" +{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy837, TSDB_ALTER_USER_CREATEDB, &yymsp[0].minor.yy0); } break; case 38: /* cmd ::= ALTER USER user_name ADD white_list */ -#line 117 "sql.y" -{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy1109, TSDB_ALTER_USER_ADD_WHITE_LIST, yymsp[0].minor.yy316); } -#line 5710 "sql.c" +{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy837, TSDB_ALTER_USER_ADD_WHITE_LIST, yymsp[0].minor.yy652); } break; case 39: /* cmd ::= ALTER USER user_name DROP white_list */ -#line 118 "sql.y" -{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy1109, TSDB_ALTER_USER_DROP_WHITE_LIST, yymsp[0].minor.yy316); } -#line 5715 "sql.c" +{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy837, TSDB_ALTER_USER_DROP_WHITE_LIST, yymsp[0].minor.yy652); } break; case 40: /* cmd ::= DROP USER user_name */ -#line 119 "sql.y" -{ pCxt->pRootNode = createDropUserStmt(pCxt, &yymsp[0].minor.yy1109); } -#line 5720 "sql.c" +{ pCxt->pRootNode = createDropUserStmt(pCxt, &yymsp[0].minor.yy837); } break; case 41: /* sysinfo_opt ::= */ -#line 123 "sql.y" -{ yymsp[1].minor.yy1043 = 1; } -#line 5725 "sql.c" +{ yymsp[1].minor.yy535 = 1; } break; case 43: /* cmd ::= GRANT privileges ON priv_level with_opt TO user_name */ -#line 127 "sql.y" -{ pCxt->pRootNode = createGrantStmt(pCxt, yymsp[-5].minor.yy1089, &yymsp[-3].minor.yy849, &yymsp[0].minor.yy1109, yymsp[-2].minor.yy416); } -#line 5730 "sql.c" +{ pCxt->pRootNode = createGrantStmt(pCxt, yymsp[-5].minor.yy909, &yymsp[-3].minor.yy495, &yymsp[0].minor.yy837, yymsp[-2].minor.yy54); } break; case 44: /* cmd ::= REVOKE privileges ON priv_level with_opt FROM user_name */ -#line 128 "sql.y" -{ pCxt->pRootNode = createRevokeStmt(pCxt, yymsp[-5].minor.yy1089, &yymsp[-3].minor.yy849, &yymsp[0].minor.yy1109, yymsp[-2].minor.yy416); } -#line 5735 "sql.c" +{ pCxt->pRootNode = createRevokeStmt(pCxt, yymsp[-5].minor.yy909, &yymsp[-3].minor.yy495, &yymsp[0].minor.yy837, yymsp[-2].minor.yy54); } break; case 45: /* privileges ::= ALL */ -#line 132 "sql.y" -{ yymsp[0].minor.yy1089 = PRIVILEGE_TYPE_ALL; } -#line 5740 "sql.c" +{ yymsp[0].minor.yy909 = PRIVILEGE_TYPE_ALL; } break; case 46: /* privileges ::= priv_type_list */ case 48: /* priv_type_list ::= priv_type */ yytestcase(yyruleno==48); -#line 133 "sql.y" -{ yylhsminor.yy1089 = yymsp[0].minor.yy1089; } -#line 5746 "sql.c" - yymsp[0].minor.yy1089 = yylhsminor.yy1089; +{ yylhsminor.yy909 = yymsp[0].minor.yy909; } + yymsp[0].minor.yy909 = yylhsminor.yy909; break; case 47: /* privileges ::= SUBSCRIBE */ -#line 134 "sql.y" -{ yymsp[0].minor.yy1089 = PRIVILEGE_TYPE_SUBSCRIBE; } -#line 5752 "sql.c" +{ yymsp[0].minor.yy909 = PRIVILEGE_TYPE_SUBSCRIBE; } break; case 49: /* priv_type_list ::= priv_type_list NK_COMMA priv_type */ -#line 139 "sql.y" -{ yylhsminor.yy1089 = yymsp[-2].minor.yy1089 | yymsp[0].minor.yy1089; } -#line 5757 "sql.c" - yymsp[-2].minor.yy1089 = yylhsminor.yy1089; +{ yylhsminor.yy909 = yymsp[-2].minor.yy909 | yymsp[0].minor.yy909; } + yymsp[-2].minor.yy909 = yylhsminor.yy909; break; case 50: /* priv_type ::= READ */ -#line 143 "sql.y" -{ yymsp[0].minor.yy1089 = PRIVILEGE_TYPE_READ; } -#line 5763 "sql.c" +{ yymsp[0].minor.yy909 = PRIVILEGE_TYPE_READ; } break; case 51: /* priv_type ::= WRITE */ -#line 144 "sql.y" -{ yymsp[0].minor.yy1089 = PRIVILEGE_TYPE_WRITE; } -#line 5768 "sql.c" +{ yymsp[0].minor.yy909 = PRIVILEGE_TYPE_WRITE; } break; case 52: /* priv_type ::= ALTER */ -#line 145 "sql.y" -{ yymsp[0].minor.yy1089 = PRIVILEGE_TYPE_ALTER; } -#line 5773 "sql.c" +{ yymsp[0].minor.yy909 = PRIVILEGE_TYPE_ALTER; } break; case 53: /* priv_level ::= NK_STAR NK_DOT NK_STAR */ -#line 149 "sql.y" -{ yylhsminor.yy849.first = yymsp[-2].minor.yy0; yylhsminor.yy849.second = yymsp[0].minor.yy0; } -#line 5778 "sql.c" - yymsp[-2].minor.yy849 = yylhsminor.yy849; +{ yylhsminor.yy495.first = yymsp[-2].minor.yy0; yylhsminor.yy495.second = yymsp[0].minor.yy0; } + yymsp[-2].minor.yy495 = yylhsminor.yy495; break; case 54: /* priv_level ::= db_name NK_DOT NK_STAR */ -#line 150 "sql.y" -{ yylhsminor.yy849.first = yymsp[-2].minor.yy1109; yylhsminor.yy849.second = yymsp[0].minor.yy0; } -#line 5784 "sql.c" - yymsp[-2].minor.yy849 = yylhsminor.yy849; +{ yylhsminor.yy495.first = yymsp[-2].minor.yy837; yylhsminor.yy495.second = yymsp[0].minor.yy0; } + yymsp[-2].minor.yy495 = yylhsminor.yy495; break; case 55: /* priv_level ::= db_name NK_DOT table_name */ -#line 151 "sql.y" -{ yylhsminor.yy849.first = yymsp[-2].minor.yy1109; yylhsminor.yy849.second = yymsp[0].minor.yy1109; } -#line 5790 "sql.c" - yymsp[-2].minor.yy849 = yylhsminor.yy849; +{ yylhsminor.yy495.first = yymsp[-2].minor.yy837; yylhsminor.yy495.second = yymsp[0].minor.yy837; } + yymsp[-2].minor.yy495 = yylhsminor.yy495; break; case 56: /* priv_level ::= topic_name */ -#line 152 "sql.y" -{ yylhsminor.yy849.first = yymsp[0].minor.yy1109; yylhsminor.yy849.second = nil_token; } -#line 5796 "sql.c" - yymsp[0].minor.yy849 = yylhsminor.yy849; +{ yylhsminor.yy495.first = yymsp[0].minor.yy837; yylhsminor.yy495.second = nil_token; } + yymsp[0].minor.yy495 = yylhsminor.yy495; break; case 57: /* with_opt ::= */ case 173: /* start_opt ::= */ yytestcase(yyruleno==173); case 177: /* end_opt ::= */ yytestcase(yyruleno==177); case 330: /* like_pattern_opt ::= */ yytestcase(yyruleno==330); - case 427: /* subtable_opt ::= */ yytestcase(yyruleno==427); - case 596: /* case_when_else_opt ::= */ yytestcase(yyruleno==596); - case 626: /* from_clause_opt ::= */ yytestcase(yyruleno==626); - case 653: /* join_on_clause_opt ::= */ yytestcase(yyruleno==653); - case 655: /* window_offset_clause_opt ::= */ yytestcase(yyruleno==655); - case 659: /* jlimit_clause_opt ::= */ yytestcase(yyruleno==659); - case 676: /* where_clause_opt ::= */ yytestcase(yyruleno==676); - case 685: /* twindow_clause_opt ::= */ yytestcase(yyruleno==685); - case 693: /* sliding_opt ::= */ yytestcase(yyruleno==693); - case 698: /* fill_opt ::= */ yytestcase(yyruleno==698); - case 712: /* having_clause_opt ::= */ yytestcase(yyruleno==712); - case 714: /* range_opt ::= */ yytestcase(yyruleno==714); - case 717: /* every_opt ::= */ yytestcase(yyruleno==717); - case 730: /* slimit_clause_opt ::= */ yytestcase(yyruleno==730); - case 734: /* limit_clause_opt ::= */ yytestcase(yyruleno==734); -#line 154 "sql.y" -{ yymsp[1].minor.yy416 = NULL; } -#line 5820 "sql.c" + case 428: /* subtable_opt ::= */ yytestcase(yyruleno==428); + case 597: /* case_when_else_opt ::= */ yytestcase(yyruleno==597); + case 627: /* from_clause_opt ::= */ yytestcase(yyruleno==627); + case 654: /* join_on_clause_opt ::= */ yytestcase(yyruleno==654); + case 656: /* window_offset_clause_opt ::= */ yytestcase(yyruleno==656); + case 660: /* jlimit_clause_opt ::= */ yytestcase(yyruleno==660); + case 677: /* where_clause_opt ::= */ yytestcase(yyruleno==677); + case 686: /* twindow_clause_opt ::= */ yytestcase(yyruleno==686); + case 694: /* sliding_opt ::= */ yytestcase(yyruleno==694); + case 699: /* fill_opt ::= */ yytestcase(yyruleno==699); + case 713: /* having_clause_opt ::= */ yytestcase(yyruleno==713); + case 715: /* range_opt ::= */ yytestcase(yyruleno==715); + case 718: /* every_opt ::= */ yytestcase(yyruleno==718); + case 731: /* slimit_clause_opt ::= */ yytestcase(yyruleno==731); + case 735: /* limit_clause_opt ::= */ yytestcase(yyruleno==735); +{ yymsp[1].minor.yy54 = NULL; } break; case 58: /* with_opt ::= WITH search_condition */ - case 627: /* from_clause_opt ::= FROM table_reference_list */ yytestcase(yyruleno==627); - case 654: /* join_on_clause_opt ::= ON search_condition */ yytestcase(yyruleno==654); - case 677: /* where_clause_opt ::= WHERE search_condition */ yytestcase(yyruleno==677); - case 713: /* having_clause_opt ::= HAVING search_condition */ yytestcase(yyruleno==713); -#line 155 "sql.y" -{ yymsp[-1].minor.yy416 = yymsp[0].minor.yy416; } -#line 5829 "sql.c" + case 628: /* from_clause_opt ::= FROM table_reference_list */ yytestcase(yyruleno==628); + case 655: /* join_on_clause_opt ::= ON search_condition */ yytestcase(yyruleno==655); + case 678: /* where_clause_opt ::= WHERE search_condition */ yytestcase(yyruleno==678); + case 714: /* having_clause_opt ::= HAVING search_condition */ yytestcase(yyruleno==714); +{ yymsp[-1].minor.yy54 = yymsp[0].minor.yy54; } break; case 59: /* cmd ::= CREATE ENCRYPT_KEY NK_STRING */ -#line 158 "sql.y" { pCxt->pRootNode = createEncryptKeyStmt(pCxt, &yymsp[0].minor.yy0); } -#line 5834 "sql.c" break; case 60: /* cmd ::= CREATE DNODE dnode_endpoint */ -#line 161 "sql.y" -{ pCxt->pRootNode = createCreateDnodeStmt(pCxt, &yymsp[0].minor.yy1109, NULL); } -#line 5839 "sql.c" +{ pCxt->pRootNode = createCreateDnodeStmt(pCxt, &yymsp[0].minor.yy837, NULL); } break; case 61: /* cmd ::= CREATE DNODE dnode_endpoint PORT NK_INTEGER */ -#line 162 "sql.y" -{ pCxt->pRootNode = createCreateDnodeStmt(pCxt, &yymsp[-2].minor.yy1109, &yymsp[0].minor.yy0); } -#line 5844 "sql.c" +{ pCxt->pRootNode = createCreateDnodeStmt(pCxt, &yymsp[-2].minor.yy837, &yymsp[0].minor.yy0); } break; case 62: /* cmd ::= DROP DNODE NK_INTEGER force_opt */ -#line 163 "sql.y" -{ pCxt->pRootNode = createDropDnodeStmt(pCxt, &yymsp[-1].minor.yy0, yymsp[0].minor.yy209, false); } -#line 5849 "sql.c" +{ pCxt->pRootNode = createDropDnodeStmt(pCxt, &yymsp[-1].minor.yy0, yymsp[0].minor.yy325, false); } break; case 63: /* cmd ::= DROP DNODE dnode_endpoint force_opt */ -#line 164 "sql.y" -{ pCxt->pRootNode = createDropDnodeStmt(pCxt, &yymsp[-1].minor.yy1109, yymsp[0].minor.yy209, false); } -#line 5854 "sql.c" +{ pCxt->pRootNode = createDropDnodeStmt(pCxt, &yymsp[-1].minor.yy837, yymsp[0].minor.yy325, false); } break; case 64: /* cmd ::= DROP DNODE NK_INTEGER unsafe_opt */ -#line 165 "sql.y" -{ pCxt->pRootNode = createDropDnodeStmt(pCxt, &yymsp[-1].minor.yy0, false, yymsp[0].minor.yy209); } -#line 5859 "sql.c" +{ pCxt->pRootNode = createDropDnodeStmt(pCxt, &yymsp[-1].minor.yy0, false, yymsp[0].minor.yy325); } break; case 65: /* cmd ::= DROP DNODE dnode_endpoint unsafe_opt */ -#line 166 "sql.y" -{ pCxt->pRootNode = createDropDnodeStmt(pCxt, &yymsp[-1].minor.yy1109, false, yymsp[0].minor.yy209); } -#line 5864 "sql.c" +{ pCxt->pRootNode = createDropDnodeStmt(pCxt, &yymsp[-1].minor.yy837, false, yymsp[0].minor.yy325); } break; case 66: /* cmd ::= ALTER DNODE NK_INTEGER NK_STRING */ -#line 167 "sql.y" { pCxt->pRootNode = createAlterDnodeStmt(pCxt, &yymsp[-1].minor.yy0, &yymsp[0].minor.yy0, NULL); } -#line 5869 "sql.c" break; case 67: /* cmd ::= ALTER DNODE NK_INTEGER NK_STRING NK_STRING */ -#line 168 "sql.y" { pCxt->pRootNode = createAlterDnodeStmt(pCxt, &yymsp[-2].minor.yy0, &yymsp[-1].minor.yy0, &yymsp[0].minor.yy0); } -#line 5874 "sql.c" break; case 68: /* cmd ::= ALTER ALL DNODES NK_STRING */ -#line 169 "sql.y" { pCxt->pRootNode = createAlterDnodeStmt(pCxt, NULL, &yymsp[0].minor.yy0, NULL); } -#line 5879 "sql.c" break; case 69: /* cmd ::= ALTER ALL DNODES NK_STRING NK_STRING */ -#line 170 "sql.y" { pCxt->pRootNode = createAlterDnodeStmt(pCxt, NULL, &yymsp[-1].minor.yy0, &yymsp[0].minor.yy0); } -#line 5884 "sql.c" break; case 70: /* cmd ::= RESTORE DNODE NK_INTEGER */ -#line 171 "sql.y" { pCxt->pRootNode = createRestoreComponentNodeStmt(pCxt, QUERY_NODE_RESTORE_DNODE_STMT, &yymsp[0].minor.yy0); } -#line 5889 "sql.c" break; case 71: /* dnode_endpoint ::= NK_STRING */ case 72: /* dnode_endpoint ::= NK_ID */ yytestcase(yyruleno==72); @@ -5895,37 +5796,35 @@ static YYACTIONTYPE yy_reduce( case 365: /* sma_func_name ::= FIRST */ yytestcase(yyruleno==365); case 366: /* sma_func_name ::= LAST */ yytestcase(yyruleno==366); case 367: /* sma_func_name ::= LAST_ROW */ yytestcase(yyruleno==367); - case 515: /* db_name ::= NK_ID */ yytestcase(yyruleno==515); - case 516: /* table_name ::= NK_ID */ yytestcase(yyruleno==516); - case 517: /* column_name ::= NK_ID */ yytestcase(yyruleno==517); - case 518: /* function_name ::= NK_ID */ yytestcase(yyruleno==518); - case 519: /* view_name ::= NK_ID */ yytestcase(yyruleno==519); - case 520: /* table_alias ::= NK_ID */ yytestcase(yyruleno==520); - case 521: /* column_alias ::= NK_ID */ yytestcase(yyruleno==521); - case 522: /* column_alias ::= NK_ALIAS */ yytestcase(yyruleno==522); - case 523: /* user_name ::= NK_ID */ yytestcase(yyruleno==523); - case 524: /* topic_name ::= NK_ID */ yytestcase(yyruleno==524); - case 525: /* stream_name ::= NK_ID */ yytestcase(yyruleno==525); - case 526: /* cgroup_name ::= NK_ID */ yytestcase(yyruleno==526); - case 527: /* index_name ::= NK_ID */ yytestcase(yyruleno==527); - case 528: /* tsma_name ::= NK_ID */ yytestcase(yyruleno==528); - case 572: /* noarg_func ::= NOW */ yytestcase(yyruleno==572); - case 573: /* noarg_func ::= TODAY */ yytestcase(yyruleno==573); - case 574: /* noarg_func ::= TIMEZONE */ yytestcase(yyruleno==574); - case 575: /* noarg_func ::= DATABASE */ yytestcase(yyruleno==575); - case 576: /* noarg_func ::= CLIENT_VERSION */ yytestcase(yyruleno==576); - case 577: /* noarg_func ::= SERVER_VERSION */ yytestcase(yyruleno==577); - case 578: /* noarg_func ::= SERVER_STATUS */ yytestcase(yyruleno==578); - case 579: /* noarg_func ::= CURRENT_USER */ yytestcase(yyruleno==579); - case 580: /* noarg_func ::= USER */ yytestcase(yyruleno==580); - case 581: /* star_func ::= COUNT */ yytestcase(yyruleno==581); - case 582: /* star_func ::= FIRST */ yytestcase(yyruleno==582); - case 583: /* star_func ::= LAST */ yytestcase(yyruleno==583); - case 584: /* star_func ::= LAST_ROW */ yytestcase(yyruleno==584); -#line 175 "sql.y" -{ yylhsminor.yy1109 = yymsp[0].minor.yy0; } -#line 5927 "sql.c" - yymsp[0].minor.yy1109 = yylhsminor.yy1109; + case 516: /* db_name ::= NK_ID */ yytestcase(yyruleno==516); + case 517: /* table_name ::= NK_ID */ yytestcase(yyruleno==517); + case 518: /* column_name ::= NK_ID */ yytestcase(yyruleno==518); + case 519: /* function_name ::= NK_ID */ yytestcase(yyruleno==519); + case 520: /* view_name ::= NK_ID */ yytestcase(yyruleno==520); + case 521: /* table_alias ::= NK_ID */ yytestcase(yyruleno==521); + case 522: /* column_alias ::= NK_ID */ yytestcase(yyruleno==522); + case 523: /* column_alias ::= NK_ALIAS */ yytestcase(yyruleno==523); + case 524: /* user_name ::= NK_ID */ yytestcase(yyruleno==524); + case 525: /* topic_name ::= NK_ID */ yytestcase(yyruleno==525); + case 526: /* stream_name ::= NK_ID */ yytestcase(yyruleno==526); + case 527: /* cgroup_name ::= NK_ID */ yytestcase(yyruleno==527); + case 528: /* index_name ::= NK_ID */ yytestcase(yyruleno==528); + case 529: /* tsma_name ::= NK_ID */ yytestcase(yyruleno==529); + case 573: /* noarg_func ::= NOW */ yytestcase(yyruleno==573); + case 574: /* noarg_func ::= TODAY */ yytestcase(yyruleno==574); + case 575: /* noarg_func ::= TIMEZONE */ yytestcase(yyruleno==575); + case 576: /* noarg_func ::= DATABASE */ yytestcase(yyruleno==576); + case 577: /* noarg_func ::= CLIENT_VERSION */ yytestcase(yyruleno==577); + case 578: /* noarg_func ::= SERVER_VERSION */ yytestcase(yyruleno==578); + case 579: /* noarg_func ::= SERVER_STATUS */ yytestcase(yyruleno==579); + case 580: /* noarg_func ::= CURRENT_USER */ yytestcase(yyruleno==580); + case 581: /* noarg_func ::= USER */ yytestcase(yyruleno==581); + case 582: /* star_func ::= COUNT */ yytestcase(yyruleno==582); + case 583: /* star_func ::= FIRST */ yytestcase(yyruleno==583); + case 584: /* star_func ::= LAST */ yytestcase(yyruleno==584); + case 585: /* star_func ::= LAST_ROW */ yytestcase(yyruleno==585); +{ yylhsminor.yy837 = yymsp[0].minor.yy0; } + yymsp[0].minor.yy837 = yylhsminor.yy837; break; case 74: /* force_opt ::= */ case 101: /* not_exists_opt ::= */ yytestcase(yyruleno==101); @@ -5933,500 +5832,330 @@ static YYACTIONTYPE yy_reduce( case 385: /* analyze_opt ::= */ yytestcase(yyruleno==385); case 392: /* agg_func_opt ::= */ yytestcase(yyruleno==392); case 398: /* or_replace_opt ::= */ yytestcase(yyruleno==398); - case 429: /* ignore_opt ::= */ yytestcase(yyruleno==429); - case 664: /* tag_mode_opt ::= */ yytestcase(yyruleno==664); - case 666: /* set_quantifier_opt ::= */ yytestcase(yyruleno==666); -#line 181 "sql.y" -{ yymsp[1].minor.yy209 = false; } -#line 5941 "sql.c" + case 430: /* ignore_opt ::= */ yytestcase(yyruleno==430); + case 665: /* tag_mode_opt ::= */ yytestcase(yyruleno==665); + case 667: /* set_quantifier_opt ::= */ yytestcase(yyruleno==667); +{ yymsp[1].minor.yy325 = false; } break; case 75: /* force_opt ::= FORCE */ case 76: /* unsafe_opt ::= UNSAFE */ yytestcase(yyruleno==76); case 386: /* analyze_opt ::= ANALYZE */ yytestcase(yyruleno==386); case 393: /* agg_func_opt ::= AGGREGATE */ yytestcase(yyruleno==393); - case 665: /* tag_mode_opt ::= TAGS */ yytestcase(yyruleno==665); - case 667: /* set_quantifier_opt ::= DISTINCT */ yytestcase(yyruleno==667); -#line 182 "sql.y" -{ yymsp[0].minor.yy209 = true; } -#line 5951 "sql.c" + case 666: /* tag_mode_opt ::= TAGS */ yytestcase(yyruleno==666); + case 668: /* set_quantifier_opt ::= DISTINCT */ yytestcase(yyruleno==668); +{ yymsp[0].minor.yy325 = true; } break; case 77: /* cmd ::= ALTER CLUSTER NK_STRING */ -#line 189 "sql.y" { pCxt->pRootNode = createAlterClusterStmt(pCxt, &yymsp[0].minor.yy0, NULL); } -#line 5956 "sql.c" break; case 78: /* cmd ::= ALTER CLUSTER NK_STRING NK_STRING */ -#line 190 "sql.y" { pCxt->pRootNode = createAlterClusterStmt(pCxt, &yymsp[-1].minor.yy0, &yymsp[0].minor.yy0); } -#line 5961 "sql.c" break; case 79: /* cmd ::= ALTER LOCAL NK_STRING */ -#line 193 "sql.y" { pCxt->pRootNode = createAlterLocalStmt(pCxt, &yymsp[0].minor.yy0, NULL); } -#line 5966 "sql.c" break; case 80: /* cmd ::= ALTER LOCAL NK_STRING NK_STRING */ -#line 194 "sql.y" { pCxt->pRootNode = createAlterLocalStmt(pCxt, &yymsp[-1].minor.yy0, &yymsp[0].minor.yy0); } -#line 5971 "sql.c" break; case 81: /* cmd ::= CREATE QNODE ON DNODE NK_INTEGER */ -#line 197 "sql.y" { pCxt->pRootNode = createCreateComponentNodeStmt(pCxt, QUERY_NODE_CREATE_QNODE_STMT, &yymsp[0].minor.yy0); } -#line 5976 "sql.c" break; case 82: /* cmd ::= DROP QNODE ON DNODE NK_INTEGER */ -#line 198 "sql.y" { pCxt->pRootNode = createDropComponentNodeStmt(pCxt, QUERY_NODE_DROP_QNODE_STMT, &yymsp[0].minor.yy0); } -#line 5981 "sql.c" break; case 83: /* cmd ::= RESTORE QNODE ON DNODE NK_INTEGER */ -#line 199 "sql.y" { pCxt->pRootNode = createRestoreComponentNodeStmt(pCxt, QUERY_NODE_RESTORE_QNODE_STMT, &yymsp[0].minor.yy0); } -#line 5986 "sql.c" break; case 84: /* cmd ::= CREATE BNODE ON DNODE NK_INTEGER */ -#line 202 "sql.y" { pCxt->pRootNode = createCreateComponentNodeStmt(pCxt, QUERY_NODE_CREATE_BNODE_STMT, &yymsp[0].minor.yy0); } -#line 5991 "sql.c" break; case 85: /* cmd ::= DROP BNODE ON DNODE NK_INTEGER */ -#line 203 "sql.y" { pCxt->pRootNode = createDropComponentNodeStmt(pCxt, QUERY_NODE_DROP_BNODE_STMT, &yymsp[0].minor.yy0); } -#line 5996 "sql.c" break; case 86: /* cmd ::= CREATE SNODE ON DNODE NK_INTEGER */ -#line 206 "sql.y" { pCxt->pRootNode = createCreateComponentNodeStmt(pCxt, QUERY_NODE_CREATE_SNODE_STMT, &yymsp[0].minor.yy0); } -#line 6001 "sql.c" break; case 87: /* cmd ::= DROP SNODE ON DNODE NK_INTEGER */ -#line 207 "sql.y" { pCxt->pRootNode = createDropComponentNodeStmt(pCxt, QUERY_NODE_DROP_SNODE_STMT, &yymsp[0].minor.yy0); } -#line 6006 "sql.c" break; case 88: /* cmd ::= CREATE MNODE ON DNODE NK_INTEGER */ -#line 210 "sql.y" { pCxt->pRootNode = createCreateComponentNodeStmt(pCxt, QUERY_NODE_CREATE_MNODE_STMT, &yymsp[0].minor.yy0); } -#line 6011 "sql.c" break; case 89: /* cmd ::= DROP MNODE ON DNODE NK_INTEGER */ -#line 211 "sql.y" { pCxt->pRootNode = createDropComponentNodeStmt(pCxt, QUERY_NODE_DROP_MNODE_STMT, &yymsp[0].minor.yy0); } -#line 6016 "sql.c" break; case 90: /* cmd ::= RESTORE MNODE ON DNODE NK_INTEGER */ -#line 212 "sql.y" { pCxt->pRootNode = createRestoreComponentNodeStmt(pCxt, QUERY_NODE_RESTORE_MNODE_STMT, &yymsp[0].minor.yy0); } -#line 6021 "sql.c" break; case 91: /* cmd ::= RESTORE VNODE ON DNODE NK_INTEGER */ -#line 215 "sql.y" { pCxt->pRootNode = createRestoreComponentNodeStmt(pCxt, QUERY_NODE_RESTORE_VNODE_STMT, &yymsp[0].minor.yy0); } -#line 6026 "sql.c" break; case 92: /* cmd ::= CREATE DATABASE not_exists_opt db_name db_options */ -#line 218 "sql.y" -{ pCxt->pRootNode = createCreateDatabaseStmt(pCxt, yymsp[-2].minor.yy209, &yymsp[-1].minor.yy1109, yymsp[0].minor.yy416); } -#line 6031 "sql.c" +{ pCxt->pRootNode = createCreateDatabaseStmt(pCxt, yymsp[-2].minor.yy325, &yymsp[-1].minor.yy837, yymsp[0].minor.yy54); } break; case 93: /* cmd ::= DROP DATABASE exists_opt db_name */ -#line 219 "sql.y" -{ pCxt->pRootNode = createDropDatabaseStmt(pCxt, yymsp[-1].minor.yy209, &yymsp[0].minor.yy1109); } -#line 6036 "sql.c" +{ pCxt->pRootNode = createDropDatabaseStmt(pCxt, yymsp[-1].minor.yy325, &yymsp[0].minor.yy837); } break; case 94: /* cmd ::= USE db_name */ -#line 220 "sql.y" -{ pCxt->pRootNode = createUseDatabaseStmt(pCxt, &yymsp[0].minor.yy1109); } -#line 6041 "sql.c" +{ pCxt->pRootNode = createUseDatabaseStmt(pCxt, &yymsp[0].minor.yy837); } break; case 95: /* cmd ::= ALTER DATABASE db_name alter_db_options */ -#line 221 "sql.y" -{ pCxt->pRootNode = createAlterDatabaseStmt(pCxt, &yymsp[-1].minor.yy1109, yymsp[0].minor.yy416); } -#line 6046 "sql.c" +{ pCxt->pRootNode = createAlterDatabaseStmt(pCxt, &yymsp[-1].minor.yy837, yymsp[0].minor.yy54); } break; case 96: /* cmd ::= FLUSH DATABASE db_name */ -#line 222 "sql.y" -{ pCxt->pRootNode = createFlushDatabaseStmt(pCxt, &yymsp[0].minor.yy1109); } -#line 6051 "sql.c" +{ pCxt->pRootNode = createFlushDatabaseStmt(pCxt, &yymsp[0].minor.yy837); } break; case 97: /* cmd ::= TRIM DATABASE db_name speed_opt */ -#line 223 "sql.y" -{ pCxt->pRootNode = createTrimDatabaseStmt(pCxt, &yymsp[-1].minor.yy1109, yymsp[0].minor.yy820); } -#line 6056 "sql.c" +{ pCxt->pRootNode = createTrimDatabaseStmt(pCxt, &yymsp[-1].minor.yy837, yymsp[0].minor.yy332); } break; case 98: /* cmd ::= S3MIGRATE DATABASE db_name */ -#line 224 "sql.y" -{ pCxt->pRootNode = createS3MigrateDatabaseStmt(pCxt, &yymsp[0].minor.yy1109); } -#line 6061 "sql.c" +{ pCxt->pRootNode = createS3MigrateDatabaseStmt(pCxt, &yymsp[0].minor.yy837); } break; case 99: /* cmd ::= COMPACT DATABASE db_name start_opt end_opt */ -#line 225 "sql.y" -{ pCxt->pRootNode = createCompactStmt(pCxt, &yymsp[-2].minor.yy1109, yymsp[-1].minor.yy416, yymsp[0].minor.yy416); } -#line 6066 "sql.c" +{ pCxt->pRootNode = createCompactStmt(pCxt, &yymsp[-2].minor.yy837, yymsp[-1].minor.yy54, yymsp[0].minor.yy54); } break; case 100: /* not_exists_opt ::= IF NOT EXISTS */ -#line 229 "sql.y" -{ yymsp[-2].minor.yy209 = true; } -#line 6071 "sql.c" +{ yymsp[-2].minor.yy325 = true; } break; case 102: /* exists_opt ::= IF EXISTS */ case 399: /* or_replace_opt ::= OR REPLACE */ yytestcase(yyruleno==399); - case 430: /* ignore_opt ::= IGNORE UNTREATED */ yytestcase(yyruleno==430); -#line 234 "sql.y" -{ yymsp[-1].minor.yy209 = true; } -#line 6078 "sql.c" + case 431: /* ignore_opt ::= IGNORE UNTREATED */ yytestcase(yyruleno==431); +{ yymsp[-1].minor.yy325 = true; } break; case 104: /* db_options ::= */ -#line 237 "sql.y" -{ yymsp[1].minor.yy416 = createDefaultDatabaseOptions(pCxt); } -#line 6083 "sql.c" +{ yymsp[1].minor.yy54 = createDefaultDatabaseOptions(pCxt); } break; case 105: /* db_options ::= db_options BUFFER NK_INTEGER */ -#line 238 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_BUFFER, &yymsp[0].minor.yy0); } -#line 6088 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_BUFFER, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 106: /* db_options ::= db_options CACHEMODEL NK_STRING */ -#line 239 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_CACHEMODEL, &yymsp[0].minor.yy0); } -#line 6094 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_CACHEMODEL, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 107: /* db_options ::= db_options CACHESIZE NK_INTEGER */ -#line 240 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_CACHESIZE, &yymsp[0].minor.yy0); } -#line 6100 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_CACHESIZE, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 108: /* db_options ::= db_options COMP NK_INTEGER */ -#line 241 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_COMP, &yymsp[0].minor.yy0); } -#line 6106 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_COMP, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 109: /* db_options ::= db_options DURATION NK_INTEGER */ case 110: /* db_options ::= db_options DURATION NK_VARIABLE */ yytestcase(yyruleno==110); -#line 242 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_DAYS, &yymsp[0].minor.yy0); } -#line 6113 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_DAYS, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 111: /* db_options ::= db_options MAXROWS NK_INTEGER */ -#line 244 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_MAXROWS, &yymsp[0].minor.yy0); } -#line 6119 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_MAXROWS, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 112: /* db_options ::= db_options MINROWS NK_INTEGER */ -#line 245 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_MINROWS, &yymsp[0].minor.yy0); } -#line 6125 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_MINROWS, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 113: /* db_options ::= db_options KEEP integer_list */ case 114: /* db_options ::= db_options KEEP variable_list */ yytestcase(yyruleno==114); -#line 246 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_KEEP, yymsp[0].minor.yy316); } -#line 6132 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_KEEP, yymsp[0].minor.yy652); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 115: /* db_options ::= db_options PAGES NK_INTEGER */ -#line 248 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_PAGES, &yymsp[0].minor.yy0); } -#line 6138 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_PAGES, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 116: /* db_options ::= db_options PAGESIZE NK_INTEGER */ -#line 249 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_PAGESIZE, &yymsp[0].minor.yy0); } -#line 6144 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_PAGESIZE, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 117: /* db_options ::= db_options TSDB_PAGESIZE NK_INTEGER */ -#line 250 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_TSDB_PAGESIZE, &yymsp[0].minor.yy0); } -#line 6150 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_TSDB_PAGESIZE, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 118: /* db_options ::= db_options PRECISION NK_STRING */ -#line 251 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_PRECISION, &yymsp[0].minor.yy0); } -#line 6156 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_PRECISION, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 119: /* db_options ::= db_options REPLICA NK_INTEGER */ -#line 252 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_REPLICA, &yymsp[0].minor.yy0); } -#line 6162 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_REPLICA, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 120: /* db_options ::= db_options VGROUPS NK_INTEGER */ -#line 254 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_VGROUPS, &yymsp[0].minor.yy0); } -#line 6168 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_VGROUPS, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 121: /* db_options ::= db_options SINGLE_STABLE NK_INTEGER */ -#line 255 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_SINGLE_STABLE, &yymsp[0].minor.yy0); } -#line 6174 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_SINGLE_STABLE, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 122: /* db_options ::= db_options RETENTIONS retention_list */ -#line 256 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_RETENTIONS, yymsp[0].minor.yy316); } -#line 6180 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_RETENTIONS, yymsp[0].minor.yy652); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 123: /* db_options ::= db_options SCHEMALESS NK_INTEGER */ -#line 257 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_SCHEMALESS, &yymsp[0].minor.yy0); } -#line 6186 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_SCHEMALESS, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 124: /* db_options ::= db_options WAL_LEVEL NK_INTEGER */ -#line 258 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_WAL, &yymsp[0].minor.yy0); } -#line 6192 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_WAL, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 125: /* db_options ::= db_options WAL_FSYNC_PERIOD NK_INTEGER */ -#line 259 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_FSYNC, &yymsp[0].minor.yy0); } -#line 6198 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_FSYNC, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 126: /* db_options ::= db_options WAL_RETENTION_PERIOD NK_INTEGER */ -#line 260 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_WAL_RETENTION_PERIOD, &yymsp[0].minor.yy0); } -#line 6204 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_WAL_RETENTION_PERIOD, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 127: /* db_options ::= db_options WAL_RETENTION_PERIOD NK_MINUS NK_INTEGER */ -#line 261 "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.yy416 = setDatabaseOption(pCxt, yymsp[-3].minor.yy416, DB_OPTION_WAL_RETENTION_PERIOD, &t); + yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-3].minor.yy54, DB_OPTION_WAL_RETENTION_PERIOD, &t); } -#line 6214 "sql.c" - yymsp[-3].minor.yy416 = yylhsminor.yy416; + yymsp[-3].minor.yy54 = yylhsminor.yy54; break; case 128: /* db_options ::= db_options WAL_RETENTION_SIZE NK_INTEGER */ -#line 266 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_WAL_RETENTION_SIZE, &yymsp[0].minor.yy0); } -#line 6220 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_WAL_RETENTION_SIZE, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 129: /* db_options ::= db_options WAL_RETENTION_SIZE NK_MINUS NK_INTEGER */ -#line 267 "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.yy416 = setDatabaseOption(pCxt, yymsp[-3].minor.yy416, DB_OPTION_WAL_RETENTION_SIZE, &t); + yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-3].minor.yy54, DB_OPTION_WAL_RETENTION_SIZE, &t); } -#line 6230 "sql.c" - yymsp[-3].minor.yy416 = yylhsminor.yy416; + yymsp[-3].minor.yy54 = yylhsminor.yy54; break; case 130: /* db_options ::= db_options WAL_ROLL_PERIOD NK_INTEGER */ -#line 272 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_WAL_ROLL_PERIOD, &yymsp[0].minor.yy0); } -#line 6236 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_WAL_ROLL_PERIOD, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 131: /* db_options ::= db_options WAL_SEGMENT_SIZE NK_INTEGER */ -#line 273 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_WAL_SEGMENT_SIZE, &yymsp[0].minor.yy0); } -#line 6242 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_WAL_SEGMENT_SIZE, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 132: /* db_options ::= db_options STT_TRIGGER NK_INTEGER */ -#line 274 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_STT_TRIGGER, &yymsp[0].minor.yy0); } -#line 6248 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_STT_TRIGGER, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 133: /* db_options ::= db_options TABLE_PREFIX signed */ -#line 275 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_TABLE_PREFIX, yymsp[0].minor.yy416); } -#line 6254 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_TABLE_PREFIX, yymsp[0].minor.yy54); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 134: /* db_options ::= db_options TABLE_SUFFIX signed */ -#line 276 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_TABLE_SUFFIX, yymsp[0].minor.yy416); } -#line 6260 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_TABLE_SUFFIX, yymsp[0].minor.yy54); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 135: /* db_options ::= db_options S3_CHUNKSIZE NK_INTEGER */ -#line 277 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_S3_CHUNKSIZE, &yymsp[0].minor.yy0); } -#line 6266 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_S3_CHUNKSIZE, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 136: /* db_options ::= db_options S3_KEEPLOCAL NK_INTEGER */ case 137: /* db_options ::= db_options S3_KEEPLOCAL NK_VARIABLE */ yytestcase(yyruleno==137); -#line 278 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_S3_KEEPLOCAL, &yymsp[0].minor.yy0); } -#line 6273 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_S3_KEEPLOCAL, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 138: /* db_options ::= db_options S3_COMPACT NK_INTEGER */ -#line 280 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_S3_COMPACT, &yymsp[0].minor.yy0); } -#line 6279 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_S3_COMPACT, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 139: /* db_options ::= db_options KEEP_TIME_OFFSET NK_INTEGER */ -#line 281 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_KEEP_TIME_OFFSET, &yymsp[0].minor.yy0); } -#line 6285 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_KEEP_TIME_OFFSET, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 140: /* db_options ::= db_options ENCRYPT_ALGORITHM NK_STRING */ -#line 282 "sql.y" -{ yylhsminor.yy416 = setDatabaseOption(pCxt, yymsp[-2].minor.yy416, DB_OPTION_ENCRYPT_ALGORITHM, &yymsp[0].minor.yy0); } -#line 6291 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setDatabaseOption(pCxt, yymsp[-2].minor.yy54, DB_OPTION_ENCRYPT_ALGORITHM, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 141: /* alter_db_options ::= alter_db_option */ -#line 284 "sql.y" -{ yylhsminor.yy416 = createAlterDatabaseOptions(pCxt); yylhsminor.yy416 = setAlterDatabaseOption(pCxt, yylhsminor.yy416, &yymsp[0].minor.yy101); } -#line 6297 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createAlterDatabaseOptions(pCxt); yylhsminor.yy54 = setAlterDatabaseOption(pCxt, yylhsminor.yy54, &yymsp[0].minor.yy663); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; case 142: /* alter_db_options ::= alter_db_options alter_db_option */ -#line 285 "sql.y" -{ yylhsminor.yy416 = setAlterDatabaseOption(pCxt, yymsp[-1].minor.yy416, &yymsp[0].minor.yy101); } -#line 6303 "sql.c" - yymsp[-1].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setAlterDatabaseOption(pCxt, yymsp[-1].minor.yy54, &yymsp[0].minor.yy663); } + yymsp[-1].minor.yy54 = yylhsminor.yy54; break; case 143: /* alter_db_option ::= BUFFER NK_INTEGER */ -#line 289 "sql.y" -{ yymsp[-1].minor.yy101.type = DB_OPTION_BUFFER; yymsp[-1].minor.yy101.val = yymsp[0].minor.yy0; } -#line 6309 "sql.c" +{ yymsp[-1].minor.yy663.type = DB_OPTION_BUFFER; yymsp[-1].minor.yy663.val = yymsp[0].minor.yy0; } break; case 144: /* alter_db_option ::= CACHEMODEL NK_STRING */ -#line 290 "sql.y" -{ yymsp[-1].minor.yy101.type = DB_OPTION_CACHEMODEL; yymsp[-1].minor.yy101.val = yymsp[0].minor.yy0; } -#line 6314 "sql.c" +{ yymsp[-1].minor.yy663.type = DB_OPTION_CACHEMODEL; yymsp[-1].minor.yy663.val = yymsp[0].minor.yy0; } break; case 145: /* alter_db_option ::= CACHESIZE NK_INTEGER */ -#line 291 "sql.y" -{ yymsp[-1].minor.yy101.type = DB_OPTION_CACHESIZE; yymsp[-1].minor.yy101.val = yymsp[0].minor.yy0; } -#line 6319 "sql.c" +{ yymsp[-1].minor.yy663.type = DB_OPTION_CACHESIZE; yymsp[-1].minor.yy663.val = yymsp[0].minor.yy0; } break; case 146: /* alter_db_option ::= WAL_FSYNC_PERIOD NK_INTEGER */ -#line 292 "sql.y" -{ yymsp[-1].minor.yy101.type = DB_OPTION_FSYNC; yymsp[-1].minor.yy101.val = yymsp[0].minor.yy0; } -#line 6324 "sql.c" +{ yymsp[-1].minor.yy663.type = DB_OPTION_FSYNC; yymsp[-1].minor.yy663.val = yymsp[0].minor.yy0; } break; case 147: /* alter_db_option ::= KEEP integer_list */ case 148: /* alter_db_option ::= KEEP variable_list */ yytestcase(yyruleno==148); -#line 293 "sql.y" -{ yymsp[-1].minor.yy101.type = DB_OPTION_KEEP; yymsp[-1].minor.yy101.pList = yymsp[0].minor.yy316; } -#line 6330 "sql.c" +{ yymsp[-1].minor.yy663.type = DB_OPTION_KEEP; yymsp[-1].minor.yy663.pList = yymsp[0].minor.yy652; } break; case 149: /* alter_db_option ::= PAGES NK_INTEGER */ -#line 295 "sql.y" -{ yymsp[-1].minor.yy101.type = DB_OPTION_PAGES; yymsp[-1].minor.yy101.val = yymsp[0].minor.yy0; } -#line 6335 "sql.c" +{ yymsp[-1].minor.yy663.type = DB_OPTION_PAGES; yymsp[-1].minor.yy663.val = yymsp[0].minor.yy0; } break; case 150: /* alter_db_option ::= REPLICA NK_INTEGER */ -#line 296 "sql.y" -{ yymsp[-1].minor.yy101.type = DB_OPTION_REPLICA; yymsp[-1].minor.yy101.val = yymsp[0].minor.yy0; } -#line 6340 "sql.c" +{ yymsp[-1].minor.yy663.type = DB_OPTION_REPLICA; yymsp[-1].minor.yy663.val = yymsp[0].minor.yy0; } break; case 151: /* alter_db_option ::= WAL_LEVEL NK_INTEGER */ -#line 298 "sql.y" -{ yymsp[-1].minor.yy101.type = DB_OPTION_WAL; yymsp[-1].minor.yy101.val = yymsp[0].minor.yy0; } -#line 6345 "sql.c" +{ yymsp[-1].minor.yy663.type = DB_OPTION_WAL; yymsp[-1].minor.yy663.val = yymsp[0].minor.yy0; } break; case 152: /* alter_db_option ::= STT_TRIGGER NK_INTEGER */ -#line 299 "sql.y" -{ yymsp[-1].minor.yy101.type = DB_OPTION_STT_TRIGGER; yymsp[-1].minor.yy101.val = yymsp[0].minor.yy0; } -#line 6350 "sql.c" +{ yymsp[-1].minor.yy663.type = DB_OPTION_STT_TRIGGER; yymsp[-1].minor.yy663.val = yymsp[0].minor.yy0; } break; case 153: /* alter_db_option ::= MINROWS NK_INTEGER */ -#line 300 "sql.y" -{ yymsp[-1].minor.yy101.type = DB_OPTION_MINROWS; yymsp[-1].minor.yy101.val = yymsp[0].minor.yy0; } -#line 6355 "sql.c" +{ yymsp[-1].minor.yy663.type = DB_OPTION_MINROWS; yymsp[-1].minor.yy663.val = yymsp[0].minor.yy0; } break; case 154: /* alter_db_option ::= WAL_RETENTION_PERIOD NK_INTEGER */ -#line 301 "sql.y" -{ yymsp[-1].minor.yy101.type = DB_OPTION_WAL_RETENTION_PERIOD; yymsp[-1].minor.yy101.val = yymsp[0].minor.yy0; } -#line 6360 "sql.c" +{ yymsp[-1].minor.yy663.type = DB_OPTION_WAL_RETENTION_PERIOD; yymsp[-1].minor.yy663.val = yymsp[0].minor.yy0; } break; case 155: /* alter_db_option ::= WAL_RETENTION_PERIOD NK_MINUS NK_INTEGER */ -#line 302 "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.yy101.type = DB_OPTION_WAL_RETENTION_PERIOD; yymsp[-2].minor.yy101.val = t; + yymsp[-2].minor.yy663.type = DB_OPTION_WAL_RETENTION_PERIOD; yymsp[-2].minor.yy663.val = t; } -#line 6369 "sql.c" break; case 156: /* alter_db_option ::= WAL_RETENTION_SIZE NK_INTEGER */ -#line 307 "sql.y" -{ yymsp[-1].minor.yy101.type = DB_OPTION_WAL_RETENTION_SIZE; yymsp[-1].minor.yy101.val = yymsp[0].minor.yy0; } -#line 6374 "sql.c" +{ yymsp[-1].minor.yy663.type = DB_OPTION_WAL_RETENTION_SIZE; yymsp[-1].minor.yy663.val = yymsp[0].minor.yy0; } break; case 157: /* alter_db_option ::= WAL_RETENTION_SIZE NK_MINUS NK_INTEGER */ -#line 308 "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.yy101.type = DB_OPTION_WAL_RETENTION_SIZE; yymsp[-2].minor.yy101.val = t; + yymsp[-2].minor.yy663.type = DB_OPTION_WAL_RETENTION_SIZE; yymsp[-2].minor.yy663.val = t; } -#line 6383 "sql.c" break; case 158: /* alter_db_option ::= S3_KEEPLOCAL NK_INTEGER */ case 159: /* alter_db_option ::= S3_KEEPLOCAL NK_VARIABLE */ yytestcase(yyruleno==159); -#line 313 "sql.y" -{ yymsp[-1].minor.yy101.type = DB_OPTION_S3_KEEPLOCAL; yymsp[-1].minor.yy101.val = yymsp[0].minor.yy0; } -#line 6389 "sql.c" +{ yymsp[-1].minor.yy663.type = DB_OPTION_S3_KEEPLOCAL; yymsp[-1].minor.yy663.val = yymsp[0].minor.yy0; } break; case 160: /* alter_db_option ::= S3_COMPACT NK_INTEGER */ -#line 315 "sql.y" -{ yymsp[-1].minor.yy101.type = DB_OPTION_S3_COMPACT, yymsp[-1].minor.yy101.val = yymsp[0].minor.yy0; } -#line 6394 "sql.c" +{ yymsp[-1].minor.yy663.type = DB_OPTION_S3_COMPACT, yymsp[-1].minor.yy663.val = yymsp[0].minor.yy0; } break; case 161: /* alter_db_option ::= KEEP_TIME_OFFSET NK_INTEGER */ -#line 316 "sql.y" -{ yymsp[-1].minor.yy101.type = DB_OPTION_KEEP_TIME_OFFSET; yymsp[-1].minor.yy101.val = yymsp[0].minor.yy0; } -#line 6399 "sql.c" +{ yymsp[-1].minor.yy663.type = DB_OPTION_KEEP_TIME_OFFSET; yymsp[-1].minor.yy663.val = yymsp[0].minor.yy0; } break; case 162: /* alter_db_option ::= ENCRYPT_ALGORITHM NK_STRING */ -#line 317 "sql.y" -{ yymsp[-1].minor.yy101.type = DB_OPTION_ENCRYPT_ALGORITHM; yymsp[-1].minor.yy101.val = yymsp[0].minor.yy0; } -#line 6404 "sql.c" +{ yymsp[-1].minor.yy663.type = DB_OPTION_ENCRYPT_ALGORITHM; yymsp[-1].minor.yy663.val = yymsp[0].minor.yy0; } break; case 163: /* integer_list ::= NK_INTEGER */ -#line 321 "sql.y" -{ yylhsminor.yy316 = createNodeList(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0)); } -#line 6409 "sql.c" - yymsp[0].minor.yy316 = yylhsminor.yy316; +{ yylhsminor.yy652 = createNodeList(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy652 = yylhsminor.yy652; break; case 164: /* integer_list ::= integer_list NK_COMMA NK_INTEGER */ - case 444: /* dnode_list ::= dnode_list DNODE NK_INTEGER */ yytestcase(yyruleno==444); -#line 322 "sql.y" -{ yylhsminor.yy316 = addNodeToList(pCxt, yymsp[-2].minor.yy316, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0)); } -#line 6416 "sql.c" - yymsp[-2].minor.yy316 = yylhsminor.yy316; + case 445: /* dnode_list ::= dnode_list DNODE NK_INTEGER */ yytestcase(yyruleno==445); +{ yylhsminor.yy652 = addNodeToList(pCxt, yymsp[-2].minor.yy652, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0)); } + yymsp[-2].minor.yy652 = yylhsminor.yy652; break; case 165: /* variable_list ::= NK_VARIABLE */ -#line 326 "sql.y" -{ yylhsminor.yy316 = createNodeList(pCxt, createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } -#line 6422 "sql.c" - yymsp[0].minor.yy316 = yylhsminor.yy316; +{ yylhsminor.yy652 = createNodeList(pCxt, createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy652 = yylhsminor.yy652; break; case 166: /* variable_list ::= variable_list NK_COMMA NK_VARIABLE */ -#line 327 "sql.y" -{ yylhsminor.yy316 = addNodeToList(pCxt, yymsp[-2].minor.yy316, createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } -#line 6428 "sql.c" - yymsp[-2].minor.yy316 = yylhsminor.yy316; +{ yylhsminor.yy652 = addNodeToList(pCxt, yymsp[-2].minor.yy652, createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } + yymsp[-2].minor.yy652 = yylhsminor.yy652; break; case 167: /* retention_list ::= retention */ case 200: /* multi_create_clause ::= create_subtable_clause */ yytestcase(yyruleno==200); @@ -6438,17 +6167,15 @@ static YYACTIONTYPE yy_reduce( case 336: /* tag_list_opt ::= tag_item */ yytestcase(yyruleno==336); case 360: /* func_list ::= func */ yytestcase(yyruleno==360); case 410: /* column_stream_def_list ::= column_stream_def */ yytestcase(yyruleno==410); - case 488: /* tags_literal_list ::= tags_literal */ yytestcase(yyruleno==488); - case 513: /* literal_list ::= signed_literal */ yytestcase(yyruleno==513); - case 587: /* other_para_list ::= star_func_para */ yytestcase(yyruleno==587); - case 593: /* when_then_list ::= when_then_expr */ yytestcase(yyruleno==593); - case 669: /* select_list ::= select_item */ yytestcase(yyruleno==669); - case 680: /* partition_list ::= partition_item */ yytestcase(yyruleno==680); - case 741: /* sort_specification_list ::= sort_specification */ yytestcase(yyruleno==741); -#line 331 "sql.y" -{ yylhsminor.yy316 = createNodeList(pCxt, yymsp[0].minor.yy416); } -#line 6450 "sql.c" - yymsp[0].minor.yy316 = yylhsminor.yy316; + case 489: /* tags_literal_list ::= tags_literal */ yytestcase(yyruleno==489); + case 514: /* literal_list ::= signed_literal */ yytestcase(yyruleno==514); + case 588: /* other_para_list ::= star_func_para */ yytestcase(yyruleno==588); + case 594: /* when_then_list ::= when_then_expr */ yytestcase(yyruleno==594); + case 670: /* select_list ::= select_item */ yytestcase(yyruleno==670); + case 681: /* partition_list ::= partition_item */ yytestcase(yyruleno==681); + case 742: /* sort_specification_list ::= sort_specification */ yytestcase(yyruleno==742); +{ yylhsminor.yy652 = createNodeList(pCxt, yymsp[0].minor.yy54); } + yymsp[0].minor.yy652 = yylhsminor.yy652; break; case 168: /* retention_list ::= retention_list NK_COMMA retention */ case 204: /* multi_drop_clause ::= multi_drop_clause NK_COMMA drop_table_clause */ yytestcase(yyruleno==204); @@ -6459,2214 +6186,1489 @@ static YYACTIONTYPE yy_reduce( case 337: /* tag_list_opt ::= tag_list_opt NK_COMMA tag_item */ yytestcase(yyruleno==337); case 361: /* func_list ::= func_list NK_COMMA func */ yytestcase(yyruleno==361); case 411: /* column_stream_def_list ::= column_stream_def_list NK_COMMA column_stream_def */ yytestcase(yyruleno==411); - case 489: /* tags_literal_list ::= tags_literal_list NK_COMMA tags_literal */ yytestcase(yyruleno==489); - case 514: /* literal_list ::= literal_list NK_COMMA signed_literal */ yytestcase(yyruleno==514); - case 588: /* other_para_list ::= other_para_list NK_COMMA star_func_para */ yytestcase(yyruleno==588); - case 670: /* select_list ::= select_list NK_COMMA select_item */ yytestcase(yyruleno==670); - case 681: /* partition_list ::= partition_list NK_COMMA partition_item */ yytestcase(yyruleno==681); - case 742: /* sort_specification_list ::= sort_specification_list NK_COMMA sort_specification */ yytestcase(yyruleno==742); -#line 332 "sql.y" -{ yylhsminor.yy316 = addNodeToList(pCxt, yymsp[-2].minor.yy316, yymsp[0].minor.yy416); } -#line 6470 "sql.c" - yymsp[-2].minor.yy316 = yylhsminor.yy316; + case 490: /* tags_literal_list ::= tags_literal_list NK_COMMA tags_literal */ yytestcase(yyruleno==490); + case 515: /* literal_list ::= literal_list NK_COMMA signed_literal */ yytestcase(yyruleno==515); + case 589: /* other_para_list ::= other_para_list NK_COMMA star_func_para */ yytestcase(yyruleno==589); + case 671: /* select_list ::= select_list NK_COMMA select_item */ yytestcase(yyruleno==671); + case 682: /* partition_list ::= partition_list NK_COMMA partition_item */ yytestcase(yyruleno==682); + case 743: /* sort_specification_list ::= sort_specification_list NK_COMMA sort_specification */ yytestcase(yyruleno==743); +{ yylhsminor.yy652 = addNodeToList(pCxt, yymsp[-2].minor.yy652, yymsp[0].minor.yy54); } + yymsp[-2].minor.yy652 = yylhsminor.yy652; break; case 169: /* retention ::= NK_VARIABLE NK_COLON NK_VARIABLE */ case 170: /* retention ::= NK_MINUS NK_COLON NK_VARIABLE */ yytestcase(yyruleno==170); -#line 334 "sql.y" -{ yylhsminor.yy416 = createNodeListNodeEx(pCxt, createDurationValueNode(pCxt, &yymsp[-2].minor.yy0), createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } -#line 6477 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createNodeListNodeEx(pCxt, createDurationValueNode(pCxt, &yymsp[-2].minor.yy0), createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 171: /* speed_opt ::= */ case 394: /* bufsize_opt ::= */ yytestcase(yyruleno==394); -#line 339 "sql.y" -{ yymsp[1].minor.yy820 = 0; } -#line 6484 "sql.c" +{ yymsp[1].minor.yy332 = 0; } break; case 172: /* speed_opt ::= BWLIMIT NK_INTEGER */ case 395: /* bufsize_opt ::= BUFSIZE NK_INTEGER */ yytestcase(yyruleno==395); -#line 340 "sql.y" -{ yymsp[-1].minor.yy820 = taosStr2Int32(yymsp[0].minor.yy0.z, NULL, 10); } -#line 6490 "sql.c" +{ yymsp[-1].minor.yy332 = taosStr2Int32(yymsp[0].minor.yy0.z, NULL, 10); } break; case 174: /* start_opt ::= START WITH NK_INTEGER */ case 178: /* end_opt ::= END WITH NK_INTEGER */ yytestcase(yyruleno==178); -#line 343 "sql.y" -{ yymsp[-2].minor.yy416 = createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0); } -#line 6496 "sql.c" +{ yymsp[-2].minor.yy54 = createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0); } break; case 175: /* start_opt ::= START WITH NK_STRING */ case 179: /* end_opt ::= END WITH NK_STRING */ yytestcase(yyruleno==179); -#line 344 "sql.y" -{ yymsp[-2].minor.yy416 = createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0); } -#line 6502 "sql.c" +{ yymsp[-2].minor.yy54 = createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0); } break; case 176: /* start_opt ::= START WITH TIMESTAMP NK_STRING */ case 180: /* end_opt ::= END WITH TIMESTAMP NK_STRING */ yytestcase(yyruleno==180); -#line 345 "sql.y" -{ yymsp[-3].minor.yy416 = createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0); } -#line 6508 "sql.c" +{ yymsp[-3].minor.yy54 = createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0); } break; case 181: /* cmd ::= CREATE TABLE not_exists_opt full_table_name NK_LP column_def_list NK_RP tags_def_opt table_options */ case 184: /* cmd ::= CREATE STABLE not_exists_opt full_table_name NK_LP column_def_list NK_RP tags_def table_options */ yytestcase(yyruleno==184); -#line 354 "sql.y" -{ pCxt->pRootNode = createCreateTableStmt(pCxt, yymsp[-6].minor.yy209, yymsp[-5].minor.yy416, yymsp[-3].minor.yy316, yymsp[-1].minor.yy316, yymsp[0].minor.yy416); } -#line 6514 "sql.c" +{ pCxt->pRootNode = createCreateTableStmt(pCxt, yymsp[-6].minor.yy325, yymsp[-5].minor.yy54, yymsp[-3].minor.yy652, yymsp[-1].minor.yy652, yymsp[0].minor.yy54); } break; case 182: /* cmd ::= CREATE TABLE multi_create_clause */ -#line 355 "sql.y" -{ pCxt->pRootNode = createCreateMultiTableStmt(pCxt, yymsp[0].minor.yy316); } -#line 6519 "sql.c" +{ pCxt->pRootNode = createCreateMultiTableStmt(pCxt, yymsp[0].minor.yy652); } break; case 183: /* cmd ::= CREATE TABLE not_exists_opt USING full_table_name NK_LP tag_list_opt NK_RP FILE NK_STRING */ -#line 357 "sql.y" -{ pCxt->pRootNode = createCreateSubTableFromFileClause(pCxt, yymsp[-7].minor.yy209, yymsp[-5].minor.yy416, yymsp[-3].minor.yy316, &yymsp[0].minor.yy0); } -#line 6524 "sql.c" +{ pCxt->pRootNode = createCreateSubTableFromFileClause(pCxt, yymsp[-7].minor.yy325, yymsp[-5].minor.yy54, yymsp[-3].minor.yy652, &yymsp[0].minor.yy0); } break; case 185: /* cmd ::= DROP TABLE multi_drop_clause */ -#line 360 "sql.y" -{ pCxt->pRootNode = createDropTableStmt(pCxt, yymsp[0].minor.yy316); } -#line 6529 "sql.c" +{ pCxt->pRootNode = createDropTableStmt(pCxt, yymsp[0].minor.yy652); } break; case 186: /* cmd ::= DROP STABLE exists_opt full_table_name */ -#line 361 "sql.y" -{ pCxt->pRootNode = createDropSuperTableStmt(pCxt, yymsp[-1].minor.yy209, yymsp[0].minor.yy416); } -#line 6534 "sql.c" +{ pCxt->pRootNode = createDropSuperTableStmt(pCxt, yymsp[-1].minor.yy325, yymsp[0].minor.yy54); } break; case 187: /* cmd ::= ALTER TABLE alter_table_clause */ - case 446: /* cmd ::= query_or_subquery */ yytestcase(yyruleno==446); - case 447: /* cmd ::= insert_query */ yytestcase(yyruleno==447); -#line 363 "sql.y" -{ pCxt->pRootNode = yymsp[0].minor.yy416; } -#line 6541 "sql.c" + case 447: /* cmd ::= query_or_subquery */ yytestcase(yyruleno==447); + case 448: /* cmd ::= insert_query */ yytestcase(yyruleno==448); +{ pCxt->pRootNode = yymsp[0].minor.yy54; } break; case 188: /* cmd ::= ALTER STABLE alter_table_clause */ -#line 364 "sql.y" -{ pCxt->pRootNode = setAlterSuperTableType(yymsp[0].minor.yy416); } -#line 6546 "sql.c" +{ pCxt->pRootNode = setAlterSuperTableType(yymsp[0].minor.yy54); } break; case 189: /* alter_table_clause ::= full_table_name alter_table_options */ -#line 366 "sql.y" -{ yylhsminor.yy416 = createAlterTableModifyOptions(pCxt, yymsp[-1].minor.yy416, yymsp[0].minor.yy416); } -#line 6551 "sql.c" - yymsp[-1].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createAlterTableModifyOptions(pCxt, yymsp[-1].minor.yy54, yymsp[0].minor.yy54); } + yymsp[-1].minor.yy54 = yylhsminor.yy54; break; case 190: /* alter_table_clause ::= full_table_name ADD COLUMN column_name type_name column_options */ -#line 368 "sql.y" -{ yylhsminor.yy416 = createAlterTableAddModifyColOptions2(pCxt, yymsp[-5].minor.yy416, TSDB_ALTER_TABLE_ADD_COLUMN, &yymsp[-2].minor.yy1109, yymsp[-1].minor.yy952, yymsp[0].minor.yy416); } -#line 6557 "sql.c" - yymsp[-5].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createAlterTableAddModifyColOptions2(pCxt, yymsp[-5].minor.yy54, TSDB_ALTER_TABLE_ADD_COLUMN, &yymsp[-2].minor.yy837, yymsp[-1].minor.yy84, yymsp[0].minor.yy54); } + yymsp[-5].minor.yy54 = yylhsminor.yy54; break; case 191: /* alter_table_clause ::= full_table_name DROP COLUMN column_name */ -#line 369 "sql.y" -{ yylhsminor.yy416 = createAlterTableDropCol(pCxt, yymsp[-3].minor.yy416, TSDB_ALTER_TABLE_DROP_COLUMN, &yymsp[0].minor.yy1109); } -#line 6563 "sql.c" - yymsp[-3].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createAlterTableDropCol(pCxt, yymsp[-3].minor.yy54, TSDB_ALTER_TABLE_DROP_COLUMN, &yymsp[0].minor.yy837); } + yymsp[-3].minor.yy54 = yylhsminor.yy54; break; case 192: /* alter_table_clause ::= full_table_name MODIFY COLUMN column_name type_name */ -#line 371 "sql.y" -{ yylhsminor.yy416 = createAlterTableAddModifyCol(pCxt, yymsp[-4].minor.yy416, TSDB_ALTER_TABLE_UPDATE_COLUMN_BYTES, &yymsp[-1].minor.yy1109, yymsp[0].minor.yy952); } -#line 6569 "sql.c" - yymsp[-4].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createAlterTableAddModifyCol(pCxt, yymsp[-4].minor.yy54, TSDB_ALTER_TABLE_UPDATE_COLUMN_BYTES, &yymsp[-1].minor.yy837, yymsp[0].minor.yy84); } + yymsp[-4].minor.yy54 = yylhsminor.yy54; break; case 193: /* alter_table_clause ::= full_table_name MODIFY COLUMN column_name column_options */ -#line 373 "sql.y" -{ yylhsminor.yy416 = createAlterTableAddModifyColOptions(pCxt, yymsp[-4].minor.yy416, TSDB_ALTER_TABLE_UPDATE_COLUMN_COMPRESS, &yymsp[-1].minor.yy1109, yymsp[0].minor.yy416); } -#line 6575 "sql.c" - yymsp[-4].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createAlterTableAddModifyColOptions(pCxt, yymsp[-4].minor.yy54, TSDB_ALTER_TABLE_UPDATE_COLUMN_COMPRESS, &yymsp[-1].minor.yy837, yymsp[0].minor.yy54); } + yymsp[-4].minor.yy54 = yylhsminor.yy54; break; case 194: /* alter_table_clause ::= full_table_name RENAME COLUMN column_name column_name */ -#line 375 "sql.y" -{ yylhsminor.yy416 = createAlterTableRenameCol(pCxt, yymsp[-4].minor.yy416, TSDB_ALTER_TABLE_UPDATE_COLUMN_NAME, &yymsp[-1].minor.yy1109, &yymsp[0].minor.yy1109); } -#line 6581 "sql.c" - yymsp[-4].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createAlterTableRenameCol(pCxt, yymsp[-4].minor.yy54, TSDB_ALTER_TABLE_UPDATE_COLUMN_NAME, &yymsp[-1].minor.yy837, &yymsp[0].minor.yy837); } + yymsp[-4].minor.yy54 = yylhsminor.yy54; break; case 195: /* alter_table_clause ::= full_table_name ADD TAG column_name type_name */ -#line 377 "sql.y" -{ yylhsminor.yy416 = createAlterTableAddModifyCol(pCxt, yymsp[-4].minor.yy416, TSDB_ALTER_TABLE_ADD_TAG, &yymsp[-1].minor.yy1109, yymsp[0].minor.yy952); } -#line 6587 "sql.c" - yymsp[-4].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createAlterTableAddModifyCol(pCxt, yymsp[-4].minor.yy54, TSDB_ALTER_TABLE_ADD_TAG, &yymsp[-1].minor.yy837, yymsp[0].minor.yy84); } + yymsp[-4].minor.yy54 = yylhsminor.yy54; break; case 196: /* alter_table_clause ::= full_table_name DROP TAG column_name */ -#line 378 "sql.y" -{ yylhsminor.yy416 = createAlterTableDropCol(pCxt, yymsp[-3].minor.yy416, TSDB_ALTER_TABLE_DROP_TAG, &yymsp[0].minor.yy1109); } -#line 6593 "sql.c" - yymsp[-3].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createAlterTableDropCol(pCxt, yymsp[-3].minor.yy54, TSDB_ALTER_TABLE_DROP_TAG, &yymsp[0].minor.yy837); } + yymsp[-3].minor.yy54 = yylhsminor.yy54; break; case 197: /* alter_table_clause ::= full_table_name MODIFY TAG column_name type_name */ -#line 380 "sql.y" -{ yylhsminor.yy416 = createAlterTableAddModifyCol(pCxt, yymsp[-4].minor.yy416, TSDB_ALTER_TABLE_UPDATE_TAG_BYTES, &yymsp[-1].minor.yy1109, yymsp[0].minor.yy952); } -#line 6599 "sql.c" - yymsp[-4].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createAlterTableAddModifyCol(pCxt, yymsp[-4].minor.yy54, TSDB_ALTER_TABLE_UPDATE_TAG_BYTES, &yymsp[-1].minor.yy837, yymsp[0].minor.yy84); } + yymsp[-4].minor.yy54 = yylhsminor.yy54; break; case 198: /* alter_table_clause ::= full_table_name RENAME TAG column_name column_name */ -#line 382 "sql.y" -{ yylhsminor.yy416 = createAlterTableRenameCol(pCxt, yymsp[-4].minor.yy416, TSDB_ALTER_TABLE_UPDATE_TAG_NAME, &yymsp[-1].minor.yy1109, &yymsp[0].minor.yy1109); } -#line 6605 "sql.c" - yymsp[-4].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createAlterTableRenameCol(pCxt, yymsp[-4].minor.yy54, TSDB_ALTER_TABLE_UPDATE_TAG_NAME, &yymsp[-1].minor.yy837, &yymsp[0].minor.yy837); } + yymsp[-4].minor.yy54 = yylhsminor.yy54; break; case 199: /* alter_table_clause ::= full_table_name SET TAG column_name NK_EQ tags_literal */ -#line 384 "sql.y" -{ yylhsminor.yy416 = createAlterTableSetTag(pCxt, yymsp[-5].minor.yy416, &yymsp[-2].minor.yy1109, yymsp[0].minor.yy416); } -#line 6611 "sql.c" - yymsp[-5].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createAlterTableSetTag(pCxt, yymsp[-5].minor.yy54, &yymsp[-2].minor.yy837, yymsp[0].minor.yy54); } + yymsp[-5].minor.yy54 = yylhsminor.yy54; break; case 201: /* multi_create_clause ::= multi_create_clause create_subtable_clause */ - case 594: /* when_then_list ::= when_then_list when_then_expr */ yytestcase(yyruleno==594); -#line 389 "sql.y" -{ yylhsminor.yy316 = addNodeToList(pCxt, yymsp[-1].minor.yy316, yymsp[0].minor.yy416); } -#line 6618 "sql.c" - yymsp[-1].minor.yy316 = yylhsminor.yy316; + case 595: /* when_then_list ::= when_then_list when_then_expr */ yytestcase(yyruleno==595); +{ yylhsminor.yy652 = addNodeToList(pCxt, yymsp[-1].minor.yy652, yymsp[0].minor.yy54); } + yymsp[-1].minor.yy652 = yylhsminor.yy652; break; case 202: /* create_subtable_clause ::= not_exists_opt full_table_name USING full_table_name specific_cols_opt TAGS NK_LP tags_literal_list NK_RP table_options */ -#line 393 "sql.y" -{ yylhsminor.yy416 = createCreateSubTableClause(pCxt, yymsp[-9].minor.yy209, yymsp[-8].minor.yy416, yymsp[-6].minor.yy416, yymsp[-5].minor.yy316, yymsp[-2].minor.yy316, yymsp[0].minor.yy416); } -#line 6624 "sql.c" - yymsp[-9].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createCreateSubTableClause(pCxt, yymsp[-9].minor.yy325, yymsp[-8].minor.yy54, yymsp[-6].minor.yy54, yymsp[-5].minor.yy652, yymsp[-2].minor.yy652, yymsp[0].minor.yy54); } + yymsp[-9].minor.yy54 = yylhsminor.yy54; break; case 205: /* drop_table_clause ::= exists_opt full_table_name */ -#line 400 "sql.y" -{ yylhsminor.yy416 = createDropTableClause(pCxt, yymsp[-1].minor.yy209, yymsp[0].minor.yy416); } -#line 6630 "sql.c" - yymsp[-1].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createDropTableClause(pCxt, yymsp[-1].minor.yy325, yymsp[0].minor.yy54); } + yymsp[-1].minor.yy54 = yylhsminor.yy54; break; case 207: /* specific_cols_opt ::= NK_LP col_name_list NK_RP */ case 409: /* col_list_opt ::= NK_LP column_stream_def_list NK_RP */ yytestcase(yyruleno==409); -#line 405 "sql.y" -{ yymsp[-2].minor.yy316 = yymsp[-1].minor.yy316; } -#line 6637 "sql.c" +{ yymsp[-2].minor.yy652 = yymsp[-1].minor.yy652; } break; case 208: /* full_table_name ::= table_name */ case 350: /* full_tsma_name ::= tsma_name */ yytestcase(yyruleno==350); -#line 407 "sql.y" -{ yylhsminor.yy416 = createRealTableNode(pCxt, NULL, &yymsp[0].minor.yy1109, NULL); } -#line 6643 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createRealTableNode(pCxt, NULL, &yymsp[0].minor.yy837, NULL); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; case 209: /* full_table_name ::= db_name NK_DOT table_name */ case 351: /* full_tsma_name ::= db_name NK_DOT tsma_name */ yytestcase(yyruleno==351); -#line 408 "sql.y" -{ yylhsminor.yy416 = createRealTableNode(pCxt, &yymsp[-2].minor.yy1109, &yymsp[0].minor.yy1109, NULL); } -#line 6650 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createRealTableNode(pCxt, &yymsp[-2].minor.yy837, &yymsp[0].minor.yy837, NULL); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 212: /* tag_def ::= column_name type_name */ -#line 414 "sql.y" -{ yylhsminor.yy416 = createColumnDefNode(pCxt, &yymsp[-1].minor.yy1109, yymsp[0].minor.yy952, NULL); } -#line 6656 "sql.c" - yymsp[-1].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createColumnDefNode(pCxt, &yymsp[-1].minor.yy837, yymsp[0].minor.yy84, NULL); } + yymsp[-1].minor.yy54 = yylhsminor.yy54; break; case 215: /* column_def ::= column_name type_name column_options */ -#line 422 "sql.y" -{ yylhsminor.yy416 = createColumnDefNode(pCxt, &yymsp[-2].minor.yy1109, yymsp[-1].minor.yy952, yymsp[0].minor.yy416); } -#line 6662 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createColumnDefNode(pCxt, &yymsp[-2].minor.yy837, yymsp[-1].minor.yy84, yymsp[0].minor.yy54); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 216: /* type_name ::= BOOL */ -#line 426 "sql.y" -{ yymsp[0].minor.yy952 = createDataType(TSDB_DATA_TYPE_BOOL); } -#line 6668 "sql.c" +{ yymsp[0].minor.yy84 = createDataType(TSDB_DATA_TYPE_BOOL); } break; case 217: /* type_name ::= TINYINT */ -#line 427 "sql.y" -{ yymsp[0].minor.yy952 = createDataType(TSDB_DATA_TYPE_TINYINT); } -#line 6673 "sql.c" +{ yymsp[0].minor.yy84 = createDataType(TSDB_DATA_TYPE_TINYINT); } break; case 218: /* type_name ::= SMALLINT */ -#line 428 "sql.y" -{ yymsp[0].minor.yy952 = createDataType(TSDB_DATA_TYPE_SMALLINT); } -#line 6678 "sql.c" +{ yymsp[0].minor.yy84 = createDataType(TSDB_DATA_TYPE_SMALLINT); } break; case 219: /* type_name ::= INT */ case 220: /* type_name ::= INTEGER */ yytestcase(yyruleno==220); -#line 429 "sql.y" -{ yymsp[0].minor.yy952 = createDataType(TSDB_DATA_TYPE_INT); } -#line 6684 "sql.c" +{ yymsp[0].minor.yy84 = createDataType(TSDB_DATA_TYPE_INT); } break; case 221: /* type_name ::= BIGINT */ -#line 431 "sql.y" -{ yymsp[0].minor.yy952 = createDataType(TSDB_DATA_TYPE_BIGINT); } -#line 6689 "sql.c" +{ yymsp[0].minor.yy84 = createDataType(TSDB_DATA_TYPE_BIGINT); } break; case 222: /* type_name ::= FLOAT */ -#line 432 "sql.y" -{ yymsp[0].minor.yy952 = createDataType(TSDB_DATA_TYPE_FLOAT); } -#line 6694 "sql.c" +{ yymsp[0].minor.yy84 = createDataType(TSDB_DATA_TYPE_FLOAT); } break; case 223: /* type_name ::= DOUBLE */ -#line 433 "sql.y" -{ yymsp[0].minor.yy952 = createDataType(TSDB_DATA_TYPE_DOUBLE); } -#line 6699 "sql.c" +{ yymsp[0].minor.yy84 = createDataType(TSDB_DATA_TYPE_DOUBLE); } break; case 224: /* type_name ::= BINARY NK_LP NK_INTEGER NK_RP */ -#line 434 "sql.y" -{ yymsp[-3].minor.yy952 = createVarLenDataType(TSDB_DATA_TYPE_BINARY, &yymsp[-1].minor.yy0); } -#line 6704 "sql.c" +{ yymsp[-3].minor.yy84 = createVarLenDataType(TSDB_DATA_TYPE_BINARY, &yymsp[-1].minor.yy0); } break; case 225: /* type_name ::= TIMESTAMP */ -#line 435 "sql.y" -{ yymsp[0].minor.yy952 = createDataType(TSDB_DATA_TYPE_TIMESTAMP); } -#line 6709 "sql.c" +{ yymsp[0].minor.yy84 = createDataType(TSDB_DATA_TYPE_TIMESTAMP); } break; case 226: /* type_name ::= NCHAR NK_LP NK_INTEGER NK_RP */ -#line 436 "sql.y" -{ yymsp[-3].minor.yy952 = createVarLenDataType(TSDB_DATA_TYPE_NCHAR, &yymsp[-1].minor.yy0); } -#line 6714 "sql.c" +{ yymsp[-3].minor.yy84 = createVarLenDataType(TSDB_DATA_TYPE_NCHAR, &yymsp[-1].minor.yy0); } break; case 227: /* type_name ::= TINYINT UNSIGNED */ -#line 437 "sql.y" -{ yymsp[-1].minor.yy952 = createDataType(TSDB_DATA_TYPE_UTINYINT); } -#line 6719 "sql.c" +{ yymsp[-1].minor.yy84 = createDataType(TSDB_DATA_TYPE_UTINYINT); } break; case 228: /* type_name ::= SMALLINT UNSIGNED */ -#line 438 "sql.y" -{ yymsp[-1].minor.yy952 = createDataType(TSDB_DATA_TYPE_USMALLINT); } -#line 6724 "sql.c" +{ yymsp[-1].minor.yy84 = createDataType(TSDB_DATA_TYPE_USMALLINT); } break; case 229: /* type_name ::= INT UNSIGNED */ -#line 439 "sql.y" -{ yymsp[-1].minor.yy952 = createDataType(TSDB_DATA_TYPE_UINT); } -#line 6729 "sql.c" +{ yymsp[-1].minor.yy84 = createDataType(TSDB_DATA_TYPE_UINT); } break; case 230: /* type_name ::= BIGINT UNSIGNED */ -#line 440 "sql.y" -{ yymsp[-1].minor.yy952 = createDataType(TSDB_DATA_TYPE_UBIGINT); } -#line 6734 "sql.c" +{ yymsp[-1].minor.yy84 = createDataType(TSDB_DATA_TYPE_UBIGINT); } break; case 231: /* type_name ::= JSON */ -#line 441 "sql.y" -{ yymsp[0].minor.yy952 = createDataType(TSDB_DATA_TYPE_JSON); } -#line 6739 "sql.c" +{ yymsp[0].minor.yy84 = createDataType(TSDB_DATA_TYPE_JSON); } break; case 232: /* type_name ::= VARCHAR NK_LP NK_INTEGER NK_RP */ -#line 442 "sql.y" -{ yymsp[-3].minor.yy952 = createVarLenDataType(TSDB_DATA_TYPE_VARCHAR, &yymsp[-1].minor.yy0); } -#line 6744 "sql.c" +{ yymsp[-3].minor.yy84 = createVarLenDataType(TSDB_DATA_TYPE_VARCHAR, &yymsp[-1].minor.yy0); } break; case 233: /* type_name ::= MEDIUMBLOB */ -#line 443 "sql.y" -{ yymsp[0].minor.yy952 = createDataType(TSDB_DATA_TYPE_MEDIUMBLOB); } -#line 6749 "sql.c" +{ yymsp[0].minor.yy84 = createDataType(TSDB_DATA_TYPE_MEDIUMBLOB); } break; case 234: /* type_name ::= BLOB */ -#line 444 "sql.y" -{ yymsp[0].minor.yy952 = createDataType(TSDB_DATA_TYPE_BLOB); } -#line 6754 "sql.c" +{ yymsp[0].minor.yy84 = createDataType(TSDB_DATA_TYPE_BLOB); } break; case 235: /* type_name ::= VARBINARY NK_LP NK_INTEGER NK_RP */ -#line 445 "sql.y" -{ yymsp[-3].minor.yy952 = createVarLenDataType(TSDB_DATA_TYPE_VARBINARY, &yymsp[-1].minor.yy0); } -#line 6759 "sql.c" +{ yymsp[-3].minor.yy84 = createVarLenDataType(TSDB_DATA_TYPE_VARBINARY, &yymsp[-1].minor.yy0); } break; case 236: /* type_name ::= GEOMETRY NK_LP NK_INTEGER NK_RP */ -#line 446 "sql.y" -{ yymsp[-3].minor.yy952 = createVarLenDataType(TSDB_DATA_TYPE_GEOMETRY, &yymsp[-1].minor.yy0); } -#line 6764 "sql.c" +{ yymsp[-3].minor.yy84 = createVarLenDataType(TSDB_DATA_TYPE_GEOMETRY, &yymsp[-1].minor.yy0); } break; case 237: /* type_name ::= DECIMAL */ -#line 447 "sql.y" -{ yymsp[0].minor.yy952 = createDataType(TSDB_DATA_TYPE_DECIMAL); } -#line 6769 "sql.c" +{ yymsp[0].minor.yy84 = createDataType(TSDB_DATA_TYPE_DECIMAL); } break; case 238: /* type_name ::= DECIMAL NK_LP NK_INTEGER NK_RP */ -#line 448 "sql.y" -{ yymsp[-3].minor.yy952 = createDataType(TSDB_DATA_TYPE_DECIMAL); } -#line 6774 "sql.c" +{ yymsp[-3].minor.yy84 = createDataType(TSDB_DATA_TYPE_DECIMAL); } break; case 239: /* type_name ::= DECIMAL NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP */ -#line 449 "sql.y" -{ yymsp[-5].minor.yy952 = createDataType(TSDB_DATA_TYPE_DECIMAL); } -#line 6779 "sql.c" +{ yymsp[-5].minor.yy84 = createDataType(TSDB_DATA_TYPE_DECIMAL); } break; case 240: /* type_name_default_len ::= BINARY */ -#line 453 "sql.y" -{ yymsp[0].minor.yy952 = createVarLenDataType(TSDB_DATA_TYPE_BINARY, NULL); } -#line 6784 "sql.c" +{ yymsp[0].minor.yy84 = createVarLenDataType(TSDB_DATA_TYPE_BINARY, NULL); } break; case 241: /* type_name_default_len ::= NCHAR */ -#line 454 "sql.y" -{ yymsp[0].minor.yy952 = createVarLenDataType(TSDB_DATA_TYPE_NCHAR, NULL); } -#line 6789 "sql.c" +{ yymsp[0].minor.yy84 = createVarLenDataType(TSDB_DATA_TYPE_NCHAR, NULL); } break; case 242: /* type_name_default_len ::= VARCHAR */ -#line 455 "sql.y" -{ yymsp[0].minor.yy952 = createVarLenDataType(TSDB_DATA_TYPE_VARCHAR, NULL); } -#line 6794 "sql.c" +{ yymsp[0].minor.yy84 = createVarLenDataType(TSDB_DATA_TYPE_VARCHAR, NULL); } break; case 243: /* type_name_default_len ::= VARBINARY */ -#line 456 "sql.y" -{ yymsp[0].minor.yy952 = createVarLenDataType(TSDB_DATA_TYPE_VARBINARY, NULL); } -#line 6799 "sql.c" +{ yymsp[0].minor.yy84 = createVarLenDataType(TSDB_DATA_TYPE_VARBINARY, NULL); } break; case 246: /* tags_def ::= TAGS NK_LP tag_def_list NK_RP */ case 417: /* tag_def_or_ref_opt ::= TAGS NK_LP column_stream_def_list NK_RP */ yytestcase(yyruleno==417); -#line 465 "sql.y" -{ yymsp[-3].minor.yy316 = yymsp[-1].minor.yy316; } -#line 6805 "sql.c" +{ yymsp[-3].minor.yy652 = yymsp[-1].minor.yy652; } break; case 247: /* table_options ::= */ -#line 467 "sql.y" -{ yymsp[1].minor.yy416 = createDefaultTableOptions(pCxt); } -#line 6810 "sql.c" +{ yymsp[1].minor.yy54 = createDefaultTableOptions(pCxt); } break; case 248: /* table_options ::= table_options COMMENT NK_STRING */ -#line 468 "sql.y" -{ yylhsminor.yy416 = setTableOption(pCxt, yymsp[-2].minor.yy416, TABLE_OPTION_COMMENT, &yymsp[0].minor.yy0); } -#line 6815 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setTableOption(pCxt, yymsp[-2].minor.yy54, TABLE_OPTION_COMMENT, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 249: /* table_options ::= table_options MAX_DELAY duration_list */ -#line 469 "sql.y" -{ yylhsminor.yy416 = setTableOption(pCxt, yymsp[-2].minor.yy416, TABLE_OPTION_MAXDELAY, yymsp[0].minor.yy316); } -#line 6821 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setTableOption(pCxt, yymsp[-2].minor.yy54, TABLE_OPTION_MAXDELAY, yymsp[0].minor.yy652); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 250: /* table_options ::= table_options WATERMARK duration_list */ -#line 470 "sql.y" -{ yylhsminor.yy416 = setTableOption(pCxt, yymsp[-2].minor.yy416, TABLE_OPTION_WATERMARK, yymsp[0].minor.yy316); } -#line 6827 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setTableOption(pCxt, yymsp[-2].minor.yy54, TABLE_OPTION_WATERMARK, yymsp[0].minor.yy652); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 251: /* table_options ::= table_options ROLLUP NK_LP rollup_func_list NK_RP */ -#line 471 "sql.y" -{ yylhsminor.yy416 = setTableOption(pCxt, yymsp[-4].minor.yy416, TABLE_OPTION_ROLLUP, yymsp[-1].minor.yy316); } -#line 6833 "sql.c" - yymsp[-4].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setTableOption(pCxt, yymsp[-4].minor.yy54, TABLE_OPTION_ROLLUP, yymsp[-1].minor.yy652); } + yymsp[-4].minor.yy54 = yylhsminor.yy54; break; case 252: /* table_options ::= table_options TTL NK_INTEGER */ -#line 472 "sql.y" -{ yylhsminor.yy416 = setTableOption(pCxt, yymsp[-2].minor.yy416, TABLE_OPTION_TTL, &yymsp[0].minor.yy0); } -#line 6839 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setTableOption(pCxt, yymsp[-2].minor.yy54, TABLE_OPTION_TTL, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 253: /* table_options ::= table_options SMA NK_LP col_name_list NK_RP */ -#line 473 "sql.y" -{ yylhsminor.yy416 = setTableOption(pCxt, yymsp[-4].minor.yy416, TABLE_OPTION_SMA, yymsp[-1].minor.yy316); } -#line 6845 "sql.c" - yymsp[-4].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setTableOption(pCxt, yymsp[-4].minor.yy54, TABLE_OPTION_SMA, yymsp[-1].minor.yy652); } + yymsp[-4].minor.yy54 = yylhsminor.yy54; break; case 254: /* table_options ::= table_options DELETE_MARK duration_list */ -#line 474 "sql.y" -{ yylhsminor.yy416 = setTableOption(pCxt, yymsp[-2].minor.yy416, TABLE_OPTION_DELETE_MARK, yymsp[0].minor.yy316); } -#line 6851 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setTableOption(pCxt, yymsp[-2].minor.yy54, TABLE_OPTION_DELETE_MARK, yymsp[0].minor.yy652); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 255: /* alter_table_options ::= alter_table_option */ -#line 476 "sql.y" -{ yylhsminor.yy416 = createAlterTableOptions(pCxt); yylhsminor.yy416 = setTableOption(pCxt, yylhsminor.yy416, yymsp[0].minor.yy101.type, &yymsp[0].minor.yy101.val); } -#line 6857 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createAlterTableOptions(pCxt); yylhsminor.yy54 = setTableOption(pCxt, yylhsminor.yy54, yymsp[0].minor.yy663.type, &yymsp[0].minor.yy663.val); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; case 256: /* alter_table_options ::= alter_table_options alter_table_option */ -#line 477 "sql.y" -{ yylhsminor.yy416 = setTableOption(pCxt, yymsp[-1].minor.yy416, yymsp[0].minor.yy101.type, &yymsp[0].minor.yy101.val); } -#line 6863 "sql.c" - yymsp[-1].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setTableOption(pCxt, yymsp[-1].minor.yy54, yymsp[0].minor.yy663.type, &yymsp[0].minor.yy663.val); } + yymsp[-1].minor.yy54 = yylhsminor.yy54; break; case 257: /* alter_table_option ::= COMMENT NK_STRING */ -#line 481 "sql.y" -{ yymsp[-1].minor.yy101.type = TABLE_OPTION_COMMENT; yymsp[-1].minor.yy101.val = yymsp[0].minor.yy0; } -#line 6869 "sql.c" +{ yymsp[-1].minor.yy663.type = TABLE_OPTION_COMMENT; yymsp[-1].minor.yy663.val = yymsp[0].minor.yy0; } break; case 258: /* alter_table_option ::= TTL NK_INTEGER */ -#line 482 "sql.y" -{ yymsp[-1].minor.yy101.type = TABLE_OPTION_TTL; yymsp[-1].minor.yy101.val = yymsp[0].minor.yy0; } -#line 6874 "sql.c" +{ yymsp[-1].minor.yy663.type = TABLE_OPTION_TTL; yymsp[-1].minor.yy663.val = yymsp[0].minor.yy0; } break; case 259: /* duration_list ::= duration_literal */ - case 546: /* expression_list ::= expr_or_subquery */ yytestcase(yyruleno==546); -#line 486 "sql.y" -{ yylhsminor.yy316 = createNodeList(pCxt, releaseRawExprNode(pCxt, yymsp[0].minor.yy416)); } -#line 6880 "sql.c" - yymsp[0].minor.yy316 = yylhsminor.yy316; + case 547: /* expression_list ::= expr_or_subquery */ yytestcase(yyruleno==547); +{ yylhsminor.yy652 = createNodeList(pCxt, releaseRawExprNode(pCxt, yymsp[0].minor.yy54)); } + yymsp[0].minor.yy652 = yylhsminor.yy652; break; case 260: /* duration_list ::= duration_list NK_COMMA duration_literal */ - case 547: /* expression_list ::= expression_list NK_COMMA expr_or_subquery */ yytestcase(yyruleno==547); -#line 487 "sql.y" -{ yylhsminor.yy316 = addNodeToList(pCxt, yymsp[-2].minor.yy316, releaseRawExprNode(pCxt, yymsp[0].minor.yy416)); } -#line 6887 "sql.c" - yymsp[-2].minor.yy316 = yylhsminor.yy316; + case 548: /* expression_list ::= expression_list NK_COMMA expr_or_subquery */ yytestcase(yyruleno==548); +{ yylhsminor.yy652 = addNodeToList(pCxt, yymsp[-2].minor.yy652, releaseRawExprNode(pCxt, yymsp[0].minor.yy54)); } + yymsp[-2].minor.yy652 = yylhsminor.yy652; break; case 263: /* rollup_func_name ::= function_name */ -#line 494 "sql.y" -{ yylhsminor.yy416 = createFunctionNode(pCxt, &yymsp[0].minor.yy1109, NULL); } -#line 6893 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createFunctionNode(pCxt, &yymsp[0].minor.yy837, NULL); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; case 264: /* rollup_func_name ::= FIRST */ case 265: /* rollup_func_name ::= LAST */ yytestcase(yyruleno==265); case 339: /* tag_item ::= QTAGS */ yytestcase(yyruleno==339); -#line 495 "sql.y" -{ yylhsminor.yy416 = createFunctionNode(pCxt, &yymsp[0].minor.yy0, NULL); } -#line 6901 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createFunctionNode(pCxt, &yymsp[0].minor.yy0, NULL); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; case 268: /* col_name ::= column_name */ case 340: /* tag_item ::= column_name */ yytestcase(yyruleno==340); -#line 503 "sql.y" -{ yylhsminor.yy416 = createColumnNode(pCxt, NULL, &yymsp[0].minor.yy1109); } -#line 6908 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createColumnNode(pCxt, NULL, &yymsp[0].minor.yy837); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; case 269: /* cmd ::= SHOW DNODES */ -#line 506 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_DNODES_STMT); } -#line 6914 "sql.c" break; case 270: /* cmd ::= SHOW USERS */ -#line 507 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_USERS_STMT); } -#line 6919 "sql.c" break; case 271: /* cmd ::= SHOW USERS FULL */ -#line 508 "sql.y" { pCxt->pRootNode = createShowStmtWithFull(pCxt, QUERY_NODE_SHOW_USERS_FULL_STMT); } -#line 6924 "sql.c" break; case 272: /* cmd ::= SHOW USER PRIVILEGES */ -#line 509 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_USER_PRIVILEGES_STMT); } -#line 6929 "sql.c" break; case 273: /* cmd ::= SHOW db_kind_opt DATABASES */ -#line 510 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_DATABASES_STMT); - setShowKind(pCxt, pCxt->pRootNode, yymsp[-1].minor.yy681); + (void)setShowKind(pCxt, pCxt->pRootNode, yymsp[-1].minor.yy719); } -#line 6937 "sql.c" break; case 274: /* cmd ::= SHOW table_kind_db_name_cond_opt TABLES like_pattern_opt */ -#line 514 "sql.y" { - pCxt->pRootNode = createShowTablesStmt(pCxt, yymsp[-2].minor.yy925, yymsp[0].minor.yy416, OP_TYPE_LIKE); + pCxt->pRootNode = createShowTablesStmt(pCxt, yymsp[-2].minor.yy579, yymsp[0].minor.yy54, OP_TYPE_LIKE); } -#line 6944 "sql.c" break; case 275: /* cmd ::= SHOW db_name_cond_opt STABLES like_pattern_opt */ -#line 517 "sql.y" -{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_STABLES_STMT, yymsp[-2].minor.yy416, yymsp[0].minor.yy416, OP_TYPE_LIKE); } -#line 6949 "sql.c" +{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_STABLES_STMT, yymsp[-2].minor.yy54, yymsp[0].minor.yy54, OP_TYPE_LIKE); } break; case 276: /* cmd ::= SHOW db_name_cond_opt VGROUPS */ -#line 518 "sql.y" -{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_VGROUPS_STMT, yymsp[-1].minor.yy416, NULL, OP_TYPE_LIKE); } -#line 6954 "sql.c" +{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_VGROUPS_STMT, yymsp[-1].minor.yy54, NULL, OP_TYPE_LIKE); } break; case 277: /* cmd ::= SHOW MNODES */ -#line 519 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_MNODES_STMT); } -#line 6959 "sql.c" break; case 278: /* cmd ::= SHOW QNODES */ -#line 521 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_QNODES_STMT); } -#line 6964 "sql.c" break; case 279: /* cmd ::= SHOW ARBGROUPS */ -#line 522 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_ARBGROUPS_STMT); } -#line 6969 "sql.c" break; case 280: /* cmd ::= SHOW FUNCTIONS */ -#line 523 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_FUNCTIONS_STMT); } -#line 6974 "sql.c" break; case 281: /* cmd ::= SHOW INDEXES FROM table_name_cond from_db_opt */ -#line 524 "sql.y" -{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_INDEXES_STMT, yymsp[0].minor.yy416, yymsp[-1].minor.yy416, OP_TYPE_EQUAL); } -#line 6979 "sql.c" +{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_INDEXES_STMT, yymsp[0].minor.yy54, yymsp[-1].minor.yy54, OP_TYPE_EQUAL); } break; case 282: /* cmd ::= SHOW INDEXES FROM db_name NK_DOT table_name */ -#line 525 "sql.y" -{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_INDEXES_STMT, createIdentifierValueNode(pCxt, &yymsp[-2].minor.yy1109), createIdentifierValueNode(pCxt, &yymsp[0].minor.yy1109), OP_TYPE_EQUAL); } -#line 6984 "sql.c" +{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_INDEXES_STMT, createIdentifierValueNode(pCxt, &yymsp[-2].minor.yy837), createIdentifierValueNode(pCxt, &yymsp[0].minor.yy837), OP_TYPE_EQUAL); } break; case 283: /* cmd ::= SHOW STREAMS */ -#line 526 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_STREAMS_STMT); } -#line 6989 "sql.c" break; case 284: /* cmd ::= SHOW ACCOUNTS */ -#line 527 "sql.y" { pCxt->errCode = generateSyntaxErrMsg(&pCxt->msgBuf, TSDB_CODE_PAR_EXPRIE_STATEMENT); } -#line 6994 "sql.c" break; case 285: /* cmd ::= SHOW APPS */ -#line 528 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_APPS_STMT); } -#line 6999 "sql.c" break; case 286: /* cmd ::= SHOW CONNECTIONS */ -#line 529 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_CONNECTIONS_STMT); } -#line 7004 "sql.c" break; case 287: /* cmd ::= SHOW LICENCES */ case 288: /* cmd ::= SHOW GRANTS */ yytestcase(yyruleno==288); -#line 530 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_LICENCES_STMT); } -#line 7010 "sql.c" break; case 289: /* cmd ::= SHOW GRANTS FULL */ -#line 532 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_GRANTS_FULL_STMT); } -#line 7015 "sql.c" break; case 290: /* cmd ::= SHOW GRANTS LOGS */ -#line 533 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_GRANTS_LOGS_STMT); } -#line 7020 "sql.c" break; case 291: /* cmd ::= SHOW CLUSTER MACHINES */ -#line 534 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_CLUSTER_MACHINES_STMT); } -#line 7025 "sql.c" break; case 292: /* cmd ::= SHOW CREATE DATABASE db_name */ -#line 535 "sql.y" -{ pCxt->pRootNode = createShowCreateDatabaseStmt(pCxt, &yymsp[0].minor.yy1109); } -#line 7030 "sql.c" +{ pCxt->pRootNode = createShowCreateDatabaseStmt(pCxt, &yymsp[0].minor.yy837); } break; case 293: /* cmd ::= SHOW CREATE TABLE full_table_name */ -#line 536 "sql.y" -{ pCxt->pRootNode = createShowCreateTableStmt(pCxt, QUERY_NODE_SHOW_CREATE_TABLE_STMT, yymsp[0].minor.yy416); } -#line 7035 "sql.c" +{ pCxt->pRootNode = createShowCreateTableStmt(pCxt, QUERY_NODE_SHOW_CREATE_TABLE_STMT, yymsp[0].minor.yy54); } break; case 294: /* cmd ::= SHOW CREATE STABLE full_table_name */ -#line 537 "sql.y" { pCxt->pRootNode = createShowCreateTableStmt(pCxt, QUERY_NODE_SHOW_CREATE_STABLE_STMT, -yymsp[0].minor.yy416); } -#line 7041 "sql.c" +yymsp[0].minor.yy54); } break; case 295: /* cmd ::= SHOW ENCRYPTIONS */ -#line 539 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_ENCRYPTIONS_STMT); } -#line 7046 "sql.c" break; case 296: /* cmd ::= SHOW QUERIES */ -#line 540 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_QUERIES_STMT); } -#line 7051 "sql.c" break; case 297: /* cmd ::= SHOW SCORES */ -#line 541 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_SCORES_STMT); } -#line 7056 "sql.c" break; case 298: /* cmd ::= SHOW TOPICS */ -#line 542 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_TOPICS_STMT); } -#line 7061 "sql.c" break; case 299: /* cmd ::= SHOW VARIABLES */ case 300: /* cmd ::= SHOW CLUSTER VARIABLES */ yytestcase(yyruleno==300); -#line 543 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_VARIABLES_STMT); } -#line 7067 "sql.c" break; case 301: /* cmd ::= SHOW LOCAL VARIABLES */ -#line 545 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_LOCAL_VARIABLES_STMT); } -#line 7072 "sql.c" break; case 302: /* cmd ::= SHOW DNODE NK_INTEGER VARIABLES like_pattern_opt */ -#line 546 "sql.y" -{ pCxt->pRootNode = createShowDnodeVariablesStmt(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[-2].minor.yy0), yymsp[0].minor.yy416); } -#line 7077 "sql.c" +{ pCxt->pRootNode = createShowDnodeVariablesStmt(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[-2].minor.yy0), yymsp[0].minor.yy54); } break; case 303: /* cmd ::= SHOW BNODES */ -#line 547 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_BNODES_STMT); } -#line 7082 "sql.c" break; case 304: /* cmd ::= SHOW SNODES */ -#line 548 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_SNODES_STMT); } -#line 7087 "sql.c" break; case 305: /* cmd ::= SHOW CLUSTER */ -#line 549 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_CLUSTER_STMT); } -#line 7092 "sql.c" break; case 306: /* cmd ::= SHOW TRANSACTIONS */ -#line 550 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_TRANSACTIONS_STMT); } -#line 7097 "sql.c" break; case 307: /* cmd ::= SHOW TABLE DISTRIBUTED full_table_name */ -#line 551 "sql.y" -{ pCxt->pRootNode = createShowTableDistributedStmt(pCxt, yymsp[0].minor.yy416); } -#line 7102 "sql.c" +{ pCxt->pRootNode = createShowTableDistributedStmt(pCxt, yymsp[0].minor.yy54); } break; case 308: /* cmd ::= SHOW CONSUMERS */ -#line 552 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_CONSUMERS_STMT); } -#line 7107 "sql.c" break; case 309: /* cmd ::= SHOW SUBSCRIPTIONS */ -#line 553 "sql.y" { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_SUBSCRIPTIONS_STMT); } -#line 7112 "sql.c" break; case 310: /* cmd ::= SHOW TAGS FROM table_name_cond from_db_opt */ -#line 554 "sql.y" -{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_TAGS_STMT, yymsp[0].minor.yy416, yymsp[-1].minor.yy416, OP_TYPE_EQUAL); } -#line 7117 "sql.c" +{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_TAGS_STMT, yymsp[0].minor.yy54, yymsp[-1].minor.yy54, OP_TYPE_EQUAL); } break; case 311: /* cmd ::= SHOW TAGS FROM db_name NK_DOT table_name */ -#line 555 "sql.y" -{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_TAGS_STMT, createIdentifierValueNode(pCxt, &yymsp[-2].minor.yy1109), createIdentifierValueNode(pCxt, &yymsp[0].minor.yy1109), OP_TYPE_EQUAL); } -#line 7122 "sql.c" +{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_TAGS_STMT, createIdentifierValueNode(pCxt, &yymsp[-2].minor.yy837), createIdentifierValueNode(pCxt, &yymsp[0].minor.yy837), OP_TYPE_EQUAL); } break; case 312: /* cmd ::= SHOW TABLE TAGS tag_list_opt FROM table_name_cond from_db_opt */ -#line 556 "sql.y" -{ pCxt->pRootNode = createShowTableTagsStmt(pCxt, yymsp[-1].minor.yy416, yymsp[0].minor.yy416, yymsp[-3].minor.yy316); } -#line 7127 "sql.c" +{ pCxt->pRootNode = createShowTableTagsStmt(pCxt, yymsp[-1].minor.yy54, yymsp[0].minor.yy54, yymsp[-3].minor.yy652); } break; case 313: /* cmd ::= SHOW TABLE TAGS tag_list_opt FROM db_name NK_DOT table_name */ -#line 557 "sql.y" -{ pCxt->pRootNode = createShowTableTagsStmt(pCxt, createIdentifierValueNode(pCxt, &yymsp[0].minor.yy1109), createIdentifierValueNode(pCxt, &yymsp[-2].minor.yy1109), yymsp[-4].minor.yy316); } -#line 7132 "sql.c" +{ pCxt->pRootNode = createShowTableTagsStmt(pCxt, createIdentifierValueNode(pCxt, &yymsp[0].minor.yy837), createIdentifierValueNode(pCxt, &yymsp[-2].minor.yy837), yymsp[-4].minor.yy652); } break; case 314: /* cmd ::= SHOW VNODES ON DNODE NK_INTEGER */ -#line 558 "sql.y" { pCxt->pRootNode = createShowVnodesStmt(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0), NULL); } -#line 7137 "sql.c" break; case 315: /* cmd ::= SHOW VNODES */ -#line 559 "sql.y" { pCxt->pRootNode = createShowVnodesStmt(pCxt, NULL, NULL); } -#line 7142 "sql.c" break; case 316: /* cmd ::= SHOW db_name_cond_opt ALIVE */ -#line 561 "sql.y" -{ pCxt->pRootNode = createShowAliveStmt(pCxt, yymsp[-1].minor.yy416, QUERY_NODE_SHOW_DB_ALIVE_STMT); } -#line 7147 "sql.c" +{ pCxt->pRootNode = createShowAliveStmt(pCxt, yymsp[-1].minor.yy54, QUERY_NODE_SHOW_DB_ALIVE_STMT); } break; case 317: /* cmd ::= SHOW CLUSTER ALIVE */ -#line 562 "sql.y" { pCxt->pRootNode = createShowAliveStmt(pCxt, NULL, QUERY_NODE_SHOW_CLUSTER_ALIVE_STMT); } -#line 7152 "sql.c" break; case 318: /* cmd ::= SHOW db_name_cond_opt VIEWS like_pattern_opt */ -#line 563 "sql.y" -{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_VIEWS_STMT, yymsp[-2].minor.yy416, yymsp[0].minor.yy416, OP_TYPE_LIKE); } -#line 7157 "sql.c" +{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_VIEWS_STMT, yymsp[-2].minor.yy54, yymsp[0].minor.yy54, OP_TYPE_LIKE); } break; case 319: /* cmd ::= SHOW CREATE VIEW full_table_name */ -#line 564 "sql.y" -{ pCxt->pRootNode = createShowCreateViewStmt(pCxt, QUERY_NODE_SHOW_CREATE_VIEW_STMT, yymsp[0].minor.yy416); } -#line 7162 "sql.c" +{ pCxt->pRootNode = createShowCreateViewStmt(pCxt, QUERY_NODE_SHOW_CREATE_VIEW_STMT, yymsp[0].minor.yy54); } break; case 320: /* cmd ::= SHOW COMPACTS */ -#line 565 "sql.y" { pCxt->pRootNode = createShowCompactsStmt(pCxt, QUERY_NODE_SHOW_COMPACTS_STMT); } -#line 7167 "sql.c" break; case 321: /* cmd ::= SHOW COMPACT NK_INTEGER */ -#line 566 "sql.y" { pCxt->pRootNode = createShowCompactDetailsStmt(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0)); } -#line 7172 "sql.c" break; case 322: /* table_kind_db_name_cond_opt ::= */ -#line 570 "sql.y" -{ yymsp[1].minor.yy925.kind = SHOW_KIND_ALL; yymsp[1].minor.yy925.dbName = nil_token; } -#line 7177 "sql.c" +{ yymsp[1].minor.yy579.kind = SHOW_KIND_ALL; yymsp[1].minor.yy579.dbName = nil_token; } break; case 323: /* table_kind_db_name_cond_opt ::= table_kind */ -#line 571 "sql.y" -{ yylhsminor.yy925.kind = yymsp[0].minor.yy681; yylhsminor.yy925.dbName = nil_token; } -#line 7182 "sql.c" - yymsp[0].minor.yy925 = yylhsminor.yy925; +{ yylhsminor.yy579.kind = yymsp[0].minor.yy719; yylhsminor.yy579.dbName = nil_token; } + yymsp[0].minor.yy579 = yylhsminor.yy579; break; case 324: /* table_kind_db_name_cond_opt ::= db_name NK_DOT */ -#line 572 "sql.y" -{ yylhsminor.yy925.kind = SHOW_KIND_ALL; yylhsminor.yy925.dbName = yymsp[-1].minor.yy1109; } -#line 7188 "sql.c" - yymsp[-1].minor.yy925 = yylhsminor.yy925; +{ yylhsminor.yy579.kind = SHOW_KIND_ALL; yylhsminor.yy579.dbName = yymsp[-1].minor.yy837; } + yymsp[-1].minor.yy579 = yylhsminor.yy579; break; case 325: /* table_kind_db_name_cond_opt ::= table_kind db_name NK_DOT */ -#line 573 "sql.y" -{ yylhsminor.yy925.kind = yymsp[-2].minor.yy681; yylhsminor.yy925.dbName = yymsp[-1].minor.yy1109; } -#line 7194 "sql.c" - yymsp[-2].minor.yy925 = yylhsminor.yy925; +{ yylhsminor.yy579.kind = yymsp[-2].minor.yy719; yylhsminor.yy579.dbName = yymsp[-1].minor.yy837; } + yymsp[-2].minor.yy579 = yylhsminor.yy579; break; case 326: /* table_kind ::= NORMAL */ -#line 577 "sql.y" -{ yymsp[0].minor.yy681 = SHOW_KIND_TABLES_NORMAL; } -#line 7200 "sql.c" +{ yymsp[0].minor.yy719 = SHOW_KIND_TABLES_NORMAL; } break; case 327: /* table_kind ::= CHILD */ -#line 578 "sql.y" -{ yymsp[0].minor.yy681 = SHOW_KIND_TABLES_CHILD; } -#line 7205 "sql.c" +{ yymsp[0].minor.yy719 = SHOW_KIND_TABLES_CHILD; } break; case 328: /* db_name_cond_opt ::= */ case 333: /* from_db_opt ::= */ yytestcase(yyruleno==333); -#line 580 "sql.y" -{ yymsp[1].minor.yy416 = createDefaultDatabaseCondValue(pCxt); } -#line 7211 "sql.c" +{ yymsp[1].minor.yy54 = createDefaultDatabaseCondValue(pCxt); } break; case 329: /* db_name_cond_opt ::= db_name NK_DOT */ -#line 581 "sql.y" -{ yylhsminor.yy416 = createIdentifierValueNode(pCxt, &yymsp[-1].minor.yy1109); } -#line 7216 "sql.c" - yymsp[-1].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createIdentifierValueNode(pCxt, &yymsp[-1].minor.yy837); } + yymsp[-1].minor.yy54 = yylhsminor.yy54; break; case 331: /* like_pattern_opt ::= LIKE NK_STRING */ -#line 584 "sql.y" -{ yymsp[-1].minor.yy416 = createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0); } -#line 7222 "sql.c" +{ yymsp[-1].minor.yy54 = createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0); } break; case 332: /* table_name_cond ::= table_name */ -#line 586 "sql.y" -{ yylhsminor.yy416 = createIdentifierValueNode(pCxt, &yymsp[0].minor.yy1109); } -#line 7227 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createIdentifierValueNode(pCxt, &yymsp[0].minor.yy837); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; case 334: /* from_db_opt ::= FROM db_name */ -#line 589 "sql.y" -{ yymsp[-1].minor.yy416 = createIdentifierValueNode(pCxt, &yymsp[0].minor.yy1109); } -#line 7233 "sql.c" +{ yymsp[-1].minor.yy54 = createIdentifierValueNode(pCxt, &yymsp[0].minor.yy837); } break; case 338: /* tag_item ::= TBNAME */ -#line 597 "sql.y" -{ yylhsminor.yy416 = setProjectionAlias(pCxt, createFunctionNode(pCxt, &yymsp[0].minor.yy0, NULL), &yymsp[0].minor.yy0); } -#line 7238 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setProjectionAlias(pCxt, createFunctionNode(pCxt, &yymsp[0].minor.yy0, NULL), &yymsp[0].minor.yy0); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; case 341: /* tag_item ::= column_name column_alias */ -#line 600 "sql.y" -{ yylhsminor.yy416 = setProjectionAlias(pCxt, createColumnNode(pCxt, NULL, &yymsp[-1].minor.yy1109), &yymsp[0].minor.yy1109); } -#line 7244 "sql.c" - yymsp[-1].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setProjectionAlias(pCxt, createColumnNode(pCxt, NULL, &yymsp[-1].minor.yy837), &yymsp[0].minor.yy837); } + yymsp[-1].minor.yy54 = yylhsminor.yy54; break; case 342: /* tag_item ::= column_name AS column_alias */ -#line 601 "sql.y" -{ yylhsminor.yy416 = setProjectionAlias(pCxt, createColumnNode(pCxt, NULL, &yymsp[-2].minor.yy1109), &yymsp[0].minor.yy1109); } -#line 7250 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setProjectionAlias(pCxt, createColumnNode(pCxt, NULL, &yymsp[-2].minor.yy837), &yymsp[0].minor.yy837); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 343: /* db_kind_opt ::= */ -#line 605 "sql.y" -{ yymsp[1].minor.yy681 = SHOW_KIND_ALL; } -#line 7256 "sql.c" +{ yymsp[1].minor.yy719 = SHOW_KIND_ALL; } break; case 344: /* db_kind_opt ::= USER */ -#line 606 "sql.y" -{ yymsp[0].minor.yy681 = SHOW_KIND_DATABASES_USER; } -#line 7261 "sql.c" +{ yymsp[0].minor.yy719 = SHOW_KIND_DATABASES_USER; } break; case 345: /* db_kind_opt ::= SYSTEM */ -#line 607 "sql.y" -{ yymsp[0].minor.yy681 = SHOW_KIND_DATABASES_SYSTEM; } -#line 7266 "sql.c" +{ yymsp[0].minor.yy719 = SHOW_KIND_DATABASES_SYSTEM; } break; case 346: /* cmd ::= CREATE TSMA not_exists_opt tsma_name ON full_table_name tsma_func_list INTERVAL NK_LP duration_literal NK_RP */ -#line 613 "sql.y" -{ pCxt->pRootNode = createCreateTSMAStmt(pCxt, yymsp[-8].minor.yy209, &yymsp[-7].minor.yy1109, yymsp[-4].minor.yy416, yymsp[-5].minor.yy416, releaseRawExprNode(pCxt, yymsp[-1].minor.yy416)); } -#line 7271 "sql.c" +{ pCxt->pRootNode = createCreateTSMAStmt(pCxt, yymsp[-8].minor.yy325, &yymsp[-7].minor.yy837, yymsp[-4].minor.yy54, yymsp[-5].minor.yy54, releaseRawExprNode(pCxt, yymsp[-1].minor.yy54)); } break; case 347: /* cmd ::= CREATE RECURSIVE TSMA not_exists_opt tsma_name ON full_table_name INTERVAL NK_LP duration_literal NK_RP */ -#line 615 "sql.y" -{ pCxt->pRootNode = createCreateTSMAStmt(pCxt, yymsp[-7].minor.yy209, &yymsp[-6].minor.yy1109, NULL, yymsp[-4].minor.yy416, releaseRawExprNode(pCxt, yymsp[-1].minor.yy416)); } -#line 7276 "sql.c" +{ pCxt->pRootNode = createCreateTSMAStmt(pCxt, yymsp[-7].minor.yy325, &yymsp[-6].minor.yy837, NULL, yymsp[-4].minor.yy54, releaseRawExprNode(pCxt, yymsp[-1].minor.yy54)); } break; case 348: /* cmd ::= DROP TSMA exists_opt full_tsma_name */ -#line 616 "sql.y" -{ pCxt->pRootNode = createDropTSMAStmt(pCxt, yymsp[-1].minor.yy209, yymsp[0].minor.yy416); } -#line 7281 "sql.c" +{ pCxt->pRootNode = createDropTSMAStmt(pCxt, yymsp[-1].minor.yy325, yymsp[0].minor.yy54); } break; case 349: /* cmd ::= SHOW db_name_cond_opt TSMAS */ -#line 617 "sql.y" -{ pCxt->pRootNode = createShowTSMASStmt(pCxt, yymsp[-1].minor.yy416); } -#line 7286 "sql.c" +{ pCxt->pRootNode = createShowTSMASStmt(pCxt, yymsp[-1].minor.yy54); } break; case 352: /* tsma_func_list ::= FUNCTION NK_LP func_list NK_RP */ -#line 624 "sql.y" -{ yymsp[-3].minor.yy416 = createTSMAOptions(pCxt, yymsp[-1].minor.yy316); } -#line 7291 "sql.c" +{ yymsp[-3].minor.yy54 = createTSMAOptions(pCxt, yymsp[-1].minor.yy652); } break; case 353: /* cmd ::= CREATE SMA INDEX not_exists_opt col_name ON full_table_name index_options */ -#line 628 "sql.y" -{ pCxt->pRootNode = createCreateIndexStmt(pCxt, INDEX_TYPE_SMA, yymsp[-4].minor.yy209, yymsp[-3].minor.yy416, yymsp[-1].minor.yy416, NULL, yymsp[0].minor.yy416); } -#line 7296 "sql.c" +{ pCxt->pRootNode = createCreateIndexStmt(pCxt, INDEX_TYPE_SMA, yymsp[-4].minor.yy325, yymsp[-3].minor.yy54, yymsp[-1].minor.yy54, NULL, yymsp[0].minor.yy54); } break; case 354: /* cmd ::= CREATE INDEX not_exists_opt col_name ON full_table_name NK_LP col_name_list NK_RP */ -#line 630 "sql.y" -{ pCxt->pRootNode = createCreateIndexStmt(pCxt, INDEX_TYPE_NORMAL, yymsp[-6].minor.yy209, yymsp[-5].minor.yy416, yymsp[-3].minor.yy416, yymsp[-1].minor.yy316, NULL); } -#line 7301 "sql.c" +{ pCxt->pRootNode = createCreateIndexStmt(pCxt, INDEX_TYPE_NORMAL, yymsp[-6].minor.yy325, yymsp[-5].minor.yy54, yymsp[-3].minor.yy54, yymsp[-1].minor.yy652, NULL); } break; case 355: /* cmd ::= DROP INDEX exists_opt full_index_name */ -#line 631 "sql.y" -{ pCxt->pRootNode = createDropIndexStmt(pCxt, yymsp[-1].minor.yy209, yymsp[0].minor.yy416); } -#line 7306 "sql.c" +{ pCxt->pRootNode = createDropIndexStmt(pCxt, yymsp[-1].minor.yy325, yymsp[0].minor.yy54); } break; case 356: /* full_index_name ::= index_name */ -#line 633 "sql.y" -{ yylhsminor.yy416 = createRealTableNodeForIndexName(pCxt, NULL, &yymsp[0].minor.yy1109); } -#line 7311 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createRealTableNodeForIndexName(pCxt, NULL, &yymsp[0].minor.yy837); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; case 357: /* full_index_name ::= db_name NK_DOT index_name */ -#line 634 "sql.y" -{ yylhsminor.yy416 = createRealTableNodeForIndexName(pCxt, &yymsp[-2].minor.yy1109, &yymsp[0].minor.yy1109); } -#line 7317 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createRealTableNodeForIndexName(pCxt, &yymsp[-2].minor.yy837, &yymsp[0].minor.yy837); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 358: /* index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_RP sliding_opt sma_stream_opt */ -#line 637 "sql.y" -{ yymsp[-9].minor.yy416 = createIndexOption(pCxt, yymsp[-7].minor.yy316, releaseRawExprNode(pCxt, yymsp[-3].minor.yy416), NULL, yymsp[-1].minor.yy416, yymsp[0].minor.yy416); } -#line 7323 "sql.c" +{ yymsp[-9].minor.yy54 = createIndexOption(pCxt, yymsp[-7].minor.yy652, releaseRawExprNode(pCxt, yymsp[-3].minor.yy54), NULL, yymsp[-1].minor.yy54, yymsp[0].minor.yy54); } break; case 359: /* 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 640 "sql.y" -{ yymsp[-11].minor.yy416 = createIndexOption(pCxt, yymsp[-9].minor.yy316, releaseRawExprNode(pCxt, yymsp[-5].minor.yy416), releaseRawExprNode(pCxt, yymsp[-3].minor.yy416), yymsp[-1].minor.yy416, yymsp[0].minor.yy416); } -#line 7328 "sql.c" +{ yymsp[-11].minor.yy54 = createIndexOption(pCxt, yymsp[-9].minor.yy652, releaseRawExprNode(pCxt, yymsp[-5].minor.yy54), releaseRawExprNode(pCxt, yymsp[-3].minor.yy54), yymsp[-1].minor.yy54, yymsp[0].minor.yy54); } break; case 362: /* func ::= sma_func_name NK_LP expression_list NK_RP */ -#line 647 "sql.y" -{ yylhsminor.yy416 = createFunctionNode(pCxt, &yymsp[-3].minor.yy1109, yymsp[-1].minor.yy316); } -#line 7333 "sql.c" - yymsp[-3].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createFunctionNode(pCxt, &yymsp[-3].minor.yy837, yymsp[-1].minor.yy652); } + yymsp[-3].minor.yy54 = yylhsminor.yy54; break; case 363: /* sma_func_name ::= function_name */ - case 637: /* alias_opt ::= table_alias */ yytestcase(yyruleno==637); -#line 651 "sql.y" -{ yylhsminor.yy1109 = yymsp[0].minor.yy1109; } -#line 7340 "sql.c" - yymsp[0].minor.yy1109 = yylhsminor.yy1109; + case 638: /* alias_opt ::= table_alias */ yytestcase(yyruleno==638); +{ yylhsminor.yy837 = yymsp[0].minor.yy837; } + yymsp[0].minor.yy837 = yylhsminor.yy837; break; case 368: /* sma_stream_opt ::= */ case 418: /* stream_options ::= */ yytestcase(yyruleno==418); -#line 657 "sql.y" -{ yymsp[1].minor.yy416 = createStreamOptions(pCxt); } -#line 7347 "sql.c" +{ yymsp[1].minor.yy54 = createStreamOptions(pCxt); } break; case 369: /* sma_stream_opt ::= sma_stream_opt WATERMARK duration_literal */ -#line 658 "sql.y" -{ ((SStreamOptions*)yymsp[-2].minor.yy416)->pWatermark = releaseRawExprNode(pCxt, yymsp[0].minor.yy416); yylhsminor.yy416 = yymsp[-2].minor.yy416; } -#line 7352 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ ((SStreamOptions*)yymsp[-2].minor.yy54)->pWatermark = releaseRawExprNode(pCxt, yymsp[0].minor.yy54); yylhsminor.yy54 = yymsp[-2].minor.yy54; } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 370: /* sma_stream_opt ::= sma_stream_opt MAX_DELAY duration_literal */ -#line 659 "sql.y" -{ ((SStreamOptions*)yymsp[-2].minor.yy416)->pDelay = releaseRawExprNode(pCxt, yymsp[0].minor.yy416); yylhsminor.yy416 = yymsp[-2].minor.yy416; } -#line 7358 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ ((SStreamOptions*)yymsp[-2].minor.yy54)->pDelay = releaseRawExprNode(pCxt, yymsp[0].minor.yy54); yylhsminor.yy54 = yymsp[-2].minor.yy54; } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 371: /* sma_stream_opt ::= sma_stream_opt DELETE_MARK duration_literal */ -#line 660 "sql.y" -{ ((SStreamOptions*)yymsp[-2].minor.yy416)->pDeleteMark = releaseRawExprNode(pCxt, yymsp[0].minor.yy416); yylhsminor.yy416 = yymsp[-2].minor.yy416; } -#line 7364 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ ((SStreamOptions*)yymsp[-2].minor.yy54)->pDeleteMark = releaseRawExprNode(pCxt, yymsp[0].minor.yy54); yylhsminor.yy54 = yymsp[-2].minor.yy54; } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 372: /* with_meta ::= AS */ -#line 665 "sql.y" -{ yymsp[0].minor.yy820 = 0; } -#line 7370 "sql.c" +{ yymsp[0].minor.yy332 = 0; } break; case 373: /* with_meta ::= WITH META AS */ -#line 666 "sql.y" -{ yymsp[-2].minor.yy820 = 1; } -#line 7375 "sql.c" +{ yymsp[-2].minor.yy332 = 1; } break; case 374: /* with_meta ::= ONLY META AS */ -#line 667 "sql.y" -{ yymsp[-2].minor.yy820 = 2; } -#line 7380 "sql.c" +{ yymsp[-2].minor.yy332 = 2; } break; case 375: /* cmd ::= CREATE TOPIC not_exists_opt topic_name AS query_or_subquery */ -#line 669 "sql.y" -{ pCxt->pRootNode = createCreateTopicStmtUseQuery(pCxt, yymsp[-3].minor.yy209, &yymsp[-2].minor.yy1109, yymsp[0].minor.yy416); } -#line 7385 "sql.c" +{ pCxt->pRootNode = createCreateTopicStmtUseQuery(pCxt, yymsp[-3].minor.yy325, &yymsp[-2].minor.yy837, yymsp[0].minor.yy54); } break; case 376: /* cmd ::= CREATE TOPIC not_exists_opt topic_name with_meta DATABASE db_name */ -#line 671 "sql.y" -{ pCxt->pRootNode = createCreateTopicStmtUseDb(pCxt, yymsp[-4].minor.yy209, &yymsp[-3].minor.yy1109, &yymsp[0].minor.yy1109, yymsp[-2].minor.yy820); } -#line 7390 "sql.c" +{ pCxt->pRootNode = createCreateTopicStmtUseDb(pCxt, yymsp[-4].minor.yy325, &yymsp[-3].minor.yy837, &yymsp[0].minor.yy837, yymsp[-2].minor.yy332); } break; case 377: /* cmd ::= CREATE TOPIC not_exists_opt topic_name with_meta STABLE full_table_name where_clause_opt */ -#line 673 "sql.y" -{ pCxt->pRootNode = createCreateTopicStmtUseTable(pCxt, yymsp[-5].minor.yy209, &yymsp[-4].minor.yy1109, yymsp[-1].minor.yy416, yymsp[-3].minor.yy820, yymsp[0].minor.yy416); } -#line 7395 "sql.c" +{ pCxt->pRootNode = createCreateTopicStmtUseTable(pCxt, yymsp[-5].minor.yy325, &yymsp[-4].minor.yy837, yymsp[-1].minor.yy54, yymsp[-3].minor.yy332, yymsp[0].minor.yy54); } break; case 378: /* cmd ::= DROP TOPIC exists_opt topic_name */ -#line 675 "sql.y" -{ pCxt->pRootNode = createDropTopicStmt(pCxt, yymsp[-1].minor.yy209, &yymsp[0].minor.yy1109); } -#line 7400 "sql.c" +{ pCxt->pRootNode = createDropTopicStmt(pCxt, yymsp[-1].minor.yy325, &yymsp[0].minor.yy837); } break; case 379: /* cmd ::= DROP CONSUMER GROUP exists_opt cgroup_name ON topic_name */ -#line 676 "sql.y" -{ pCxt->pRootNode = createDropCGroupStmt(pCxt, yymsp[-3].minor.yy209, &yymsp[-2].minor.yy1109, &yymsp[0].minor.yy1109); } -#line 7405 "sql.c" +{ pCxt->pRootNode = createDropCGroupStmt(pCxt, yymsp[-3].minor.yy325, &yymsp[-2].minor.yy837, &yymsp[0].minor.yy837); } break; case 380: /* cmd ::= DESC full_table_name */ case 381: /* cmd ::= DESCRIBE full_table_name */ yytestcase(yyruleno==381); -#line 679 "sql.y" -{ pCxt->pRootNode = createDescribeStmt(pCxt, yymsp[0].minor.yy416); } -#line 7411 "sql.c" +{ pCxt->pRootNode = createDescribeStmt(pCxt, yymsp[0].minor.yy54); } break; case 382: /* cmd ::= RESET QUERY CACHE */ -#line 683 "sql.y" { pCxt->pRootNode = createResetQueryCacheStmt(pCxt); } -#line 7416 "sql.c" break; case 383: /* cmd ::= EXPLAIN analyze_opt explain_options query_or_subquery */ case 384: /* cmd ::= EXPLAIN analyze_opt explain_options insert_query */ yytestcase(yyruleno==384); -#line 686 "sql.y" -{ pCxt->pRootNode = createExplainStmt(pCxt, yymsp[-2].minor.yy209, yymsp[-1].minor.yy416, yymsp[0].minor.yy416); } -#line 7422 "sql.c" +{ pCxt->pRootNode = createExplainStmt(pCxt, yymsp[-2].minor.yy325, yymsp[-1].minor.yy54, yymsp[0].minor.yy54); } break; case 387: /* explain_options ::= */ -#line 694 "sql.y" -{ yymsp[1].minor.yy416 = createDefaultExplainOptions(pCxt); } -#line 7427 "sql.c" +{ yymsp[1].minor.yy54 = createDefaultExplainOptions(pCxt); } break; case 388: /* explain_options ::= explain_options VERBOSE NK_BOOL */ -#line 695 "sql.y" -{ yylhsminor.yy416 = setExplainVerbose(pCxt, yymsp[-2].minor.yy416, &yymsp[0].minor.yy0); } -#line 7432 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setExplainVerbose(pCxt, yymsp[-2].minor.yy54, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 389: /* explain_options ::= explain_options RATIO NK_FLOAT */ -#line 696 "sql.y" -{ yylhsminor.yy416 = setExplainRatio(pCxt, yymsp[-2].minor.yy416, &yymsp[0].minor.yy0); } -#line 7438 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = setExplainRatio(pCxt, yymsp[-2].minor.yy54, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 390: /* 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 701 "sql.y" -{ pCxt->pRootNode = createCreateFunctionStmt(pCxt, yymsp[-7].minor.yy209, yymsp[-9].minor.yy209, &yymsp[-6].minor.yy1109, &yymsp[-4].minor.yy0, yymsp[-2].minor.yy952, yymsp[-1].minor.yy820, &yymsp[0].minor.yy1109, yymsp[-10].minor.yy209); } -#line 7444 "sql.c" +{ pCxt->pRootNode = createCreateFunctionStmt(pCxt, yymsp[-7].minor.yy325, yymsp[-9].minor.yy325, &yymsp[-6].minor.yy837, &yymsp[-4].minor.yy0, yymsp[-2].minor.yy84, yymsp[-1].minor.yy332, &yymsp[0].minor.yy837, yymsp[-10].minor.yy325); } break; case 391: /* cmd ::= DROP FUNCTION exists_opt function_name */ -#line 702 "sql.y" -{ pCxt->pRootNode = createDropFunctionStmt(pCxt, yymsp[-1].minor.yy209, &yymsp[0].minor.yy1109); } -#line 7449 "sql.c" +{ pCxt->pRootNode = createDropFunctionStmt(pCxt, yymsp[-1].minor.yy325, &yymsp[0].minor.yy837); } break; case 396: /* language_opt ::= */ - case 441: /* on_vgroup_id ::= */ yytestcase(yyruleno==441); -#line 716 "sql.y" -{ yymsp[1].minor.yy1109 = nil_token; } -#line 7455 "sql.c" + case 442: /* on_vgroup_id ::= */ yytestcase(yyruleno==442); +{ yymsp[1].minor.yy837 = nil_token; } break; case 397: /* language_opt ::= LANGUAGE NK_STRING */ - case 442: /* on_vgroup_id ::= ON NK_INTEGER */ yytestcase(yyruleno==442); -#line 717 "sql.y" -{ yymsp[-1].minor.yy1109 = yymsp[0].minor.yy0; } -#line 7461 "sql.c" + case 443: /* on_vgroup_id ::= ON NK_INTEGER */ yytestcase(yyruleno==443); +{ yymsp[-1].minor.yy837 = yymsp[0].minor.yy0; } break; case 400: /* cmd ::= CREATE or_replace_opt VIEW full_view_name AS query_or_subquery */ -#line 726 "sql.y" -{ pCxt->pRootNode = createCreateViewStmt(pCxt, yymsp[-4].minor.yy209, yymsp[-2].minor.yy416, &yymsp[-1].minor.yy0, yymsp[0].minor.yy416); } -#line 7466 "sql.c" +{ pCxt->pRootNode = createCreateViewStmt(pCxt, yymsp[-4].minor.yy325, yymsp[-2].minor.yy54, &yymsp[-1].minor.yy0, yymsp[0].minor.yy54); } break; case 401: /* cmd ::= DROP VIEW exists_opt full_view_name */ -#line 727 "sql.y" -{ pCxt->pRootNode = createDropViewStmt(pCxt, yymsp[-1].minor.yy209, yymsp[0].minor.yy416); } -#line 7471 "sql.c" +{ pCxt->pRootNode = createDropViewStmt(pCxt, yymsp[-1].minor.yy325, yymsp[0].minor.yy54); } break; case 402: /* full_view_name ::= view_name */ -#line 729 "sql.y" -{ yylhsminor.yy416 = createViewNode(pCxt, NULL, &yymsp[0].minor.yy1109); } -#line 7476 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createViewNode(pCxt, NULL, &yymsp[0].minor.yy837); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; case 403: /* full_view_name ::= db_name NK_DOT view_name */ -#line 730 "sql.y" -{ yylhsminor.yy416 = createViewNode(pCxt, &yymsp[-2].minor.yy1109, &yymsp[0].minor.yy1109); } -#line 7482 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createViewNode(pCxt, &yymsp[-2].minor.yy837, &yymsp[0].minor.yy837); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 404: /* 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 735 "sql.y" -{ pCxt->pRootNode = createCreateStreamStmt(pCxt, yymsp[-9].minor.yy209, &yymsp[-8].minor.yy1109, yymsp[-5].minor.yy416, yymsp[-7].minor.yy416, yymsp[-3].minor.yy316, yymsp[-2].minor.yy416, yymsp[0].minor.yy416, yymsp[-4].minor.yy316); } -#line 7488 "sql.c" +{ pCxt->pRootNode = createCreateStreamStmt(pCxt, yymsp[-9].minor.yy325, &yymsp[-8].minor.yy837, yymsp[-5].minor.yy54, yymsp[-7].minor.yy54, yymsp[-3].minor.yy652, yymsp[-2].minor.yy54, yymsp[0].minor.yy54, yymsp[-4].minor.yy652); } break; case 405: /* cmd ::= DROP STREAM exists_opt stream_name */ -#line 736 "sql.y" -{ pCxt->pRootNode = createDropStreamStmt(pCxt, yymsp[-1].minor.yy209, &yymsp[0].minor.yy1109); } -#line 7493 "sql.c" +{ pCxt->pRootNode = createDropStreamStmt(pCxt, yymsp[-1].minor.yy325, &yymsp[0].minor.yy837); } break; case 406: /* cmd ::= PAUSE STREAM exists_opt stream_name */ -#line 737 "sql.y" -{ pCxt->pRootNode = createPauseStreamStmt(pCxt, yymsp[-1].minor.yy209, &yymsp[0].minor.yy1109); } -#line 7498 "sql.c" +{ pCxt->pRootNode = createPauseStreamStmt(pCxt, yymsp[-1].minor.yy325, &yymsp[0].minor.yy837); } break; case 407: /* cmd ::= RESUME STREAM exists_opt ignore_opt stream_name */ -#line 738 "sql.y" -{ pCxt->pRootNode = createResumeStreamStmt(pCxt, yymsp[-2].minor.yy209, yymsp[-1].minor.yy209, &yymsp[0].minor.yy1109); } -#line 7503 "sql.c" +{ pCxt->pRootNode = createResumeStreamStmt(pCxt, yymsp[-2].minor.yy325, yymsp[-1].minor.yy325, &yymsp[0].minor.yy837); } break; case 412: /* column_stream_def ::= column_name stream_col_options */ -#line 751 "sql.y" -{ yylhsminor.yy416 = createColumnDefNode(pCxt, &yymsp[-1].minor.yy1109, createDataType(TSDB_DATA_TYPE_NULL), yymsp[0].minor.yy416); } -#line 7508 "sql.c" - yymsp[-1].minor.yy416 = yylhsminor.yy416; +{ yylhsminor.yy54 = createColumnDefNode(pCxt, &yymsp[-1].minor.yy837, createDataType(TSDB_DATA_TYPE_NULL), yymsp[0].minor.yy54); } + yymsp[-1].minor.yy54 = yylhsminor.yy54; break; case 413: /* stream_col_options ::= */ - case 750: /* column_options ::= */ yytestcase(yyruleno==750); -#line 752 "sql.y" -{ yymsp[1].minor.yy416 = createDefaultColumnOptions(pCxt); } -#line 7515 "sql.c" + case 751: /* column_options ::= */ yytestcase(yyruleno==751); +{ yymsp[1].minor.yy54 = createDefaultColumnOptions(pCxt); } break; case 414: /* stream_col_options ::= stream_col_options PRIMARY KEY */ - case 751: /* column_options ::= column_options PRIMARY KEY */ yytestcase(yyruleno==751); -#line 753 "sql.y" -{ yylhsminor.yy416 = setColumnOptions(pCxt, yymsp[-2].minor.yy416, COLUMN_OPTION_PRIMARYKEY, NULL); } -#line 7521 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + case 752: /* column_options ::= column_options PRIMARY KEY */ yytestcase(yyruleno==752); +{ yylhsminor.yy54 = setColumnOptions(pCxt, yymsp[-2].minor.yy54, COLUMN_OPTION_PRIMARYKEY, NULL); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; case 419: /* stream_options ::= stream_options TRIGGER AT_ONCE */ case 420: /* stream_options ::= stream_options TRIGGER WINDOW_CLOSE */ yytestcase(yyruleno==420); -#line 763 "sql.y" -{ yylhsminor.yy416 = setStreamOptions(pCxt, yymsp[-2].minor.yy416, SOPT_TRIGGER_TYPE_SET, &yymsp[0].minor.yy0, NULL); } -#line 7528 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + case 421: /* stream_options ::= stream_options TRIGGER FORCE_WINDOW_CLOSE */ yytestcase(yyruleno==421); +{ yylhsminor.yy54 = setStreamOptions(pCxt, yymsp[-2].minor.yy54, SOPT_TRIGGER_TYPE_SET, &yymsp[0].minor.yy0, NULL); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 421: /* stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal */ -#line 765 "sql.y" -{ yylhsminor.yy416 = setStreamOptions(pCxt, yymsp[-3].minor.yy416, SOPT_TRIGGER_TYPE_SET, &yymsp[-1].minor.yy0, releaseRawExprNode(pCxt, yymsp[0].minor.yy416)); } -#line 7534 "sql.c" - yymsp[-3].minor.yy416 = yylhsminor.yy416; + case 422: /* stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal */ +{ yylhsminor.yy54 = setStreamOptions(pCxt, yymsp[-3].minor.yy54, SOPT_TRIGGER_TYPE_SET, &yymsp[-1].minor.yy0, releaseRawExprNode(pCxt, yymsp[0].minor.yy54)); } + yymsp[-3].minor.yy54 = yylhsminor.yy54; break; - case 422: /* stream_options ::= stream_options WATERMARK duration_literal */ -#line 766 "sql.y" -{ yylhsminor.yy416 = setStreamOptions(pCxt, yymsp[-2].minor.yy416, SOPT_WATERMARK_SET, NULL, releaseRawExprNode(pCxt, yymsp[0].minor.yy416)); } -#line 7540 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + case 423: /* stream_options ::= stream_options WATERMARK duration_literal */ +{ yylhsminor.yy54 = setStreamOptions(pCxt, yymsp[-2].minor.yy54, SOPT_WATERMARK_SET, NULL, releaseRawExprNode(pCxt, yymsp[0].minor.yy54)); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 423: /* stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER */ -#line 767 "sql.y" -{ yylhsminor.yy416 = setStreamOptions(pCxt, yymsp[-3].minor.yy416, SOPT_IGNORE_EXPIRED_SET, &yymsp[0].minor.yy0, NULL); } -#line 7546 "sql.c" - yymsp[-3].minor.yy416 = yylhsminor.yy416; + case 424: /* stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER */ +{ yylhsminor.yy54 = setStreamOptions(pCxt, yymsp[-3].minor.yy54, SOPT_IGNORE_EXPIRED_SET, &yymsp[0].minor.yy0, NULL); } + yymsp[-3].minor.yy54 = yylhsminor.yy54; break; - case 424: /* stream_options ::= stream_options FILL_HISTORY NK_INTEGER */ -#line 768 "sql.y" -{ yylhsminor.yy416 = setStreamOptions(pCxt, yymsp[-2].minor.yy416, SOPT_FILL_HISTORY_SET, &yymsp[0].minor.yy0, NULL); } -#line 7552 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + case 425: /* stream_options ::= stream_options FILL_HISTORY NK_INTEGER */ +{ yylhsminor.yy54 = setStreamOptions(pCxt, yymsp[-2].minor.yy54, SOPT_FILL_HISTORY_SET, &yymsp[0].minor.yy0, NULL); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 425: /* stream_options ::= stream_options DELETE_MARK duration_literal */ -#line 769 "sql.y" -{ yylhsminor.yy416 = setStreamOptions(pCxt, yymsp[-2].minor.yy416, SOPT_DELETE_MARK_SET, NULL, releaseRawExprNode(pCxt, yymsp[0].minor.yy416)); } -#line 7558 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + case 426: /* stream_options ::= stream_options DELETE_MARK duration_literal */ +{ yylhsminor.yy54 = setStreamOptions(pCxt, yymsp[-2].minor.yy54, SOPT_DELETE_MARK_SET, NULL, releaseRawExprNode(pCxt, yymsp[0].minor.yy54)); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 426: /* stream_options ::= stream_options IGNORE UPDATE NK_INTEGER */ -#line 770 "sql.y" -{ yylhsminor.yy416 = setStreamOptions(pCxt, yymsp[-3].minor.yy416, SOPT_IGNORE_UPDATE_SET, &yymsp[0].minor.yy0, NULL); } -#line 7564 "sql.c" - yymsp[-3].minor.yy416 = yylhsminor.yy416; + case 427: /* stream_options ::= stream_options IGNORE UPDATE NK_INTEGER */ +{ yylhsminor.yy54 = setStreamOptions(pCxt, yymsp[-3].minor.yy54, SOPT_IGNORE_UPDATE_SET, &yymsp[0].minor.yy0, NULL); } + yymsp[-3].minor.yy54 = yylhsminor.yy54; break; - case 428: /* subtable_opt ::= SUBTABLE NK_LP expression NK_RP */ - case 694: /* sliding_opt ::= SLIDING NK_LP interval_sliding_duration_literal NK_RP */ yytestcase(yyruleno==694); - case 718: /* every_opt ::= EVERY NK_LP duration_literal NK_RP */ yytestcase(yyruleno==718); -#line 773 "sql.y" -{ yymsp[-3].minor.yy416 = releaseRawExprNode(pCxt, yymsp[-1].minor.yy416); } -#line 7572 "sql.c" + case 429: /* subtable_opt ::= SUBTABLE NK_LP expression NK_RP */ + case 695: /* sliding_opt ::= SLIDING NK_LP interval_sliding_duration_literal NK_RP */ yytestcase(yyruleno==695); + case 719: /* every_opt ::= EVERY NK_LP duration_literal NK_RP */ yytestcase(yyruleno==719); +{ yymsp[-3].minor.yy54 = releaseRawExprNode(pCxt, yymsp[-1].minor.yy54); } break; - case 431: /* cmd ::= KILL CONNECTION NK_INTEGER */ -#line 781 "sql.y" + case 432: /* cmd ::= KILL CONNECTION NK_INTEGER */ { pCxt->pRootNode = createKillStmt(pCxt, QUERY_NODE_KILL_CONNECTION_STMT, &yymsp[0].minor.yy0); } -#line 7577 "sql.c" break; - case 432: /* cmd ::= KILL QUERY NK_STRING */ -#line 782 "sql.y" + case 433: /* cmd ::= KILL QUERY NK_STRING */ { pCxt->pRootNode = createKillQueryStmt(pCxt, &yymsp[0].minor.yy0); } -#line 7582 "sql.c" break; - case 433: /* cmd ::= KILL TRANSACTION NK_INTEGER */ -#line 783 "sql.y" + case 434: /* cmd ::= KILL TRANSACTION NK_INTEGER */ { pCxt->pRootNode = createKillStmt(pCxt, QUERY_NODE_KILL_TRANSACTION_STMT, &yymsp[0].minor.yy0); } -#line 7587 "sql.c" break; - case 434: /* cmd ::= KILL COMPACT NK_INTEGER */ -#line 784 "sql.y" + case 435: /* cmd ::= KILL COMPACT NK_INTEGER */ { pCxt->pRootNode = createKillStmt(pCxt, QUERY_NODE_KILL_COMPACT_STMT, &yymsp[0].minor.yy0); } -#line 7592 "sql.c" break; - case 435: /* cmd ::= BALANCE VGROUP */ -#line 787 "sql.y" + case 436: /* cmd ::= BALANCE VGROUP */ { pCxt->pRootNode = createBalanceVgroupStmt(pCxt); } -#line 7597 "sql.c" break; - case 436: /* cmd ::= BALANCE VGROUP LEADER on_vgroup_id */ -#line 788 "sql.y" -{ pCxt->pRootNode = createBalanceVgroupLeaderStmt(pCxt, &yymsp[0].minor.yy1109); } -#line 7602 "sql.c" + case 437: /* cmd ::= BALANCE VGROUP LEADER on_vgroup_id */ +{ pCxt->pRootNode = createBalanceVgroupLeaderStmt(pCxt, &yymsp[0].minor.yy837); } break; - case 437: /* cmd ::= BALANCE VGROUP LEADER DATABASE db_name */ -#line 789 "sql.y" -{ pCxt->pRootNode = createBalanceVgroupLeaderDBNameStmt(pCxt, &yymsp[0].minor.yy1109); } -#line 7607 "sql.c" + case 438: /* cmd ::= BALANCE VGROUP LEADER DATABASE db_name */ +{ pCxt->pRootNode = createBalanceVgroupLeaderDBNameStmt(pCxt, &yymsp[0].minor.yy837); } break; - case 438: /* cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER */ -#line 790 "sql.y" + case 439: /* cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER */ { pCxt->pRootNode = createMergeVgroupStmt(pCxt, &yymsp[-1].minor.yy0, &yymsp[0].minor.yy0); } -#line 7612 "sql.c" break; - case 439: /* cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list */ -#line 791 "sql.y" -{ pCxt->pRootNode = createRedistributeVgroupStmt(pCxt, &yymsp[-1].minor.yy0, yymsp[0].minor.yy316); } -#line 7617 "sql.c" + case 440: /* cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list */ +{ pCxt->pRootNode = createRedistributeVgroupStmt(pCxt, &yymsp[-1].minor.yy0, yymsp[0].minor.yy652); } break; - case 440: /* cmd ::= SPLIT VGROUP NK_INTEGER */ -#line 792 "sql.y" + case 441: /* cmd ::= SPLIT VGROUP NK_INTEGER */ { pCxt->pRootNode = createSplitVgroupStmt(pCxt, &yymsp[0].minor.yy0); } -#line 7622 "sql.c" break; - case 443: /* dnode_list ::= DNODE NK_INTEGER */ -#line 801 "sql.y" -{ yymsp[-1].minor.yy316 = createNodeList(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0)); } -#line 7627 "sql.c" + case 444: /* dnode_list ::= DNODE NK_INTEGER */ +{ yymsp[-1].minor.yy652 = createNodeList(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0)); } break; - case 445: /* cmd ::= DELETE FROM full_table_name where_clause_opt */ -#line 808 "sql.y" -{ pCxt->pRootNode = createDeleteStmt(pCxt, yymsp[-1].minor.yy416, yymsp[0].minor.yy416); } -#line 7632 "sql.c" + case 446: /* cmd ::= DELETE FROM full_table_name where_clause_opt */ +{ pCxt->pRootNode = createDeleteStmt(pCxt, yymsp[-1].minor.yy54, yymsp[0].minor.yy54); } break; - case 448: /* insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery */ -#line 817 "sql.y" -{ yymsp[-6].minor.yy416 = createInsertStmt(pCxt, yymsp[-4].minor.yy416, yymsp[-2].minor.yy316, yymsp[0].minor.yy416); } -#line 7637 "sql.c" + case 449: /* insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery */ +{ yymsp[-6].minor.yy54 = createInsertStmt(pCxt, yymsp[-4].minor.yy54, yymsp[-2].minor.yy652, yymsp[0].minor.yy54); } break; - case 449: /* insert_query ::= INSERT INTO full_table_name query_or_subquery */ -#line 818 "sql.y" -{ yymsp[-3].minor.yy416 = createInsertStmt(pCxt, yymsp[-1].minor.yy416, NULL, yymsp[0].minor.yy416); } -#line 7642 "sql.c" + case 450: /* insert_query ::= INSERT INTO full_table_name query_or_subquery */ +{ yymsp[-3].minor.yy54 = createInsertStmt(pCxt, yymsp[-1].minor.yy54, NULL, yymsp[0].minor.yy54); } break; - case 450: /* tags_literal ::= NK_INTEGER */ - case 462: /* tags_literal ::= NK_BIN */ yytestcase(yyruleno==462); - case 471: /* tags_literal ::= NK_HEX */ yytestcase(yyruleno==471); -#line 821 "sql.y" -{ yylhsminor.yy416 = createRawValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &yymsp[0].minor.yy0, NULL); } -#line 7649 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; + case 451: /* tags_literal ::= NK_INTEGER */ + case 463: /* tags_literal ::= NK_BIN */ yytestcase(yyruleno==463); + case 472: /* tags_literal ::= NK_HEX */ yytestcase(yyruleno==472); +{ yylhsminor.yy54 = createRawValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &yymsp[0].minor.yy0, NULL); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; - case 451: /* tags_literal ::= NK_INTEGER NK_PLUS duration_literal */ - case 452: /* tags_literal ::= NK_INTEGER NK_MINUS duration_literal */ yytestcase(yyruleno==452); - case 463: /* tags_literal ::= NK_BIN NK_PLUS duration_literal */ yytestcase(yyruleno==463); - case 464: /* tags_literal ::= NK_BIN NK_MINUS duration_literal */ yytestcase(yyruleno==464); - case 472: /* tags_literal ::= NK_HEX NK_PLUS duration_literal */ yytestcase(yyruleno==472); - case 473: /* tags_literal ::= NK_HEX NK_MINUS duration_literal */ yytestcase(yyruleno==473); - case 481: /* tags_literal ::= NK_STRING NK_PLUS duration_literal */ yytestcase(yyruleno==481); - case 482: /* tags_literal ::= NK_STRING NK_MINUS duration_literal */ yytestcase(yyruleno==482); -#line 822 "sql.y" + case 452: /* tags_literal ::= NK_INTEGER NK_PLUS duration_literal */ + case 453: /* tags_literal ::= NK_INTEGER NK_MINUS duration_literal */ yytestcase(yyruleno==453); + case 464: /* tags_literal ::= NK_BIN NK_PLUS duration_literal */ yytestcase(yyruleno==464); + case 465: /* tags_literal ::= NK_BIN NK_MINUS duration_literal */ yytestcase(yyruleno==465); + case 473: /* tags_literal ::= NK_HEX NK_PLUS duration_literal */ yytestcase(yyruleno==473); + case 474: /* tags_literal ::= NK_HEX NK_MINUS duration_literal */ yytestcase(yyruleno==474); + case 482: /* tags_literal ::= NK_STRING NK_PLUS duration_literal */ yytestcase(yyruleno==482); + case 483: /* tags_literal ::= NK_STRING NK_MINUS duration_literal */ yytestcase(yyruleno==483); { SToken l = yymsp[-2].minor.yy0; - SToken r = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy416); + SToken r = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy54); l.n = (r.z + r.n) - l.z; - yylhsminor.yy416 = createRawValueNodeExt(pCxt, TSDB_DATA_TYPE_BINARY, &l, NULL, yymsp[0].minor.yy416); + yylhsminor.yy54 = createRawValueNodeExt(pCxt, TSDB_DATA_TYPE_BINARY, &l, NULL, yymsp[0].minor.yy54); } -#line 7667 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 453: /* tags_literal ::= NK_PLUS NK_INTEGER */ - case 456: /* tags_literal ::= NK_MINUS NK_INTEGER */ yytestcase(yyruleno==456); - case 465: /* tags_literal ::= NK_PLUS NK_BIN */ yytestcase(yyruleno==465); - case 468: /* tags_literal ::= NK_MINUS NK_BIN */ yytestcase(yyruleno==468); - case 474: /* tags_literal ::= NK_PLUS NK_HEX */ yytestcase(yyruleno==474); - case 477: /* tags_literal ::= NK_MINUS NK_HEX */ yytestcase(yyruleno==477); -#line 834 "sql.y" + case 454: /* tags_literal ::= NK_PLUS NK_INTEGER */ + case 457: /* tags_literal ::= NK_MINUS NK_INTEGER */ yytestcase(yyruleno==457); + case 466: /* tags_literal ::= NK_PLUS NK_BIN */ yytestcase(yyruleno==466); + case 469: /* tags_literal ::= NK_MINUS NK_BIN */ yytestcase(yyruleno==469); + case 475: /* tags_literal ::= NK_PLUS NK_HEX */ yytestcase(yyruleno==475); + case 478: /* tags_literal ::= NK_MINUS NK_HEX */ yytestcase(yyruleno==478); { SToken t = yymsp[-1].minor.yy0; t.n = (yymsp[0].minor.yy0.z + yymsp[0].minor.yy0.n) - yymsp[-1].minor.yy0.z; - yylhsminor.yy416 = createRawValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &t, NULL); + yylhsminor.yy54 = createRawValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &t, NULL); } -#line 7682 "sql.c" - yymsp[-1].minor.yy416 = yylhsminor.yy416; + yymsp[-1].minor.yy54 = yylhsminor.yy54; break; - case 454: /* tags_literal ::= NK_PLUS NK_INTEGER NK_PLUS duration_literal */ - case 455: /* tags_literal ::= NK_PLUS NK_INTEGER NK_MINUS duration_literal */ yytestcase(yyruleno==455); - case 457: /* tags_literal ::= NK_MINUS NK_INTEGER NK_PLUS duration_literal */ yytestcase(yyruleno==457); - case 458: /* tags_literal ::= NK_MINUS NK_INTEGER NK_MINUS duration_literal */ yytestcase(yyruleno==458); - case 466: /* tags_literal ::= NK_PLUS NK_BIN NK_PLUS duration_literal */ yytestcase(yyruleno==466); - case 467: /* tags_literal ::= NK_PLUS NK_BIN NK_MINUS duration_literal */ yytestcase(yyruleno==467); - case 469: /* tags_literal ::= NK_MINUS NK_BIN NK_PLUS duration_literal */ yytestcase(yyruleno==469); - case 470: /* tags_literal ::= NK_MINUS NK_BIN NK_MINUS duration_literal */ yytestcase(yyruleno==470); - case 475: /* tags_literal ::= NK_PLUS NK_HEX NK_PLUS duration_literal */ yytestcase(yyruleno==475); - case 476: /* tags_literal ::= NK_PLUS NK_HEX NK_MINUS duration_literal */ yytestcase(yyruleno==476); - case 478: /* tags_literal ::= NK_MINUS NK_HEX NK_PLUS duration_literal */ yytestcase(yyruleno==478); - case 479: /* tags_literal ::= NK_MINUS NK_HEX NK_MINUS duration_literal */ yytestcase(yyruleno==479); -#line 839 "sql.y" + case 455: /* tags_literal ::= NK_PLUS NK_INTEGER NK_PLUS duration_literal */ + case 456: /* tags_literal ::= NK_PLUS NK_INTEGER NK_MINUS duration_literal */ yytestcase(yyruleno==456); + case 458: /* tags_literal ::= NK_MINUS NK_INTEGER NK_PLUS duration_literal */ yytestcase(yyruleno==458); + case 459: /* tags_literal ::= NK_MINUS NK_INTEGER NK_MINUS duration_literal */ yytestcase(yyruleno==459); + case 467: /* tags_literal ::= NK_PLUS NK_BIN NK_PLUS duration_literal */ yytestcase(yyruleno==467); + case 468: /* tags_literal ::= NK_PLUS NK_BIN NK_MINUS duration_literal */ yytestcase(yyruleno==468); + case 470: /* tags_literal ::= NK_MINUS NK_BIN NK_PLUS duration_literal */ yytestcase(yyruleno==470); + case 471: /* tags_literal ::= NK_MINUS NK_BIN NK_MINUS duration_literal */ yytestcase(yyruleno==471); + case 476: /* tags_literal ::= NK_PLUS NK_HEX NK_PLUS duration_literal */ yytestcase(yyruleno==476); + case 477: /* tags_literal ::= NK_PLUS NK_HEX NK_MINUS duration_literal */ yytestcase(yyruleno==477); + case 479: /* tags_literal ::= NK_MINUS NK_HEX NK_PLUS duration_literal */ yytestcase(yyruleno==479); + case 480: /* tags_literal ::= NK_MINUS NK_HEX NK_MINUS duration_literal */ yytestcase(yyruleno==480); { SToken l = yymsp[-3].minor.yy0; - SToken r = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy416); + SToken r = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy54); l.n = (r.z + r.n) - l.z; - yylhsminor.yy416 = createRawValueNodeExt(pCxt, TSDB_DATA_TYPE_BINARY, &l, NULL, yymsp[0].minor.yy416); + yylhsminor.yy54 = createRawValueNodeExt(pCxt, TSDB_DATA_TYPE_BINARY, &l, NULL, yymsp[0].minor.yy54); } -#line 7704 "sql.c" - yymsp[-3].minor.yy416 = yylhsminor.yy416; + yymsp[-3].minor.yy54 = yylhsminor.yy54; break; - case 459: /* tags_literal ::= NK_FLOAT */ -#line 868 "sql.y" -{ yylhsminor.yy416 = createRawValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &yymsp[0].minor.yy0, NULL); } -#line 7710 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; + case 460: /* tags_literal ::= NK_FLOAT */ +{ yylhsminor.yy54 = createRawValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &yymsp[0].minor.yy0, NULL); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; - case 460: /* tags_literal ::= NK_PLUS NK_FLOAT */ - case 461: /* tags_literal ::= NK_MINUS NK_FLOAT */ yytestcase(yyruleno==461); -#line 869 "sql.y" + case 461: /* tags_literal ::= NK_PLUS NK_FLOAT */ + case 462: /* tags_literal ::= NK_MINUS NK_FLOAT */ yytestcase(yyruleno==462); { SToken t = yymsp[-1].minor.yy0; t.n = (yymsp[0].minor.yy0.z + yymsp[0].minor.yy0.n) - yymsp[-1].minor.yy0.z; - yylhsminor.yy416 = createRawValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &t, NULL); + yylhsminor.yy54 = createRawValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &t, NULL); } -#line 7721 "sql.c" - yymsp[-1].minor.yy416 = yylhsminor.yy416; + yymsp[-1].minor.yy54 = yylhsminor.yy54; break; - case 480: /* tags_literal ::= NK_STRING */ -#line 975 "sql.y" -{ yylhsminor.yy416 = createRawValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0, NULL); } -#line 7727 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; + case 481: /* tags_literal ::= NK_STRING */ +{ yylhsminor.yy54 = createRawValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0, NULL); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; - case 483: /* tags_literal ::= NK_BOOL */ -#line 988 "sql.y" -{ yylhsminor.yy416 = createRawValueNode(pCxt, TSDB_DATA_TYPE_BOOL, &yymsp[0].minor.yy0, NULL); } -#line 7733 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; + case 484: /* tags_literal ::= NK_BOOL */ +{ yylhsminor.yy54 = createRawValueNode(pCxt, TSDB_DATA_TYPE_BOOL, &yymsp[0].minor.yy0, NULL); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; - case 484: /* tags_literal ::= NULL */ -#line 989 "sql.y" -{ yylhsminor.yy416 = createRawValueNode(pCxt, TSDB_DATA_TYPE_NULL, &yymsp[0].minor.yy0, NULL); } -#line 7739 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; + case 485: /* tags_literal ::= NULL */ +{ yylhsminor.yy54 = createRawValueNode(pCxt, TSDB_DATA_TYPE_NULL, &yymsp[0].minor.yy0, NULL); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; - case 485: /* tags_literal ::= literal_func */ -#line 991 "sql.y" -{ yylhsminor.yy416 = createRawValueNode(pCxt, TSDB_DATA_TYPE_BINARY, NULL, yymsp[0].minor.yy416); } -#line 7745 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; + case 486: /* tags_literal ::= literal_func */ +{ yylhsminor.yy54 = createRawValueNode(pCxt, TSDB_DATA_TYPE_BINARY, NULL, yymsp[0].minor.yy54); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; - case 486: /* tags_literal ::= literal_func NK_PLUS duration_literal */ - case 487: /* tags_literal ::= literal_func NK_MINUS duration_literal */ yytestcase(yyruleno==487); -#line 992 "sql.y" + case 487: /* tags_literal ::= literal_func NK_PLUS duration_literal */ + case 488: /* tags_literal ::= literal_func NK_MINUS duration_literal */ yytestcase(yyruleno==488); { - SToken l = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy416); - SToken r = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy416); + SToken l = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy54); + SToken r = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy54); l.n = (r.z + r.n) - l.z; - yylhsminor.yy416 = createRawValueNodeExt(pCxt, TSDB_DATA_TYPE_BINARY, &l, yymsp[-2].minor.yy416, yymsp[0].minor.yy416); + yylhsminor.yy54 = createRawValueNodeExt(pCxt, TSDB_DATA_TYPE_BINARY, &l, yymsp[-2].minor.yy54, yymsp[0].minor.yy54); } -#line 7757 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 490: /* literal ::= NK_INTEGER */ -#line 1011 "sql.y" -{ yylhsminor.yy416 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &yymsp[0].minor.yy0)); } -#line 7763 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; + case 491: /* literal ::= NK_INTEGER */ +{ yylhsminor.yy54 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; - case 491: /* literal ::= NK_FLOAT */ -#line 1012 "sql.y" -{ yylhsminor.yy416 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &yymsp[0].minor.yy0)); } -#line 7769 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; + case 492: /* literal ::= NK_FLOAT */ +{ yylhsminor.yy54 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; - case 492: /* literal ::= NK_STRING */ -#line 1013 "sql.y" -{ yylhsminor.yy416 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0)); } -#line 7775 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; + case 493: /* literal ::= NK_STRING */ +{ yylhsminor.yy54 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; - case 493: /* literal ::= NK_BOOL */ -#line 1014 "sql.y" -{ yylhsminor.yy416 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_BOOL, &yymsp[0].minor.yy0)); } -#line 7781 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; + case 494: /* literal ::= NK_BOOL */ +{ yylhsminor.yy54 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_BOOL, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; - case 494: /* literal ::= TIMESTAMP NK_STRING */ -#line 1015 "sql.y" -{ yylhsminor.yy416 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0)); } -#line 7787 "sql.c" - yymsp[-1].minor.yy416 = yylhsminor.yy416; + case 495: /* literal ::= TIMESTAMP NK_STRING */ +{ yylhsminor.yy54 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0)); } + yymsp[-1].minor.yy54 = yylhsminor.yy54; break; - case 495: /* literal ::= duration_literal */ - case 505: /* signed_literal ::= signed */ yytestcase(yyruleno==505); - case 529: /* expr_or_subquery ::= expression */ yytestcase(yyruleno==529); - case 530: /* expression ::= literal */ yytestcase(yyruleno==530); - case 532: /* expression ::= column_reference */ yytestcase(yyruleno==532); - case 533: /* expression ::= function_expression */ yytestcase(yyruleno==533); - case 534: /* expression ::= case_when_expression */ yytestcase(yyruleno==534); - case 568: /* function_expression ::= literal_func */ yytestcase(yyruleno==568); - case 618: /* boolean_value_expression ::= boolean_primary */ yytestcase(yyruleno==618); - case 622: /* boolean_primary ::= predicate */ yytestcase(yyruleno==622); - case 624: /* common_expression ::= expr_or_subquery */ yytestcase(yyruleno==624); - case 625: /* common_expression ::= boolean_value_expression */ yytestcase(yyruleno==625); - case 628: /* table_reference_list ::= table_reference */ yytestcase(yyruleno==628); - case 630: /* table_reference ::= table_primary */ yytestcase(yyruleno==630); - case 631: /* table_reference ::= joined_table */ yytestcase(yyruleno==631); - case 635: /* table_primary ::= parenthesized_joined_table */ yytestcase(yyruleno==635); - case 720: /* query_simple ::= query_specification */ yytestcase(yyruleno==720); - case 721: /* query_simple ::= union_query_expression */ yytestcase(yyruleno==721); - case 724: /* query_simple_or_subquery ::= query_simple */ yytestcase(yyruleno==724); - case 726: /* query_or_subquery ::= query_expression */ yytestcase(yyruleno==726); -#line 1016 "sql.y" -{ yylhsminor.yy416 = yymsp[0].minor.yy416; } -#line 7812 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; + case 496: /* literal ::= duration_literal */ + case 506: /* signed_literal ::= signed */ yytestcase(yyruleno==506); + case 530: /* expr_or_subquery ::= expression */ yytestcase(yyruleno==530); + case 531: /* expression ::= literal */ yytestcase(yyruleno==531); + case 533: /* expression ::= column_reference */ yytestcase(yyruleno==533); + case 534: /* expression ::= function_expression */ yytestcase(yyruleno==534); + case 535: /* expression ::= case_when_expression */ yytestcase(yyruleno==535); + case 569: /* function_expression ::= literal_func */ yytestcase(yyruleno==569); + case 619: /* boolean_value_expression ::= boolean_primary */ yytestcase(yyruleno==619); + case 623: /* boolean_primary ::= predicate */ yytestcase(yyruleno==623); + case 625: /* common_expression ::= expr_or_subquery */ yytestcase(yyruleno==625); + case 626: /* common_expression ::= boolean_value_expression */ yytestcase(yyruleno==626); + case 629: /* table_reference_list ::= table_reference */ yytestcase(yyruleno==629); + case 631: /* table_reference ::= table_primary */ yytestcase(yyruleno==631); + case 632: /* table_reference ::= joined_table */ yytestcase(yyruleno==632); + case 636: /* table_primary ::= parenthesized_joined_table */ yytestcase(yyruleno==636); + case 721: /* query_simple ::= query_specification */ yytestcase(yyruleno==721); + case 722: /* query_simple ::= union_query_expression */ yytestcase(yyruleno==722); + case 725: /* query_simple_or_subquery ::= query_simple */ yytestcase(yyruleno==725); + case 727: /* query_or_subquery ::= query_expression */ yytestcase(yyruleno==727); +{ yylhsminor.yy54 = yymsp[0].minor.yy54; } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; - case 496: /* literal ::= NULL */ -#line 1017 "sql.y" -{ yylhsminor.yy416 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_NULL, &yymsp[0].minor.yy0)); } -#line 7818 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; + case 497: /* literal ::= NULL */ +{ yylhsminor.yy54 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_NULL, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; - case 497: /* literal ::= NK_QUESTION */ -#line 1018 "sql.y" -{ yylhsminor.yy416 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createPlaceholderValueNode(pCxt, &yymsp[0].minor.yy0)); } -#line 7824 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; + case 498: /* literal ::= NK_QUESTION */ +{ yylhsminor.yy54 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createPlaceholderValueNode(pCxt, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; - case 498: /* duration_literal ::= NK_VARIABLE */ - case 695: /* interval_sliding_duration_literal ::= NK_VARIABLE */ yytestcase(yyruleno==695); - case 696: /* interval_sliding_duration_literal ::= NK_STRING */ yytestcase(yyruleno==696); - case 697: /* interval_sliding_duration_literal ::= NK_INTEGER */ yytestcase(yyruleno==697); -#line 1020 "sql.y" -{ yylhsminor.yy416 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } -#line 7833 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; + case 499: /* duration_literal ::= NK_VARIABLE */ + case 696: /* interval_sliding_duration_literal ::= NK_VARIABLE */ yytestcase(yyruleno==696); + case 697: /* interval_sliding_duration_literal ::= NK_STRING */ yytestcase(yyruleno==697); + case 698: /* interval_sliding_duration_literal ::= NK_INTEGER */ yytestcase(yyruleno==698); +{ yylhsminor.yy54 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; - case 499: /* signed ::= NK_INTEGER */ -#line 1022 "sql.y" -{ yylhsminor.yy416 = createValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &yymsp[0].minor.yy0); } -#line 7839 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; + case 500: /* signed ::= NK_INTEGER */ +{ yylhsminor.yy54 = createValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &yymsp[0].minor.yy0); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; - case 500: /* signed ::= NK_PLUS NK_INTEGER */ -#line 1023 "sql.y" -{ yymsp[-1].minor.yy416 = createValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &yymsp[0].minor.yy0); } -#line 7845 "sql.c" + case 501: /* signed ::= NK_PLUS NK_INTEGER */ +{ yymsp[-1].minor.yy54 = createValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &yymsp[0].minor.yy0); } break; - case 501: /* signed ::= NK_MINUS NK_INTEGER */ -#line 1024 "sql.y" + case 502: /* signed ::= NK_MINUS NK_INTEGER */ { SToken t = yymsp[-1].minor.yy0; t.n = (yymsp[0].minor.yy0.z + yymsp[0].minor.yy0.n) - yymsp[-1].minor.yy0.z; - yylhsminor.yy416 = createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &t); + yylhsminor.yy54 = createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &t); } -#line 7854 "sql.c" - yymsp[-1].minor.yy416 = yylhsminor.yy416; + yymsp[-1].minor.yy54 = yylhsminor.yy54; break; - case 502: /* signed ::= NK_FLOAT */ -#line 1029 "sql.y" -{ yylhsminor.yy416 = createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &yymsp[0].minor.yy0); } -#line 7860 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; + case 503: /* signed ::= NK_FLOAT */ +{ yylhsminor.yy54 = createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &yymsp[0].minor.yy0); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; - case 503: /* signed ::= NK_PLUS NK_FLOAT */ -#line 1030 "sql.y" -{ yymsp[-1].minor.yy416 = createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &yymsp[0].minor.yy0); } -#line 7866 "sql.c" + case 504: /* signed ::= NK_PLUS NK_FLOAT */ +{ yymsp[-1].minor.yy54 = createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &yymsp[0].minor.yy0); } break; - case 504: /* signed ::= NK_MINUS NK_FLOAT */ -#line 1031 "sql.y" + case 505: /* signed ::= NK_MINUS NK_FLOAT */ { SToken t = yymsp[-1].minor.yy0; t.n = (yymsp[0].minor.yy0.z + yymsp[0].minor.yy0.n) - yymsp[-1].minor.yy0.z; - yylhsminor.yy416 = createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &t); + yylhsminor.yy54 = createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &t); } -#line 7875 "sql.c" - yymsp[-1].minor.yy416 = yylhsminor.yy416; + yymsp[-1].minor.yy54 = yylhsminor.yy54; break; - case 506: /* signed_literal ::= NK_STRING */ -#line 1038 "sql.y" -{ yylhsminor.yy416 = createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0); } -#line 7881 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; + case 507: /* signed_literal ::= NK_STRING */ +{ yylhsminor.yy54 = createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; - case 507: /* signed_literal ::= NK_BOOL */ -#line 1039 "sql.y" -{ yylhsminor.yy416 = createValueNode(pCxt, TSDB_DATA_TYPE_BOOL, &yymsp[0].minor.yy0); } -#line 7887 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; + case 508: /* signed_literal ::= NK_BOOL */ +{ yylhsminor.yy54 = createValueNode(pCxt, TSDB_DATA_TYPE_BOOL, &yymsp[0].minor.yy0); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; - case 508: /* signed_literal ::= TIMESTAMP NK_STRING */ -#line 1040 "sql.y" -{ yymsp[-1].minor.yy416 = createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0); } -#line 7893 "sql.c" + case 509: /* signed_literal ::= TIMESTAMP NK_STRING */ +{ yymsp[-1].minor.yy54 = createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0); } break; - case 509: /* signed_literal ::= duration_literal */ - case 511: /* signed_literal ::= literal_func */ yytestcase(yyruleno==511); - case 589: /* star_func_para ::= expr_or_subquery */ yytestcase(yyruleno==589); - case 672: /* select_item ::= common_expression */ yytestcase(yyruleno==672); - case 682: /* partition_item ::= expr_or_subquery */ yytestcase(yyruleno==682); - case 725: /* query_simple_or_subquery ::= subquery */ yytestcase(yyruleno==725); - case 727: /* query_or_subquery ::= subquery */ yytestcase(yyruleno==727); - case 740: /* search_condition ::= common_expression */ yytestcase(yyruleno==740); -#line 1041 "sql.y" -{ yylhsminor.yy416 = releaseRawExprNode(pCxt, yymsp[0].minor.yy416); } -#line 7905 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; + case 510: /* signed_literal ::= duration_literal */ + case 512: /* signed_literal ::= literal_func */ yytestcase(yyruleno==512); + case 590: /* star_func_para ::= expr_or_subquery */ yytestcase(yyruleno==590); + case 673: /* select_item ::= common_expression */ yytestcase(yyruleno==673); + case 683: /* partition_item ::= expr_or_subquery */ yytestcase(yyruleno==683); + case 726: /* query_simple_or_subquery ::= subquery */ yytestcase(yyruleno==726); + case 728: /* query_or_subquery ::= subquery */ yytestcase(yyruleno==728); + case 741: /* search_condition ::= common_expression */ yytestcase(yyruleno==741); +{ yylhsminor.yy54 = releaseRawExprNode(pCxt, yymsp[0].minor.yy54); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; - case 510: /* signed_literal ::= NULL */ -#line 1042 "sql.y" -{ yylhsminor.yy416 = createValueNode(pCxt, TSDB_DATA_TYPE_NULL, &yymsp[0].minor.yy0); } -#line 7911 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; + case 511: /* signed_literal ::= NULL */ +{ yylhsminor.yy54 = createValueNode(pCxt, TSDB_DATA_TYPE_NULL, &yymsp[0].minor.yy0); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; - case 512: /* signed_literal ::= NK_QUESTION */ -#line 1044 "sql.y" -{ yylhsminor.yy416 = createPlaceholderValueNode(pCxt, &yymsp[0].minor.yy0); } -#line 7917 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; + case 513: /* signed_literal ::= NK_QUESTION */ +{ yylhsminor.yy54 = createPlaceholderValueNode(pCxt, &yymsp[0].minor.yy0); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; - case 531: /* expression ::= pseudo_column */ -#line 1110 "sql.y" -{ yylhsminor.yy416 = yymsp[0].minor.yy416; setRawExprNodeIsPseudoColumn(pCxt, yylhsminor.yy416, true); } -#line 7923 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; + case 532: /* expression ::= pseudo_column */ +{ yylhsminor.yy54 = yymsp[0].minor.yy54; (void)setRawExprNodeIsPseudoColumn(pCxt, yylhsminor.yy54, true); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; - case 535: /* expression ::= NK_LP expression NK_RP */ - case 623: /* boolean_primary ::= NK_LP boolean_value_expression NK_RP */ yytestcase(yyruleno==623); - case 739: /* subquery ::= NK_LP subquery NK_RP */ yytestcase(yyruleno==739); -#line 1114 "sql.y" -{ yylhsminor.yy416 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0, releaseRawExprNode(pCxt, yymsp[-1].minor.yy416)); } -#line 7931 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + case 536: /* expression ::= NK_LP expression NK_RP */ + case 624: /* boolean_primary ::= NK_LP boolean_value_expression NK_RP */ yytestcase(yyruleno==624); + case 740: /* subquery ::= NK_LP subquery NK_RP */ yytestcase(yyruleno==740); +{ yylhsminor.yy54 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0, releaseRawExprNode(pCxt, yymsp[-1].minor.yy54)); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 536: /* expression ::= NK_PLUS expr_or_subquery */ -#line 1115 "sql.y" + case 537: /* expression ::= NK_PLUS expr_or_subquery */ { - SToken t = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy416); - yylhsminor.yy416 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &t, releaseRawExprNode(pCxt, yymsp[0].minor.yy416)); + SToken t = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy54); + yylhsminor.yy54 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &t, releaseRawExprNode(pCxt, yymsp[0].minor.yy54)); } -#line 7940 "sql.c" - yymsp[-1].minor.yy416 = yylhsminor.yy416; + yymsp[-1].minor.yy54 = yylhsminor.yy54; break; - case 537: /* expression ::= NK_MINUS expr_or_subquery */ -#line 1119 "sql.y" + case 538: /* expression ::= NK_MINUS expr_or_subquery */ { - SToken t = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy416); - yylhsminor.yy416 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &t, createOperatorNode(pCxt, OP_TYPE_MINUS, releaseRawExprNode(pCxt, yymsp[0].minor.yy416), NULL)); + SToken t = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy54); + yylhsminor.yy54 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &t, createOperatorNode(pCxt, OP_TYPE_MINUS, releaseRawExprNode(pCxt, yymsp[0].minor.yy54), NULL)); } -#line 7949 "sql.c" - yymsp[-1].minor.yy416 = yylhsminor.yy416; + yymsp[-1].minor.yy54 = yylhsminor.yy54; break; - case 538: /* expression ::= expr_or_subquery NK_PLUS expr_or_subquery */ -#line 1123 "sql.y" + case 539: /* expression ::= expr_or_subquery NK_PLUS expr_or_subquery */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy416); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy416); - yylhsminor.yy416 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_ADD, releaseRawExprNode(pCxt, yymsp[-2].minor.yy416), releaseRawExprNode(pCxt, yymsp[0].minor.yy416))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy54); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy54); + yylhsminor.yy54 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_ADD, releaseRawExprNode(pCxt, yymsp[-2].minor.yy54), releaseRawExprNode(pCxt, yymsp[0].minor.yy54))); } -#line 7959 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 539: /* expression ::= expr_or_subquery NK_MINUS expr_or_subquery */ -#line 1128 "sql.y" + case 540: /* expression ::= expr_or_subquery NK_MINUS expr_or_subquery */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy416); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy416); - yylhsminor.yy416 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_SUB, releaseRawExprNode(pCxt, yymsp[-2].minor.yy416), releaseRawExprNode(pCxt, yymsp[0].minor.yy416))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy54); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy54); + yylhsminor.yy54 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_SUB, releaseRawExprNode(pCxt, yymsp[-2].minor.yy54), releaseRawExprNode(pCxt, yymsp[0].minor.yy54))); } -#line 7969 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 540: /* expression ::= expr_or_subquery NK_STAR expr_or_subquery */ -#line 1133 "sql.y" + case 541: /* expression ::= expr_or_subquery NK_STAR expr_or_subquery */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy416); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy416); - yylhsminor.yy416 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_MULTI, releaseRawExprNode(pCxt, yymsp[-2].minor.yy416), releaseRawExprNode(pCxt, yymsp[0].minor.yy416))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy54); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy54); + yylhsminor.yy54 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_MULTI, releaseRawExprNode(pCxt, yymsp[-2].minor.yy54), releaseRawExprNode(pCxt, yymsp[0].minor.yy54))); } -#line 7979 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 541: /* expression ::= expr_or_subquery NK_SLASH expr_or_subquery */ -#line 1138 "sql.y" + case 542: /* expression ::= expr_or_subquery NK_SLASH expr_or_subquery */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy416); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy416); - yylhsminor.yy416 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_DIV, releaseRawExprNode(pCxt, yymsp[-2].minor.yy416), releaseRawExprNode(pCxt, yymsp[0].minor.yy416))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy54); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy54); + yylhsminor.yy54 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_DIV, releaseRawExprNode(pCxt, yymsp[-2].minor.yy54), releaseRawExprNode(pCxt, yymsp[0].minor.yy54))); } -#line 7989 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 542: /* expression ::= expr_or_subquery NK_REM expr_or_subquery */ -#line 1143 "sql.y" + case 543: /* expression ::= expr_or_subquery NK_REM expr_or_subquery */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy416); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy416); - yylhsminor.yy416 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_REM, releaseRawExprNode(pCxt, yymsp[-2].minor.yy416), releaseRawExprNode(pCxt, yymsp[0].minor.yy416))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy54); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy54); + yylhsminor.yy54 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_REM, releaseRawExprNode(pCxt, yymsp[-2].minor.yy54), releaseRawExprNode(pCxt, yymsp[0].minor.yy54))); } -#line 7999 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 543: /* expression ::= column_reference NK_ARROW NK_STRING */ -#line 1148 "sql.y" + case 544: /* expression ::= column_reference NK_ARROW NK_STRING */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy416); - yylhsminor.yy416 = createRawExprNodeExt(pCxt, &s, &yymsp[0].minor.yy0, createOperatorNode(pCxt, OP_TYPE_JSON_GET_VALUE, releaseRawExprNode(pCxt, yymsp[-2].minor.yy416), createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy54); + yylhsminor.yy54 = createRawExprNodeExt(pCxt, &s, &yymsp[0].minor.yy0, createOperatorNode(pCxt, OP_TYPE_JSON_GET_VALUE, releaseRawExprNode(pCxt, yymsp[-2].minor.yy54), createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0))); } -#line 8008 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 544: /* expression ::= expr_or_subquery NK_BITAND expr_or_subquery */ -#line 1152 "sql.y" + case 545: /* expression ::= expr_or_subquery NK_BITAND expr_or_subquery */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy416); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy416); - yylhsminor.yy416 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_BIT_AND, releaseRawExprNode(pCxt, yymsp[-2].minor.yy416), releaseRawExprNode(pCxt, yymsp[0].minor.yy416))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy54); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy54); + yylhsminor.yy54 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_BIT_AND, releaseRawExprNode(pCxt, yymsp[-2].minor.yy54), releaseRawExprNode(pCxt, yymsp[0].minor.yy54))); } -#line 8018 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 545: /* expression ::= expr_or_subquery NK_BITOR expr_or_subquery */ -#line 1157 "sql.y" + case 546: /* expression ::= expr_or_subquery NK_BITOR expr_or_subquery */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy416); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy416); - yylhsminor.yy416 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_BIT_OR, releaseRawExprNode(pCxt, yymsp[-2].minor.yy416), releaseRawExprNode(pCxt, yymsp[0].minor.yy416))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy54); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy54); + yylhsminor.yy54 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_BIT_OR, releaseRawExprNode(pCxt, yymsp[-2].minor.yy54), releaseRawExprNode(pCxt, yymsp[0].minor.yy54))); } -#line 8028 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 548: /* column_reference ::= column_name */ -#line 1168 "sql.y" -{ yylhsminor.yy416 = createRawExprNode(pCxt, &yymsp[0].minor.yy1109, createColumnNode(pCxt, NULL, &yymsp[0].minor.yy1109)); } -#line 8034 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; + case 549: /* column_reference ::= column_name */ +{ yylhsminor.yy54 = createRawExprNode(pCxt, &yymsp[0].minor.yy837, createColumnNode(pCxt, NULL, &yymsp[0].minor.yy837)); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; - case 549: /* column_reference ::= table_name NK_DOT column_name */ -#line 1169 "sql.y" -{ yylhsminor.yy416 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy1109, &yymsp[0].minor.yy1109, createColumnNode(pCxt, &yymsp[-2].minor.yy1109, &yymsp[0].minor.yy1109)); } -#line 8040 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + case 550: /* column_reference ::= table_name NK_DOT column_name */ +{ yylhsminor.yy54 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy837, &yymsp[0].minor.yy837, createColumnNode(pCxt, &yymsp[-2].minor.yy837, &yymsp[0].minor.yy837)); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 550: /* column_reference ::= NK_ALIAS */ -#line 1170 "sql.y" -{ yylhsminor.yy416 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createColumnNode(pCxt, NULL, &yymsp[0].minor.yy0)); } -#line 8046 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; + case 551: /* column_reference ::= NK_ALIAS */ +{ yylhsminor.yy54 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createColumnNode(pCxt, NULL, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; - case 551: /* column_reference ::= table_name NK_DOT NK_ALIAS */ -#line 1171 "sql.y" -{ yylhsminor.yy416 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy1109, &yymsp[0].minor.yy0, createColumnNode(pCxt, &yymsp[-2].minor.yy1109, &yymsp[0].minor.yy0)); } -#line 8052 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + case 552: /* column_reference ::= table_name NK_DOT NK_ALIAS */ +{ yylhsminor.yy54 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy837, &yymsp[0].minor.yy0, createColumnNode(pCxt, &yymsp[-2].minor.yy837, &yymsp[0].minor.yy0)); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 552: /* pseudo_column ::= ROWTS */ - case 553: /* pseudo_column ::= TBNAME */ yytestcase(yyruleno==553); - case 555: /* pseudo_column ::= QSTART */ yytestcase(yyruleno==555); - case 556: /* pseudo_column ::= QEND */ yytestcase(yyruleno==556); - case 557: /* pseudo_column ::= QDURATION */ yytestcase(yyruleno==557); - case 558: /* pseudo_column ::= WSTART */ yytestcase(yyruleno==558); - case 559: /* pseudo_column ::= WEND */ yytestcase(yyruleno==559); - case 560: /* pseudo_column ::= WDURATION */ yytestcase(yyruleno==560); - case 561: /* pseudo_column ::= IROWTS */ yytestcase(yyruleno==561); - case 562: /* pseudo_column ::= ISFILLED */ yytestcase(yyruleno==562); - case 563: /* pseudo_column ::= QTAGS */ yytestcase(yyruleno==563); - case 570: /* literal_func ::= NOW */ yytestcase(yyruleno==570); - case 571: /* literal_func ::= TODAY */ yytestcase(yyruleno==571); -#line 1173 "sql.y" -{ yylhsminor.yy416 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[0].minor.yy0, NULL)); } -#line 8070 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; + case 553: /* pseudo_column ::= ROWTS */ + case 554: /* pseudo_column ::= TBNAME */ yytestcase(yyruleno==554); + case 556: /* pseudo_column ::= QSTART */ yytestcase(yyruleno==556); + case 557: /* pseudo_column ::= QEND */ yytestcase(yyruleno==557); + case 558: /* pseudo_column ::= QDURATION */ yytestcase(yyruleno==558); + case 559: /* pseudo_column ::= WSTART */ yytestcase(yyruleno==559); + case 560: /* pseudo_column ::= WEND */ yytestcase(yyruleno==560); + case 561: /* pseudo_column ::= WDURATION */ yytestcase(yyruleno==561); + case 562: /* pseudo_column ::= IROWTS */ yytestcase(yyruleno==562); + case 563: /* pseudo_column ::= ISFILLED */ yytestcase(yyruleno==563); + case 564: /* pseudo_column ::= QTAGS */ yytestcase(yyruleno==564); + case 571: /* literal_func ::= NOW */ yytestcase(yyruleno==571); + case 572: /* literal_func ::= TODAY */ yytestcase(yyruleno==572); +{ yylhsminor.yy54 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[0].minor.yy0, NULL)); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; - case 554: /* pseudo_column ::= table_name NK_DOT TBNAME */ -#line 1175 "sql.y" -{ yylhsminor.yy416 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy1109, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[0].minor.yy0, createNodeList(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[-2].minor.yy1109)))); } -#line 8076 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + case 555: /* pseudo_column ::= table_name NK_DOT TBNAME */ +{ yylhsminor.yy54 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy837, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[0].minor.yy0, createNodeList(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[-2].minor.yy837)))); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 564: /* function_expression ::= function_name NK_LP expression_list NK_RP */ - case 565: /* function_expression ::= star_func NK_LP star_func_para_list NK_RP */ yytestcase(yyruleno==565); -#line 1186 "sql.y" -{ yylhsminor.yy416 = createRawExprNodeExt(pCxt, &yymsp[-3].minor.yy1109, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[-3].minor.yy1109, yymsp[-1].minor.yy316)); } -#line 8083 "sql.c" - yymsp[-3].minor.yy416 = yylhsminor.yy416; + case 565: /* function_expression ::= function_name NK_LP expression_list NK_RP */ + case 566: /* function_expression ::= star_func NK_LP star_func_para_list NK_RP */ yytestcase(yyruleno==566); +{ yylhsminor.yy54 = createRawExprNodeExt(pCxt, &yymsp[-3].minor.yy837, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[-3].minor.yy837, yymsp[-1].minor.yy652)); } + yymsp[-3].minor.yy54 = yylhsminor.yy54; break; - case 566: /* function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP */ - case 567: /* function_expression ::= CAST NK_LP expr_or_subquery AS type_name_default_len NK_RP */ yytestcase(yyruleno==567); -#line 1189 "sql.y" -{ yylhsminor.yy416 = createRawExprNodeExt(pCxt, &yymsp[-5].minor.yy0, &yymsp[0].minor.yy0, createCastFunctionNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy416), yymsp[-1].minor.yy952)); } -#line 8090 "sql.c" - yymsp[-5].minor.yy416 = yylhsminor.yy416; + case 567: /* function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP */ + case 568: /* function_expression ::= CAST NK_LP expr_or_subquery AS type_name_default_len NK_RP */ yytestcase(yyruleno==568); +{ yylhsminor.yy54 = createRawExprNodeExt(pCxt, &yymsp[-5].minor.yy0, &yymsp[0].minor.yy0, createCastFunctionNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy54), yymsp[-1].minor.yy84)); } + yymsp[-5].minor.yy54 = yylhsminor.yy54; break; - case 569: /* literal_func ::= noarg_func NK_LP NK_RP */ -#line 1195 "sql.y" -{ yylhsminor.yy416 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy1109, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[-2].minor.yy1109, NULL)); } -#line 8096 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + case 570: /* literal_func ::= noarg_func NK_LP NK_RP */ +{ yylhsminor.yy54 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy837, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[-2].minor.yy837, NULL)); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 585: /* star_func_para_list ::= NK_STAR */ -#line 1220 "sql.y" -{ yylhsminor.yy316 = createNodeList(pCxt, createColumnNode(pCxt, NULL, &yymsp[0].minor.yy0)); } -#line 8102 "sql.c" - yymsp[0].minor.yy316 = yylhsminor.yy316; + case 586: /* star_func_para_list ::= NK_STAR */ +{ yylhsminor.yy652 = createNodeList(pCxt, createColumnNode(pCxt, NULL, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy652 = yylhsminor.yy652; break; - case 590: /* star_func_para ::= table_name NK_DOT NK_STAR */ - case 675: /* select_item ::= table_name NK_DOT NK_STAR */ yytestcase(yyruleno==675); -#line 1229 "sql.y" -{ yylhsminor.yy416 = createColumnNode(pCxt, &yymsp[-2].minor.yy1109, &yymsp[0].minor.yy0); } -#line 8109 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + case 591: /* star_func_para ::= table_name NK_DOT NK_STAR */ + case 676: /* select_item ::= table_name NK_DOT NK_STAR */ yytestcase(yyruleno==676); +{ yylhsminor.yy54 = createColumnNode(pCxt, &yymsp[-2].minor.yy837, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 591: /* case_when_expression ::= CASE when_then_list case_when_else_opt END */ -#line 1232 "sql.y" -{ yylhsminor.yy416 = createRawExprNodeExt(pCxt, &yymsp[-3].minor.yy0, &yymsp[0].minor.yy0, createCaseWhenNode(pCxt, NULL, yymsp[-2].minor.yy316, yymsp[-1].minor.yy416)); } -#line 8115 "sql.c" - yymsp[-3].minor.yy416 = yylhsminor.yy416; + case 592: /* case_when_expression ::= CASE when_then_list case_when_else_opt END */ +{ yylhsminor.yy54 = createRawExprNodeExt(pCxt, &yymsp[-3].minor.yy0, &yymsp[0].minor.yy0, createCaseWhenNode(pCxt, NULL, yymsp[-2].minor.yy652, yymsp[-1].minor.yy54)); } + yymsp[-3].minor.yy54 = yylhsminor.yy54; break; - case 592: /* case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END */ -#line 1234 "sql.y" -{ yylhsminor.yy416 = createRawExprNodeExt(pCxt, &yymsp[-4].minor.yy0, &yymsp[0].minor.yy0, createCaseWhenNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy416), yymsp[-2].minor.yy316, yymsp[-1].minor.yy416)); } -#line 8121 "sql.c" - yymsp[-4].minor.yy416 = yylhsminor.yy416; + case 593: /* case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END */ +{ yylhsminor.yy54 = createRawExprNodeExt(pCxt, &yymsp[-4].minor.yy0, &yymsp[0].minor.yy0, createCaseWhenNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy54), yymsp[-2].minor.yy652, yymsp[-1].minor.yy54)); } + yymsp[-4].minor.yy54 = yylhsminor.yy54; break; - case 595: /* when_then_expr ::= WHEN common_expression THEN common_expression */ -#line 1241 "sql.y" -{ yymsp[-3].minor.yy416 = createWhenThenNode(pCxt, releaseRawExprNode(pCxt, yymsp[-2].minor.yy416), releaseRawExprNode(pCxt, yymsp[0].minor.yy416)); } -#line 8127 "sql.c" + case 596: /* when_then_expr ::= WHEN common_expression THEN common_expression */ +{ yymsp[-3].minor.yy54 = createWhenThenNode(pCxt, releaseRawExprNode(pCxt, yymsp[-2].minor.yy54), releaseRawExprNode(pCxt, yymsp[0].minor.yy54)); } break; - case 597: /* case_when_else_opt ::= ELSE common_expression */ -#line 1244 "sql.y" -{ yymsp[-1].minor.yy416 = releaseRawExprNode(pCxt, yymsp[0].minor.yy416); } -#line 8132 "sql.c" + case 598: /* case_when_else_opt ::= ELSE common_expression */ +{ yymsp[-1].minor.yy54 = releaseRawExprNode(pCxt, yymsp[0].minor.yy54); } break; - case 598: /* predicate ::= expr_or_subquery compare_op expr_or_subquery */ - case 603: /* predicate ::= expr_or_subquery in_op in_predicate_value */ yytestcase(yyruleno==603); -#line 1247 "sql.y" + case 599: /* predicate ::= expr_or_subquery compare_op expr_or_subquery */ + case 604: /* predicate ::= expr_or_subquery in_op in_predicate_value */ yytestcase(yyruleno==604); { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy416); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy416); - yylhsminor.yy416 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, yymsp[-1].minor.yy848, releaseRawExprNode(pCxt, yymsp[-2].minor.yy416), releaseRawExprNode(pCxt, yymsp[0].minor.yy416))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy54); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy54); + yylhsminor.yy54 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, yymsp[-1].minor.yy922, releaseRawExprNode(pCxt, yymsp[-2].minor.yy54), releaseRawExprNode(pCxt, yymsp[0].minor.yy54))); } -#line 8142 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 599: /* predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery */ -#line 1254 "sql.y" + case 600: /* predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-4].minor.yy416); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy416); - yylhsminor.yy416 = createRawExprNodeExt(pCxt, &s, &e, createBetweenAnd(pCxt, releaseRawExprNode(pCxt, yymsp[-4].minor.yy416), releaseRawExprNode(pCxt, yymsp[-2].minor.yy416), releaseRawExprNode(pCxt, yymsp[0].minor.yy416))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-4].minor.yy54); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy54); + yylhsminor.yy54 = createRawExprNodeExt(pCxt, &s, &e, createBetweenAnd(pCxt, releaseRawExprNode(pCxt, yymsp[-4].minor.yy54), releaseRawExprNode(pCxt, yymsp[-2].minor.yy54), releaseRawExprNode(pCxt, yymsp[0].minor.yy54))); } -#line 8152 "sql.c" - yymsp[-4].minor.yy416 = yylhsminor.yy416; + yymsp[-4].minor.yy54 = yylhsminor.yy54; break; - case 600: /* predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery */ -#line 1260 "sql.y" + case 601: /* predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-5].minor.yy416); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy416); - yylhsminor.yy416 = createRawExprNodeExt(pCxt, &s, &e, createNotBetweenAnd(pCxt, releaseRawExprNode(pCxt, yymsp[-5].minor.yy416), releaseRawExprNode(pCxt, yymsp[-2].minor.yy416), releaseRawExprNode(pCxt, yymsp[0].minor.yy416))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-5].minor.yy54); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy54); + yylhsminor.yy54 = createRawExprNodeExt(pCxt, &s, &e, createNotBetweenAnd(pCxt, releaseRawExprNode(pCxt, yymsp[-5].minor.yy54), releaseRawExprNode(pCxt, yymsp[-2].minor.yy54), releaseRawExprNode(pCxt, yymsp[0].minor.yy54))); } -#line 8162 "sql.c" - yymsp[-5].minor.yy416 = yylhsminor.yy416; + yymsp[-5].minor.yy54 = yylhsminor.yy54; break; - case 601: /* predicate ::= expr_or_subquery IS NULL */ -#line 1265 "sql.y" + case 602: /* predicate ::= expr_or_subquery IS NULL */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy416); - yylhsminor.yy416 = createRawExprNodeExt(pCxt, &s, &yymsp[0].minor.yy0, createOperatorNode(pCxt, OP_TYPE_IS_NULL, releaseRawExprNode(pCxt, yymsp[-2].minor.yy416), NULL)); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy54); + yylhsminor.yy54 = createRawExprNodeExt(pCxt, &s, &yymsp[0].minor.yy0, createOperatorNode(pCxt, OP_TYPE_IS_NULL, releaseRawExprNode(pCxt, yymsp[-2].minor.yy54), NULL)); } -#line 8171 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 602: /* predicate ::= expr_or_subquery IS NOT NULL */ -#line 1269 "sql.y" + case 603: /* predicate ::= expr_or_subquery IS NOT NULL */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-3].minor.yy416); - yylhsminor.yy416 = createRawExprNodeExt(pCxt, &s, &yymsp[0].minor.yy0, createOperatorNode(pCxt, OP_TYPE_IS_NOT_NULL, releaseRawExprNode(pCxt, yymsp[-3].minor.yy416), NULL)); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-3].minor.yy54); + yylhsminor.yy54 = createRawExprNodeExt(pCxt, &s, &yymsp[0].minor.yy0, createOperatorNode(pCxt, OP_TYPE_IS_NOT_NULL, releaseRawExprNode(pCxt, yymsp[-3].minor.yy54), NULL)); } -#line 8180 "sql.c" - yymsp[-3].minor.yy416 = yylhsminor.yy416; + yymsp[-3].minor.yy54 = yylhsminor.yy54; break; - case 604: /* compare_op ::= NK_LT */ -#line 1281 "sql.y" -{ yymsp[0].minor.yy848 = OP_TYPE_LOWER_THAN; } -#line 8186 "sql.c" + case 605: /* compare_op ::= NK_LT */ +{ yymsp[0].minor.yy922 = OP_TYPE_LOWER_THAN; } break; - case 605: /* compare_op ::= NK_GT */ -#line 1282 "sql.y" -{ yymsp[0].minor.yy848 = OP_TYPE_GREATER_THAN; } -#line 8191 "sql.c" + case 606: /* compare_op ::= NK_GT */ +{ yymsp[0].minor.yy922 = OP_TYPE_GREATER_THAN; } break; - case 606: /* compare_op ::= NK_LE */ -#line 1283 "sql.y" -{ yymsp[0].minor.yy848 = OP_TYPE_LOWER_EQUAL; } -#line 8196 "sql.c" + case 607: /* compare_op ::= NK_LE */ +{ yymsp[0].minor.yy922 = OP_TYPE_LOWER_EQUAL; } break; - case 607: /* compare_op ::= NK_GE */ -#line 1284 "sql.y" -{ yymsp[0].minor.yy848 = OP_TYPE_GREATER_EQUAL; } -#line 8201 "sql.c" + case 608: /* compare_op ::= NK_GE */ +{ yymsp[0].minor.yy922 = OP_TYPE_GREATER_EQUAL; } break; - case 608: /* compare_op ::= NK_NE */ -#line 1285 "sql.y" -{ yymsp[0].minor.yy848 = OP_TYPE_NOT_EQUAL; } -#line 8206 "sql.c" + case 609: /* compare_op ::= NK_NE */ +{ yymsp[0].minor.yy922 = OP_TYPE_NOT_EQUAL; } break; - case 609: /* compare_op ::= NK_EQ */ -#line 1286 "sql.y" -{ yymsp[0].minor.yy848 = OP_TYPE_EQUAL; } -#line 8211 "sql.c" + case 610: /* compare_op ::= NK_EQ */ +{ yymsp[0].minor.yy922 = OP_TYPE_EQUAL; } break; - case 610: /* compare_op ::= LIKE */ -#line 1287 "sql.y" -{ yymsp[0].minor.yy848 = OP_TYPE_LIKE; } -#line 8216 "sql.c" + case 611: /* compare_op ::= LIKE */ +{ yymsp[0].minor.yy922 = OP_TYPE_LIKE; } break; - case 611: /* compare_op ::= NOT LIKE */ -#line 1288 "sql.y" -{ yymsp[-1].minor.yy848 = OP_TYPE_NOT_LIKE; } -#line 8221 "sql.c" + case 612: /* compare_op ::= NOT LIKE */ +{ yymsp[-1].minor.yy922 = OP_TYPE_NOT_LIKE; } break; - case 612: /* compare_op ::= MATCH */ -#line 1289 "sql.y" -{ yymsp[0].minor.yy848 = OP_TYPE_MATCH; } -#line 8226 "sql.c" + case 613: /* compare_op ::= MATCH */ +{ yymsp[0].minor.yy922 = OP_TYPE_MATCH; } break; - case 613: /* compare_op ::= NMATCH */ -#line 1290 "sql.y" -{ yymsp[0].minor.yy848 = OP_TYPE_NMATCH; } -#line 8231 "sql.c" + case 614: /* compare_op ::= NMATCH */ +{ yymsp[0].minor.yy922 = OP_TYPE_NMATCH; } break; - case 614: /* compare_op ::= CONTAINS */ -#line 1291 "sql.y" -{ yymsp[0].minor.yy848 = OP_TYPE_JSON_CONTAINS; } -#line 8236 "sql.c" + case 615: /* compare_op ::= CONTAINS */ +{ yymsp[0].minor.yy922 = OP_TYPE_JSON_CONTAINS; } break; - case 615: /* in_op ::= IN */ -#line 1295 "sql.y" -{ yymsp[0].minor.yy848 = OP_TYPE_IN; } -#line 8241 "sql.c" + case 616: /* in_op ::= IN */ +{ yymsp[0].minor.yy922 = OP_TYPE_IN; } break; - case 616: /* in_op ::= NOT IN */ -#line 1296 "sql.y" -{ yymsp[-1].minor.yy848 = OP_TYPE_NOT_IN; } -#line 8246 "sql.c" + case 617: /* in_op ::= NOT IN */ +{ yymsp[-1].minor.yy922 = OP_TYPE_NOT_IN; } break; - case 617: /* in_predicate_value ::= NK_LP literal_list NK_RP */ -#line 1298 "sql.y" -{ yylhsminor.yy416 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0, createNodeListNode(pCxt, yymsp[-1].minor.yy316)); } -#line 8251 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + case 618: /* in_predicate_value ::= NK_LP literal_list NK_RP */ +{ yylhsminor.yy54 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0, createNodeListNode(pCxt, yymsp[-1].minor.yy652)); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 619: /* boolean_value_expression ::= NOT boolean_primary */ -#line 1302 "sql.y" + case 620: /* boolean_value_expression ::= NOT boolean_primary */ { - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy416); - yylhsminor.yy416 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &e, createLogicConditionNode(pCxt, LOGIC_COND_TYPE_NOT, releaseRawExprNode(pCxt, yymsp[0].minor.yy416), NULL)); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy54); + yylhsminor.yy54 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &e, createLogicConditionNode(pCxt, LOGIC_COND_TYPE_NOT, releaseRawExprNode(pCxt, yymsp[0].minor.yy54), NULL)); } -#line 8260 "sql.c" - yymsp[-1].minor.yy416 = yylhsminor.yy416; + yymsp[-1].minor.yy54 = yylhsminor.yy54; break; - case 620: /* boolean_value_expression ::= boolean_value_expression OR boolean_value_expression */ -#line 1307 "sql.y" + case 621: /* boolean_value_expression ::= boolean_value_expression OR boolean_value_expression */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy416); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy416); - yylhsminor.yy416 = createRawExprNodeExt(pCxt, &s, &e, createLogicConditionNode(pCxt, LOGIC_COND_TYPE_OR, releaseRawExprNode(pCxt, yymsp[-2].minor.yy416), releaseRawExprNode(pCxt, yymsp[0].minor.yy416))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy54); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy54); + yylhsminor.yy54 = createRawExprNodeExt(pCxt, &s, &e, createLogicConditionNode(pCxt, LOGIC_COND_TYPE_OR, releaseRawExprNode(pCxt, yymsp[-2].minor.yy54), releaseRawExprNode(pCxt, yymsp[0].minor.yy54))); } -#line 8270 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 621: /* boolean_value_expression ::= boolean_value_expression AND boolean_value_expression */ -#line 1313 "sql.y" + case 622: /* boolean_value_expression ::= boolean_value_expression AND boolean_value_expression */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy416); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy416); - yylhsminor.yy416 = createRawExprNodeExt(pCxt, &s, &e, createLogicConditionNode(pCxt, LOGIC_COND_TYPE_AND, releaseRawExprNode(pCxt, yymsp[-2].minor.yy416), releaseRawExprNode(pCxt, yymsp[0].minor.yy416))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy54); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy54); + yylhsminor.yy54 = createRawExprNodeExt(pCxt, &s, &e, createLogicConditionNode(pCxt, LOGIC_COND_TYPE_AND, releaseRawExprNode(pCxt, yymsp[-2].minor.yy54), releaseRawExprNode(pCxt, yymsp[0].minor.yy54))); } -#line 8280 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 629: /* table_reference_list ::= table_reference_list NK_COMMA table_reference */ -#line 1331 "sql.y" -{ yylhsminor.yy416 = createJoinTableNode(pCxt, JOIN_TYPE_INNER, JOIN_STYPE_NONE, yymsp[-2].minor.yy416, yymsp[0].minor.yy416, NULL); } -#line 8286 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + case 630: /* table_reference_list ::= table_reference_list NK_COMMA table_reference */ +{ yylhsminor.yy54 = createJoinTableNode(pCxt, JOIN_TYPE_INNER, JOIN_STYPE_NONE, yymsp[-2].minor.yy54, yymsp[0].minor.yy54, NULL); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 632: /* table_primary ::= table_name alias_opt */ -#line 1337 "sql.y" -{ yylhsminor.yy416 = createRealTableNode(pCxt, NULL, &yymsp[-1].minor.yy1109, &yymsp[0].minor.yy1109); } -#line 8292 "sql.c" - yymsp[-1].minor.yy416 = yylhsminor.yy416; + case 633: /* table_primary ::= table_name alias_opt */ +{ yylhsminor.yy54 = createRealTableNode(pCxt, NULL, &yymsp[-1].minor.yy837, &yymsp[0].minor.yy837); } + yymsp[-1].minor.yy54 = yylhsminor.yy54; break; - case 633: /* table_primary ::= db_name NK_DOT table_name alias_opt */ -#line 1338 "sql.y" -{ yylhsminor.yy416 = createRealTableNode(pCxt, &yymsp[-3].minor.yy1109, &yymsp[-1].minor.yy1109, &yymsp[0].minor.yy1109); } -#line 8298 "sql.c" - yymsp[-3].minor.yy416 = yylhsminor.yy416; + case 634: /* table_primary ::= db_name NK_DOT table_name alias_opt */ +{ yylhsminor.yy54 = createRealTableNode(pCxt, &yymsp[-3].minor.yy837, &yymsp[-1].minor.yy837, &yymsp[0].minor.yy837); } + yymsp[-3].minor.yy54 = yylhsminor.yy54; break; - case 634: /* table_primary ::= subquery alias_opt */ -#line 1339 "sql.y" -{ yylhsminor.yy416 = createTempTableNode(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy416), &yymsp[0].minor.yy1109); } -#line 8304 "sql.c" - yymsp[-1].minor.yy416 = yylhsminor.yy416; + case 635: /* table_primary ::= subquery alias_opt */ +{ yylhsminor.yy54 = createTempTableNode(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy54), &yymsp[0].minor.yy837); } + yymsp[-1].minor.yy54 = yylhsminor.yy54; break; - case 636: /* alias_opt ::= */ -#line 1344 "sql.y" -{ yymsp[1].minor.yy1109 = nil_token; } -#line 8310 "sql.c" + case 637: /* alias_opt ::= */ +{ yymsp[1].minor.yy837 = nil_token; } break; - case 638: /* alias_opt ::= AS table_alias */ -#line 1346 "sql.y" -{ yymsp[-1].minor.yy1109 = yymsp[0].minor.yy1109; } -#line 8315 "sql.c" + case 639: /* alias_opt ::= AS table_alias */ +{ yymsp[-1].minor.yy837 = yymsp[0].minor.yy837; } break; - case 639: /* parenthesized_joined_table ::= NK_LP joined_table NK_RP */ - case 640: /* parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP */ yytestcase(yyruleno==640); -#line 1348 "sql.y" -{ yymsp[-2].minor.yy416 = yymsp[-1].minor.yy416; } -#line 8321 "sql.c" + case 640: /* parenthesized_joined_table ::= NK_LP joined_table NK_RP */ + case 641: /* parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP */ yytestcase(yyruleno==641); +{ yymsp[-2].minor.yy54 = yymsp[-1].minor.yy54; } break; - case 641: /* joined_table ::= table_reference join_type join_subtype JOIN table_reference join_on_clause_opt window_offset_clause_opt jlimit_clause_opt */ -#line 1354 "sql.y" + case 642: /* joined_table ::= table_reference join_type join_subtype JOIN table_reference join_on_clause_opt window_offset_clause_opt jlimit_clause_opt */ { - yylhsminor.yy416 = createJoinTableNode(pCxt, yymsp[-6].minor.yy972, yymsp[-5].minor.yy630, yymsp[-7].minor.yy416, yymsp[-3].minor.yy416, yymsp[-2].minor.yy416); - yylhsminor.yy416 = addWindowOffsetClause(pCxt, yylhsminor.yy416, yymsp[-1].minor.yy416); - yylhsminor.yy416 = addJLimitClause(pCxt, yylhsminor.yy416, yymsp[0].minor.yy416); + yylhsminor.yy54 = createJoinTableNode(pCxt, yymsp[-6].minor.yy230, yymsp[-5].minor.yy948, yymsp[-7].minor.yy54, yymsp[-3].minor.yy54, yymsp[-2].minor.yy54); + yylhsminor.yy54 = addWindowOffsetClause(pCxt, yylhsminor.yy54, yymsp[-1].minor.yy54); + yylhsminor.yy54 = addJLimitClause(pCxt, yylhsminor.yy54, yymsp[0].minor.yy54); } -#line 8330 "sql.c" - yymsp[-7].minor.yy416 = yylhsminor.yy416; + yymsp[-7].minor.yy54 = yylhsminor.yy54; break; - case 642: /* join_type ::= */ -#line 1362 "sql.y" -{ yymsp[1].minor.yy972 = JOIN_TYPE_INNER; } -#line 8336 "sql.c" + case 643: /* join_type ::= */ +{ yymsp[1].minor.yy230 = JOIN_TYPE_INNER; } break; - case 643: /* join_type ::= INNER */ -#line 1363 "sql.y" -{ yymsp[0].minor.yy972 = JOIN_TYPE_INNER; } -#line 8341 "sql.c" + case 644: /* join_type ::= INNER */ +{ yymsp[0].minor.yy230 = JOIN_TYPE_INNER; } break; - case 644: /* join_type ::= LEFT */ -#line 1364 "sql.y" -{ yymsp[0].minor.yy972 = JOIN_TYPE_LEFT; } -#line 8346 "sql.c" + case 645: /* join_type ::= LEFT */ +{ yymsp[0].minor.yy230 = JOIN_TYPE_LEFT; } break; - case 645: /* join_type ::= RIGHT */ -#line 1365 "sql.y" -{ yymsp[0].minor.yy972 = JOIN_TYPE_RIGHT; } -#line 8351 "sql.c" + case 646: /* join_type ::= RIGHT */ +{ yymsp[0].minor.yy230 = JOIN_TYPE_RIGHT; } break; - case 646: /* join_type ::= FULL */ -#line 1366 "sql.y" -{ yymsp[0].minor.yy972 = JOIN_TYPE_FULL; } -#line 8356 "sql.c" + case 647: /* join_type ::= FULL */ +{ yymsp[0].minor.yy230 = JOIN_TYPE_FULL; } break; - case 647: /* join_subtype ::= */ -#line 1370 "sql.y" -{ yymsp[1].minor.yy630 = JOIN_STYPE_NONE; } -#line 8361 "sql.c" + case 648: /* join_subtype ::= */ +{ yymsp[1].minor.yy948 = JOIN_STYPE_NONE; } break; - case 648: /* join_subtype ::= OUTER */ -#line 1371 "sql.y" -{ yymsp[0].minor.yy630 = JOIN_STYPE_OUTER; } -#line 8366 "sql.c" + case 649: /* join_subtype ::= OUTER */ +{ yymsp[0].minor.yy948 = JOIN_STYPE_OUTER; } break; - case 649: /* join_subtype ::= SEMI */ -#line 1372 "sql.y" -{ yymsp[0].minor.yy630 = JOIN_STYPE_SEMI; } -#line 8371 "sql.c" + case 650: /* join_subtype ::= SEMI */ +{ yymsp[0].minor.yy948 = JOIN_STYPE_SEMI; } break; - case 650: /* join_subtype ::= ANTI */ -#line 1373 "sql.y" -{ yymsp[0].minor.yy630 = JOIN_STYPE_ANTI; } -#line 8376 "sql.c" + case 651: /* join_subtype ::= ANTI */ +{ yymsp[0].minor.yy948 = JOIN_STYPE_ANTI; } break; - case 651: /* join_subtype ::= ASOF */ -#line 1374 "sql.y" -{ yymsp[0].minor.yy630 = JOIN_STYPE_ASOF; } -#line 8381 "sql.c" + case 652: /* join_subtype ::= ASOF */ +{ yymsp[0].minor.yy948 = JOIN_STYPE_ASOF; } break; - case 652: /* join_subtype ::= WINDOW */ -#line 1375 "sql.y" -{ yymsp[0].minor.yy630 = JOIN_STYPE_WIN; } -#line 8386 "sql.c" + case 653: /* join_subtype ::= WINDOW */ +{ yymsp[0].minor.yy948 = JOIN_STYPE_WIN; } break; - case 656: /* window_offset_clause_opt ::= WINDOW_OFFSET NK_LP window_offset_literal NK_COMMA window_offset_literal NK_RP */ -#line 1382 "sql.y" -{ yymsp[-5].minor.yy416 = createWindowOffsetNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy416), releaseRawExprNode(pCxt, yymsp[-1].minor.yy416)); } -#line 8391 "sql.c" + case 657: /* window_offset_clause_opt ::= WINDOW_OFFSET NK_LP window_offset_literal NK_COMMA window_offset_literal NK_RP */ +{ yymsp[-5].minor.yy54 = createWindowOffsetNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy54), releaseRawExprNode(pCxt, yymsp[-1].minor.yy54)); } break; - case 657: /* window_offset_literal ::= NK_VARIABLE */ -#line 1384 "sql.y" -{ yylhsminor.yy416 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createTimeOffsetValueNode(pCxt, &yymsp[0].minor.yy0)); } -#line 8396 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; + case 658: /* window_offset_literal ::= NK_VARIABLE */ +{ yylhsminor.yy54 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createTimeOffsetValueNode(pCxt, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; - case 658: /* window_offset_literal ::= NK_MINUS NK_VARIABLE */ -#line 1385 "sql.y" + case 659: /* window_offset_literal ::= NK_MINUS NK_VARIABLE */ { SToken t = yymsp[-1].minor.yy0; t.n = (yymsp[0].minor.yy0.z + yymsp[0].minor.yy0.n) - yymsp[-1].minor.yy0.z; - yylhsminor.yy416 = createRawExprNode(pCxt, &t, createTimeOffsetValueNode(pCxt, &t)); + yylhsminor.yy54 = createRawExprNode(pCxt, &t, createTimeOffsetValueNode(pCxt, &t)); } -#line 8406 "sql.c" - yymsp[-1].minor.yy416 = yylhsminor.yy416; + yymsp[-1].minor.yy54 = yylhsminor.yy54; break; - case 660: /* jlimit_clause_opt ::= JLIMIT NK_INTEGER */ - case 731: /* slimit_clause_opt ::= SLIMIT NK_INTEGER */ yytestcase(yyruleno==731); - case 735: /* limit_clause_opt ::= LIMIT NK_INTEGER */ yytestcase(yyruleno==735); -#line 1392 "sql.y" -{ yymsp[-1].minor.yy416 = createLimitNode(pCxt, &yymsp[0].minor.yy0, NULL); } -#line 8414 "sql.c" + case 661: /* jlimit_clause_opt ::= JLIMIT NK_INTEGER */ + case 732: /* slimit_clause_opt ::= SLIMIT NK_INTEGER */ yytestcase(yyruleno==732); + case 736: /* limit_clause_opt ::= LIMIT NK_INTEGER */ yytestcase(yyruleno==736); +{ yymsp[-1].minor.yy54 = createLimitNode(pCxt, &yymsp[0].minor.yy0, NULL); } break; - case 661: /* query_specification ::= SELECT hint_list set_quantifier_opt tag_mode_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 1398 "sql.y" + case 662: /* query_specification ::= SELECT hint_list set_quantifier_opt tag_mode_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[-13].minor.yy416 = createSelectStmt(pCxt, yymsp[-11].minor.yy209, yymsp[-9].minor.yy316, yymsp[-8].minor.yy416, yymsp[-12].minor.yy316); - yymsp[-13].minor.yy416 = setSelectStmtTagMode(pCxt, yymsp[-13].minor.yy416, yymsp[-10].minor.yy209); - yymsp[-13].minor.yy416 = addWhereClause(pCxt, yymsp[-13].minor.yy416, yymsp[-7].minor.yy416); - yymsp[-13].minor.yy416 = addPartitionByClause(pCxt, yymsp[-13].minor.yy416, yymsp[-6].minor.yy316); - yymsp[-13].minor.yy416 = addWindowClauseClause(pCxt, yymsp[-13].minor.yy416, yymsp[-2].minor.yy416); - yymsp[-13].minor.yy416 = addGroupByClause(pCxt, yymsp[-13].minor.yy416, yymsp[-1].minor.yy316); - yymsp[-13].minor.yy416 = addHavingClause(pCxt, yymsp[-13].minor.yy416, yymsp[0].minor.yy416); - yymsp[-13].minor.yy416 = addRangeClause(pCxt, yymsp[-13].minor.yy416, yymsp[-5].minor.yy416); - yymsp[-13].minor.yy416 = addEveryClause(pCxt, yymsp[-13].minor.yy416, yymsp[-4].minor.yy416); - yymsp[-13].minor.yy416 = addFillClause(pCxt, yymsp[-13].minor.yy416, yymsp[-3].minor.yy416); + yymsp[-13].minor.yy54 = createSelectStmt(pCxt, yymsp[-11].minor.yy325, yymsp[-9].minor.yy652, yymsp[-8].minor.yy54, yymsp[-12].minor.yy652); + yymsp[-13].minor.yy54 = setSelectStmtTagMode(pCxt, yymsp[-13].minor.yy54, yymsp[-10].minor.yy325); + yymsp[-13].minor.yy54 = addWhereClause(pCxt, yymsp[-13].minor.yy54, yymsp[-7].minor.yy54); + yymsp[-13].minor.yy54 = addPartitionByClause(pCxt, yymsp[-13].minor.yy54, yymsp[-6].minor.yy652); + yymsp[-13].minor.yy54 = addWindowClauseClause(pCxt, yymsp[-13].minor.yy54, yymsp[-2].minor.yy54); + yymsp[-13].minor.yy54 = addGroupByClause(pCxt, yymsp[-13].minor.yy54, yymsp[-1].minor.yy652); + yymsp[-13].minor.yy54 = addHavingClause(pCxt, yymsp[-13].minor.yy54, yymsp[0].minor.yy54); + yymsp[-13].minor.yy54 = addRangeClause(pCxt, yymsp[-13].minor.yy54, yymsp[-5].minor.yy54); + yymsp[-13].minor.yy54 = addEveryClause(pCxt, yymsp[-13].minor.yy54, yymsp[-4].minor.yy54); + yymsp[-13].minor.yy54 = addFillClause(pCxt, yymsp[-13].minor.yy54, yymsp[-3].minor.yy54); } -#line 8430 "sql.c" break; - case 662: /* hint_list ::= */ -#line 1413 "sql.y" -{ yymsp[1].minor.yy316 = createHintNodeList(pCxt, NULL); } -#line 8435 "sql.c" + case 663: /* hint_list ::= */ +{ yymsp[1].minor.yy652 = createHintNodeList(pCxt, NULL); } break; - case 663: /* hint_list ::= NK_HINT */ -#line 1414 "sql.y" -{ yylhsminor.yy316 = createHintNodeList(pCxt, &yymsp[0].minor.yy0); } -#line 8440 "sql.c" - yymsp[0].minor.yy316 = yylhsminor.yy316; + case 664: /* hint_list ::= NK_HINT */ +{ yylhsminor.yy652 = createHintNodeList(pCxt, &yymsp[0].minor.yy0); } + yymsp[0].minor.yy652 = yylhsminor.yy652; break; - case 668: /* set_quantifier_opt ::= ALL */ -#line 1425 "sql.y" -{ yymsp[0].minor.yy209 = false; } -#line 8446 "sql.c" + case 669: /* set_quantifier_opt ::= ALL */ +{ yymsp[0].minor.yy325 = false; } break; - case 671: /* select_item ::= NK_STAR */ -#line 1432 "sql.y" -{ yylhsminor.yy416 = createColumnNode(pCxt, NULL, &yymsp[0].minor.yy0); } -#line 8451 "sql.c" - yymsp[0].minor.yy416 = yylhsminor.yy416; + case 672: /* select_item ::= NK_STAR */ +{ yylhsminor.yy54 = createColumnNode(pCxt, NULL, &yymsp[0].minor.yy0); } + yymsp[0].minor.yy54 = yylhsminor.yy54; break; - case 673: /* select_item ::= common_expression column_alias */ - case 683: /* partition_item ::= expr_or_subquery column_alias */ yytestcase(yyruleno==683); -#line 1434 "sql.y" -{ yylhsminor.yy416 = setProjectionAlias(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy416), &yymsp[0].minor.yy1109); } -#line 8458 "sql.c" - yymsp[-1].minor.yy416 = yylhsminor.yy416; + case 674: /* select_item ::= common_expression column_alias */ + case 684: /* partition_item ::= expr_or_subquery column_alias */ yytestcase(yyruleno==684); +{ yylhsminor.yy54 = setProjectionAlias(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy54), &yymsp[0].minor.yy837); } + yymsp[-1].minor.yy54 = yylhsminor.yy54; break; - case 674: /* select_item ::= common_expression AS column_alias */ - case 684: /* partition_item ::= expr_or_subquery AS column_alias */ yytestcase(yyruleno==684); -#line 1435 "sql.y" -{ yylhsminor.yy416 = setProjectionAlias(pCxt, releaseRawExprNode(pCxt, yymsp[-2].minor.yy416), &yymsp[0].minor.yy1109); } -#line 8465 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + case 675: /* select_item ::= common_expression AS column_alias */ + case 685: /* partition_item ::= expr_or_subquery AS column_alias */ yytestcase(yyruleno==685); +{ yylhsminor.yy54 = setProjectionAlias(pCxt, releaseRawExprNode(pCxt, yymsp[-2].minor.yy54), &yymsp[0].minor.yy837); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 679: /* partition_by_clause_opt ::= PARTITION BY partition_list */ - case 709: /* group_by_clause_opt ::= GROUP BY group_by_list */ yytestcase(yyruleno==709); - case 729: /* order_by_clause_opt ::= ORDER BY sort_specification_list */ yytestcase(yyruleno==729); -#line 1444 "sql.y" -{ yymsp[-2].minor.yy316 = yymsp[0].minor.yy316; } -#line 8473 "sql.c" + case 680: /* partition_by_clause_opt ::= PARTITION BY partition_list */ + case 710: /* group_by_clause_opt ::= GROUP BY group_by_list */ yytestcase(yyruleno==710); + case 730: /* order_by_clause_opt ::= ORDER BY sort_specification_list */ yytestcase(yyruleno==730); +{ yymsp[-2].minor.yy652 = yymsp[0].minor.yy652; } break; - case 686: /* twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA interval_sliding_duration_literal NK_RP */ -#line 1457 "sql.y" -{ yymsp[-5].minor.yy416 = createSessionWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy416), releaseRawExprNode(pCxt, yymsp[-1].minor.yy416)); } -#line 8478 "sql.c" + case 687: /* twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA interval_sliding_duration_literal NK_RP */ +{ yymsp[-5].minor.yy54 = createSessionWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy54), releaseRawExprNode(pCxt, yymsp[-1].minor.yy54)); } break; - case 687: /* twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP */ -#line 1458 "sql.y" -{ yymsp[-3].minor.yy416 = createStateWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy416)); } -#line 8483 "sql.c" + case 688: /* twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP */ +{ yymsp[-3].minor.yy54 = createStateWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy54)); } break; - case 688: /* twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_RP sliding_opt fill_opt */ -#line 1460 "sql.y" -{ yymsp[-5].minor.yy416 = createIntervalWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy416), NULL, yymsp[-1].minor.yy416, yymsp[0].minor.yy416); } -#line 8488 "sql.c" + case 689: /* twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_RP sliding_opt fill_opt */ +{ yymsp[-5].minor.yy54 = createIntervalWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy54), NULL, yymsp[-1].minor.yy54, yymsp[0].minor.yy54); } break; - case 689: /* twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_COMMA interval_sliding_duration_literal NK_RP sliding_opt fill_opt */ -#line 1464 "sql.y" -{ yymsp[-7].minor.yy416 = createIntervalWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-5].minor.yy416), releaseRawExprNode(pCxt, yymsp[-3].minor.yy416), yymsp[-1].minor.yy416, yymsp[0].minor.yy416); } -#line 8493 "sql.c" + case 690: /* twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_COMMA interval_sliding_duration_literal NK_RP sliding_opt fill_opt */ +{ yymsp[-7].minor.yy54 = createIntervalWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-5].minor.yy54), releaseRawExprNode(pCxt, yymsp[-3].minor.yy54), yymsp[-1].minor.yy54, yymsp[0].minor.yy54); } break; - case 690: /* twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition */ -#line 1466 "sql.y" -{ yymsp[-6].minor.yy416 = createEventWindowNode(pCxt, yymsp[-3].minor.yy416, yymsp[0].minor.yy416); } -#line 8498 "sql.c" + case 691: /* twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition */ +{ yymsp[-6].minor.yy54 = createEventWindowNode(pCxt, yymsp[-3].minor.yy54, yymsp[0].minor.yy54); } break; - case 691: /* twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_RP */ -#line 1468 "sql.y" -{ yymsp[-3].minor.yy416 = createCountWindowNode(pCxt, &yymsp[-1].minor.yy0, &yymsp[-1].minor.yy0); } -#line 8503 "sql.c" + case 692: /* twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_RP */ +{ yymsp[-3].minor.yy54 = createCountWindowNode(pCxt, &yymsp[-1].minor.yy0, &yymsp[-1].minor.yy0); } break; - case 692: /* twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP */ -#line 1470 "sql.y" -{ yymsp[-5].minor.yy416 = createCountWindowNode(pCxt, &yymsp[-3].minor.yy0, &yymsp[-1].minor.yy0); } -#line 8508 "sql.c" + case 693: /* twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP */ +{ yymsp[-5].minor.yy54 = createCountWindowNode(pCxt, &yymsp[-3].minor.yy0, &yymsp[-1].minor.yy0); } break; - case 699: /* fill_opt ::= FILL NK_LP fill_mode NK_RP */ -#line 1480 "sql.y" -{ yymsp[-3].minor.yy416 = createFillNode(pCxt, yymsp[-1].minor.yy882, NULL); } -#line 8513 "sql.c" + case 700: /* fill_opt ::= FILL NK_LP fill_mode NK_RP */ +{ yymsp[-3].minor.yy54 = createFillNode(pCxt, yymsp[-1].minor.yy478, NULL); } break; - case 700: /* fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP */ -#line 1481 "sql.y" -{ yymsp[-5].minor.yy416 = createFillNode(pCxt, FILL_MODE_VALUE, createNodeListNode(pCxt, yymsp[-1].minor.yy316)); } -#line 8518 "sql.c" + case 701: /* fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP */ +{ yymsp[-5].minor.yy54 = createFillNode(pCxt, FILL_MODE_VALUE, createNodeListNode(pCxt, yymsp[-1].minor.yy652)); } break; - case 701: /* fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP */ -#line 1482 "sql.y" -{ yymsp[-5].minor.yy416 = createFillNode(pCxt, FILL_MODE_VALUE_F, createNodeListNode(pCxt, yymsp[-1].minor.yy316)); } -#line 8523 "sql.c" + case 702: /* fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP */ +{ yymsp[-5].minor.yy54 = createFillNode(pCxt, FILL_MODE_VALUE_F, createNodeListNode(pCxt, yymsp[-1].minor.yy652)); } break; - case 702: /* fill_mode ::= NONE */ -#line 1486 "sql.y" -{ yymsp[0].minor.yy882 = FILL_MODE_NONE; } -#line 8528 "sql.c" + case 703: /* fill_mode ::= NONE */ +{ yymsp[0].minor.yy478 = FILL_MODE_NONE; } break; - case 703: /* fill_mode ::= PREV */ -#line 1487 "sql.y" -{ yymsp[0].minor.yy882 = FILL_MODE_PREV; } -#line 8533 "sql.c" + case 704: /* fill_mode ::= PREV */ +{ yymsp[0].minor.yy478 = FILL_MODE_PREV; } break; - case 704: /* fill_mode ::= NULL */ -#line 1488 "sql.y" -{ yymsp[0].minor.yy882 = FILL_MODE_NULL; } -#line 8538 "sql.c" + case 705: /* fill_mode ::= NULL */ +{ yymsp[0].minor.yy478 = FILL_MODE_NULL; } break; - case 705: /* fill_mode ::= NULL_F */ -#line 1489 "sql.y" -{ yymsp[0].minor.yy882 = FILL_MODE_NULL_F; } -#line 8543 "sql.c" + case 706: /* fill_mode ::= NULL_F */ +{ yymsp[0].minor.yy478 = FILL_MODE_NULL_F; } break; - case 706: /* fill_mode ::= LINEAR */ -#line 1490 "sql.y" -{ yymsp[0].minor.yy882 = FILL_MODE_LINEAR; } -#line 8548 "sql.c" + case 707: /* fill_mode ::= LINEAR */ +{ yymsp[0].minor.yy478 = FILL_MODE_LINEAR; } break; - case 707: /* fill_mode ::= NEXT */ -#line 1491 "sql.y" -{ yymsp[0].minor.yy882 = FILL_MODE_NEXT; } -#line 8553 "sql.c" + case 708: /* fill_mode ::= NEXT */ +{ yymsp[0].minor.yy478 = FILL_MODE_NEXT; } break; - case 710: /* group_by_list ::= expr_or_subquery */ -#line 1500 "sql.y" -{ yylhsminor.yy316 = createNodeList(pCxt, createGroupingSetNode(pCxt, releaseRawExprNode(pCxt, yymsp[0].minor.yy416))); } -#line 8558 "sql.c" - yymsp[0].minor.yy316 = yylhsminor.yy316; + case 711: /* group_by_list ::= expr_or_subquery */ +{ yylhsminor.yy652 = createNodeList(pCxt, createGroupingSetNode(pCxt, releaseRawExprNode(pCxt, yymsp[0].minor.yy54))); } + yymsp[0].minor.yy652 = yylhsminor.yy652; break; - case 711: /* group_by_list ::= group_by_list NK_COMMA expr_or_subquery */ -#line 1501 "sql.y" -{ yylhsminor.yy316 = addNodeToList(pCxt, yymsp[-2].minor.yy316, createGroupingSetNode(pCxt, releaseRawExprNode(pCxt, yymsp[0].minor.yy416))); } -#line 8564 "sql.c" - yymsp[-2].minor.yy316 = yylhsminor.yy316; + case 712: /* group_by_list ::= group_by_list NK_COMMA expr_or_subquery */ +{ yylhsminor.yy652 = addNodeToList(pCxt, yymsp[-2].minor.yy652, createGroupingSetNode(pCxt, releaseRawExprNode(pCxt, yymsp[0].minor.yy54))); } + yymsp[-2].minor.yy652 = yylhsminor.yy652; break; - case 715: /* range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP */ -#line 1508 "sql.y" -{ yymsp[-5].minor.yy416 = createInterpTimeRange(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy416), releaseRawExprNode(pCxt, yymsp[-1].minor.yy416)); } -#line 8570 "sql.c" + case 716: /* range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP */ +{ yymsp[-5].minor.yy54 = createInterpTimeRange(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy54), releaseRawExprNode(pCxt, yymsp[-1].minor.yy54)); } break; - case 716: /* range_opt ::= RANGE NK_LP expr_or_subquery NK_RP */ -#line 1510 "sql.y" -{ yymsp[-3].minor.yy416 = createInterpTimePoint(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy416)); } -#line 8575 "sql.c" + case 717: /* range_opt ::= RANGE NK_LP expr_or_subquery NK_RP */ +{ yymsp[-3].minor.yy54 = createInterpTimePoint(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy54)); } break; - case 719: /* query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt */ -#line 1517 "sql.y" + case 720: /* query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt */ { - yylhsminor.yy416 = addOrderByClause(pCxt, yymsp[-3].minor.yy416, yymsp[-2].minor.yy316); - yylhsminor.yy416 = addSlimitClause(pCxt, yylhsminor.yy416, yymsp[-1].minor.yy416); - yylhsminor.yy416 = addLimitClause(pCxt, yylhsminor.yy416, yymsp[0].minor.yy416); + yylhsminor.yy54 = addOrderByClause(pCxt, yymsp[-3].minor.yy54, yymsp[-2].minor.yy652); + yylhsminor.yy54 = addSlimitClause(pCxt, yylhsminor.yy54, yymsp[-1].minor.yy54); + yylhsminor.yy54 = addLimitClause(pCxt, yylhsminor.yy54, yymsp[0].minor.yy54); } -#line 8584 "sql.c" - yymsp[-3].minor.yy416 = yylhsminor.yy416; + yymsp[-3].minor.yy54 = yylhsminor.yy54; break; - case 722: /* union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery */ -#line 1527 "sql.y" -{ yylhsminor.yy416 = createSetOperator(pCxt, SET_OP_TYPE_UNION_ALL, yymsp[-3].minor.yy416, yymsp[0].minor.yy416); } -#line 8590 "sql.c" - yymsp[-3].minor.yy416 = yylhsminor.yy416; + case 723: /* union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery */ +{ yylhsminor.yy54 = createSetOperator(pCxt, SET_OP_TYPE_UNION_ALL, yymsp[-3].minor.yy54, yymsp[0].minor.yy54); } + yymsp[-3].minor.yy54 = yylhsminor.yy54; break; - case 723: /* union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery */ -#line 1529 "sql.y" -{ yylhsminor.yy416 = createSetOperator(pCxt, SET_OP_TYPE_UNION, yymsp[-2].minor.yy416, yymsp[0].minor.yy416); } -#line 8596 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + case 724: /* union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery */ +{ yylhsminor.yy54 = createSetOperator(pCxt, SET_OP_TYPE_UNION, yymsp[-2].minor.yy54, yymsp[0].minor.yy54); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 732: /* slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER */ - case 736: /* limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER */ yytestcase(yyruleno==736); -#line 1544 "sql.y" -{ yymsp[-3].minor.yy416 = createLimitNode(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0); } -#line 8603 "sql.c" + case 733: /* slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER */ + case 737: /* limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER */ yytestcase(yyruleno==737); +{ yymsp[-3].minor.yy54 = createLimitNode(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0); } break; - case 733: /* slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER */ - case 737: /* limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER */ yytestcase(yyruleno==737); -#line 1545 "sql.y" -{ yymsp[-3].minor.yy416 = createLimitNode(pCxt, &yymsp[0].minor.yy0, &yymsp[-2].minor.yy0); } -#line 8609 "sql.c" + case 734: /* slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER */ + case 738: /* limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER */ yytestcase(yyruleno==738); +{ yymsp[-3].minor.yy54 = createLimitNode(pCxt, &yymsp[0].minor.yy0, &yymsp[-2].minor.yy0); } break; - case 738: /* subquery ::= NK_LP query_expression NK_RP */ -#line 1553 "sql.y" -{ yylhsminor.yy416 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0, yymsp[-1].minor.yy416); } -#line 8614 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + case 739: /* subquery ::= NK_LP query_expression NK_RP */ +{ yylhsminor.yy54 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0, yymsp[-1].minor.yy54); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 743: /* sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt */ -#line 1567 "sql.y" -{ yylhsminor.yy416 = createOrderByExprNode(pCxt, releaseRawExprNode(pCxt, yymsp[-2].minor.yy416), yymsp[-1].minor.yy506, yymsp[0].minor.yy1045); } -#line 8620 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + case 744: /* sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt */ +{ yylhsminor.yy54 = createOrderByExprNode(pCxt, releaseRawExprNode(pCxt, yymsp[-2].minor.yy54), yymsp[-1].minor.yy760, yymsp[0].minor.yy503); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 744: /* ordering_specification_opt ::= */ -#line 1571 "sql.y" -{ yymsp[1].minor.yy506 = ORDER_ASC; } -#line 8626 "sql.c" + case 745: /* ordering_specification_opt ::= */ +{ yymsp[1].minor.yy760 = ORDER_ASC; } break; - case 745: /* ordering_specification_opt ::= ASC */ -#line 1572 "sql.y" -{ yymsp[0].minor.yy506 = ORDER_ASC; } -#line 8631 "sql.c" + case 746: /* ordering_specification_opt ::= ASC */ +{ yymsp[0].minor.yy760 = ORDER_ASC; } break; - case 746: /* ordering_specification_opt ::= DESC */ -#line 1573 "sql.y" -{ yymsp[0].minor.yy506 = ORDER_DESC; } -#line 8636 "sql.c" + case 747: /* ordering_specification_opt ::= DESC */ +{ yymsp[0].minor.yy760 = ORDER_DESC; } break; - case 747: /* null_ordering_opt ::= */ -#line 1577 "sql.y" -{ yymsp[1].minor.yy1045 = NULL_ORDER_DEFAULT; } -#line 8641 "sql.c" + case 748: /* null_ordering_opt ::= */ +{ yymsp[1].minor.yy503 = NULL_ORDER_DEFAULT; } break; - case 748: /* null_ordering_opt ::= NULLS FIRST */ -#line 1578 "sql.y" -{ yymsp[-1].minor.yy1045 = NULL_ORDER_FIRST; } -#line 8646 "sql.c" + case 749: /* null_ordering_opt ::= NULLS FIRST */ +{ yymsp[-1].minor.yy503 = NULL_ORDER_FIRST; } break; - case 749: /* null_ordering_opt ::= NULLS LAST */ -#line 1579 "sql.y" -{ yymsp[-1].minor.yy1045 = NULL_ORDER_LAST; } -#line 8651 "sql.c" + case 750: /* null_ordering_opt ::= NULLS LAST */ +{ yymsp[-1].minor.yy503 = NULL_ORDER_LAST; } break; - case 752: /* column_options ::= column_options ENCODE NK_STRING */ -#line 1587 "sql.y" -{ yylhsminor.yy416 = setColumnOptions(pCxt, yymsp[-2].minor.yy416, COLUMN_OPTION_ENCODE, &yymsp[0].minor.yy0); } -#line 8656 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + case 753: /* column_options ::= column_options ENCODE NK_STRING */ +{ yylhsminor.yy54 = setColumnOptions(pCxt, yymsp[-2].minor.yy54, COLUMN_OPTION_ENCODE, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 753: /* column_options ::= column_options COMPRESS NK_STRING */ -#line 1588 "sql.y" -{ yylhsminor.yy416 = setColumnOptions(pCxt, yymsp[-2].minor.yy416, COLUMN_OPTION_COMPRESS, &yymsp[0].minor.yy0); } -#line 8662 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + case 754: /* column_options ::= column_options COMPRESS NK_STRING */ +{ yylhsminor.yy54 = setColumnOptions(pCxt, yymsp[-2].minor.yy54, COLUMN_OPTION_COMPRESS, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; - case 754: /* column_options ::= column_options LEVEL NK_STRING */ -#line 1589 "sql.y" -{ yylhsminor.yy416 = setColumnOptions(pCxt, yymsp[-2].minor.yy416, COLUMN_OPTION_LEVEL, &yymsp[0].minor.yy0); } -#line 8668 "sql.c" - yymsp[-2].minor.yy416 = yylhsminor.yy416; + case 755: /* column_options ::= column_options LEVEL NK_STRING */ +{ yylhsminor.yy54 = setColumnOptions(pCxt, yymsp[-2].minor.yy54, COLUMN_OPTION_LEVEL, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy54 = yylhsminor.yy54; break; default: break; @@ -8728,7 +7730,6 @@ 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) { @@ -8739,7 +7740,6 @@ 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 8742 "sql.c" /************ End %syntax_error code ******************************************/ ParseARG_STORE /* Suppress warning about unused %extra_argument variable */ ParseCTX_STORE diff --git a/source/libs/stream/inc/streamBackendRocksdb.h b/source/libs/stream/inc/streamBackendRocksdb.h index df513f43b3..c965d620f4 100644 --- a/source/libs/stream/inc/streamBackendRocksdb.h +++ b/source/libs/stream/inc/streamBackendRocksdb.h @@ -211,6 +211,8 @@ SStreamStateCur* streamStateFillSeekToLast_rocksdb(SStreamState* pState); // partag cf int32_t streamStatePutParTag_rocksdb(SStreamState* pState, int64_t groupId, const void* tag, int32_t tagLen); int32_t streamStateGetParTag_rocksdb(SStreamState* pState, int64_t groupId, void** tagVal, int32_t* tagLen); +void streamStateParTagSeekKeyNext_rocksdb(SStreamState* pState, const int64_t groupId, SStreamStateCur* pCur); +int32_t streamStateParTagGetKVByCur_rocksdb(SStreamStateCur* pCur, int64_t* pGroupId, const void** pVal, int32_t* pVLen); // parname cf int32_t streamStatePutParName_rocksdb(SStreamState* pState, int64_t groupId, const char tbname[TSDB_TABLE_NAME_LEN]); diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 7e7240d0ef..043bd2e598 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -4166,7 +4166,6 @@ _end: return res; } -#ifdef BUILD_NO_CALL // partag cf int32_t streamStatePutParTag_rocksdb(SStreamState* pState, int64_t groupId, const void* tag, int32_t tagLen) { int code = 0; @@ -4174,6 +4173,60 @@ int32_t streamStatePutParTag_rocksdb(SStreamState* pState, int64_t groupId, cons return code; } +void streamStateParTagSeekKeyNext_rocksdb(SStreamState* pState, const int64_t groupId, SStreamStateCur* pCur) { + if (pCur == NULL) { + return ; + } + STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + pCur->number = pState->number; + pCur->db = wrapper->db; + pCur->iter = streamStateIterCreate(pState, "partag", (rocksdb_snapshot_t**)&pCur->snapshot, + (rocksdb_readoptions_t**)&pCur->readOpt); + int i = streamStateGetCfIdx(pState, "partag"); + if (i < 0) { + stError("streamState failed to put to cf name:%s", "partag"); + return ; + } + + char buf[128] = {0}; + int32_t klen = ginitDict[i].enFunc((void*)&groupId, buf); + if (!streamStateIterSeekAndValid(pCur->iter, buf, klen)) { + streamStateFreeCur(pCur); + return ; + } + // skip ttl expired data + while (rocksdb_iter_valid(pCur->iter) && iterValueIsStale(pCur->iter)) { + rocksdb_iter_next(pCur->iter); + } +} + +int32_t streamStateParTagGetKVByCur_rocksdb(SStreamStateCur* pCur, int64_t* pGroupId, const void** pVal, int32_t* pVLen) { + stDebug("streamStateFillGetKVByCur_rocksdb"); + if (!pCur) { + return -1; + } + SWinKey winKey; + if (!rocksdb_iter_valid(pCur->iter) || iterValueIsStale(pCur->iter)) { + return -1; + } + + size_t klen, vlen; + char* keyStr = (char*)rocksdb_iter_key(pCur->iter, &klen); + (void)parKeyDecode(pGroupId, keyStr); + + if (pVal) { + const char* valStr = rocksdb_iter_value(pCur->iter, &vlen); + int32_t len = valueDecode((void*)valStr, vlen, NULL, (char**)pVal); + if (len < 0) { + return -1; + } + if (pVLen != NULL) *pVLen = len; + } + + return 0; +} + +#ifdef BUILD_NO_CALL int32_t streamStateGetParTag_rocksdb(SStreamState* pState, int64_t groupId, void** tagVal, int32_t* tagLen) { int code = 0; STREAM_STATE_GET_ROCKSDB(pState, "partag", &groupId, tagVal, tagLen); diff --git a/source/libs/stream/src/streamSessionState.c b/source/libs/stream/src/streamSessionState.c index 0d2303ba65..1a545176be 100644 --- a/source/libs/stream/src/streamSessionState.c +++ b/source/libs/stream/src/streamSessionState.c @@ -73,7 +73,7 @@ bool inSessionWindow(SSessionKey* pKey, TSKEY ts, int64_t gap) { return false; } -SStreamStateCur* createSessionStateCursor(SStreamFileState* pFileState) { +SStreamStateCur* createStateCursor(SStreamFileState* pFileState) { SStreamStateCur* pCur = createStreamStateCursor(); pCur->pStreamFileState = pFileState; return pCur; @@ -527,7 +527,7 @@ static SStreamStateCur* seekKeyCurrentPrev_buff(SStreamFileState* pFileState, co } if (index >= 0) { - pCur = createSessionStateCursor(pFileState); + pCur = createStateCursor(pFileState); pCur->buffIndex = index; if (pIndex) { *pIndex = index; @@ -568,7 +568,7 @@ static void checkAndTransformCursor(SStreamFileState* pFileState, const uint64_t if (taosArrayGetSize(pWinStates) > 0 && (code == TSDB_CODE_FAILED || sessionStateKeyCompare(&key, pWinStates, 0) >= 0)) { if (!(*ppCur)) { - (*ppCur) = createSessionStateCursor(pFileState); + (*ppCur) = createStateCursor(pFileState); } transformCursor(pFileState, *ppCur); } else if (*ppCur) { @@ -628,7 +628,7 @@ SStreamStateCur* countWinStateSeekKeyPrev(SStreamFileState* pFileState, const SS } pBuffCur->buffIndex = 0; } else if (taosArrayGetSize(pWinStates) > 0) { - pBuffCur = createSessionStateCursor(pFileState); + pBuffCur = createStateCursor(pFileState); pBuffCur->buffIndex = 0; } diff --git a/source/libs/stream/src/streamSliceState.c b/source/libs/stream/src/streamSliceState.c index 5dfc6e03e1..ea5e35e245 100644 --- a/source/libs/stream/src/streamSliceState.c +++ b/source/libs/stream/src/streamSliceState.c @@ -32,7 +32,7 @@ int32_t getHashSortRowBuff(SStreamFileState* pFileState, const SWinKey* pKey, vo int32_t code = TSDB_CODE_SUCCESS; int32_t lino = 0; - code = getRowBuff(pFileState, (void*)pKey, sizeof(SWinKey), pVal, pVLen, pWinCode); + code = addRowBuffIfNotExist(pFileState, (void*)pKey, sizeof(SWinKey), pVal, pVLen, pWinCode); QUERY_CHECK_CODE(code, lino, _end); SArray* pWinStates = NULL; diff --git a/source/libs/stream/src/streamState.c b/source/libs/stream/src/streamState.c index 7706a6507f..ed4c639aaa 100644 --- a/source/libs/stream/src/streamState.c +++ b/source/libs/stream/src/streamState.c @@ -128,10 +128,8 @@ SStreamState* streamStateOpen(const char* path, void* pTask, int64_t streamId, i pState->pFileState = NULL; _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT); pState->parNameMap = tSimpleHashInit(1024, hashFn); - if (!pState->parNameMap) { - code = TSDB_CODE_OUT_OF_MEMORY; - QUERY_CHECK_CODE(code, lino, _end); - } + QUERY_CHECK_NULL(pState->parNameMap, code, lino, _end, terrno); + stInfo("open state %p on backend %p 0x%" PRIx64 "-%d succ", pState, pMeta->streamBackend, pState->streamId, pState->taskId); return pState; @@ -201,14 +199,10 @@ _end: return code; } -// todo refactor -int32_t streamStatePut(SStreamState* pState, const SWinKey* key, const void* value, int32_t vLen) { - return 0; - // return streamStatePut_rocksdb(pState, key, value, vLen); -} +int32_t streamStatePut(SStreamState* pState, const SWinKey* key, const void* value, int32_t vLen) { return 0; } int32_t streamStateGet(SStreamState* pState, const SWinKey* key, void** pVal, int32_t* pVLen, int32_t* pWinCode) { - return getRowBuff(pState->pFileState, (void*)key, sizeof(SWinKey), pVal, pVLen, pWinCode); + return addRowBuffIfNotExist(pState->pFileState, (void*)key, sizeof(SWinKey), pVal, pVLen, pWinCode); } bool streamStateCheck(SStreamState* pState, const SWinKey* key) { @@ -221,33 +215,36 @@ int32_t streamStateGetByPos(SStreamState* pState, void* pos, void** pVal) { return code; } -// todo refactor void streamStateDel(SStreamState* pState, const SWinKey* key) { deleteRowBuff(pState->pFileState, key, sizeof(SWinKey)); } -// todo refactor int32_t streamStateFillPut(SStreamState* pState, const SWinKey* key, const void* value, int32_t vLen) { return streamStateFillPut_rocksdb(pState, key, value, vLen); } -// todo refactor int32_t streamStateFillGet(SStreamState* pState, const SWinKey* key, void** pVal, int32_t* pVLen, int32_t* pWinCode) { if (pState->pFileState) { - return getHashSortRowBuff(pState->pFileState, key, pVal, pVLen, pWinCode); + return getRowBuff(pState->pFileState, (void*)key, sizeof(SWinKey), pVal, pVLen, pWinCode); } return streamStateFillGet_rocksdb(pState, key, pVal, pVLen); } -int32_t streamStateFillGetNext(SStreamState* pState, const SWinKey* pKey, SWinKey* pResKey, void** pVal, int32_t* pVLen, int32_t* pWinCode) { - return getHashSortNextRow(pState->pFileState, pKey, pResKey, pVal, pVLen, pWinCode); +int32_t streamStateFillAddIfNotExist(SStreamState* pState, const SWinKey* key, void** pVal, int32_t* pVLen, + int32_t* pWinCode) { + return getHashSortRowBuff(pState->pFileState, key, pVal, pVLen, pWinCode); } -int32_t streamStateFillGetPrev(SStreamState* pState, const SWinKey* pKey, SWinKey* pResKey, void** pVal, int32_t* pVLen, int32_t* pWinCode) { +int32_t streamStateFillGetNext(SStreamState* pState, const SWinKey* pKey, SWinKey* pResKey, void** pVal, int32_t* pVLen, + int32_t* pWinCode) { + return getHashSortNextRow(pState->pFileState, pKey, pResKey, pVal, pVLen, pWinCode); +} + +int32_t streamStateFillGetPrev(SStreamState* pState, const SWinKey* pKey, SWinKey* pResKey, void** pVal, int32_t* pVLen, + int32_t* pWinCode) { return getHashSortPrevRow(pState->pFileState, pKey, pResKey, pVal, pVLen, pWinCode); } -// todo refactor void streamStateFillDel(SStreamState* pState, const SWinKey* key) { if (pState->pFileState) { deleteHashSortRowBuff(pState->pFileState, key); @@ -295,7 +292,6 @@ int32_t streamStateAddIfNotExist(SStreamState* pState, const SWinKey* key, void* } void streamStateReleaseBuf(SStreamState* pState, void* pVal, bool used) { - // todo refactor if (!pVal) { return; } @@ -458,7 +454,6 @@ int32_t streamStateSessionAddIfNotExist(SStreamState* pState, SSessionKey* key, int32_t streamStateStateAddIfNotExist(SStreamState* pState, SSessionKey* key, char* pKeyData, int32_t keyDataLen, state_key_cmpr_fn fn, void** pVal, int32_t* pVLen, int32_t* pWinCode) { - // todo refactor return getStateWinResultBuff(pState->pFileState, key, pKeyData, keyDataLen, fn, pVal, pVLen, pWinCode); } @@ -556,3 +551,25 @@ int32_t streamStateCountWinAddIfNotExist(SStreamState* pState, SSessionKey* pKey int32_t streamStateCountWinAdd(SStreamState* pState, SSessionKey* pKey, void** pVal, int32_t* pVLen) { return createCountWinResultBuff(pState->pFileState, pKey, pVal, pVLen); } + +int32_t streamStateGroupPut(SStreamState* pState, int64_t groupId, void* value, int32_t vLen) { + return streamFileStateGroupPut(pState->pFileState, groupId, value, vLen); +} + +SStreamStateCur* streamStateGroupGetCur(SStreamState* pState) { + SStreamStateCur* pCur = createStateCursor(pState->pFileState); + pCur->hashIter = 0; + pCur->pHashData = NULL; + return pCur; +} + +void streamStateGroupCurNext(SStreamStateCur* pCur) { + streamFileStateGroupCurNext(pCur); +} + +int32_t streamStateGroupGetKVByCur(SStreamStateCur* pCur, int64_t* pKey, void** pVal, int32_t* pVLen) { + if (pVal != NULL) { + return -1; + } + return streamFileStateGroupGetKVByCur(pCur, pKey, pVal, pVLen); +} diff --git a/source/libs/stream/src/tstreamFileState.c b/source/libs/stream/src/tstreamFileState.c index ca0623c47e..80998ca087 100644 --- a/source/libs/stream/src/tstreamFileState.c +++ b/source/libs/stream/src/tstreamFileState.c @@ -29,29 +29,31 @@ #define MIN_NUM_OF_RECOVER_ROW_BUFF 128 #define MIN_NUM_SEARCH_BUCKET 128 #define MAX_ARRAY_SIZE 1024 +#define MAX_GROUP_ID_NUM 200000 #define TASK_KEY "streamFileState" #define STREAM_STATE_INFO_NAME "StreamStateCheckPoint" struct SStreamFileState { - SList* usedBuffs; - SList* freeBuffs; - void* rowStateBuff; - void* pFileStore; - int32_t rowSize; - int32_t selectivityRowSize; - int32_t keyLen; - uint64_t preCheckPointVersion; - uint64_t checkPointVersion; - TSKEY maxTs; - TSKEY deleteMark; - TSKEY flushMark; - uint64_t maxRowCount; - uint64_t curRowCount; - GetTsFun getTs; - char* id; - char* cfName; - void* searchBuff; + SList* usedBuffs; + SList* freeBuffs; + void* rowStateBuff; + void* pFileStore; + int32_t rowSize; + int32_t selectivityRowSize; + int32_t keyLen; + uint64_t preCheckPointVersion; + uint64_t checkPointVersion; + TSKEY maxTs; + TSKEY deleteMark; + TSKEY flushMark; + uint64_t maxRowCount; + uint64_t curRowCount; + GetTsFun getTs; + char* id; + char* cfName; + void* searchBuff; + SSHashObj* pGroupIdMap; _state_buff_cleanup_fn stateBuffCleanupFn; _state_buff_remove_fn stateBuffRemoveFn; @@ -134,17 +136,19 @@ static void streamFileStateEncode(TSKEY* pKey, void** pVal, int32_t* pLen) { SStreamFileState* streamFileStateInit(int64_t memSize, uint32_t keySize, uint32_t rowSize, uint32_t selectRowSize, GetTsFun fp, void* pFile, TSKEY delMark, const char* taskId, int64_t checkpointId, int8_t type) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; if (memSize <= 0) { memSize = DEFAULT_MAX_STREAM_BUFFER_SIZE; } if (rowSize == 0) { - goto _error; + code = TSDB_CODE_INVALID_PARA; + QUERY_CHECK_CODE(code, lino, _end); } SStreamFileState* pFileState = taosMemoryCalloc(1, sizeof(SStreamFileState)); - if (!pFileState) { - goto _error; - } + QUERY_CHECK_NULL(pFileState, code, lino, _end, terrno); + rowSize += selectRowSize; pFileState->maxRowCount = TMAX((uint64_t)memSize / rowSize, FLUSH_NUM * 2); pFileState->usedBuffs = tdListNew(POINTER_BYTES); @@ -161,7 +165,7 @@ SStreamFileState* streamFileStateInit(int64_t memSize, uint32_t keySize, uint32_ pFileState->stateFileRemoveFn = intervalFileRemoveFn; pFileState->stateFileGetFn = intervalFileGetFn; pFileState->cfName = taosStrdup("state"); - pFileState->stateFunctionGetFn = getRowBuff; + pFileState->stateFunctionGetFn = addRowBuffIfNotExist; } else if (type == STREAM_STATE_BUFF_SORT) { pFileState->rowStateBuff = tSimpleHashInit(cap, hashFn); pFileState->stateBuffCleanupFn = sessionWinStateCleanup; @@ -176,6 +180,7 @@ SStreamFileState* streamFileStateInit(int64_t memSize, uint32_t keySize, uint32_ } else if (type == STREAM_STATE_BUFF_HASH_SORT) { pFileState->rowStateBuff = tSimpleHashInit(cap, hashFn); pFileState->searchBuff = tSimpleHashInit(MIN_NUM_SEARCH_BUCKET, hashFn); + QUERY_CHECK_NULL(pFileState->searchBuff, code, lino, _end, terrno); pFileState->stateBuffCleanupFn = stateHashBuffCleanupFn; pFileState->stateBuffRemoveFn = stateHashBuffRemoveFn; pFileState->stateBuffRemoveByPosFn = stateHashBuffRemoveByPosFn; @@ -187,9 +192,10 @@ SStreamFileState* streamFileStateInit(int64_t memSize, uint32_t keySize, uint32_ pFileState->stateFunctionGetFn = NULL; } - if (!pFileState->usedBuffs || !pFileState->freeBuffs || !pFileState->rowStateBuff) { - goto _error; - } + QUERY_CHECK_NULL(pFileState->usedBuffs, code, lino, _end, terrno); + QUERY_CHECK_NULL(pFileState->freeBuffs, code, lino, _end, terrno); + QUERY_CHECK_NULL(pFileState->rowStateBuff, code, lino, _end, terrno); + QUERY_CHECK_NULL(pFileState->cfName, code, lino, _end, terrno); pFileState->keyLen = keySize; pFileState->rowSize = rowSize; @@ -203,6 +209,10 @@ SStreamFileState* streamFileStateInit(int64_t memSize, uint32_t keySize, uint32_ pFileState->flushMark = INT64_MIN; pFileState->maxTs = INT64_MIN; pFileState->id = taosStrdup(taskId); + QUERY_CHECK_NULL(pFileState->id, code, lino, _end, terrno); + + pFileState->pGroupIdMap = tSimpleHashInit(1024, hashFn); + QUERY_CHECK_NULL(pFileState->pGroupIdMap, code, lino, _end, terrno); // todo(liuyao) optimize if (type == STREAM_STATE_BUFF_HASH) { @@ -215,18 +225,21 @@ SStreamFileState* streamFileStateInit(int64_t memSize, uint32_t keySize, uint32_ void* valBuf = NULL; int32_t len = 0; - int32_t code = streamDefaultGet_rocksdb(pFileState->pFileStore, STREAM_STATE_INFO_NAME, &valBuf, &len); - if (code == TSDB_CODE_SUCCESS) { + int32_t tmpRes = streamDefaultGet_rocksdb(pFileState->pFileStore, STREAM_STATE_INFO_NAME, &valBuf, &len); + if (tmpRes == TSDB_CODE_SUCCESS) { ASSERT(len == sizeof(TSKEY)); streamFileStateDecode(&pFileState->flushMark, valBuf, len); qDebug("===stream===flushMark read:%" PRId64, pFileState->flushMark); } taosMemoryFreeClear(valBuf); - return pFileState; -_error: - streamFileStateDestroy(pFileState); - return NULL; +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + streamFileStateDestroy(pFileState); + pFileState = NULL; + } + return pFileState; } void destroyRowBuffPos(SRowBuffPos* pPos) { @@ -332,11 +345,11 @@ int32_t clearFlushedRowBuff(SStreamFileState* pFileState, SStreamSnapshot* pFlus if (all || !pPos->beUsed) { if (all && !pPos->pRowBuff) { continue; - } + } code = tdListAppend(pFlushList, &pPos); QUERY_CHECK_CODE(code, lino, _end); - pFileState->flushMark = TMAX(pFileState->flushMark, pFileState->getTs(pPos->pKey)); + pFileState->flushMark = TMAX(pFileState->flushMark, pFileState->getTs(pPos->pKey)); pFileState->stateBuffRemoveByPosFn(pFileState, pPos); SListNode* tmp = tdListPopNode(pFileState->usedBuffs, pNode); taosMemoryFreeClear(tmp); @@ -429,7 +442,6 @@ int32_t flushRowBuff(SStreamFileState* pFileState) { if (pFileState->searchBuff) { clearFlushedRowBuff(pFileState, pFlushList, pFileState->curRowCount, true); } - flushSnapshot(pFileState, pFlushList, false); @@ -548,8 +560,8 @@ _error: return NULL; } -int32_t getRowBuff(SStreamFileState* pFileState, void* pKey, int32_t keyLen, void** pVal, int32_t* pVLen, - int32_t* pWinCode) { +int32_t addRowBuffIfNotExist(SStreamFileState* pFileState, void* pKey, int32_t keyLen, void** pVal, int32_t* pVLen, + int32_t* pWinCode) { int32_t code = TSDB_CODE_SUCCESS; int32_t lino = 0; (*pWinCode) = TSDB_CODE_SUCCESS; @@ -938,7 +950,7 @@ bool isDeteled(SStreamFileState* pFileState, TSKEY ts) { bool isFlushedState(SStreamFileState* pFileState, TSKEY ts, TSKEY gap) { return ts <= (pFileState->flushMark + gap); } -TSKEY getFlushMark(SStreamFileState* pFileState) {return pFileState->flushMark;}; +TSKEY getFlushMark(SStreamFileState* pFileState) { return pFileState->flushMark; }; int32_t getRowStateRowSize(SStreamFileState* pFileState) { return pFileState->rowSize; } @@ -970,7 +982,7 @@ int32_t recoverFillSnapshot(SStreamFileState* pFileState, int64_t ckId) { int32_t vlen = 0; SRowBuffPos* pNewPos = getNewRowPosForWrite(pFileState); code = streamStateFillGetKVByCur_rocksdb(pCur, pNewPos->pKey, (const void**)&pVal, &vlen); - if (code != TSDB_CODE_SUCCESS || isFlushedState(pFileState, pFileState->getTs(pNewPos->pKey), 0) ) { + if (code != TSDB_CODE_SUCCESS || isFlushedState(pFileState, pFileState->getTs(pNewPos->pKey), 0)) { destroyRowBuffPos(pNewPos); SListNode* pNode = tdListPopTail(pFileState->usedBuffs); taosMemoryFreeClear(pNode); @@ -992,3 +1004,99 @@ int32_t recoverFillSnapshot(SStreamFileState* pFileState, int64_t ckId) { return TSDB_CODE_SUCCESS; } + +int32_t getRowBuff(SStreamFileState* pFileState, void* pKey, int32_t keyLen, void** pVal, int32_t* pVLen, + int32_t* pWinCode) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + (*pWinCode) = TSDB_CODE_FAILED; + pFileState->maxTs = TMAX(pFileState->maxTs, pFileState->getTs(pKey)); + SRowBuffPos** pos = tSimpleHashGet(pFileState->rowStateBuff, pKey, keyLen); + if (pos) { + *pVLen = pFileState->rowSize; + *pVal = *pos; + (*pos)->beUsed = true; + (*pos)->beFlushed = false; + (*pWinCode) = TSDB_CODE_SUCCESS; + } + TSKEY ts = pFileState->getTs(pKey); + if (!isDeteled(pFileState, ts) && isFlushedState(pFileState, ts, 0)) { + int32_t len = 0; + void* p = NULL; + (*pWinCode) = pFileState->stateFileGetFn(pFileState, pKey, &p, &len); + qDebug("===stream===get %" PRId64 " from disc, res %d", ts, (*pWinCode)); + if ((*pWinCode) == TSDB_CODE_SUCCESS) { + SRowBuffPos* pNewPos = getNewRowPosForWrite(pFileState); + if (!pNewPos || !pNewPos->pRowBuff) { + code = TSDB_CODE_OUT_OF_MEMORY; + QUERY_CHECK_CODE(code, lino, _end); + } + + memcpy(pNewPos->pKey, pKey, keyLen); + memcpy(pNewPos->pRowBuff, p, len); + code = tSimpleHashPut(pFileState->rowStateBuff, pKey, keyLen, &pNewPos, POINTER_BYTES); + QUERY_CHECK_CODE(code, lino, _end); + + if (pVal) { + *pVLen = pFileState->rowSize; + *pVal = pNewPos; + } + } + taosMemoryFree(p); + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +int32_t streamFileStateGroupPut(SStreamFileState* pFileState, int64_t groupId, void* value, int32_t vLen) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + if (value != NULL) { + code = TSDB_CODE_INVALID_PARA; + QUERY_CHECK_CODE(code, lino, _end); + } + + if (tSimpleHashGet(pFileState->pGroupIdMap, &groupId, sizeof(int64_t)) == NULL) { + if (tSimpleHashGetSize(pFileState->pGroupIdMap) <= MAX_GROUP_ID_NUM) { + code = tSimpleHashPut(pFileState->pGroupIdMap, &groupId, sizeof(int64_t), NULL, 0); + QUERY_CHECK_CODE(code, lino, _end); + } + code = streamStatePutParTag_rocksdb(pFileState->pFileStore, groupId, value, vLen); + QUERY_CHECK_CODE(code, lino, _end); + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +void streamFileStateGroupCurNext(SStreamStateCur* pCur) { + SStreamFileState* pFileState = (SStreamFileState*)pCur->pStreamFileState; + if (pCur->hashIter == -1) { + streamStateCurNext(pFileState->pFileStore, pCur); + } + + SSHashObj* pHash = pFileState->pGroupIdMap; + pCur->pHashData = tSimpleHashIterate(pHash, pCur->pHashData, &pCur->hashIter); + if (!pCur->pHashData) { + pCur->hashIter = -1; + streamStateParTagSeekKeyNext_rocksdb(pFileState->pFileStore, pCur->minGpId, pCur); + } + int64_t gpId = *(int64_t*)tSimpleHashGetKey(pCur->pHashData, NULL); + pCur->minGpId = TMIN(pCur->minGpId, gpId); +} + +int32_t streamFileStateGroupGetKVByCur(SStreamStateCur* pCur, int64_t* pKey, void** pVal, int32_t* pVLen) { + int32_t code = TSDB_CODE_SUCCESS; + if (pCur->pHashData) { + *pKey = *(int64_t*)tSimpleHashGetKey(pCur->pHashData, NULL); + return code; + } + return streamStateParTagGetKVByCur_rocksdb(pCur, pKey, NULL, NULL); +} diff --git a/tests/script/tsim/stream/checkpointSession1.sim b/tests/script/tsim/stream/checkpointSession1.sim index 0427585122..5ad7ef0b96 100644 --- a/tests/script/tsim/stream/checkpointSession1.sim +++ b/tests/script/tsim/stream/checkpointSession1.sim @@ -62,6 +62,8 @@ system sh/exec.sh -n dnode1 -s start sleep 2000 +run tsim/stream/checkTaskStatus.sim + sql insert into t1 values(1648791213002,3,2,3,1.1); sql insert into t2 values(1648791233003,4,2,3,1.1);