From 8e15c64499caabc5ce87b49994654bbe53cc6da2 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 13 May 2023 23:22:36 +0800 Subject: [PATCH 01/59] refactor: do some internal refactor. --- include/libs/stream/tstream.h | 10 ++--- source/dnode/vnode/src/inc/tq.h | 2 +- source/dnode/vnode/src/tq/tqRead.c | 1 - source/dnode/vnode/src/tq/tqRestore.c | 14 +------ source/dnode/vnode/src/tq/tqUtil.c | 11 +++-- source/libs/stream/src/stream.c | 37 +++++++++-------- source/libs/stream/src/streamData.c | 36 ++++++++--------- source/libs/stream/src/streamExec.c | 8 ++-- source/libs/stream/src/streamQueue.c | 58 +++++++++++++++++++++++++++ 9 files changed, 115 insertions(+), 62 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 9d7786cfba..72309620d1 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -113,14 +113,14 @@ typedef struct { int64_t ver; int32_t* dataRef; SPackedData submit; -} SStreamDataSubmit2; +} SStreamDataSubmit; typedef struct { int8_t type; int64_t ver; SArray* dataRefs; // SArray SArray* submits; // SArray -} SStreamMergedSubmit2; +} SStreamMergedSubmit; typedef struct { int8_t type; @@ -209,10 +209,10 @@ static FORCE_INLINE void streamQueueProcessFail(SStreamQueue* queue) { void* streamQueueNextItem(SStreamQueue* queue); -SStreamDataSubmit2* streamDataSubmitNew(SPackedData submit, int32_t type); -void streamDataSubmitDestroy(SStreamDataSubmit2* pDataSubmit); +SStreamDataSubmit* streamDataSubmitNew(SPackedData* pData, int32_t type); +void streamDataSubmitDestroy(SStreamDataSubmit* pDataSubmit); -SStreamDataSubmit2* streamSubmitBlockClone(SStreamDataSubmit2* pSubmit); +SStreamDataSubmit* streamSubmitBlockClone(SStreamDataSubmit* pSubmit); typedef struct { char* qmsg; diff --git a/source/dnode/vnode/src/inc/tq.h b/source/dnode/vnode/src/inc/tq.h index cfed5f0529..2a4a471b97 100644 --- a/source/dnode/vnode/src/inc/tq.h +++ b/source/dnode/vnode/src/inc/tq.h @@ -183,7 +183,7 @@ int32_t tqStreamTasksScanWal(STQ* pTq); // tq util char* createStreamTaskIdStr(int64_t streamId, int32_t taskId); -int32_t tqAddInputBlockNLaunchTask(SStreamTask* pTask, SStreamQueueItem* pQueueItem, int64_t ver); +int32_t tqAddBlockNLaunchTask(SStreamTask* pTask, SPackedData* pData); int32_t tqExtractDataForMq(STQ* pTq, STqHandle* pHandle, const SMqPollReq* pRequest, SRpcMsg* pMsg); #ifdef __cplusplus diff --git a/source/dnode/vnode/src/tq/tqRead.c b/source/dnode/vnode/src/tq/tqRead.c index edea7724b5..bd1ccfe5dc 100644 --- a/source/dnode/vnode/src/tq/tqRead.c +++ b/source/dnode/vnode/src/tq/tqRead.c @@ -312,7 +312,6 @@ int32_t extractSubmitMsgFromWal(SWalReader* pReader, SPackedData* pPackedData) { void* data = taosMemoryMalloc(len); if (data == NULL) { - // todo: for all stream in this vnode, keep this offset in the offset files, and wait for a moment, and then retry terrno = TSDB_CODE_OUT_OF_MEMORY; tqError("vgId:%d, failed to copy submit data for stream processing, since out of memory", 0); return -1; diff --git a/source/dnode/vnode/src/tq/tqRestore.c b/source/dnode/vnode/src/tq/tqRestore.c index b956027741..8ada268c4d 100644 --- a/source/dnode/vnode/src/tq/tqRestore.c +++ b/source/dnode/vnode/src/tq/tqRestore.c @@ -120,8 +120,6 @@ int32_t createStreamRunReq(SStreamMeta* pStreamMeta, bool* pScanIdle) { continue; } - - // append the data for the stream tqDebug("vgId:%d s-task:%s wal reader seek to ver:%" PRId64, vgId, pTask->id.idStr, pTask->chkInfo.currentVer); } else { @@ -145,17 +143,9 @@ int32_t createStreamRunReq(SStreamMeta* pStreamMeta, bool* pScanIdle) { continue; } - SStreamDataSubmit2* p = streamDataSubmitNew(packData, STREAM_INPUT__DATA_SUBMIT); - if (p == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - tqError("%s failed to create data submit for stream since out of memory", pTask->id.idStr); - streamMetaReleaseTask(pStreamMeta, pTask); - continue; - } - noNewDataInWal = false; - code = tqAddInputBlockNLaunchTask(pTask, (SStreamQueueItem*)p, packData.ver); + code = tqAddBlockNLaunchTask(pTask, &packData); if (code == TSDB_CODE_SUCCESS) { pTask->chkInfo.currentVer = walReaderGetCurrentVer(pTask->exec.pWalReader); tqDebug("s-task:%s set the ver:%" PRId64 " from WALReader after extract block from WAL", pTask->id.idStr, @@ -164,8 +154,6 @@ int32_t createStreamRunReq(SStreamMeta* pStreamMeta, bool* pScanIdle) { tqError("s-task:%s append input queue failed, ver:%" PRId64, pTask->id.idStr, pTask->chkInfo.currentVer); } - streamDataSubmitDestroy(p); - taosFreeQitem(p); streamMetaReleaseTask(pStreamMeta, pTask); } diff --git a/source/dnode/vnode/src/tq/tqUtil.c b/source/dnode/vnode/src/tq/tqUtil.c index 93acd3a2b0..8f9a490048 100644 --- a/source/dnode/vnode/src/tq/tqUtil.c +++ b/source/dnode/vnode/src/tq/tqUtil.c @@ -26,10 +26,15 @@ char* createStreamTaskIdStr(int64_t streamId, int32_t taskId) { return taosStrdup(buf); } -int32_t tqAddInputBlockNLaunchTask(SStreamTask* pTask, SStreamQueueItem* pQueueItem, int64_t ver) { - int32_t code = tAppendDataToInputQueue(pTask, pQueueItem); +int32_t tqAddBlockNLaunchTask(SStreamTask* pTask, SPackedData *pPackedData) { + SStreamDataSubmit* p = streamDataSubmitNew(pPackedData, STREAM_INPUT__DATA_SUBMIT); + + int32_t code = tAppendDataToInputQueue(pTask, (SStreamQueueItem*) p); + streamDataSubmitDestroy(p); + taosFreeQitem(p); + if (code < 0) { - tqError("s-task:%s failed to put into queue, too many, next start ver:%" PRId64, pTask->id.idStr, ver); + tqError("s-task:%s failed to put into queue, too many, next start ver:%" PRId64, pTask->id.idStr, pPackedData->ver); return -1; } diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 635024519e..90f4bac242 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -18,6 +18,9 @@ #define STREAM_TASK_INPUT_QUEUEU_CAPACITY 20480 #define STREAM_TASK_INPUT_QUEUEU_CAPACITY_IN_SIZE (100) +#define ONE_MB_F (1048576.0) + +#define QUEUE_MEM_SIZE_IN_MB(_q) (taosQueueMemorySize(_q)/ONE_MB_F) int32_t streamInit() { int8_t old; @@ -288,7 +291,22 @@ int32_t tAppendDataToInputQueue(SStreamTask* pTask, SStreamQueueItem* pItem) { int8_t type = pItem->type; if (type == STREAM_INPUT__DATA_SUBMIT) { - SStreamDataSubmit2* pSubmitBlock = streamSubmitBlockClone((SStreamDataSubmit2*)pItem); + int32_t numOfBlocks = taosQueueItemSize(pTask->inputQueue->queue) + 1; + double size = QUEUE_MEM_SIZE_IN_MB(pTask->inputQueue->queue); + + SStreamDataSubmit* px = (SStreamDataSubmit*)pItem; + qDebug("s-task:%s submit enqueue msgLen:%d ver:%" PRId64 ", total in queue:%d, size:%.2fMiB", pTask->id.idStr, + px->submit.msgLen, px->submit.ver, numOfBlocks, size); + + if ((pTask->taskLevel == TASK_LEVEL__SOURCE) && + (numOfBlocks > STREAM_TASK_INPUT_QUEUEU_CAPACITY || (size >= STREAM_TASK_INPUT_QUEUEU_CAPACITY_IN_SIZE))) { + qError("s-task:%s input queue is full, capacity(size:%d num:%dMiB), current(blocks:%d, size:%.2fMiB) abort", pTask->id.idStr, + STREAM_TASK_INPUT_QUEUEU_CAPACITY, STREAM_TASK_INPUT_QUEUEU_CAPACITY_IN_SIZE, + numOfBlocks, size); + return -1; + } + + SStreamDataSubmit* pSubmitBlock = streamSubmitBlockClone((SStreamDataSubmit*)pItem); if (pSubmitBlock == NULL) { qDebug("task %d %p submit enqueue failed since out of memory", pTask->id.taskId, pTask); terrno = TSDB_CODE_OUT_OF_MEMORY; @@ -296,26 +314,11 @@ int32_t tAppendDataToInputQueue(SStreamTask* pTask, SStreamQueueItem* pItem) { return -1; } - int32_t numOfBlocks = taosQueueItemSize(pTask->inputQueue->queue) + 1; - double size = taosQueueMemorySize(pTask->inputQueue->queue) / 1048576.0; - - qDebug("s-task:%s submit enqueue msgLen:%d ver:%" PRId64 ", total in queue:%d, size:%.2fMiB", pTask->id.idStr, - pSubmitBlock->submit.msgLen, pSubmitBlock->submit.ver, numOfBlocks, size); - - if ((pTask->taskLevel == TASK_LEVEL__SOURCE) && - (numOfBlocks > STREAM_TASK_INPUT_QUEUEU_CAPACITY || (size >= STREAM_TASK_INPUT_QUEUEU_CAPACITY_IN_SIZE))) { - qError("s-task:%s input queue is full, capacity(size:%d num:%dMiB), current(blocks:%d, size:%.2fMiB) abort", pTask->id.idStr, - STREAM_TASK_INPUT_QUEUEU_CAPACITY, STREAM_TASK_INPUT_QUEUEU_CAPACITY_IN_SIZE, - numOfBlocks, size); - streamDataSubmitDestroy(pSubmitBlock); - return -1; - } - taosWriteQitem(pTask->inputQueue->queue, pSubmitBlock); } else if (type == STREAM_INPUT__DATA_BLOCK || type == STREAM_INPUT__DATA_RETRIEVE || type == STREAM_INPUT__REF_DATA_BLOCK) { int32_t numOfBlocks = taosQueueItemSize(pTask->inputQueue->queue) + 1; - double size = taosQueueMemorySize(pTask->inputQueue->queue) / 1048576.0; + double size = QUEUE_MEM_SIZE_IN_MB(pTask->inputQueue->queue); if ((pTask->taskLevel == TASK_LEVEL__SOURCE) && (numOfBlocks > STREAM_TASK_INPUT_QUEUEU_CAPACITY || (size >= STREAM_TASK_INPUT_QUEUEU_CAPACITY_IN_SIZE))) { diff --git a/source/libs/stream/src/streamData.c b/source/libs/stream/src/streamData.c index e574cdbe8a..67177268d3 100644 --- a/source/libs/stream/src/streamData.c +++ b/source/libs/stream/src/streamData.c @@ -67,8 +67,8 @@ int32_t streamRetrieveReqToData(const SStreamRetrieveReq* pReq, SStreamDataBlock return 0; } -SStreamDataSubmit2* streamDataSubmitNew(SPackedData submit, int32_t type) { - SStreamDataSubmit2* pDataSubmit = (SStreamDataSubmit2*)taosAllocateQitem(sizeof(SStreamDataSubmit2), DEF_QITEM, submit.msgLen); +SStreamDataSubmit* streamDataSubmitNew(SPackedData* pData, int32_t type) { + SStreamDataSubmit* pDataSubmit = (SStreamDataSubmit*)taosAllocateQitem(sizeof(SStreamDataSubmit), DEF_QITEM, pData->msgLen); if (pDataSubmit == NULL) { return NULL; } @@ -79,14 +79,14 @@ SStreamDataSubmit2* streamDataSubmitNew(SPackedData submit, int32_t type) { return NULL; } - pDataSubmit->submit = submit; + pDataSubmit->submit = *pData; *pDataSubmit->dataRef = 1; // initialize the reference count to be 1 pDataSubmit->type = type; return pDataSubmit; } -void streamDataSubmitDestroy(SStreamDataSubmit2* pDataSubmit) { +void streamDataSubmitDestroy(SStreamDataSubmit* pDataSubmit) { int32_t ref = atomic_sub_fetch_32(pDataSubmit->dataRef, 1); ASSERT(ref >= 0 && pDataSubmit->type == STREAM_INPUT__DATA_SUBMIT); @@ -96,8 +96,8 @@ void streamDataSubmitDestroy(SStreamDataSubmit2* pDataSubmit) { } } -SStreamMergedSubmit2* streamMergedSubmitNew() { - SStreamMergedSubmit2* pMerged = (SStreamMergedSubmit2*)taosAllocateQitem(sizeof(SStreamMergedSubmit2), DEF_QITEM, 0); +SStreamMergedSubmit* streamMergedSubmitNew() { + SStreamMergedSubmit* pMerged = (SStreamMergedSubmit*)taosAllocateQitem(sizeof(SStreamMergedSubmit), DEF_QITEM, 0); if (pMerged == NULL) { return NULL; } @@ -116,30 +116,30 @@ SStreamMergedSubmit2* streamMergedSubmitNew() { return pMerged; } -int32_t streamMergeSubmit(SStreamMergedSubmit2* pMerged, SStreamDataSubmit2* pSubmit) { +int32_t streamMergeSubmit(SStreamMergedSubmit* pMerged, SStreamDataSubmit* pSubmit) { taosArrayPush(pMerged->dataRefs, &pSubmit->dataRef); taosArrayPush(pMerged->submits, &pSubmit->submit); pMerged->ver = pSubmit->ver; return 0; } -static FORCE_INLINE void streamDataSubmitRefInc(SStreamDataSubmit2* pDataSubmit) { +static FORCE_INLINE void streamDataSubmitRefInc(SStreamDataSubmit* pDataSubmit) { atomic_add_fetch_32(pDataSubmit->dataRef, 1); } -SStreamDataSubmit2* streamSubmitBlockClone(SStreamDataSubmit2* pSubmit) { +SStreamDataSubmit* streamSubmitBlockClone(SStreamDataSubmit* pSubmit) { int32_t len = 0; if (pSubmit->type == STREAM_INPUT__DATA_SUBMIT) { len = pSubmit->submit.msgLen; } - SStreamDataSubmit2* pSubmitClone = taosAllocateQitem(sizeof(SStreamDataSubmit2), DEF_QITEM, len); + SStreamDataSubmit* pSubmitClone = taosAllocateQitem(sizeof(SStreamDataSubmit), DEF_QITEM, len); if (pSubmitClone == NULL) { return NULL; } streamDataSubmitRefInc(pSubmit); - memcpy(pSubmitClone, pSubmit, sizeof(SStreamDataSubmit2)); + memcpy(pSubmitClone, pSubmit, sizeof(SStreamDataSubmit)); return pSubmitClone; } @@ -152,17 +152,17 @@ SStreamQueueItem* streamMergeQueueItem(SStreamQueueItem* dst, SStreamQueueItem* taosFreeQitem(pElem); return dst; } else if (dst->type == STREAM_INPUT__MERGED_SUBMIT && pElem->type == STREAM_INPUT__DATA_SUBMIT) { - SStreamMergedSubmit2* pMerged = (SStreamMergedSubmit2*)dst; - SStreamDataSubmit2* pBlockSrc = (SStreamDataSubmit2*)pElem; + SStreamMergedSubmit* pMerged = (SStreamMergedSubmit*)dst; + SStreamDataSubmit* pBlockSrc = (SStreamDataSubmit*)pElem; streamMergeSubmit(pMerged, pBlockSrc); taosFreeQitem(pElem); return dst; } else if (dst->type == STREAM_INPUT__DATA_SUBMIT && pElem->type == STREAM_INPUT__DATA_SUBMIT) { - SStreamMergedSubmit2* pMerged = streamMergedSubmitNew(); + SStreamMergedSubmit* pMerged = streamMergedSubmitNew(); // todo handle error - streamMergeSubmit(pMerged, (SStreamDataSubmit2*)dst); - streamMergeSubmit(pMerged, (SStreamDataSubmit2*)pElem); + streamMergeSubmit(pMerged, (SStreamDataSubmit*)dst); + streamMergeSubmit(pMerged, (SStreamDataSubmit*)pElem); taosFreeQitem(dst); taosFreeQitem(pElem); return (SStreamQueueItem*)pMerged; @@ -180,10 +180,10 @@ void streamFreeQitem(SStreamQueueItem* data) { taosArrayDestroyEx(((SStreamDataBlock*)data)->blocks, (FDelete)blockDataFreeRes); taosFreeQitem(data); } else if (type == STREAM_INPUT__DATA_SUBMIT) { - streamDataSubmitDestroy((SStreamDataSubmit2*)data); + streamDataSubmitDestroy((SStreamDataSubmit*)data); taosFreeQitem(data); } else if (type == STREAM_INPUT__MERGED_SUBMIT) { - SStreamMergedSubmit2* pMerge = (SStreamMergedSubmit2*)data; + SStreamMergedSubmit* pMerge = (SStreamMergedSubmit*)data; int32_t sz = taosArrayGetSize(pMerge->submits); for (int32_t i = 0; i < sz; i++) { int32_t* pRef = taosArrayGetP(pMerge->dataRefs, i); diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 0fb78fb589..c2e32f1027 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -51,7 +51,7 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, const void* data, SArray* qSetMultiStreamInput(pExecutor, pTrigger->pBlock, 1, STREAM_INPUT__DATA_BLOCK); } else if (pItem->type == STREAM_INPUT__DATA_SUBMIT) { ASSERT(pTask->taskLevel == TASK_LEVEL__SOURCE); - const SStreamDataSubmit2* pSubmit = (const SStreamDataSubmit2*)data; + const SStreamDataSubmit* pSubmit = (const SStreamDataSubmit*)data; qSetMultiStreamInput(pExecutor, &pSubmit->submit, 1, STREAM_INPUT__DATA_SUBMIT); qDebug("s-task:%s set submit blocks as source block completed, %p %p len:%d ver:%" PRId64, pTask->id.idStr, pSubmit, pSubmit->submit.msgStr, pSubmit->submit.msgLen, pSubmit->submit.ver); @@ -63,7 +63,7 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, const void* data, SArray* qDebug("s-task:%s set sdata blocks as input num:%d, ver:%"PRId64, pTask->id.idStr, numOfBlocks, pBlock->sourceVer); qSetMultiStreamInput(pExecutor, pBlockList->pData, numOfBlocks, STREAM_INPUT__DATA_BLOCK); } else if (pItem->type == STREAM_INPUT__MERGED_SUBMIT) { - const SStreamMergedSubmit2* pMerged = (const SStreamMergedSubmit2*)data; + const SStreamMergedSubmit* pMerged = (const SStreamMergedSubmit*)data; SArray* pBlockList = pMerged->submits; int32_t numOfBlocks = taosArrayGetSize(pBlockList); @@ -366,11 +366,11 @@ int32_t streamExecForAll(SStreamTask* pTask) { qRes->blocks = pRes; if (((SStreamQueueItem*)pInput)->type == STREAM_INPUT__DATA_SUBMIT) { - SStreamDataSubmit2* pSubmit = (SStreamDataSubmit2*)pInput; + SStreamDataSubmit* pSubmit = (SStreamDataSubmit*)pInput; qRes->childId = pTask->selfChildId; qRes->sourceVer = pSubmit->ver; } else if (((SStreamQueueItem*)pInput)->type == STREAM_INPUT__MERGED_SUBMIT) { - SStreamMergedSubmit2* pMerged = (SStreamMergedSubmit2*)pInput; + SStreamMergedSubmit* pMerged = (SStreamMergedSubmit*)pInput; qRes->childId = pTask->selfChildId; qRes->sourceVer = pMerged->ver; } diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 20abcca197..52d6525769 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -105,3 +105,61 @@ SStreamQueueRes streamQueueGetRes(SStreamQueue1* pQueue) { return (SStreamQueueRes){0}; } #endif + +#define MAX_STREAM_EXEC_BATCH_NUM 128 +#define MIN_STREAM_EXEC_BATCH_NUM 16 + +// todo refactor: +// read data from input queue +typedef struct SQueueReader { + SStreamQueue* pQueue; + int32_t taskLevel; + int32_t maxBlocks; // maximum block in one batch + int32_t waitDuration; // maximum wait time to format several block into a batch to process, unit: ms +} SQueueReader; + +SStreamQueueItem* doReadMultiBlocksFromQueue(SQueueReader* pReader, const char* idstr) { + int32_t numOfBlocks = 0; + int32_t tryCount = 0; + SStreamQueueItem* pRet = NULL; + + while (1) { + SStreamQueueItem* qItem = streamQueueNextItem(pReader->pQueue); + if (qItem == NULL) { + if (pReader->taskLevel == TASK_LEVEL__SOURCE && numOfBlocks < MIN_STREAM_EXEC_BATCH_NUM && tryCount < pReader->waitDuration) { + tryCount++; + taosMsleep(1); + qDebug("===stream===try again batchSize:%d", numOfBlocks); + continue; + } + + qDebug("===stream===break batchSize:%d", numOfBlocks); + break; + } + + if (pRet == NULL) { + pRet = qItem; + streamQueueProcessSuccess(pReader->pQueue); + if (pReader->taskLevel == TASK_LEVEL__SINK) { + break; + } + } else { + // todo we need to sort the data block, instead of just appending into the array list. + void* newRet = NULL; + if ((newRet = streamMergeQueueItem(pRet, qItem)) == NULL) { + streamQueueProcessFail(pReader->pQueue); + break; + } else { + numOfBlocks++; + pRet = newRet; + streamQueueProcessSuccess(pReader->pQueue); + if (numOfBlocks > pReader->maxBlocks) { + qDebug("maximum blocks limit:%d reached, processing, %s", pReader->maxBlocks, idstr); + break; + } + } + } + } + + return pRet; +} From 3d00f6ce547f00a7a857b72cec7af5c6241a6aef Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 14 May 2023 22:08:17 +0800 Subject: [PATCH 02/59] refactor: do some internal refactor. --- source/dnode/vnode/src/tq/tqSink.c | 39 +++++++++++++++--------------- source/libs/stream/src/stream.c | 10 ++++---- 2 files changed, 25 insertions(+), 24 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqSink.c b/source/dnode/vnode/src/tq/tqSink.c index 4a9e3dcee7..e2f9089730 100644 --- a/source/dnode/vnode/src/tq/tqSink.c +++ b/source/dnode/vnode/src/tq/tqSink.c @@ -19,10 +19,10 @@ #define MAX_CATCH_NUM 10240 -typedef struct STblInfo { +typedef struct STableSinkInfo { uint64_t uid; char tbName[TSDB_TABLE_NAME_LEN]; -} STblInfo; +} STableSinkInfo; int32_t tqBuildDeleteReq(const char* stbFullName, const SSDataBlock* pDataBlock, SBatchDeleteReq* deleteReq, const char* pIdStr) { @@ -97,16 +97,17 @@ end: return ret; } -int32_t tqGetTableInfo(SSHashObj* tblInfo ,uint64_t groupId, STblInfo** pTbl) { - void* pVal = tSimpleHashGet(tblInfo, &groupId, sizeof(uint64_t)); +static int32_t tqGetTableInfo(SSHashObj* pTableInfoMap,uint64_t groupId, STableSinkInfo** pInfo) { + void* pVal = tSimpleHashGet(pTableInfoMap, &groupId, sizeof(uint64_t)); if (pVal) { - *pTbl = *(STblInfo**)pVal; + *pInfo = *(STableSinkInfo**)pVal; return TSDB_CODE_SUCCESS; } + return TSDB_CODE_FAILED; } -int32_t tqPutTableInfo(SSHashObj* tblInfo ,uint64_t groupId, STblInfo* pTbl) { +int32_t tqPutTableInfo(SSHashObj* tblInfo ,uint64_t groupId, STableSinkInfo* pTbl) { if (tSimpleHashGetSize(tblInfo) > MAX_CATCH_NUM) { return TSDB_CODE_SUCCESS; } @@ -274,7 +275,7 @@ void tqSinkToTablePipeline(SStreamTask* pTask, void* vnode, int64_t ver, void* d crTblArray = NULL; } else { SSubmitTbData tbData = {0}; - tqDebug("tq sink pipe, convert block1 %d, rows: %d", i, rows); + tqDebug("tq sink pipe, convert block:%d, rows:%d", i, rows); if (!(tbData.aRowP = taosArrayInit(rows, sizeof(SRow*)))) { goto _end; @@ -283,35 +284,35 @@ void tqSinkToTablePipeline(SStreamTask* pTask, void* vnode, int64_t ver, void* d tbData.suid = suid; tbData.uid = 0; // uid is assigned by vnode tbData.sver = pTSchema->version; - STblInfo* pTblMeta = NULL; - int32_t res = tqGetTableInfo(pTask->tbSink.pTblInfo, pDataBlock->info.id.groupId, &pTblMeta); + STableSinkInfo* pTableSinkInfo = NULL; + int32_t res = tqGetTableInfo(pTask->tbSink.pTblInfo, pDataBlock->info.id.groupId, &pTableSinkInfo); if (res != TSDB_CODE_SUCCESS) { - pTblMeta = taosMemoryCalloc(1, sizeof(STblInfo)); + pTableSinkInfo = taosMemoryCalloc(1, sizeof(STableSinkInfo)); } char* ctbName = pDataBlock->info.parTbName; if (!ctbName[0]) { if (res == TSDB_CODE_SUCCESS) { - memcpy(ctbName, pTblMeta->tbName, strlen(pTblMeta->tbName)); + memcpy(ctbName, pTableSinkInfo->tbName, strlen(pTableSinkInfo->tbName)); } else { char* tmp = buildCtbNameByGroupId(stbFullName, pDataBlock->info.id.groupId); memcpy(ctbName, tmp, strlen(tmp)); - memcpy(pTblMeta->tbName, tmp, strlen(tmp)); + memcpy(pTableSinkInfo->tbName, tmp, strlen(tmp)); taosMemoryFree(tmp); - tqDebug("vgId:%d, gropuid:%" PRIu64 " datablock tabel name is null", TD_VID(pVnode), + tqDebug("vgId:%d, gropuId:%" PRIu64 " datablock table name is null", TD_VID(pVnode), pDataBlock->info.id.groupId); } } if (res == TSDB_CODE_SUCCESS) { - tbData.uid = pTblMeta->uid; + tbData.uid = pTableSinkInfo->uid; } else { SMetaReader mr = {0}; metaReaderInit(&mr, pVnode->pMeta, 0); if (metaGetTableEntryByName(&mr, ctbName) < 0) { metaReaderClear(&mr); - taosMemoryFree(pTblMeta); + taosMemoryFree(pTableSinkInfo); tqDebug("vgId:%d, stream write into %s, table auto created", TD_VID(pVnode), ctbName); SVCreateTbReq* pCreateTbReq = NULL; @@ -371,7 +372,7 @@ void tqSinkToTablePipeline(SStreamTask* pTask, void* vnode, int64_t ver, void* d tqError("vgId:%d, failed to write into %s, since table type incorrect, type %d", TD_VID(pVnode), ctbName, mr.me.type); metaReaderClear(&mr); - taosMemoryFree(pTblMeta); + taosMemoryFree(pTableSinkInfo); continue; } @@ -380,13 +381,13 @@ void tqSinkToTablePipeline(SStreamTask* pTask, void* vnode, int64_t ver, void* d ", actual suid %" PRId64 "", TD_VID(pVnode), ctbName, suid, mr.me.ctbEntry.suid); metaReaderClear(&mr); - taosMemoryFree(pTblMeta); + taosMemoryFree(pTableSinkInfo); continue; } tbData.uid = mr.me.uid; - pTblMeta->uid = mr.me.uid; - tqPutTableInfo(pTask->tbSink.pTblInfo, pDataBlock->info.id.groupId, pTblMeta); + pTableSinkInfo->uid = mr.me.uid; + tqPutTableInfo(pTask->tbSink.pTblInfo, pDataBlock->info.id.groupId, pTableSinkInfo); metaReaderClear(&mr); } } diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 90f4bac242..6047f74ab2 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -284,7 +284,9 @@ int32_t streamProcessRetrieveReq(SStreamTask* pTask, SStreamRetrieveReq* pReq, S } bool tInputQueueIsFull(const SStreamTask* pTask) { - return taosQueueItemSize((pTask->inputQueue->queue)) >= STREAM_TASK_INPUT_QUEUEU_CAPACITY; + bool isFull = taosQueueItemSize((pTask->inputQueue->queue)) >= STREAM_TASK_INPUT_QUEUEU_CAPACITY; + double size = QUEUE_MEM_SIZE_IN_MB(pTask->inputQueue->queue); + return (isFull || size >= STREAM_TASK_INPUT_QUEUEU_CAPACITY_IN_SIZE); } int32_t tAppendDataToInputQueue(SStreamTask* pTask, SStreamQueueItem* pItem) { @@ -298,8 +300,7 @@ int32_t tAppendDataToInputQueue(SStreamTask* pTask, SStreamQueueItem* pItem) { qDebug("s-task:%s submit enqueue msgLen:%d ver:%" PRId64 ", total in queue:%d, size:%.2fMiB", pTask->id.idStr, px->submit.msgLen, px->submit.ver, numOfBlocks, size); - if ((pTask->taskLevel == TASK_LEVEL__SOURCE) && - (numOfBlocks > STREAM_TASK_INPUT_QUEUEU_CAPACITY || (size >= STREAM_TASK_INPUT_QUEUEU_CAPACITY_IN_SIZE))) { + if ((pTask->taskLevel == TASK_LEVEL__SOURCE) && tInputQueueIsFull(pTask)) { qError("s-task:%s input queue is full, capacity(size:%d num:%dMiB), current(blocks:%d, size:%.2fMiB) abort", pTask->id.idStr, STREAM_TASK_INPUT_QUEUEU_CAPACITY, STREAM_TASK_INPUT_QUEUEU_CAPACITY_IN_SIZE, numOfBlocks, size); @@ -320,8 +321,7 @@ int32_t tAppendDataToInputQueue(SStreamTask* pTask, SStreamQueueItem* pItem) { int32_t numOfBlocks = taosQueueItemSize(pTask->inputQueue->queue) + 1; double size = QUEUE_MEM_SIZE_IN_MB(pTask->inputQueue->queue); - if ((pTask->taskLevel == TASK_LEVEL__SOURCE) && - (numOfBlocks > STREAM_TASK_INPUT_QUEUEU_CAPACITY || (size >= STREAM_TASK_INPUT_QUEUEU_CAPACITY_IN_SIZE))) { + if ((pTask->taskLevel == TASK_LEVEL__SOURCE) && (tInputQueueIsFull(pTask))) { qError("s-task:%s input queue is full, capacity:%d size:%d MiB, current(blocks:%d, size:%.2fMiB) abort", pTask->id.idStr, STREAM_TASK_INPUT_QUEUEU_CAPACITY, STREAM_TASK_INPUT_QUEUEU_CAPACITY_IN_SIZE, numOfBlocks, size); From ac287572efdc2f54a5cb33aba6fa5fca64629e84 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 15 May 2023 00:03:53 +0800 Subject: [PATCH 03/59] fix: free table sink info --- source/dnode/vnode/src/tq/tqSink.c | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqSink.c b/source/dnode/vnode/src/tq/tqSink.c index e2f9089730..0bd7d9a57b 100644 --- a/source/dnode/vnode/src/tq/tqSink.c +++ b/source/dnode/vnode/src/tq/tqSink.c @@ -17,7 +17,7 @@ #include "tmsg.h" #include "tq.h" -#define MAX_CATCH_NUM 10240 +#define MAX_CACHE_TABLE_INFO_NUM 10240 typedef struct STableSinkInfo { uint64_t uid; @@ -108,9 +108,10 @@ static int32_t tqGetTableInfo(SSHashObj* pTableInfoMap,uint64_t groupId, STableS } int32_t tqPutTableInfo(SSHashObj* tblInfo ,uint64_t groupId, STableSinkInfo* pTbl) { - if (tSimpleHashGetSize(tblInfo) > MAX_CATCH_NUM) { - return TSDB_CODE_SUCCESS; + if (tSimpleHashGetSize(tblInfo) > MAX_CACHE_TABLE_INFO_NUM) { + return TSDB_CODE_FAILED; } + return tSimpleHashPut(tblInfo, &groupId, sizeof(uint64_t), &pTbl, POINTER_BYTES); } @@ -387,7 +388,10 @@ void tqSinkToTablePipeline(SStreamTask* pTask, void* vnode, int64_t ver, void* d tbData.uid = mr.me.uid; pTableSinkInfo->uid = mr.me.uid; - tqPutTableInfo(pTask->tbSink.pTblInfo, pDataBlock->info.id.groupId, pTableSinkInfo); + int32_t code = tqPutTableInfo(pTask->tbSink.pTblInfo, pDataBlock->info.id.groupId, pTableSinkInfo); + if (code != TSDB_CODE_SUCCESS) { + taosMemoryFreeClear(pTableSinkInfo); + } metaReaderClear(&mr); } } From 3b6e052c6b1d895113eed54e064cb99479703248 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 15 May 2023 10:11:08 +0800 Subject: [PATCH 04/59] enh(stream): avoid clone submit block. --- source/libs/stream/src/stream.c | 14 +------------- 1 file changed, 1 insertion(+), 13 deletions(-) diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 6047f74ab2..a0a7d5caf7 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -307,15 +307,7 @@ int32_t tAppendDataToInputQueue(SStreamTask* pTask, SStreamQueueItem* pItem) { return -1; } - SStreamDataSubmit* pSubmitBlock = streamSubmitBlockClone((SStreamDataSubmit*)pItem); - if (pSubmitBlock == NULL) { - qDebug("task %d %p submit enqueue failed since out of memory", pTask->id.taskId, pTask); - terrno = TSDB_CODE_OUT_OF_MEMORY; - atomic_store_8(&pTask->inputStatus, TASK_INPUT_STATUS__FAILED); - return -1; - } - - taosWriteQitem(pTask->inputQueue->queue, pSubmitBlock); + taosWriteQitem(pTask->inputQueue->queue, pItem); } else if (type == STREAM_INPUT__DATA_BLOCK || type == STREAM_INPUT__DATA_RETRIEVE || type == STREAM_INPUT__REF_DATA_BLOCK) { int32_t numOfBlocks = taosQueueItemSize(pTask->inputQueue->queue) + 1; @@ -340,10 +332,6 @@ int32_t tAppendDataToInputQueue(SStreamTask* pTask, SStreamQueueItem* pItem) { atomic_val_compare_exchange_8(&pTask->triggerStatus, TASK_TRIGGER_STATUS__INACTIVE, TASK_TRIGGER_STATUS__ACTIVE); } -#if 0 - atomic_store_8(&pTask->inputStatus, TASK_INPUT_STATUS__NORMAL); -#endif - return 0; } From 9258dcd7404e81748758b3986ab1ebbe4250d129 Mon Sep 17 00:00:00 2001 From: liuyao <54liuyao@163.com> Date: Mon, 15 May 2023 11:19:17 +0800 Subject: [PATCH 05/59] fix:mem leak --- source/libs/executor/inc/executorInt.h | 1 - source/libs/executor/src/groupoperator.c | 6 ++++++ source/libs/stream/src/streamExec.c | 3 ++- 3 files changed, 8 insertions(+), 2 deletions(-) diff --git a/source/libs/executor/inc/executorInt.h b/source/libs/executor/inc/executorInt.h index c9d0eebe2b..d561d86472 100644 --- a/source/libs/executor/inc/executorInt.h +++ b/source/libs/executor/inc/executorInt.h @@ -297,7 +297,6 @@ typedef struct SPartitionBySupporter { typedef struct SPartitionDataInfo { uint64_t groupId; char* tbname; - SArray* tags; SArray* rowIds; } SPartitionDataInfo; diff --git a/source/libs/executor/src/groupoperator.c b/source/libs/executor/src/groupoperator.c index 280edf8b53..e2ba2a07c8 100644 --- a/source/libs/executor/src/groupoperator.c +++ b/source/libs/executor/src/groupoperator.c @@ -1215,6 +1215,11 @@ SSDataBlock* buildCreateTableBlock(SExprSupp* tbName, SExprSupp* tag) { return pBlock; } +void freePartItem(void* ptr) { + SPartitionDataInfo* pPart = (SPartitionDataInfo*)ptr; + taosArrayDestroy(pPart->rowIds); +} + SOperatorInfo* createStreamPartitionOperatorInfo(SOperatorInfo* downstream, SStreamPartitionPhysiNode* pPartNode, SExecTaskInfo* pTaskInfo) { int32_t code = TSDB_CODE_SUCCESS; @@ -1293,6 +1298,7 @@ SOperatorInfo* createStreamPartitionOperatorInfo(SOperatorInfo* downstream, SStr _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); pInfo->pPartitions = taosHashInit(1024, hashFn, false, HASH_NO_LOCK); + taosHashSetFreeFp(pInfo->pPartitions, freePartItem); pInfo->tsColIndex = 0; pInfo->pDelRes = createSpecialDataBlock(STREAM_DELETE_RESULT); diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 0fb78fb589..918a203df4 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -149,7 +149,7 @@ int32_t streamScanExec(SStreamTask* pTask, int32_t batchSz) { int32_t batchCnt = 0; while (1) { if (streamTaskShouldStop(&pTask->status) || streamTaskShouldPause(&pTask->status)) { - taosArrayDestroy(pRes); + taosArrayDestroyEx(pRes, (FDelete)blockDataFreeRes); return 0; } @@ -203,6 +203,7 @@ int32_t streamScanExec(SStreamTask* pTask, int32_t batchSz) { code = streamTaskOutput(pTask, qRes); if (code == TSDB_CODE_UTIL_QUEUE_OUT_OF_MEMORY) { taosFreeQitem(pRes); + taosArrayDestroyEx(pRes, (FDelete)blockDataFreeRes); return code; } From 72c543563fd859806e0a274315c972335548a94d Mon Sep 17 00:00:00 2001 From: cadem Date: Mon, 15 May 2023 11:20:27 +0800 Subject: [PATCH 06/59] force drop dnode --- source/dnode/mnode/impl/inc/mndStb.h | 1 + source/dnode/mnode/impl/src/mndStb.c | 31 +++++++++++++++++++++++++ source/dnode/mnode/impl/src/mndVgroup.c | 22 +++++++++++++++++- 3 files changed, 53 insertions(+), 1 deletion(-) diff --git a/source/dnode/mnode/impl/inc/mndStb.h b/source/dnode/mnode/impl/inc/mndStb.h index 66d0ed1d12..99af413539 100644 --- a/source/dnode/mnode/impl/inc/mndStb.h +++ b/source/dnode/mnode/impl/inc/mndStb.h @@ -47,6 +47,7 @@ int32_t mndAllocStbSchemas(const SStbObj *pOld, SStbObj *pNew); int32_t mndCheckColAndTagModifiable(SMnode *pMnode, const char *stbFullName, int64_t suid, col_id_t colId); void *mndBuildVCreateStbReq(SMnode *pMnode, SVgObj *pVgroup, SStbObj *pStb, int32_t *pContLen, void *alterOriData, int32_t alterOriDataLen); +int32_t mndSetForceDropCreateStbRedoActions(SMnode *pMnode, STrans *pTrans, SVgObj *pVgroup, SStbObj *pStb); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/src/mndStb.c b/source/dnode/mnode/impl/src/mndStb.c index 8b708c3e0f..ef72f66a7a 100644 --- a/source/dnode/mnode/impl/src/mndStb.c +++ b/source/dnode/mnode/impl/src/mndStb.c @@ -687,6 +687,37 @@ static int32_t mndSetCreateStbRedoActions(SMnode *pMnode, STrans *pTrans, SDbObj return 0; } +int32_t mndSetForceDropCreateStbRedoActions(SMnode *pMnode, STrans *pTrans, SVgObj *pVgroup, SStbObj *pStb) { + SSdb *pSdb = pMnode->pSdb; + void *pIter = NULL; + int32_t contLen; + + void *pReq = mndBuildVCreateStbReq(pMnode, pVgroup, pStb, &contLen, NULL, 0); + if (pReq == NULL) { + sdbCancelFetch(pSdb, pIter); + sdbRelease(pSdb, pVgroup); + return -1; + } + + STransAction action = {0}; + action.mTraceId = pTrans->mTraceId; + action.epSet = mndGetVgroupEpset(pMnode, pVgroup); + action.pCont = pReq; + action.contLen = contLen; + action.msgType = TDMT_VND_CREATE_STB; + action.acceptableCode = TSDB_CODE_TDB_STB_ALREADY_EXIST; + action.retryCode = TSDB_CODE_TDB_STB_NOT_EXIST; + if (mndTransAppendRedoAction(pTrans, &action) != 0) { + taosMemoryFree(pReq); + sdbCancelFetch(pSdb, pIter); + sdbRelease(pSdb, pVgroup); + return -1; + } + sdbRelease(pSdb, pVgroup); + + return 0; +} + static int32_t mndSetCreateStbUndoActions(SMnode *pMnode, STrans *pTrans, SDbObj *pDb, SStbObj *pStb) { SSdb *pSdb = pMnode->pSdb; SVgObj *pVgroup = NULL; diff --git a/source/dnode/mnode/impl/src/mndVgroup.c b/source/dnode/mnode/impl/src/mndVgroup.c index a8e9db28e9..5cf286354c 100644 --- a/source/dnode/mnode/impl/src/mndVgroup.c +++ b/source/dnode/mnode/impl/src/mndVgroup.c @@ -23,6 +23,7 @@ #include "mndTrans.h" #include "mndUser.h" #include "tmisce.h" +#include "mndStb.h" #define VGROUP_VER_NUMBER 1 #define VGROUP_RESERVE_SIZE 64 @@ -1476,7 +1477,26 @@ int32_t mndSetMoveVgroupInfoToTrans(SMnode *pMnode, STrans *pTrans, SDbObj *pDb, if (mndAddCreateVnodeAction(pMnode, pTrans, pDb, &newVg, &newVg.vnodeGid[vnIndex]) != 0) return -1; if (mndAddAlterVnodeConfirmAction(pMnode, pTrans, pDb, &newVg) != 0) return -1; - if (newVg.replica == 1) { + if(newVg.replica == 1){ + SSdb *pSdb = pMnode->pSdb; + void *pIter = NULL; + + while (1) { + SStbObj *pStb = NULL; + pIter = sdbFetch(pSdb, SDB_STB, pIter, (void **)&pStb); + if (pIter == NULL) break; + + if (strcmp(pStb->db, pDb->name) == 0) { + if (mndSetForceDropCreateStbRedoActions(pMnode, pTrans, &newVg, pStb) != 0) { + sdbCancelFetch(pSdb, pIter); + sdbRelease(pSdb, pStb); + return -1; + } + } + + sdbRelease(pSdb, pStb); + } + mInfo("vgId:%d, all data is dropped since replica=1", pVgroup->vgId); } } From 17592ea2947bd2983781da16485ee25a81ef9d33 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 15 May 2023 13:49:26 +0800 Subject: [PATCH 07/59] refactor: do some internal refactor. --- source/dnode/vnode/src/tq/tqUtil.c | 4 ---- 1 file changed, 4 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqUtil.c b/source/dnode/vnode/src/tq/tqUtil.c index 8f9a490048..e78820bad8 100644 --- a/source/dnode/vnode/src/tq/tqUtil.c +++ b/source/dnode/vnode/src/tq/tqUtil.c @@ -28,11 +28,7 @@ char* createStreamTaskIdStr(int64_t streamId, int32_t taskId) { int32_t tqAddBlockNLaunchTask(SStreamTask* pTask, SPackedData *pPackedData) { SStreamDataSubmit* p = streamDataSubmitNew(pPackedData, STREAM_INPUT__DATA_SUBMIT); - int32_t code = tAppendDataToInputQueue(pTask, (SStreamQueueItem*) p); - streamDataSubmitDestroy(p); - taosFreeQitem(p); - if (code < 0) { tqError("s-task:%s failed to put into queue, too many, next start ver:%" PRId64, pTask->id.idStr, pPackedData->ver); return -1; From 4c64260a7afd78a7cc91ee0aa5b7eed0d1d8b6d0 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 15 May 2023 14:15:10 +0800 Subject: [PATCH 08/59] refactor: do internal refactor. --- source/libs/stream/src/stream.c | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index a0a7d5caf7..1c75519647 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -16,11 +16,10 @@ #include "streamInc.h" #include "ttimer.h" -#define STREAM_TASK_INPUT_QUEUEU_CAPACITY 20480 -#define STREAM_TASK_INPUT_QUEUEU_CAPACITY_IN_SIZE (100) -#define ONE_MB_F (1048576.0) - -#define QUEUE_MEM_SIZE_IN_MB(_q) (taosQueueMemorySize(_q)/ONE_MB_F) +#define STREAM_TASK_INPUT_QUEUEU_CAPACITY 20480 +#define STREAM_TASK_INPUT_QUEUEU_CAPACITY_IN_SIZE (50) +#define ONE_MB_F (1048576.0) +#define QUEUE_MEM_SIZE_IN_MB(_q) (taosQueueMemorySize(_q)/ONE_MB_F) int32_t streamInit() { int8_t old; From 502efb3d019ebe3dc3db2848e8983270b5c41f47 Mon Sep 17 00:00:00 2001 From: liuyao <54liuyao@163.com> Date: Mon, 15 May 2023 16:26:24 +0800 Subject: [PATCH 09/59] op stream selectivity buff --- include/common/tcommon.h | 2 +- include/libs/stream/streamState.h | 5 +- include/libs/stream/tstreamFileState.h | 5 +- source/libs/executor/src/timewindowoperator.c | 48 ++++++++++++++----- source/libs/function/src/builtinsimpl.c | 9 +--- source/libs/stream/src/streamState.c | 30 +++++++----- source/libs/stream/src/tstreamFileState.c | 16 +++++-- 7 files changed, 73 insertions(+), 42 deletions(-) diff --git a/include/common/tcommon.h b/include/common/tcommon.h index 2bc67e439f..51d31ca91f 100644 --- a/include/common/tcommon.h +++ b/include/common/tcommon.h @@ -82,7 +82,7 @@ typedef struct STuplePos { int32_t pageId; int32_t offset; }; - STupleKey streamTupleKey; + SWinKey streamTupleKey; }; } STuplePos; diff --git a/include/libs/stream/streamState.h b/include/libs/stream/streamState.h index 9b80ce2786..0de091bc4e 100644 --- a/include/libs/stream/streamState.h +++ b/include/libs/stream/streamState.h @@ -77,9 +77,8 @@ typedef struct { int64_t number; } SStreamStateCur; -int32_t streamStateFuncPut(SStreamState* pState, const STupleKey* key, const void* value, int32_t vLen); -int32_t streamStateFuncGet(SStreamState* pState, const STupleKey* key, void** pVal, int32_t* pVLen); -int32_t streamStateFuncDel(SStreamState* pState, const STupleKey* key); +int32_t streamStateFuncPut(SStreamState* pState, const SWinKey* key, const void* value, int32_t vLen); +int32_t streamStateFuncGet(SStreamState* pState, const SWinKey* key, void** ppVal, int32_t* pVLen); int32_t streamStatePut(SStreamState* pState, const SWinKey* key, const void* value, int32_t vLen); int32_t streamStateGet(SStreamState* pState, const SWinKey* key, void** pVal, int32_t* pVLen); diff --git a/include/libs/stream/tstreamFileState.h b/include/libs/stream/tstreamFileState.h index d50f0e0a31..7124e2d251 100644 --- a/include/libs/stream/tstreamFileState.h +++ b/include/libs/stream/tstreamFileState.h @@ -38,8 +38,8 @@ typedef SList SStreamSnapshot; typedef TSKEY (*GetTsFun)(void*); -SStreamFileState* streamFileStateInit(int64_t memSize, uint32_t keySize, uint32_t rowSize, GetTsFun fp, void* pFile, - TSKEY delMark); +SStreamFileState* streamFileStateInit(int64_t memSize, uint32_t keySize, uint32_t rowSize, uint32_t selectRowSize, + GetTsFun fp, void* pFile, TSKEY delMark); void streamFileStateDestroy(SStreamFileState* pFileState); void streamFileStateClear(SStreamFileState* pFileState); bool needClearDiskBuff(SStreamFileState* pFileState); @@ -56,6 +56,7 @@ int32_t recoverSnapshot(SStreamFileState* pFileState); int32_t getSnapshotIdList(SStreamFileState* pFileState, SArray* list); int32_t deleteExpiredCheckPoint(SStreamFileState* pFileState, TSKEY mark); +int32_t streamFileStateGeSelectRowSize(SStreamFileState* pFileState); #ifdef __cplusplus } diff --git a/source/libs/executor/src/timewindowoperator.c b/source/libs/executor/src/timewindowoperator.c index ca3165bf92..7570e3c112 100644 --- a/source/libs/executor/src/timewindowoperator.c +++ b/source/libs/executor/src/timewindowoperator.c @@ -2675,6 +2675,29 @@ TSKEY compareTs(void* pKey) { return pWinKey->ts; } +int32_t getSelectivityBufSize(SqlFunctionCtx* pCtx) { + if (pCtx->subsidiaries.rowLen == 0) { + int32_t rowLen = 0; + for (int32_t j = 0; j < pCtx->subsidiaries.num; ++j) { + SqlFunctionCtx* pc = pCtx->subsidiaries.pCtx[j]; + rowLen += pc->pExpr->base.resSchema.bytes; + } + + return rowLen + pCtx->subsidiaries.num * sizeof(bool); + } else { + return pCtx->subsidiaries.rowLen; + } +} + +int32_t getMaxFunResSize(SExprSupp* pSup, int32_t numOfCols) { + int32_t size = 0; + for (int32_t i = 0; i < numOfCols; ++i) { + int32_t resSize = getSelectivityBufSize(pSup->pCtx + i); + size = TMAX(size, resSize); + } + return size; +} + SOperatorInfo* createStreamFinalIntervalOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo, int32_t numOfChild) { SIntervalPhysiNode* pIntervalPhyNode = (SIntervalPhysiNode*)pPhyNode; @@ -2721,8 +2744,11 @@ SOperatorInfo* createStreamFinalIntervalOperatorInfo(SOperatorInfo* downstream, SSDataBlock* pResBlock = createDataBlockFromDescNode(pPhyNode->pOutputDataBlockDesc); initBasicInfo(&pInfo->binfo, pResBlock); + pInfo->pState = taosMemoryCalloc(1, sizeof(SStreamState)); + *(pInfo->pState) = *(pTaskInfo->streamInfo.pState); + streamStateSetNumber(pInfo->pState, -1); int32_t code = initAggSup(&pOperator->exprSupp, &pInfo->aggSup, pExprInfo, numOfCols, keyBufSize, pTaskInfo->id.str, - pTaskInfo->streamInfo.pState); + pInfo->pState); if (code != TSDB_CODE_SUCCESS) { goto _error; } @@ -2731,10 +2757,6 @@ SOperatorInfo* createStreamFinalIntervalOperatorInfo(SOperatorInfo* downstream, initExecTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &pTaskInfo->window); - pInfo->pState = taosMemoryCalloc(1, sizeof(SStreamState)); - *(pInfo->pState) = *(pTaskInfo->streamInfo.pState); - streamStateSetNumber(pInfo->pState, -1); - initResultRowInfo(&pInfo->binfo.resultRowInfo); pInfo->numOfChild = numOfChild; @@ -2767,7 +2789,8 @@ SOperatorInfo* createStreamFinalIntervalOperatorInfo(SOperatorInfo* downstream, pInfo->numOfDatapack = 0; pInfo->pUpdated = NULL; pInfo->pUpdatedMap = NULL; - pInfo->pState->pFileState = streamFileStateInit(tsStreamBufferSize, sizeof(SWinKey), pInfo->aggSup.resultRowSize, + int32_t funResSize= getMaxFunResSize(&pOperator->exprSupp, numOfCols); + pInfo->pState->pFileState = streamFileStateInit(tsStreamBufferSize, sizeof(SWinKey), pInfo->aggSup.resultRowSize, funResSize, compareTs, pInfo->pState, pInfo->twAggSup.deleteMark); pInfo->dataVersion = 0; @@ -4886,9 +4909,13 @@ SOperatorInfo* createStreamIntervalOperatorInfo(SOperatorInfo* downstream, SPhys pInfo->primaryTsIndex = ((SColumnNode*)pIntervalPhyNode->window.pTspk)->slotId; initResultSizeInfo(&pOperator->resultInfo, 4096); + pInfo->pState = taosMemoryCalloc(1, sizeof(SStreamState)); + *(pInfo->pState) = *(pTaskInfo->streamInfo.pState); + streamStateSetNumber(pInfo->pState, -1); + size_t keyBufSize = sizeof(int64_t) + sizeof(int64_t) + POINTER_BYTES; code = initAggSup(pSup, &pInfo->aggSup, pExprInfo, numOfCols, keyBufSize, pTaskInfo->id.str, - pTaskInfo->streamInfo.pState); + pInfo->pState); if (code != TSDB_CODE_SUCCESS) { goto _error; } @@ -4909,10 +4936,6 @@ SOperatorInfo* createStreamIntervalOperatorInfo(SOperatorInfo* downstream, SPhys pInfo->pDelRes = createSpecialDataBlock(STREAM_DELETE_RESULT); initResultRowInfo(&pInfo->binfo.resultRowInfo); - pInfo->pState = taosMemoryCalloc(1, sizeof(SStreamState)); - *(pInfo->pState) = *(pTaskInfo->streamInfo.pState); - streamStateSetNumber(pInfo->pState, -1); - pInfo->pPhyNode = NULL; // create new child pInfo->pPullDataMap = NULL; pInfo->pPullWins = NULL; // SPullWindowInfo @@ -4925,7 +4948,8 @@ SOperatorInfo* createStreamIntervalOperatorInfo(SOperatorInfo* downstream, SPhys pInfo->numOfDatapack = 0; pInfo->pUpdated = NULL; pInfo->pUpdatedMap = NULL; - pInfo->pState->pFileState = streamFileStateInit(tsStreamBufferSize, sizeof(SWinKey), pInfo->aggSup.resultRowSize, + int32_t funResSize= getMaxFunResSize(pSup, numOfCols); + pInfo->pState->pFileState = streamFileStateInit(tsStreamBufferSize, sizeof(SWinKey), pInfo->aggSup.resultRowSize, funResSize, compareTs, pInfo->pState, pInfo->twAggSup.deleteMark); setOperatorInfo(pOperator, "StreamIntervalOperator", QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL, true, OP_NOT_OPENED, diff --git a/source/libs/function/src/builtinsimpl.c b/source/libs/function/src/builtinsimpl.c index af318a6bc5..e50eb0b784 100644 --- a/source/libs/function/src/builtinsimpl.c +++ b/source/libs/function/src/builtinsimpl.c @@ -881,10 +881,6 @@ int32_t setSelectivityValue(SqlFunctionCtx* pCtx, SSDataBlock* pBlock, const STu } pStart += pDstCol->info.bytes; } - - if (pCtx->saveHandle.pState) { - streamFreeVal((void*)p); - } } return TSDB_CODE_SUCCESS; @@ -3121,7 +3117,7 @@ void* serializeTupleData(const SSDataBlock* pSrcBlock, int32_t rowIndex, SSubsid return buf; } -static int32_t doSaveTupleData(SSerializeDataHandle* pHandle, const void* pBuf, size_t length, STupleKey* key, +static int32_t doSaveTupleData(SSerializeDataHandle* pHandle, const void* pBuf, size_t length, SWinKey* key, STuplePos* pPos) { STuplePos p = {0}; if (pHandle->pBuf != NULL) { @@ -3169,7 +3165,7 @@ static int32_t doSaveTupleData(SSerializeDataHandle* pHandle, const void* pBuf, int32_t saveTupleData(SqlFunctionCtx* pCtx, int32_t rowIndex, const SSDataBlock* pSrcBlock, STuplePos* pPos) { prepareBuf(pCtx); - STupleKey key; + SWinKey key; if (pCtx->saveHandle.pBuf == NULL) { SColumnInfoData* pColInfo = taosArrayGet(pSrcBlock->pDataBlock, 0); if (pColInfo->info.type == TSDB_DATA_TYPE_TIMESTAMP) { @@ -3177,7 +3173,6 @@ int32_t saveTupleData(SqlFunctionCtx* pCtx, int32_t rowIndex, const SSDataBlock* key.groupId = pSrcBlock->info.id.groupId; key.ts = skey; - key.exprIdx = pCtx->exprIdx; } } diff --git a/source/libs/stream/src/streamState.c b/source/libs/stream/src/streamState.c index fa9bfe6ca5..a607d6af74 100644 --- a/source/libs/stream/src/streamState.c +++ b/source/libs/stream/src/streamState.c @@ -262,26 +262,30 @@ int32_t streamStateCommit(SStreamState* pState) { #endif } -int32_t streamStateFuncPut(SStreamState* pState, const STupleKey* key, const void* value, int32_t vLen) { +int32_t streamStateFuncPut(SStreamState* pState, const SWinKey* key, const void* value, int32_t vLen) { #ifdef USE_ROCKSDB - return streamStateFuncPut_rocksdb(pState, key, value, vLen); + void* pVal = NULL; + int32_t len = 0; + int32_t code = getRowBuff(pState->pFileState, (void*)key, sizeof(SWinKey), &pVal, &len); + char* buf = ((SRowBuffPos*)pVal)->pRowBuff; + uint32_t rowSize = streamFileStateGeSelectRowSize(pState->pFileState); + memcpy(buf + len - rowSize, value, vLen); + return code; #else return tdbTbUpsert(pState->pTdbState->pFuncStateDb, key, sizeof(STupleKey), value, vLen, pState->pTdbState->txn); #endif } -int32_t streamStateFuncGet(SStreamState* pState, const STupleKey* key, void** pVal, int32_t* pVLen) { +int32_t streamStateFuncGet(SStreamState* pState, const SWinKey* key, void** ppVal, int32_t* pVLen) { #ifdef USE_ROCKSDB - return streamStateFuncGet_rocksdb(pState, key, pVal, pVLen); + void* pVal = NULL; + int32_t len = 0; + int32_t code = getRowBuff(pState->pFileState, (void*)key, sizeof(SWinKey), (void**)(&pVal), &len); + char* buf = ((SRowBuffPos*)pVal)->pRowBuff; + uint32_t rowSize = streamFileStateGeSelectRowSize(pState->pFileState); + *ppVal = buf + len - rowSize; + return code; #else - return tdbTbGet(pState->pTdbState->pFuncStateDb, key, sizeof(STupleKey), pVal, pVLen); -#endif -} - -int32_t streamStateFuncDel(SStreamState* pState, const STupleKey* key) { -#ifdef USE_ROCKSDB - return streamStateFuncDel_rocksdb(pState, key); -#else - return tdbTbDelete(pState->pTdbState->pFuncStateDb, key, sizeof(STupleKey), pState->pTdbState->txn); + return tdbTbGet(pState->pTdbState->pFuncStateDb, key, sizeof(STupleKey), ppVal, pVLen); #endif } diff --git a/source/libs/stream/src/tstreamFileState.c b/source/libs/stream/src/tstreamFileState.c index b7401ec5d9..1b62b80b3d 100644 --- a/source/libs/stream/src/tstreamFileState.c +++ b/source/libs/stream/src/tstreamFileState.c @@ -31,6 +31,7 @@ struct SStreamFileState { SSHashObj* rowBuffMap; void* pFileStore; int32_t rowSize; + int32_t selectivityRowSize; int32_t keyLen; uint64_t preCheckPointVersion; uint64_t checkPointVersion; @@ -44,7 +45,7 @@ struct SStreamFileState { typedef SRowBuffPos SRowBuffInfo; -SStreamFileState* streamFileStateInit(int64_t memSize, uint32_t keySize, uint32_t rowSize, GetTsFun fp, void* pFile, +SStreamFileState* streamFileStateInit(int64_t memSize, uint32_t keySize, uint32_t rowSize, uint32_t selectRowSize, GetTsFun fp, void* pFile, TSKEY delMark) { if (memSize <= 0) { memSize = DEFAULT_MAX_STREAM_BUFFER_SIZE; @@ -57,6 +58,7 @@ SStreamFileState* streamFileStateInit(int64_t memSize, uint32_t keySize, uint32_ if (!pFileState) { goto _error; } + rowSize += selectRowSize; pFileState->maxRowCount = TMAX((uint64_t)memSize / rowSize, FLUSH_NUM * 2); pFileState->usedBuffs = tdListNew(POINTER_BYTES); pFileState->freeBuffs = tdListNew(POINTER_BYTES); @@ -68,11 +70,11 @@ SStreamFileState* streamFileStateInit(int64_t memSize, uint32_t keySize, uint32_ } pFileState->keyLen = keySize; pFileState->rowSize = rowSize; + pFileState->selectivityRowSize = selectRowSize; pFileState->preCheckPointVersion = 0; pFileState->checkPointVersion = 1; pFileState->pFileStore = pFile; pFileState->getTs = fp; - pFileState->maxRowCount = TMAX((uint64_t)memSize / rowSize, FLUSH_NUM * 2); pFileState->curRowCount = 0; pFileState->deleteMark = delMark; pFileState->flushMark = INT64_MIN; @@ -440,7 +442,9 @@ int32_t deleteExpiredCheckPoint(SStreamFileState* pFileState, TSKEY mark) { int32_t recoverSnapshot(SStreamFileState* pFileState) { int32_t code = TSDB_CODE_SUCCESS; - deleteExpiredCheckPoint(pFileState, pFileState->maxTs - pFileState->deleteMark); + if (pFileState->maxTs != INT64_MIN) { + deleteExpiredCheckPoint(pFileState, pFileState->maxTs - pFileState->deleteMark); + } void* pStVal = NULL; int32_t len = 0; @@ -475,4 +479,8 @@ int32_t recoverSnapshot(SStreamFileState* pFileState) { streamStateFreeCur(pCur); return TSDB_CODE_SUCCESS; -} \ No newline at end of file +} + +int32_t streamFileStateGeSelectRowSize(SStreamFileState* pFileState) { + return pFileState->selectivityRowSize; +} From 04c8c933c850cb471d0578acae6f1815b402058f Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 15 May 2023 17:15:03 +0800 Subject: [PATCH 10/59] fix(stream): add initial offset check. --- source/dnode/vnode/src/tq/tqRestore.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/tq/tqRestore.c b/source/dnode/vnode/src/tq/tqRestore.c index 8ada268c4d..091b8b9ef8 100644 --- a/source/dnode/vnode/src/tq/tqRestore.c +++ b/source/dnode/vnode/src/tq/tqRestore.c @@ -124,7 +124,7 @@ int32_t createStreamRunReq(SStreamMeta* pStreamMeta, bool* pScanIdle) { tqDebug("vgId:%d s-task:%s wal reader seek to ver:%" PRId64, vgId, pTask->id.idStr, pTask->chkInfo.currentVer); } else { int64_t currentVer = walReaderGetCurrentVer(pTask->exec.pWalReader); - if (currentVer != pTask->chkInfo.currentVer) { + if (currentVer != -1 && currentVer != pTask->chkInfo.currentVer) { int32_t code = walReaderSeekVer(pTask->exec.pWalReader, pTask->chkInfo.currentVer); if (code != TSDB_CODE_SUCCESS) { // no data in wal, quit streamMetaReleaseTask(pStreamMeta, pTask); From cf7b4593e790bd56e443c24ccadda12e89f9a43d Mon Sep 17 00:00:00 2001 From: dapan1121 Date: Tue, 16 May 2023 09:50:10 +0800 Subject: [PATCH 11/59] feat: add drop dnode unsafe command --- include/common/tmsg.h | 1 + include/common/ttokendef.h | 562 +- include/libs/nodes/cmdnodes.h | 1 + source/common/src/tmsg.c | 2 + source/libs/nodes/src/nodesCodeFuncs.c | 7 + source/libs/parser/inc/parAst.h | 2 +- source/libs/parser/inc/sql.y | 10 +- source/libs/parser/src/parAstCreater.c | 3 +- source/libs/parser/src/parTokenizer.c | 1 + source/libs/parser/src/parTranslater.c | 1 + source/libs/parser/src/sql.c | 9484 +++++++++---------- source/libs/parser/test/parInitialDTest.cpp | 19 +- 12 files changed, 4659 insertions(+), 5434 deletions(-) diff --git a/include/common/tmsg.h b/include/common/tmsg.h index 2daa37d453..2af9c6c299 100644 --- a/include/common/tmsg.h +++ b/include/common/tmsg.h @@ -1629,6 +1629,7 @@ typedef struct { char fqdn[TSDB_FQDN_LEN]; int32_t port; int8_t force; + int8_t unsafe; } SDropDnodeReq; int32_t tSerializeSDropDnodeReq(void* buf, int32_t bufLen, SDropDnodeReq* pReq); diff --git a/include/common/ttokendef.h b/include/common/ttokendef.h index 79e8cc1bf1..f055ff14dd 100644 --- a/include/common/ttokendef.h +++ b/include/common/ttokendef.h @@ -70,286 +70,288 @@ #define TK_RESTORE 52 #define TK_NK_IPTOKEN 53 #define TK_FORCE 54 -#define TK_LOCAL 55 -#define TK_QNODE 56 -#define TK_BNODE 57 -#define TK_SNODE 58 -#define TK_MNODE 59 -#define TK_VNODE 60 -#define TK_DATABASE 61 -#define TK_USE 62 -#define TK_FLUSH 63 -#define TK_TRIM 64 -#define TK_COMPACT 65 -#define TK_IF 66 -#define TK_NOT 67 -#define TK_EXISTS 68 -#define TK_BUFFER 69 -#define TK_CACHEMODEL 70 -#define TK_CACHESIZE 71 -#define TK_COMP 72 -#define TK_DURATION 73 -#define TK_NK_VARIABLE 74 -#define TK_MAXROWS 75 -#define TK_MINROWS 76 -#define TK_KEEP 77 -#define TK_PAGES 78 -#define TK_PAGESIZE 79 -#define TK_TSDB_PAGESIZE 80 -#define TK_PRECISION 81 -#define TK_REPLICA 82 -#define TK_VGROUPS 83 -#define TK_SINGLE_STABLE 84 -#define TK_RETENTIONS 85 -#define TK_SCHEMALESS 86 -#define TK_WAL_LEVEL 87 -#define TK_WAL_FSYNC_PERIOD 88 -#define TK_WAL_RETENTION_PERIOD 89 -#define TK_WAL_RETENTION_SIZE 90 -#define TK_WAL_ROLL_PERIOD 91 -#define TK_WAL_SEGMENT_SIZE 92 -#define TK_STT_TRIGGER 93 -#define TK_TABLE_PREFIX 94 -#define TK_TABLE_SUFFIX 95 -#define TK_NK_COLON 96 -#define TK_MAX_SPEED 97 -#define TK_START 98 -#define TK_TIMESTAMP 99 -#define TK_END 100 -#define TK_TABLE 101 -#define TK_NK_LP 102 -#define TK_NK_RP 103 -#define TK_STABLE 104 -#define TK_ADD 105 -#define TK_COLUMN 106 -#define TK_MODIFY 107 -#define TK_RENAME 108 -#define TK_TAG 109 -#define TK_SET 110 -#define TK_NK_EQ 111 -#define TK_USING 112 -#define TK_TAGS 113 -#define TK_BOOL 114 -#define TK_TINYINT 115 -#define TK_SMALLINT 116 -#define TK_INT 117 -#define TK_INTEGER 118 -#define TK_BIGINT 119 -#define TK_FLOAT 120 -#define TK_DOUBLE 121 -#define TK_BINARY 122 -#define TK_NCHAR 123 -#define TK_UNSIGNED 124 -#define TK_JSON 125 -#define TK_VARCHAR 126 -#define TK_MEDIUMBLOB 127 -#define TK_BLOB 128 -#define TK_VARBINARY 129 -#define TK_DECIMAL 130 -#define TK_COMMENT 131 -#define TK_MAX_DELAY 132 -#define TK_WATERMARK 133 -#define TK_ROLLUP 134 -#define TK_TTL 135 -#define TK_SMA 136 -#define TK_DELETE_MARK 137 -#define TK_FIRST 138 -#define TK_LAST 139 -#define TK_SHOW 140 -#define TK_PRIVILEGES 141 -#define TK_DATABASES 142 -#define TK_TABLES 143 -#define TK_STABLES 144 -#define TK_MNODES 145 -#define TK_QNODES 146 -#define TK_FUNCTIONS 147 -#define TK_INDEXES 148 -#define TK_ACCOUNTS 149 -#define TK_APPS 150 -#define TK_CONNECTIONS 151 -#define TK_LICENCES 152 -#define TK_GRANTS 153 -#define TK_QUERIES 154 -#define TK_SCORES 155 -#define TK_TOPICS 156 -#define TK_VARIABLES 157 -#define TK_CLUSTER 158 -#define TK_BNODES 159 -#define TK_SNODES 160 -#define TK_TRANSACTIONS 161 -#define TK_DISTRIBUTED 162 -#define TK_CONSUMERS 163 -#define TK_SUBSCRIPTIONS 164 -#define TK_VNODES 165 -#define TK_ALIVE 166 -#define TK_LIKE 167 -#define TK_TBNAME 168 -#define TK_QTAGS 169 -#define TK_AS 170 -#define TK_INDEX 171 -#define TK_FUNCTION 172 -#define TK_INTERVAL 173 -#define TK_COUNT 174 -#define TK_LAST_ROW 175 -#define TK_TOPIC 176 -#define TK_META 177 -#define TK_CONSUMER 178 -#define TK_GROUP 179 -#define TK_DESC 180 -#define TK_DESCRIBE 181 -#define TK_RESET 182 -#define TK_QUERY 183 -#define TK_CACHE 184 -#define TK_EXPLAIN 185 -#define TK_ANALYZE 186 -#define TK_VERBOSE 187 -#define TK_NK_BOOL 188 -#define TK_RATIO 189 -#define TK_NK_FLOAT 190 -#define TK_OUTPUTTYPE 191 -#define TK_AGGREGATE 192 -#define TK_BUFSIZE 193 -#define TK_LANGUAGE 194 -#define TK_REPLACE 195 -#define TK_STREAM 196 -#define TK_INTO 197 -#define TK_PAUSE 198 -#define TK_RESUME 199 -#define TK_TRIGGER 200 -#define TK_AT_ONCE 201 -#define TK_WINDOW_CLOSE 202 -#define TK_IGNORE 203 -#define TK_EXPIRED 204 -#define TK_FILL_HISTORY 205 -#define TK_UPDATE 206 -#define TK_SUBTABLE 207 -#define TK_UNTREATED 208 -#define TK_KILL 209 -#define TK_CONNECTION 210 -#define TK_TRANSACTION 211 -#define TK_BALANCE 212 -#define TK_VGROUP 213 -#define TK_LEADER 214 -#define TK_MERGE 215 -#define TK_REDISTRIBUTE 216 -#define TK_SPLIT 217 -#define TK_DELETE 218 -#define TK_INSERT 219 -#define TK_NULL 220 -#define TK_NK_QUESTION 221 -#define TK_NK_ARROW 222 -#define TK_ROWTS 223 -#define TK_QSTART 224 -#define TK_QEND 225 -#define TK_QDURATION 226 -#define TK_WSTART 227 -#define TK_WEND 228 -#define TK_WDURATION 229 -#define TK_IROWTS 230 -#define TK_ISFILLED 231 -#define TK_CAST 232 -#define TK_NOW 233 -#define TK_TODAY 234 -#define TK_TIMEZONE 235 -#define TK_CLIENT_VERSION 236 -#define TK_SERVER_VERSION 237 -#define TK_SERVER_STATUS 238 -#define TK_CURRENT_USER 239 -#define TK_CASE 240 -#define TK_WHEN 241 -#define TK_THEN 242 -#define TK_ELSE 243 -#define TK_BETWEEN 244 -#define TK_IS 245 -#define TK_NK_LT 246 -#define TK_NK_GT 247 -#define TK_NK_LE 248 -#define TK_NK_GE 249 -#define TK_NK_NE 250 -#define TK_MATCH 251 -#define TK_NMATCH 252 -#define TK_CONTAINS 253 -#define TK_IN 254 -#define TK_JOIN 255 -#define TK_INNER 256 -#define TK_SELECT 257 -#define TK_DISTINCT 258 -#define TK_WHERE 259 -#define TK_PARTITION 260 -#define TK_BY 261 -#define TK_SESSION 262 -#define TK_STATE_WINDOW 263 -#define TK_EVENT_WINDOW 264 -#define TK_SLIDING 265 -#define TK_FILL 266 -#define TK_VALUE 267 -#define TK_VALUE_F 268 -#define TK_NONE 269 -#define TK_PREV 270 -#define TK_NULL_F 271 -#define TK_LINEAR 272 -#define TK_NEXT 273 -#define TK_HAVING 274 -#define TK_RANGE 275 -#define TK_EVERY 276 -#define TK_ORDER 277 -#define TK_SLIMIT 278 -#define TK_SOFFSET 279 -#define TK_LIMIT 280 -#define TK_OFFSET 281 -#define TK_ASC 282 -#define TK_NULLS 283 -#define TK_ABORT 284 -#define TK_AFTER 285 -#define TK_ATTACH 286 -#define TK_BEFORE 287 -#define TK_BEGIN 288 -#define TK_BITAND 289 -#define TK_BITNOT 290 -#define TK_BITOR 291 -#define TK_BLOCKS 292 -#define TK_CHANGE 293 -#define TK_COMMA 294 -#define TK_CONCAT 295 -#define TK_CONFLICT 296 -#define TK_COPY 297 -#define TK_DEFERRED 298 -#define TK_DELIMITERS 299 -#define TK_DETACH 300 -#define TK_DIVIDE 301 -#define TK_DOT 302 -#define TK_EACH 303 -#define TK_FAIL 304 -#define TK_FILE 305 -#define TK_FOR 306 -#define TK_GLOB 307 -#define TK_ID 308 -#define TK_IMMEDIATE 309 -#define TK_IMPORT 310 -#define TK_INITIALLY 311 -#define TK_INSTEAD 312 -#define TK_ISNULL 313 -#define TK_KEY 314 -#define TK_MODULES 315 -#define TK_NK_BITNOT 316 -#define TK_NK_SEMI 317 -#define TK_NOTNULL 318 -#define TK_OF 319 -#define TK_PLUS 320 -#define TK_PRIVILEGE 321 -#define TK_RAISE 322 -#define TK_RESTRICT 323 -#define TK_ROW 324 -#define TK_SEMI 325 -#define TK_STAR 326 -#define TK_STATEMENT 327 -#define TK_STRICT 328 -#define TK_STRING 329 -#define TK_TIMES 330 -#define TK_VALUES 331 -#define TK_VARIABLE 332 -#define TK_VIEW 333 -#define TK_WAL 334 +#define TK_UNSAFE 55 +#define TK_LOCAL 56 +#define TK_QNODE 57 +#define TK_BNODE 58 +#define TK_SNODE 59 +#define TK_MNODE 60 +#define TK_VNODE 61 +#define TK_DATABASE 62 +#define TK_USE 63 +#define TK_FLUSH 64 +#define TK_TRIM 65 +#define TK_COMPACT 66 +#define TK_IF 67 +#define TK_NOT 68 +#define TK_EXISTS 69 +#define TK_BUFFER 70 +#define TK_CACHEMODEL 71 +#define TK_CACHESIZE 72 +#define TK_COMP 73 +#define TK_DURATION 74 +#define TK_NK_VARIABLE 75 +#define TK_MAXROWS 76 +#define TK_MINROWS 77 +#define TK_KEEP 78 +#define TK_PAGES 79 +#define TK_PAGESIZE 80 +#define TK_TSDB_PAGESIZE 81 +#define TK_PRECISION 82 +#define TK_REPLICA 83 +#define TK_VGROUPS 84 +#define TK_SINGLE_STABLE 85 +#define TK_RETENTIONS 86 +#define TK_SCHEMALESS 87 +#define TK_WAL_LEVEL 88 +#define TK_WAL_FSYNC_PERIOD 89 +#define TK_WAL_RETENTION_PERIOD 90 +#define TK_WAL_RETENTION_SIZE 91 +#define TK_WAL_ROLL_PERIOD 92 +#define TK_WAL_SEGMENT_SIZE 93 +#define TK_STT_TRIGGER 94 +#define TK_TABLE_PREFIX 95 +#define TK_TABLE_SUFFIX 96 +#define TK_NK_COLON 97 +#define TK_MAX_SPEED 98 +#define TK_START 99 +#define TK_TIMESTAMP 100 +#define TK_END 101 +#define TK_TABLE 102 +#define TK_NK_LP 103 +#define TK_NK_RP 104 +#define TK_STABLE 105 +#define TK_ADD 106 +#define TK_COLUMN 107 +#define TK_MODIFY 108 +#define TK_RENAME 109 +#define TK_TAG 110 +#define TK_SET 111 +#define TK_NK_EQ 112 +#define TK_USING 113 +#define TK_TAGS 114 +#define TK_BOOL 115 +#define TK_TINYINT 116 +#define TK_SMALLINT 117 +#define TK_INT 118 +#define TK_INTEGER 119 +#define TK_BIGINT 120 +#define TK_FLOAT 121 +#define TK_DOUBLE 122 +#define TK_BINARY 123 +#define TK_NCHAR 124 +#define TK_UNSIGNED 125 +#define TK_JSON 126 +#define TK_VARCHAR 127 +#define TK_MEDIUMBLOB 128 +#define TK_BLOB 129 +#define TK_VARBINARY 130 +#define TK_DECIMAL 131 +#define TK_COMMENT 132 +#define TK_MAX_DELAY 133 +#define TK_WATERMARK 134 +#define TK_ROLLUP 135 +#define TK_TTL 136 +#define TK_SMA 137 +#define TK_DELETE_MARK 138 +#define TK_FIRST 139 +#define TK_LAST 140 +#define TK_SHOW 141 +#define TK_PRIVILEGES 142 +#define TK_DATABASES 143 +#define TK_TABLES 144 +#define TK_STABLES 145 +#define TK_MNODES 146 +#define TK_QNODES 147 +#define TK_FUNCTIONS 148 +#define TK_INDEXES 149 +#define TK_ACCOUNTS 150 +#define TK_APPS 151 +#define TK_CONNECTIONS 152 +#define TK_LICENCES 153 +#define TK_GRANTS 154 +#define TK_QUERIES 155 +#define TK_SCORES 156 +#define TK_TOPICS 157 +#define TK_VARIABLES 158 +#define TK_CLUSTER 159 +#define TK_BNODES 160 +#define TK_SNODES 161 +#define TK_TRANSACTIONS 162 +#define TK_DISTRIBUTED 163 +#define TK_CONSUMERS 164 +#define TK_SUBSCRIPTIONS 165 +#define TK_VNODES 166 +#define TK_ALIVE 167 +#define TK_LIKE 168 +#define TK_TBNAME 169 +#define TK_QTAGS 170 +#define TK_AS 171 +#define TK_INDEX 172 +#define TK_FUNCTION 173 +#define TK_INTERVAL 174 +#define TK_COUNT 175 +#define TK_LAST_ROW 176 +#define TK_TOPIC 177 +#define TK_META 178 +#define TK_CONSUMER 179 +#define TK_GROUP 180 +#define TK_DESC 181 +#define TK_DESCRIBE 182 +#define TK_RESET 183 +#define TK_QUERY 184 +#define TK_CACHE 185 +#define TK_EXPLAIN 186 +#define TK_ANALYZE 187 +#define TK_VERBOSE 188 +#define TK_NK_BOOL 189 +#define TK_RATIO 190 +#define TK_NK_FLOAT 191 +#define TK_OUTPUTTYPE 192 +#define TK_AGGREGATE 193 +#define TK_BUFSIZE 194 +#define TK_LANGUAGE 195 +#define TK_REPLACE 196 +#define TK_STREAM 197 +#define TK_INTO 198 +#define TK_PAUSE 199 +#define TK_RESUME 200 +#define TK_TRIGGER 201 +#define TK_AT_ONCE 202 +#define TK_WINDOW_CLOSE 203 +#define TK_IGNORE 204 +#define TK_EXPIRED 205 +#define TK_FILL_HISTORY 206 +#define TK_UPDATE 207 +#define TK_SUBTABLE 208 +#define TK_UNTREATED 209 +#define TK_KILL 210 +#define TK_CONNECTION 211 +#define TK_TRANSACTION 212 +#define TK_BALANCE 213 +#define TK_VGROUP 214 +#define TK_LEADER 215 +#define TK_MERGE 216 +#define TK_REDISTRIBUTE 217 +#define TK_SPLIT 218 +#define TK_DELETE 219 +#define TK_INSERT 220 +#define TK_NULL 221 +#define TK_NK_QUESTION 222 +#define TK_NK_ARROW 223 +#define TK_ROWTS 224 +#define TK_QSTART 225 +#define TK_QEND 226 +#define TK_QDURATION 227 +#define TK_WSTART 228 +#define TK_WEND 229 +#define TK_WDURATION 230 +#define TK_IROWTS 231 +#define TK_ISFILLED 232 +#define TK_CAST 233 +#define TK_NOW 234 +#define TK_TODAY 235 +#define TK_TIMEZONE 236 +#define TK_CLIENT_VERSION 237 +#define TK_SERVER_VERSION 238 +#define TK_SERVER_STATUS 239 +#define TK_CURRENT_USER 240 +#define TK_CASE 241 +#define TK_WHEN 242 +#define TK_THEN 243 +#define TK_ELSE 244 +#define TK_BETWEEN 245 +#define TK_IS 246 +#define TK_NK_LT 247 +#define TK_NK_GT 248 +#define TK_NK_LE 249 +#define TK_NK_GE 250 +#define TK_NK_NE 251 +#define TK_MATCH 252 +#define TK_NMATCH 253 +#define TK_CONTAINS 254 +#define TK_IN 255 +#define TK_JOIN 256 +#define TK_INNER 257 +#define TK_SELECT 258 +#define TK_DISTINCT 259 +#define TK_WHERE 260 +#define TK_PARTITION 261 +#define TK_BY 262 +#define TK_SESSION 263 +#define TK_STATE_WINDOW 264 +#define TK_EVENT_WINDOW 265 +#define TK_SLIDING 266 +#define TK_FILL 267 +#define TK_VALUE 268 +#define TK_VALUE_F 269 +#define TK_NONE 270 +#define TK_PREV 271 +#define TK_NULL_F 272 +#define TK_LINEAR 273 +#define TK_NEXT 274 +#define TK_HAVING 275 +#define TK_RANGE 276 +#define TK_EVERY 277 +#define TK_ORDER 278 +#define TK_SLIMIT 279 +#define TK_SOFFSET 280 +#define TK_LIMIT 281 +#define TK_OFFSET 282 +#define TK_ASC 283 +#define TK_NULLS 284 +#define TK_ABORT 285 +#define TK_AFTER 286 +#define TK_ATTACH 287 +#define TK_BEFORE 288 +#define TK_BEGIN 289 +#define TK_BITAND 290 +#define TK_BITNOT 291 +#define TK_BITOR 292 +#define TK_BLOCKS 293 +#define TK_CHANGE 294 +#define TK_COMMA 295 +#define TK_CONCAT 296 +#define TK_CONFLICT 297 +#define TK_COPY 298 +#define TK_DEFERRED 299 +#define TK_DELIMITERS 300 +#define TK_DETACH 301 +#define TK_DIVIDE 302 +#define TK_DOT 303 +#define TK_EACH 304 +#define TK_FAIL 305 +#define TK_FILE 306 +#define TK_FOR 307 +#define TK_GLOB 308 +#define TK_ID 309 +#define TK_IMMEDIATE 310 +#define TK_IMPORT 311 +#define TK_INITIALLY 312 +#define TK_INSTEAD 313 +#define TK_ISNULL 314 +#define TK_KEY 315 +#define TK_MODULES 316 +#define TK_NK_BITNOT 317 +#define TK_NK_SEMI 318 +#define TK_NOTNULL 319 +#define TK_OF 320 +#define TK_PLUS 321 +#define TK_PRIVILEGE 322 +#define TK_RAISE 323 +#define TK_RESTRICT 324 +#define TK_ROW 325 +#define TK_SEMI 326 +#define TK_STAR 327 +#define TK_STATEMENT 328 +#define TK_STRICT 329 +#define TK_STRING 330 +#define TK_TIMES 331 +#define TK_VALUES 332 +#define TK_VARIABLE 333 +#define TK_VIEW 334 +#define TK_WAL 335 + #define TK_NK_SPACE 600 diff --git a/include/libs/nodes/cmdnodes.h b/include/libs/nodes/cmdnodes.h index d899307d74..9e0cac066d 100644 --- a/include/libs/nodes/cmdnodes.h +++ b/include/libs/nodes/cmdnodes.h @@ -249,6 +249,7 @@ typedef struct SDropDnodeStmt { char fqdn[TSDB_FQDN_LEN]; int32_t port; bool force; + bool unsafe; } SDropDnodeStmt; typedef struct SAlterDnodeStmt { diff --git a/source/common/src/tmsg.c b/source/common/src/tmsg.c index aff213fea3..dbf7b88ea8 100644 --- a/source/common/src/tmsg.c +++ b/source/common/src/tmsg.c @@ -1701,6 +1701,7 @@ int32_t tSerializeSDropDnodeReq(void *buf, int32_t bufLen, SDropDnodeReq *pReq) if (tEncodeCStr(&encoder, pReq->fqdn) < 0) return -1; if (tEncodeI32(&encoder, pReq->port) < 0) return -1; if (tEncodeI8(&encoder, pReq->force) < 0) return -1; + if (tEncodeI8(&encoder, pReq->unsafe) < 0) return -1; tEndEncode(&encoder); int32_t tlen = encoder.pos; @@ -1717,6 +1718,7 @@ int32_t tDeserializeSDropDnodeReq(void *buf, int32_t bufLen, SDropDnodeReq *pReq if (tDecodeCStrTo(&decoder, pReq->fqdn) < 0) return -1; if (tDecodeI32(&decoder, &pReq->port) < 0) return -1; if (tDecodeI8(&decoder, &pReq->force) < 0) return -1; + if (tDecodeI8(&decoder, &pReq->unsafe) < 0) return -1; tEndDecode(&decoder); tDecoderClear(&decoder); diff --git a/source/libs/nodes/src/nodesCodeFuncs.c b/source/libs/nodes/src/nodesCodeFuncs.c index 136d1fc391..459e389c9c 100644 --- a/source/libs/nodes/src/nodesCodeFuncs.c +++ b/source/libs/nodes/src/nodesCodeFuncs.c @@ -5510,6 +5510,7 @@ static const char* jkDropDnodeStmtDnodeId = "DnodeId"; static const char* jkDropDnodeStmtFqdn = "Fqdn"; static const char* jkDropDnodeStmtPort = "Port"; static const char* jkDropDnodeStmtForce = "Force"; +static const char* jkDropDnodeStmtUnsafe = "Unsafe"; static int32_t dropDnodeStmtToJson(const void* pObj, SJson* pJson) { const SDropDnodeStmt* pNode = (const SDropDnodeStmt*)pObj; @@ -5524,6 +5525,9 @@ static int32_t dropDnodeStmtToJson(const void* pObj, SJson* pJson) { if (TSDB_CODE_SUCCESS == code) { code = tjsonAddBoolToObject(pJson, jkDropDnodeStmtForce, pNode->force); } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonAddBoolToObject(pJson, jkDropDnodeStmtUnsafe, pNode->unsafe); + } return code; } @@ -5541,6 +5545,9 @@ static int32_t jsonToDropDnodeStmt(const SJson* pJson, void* pObj) { if (TSDB_CODE_SUCCESS == code) { code = tjsonGetBoolValue(pJson, jkDropDnodeStmtForce, &pNode->force); } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonGetBoolValue(pJson, jkDropDnodeStmtUnsafe, &pNode->unsafe); + } return code; } diff --git a/source/libs/parser/inc/parAst.h b/source/libs/parser/inc/parAst.h index 1d1a522d01..43765c8112 100644 --- a/source/libs/parser/inc/parAst.h +++ b/source/libs/parser/inc/parAst.h @@ -192,7 +192,7 @@ SNode* createCreateUserStmt(SAstCreateContext* pCxt, SToken* pUserName, const ST SNode* createAlterUserStmt(SAstCreateContext* pCxt, SToken* pUserName, int8_t alterType, const SToken* pVal); SNode* createDropUserStmt(SAstCreateContext* pCxt, SToken* pUserName); SNode* createCreateDnodeStmt(SAstCreateContext* pCxt, const SToken* pFqdn, const SToken* pPort); -SNode* createDropDnodeStmt(SAstCreateContext* pCxt, const SToken* pDnode, bool force); +SNode* createDropDnodeStmt(SAstCreateContext* pCxt, const SToken* pDnode, bool force, bool unsafe); SNode* createAlterDnodeStmt(SAstCreateContext* pCxt, const SToken* pDnode, const SToken* pConfig, const SToken* pValue); SNode* createRealTableNodeForIndexName(SAstCreateContext* pCxt, SToken* pDbName, SToken* pIndexName); SNode* createCreateIndexStmt(SAstCreateContext* pCxt, EIndexType type, bool ignoreExists, SNode* pIndexName, diff --git a/source/libs/parser/inc/sql.y b/source/libs/parser/inc/sql.y index b682cdd4fd..62a5a54a4a 100755 --- a/source/libs/parser/inc/sql.y +++ b/source/libs/parser/inc/sql.y @@ -126,8 +126,10 @@ with_opt(A) ::= WITH search_condition(B). /************************************************ create/drop/alter/restore dnode *********************************************/ cmd ::= CREATE DNODE dnode_endpoint(A). { pCxt->pRootNode = createCreateDnodeStmt(pCxt, &A, NULL); } cmd ::= CREATE DNODE dnode_endpoint(A) PORT NK_INTEGER(B). { pCxt->pRootNode = createCreateDnodeStmt(pCxt, &A, &B); } -cmd ::= DROP DNODE NK_INTEGER(A) force_opt(B). { pCxt->pRootNode = createDropDnodeStmt(pCxt, &A, B); } -cmd ::= DROP DNODE dnode_endpoint(A) force_opt(B). { pCxt->pRootNode = createDropDnodeStmt(pCxt, &A, B); } +cmd ::= DROP DNODE NK_INTEGER(A) force_opt(B). { pCxt->pRootNode = createDropDnodeStmt(pCxt, &A, B, false); } +cmd ::= DROP DNODE dnode_endpoint(A) force_opt(B). { pCxt->pRootNode = createDropDnodeStmt(pCxt, &A, B, false); } +cmd ::= DROP DNODE NK_INTEGER(A) unsafe_opt(B). { pCxt->pRootNode = createDropDnodeStmt(pCxt, &A, false, B); } +cmd ::= DROP DNODE dnode_endpoint(A) unsafe_opt(B). { pCxt->pRootNode = createDropDnodeStmt(pCxt, &A, false, B); } cmd ::= ALTER DNODE NK_INTEGER(A) NK_STRING(B). { pCxt->pRootNode = createAlterDnodeStmt(pCxt, &A, &B, NULL); } cmd ::= ALTER DNODE NK_INTEGER(A) NK_STRING(B) NK_STRING(C). { pCxt->pRootNode = createAlterDnodeStmt(pCxt, &A, &B, &C); } cmd ::= ALTER ALL DNODES NK_STRING(A). { pCxt->pRootNode = createAlterDnodeStmt(pCxt, NULL, &A, NULL); } @@ -145,6 +147,10 @@ dnode_endpoint(A) ::= NK_IPTOKEN(B). force_opt(A) ::= . { A = false; } force_opt(A) ::= FORCE. { A = true; } +%type unsafe_opt { bool } +%destructor unsafe_opt { } +unsafe_opt(A) ::= UNSAFE. { A = true; } + /************************************************ alter local *********************************************************/ cmd ::= ALTER LOCAL NK_STRING(A). { pCxt->pRootNode = createAlterLocalStmt(pCxt, &A, NULL); } cmd ::= ALTER LOCAL NK_STRING(A) NK_STRING(B). { pCxt->pRootNode = createAlterLocalStmt(pCxt, &A, &B); } diff --git a/source/libs/parser/src/parAstCreater.c b/source/libs/parser/src/parAstCreater.c index eeccf18c7b..d0c8355b01 100644 --- a/source/libs/parser/src/parAstCreater.c +++ b/source/libs/parser/src/parAstCreater.c @@ -1576,7 +1576,7 @@ SNode* createCreateDnodeStmt(SAstCreateContext* pCxt, const SToken* pFqdn, const return (SNode*)pStmt; } -SNode* createDropDnodeStmt(SAstCreateContext* pCxt, const SToken* pDnode, bool force) { +SNode* createDropDnodeStmt(SAstCreateContext* pCxt, const SToken* pDnode, bool force, bool unsafe) { CHECK_PARSER_STATUS(pCxt); SDropDnodeStmt* pStmt = (SDropDnodeStmt*)nodesMakeNode(QUERY_NODE_DROP_DNODE_STMT); CHECK_OUT_OF_MEM(pStmt); @@ -1589,6 +1589,7 @@ SNode* createDropDnodeStmt(SAstCreateContext* pCxt, const SToken* pDnode, bool f } } pStmt->force = force; + pStmt->unsafe = unsafe; return (SNode*)pStmt; } diff --git a/source/libs/parser/src/parTokenizer.c b/source/libs/parser/src/parTokenizer.c index 5c1f4bf98c..e1a1244b98 100644 --- a/source/libs/parser/src/parTokenizer.c +++ b/source/libs/parser/src/parTokenizer.c @@ -241,6 +241,7 @@ static SKeyword keywordTable[] = { {"TSERIES", TK_TSERIES}, {"TTL", TK_TTL}, {"UNION", TK_UNION}, + {"UNSAFE", TK_UNSAFE}, {"UNSIGNED", TK_UNSIGNED}, {"UNTREATED", TK_UNTREATED}, {"UPDATE", TK_UPDATE}, diff --git a/source/libs/parser/src/parTranslater.c b/source/libs/parser/src/parTranslater.c index c5fe3a1f73..285102beeb 100644 --- a/source/libs/parser/src/parTranslater.c +++ b/source/libs/parser/src/parTranslater.c @@ -5517,6 +5517,7 @@ static int32_t translateDropDnode(STranslateContext* pCxt, SDropDnodeStmt* pStmt strcpy(dropReq.fqdn, pStmt->fqdn); dropReq.port = pStmt->port; dropReq.force = pStmt->force; + dropReq.unsafe = pStmt->unsafe; return buildCmdMsg(pCxt, TDMT_MND_DROP_DNODE, (FSerializeFunc)tSerializeSDropDnodeReq, &dropReq); } diff --git a/source/libs/parser/src/sql.c b/source/libs/parser/src/sql.c index bb0b040035..0aebd8f47f 100644 --- a/source/libs/parser/src/sql.c +++ b/source/libs/parser/src/sql.c @@ -25,7 +25,6 @@ #include #include /************ Begin %include sections from the grammar ************************/ -#line 11 "sql.y" #include #include @@ -42,7 +41,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 ** in a format understandable to "makeheaders". This section is blank unless @@ -106,27 +104,27 @@ #endif /************* Begin control #defines *****************************************/ #define YYCODETYPE unsigned short int -#define YYNOCODE 482 +#define YYNOCODE 484 #define YYACTIONTYPE unsigned short int #define ParseTOKENTYPE SToken typedef union { int yyinit; ParseTOKENTYPE yy0; - SNodeList* yy72; - SNode* yy164; - EJoinType yy196; - bool yy441; - EFillMode yy446; - SToken yy497; - ENullOrder yy517; - EOrder yy550; - int32_t yy560; - int8_t yy563; - int64_t yy693; - SDataType yy700; - SAlterOption yy761; - EOperatorType yy796; - STokenPair yy953; + int64_t yy85; + ENullOrder yy257; + EFillMode yy470; + STokenPair yy489; + SNode* yy520; + EOperatorType yy524; + bool yy537; + int8_t yy575; + EJoinType yy596; + SAlterOption yy805; + SToken yy833; + SDataType yy840; + int32_t yy860; + SNodeList* yy904; + EOrder yy906; } YYMINORTYPE; #ifndef YYSTACKDEPTH #define YYSTACKDEPTH 100 @@ -142,18 +140,18 @@ typedef union { #define ParseCTX_FETCH #define ParseCTX_STORE #define YYFALLBACK 1 -#define YYNSTATE 787 -#define YYNRULE 591 -#define YYNRULE_WITH_ACTION 591 -#define YYNTOKEN 335 -#define YY_MAX_SHIFT 786 -#define YY_MIN_SHIFTREDUCE 1162 -#define YY_MAX_SHIFTREDUCE 1752 -#define YY_ERROR_ACTION 1753 -#define YY_ACCEPT_ACTION 1754 -#define YY_NO_ACTION 1755 -#define YY_MIN_REDUCE 1756 -#define YY_MAX_REDUCE 2346 +#define YYNSTATE 789 +#define YYNRULE 594 +#define YYNRULE_WITH_ACTION 594 +#define YYNTOKEN 336 +#define YY_MAX_SHIFT 788 +#define YY_MIN_SHIFTREDUCE 1165 +#define YY_MAX_SHIFTREDUCE 1758 +#define YY_ERROR_ACTION 1759 +#define YY_ACCEPT_ACTION 1760 +#define YY_NO_ACTION 1761 +#define YY_MIN_REDUCE 1762 +#define YY_MAX_REDUCE 2355 /************* End control #defines *******************************************/ #define YY_NLOOKAHEAD ((int)(sizeof(yy_lookahead)/sizeof(yy_lookahead[0]))) @@ -220,842 +218,840 @@ typedef union { ** yy_default[] Default action for each state. ** *********** Begin parsing tables **********************************************/ -#define YY_ACTTAB_COUNT (2980) +#define YY_ACTTAB_COUNT (2969) static const YYACTIONTYPE yy_action[] = { - /* 0 */ 2158, 402, 680, 1930, 38, 303, 182, 2091, 643, 162, - /* 10 */ 694, 2317, 48, 46, 1680, 1757, 380, 1932, 1969, 216, - /* 20 */ 399, 133, 1529, 529, 1980, 1799, 642, 186, 564, 655, - /* 30 */ 140, 2318, 644, 1610, 1829, 1527, 123, 2176, 107, 122, - /* 40 */ 121, 120, 119, 118, 117, 116, 115, 114, 1555, 2126, - /* 50 */ 1907, 696, 41, 40, 251, 141, 47, 45, 44, 43, - /* 60 */ 42, 1605, 531, 1922, 1557, 41, 40, 19, 528, 47, - /* 70 */ 45, 44, 43, 42, 1535, 1905, 1554, 47, 45, 44, - /* 80 */ 43, 42, 142, 2157, 526, 2217, 2193, 527, 1792, 339, - /* 90 */ 2159, 700, 2161, 2162, 695, 693, 690, 681, 2211, 783, - /* 100 */ 167, 30, 15, 760, 759, 758, 757, 411, 1871, 756, - /* 110 */ 755, 144, 750, 749, 748, 747, 746, 745, 744, 157, - /* 120 */ 740, 739, 738, 410, 409, 735, 734, 733, 175, 174, - /* 130 */ 657, 184, 2254, 2255, 1350, 138, 2259, 123, 1612, 1613, - /* 140 */ 122, 121, 120, 119, 118, 117, 116, 115, 114, 1341, - /* 150 */ 722, 721, 720, 1345, 719, 1347, 1348, 718, 715, 679, - /* 160 */ 1356, 712, 1358, 1359, 709, 706, 1384, 1385, 1585, 1595, - /* 170 */ 680, 1930, 41, 40, 1611, 1614, 47, 45, 44, 43, - /* 180 */ 42, 730, 155, 154, 727, 726, 725, 152, 1530, 133, - /* 190 */ 1528, 62, 666, 408, 407, 568, 569, 41, 40, 567, - /* 200 */ 661, 47, 45, 44, 43, 42, 730, 155, 154, 727, - /* 210 */ 726, 725, 152, 665, 262, 534, 1536, 1554, 527, 1792, - /* 220 */ 1533, 1534, 543, 1584, 1587, 1588, 1589, 1590, 1591, 1592, - /* 230 */ 1593, 1594, 692, 688, 1603, 1604, 1606, 1607, 1608, 1609, - /* 240 */ 2, 48, 46, 541, 2158, 2041, 349, 618, 1552, 399, - /* 250 */ 2317, 1529, 14, 13, 658, 482, 358, 2048, 496, 638, - /* 260 */ 679, 495, 1610, 52, 1527, 2323, 186, 582, 581, 580, - /* 270 */ 2318, 644, 2046, 667, 572, 137, 576, 465, 2136, 497, - /* 280 */ 575, 2176, 234, 467, 1195, 574, 579, 374, 373, 1756, - /* 290 */ 1605, 573, 1921, 2126, 1982, 696, 19, 1982, 172, 1677, - /* 300 */ 1639, 379, 2140, 1535, 389, 560, 556, 552, 548, 1980, - /* 310 */ 231, 179, 1980, 132, 131, 130, 129, 128, 127, 126, - /* 320 */ 125, 124, 168, 1197, 1768, 1200, 1201, 2157, 783, 367, - /* 330 */ 2193, 15, 2032, 110, 2159, 700, 2161, 2162, 695, 2142, - /* 340 */ 690, 455, 51, 2261, 1749, 183, 189, 2246, 1982, 690, - /* 350 */ 88, 395, 2242, 229, 1653, 394, 1640, 404, 445, 732, - /* 360 */ 1975, 1977, 444, 1980, 188, 1439, 1440, 1612, 1613, 2258, - /* 370 */ 493, 1539, 2272, 487, 486, 485, 484, 481, 480, 479, - /* 380 */ 478, 477, 473, 472, 471, 470, 348, 462, 461, 460, - /* 390 */ 218, 457, 456, 365, 529, 679, 1799, 1585, 1595, 2012, - /* 400 */ 1555, 655, 140, 1611, 1614, 618, 41, 40, 2317, 2322, - /* 410 */ 47, 45, 44, 43, 42, 1982, 371, 1530, 189, 1528, - /* 420 */ 228, 222, 364, 2323, 186, 227, 1556, 539, 2318, 644, - /* 430 */ 1980, 1754, 2048, 37, 397, 1634, 1635, 1636, 1637, 1638, - /* 440 */ 1642, 1643, 1644, 1645, 392, 220, 1748, 2045, 667, 1533, - /* 450 */ 1534, 637, 1584, 1587, 1588, 1589, 1590, 1591, 1592, 1593, - /* 460 */ 1594, 692, 688, 1603, 1604, 1606, 1607, 1608, 1609, 2, - /* 470 */ 12, 48, 46, 87, 87, 393, 666, 51, 2176, 399, - /* 480 */ 451, 1529, 1779, 165, 372, 666, 370, 369, 2158, 566, - /* 490 */ 368, 1932, 1610, 643, 1527, 1708, 2317, 189, 697, 1925, - /* 500 */ 1926, 1719, 414, 185, 2254, 2255, 413, 138, 2259, 1742, - /* 510 */ 568, 642, 186, 2158, 567, 1982, 2318, 644, 1456, 1457, - /* 520 */ 1605, 680, 1930, 697, 345, 2176, 19, 664, 489, 2041, - /* 530 */ 1981, 2126, 636, 1535, 655, 140, 675, 2126, 2041, 696, - /* 540 */ 191, 633, 630, 629, 1706, 1707, 1709, 1710, 1711, 618, - /* 550 */ 2176, 1778, 2317, 1676, 1455, 1458, 469, 66, 783, 523, - /* 560 */ 101, 15, 2126, 1553, 696, 468, 521, 2323, 186, 517, - /* 570 */ 513, 2157, 2318, 644, 2193, 1832, 284, 169, 2159, 700, - /* 580 */ 2161, 2162, 695, 12, 690, 1923, 41, 40, 207, 206, - /* 590 */ 47, 45, 44, 43, 42, 596, 2157, 1612, 1613, 2193, - /* 600 */ 2126, 605, 110, 2159, 700, 2161, 2162, 695, 594, 690, - /* 610 */ 592, 488, 143, 1684, 150, 2217, 2246, 619, 2283, 1554, - /* 620 */ 395, 2242, 179, 1554, 294, 295, 1529, 1585, 1595, 293, - /* 630 */ 639, 634, 627, 1611, 1614, 2261, 187, 2254, 2255, 1527, - /* 640 */ 138, 2259, 366, 2031, 582, 581, 580, 1530, 165, 1528, - /* 650 */ 249, 572, 137, 576, 248, 62, 1933, 575, 1777, 723, - /* 660 */ 1620, 2257, 574, 579, 374, 373, 1554, 2158, 573, 730, - /* 670 */ 155, 154, 727, 726, 725, 152, 260, 658, 1535, 1533, - /* 680 */ 1534, 1696, 1584, 1587, 1588, 1589, 1590, 1591, 1592, 1593, - /* 690 */ 1594, 692, 688, 1603, 1604, 1606, 1607, 1608, 1609, 2, - /* 700 */ 48, 46, 1615, 783, 2176, 62, 1535, 2126, 399, 1294, - /* 710 */ 1529, 1218, 62, 1217, 90, 2158, 2126, 353, 696, 91, - /* 720 */ 378, 1610, 598, 1527, 34, 697, 233, 1801, 680, 1930, - /* 730 */ 41, 40, 680, 1930, 47, 45, 44, 43, 42, 9, - /* 740 */ 655, 140, 1776, 2261, 1219, 285, 742, 57, 1296, 1605, - /* 750 */ 2157, 449, 2176, 2193, 680, 1930, 110, 2159, 700, 2161, - /* 760 */ 2162, 695, 1535, 690, 2126, 2119, 696, 1586, 183, 2256, - /* 770 */ 2246, 36, 1775, 450, 395, 2242, 1538, 41, 40, 408, - /* 780 */ 407, 47, 45, 44, 43, 42, 62, 783, 93, 1543, - /* 790 */ 49, 2126, 1530, 1906, 1528, 2273, 153, 443, 2157, 442, - /* 800 */ 1610, 2193, 1536, 2158, 110, 2159, 700, 2161, 2162, 695, - /* 810 */ 189, 690, 618, 697, 1586, 2317, 2337, 570, 2246, 680, - /* 820 */ 1930, 2126, 395, 2242, 1533, 1534, 1612, 1613, 1605, 441, - /* 830 */ 2323, 186, 680, 1930, 1919, 2318, 644, 402, 459, 1292, - /* 840 */ 2176, 1535, 282, 2254, 654, 165, 134, 653, 12, 2317, - /* 850 */ 10, 474, 2126, 1932, 696, 56, 1585, 1595, 1774, 732, - /* 860 */ 189, 1773, 1611, 1614, 642, 186, 686, 189, 1982, 2318, - /* 870 */ 644, 680, 1930, 578, 577, 403, 1530, 2136, 1528, 2322, - /* 880 */ 1915, 2322, 2317, 1980, 2317, 1772, 2157, 680, 1930, 2193, - /* 890 */ 475, 2145, 170, 2159, 700, 2161, 2162, 695, 2321, 690, - /* 900 */ 2321, 2140, 2318, 2320, 2318, 2319, 542, 2126, 1533, 1534, - /* 910 */ 2126, 1584, 1587, 1588, 1589, 1590, 1591, 1592, 1593, 1594, - /* 920 */ 692, 688, 1603, 1604, 1606, 1607, 1608, 1609, 2, 48, - /* 930 */ 46, 1541, 680, 1930, 2126, 2158, 1771, 399, 2142, 1529, - /* 940 */ 1305, 189, 2321, 645, 2338, 697, 405, 2280, 690, 1917, - /* 950 */ 1610, 1927, 1527, 1304, 165, 1544, 1673, 1539, 1770, 41, - /* 960 */ 40, 1767, 1932, 47, 45, 44, 43, 42, 317, 41, - /* 970 */ 40, 1959, 2176, 47, 45, 44, 43, 42, 1605, 1554, - /* 980 */ 680, 1930, 1976, 1977, 2126, 2126, 696, 1547, 1549, 680, - /* 990 */ 1930, 1535, 680, 1930, 680, 1930, 1309, 2109, 2120, 252, - /* 1000 */ 688, 1603, 1604, 1606, 1607, 1608, 1609, 2126, 614, 1308, - /* 1010 */ 2126, 659, 2027, 663, 1556, 1913, 783, 498, 2157, 49, - /* 1020 */ 1218, 2193, 1217, 2158, 110, 2159, 700, 2161, 2162, 695, - /* 1030 */ 250, 690, 587, 697, 724, 2293, 2337, 1973, 2246, 680, - /* 1040 */ 1930, 728, 395, 2242, 1973, 618, 1641, 597, 2317, 1766, - /* 1050 */ 203, 680, 1930, 1219, 433, 1612, 1613, 199, 298, 1765, - /* 1060 */ 2176, 247, 1764, 2323, 186, 680, 1930, 1557, 2318, 644, - /* 1070 */ 677, 1763, 2126, 500, 696, 680, 1930, 590, 680, 1930, - /* 1080 */ 1557, 435, 431, 584, 678, 1585, 1595, 754, 752, 246, - /* 1090 */ 84, 1611, 1614, 83, 304, 1200, 1201, 406, 2126, 1500, - /* 1100 */ 1501, 44, 43, 42, 1203, 1530, 2157, 1528, 2126, 2193, - /* 1110 */ 1553, 2126, 110, 2159, 700, 2161, 2162, 695, 1762, 690, - /* 1120 */ 2126, 164, 603, 35, 2337, 1761, 2246, 1586, 1760, 70, - /* 1130 */ 395, 2242, 69, 1646, 1759, 2266, 1673, 1533, 1534, 2027, - /* 1140 */ 1584, 1587, 1588, 1589, 1590, 1591, 1592, 1593, 1594, 692, - /* 1150 */ 688, 1603, 1604, 1606, 1607, 1608, 1609, 2, 48, 46, - /* 1160 */ 2027, 2158, 2136, 682, 284, 2218, 399, 2126, 1529, 618, - /* 1170 */ 2112, 697, 2317, 625, 2126, 743, 2144, 2126, 1892, 1610, - /* 1180 */ 684, 1527, 2218, 2126, 201, 74, 2140, 2323, 186, 452, - /* 1190 */ 617, 729, 2318, 644, 1973, 54, 1908, 3, 2176, 146, - /* 1200 */ 239, 135, 453, 237, 1816, 205, 241, 1605, 153, 240, - /* 1210 */ 2126, 428, 696, 571, 1808, 243, 646, 200, 242, 421, - /* 1220 */ 1535, 1806, 245, 2142, 396, 244, 583, 600, 153, 599, - /* 1230 */ 261, 50, 50, 690, 82, 1290, 585, 266, 153, 647, - /* 1240 */ 148, 1751, 1752, 588, 2157, 783, 50, 2193, 15, 687, - /* 1250 */ 110, 2159, 700, 2161, 2162, 695, 650, 690, 2147, 291, - /* 1260 */ 71, 1802, 2337, 1537, 2246, 106, 151, 1495, 395, 2242, - /* 1270 */ 153, 14, 13, 64, 50, 103, 50, 736, 737, 704, - /* 1280 */ 151, 153, 136, 151, 1612, 1613, 1934, 1498, 1251, 256, - /* 1290 */ 1705, 1704, 691, 259, 1872, 1769, 268, 662, 2286, 1270, - /* 1300 */ 1268, 279, 631, 166, 232, 1453, 273, 1870, 323, 1869, - /* 1310 */ 2177, 778, 55, 2149, 1585, 1595, 412, 2036, 296, 672, - /* 1320 */ 1611, 1614, 321, 73, 1793, 300, 72, 1252, 1798, 1335, - /* 1330 */ 2276, 1970, 1647, 1596, 1530, 316, 1528, 346, 1362, 1366, - /* 1340 */ 1373, 1371, 156, 656, 278, 281, 1, 214, 508, 506, - /* 1350 */ 503, 5, 415, 420, 362, 2158, 1560, 437, 436, 194, - /* 1360 */ 193, 196, 439, 1476, 204, 697, 1533, 1534, 311, 1584, - /* 1370 */ 1587, 1588, 1589, 1590, 1591, 1592, 1593, 1594, 692, 688, - /* 1380 */ 1603, 1604, 1606, 1607, 1608, 1609, 2, 62, 1557, 454, - /* 1390 */ 2037, 491, 2176, 458, 463, 1552, 476, 483, 2029, 490, - /* 1400 */ 492, 501, 1631, 502, 2126, 499, 696, 504, 209, 211, - /* 1410 */ 208, 505, 507, 509, 1558, 524, 535, 4, 1540, 532, - /* 1420 */ 525, 533, 219, 1555, 2158, 109, 536, 1559, 221, 537, - /* 1430 */ 1561, 538, 540, 224, 697, 544, 2311, 226, 2157, 85, - /* 1440 */ 86, 2193, 1221, 561, 110, 2159, 700, 2161, 2162, 695, - /* 1450 */ 230, 690, 563, 562, 352, 112, 2221, 2100, 2246, 602, - /* 1460 */ 604, 2176, 395, 2242, 565, 81, 80, 448, 1920, 89, - /* 1470 */ 198, 236, 1916, 2126, 648, 696, 149, 238, 312, 158, - /* 1480 */ 159, 1918, 1914, 440, 438, 160, 2158, 253, 161, 608, - /* 1490 */ 607, 257, 1483, 651, 347, 609, 697, 429, 2265, 615, - /* 1500 */ 427, 423, 419, 416, 441, 2097, 2096, 2157, 2277, 612, - /* 1510 */ 2193, 255, 2158, 110, 2159, 700, 2161, 2162, 695, 2287, - /* 1520 */ 690, 632, 697, 2176, 622, 2337, 2292, 2246, 670, 613, - /* 1530 */ 264, 395, 2242, 2291, 628, 2126, 384, 696, 267, 2268, - /* 1540 */ 8, 635, 189, 641, 623, 385, 621, 620, 277, 2176, - /* 1550 */ 2340, 649, 652, 139, 1673, 1556, 2262, 660, 388, 286, - /* 1560 */ 274, 2126, 1562, 696, 313, 2042, 96, 668, 669, 2157, - /* 1570 */ 2056, 314, 2193, 2055, 276, 110, 2159, 700, 2161, 2162, - /* 1580 */ 695, 673, 690, 275, 2054, 272, 2158, 2337, 173, 2246, - /* 1590 */ 391, 674, 61, 395, 2242, 2157, 697, 98, 2193, 2316, - /* 1600 */ 100, 110, 2159, 700, 2161, 2162, 695, 280, 690, 102, - /* 1610 */ 2227, 315, 702, 2219, 1974, 2246, 1893, 1931, 779, 395, - /* 1620 */ 2242, 318, 780, 2176, 782, 327, 53, 341, 307, 331, - /* 1630 */ 320, 2118, 322, 2117, 342, 2126, 2116, 696, 78, 2113, - /* 1640 */ 417, 418, 1520, 1521, 354, 355, 192, 2158, 2111, 422, - /* 1650 */ 424, 425, 426, 2110, 363, 2108, 430, 697, 434, 432, - /* 1660 */ 2106, 1511, 2087, 195, 2086, 197, 1479, 79, 1478, 2157, - /* 1670 */ 2107, 2068, 2193, 2158, 2067, 110, 2159, 700, 2161, 2162, - /* 1680 */ 695, 2066, 690, 697, 2176, 446, 447, 683, 2065, 2246, - /* 1690 */ 2064, 1430, 2020, 395, 2242, 2019, 2126, 2017, 696, 145, - /* 1700 */ 2016, 2015, 2018, 2014, 2013, 2011, 2010, 2009, 202, 2158, - /* 1710 */ 2176, 464, 2008, 466, 2022, 2007, 2006, 2005, 2004, 697, - /* 1720 */ 147, 1992, 2126, 2003, 696, 2002, 2001, 2000, 1999, 1998, - /* 1730 */ 2157, 1997, 1996, 2193, 1995, 1994, 111, 2159, 700, 2161, - /* 1740 */ 2162, 695, 1993, 690, 1991, 1990, 2176, 2021, 1989, 1988, - /* 1750 */ 2246, 1987, 1432, 1986, 2245, 2242, 2157, 1985, 2126, 2193, - /* 1760 */ 696, 494, 111, 2159, 700, 2161, 2162, 695, 1984, 690, - /* 1770 */ 1983, 1835, 1306, 2158, 1310, 1302, 2246, 1834, 1833, 1831, - /* 1780 */ 685, 2242, 225, 697, 350, 351, 1828, 511, 1827, 1820, - /* 1790 */ 510, 1810, 698, 514, 210, 2193, 518, 2158, 111, 2159, - /* 1800 */ 700, 2161, 2162, 695, 212, 690, 512, 697, 515, 519, - /* 1810 */ 2176, 516, 2246, 520, 1788, 213, 357, 2242, 215, 522, - /* 1820 */ 76, 1202, 2126, 1787, 696, 2085, 77, 2146, 2075, 180, - /* 1830 */ 2063, 2062, 2040, 217, 2176, 223, 1909, 181, 1830, 530, - /* 1840 */ 1826, 1244, 547, 545, 546, 1824, 2126, 549, 696, 550, - /* 1850 */ 551, 1822, 553, 554, 555, 1819, 2157, 557, 558, 2193, - /* 1860 */ 559, 2158, 111, 2159, 700, 2161, 2162, 695, 1805, 690, - /* 1870 */ 1804, 697, 1784, 1911, 63, 235, 2246, 1378, 1377, 1910, - /* 1880 */ 2157, 2243, 606, 2193, 1293, 1291, 169, 2159, 700, 2161, - /* 1890 */ 2162, 695, 751, 690, 1289, 1288, 1287, 1286, 2176, 1285, - /* 1900 */ 786, 1282, 753, 382, 1281, 1280, 1279, 1817, 375, 1809, - /* 1910 */ 2126, 376, 696, 1807, 310, 377, 586, 589, 1783, 2158, - /* 1920 */ 591, 1782, 593, 1781, 595, 113, 1505, 2284, 29, 697, - /* 1930 */ 178, 1507, 1504, 2084, 58, 2158, 1509, 776, 772, 768, - /* 1940 */ 764, 67, 308, 1485, 2157, 697, 1487, 2193, 2074, 610, - /* 1950 */ 340, 2159, 700, 2161, 2162, 695, 2176, 690, 2061, 2059, - /* 1960 */ 1489, 383, 6, 2322, 611, 31, 163, 624, 2126, 258, - /* 1970 */ 696, 381, 2176, 20, 17, 616, 7, 1721, 263, 21, - /* 1980 */ 22, 265, 108, 1703, 2126, 301, 696, 1695, 626, 271, - /* 1990 */ 171, 270, 65, 2147, 269, 33, 2158, 32, 24, 1736, - /* 2000 */ 92, 1735, 2157, 1741, 1742, 2193, 697, 23, 340, 2159, - /* 2010 */ 700, 2161, 2162, 695, 386, 690, 1740, 676, 2157, 1739, - /* 2020 */ 387, 2193, 2158, 18, 333, 2159, 700, 2161, 2162, 695, - /* 2030 */ 283, 690, 697, 2176, 1670, 1669, 60, 176, 59, 2060, - /* 2040 */ 2058, 2057, 2039, 94, 95, 2126, 289, 696, 671, 2038, - /* 2050 */ 97, 25, 288, 290, 302, 1701, 103, 287, 2158, 2176, - /* 2060 */ 26, 11, 177, 292, 390, 297, 13, 640, 694, 68, - /* 2070 */ 299, 2126, 99, 696, 1622, 1621, 1545, 254, 2196, 2157, - /* 2080 */ 1600, 1598, 2193, 1597, 689, 170, 2159, 700, 2161, 2162, - /* 2090 */ 695, 190, 690, 39, 2158, 2176, 16, 27, 1577, 1569, - /* 2100 */ 28, 701, 703, 1363, 697, 2157, 401, 2126, 2193, 696, - /* 2110 */ 705, 340, 2159, 700, 2161, 2162, 695, 1360, 690, 707, - /* 2120 */ 2158, 708, 710, 1632, 1357, 711, 699, 1351, 713, 714, - /* 2130 */ 697, 2176, 1349, 716, 717, 104, 398, 2339, 305, 105, - /* 2140 */ 1372, 2157, 1355, 2126, 2193, 696, 1354, 339, 2159, 700, - /* 2150 */ 2161, 2162, 695, 75, 690, 1353, 2212, 2176, 1352, 1368, - /* 2160 */ 1242, 1274, 400, 731, 1273, 1272, 1271, 1269, 1267, 2126, - /* 2170 */ 1266, 696, 306, 1300, 1265, 741, 1263, 2157, 1262, 1261, - /* 2180 */ 2193, 1260, 1259, 340, 2159, 700, 2161, 2162, 695, 1258, - /* 2190 */ 690, 1257, 1297, 1295, 1248, 2158, 1254, 1253, 1250, 1249, - /* 2200 */ 1247, 1825, 761, 2157, 762, 697, 2193, 763, 1823, 340, - /* 2210 */ 2159, 700, 2161, 2162, 695, 2158, 690, 765, 766, 767, - /* 2220 */ 1821, 1818, 769, 770, 771, 697, 773, 774, 775, 1803, - /* 2230 */ 777, 1192, 2176, 1780, 309, 781, 1755, 1531, 319, 784, - /* 2240 */ 1755, 1755, 785, 1755, 2126, 1755, 696, 1755, 1755, 1755, - /* 2250 */ 1755, 1755, 2176, 1755, 1755, 1755, 1755, 1755, 1755, 1755, - /* 2260 */ 1755, 1755, 1755, 1755, 2126, 1755, 696, 1755, 1755, 1755, - /* 2270 */ 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 601, 1755, - /* 2280 */ 1755, 2193, 1755, 2158, 335, 2159, 700, 2161, 2162, 695, - /* 2290 */ 1755, 690, 1755, 697, 1755, 1755, 1755, 1755, 2157, 1755, - /* 2300 */ 2158, 2193, 1755, 1755, 324, 2159, 700, 2161, 2162, 695, - /* 2310 */ 697, 690, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, - /* 2320 */ 2176, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, - /* 2330 */ 1755, 1755, 2126, 1755, 696, 1755, 1755, 2176, 1755, 1755, - /* 2340 */ 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 2158, 2126, - /* 2350 */ 1755, 696, 1755, 1755, 1755, 1755, 1755, 1755, 697, 1755, - /* 2360 */ 1755, 1755, 1755, 1755, 1755, 1755, 2157, 1755, 2158, 2193, - /* 2370 */ 1755, 1755, 325, 2159, 700, 2161, 2162, 695, 697, 690, - /* 2380 */ 1755, 1755, 1755, 2157, 1755, 2176, 2193, 1755, 1755, 326, - /* 2390 */ 2159, 700, 2161, 2162, 695, 1755, 690, 2126, 2158, 696, - /* 2400 */ 1755, 1755, 1755, 1755, 1755, 2176, 1755, 1755, 697, 1755, - /* 2410 */ 1755, 1755, 1755, 1755, 1755, 1755, 2158, 2126, 1755, 696, - /* 2420 */ 1755, 1755, 1755, 1755, 1755, 1755, 697, 1755, 1755, 1755, - /* 2430 */ 1755, 2157, 1755, 1755, 2193, 2176, 1755, 332, 2159, 700, - /* 2440 */ 2161, 2162, 695, 1755, 690, 1755, 2158, 2126, 1755, 696, - /* 2450 */ 1755, 2157, 1755, 2176, 2193, 1755, 697, 336, 2159, 700, - /* 2460 */ 2161, 2162, 695, 1755, 690, 2126, 1755, 696, 1755, 1755, - /* 2470 */ 1755, 1755, 1755, 1755, 1755, 1755, 2158, 1755, 1755, 1755, - /* 2480 */ 1755, 2157, 1755, 2176, 2193, 1755, 697, 328, 2159, 700, - /* 2490 */ 2161, 2162, 695, 1755, 690, 2126, 1755, 696, 1755, 2157, - /* 2500 */ 1755, 1755, 2193, 1755, 1755, 337, 2159, 700, 2161, 2162, - /* 2510 */ 695, 1755, 690, 2176, 1755, 1755, 1755, 1755, 1755, 1755, - /* 2520 */ 1755, 1755, 1755, 1755, 2158, 2126, 1755, 696, 1755, 2157, - /* 2530 */ 1755, 1755, 2193, 1755, 697, 329, 2159, 700, 2161, 2162, - /* 2540 */ 695, 1755, 690, 1755, 2158, 1755, 1755, 1755, 1755, 1755, - /* 2550 */ 1755, 1755, 1755, 1755, 697, 1755, 1755, 1755, 1755, 2157, - /* 2560 */ 1755, 2176, 2193, 1755, 1755, 338, 2159, 700, 2161, 2162, - /* 2570 */ 695, 1755, 690, 2126, 1755, 696, 1755, 1755, 1755, 1755, - /* 2580 */ 1755, 2176, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, - /* 2590 */ 1755, 1755, 1755, 2126, 1755, 696, 1755, 1755, 1755, 1755, - /* 2600 */ 1755, 1755, 1755, 1755, 1755, 1755, 1755, 2157, 1755, 1755, - /* 2610 */ 2193, 1755, 1755, 330, 2159, 700, 2161, 2162, 695, 1755, - /* 2620 */ 690, 1755, 2158, 1755, 1755, 1755, 1755, 2157, 1755, 1755, - /* 2630 */ 2193, 1755, 697, 343, 2159, 700, 2161, 2162, 695, 2158, - /* 2640 */ 690, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 697, - /* 2650 */ 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 2176, - /* 2660 */ 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, - /* 2670 */ 1755, 2126, 1755, 696, 1755, 1755, 2176, 1755, 1755, 1755, - /* 2680 */ 1755, 1755, 1755, 1755, 1755, 1755, 1755, 2158, 2126, 1755, - /* 2690 */ 696, 1755, 1755, 1755, 1755, 1755, 1755, 697, 1755, 1755, - /* 2700 */ 1755, 1755, 1755, 1755, 1755, 2157, 1755, 2158, 2193, 1755, - /* 2710 */ 1755, 344, 2159, 700, 2161, 2162, 695, 697, 690, 1755, - /* 2720 */ 1755, 1755, 2157, 1755, 2176, 2193, 1755, 1755, 2170, 2159, - /* 2730 */ 700, 2161, 2162, 695, 1755, 690, 2126, 2158, 696, 1755, - /* 2740 */ 1755, 1755, 1755, 1755, 2176, 1755, 1755, 697, 1755, 1755, - /* 2750 */ 1755, 1755, 1755, 1755, 1755, 2158, 2126, 1755, 696, 1755, - /* 2760 */ 1755, 1755, 1755, 1755, 1755, 697, 1755, 1755, 1755, 1755, - /* 2770 */ 2157, 1755, 1755, 2193, 2176, 1755, 2169, 2159, 700, 2161, - /* 2780 */ 2162, 695, 1755, 690, 1755, 2158, 2126, 1755, 696, 1755, - /* 2790 */ 2157, 1755, 2176, 2193, 1755, 697, 2168, 2159, 700, 2161, - /* 2800 */ 2162, 695, 1755, 690, 2126, 1755, 696, 1755, 1755, 1755, - /* 2810 */ 1755, 1755, 1755, 1755, 1755, 2158, 1755, 1755, 1755, 1755, - /* 2820 */ 2157, 1755, 2176, 2193, 1755, 697, 359, 2159, 700, 2161, - /* 2830 */ 2162, 695, 1755, 690, 2126, 1755, 696, 1755, 2157, 1755, - /* 2840 */ 1755, 2193, 1755, 1755, 360, 2159, 700, 2161, 2162, 695, - /* 2850 */ 1755, 690, 2176, 1755, 1755, 1755, 1755, 1755, 1755, 1755, - /* 2860 */ 1755, 1755, 1755, 2158, 2126, 1755, 696, 1755, 2157, 1755, - /* 2870 */ 1755, 2193, 1755, 697, 356, 2159, 700, 2161, 2162, 695, - /* 2880 */ 1755, 690, 1755, 2158, 1755, 1755, 1755, 1755, 1755, 1755, - /* 2890 */ 1755, 1755, 1755, 697, 1755, 1755, 1755, 1755, 2157, 1755, - /* 2900 */ 2176, 2193, 1755, 1755, 361, 2159, 700, 2161, 2162, 695, - /* 2910 */ 1755, 690, 2126, 1755, 696, 1755, 1755, 1755, 1755, 1755, - /* 2920 */ 2176, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, - /* 2930 */ 1755, 1755, 2126, 1755, 696, 1755, 1755, 1755, 1755, 1755, - /* 2940 */ 1755, 1755, 1755, 1755, 1755, 1755, 698, 1755, 1755, 2193, - /* 2950 */ 1755, 1755, 335, 2159, 700, 2161, 2162, 695, 1755, 690, - /* 2960 */ 1755, 1755, 1755, 1755, 1755, 1755, 2157, 1755, 1755, 2193, - /* 2970 */ 1755, 1755, 334, 2159, 700, 2161, 2162, 695, 1755, 690, + /* 0 */ 2167, 445, 2145, 2057, 404, 444, 144, 1984, 1986, 2226, + /* 10 */ 696, 451, 48, 46, 1686, 1763, 2153, 218, 2055, 669, + /* 20 */ 399, 529, 1535, 1805, 41, 40, 2149, 639, 47, 45, + /* 30 */ 44, 43, 42, 1616, 1838, 1533, 123, 1560, 2185, 122, + /* 40 */ 121, 120, 119, 118, 117, 116, 115, 114, 620, 681, + /* 50 */ 2135, 2326, 698, 41, 40, 2185, 345, 47, 45, 44, + /* 60 */ 43, 42, 1611, 2151, 396, 649, 2332, 188, 19, 1560, + /* 70 */ 2057, 2327, 646, 692, 681, 1541, 47, 45, 44, 43, + /* 80 */ 42, 170, 392, 1774, 2166, 2054, 669, 2202, 1561, 30, + /* 90 */ 339, 2168, 702, 2170, 2171, 697, 695, 692, 683, 2220, + /* 100 */ 785, 66, 2118, 15, 762, 761, 760, 759, 411, 638, + /* 110 */ 758, 757, 146, 752, 751, 750, 749, 748, 747, 746, + /* 120 */ 159, 742, 741, 740, 410, 409, 737, 736, 735, 177, + /* 130 */ 176, 2100, 51, 526, 2270, 1356, 527, 1798, 123, 1618, + /* 140 */ 1619, 122, 121, 120, 119, 118, 117, 116, 115, 114, + /* 150 */ 1347, 724, 723, 722, 1351, 721, 1353, 1354, 720, 717, + /* 160 */ 2267, 1362, 714, 1364, 1365, 711, 708, 38, 303, 1591, + /* 170 */ 1601, 2331, 181, 1991, 2326, 1617, 1620, 285, 251, 62, + /* 180 */ 364, 1760, 667, 657, 142, 1762, 1445, 1446, 1989, 1536, + /* 190 */ 2330, 1534, 366, 2040, 2327, 2329, 543, 1755, 41, 40, + /* 200 */ 62, 1562, 47, 45, 44, 43, 42, 1390, 1391, 132, + /* 210 */ 131, 130, 129, 128, 127, 126, 125, 124, 62, 640, + /* 220 */ 93, 1539, 1540, 181, 1590, 1593, 1594, 1595, 1596, 1597, + /* 230 */ 1598, 1599, 1600, 694, 690, 1609, 1610, 1612, 1613, 1614, + /* 240 */ 1615, 2, 48, 46, 2041, 2167, 1223, 349, 1222, 1558, + /* 250 */ 399, 645, 1535, 414, 2326, 660, 482, 413, 358, 496, + /* 260 */ 393, 138, 495, 1616, 443, 1533, 442, 62, 167, 644, + /* 270 */ 188, 408, 407, 402, 2327, 646, 1941, 1561, 465, 1224, + /* 280 */ 497, 167, 681, 2185, 1560, 467, 282, 2263, 656, 1941, + /* 290 */ 134, 655, 1611, 2326, 1542, 2135, 441, 698, 19, 1754, + /* 300 */ 620, 650, 1645, 2326, 523, 1541, 9, 534, 644, 188, + /* 310 */ 527, 1798, 521, 2327, 646, 517, 513, 428, 2332, 188, + /* 320 */ 1985, 1986, 2331, 2327, 646, 2326, 44, 43, 42, 2166, + /* 330 */ 785, 367, 2202, 15, 191, 110, 2168, 702, 2170, 2171, + /* 340 */ 697, 2330, 692, 455, 1991, 2327, 2328, 185, 12, 2255, + /* 350 */ 10, 379, 284, 395, 2251, 191, 41, 40, 1646, 1989, + /* 360 */ 47, 45, 44, 43, 42, 51, 190, 62, 1563, 1618, + /* 370 */ 1619, 1198, 493, 191, 2281, 487, 486, 485, 484, 481, + /* 380 */ 480, 479, 478, 477, 473, 472, 471, 470, 348, 462, + /* 390 */ 461, 460, 531, 457, 456, 365, 1462, 1463, 528, 1591, + /* 400 */ 1601, 657, 142, 682, 1939, 1617, 1620, 584, 583, 582, + /* 410 */ 1200, 452, 1203, 1204, 574, 139, 578, 371, 489, 1536, + /* 420 */ 577, 1534, 191, 133, 453, 576, 581, 374, 373, 734, + /* 430 */ 566, 575, 1461, 1464, 598, 37, 397, 1640, 1641, 1642, + /* 440 */ 1643, 1644, 1648, 1649, 1650, 1651, 2021, 596, 52, 594, + /* 450 */ 1545, 1539, 1540, 1559, 1590, 1593, 1594, 1595, 1596, 1597, + /* 460 */ 1598, 1599, 1600, 694, 690, 1609, 1610, 1612, 1613, 1614, + /* 470 */ 1615, 2, 12, 48, 46, 682, 1939, 1991, 209, 208, + /* 480 */ 635, 399, 726, 1535, 389, 1982, 372, 668, 370, 369, + /* 490 */ 2167, 568, 1989, 1785, 1616, 133, 1533, 1714, 682, 1939, + /* 500 */ 699, 488, 571, 659, 186, 2263, 2264, 220, 140, 2268, + /* 510 */ 2331, 529, 570, 1805, 645, 2167, 569, 2326, 193, 262, + /* 520 */ 191, 1915, 191, 1611, 1679, 699, 663, 1562, 2185, 19, + /* 530 */ 1226, 1227, 644, 188, 682, 1939, 1541, 2327, 646, 541, + /* 540 */ 2135, 2050, 698, 2135, 632, 631, 1712, 1713, 1715, 1716, + /* 550 */ 1717, 41, 40, 2185, 57, 47, 45, 44, 43, 42, + /* 560 */ 87, 785, 249, 184, 15, 2135, 248, 698, 1841, 641, + /* 570 */ 636, 629, 87, 620, 2166, 1978, 2326, 2202, 1784, 607, + /* 580 */ 171, 2168, 702, 2170, 2171, 697, 1935, 692, 734, 368, + /* 590 */ 668, 2332, 188, 12, 682, 1939, 2327, 646, 1934, 2166, + /* 600 */ 1618, 1619, 2202, 191, 469, 110, 2168, 702, 2170, 2171, + /* 610 */ 697, 1748, 692, 468, 449, 145, 107, 152, 2226, 2255, + /* 620 */ 621, 2292, 1783, 395, 2251, 2167, 90, 668, 2135, 353, + /* 630 */ 1591, 1601, 378, 143, 600, 660, 1617, 1620, 584, 583, + /* 640 */ 582, 1931, 666, 433, 2050, 574, 139, 578, 562, 561, + /* 650 */ 1536, 577, 1534, 682, 1939, 1541, 576, 581, 374, 373, + /* 660 */ 41, 40, 575, 2185, 47, 45, 44, 43, 42, 1991, + /* 670 */ 435, 431, 2135, 450, 2156, 2135, 394, 698, 284, 677, + /* 680 */ 1647, 2050, 1539, 1540, 1989, 1590, 1593, 1594, 1595, 1596, + /* 690 */ 1597, 1598, 1599, 1600, 694, 690, 1609, 1610, 1612, 1613, + /* 700 */ 1614, 1615, 2, 48, 46, 1621, 684, 744, 2227, 2166, + /* 710 */ 250, 399, 2202, 1535, 166, 110, 2168, 702, 2170, 2171, + /* 720 */ 697, 1991, 692, 402, 1616, 2167, 1533, 185, 403, 2255, + /* 730 */ 2158, 164, 1311, 395, 2251, 699, 1989, 1807, 380, 1941, + /* 740 */ 589, 1916, 1563, 41, 40, 1310, 1989, 47, 45, 44, + /* 750 */ 43, 42, 405, 1611, 2282, 599, 1725, 35, 682, 1939, + /* 760 */ 167, 2167, 169, 2185, 682, 1939, 1541, 1652, 1941, 247, + /* 770 */ 1880, 699, 725, 2289, 1300, 2135, 34, 698, 459, 1506, + /* 780 */ 1507, 1928, 41, 40, 474, 592, 47, 45, 44, 43, + /* 790 */ 42, 785, 586, 686, 49, 2227, 1782, 2167, 246, 2185, + /* 800 */ 732, 157, 156, 729, 728, 727, 154, 699, 1315, 2166, + /* 810 */ 498, 2135, 2202, 698, 1302, 110, 2168, 702, 2170, 2171, + /* 820 */ 697, 1314, 692, 682, 1939, 564, 563, 2346, 1914, 2255, + /* 830 */ 1618, 1619, 167, 395, 2251, 2185, 580, 579, 70, 1702, + /* 840 */ 1942, 69, 572, 475, 1991, 2166, 2135, 2135, 2202, 698, + /* 850 */ 1781, 110, 2168, 702, 2170, 2171, 697, 2270, 692, 1990, + /* 860 */ 1591, 1601, 1690, 2346, 1298, 2255, 1617, 1620, 1560, 395, + /* 870 */ 2251, 14, 13, 732, 157, 156, 729, 728, 727, 154, + /* 880 */ 1536, 2166, 1534, 2266, 2202, 1780, 500, 110, 2168, 702, + /* 890 */ 2170, 2171, 697, 202, 692, 657, 142, 36, 1779, 2230, + /* 900 */ 2135, 2255, 1778, 41, 40, 395, 2251, 47, 45, 44, + /* 910 */ 43, 42, 1539, 1540, 1924, 1590, 1593, 1594, 1595, 1596, + /* 920 */ 1597, 1598, 1599, 1600, 694, 690, 1609, 1610, 1612, 1613, + /* 930 */ 1614, 1615, 2, 48, 46, 2135, 1223, 155, 1222, 2167, + /* 940 */ 1535, 399, 2036, 1535, 1777, 2128, 657, 142, 2135, 699, + /* 950 */ 2330, 2302, 2135, 1533, 1616, 1776, 1533, 1773, 682, 1939, + /* 960 */ 732, 157, 156, 729, 728, 727, 154, 294, 295, 1224, + /* 970 */ 682, 1939, 293, 730, 1659, 2167, 1982, 2185, 542, 205, + /* 980 */ 1683, 682, 1939, 1611, 101, 699, 2270, 627, 201, 2135, + /* 990 */ 1936, 698, 620, 1541, 2135, 2326, 1541, 56, 187, 2263, + /* 1000 */ 2264, 252, 140, 2268, 1926, 2135, 1772, 2135, 1771, 1932, + /* 1010 */ 2332, 188, 2265, 2185, 1206, 2327, 646, 1592, 785, 84, + /* 1020 */ 1559, 785, 83, 2166, 49, 2135, 2202, 698, 2129, 110, + /* 1030 */ 2168, 702, 2170, 2171, 697, 2167, 692, 682, 1939, 756, + /* 1040 */ 754, 2346, 1770, 2255, 54, 699, 3, 395, 2251, 189, + /* 1050 */ 2263, 2264, 1563, 140, 2268, 1769, 2135, 616, 2135, 2166, + /* 1060 */ 1618, 1619, 2202, 1203, 1204, 110, 2168, 702, 2170, 2171, + /* 1070 */ 697, 731, 692, 2185, 1982, 620, 1626, 2346, 2326, 2255, + /* 1080 */ 682, 1939, 1560, 395, 2251, 2135, 1768, 698, 682, 1939, + /* 1090 */ 1591, 1601, 2135, 2332, 188, 2121, 1617, 1620, 2327, 646, + /* 1100 */ 661, 682, 1939, 682, 1939, 2135, 1560, 1536, 665, 1534, + /* 1110 */ 1536, 652, 1534, 682, 1939, 1767, 1766, 2145, 259, 2166, + /* 1120 */ 1765, 406, 2202, 298, 1922, 172, 2168, 702, 2170, 2171, + /* 1130 */ 697, 1930, 692, 679, 2036, 605, 2135, 2036, 1943, 1539, + /* 1140 */ 1540, 2149, 1539, 1540, 421, 1590, 1593, 1594, 1595, 1596, + /* 1150 */ 1597, 1598, 1599, 1600, 694, 690, 1609, 1610, 1612, 1613, + /* 1160 */ 1614, 1615, 2, 48, 46, 2135, 2135, 682, 1939, 2167, + /* 1170 */ 2135, 399, 148, 1535, 135, 2145, 647, 2347, 2151, 699, + /* 1180 */ 203, 2320, 620, 207, 1616, 2326, 1533, 680, 692, 2154, + /* 1190 */ 41, 40, 2167, 1917, 47, 45, 44, 43, 42, 2149, + /* 1200 */ 2332, 188, 699, 261, 2274, 2327, 646, 2185, 317, 682, + /* 1210 */ 1939, 1968, 570, 1611, 745, 155, 569, 1901, 74, 2135, + /* 1220 */ 155, 698, 2275, 1679, 239, 648, 1541, 237, 573, 304, + /* 1230 */ 2185, 1592, 602, 50, 601, 1682, 2151, 150, 241, 50, + /* 1240 */ 243, 240, 2135, 242, 698, 245, 692, 1825, 244, 689, + /* 1250 */ 1296, 785, 1816, 2166, 15, 1592, 2202, 266, 155, 110, + /* 1260 */ 2168, 702, 2170, 2171, 697, 260, 692, 82, 256, 585, + /* 1270 */ 1814, 2346, 1544, 2255, 587, 1501, 2166, 395, 2251, 2202, + /* 1280 */ 1504, 619, 110, 2168, 702, 2170, 2171, 697, 50, 692, + /* 1290 */ 1618, 1619, 590, 1711, 2346, 1881, 2255, 1757, 1758, 1710, + /* 1300 */ 395, 2251, 291, 71, 153, 155, 14, 13, 91, 168, + /* 1310 */ 64, 50, 106, 1543, 323, 693, 633, 268, 664, 1257, + /* 1320 */ 1591, 1601, 103, 738, 1808, 739, 1617, 1620, 321, 73, + /* 1330 */ 1775, 2295, 72, 50, 279, 137, 273, 706, 153, 155, + /* 1340 */ 1536, 136, 1534, 346, 1879, 1276, 153, 1274, 1459, 653, + /* 1350 */ 1878, 2186, 2045, 412, 216, 508, 506, 503, 1799, 1258, + /* 1360 */ 1979, 1804, 296, 674, 300, 1341, 2285, 658, 278, 281, + /* 1370 */ 1653, 1602, 1539, 1540, 780, 1590, 1593, 1594, 1595, 1596, + /* 1380 */ 1597, 1598, 1599, 1600, 694, 690, 1609, 1610, 1612, 1613, + /* 1390 */ 1614, 1615, 2, 316, 62, 408, 407, 1368, 1372, 1379, + /* 1400 */ 1, 1377, 1637, 5, 55, 1549, 158, 415, 420, 1566, + /* 1410 */ 362, 437, 436, 195, 196, 439, 1616, 234, 1542, 1482, + /* 1420 */ 198, 206, 454, 311, 1563, 458, 1558, 2046, 1547, 463, + /* 1430 */ 2167, 491, 109, 174, 483, 476, 2038, 490, 492, 501, + /* 1440 */ 699, 560, 556, 552, 548, 1611, 233, 499, 211, 504, + /* 1450 */ 502, 213, 210, 505, 507, 509, 1564, 524, 1541, 4, + /* 1460 */ 525, 532, 535, 533, 1561, 221, 536, 223, 2185, 1546, + /* 1470 */ 1565, 537, 81, 80, 448, 1567, 226, 200, 538, 540, + /* 1480 */ 2135, 228, 698, 688, 85, 86, 88, 544, 565, 231, + /* 1490 */ 440, 438, 2167, 232, 112, 604, 352, 606, 2109, 567, + /* 1500 */ 1929, 347, 699, 236, 429, 89, 610, 427, 423, 419, + /* 1510 */ 416, 441, 1925, 611, 2166, 151, 312, 2202, 238, 160, + /* 1520 */ 110, 2168, 702, 2170, 2171, 697, 161, 692, 253, 1927, + /* 1530 */ 2185, 1923, 2228, 162, 2255, 609, 163, 2106, 395, 2251, + /* 1540 */ 255, 614, 2135, 257, 698, 1489, 2105, 617, 615, 191, + /* 1550 */ 634, 624, 672, 630, 2301, 8, 230, 224, 384, 2167, + /* 1560 */ 2286, 229, 2296, 539, 264, 2300, 637, 643, 272, 699, + /* 1570 */ 267, 654, 1550, 625, 1545, 274, 2166, 175, 2277, 2202, + /* 1580 */ 623, 222, 110, 2168, 702, 2170, 2171, 697, 622, 692, + /* 1590 */ 277, 385, 651, 276, 685, 141, 2255, 2185, 1562, 275, + /* 1600 */ 395, 2251, 1679, 2271, 1553, 1555, 388, 1568, 286, 2135, + /* 1610 */ 662, 698, 96, 2051, 313, 670, 280, 690, 1609, 1610, + /* 1620 */ 1612, 1613, 1614, 1615, 2167, 671, 314, 2349, 2065, 2064, + /* 1630 */ 2063, 391, 675, 61, 699, 676, 98, 315, 2236, 102, + /* 1640 */ 1983, 781, 1902, 2166, 2325, 342, 2202, 100, 1940, 111, + /* 1650 */ 2168, 702, 2170, 2171, 697, 704, 692, 318, 307, 782, + /* 1660 */ 2167, 784, 2185, 2255, 320, 53, 322, 2254, 2251, 327, + /* 1670 */ 699, 341, 331, 354, 2135, 2127, 698, 355, 2126, 2125, + /* 1680 */ 78, 2122, 417, 418, 1526, 1527, 194, 422, 2120, 424, + /* 1690 */ 425, 426, 2119, 363, 2117, 430, 2167, 434, 2185, 2116, + /* 1700 */ 2115, 1517, 432, 2096, 197, 2095, 699, 199, 2166, 79, + /* 1710 */ 2135, 2202, 698, 1485, 111, 2168, 702, 2170, 2171, 697, + /* 1720 */ 2167, 692, 1484, 2077, 2076, 2075, 446, 447, 2255, 2074, + /* 1730 */ 699, 2073, 687, 2251, 2185, 2029, 1436, 2028, 2026, 2025, + /* 1740 */ 147, 2024, 2027, 2023, 700, 2022, 2135, 2202, 698, 2020, + /* 1750 */ 111, 2168, 702, 2170, 2171, 697, 2019, 692, 2185, 2018, + /* 1760 */ 204, 464, 2017, 2031, 2255, 466, 2016, 2015, 357, 2251, + /* 1770 */ 2135, 2014, 698, 2013, 2012, 2011, 2010, 2009, 2008, 2007, + /* 1780 */ 2166, 2006, 2005, 2202, 2004, 2167, 111, 2168, 702, 2170, + /* 1790 */ 2171, 697, 2003, 692, 2002, 699, 149, 2001, 2000, 1999, + /* 1800 */ 2255, 2030, 1998, 1997, 2166, 2252, 1996, 2202, 1995, 1438, + /* 1810 */ 171, 2168, 702, 2170, 2171, 697, 1994, 692, 494, 1993, + /* 1820 */ 1992, 1844, 1312, 2185, 1308, 1843, 1316, 212, 382, 1842, + /* 1830 */ 214, 350, 215, 351, 1840, 2135, 1837, 698, 1836, 511, + /* 1840 */ 515, 1829, 510, 1818, 519, 514, 1794, 2167, 512, 518, + /* 1850 */ 516, 2293, 520, 522, 217, 1793, 76, 699, 1205, 182, + /* 1860 */ 2094, 2084, 219, 2167, 2072, 77, 227, 225, 2071, 2166, + /* 1870 */ 2155, 183, 2202, 699, 530, 340, 2168, 702, 2170, 2171, + /* 1880 */ 697, 2049, 692, 1918, 1839, 2185, 1835, 545, 546, 547, + /* 1890 */ 383, 549, 551, 1831, 550, 1828, 554, 2135, 1833, 698, + /* 1900 */ 1250, 2185, 553, 558, 555, 557, 1813, 1811, 559, 1812, + /* 1910 */ 1810, 1790, 1920, 2135, 2167, 698, 1383, 1384, 1919, 753, + /* 1920 */ 1299, 755, 1297, 1295, 699, 1294, 1286, 1293, 1826, 588, + /* 1930 */ 1815, 2166, 1292, 1817, 2202, 1291, 2167, 340, 2168, 702, + /* 1940 */ 2170, 2171, 697, 1288, 692, 63, 699, 2166, 1287, 1285, + /* 1950 */ 2202, 375, 2185, 333, 2168, 702, 2170, 2171, 697, 235, + /* 1960 */ 692, 591, 376, 377, 2135, 1789, 698, 593, 1788, 595, + /* 1970 */ 1787, 597, 113, 1511, 2185, 1515, 29, 1513, 1510, 390, + /* 1980 */ 2093, 1491, 67, 1493, 1495, 2083, 2135, 2331, 698, 612, + /* 1990 */ 2070, 2068, 20, 1727, 17, 6, 642, 21, 2166, 31, + /* 2000 */ 2167, 2202, 263, 608, 172, 2168, 702, 2170, 2171, 697, + /* 2010 */ 696, 692, 7, 58, 23, 618, 626, 22, 628, 613, + /* 2020 */ 2166, 788, 165, 2202, 258, 265, 340, 2168, 702, 2170, + /* 2030 */ 2171, 697, 270, 692, 381, 310, 2167, 33, 2185, 1709, + /* 2040 */ 173, 271, 2156, 65, 24, 1742, 699, 269, 32, 1741, + /* 2050 */ 2135, 180, 698, 386, 1701, 1747, 2348, 92, 2167, 778, + /* 2060 */ 774, 770, 766, 1748, 308, 1746, 1745, 387, 699, 283, + /* 2070 */ 1676, 60, 178, 1675, 2185, 94, 95, 25, 292, 398, + /* 2080 */ 2069, 2067, 2066, 2048, 2166, 289, 2135, 2202, 698, 297, + /* 2090 */ 339, 2168, 702, 2170, 2171, 697, 2185, 692, 673, 2221, + /* 2100 */ 290, 400, 18, 1707, 108, 2047, 68, 301, 2135, 97, + /* 2110 */ 698, 99, 302, 59, 103, 26, 1628, 1627, 13, 11, + /* 2120 */ 2166, 1551, 1638, 2202, 2205, 2167, 340, 2168, 702, 2170, + /* 2130 */ 2171, 697, 299, 692, 179, 699, 192, 1606, 691, 678, + /* 2140 */ 1604, 1583, 2166, 1603, 2167, 2202, 39, 16, 340, 2168, + /* 2150 */ 702, 2170, 2171, 697, 699, 692, 27, 705, 1575, 701, + /* 2160 */ 28, 703, 1369, 2185, 401, 707, 709, 1366, 1363, 710, + /* 2170 */ 712, 713, 715, 1357, 288, 2135, 716, 698, 1355, 287, + /* 2180 */ 718, 719, 2185, 305, 1361, 1360, 1359, 104, 105, 1378, + /* 2190 */ 1374, 75, 1358, 1248, 2135, 733, 698, 1280, 1279, 254, + /* 2200 */ 1278, 1277, 1275, 1306, 1273, 1272, 1271, 743, 306, 603, + /* 2210 */ 1266, 1269, 2202, 2167, 1268, 335, 2168, 702, 2170, 2171, + /* 2220 */ 697, 1267, 692, 699, 1265, 1264, 1263, 1303, 2166, 1301, + /* 2230 */ 2167, 2202, 1260, 1259, 324, 2168, 702, 2170, 2171, 697, + /* 2240 */ 699, 692, 1256, 1255, 1254, 1253, 1834, 763, 764, 1832, + /* 2250 */ 765, 2185, 767, 768, 769, 1830, 771, 773, 772, 1827, + /* 2260 */ 775, 1809, 776, 2135, 779, 698, 777, 1195, 2185, 1786, + /* 2270 */ 309, 783, 1761, 1537, 319, 786, 787, 1761, 2167, 1761, + /* 2280 */ 2135, 1761, 698, 1761, 1761, 1761, 1761, 1761, 699, 1761, + /* 2290 */ 1761, 1761, 1761, 1761, 1761, 1761, 1761, 2166, 1761, 1761, + /* 2300 */ 2202, 1761, 1761, 325, 2168, 702, 2170, 2171, 697, 1761, + /* 2310 */ 692, 1761, 1761, 1761, 2166, 1761, 2185, 2202, 1761, 1761, + /* 2320 */ 326, 2168, 702, 2170, 2171, 697, 2167, 692, 2135, 1761, + /* 2330 */ 698, 1761, 1761, 1761, 1761, 1761, 699, 1761, 1761, 1761, + /* 2340 */ 1761, 1761, 1761, 2167, 1761, 1761, 1761, 1761, 1761, 1761, + /* 2350 */ 1761, 1761, 1761, 699, 1761, 1761, 1761, 1761, 1761, 1761, + /* 2360 */ 1761, 1761, 2166, 1761, 2185, 2202, 1761, 1761, 332, 2168, + /* 2370 */ 702, 2170, 2171, 697, 1761, 692, 2135, 1761, 698, 1761, + /* 2380 */ 1761, 2185, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, + /* 2390 */ 1761, 2167, 1761, 2135, 1761, 698, 1761, 1761, 1761, 1761, + /* 2400 */ 1761, 699, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, + /* 2410 */ 2166, 1761, 1761, 2202, 2167, 1761, 336, 2168, 702, 2170, + /* 2420 */ 2171, 697, 1761, 692, 699, 1761, 1761, 2166, 1761, 2185, + /* 2430 */ 2202, 1761, 1761, 328, 2168, 702, 2170, 2171, 697, 2167, + /* 2440 */ 692, 2135, 1761, 698, 1761, 1761, 1761, 1761, 1761, 699, + /* 2450 */ 1761, 1761, 2185, 1761, 1761, 1761, 1761, 1761, 1761, 1761, + /* 2460 */ 1761, 1761, 1761, 1761, 2135, 1761, 698, 1761, 1761, 1761, + /* 2470 */ 1761, 1761, 1761, 1761, 1761, 2166, 1761, 2185, 2202, 1761, + /* 2480 */ 1761, 337, 2168, 702, 2170, 2171, 697, 1761, 692, 2135, + /* 2490 */ 2167, 698, 1761, 1761, 1761, 1761, 1761, 1761, 2166, 1761, + /* 2500 */ 699, 2202, 1761, 1761, 329, 2168, 702, 2170, 2171, 697, + /* 2510 */ 1761, 692, 1761, 1761, 2167, 1761, 1761, 1761, 1761, 1761, + /* 2520 */ 1761, 1761, 1761, 2166, 699, 1761, 2202, 1761, 2185, 338, + /* 2530 */ 2168, 702, 2170, 2171, 697, 1761, 692, 1761, 1761, 1761, + /* 2540 */ 2135, 1761, 698, 1761, 1761, 1761, 1761, 1761, 1761, 1761, + /* 2550 */ 1761, 1761, 2185, 1761, 1761, 1761, 1761, 1761, 1761, 1761, + /* 2560 */ 1761, 1761, 2167, 1761, 2135, 1761, 698, 1761, 1761, 1761, + /* 2570 */ 1761, 1761, 699, 1761, 2166, 1761, 1761, 2202, 2167, 1761, + /* 2580 */ 330, 2168, 702, 2170, 2171, 697, 1761, 692, 699, 1761, + /* 2590 */ 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 2166, 1761, + /* 2600 */ 2185, 2202, 1761, 1761, 343, 2168, 702, 2170, 2171, 697, + /* 2610 */ 1761, 692, 2135, 1761, 698, 1761, 2185, 1761, 1761, 1761, + /* 2620 */ 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 2135, 1761, + /* 2630 */ 698, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, + /* 2640 */ 2167, 1761, 1761, 1761, 1761, 1761, 2166, 1761, 1761, 2202, + /* 2650 */ 699, 1761, 344, 2168, 702, 2170, 2171, 697, 1761, 692, + /* 2660 */ 1761, 1761, 2166, 1761, 2167, 2202, 1761, 1761, 2179, 2168, + /* 2670 */ 702, 2170, 2171, 697, 699, 692, 1761, 1761, 2185, 1761, + /* 2680 */ 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, + /* 2690 */ 2135, 1761, 698, 1761, 1761, 1761, 1761, 1761, 1761, 1761, + /* 2700 */ 1761, 1761, 2185, 1761, 1761, 1761, 1761, 1761, 1761, 1761, + /* 2710 */ 1761, 1761, 2167, 1761, 2135, 1761, 698, 1761, 1761, 1761, + /* 2720 */ 1761, 1761, 699, 1761, 2166, 1761, 1761, 2202, 1761, 1761, + /* 2730 */ 2178, 2168, 702, 2170, 2171, 697, 1761, 692, 1761, 1761, + /* 2740 */ 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 2166, 1761, + /* 2750 */ 2185, 2202, 1761, 1761, 2177, 2168, 702, 2170, 2171, 697, + /* 2760 */ 2167, 692, 2135, 1761, 698, 1761, 1761, 1761, 1761, 1761, + /* 2770 */ 699, 1761, 1761, 1761, 1761, 2167, 1761, 1761, 1761, 1761, + /* 2780 */ 1761, 1761, 1761, 1761, 1761, 699, 1761, 1761, 1761, 1761, + /* 2790 */ 1761, 1761, 1761, 1761, 1761, 1761, 2166, 1761, 2185, 2202, + /* 2800 */ 1761, 1761, 359, 2168, 702, 2170, 2171, 697, 1761, 692, + /* 2810 */ 2135, 1761, 698, 2185, 1761, 1761, 1761, 1761, 1761, 1761, + /* 2820 */ 1761, 1761, 1761, 2167, 1761, 2135, 1761, 698, 1761, 1761, + /* 2830 */ 1761, 1761, 1761, 699, 1761, 1761, 1761, 1761, 2167, 1761, + /* 2840 */ 1761, 1761, 1761, 1761, 2166, 1761, 1761, 2202, 699, 1761, + /* 2850 */ 360, 2168, 702, 2170, 2171, 697, 1761, 692, 1761, 2166, + /* 2860 */ 1761, 2185, 2202, 1761, 1761, 356, 2168, 702, 2170, 2171, + /* 2870 */ 697, 2167, 692, 2135, 1761, 698, 2185, 1761, 1761, 1761, + /* 2880 */ 1761, 699, 1761, 1761, 1761, 1761, 1761, 1761, 2135, 1761, + /* 2890 */ 698, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, + /* 2900 */ 1761, 1761, 1761, 1761, 1761, 1761, 1761, 2166, 1761, 2185, + /* 2910 */ 2202, 1761, 1761, 361, 2168, 702, 2170, 2171, 697, 1761, + /* 2920 */ 692, 2135, 700, 698, 1761, 2202, 1761, 1761, 335, 2168, + /* 2930 */ 702, 2170, 2171, 697, 1761, 692, 1761, 1761, 1761, 1761, + /* 2940 */ 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, + /* 2950 */ 1761, 1761, 1761, 1761, 1761, 2166, 1761, 1761, 2202, 1761, + /* 2960 */ 1761, 334, 2168, 702, 2170, 2171, 697, 1761, 692, }; static const YYCODETYPE yy_lookahead[] = { - /* 0 */ 338, 367, 347, 348, 442, 443, 374, 371, 453, 375, - /* 10 */ 348, 456, 12, 13, 14, 0, 382, 383, 386, 343, - /* 20 */ 20, 366, 22, 347, 390, 349, 471, 472, 373, 347, - /* 30 */ 348, 476, 477, 33, 0, 35, 21, 375, 353, 24, - /* 40 */ 25, 26, 27, 28, 29, 30, 31, 32, 20, 387, - /* 50 */ 0, 389, 8, 9, 418, 370, 12, 13, 14, 15, - /* 60 */ 16, 61, 14, 378, 20, 8, 9, 67, 20, 12, - /* 70 */ 13, 14, 15, 16, 74, 0, 20, 12, 13, 14, - /* 80 */ 15, 16, 437, 421, 342, 440, 424, 345, 346, 427, - /* 90 */ 428, 429, 430, 431, 432, 433, 434, 435, 436, 99, - /* 100 */ 356, 44, 102, 69, 70, 71, 72, 73, 364, 75, + /* 0 */ 339, 408, 365, 391, 387, 412, 439, 390, 391, 442, + /* 10 */ 349, 348, 12, 13, 14, 0, 379, 344, 406, 407, + /* 20 */ 20, 348, 22, 350, 8, 9, 389, 349, 12, 13, + /* 30 */ 14, 15, 16, 33, 0, 35, 21, 20, 377, 24, + /* 40 */ 25, 26, 27, 28, 29, 30, 31, 32, 455, 20, + /* 50 */ 389, 458, 391, 8, 9, 377, 393, 12, 13, 14, + /* 60 */ 15, 16, 62, 426, 427, 44, 473, 474, 68, 20, + /* 70 */ 391, 478, 479, 436, 20, 75, 12, 13, 14, 15, + /* 80 */ 16, 338, 403, 340, 423, 406, 407, 426, 20, 44, + /* 90 */ 429, 430, 431, 432, 433, 434, 435, 436, 437, 438, + /* 100 */ 100, 4, 0, 103, 70, 71, 72, 73, 74, 431, /* 110 */ 76, 77, 78, 79, 80, 81, 82, 83, 84, 85, /* 120 */ 86, 87, 88, 89, 90, 91, 92, 93, 94, 95, - /* 130 */ 448, 449, 450, 451, 99, 453, 454, 21, 138, 139, - /* 140 */ 24, 25, 26, 27, 28, 29, 30, 31, 32, 114, - /* 150 */ 115, 116, 117, 118, 119, 120, 121, 122, 123, 20, - /* 160 */ 125, 126, 127, 128, 129, 130, 138, 139, 168, 169, - /* 170 */ 347, 348, 8, 9, 174, 175, 12, 13, 14, 15, - /* 180 */ 16, 131, 132, 133, 134, 135, 136, 137, 188, 366, - /* 190 */ 190, 102, 347, 12, 13, 131, 373, 8, 9, 135, - /* 200 */ 406, 12, 13, 14, 15, 16, 131, 132, 133, 134, - /* 210 */ 135, 136, 137, 20, 170, 342, 35, 20, 345, 346, - /* 220 */ 220, 221, 66, 223, 224, 225, 226, 227, 228, 229, + /* 130 */ 96, 373, 103, 343, 428, 100, 346, 347, 21, 139, + /* 140 */ 140, 24, 25, 26, 27, 28, 29, 30, 31, 32, + /* 150 */ 115, 116, 117, 118, 119, 120, 121, 122, 123, 124, + /* 160 */ 454, 126, 127, 128, 129, 130, 131, 444, 445, 169, + /* 170 */ 170, 455, 377, 377, 458, 175, 176, 62, 420, 103, + /* 180 */ 384, 336, 20, 348, 349, 0, 169, 170, 392, 189, + /* 190 */ 474, 191, 397, 398, 478, 479, 67, 181, 8, 9, + /* 200 */ 103, 20, 12, 13, 14, 15, 16, 139, 140, 24, + /* 210 */ 25, 26, 27, 28, 29, 30, 31, 32, 103, 20, + /* 220 */ 105, 221, 222, 377, 224, 225, 226, 227, 228, 229, /* 230 */ 230, 231, 232, 233, 234, 235, 236, 237, 238, 239, - /* 240 */ 240, 12, 13, 398, 338, 400, 18, 453, 20, 20, - /* 250 */ 456, 22, 1, 2, 348, 27, 67, 389, 30, 20, - /* 260 */ 20, 33, 33, 102, 35, 471, 472, 69, 70, 71, - /* 270 */ 476, 477, 404, 405, 76, 77, 78, 49, 363, 51, - /* 280 */ 82, 375, 33, 55, 4, 87, 88, 89, 90, 0, - /* 290 */ 61, 93, 377, 387, 375, 389, 67, 375, 49, 4, - /* 300 */ 111, 382, 387, 74, 382, 56, 57, 58, 59, 390, - /* 310 */ 61, 375, 390, 24, 25, 26, 27, 28, 29, 30, - /* 320 */ 31, 32, 337, 43, 339, 45, 46, 421, 99, 101, - /* 330 */ 424, 102, 396, 427, 428, 429, 430, 431, 432, 424, - /* 340 */ 434, 113, 102, 426, 180, 439, 257, 441, 375, 434, - /* 350 */ 101, 445, 446, 104, 103, 382, 167, 385, 406, 66, - /* 360 */ 388, 389, 410, 390, 458, 168, 169, 138, 139, 452, - /* 370 */ 142, 190, 466, 145, 146, 147, 148, 149, 150, 151, - /* 380 */ 152, 153, 154, 155, 156, 157, 158, 159, 160, 161, - /* 390 */ 343, 163, 164, 165, 347, 20, 349, 168, 169, 0, - /* 400 */ 20, 347, 348, 174, 175, 453, 8, 9, 456, 3, - /* 410 */ 12, 13, 14, 15, 16, 375, 37, 188, 257, 190, - /* 420 */ 171, 172, 382, 471, 472, 176, 20, 178, 476, 477, - /* 430 */ 390, 335, 389, 244, 245, 246, 247, 248, 249, 250, - /* 440 */ 251, 252, 253, 254, 401, 196, 282, 404, 405, 220, - /* 450 */ 221, 348, 223, 224, 225, 226, 227, 228, 229, 230, - /* 460 */ 231, 232, 233, 234, 235, 236, 237, 238, 239, 240, - /* 470 */ 241, 12, 13, 355, 355, 367, 347, 102, 375, 20, - /* 480 */ 347, 22, 338, 375, 105, 347, 107, 108, 338, 110, - /* 490 */ 372, 383, 33, 453, 35, 220, 456, 257, 348, 381, - /* 500 */ 381, 103, 406, 449, 450, 451, 410, 453, 454, 103, - /* 510 */ 131, 471, 472, 338, 135, 375, 476, 477, 138, 139, - /* 520 */ 61, 347, 348, 348, 391, 375, 67, 398, 83, 400, - /* 530 */ 390, 387, 429, 74, 347, 348, 398, 387, 400, 389, - /* 540 */ 366, 173, 267, 268, 269, 270, 271, 272, 273, 453, - /* 550 */ 375, 338, 456, 258, 174, 175, 157, 4, 99, 49, - /* 560 */ 353, 102, 387, 20, 389, 166, 56, 471, 472, 59, - /* 570 */ 60, 421, 476, 477, 424, 0, 170, 427, 428, 429, - /* 580 */ 430, 431, 432, 241, 434, 378, 8, 9, 143, 144, - /* 590 */ 12, 13, 14, 15, 16, 21, 421, 138, 139, 424, - /* 600 */ 387, 113, 427, 428, 429, 430, 431, 432, 34, 434, - /* 610 */ 36, 166, 437, 14, 439, 440, 441, 467, 468, 20, - /* 620 */ 445, 446, 375, 20, 132, 133, 22, 168, 169, 137, - /* 630 */ 262, 263, 264, 174, 175, 426, 449, 450, 451, 35, - /* 640 */ 453, 454, 395, 396, 69, 70, 71, 188, 375, 190, - /* 650 */ 133, 76, 77, 78, 137, 102, 383, 82, 338, 113, - /* 660 */ 14, 452, 87, 88, 89, 90, 20, 338, 93, 131, - /* 670 */ 132, 133, 134, 135, 136, 137, 61, 348, 74, 220, - /* 680 */ 221, 103, 223, 224, 225, 226, 227, 228, 229, 230, - /* 690 */ 231, 232, 233, 234, 235, 236, 237, 238, 239, 240, - /* 700 */ 12, 13, 14, 99, 375, 102, 74, 387, 20, 35, - /* 710 */ 22, 20, 102, 22, 197, 338, 387, 200, 389, 104, - /* 720 */ 203, 33, 205, 35, 2, 348, 35, 350, 347, 348, - /* 730 */ 8, 9, 347, 348, 12, 13, 14, 15, 16, 39, - /* 740 */ 347, 348, 338, 426, 53, 61, 74, 366, 74, 61, - /* 750 */ 421, 366, 375, 424, 347, 348, 427, 428, 429, 430, - /* 760 */ 431, 432, 74, 434, 387, 406, 389, 168, 439, 452, - /* 770 */ 441, 2, 338, 366, 445, 446, 35, 8, 9, 12, - /* 780 */ 13, 12, 13, 14, 15, 16, 102, 99, 104, 22, - /* 790 */ 102, 387, 188, 0, 190, 466, 44, 187, 421, 189, - /* 800 */ 33, 424, 35, 338, 427, 428, 429, 430, 431, 432, - /* 810 */ 257, 434, 453, 348, 168, 456, 439, 13, 441, 347, - /* 820 */ 348, 387, 445, 446, 220, 221, 138, 139, 61, 219, - /* 830 */ 471, 472, 347, 348, 376, 476, 477, 367, 366, 35, - /* 840 */ 375, 74, 449, 450, 451, 375, 453, 454, 241, 456, - /* 850 */ 243, 366, 387, 383, 389, 103, 168, 169, 338, 66, - /* 860 */ 257, 338, 174, 175, 471, 472, 99, 257, 375, 476, - /* 870 */ 477, 347, 348, 360, 361, 382, 188, 363, 190, 453, - /* 880 */ 376, 453, 456, 390, 456, 338, 421, 347, 348, 424, - /* 890 */ 366, 377, 427, 428, 429, 430, 431, 432, 472, 434, - /* 900 */ 472, 387, 476, 477, 476, 477, 366, 387, 220, 221, - /* 910 */ 387, 223, 224, 225, 226, 227, 228, 229, 230, 231, - /* 920 */ 232, 233, 234, 235, 236, 237, 238, 239, 240, 12, - /* 930 */ 13, 190, 347, 348, 387, 338, 338, 20, 424, 22, - /* 940 */ 22, 257, 3, 478, 479, 348, 367, 350, 434, 376, - /* 950 */ 33, 366, 35, 35, 375, 188, 256, 190, 338, 8, - /* 960 */ 9, 338, 383, 12, 13, 14, 15, 16, 368, 8, - /* 970 */ 9, 371, 375, 12, 13, 14, 15, 16, 61, 20, - /* 980 */ 347, 348, 388, 389, 387, 387, 389, 220, 221, 347, - /* 990 */ 348, 74, 347, 348, 347, 348, 22, 0, 406, 366, - /* 1000 */ 233, 234, 235, 236, 237, 238, 239, 387, 366, 35, - /* 1010 */ 387, 366, 348, 366, 20, 376, 99, 99, 421, 102, - /* 1020 */ 20, 424, 22, 338, 427, 428, 429, 430, 431, 432, - /* 1030 */ 132, 434, 4, 348, 384, 350, 439, 387, 441, 347, - /* 1040 */ 348, 384, 445, 446, 387, 453, 167, 19, 456, 338, - /* 1050 */ 61, 347, 348, 53, 183, 138, 139, 393, 366, 338, - /* 1060 */ 375, 33, 338, 471, 472, 347, 348, 20, 476, 477, - /* 1070 */ 366, 338, 387, 99, 389, 347, 348, 49, 347, 348, - /* 1080 */ 20, 210, 211, 55, 366, 168, 169, 360, 361, 61, - /* 1090 */ 101, 174, 175, 104, 366, 45, 46, 366, 387, 201, - /* 1100 */ 202, 14, 15, 16, 14, 188, 421, 190, 387, 424, - /* 1110 */ 20, 387, 427, 428, 429, 430, 431, 432, 338, 434, - /* 1120 */ 387, 170, 406, 244, 439, 338, 441, 168, 338, 101, - /* 1130 */ 445, 446, 104, 254, 338, 255, 256, 220, 221, 348, - /* 1140 */ 223, 224, 225, 226, 227, 228, 229, 230, 231, 232, - /* 1150 */ 233, 234, 235, 236, 237, 238, 239, 240, 12, 13, - /* 1160 */ 348, 338, 363, 438, 170, 440, 20, 387, 22, 453, - /* 1170 */ 0, 348, 456, 350, 387, 362, 377, 387, 365, 33, - /* 1180 */ 438, 35, 440, 387, 393, 113, 387, 471, 472, 22, - /* 1190 */ 48, 384, 476, 477, 387, 42, 0, 44, 375, 42, - /* 1200 */ 106, 44, 35, 109, 0, 393, 106, 61, 44, 109, - /* 1210 */ 387, 214, 389, 13, 0, 106, 277, 170, 109, 49, - /* 1220 */ 74, 0, 106, 424, 425, 109, 22, 204, 44, 206, - /* 1230 */ 170, 44, 44, 434, 162, 35, 22, 44, 44, 44, - /* 1240 */ 44, 138, 139, 22, 421, 99, 44, 424, 102, 67, - /* 1250 */ 427, 428, 429, 430, 431, 432, 44, 434, 47, 44, - /* 1260 */ 44, 0, 439, 35, 441, 102, 44, 103, 445, 446, - /* 1270 */ 44, 1, 2, 44, 44, 112, 44, 13, 13, 44, - /* 1280 */ 44, 44, 44, 44, 138, 139, 376, 103, 35, 376, - /* 1290 */ 103, 103, 376, 411, 364, 339, 103, 103, 397, 35, - /* 1300 */ 35, 480, 469, 18, 351, 103, 463, 363, 23, 363, - /* 1310 */ 375, 50, 170, 102, 168, 169, 351, 397, 103, 103, - /* 1320 */ 174, 175, 37, 38, 346, 103, 41, 74, 348, 103, - /* 1330 */ 397, 386, 103, 103, 188, 103, 190, 52, 103, 103, - /* 1340 */ 103, 103, 103, 455, 447, 473, 457, 62, 63, 64, - /* 1350 */ 65, 259, 423, 49, 422, 338, 20, 415, 203, 355, - /* 1360 */ 420, 355, 415, 186, 42, 348, 220, 221, 408, 223, - /* 1370 */ 224, 225, 226, 227, 228, 229, 230, 231, 232, 233, - /* 1380 */ 234, 235, 236, 237, 238, 239, 240, 102, 20, 394, - /* 1390 */ 397, 167, 375, 394, 392, 20, 347, 394, 347, 392, - /* 1400 */ 392, 100, 220, 359, 387, 98, 389, 97, 347, 347, - /* 1410 */ 358, 357, 347, 347, 20, 340, 415, 48, 190, 340, - /* 1420 */ 344, 344, 355, 20, 338, 140, 389, 20, 355, 349, - /* 1430 */ 20, 407, 349, 355, 348, 347, 350, 355, 421, 355, - /* 1440 */ 355, 424, 54, 352, 427, 428, 429, 430, 431, 432, - /* 1450 */ 355, 434, 340, 352, 340, 347, 439, 387, 441, 207, - /* 1460 */ 419, 375, 445, 446, 375, 180, 181, 182, 375, 102, - /* 1470 */ 185, 375, 375, 387, 279, 389, 417, 375, 415, 375, - /* 1480 */ 375, 375, 375, 198, 199, 375, 338, 353, 375, 194, - /* 1490 */ 193, 353, 192, 281, 209, 414, 348, 212, 350, 347, - /* 1500 */ 215, 216, 217, 218, 219, 387, 387, 421, 397, 389, - /* 1510 */ 424, 413, 338, 427, 428, 429, 430, 431, 432, 397, - /* 1520 */ 434, 266, 348, 375, 387, 439, 462, 441, 265, 412, - /* 1530 */ 402, 445, 446, 462, 387, 387, 387, 389, 402, 465, - /* 1540 */ 274, 387, 257, 179, 276, 283, 275, 260, 423, 375, - /* 1550 */ 481, 278, 280, 348, 256, 20, 426, 347, 349, 353, - /* 1560 */ 461, 387, 20, 389, 402, 400, 353, 387, 387, 421, - /* 1570 */ 387, 402, 424, 387, 459, 427, 428, 429, 430, 431, - /* 1580 */ 432, 172, 434, 460, 387, 464, 338, 439, 462, 441, - /* 1590 */ 387, 399, 102, 445, 446, 421, 348, 353, 424, 475, - /* 1600 */ 353, 427, 428, 429, 430, 431, 432, 474, 434, 102, - /* 1610 */ 444, 371, 379, 439, 387, 441, 365, 348, 36, 445, - /* 1620 */ 446, 347, 341, 375, 340, 369, 409, 369, 353, 369, - /* 1630 */ 354, 0, 336, 0, 416, 387, 0, 389, 42, 0, - /* 1640 */ 35, 213, 35, 35, 403, 403, 35, 338, 0, 213, - /* 1650 */ 35, 35, 213, 0, 213, 0, 35, 348, 35, 22, - /* 1660 */ 0, 208, 0, 196, 0, 196, 190, 197, 188, 421, - /* 1670 */ 0, 0, 424, 338, 0, 427, 428, 429, 430, 431, - /* 1680 */ 432, 0, 434, 348, 375, 184, 183, 439, 0, 441, - /* 1690 */ 0, 47, 0, 445, 446, 0, 387, 0, 389, 42, - /* 1700 */ 0, 0, 0, 0, 0, 0, 0, 0, 157, 338, - /* 1710 */ 375, 35, 0, 157, 0, 0, 0, 0, 0, 348, - /* 1720 */ 42, 0, 387, 0, 389, 0, 0, 0, 0, 0, - /* 1730 */ 421, 0, 0, 424, 0, 0, 427, 428, 429, 430, - /* 1740 */ 431, 432, 0, 434, 0, 0, 375, 0, 0, 0, - /* 1750 */ 441, 0, 22, 0, 445, 446, 421, 0, 387, 424, - /* 1760 */ 389, 141, 427, 428, 429, 430, 431, 432, 0, 434, - /* 1770 */ 0, 0, 22, 338, 22, 35, 441, 0, 0, 0, - /* 1780 */ 445, 446, 179, 348, 48, 48, 0, 49, 0, 0, - /* 1790 */ 35, 0, 421, 35, 61, 424, 35, 338, 427, 428, - /* 1800 */ 429, 430, 431, 432, 61, 434, 39, 348, 49, 49, - /* 1810 */ 375, 39, 441, 39, 0, 61, 445, 446, 42, 35, - /* 1820 */ 39, 14, 387, 0, 389, 0, 39, 47, 0, 44, - /* 1830 */ 0, 0, 0, 40, 375, 39, 0, 47, 0, 47, - /* 1840 */ 0, 68, 39, 35, 49, 0, 387, 35, 389, 49, - /* 1850 */ 39, 0, 35, 49, 39, 0, 421, 35, 49, 424, - /* 1860 */ 39, 338, 427, 428, 429, 430, 431, 432, 0, 434, - /* 1870 */ 0, 348, 0, 0, 111, 109, 441, 35, 22, 0, - /* 1880 */ 421, 446, 1, 424, 35, 35, 427, 428, 429, 430, - /* 1890 */ 431, 432, 44, 434, 35, 35, 35, 35, 375, 35, - /* 1900 */ 19, 35, 44, 380, 35, 22, 35, 0, 22, 0, - /* 1910 */ 387, 22, 389, 0, 33, 22, 51, 35, 0, 338, - /* 1920 */ 35, 0, 35, 0, 22, 20, 35, 468, 102, 348, - /* 1930 */ 49, 35, 35, 0, 170, 338, 103, 56, 57, 58, - /* 1940 */ 59, 102, 61, 35, 421, 348, 22, 424, 0, 22, - /* 1950 */ 427, 428, 429, 430, 431, 432, 375, 434, 0, 0, - /* 1960 */ 195, 380, 48, 3, 170, 102, 191, 100, 387, 172, - /* 1970 */ 389, 170, 375, 44, 261, 177, 48, 103, 102, 44, - /* 1980 */ 44, 103, 101, 103, 387, 104, 389, 103, 98, 47, - /* 1990 */ 102, 44, 3, 47, 102, 44, 338, 102, 44, 35, - /* 2000 */ 102, 35, 421, 103, 103, 424, 348, 261, 427, 428, - /* 2010 */ 429, 430, 431, 432, 35, 434, 35, 136, 421, 35, - /* 2020 */ 35, 424, 338, 261, 427, 428, 429, 430, 431, 432, - /* 2030 */ 47, 434, 348, 375, 103, 103, 44, 47, 255, 0, - /* 2040 */ 0, 0, 0, 102, 39, 387, 47, 389, 173, 0, - /* 2050 */ 39, 102, 171, 103, 47, 103, 112, 176, 338, 375, - /* 2060 */ 44, 242, 47, 102, 380, 102, 2, 470, 348, 102, - /* 2070 */ 171, 387, 102, 389, 100, 100, 22, 196, 102, 421, - /* 2080 */ 103, 103, 424, 103, 102, 427, 428, 429, 430, 431, - /* 2090 */ 432, 47, 434, 102, 338, 375, 102, 102, 22, 103, - /* 2100 */ 102, 113, 35, 103, 348, 421, 35, 387, 424, 389, - /* 2110 */ 102, 427, 428, 429, 430, 431, 432, 103, 434, 35, - /* 2120 */ 338, 102, 35, 220, 103, 102, 222, 103, 35, 102, - /* 2130 */ 348, 375, 103, 35, 102, 102, 380, 479, 44, 102, - /* 2140 */ 35, 421, 124, 387, 424, 389, 124, 427, 428, 429, - /* 2150 */ 430, 431, 432, 102, 434, 124, 436, 375, 124, 22, - /* 2160 */ 68, 35, 380, 67, 35, 35, 35, 35, 35, 387, - /* 2170 */ 35, 389, 44, 74, 35, 96, 35, 421, 35, 35, - /* 2180 */ 424, 22, 35, 427, 428, 429, 430, 431, 432, 35, - /* 2190 */ 434, 35, 74, 35, 22, 338, 35, 35, 35, 35, - /* 2200 */ 35, 0, 35, 421, 49, 348, 424, 39, 0, 427, - /* 2210 */ 428, 429, 430, 431, 432, 338, 434, 35, 49, 39, - /* 2220 */ 0, 0, 35, 49, 39, 348, 35, 49, 39, 0, - /* 2230 */ 35, 35, 375, 0, 22, 21, 482, 22, 22, 21, - /* 2240 */ 482, 482, 20, 482, 387, 482, 389, 482, 482, 482, - /* 2250 */ 482, 482, 375, 482, 482, 482, 482, 482, 482, 482, - /* 2260 */ 482, 482, 482, 482, 387, 482, 389, 482, 482, 482, - /* 2270 */ 482, 482, 482, 482, 482, 482, 482, 482, 421, 482, - /* 2280 */ 482, 424, 482, 338, 427, 428, 429, 430, 431, 432, - /* 2290 */ 482, 434, 482, 348, 482, 482, 482, 482, 421, 482, - /* 2300 */ 338, 424, 482, 482, 427, 428, 429, 430, 431, 432, - /* 2310 */ 348, 434, 482, 482, 482, 482, 482, 482, 482, 482, - /* 2320 */ 375, 482, 482, 482, 482, 482, 482, 482, 482, 482, - /* 2330 */ 482, 482, 387, 482, 389, 482, 482, 375, 482, 482, - /* 2340 */ 482, 482, 482, 482, 482, 482, 482, 482, 338, 387, - /* 2350 */ 482, 389, 482, 482, 482, 482, 482, 482, 348, 482, - /* 2360 */ 482, 482, 482, 482, 482, 482, 421, 482, 338, 424, - /* 2370 */ 482, 482, 427, 428, 429, 430, 431, 432, 348, 434, - /* 2380 */ 482, 482, 482, 421, 482, 375, 424, 482, 482, 427, - /* 2390 */ 428, 429, 430, 431, 432, 482, 434, 387, 338, 389, - /* 2400 */ 482, 482, 482, 482, 482, 375, 482, 482, 348, 482, - /* 2410 */ 482, 482, 482, 482, 482, 482, 338, 387, 482, 389, - /* 2420 */ 482, 482, 482, 482, 482, 482, 348, 482, 482, 482, - /* 2430 */ 482, 421, 482, 482, 424, 375, 482, 427, 428, 429, - /* 2440 */ 430, 431, 432, 482, 434, 482, 338, 387, 482, 389, - /* 2450 */ 482, 421, 482, 375, 424, 482, 348, 427, 428, 429, - /* 2460 */ 430, 431, 432, 482, 434, 387, 482, 389, 482, 482, - /* 2470 */ 482, 482, 482, 482, 482, 482, 338, 482, 482, 482, - /* 2480 */ 482, 421, 482, 375, 424, 482, 348, 427, 428, 429, - /* 2490 */ 430, 431, 432, 482, 434, 387, 482, 389, 482, 421, - /* 2500 */ 482, 482, 424, 482, 482, 427, 428, 429, 430, 431, - /* 2510 */ 432, 482, 434, 375, 482, 482, 482, 482, 482, 482, - /* 2520 */ 482, 482, 482, 482, 338, 387, 482, 389, 482, 421, - /* 2530 */ 482, 482, 424, 482, 348, 427, 428, 429, 430, 431, - /* 2540 */ 432, 482, 434, 482, 338, 482, 482, 482, 482, 482, - /* 2550 */ 482, 482, 482, 482, 348, 482, 482, 482, 482, 421, - /* 2560 */ 482, 375, 424, 482, 482, 427, 428, 429, 430, 431, - /* 2570 */ 432, 482, 434, 387, 482, 389, 482, 482, 482, 482, - /* 2580 */ 482, 375, 482, 482, 482, 482, 482, 482, 482, 482, - /* 2590 */ 482, 482, 482, 387, 482, 389, 482, 482, 482, 482, - /* 2600 */ 482, 482, 482, 482, 482, 482, 482, 421, 482, 482, - /* 2610 */ 424, 482, 482, 427, 428, 429, 430, 431, 432, 482, - /* 2620 */ 434, 482, 338, 482, 482, 482, 482, 421, 482, 482, - /* 2630 */ 424, 482, 348, 427, 428, 429, 430, 431, 432, 338, - /* 2640 */ 434, 482, 482, 482, 482, 482, 482, 482, 482, 348, - /* 2650 */ 482, 482, 482, 482, 482, 482, 482, 482, 482, 375, - /* 2660 */ 482, 482, 482, 482, 482, 482, 482, 482, 482, 482, - /* 2670 */ 482, 387, 482, 389, 482, 482, 375, 482, 482, 482, - /* 2680 */ 482, 482, 482, 482, 482, 482, 482, 338, 387, 482, - /* 2690 */ 389, 482, 482, 482, 482, 482, 482, 348, 482, 482, - /* 2700 */ 482, 482, 482, 482, 482, 421, 482, 338, 424, 482, - /* 2710 */ 482, 427, 428, 429, 430, 431, 432, 348, 434, 482, - /* 2720 */ 482, 482, 421, 482, 375, 424, 482, 482, 427, 428, - /* 2730 */ 429, 430, 431, 432, 482, 434, 387, 338, 389, 482, - /* 2740 */ 482, 482, 482, 482, 375, 482, 482, 348, 482, 482, - /* 2750 */ 482, 482, 482, 482, 482, 338, 387, 482, 389, 482, - /* 2760 */ 482, 482, 482, 482, 482, 348, 482, 482, 482, 482, - /* 2770 */ 421, 482, 482, 424, 375, 482, 427, 428, 429, 430, - /* 2780 */ 431, 432, 482, 434, 482, 338, 387, 482, 389, 482, - /* 2790 */ 421, 482, 375, 424, 482, 348, 427, 428, 429, 430, - /* 2800 */ 431, 432, 482, 434, 387, 482, 389, 482, 482, 482, - /* 2810 */ 482, 482, 482, 482, 482, 338, 482, 482, 482, 482, - /* 2820 */ 421, 482, 375, 424, 482, 348, 427, 428, 429, 430, - /* 2830 */ 431, 432, 482, 434, 387, 482, 389, 482, 421, 482, - /* 2840 */ 482, 424, 482, 482, 427, 428, 429, 430, 431, 432, - /* 2850 */ 482, 434, 375, 482, 482, 482, 482, 482, 482, 482, - /* 2860 */ 482, 482, 482, 338, 387, 482, 389, 482, 421, 482, - /* 2870 */ 482, 424, 482, 348, 427, 428, 429, 430, 431, 432, - /* 2880 */ 482, 434, 482, 338, 482, 482, 482, 482, 482, 482, - /* 2890 */ 482, 482, 482, 348, 482, 482, 482, 482, 421, 482, - /* 2900 */ 375, 424, 482, 482, 427, 428, 429, 430, 431, 432, - /* 2910 */ 482, 434, 387, 482, 389, 482, 482, 482, 482, 482, - /* 2920 */ 375, 482, 482, 482, 482, 482, 482, 482, 482, 482, - /* 2930 */ 482, 482, 387, 482, 389, 482, 482, 482, 482, 482, - /* 2940 */ 482, 482, 482, 482, 482, 482, 421, 482, 482, 424, - /* 2950 */ 482, 482, 427, 428, 429, 430, 431, 432, 482, 434, - /* 2960 */ 482, 482, 482, 482, 482, 482, 421, 482, 482, 424, - /* 2970 */ 482, 482, 427, 428, 429, 430, 431, 432, 482, 434, - /* 2980 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 2990 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3000 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3010 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3020 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3030 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3040 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3050 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3060 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3070 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3080 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3090 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3100 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3110 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3120 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3130 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3140 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3150 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3160 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3170 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3180 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3190 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3200 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3210 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3220 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3230 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3240 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3250 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3260 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3270 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3280 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3290 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3300 */ 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, - /* 3310 */ 335, 335, 335, 335, 335, + /* 240 */ 240, 241, 12, 13, 398, 339, 20, 18, 22, 20, + /* 250 */ 20, 455, 22, 408, 458, 349, 27, 412, 68, 30, + /* 260 */ 369, 35, 33, 33, 188, 35, 190, 103, 377, 473, + /* 270 */ 474, 12, 13, 369, 478, 479, 385, 20, 49, 53, + /* 280 */ 51, 377, 20, 377, 20, 56, 451, 452, 453, 385, + /* 290 */ 455, 456, 62, 458, 35, 389, 220, 391, 68, 283, + /* 300 */ 455, 280, 112, 458, 49, 75, 39, 343, 473, 474, + /* 310 */ 346, 347, 57, 478, 479, 60, 61, 215, 473, 474, + /* 320 */ 390, 391, 455, 478, 479, 458, 14, 15, 16, 423, + /* 330 */ 100, 102, 426, 103, 258, 429, 430, 431, 432, 433, + /* 340 */ 434, 474, 436, 114, 377, 478, 479, 441, 242, 443, + /* 350 */ 244, 384, 171, 447, 448, 258, 8, 9, 168, 392, + /* 360 */ 12, 13, 14, 15, 16, 103, 460, 103, 20, 139, + /* 370 */ 140, 4, 143, 258, 468, 146, 147, 148, 149, 150, + /* 380 */ 151, 152, 153, 154, 155, 156, 157, 158, 159, 160, + /* 390 */ 161, 162, 14, 164, 165, 166, 139, 140, 20, 169, + /* 400 */ 170, 348, 349, 348, 349, 175, 176, 70, 71, 72, + /* 410 */ 43, 22, 45, 46, 77, 78, 79, 37, 84, 189, + /* 420 */ 83, 191, 258, 368, 35, 88, 89, 90, 91, 67, + /* 430 */ 375, 94, 175, 176, 21, 245, 246, 247, 248, 249, + /* 440 */ 250, 251, 252, 253, 254, 255, 0, 34, 103, 36, + /* 450 */ 191, 221, 222, 20, 224, 225, 226, 227, 228, 229, + /* 460 */ 230, 231, 232, 233, 234, 235, 236, 237, 238, 239, + /* 470 */ 240, 241, 242, 12, 13, 348, 349, 377, 144, 145, + /* 480 */ 174, 20, 386, 22, 384, 389, 106, 348, 108, 109, + /* 490 */ 339, 111, 392, 339, 33, 368, 35, 221, 348, 349, + /* 500 */ 349, 167, 375, 450, 451, 452, 453, 344, 455, 456, + /* 510 */ 3, 348, 132, 350, 455, 339, 136, 458, 368, 171, + /* 520 */ 258, 0, 258, 62, 257, 349, 408, 20, 377, 68, + /* 530 */ 54, 55, 473, 474, 348, 349, 75, 478, 479, 400, + /* 540 */ 389, 402, 391, 389, 268, 269, 270, 271, 272, 273, + /* 550 */ 274, 8, 9, 377, 368, 12, 13, 14, 15, 16, + /* 560 */ 357, 100, 134, 376, 103, 389, 138, 391, 0, 263, + /* 570 */ 264, 265, 357, 455, 423, 388, 458, 426, 339, 114, + /* 580 */ 429, 430, 431, 432, 433, 434, 383, 436, 67, 374, + /* 590 */ 348, 473, 474, 242, 348, 349, 478, 479, 383, 423, + /* 600 */ 139, 140, 426, 258, 158, 429, 430, 431, 432, 433, + /* 610 */ 434, 104, 436, 167, 368, 439, 355, 441, 442, 443, + /* 620 */ 469, 470, 339, 447, 448, 339, 198, 348, 389, 201, + /* 630 */ 169, 170, 204, 372, 206, 349, 175, 176, 70, 71, + /* 640 */ 72, 380, 400, 184, 402, 77, 78, 79, 353, 354, + /* 650 */ 189, 83, 191, 348, 349, 75, 88, 89, 90, 91, + /* 660 */ 8, 9, 94, 377, 12, 13, 14, 15, 16, 377, + /* 670 */ 211, 212, 389, 368, 47, 389, 384, 391, 171, 400, + /* 680 */ 168, 402, 221, 222, 392, 224, 225, 226, 227, 228, + /* 690 */ 229, 230, 231, 232, 233, 234, 235, 236, 237, 238, + /* 700 */ 239, 240, 241, 12, 13, 14, 440, 75, 442, 423, + /* 710 */ 133, 20, 426, 22, 171, 429, 430, 431, 432, 433, + /* 720 */ 434, 377, 436, 369, 33, 339, 35, 441, 384, 443, + /* 730 */ 103, 377, 22, 447, 448, 349, 392, 351, 384, 385, + /* 740 */ 4, 0, 20, 8, 9, 35, 392, 12, 13, 14, + /* 750 */ 15, 16, 369, 62, 468, 19, 104, 245, 348, 349, + /* 760 */ 377, 339, 358, 377, 348, 349, 75, 255, 385, 33, + /* 770 */ 366, 349, 114, 351, 35, 389, 2, 391, 368, 202, + /* 780 */ 203, 378, 8, 9, 368, 49, 12, 13, 14, 15, + /* 790 */ 16, 100, 56, 440, 103, 442, 339, 339, 62, 377, + /* 800 */ 132, 133, 134, 135, 136, 137, 138, 349, 22, 423, + /* 810 */ 100, 389, 426, 391, 75, 429, 430, 431, 432, 433, + /* 820 */ 434, 35, 436, 348, 349, 353, 354, 441, 0, 443, + /* 830 */ 139, 140, 377, 447, 448, 377, 362, 363, 102, 104, + /* 840 */ 385, 105, 13, 368, 377, 423, 389, 389, 426, 391, + /* 850 */ 339, 429, 430, 431, 432, 433, 434, 428, 436, 392, + /* 860 */ 169, 170, 14, 441, 35, 443, 175, 176, 20, 447, + /* 870 */ 448, 1, 2, 132, 133, 134, 135, 136, 137, 138, + /* 880 */ 189, 423, 191, 454, 426, 339, 100, 429, 430, 431, + /* 890 */ 432, 433, 434, 171, 436, 348, 349, 2, 339, 441, + /* 900 */ 389, 443, 339, 8, 9, 447, 448, 12, 13, 14, + /* 910 */ 15, 16, 221, 222, 378, 224, 225, 226, 227, 228, + /* 920 */ 229, 230, 231, 232, 233, 234, 235, 236, 237, 238, + /* 930 */ 239, 240, 241, 12, 13, 389, 20, 44, 22, 339, + /* 940 */ 22, 20, 349, 22, 339, 408, 348, 349, 389, 349, + /* 950 */ 3, 351, 389, 35, 33, 339, 35, 339, 348, 349, + /* 960 */ 132, 133, 134, 135, 136, 137, 138, 133, 134, 53, + /* 970 */ 348, 349, 138, 386, 104, 339, 389, 377, 368, 62, + /* 980 */ 4, 348, 349, 62, 355, 349, 428, 351, 395, 389, + /* 990 */ 368, 391, 455, 75, 389, 458, 75, 104, 451, 452, + /* 1000 */ 453, 368, 455, 456, 378, 389, 339, 389, 339, 380, + /* 1010 */ 473, 474, 454, 377, 14, 478, 479, 169, 100, 102, + /* 1020 */ 20, 100, 105, 423, 103, 389, 426, 391, 408, 429, + /* 1030 */ 430, 431, 432, 433, 434, 339, 436, 348, 349, 362, + /* 1040 */ 363, 441, 339, 443, 42, 349, 44, 447, 448, 451, + /* 1050 */ 452, 453, 20, 455, 456, 339, 389, 368, 389, 423, + /* 1060 */ 139, 140, 426, 45, 46, 429, 430, 431, 432, 433, + /* 1070 */ 434, 386, 436, 377, 389, 455, 14, 441, 458, 443, + /* 1080 */ 348, 349, 20, 447, 448, 389, 339, 391, 348, 349, + /* 1090 */ 169, 170, 389, 473, 474, 0, 175, 176, 478, 479, + /* 1100 */ 368, 348, 349, 348, 349, 389, 20, 189, 368, 191, + /* 1110 */ 189, 44, 191, 348, 349, 339, 339, 365, 413, 423, + /* 1120 */ 339, 368, 426, 368, 378, 429, 430, 431, 432, 433, + /* 1130 */ 434, 379, 436, 368, 349, 408, 389, 349, 378, 221, + /* 1140 */ 222, 389, 221, 222, 49, 224, 225, 226, 227, 228, + /* 1150 */ 229, 230, 231, 232, 233, 234, 235, 236, 237, 238, + /* 1160 */ 239, 240, 241, 12, 13, 389, 389, 348, 349, 339, + /* 1170 */ 389, 20, 42, 22, 44, 365, 480, 481, 426, 349, + /* 1180 */ 395, 351, 455, 395, 33, 458, 35, 368, 436, 379, + /* 1190 */ 8, 9, 339, 0, 12, 13, 14, 15, 16, 389, + /* 1200 */ 473, 474, 349, 171, 351, 478, 479, 377, 370, 348, + /* 1210 */ 349, 373, 132, 62, 364, 44, 136, 367, 114, 389, + /* 1220 */ 44, 391, 256, 257, 107, 278, 75, 110, 13, 368, + /* 1230 */ 377, 169, 205, 44, 207, 259, 426, 44, 107, 44, + /* 1240 */ 107, 110, 389, 110, 391, 107, 436, 0, 110, 68, + /* 1250 */ 35, 100, 0, 423, 103, 169, 426, 44, 44, 429, + /* 1260 */ 430, 431, 432, 433, 434, 62, 436, 163, 378, 22, + /* 1270 */ 0, 441, 35, 443, 22, 104, 423, 447, 448, 426, + /* 1280 */ 104, 48, 429, 430, 431, 432, 433, 434, 44, 436, + /* 1290 */ 139, 140, 22, 104, 441, 366, 443, 139, 140, 104, + /* 1300 */ 447, 448, 44, 44, 44, 44, 1, 2, 105, 18, + /* 1310 */ 44, 44, 103, 35, 23, 378, 471, 104, 104, 35, + /* 1320 */ 169, 170, 113, 13, 0, 13, 175, 176, 37, 38, + /* 1330 */ 340, 399, 41, 44, 482, 352, 465, 44, 44, 44, + /* 1340 */ 189, 44, 191, 52, 365, 35, 44, 35, 104, 282, + /* 1350 */ 365, 377, 399, 352, 63, 64, 65, 66, 347, 75, + /* 1360 */ 388, 349, 104, 104, 104, 104, 399, 457, 449, 475, + /* 1370 */ 104, 104, 221, 222, 50, 224, 225, 226, 227, 228, + /* 1380 */ 229, 230, 231, 232, 233, 234, 235, 236, 237, 238, + /* 1390 */ 239, 240, 241, 104, 103, 12, 13, 104, 104, 104, + /* 1400 */ 459, 104, 221, 260, 171, 22, 104, 425, 49, 20, + /* 1410 */ 424, 417, 204, 422, 357, 417, 33, 33, 35, 187, + /* 1420 */ 357, 42, 396, 410, 20, 396, 20, 399, 191, 394, + /* 1430 */ 339, 168, 141, 49, 396, 348, 348, 394, 394, 101, + /* 1440 */ 349, 57, 58, 59, 60, 62, 62, 99, 348, 98, + /* 1450 */ 361, 348, 360, 359, 348, 348, 20, 341, 75, 48, + /* 1460 */ 345, 341, 417, 345, 20, 357, 391, 357, 377, 191, + /* 1470 */ 20, 350, 181, 182, 183, 20, 357, 186, 409, 350, + /* 1480 */ 389, 357, 391, 100, 357, 357, 102, 348, 341, 105, + /* 1490 */ 199, 200, 339, 357, 348, 208, 341, 421, 389, 377, + /* 1500 */ 377, 210, 349, 377, 213, 103, 195, 216, 217, 218, + /* 1510 */ 219, 220, 377, 416, 423, 419, 417, 426, 377, 377, + /* 1520 */ 429, 430, 431, 432, 433, 434, 377, 436, 355, 377, + /* 1530 */ 377, 377, 441, 377, 443, 194, 377, 389, 447, 448, + /* 1540 */ 415, 391, 389, 355, 391, 193, 389, 348, 414, 258, + /* 1550 */ 267, 389, 266, 389, 464, 275, 172, 173, 389, 339, + /* 1560 */ 399, 177, 399, 179, 404, 464, 389, 180, 466, 349, + /* 1570 */ 404, 281, 189, 277, 191, 463, 423, 464, 467, 426, + /* 1580 */ 276, 197, 429, 430, 431, 432, 433, 434, 261, 436, + /* 1590 */ 425, 284, 279, 461, 441, 349, 443, 377, 20, 462, + /* 1600 */ 447, 448, 257, 428, 221, 222, 350, 20, 355, 389, + /* 1610 */ 348, 391, 355, 402, 404, 389, 476, 234, 235, 236, + /* 1620 */ 237, 238, 239, 240, 339, 389, 404, 483, 389, 389, + /* 1630 */ 389, 389, 173, 103, 349, 401, 355, 373, 446, 103, + /* 1640 */ 389, 36, 367, 423, 477, 418, 426, 355, 349, 429, + /* 1650 */ 430, 431, 432, 433, 434, 381, 436, 348, 355, 342, + /* 1660 */ 339, 341, 377, 443, 356, 411, 337, 447, 448, 371, + /* 1670 */ 349, 371, 371, 405, 389, 0, 391, 405, 0, 0, + /* 1680 */ 42, 0, 35, 214, 35, 35, 35, 214, 0, 35, + /* 1690 */ 35, 214, 0, 214, 0, 35, 339, 35, 377, 0, + /* 1700 */ 0, 209, 22, 0, 197, 0, 349, 197, 423, 198, + /* 1710 */ 389, 426, 391, 191, 429, 430, 431, 432, 433, 434, + /* 1720 */ 339, 436, 189, 0, 0, 0, 185, 184, 443, 0, + /* 1730 */ 349, 0, 447, 448, 377, 0, 47, 0, 0, 0, + /* 1740 */ 42, 0, 0, 0, 423, 0, 389, 426, 391, 0, + /* 1750 */ 429, 430, 431, 432, 433, 434, 0, 436, 377, 0, + /* 1760 */ 158, 35, 0, 0, 443, 158, 0, 0, 447, 448, + /* 1770 */ 389, 0, 391, 0, 0, 0, 0, 0, 0, 0, + /* 1780 */ 423, 0, 0, 426, 0, 339, 429, 430, 431, 432, + /* 1790 */ 433, 434, 0, 436, 0, 349, 42, 0, 0, 0, + /* 1800 */ 443, 0, 0, 0, 423, 448, 0, 426, 0, 22, + /* 1810 */ 429, 430, 431, 432, 433, 434, 0, 436, 142, 0, + /* 1820 */ 0, 0, 22, 377, 35, 0, 22, 62, 382, 0, + /* 1830 */ 62, 48, 62, 48, 0, 389, 0, 391, 0, 49, + /* 1840 */ 49, 0, 35, 0, 49, 35, 0, 339, 39, 35, + /* 1850 */ 39, 470, 39, 35, 42, 0, 39, 349, 14, 44, + /* 1860 */ 0, 0, 40, 339, 0, 39, 180, 39, 0, 423, + /* 1870 */ 47, 47, 426, 349, 47, 429, 430, 431, 432, 433, + /* 1880 */ 434, 0, 436, 0, 0, 377, 0, 35, 49, 39, + /* 1890 */ 382, 35, 39, 0, 49, 0, 49, 389, 0, 391, + /* 1900 */ 69, 377, 35, 49, 39, 35, 0, 0, 39, 0, + /* 1910 */ 0, 0, 0, 389, 339, 391, 22, 35, 0, 44, + /* 1920 */ 35, 44, 35, 35, 349, 35, 22, 35, 0, 51, + /* 1930 */ 0, 423, 35, 0, 426, 35, 339, 429, 430, 431, + /* 1940 */ 432, 433, 434, 35, 436, 112, 349, 423, 35, 35, + /* 1950 */ 426, 22, 377, 429, 430, 431, 432, 433, 434, 110, + /* 1960 */ 436, 35, 22, 22, 389, 0, 391, 35, 0, 35, + /* 1970 */ 0, 22, 20, 35, 377, 104, 103, 35, 35, 382, + /* 1980 */ 0, 35, 103, 22, 196, 0, 389, 3, 391, 22, + /* 1990 */ 0, 0, 44, 104, 262, 48, 472, 44, 423, 103, + /* 2000 */ 339, 426, 103, 1, 429, 430, 431, 432, 433, 434, + /* 2010 */ 349, 436, 48, 171, 262, 178, 101, 44, 99, 171, + /* 2020 */ 423, 19, 192, 426, 173, 104, 429, 430, 431, 432, + /* 2030 */ 433, 434, 44, 436, 171, 33, 339, 44, 377, 104, + /* 2040 */ 103, 47, 47, 3, 44, 35, 349, 103, 103, 35, + /* 2050 */ 389, 49, 391, 35, 104, 104, 481, 103, 339, 57, + /* 2060 */ 58, 59, 60, 104, 62, 35, 35, 35, 349, 47, + /* 2070 */ 104, 44, 47, 104, 377, 103, 39, 103, 103, 382, + /* 2080 */ 0, 0, 0, 0, 423, 47, 389, 426, 391, 103, + /* 2090 */ 429, 430, 431, 432, 433, 434, 377, 436, 174, 438, + /* 2100 */ 104, 382, 262, 104, 102, 0, 103, 105, 389, 39, + /* 2110 */ 391, 103, 47, 256, 113, 44, 101, 101, 2, 243, + /* 2120 */ 423, 22, 221, 426, 103, 339, 429, 430, 431, 432, + /* 2130 */ 433, 434, 172, 436, 47, 349, 47, 104, 103, 137, + /* 2140 */ 104, 22, 423, 104, 339, 426, 103, 103, 429, 430, + /* 2150 */ 431, 432, 433, 434, 349, 436, 103, 35, 104, 223, + /* 2160 */ 103, 114, 104, 377, 35, 103, 35, 104, 104, 103, + /* 2170 */ 35, 103, 35, 104, 172, 389, 103, 391, 104, 177, + /* 2180 */ 35, 103, 377, 44, 125, 125, 125, 103, 103, 35, + /* 2190 */ 22, 103, 125, 69, 389, 68, 391, 35, 35, 197, + /* 2200 */ 35, 35, 35, 75, 35, 35, 35, 97, 44, 423, + /* 2210 */ 22, 35, 426, 339, 35, 429, 430, 431, 432, 433, + /* 2220 */ 434, 35, 436, 349, 35, 35, 35, 75, 423, 35, + /* 2230 */ 339, 426, 35, 35, 429, 430, 431, 432, 433, 434, + /* 2240 */ 349, 436, 35, 35, 22, 35, 0, 35, 49, 0, + /* 2250 */ 39, 377, 35, 49, 39, 0, 35, 39, 49, 0, + /* 2260 */ 35, 0, 49, 389, 35, 391, 39, 35, 377, 0, + /* 2270 */ 22, 21, 484, 22, 22, 21, 20, 484, 339, 484, + /* 2280 */ 389, 484, 391, 484, 484, 484, 484, 484, 349, 484, + /* 2290 */ 484, 484, 484, 484, 484, 484, 484, 423, 484, 484, + /* 2300 */ 426, 484, 484, 429, 430, 431, 432, 433, 434, 484, + /* 2310 */ 436, 484, 484, 484, 423, 484, 377, 426, 484, 484, + /* 2320 */ 429, 430, 431, 432, 433, 434, 339, 436, 389, 484, + /* 2330 */ 391, 484, 484, 484, 484, 484, 349, 484, 484, 484, + /* 2340 */ 484, 484, 484, 339, 484, 484, 484, 484, 484, 484, + /* 2350 */ 484, 484, 484, 349, 484, 484, 484, 484, 484, 484, + /* 2360 */ 484, 484, 423, 484, 377, 426, 484, 484, 429, 430, + /* 2370 */ 431, 432, 433, 434, 484, 436, 389, 484, 391, 484, + /* 2380 */ 484, 377, 484, 484, 484, 484, 484, 484, 484, 484, + /* 2390 */ 484, 339, 484, 389, 484, 391, 484, 484, 484, 484, + /* 2400 */ 484, 349, 484, 484, 484, 484, 484, 484, 484, 484, + /* 2410 */ 423, 484, 484, 426, 339, 484, 429, 430, 431, 432, + /* 2420 */ 433, 434, 484, 436, 349, 484, 484, 423, 484, 377, + /* 2430 */ 426, 484, 484, 429, 430, 431, 432, 433, 434, 339, + /* 2440 */ 436, 389, 484, 391, 484, 484, 484, 484, 484, 349, + /* 2450 */ 484, 484, 377, 484, 484, 484, 484, 484, 484, 484, + /* 2460 */ 484, 484, 484, 484, 389, 484, 391, 484, 484, 484, + /* 2470 */ 484, 484, 484, 484, 484, 423, 484, 377, 426, 484, + /* 2480 */ 484, 429, 430, 431, 432, 433, 434, 484, 436, 389, + /* 2490 */ 339, 391, 484, 484, 484, 484, 484, 484, 423, 484, + /* 2500 */ 349, 426, 484, 484, 429, 430, 431, 432, 433, 434, + /* 2510 */ 484, 436, 484, 484, 339, 484, 484, 484, 484, 484, + /* 2520 */ 484, 484, 484, 423, 349, 484, 426, 484, 377, 429, + /* 2530 */ 430, 431, 432, 433, 434, 484, 436, 484, 484, 484, + /* 2540 */ 389, 484, 391, 484, 484, 484, 484, 484, 484, 484, + /* 2550 */ 484, 484, 377, 484, 484, 484, 484, 484, 484, 484, + /* 2560 */ 484, 484, 339, 484, 389, 484, 391, 484, 484, 484, + /* 2570 */ 484, 484, 349, 484, 423, 484, 484, 426, 339, 484, + /* 2580 */ 429, 430, 431, 432, 433, 434, 484, 436, 349, 484, + /* 2590 */ 484, 484, 484, 484, 484, 484, 484, 484, 423, 484, + /* 2600 */ 377, 426, 484, 484, 429, 430, 431, 432, 433, 434, + /* 2610 */ 484, 436, 389, 484, 391, 484, 377, 484, 484, 484, + /* 2620 */ 484, 484, 484, 484, 484, 484, 484, 484, 389, 484, + /* 2630 */ 391, 484, 484, 484, 484, 484, 484, 484, 484, 484, + /* 2640 */ 339, 484, 484, 484, 484, 484, 423, 484, 484, 426, + /* 2650 */ 349, 484, 429, 430, 431, 432, 433, 434, 484, 436, + /* 2660 */ 484, 484, 423, 484, 339, 426, 484, 484, 429, 430, + /* 2670 */ 431, 432, 433, 434, 349, 436, 484, 484, 377, 484, + /* 2680 */ 484, 484, 484, 484, 484, 484, 484, 484, 484, 484, + /* 2690 */ 389, 484, 391, 484, 484, 484, 484, 484, 484, 484, + /* 2700 */ 484, 484, 377, 484, 484, 484, 484, 484, 484, 484, + /* 2710 */ 484, 484, 339, 484, 389, 484, 391, 484, 484, 484, + /* 2720 */ 484, 484, 349, 484, 423, 484, 484, 426, 484, 484, + /* 2730 */ 429, 430, 431, 432, 433, 434, 484, 436, 484, 484, + /* 2740 */ 484, 484, 484, 484, 484, 484, 484, 484, 423, 484, + /* 2750 */ 377, 426, 484, 484, 429, 430, 431, 432, 433, 434, + /* 2760 */ 339, 436, 389, 484, 391, 484, 484, 484, 484, 484, + /* 2770 */ 349, 484, 484, 484, 484, 339, 484, 484, 484, 484, + /* 2780 */ 484, 484, 484, 484, 484, 349, 484, 484, 484, 484, + /* 2790 */ 484, 484, 484, 484, 484, 484, 423, 484, 377, 426, + /* 2800 */ 484, 484, 429, 430, 431, 432, 433, 434, 484, 436, + /* 2810 */ 389, 484, 391, 377, 484, 484, 484, 484, 484, 484, + /* 2820 */ 484, 484, 484, 339, 484, 389, 484, 391, 484, 484, + /* 2830 */ 484, 484, 484, 349, 484, 484, 484, 484, 339, 484, + /* 2840 */ 484, 484, 484, 484, 423, 484, 484, 426, 349, 484, + /* 2850 */ 429, 430, 431, 432, 433, 434, 484, 436, 484, 423, + /* 2860 */ 484, 377, 426, 484, 484, 429, 430, 431, 432, 433, + /* 2870 */ 434, 339, 436, 389, 484, 391, 377, 484, 484, 484, + /* 2880 */ 484, 349, 484, 484, 484, 484, 484, 484, 389, 484, + /* 2890 */ 391, 484, 484, 484, 484, 484, 484, 484, 484, 484, + /* 2900 */ 484, 484, 484, 484, 484, 484, 484, 423, 484, 377, + /* 2910 */ 426, 484, 484, 429, 430, 431, 432, 433, 434, 484, + /* 2920 */ 436, 389, 423, 391, 484, 426, 484, 484, 429, 430, + /* 2930 */ 431, 432, 433, 434, 484, 436, 484, 484, 484, 484, + /* 2940 */ 484, 484, 484, 484, 484, 484, 484, 484, 484, 484, + /* 2950 */ 484, 484, 484, 484, 484, 423, 484, 484, 426, 484, + /* 2960 */ 484, 429, 430, 431, 432, 433, 434, 484, 436, 336, + /* 2970 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 2980 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 2990 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3000 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3010 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3020 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3030 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3040 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3050 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3060 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3070 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3080 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3090 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3100 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3110 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3120 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3130 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3140 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3150 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3160 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3170 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3180 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3190 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3200 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3210 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3220 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3230 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3240 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3250 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3260 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3270 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3280 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3290 */ 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, + /* 3300 */ 336, 336, 336, 336, 336, }; -#define YY_SHIFT_COUNT (786) +#define YY_SHIFT_COUNT (788) #define YY_SHIFT_MIN (0) -#define YY_SHIFT_MAX (2233) +#define YY_SHIFT_MAX (2269) static const unsigned short int yy_shift_ofst[] = { - /* 0 */ 1285, 0, 229, 0, 459, 459, 459, 459, 459, 459, - /* 10 */ 459, 459, 459, 459, 459, 459, 688, 917, 917, 1146, - /* 20 */ 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, - /* 30 */ 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, - /* 40 */ 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, - /* 50 */ 917, 240, 603, 610, 375, 684, 89, 161, 89, 375, - /* 60 */ 375, 767, 89, 767, 767, 553, 89, 56, 380, 139, - /* 70 */ 139, 380, 280, 280, 197, 28, 48, 48, 139, 139, - /* 80 */ 139, 139, 139, 139, 139, 193, 139, 139, 156, 56, - /* 90 */ 139, 139, 239, 139, 56, 139, 193, 139, 193, 56, - /* 100 */ 139, 139, 56, 139, 56, 56, 56, 139, 293, 228, - /* 110 */ 189, 189, 198, 116, 604, 604, 604, 604, 604, 604, - /* 120 */ 604, 604, 604, 604, 604, 604, 604, 604, 604, 604, - /* 130 */ 604, 604, 604, 379, 406, 197, 28, 674, 994, 994, - /* 140 */ 994, 793, 607, 607, 674, 543, 543, 543, 156, 488, - /* 150 */ 342, 56, 632, 56, 632, 632, 546, 672, 35, 35, - /* 160 */ 35, 35, 35, 35, 35, 35, 1881, 575, 15, 44, - /* 170 */ 164, 275, 691, 368, 181, 181, 599, 646, 1000, 1047, - /* 180 */ 1050, 1090, 64, 1060, 880, 700, 939, 880, 1153, 295, - /* 190 */ 959, 1092, 1304, 1336, 1155, 156, 1336, 156, 1177, 1322, - /* 200 */ 1368, 1322, 1224, 1375, 1375, 1322, 1224, 1224, 1301, 1307, - /* 210 */ 1375, 1310, 1375, 1375, 1375, 1394, 1369, 1394, 1369, 1336, - /* 220 */ 156, 1403, 156, 1407, 1410, 156, 1407, 156, 156, 156, - /* 230 */ 1375, 156, 1388, 1388, 1394, 56, 56, 56, 56, 56, - /* 240 */ 56, 56, 56, 56, 56, 56, 1375, 1394, 632, 632, - /* 250 */ 632, 1252, 1367, 1336, 293, 1295, 1297, 1403, 293, 1300, - /* 260 */ 1375, 1368, 1368, 632, 1255, 1263, 632, 1255, 1263, 632, - /* 270 */ 632, 56, 1266, 1364, 1255, 1268, 1271, 1287, 1092, 1262, - /* 280 */ 1272, 1273, 1298, 543, 1535, 1375, 1407, 293, 293, 1542, - /* 290 */ 1263, 632, 632, 632, 632, 632, 1263, 632, 1409, 293, - /* 300 */ 546, 293, 543, 1490, 1507, 632, 672, 1375, 293, 1582, - /* 310 */ 1394, 2980, 2980, 2980, 2980, 2980, 2980, 2980, 2980, 2980, - /* 320 */ 34, 249, 289, 1028, 398, 57, 578, 50, 722, 769, - /* 330 */ 951, 75, 961, 961, 961, 961, 961, 961, 961, 961, - /* 340 */ 961, 538, 517, 65, 65, 445, 510, 871, 399, 989, - /* 350 */ 918, 974, 574, 898, 492, 492, 1087, 251, 879, 1087, - /* 360 */ 1087, 1087, 1170, 997, 752, 1167, 1157, 1072, 1196, 1094, - /* 370 */ 1100, 1109, 1116, 804, 1200, 1204, 1214, 1221, 1023, 1164, - /* 380 */ 1184, 615, 1187, 1188, 1193, 1103, 1195, 1212, 1142, 1194, - /* 390 */ 1202, 1215, 1216, 1222, 1226, 1270, 1229, 1182, 1230, 1211, - /* 400 */ 1232, 1235, 1236, 1237, 1238, 1239, 1163, 741, 1228, 1264, - /* 410 */ 1265, 1253, 1261, 1631, 1633, 1636, 1596, 1639, 1605, 1428, - /* 420 */ 1607, 1608, 1611, 1436, 1648, 1615, 1616, 1439, 1653, 1441, - /* 430 */ 1655, 1621, 1670, 1637, 1660, 1623, 1453, 1662, 1467, 1664, - /* 440 */ 1469, 1470, 1476, 1480, 1671, 1674, 1681, 1501, 1503, 1688, - /* 450 */ 1690, 1644, 1692, 1695, 1697, 1657, 1700, 1701, 1702, 1703, - /* 460 */ 1704, 1705, 1706, 1707, 1551, 1676, 1712, 1556, 1714, 1715, - /* 470 */ 1716, 1717, 1718, 1723, 1725, 1726, 1727, 1728, 1729, 1731, - /* 480 */ 1732, 1734, 1735, 1742, 1678, 1721, 1744, 1745, 1747, 1748, - /* 490 */ 1749, 1730, 1751, 1753, 1757, 1620, 1768, 1770, 1750, 1736, - /* 500 */ 1752, 1737, 1771, 1733, 1740, 1777, 1743, 1778, 1754, 1779, - /* 510 */ 1786, 1755, 1738, 1767, 1788, 1758, 1759, 1772, 1789, 1761, - /* 520 */ 1760, 1774, 1791, 1784, 1814, 1776, 1781, 1785, 1780, 1790, - /* 530 */ 1807, 1792, 1823, 1793, 1787, 1825, 1828, 1830, 1796, 1603, - /* 540 */ 1831, 1832, 1836, 1773, 1838, 1840, 1808, 1795, 1803, 1845, - /* 550 */ 1812, 1800, 1811, 1851, 1817, 1804, 1815, 1855, 1822, 1809, - /* 560 */ 1821, 1868, 1870, 1872, 1873, 1763, 1766, 1842, 1856, 1879, - /* 570 */ 1849, 1850, 1859, 1860, 1861, 1862, 1864, 1848, 1858, 1866, - /* 580 */ 1869, 1883, 1871, 1907, 1886, 1909, 1889, 1865, 1913, 1893, - /* 590 */ 1882, 1918, 1885, 1921, 1887, 1923, 1902, 1905, 1891, 1896, - /* 600 */ 1897, 1833, 1826, 1933, 1764, 1839, 1765, 1908, 1924, 1948, - /* 610 */ 1775, 1927, 1794, 1797, 1958, 1959, 1801, 1798, 1960, 1929, - /* 620 */ 1713, 1863, 1874, 1876, 1914, 1867, 1928, 1890, 1878, 1935, - /* 630 */ 1936, 1880, 1888, 1892, 1895, 1884, 1947, 1942, 1946, 1898, - /* 640 */ 1951, 1746, 1900, 1901, 1989, 1954, 1762, 1964, 1966, 1979, - /* 650 */ 1981, 1984, 1985, 1931, 1932, 1983, 1783, 1992, 1990, 2039, - /* 660 */ 2040, 2041, 2042, 1941, 2005, 1780, 1999, 1949, 1950, 1952, - /* 670 */ 1961, 1963, 1875, 1967, 2049, 2011, 1899, 1970, 1944, 1780, - /* 680 */ 2007, 2016, 1974, 1819, 1975, 2064, 2054, 1903, 1976, 1977, - /* 690 */ 1982, 1978, 1991, 1980, 2015, 1994, 1995, 2044, 1996, 2076, - /* 700 */ 1904, 1998, 1988, 2000, 2067, 2071, 2008, 2014, 2084, 2019, - /* 710 */ 2021, 2087, 2023, 2024, 2093, 2027, 2029, 2098, 2032, 2018, - /* 720 */ 2022, 2031, 2034, 2033, 2094, 2037, 2105, 2051, 2094, 2094, - /* 730 */ 2137, 2092, 2096, 2126, 2129, 2130, 2131, 2132, 2133, 2135, - /* 740 */ 2139, 2099, 2079, 2128, 2141, 2143, 2144, 2159, 2147, 2154, - /* 750 */ 2156, 2118, 1848, 2158, 1858, 2161, 2162, 2163, 2164, 2172, - /* 760 */ 2165, 2201, 2167, 2155, 2168, 2208, 2182, 2169, 2180, 2220, - /* 770 */ 2187, 2174, 2185, 2221, 2191, 2178, 2189, 2229, 2195, 2196, - /* 780 */ 2233, 2212, 2214, 2215, 2216, 2218, 2222, + /* 0 */ 1291, 0, 230, 0, 461, 461, 461, 461, 461, 461, + /* 10 */ 461, 461, 461, 461, 461, 461, 691, 921, 921, 1151, + /* 20 */ 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, + /* 30 */ 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, + /* 40 */ 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, + /* 50 */ 921, 262, 264, 76, 29, 115, 164, 345, 164, 29, + /* 60 */ 29, 1383, 164, 1383, 1383, 97, 164, 49, 257, 54, + /* 70 */ 54, 257, 367, 367, 17, 68, 378, 378, 54, 54, + /* 80 */ 54, 54, 54, 54, 54, 162, 54, 54, 129, 49, + /* 90 */ 54, 54, 199, 54, 49, 54, 162, 54, 162, 49, + /* 100 */ 54, 54, 49, 54, 49, 49, 49, 54, 362, 229, + /* 110 */ 190, 190, 337, 117, 918, 918, 918, 918, 918, 918, + /* 120 */ 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, + /* 130 */ 918, 918, 918, 380, 507, 17, 68, 476, 476, 739, + /* 140 */ 181, 181, 181, 521, 106, 106, 739, 433, 433, 433, + /* 150 */ 129, 465, 351, 49, 580, 49, 580, 580, 658, 632, + /* 160 */ 35, 35, 35, 35, 35, 35, 35, 35, 2002, 568, + /* 170 */ 15, 348, 16, 276, 226, 306, 259, 259, 848, 1062, + /* 180 */ 916, 722, 1018, 1000, 1080, 1032, 966, 267, 947, 966, + /* 190 */ 1002, 976, 1086, 1143, 1359, 1389, 1208, 129, 1389, 129, + /* 200 */ 1232, 1379, 1404, 1379, 1263, 1406, 1406, 1379, 1263, 1263, + /* 210 */ 1338, 1348, 1406, 1351, 1406, 1406, 1406, 1436, 1411, 1436, + /* 220 */ 1411, 1389, 129, 1444, 129, 1450, 1455, 129, 1450, 129, + /* 230 */ 129, 129, 1406, 129, 1436, 49, 49, 49, 49, 49, + /* 240 */ 49, 49, 49, 49, 49, 49, 1406, 1436, 580, 580, + /* 250 */ 580, 1287, 1402, 1389, 362, 1311, 1341, 1444, 362, 1352, + /* 260 */ 1406, 1404, 1404, 580, 1283, 1286, 580, 1283, 1286, 580, + /* 270 */ 580, 49, 1280, 1387, 1283, 1296, 1304, 1327, 1143, 1307, + /* 280 */ 1290, 1313, 1345, 433, 1578, 1406, 1450, 362, 362, 1587, + /* 290 */ 1286, 580, 580, 580, 580, 580, 1286, 580, 1459, 362, + /* 300 */ 658, 362, 433, 1530, 1536, 580, 632, 1406, 362, 1605, + /* 310 */ 1436, 2969, 2969, 2969, 2969, 2969, 2969, 2969, 2969, 2969, + /* 320 */ 34, 1384, 185, 736, 652, 45, 735, 741, 774, 895, + /* 330 */ 543, 828, 1182, 1182, 1182, 1182, 1182, 1182, 1182, 1182, + /* 340 */ 1182, 668, 428, 64, 64, 334, 255, 459, 446, 917, + /* 350 */ 710, 786, 413, 577, 834, 834, 312, 870, 512, 312, + /* 360 */ 312, 312, 1095, 102, 893, 389, 1130, 1104, 1193, 1117, + /* 370 */ 1131, 1133, 1138, 829, 1215, 1247, 1252, 1270, 1027, 1171, + /* 380 */ 1176, 1203, 1189, 1195, 1213, 1158, 21, 1067, 1233, 1214, + /* 390 */ 1244, 1258, 1259, 1260, 1261, 1305, 1266, 1181, 1267, 627, + /* 400 */ 1289, 1293, 1294, 1295, 1297, 1302, 1209, 1237, 1278, 1310, + /* 410 */ 1312, 1284, 1324, 1675, 1678, 1679, 1638, 1681, 1647, 1469, + /* 420 */ 1649, 1650, 1651, 1473, 1688, 1654, 1655, 1477, 1692, 1479, + /* 430 */ 1694, 1660, 1699, 1680, 1700, 1662, 1492, 1703, 1507, 1705, + /* 440 */ 1510, 1511, 1522, 1533, 1723, 1724, 1725, 1541, 1543, 1729, + /* 450 */ 1731, 1689, 1735, 1737, 1738, 1698, 1739, 1741, 1742, 1743, + /* 460 */ 1745, 1749, 1756, 1759, 1602, 1726, 1762, 1607, 1763, 1766, + /* 470 */ 1767, 1771, 1773, 1774, 1775, 1776, 1777, 1778, 1779, 1781, + /* 480 */ 1782, 1784, 1792, 1794, 1754, 1797, 1798, 1799, 1801, 1802, + /* 490 */ 1803, 1787, 1806, 1808, 1816, 1676, 1819, 1820, 1800, 1783, + /* 500 */ 1804, 1785, 1821, 1765, 1789, 1825, 1768, 1829, 1770, 1834, + /* 510 */ 1836, 1807, 1790, 1809, 1838, 1810, 1791, 1811, 1841, 1814, + /* 520 */ 1795, 1813, 1843, 1818, 1846, 1812, 1817, 1815, 1823, 1824, + /* 530 */ 1844, 1827, 1855, 1822, 1826, 1860, 1861, 1864, 1828, 1686, + /* 540 */ 1868, 1881, 1883, 1831, 1884, 1886, 1852, 1839, 1850, 1898, + /* 550 */ 1856, 1845, 1853, 1893, 1867, 1847, 1865, 1895, 1870, 1854, + /* 560 */ 1869, 1906, 1907, 1909, 1910, 1911, 1912, 1833, 1849, 1882, + /* 570 */ 1894, 1918, 1885, 1887, 1888, 1890, 1892, 1897, 1900, 1875, + /* 580 */ 1877, 1908, 1913, 1904, 1914, 1928, 1929, 1933, 1940, 1878, + /* 590 */ 1930, 1941, 1926, 1965, 1932, 1968, 1934, 1970, 1949, 1952, + /* 600 */ 1938, 1942, 1943, 1871, 1873, 1980, 1842, 1879, 1788, 1946, + /* 610 */ 1961, 1985, 1830, 1967, 1848, 1851, 1990, 1991, 1863, 1837, + /* 620 */ 1984, 1948, 1732, 1896, 1889, 1899, 1947, 1915, 1964, 1919, + /* 630 */ 1921, 1953, 1973, 1935, 1937, 1944, 1945, 1950, 1988, 1994, + /* 640 */ 1995, 1954, 1993, 1752, 1951, 1959, 2040, 2000, 1840, 2010, + /* 650 */ 2014, 2018, 2030, 2031, 2032, 1966, 1969, 2022, 1857, 2027, + /* 660 */ 2025, 2080, 2081, 2082, 2083, 1972, 2037, 1823, 2038, 1974, + /* 670 */ 1996, 1999, 1975, 1986, 1924, 2003, 2105, 2070, 1960, 2008, + /* 680 */ 2001, 1823, 2065, 2071, 2015, 1876, 2016, 2116, 2099, 1901, + /* 690 */ 2021, 2033, 2035, 2036, 2043, 2039, 2087, 2044, 2053, 2089, + /* 700 */ 2054, 2119, 1936, 2057, 2047, 2058, 2122, 2129, 2062, 2063, + /* 710 */ 2131, 2066, 2064, 2135, 2068, 2069, 2137, 2073, 2074, 2145, + /* 720 */ 2078, 2059, 2060, 2061, 2067, 2084, 2139, 2085, 2154, 2088, + /* 730 */ 2139, 2139, 2168, 2124, 2127, 2162, 2163, 2165, 2166, 2167, + /* 740 */ 2169, 2170, 2171, 2128, 2110, 2164, 2176, 2179, 2186, 2188, + /* 750 */ 2189, 2190, 2191, 2152, 1875, 2194, 1877, 2197, 2198, 2207, + /* 760 */ 2208, 2222, 2210, 2246, 2212, 2199, 2211, 2249, 2217, 2204, + /* 770 */ 2215, 2255, 2221, 2209, 2218, 2259, 2225, 2213, 2227, 2261, + /* 780 */ 2229, 2232, 2269, 2248, 2250, 2251, 2252, 2254, 2256, }; #define YY_REDUCE_COUNT (319) -#define YY_REDUCE_MIN (-445) -#define YY_REDUCE_MAX (2545) +#define YY_REDUCE_MIN (-433) +#define YY_REDUCE_MAX (2532) static const short yy_reduce_ofst[] = { - /* 0 */ 96, -94, 175, 329, 377, 597, 685, 823, 1086, 1148, - /* 10 */ 1017, 1174, 1248, 1309, 1335, 1371, -338, 150, 465, 1435, - /* 20 */ 1459, 1523, 1581, 1597, 1658, 1684, 1720, 1756, 1782, 1857, - /* 30 */ 1877, 1945, 1962, 2010, 2030, 2060, 2078, 2108, 2138, 2186, - /* 40 */ 2206, 2284, 2301, 2349, 2369, 2399, 2417, 2447, 2477, 2525, - /* 50 */ 2545, 393, 40, -48, -318, -206, 359, 592, 716, 54, - /* 60 */ 187, 799, -445, -85, 514, 426, 428, -366, 43, -345, - /* 70 */ -177, -132, -258, -127, 247, -28, -324, 47, 174, 381, - /* 80 */ 385, 407, 472, 485, 524, -155, 540, 585, 118, -81, - /* 90 */ 633, 642, 103, 645, -78, 647, 129, 692, 138, 108, - /* 100 */ 704, 718, -27, 728, 470, 493, 579, 731, -315, 133, - /* 110 */ -438, -438, -256, -15, 144, 213, 320, 404, 434, 520, - /* 120 */ 523, 547, 598, 620, 623, 711, 721, 724, 733, 780, - /* 130 */ 787, 790, 796, -368, -83, -64, 594, 513, -83, 209, - /* 140 */ 317, 207, 725, 742, 727, 664, 791, 812, 119, -364, - /* 150 */ -355, 273, 650, 140, 657, 807, 600, 813, 458, 504, - /* 160 */ 573, 639, 910, 913, 916, 910, 882, 930, 956, 901, - /* 170 */ 821, 833, 953, 843, 944, 946, 935, 935, 965, 920, - /* 180 */ 978, 980, 945, 933, 888, 888, 872, 888, 897, 889, - /* 190 */ 935, 929, 932, 942, 940, 1004, 947, 1006, 960, 995, - /* 200 */ 993, 999, 1002, 1049, 1051, 1003, 1007, 1008, 1044, 1052, - /* 210 */ 1061, 1054, 1062, 1065, 1066, 1075, 1076, 1079, 1077, 1001, - /* 220 */ 1067, 1037, 1073, 1080, 1024, 1078, 1083, 1082, 1084, 1085, - /* 230 */ 1088, 1095, 1091, 1101, 1112, 1089, 1093, 1096, 1097, 1102, - /* 240 */ 1104, 1105, 1106, 1107, 1110, 1113, 1108, 1114, 1070, 1118, - /* 250 */ 1119, 1041, 1059, 1063, 1134, 1081, 1098, 1120, 1138, 1117, - /* 260 */ 1152, 1111, 1122, 1137, 1064, 1128, 1147, 1071, 1136, 1149, - /* 270 */ 1154, 935, 1074, 1121, 1126, 1099, 1123, 1115, 1125, 1069, - /* 280 */ 1124, 1133, 888, 1205, 1130, 1210, 1209, 1206, 1213, 1165, - /* 290 */ 1162, 1180, 1181, 1183, 1186, 1197, 1169, 1203, 1192, 1244, - /* 300 */ 1240, 1247, 1269, 1166, 1233, 1227, 1251, 1274, 1275, 1281, - /* 310 */ 1284, 1217, 1218, 1241, 1242, 1256, 1258, 1260, 1276, 1296, + /* 0 */ -155, -94, 176, 286, 386, 422, 600, 636, 830, 853, + /* 10 */ 458, 1091, 1153, 1220, 1285, 1321, -339, 151, 696, 1357, + /* 20 */ 1381, 1446, 1508, 1524, 1575, 1597, 1661, 1697, 1719, 1786, + /* 30 */ 1805, 1874, 1891, 1939, 1987, 2004, 2052, 2075, 2100, 2151, + /* 40 */ 2175, 2223, 2239, 2301, 2325, 2373, 2421, 2436, 2484, 2499, + /* 50 */ 2532, -165, -204, -407, 53, 118, 537, 620, 727, 547, + /* 60 */ 598, -363, 59, 752, 810, -284, -133, 354, -321, 55, + /* 70 */ 127, -388, -210, -36, -205, -383, -327, 163, 150, 186, + /* 80 */ 246, 305, 410, 416, 475, 139, 610, 622, 215, -33, + /* 90 */ 633, 689, -322, 732, 100, 740, 242, 755, 279, -109, + /* 100 */ 765, 819, 292, 861, -96, 344, 383, 753, 261, -337, + /* 110 */ -277, -277, 404, -257, 154, 239, 283, 457, 511, 546, + /* 120 */ 559, 563, 605, 616, 618, 667, 669, 703, 716, 747, + /* 130 */ 776, 777, 781, 187, -294, -154, -70, 295, 472, 474, + /* 140 */ -294, 429, 558, 629, 266, 353, 677, 593, 785, 788, + /* 150 */ 203, -242, -433, 455, 96, 467, 587, 685, 838, 850, + /* 160 */ 403, 536, 626, 746, 760, 890, 937, 760, 705, 929, + /* 170 */ 990, 932, 852, 845, 983, 871, 979, 985, 974, 974, + /* 180 */ 1001, 953, 1011, 1012, 972, 967, 910, 910, 894, 910, + /* 190 */ 919, 941, 974, 982, 986, 994, 991, 1057, 998, 1063, + /* 200 */ 1013, 1026, 1028, 1029, 1035, 1087, 1088, 1038, 1043, 1044, + /* 210 */ 1089, 1092, 1100, 1094, 1103, 1106, 1107, 1116, 1115, 1120, + /* 220 */ 1118, 1045, 1108, 1075, 1110, 1121, 1069, 1119, 1129, 1124, + /* 230 */ 1127, 1128, 1139, 1136, 1147, 1122, 1123, 1126, 1135, 1141, + /* 240 */ 1142, 1149, 1152, 1154, 1156, 1159, 1146, 1155, 1109, 1148, + /* 250 */ 1157, 1076, 1096, 1099, 1173, 1097, 1125, 1150, 1188, 1134, + /* 260 */ 1199, 1161, 1163, 1162, 1090, 1160, 1164, 1101, 1166, 1169, + /* 270 */ 1177, 974, 1111, 1102, 1113, 1112, 1137, 1132, 1165, 1144, + /* 280 */ 1167, 1140, 910, 1246, 1175, 1262, 1256, 1253, 1257, 1211, + /* 290 */ 1210, 1226, 1236, 1239, 1240, 1241, 1222, 1242, 1234, 1281, + /* 300 */ 1264, 1292, 1299, 1192, 1274, 1251, 1275, 1309, 1303, 1317, + /* 310 */ 1320, 1254, 1227, 1268, 1272, 1298, 1300, 1301, 1308, 1329, }; static const YYACTIONTYPE yy_default[] = { - /* 0 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 10 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 20 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 30 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 40 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 50 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 60 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 70 */ 1753, 1753, 1753, 1753, 2030, 1753, 1753, 1753, 1753, 1753, - /* 80 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1839, 1753, - /* 90 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 100 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1837, 2023, - /* 110 */ 2248, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 120 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 130 */ 1753, 1753, 1753, 1753, 2260, 1753, 1753, 1753, 2260, 2260, - /* 140 */ 2260, 1837, 2220, 2220, 1753, 1753, 1753, 1753, 1839, 2090, - /* 150 */ 1753, 1753, 1753, 1753, 1753, 1753, 1958, 1753, 1753, 1753, - /* 160 */ 1753, 1753, 1982, 1753, 1753, 1753, 2082, 1753, 1753, 2285, - /* 170 */ 2341, 1753, 1753, 2288, 1753, 1753, 1753, 1753, 1753, 2035, - /* 180 */ 1753, 1753, 1912, 2275, 2252, 2266, 2325, 2253, 2250, 2269, - /* 190 */ 1753, 2279, 1753, 1753, 2104, 1839, 1753, 1839, 2069, 2028, - /* 200 */ 1753, 2028, 2025, 1753, 1753, 2028, 2025, 2025, 1901, 1897, - /* 210 */ 1753, 1895, 1753, 1753, 1753, 1753, 1800, 1753, 1800, 1753, - /* 220 */ 1839, 1753, 1839, 1753, 1753, 1839, 1753, 1839, 1839, 1839, - /* 230 */ 1753, 1839, 1814, 1814, 1753, 1753, 1753, 1753, 1753, 1753, - /* 240 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 250 */ 1753, 2102, 2088, 1753, 1837, 2080, 2078, 1753, 1837, 2076, - /* 260 */ 1753, 1753, 1753, 1753, 2296, 2294, 1753, 2296, 2294, 1753, - /* 270 */ 1753, 1753, 2310, 2306, 2296, 2314, 2312, 2281, 2279, 2344, - /* 280 */ 2331, 2327, 2266, 1753, 1753, 1753, 1753, 1837, 1837, 1753, - /* 290 */ 2294, 1753, 1753, 1753, 1753, 1753, 2294, 1753, 1753, 1837, - /* 300 */ 1753, 1837, 1753, 1753, 1928, 1753, 1753, 1753, 1837, 1785, - /* 310 */ 1753, 2071, 2093, 2053, 2053, 1961, 1961, 1961, 1840, 1758, - /* 320 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 330 */ 1753, 1753, 2309, 2308, 2175, 1753, 2224, 2223, 2222, 2213, - /* 340 */ 2174, 1924, 1753, 2173, 2172, 1753, 1753, 1753, 1753, 1753, - /* 350 */ 1753, 1753, 1753, 1753, 2044, 2043, 2166, 1753, 1753, 2167, - /* 360 */ 2165, 2164, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 370 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 380 */ 1753, 1753, 1753, 1753, 1753, 1753, 2328, 2332, 1753, 1753, - /* 390 */ 1753, 1753, 1753, 1753, 1753, 2249, 1753, 1753, 1753, 2148, - /* 400 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 410 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 420 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 430 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 440 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 450 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 460 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 470 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 480 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 490 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 500 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 510 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 520 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1790, 2153, 1753, - /* 530 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 540 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 550 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 560 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 570 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1878, 1877, 1753, - /* 580 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 590 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 600 */ 1753, 2157, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 610 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 2324, 2282, - /* 620 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 630 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 2148, 1753, - /* 640 */ 2307, 1753, 1753, 2322, 1753, 2326, 1753, 1753, 1753, 1753, - /* 650 */ 1753, 1753, 1753, 2259, 2255, 1753, 1753, 2251, 1753, 1753, - /* 660 */ 1753, 1753, 1753, 1753, 1753, 2156, 1753, 1753, 1753, 1753, - /* 670 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 2147, - /* 680 */ 1753, 2210, 1753, 1753, 1753, 2244, 1753, 1753, 2195, 1753, - /* 690 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 2157, 1753, - /* 700 */ 2160, 1753, 1753, 1753, 1753, 1753, 1955, 1753, 1753, 1753, - /* 710 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1940, - /* 720 */ 1938, 1937, 1936, 1753, 1968, 1753, 1753, 1753, 1964, 1963, - /* 730 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 740 */ 1753, 1753, 1753, 1858, 1753, 1753, 1753, 1753, 1753, 1753, - /* 750 */ 1753, 1753, 1850, 1753, 1849, 1753, 1753, 1753, 1753, 1753, - /* 760 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 770 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, - /* 780 */ 1753, 1753, 1753, 1753, 1753, 1753, 1753, + /* 0 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 10 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 20 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 30 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 40 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 50 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 60 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 70 */ 1759, 1759, 1759, 1759, 2039, 1759, 1759, 1759, 1759, 1759, + /* 80 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1848, 1759, + /* 90 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 100 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1846, 2032, + /* 110 */ 2257, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 120 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 130 */ 1759, 1759, 1759, 1759, 2269, 1759, 1759, 1822, 1822, 1759, + /* 140 */ 2269, 2269, 2269, 1846, 2229, 2229, 1759, 1759, 1759, 1759, + /* 150 */ 1848, 2099, 1759, 1759, 1759, 1759, 1759, 1759, 1967, 1759, + /* 160 */ 1759, 1759, 1759, 1759, 1991, 1759, 1759, 1759, 2091, 1759, + /* 170 */ 1759, 2294, 2350, 1759, 1759, 2297, 1759, 1759, 1759, 1759, + /* 180 */ 1759, 2044, 1759, 1759, 1921, 2284, 2261, 2275, 2334, 2262, + /* 190 */ 2259, 2278, 1759, 2288, 1759, 1759, 2113, 1848, 1759, 1848, + /* 200 */ 2078, 2037, 1759, 2037, 2034, 1759, 1759, 2037, 2034, 2034, + /* 210 */ 1910, 1906, 1759, 1904, 1759, 1759, 1759, 1759, 1806, 1759, + /* 220 */ 1806, 1759, 1848, 1759, 1848, 1759, 1759, 1848, 1759, 1848, + /* 230 */ 1848, 1848, 1759, 1848, 1759, 1759, 1759, 1759, 1759, 1759, + /* 240 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 250 */ 1759, 2111, 2097, 1759, 1846, 2089, 2087, 1759, 1846, 2085, + /* 260 */ 1759, 1759, 1759, 1759, 2305, 2303, 1759, 2305, 2303, 1759, + /* 270 */ 1759, 1759, 2319, 2315, 2305, 2323, 2321, 2290, 2288, 2353, + /* 280 */ 2340, 2336, 2275, 1759, 1759, 1759, 1759, 1846, 1846, 1759, + /* 290 */ 2303, 1759, 1759, 1759, 1759, 1759, 2303, 1759, 1759, 1846, + /* 300 */ 1759, 1846, 1759, 1759, 1937, 1759, 1759, 1759, 1846, 1791, + /* 310 */ 1759, 2080, 2102, 2062, 2062, 1970, 1970, 1970, 1849, 1764, + /* 320 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 330 */ 1759, 1759, 2318, 2317, 2184, 1759, 2233, 2232, 2231, 2222, + /* 340 */ 2183, 1933, 1759, 2182, 2181, 1759, 1759, 1759, 1759, 1759, + /* 350 */ 1759, 1759, 1759, 1759, 2053, 2052, 2175, 1759, 1759, 2176, + /* 360 */ 2174, 2173, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 370 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 380 */ 1759, 1759, 1759, 1759, 1759, 1759, 2337, 2341, 1759, 1759, + /* 390 */ 1759, 1759, 1759, 1759, 1759, 2258, 1759, 1759, 1759, 2157, + /* 400 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 410 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 420 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 430 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 440 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 450 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 460 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 470 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 480 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 490 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 500 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 510 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 520 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1796, 2162, 1759, + /* 530 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 540 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 550 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 560 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 570 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1887, + /* 580 */ 1886, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 590 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 600 */ 1759, 1759, 1759, 2166, 1759, 1759, 1759, 1759, 1759, 1759, + /* 610 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 620 */ 2333, 2291, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 630 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 640 */ 2157, 1759, 2316, 1759, 1759, 2331, 1759, 2335, 1759, 1759, + /* 650 */ 1759, 1759, 1759, 1759, 1759, 2268, 2264, 1759, 1759, 2260, + /* 660 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 2165, 1759, 1759, + /* 670 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 680 */ 1759, 2156, 1759, 2219, 1759, 1759, 1759, 2253, 1759, 1759, + /* 690 */ 2204, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 700 */ 2166, 1759, 2169, 1759, 1759, 1759, 1759, 1759, 1964, 1759, + /* 710 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 720 */ 1759, 1949, 1947, 1946, 1945, 1759, 1977, 1759, 1759, 1759, + /* 730 */ 1973, 1972, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 740 */ 1759, 1759, 1759, 1759, 1759, 1867, 1759, 1759, 1759, 1759, + /* 750 */ 1759, 1759, 1759, 1759, 1859, 1759, 1858, 1759, 1759, 1759, + /* 760 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 770 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, + /* 780 */ 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, }; /********** End of lemon-generated parsing tables *****************************/ @@ -1130,6 +1126,7 @@ static const YYCODETYPE yyFallback[] = { 0, /* RESTORE => nothing */ 0, /* NK_IPTOKEN => nothing */ 0, /* FORCE => nothing */ + 0, /* UNSAFE => nothing */ 0, /* LOCAL => nothing */ 0, /* QNODE => nothing */ 0, /* BNODE => nothing */ @@ -1175,7 +1172,7 @@ static const YYCODETYPE yyFallback[] = { 0, /* MAX_SPEED => nothing */ 0, /* START => nothing */ 0, /* TIMESTAMP => nothing */ - 284, /* END => ABORT */ + 285, /* END => ABORT */ 0, /* TABLE => nothing */ 0, /* NK_LP => nothing */ 0, /* NK_RP => nothing */ @@ -1360,56 +1357,56 @@ static const YYCODETYPE yyFallback[] = { 0, /* ASC => nothing */ 0, /* NULLS => nothing */ 0, /* ABORT => nothing */ - 284, /* AFTER => ABORT */ - 284, /* ATTACH => ABORT */ - 284, /* BEFORE => ABORT */ - 284, /* BEGIN => ABORT */ - 284, /* BITAND => ABORT */ - 284, /* BITNOT => ABORT */ - 284, /* BITOR => ABORT */ - 284, /* BLOCKS => ABORT */ - 284, /* CHANGE => ABORT */ - 284, /* COMMA => ABORT */ - 284, /* CONCAT => ABORT */ - 284, /* CONFLICT => ABORT */ - 284, /* COPY => ABORT */ - 284, /* DEFERRED => ABORT */ - 284, /* DELIMITERS => ABORT */ - 284, /* DETACH => ABORT */ - 284, /* DIVIDE => ABORT */ - 284, /* DOT => ABORT */ - 284, /* EACH => ABORT */ - 284, /* FAIL => ABORT */ - 284, /* FILE => ABORT */ - 284, /* FOR => ABORT */ - 284, /* GLOB => ABORT */ - 284, /* ID => ABORT */ - 284, /* IMMEDIATE => ABORT */ - 284, /* IMPORT => ABORT */ - 284, /* INITIALLY => ABORT */ - 284, /* INSTEAD => ABORT */ - 284, /* ISNULL => ABORT */ - 284, /* KEY => ABORT */ - 284, /* MODULES => ABORT */ - 284, /* NK_BITNOT => ABORT */ - 284, /* NK_SEMI => ABORT */ - 284, /* NOTNULL => ABORT */ - 284, /* OF => ABORT */ - 284, /* PLUS => ABORT */ - 284, /* PRIVILEGE => ABORT */ - 284, /* RAISE => ABORT */ - 284, /* RESTRICT => ABORT */ - 284, /* ROW => ABORT */ - 284, /* SEMI => ABORT */ - 284, /* STAR => ABORT */ - 284, /* STATEMENT => ABORT */ - 284, /* STRICT => ABORT */ - 284, /* STRING => ABORT */ - 284, /* TIMES => ABORT */ - 284, /* VALUES => ABORT */ - 284, /* VARIABLE => ABORT */ - 284, /* VIEW => ABORT */ - 284, /* WAL => ABORT */ + 285, /* AFTER => ABORT */ + 285, /* ATTACH => ABORT */ + 285, /* BEFORE => ABORT */ + 285, /* BEGIN => ABORT */ + 285, /* BITAND => ABORT */ + 285, /* BITNOT => ABORT */ + 285, /* BITOR => ABORT */ + 285, /* BLOCKS => ABORT */ + 285, /* CHANGE => ABORT */ + 285, /* COMMA => ABORT */ + 285, /* CONCAT => ABORT */ + 285, /* CONFLICT => ABORT */ + 285, /* COPY => ABORT */ + 285, /* DEFERRED => ABORT */ + 285, /* DELIMITERS => ABORT */ + 285, /* DETACH => ABORT */ + 285, /* DIVIDE => ABORT */ + 285, /* DOT => ABORT */ + 285, /* EACH => ABORT */ + 285, /* FAIL => ABORT */ + 285, /* FILE => ABORT */ + 285, /* FOR => ABORT */ + 285, /* GLOB => ABORT */ + 285, /* ID => ABORT */ + 285, /* IMMEDIATE => ABORT */ + 285, /* IMPORT => ABORT */ + 285, /* INITIALLY => ABORT */ + 285, /* INSTEAD => ABORT */ + 285, /* ISNULL => ABORT */ + 285, /* KEY => ABORT */ + 285, /* MODULES => ABORT */ + 285, /* NK_BITNOT => ABORT */ + 285, /* NK_SEMI => ABORT */ + 285, /* NOTNULL => ABORT */ + 285, /* OF => ABORT */ + 285, /* PLUS => ABORT */ + 285, /* PRIVILEGE => ABORT */ + 285, /* RAISE => ABORT */ + 285, /* RESTRICT => ABORT */ + 285, /* ROW => ABORT */ + 285, /* SEMI => ABORT */ + 285, /* STAR => ABORT */ + 285, /* STATEMENT => ABORT */ + 285, /* STRICT => ABORT */ + 285, /* STRING => ABORT */ + 285, /* TIMES => ABORT */ + 285, /* VALUES => ABORT */ + 285, /* VARIABLE => ABORT */ + 285, /* VIEW => ABORT */ + 285, /* WAL => ABORT */ }; #endif /* YYFALLBACK */ @@ -1552,433 +1549,435 @@ static const char *const yyTokenName[] = { /* 52 */ "RESTORE", /* 53 */ "NK_IPTOKEN", /* 54 */ "FORCE", - /* 55 */ "LOCAL", - /* 56 */ "QNODE", - /* 57 */ "BNODE", - /* 58 */ "SNODE", - /* 59 */ "MNODE", - /* 60 */ "VNODE", - /* 61 */ "DATABASE", - /* 62 */ "USE", - /* 63 */ "FLUSH", - /* 64 */ "TRIM", - /* 65 */ "COMPACT", - /* 66 */ "IF", - /* 67 */ "NOT", - /* 68 */ "EXISTS", - /* 69 */ "BUFFER", - /* 70 */ "CACHEMODEL", - /* 71 */ "CACHESIZE", - /* 72 */ "COMP", - /* 73 */ "DURATION", - /* 74 */ "NK_VARIABLE", - /* 75 */ "MAXROWS", - /* 76 */ "MINROWS", - /* 77 */ "KEEP", - /* 78 */ "PAGES", - /* 79 */ "PAGESIZE", - /* 80 */ "TSDB_PAGESIZE", - /* 81 */ "PRECISION", - /* 82 */ "REPLICA", - /* 83 */ "VGROUPS", - /* 84 */ "SINGLE_STABLE", - /* 85 */ "RETENTIONS", - /* 86 */ "SCHEMALESS", - /* 87 */ "WAL_LEVEL", - /* 88 */ "WAL_FSYNC_PERIOD", - /* 89 */ "WAL_RETENTION_PERIOD", - /* 90 */ "WAL_RETENTION_SIZE", - /* 91 */ "WAL_ROLL_PERIOD", - /* 92 */ "WAL_SEGMENT_SIZE", - /* 93 */ "STT_TRIGGER", - /* 94 */ "TABLE_PREFIX", - /* 95 */ "TABLE_SUFFIX", - /* 96 */ "NK_COLON", - /* 97 */ "MAX_SPEED", - /* 98 */ "START", - /* 99 */ "TIMESTAMP", - /* 100 */ "END", - /* 101 */ "TABLE", - /* 102 */ "NK_LP", - /* 103 */ "NK_RP", - /* 104 */ "STABLE", - /* 105 */ "ADD", - /* 106 */ "COLUMN", - /* 107 */ "MODIFY", - /* 108 */ "RENAME", - /* 109 */ "TAG", - /* 110 */ "SET", - /* 111 */ "NK_EQ", - /* 112 */ "USING", - /* 113 */ "TAGS", - /* 114 */ "BOOL", - /* 115 */ "TINYINT", - /* 116 */ "SMALLINT", - /* 117 */ "INT", - /* 118 */ "INTEGER", - /* 119 */ "BIGINT", - /* 120 */ "FLOAT", - /* 121 */ "DOUBLE", - /* 122 */ "BINARY", - /* 123 */ "NCHAR", - /* 124 */ "UNSIGNED", - /* 125 */ "JSON", - /* 126 */ "VARCHAR", - /* 127 */ "MEDIUMBLOB", - /* 128 */ "BLOB", - /* 129 */ "VARBINARY", - /* 130 */ "DECIMAL", - /* 131 */ "COMMENT", - /* 132 */ "MAX_DELAY", - /* 133 */ "WATERMARK", - /* 134 */ "ROLLUP", - /* 135 */ "TTL", - /* 136 */ "SMA", - /* 137 */ "DELETE_MARK", - /* 138 */ "FIRST", - /* 139 */ "LAST", - /* 140 */ "SHOW", - /* 141 */ "PRIVILEGES", - /* 142 */ "DATABASES", - /* 143 */ "TABLES", - /* 144 */ "STABLES", - /* 145 */ "MNODES", - /* 146 */ "QNODES", - /* 147 */ "FUNCTIONS", - /* 148 */ "INDEXES", - /* 149 */ "ACCOUNTS", - /* 150 */ "APPS", - /* 151 */ "CONNECTIONS", - /* 152 */ "LICENCES", - /* 153 */ "GRANTS", - /* 154 */ "QUERIES", - /* 155 */ "SCORES", - /* 156 */ "TOPICS", - /* 157 */ "VARIABLES", - /* 158 */ "CLUSTER", - /* 159 */ "BNODES", - /* 160 */ "SNODES", - /* 161 */ "TRANSACTIONS", - /* 162 */ "DISTRIBUTED", - /* 163 */ "CONSUMERS", - /* 164 */ "SUBSCRIPTIONS", - /* 165 */ "VNODES", - /* 166 */ "ALIVE", - /* 167 */ "LIKE", - /* 168 */ "TBNAME", - /* 169 */ "QTAGS", - /* 170 */ "AS", - /* 171 */ "INDEX", - /* 172 */ "FUNCTION", - /* 173 */ "INTERVAL", - /* 174 */ "COUNT", - /* 175 */ "LAST_ROW", - /* 176 */ "TOPIC", - /* 177 */ "META", - /* 178 */ "CONSUMER", - /* 179 */ "GROUP", - /* 180 */ "DESC", - /* 181 */ "DESCRIBE", - /* 182 */ "RESET", - /* 183 */ "QUERY", - /* 184 */ "CACHE", - /* 185 */ "EXPLAIN", - /* 186 */ "ANALYZE", - /* 187 */ "VERBOSE", - /* 188 */ "NK_BOOL", - /* 189 */ "RATIO", - /* 190 */ "NK_FLOAT", - /* 191 */ "OUTPUTTYPE", - /* 192 */ "AGGREGATE", - /* 193 */ "BUFSIZE", - /* 194 */ "LANGUAGE", - /* 195 */ "REPLACE", - /* 196 */ "STREAM", - /* 197 */ "INTO", - /* 198 */ "PAUSE", - /* 199 */ "RESUME", - /* 200 */ "TRIGGER", - /* 201 */ "AT_ONCE", - /* 202 */ "WINDOW_CLOSE", - /* 203 */ "IGNORE", - /* 204 */ "EXPIRED", - /* 205 */ "FILL_HISTORY", - /* 206 */ "UPDATE", - /* 207 */ "SUBTABLE", - /* 208 */ "UNTREATED", - /* 209 */ "KILL", - /* 210 */ "CONNECTION", - /* 211 */ "TRANSACTION", - /* 212 */ "BALANCE", - /* 213 */ "VGROUP", - /* 214 */ "LEADER", - /* 215 */ "MERGE", - /* 216 */ "REDISTRIBUTE", - /* 217 */ "SPLIT", - /* 218 */ "DELETE", - /* 219 */ "INSERT", - /* 220 */ "NULL", - /* 221 */ "NK_QUESTION", - /* 222 */ "NK_ARROW", - /* 223 */ "ROWTS", - /* 224 */ "QSTART", - /* 225 */ "QEND", - /* 226 */ "QDURATION", - /* 227 */ "WSTART", - /* 228 */ "WEND", - /* 229 */ "WDURATION", - /* 230 */ "IROWTS", - /* 231 */ "ISFILLED", - /* 232 */ "CAST", - /* 233 */ "NOW", - /* 234 */ "TODAY", - /* 235 */ "TIMEZONE", - /* 236 */ "CLIENT_VERSION", - /* 237 */ "SERVER_VERSION", - /* 238 */ "SERVER_STATUS", - /* 239 */ "CURRENT_USER", - /* 240 */ "CASE", - /* 241 */ "WHEN", - /* 242 */ "THEN", - /* 243 */ "ELSE", - /* 244 */ "BETWEEN", - /* 245 */ "IS", - /* 246 */ "NK_LT", - /* 247 */ "NK_GT", - /* 248 */ "NK_LE", - /* 249 */ "NK_GE", - /* 250 */ "NK_NE", - /* 251 */ "MATCH", - /* 252 */ "NMATCH", - /* 253 */ "CONTAINS", - /* 254 */ "IN", - /* 255 */ "JOIN", - /* 256 */ "INNER", - /* 257 */ "SELECT", - /* 258 */ "DISTINCT", - /* 259 */ "WHERE", - /* 260 */ "PARTITION", - /* 261 */ "BY", - /* 262 */ "SESSION", - /* 263 */ "STATE_WINDOW", - /* 264 */ "EVENT_WINDOW", - /* 265 */ "SLIDING", - /* 266 */ "FILL", - /* 267 */ "VALUE", - /* 268 */ "VALUE_F", - /* 269 */ "NONE", - /* 270 */ "PREV", - /* 271 */ "NULL_F", - /* 272 */ "LINEAR", - /* 273 */ "NEXT", - /* 274 */ "HAVING", - /* 275 */ "RANGE", - /* 276 */ "EVERY", - /* 277 */ "ORDER", - /* 278 */ "SLIMIT", - /* 279 */ "SOFFSET", - /* 280 */ "LIMIT", - /* 281 */ "OFFSET", - /* 282 */ "ASC", - /* 283 */ "NULLS", - /* 284 */ "ABORT", - /* 285 */ "AFTER", - /* 286 */ "ATTACH", - /* 287 */ "BEFORE", - /* 288 */ "BEGIN", - /* 289 */ "BITAND", - /* 290 */ "BITNOT", - /* 291 */ "BITOR", - /* 292 */ "BLOCKS", - /* 293 */ "CHANGE", - /* 294 */ "COMMA", - /* 295 */ "CONCAT", - /* 296 */ "CONFLICT", - /* 297 */ "COPY", - /* 298 */ "DEFERRED", - /* 299 */ "DELIMITERS", - /* 300 */ "DETACH", - /* 301 */ "DIVIDE", - /* 302 */ "DOT", - /* 303 */ "EACH", - /* 304 */ "FAIL", - /* 305 */ "FILE", - /* 306 */ "FOR", - /* 307 */ "GLOB", - /* 308 */ "ID", - /* 309 */ "IMMEDIATE", - /* 310 */ "IMPORT", - /* 311 */ "INITIALLY", - /* 312 */ "INSTEAD", - /* 313 */ "ISNULL", - /* 314 */ "KEY", - /* 315 */ "MODULES", - /* 316 */ "NK_BITNOT", - /* 317 */ "NK_SEMI", - /* 318 */ "NOTNULL", - /* 319 */ "OF", - /* 320 */ "PLUS", - /* 321 */ "PRIVILEGE", - /* 322 */ "RAISE", - /* 323 */ "RESTRICT", - /* 324 */ "ROW", - /* 325 */ "SEMI", - /* 326 */ "STAR", - /* 327 */ "STATEMENT", - /* 328 */ "STRICT", - /* 329 */ "STRING", - /* 330 */ "TIMES", - /* 331 */ "VALUES", - /* 332 */ "VARIABLE", - /* 333 */ "VIEW", - /* 334 */ "WAL", - /* 335 */ "cmd", - /* 336 */ "account_options", - /* 337 */ "alter_account_options", - /* 338 */ "literal", - /* 339 */ "alter_account_option", - /* 340 */ "user_name", - /* 341 */ "sysinfo_opt", - /* 342 */ "privileges", - /* 343 */ "priv_level", - /* 344 */ "with_opt", - /* 345 */ "priv_type_list", - /* 346 */ "priv_type", - /* 347 */ "db_name", - /* 348 */ "table_name", - /* 349 */ "topic_name", - /* 350 */ "search_condition", - /* 351 */ "dnode_endpoint", - /* 352 */ "force_opt", - /* 353 */ "not_exists_opt", - /* 354 */ "db_options", - /* 355 */ "exists_opt", - /* 356 */ "alter_db_options", - /* 357 */ "speed_opt", - /* 358 */ "start_opt", - /* 359 */ "end_opt", - /* 360 */ "integer_list", - /* 361 */ "variable_list", - /* 362 */ "retention_list", - /* 363 */ "signed", - /* 364 */ "alter_db_option", - /* 365 */ "retention", - /* 366 */ "full_table_name", - /* 367 */ "column_def_list", - /* 368 */ "tags_def_opt", - /* 369 */ "table_options", - /* 370 */ "multi_create_clause", - /* 371 */ "tags_def", - /* 372 */ "multi_drop_clause", - /* 373 */ "alter_table_clause", - /* 374 */ "alter_table_options", - /* 375 */ "column_name", - /* 376 */ "type_name", - /* 377 */ "signed_literal", - /* 378 */ "create_subtable_clause", - /* 379 */ "specific_cols_opt", - /* 380 */ "expression_list", - /* 381 */ "drop_table_clause", - /* 382 */ "col_name_list", - /* 383 */ "column_def", - /* 384 */ "duration_list", - /* 385 */ "rollup_func_list", - /* 386 */ "alter_table_option", - /* 387 */ "duration_literal", - /* 388 */ "rollup_func_name", - /* 389 */ "function_name", - /* 390 */ "col_name", - /* 391 */ "db_name_cond_opt", - /* 392 */ "like_pattern_opt", - /* 393 */ "table_name_cond", - /* 394 */ "from_db_opt", - /* 395 */ "tag_list_opt", - /* 396 */ "tag_item", - /* 397 */ "column_alias", - /* 398 */ "full_index_name", - /* 399 */ "index_options", - /* 400 */ "index_name", - /* 401 */ "func_list", - /* 402 */ "sliding_opt", - /* 403 */ "sma_stream_opt", - /* 404 */ "func", - /* 405 */ "sma_func_name", - /* 406 */ "query_or_subquery", - /* 407 */ "cgroup_name", - /* 408 */ "analyze_opt", - /* 409 */ "explain_options", - /* 410 */ "insert_query", - /* 411 */ "or_replace_opt", - /* 412 */ "agg_func_opt", - /* 413 */ "bufsize_opt", - /* 414 */ "language_opt", - /* 415 */ "stream_name", - /* 416 */ "stream_options", - /* 417 */ "col_list_opt", - /* 418 */ "tag_def_or_ref_opt", - /* 419 */ "subtable_opt", - /* 420 */ "ignore_opt", - /* 421 */ "expression", - /* 422 */ "dnode_list", - /* 423 */ "where_clause_opt", - /* 424 */ "literal_func", - /* 425 */ "literal_list", - /* 426 */ "table_alias", - /* 427 */ "expr_or_subquery", - /* 428 */ "pseudo_column", - /* 429 */ "column_reference", - /* 430 */ "function_expression", - /* 431 */ "case_when_expression", - /* 432 */ "star_func", - /* 433 */ "star_func_para_list", - /* 434 */ "noarg_func", - /* 435 */ "other_para_list", - /* 436 */ "star_func_para", - /* 437 */ "when_then_list", - /* 438 */ "case_when_else_opt", - /* 439 */ "common_expression", - /* 440 */ "when_then_expr", - /* 441 */ "predicate", - /* 442 */ "compare_op", - /* 443 */ "in_op", - /* 444 */ "in_predicate_value", - /* 445 */ "boolean_value_expression", - /* 446 */ "boolean_primary", - /* 447 */ "from_clause_opt", - /* 448 */ "table_reference_list", - /* 449 */ "table_reference", - /* 450 */ "table_primary", - /* 451 */ "joined_table", - /* 452 */ "alias_opt", - /* 453 */ "subquery", - /* 454 */ "parenthesized_joined_table", - /* 455 */ "join_type", - /* 456 */ "query_specification", - /* 457 */ "set_quantifier_opt", - /* 458 */ "select_list", - /* 459 */ "partition_by_clause_opt", - /* 460 */ "range_opt", - /* 461 */ "every_opt", - /* 462 */ "fill_opt", - /* 463 */ "twindow_clause_opt", - /* 464 */ "group_by_clause_opt", - /* 465 */ "having_clause_opt", - /* 466 */ "select_item", - /* 467 */ "partition_list", - /* 468 */ "partition_item", - /* 469 */ "fill_mode", - /* 470 */ "group_by_list", - /* 471 */ "query_expression", - /* 472 */ "query_simple", - /* 473 */ "order_by_clause_opt", - /* 474 */ "slimit_clause_opt", - /* 475 */ "limit_clause_opt", - /* 476 */ "union_query_expression", - /* 477 */ "query_simple_or_subquery", - /* 478 */ "sort_specification_list", - /* 479 */ "sort_specification", - /* 480 */ "ordering_specification_opt", - /* 481 */ "null_ordering_opt", + /* 55 */ "UNSAFE", + /* 56 */ "LOCAL", + /* 57 */ "QNODE", + /* 58 */ "BNODE", + /* 59 */ "SNODE", + /* 60 */ "MNODE", + /* 61 */ "VNODE", + /* 62 */ "DATABASE", + /* 63 */ "USE", + /* 64 */ "FLUSH", + /* 65 */ "TRIM", + /* 66 */ "COMPACT", + /* 67 */ "IF", + /* 68 */ "NOT", + /* 69 */ "EXISTS", + /* 70 */ "BUFFER", + /* 71 */ "CACHEMODEL", + /* 72 */ "CACHESIZE", + /* 73 */ "COMP", + /* 74 */ "DURATION", + /* 75 */ "NK_VARIABLE", + /* 76 */ "MAXROWS", + /* 77 */ "MINROWS", + /* 78 */ "KEEP", + /* 79 */ "PAGES", + /* 80 */ "PAGESIZE", + /* 81 */ "TSDB_PAGESIZE", + /* 82 */ "PRECISION", + /* 83 */ "REPLICA", + /* 84 */ "VGROUPS", + /* 85 */ "SINGLE_STABLE", + /* 86 */ "RETENTIONS", + /* 87 */ "SCHEMALESS", + /* 88 */ "WAL_LEVEL", + /* 89 */ "WAL_FSYNC_PERIOD", + /* 90 */ "WAL_RETENTION_PERIOD", + /* 91 */ "WAL_RETENTION_SIZE", + /* 92 */ "WAL_ROLL_PERIOD", + /* 93 */ "WAL_SEGMENT_SIZE", + /* 94 */ "STT_TRIGGER", + /* 95 */ "TABLE_PREFIX", + /* 96 */ "TABLE_SUFFIX", + /* 97 */ "NK_COLON", + /* 98 */ "MAX_SPEED", + /* 99 */ "START", + /* 100 */ "TIMESTAMP", + /* 101 */ "END", + /* 102 */ "TABLE", + /* 103 */ "NK_LP", + /* 104 */ "NK_RP", + /* 105 */ "STABLE", + /* 106 */ "ADD", + /* 107 */ "COLUMN", + /* 108 */ "MODIFY", + /* 109 */ "RENAME", + /* 110 */ "TAG", + /* 111 */ "SET", + /* 112 */ "NK_EQ", + /* 113 */ "USING", + /* 114 */ "TAGS", + /* 115 */ "BOOL", + /* 116 */ "TINYINT", + /* 117 */ "SMALLINT", + /* 118 */ "INT", + /* 119 */ "INTEGER", + /* 120 */ "BIGINT", + /* 121 */ "FLOAT", + /* 122 */ "DOUBLE", + /* 123 */ "BINARY", + /* 124 */ "NCHAR", + /* 125 */ "UNSIGNED", + /* 126 */ "JSON", + /* 127 */ "VARCHAR", + /* 128 */ "MEDIUMBLOB", + /* 129 */ "BLOB", + /* 130 */ "VARBINARY", + /* 131 */ "DECIMAL", + /* 132 */ "COMMENT", + /* 133 */ "MAX_DELAY", + /* 134 */ "WATERMARK", + /* 135 */ "ROLLUP", + /* 136 */ "TTL", + /* 137 */ "SMA", + /* 138 */ "DELETE_MARK", + /* 139 */ "FIRST", + /* 140 */ "LAST", + /* 141 */ "SHOW", + /* 142 */ "PRIVILEGES", + /* 143 */ "DATABASES", + /* 144 */ "TABLES", + /* 145 */ "STABLES", + /* 146 */ "MNODES", + /* 147 */ "QNODES", + /* 148 */ "FUNCTIONS", + /* 149 */ "INDEXES", + /* 150 */ "ACCOUNTS", + /* 151 */ "APPS", + /* 152 */ "CONNECTIONS", + /* 153 */ "LICENCES", + /* 154 */ "GRANTS", + /* 155 */ "QUERIES", + /* 156 */ "SCORES", + /* 157 */ "TOPICS", + /* 158 */ "VARIABLES", + /* 159 */ "CLUSTER", + /* 160 */ "BNODES", + /* 161 */ "SNODES", + /* 162 */ "TRANSACTIONS", + /* 163 */ "DISTRIBUTED", + /* 164 */ "CONSUMERS", + /* 165 */ "SUBSCRIPTIONS", + /* 166 */ "VNODES", + /* 167 */ "ALIVE", + /* 168 */ "LIKE", + /* 169 */ "TBNAME", + /* 170 */ "QTAGS", + /* 171 */ "AS", + /* 172 */ "INDEX", + /* 173 */ "FUNCTION", + /* 174 */ "INTERVAL", + /* 175 */ "COUNT", + /* 176 */ "LAST_ROW", + /* 177 */ "TOPIC", + /* 178 */ "META", + /* 179 */ "CONSUMER", + /* 180 */ "GROUP", + /* 181 */ "DESC", + /* 182 */ "DESCRIBE", + /* 183 */ "RESET", + /* 184 */ "QUERY", + /* 185 */ "CACHE", + /* 186 */ "EXPLAIN", + /* 187 */ "ANALYZE", + /* 188 */ "VERBOSE", + /* 189 */ "NK_BOOL", + /* 190 */ "RATIO", + /* 191 */ "NK_FLOAT", + /* 192 */ "OUTPUTTYPE", + /* 193 */ "AGGREGATE", + /* 194 */ "BUFSIZE", + /* 195 */ "LANGUAGE", + /* 196 */ "REPLACE", + /* 197 */ "STREAM", + /* 198 */ "INTO", + /* 199 */ "PAUSE", + /* 200 */ "RESUME", + /* 201 */ "TRIGGER", + /* 202 */ "AT_ONCE", + /* 203 */ "WINDOW_CLOSE", + /* 204 */ "IGNORE", + /* 205 */ "EXPIRED", + /* 206 */ "FILL_HISTORY", + /* 207 */ "UPDATE", + /* 208 */ "SUBTABLE", + /* 209 */ "UNTREATED", + /* 210 */ "KILL", + /* 211 */ "CONNECTION", + /* 212 */ "TRANSACTION", + /* 213 */ "BALANCE", + /* 214 */ "VGROUP", + /* 215 */ "LEADER", + /* 216 */ "MERGE", + /* 217 */ "REDISTRIBUTE", + /* 218 */ "SPLIT", + /* 219 */ "DELETE", + /* 220 */ "INSERT", + /* 221 */ "NULL", + /* 222 */ "NK_QUESTION", + /* 223 */ "NK_ARROW", + /* 224 */ "ROWTS", + /* 225 */ "QSTART", + /* 226 */ "QEND", + /* 227 */ "QDURATION", + /* 228 */ "WSTART", + /* 229 */ "WEND", + /* 230 */ "WDURATION", + /* 231 */ "IROWTS", + /* 232 */ "ISFILLED", + /* 233 */ "CAST", + /* 234 */ "NOW", + /* 235 */ "TODAY", + /* 236 */ "TIMEZONE", + /* 237 */ "CLIENT_VERSION", + /* 238 */ "SERVER_VERSION", + /* 239 */ "SERVER_STATUS", + /* 240 */ "CURRENT_USER", + /* 241 */ "CASE", + /* 242 */ "WHEN", + /* 243 */ "THEN", + /* 244 */ "ELSE", + /* 245 */ "BETWEEN", + /* 246 */ "IS", + /* 247 */ "NK_LT", + /* 248 */ "NK_GT", + /* 249 */ "NK_LE", + /* 250 */ "NK_GE", + /* 251 */ "NK_NE", + /* 252 */ "MATCH", + /* 253 */ "NMATCH", + /* 254 */ "CONTAINS", + /* 255 */ "IN", + /* 256 */ "JOIN", + /* 257 */ "INNER", + /* 258 */ "SELECT", + /* 259 */ "DISTINCT", + /* 260 */ "WHERE", + /* 261 */ "PARTITION", + /* 262 */ "BY", + /* 263 */ "SESSION", + /* 264 */ "STATE_WINDOW", + /* 265 */ "EVENT_WINDOW", + /* 266 */ "SLIDING", + /* 267 */ "FILL", + /* 268 */ "VALUE", + /* 269 */ "VALUE_F", + /* 270 */ "NONE", + /* 271 */ "PREV", + /* 272 */ "NULL_F", + /* 273 */ "LINEAR", + /* 274 */ "NEXT", + /* 275 */ "HAVING", + /* 276 */ "RANGE", + /* 277 */ "EVERY", + /* 278 */ "ORDER", + /* 279 */ "SLIMIT", + /* 280 */ "SOFFSET", + /* 281 */ "LIMIT", + /* 282 */ "OFFSET", + /* 283 */ "ASC", + /* 284 */ "NULLS", + /* 285 */ "ABORT", + /* 286 */ "AFTER", + /* 287 */ "ATTACH", + /* 288 */ "BEFORE", + /* 289 */ "BEGIN", + /* 290 */ "BITAND", + /* 291 */ "BITNOT", + /* 292 */ "BITOR", + /* 293 */ "BLOCKS", + /* 294 */ "CHANGE", + /* 295 */ "COMMA", + /* 296 */ "CONCAT", + /* 297 */ "CONFLICT", + /* 298 */ "COPY", + /* 299 */ "DEFERRED", + /* 300 */ "DELIMITERS", + /* 301 */ "DETACH", + /* 302 */ "DIVIDE", + /* 303 */ "DOT", + /* 304 */ "EACH", + /* 305 */ "FAIL", + /* 306 */ "FILE", + /* 307 */ "FOR", + /* 308 */ "GLOB", + /* 309 */ "ID", + /* 310 */ "IMMEDIATE", + /* 311 */ "IMPORT", + /* 312 */ "INITIALLY", + /* 313 */ "INSTEAD", + /* 314 */ "ISNULL", + /* 315 */ "KEY", + /* 316 */ "MODULES", + /* 317 */ "NK_BITNOT", + /* 318 */ "NK_SEMI", + /* 319 */ "NOTNULL", + /* 320 */ "OF", + /* 321 */ "PLUS", + /* 322 */ "PRIVILEGE", + /* 323 */ "RAISE", + /* 324 */ "RESTRICT", + /* 325 */ "ROW", + /* 326 */ "SEMI", + /* 327 */ "STAR", + /* 328 */ "STATEMENT", + /* 329 */ "STRICT", + /* 330 */ "STRING", + /* 331 */ "TIMES", + /* 332 */ "VALUES", + /* 333 */ "VARIABLE", + /* 334 */ "VIEW", + /* 335 */ "WAL", + /* 336 */ "cmd", + /* 337 */ "account_options", + /* 338 */ "alter_account_options", + /* 339 */ "literal", + /* 340 */ "alter_account_option", + /* 341 */ "user_name", + /* 342 */ "sysinfo_opt", + /* 343 */ "privileges", + /* 344 */ "priv_level", + /* 345 */ "with_opt", + /* 346 */ "priv_type_list", + /* 347 */ "priv_type", + /* 348 */ "db_name", + /* 349 */ "table_name", + /* 350 */ "topic_name", + /* 351 */ "search_condition", + /* 352 */ "dnode_endpoint", + /* 353 */ "force_opt", + /* 354 */ "unsafe_opt", + /* 355 */ "not_exists_opt", + /* 356 */ "db_options", + /* 357 */ "exists_opt", + /* 358 */ "alter_db_options", + /* 359 */ "speed_opt", + /* 360 */ "start_opt", + /* 361 */ "end_opt", + /* 362 */ "integer_list", + /* 363 */ "variable_list", + /* 364 */ "retention_list", + /* 365 */ "signed", + /* 366 */ "alter_db_option", + /* 367 */ "retention", + /* 368 */ "full_table_name", + /* 369 */ "column_def_list", + /* 370 */ "tags_def_opt", + /* 371 */ "table_options", + /* 372 */ "multi_create_clause", + /* 373 */ "tags_def", + /* 374 */ "multi_drop_clause", + /* 375 */ "alter_table_clause", + /* 376 */ "alter_table_options", + /* 377 */ "column_name", + /* 378 */ "type_name", + /* 379 */ "signed_literal", + /* 380 */ "create_subtable_clause", + /* 381 */ "specific_cols_opt", + /* 382 */ "expression_list", + /* 383 */ "drop_table_clause", + /* 384 */ "col_name_list", + /* 385 */ "column_def", + /* 386 */ "duration_list", + /* 387 */ "rollup_func_list", + /* 388 */ "alter_table_option", + /* 389 */ "duration_literal", + /* 390 */ "rollup_func_name", + /* 391 */ "function_name", + /* 392 */ "col_name", + /* 393 */ "db_name_cond_opt", + /* 394 */ "like_pattern_opt", + /* 395 */ "table_name_cond", + /* 396 */ "from_db_opt", + /* 397 */ "tag_list_opt", + /* 398 */ "tag_item", + /* 399 */ "column_alias", + /* 400 */ "full_index_name", + /* 401 */ "index_options", + /* 402 */ "index_name", + /* 403 */ "func_list", + /* 404 */ "sliding_opt", + /* 405 */ "sma_stream_opt", + /* 406 */ "func", + /* 407 */ "sma_func_name", + /* 408 */ "query_or_subquery", + /* 409 */ "cgroup_name", + /* 410 */ "analyze_opt", + /* 411 */ "explain_options", + /* 412 */ "insert_query", + /* 413 */ "or_replace_opt", + /* 414 */ "agg_func_opt", + /* 415 */ "bufsize_opt", + /* 416 */ "language_opt", + /* 417 */ "stream_name", + /* 418 */ "stream_options", + /* 419 */ "col_list_opt", + /* 420 */ "tag_def_or_ref_opt", + /* 421 */ "subtable_opt", + /* 422 */ "ignore_opt", + /* 423 */ "expression", + /* 424 */ "dnode_list", + /* 425 */ "where_clause_opt", + /* 426 */ "literal_func", + /* 427 */ "literal_list", + /* 428 */ "table_alias", + /* 429 */ "expr_or_subquery", + /* 430 */ "pseudo_column", + /* 431 */ "column_reference", + /* 432 */ "function_expression", + /* 433 */ "case_when_expression", + /* 434 */ "star_func", + /* 435 */ "star_func_para_list", + /* 436 */ "noarg_func", + /* 437 */ "other_para_list", + /* 438 */ "star_func_para", + /* 439 */ "when_then_list", + /* 440 */ "case_when_else_opt", + /* 441 */ "common_expression", + /* 442 */ "when_then_expr", + /* 443 */ "predicate", + /* 444 */ "compare_op", + /* 445 */ "in_op", + /* 446 */ "in_predicate_value", + /* 447 */ "boolean_value_expression", + /* 448 */ "boolean_primary", + /* 449 */ "from_clause_opt", + /* 450 */ "table_reference_list", + /* 451 */ "table_reference", + /* 452 */ "table_primary", + /* 453 */ "joined_table", + /* 454 */ "alias_opt", + /* 455 */ "subquery", + /* 456 */ "parenthesized_joined_table", + /* 457 */ "join_type", + /* 458 */ "query_specification", + /* 459 */ "set_quantifier_opt", + /* 460 */ "select_list", + /* 461 */ "partition_by_clause_opt", + /* 462 */ "range_opt", + /* 463 */ "every_opt", + /* 464 */ "fill_opt", + /* 465 */ "twindow_clause_opt", + /* 466 */ "group_by_clause_opt", + /* 467 */ "having_clause_opt", + /* 468 */ "select_item", + /* 469 */ "partition_list", + /* 470 */ "partition_item", + /* 471 */ "fill_mode", + /* 472 */ "group_by_list", + /* 473 */ "query_expression", + /* 474 */ "query_simple", + /* 475 */ "order_by_clause_opt", + /* 476 */ "slimit_clause_opt", + /* 477 */ "limit_clause_opt", + /* 478 */ "union_query_expression", + /* 479 */ "query_simple_or_subquery", + /* 480 */ "sort_specification_list", + /* 481 */ "sort_specification", + /* 482 */ "ordering_specification_opt", + /* 483 */ "null_ordering_opt", }; #endif /* defined(YYCOVERAGE) || !defined(NDEBUG) */ @@ -2036,547 +2035,550 @@ static const char *const yyRuleName[] = { /* 47 */ "cmd ::= CREATE DNODE dnode_endpoint PORT NK_INTEGER", /* 48 */ "cmd ::= DROP DNODE NK_INTEGER force_opt", /* 49 */ "cmd ::= DROP DNODE dnode_endpoint force_opt", - /* 50 */ "cmd ::= ALTER DNODE NK_INTEGER NK_STRING", - /* 51 */ "cmd ::= ALTER DNODE NK_INTEGER NK_STRING NK_STRING", - /* 52 */ "cmd ::= ALTER ALL DNODES NK_STRING", - /* 53 */ "cmd ::= ALTER ALL DNODES NK_STRING NK_STRING", - /* 54 */ "cmd ::= RESTORE DNODE NK_INTEGER", - /* 55 */ "dnode_endpoint ::= NK_STRING", - /* 56 */ "dnode_endpoint ::= NK_ID", - /* 57 */ "dnode_endpoint ::= NK_IPTOKEN", - /* 58 */ "force_opt ::=", - /* 59 */ "force_opt ::= FORCE", - /* 60 */ "cmd ::= ALTER LOCAL NK_STRING", - /* 61 */ "cmd ::= ALTER LOCAL NK_STRING NK_STRING", - /* 62 */ "cmd ::= CREATE QNODE ON DNODE NK_INTEGER", - /* 63 */ "cmd ::= DROP QNODE ON DNODE NK_INTEGER", - /* 64 */ "cmd ::= RESTORE QNODE ON DNODE NK_INTEGER", - /* 65 */ "cmd ::= CREATE BNODE ON DNODE NK_INTEGER", - /* 66 */ "cmd ::= DROP BNODE ON DNODE NK_INTEGER", - /* 67 */ "cmd ::= CREATE SNODE ON DNODE NK_INTEGER", - /* 68 */ "cmd ::= DROP SNODE ON DNODE NK_INTEGER", - /* 69 */ "cmd ::= CREATE MNODE ON DNODE NK_INTEGER", - /* 70 */ "cmd ::= DROP MNODE ON DNODE NK_INTEGER", - /* 71 */ "cmd ::= RESTORE MNODE ON DNODE NK_INTEGER", - /* 72 */ "cmd ::= RESTORE VNODE ON DNODE NK_INTEGER", - /* 73 */ "cmd ::= CREATE DATABASE not_exists_opt db_name db_options", - /* 74 */ "cmd ::= DROP DATABASE exists_opt db_name", - /* 75 */ "cmd ::= USE db_name", - /* 76 */ "cmd ::= ALTER DATABASE db_name alter_db_options", - /* 77 */ "cmd ::= FLUSH DATABASE db_name", - /* 78 */ "cmd ::= TRIM DATABASE db_name speed_opt", - /* 79 */ "cmd ::= COMPACT DATABASE db_name start_opt end_opt", - /* 80 */ "not_exists_opt ::= IF NOT EXISTS", - /* 81 */ "not_exists_opt ::=", - /* 82 */ "exists_opt ::= IF EXISTS", - /* 83 */ "exists_opt ::=", - /* 84 */ "db_options ::=", - /* 85 */ "db_options ::= db_options BUFFER NK_INTEGER", - /* 86 */ "db_options ::= db_options CACHEMODEL NK_STRING", - /* 87 */ "db_options ::= db_options CACHESIZE NK_INTEGER", - /* 88 */ "db_options ::= db_options COMP NK_INTEGER", - /* 89 */ "db_options ::= db_options DURATION NK_INTEGER", - /* 90 */ "db_options ::= db_options DURATION NK_VARIABLE", - /* 91 */ "db_options ::= db_options MAXROWS NK_INTEGER", - /* 92 */ "db_options ::= db_options MINROWS NK_INTEGER", - /* 93 */ "db_options ::= db_options KEEP integer_list", - /* 94 */ "db_options ::= db_options KEEP variable_list", - /* 95 */ "db_options ::= db_options PAGES NK_INTEGER", - /* 96 */ "db_options ::= db_options PAGESIZE NK_INTEGER", - /* 97 */ "db_options ::= db_options TSDB_PAGESIZE NK_INTEGER", - /* 98 */ "db_options ::= db_options PRECISION NK_STRING", - /* 99 */ "db_options ::= db_options REPLICA NK_INTEGER", - /* 100 */ "db_options ::= db_options VGROUPS NK_INTEGER", - /* 101 */ "db_options ::= db_options SINGLE_STABLE NK_INTEGER", - /* 102 */ "db_options ::= db_options RETENTIONS retention_list", - /* 103 */ "db_options ::= db_options SCHEMALESS NK_INTEGER", - /* 104 */ "db_options ::= db_options WAL_LEVEL NK_INTEGER", - /* 105 */ "db_options ::= db_options WAL_FSYNC_PERIOD NK_INTEGER", - /* 106 */ "db_options ::= db_options WAL_RETENTION_PERIOD NK_INTEGER", - /* 107 */ "db_options ::= db_options WAL_RETENTION_PERIOD NK_MINUS NK_INTEGER", - /* 108 */ "db_options ::= db_options WAL_RETENTION_SIZE NK_INTEGER", - /* 109 */ "db_options ::= db_options WAL_RETENTION_SIZE NK_MINUS NK_INTEGER", - /* 110 */ "db_options ::= db_options WAL_ROLL_PERIOD NK_INTEGER", - /* 111 */ "db_options ::= db_options WAL_SEGMENT_SIZE NK_INTEGER", - /* 112 */ "db_options ::= db_options STT_TRIGGER NK_INTEGER", - /* 113 */ "db_options ::= db_options TABLE_PREFIX signed", - /* 114 */ "db_options ::= db_options TABLE_SUFFIX signed", - /* 115 */ "alter_db_options ::= alter_db_option", - /* 116 */ "alter_db_options ::= alter_db_options alter_db_option", - /* 117 */ "alter_db_option ::= BUFFER NK_INTEGER", - /* 118 */ "alter_db_option ::= CACHEMODEL NK_STRING", - /* 119 */ "alter_db_option ::= CACHESIZE NK_INTEGER", - /* 120 */ "alter_db_option ::= WAL_FSYNC_PERIOD NK_INTEGER", - /* 121 */ "alter_db_option ::= KEEP integer_list", - /* 122 */ "alter_db_option ::= KEEP variable_list", - /* 123 */ "alter_db_option ::= PAGES NK_INTEGER", - /* 124 */ "alter_db_option ::= REPLICA NK_INTEGER", - /* 125 */ "alter_db_option ::= WAL_LEVEL NK_INTEGER", - /* 126 */ "alter_db_option ::= STT_TRIGGER NK_INTEGER", - /* 127 */ "alter_db_option ::= MINROWS NK_INTEGER", - /* 128 */ "alter_db_option ::= WAL_RETENTION_PERIOD NK_INTEGER", - /* 129 */ "alter_db_option ::= WAL_RETENTION_PERIOD NK_MINUS NK_INTEGER", - /* 130 */ "alter_db_option ::= WAL_RETENTION_SIZE NK_INTEGER", - /* 131 */ "alter_db_option ::= WAL_RETENTION_SIZE NK_MINUS NK_INTEGER", - /* 132 */ "integer_list ::= NK_INTEGER", - /* 133 */ "integer_list ::= integer_list NK_COMMA NK_INTEGER", - /* 134 */ "variable_list ::= NK_VARIABLE", - /* 135 */ "variable_list ::= variable_list NK_COMMA NK_VARIABLE", - /* 136 */ "retention_list ::= retention", - /* 137 */ "retention_list ::= retention_list NK_COMMA retention", - /* 138 */ "retention ::= NK_VARIABLE NK_COLON NK_VARIABLE", - /* 139 */ "speed_opt ::=", - /* 140 */ "speed_opt ::= MAX_SPEED NK_INTEGER", - /* 141 */ "start_opt ::=", - /* 142 */ "start_opt ::= START WITH NK_INTEGER", - /* 143 */ "start_opt ::= START WITH NK_STRING", - /* 144 */ "start_opt ::= START WITH TIMESTAMP NK_STRING", - /* 145 */ "end_opt ::=", - /* 146 */ "end_opt ::= END WITH NK_INTEGER", - /* 147 */ "end_opt ::= END WITH NK_STRING", - /* 148 */ "end_opt ::= END WITH TIMESTAMP NK_STRING", - /* 149 */ "cmd ::= CREATE TABLE not_exists_opt full_table_name NK_LP column_def_list NK_RP tags_def_opt table_options", - /* 150 */ "cmd ::= CREATE TABLE multi_create_clause", - /* 151 */ "cmd ::= CREATE STABLE not_exists_opt full_table_name NK_LP column_def_list NK_RP tags_def table_options", - /* 152 */ "cmd ::= DROP TABLE multi_drop_clause", - /* 153 */ "cmd ::= DROP STABLE exists_opt full_table_name", - /* 154 */ "cmd ::= ALTER TABLE alter_table_clause", - /* 155 */ "cmd ::= ALTER STABLE alter_table_clause", - /* 156 */ "alter_table_clause ::= full_table_name alter_table_options", - /* 157 */ "alter_table_clause ::= full_table_name ADD COLUMN column_name type_name", - /* 158 */ "alter_table_clause ::= full_table_name DROP COLUMN column_name", - /* 159 */ "alter_table_clause ::= full_table_name MODIFY COLUMN column_name type_name", - /* 160 */ "alter_table_clause ::= full_table_name RENAME COLUMN column_name column_name", - /* 161 */ "alter_table_clause ::= full_table_name ADD TAG column_name type_name", - /* 162 */ "alter_table_clause ::= full_table_name DROP TAG column_name", - /* 163 */ "alter_table_clause ::= full_table_name MODIFY TAG column_name type_name", - /* 164 */ "alter_table_clause ::= full_table_name RENAME TAG column_name column_name", - /* 165 */ "alter_table_clause ::= full_table_name SET TAG column_name NK_EQ signed_literal", - /* 166 */ "multi_create_clause ::= create_subtable_clause", - /* 167 */ "multi_create_clause ::= multi_create_clause create_subtable_clause", - /* 168 */ "create_subtable_clause ::= not_exists_opt full_table_name USING full_table_name specific_cols_opt TAGS NK_LP expression_list NK_RP table_options", - /* 169 */ "multi_drop_clause ::= drop_table_clause", - /* 170 */ "multi_drop_clause ::= multi_drop_clause NK_COMMA drop_table_clause", - /* 171 */ "drop_table_clause ::= exists_opt full_table_name", - /* 172 */ "specific_cols_opt ::=", - /* 173 */ "specific_cols_opt ::= NK_LP col_name_list NK_RP", - /* 174 */ "full_table_name ::= table_name", - /* 175 */ "full_table_name ::= db_name NK_DOT table_name", - /* 176 */ "column_def_list ::= column_def", - /* 177 */ "column_def_list ::= column_def_list NK_COMMA column_def", - /* 178 */ "column_def ::= column_name type_name", - /* 179 */ "type_name ::= BOOL", - /* 180 */ "type_name ::= TINYINT", - /* 181 */ "type_name ::= SMALLINT", - /* 182 */ "type_name ::= INT", - /* 183 */ "type_name ::= INTEGER", - /* 184 */ "type_name ::= BIGINT", - /* 185 */ "type_name ::= FLOAT", - /* 186 */ "type_name ::= DOUBLE", - /* 187 */ "type_name ::= BINARY NK_LP NK_INTEGER NK_RP", - /* 188 */ "type_name ::= TIMESTAMP", - /* 189 */ "type_name ::= NCHAR NK_LP NK_INTEGER NK_RP", - /* 190 */ "type_name ::= TINYINT UNSIGNED", - /* 191 */ "type_name ::= SMALLINT UNSIGNED", - /* 192 */ "type_name ::= INT UNSIGNED", - /* 193 */ "type_name ::= BIGINT UNSIGNED", - /* 194 */ "type_name ::= JSON", - /* 195 */ "type_name ::= VARCHAR NK_LP NK_INTEGER NK_RP", - /* 196 */ "type_name ::= MEDIUMBLOB", - /* 197 */ "type_name ::= BLOB", - /* 198 */ "type_name ::= VARBINARY NK_LP NK_INTEGER NK_RP", - /* 199 */ "type_name ::= DECIMAL", - /* 200 */ "type_name ::= DECIMAL NK_LP NK_INTEGER NK_RP", - /* 201 */ "type_name ::= DECIMAL NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP", - /* 202 */ "tags_def_opt ::=", - /* 203 */ "tags_def_opt ::= tags_def", - /* 204 */ "tags_def ::= TAGS NK_LP column_def_list NK_RP", - /* 205 */ "table_options ::=", - /* 206 */ "table_options ::= table_options COMMENT NK_STRING", - /* 207 */ "table_options ::= table_options MAX_DELAY duration_list", - /* 208 */ "table_options ::= table_options WATERMARK duration_list", - /* 209 */ "table_options ::= table_options ROLLUP NK_LP rollup_func_list NK_RP", - /* 210 */ "table_options ::= table_options TTL NK_INTEGER", - /* 211 */ "table_options ::= table_options SMA NK_LP col_name_list NK_RP", - /* 212 */ "table_options ::= table_options DELETE_MARK duration_list", - /* 213 */ "alter_table_options ::= alter_table_option", - /* 214 */ "alter_table_options ::= alter_table_options alter_table_option", - /* 215 */ "alter_table_option ::= COMMENT NK_STRING", - /* 216 */ "alter_table_option ::= TTL NK_INTEGER", - /* 217 */ "duration_list ::= duration_literal", - /* 218 */ "duration_list ::= duration_list NK_COMMA duration_literal", - /* 219 */ "rollup_func_list ::= rollup_func_name", - /* 220 */ "rollup_func_list ::= rollup_func_list NK_COMMA rollup_func_name", - /* 221 */ "rollup_func_name ::= function_name", - /* 222 */ "rollup_func_name ::= FIRST", - /* 223 */ "rollup_func_name ::= LAST", - /* 224 */ "col_name_list ::= col_name", - /* 225 */ "col_name_list ::= col_name_list NK_COMMA col_name", - /* 226 */ "col_name ::= column_name", - /* 227 */ "cmd ::= SHOW DNODES", - /* 228 */ "cmd ::= SHOW USERS", - /* 229 */ "cmd ::= SHOW USER PRIVILEGES", - /* 230 */ "cmd ::= SHOW DATABASES", - /* 231 */ "cmd ::= SHOW db_name_cond_opt TABLES like_pattern_opt", - /* 232 */ "cmd ::= SHOW db_name_cond_opt STABLES like_pattern_opt", - /* 233 */ "cmd ::= SHOW db_name_cond_opt VGROUPS", - /* 234 */ "cmd ::= SHOW MNODES", - /* 235 */ "cmd ::= SHOW QNODES", - /* 236 */ "cmd ::= SHOW FUNCTIONS", - /* 237 */ "cmd ::= SHOW INDEXES FROM table_name_cond from_db_opt", - /* 238 */ "cmd ::= SHOW STREAMS", - /* 239 */ "cmd ::= SHOW ACCOUNTS", - /* 240 */ "cmd ::= SHOW APPS", - /* 241 */ "cmd ::= SHOW CONNECTIONS", - /* 242 */ "cmd ::= SHOW LICENCES", - /* 243 */ "cmd ::= SHOW GRANTS", - /* 244 */ "cmd ::= SHOW CREATE DATABASE db_name", - /* 245 */ "cmd ::= SHOW CREATE TABLE full_table_name", - /* 246 */ "cmd ::= SHOW CREATE STABLE full_table_name", - /* 247 */ "cmd ::= SHOW QUERIES", - /* 248 */ "cmd ::= SHOW SCORES", - /* 249 */ "cmd ::= SHOW TOPICS", - /* 250 */ "cmd ::= SHOW VARIABLES", - /* 251 */ "cmd ::= SHOW CLUSTER VARIABLES", - /* 252 */ "cmd ::= SHOW LOCAL VARIABLES", - /* 253 */ "cmd ::= SHOW DNODE NK_INTEGER VARIABLES like_pattern_opt", - /* 254 */ "cmd ::= SHOW BNODES", - /* 255 */ "cmd ::= SHOW SNODES", - /* 256 */ "cmd ::= SHOW CLUSTER", - /* 257 */ "cmd ::= SHOW TRANSACTIONS", - /* 258 */ "cmd ::= SHOW TABLE DISTRIBUTED full_table_name", - /* 259 */ "cmd ::= SHOW CONSUMERS", - /* 260 */ "cmd ::= SHOW SUBSCRIPTIONS", - /* 261 */ "cmd ::= SHOW TAGS FROM table_name_cond from_db_opt", - /* 262 */ "cmd ::= SHOW TABLE TAGS tag_list_opt FROM table_name_cond from_db_opt", - /* 263 */ "cmd ::= SHOW VNODES NK_INTEGER", - /* 264 */ "cmd ::= SHOW VNODES NK_STRING", - /* 265 */ "cmd ::= SHOW db_name_cond_opt ALIVE", - /* 266 */ "cmd ::= SHOW CLUSTER ALIVE", - /* 267 */ "db_name_cond_opt ::=", - /* 268 */ "db_name_cond_opt ::= db_name NK_DOT", - /* 269 */ "like_pattern_opt ::=", - /* 270 */ "like_pattern_opt ::= LIKE NK_STRING", - /* 271 */ "table_name_cond ::= table_name", - /* 272 */ "from_db_opt ::=", - /* 273 */ "from_db_opt ::= FROM db_name", - /* 274 */ "tag_list_opt ::=", - /* 275 */ "tag_list_opt ::= tag_item", - /* 276 */ "tag_list_opt ::= tag_list_opt NK_COMMA tag_item", - /* 277 */ "tag_item ::= TBNAME", - /* 278 */ "tag_item ::= QTAGS", - /* 279 */ "tag_item ::= column_name", - /* 280 */ "tag_item ::= column_name column_alias", - /* 281 */ "tag_item ::= column_name AS column_alias", - /* 282 */ "cmd ::= CREATE SMA INDEX not_exists_opt full_index_name ON full_table_name index_options", - /* 283 */ "cmd ::= CREATE INDEX not_exists_opt full_index_name ON full_table_name NK_LP col_name_list NK_RP", - /* 284 */ "cmd ::= DROP INDEX exists_opt full_index_name", - /* 285 */ "full_index_name ::= index_name", - /* 286 */ "full_index_name ::= db_name NK_DOT index_name", - /* 287 */ "index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_RP sliding_opt sma_stream_opt", - /* 288 */ "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", - /* 289 */ "func_list ::= func", - /* 290 */ "func_list ::= func_list NK_COMMA func", - /* 291 */ "func ::= sma_func_name NK_LP expression_list NK_RP", - /* 292 */ "sma_func_name ::= function_name", - /* 293 */ "sma_func_name ::= COUNT", - /* 294 */ "sma_func_name ::= FIRST", - /* 295 */ "sma_func_name ::= LAST", - /* 296 */ "sma_func_name ::= LAST_ROW", - /* 297 */ "sma_stream_opt ::=", - /* 298 */ "sma_stream_opt ::= sma_stream_opt WATERMARK duration_literal", - /* 299 */ "sma_stream_opt ::= sma_stream_opt MAX_DELAY duration_literal", - /* 300 */ "sma_stream_opt ::= sma_stream_opt DELETE_MARK duration_literal", - /* 301 */ "cmd ::= CREATE TOPIC not_exists_opt topic_name AS query_or_subquery", - /* 302 */ "cmd ::= CREATE TOPIC not_exists_opt topic_name AS DATABASE db_name", - /* 303 */ "cmd ::= CREATE TOPIC not_exists_opt topic_name WITH META AS DATABASE db_name", - /* 304 */ "cmd ::= CREATE TOPIC not_exists_opt topic_name AS STABLE full_table_name", - /* 305 */ "cmd ::= CREATE TOPIC not_exists_opt topic_name WITH META AS STABLE full_table_name", - /* 306 */ "cmd ::= DROP TOPIC exists_opt topic_name", - /* 307 */ "cmd ::= DROP CONSUMER GROUP exists_opt cgroup_name ON topic_name", - /* 308 */ "cmd ::= DESC full_table_name", - /* 309 */ "cmd ::= DESCRIBE full_table_name", - /* 310 */ "cmd ::= RESET QUERY CACHE", - /* 311 */ "cmd ::= EXPLAIN analyze_opt explain_options query_or_subquery", - /* 312 */ "cmd ::= EXPLAIN analyze_opt explain_options insert_query", - /* 313 */ "analyze_opt ::=", - /* 314 */ "analyze_opt ::= ANALYZE", - /* 315 */ "explain_options ::=", - /* 316 */ "explain_options ::= explain_options VERBOSE NK_BOOL", - /* 317 */ "explain_options ::= explain_options RATIO NK_FLOAT", - /* 318 */ "cmd ::= CREATE or_replace_opt agg_func_opt FUNCTION not_exists_opt function_name AS NK_STRING OUTPUTTYPE type_name bufsize_opt language_opt", - /* 319 */ "cmd ::= DROP FUNCTION exists_opt function_name", - /* 320 */ "agg_func_opt ::=", - /* 321 */ "agg_func_opt ::= AGGREGATE", - /* 322 */ "bufsize_opt ::=", - /* 323 */ "bufsize_opt ::= BUFSIZE NK_INTEGER", - /* 324 */ "language_opt ::=", - /* 325 */ "language_opt ::= LANGUAGE NK_STRING", - /* 326 */ "or_replace_opt ::=", - /* 327 */ "or_replace_opt ::= OR REPLACE", - /* 328 */ "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", - /* 329 */ "cmd ::= DROP STREAM exists_opt stream_name", - /* 330 */ "cmd ::= PAUSE STREAM exists_opt stream_name", - /* 331 */ "cmd ::= RESUME STREAM exists_opt ignore_opt stream_name", - /* 332 */ "col_list_opt ::=", - /* 333 */ "col_list_opt ::= NK_LP col_name_list NK_RP", - /* 334 */ "tag_def_or_ref_opt ::=", - /* 335 */ "tag_def_or_ref_opt ::= tags_def", - /* 336 */ "tag_def_or_ref_opt ::= TAGS NK_LP col_name_list NK_RP", - /* 337 */ "stream_options ::=", - /* 338 */ "stream_options ::= stream_options TRIGGER AT_ONCE", - /* 339 */ "stream_options ::= stream_options TRIGGER WINDOW_CLOSE", - /* 340 */ "stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal", - /* 341 */ "stream_options ::= stream_options WATERMARK duration_literal", - /* 342 */ "stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER", - /* 343 */ "stream_options ::= stream_options FILL_HISTORY NK_INTEGER", - /* 344 */ "stream_options ::= stream_options DELETE_MARK duration_literal", - /* 345 */ "stream_options ::= stream_options IGNORE UPDATE NK_INTEGER", - /* 346 */ "subtable_opt ::=", - /* 347 */ "subtable_opt ::= SUBTABLE NK_LP expression NK_RP", - /* 348 */ "ignore_opt ::=", - /* 349 */ "ignore_opt ::= IGNORE UNTREATED", - /* 350 */ "cmd ::= KILL CONNECTION NK_INTEGER", - /* 351 */ "cmd ::= KILL QUERY NK_STRING", - /* 352 */ "cmd ::= KILL TRANSACTION NK_INTEGER", - /* 353 */ "cmd ::= BALANCE VGROUP", - /* 354 */ "cmd ::= BALANCE VGROUP LEADER", - /* 355 */ "cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER", - /* 356 */ "cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list", - /* 357 */ "cmd ::= SPLIT VGROUP NK_INTEGER", - /* 358 */ "dnode_list ::= DNODE NK_INTEGER", - /* 359 */ "dnode_list ::= dnode_list DNODE NK_INTEGER", - /* 360 */ "cmd ::= DELETE FROM full_table_name where_clause_opt", - /* 361 */ "cmd ::= query_or_subquery", - /* 362 */ "cmd ::= insert_query", - /* 363 */ "insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery", - /* 364 */ "insert_query ::= INSERT INTO full_table_name query_or_subquery", - /* 365 */ "literal ::= NK_INTEGER", - /* 366 */ "literal ::= NK_FLOAT", - /* 367 */ "literal ::= NK_STRING", - /* 368 */ "literal ::= NK_BOOL", - /* 369 */ "literal ::= TIMESTAMP NK_STRING", - /* 370 */ "literal ::= duration_literal", - /* 371 */ "literal ::= NULL", - /* 372 */ "literal ::= NK_QUESTION", - /* 373 */ "duration_literal ::= NK_VARIABLE", - /* 374 */ "signed ::= NK_INTEGER", - /* 375 */ "signed ::= NK_PLUS NK_INTEGER", - /* 376 */ "signed ::= NK_MINUS NK_INTEGER", - /* 377 */ "signed ::= NK_FLOAT", - /* 378 */ "signed ::= NK_PLUS NK_FLOAT", - /* 379 */ "signed ::= NK_MINUS NK_FLOAT", - /* 380 */ "signed_literal ::= signed", - /* 381 */ "signed_literal ::= NK_STRING", - /* 382 */ "signed_literal ::= NK_BOOL", - /* 383 */ "signed_literal ::= TIMESTAMP NK_STRING", - /* 384 */ "signed_literal ::= duration_literal", - /* 385 */ "signed_literal ::= NULL", - /* 386 */ "signed_literal ::= literal_func", - /* 387 */ "signed_literal ::= NK_QUESTION", - /* 388 */ "literal_list ::= signed_literal", - /* 389 */ "literal_list ::= literal_list NK_COMMA signed_literal", - /* 390 */ "db_name ::= NK_ID", - /* 391 */ "table_name ::= NK_ID", - /* 392 */ "column_name ::= NK_ID", - /* 393 */ "function_name ::= NK_ID", - /* 394 */ "table_alias ::= NK_ID", - /* 395 */ "column_alias ::= NK_ID", - /* 396 */ "user_name ::= NK_ID", - /* 397 */ "topic_name ::= NK_ID", - /* 398 */ "stream_name ::= NK_ID", - /* 399 */ "cgroup_name ::= NK_ID", - /* 400 */ "index_name ::= NK_ID", - /* 401 */ "expr_or_subquery ::= expression", - /* 402 */ "expression ::= literal", - /* 403 */ "expression ::= pseudo_column", - /* 404 */ "expression ::= column_reference", - /* 405 */ "expression ::= function_expression", - /* 406 */ "expression ::= case_when_expression", - /* 407 */ "expression ::= NK_LP expression NK_RP", - /* 408 */ "expression ::= NK_PLUS expr_or_subquery", - /* 409 */ "expression ::= NK_MINUS expr_or_subquery", - /* 410 */ "expression ::= expr_or_subquery NK_PLUS expr_or_subquery", - /* 411 */ "expression ::= expr_or_subquery NK_MINUS expr_or_subquery", - /* 412 */ "expression ::= expr_or_subquery NK_STAR expr_or_subquery", - /* 413 */ "expression ::= expr_or_subquery NK_SLASH expr_or_subquery", - /* 414 */ "expression ::= expr_or_subquery NK_REM expr_or_subquery", - /* 415 */ "expression ::= column_reference NK_ARROW NK_STRING", - /* 416 */ "expression ::= expr_or_subquery NK_BITAND expr_or_subquery", - /* 417 */ "expression ::= expr_or_subquery NK_BITOR expr_or_subquery", - /* 418 */ "expression_list ::= expr_or_subquery", - /* 419 */ "expression_list ::= expression_list NK_COMMA expr_or_subquery", - /* 420 */ "column_reference ::= column_name", - /* 421 */ "column_reference ::= table_name NK_DOT column_name", - /* 422 */ "pseudo_column ::= ROWTS", - /* 423 */ "pseudo_column ::= TBNAME", - /* 424 */ "pseudo_column ::= table_name NK_DOT TBNAME", - /* 425 */ "pseudo_column ::= QSTART", - /* 426 */ "pseudo_column ::= QEND", - /* 427 */ "pseudo_column ::= QDURATION", - /* 428 */ "pseudo_column ::= WSTART", - /* 429 */ "pseudo_column ::= WEND", - /* 430 */ "pseudo_column ::= WDURATION", - /* 431 */ "pseudo_column ::= IROWTS", - /* 432 */ "pseudo_column ::= ISFILLED", - /* 433 */ "pseudo_column ::= QTAGS", - /* 434 */ "function_expression ::= function_name NK_LP expression_list NK_RP", - /* 435 */ "function_expression ::= star_func NK_LP star_func_para_list NK_RP", - /* 436 */ "function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP", - /* 437 */ "function_expression ::= literal_func", - /* 438 */ "literal_func ::= noarg_func NK_LP NK_RP", - /* 439 */ "literal_func ::= NOW", - /* 440 */ "noarg_func ::= NOW", - /* 441 */ "noarg_func ::= TODAY", - /* 442 */ "noarg_func ::= TIMEZONE", - /* 443 */ "noarg_func ::= DATABASE", - /* 444 */ "noarg_func ::= CLIENT_VERSION", - /* 445 */ "noarg_func ::= SERVER_VERSION", - /* 446 */ "noarg_func ::= SERVER_STATUS", - /* 447 */ "noarg_func ::= CURRENT_USER", - /* 448 */ "noarg_func ::= USER", - /* 449 */ "star_func ::= COUNT", - /* 450 */ "star_func ::= FIRST", - /* 451 */ "star_func ::= LAST", - /* 452 */ "star_func ::= LAST_ROW", - /* 453 */ "star_func_para_list ::= NK_STAR", - /* 454 */ "star_func_para_list ::= other_para_list", - /* 455 */ "other_para_list ::= star_func_para", - /* 456 */ "other_para_list ::= other_para_list NK_COMMA star_func_para", - /* 457 */ "star_func_para ::= expr_or_subquery", - /* 458 */ "star_func_para ::= table_name NK_DOT NK_STAR", - /* 459 */ "case_when_expression ::= CASE when_then_list case_when_else_opt END", - /* 460 */ "case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END", - /* 461 */ "when_then_list ::= when_then_expr", - /* 462 */ "when_then_list ::= when_then_list when_then_expr", - /* 463 */ "when_then_expr ::= WHEN common_expression THEN common_expression", - /* 464 */ "case_when_else_opt ::=", - /* 465 */ "case_when_else_opt ::= ELSE common_expression", - /* 466 */ "predicate ::= expr_or_subquery compare_op expr_or_subquery", - /* 467 */ "predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery", - /* 468 */ "predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery", - /* 469 */ "predicate ::= expr_or_subquery IS NULL", - /* 470 */ "predicate ::= expr_or_subquery IS NOT NULL", - /* 471 */ "predicate ::= expr_or_subquery in_op in_predicate_value", - /* 472 */ "compare_op ::= NK_LT", - /* 473 */ "compare_op ::= NK_GT", - /* 474 */ "compare_op ::= NK_LE", - /* 475 */ "compare_op ::= NK_GE", - /* 476 */ "compare_op ::= NK_NE", - /* 477 */ "compare_op ::= NK_EQ", - /* 478 */ "compare_op ::= LIKE", - /* 479 */ "compare_op ::= NOT LIKE", - /* 480 */ "compare_op ::= MATCH", - /* 481 */ "compare_op ::= NMATCH", - /* 482 */ "compare_op ::= CONTAINS", - /* 483 */ "in_op ::= IN", - /* 484 */ "in_op ::= NOT IN", - /* 485 */ "in_predicate_value ::= NK_LP literal_list NK_RP", - /* 486 */ "boolean_value_expression ::= boolean_primary", - /* 487 */ "boolean_value_expression ::= NOT boolean_primary", - /* 488 */ "boolean_value_expression ::= boolean_value_expression OR boolean_value_expression", - /* 489 */ "boolean_value_expression ::= boolean_value_expression AND boolean_value_expression", - /* 490 */ "boolean_primary ::= predicate", - /* 491 */ "boolean_primary ::= NK_LP boolean_value_expression NK_RP", - /* 492 */ "common_expression ::= expr_or_subquery", - /* 493 */ "common_expression ::= boolean_value_expression", - /* 494 */ "from_clause_opt ::=", - /* 495 */ "from_clause_opt ::= FROM table_reference_list", - /* 496 */ "table_reference_list ::= table_reference", - /* 497 */ "table_reference_list ::= table_reference_list NK_COMMA table_reference", - /* 498 */ "table_reference ::= table_primary", - /* 499 */ "table_reference ::= joined_table", - /* 500 */ "table_primary ::= table_name alias_opt", - /* 501 */ "table_primary ::= db_name NK_DOT table_name alias_opt", - /* 502 */ "table_primary ::= subquery alias_opt", - /* 503 */ "table_primary ::= parenthesized_joined_table", - /* 504 */ "alias_opt ::=", - /* 505 */ "alias_opt ::= table_alias", - /* 506 */ "alias_opt ::= AS table_alias", - /* 507 */ "parenthesized_joined_table ::= NK_LP joined_table NK_RP", - /* 508 */ "parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP", - /* 509 */ "joined_table ::= table_reference join_type JOIN table_reference ON search_condition", - /* 510 */ "join_type ::=", - /* 511 */ "join_type ::= INNER", - /* 512 */ "query_specification ::= SELECT set_quantifier_opt select_list from_clause_opt where_clause_opt partition_by_clause_opt range_opt every_opt fill_opt twindow_clause_opt group_by_clause_opt having_clause_opt", - /* 513 */ "set_quantifier_opt ::=", - /* 514 */ "set_quantifier_opt ::= DISTINCT", - /* 515 */ "set_quantifier_opt ::= ALL", - /* 516 */ "select_list ::= select_item", - /* 517 */ "select_list ::= select_list NK_COMMA select_item", - /* 518 */ "select_item ::= NK_STAR", - /* 519 */ "select_item ::= common_expression", - /* 520 */ "select_item ::= common_expression column_alias", - /* 521 */ "select_item ::= common_expression AS column_alias", - /* 522 */ "select_item ::= table_name NK_DOT NK_STAR", - /* 523 */ "where_clause_opt ::=", - /* 524 */ "where_clause_opt ::= WHERE search_condition", - /* 525 */ "partition_by_clause_opt ::=", - /* 526 */ "partition_by_clause_opt ::= PARTITION BY partition_list", - /* 527 */ "partition_list ::= partition_item", - /* 528 */ "partition_list ::= partition_list NK_COMMA partition_item", - /* 529 */ "partition_item ::= expr_or_subquery", - /* 530 */ "partition_item ::= expr_or_subquery column_alias", - /* 531 */ "partition_item ::= expr_or_subquery AS column_alias", - /* 532 */ "twindow_clause_opt ::=", - /* 533 */ "twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA duration_literal NK_RP", - /* 534 */ "twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP", - /* 535 */ "twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_RP sliding_opt fill_opt", - /* 536 */ "twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_COMMA duration_literal NK_RP sliding_opt fill_opt", - /* 537 */ "twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition", - /* 538 */ "sliding_opt ::=", - /* 539 */ "sliding_opt ::= SLIDING NK_LP duration_literal NK_RP", - /* 540 */ "fill_opt ::=", - /* 541 */ "fill_opt ::= FILL NK_LP fill_mode NK_RP", - /* 542 */ "fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP", - /* 543 */ "fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP", - /* 544 */ "fill_mode ::= NONE", - /* 545 */ "fill_mode ::= PREV", - /* 546 */ "fill_mode ::= NULL", - /* 547 */ "fill_mode ::= NULL_F", - /* 548 */ "fill_mode ::= LINEAR", - /* 549 */ "fill_mode ::= NEXT", - /* 550 */ "group_by_clause_opt ::=", - /* 551 */ "group_by_clause_opt ::= GROUP BY group_by_list", - /* 552 */ "group_by_list ::= expr_or_subquery", - /* 553 */ "group_by_list ::= group_by_list NK_COMMA expr_or_subquery", - /* 554 */ "having_clause_opt ::=", - /* 555 */ "having_clause_opt ::= HAVING search_condition", - /* 556 */ "range_opt ::=", - /* 557 */ "range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP", - /* 558 */ "every_opt ::=", - /* 559 */ "every_opt ::= EVERY NK_LP duration_literal NK_RP", - /* 560 */ "query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt", - /* 561 */ "query_simple ::= query_specification", - /* 562 */ "query_simple ::= union_query_expression", - /* 563 */ "union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery", - /* 564 */ "union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery", - /* 565 */ "query_simple_or_subquery ::= query_simple", - /* 566 */ "query_simple_or_subquery ::= subquery", - /* 567 */ "query_or_subquery ::= query_expression", - /* 568 */ "query_or_subquery ::= subquery", - /* 569 */ "order_by_clause_opt ::=", - /* 570 */ "order_by_clause_opt ::= ORDER BY sort_specification_list", - /* 571 */ "slimit_clause_opt ::=", - /* 572 */ "slimit_clause_opt ::= SLIMIT NK_INTEGER", - /* 573 */ "slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER", - /* 574 */ "slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER", - /* 575 */ "limit_clause_opt ::=", - /* 576 */ "limit_clause_opt ::= LIMIT NK_INTEGER", - /* 577 */ "limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER", - /* 578 */ "limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER", - /* 579 */ "subquery ::= NK_LP query_expression NK_RP", - /* 580 */ "subquery ::= NK_LP subquery NK_RP", - /* 581 */ "search_condition ::= common_expression", - /* 582 */ "sort_specification_list ::= sort_specification", - /* 583 */ "sort_specification_list ::= sort_specification_list NK_COMMA sort_specification", - /* 584 */ "sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt", - /* 585 */ "ordering_specification_opt ::=", - /* 586 */ "ordering_specification_opt ::= ASC", - /* 587 */ "ordering_specification_opt ::= DESC", - /* 588 */ "null_ordering_opt ::=", - /* 589 */ "null_ordering_opt ::= NULLS FIRST", - /* 590 */ "null_ordering_opt ::= NULLS LAST", + /* 50 */ "cmd ::= DROP DNODE NK_INTEGER unsafe_opt", + /* 51 */ "cmd ::= DROP DNODE dnode_endpoint unsafe_opt", + /* 52 */ "cmd ::= ALTER DNODE NK_INTEGER NK_STRING", + /* 53 */ "cmd ::= ALTER DNODE NK_INTEGER NK_STRING NK_STRING", + /* 54 */ "cmd ::= ALTER ALL DNODES NK_STRING", + /* 55 */ "cmd ::= ALTER ALL DNODES NK_STRING NK_STRING", + /* 56 */ "cmd ::= RESTORE DNODE NK_INTEGER", + /* 57 */ "dnode_endpoint ::= NK_STRING", + /* 58 */ "dnode_endpoint ::= NK_ID", + /* 59 */ "dnode_endpoint ::= NK_IPTOKEN", + /* 60 */ "force_opt ::=", + /* 61 */ "force_opt ::= FORCE", + /* 62 */ "unsafe_opt ::= UNSAFE", + /* 63 */ "cmd ::= ALTER LOCAL NK_STRING", + /* 64 */ "cmd ::= ALTER LOCAL NK_STRING NK_STRING", + /* 65 */ "cmd ::= CREATE QNODE ON DNODE NK_INTEGER", + /* 66 */ "cmd ::= DROP QNODE ON DNODE NK_INTEGER", + /* 67 */ "cmd ::= RESTORE QNODE ON DNODE NK_INTEGER", + /* 68 */ "cmd ::= CREATE BNODE ON DNODE NK_INTEGER", + /* 69 */ "cmd ::= DROP BNODE ON DNODE NK_INTEGER", + /* 70 */ "cmd ::= CREATE SNODE ON DNODE NK_INTEGER", + /* 71 */ "cmd ::= DROP SNODE ON DNODE NK_INTEGER", + /* 72 */ "cmd ::= CREATE MNODE ON DNODE NK_INTEGER", + /* 73 */ "cmd ::= DROP MNODE ON DNODE NK_INTEGER", + /* 74 */ "cmd ::= RESTORE MNODE ON DNODE NK_INTEGER", + /* 75 */ "cmd ::= RESTORE VNODE ON DNODE NK_INTEGER", + /* 76 */ "cmd ::= CREATE DATABASE not_exists_opt db_name db_options", + /* 77 */ "cmd ::= DROP DATABASE exists_opt db_name", + /* 78 */ "cmd ::= USE db_name", + /* 79 */ "cmd ::= ALTER DATABASE db_name alter_db_options", + /* 80 */ "cmd ::= FLUSH DATABASE db_name", + /* 81 */ "cmd ::= TRIM DATABASE db_name speed_opt", + /* 82 */ "cmd ::= COMPACT DATABASE db_name start_opt end_opt", + /* 83 */ "not_exists_opt ::= IF NOT EXISTS", + /* 84 */ "not_exists_opt ::=", + /* 85 */ "exists_opt ::= IF EXISTS", + /* 86 */ "exists_opt ::=", + /* 87 */ "db_options ::=", + /* 88 */ "db_options ::= db_options BUFFER NK_INTEGER", + /* 89 */ "db_options ::= db_options CACHEMODEL NK_STRING", + /* 90 */ "db_options ::= db_options CACHESIZE NK_INTEGER", + /* 91 */ "db_options ::= db_options COMP NK_INTEGER", + /* 92 */ "db_options ::= db_options DURATION NK_INTEGER", + /* 93 */ "db_options ::= db_options DURATION NK_VARIABLE", + /* 94 */ "db_options ::= db_options MAXROWS NK_INTEGER", + /* 95 */ "db_options ::= db_options MINROWS NK_INTEGER", + /* 96 */ "db_options ::= db_options KEEP integer_list", + /* 97 */ "db_options ::= db_options KEEP variable_list", + /* 98 */ "db_options ::= db_options PAGES NK_INTEGER", + /* 99 */ "db_options ::= db_options PAGESIZE NK_INTEGER", + /* 100 */ "db_options ::= db_options TSDB_PAGESIZE NK_INTEGER", + /* 101 */ "db_options ::= db_options PRECISION NK_STRING", + /* 102 */ "db_options ::= db_options REPLICA NK_INTEGER", + /* 103 */ "db_options ::= db_options VGROUPS NK_INTEGER", + /* 104 */ "db_options ::= db_options SINGLE_STABLE NK_INTEGER", + /* 105 */ "db_options ::= db_options RETENTIONS retention_list", + /* 106 */ "db_options ::= db_options SCHEMALESS NK_INTEGER", + /* 107 */ "db_options ::= db_options WAL_LEVEL NK_INTEGER", + /* 108 */ "db_options ::= db_options WAL_FSYNC_PERIOD NK_INTEGER", + /* 109 */ "db_options ::= db_options WAL_RETENTION_PERIOD NK_INTEGER", + /* 110 */ "db_options ::= db_options WAL_RETENTION_PERIOD NK_MINUS NK_INTEGER", + /* 111 */ "db_options ::= db_options WAL_RETENTION_SIZE NK_INTEGER", + /* 112 */ "db_options ::= db_options WAL_RETENTION_SIZE NK_MINUS NK_INTEGER", + /* 113 */ "db_options ::= db_options WAL_ROLL_PERIOD NK_INTEGER", + /* 114 */ "db_options ::= db_options WAL_SEGMENT_SIZE NK_INTEGER", + /* 115 */ "db_options ::= db_options STT_TRIGGER NK_INTEGER", + /* 116 */ "db_options ::= db_options TABLE_PREFIX signed", + /* 117 */ "db_options ::= db_options TABLE_SUFFIX signed", + /* 118 */ "alter_db_options ::= alter_db_option", + /* 119 */ "alter_db_options ::= alter_db_options alter_db_option", + /* 120 */ "alter_db_option ::= BUFFER NK_INTEGER", + /* 121 */ "alter_db_option ::= CACHEMODEL NK_STRING", + /* 122 */ "alter_db_option ::= CACHESIZE NK_INTEGER", + /* 123 */ "alter_db_option ::= WAL_FSYNC_PERIOD NK_INTEGER", + /* 124 */ "alter_db_option ::= KEEP integer_list", + /* 125 */ "alter_db_option ::= KEEP variable_list", + /* 126 */ "alter_db_option ::= PAGES NK_INTEGER", + /* 127 */ "alter_db_option ::= REPLICA NK_INTEGER", + /* 128 */ "alter_db_option ::= WAL_LEVEL NK_INTEGER", + /* 129 */ "alter_db_option ::= STT_TRIGGER NK_INTEGER", + /* 130 */ "alter_db_option ::= MINROWS NK_INTEGER", + /* 131 */ "alter_db_option ::= WAL_RETENTION_PERIOD NK_INTEGER", + /* 132 */ "alter_db_option ::= WAL_RETENTION_PERIOD NK_MINUS NK_INTEGER", + /* 133 */ "alter_db_option ::= WAL_RETENTION_SIZE NK_INTEGER", + /* 134 */ "alter_db_option ::= WAL_RETENTION_SIZE NK_MINUS NK_INTEGER", + /* 135 */ "integer_list ::= NK_INTEGER", + /* 136 */ "integer_list ::= integer_list NK_COMMA NK_INTEGER", + /* 137 */ "variable_list ::= NK_VARIABLE", + /* 138 */ "variable_list ::= variable_list NK_COMMA NK_VARIABLE", + /* 139 */ "retention_list ::= retention", + /* 140 */ "retention_list ::= retention_list NK_COMMA retention", + /* 141 */ "retention ::= NK_VARIABLE NK_COLON NK_VARIABLE", + /* 142 */ "speed_opt ::=", + /* 143 */ "speed_opt ::= MAX_SPEED NK_INTEGER", + /* 144 */ "start_opt ::=", + /* 145 */ "start_opt ::= START WITH NK_INTEGER", + /* 146 */ "start_opt ::= START WITH NK_STRING", + /* 147 */ "start_opt ::= START WITH TIMESTAMP NK_STRING", + /* 148 */ "end_opt ::=", + /* 149 */ "end_opt ::= END WITH NK_INTEGER", + /* 150 */ "end_opt ::= END WITH NK_STRING", + /* 151 */ "end_opt ::= END WITH TIMESTAMP NK_STRING", + /* 152 */ "cmd ::= CREATE TABLE not_exists_opt full_table_name NK_LP column_def_list NK_RP tags_def_opt table_options", + /* 153 */ "cmd ::= CREATE TABLE multi_create_clause", + /* 154 */ "cmd ::= CREATE STABLE not_exists_opt full_table_name NK_LP column_def_list NK_RP tags_def table_options", + /* 155 */ "cmd ::= DROP TABLE multi_drop_clause", + /* 156 */ "cmd ::= DROP STABLE exists_opt full_table_name", + /* 157 */ "cmd ::= ALTER TABLE alter_table_clause", + /* 158 */ "cmd ::= ALTER STABLE alter_table_clause", + /* 159 */ "alter_table_clause ::= full_table_name alter_table_options", + /* 160 */ "alter_table_clause ::= full_table_name ADD COLUMN column_name type_name", + /* 161 */ "alter_table_clause ::= full_table_name DROP COLUMN column_name", + /* 162 */ "alter_table_clause ::= full_table_name MODIFY COLUMN column_name type_name", + /* 163 */ "alter_table_clause ::= full_table_name RENAME COLUMN column_name column_name", + /* 164 */ "alter_table_clause ::= full_table_name ADD TAG column_name type_name", + /* 165 */ "alter_table_clause ::= full_table_name DROP TAG column_name", + /* 166 */ "alter_table_clause ::= full_table_name MODIFY TAG column_name type_name", + /* 167 */ "alter_table_clause ::= full_table_name RENAME TAG column_name column_name", + /* 168 */ "alter_table_clause ::= full_table_name SET TAG column_name NK_EQ signed_literal", + /* 169 */ "multi_create_clause ::= create_subtable_clause", + /* 170 */ "multi_create_clause ::= multi_create_clause create_subtable_clause", + /* 171 */ "create_subtable_clause ::= not_exists_opt full_table_name USING full_table_name specific_cols_opt TAGS NK_LP expression_list NK_RP table_options", + /* 172 */ "multi_drop_clause ::= drop_table_clause", + /* 173 */ "multi_drop_clause ::= multi_drop_clause NK_COMMA drop_table_clause", + /* 174 */ "drop_table_clause ::= exists_opt full_table_name", + /* 175 */ "specific_cols_opt ::=", + /* 176 */ "specific_cols_opt ::= NK_LP col_name_list NK_RP", + /* 177 */ "full_table_name ::= table_name", + /* 178 */ "full_table_name ::= db_name NK_DOT table_name", + /* 179 */ "column_def_list ::= column_def", + /* 180 */ "column_def_list ::= column_def_list NK_COMMA column_def", + /* 181 */ "column_def ::= column_name type_name", + /* 182 */ "type_name ::= BOOL", + /* 183 */ "type_name ::= TINYINT", + /* 184 */ "type_name ::= SMALLINT", + /* 185 */ "type_name ::= INT", + /* 186 */ "type_name ::= INTEGER", + /* 187 */ "type_name ::= BIGINT", + /* 188 */ "type_name ::= FLOAT", + /* 189 */ "type_name ::= DOUBLE", + /* 190 */ "type_name ::= BINARY NK_LP NK_INTEGER NK_RP", + /* 191 */ "type_name ::= TIMESTAMP", + /* 192 */ "type_name ::= NCHAR NK_LP NK_INTEGER NK_RP", + /* 193 */ "type_name ::= TINYINT UNSIGNED", + /* 194 */ "type_name ::= SMALLINT UNSIGNED", + /* 195 */ "type_name ::= INT UNSIGNED", + /* 196 */ "type_name ::= BIGINT UNSIGNED", + /* 197 */ "type_name ::= JSON", + /* 198 */ "type_name ::= VARCHAR NK_LP NK_INTEGER NK_RP", + /* 199 */ "type_name ::= MEDIUMBLOB", + /* 200 */ "type_name ::= BLOB", + /* 201 */ "type_name ::= VARBINARY NK_LP NK_INTEGER NK_RP", + /* 202 */ "type_name ::= DECIMAL", + /* 203 */ "type_name ::= DECIMAL NK_LP NK_INTEGER NK_RP", + /* 204 */ "type_name ::= DECIMAL NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP", + /* 205 */ "tags_def_opt ::=", + /* 206 */ "tags_def_opt ::= tags_def", + /* 207 */ "tags_def ::= TAGS NK_LP column_def_list NK_RP", + /* 208 */ "table_options ::=", + /* 209 */ "table_options ::= table_options COMMENT NK_STRING", + /* 210 */ "table_options ::= table_options MAX_DELAY duration_list", + /* 211 */ "table_options ::= table_options WATERMARK duration_list", + /* 212 */ "table_options ::= table_options ROLLUP NK_LP rollup_func_list NK_RP", + /* 213 */ "table_options ::= table_options TTL NK_INTEGER", + /* 214 */ "table_options ::= table_options SMA NK_LP col_name_list NK_RP", + /* 215 */ "table_options ::= table_options DELETE_MARK duration_list", + /* 216 */ "alter_table_options ::= alter_table_option", + /* 217 */ "alter_table_options ::= alter_table_options alter_table_option", + /* 218 */ "alter_table_option ::= COMMENT NK_STRING", + /* 219 */ "alter_table_option ::= TTL NK_INTEGER", + /* 220 */ "duration_list ::= duration_literal", + /* 221 */ "duration_list ::= duration_list NK_COMMA duration_literal", + /* 222 */ "rollup_func_list ::= rollup_func_name", + /* 223 */ "rollup_func_list ::= rollup_func_list NK_COMMA rollup_func_name", + /* 224 */ "rollup_func_name ::= function_name", + /* 225 */ "rollup_func_name ::= FIRST", + /* 226 */ "rollup_func_name ::= LAST", + /* 227 */ "col_name_list ::= col_name", + /* 228 */ "col_name_list ::= col_name_list NK_COMMA col_name", + /* 229 */ "col_name ::= column_name", + /* 230 */ "cmd ::= SHOW DNODES", + /* 231 */ "cmd ::= SHOW USERS", + /* 232 */ "cmd ::= SHOW USER PRIVILEGES", + /* 233 */ "cmd ::= SHOW DATABASES", + /* 234 */ "cmd ::= SHOW db_name_cond_opt TABLES like_pattern_opt", + /* 235 */ "cmd ::= SHOW db_name_cond_opt STABLES like_pattern_opt", + /* 236 */ "cmd ::= SHOW db_name_cond_opt VGROUPS", + /* 237 */ "cmd ::= SHOW MNODES", + /* 238 */ "cmd ::= SHOW QNODES", + /* 239 */ "cmd ::= SHOW FUNCTIONS", + /* 240 */ "cmd ::= SHOW INDEXES FROM table_name_cond from_db_opt", + /* 241 */ "cmd ::= SHOW STREAMS", + /* 242 */ "cmd ::= SHOW ACCOUNTS", + /* 243 */ "cmd ::= SHOW APPS", + /* 244 */ "cmd ::= SHOW CONNECTIONS", + /* 245 */ "cmd ::= SHOW LICENCES", + /* 246 */ "cmd ::= SHOW GRANTS", + /* 247 */ "cmd ::= SHOW CREATE DATABASE db_name", + /* 248 */ "cmd ::= SHOW CREATE TABLE full_table_name", + /* 249 */ "cmd ::= SHOW CREATE STABLE full_table_name", + /* 250 */ "cmd ::= SHOW QUERIES", + /* 251 */ "cmd ::= SHOW SCORES", + /* 252 */ "cmd ::= SHOW TOPICS", + /* 253 */ "cmd ::= SHOW VARIABLES", + /* 254 */ "cmd ::= SHOW CLUSTER VARIABLES", + /* 255 */ "cmd ::= SHOW LOCAL VARIABLES", + /* 256 */ "cmd ::= SHOW DNODE NK_INTEGER VARIABLES like_pattern_opt", + /* 257 */ "cmd ::= SHOW BNODES", + /* 258 */ "cmd ::= SHOW SNODES", + /* 259 */ "cmd ::= SHOW CLUSTER", + /* 260 */ "cmd ::= SHOW TRANSACTIONS", + /* 261 */ "cmd ::= SHOW TABLE DISTRIBUTED full_table_name", + /* 262 */ "cmd ::= SHOW CONSUMERS", + /* 263 */ "cmd ::= SHOW SUBSCRIPTIONS", + /* 264 */ "cmd ::= SHOW TAGS FROM table_name_cond from_db_opt", + /* 265 */ "cmd ::= SHOW TABLE TAGS tag_list_opt FROM table_name_cond from_db_opt", + /* 266 */ "cmd ::= SHOW VNODES NK_INTEGER", + /* 267 */ "cmd ::= SHOW VNODES NK_STRING", + /* 268 */ "cmd ::= SHOW db_name_cond_opt ALIVE", + /* 269 */ "cmd ::= SHOW CLUSTER ALIVE", + /* 270 */ "db_name_cond_opt ::=", + /* 271 */ "db_name_cond_opt ::= db_name NK_DOT", + /* 272 */ "like_pattern_opt ::=", + /* 273 */ "like_pattern_opt ::= LIKE NK_STRING", + /* 274 */ "table_name_cond ::= table_name", + /* 275 */ "from_db_opt ::=", + /* 276 */ "from_db_opt ::= FROM db_name", + /* 277 */ "tag_list_opt ::=", + /* 278 */ "tag_list_opt ::= tag_item", + /* 279 */ "tag_list_opt ::= tag_list_opt NK_COMMA tag_item", + /* 280 */ "tag_item ::= TBNAME", + /* 281 */ "tag_item ::= QTAGS", + /* 282 */ "tag_item ::= column_name", + /* 283 */ "tag_item ::= column_name column_alias", + /* 284 */ "tag_item ::= column_name AS column_alias", + /* 285 */ "cmd ::= CREATE SMA INDEX not_exists_opt full_index_name ON full_table_name index_options", + /* 286 */ "cmd ::= CREATE INDEX not_exists_opt full_index_name ON full_table_name NK_LP col_name_list NK_RP", + /* 287 */ "cmd ::= DROP INDEX exists_opt full_index_name", + /* 288 */ "full_index_name ::= index_name", + /* 289 */ "full_index_name ::= db_name NK_DOT index_name", + /* 290 */ "index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_RP sliding_opt sma_stream_opt", + /* 291 */ "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", + /* 292 */ "func_list ::= func", + /* 293 */ "func_list ::= func_list NK_COMMA func", + /* 294 */ "func ::= sma_func_name NK_LP expression_list NK_RP", + /* 295 */ "sma_func_name ::= function_name", + /* 296 */ "sma_func_name ::= COUNT", + /* 297 */ "sma_func_name ::= FIRST", + /* 298 */ "sma_func_name ::= LAST", + /* 299 */ "sma_func_name ::= LAST_ROW", + /* 300 */ "sma_stream_opt ::=", + /* 301 */ "sma_stream_opt ::= sma_stream_opt WATERMARK duration_literal", + /* 302 */ "sma_stream_opt ::= sma_stream_opt MAX_DELAY duration_literal", + /* 303 */ "sma_stream_opt ::= sma_stream_opt DELETE_MARK duration_literal", + /* 304 */ "cmd ::= CREATE TOPIC not_exists_opt topic_name AS query_or_subquery", + /* 305 */ "cmd ::= CREATE TOPIC not_exists_opt topic_name AS DATABASE db_name", + /* 306 */ "cmd ::= CREATE TOPIC not_exists_opt topic_name WITH META AS DATABASE db_name", + /* 307 */ "cmd ::= CREATE TOPIC not_exists_opt topic_name AS STABLE full_table_name", + /* 308 */ "cmd ::= CREATE TOPIC not_exists_opt topic_name WITH META AS STABLE full_table_name", + /* 309 */ "cmd ::= DROP TOPIC exists_opt topic_name", + /* 310 */ "cmd ::= DROP CONSUMER GROUP exists_opt cgroup_name ON topic_name", + /* 311 */ "cmd ::= DESC full_table_name", + /* 312 */ "cmd ::= DESCRIBE full_table_name", + /* 313 */ "cmd ::= RESET QUERY CACHE", + /* 314 */ "cmd ::= EXPLAIN analyze_opt explain_options query_or_subquery", + /* 315 */ "cmd ::= EXPLAIN analyze_opt explain_options insert_query", + /* 316 */ "analyze_opt ::=", + /* 317 */ "analyze_opt ::= ANALYZE", + /* 318 */ "explain_options ::=", + /* 319 */ "explain_options ::= explain_options VERBOSE NK_BOOL", + /* 320 */ "explain_options ::= explain_options RATIO NK_FLOAT", + /* 321 */ "cmd ::= CREATE or_replace_opt agg_func_opt FUNCTION not_exists_opt function_name AS NK_STRING OUTPUTTYPE type_name bufsize_opt language_opt", + /* 322 */ "cmd ::= DROP FUNCTION exists_opt function_name", + /* 323 */ "agg_func_opt ::=", + /* 324 */ "agg_func_opt ::= AGGREGATE", + /* 325 */ "bufsize_opt ::=", + /* 326 */ "bufsize_opt ::= BUFSIZE NK_INTEGER", + /* 327 */ "language_opt ::=", + /* 328 */ "language_opt ::= LANGUAGE NK_STRING", + /* 329 */ "or_replace_opt ::=", + /* 330 */ "or_replace_opt ::= OR REPLACE", + /* 331 */ "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", + /* 332 */ "cmd ::= DROP STREAM exists_opt stream_name", + /* 333 */ "cmd ::= PAUSE STREAM exists_opt stream_name", + /* 334 */ "cmd ::= RESUME STREAM exists_opt ignore_opt stream_name", + /* 335 */ "col_list_opt ::=", + /* 336 */ "col_list_opt ::= NK_LP col_name_list NK_RP", + /* 337 */ "tag_def_or_ref_opt ::=", + /* 338 */ "tag_def_or_ref_opt ::= tags_def", + /* 339 */ "tag_def_or_ref_opt ::= TAGS NK_LP col_name_list NK_RP", + /* 340 */ "stream_options ::=", + /* 341 */ "stream_options ::= stream_options TRIGGER AT_ONCE", + /* 342 */ "stream_options ::= stream_options TRIGGER WINDOW_CLOSE", + /* 343 */ "stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal", + /* 344 */ "stream_options ::= stream_options WATERMARK duration_literal", + /* 345 */ "stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER", + /* 346 */ "stream_options ::= stream_options FILL_HISTORY NK_INTEGER", + /* 347 */ "stream_options ::= stream_options DELETE_MARK duration_literal", + /* 348 */ "stream_options ::= stream_options IGNORE UPDATE NK_INTEGER", + /* 349 */ "subtable_opt ::=", + /* 350 */ "subtable_opt ::= SUBTABLE NK_LP expression NK_RP", + /* 351 */ "ignore_opt ::=", + /* 352 */ "ignore_opt ::= IGNORE UNTREATED", + /* 353 */ "cmd ::= KILL CONNECTION NK_INTEGER", + /* 354 */ "cmd ::= KILL QUERY NK_STRING", + /* 355 */ "cmd ::= KILL TRANSACTION NK_INTEGER", + /* 356 */ "cmd ::= BALANCE VGROUP", + /* 357 */ "cmd ::= BALANCE VGROUP LEADER", + /* 358 */ "cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER", + /* 359 */ "cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list", + /* 360 */ "cmd ::= SPLIT VGROUP NK_INTEGER", + /* 361 */ "dnode_list ::= DNODE NK_INTEGER", + /* 362 */ "dnode_list ::= dnode_list DNODE NK_INTEGER", + /* 363 */ "cmd ::= DELETE FROM full_table_name where_clause_opt", + /* 364 */ "cmd ::= query_or_subquery", + /* 365 */ "cmd ::= insert_query", + /* 366 */ "insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery", + /* 367 */ "insert_query ::= INSERT INTO full_table_name query_or_subquery", + /* 368 */ "literal ::= NK_INTEGER", + /* 369 */ "literal ::= NK_FLOAT", + /* 370 */ "literal ::= NK_STRING", + /* 371 */ "literal ::= NK_BOOL", + /* 372 */ "literal ::= TIMESTAMP NK_STRING", + /* 373 */ "literal ::= duration_literal", + /* 374 */ "literal ::= NULL", + /* 375 */ "literal ::= NK_QUESTION", + /* 376 */ "duration_literal ::= NK_VARIABLE", + /* 377 */ "signed ::= NK_INTEGER", + /* 378 */ "signed ::= NK_PLUS NK_INTEGER", + /* 379 */ "signed ::= NK_MINUS NK_INTEGER", + /* 380 */ "signed ::= NK_FLOAT", + /* 381 */ "signed ::= NK_PLUS NK_FLOAT", + /* 382 */ "signed ::= NK_MINUS NK_FLOAT", + /* 383 */ "signed_literal ::= signed", + /* 384 */ "signed_literal ::= NK_STRING", + /* 385 */ "signed_literal ::= NK_BOOL", + /* 386 */ "signed_literal ::= TIMESTAMP NK_STRING", + /* 387 */ "signed_literal ::= duration_literal", + /* 388 */ "signed_literal ::= NULL", + /* 389 */ "signed_literal ::= literal_func", + /* 390 */ "signed_literal ::= NK_QUESTION", + /* 391 */ "literal_list ::= signed_literal", + /* 392 */ "literal_list ::= literal_list NK_COMMA signed_literal", + /* 393 */ "db_name ::= NK_ID", + /* 394 */ "table_name ::= NK_ID", + /* 395 */ "column_name ::= NK_ID", + /* 396 */ "function_name ::= NK_ID", + /* 397 */ "table_alias ::= NK_ID", + /* 398 */ "column_alias ::= NK_ID", + /* 399 */ "user_name ::= NK_ID", + /* 400 */ "topic_name ::= NK_ID", + /* 401 */ "stream_name ::= NK_ID", + /* 402 */ "cgroup_name ::= NK_ID", + /* 403 */ "index_name ::= NK_ID", + /* 404 */ "expr_or_subquery ::= expression", + /* 405 */ "expression ::= literal", + /* 406 */ "expression ::= pseudo_column", + /* 407 */ "expression ::= column_reference", + /* 408 */ "expression ::= function_expression", + /* 409 */ "expression ::= case_when_expression", + /* 410 */ "expression ::= NK_LP expression NK_RP", + /* 411 */ "expression ::= NK_PLUS expr_or_subquery", + /* 412 */ "expression ::= NK_MINUS expr_or_subquery", + /* 413 */ "expression ::= expr_or_subquery NK_PLUS expr_or_subquery", + /* 414 */ "expression ::= expr_or_subquery NK_MINUS expr_or_subquery", + /* 415 */ "expression ::= expr_or_subquery NK_STAR expr_or_subquery", + /* 416 */ "expression ::= expr_or_subquery NK_SLASH expr_or_subquery", + /* 417 */ "expression ::= expr_or_subquery NK_REM expr_or_subquery", + /* 418 */ "expression ::= column_reference NK_ARROW NK_STRING", + /* 419 */ "expression ::= expr_or_subquery NK_BITAND expr_or_subquery", + /* 420 */ "expression ::= expr_or_subquery NK_BITOR expr_or_subquery", + /* 421 */ "expression_list ::= expr_or_subquery", + /* 422 */ "expression_list ::= expression_list NK_COMMA expr_or_subquery", + /* 423 */ "column_reference ::= column_name", + /* 424 */ "column_reference ::= table_name NK_DOT column_name", + /* 425 */ "pseudo_column ::= ROWTS", + /* 426 */ "pseudo_column ::= TBNAME", + /* 427 */ "pseudo_column ::= table_name NK_DOT TBNAME", + /* 428 */ "pseudo_column ::= QSTART", + /* 429 */ "pseudo_column ::= QEND", + /* 430 */ "pseudo_column ::= QDURATION", + /* 431 */ "pseudo_column ::= WSTART", + /* 432 */ "pseudo_column ::= WEND", + /* 433 */ "pseudo_column ::= WDURATION", + /* 434 */ "pseudo_column ::= IROWTS", + /* 435 */ "pseudo_column ::= ISFILLED", + /* 436 */ "pseudo_column ::= QTAGS", + /* 437 */ "function_expression ::= function_name NK_LP expression_list NK_RP", + /* 438 */ "function_expression ::= star_func NK_LP star_func_para_list NK_RP", + /* 439 */ "function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP", + /* 440 */ "function_expression ::= literal_func", + /* 441 */ "literal_func ::= noarg_func NK_LP NK_RP", + /* 442 */ "literal_func ::= NOW", + /* 443 */ "noarg_func ::= NOW", + /* 444 */ "noarg_func ::= TODAY", + /* 445 */ "noarg_func ::= TIMEZONE", + /* 446 */ "noarg_func ::= DATABASE", + /* 447 */ "noarg_func ::= CLIENT_VERSION", + /* 448 */ "noarg_func ::= SERVER_VERSION", + /* 449 */ "noarg_func ::= SERVER_STATUS", + /* 450 */ "noarg_func ::= CURRENT_USER", + /* 451 */ "noarg_func ::= USER", + /* 452 */ "star_func ::= COUNT", + /* 453 */ "star_func ::= FIRST", + /* 454 */ "star_func ::= LAST", + /* 455 */ "star_func ::= LAST_ROW", + /* 456 */ "star_func_para_list ::= NK_STAR", + /* 457 */ "star_func_para_list ::= other_para_list", + /* 458 */ "other_para_list ::= star_func_para", + /* 459 */ "other_para_list ::= other_para_list NK_COMMA star_func_para", + /* 460 */ "star_func_para ::= expr_or_subquery", + /* 461 */ "star_func_para ::= table_name NK_DOT NK_STAR", + /* 462 */ "case_when_expression ::= CASE when_then_list case_when_else_opt END", + /* 463 */ "case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END", + /* 464 */ "when_then_list ::= when_then_expr", + /* 465 */ "when_then_list ::= when_then_list when_then_expr", + /* 466 */ "when_then_expr ::= WHEN common_expression THEN common_expression", + /* 467 */ "case_when_else_opt ::=", + /* 468 */ "case_when_else_opt ::= ELSE common_expression", + /* 469 */ "predicate ::= expr_or_subquery compare_op expr_or_subquery", + /* 470 */ "predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery", + /* 471 */ "predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery", + /* 472 */ "predicate ::= expr_or_subquery IS NULL", + /* 473 */ "predicate ::= expr_or_subquery IS NOT NULL", + /* 474 */ "predicate ::= expr_or_subquery in_op in_predicate_value", + /* 475 */ "compare_op ::= NK_LT", + /* 476 */ "compare_op ::= NK_GT", + /* 477 */ "compare_op ::= NK_LE", + /* 478 */ "compare_op ::= NK_GE", + /* 479 */ "compare_op ::= NK_NE", + /* 480 */ "compare_op ::= NK_EQ", + /* 481 */ "compare_op ::= LIKE", + /* 482 */ "compare_op ::= NOT LIKE", + /* 483 */ "compare_op ::= MATCH", + /* 484 */ "compare_op ::= NMATCH", + /* 485 */ "compare_op ::= CONTAINS", + /* 486 */ "in_op ::= IN", + /* 487 */ "in_op ::= NOT IN", + /* 488 */ "in_predicate_value ::= NK_LP literal_list NK_RP", + /* 489 */ "boolean_value_expression ::= boolean_primary", + /* 490 */ "boolean_value_expression ::= NOT boolean_primary", + /* 491 */ "boolean_value_expression ::= boolean_value_expression OR boolean_value_expression", + /* 492 */ "boolean_value_expression ::= boolean_value_expression AND boolean_value_expression", + /* 493 */ "boolean_primary ::= predicate", + /* 494 */ "boolean_primary ::= NK_LP boolean_value_expression NK_RP", + /* 495 */ "common_expression ::= expr_or_subquery", + /* 496 */ "common_expression ::= boolean_value_expression", + /* 497 */ "from_clause_opt ::=", + /* 498 */ "from_clause_opt ::= FROM table_reference_list", + /* 499 */ "table_reference_list ::= table_reference", + /* 500 */ "table_reference_list ::= table_reference_list NK_COMMA table_reference", + /* 501 */ "table_reference ::= table_primary", + /* 502 */ "table_reference ::= joined_table", + /* 503 */ "table_primary ::= table_name alias_opt", + /* 504 */ "table_primary ::= db_name NK_DOT table_name alias_opt", + /* 505 */ "table_primary ::= subquery alias_opt", + /* 506 */ "table_primary ::= parenthesized_joined_table", + /* 507 */ "alias_opt ::=", + /* 508 */ "alias_opt ::= table_alias", + /* 509 */ "alias_opt ::= AS table_alias", + /* 510 */ "parenthesized_joined_table ::= NK_LP joined_table NK_RP", + /* 511 */ "parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP", + /* 512 */ "joined_table ::= table_reference join_type JOIN table_reference ON search_condition", + /* 513 */ "join_type ::=", + /* 514 */ "join_type ::= INNER", + /* 515 */ "query_specification ::= SELECT set_quantifier_opt select_list from_clause_opt where_clause_opt partition_by_clause_opt range_opt every_opt fill_opt twindow_clause_opt group_by_clause_opt having_clause_opt", + /* 516 */ "set_quantifier_opt ::=", + /* 517 */ "set_quantifier_opt ::= DISTINCT", + /* 518 */ "set_quantifier_opt ::= ALL", + /* 519 */ "select_list ::= select_item", + /* 520 */ "select_list ::= select_list NK_COMMA select_item", + /* 521 */ "select_item ::= NK_STAR", + /* 522 */ "select_item ::= common_expression", + /* 523 */ "select_item ::= common_expression column_alias", + /* 524 */ "select_item ::= common_expression AS column_alias", + /* 525 */ "select_item ::= table_name NK_DOT NK_STAR", + /* 526 */ "where_clause_opt ::=", + /* 527 */ "where_clause_opt ::= WHERE search_condition", + /* 528 */ "partition_by_clause_opt ::=", + /* 529 */ "partition_by_clause_opt ::= PARTITION BY partition_list", + /* 530 */ "partition_list ::= partition_item", + /* 531 */ "partition_list ::= partition_list NK_COMMA partition_item", + /* 532 */ "partition_item ::= expr_or_subquery", + /* 533 */ "partition_item ::= expr_or_subquery column_alias", + /* 534 */ "partition_item ::= expr_or_subquery AS column_alias", + /* 535 */ "twindow_clause_opt ::=", + /* 536 */ "twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA duration_literal NK_RP", + /* 537 */ "twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP", + /* 538 */ "twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_RP sliding_opt fill_opt", + /* 539 */ "twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_COMMA duration_literal NK_RP sliding_opt fill_opt", + /* 540 */ "twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition", + /* 541 */ "sliding_opt ::=", + /* 542 */ "sliding_opt ::= SLIDING NK_LP duration_literal NK_RP", + /* 543 */ "fill_opt ::=", + /* 544 */ "fill_opt ::= FILL NK_LP fill_mode NK_RP", + /* 545 */ "fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP", + /* 546 */ "fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP", + /* 547 */ "fill_mode ::= NONE", + /* 548 */ "fill_mode ::= PREV", + /* 549 */ "fill_mode ::= NULL", + /* 550 */ "fill_mode ::= NULL_F", + /* 551 */ "fill_mode ::= LINEAR", + /* 552 */ "fill_mode ::= NEXT", + /* 553 */ "group_by_clause_opt ::=", + /* 554 */ "group_by_clause_opt ::= GROUP BY group_by_list", + /* 555 */ "group_by_list ::= expr_or_subquery", + /* 556 */ "group_by_list ::= group_by_list NK_COMMA expr_or_subquery", + /* 557 */ "having_clause_opt ::=", + /* 558 */ "having_clause_opt ::= HAVING search_condition", + /* 559 */ "range_opt ::=", + /* 560 */ "range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP", + /* 561 */ "every_opt ::=", + /* 562 */ "every_opt ::= EVERY NK_LP duration_literal NK_RP", + /* 563 */ "query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt", + /* 564 */ "query_simple ::= query_specification", + /* 565 */ "query_simple ::= union_query_expression", + /* 566 */ "union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery", + /* 567 */ "union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery", + /* 568 */ "query_simple_or_subquery ::= query_simple", + /* 569 */ "query_simple_or_subquery ::= subquery", + /* 570 */ "query_or_subquery ::= query_expression", + /* 571 */ "query_or_subquery ::= subquery", + /* 572 */ "order_by_clause_opt ::=", + /* 573 */ "order_by_clause_opt ::= ORDER BY sort_specification_list", + /* 574 */ "slimit_clause_opt ::=", + /* 575 */ "slimit_clause_opt ::= SLIMIT NK_INTEGER", + /* 576 */ "slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER", + /* 577 */ "slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER", + /* 578 */ "limit_clause_opt ::=", + /* 579 */ "limit_clause_opt ::= LIMIT NK_INTEGER", + /* 580 */ "limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER", + /* 581 */ "limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER", + /* 582 */ "subquery ::= NK_LP query_expression NK_RP", + /* 583 */ "subquery ::= NK_LP subquery NK_RP", + /* 584 */ "search_condition ::= common_expression", + /* 585 */ "sort_specification_list ::= sort_specification", + /* 586 */ "sort_specification_list ::= sort_specification_list NK_COMMA sort_specification", + /* 587 */ "sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt", + /* 588 */ "ordering_specification_opt ::=", + /* 589 */ "ordering_specification_opt ::= ASC", + /* 590 */ "ordering_specification_opt ::= DESC", + /* 591 */ "null_ordering_opt ::=", + /* 592 */ "null_ordering_opt ::= NULLS FIRST", + /* 593 */ "null_ordering_opt ::= NULLS LAST", }; #endif /* NDEBUG */ @@ -2703,241 +2705,212 @@ static void yy_destructor( */ /********* Begin destructor definitions ***************************************/ /* Default NON-TERMINAL Destructor */ - case 335: /* cmd */ - case 338: /* literal */ - case 344: /* with_opt */ - case 350: /* search_condition */ - case 354: /* db_options */ - case 356: /* alter_db_options */ - case 358: /* start_opt */ - case 359: /* end_opt */ - case 363: /* signed */ - case 365: /* retention */ - case 366: /* full_table_name */ - case 369: /* table_options */ - case 373: /* alter_table_clause */ - case 374: /* alter_table_options */ - case 377: /* signed_literal */ - case 378: /* create_subtable_clause */ - case 381: /* drop_table_clause */ - case 383: /* column_def */ - case 387: /* duration_literal */ - case 388: /* rollup_func_name */ - case 390: /* col_name */ - case 391: /* db_name_cond_opt */ - case 392: /* like_pattern_opt */ - case 393: /* table_name_cond */ - case 394: /* from_db_opt */ - case 396: /* tag_item */ - case 398: /* full_index_name */ - case 399: /* index_options */ - case 402: /* sliding_opt */ - case 403: /* sma_stream_opt */ - case 404: /* func */ - case 406: /* query_or_subquery */ - case 409: /* explain_options */ - case 410: /* insert_query */ - case 416: /* stream_options */ - case 419: /* subtable_opt */ - case 421: /* expression */ - case 423: /* where_clause_opt */ - case 424: /* literal_func */ - case 427: /* expr_or_subquery */ - case 428: /* pseudo_column */ - case 429: /* column_reference */ - case 430: /* function_expression */ - case 431: /* case_when_expression */ - case 436: /* star_func_para */ - case 438: /* case_when_else_opt */ - case 439: /* common_expression */ - case 440: /* when_then_expr */ - case 441: /* predicate */ - case 444: /* in_predicate_value */ - case 445: /* boolean_value_expression */ - case 446: /* boolean_primary */ - case 447: /* from_clause_opt */ - case 448: /* table_reference_list */ - case 449: /* table_reference */ - case 450: /* table_primary */ - case 451: /* joined_table */ - case 453: /* subquery */ - case 454: /* parenthesized_joined_table */ - case 456: /* query_specification */ - case 460: /* range_opt */ - case 461: /* every_opt */ - case 462: /* fill_opt */ - case 463: /* twindow_clause_opt */ - case 465: /* having_clause_opt */ - case 466: /* select_item */ - case 468: /* partition_item */ - case 471: /* query_expression */ - case 472: /* query_simple */ - case 474: /* slimit_clause_opt */ - case 475: /* limit_clause_opt */ - case 476: /* union_query_expression */ - case 477: /* query_simple_or_subquery */ - case 479: /* sort_specification */ + case 336: /* cmd */ + case 339: /* literal */ + case 345: /* with_opt */ + case 351: /* search_condition */ + case 356: /* db_options */ + case 358: /* alter_db_options */ + case 360: /* start_opt */ + case 361: /* end_opt */ + case 365: /* signed */ + case 367: /* retention */ + case 368: /* full_table_name */ + case 371: /* table_options */ + case 375: /* alter_table_clause */ + case 376: /* alter_table_options */ + case 379: /* signed_literal */ + case 380: /* create_subtable_clause */ + case 383: /* drop_table_clause */ + case 385: /* column_def */ + case 389: /* duration_literal */ + case 390: /* rollup_func_name */ + case 392: /* col_name */ + case 393: /* db_name_cond_opt */ + case 394: /* like_pattern_opt */ + case 395: /* table_name_cond */ + case 396: /* from_db_opt */ + case 398: /* tag_item */ + case 400: /* full_index_name */ + case 401: /* index_options */ + case 404: /* sliding_opt */ + case 405: /* sma_stream_opt */ + case 406: /* func */ + case 408: /* query_or_subquery */ + case 411: /* explain_options */ + case 412: /* insert_query */ + case 418: /* stream_options */ + case 421: /* subtable_opt */ + case 423: /* expression */ + case 425: /* where_clause_opt */ + case 426: /* literal_func */ + case 429: /* expr_or_subquery */ + case 430: /* pseudo_column */ + case 431: /* column_reference */ + case 432: /* function_expression */ + case 433: /* case_when_expression */ + case 438: /* star_func_para */ + case 440: /* case_when_else_opt */ + case 441: /* common_expression */ + case 442: /* when_then_expr */ + case 443: /* predicate */ + case 446: /* in_predicate_value */ + case 447: /* boolean_value_expression */ + case 448: /* boolean_primary */ + case 449: /* from_clause_opt */ + case 450: /* table_reference_list */ + case 451: /* table_reference */ + case 452: /* table_primary */ + case 453: /* joined_table */ + case 455: /* subquery */ + case 456: /* parenthesized_joined_table */ + case 458: /* query_specification */ + case 462: /* range_opt */ + case 463: /* every_opt */ + case 464: /* fill_opt */ + case 465: /* twindow_clause_opt */ + case 467: /* having_clause_opt */ + case 468: /* select_item */ + case 470: /* partition_item */ + case 473: /* query_expression */ + case 474: /* query_simple */ + case 476: /* slimit_clause_opt */ + case 477: /* limit_clause_opt */ + case 478: /* union_query_expression */ + case 479: /* query_simple_or_subquery */ + case 481: /* sort_specification */ { -#line 7 "sql.y" - nodesDestroyNode((yypminor->yy164)); -#line 2784 "sql.c" + nodesDestroyNode((yypminor->yy520)); } break; - case 336: /* account_options */ - case 337: /* alter_account_options */ - case 339: /* alter_account_option */ - case 357: /* speed_opt */ - case 413: /* bufsize_opt */ + case 337: /* account_options */ + case 338: /* alter_account_options */ + case 340: /* alter_account_option */ + case 359: /* speed_opt */ + case 415: /* bufsize_opt */ { -#line 54 "sql.y" -#line 2795 "sql.c" } break; - case 340: /* user_name */ - case 347: /* db_name */ - case 348: /* table_name */ - case 349: /* topic_name */ - case 351: /* dnode_endpoint */ - case 375: /* column_name */ - case 389: /* function_name */ - case 397: /* column_alias */ - case 400: /* index_name */ - case 405: /* sma_func_name */ - case 407: /* cgroup_name */ - case 414: /* language_opt */ - case 415: /* stream_name */ - case 426: /* table_alias */ - case 432: /* star_func */ - case 434: /* noarg_func */ - case 452: /* alias_opt */ + case 341: /* user_name */ + case 348: /* db_name */ + case 349: /* table_name */ + case 350: /* topic_name */ + case 352: /* dnode_endpoint */ + case 377: /* column_name */ + case 391: /* function_name */ + case 399: /* column_alias */ + case 402: /* index_name */ + case 407: /* sma_func_name */ + case 409: /* cgroup_name */ + case 416: /* language_opt */ + case 417: /* stream_name */ + case 428: /* table_alias */ + case 434: /* star_func */ + case 436: /* noarg_func */ + case 454: /* alias_opt */ { -#line 728 "sql.y" -#line 2818 "sql.c" } break; - case 341: /* sysinfo_opt */ + case 342: /* sysinfo_opt */ { -#line 92 "sql.y" -#line 2825 "sql.c" } break; - case 342: /* privileges */ - case 345: /* priv_type_list */ - case 346: /* priv_type */ + case 343: /* privileges */ + case 346: /* priv_type_list */ + case 347: /* priv_type */ { -#line 101 "sql.y" -#line 2834 "sql.c" } break; - case 343: /* priv_level */ + case 344: /* priv_level */ { -#line 117 "sql.y" -#line 2841 "sql.c" } break; - case 352: /* force_opt */ - case 353: /* not_exists_opt */ - case 355: /* exists_opt */ - case 408: /* analyze_opt */ - case 411: /* or_replace_opt */ - case 412: /* agg_func_opt */ - case 420: /* ignore_opt */ - case 457: /* set_quantifier_opt */ + case 353: /* force_opt */ + case 354: /* unsafe_opt */ + case 355: /* not_exists_opt */ + case 357: /* exists_opt */ + case 410: /* analyze_opt */ + case 413: /* or_replace_opt */ + case 414: /* agg_func_opt */ + case 422: /* ignore_opt */ + case 459: /* set_quantifier_opt */ { -#line 144 "sql.y" -#line 2855 "sql.c" } break; - case 360: /* integer_list */ - case 361: /* variable_list */ - case 362: /* retention_list */ - case 367: /* column_def_list */ - case 368: /* tags_def_opt */ - case 370: /* multi_create_clause */ - case 371: /* tags_def */ - case 372: /* multi_drop_clause */ - case 379: /* specific_cols_opt */ - case 380: /* expression_list */ - case 382: /* col_name_list */ - case 384: /* duration_list */ - case 385: /* rollup_func_list */ - case 395: /* tag_list_opt */ - case 401: /* func_list */ - case 417: /* col_list_opt */ - case 418: /* tag_def_or_ref_opt */ - case 422: /* dnode_list */ - case 425: /* literal_list */ - case 433: /* star_func_para_list */ - case 435: /* other_para_list */ - case 437: /* when_then_list */ - case 458: /* select_list */ - case 459: /* partition_by_clause_opt */ - case 464: /* group_by_clause_opt */ - case 467: /* partition_list */ - case 470: /* group_by_list */ - case 473: /* order_by_clause_opt */ - case 478: /* sort_specification_list */ + case 362: /* integer_list */ + case 363: /* variable_list */ + case 364: /* retention_list */ + case 369: /* column_def_list */ + case 370: /* tags_def_opt */ + case 372: /* multi_create_clause */ + case 373: /* tags_def */ + case 374: /* multi_drop_clause */ + case 381: /* specific_cols_opt */ + case 382: /* expression_list */ + case 384: /* col_name_list */ + case 386: /* duration_list */ + case 387: /* rollup_func_list */ + case 397: /* tag_list_opt */ + case 403: /* func_list */ + case 419: /* col_list_opt */ + case 420: /* tag_def_or_ref_opt */ + case 424: /* dnode_list */ + case 427: /* literal_list */ + case 435: /* star_func_para_list */ + case 437: /* other_para_list */ + case 439: /* when_then_list */ + case 460: /* select_list */ + case 461: /* partition_by_clause_opt */ + case 466: /* group_by_clause_opt */ + case 469: /* partition_list */ + case 472: /* group_by_list */ + case 475: /* order_by_clause_opt */ + case 480: /* sort_specification_list */ { -#line 264 "sql.y" - nodesDestroyList((yypminor->yy72)); -#line 2890 "sql.c" + nodesDestroyList((yypminor->yy904)); } break; - case 364: /* alter_db_option */ - case 386: /* alter_table_option */ + case 366: /* alter_db_option */ + case 388: /* alter_table_option */ { -#line 237 "sql.y" -#line 2898 "sql.c" } break; - case 376: /* type_name */ + case 378: /* type_name */ { -#line 358 "sql.y" -#line 2905 "sql.c" } break; - case 442: /* compare_op */ - case 443: /* in_op */ + case 444: /* compare_op */ + case 445: /* in_op */ { -#line 916 "sql.y" -#line 2913 "sql.c" } break; - case 455: /* join_type */ + case 457: /* join_type */ { -#line 992 "sql.y" -#line 2920 "sql.c" } break; - case 469: /* fill_mode */ + case 471: /* fill_mode */ { -#line 1067 "sql.y" -#line 2927 "sql.c" } break; - case 480: /* ordering_specification_opt */ + case 482: /* ordering_specification_opt */ { -#line 1150 "sql.y" -#line 2934 "sql.c" } break; - case 481: /* null_ordering_opt */ + case 483: /* null_ordering_opt */ { -#line 1156 "sql.y" -#line 2941 "sql.c" } break; /********* End destructor definitions *****************************************/ @@ -3226,597 +3199,600 @@ static void yy_shift( /* For rule J, yyRuleInfoLhs[J] contains the symbol on the left-hand side ** of that rule */ static const YYCODETYPE yyRuleInfoLhs[] = { - 335, /* (0) cmd ::= CREATE ACCOUNT NK_ID PASS NK_STRING account_options */ - 335, /* (1) cmd ::= ALTER ACCOUNT NK_ID alter_account_options */ - 336, /* (2) account_options ::= */ - 336, /* (3) account_options ::= account_options PPS literal */ - 336, /* (4) account_options ::= account_options TSERIES literal */ - 336, /* (5) account_options ::= account_options STORAGE literal */ - 336, /* (6) account_options ::= account_options STREAMS literal */ - 336, /* (7) account_options ::= account_options QTIME literal */ - 336, /* (8) account_options ::= account_options DBS literal */ - 336, /* (9) account_options ::= account_options USERS literal */ - 336, /* (10) account_options ::= account_options CONNS literal */ - 336, /* (11) account_options ::= account_options STATE literal */ - 337, /* (12) alter_account_options ::= alter_account_option */ - 337, /* (13) alter_account_options ::= alter_account_options alter_account_option */ - 339, /* (14) alter_account_option ::= PASS literal */ - 339, /* (15) alter_account_option ::= PPS literal */ - 339, /* (16) alter_account_option ::= TSERIES literal */ - 339, /* (17) alter_account_option ::= STORAGE literal */ - 339, /* (18) alter_account_option ::= STREAMS literal */ - 339, /* (19) alter_account_option ::= QTIME literal */ - 339, /* (20) alter_account_option ::= DBS literal */ - 339, /* (21) alter_account_option ::= USERS literal */ - 339, /* (22) alter_account_option ::= CONNS literal */ - 339, /* (23) alter_account_option ::= STATE literal */ - 335, /* (24) cmd ::= CREATE USER user_name PASS NK_STRING sysinfo_opt */ - 335, /* (25) cmd ::= ALTER USER user_name PASS NK_STRING */ - 335, /* (26) cmd ::= ALTER USER user_name ENABLE NK_INTEGER */ - 335, /* (27) cmd ::= ALTER USER user_name SYSINFO NK_INTEGER */ - 335, /* (28) cmd ::= DROP USER user_name */ - 341, /* (29) sysinfo_opt ::= */ - 341, /* (30) sysinfo_opt ::= SYSINFO NK_INTEGER */ - 335, /* (31) cmd ::= GRANT privileges ON priv_level with_opt TO user_name */ - 335, /* (32) cmd ::= REVOKE privileges ON priv_level with_opt FROM user_name */ - 342, /* (33) privileges ::= ALL */ - 342, /* (34) privileges ::= priv_type_list */ - 342, /* (35) privileges ::= SUBSCRIBE */ - 345, /* (36) priv_type_list ::= priv_type */ - 345, /* (37) priv_type_list ::= priv_type_list NK_COMMA priv_type */ - 346, /* (38) priv_type ::= READ */ - 346, /* (39) priv_type ::= WRITE */ - 343, /* (40) priv_level ::= NK_STAR NK_DOT NK_STAR */ - 343, /* (41) priv_level ::= db_name NK_DOT NK_STAR */ - 343, /* (42) priv_level ::= db_name NK_DOT table_name */ - 343, /* (43) priv_level ::= topic_name */ - 344, /* (44) with_opt ::= */ - 344, /* (45) with_opt ::= WITH search_condition */ - 335, /* (46) cmd ::= CREATE DNODE dnode_endpoint */ - 335, /* (47) cmd ::= CREATE DNODE dnode_endpoint PORT NK_INTEGER */ - 335, /* (48) cmd ::= DROP DNODE NK_INTEGER force_opt */ - 335, /* (49) cmd ::= DROP DNODE dnode_endpoint force_opt */ - 335, /* (50) cmd ::= ALTER DNODE NK_INTEGER NK_STRING */ - 335, /* (51) cmd ::= ALTER DNODE NK_INTEGER NK_STRING NK_STRING */ - 335, /* (52) cmd ::= ALTER ALL DNODES NK_STRING */ - 335, /* (53) cmd ::= ALTER ALL DNODES NK_STRING NK_STRING */ - 335, /* (54) cmd ::= RESTORE DNODE NK_INTEGER */ - 351, /* (55) dnode_endpoint ::= NK_STRING */ - 351, /* (56) dnode_endpoint ::= NK_ID */ - 351, /* (57) dnode_endpoint ::= NK_IPTOKEN */ - 352, /* (58) force_opt ::= */ - 352, /* (59) force_opt ::= FORCE */ - 335, /* (60) cmd ::= ALTER LOCAL NK_STRING */ - 335, /* (61) cmd ::= ALTER LOCAL NK_STRING NK_STRING */ - 335, /* (62) cmd ::= CREATE QNODE ON DNODE NK_INTEGER */ - 335, /* (63) cmd ::= DROP QNODE ON DNODE NK_INTEGER */ - 335, /* (64) cmd ::= RESTORE QNODE ON DNODE NK_INTEGER */ - 335, /* (65) cmd ::= CREATE BNODE ON DNODE NK_INTEGER */ - 335, /* (66) cmd ::= DROP BNODE ON DNODE NK_INTEGER */ - 335, /* (67) cmd ::= CREATE SNODE ON DNODE NK_INTEGER */ - 335, /* (68) cmd ::= DROP SNODE ON DNODE NK_INTEGER */ - 335, /* (69) cmd ::= CREATE MNODE ON DNODE NK_INTEGER */ - 335, /* (70) cmd ::= DROP MNODE ON DNODE NK_INTEGER */ - 335, /* (71) cmd ::= RESTORE MNODE ON DNODE NK_INTEGER */ - 335, /* (72) cmd ::= RESTORE VNODE ON DNODE NK_INTEGER */ - 335, /* (73) cmd ::= CREATE DATABASE not_exists_opt db_name db_options */ - 335, /* (74) cmd ::= DROP DATABASE exists_opt db_name */ - 335, /* (75) cmd ::= USE db_name */ - 335, /* (76) cmd ::= ALTER DATABASE db_name alter_db_options */ - 335, /* (77) cmd ::= FLUSH DATABASE db_name */ - 335, /* (78) cmd ::= TRIM DATABASE db_name speed_opt */ - 335, /* (79) cmd ::= COMPACT DATABASE db_name start_opt end_opt */ - 353, /* (80) not_exists_opt ::= IF NOT EXISTS */ - 353, /* (81) not_exists_opt ::= */ - 355, /* (82) exists_opt ::= IF EXISTS */ - 355, /* (83) exists_opt ::= */ - 354, /* (84) db_options ::= */ - 354, /* (85) db_options ::= db_options BUFFER NK_INTEGER */ - 354, /* (86) db_options ::= db_options CACHEMODEL NK_STRING */ - 354, /* (87) db_options ::= db_options CACHESIZE NK_INTEGER */ - 354, /* (88) db_options ::= db_options COMP NK_INTEGER */ - 354, /* (89) db_options ::= db_options DURATION NK_INTEGER */ - 354, /* (90) db_options ::= db_options DURATION NK_VARIABLE */ - 354, /* (91) db_options ::= db_options MAXROWS NK_INTEGER */ - 354, /* (92) db_options ::= db_options MINROWS NK_INTEGER */ - 354, /* (93) db_options ::= db_options KEEP integer_list */ - 354, /* (94) db_options ::= db_options KEEP variable_list */ - 354, /* (95) db_options ::= db_options PAGES NK_INTEGER */ - 354, /* (96) db_options ::= db_options PAGESIZE NK_INTEGER */ - 354, /* (97) db_options ::= db_options TSDB_PAGESIZE NK_INTEGER */ - 354, /* (98) db_options ::= db_options PRECISION NK_STRING */ - 354, /* (99) db_options ::= db_options REPLICA NK_INTEGER */ - 354, /* (100) db_options ::= db_options VGROUPS NK_INTEGER */ - 354, /* (101) db_options ::= db_options SINGLE_STABLE NK_INTEGER */ - 354, /* (102) db_options ::= db_options RETENTIONS retention_list */ - 354, /* (103) db_options ::= db_options SCHEMALESS NK_INTEGER */ - 354, /* (104) db_options ::= db_options WAL_LEVEL NK_INTEGER */ - 354, /* (105) db_options ::= db_options WAL_FSYNC_PERIOD NK_INTEGER */ - 354, /* (106) db_options ::= db_options WAL_RETENTION_PERIOD NK_INTEGER */ - 354, /* (107) db_options ::= db_options WAL_RETENTION_PERIOD NK_MINUS NK_INTEGER */ - 354, /* (108) db_options ::= db_options WAL_RETENTION_SIZE NK_INTEGER */ - 354, /* (109) db_options ::= db_options WAL_RETENTION_SIZE NK_MINUS NK_INTEGER */ - 354, /* (110) db_options ::= db_options WAL_ROLL_PERIOD NK_INTEGER */ - 354, /* (111) db_options ::= db_options WAL_SEGMENT_SIZE NK_INTEGER */ - 354, /* (112) db_options ::= db_options STT_TRIGGER NK_INTEGER */ - 354, /* (113) db_options ::= db_options TABLE_PREFIX signed */ - 354, /* (114) db_options ::= db_options TABLE_SUFFIX signed */ - 356, /* (115) alter_db_options ::= alter_db_option */ - 356, /* (116) alter_db_options ::= alter_db_options alter_db_option */ - 364, /* (117) alter_db_option ::= BUFFER NK_INTEGER */ - 364, /* (118) alter_db_option ::= CACHEMODEL NK_STRING */ - 364, /* (119) alter_db_option ::= CACHESIZE NK_INTEGER */ - 364, /* (120) alter_db_option ::= WAL_FSYNC_PERIOD NK_INTEGER */ - 364, /* (121) alter_db_option ::= KEEP integer_list */ - 364, /* (122) alter_db_option ::= KEEP variable_list */ - 364, /* (123) alter_db_option ::= PAGES NK_INTEGER */ - 364, /* (124) alter_db_option ::= REPLICA NK_INTEGER */ - 364, /* (125) alter_db_option ::= WAL_LEVEL NK_INTEGER */ - 364, /* (126) alter_db_option ::= STT_TRIGGER NK_INTEGER */ - 364, /* (127) alter_db_option ::= MINROWS NK_INTEGER */ - 364, /* (128) alter_db_option ::= WAL_RETENTION_PERIOD NK_INTEGER */ - 364, /* (129) alter_db_option ::= WAL_RETENTION_PERIOD NK_MINUS NK_INTEGER */ - 364, /* (130) alter_db_option ::= WAL_RETENTION_SIZE NK_INTEGER */ - 364, /* (131) alter_db_option ::= WAL_RETENTION_SIZE NK_MINUS NK_INTEGER */ - 360, /* (132) integer_list ::= NK_INTEGER */ - 360, /* (133) integer_list ::= integer_list NK_COMMA NK_INTEGER */ - 361, /* (134) variable_list ::= NK_VARIABLE */ - 361, /* (135) variable_list ::= variable_list NK_COMMA NK_VARIABLE */ - 362, /* (136) retention_list ::= retention */ - 362, /* (137) retention_list ::= retention_list NK_COMMA retention */ - 365, /* (138) retention ::= NK_VARIABLE NK_COLON NK_VARIABLE */ - 357, /* (139) speed_opt ::= */ - 357, /* (140) speed_opt ::= MAX_SPEED NK_INTEGER */ - 358, /* (141) start_opt ::= */ - 358, /* (142) start_opt ::= START WITH NK_INTEGER */ - 358, /* (143) start_opt ::= START WITH NK_STRING */ - 358, /* (144) start_opt ::= START WITH TIMESTAMP NK_STRING */ - 359, /* (145) end_opt ::= */ - 359, /* (146) end_opt ::= END WITH NK_INTEGER */ - 359, /* (147) end_opt ::= END WITH NK_STRING */ - 359, /* (148) end_opt ::= END WITH TIMESTAMP NK_STRING */ - 335, /* (149) cmd ::= CREATE TABLE not_exists_opt full_table_name NK_LP column_def_list NK_RP tags_def_opt table_options */ - 335, /* (150) cmd ::= CREATE TABLE multi_create_clause */ - 335, /* (151) cmd ::= CREATE STABLE not_exists_opt full_table_name NK_LP column_def_list NK_RP tags_def table_options */ - 335, /* (152) cmd ::= DROP TABLE multi_drop_clause */ - 335, /* (153) cmd ::= DROP STABLE exists_opt full_table_name */ - 335, /* (154) cmd ::= ALTER TABLE alter_table_clause */ - 335, /* (155) cmd ::= ALTER STABLE alter_table_clause */ - 373, /* (156) alter_table_clause ::= full_table_name alter_table_options */ - 373, /* (157) alter_table_clause ::= full_table_name ADD COLUMN column_name type_name */ - 373, /* (158) alter_table_clause ::= full_table_name DROP COLUMN column_name */ - 373, /* (159) alter_table_clause ::= full_table_name MODIFY COLUMN column_name type_name */ - 373, /* (160) alter_table_clause ::= full_table_name RENAME COLUMN column_name column_name */ - 373, /* (161) alter_table_clause ::= full_table_name ADD TAG column_name type_name */ - 373, /* (162) alter_table_clause ::= full_table_name DROP TAG column_name */ - 373, /* (163) alter_table_clause ::= full_table_name MODIFY TAG column_name type_name */ - 373, /* (164) alter_table_clause ::= full_table_name RENAME TAG column_name column_name */ - 373, /* (165) alter_table_clause ::= full_table_name SET TAG column_name NK_EQ signed_literal */ - 370, /* (166) multi_create_clause ::= create_subtable_clause */ - 370, /* (167) multi_create_clause ::= multi_create_clause create_subtable_clause */ - 378, /* (168) create_subtable_clause ::= not_exists_opt full_table_name USING full_table_name specific_cols_opt TAGS NK_LP expression_list NK_RP table_options */ - 372, /* (169) multi_drop_clause ::= drop_table_clause */ - 372, /* (170) multi_drop_clause ::= multi_drop_clause NK_COMMA drop_table_clause */ - 381, /* (171) drop_table_clause ::= exists_opt full_table_name */ - 379, /* (172) specific_cols_opt ::= */ - 379, /* (173) specific_cols_opt ::= NK_LP col_name_list NK_RP */ - 366, /* (174) full_table_name ::= table_name */ - 366, /* (175) full_table_name ::= db_name NK_DOT table_name */ - 367, /* (176) column_def_list ::= column_def */ - 367, /* (177) column_def_list ::= column_def_list NK_COMMA column_def */ - 383, /* (178) column_def ::= column_name type_name */ - 376, /* (179) type_name ::= BOOL */ - 376, /* (180) type_name ::= TINYINT */ - 376, /* (181) type_name ::= SMALLINT */ - 376, /* (182) type_name ::= INT */ - 376, /* (183) type_name ::= INTEGER */ - 376, /* (184) type_name ::= BIGINT */ - 376, /* (185) type_name ::= FLOAT */ - 376, /* (186) type_name ::= DOUBLE */ - 376, /* (187) type_name ::= BINARY NK_LP NK_INTEGER NK_RP */ - 376, /* (188) type_name ::= TIMESTAMP */ - 376, /* (189) type_name ::= NCHAR NK_LP NK_INTEGER NK_RP */ - 376, /* (190) type_name ::= TINYINT UNSIGNED */ - 376, /* (191) type_name ::= SMALLINT UNSIGNED */ - 376, /* (192) type_name ::= INT UNSIGNED */ - 376, /* (193) type_name ::= BIGINT UNSIGNED */ - 376, /* (194) type_name ::= JSON */ - 376, /* (195) type_name ::= VARCHAR NK_LP NK_INTEGER NK_RP */ - 376, /* (196) type_name ::= MEDIUMBLOB */ - 376, /* (197) type_name ::= BLOB */ - 376, /* (198) type_name ::= VARBINARY NK_LP NK_INTEGER NK_RP */ - 376, /* (199) type_name ::= DECIMAL */ - 376, /* (200) type_name ::= DECIMAL NK_LP NK_INTEGER NK_RP */ - 376, /* (201) type_name ::= DECIMAL NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP */ - 368, /* (202) tags_def_opt ::= */ - 368, /* (203) tags_def_opt ::= tags_def */ - 371, /* (204) tags_def ::= TAGS NK_LP column_def_list NK_RP */ - 369, /* (205) table_options ::= */ - 369, /* (206) table_options ::= table_options COMMENT NK_STRING */ - 369, /* (207) table_options ::= table_options MAX_DELAY duration_list */ - 369, /* (208) table_options ::= table_options WATERMARK duration_list */ - 369, /* (209) table_options ::= table_options ROLLUP NK_LP rollup_func_list NK_RP */ - 369, /* (210) table_options ::= table_options TTL NK_INTEGER */ - 369, /* (211) table_options ::= table_options SMA NK_LP col_name_list NK_RP */ - 369, /* (212) table_options ::= table_options DELETE_MARK duration_list */ - 374, /* (213) alter_table_options ::= alter_table_option */ - 374, /* (214) alter_table_options ::= alter_table_options alter_table_option */ - 386, /* (215) alter_table_option ::= COMMENT NK_STRING */ - 386, /* (216) alter_table_option ::= TTL NK_INTEGER */ - 384, /* (217) duration_list ::= duration_literal */ - 384, /* (218) duration_list ::= duration_list NK_COMMA duration_literal */ - 385, /* (219) rollup_func_list ::= rollup_func_name */ - 385, /* (220) rollup_func_list ::= rollup_func_list NK_COMMA rollup_func_name */ - 388, /* (221) rollup_func_name ::= function_name */ - 388, /* (222) rollup_func_name ::= FIRST */ - 388, /* (223) rollup_func_name ::= LAST */ - 382, /* (224) col_name_list ::= col_name */ - 382, /* (225) col_name_list ::= col_name_list NK_COMMA col_name */ - 390, /* (226) col_name ::= column_name */ - 335, /* (227) cmd ::= SHOW DNODES */ - 335, /* (228) cmd ::= SHOW USERS */ - 335, /* (229) cmd ::= SHOW USER PRIVILEGES */ - 335, /* (230) cmd ::= SHOW DATABASES */ - 335, /* (231) cmd ::= SHOW db_name_cond_opt TABLES like_pattern_opt */ - 335, /* (232) cmd ::= SHOW db_name_cond_opt STABLES like_pattern_opt */ - 335, /* (233) cmd ::= SHOW db_name_cond_opt VGROUPS */ - 335, /* (234) cmd ::= SHOW MNODES */ - 335, /* (235) cmd ::= SHOW QNODES */ - 335, /* (236) cmd ::= SHOW FUNCTIONS */ - 335, /* (237) cmd ::= SHOW INDEXES FROM table_name_cond from_db_opt */ - 335, /* (238) cmd ::= SHOW STREAMS */ - 335, /* (239) cmd ::= SHOW ACCOUNTS */ - 335, /* (240) cmd ::= SHOW APPS */ - 335, /* (241) cmd ::= SHOW CONNECTIONS */ - 335, /* (242) cmd ::= SHOW LICENCES */ - 335, /* (243) cmd ::= SHOW GRANTS */ - 335, /* (244) cmd ::= SHOW CREATE DATABASE db_name */ - 335, /* (245) cmd ::= SHOW CREATE TABLE full_table_name */ - 335, /* (246) cmd ::= SHOW CREATE STABLE full_table_name */ - 335, /* (247) cmd ::= SHOW QUERIES */ - 335, /* (248) cmd ::= SHOW SCORES */ - 335, /* (249) cmd ::= SHOW TOPICS */ - 335, /* (250) cmd ::= SHOW VARIABLES */ - 335, /* (251) cmd ::= SHOW CLUSTER VARIABLES */ - 335, /* (252) cmd ::= SHOW LOCAL VARIABLES */ - 335, /* (253) cmd ::= SHOW DNODE NK_INTEGER VARIABLES like_pattern_opt */ - 335, /* (254) cmd ::= SHOW BNODES */ - 335, /* (255) cmd ::= SHOW SNODES */ - 335, /* (256) cmd ::= SHOW CLUSTER */ - 335, /* (257) cmd ::= SHOW TRANSACTIONS */ - 335, /* (258) cmd ::= SHOW TABLE DISTRIBUTED full_table_name */ - 335, /* (259) cmd ::= SHOW CONSUMERS */ - 335, /* (260) cmd ::= SHOW SUBSCRIPTIONS */ - 335, /* (261) cmd ::= SHOW TAGS FROM table_name_cond from_db_opt */ - 335, /* (262) cmd ::= SHOW TABLE TAGS tag_list_opt FROM table_name_cond from_db_opt */ - 335, /* (263) cmd ::= SHOW VNODES NK_INTEGER */ - 335, /* (264) cmd ::= SHOW VNODES NK_STRING */ - 335, /* (265) cmd ::= SHOW db_name_cond_opt ALIVE */ - 335, /* (266) cmd ::= SHOW CLUSTER ALIVE */ - 391, /* (267) db_name_cond_opt ::= */ - 391, /* (268) db_name_cond_opt ::= db_name NK_DOT */ - 392, /* (269) like_pattern_opt ::= */ - 392, /* (270) like_pattern_opt ::= LIKE NK_STRING */ - 393, /* (271) table_name_cond ::= table_name */ - 394, /* (272) from_db_opt ::= */ - 394, /* (273) from_db_opt ::= FROM db_name */ - 395, /* (274) tag_list_opt ::= */ - 395, /* (275) tag_list_opt ::= tag_item */ - 395, /* (276) tag_list_opt ::= tag_list_opt NK_COMMA tag_item */ - 396, /* (277) tag_item ::= TBNAME */ - 396, /* (278) tag_item ::= QTAGS */ - 396, /* (279) tag_item ::= column_name */ - 396, /* (280) tag_item ::= column_name column_alias */ - 396, /* (281) tag_item ::= column_name AS column_alias */ - 335, /* (282) cmd ::= CREATE SMA INDEX not_exists_opt full_index_name ON full_table_name index_options */ - 335, /* (283) cmd ::= CREATE INDEX not_exists_opt full_index_name ON full_table_name NK_LP col_name_list NK_RP */ - 335, /* (284) cmd ::= DROP INDEX exists_opt full_index_name */ - 398, /* (285) full_index_name ::= index_name */ - 398, /* (286) full_index_name ::= db_name NK_DOT index_name */ - 399, /* (287) index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_RP sliding_opt sma_stream_opt */ - 399, /* (288) 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 */ - 401, /* (289) func_list ::= func */ - 401, /* (290) func_list ::= func_list NK_COMMA func */ - 404, /* (291) func ::= sma_func_name NK_LP expression_list NK_RP */ - 405, /* (292) sma_func_name ::= function_name */ - 405, /* (293) sma_func_name ::= COUNT */ - 405, /* (294) sma_func_name ::= FIRST */ - 405, /* (295) sma_func_name ::= LAST */ - 405, /* (296) sma_func_name ::= LAST_ROW */ - 403, /* (297) sma_stream_opt ::= */ - 403, /* (298) sma_stream_opt ::= sma_stream_opt WATERMARK duration_literal */ - 403, /* (299) sma_stream_opt ::= sma_stream_opt MAX_DELAY duration_literal */ - 403, /* (300) sma_stream_opt ::= sma_stream_opt DELETE_MARK duration_literal */ - 335, /* (301) cmd ::= CREATE TOPIC not_exists_opt topic_name AS query_or_subquery */ - 335, /* (302) cmd ::= CREATE TOPIC not_exists_opt topic_name AS DATABASE db_name */ - 335, /* (303) cmd ::= CREATE TOPIC not_exists_opt topic_name WITH META AS DATABASE db_name */ - 335, /* (304) cmd ::= CREATE TOPIC not_exists_opt topic_name AS STABLE full_table_name */ - 335, /* (305) cmd ::= CREATE TOPIC not_exists_opt topic_name WITH META AS STABLE full_table_name */ - 335, /* (306) cmd ::= DROP TOPIC exists_opt topic_name */ - 335, /* (307) cmd ::= DROP CONSUMER GROUP exists_opt cgroup_name ON topic_name */ - 335, /* (308) cmd ::= DESC full_table_name */ - 335, /* (309) cmd ::= DESCRIBE full_table_name */ - 335, /* (310) cmd ::= RESET QUERY CACHE */ - 335, /* (311) cmd ::= EXPLAIN analyze_opt explain_options query_or_subquery */ - 335, /* (312) cmd ::= EXPLAIN analyze_opt explain_options insert_query */ - 408, /* (313) analyze_opt ::= */ - 408, /* (314) analyze_opt ::= ANALYZE */ - 409, /* (315) explain_options ::= */ - 409, /* (316) explain_options ::= explain_options VERBOSE NK_BOOL */ - 409, /* (317) explain_options ::= explain_options RATIO NK_FLOAT */ - 335, /* (318) cmd ::= CREATE or_replace_opt agg_func_opt FUNCTION not_exists_opt function_name AS NK_STRING OUTPUTTYPE type_name bufsize_opt language_opt */ - 335, /* (319) cmd ::= DROP FUNCTION exists_opt function_name */ - 412, /* (320) agg_func_opt ::= */ - 412, /* (321) agg_func_opt ::= AGGREGATE */ - 413, /* (322) bufsize_opt ::= */ - 413, /* (323) bufsize_opt ::= BUFSIZE NK_INTEGER */ - 414, /* (324) language_opt ::= */ - 414, /* (325) language_opt ::= LANGUAGE NK_STRING */ - 411, /* (326) or_replace_opt ::= */ - 411, /* (327) or_replace_opt ::= OR REPLACE */ - 335, /* (328) 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 */ - 335, /* (329) cmd ::= DROP STREAM exists_opt stream_name */ - 335, /* (330) cmd ::= PAUSE STREAM exists_opt stream_name */ - 335, /* (331) cmd ::= RESUME STREAM exists_opt ignore_opt stream_name */ - 417, /* (332) col_list_opt ::= */ - 417, /* (333) col_list_opt ::= NK_LP col_name_list NK_RP */ - 418, /* (334) tag_def_or_ref_opt ::= */ - 418, /* (335) tag_def_or_ref_opt ::= tags_def */ - 418, /* (336) tag_def_or_ref_opt ::= TAGS NK_LP col_name_list NK_RP */ - 416, /* (337) stream_options ::= */ - 416, /* (338) stream_options ::= stream_options TRIGGER AT_ONCE */ - 416, /* (339) stream_options ::= stream_options TRIGGER WINDOW_CLOSE */ - 416, /* (340) stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal */ - 416, /* (341) stream_options ::= stream_options WATERMARK duration_literal */ - 416, /* (342) stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER */ - 416, /* (343) stream_options ::= stream_options FILL_HISTORY NK_INTEGER */ - 416, /* (344) stream_options ::= stream_options DELETE_MARK duration_literal */ - 416, /* (345) stream_options ::= stream_options IGNORE UPDATE NK_INTEGER */ - 419, /* (346) subtable_opt ::= */ - 419, /* (347) subtable_opt ::= SUBTABLE NK_LP expression NK_RP */ - 420, /* (348) ignore_opt ::= */ - 420, /* (349) ignore_opt ::= IGNORE UNTREATED */ - 335, /* (350) cmd ::= KILL CONNECTION NK_INTEGER */ - 335, /* (351) cmd ::= KILL QUERY NK_STRING */ - 335, /* (352) cmd ::= KILL TRANSACTION NK_INTEGER */ - 335, /* (353) cmd ::= BALANCE VGROUP */ - 335, /* (354) cmd ::= BALANCE VGROUP LEADER */ - 335, /* (355) cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER */ - 335, /* (356) cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list */ - 335, /* (357) cmd ::= SPLIT VGROUP NK_INTEGER */ - 422, /* (358) dnode_list ::= DNODE NK_INTEGER */ - 422, /* (359) dnode_list ::= dnode_list DNODE NK_INTEGER */ - 335, /* (360) cmd ::= DELETE FROM full_table_name where_clause_opt */ - 335, /* (361) cmd ::= query_or_subquery */ - 335, /* (362) cmd ::= insert_query */ - 410, /* (363) insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery */ - 410, /* (364) insert_query ::= INSERT INTO full_table_name query_or_subquery */ - 338, /* (365) literal ::= NK_INTEGER */ - 338, /* (366) literal ::= NK_FLOAT */ - 338, /* (367) literal ::= NK_STRING */ - 338, /* (368) literal ::= NK_BOOL */ - 338, /* (369) literal ::= TIMESTAMP NK_STRING */ - 338, /* (370) literal ::= duration_literal */ - 338, /* (371) literal ::= NULL */ - 338, /* (372) literal ::= NK_QUESTION */ - 387, /* (373) duration_literal ::= NK_VARIABLE */ - 363, /* (374) signed ::= NK_INTEGER */ - 363, /* (375) signed ::= NK_PLUS NK_INTEGER */ - 363, /* (376) signed ::= NK_MINUS NK_INTEGER */ - 363, /* (377) signed ::= NK_FLOAT */ - 363, /* (378) signed ::= NK_PLUS NK_FLOAT */ - 363, /* (379) signed ::= NK_MINUS NK_FLOAT */ - 377, /* (380) signed_literal ::= signed */ - 377, /* (381) signed_literal ::= NK_STRING */ - 377, /* (382) signed_literal ::= NK_BOOL */ - 377, /* (383) signed_literal ::= TIMESTAMP NK_STRING */ - 377, /* (384) signed_literal ::= duration_literal */ - 377, /* (385) signed_literal ::= NULL */ - 377, /* (386) signed_literal ::= literal_func */ - 377, /* (387) signed_literal ::= NK_QUESTION */ - 425, /* (388) literal_list ::= signed_literal */ - 425, /* (389) literal_list ::= literal_list NK_COMMA signed_literal */ - 347, /* (390) db_name ::= NK_ID */ - 348, /* (391) table_name ::= NK_ID */ - 375, /* (392) column_name ::= NK_ID */ - 389, /* (393) function_name ::= NK_ID */ - 426, /* (394) table_alias ::= NK_ID */ - 397, /* (395) column_alias ::= NK_ID */ - 340, /* (396) user_name ::= NK_ID */ - 349, /* (397) topic_name ::= NK_ID */ - 415, /* (398) stream_name ::= NK_ID */ - 407, /* (399) cgroup_name ::= NK_ID */ - 400, /* (400) index_name ::= NK_ID */ - 427, /* (401) expr_or_subquery ::= expression */ - 421, /* (402) expression ::= literal */ - 421, /* (403) expression ::= pseudo_column */ - 421, /* (404) expression ::= column_reference */ - 421, /* (405) expression ::= function_expression */ - 421, /* (406) expression ::= case_when_expression */ - 421, /* (407) expression ::= NK_LP expression NK_RP */ - 421, /* (408) expression ::= NK_PLUS expr_or_subquery */ - 421, /* (409) expression ::= NK_MINUS expr_or_subquery */ - 421, /* (410) expression ::= expr_or_subquery NK_PLUS expr_or_subquery */ - 421, /* (411) expression ::= expr_or_subquery NK_MINUS expr_or_subquery */ - 421, /* (412) expression ::= expr_or_subquery NK_STAR expr_or_subquery */ - 421, /* (413) expression ::= expr_or_subquery NK_SLASH expr_or_subquery */ - 421, /* (414) expression ::= expr_or_subquery NK_REM expr_or_subquery */ - 421, /* (415) expression ::= column_reference NK_ARROW NK_STRING */ - 421, /* (416) expression ::= expr_or_subquery NK_BITAND expr_or_subquery */ - 421, /* (417) expression ::= expr_or_subquery NK_BITOR expr_or_subquery */ - 380, /* (418) expression_list ::= expr_or_subquery */ - 380, /* (419) expression_list ::= expression_list NK_COMMA expr_or_subquery */ - 429, /* (420) column_reference ::= column_name */ - 429, /* (421) column_reference ::= table_name NK_DOT column_name */ - 428, /* (422) pseudo_column ::= ROWTS */ - 428, /* (423) pseudo_column ::= TBNAME */ - 428, /* (424) pseudo_column ::= table_name NK_DOT TBNAME */ - 428, /* (425) pseudo_column ::= QSTART */ - 428, /* (426) pseudo_column ::= QEND */ - 428, /* (427) pseudo_column ::= QDURATION */ - 428, /* (428) pseudo_column ::= WSTART */ - 428, /* (429) pseudo_column ::= WEND */ - 428, /* (430) pseudo_column ::= WDURATION */ - 428, /* (431) pseudo_column ::= IROWTS */ - 428, /* (432) pseudo_column ::= ISFILLED */ - 428, /* (433) pseudo_column ::= QTAGS */ - 430, /* (434) function_expression ::= function_name NK_LP expression_list NK_RP */ - 430, /* (435) function_expression ::= star_func NK_LP star_func_para_list NK_RP */ - 430, /* (436) function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP */ - 430, /* (437) function_expression ::= literal_func */ - 424, /* (438) literal_func ::= noarg_func NK_LP NK_RP */ - 424, /* (439) literal_func ::= NOW */ - 434, /* (440) noarg_func ::= NOW */ - 434, /* (441) noarg_func ::= TODAY */ - 434, /* (442) noarg_func ::= TIMEZONE */ - 434, /* (443) noarg_func ::= DATABASE */ - 434, /* (444) noarg_func ::= CLIENT_VERSION */ - 434, /* (445) noarg_func ::= SERVER_VERSION */ - 434, /* (446) noarg_func ::= SERVER_STATUS */ - 434, /* (447) noarg_func ::= CURRENT_USER */ - 434, /* (448) noarg_func ::= USER */ - 432, /* (449) star_func ::= COUNT */ - 432, /* (450) star_func ::= FIRST */ - 432, /* (451) star_func ::= LAST */ - 432, /* (452) star_func ::= LAST_ROW */ - 433, /* (453) star_func_para_list ::= NK_STAR */ - 433, /* (454) star_func_para_list ::= other_para_list */ - 435, /* (455) other_para_list ::= star_func_para */ - 435, /* (456) other_para_list ::= other_para_list NK_COMMA star_func_para */ - 436, /* (457) star_func_para ::= expr_or_subquery */ - 436, /* (458) star_func_para ::= table_name NK_DOT NK_STAR */ - 431, /* (459) case_when_expression ::= CASE when_then_list case_when_else_opt END */ - 431, /* (460) case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END */ - 437, /* (461) when_then_list ::= when_then_expr */ - 437, /* (462) when_then_list ::= when_then_list when_then_expr */ - 440, /* (463) when_then_expr ::= WHEN common_expression THEN common_expression */ - 438, /* (464) case_when_else_opt ::= */ - 438, /* (465) case_when_else_opt ::= ELSE common_expression */ - 441, /* (466) predicate ::= expr_or_subquery compare_op expr_or_subquery */ - 441, /* (467) predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery */ - 441, /* (468) predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery */ - 441, /* (469) predicate ::= expr_or_subquery IS NULL */ - 441, /* (470) predicate ::= expr_or_subquery IS NOT NULL */ - 441, /* (471) predicate ::= expr_or_subquery in_op in_predicate_value */ - 442, /* (472) compare_op ::= NK_LT */ - 442, /* (473) compare_op ::= NK_GT */ - 442, /* (474) compare_op ::= NK_LE */ - 442, /* (475) compare_op ::= NK_GE */ - 442, /* (476) compare_op ::= NK_NE */ - 442, /* (477) compare_op ::= NK_EQ */ - 442, /* (478) compare_op ::= LIKE */ - 442, /* (479) compare_op ::= NOT LIKE */ - 442, /* (480) compare_op ::= MATCH */ - 442, /* (481) compare_op ::= NMATCH */ - 442, /* (482) compare_op ::= CONTAINS */ - 443, /* (483) in_op ::= IN */ - 443, /* (484) in_op ::= NOT IN */ - 444, /* (485) in_predicate_value ::= NK_LP literal_list NK_RP */ - 445, /* (486) boolean_value_expression ::= boolean_primary */ - 445, /* (487) boolean_value_expression ::= NOT boolean_primary */ - 445, /* (488) boolean_value_expression ::= boolean_value_expression OR boolean_value_expression */ - 445, /* (489) boolean_value_expression ::= boolean_value_expression AND boolean_value_expression */ - 446, /* (490) boolean_primary ::= predicate */ - 446, /* (491) boolean_primary ::= NK_LP boolean_value_expression NK_RP */ - 439, /* (492) common_expression ::= expr_or_subquery */ - 439, /* (493) common_expression ::= boolean_value_expression */ - 447, /* (494) from_clause_opt ::= */ - 447, /* (495) from_clause_opt ::= FROM table_reference_list */ - 448, /* (496) table_reference_list ::= table_reference */ - 448, /* (497) table_reference_list ::= table_reference_list NK_COMMA table_reference */ - 449, /* (498) table_reference ::= table_primary */ - 449, /* (499) table_reference ::= joined_table */ - 450, /* (500) table_primary ::= table_name alias_opt */ - 450, /* (501) table_primary ::= db_name NK_DOT table_name alias_opt */ - 450, /* (502) table_primary ::= subquery alias_opt */ - 450, /* (503) table_primary ::= parenthesized_joined_table */ - 452, /* (504) alias_opt ::= */ - 452, /* (505) alias_opt ::= table_alias */ - 452, /* (506) alias_opt ::= AS table_alias */ - 454, /* (507) parenthesized_joined_table ::= NK_LP joined_table NK_RP */ - 454, /* (508) parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP */ - 451, /* (509) joined_table ::= table_reference join_type JOIN table_reference ON search_condition */ - 455, /* (510) join_type ::= */ - 455, /* (511) join_type ::= INNER */ - 456, /* (512) query_specification ::= SELECT set_quantifier_opt select_list from_clause_opt where_clause_opt partition_by_clause_opt range_opt every_opt fill_opt twindow_clause_opt group_by_clause_opt having_clause_opt */ - 457, /* (513) set_quantifier_opt ::= */ - 457, /* (514) set_quantifier_opt ::= DISTINCT */ - 457, /* (515) set_quantifier_opt ::= ALL */ - 458, /* (516) select_list ::= select_item */ - 458, /* (517) select_list ::= select_list NK_COMMA select_item */ - 466, /* (518) select_item ::= NK_STAR */ - 466, /* (519) select_item ::= common_expression */ - 466, /* (520) select_item ::= common_expression column_alias */ - 466, /* (521) select_item ::= common_expression AS column_alias */ - 466, /* (522) select_item ::= table_name NK_DOT NK_STAR */ - 423, /* (523) where_clause_opt ::= */ - 423, /* (524) where_clause_opt ::= WHERE search_condition */ - 459, /* (525) partition_by_clause_opt ::= */ - 459, /* (526) partition_by_clause_opt ::= PARTITION BY partition_list */ - 467, /* (527) partition_list ::= partition_item */ - 467, /* (528) partition_list ::= partition_list NK_COMMA partition_item */ - 468, /* (529) partition_item ::= expr_or_subquery */ - 468, /* (530) partition_item ::= expr_or_subquery column_alias */ - 468, /* (531) partition_item ::= expr_or_subquery AS column_alias */ - 463, /* (532) twindow_clause_opt ::= */ - 463, /* (533) twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA duration_literal NK_RP */ - 463, /* (534) twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP */ - 463, /* (535) twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_RP sliding_opt fill_opt */ - 463, /* (536) twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_COMMA duration_literal NK_RP sliding_opt fill_opt */ - 463, /* (537) twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition */ - 402, /* (538) sliding_opt ::= */ - 402, /* (539) sliding_opt ::= SLIDING NK_LP duration_literal NK_RP */ - 462, /* (540) fill_opt ::= */ - 462, /* (541) fill_opt ::= FILL NK_LP fill_mode NK_RP */ - 462, /* (542) fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP */ - 462, /* (543) fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP */ - 469, /* (544) fill_mode ::= NONE */ - 469, /* (545) fill_mode ::= PREV */ - 469, /* (546) fill_mode ::= NULL */ - 469, /* (547) fill_mode ::= NULL_F */ - 469, /* (548) fill_mode ::= LINEAR */ - 469, /* (549) fill_mode ::= NEXT */ - 464, /* (550) group_by_clause_opt ::= */ - 464, /* (551) group_by_clause_opt ::= GROUP BY group_by_list */ - 470, /* (552) group_by_list ::= expr_or_subquery */ - 470, /* (553) group_by_list ::= group_by_list NK_COMMA expr_or_subquery */ - 465, /* (554) having_clause_opt ::= */ - 465, /* (555) having_clause_opt ::= HAVING search_condition */ - 460, /* (556) range_opt ::= */ - 460, /* (557) range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP */ - 461, /* (558) every_opt ::= */ - 461, /* (559) every_opt ::= EVERY NK_LP duration_literal NK_RP */ - 471, /* (560) query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt */ - 472, /* (561) query_simple ::= query_specification */ - 472, /* (562) query_simple ::= union_query_expression */ - 476, /* (563) union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery */ - 476, /* (564) union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery */ - 477, /* (565) query_simple_or_subquery ::= query_simple */ - 477, /* (566) query_simple_or_subquery ::= subquery */ - 406, /* (567) query_or_subquery ::= query_expression */ - 406, /* (568) query_or_subquery ::= subquery */ - 473, /* (569) order_by_clause_opt ::= */ - 473, /* (570) order_by_clause_opt ::= ORDER BY sort_specification_list */ - 474, /* (571) slimit_clause_opt ::= */ - 474, /* (572) slimit_clause_opt ::= SLIMIT NK_INTEGER */ - 474, /* (573) slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER */ - 474, /* (574) slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER */ - 475, /* (575) limit_clause_opt ::= */ - 475, /* (576) limit_clause_opt ::= LIMIT NK_INTEGER */ - 475, /* (577) limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER */ - 475, /* (578) limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER */ - 453, /* (579) subquery ::= NK_LP query_expression NK_RP */ - 453, /* (580) subquery ::= NK_LP subquery NK_RP */ - 350, /* (581) search_condition ::= common_expression */ - 478, /* (582) sort_specification_list ::= sort_specification */ - 478, /* (583) sort_specification_list ::= sort_specification_list NK_COMMA sort_specification */ - 479, /* (584) sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt */ - 480, /* (585) ordering_specification_opt ::= */ - 480, /* (586) ordering_specification_opt ::= ASC */ - 480, /* (587) ordering_specification_opt ::= DESC */ - 481, /* (588) null_ordering_opt ::= */ - 481, /* (589) null_ordering_opt ::= NULLS FIRST */ - 481, /* (590) null_ordering_opt ::= NULLS LAST */ + 336, /* (0) cmd ::= CREATE ACCOUNT NK_ID PASS NK_STRING account_options */ + 336, /* (1) cmd ::= ALTER ACCOUNT NK_ID alter_account_options */ + 337, /* (2) account_options ::= */ + 337, /* (3) account_options ::= account_options PPS literal */ + 337, /* (4) account_options ::= account_options TSERIES literal */ + 337, /* (5) account_options ::= account_options STORAGE literal */ + 337, /* (6) account_options ::= account_options STREAMS literal */ + 337, /* (7) account_options ::= account_options QTIME literal */ + 337, /* (8) account_options ::= account_options DBS literal */ + 337, /* (9) account_options ::= account_options USERS literal */ + 337, /* (10) account_options ::= account_options CONNS literal */ + 337, /* (11) account_options ::= account_options STATE literal */ + 338, /* (12) alter_account_options ::= alter_account_option */ + 338, /* (13) alter_account_options ::= alter_account_options alter_account_option */ + 340, /* (14) alter_account_option ::= PASS literal */ + 340, /* (15) alter_account_option ::= PPS literal */ + 340, /* (16) alter_account_option ::= TSERIES literal */ + 340, /* (17) alter_account_option ::= STORAGE literal */ + 340, /* (18) alter_account_option ::= STREAMS literal */ + 340, /* (19) alter_account_option ::= QTIME literal */ + 340, /* (20) alter_account_option ::= DBS literal */ + 340, /* (21) alter_account_option ::= USERS literal */ + 340, /* (22) alter_account_option ::= CONNS literal */ + 340, /* (23) alter_account_option ::= STATE literal */ + 336, /* (24) cmd ::= CREATE USER user_name PASS NK_STRING sysinfo_opt */ + 336, /* (25) cmd ::= ALTER USER user_name PASS NK_STRING */ + 336, /* (26) cmd ::= ALTER USER user_name ENABLE NK_INTEGER */ + 336, /* (27) cmd ::= ALTER USER user_name SYSINFO NK_INTEGER */ + 336, /* (28) cmd ::= DROP USER user_name */ + 342, /* (29) sysinfo_opt ::= */ + 342, /* (30) sysinfo_opt ::= SYSINFO NK_INTEGER */ + 336, /* (31) cmd ::= GRANT privileges ON priv_level with_opt TO user_name */ + 336, /* (32) cmd ::= REVOKE privileges ON priv_level with_opt FROM user_name */ + 343, /* (33) privileges ::= ALL */ + 343, /* (34) privileges ::= priv_type_list */ + 343, /* (35) privileges ::= SUBSCRIBE */ + 346, /* (36) priv_type_list ::= priv_type */ + 346, /* (37) priv_type_list ::= priv_type_list NK_COMMA priv_type */ + 347, /* (38) priv_type ::= READ */ + 347, /* (39) priv_type ::= WRITE */ + 344, /* (40) priv_level ::= NK_STAR NK_DOT NK_STAR */ + 344, /* (41) priv_level ::= db_name NK_DOT NK_STAR */ + 344, /* (42) priv_level ::= db_name NK_DOT table_name */ + 344, /* (43) priv_level ::= topic_name */ + 345, /* (44) with_opt ::= */ + 345, /* (45) with_opt ::= WITH search_condition */ + 336, /* (46) cmd ::= CREATE DNODE dnode_endpoint */ + 336, /* (47) cmd ::= CREATE DNODE dnode_endpoint PORT NK_INTEGER */ + 336, /* (48) cmd ::= DROP DNODE NK_INTEGER force_opt */ + 336, /* (49) cmd ::= DROP DNODE dnode_endpoint force_opt */ + 336, /* (50) cmd ::= DROP DNODE NK_INTEGER unsafe_opt */ + 336, /* (51) cmd ::= DROP DNODE dnode_endpoint unsafe_opt */ + 336, /* (52) cmd ::= ALTER DNODE NK_INTEGER NK_STRING */ + 336, /* (53) cmd ::= ALTER DNODE NK_INTEGER NK_STRING NK_STRING */ + 336, /* (54) cmd ::= ALTER ALL DNODES NK_STRING */ + 336, /* (55) cmd ::= ALTER ALL DNODES NK_STRING NK_STRING */ + 336, /* (56) cmd ::= RESTORE DNODE NK_INTEGER */ + 352, /* (57) dnode_endpoint ::= NK_STRING */ + 352, /* (58) dnode_endpoint ::= NK_ID */ + 352, /* (59) dnode_endpoint ::= NK_IPTOKEN */ + 353, /* (60) force_opt ::= */ + 353, /* (61) force_opt ::= FORCE */ + 354, /* (62) unsafe_opt ::= UNSAFE */ + 336, /* (63) cmd ::= ALTER LOCAL NK_STRING */ + 336, /* (64) cmd ::= ALTER LOCAL NK_STRING NK_STRING */ + 336, /* (65) cmd ::= CREATE QNODE ON DNODE NK_INTEGER */ + 336, /* (66) cmd ::= DROP QNODE ON DNODE NK_INTEGER */ + 336, /* (67) cmd ::= RESTORE QNODE ON DNODE NK_INTEGER */ + 336, /* (68) cmd ::= CREATE BNODE ON DNODE NK_INTEGER */ + 336, /* (69) cmd ::= DROP BNODE ON DNODE NK_INTEGER */ + 336, /* (70) cmd ::= CREATE SNODE ON DNODE NK_INTEGER */ + 336, /* (71) cmd ::= DROP SNODE ON DNODE NK_INTEGER */ + 336, /* (72) cmd ::= CREATE MNODE ON DNODE NK_INTEGER */ + 336, /* (73) cmd ::= DROP MNODE ON DNODE NK_INTEGER */ + 336, /* (74) cmd ::= RESTORE MNODE ON DNODE NK_INTEGER */ + 336, /* (75) cmd ::= RESTORE VNODE ON DNODE NK_INTEGER */ + 336, /* (76) cmd ::= CREATE DATABASE not_exists_opt db_name db_options */ + 336, /* (77) cmd ::= DROP DATABASE exists_opt db_name */ + 336, /* (78) cmd ::= USE db_name */ + 336, /* (79) cmd ::= ALTER DATABASE db_name alter_db_options */ + 336, /* (80) cmd ::= FLUSH DATABASE db_name */ + 336, /* (81) cmd ::= TRIM DATABASE db_name speed_opt */ + 336, /* (82) cmd ::= COMPACT DATABASE db_name start_opt end_opt */ + 355, /* (83) not_exists_opt ::= IF NOT EXISTS */ + 355, /* (84) not_exists_opt ::= */ + 357, /* (85) exists_opt ::= IF EXISTS */ + 357, /* (86) exists_opt ::= */ + 356, /* (87) db_options ::= */ + 356, /* (88) db_options ::= db_options BUFFER NK_INTEGER */ + 356, /* (89) db_options ::= db_options CACHEMODEL NK_STRING */ + 356, /* (90) db_options ::= db_options CACHESIZE NK_INTEGER */ + 356, /* (91) db_options ::= db_options COMP NK_INTEGER */ + 356, /* (92) db_options ::= db_options DURATION NK_INTEGER */ + 356, /* (93) db_options ::= db_options DURATION NK_VARIABLE */ + 356, /* (94) db_options ::= db_options MAXROWS NK_INTEGER */ + 356, /* (95) db_options ::= db_options MINROWS NK_INTEGER */ + 356, /* (96) db_options ::= db_options KEEP integer_list */ + 356, /* (97) db_options ::= db_options KEEP variable_list */ + 356, /* (98) db_options ::= db_options PAGES NK_INTEGER */ + 356, /* (99) db_options ::= db_options PAGESIZE NK_INTEGER */ + 356, /* (100) db_options ::= db_options TSDB_PAGESIZE NK_INTEGER */ + 356, /* (101) db_options ::= db_options PRECISION NK_STRING */ + 356, /* (102) db_options ::= db_options REPLICA NK_INTEGER */ + 356, /* (103) db_options ::= db_options VGROUPS NK_INTEGER */ + 356, /* (104) db_options ::= db_options SINGLE_STABLE NK_INTEGER */ + 356, /* (105) db_options ::= db_options RETENTIONS retention_list */ + 356, /* (106) db_options ::= db_options SCHEMALESS NK_INTEGER */ + 356, /* (107) db_options ::= db_options WAL_LEVEL NK_INTEGER */ + 356, /* (108) db_options ::= db_options WAL_FSYNC_PERIOD NK_INTEGER */ + 356, /* (109) db_options ::= db_options WAL_RETENTION_PERIOD NK_INTEGER */ + 356, /* (110) db_options ::= db_options WAL_RETENTION_PERIOD NK_MINUS NK_INTEGER */ + 356, /* (111) db_options ::= db_options WAL_RETENTION_SIZE NK_INTEGER */ + 356, /* (112) db_options ::= db_options WAL_RETENTION_SIZE NK_MINUS NK_INTEGER */ + 356, /* (113) db_options ::= db_options WAL_ROLL_PERIOD NK_INTEGER */ + 356, /* (114) db_options ::= db_options WAL_SEGMENT_SIZE NK_INTEGER */ + 356, /* (115) db_options ::= db_options STT_TRIGGER NK_INTEGER */ + 356, /* (116) db_options ::= db_options TABLE_PREFIX signed */ + 356, /* (117) db_options ::= db_options TABLE_SUFFIX signed */ + 358, /* (118) alter_db_options ::= alter_db_option */ + 358, /* (119) alter_db_options ::= alter_db_options alter_db_option */ + 366, /* (120) alter_db_option ::= BUFFER NK_INTEGER */ + 366, /* (121) alter_db_option ::= CACHEMODEL NK_STRING */ + 366, /* (122) alter_db_option ::= CACHESIZE NK_INTEGER */ + 366, /* (123) alter_db_option ::= WAL_FSYNC_PERIOD NK_INTEGER */ + 366, /* (124) alter_db_option ::= KEEP integer_list */ + 366, /* (125) alter_db_option ::= KEEP variable_list */ + 366, /* (126) alter_db_option ::= PAGES NK_INTEGER */ + 366, /* (127) alter_db_option ::= REPLICA NK_INTEGER */ + 366, /* (128) alter_db_option ::= WAL_LEVEL NK_INTEGER */ + 366, /* (129) alter_db_option ::= STT_TRIGGER NK_INTEGER */ + 366, /* (130) alter_db_option ::= MINROWS NK_INTEGER */ + 366, /* (131) alter_db_option ::= WAL_RETENTION_PERIOD NK_INTEGER */ + 366, /* (132) alter_db_option ::= WAL_RETENTION_PERIOD NK_MINUS NK_INTEGER */ + 366, /* (133) alter_db_option ::= WAL_RETENTION_SIZE NK_INTEGER */ + 366, /* (134) alter_db_option ::= WAL_RETENTION_SIZE NK_MINUS NK_INTEGER */ + 362, /* (135) integer_list ::= NK_INTEGER */ + 362, /* (136) integer_list ::= integer_list NK_COMMA NK_INTEGER */ + 363, /* (137) variable_list ::= NK_VARIABLE */ + 363, /* (138) variable_list ::= variable_list NK_COMMA NK_VARIABLE */ + 364, /* (139) retention_list ::= retention */ + 364, /* (140) retention_list ::= retention_list NK_COMMA retention */ + 367, /* (141) retention ::= NK_VARIABLE NK_COLON NK_VARIABLE */ + 359, /* (142) speed_opt ::= */ + 359, /* (143) speed_opt ::= MAX_SPEED NK_INTEGER */ + 360, /* (144) start_opt ::= */ + 360, /* (145) start_opt ::= START WITH NK_INTEGER */ + 360, /* (146) start_opt ::= START WITH NK_STRING */ + 360, /* (147) start_opt ::= START WITH TIMESTAMP NK_STRING */ + 361, /* (148) end_opt ::= */ + 361, /* (149) end_opt ::= END WITH NK_INTEGER */ + 361, /* (150) end_opt ::= END WITH NK_STRING */ + 361, /* (151) end_opt ::= END WITH TIMESTAMP NK_STRING */ + 336, /* (152) cmd ::= CREATE TABLE not_exists_opt full_table_name NK_LP column_def_list NK_RP tags_def_opt table_options */ + 336, /* (153) cmd ::= CREATE TABLE multi_create_clause */ + 336, /* (154) cmd ::= CREATE STABLE not_exists_opt full_table_name NK_LP column_def_list NK_RP tags_def table_options */ + 336, /* (155) cmd ::= DROP TABLE multi_drop_clause */ + 336, /* (156) cmd ::= DROP STABLE exists_opt full_table_name */ + 336, /* (157) cmd ::= ALTER TABLE alter_table_clause */ + 336, /* (158) cmd ::= ALTER STABLE alter_table_clause */ + 375, /* (159) alter_table_clause ::= full_table_name alter_table_options */ + 375, /* (160) alter_table_clause ::= full_table_name ADD COLUMN column_name type_name */ + 375, /* (161) alter_table_clause ::= full_table_name DROP COLUMN column_name */ + 375, /* (162) alter_table_clause ::= full_table_name MODIFY COLUMN column_name type_name */ + 375, /* (163) alter_table_clause ::= full_table_name RENAME COLUMN column_name column_name */ + 375, /* (164) alter_table_clause ::= full_table_name ADD TAG column_name type_name */ + 375, /* (165) alter_table_clause ::= full_table_name DROP TAG column_name */ + 375, /* (166) alter_table_clause ::= full_table_name MODIFY TAG column_name type_name */ + 375, /* (167) alter_table_clause ::= full_table_name RENAME TAG column_name column_name */ + 375, /* (168) alter_table_clause ::= full_table_name SET TAG column_name NK_EQ signed_literal */ + 372, /* (169) multi_create_clause ::= create_subtable_clause */ + 372, /* (170) multi_create_clause ::= multi_create_clause create_subtable_clause */ + 380, /* (171) create_subtable_clause ::= not_exists_opt full_table_name USING full_table_name specific_cols_opt TAGS NK_LP expression_list NK_RP table_options */ + 374, /* (172) multi_drop_clause ::= drop_table_clause */ + 374, /* (173) multi_drop_clause ::= multi_drop_clause NK_COMMA drop_table_clause */ + 383, /* (174) drop_table_clause ::= exists_opt full_table_name */ + 381, /* (175) specific_cols_opt ::= */ + 381, /* (176) specific_cols_opt ::= NK_LP col_name_list NK_RP */ + 368, /* (177) full_table_name ::= table_name */ + 368, /* (178) full_table_name ::= db_name NK_DOT table_name */ + 369, /* (179) column_def_list ::= column_def */ + 369, /* (180) column_def_list ::= column_def_list NK_COMMA column_def */ + 385, /* (181) column_def ::= column_name type_name */ + 378, /* (182) type_name ::= BOOL */ + 378, /* (183) type_name ::= TINYINT */ + 378, /* (184) type_name ::= SMALLINT */ + 378, /* (185) type_name ::= INT */ + 378, /* (186) type_name ::= INTEGER */ + 378, /* (187) type_name ::= BIGINT */ + 378, /* (188) type_name ::= FLOAT */ + 378, /* (189) type_name ::= DOUBLE */ + 378, /* (190) type_name ::= BINARY NK_LP NK_INTEGER NK_RP */ + 378, /* (191) type_name ::= TIMESTAMP */ + 378, /* (192) type_name ::= NCHAR NK_LP NK_INTEGER NK_RP */ + 378, /* (193) type_name ::= TINYINT UNSIGNED */ + 378, /* (194) type_name ::= SMALLINT UNSIGNED */ + 378, /* (195) type_name ::= INT UNSIGNED */ + 378, /* (196) type_name ::= BIGINT UNSIGNED */ + 378, /* (197) type_name ::= JSON */ + 378, /* (198) type_name ::= VARCHAR NK_LP NK_INTEGER NK_RP */ + 378, /* (199) type_name ::= MEDIUMBLOB */ + 378, /* (200) type_name ::= BLOB */ + 378, /* (201) type_name ::= VARBINARY NK_LP NK_INTEGER NK_RP */ + 378, /* (202) type_name ::= DECIMAL */ + 378, /* (203) type_name ::= DECIMAL NK_LP NK_INTEGER NK_RP */ + 378, /* (204) type_name ::= DECIMAL NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP */ + 370, /* (205) tags_def_opt ::= */ + 370, /* (206) tags_def_opt ::= tags_def */ + 373, /* (207) tags_def ::= TAGS NK_LP column_def_list NK_RP */ + 371, /* (208) table_options ::= */ + 371, /* (209) table_options ::= table_options COMMENT NK_STRING */ + 371, /* (210) table_options ::= table_options MAX_DELAY duration_list */ + 371, /* (211) table_options ::= table_options WATERMARK duration_list */ + 371, /* (212) table_options ::= table_options ROLLUP NK_LP rollup_func_list NK_RP */ + 371, /* (213) table_options ::= table_options TTL NK_INTEGER */ + 371, /* (214) table_options ::= table_options SMA NK_LP col_name_list NK_RP */ + 371, /* (215) table_options ::= table_options DELETE_MARK duration_list */ + 376, /* (216) alter_table_options ::= alter_table_option */ + 376, /* (217) alter_table_options ::= alter_table_options alter_table_option */ + 388, /* (218) alter_table_option ::= COMMENT NK_STRING */ + 388, /* (219) alter_table_option ::= TTL NK_INTEGER */ + 386, /* (220) duration_list ::= duration_literal */ + 386, /* (221) duration_list ::= duration_list NK_COMMA duration_literal */ + 387, /* (222) rollup_func_list ::= rollup_func_name */ + 387, /* (223) rollup_func_list ::= rollup_func_list NK_COMMA rollup_func_name */ + 390, /* (224) rollup_func_name ::= function_name */ + 390, /* (225) rollup_func_name ::= FIRST */ + 390, /* (226) rollup_func_name ::= LAST */ + 384, /* (227) col_name_list ::= col_name */ + 384, /* (228) col_name_list ::= col_name_list NK_COMMA col_name */ + 392, /* (229) col_name ::= column_name */ + 336, /* (230) cmd ::= SHOW DNODES */ + 336, /* (231) cmd ::= SHOW USERS */ + 336, /* (232) cmd ::= SHOW USER PRIVILEGES */ + 336, /* (233) cmd ::= SHOW DATABASES */ + 336, /* (234) cmd ::= SHOW db_name_cond_opt TABLES like_pattern_opt */ + 336, /* (235) cmd ::= SHOW db_name_cond_opt STABLES like_pattern_opt */ + 336, /* (236) cmd ::= SHOW db_name_cond_opt VGROUPS */ + 336, /* (237) cmd ::= SHOW MNODES */ + 336, /* (238) cmd ::= SHOW QNODES */ + 336, /* (239) cmd ::= SHOW FUNCTIONS */ + 336, /* (240) cmd ::= SHOW INDEXES FROM table_name_cond from_db_opt */ + 336, /* (241) cmd ::= SHOW STREAMS */ + 336, /* (242) cmd ::= SHOW ACCOUNTS */ + 336, /* (243) cmd ::= SHOW APPS */ + 336, /* (244) cmd ::= SHOW CONNECTIONS */ + 336, /* (245) cmd ::= SHOW LICENCES */ + 336, /* (246) cmd ::= SHOW GRANTS */ + 336, /* (247) cmd ::= SHOW CREATE DATABASE db_name */ + 336, /* (248) cmd ::= SHOW CREATE TABLE full_table_name */ + 336, /* (249) cmd ::= SHOW CREATE STABLE full_table_name */ + 336, /* (250) cmd ::= SHOW QUERIES */ + 336, /* (251) cmd ::= SHOW SCORES */ + 336, /* (252) cmd ::= SHOW TOPICS */ + 336, /* (253) cmd ::= SHOW VARIABLES */ + 336, /* (254) cmd ::= SHOW CLUSTER VARIABLES */ + 336, /* (255) cmd ::= SHOW LOCAL VARIABLES */ + 336, /* (256) cmd ::= SHOW DNODE NK_INTEGER VARIABLES like_pattern_opt */ + 336, /* (257) cmd ::= SHOW BNODES */ + 336, /* (258) cmd ::= SHOW SNODES */ + 336, /* (259) cmd ::= SHOW CLUSTER */ + 336, /* (260) cmd ::= SHOW TRANSACTIONS */ + 336, /* (261) cmd ::= SHOW TABLE DISTRIBUTED full_table_name */ + 336, /* (262) cmd ::= SHOW CONSUMERS */ + 336, /* (263) cmd ::= SHOW SUBSCRIPTIONS */ + 336, /* (264) cmd ::= SHOW TAGS FROM table_name_cond from_db_opt */ + 336, /* (265) cmd ::= SHOW TABLE TAGS tag_list_opt FROM table_name_cond from_db_opt */ + 336, /* (266) cmd ::= SHOW VNODES NK_INTEGER */ + 336, /* (267) cmd ::= SHOW VNODES NK_STRING */ + 336, /* (268) cmd ::= SHOW db_name_cond_opt ALIVE */ + 336, /* (269) cmd ::= SHOW CLUSTER ALIVE */ + 393, /* (270) db_name_cond_opt ::= */ + 393, /* (271) db_name_cond_opt ::= db_name NK_DOT */ + 394, /* (272) like_pattern_opt ::= */ + 394, /* (273) like_pattern_opt ::= LIKE NK_STRING */ + 395, /* (274) table_name_cond ::= table_name */ + 396, /* (275) from_db_opt ::= */ + 396, /* (276) from_db_opt ::= FROM db_name */ + 397, /* (277) tag_list_opt ::= */ + 397, /* (278) tag_list_opt ::= tag_item */ + 397, /* (279) tag_list_opt ::= tag_list_opt NK_COMMA tag_item */ + 398, /* (280) tag_item ::= TBNAME */ + 398, /* (281) tag_item ::= QTAGS */ + 398, /* (282) tag_item ::= column_name */ + 398, /* (283) tag_item ::= column_name column_alias */ + 398, /* (284) tag_item ::= column_name AS column_alias */ + 336, /* (285) cmd ::= CREATE SMA INDEX not_exists_opt full_index_name ON full_table_name index_options */ + 336, /* (286) cmd ::= CREATE INDEX not_exists_opt full_index_name ON full_table_name NK_LP col_name_list NK_RP */ + 336, /* (287) cmd ::= DROP INDEX exists_opt full_index_name */ + 400, /* (288) full_index_name ::= index_name */ + 400, /* (289) full_index_name ::= db_name NK_DOT index_name */ + 401, /* (290) index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_RP sliding_opt sma_stream_opt */ + 401, /* (291) 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 */ + 403, /* (292) func_list ::= func */ + 403, /* (293) func_list ::= func_list NK_COMMA func */ + 406, /* (294) func ::= sma_func_name NK_LP expression_list NK_RP */ + 407, /* (295) sma_func_name ::= function_name */ + 407, /* (296) sma_func_name ::= COUNT */ + 407, /* (297) sma_func_name ::= FIRST */ + 407, /* (298) sma_func_name ::= LAST */ + 407, /* (299) sma_func_name ::= LAST_ROW */ + 405, /* (300) sma_stream_opt ::= */ + 405, /* (301) sma_stream_opt ::= sma_stream_opt WATERMARK duration_literal */ + 405, /* (302) sma_stream_opt ::= sma_stream_opt MAX_DELAY duration_literal */ + 405, /* (303) sma_stream_opt ::= sma_stream_opt DELETE_MARK duration_literal */ + 336, /* (304) cmd ::= CREATE TOPIC not_exists_opt topic_name AS query_or_subquery */ + 336, /* (305) cmd ::= CREATE TOPIC not_exists_opt topic_name AS DATABASE db_name */ + 336, /* (306) cmd ::= CREATE TOPIC not_exists_opt topic_name WITH META AS DATABASE db_name */ + 336, /* (307) cmd ::= CREATE TOPIC not_exists_opt topic_name AS STABLE full_table_name */ + 336, /* (308) cmd ::= CREATE TOPIC not_exists_opt topic_name WITH META AS STABLE full_table_name */ + 336, /* (309) cmd ::= DROP TOPIC exists_opt topic_name */ + 336, /* (310) cmd ::= DROP CONSUMER GROUP exists_opt cgroup_name ON topic_name */ + 336, /* (311) cmd ::= DESC full_table_name */ + 336, /* (312) cmd ::= DESCRIBE full_table_name */ + 336, /* (313) cmd ::= RESET QUERY CACHE */ + 336, /* (314) cmd ::= EXPLAIN analyze_opt explain_options query_or_subquery */ + 336, /* (315) cmd ::= EXPLAIN analyze_opt explain_options insert_query */ + 410, /* (316) analyze_opt ::= */ + 410, /* (317) analyze_opt ::= ANALYZE */ + 411, /* (318) explain_options ::= */ + 411, /* (319) explain_options ::= explain_options VERBOSE NK_BOOL */ + 411, /* (320) explain_options ::= explain_options RATIO NK_FLOAT */ + 336, /* (321) cmd ::= CREATE or_replace_opt agg_func_opt FUNCTION not_exists_opt function_name AS NK_STRING OUTPUTTYPE type_name bufsize_opt language_opt */ + 336, /* (322) cmd ::= DROP FUNCTION exists_opt function_name */ + 414, /* (323) agg_func_opt ::= */ + 414, /* (324) agg_func_opt ::= AGGREGATE */ + 415, /* (325) bufsize_opt ::= */ + 415, /* (326) bufsize_opt ::= BUFSIZE NK_INTEGER */ + 416, /* (327) language_opt ::= */ + 416, /* (328) language_opt ::= LANGUAGE NK_STRING */ + 413, /* (329) or_replace_opt ::= */ + 413, /* (330) or_replace_opt ::= OR REPLACE */ + 336, /* (331) 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 */ + 336, /* (332) cmd ::= DROP STREAM exists_opt stream_name */ + 336, /* (333) cmd ::= PAUSE STREAM exists_opt stream_name */ + 336, /* (334) cmd ::= RESUME STREAM exists_opt ignore_opt stream_name */ + 419, /* (335) col_list_opt ::= */ + 419, /* (336) col_list_opt ::= NK_LP col_name_list NK_RP */ + 420, /* (337) tag_def_or_ref_opt ::= */ + 420, /* (338) tag_def_or_ref_opt ::= tags_def */ + 420, /* (339) tag_def_or_ref_opt ::= TAGS NK_LP col_name_list NK_RP */ + 418, /* (340) stream_options ::= */ + 418, /* (341) stream_options ::= stream_options TRIGGER AT_ONCE */ + 418, /* (342) stream_options ::= stream_options TRIGGER WINDOW_CLOSE */ + 418, /* (343) stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal */ + 418, /* (344) stream_options ::= stream_options WATERMARK duration_literal */ + 418, /* (345) stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER */ + 418, /* (346) stream_options ::= stream_options FILL_HISTORY NK_INTEGER */ + 418, /* (347) stream_options ::= stream_options DELETE_MARK duration_literal */ + 418, /* (348) stream_options ::= stream_options IGNORE UPDATE NK_INTEGER */ + 421, /* (349) subtable_opt ::= */ + 421, /* (350) subtable_opt ::= SUBTABLE NK_LP expression NK_RP */ + 422, /* (351) ignore_opt ::= */ + 422, /* (352) ignore_opt ::= IGNORE UNTREATED */ + 336, /* (353) cmd ::= KILL CONNECTION NK_INTEGER */ + 336, /* (354) cmd ::= KILL QUERY NK_STRING */ + 336, /* (355) cmd ::= KILL TRANSACTION NK_INTEGER */ + 336, /* (356) cmd ::= BALANCE VGROUP */ + 336, /* (357) cmd ::= BALANCE VGROUP LEADER */ + 336, /* (358) cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER */ + 336, /* (359) cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list */ + 336, /* (360) cmd ::= SPLIT VGROUP NK_INTEGER */ + 424, /* (361) dnode_list ::= DNODE NK_INTEGER */ + 424, /* (362) dnode_list ::= dnode_list DNODE NK_INTEGER */ + 336, /* (363) cmd ::= DELETE FROM full_table_name where_clause_opt */ + 336, /* (364) cmd ::= query_or_subquery */ + 336, /* (365) cmd ::= insert_query */ + 412, /* (366) insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery */ + 412, /* (367) insert_query ::= INSERT INTO full_table_name query_or_subquery */ + 339, /* (368) literal ::= NK_INTEGER */ + 339, /* (369) literal ::= NK_FLOAT */ + 339, /* (370) literal ::= NK_STRING */ + 339, /* (371) literal ::= NK_BOOL */ + 339, /* (372) literal ::= TIMESTAMP NK_STRING */ + 339, /* (373) literal ::= duration_literal */ + 339, /* (374) literal ::= NULL */ + 339, /* (375) literal ::= NK_QUESTION */ + 389, /* (376) duration_literal ::= NK_VARIABLE */ + 365, /* (377) signed ::= NK_INTEGER */ + 365, /* (378) signed ::= NK_PLUS NK_INTEGER */ + 365, /* (379) signed ::= NK_MINUS NK_INTEGER */ + 365, /* (380) signed ::= NK_FLOAT */ + 365, /* (381) signed ::= NK_PLUS NK_FLOAT */ + 365, /* (382) signed ::= NK_MINUS NK_FLOAT */ + 379, /* (383) signed_literal ::= signed */ + 379, /* (384) signed_literal ::= NK_STRING */ + 379, /* (385) signed_literal ::= NK_BOOL */ + 379, /* (386) signed_literal ::= TIMESTAMP NK_STRING */ + 379, /* (387) signed_literal ::= duration_literal */ + 379, /* (388) signed_literal ::= NULL */ + 379, /* (389) signed_literal ::= literal_func */ + 379, /* (390) signed_literal ::= NK_QUESTION */ + 427, /* (391) literal_list ::= signed_literal */ + 427, /* (392) literal_list ::= literal_list NK_COMMA signed_literal */ + 348, /* (393) db_name ::= NK_ID */ + 349, /* (394) table_name ::= NK_ID */ + 377, /* (395) column_name ::= NK_ID */ + 391, /* (396) function_name ::= NK_ID */ + 428, /* (397) table_alias ::= NK_ID */ + 399, /* (398) column_alias ::= NK_ID */ + 341, /* (399) user_name ::= NK_ID */ + 350, /* (400) topic_name ::= NK_ID */ + 417, /* (401) stream_name ::= NK_ID */ + 409, /* (402) cgroup_name ::= NK_ID */ + 402, /* (403) index_name ::= NK_ID */ + 429, /* (404) expr_or_subquery ::= expression */ + 423, /* (405) expression ::= literal */ + 423, /* (406) expression ::= pseudo_column */ + 423, /* (407) expression ::= column_reference */ + 423, /* (408) expression ::= function_expression */ + 423, /* (409) expression ::= case_when_expression */ + 423, /* (410) expression ::= NK_LP expression NK_RP */ + 423, /* (411) expression ::= NK_PLUS expr_or_subquery */ + 423, /* (412) expression ::= NK_MINUS expr_or_subquery */ + 423, /* (413) expression ::= expr_or_subquery NK_PLUS expr_or_subquery */ + 423, /* (414) expression ::= expr_or_subquery NK_MINUS expr_or_subquery */ + 423, /* (415) expression ::= expr_or_subquery NK_STAR expr_or_subquery */ + 423, /* (416) expression ::= expr_or_subquery NK_SLASH expr_or_subquery */ + 423, /* (417) expression ::= expr_or_subquery NK_REM expr_or_subquery */ + 423, /* (418) expression ::= column_reference NK_ARROW NK_STRING */ + 423, /* (419) expression ::= expr_or_subquery NK_BITAND expr_or_subquery */ + 423, /* (420) expression ::= expr_or_subquery NK_BITOR expr_or_subquery */ + 382, /* (421) expression_list ::= expr_or_subquery */ + 382, /* (422) expression_list ::= expression_list NK_COMMA expr_or_subquery */ + 431, /* (423) column_reference ::= column_name */ + 431, /* (424) column_reference ::= table_name NK_DOT column_name */ + 430, /* (425) pseudo_column ::= ROWTS */ + 430, /* (426) pseudo_column ::= TBNAME */ + 430, /* (427) pseudo_column ::= table_name NK_DOT TBNAME */ + 430, /* (428) pseudo_column ::= QSTART */ + 430, /* (429) pseudo_column ::= QEND */ + 430, /* (430) pseudo_column ::= QDURATION */ + 430, /* (431) pseudo_column ::= WSTART */ + 430, /* (432) pseudo_column ::= WEND */ + 430, /* (433) pseudo_column ::= WDURATION */ + 430, /* (434) pseudo_column ::= IROWTS */ + 430, /* (435) pseudo_column ::= ISFILLED */ + 430, /* (436) pseudo_column ::= QTAGS */ + 432, /* (437) function_expression ::= function_name NK_LP expression_list NK_RP */ + 432, /* (438) function_expression ::= star_func NK_LP star_func_para_list NK_RP */ + 432, /* (439) function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP */ + 432, /* (440) function_expression ::= literal_func */ + 426, /* (441) literal_func ::= noarg_func NK_LP NK_RP */ + 426, /* (442) literal_func ::= NOW */ + 436, /* (443) noarg_func ::= NOW */ + 436, /* (444) noarg_func ::= TODAY */ + 436, /* (445) noarg_func ::= TIMEZONE */ + 436, /* (446) noarg_func ::= DATABASE */ + 436, /* (447) noarg_func ::= CLIENT_VERSION */ + 436, /* (448) noarg_func ::= SERVER_VERSION */ + 436, /* (449) noarg_func ::= SERVER_STATUS */ + 436, /* (450) noarg_func ::= CURRENT_USER */ + 436, /* (451) noarg_func ::= USER */ + 434, /* (452) star_func ::= COUNT */ + 434, /* (453) star_func ::= FIRST */ + 434, /* (454) star_func ::= LAST */ + 434, /* (455) star_func ::= LAST_ROW */ + 435, /* (456) star_func_para_list ::= NK_STAR */ + 435, /* (457) star_func_para_list ::= other_para_list */ + 437, /* (458) other_para_list ::= star_func_para */ + 437, /* (459) other_para_list ::= other_para_list NK_COMMA star_func_para */ + 438, /* (460) star_func_para ::= expr_or_subquery */ + 438, /* (461) star_func_para ::= table_name NK_DOT NK_STAR */ + 433, /* (462) case_when_expression ::= CASE when_then_list case_when_else_opt END */ + 433, /* (463) case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END */ + 439, /* (464) when_then_list ::= when_then_expr */ + 439, /* (465) when_then_list ::= when_then_list when_then_expr */ + 442, /* (466) when_then_expr ::= WHEN common_expression THEN common_expression */ + 440, /* (467) case_when_else_opt ::= */ + 440, /* (468) case_when_else_opt ::= ELSE common_expression */ + 443, /* (469) predicate ::= expr_or_subquery compare_op expr_or_subquery */ + 443, /* (470) predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery */ + 443, /* (471) predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery */ + 443, /* (472) predicate ::= expr_or_subquery IS NULL */ + 443, /* (473) predicate ::= expr_or_subquery IS NOT NULL */ + 443, /* (474) predicate ::= expr_or_subquery in_op in_predicate_value */ + 444, /* (475) compare_op ::= NK_LT */ + 444, /* (476) compare_op ::= NK_GT */ + 444, /* (477) compare_op ::= NK_LE */ + 444, /* (478) compare_op ::= NK_GE */ + 444, /* (479) compare_op ::= NK_NE */ + 444, /* (480) compare_op ::= NK_EQ */ + 444, /* (481) compare_op ::= LIKE */ + 444, /* (482) compare_op ::= NOT LIKE */ + 444, /* (483) compare_op ::= MATCH */ + 444, /* (484) compare_op ::= NMATCH */ + 444, /* (485) compare_op ::= CONTAINS */ + 445, /* (486) in_op ::= IN */ + 445, /* (487) in_op ::= NOT IN */ + 446, /* (488) in_predicate_value ::= NK_LP literal_list NK_RP */ + 447, /* (489) boolean_value_expression ::= boolean_primary */ + 447, /* (490) boolean_value_expression ::= NOT boolean_primary */ + 447, /* (491) boolean_value_expression ::= boolean_value_expression OR boolean_value_expression */ + 447, /* (492) boolean_value_expression ::= boolean_value_expression AND boolean_value_expression */ + 448, /* (493) boolean_primary ::= predicate */ + 448, /* (494) boolean_primary ::= NK_LP boolean_value_expression NK_RP */ + 441, /* (495) common_expression ::= expr_or_subquery */ + 441, /* (496) common_expression ::= boolean_value_expression */ + 449, /* (497) from_clause_opt ::= */ + 449, /* (498) from_clause_opt ::= FROM table_reference_list */ + 450, /* (499) table_reference_list ::= table_reference */ + 450, /* (500) table_reference_list ::= table_reference_list NK_COMMA table_reference */ + 451, /* (501) table_reference ::= table_primary */ + 451, /* (502) table_reference ::= joined_table */ + 452, /* (503) table_primary ::= table_name alias_opt */ + 452, /* (504) table_primary ::= db_name NK_DOT table_name alias_opt */ + 452, /* (505) table_primary ::= subquery alias_opt */ + 452, /* (506) table_primary ::= parenthesized_joined_table */ + 454, /* (507) alias_opt ::= */ + 454, /* (508) alias_opt ::= table_alias */ + 454, /* (509) alias_opt ::= AS table_alias */ + 456, /* (510) parenthesized_joined_table ::= NK_LP joined_table NK_RP */ + 456, /* (511) parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP */ + 453, /* (512) joined_table ::= table_reference join_type JOIN table_reference ON search_condition */ + 457, /* (513) join_type ::= */ + 457, /* (514) join_type ::= INNER */ + 458, /* (515) query_specification ::= SELECT set_quantifier_opt select_list from_clause_opt where_clause_opt partition_by_clause_opt range_opt every_opt fill_opt twindow_clause_opt group_by_clause_opt having_clause_opt */ + 459, /* (516) set_quantifier_opt ::= */ + 459, /* (517) set_quantifier_opt ::= DISTINCT */ + 459, /* (518) set_quantifier_opt ::= ALL */ + 460, /* (519) select_list ::= select_item */ + 460, /* (520) select_list ::= select_list NK_COMMA select_item */ + 468, /* (521) select_item ::= NK_STAR */ + 468, /* (522) select_item ::= common_expression */ + 468, /* (523) select_item ::= common_expression column_alias */ + 468, /* (524) select_item ::= common_expression AS column_alias */ + 468, /* (525) select_item ::= table_name NK_DOT NK_STAR */ + 425, /* (526) where_clause_opt ::= */ + 425, /* (527) where_clause_opt ::= WHERE search_condition */ + 461, /* (528) partition_by_clause_opt ::= */ + 461, /* (529) partition_by_clause_opt ::= PARTITION BY partition_list */ + 469, /* (530) partition_list ::= partition_item */ + 469, /* (531) partition_list ::= partition_list NK_COMMA partition_item */ + 470, /* (532) partition_item ::= expr_or_subquery */ + 470, /* (533) partition_item ::= expr_or_subquery column_alias */ + 470, /* (534) partition_item ::= expr_or_subquery AS column_alias */ + 465, /* (535) twindow_clause_opt ::= */ + 465, /* (536) twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA duration_literal NK_RP */ + 465, /* (537) twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP */ + 465, /* (538) twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_RP sliding_opt fill_opt */ + 465, /* (539) twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_COMMA duration_literal NK_RP sliding_opt fill_opt */ + 465, /* (540) twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition */ + 404, /* (541) sliding_opt ::= */ + 404, /* (542) sliding_opt ::= SLIDING NK_LP duration_literal NK_RP */ + 464, /* (543) fill_opt ::= */ + 464, /* (544) fill_opt ::= FILL NK_LP fill_mode NK_RP */ + 464, /* (545) fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP */ + 464, /* (546) fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP */ + 471, /* (547) fill_mode ::= NONE */ + 471, /* (548) fill_mode ::= PREV */ + 471, /* (549) fill_mode ::= NULL */ + 471, /* (550) fill_mode ::= NULL_F */ + 471, /* (551) fill_mode ::= LINEAR */ + 471, /* (552) fill_mode ::= NEXT */ + 466, /* (553) group_by_clause_opt ::= */ + 466, /* (554) group_by_clause_opt ::= GROUP BY group_by_list */ + 472, /* (555) group_by_list ::= expr_or_subquery */ + 472, /* (556) group_by_list ::= group_by_list NK_COMMA expr_or_subquery */ + 467, /* (557) having_clause_opt ::= */ + 467, /* (558) having_clause_opt ::= HAVING search_condition */ + 462, /* (559) range_opt ::= */ + 462, /* (560) range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP */ + 463, /* (561) every_opt ::= */ + 463, /* (562) every_opt ::= EVERY NK_LP duration_literal NK_RP */ + 473, /* (563) query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt */ + 474, /* (564) query_simple ::= query_specification */ + 474, /* (565) query_simple ::= union_query_expression */ + 478, /* (566) union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery */ + 478, /* (567) union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery */ + 479, /* (568) query_simple_or_subquery ::= query_simple */ + 479, /* (569) query_simple_or_subquery ::= subquery */ + 408, /* (570) query_or_subquery ::= query_expression */ + 408, /* (571) query_or_subquery ::= subquery */ + 475, /* (572) order_by_clause_opt ::= */ + 475, /* (573) order_by_clause_opt ::= ORDER BY sort_specification_list */ + 476, /* (574) slimit_clause_opt ::= */ + 476, /* (575) slimit_clause_opt ::= SLIMIT NK_INTEGER */ + 476, /* (576) slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER */ + 476, /* (577) slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER */ + 477, /* (578) limit_clause_opt ::= */ + 477, /* (579) limit_clause_opt ::= LIMIT NK_INTEGER */ + 477, /* (580) limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER */ + 477, /* (581) limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER */ + 455, /* (582) subquery ::= NK_LP query_expression NK_RP */ + 455, /* (583) subquery ::= NK_LP subquery NK_RP */ + 351, /* (584) search_condition ::= common_expression */ + 480, /* (585) sort_specification_list ::= sort_specification */ + 480, /* (586) sort_specification_list ::= sort_specification_list NK_COMMA sort_specification */ + 481, /* (587) sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt */ + 482, /* (588) ordering_specification_opt ::= */ + 482, /* (589) ordering_specification_opt ::= ASC */ + 482, /* (590) ordering_specification_opt ::= DESC */ + 483, /* (591) null_ordering_opt ::= */ + 483, /* (592) null_ordering_opt ::= NULLS FIRST */ + 483, /* (593) null_ordering_opt ::= NULLS LAST */ }; /* For rule J, yyRuleInfoNRhs[J] contains the negative of the number @@ -3872,547 +3848,550 @@ static const signed char yyRuleInfoNRhs[] = { -5, /* (47) cmd ::= CREATE DNODE dnode_endpoint PORT NK_INTEGER */ -4, /* (48) cmd ::= DROP DNODE NK_INTEGER force_opt */ -4, /* (49) cmd ::= DROP DNODE dnode_endpoint force_opt */ - -4, /* (50) cmd ::= ALTER DNODE NK_INTEGER NK_STRING */ - -5, /* (51) cmd ::= ALTER DNODE NK_INTEGER NK_STRING NK_STRING */ - -4, /* (52) cmd ::= ALTER ALL DNODES NK_STRING */ - -5, /* (53) cmd ::= ALTER ALL DNODES NK_STRING NK_STRING */ - -3, /* (54) cmd ::= RESTORE DNODE NK_INTEGER */ - -1, /* (55) dnode_endpoint ::= NK_STRING */ - -1, /* (56) dnode_endpoint ::= NK_ID */ - -1, /* (57) dnode_endpoint ::= NK_IPTOKEN */ - 0, /* (58) force_opt ::= */ - -1, /* (59) force_opt ::= FORCE */ - -3, /* (60) cmd ::= ALTER LOCAL NK_STRING */ - -4, /* (61) cmd ::= ALTER LOCAL NK_STRING NK_STRING */ - -5, /* (62) cmd ::= CREATE QNODE ON DNODE NK_INTEGER */ - -5, /* (63) cmd ::= DROP QNODE ON DNODE NK_INTEGER */ - -5, /* (64) cmd ::= RESTORE QNODE ON DNODE NK_INTEGER */ - -5, /* (65) cmd ::= CREATE BNODE ON DNODE NK_INTEGER */ - -5, /* (66) cmd ::= DROP BNODE ON DNODE NK_INTEGER */ - -5, /* (67) cmd ::= CREATE SNODE ON DNODE NK_INTEGER */ - -5, /* (68) cmd ::= DROP SNODE ON DNODE NK_INTEGER */ - -5, /* (69) cmd ::= CREATE MNODE ON DNODE NK_INTEGER */ - -5, /* (70) cmd ::= DROP MNODE ON DNODE NK_INTEGER */ - -5, /* (71) cmd ::= RESTORE MNODE ON DNODE NK_INTEGER */ - -5, /* (72) cmd ::= RESTORE VNODE ON DNODE NK_INTEGER */ - -5, /* (73) cmd ::= CREATE DATABASE not_exists_opt db_name db_options */ - -4, /* (74) cmd ::= DROP DATABASE exists_opt db_name */ - -2, /* (75) cmd ::= USE db_name */ - -4, /* (76) cmd ::= ALTER DATABASE db_name alter_db_options */ - -3, /* (77) cmd ::= FLUSH DATABASE db_name */ - -4, /* (78) cmd ::= TRIM DATABASE db_name speed_opt */ - -5, /* (79) cmd ::= COMPACT DATABASE db_name start_opt end_opt */ - -3, /* (80) not_exists_opt ::= IF NOT EXISTS */ - 0, /* (81) not_exists_opt ::= */ - -2, /* (82) exists_opt ::= IF EXISTS */ - 0, /* (83) exists_opt ::= */ - 0, /* (84) db_options ::= */ - -3, /* (85) db_options ::= db_options BUFFER NK_INTEGER */ - -3, /* (86) db_options ::= db_options CACHEMODEL NK_STRING */ - -3, /* (87) db_options ::= db_options CACHESIZE NK_INTEGER */ - -3, /* (88) db_options ::= db_options COMP NK_INTEGER */ - -3, /* (89) db_options ::= db_options DURATION NK_INTEGER */ - -3, /* (90) db_options ::= db_options DURATION NK_VARIABLE */ - -3, /* (91) db_options ::= db_options MAXROWS NK_INTEGER */ - -3, /* (92) db_options ::= db_options MINROWS NK_INTEGER */ - -3, /* (93) db_options ::= db_options KEEP integer_list */ - -3, /* (94) db_options ::= db_options KEEP variable_list */ - -3, /* (95) db_options ::= db_options PAGES NK_INTEGER */ - -3, /* (96) db_options ::= db_options PAGESIZE NK_INTEGER */ - -3, /* (97) db_options ::= db_options TSDB_PAGESIZE NK_INTEGER */ - -3, /* (98) db_options ::= db_options PRECISION NK_STRING */ - -3, /* (99) db_options ::= db_options REPLICA NK_INTEGER */ - -3, /* (100) db_options ::= db_options VGROUPS NK_INTEGER */ - -3, /* (101) db_options ::= db_options SINGLE_STABLE NK_INTEGER */ - -3, /* (102) db_options ::= db_options RETENTIONS retention_list */ - -3, /* (103) db_options ::= db_options SCHEMALESS NK_INTEGER */ - -3, /* (104) db_options ::= db_options WAL_LEVEL NK_INTEGER */ - -3, /* (105) db_options ::= db_options WAL_FSYNC_PERIOD NK_INTEGER */ - -3, /* (106) db_options ::= db_options WAL_RETENTION_PERIOD NK_INTEGER */ - -4, /* (107) db_options ::= db_options WAL_RETENTION_PERIOD NK_MINUS NK_INTEGER */ - -3, /* (108) db_options ::= db_options WAL_RETENTION_SIZE NK_INTEGER */ - -4, /* (109) db_options ::= db_options WAL_RETENTION_SIZE NK_MINUS NK_INTEGER */ - -3, /* (110) db_options ::= db_options WAL_ROLL_PERIOD NK_INTEGER */ - -3, /* (111) db_options ::= db_options WAL_SEGMENT_SIZE NK_INTEGER */ - -3, /* (112) db_options ::= db_options STT_TRIGGER NK_INTEGER */ - -3, /* (113) db_options ::= db_options TABLE_PREFIX signed */ - -3, /* (114) db_options ::= db_options TABLE_SUFFIX signed */ - -1, /* (115) alter_db_options ::= alter_db_option */ - -2, /* (116) alter_db_options ::= alter_db_options alter_db_option */ - -2, /* (117) alter_db_option ::= BUFFER NK_INTEGER */ - -2, /* (118) alter_db_option ::= CACHEMODEL NK_STRING */ - -2, /* (119) alter_db_option ::= CACHESIZE NK_INTEGER */ - -2, /* (120) alter_db_option ::= WAL_FSYNC_PERIOD NK_INTEGER */ - -2, /* (121) alter_db_option ::= KEEP integer_list */ - -2, /* (122) alter_db_option ::= KEEP variable_list */ - -2, /* (123) alter_db_option ::= PAGES NK_INTEGER */ - -2, /* (124) alter_db_option ::= REPLICA NK_INTEGER */ - -2, /* (125) alter_db_option ::= WAL_LEVEL NK_INTEGER */ - -2, /* (126) alter_db_option ::= STT_TRIGGER NK_INTEGER */ - -2, /* (127) alter_db_option ::= MINROWS NK_INTEGER */ - -2, /* (128) alter_db_option ::= WAL_RETENTION_PERIOD NK_INTEGER */ - -3, /* (129) alter_db_option ::= WAL_RETENTION_PERIOD NK_MINUS NK_INTEGER */ - -2, /* (130) alter_db_option ::= WAL_RETENTION_SIZE NK_INTEGER */ - -3, /* (131) alter_db_option ::= WAL_RETENTION_SIZE NK_MINUS NK_INTEGER */ - -1, /* (132) integer_list ::= NK_INTEGER */ - -3, /* (133) integer_list ::= integer_list NK_COMMA NK_INTEGER */ - -1, /* (134) variable_list ::= NK_VARIABLE */ - -3, /* (135) variable_list ::= variable_list NK_COMMA NK_VARIABLE */ - -1, /* (136) retention_list ::= retention */ - -3, /* (137) retention_list ::= retention_list NK_COMMA retention */ - -3, /* (138) retention ::= NK_VARIABLE NK_COLON NK_VARIABLE */ - 0, /* (139) speed_opt ::= */ - -2, /* (140) speed_opt ::= MAX_SPEED NK_INTEGER */ - 0, /* (141) start_opt ::= */ - -3, /* (142) start_opt ::= START WITH NK_INTEGER */ - -3, /* (143) start_opt ::= START WITH NK_STRING */ - -4, /* (144) start_opt ::= START WITH TIMESTAMP NK_STRING */ - 0, /* (145) end_opt ::= */ - -3, /* (146) end_opt ::= END WITH NK_INTEGER */ - -3, /* (147) end_opt ::= END WITH NK_STRING */ - -4, /* (148) end_opt ::= END WITH TIMESTAMP NK_STRING */ - -9, /* (149) cmd ::= CREATE TABLE not_exists_opt full_table_name NK_LP column_def_list NK_RP tags_def_opt table_options */ - -3, /* (150) cmd ::= CREATE TABLE multi_create_clause */ - -9, /* (151) cmd ::= CREATE STABLE not_exists_opt full_table_name NK_LP column_def_list NK_RP tags_def table_options */ - -3, /* (152) cmd ::= DROP TABLE multi_drop_clause */ - -4, /* (153) cmd ::= DROP STABLE exists_opt full_table_name */ - -3, /* (154) cmd ::= ALTER TABLE alter_table_clause */ - -3, /* (155) cmd ::= ALTER STABLE alter_table_clause */ - -2, /* (156) alter_table_clause ::= full_table_name alter_table_options */ - -5, /* (157) alter_table_clause ::= full_table_name ADD COLUMN column_name type_name */ - -4, /* (158) alter_table_clause ::= full_table_name DROP COLUMN column_name */ - -5, /* (159) alter_table_clause ::= full_table_name MODIFY COLUMN column_name type_name */ - -5, /* (160) alter_table_clause ::= full_table_name RENAME COLUMN column_name column_name */ - -5, /* (161) alter_table_clause ::= full_table_name ADD TAG column_name type_name */ - -4, /* (162) alter_table_clause ::= full_table_name DROP TAG column_name */ - -5, /* (163) alter_table_clause ::= full_table_name MODIFY TAG column_name type_name */ - -5, /* (164) alter_table_clause ::= full_table_name RENAME TAG column_name column_name */ - -6, /* (165) alter_table_clause ::= full_table_name SET TAG column_name NK_EQ signed_literal */ - -1, /* (166) multi_create_clause ::= create_subtable_clause */ - -2, /* (167) multi_create_clause ::= multi_create_clause create_subtable_clause */ - -10, /* (168) create_subtable_clause ::= not_exists_opt full_table_name USING full_table_name specific_cols_opt TAGS NK_LP expression_list NK_RP table_options */ - -1, /* (169) multi_drop_clause ::= drop_table_clause */ - -3, /* (170) multi_drop_clause ::= multi_drop_clause NK_COMMA drop_table_clause */ - -2, /* (171) drop_table_clause ::= exists_opt full_table_name */ - 0, /* (172) specific_cols_opt ::= */ - -3, /* (173) specific_cols_opt ::= NK_LP col_name_list NK_RP */ - -1, /* (174) full_table_name ::= table_name */ - -3, /* (175) full_table_name ::= db_name NK_DOT table_name */ - -1, /* (176) column_def_list ::= column_def */ - -3, /* (177) column_def_list ::= column_def_list NK_COMMA column_def */ - -2, /* (178) column_def ::= column_name type_name */ - -1, /* (179) type_name ::= BOOL */ - -1, /* (180) type_name ::= TINYINT */ - -1, /* (181) type_name ::= SMALLINT */ - -1, /* (182) type_name ::= INT */ - -1, /* (183) type_name ::= INTEGER */ - -1, /* (184) type_name ::= BIGINT */ - -1, /* (185) type_name ::= FLOAT */ - -1, /* (186) type_name ::= DOUBLE */ - -4, /* (187) type_name ::= BINARY NK_LP NK_INTEGER NK_RP */ - -1, /* (188) type_name ::= TIMESTAMP */ - -4, /* (189) type_name ::= NCHAR NK_LP NK_INTEGER NK_RP */ - -2, /* (190) type_name ::= TINYINT UNSIGNED */ - -2, /* (191) type_name ::= SMALLINT UNSIGNED */ - -2, /* (192) type_name ::= INT UNSIGNED */ - -2, /* (193) type_name ::= BIGINT UNSIGNED */ - -1, /* (194) type_name ::= JSON */ - -4, /* (195) type_name ::= VARCHAR NK_LP NK_INTEGER NK_RP */ - -1, /* (196) type_name ::= MEDIUMBLOB */ - -1, /* (197) type_name ::= BLOB */ - -4, /* (198) type_name ::= VARBINARY NK_LP NK_INTEGER NK_RP */ - -1, /* (199) type_name ::= DECIMAL */ - -4, /* (200) type_name ::= DECIMAL NK_LP NK_INTEGER NK_RP */ - -6, /* (201) type_name ::= DECIMAL NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP */ - 0, /* (202) tags_def_opt ::= */ - -1, /* (203) tags_def_opt ::= tags_def */ - -4, /* (204) tags_def ::= TAGS NK_LP column_def_list NK_RP */ - 0, /* (205) table_options ::= */ - -3, /* (206) table_options ::= table_options COMMENT NK_STRING */ - -3, /* (207) table_options ::= table_options MAX_DELAY duration_list */ - -3, /* (208) table_options ::= table_options WATERMARK duration_list */ - -5, /* (209) table_options ::= table_options ROLLUP NK_LP rollup_func_list NK_RP */ - -3, /* (210) table_options ::= table_options TTL NK_INTEGER */ - -5, /* (211) table_options ::= table_options SMA NK_LP col_name_list NK_RP */ - -3, /* (212) table_options ::= table_options DELETE_MARK duration_list */ - -1, /* (213) alter_table_options ::= alter_table_option */ - -2, /* (214) alter_table_options ::= alter_table_options alter_table_option */ - -2, /* (215) alter_table_option ::= COMMENT NK_STRING */ - -2, /* (216) alter_table_option ::= TTL NK_INTEGER */ - -1, /* (217) duration_list ::= duration_literal */ - -3, /* (218) duration_list ::= duration_list NK_COMMA duration_literal */ - -1, /* (219) rollup_func_list ::= rollup_func_name */ - -3, /* (220) rollup_func_list ::= rollup_func_list NK_COMMA rollup_func_name */ - -1, /* (221) rollup_func_name ::= function_name */ - -1, /* (222) rollup_func_name ::= FIRST */ - -1, /* (223) rollup_func_name ::= LAST */ - -1, /* (224) col_name_list ::= col_name */ - -3, /* (225) col_name_list ::= col_name_list NK_COMMA col_name */ - -1, /* (226) col_name ::= column_name */ - -2, /* (227) cmd ::= SHOW DNODES */ - -2, /* (228) cmd ::= SHOW USERS */ - -3, /* (229) cmd ::= SHOW USER PRIVILEGES */ - -2, /* (230) cmd ::= SHOW DATABASES */ - -4, /* (231) cmd ::= SHOW db_name_cond_opt TABLES like_pattern_opt */ - -4, /* (232) cmd ::= SHOW db_name_cond_opt STABLES like_pattern_opt */ - -3, /* (233) cmd ::= SHOW db_name_cond_opt VGROUPS */ - -2, /* (234) cmd ::= SHOW MNODES */ - -2, /* (235) cmd ::= SHOW QNODES */ - -2, /* (236) cmd ::= SHOW FUNCTIONS */ - -5, /* (237) cmd ::= SHOW INDEXES FROM table_name_cond from_db_opt */ - -2, /* (238) cmd ::= SHOW STREAMS */ - -2, /* (239) cmd ::= SHOW ACCOUNTS */ - -2, /* (240) cmd ::= SHOW APPS */ - -2, /* (241) cmd ::= SHOW CONNECTIONS */ - -2, /* (242) cmd ::= SHOW LICENCES */ - -2, /* (243) cmd ::= SHOW GRANTS */ - -4, /* (244) cmd ::= SHOW CREATE DATABASE db_name */ - -4, /* (245) cmd ::= SHOW CREATE TABLE full_table_name */ - -4, /* (246) cmd ::= SHOW CREATE STABLE full_table_name */ - -2, /* (247) cmd ::= SHOW QUERIES */ - -2, /* (248) cmd ::= SHOW SCORES */ - -2, /* (249) cmd ::= SHOW TOPICS */ - -2, /* (250) cmd ::= SHOW VARIABLES */ - -3, /* (251) cmd ::= SHOW CLUSTER VARIABLES */ - -3, /* (252) cmd ::= SHOW LOCAL VARIABLES */ - -5, /* (253) cmd ::= SHOW DNODE NK_INTEGER VARIABLES like_pattern_opt */ - -2, /* (254) cmd ::= SHOW BNODES */ - -2, /* (255) cmd ::= SHOW SNODES */ - -2, /* (256) cmd ::= SHOW CLUSTER */ - -2, /* (257) cmd ::= SHOW TRANSACTIONS */ - -4, /* (258) cmd ::= SHOW TABLE DISTRIBUTED full_table_name */ - -2, /* (259) cmd ::= SHOW CONSUMERS */ - -2, /* (260) cmd ::= SHOW SUBSCRIPTIONS */ - -5, /* (261) cmd ::= SHOW TAGS FROM table_name_cond from_db_opt */ - -7, /* (262) cmd ::= SHOW TABLE TAGS tag_list_opt FROM table_name_cond from_db_opt */ - -3, /* (263) cmd ::= SHOW VNODES NK_INTEGER */ - -3, /* (264) cmd ::= SHOW VNODES NK_STRING */ - -3, /* (265) cmd ::= SHOW db_name_cond_opt ALIVE */ - -3, /* (266) cmd ::= SHOW CLUSTER ALIVE */ - 0, /* (267) db_name_cond_opt ::= */ - -2, /* (268) db_name_cond_opt ::= db_name NK_DOT */ - 0, /* (269) like_pattern_opt ::= */ - -2, /* (270) like_pattern_opt ::= LIKE NK_STRING */ - -1, /* (271) table_name_cond ::= table_name */ - 0, /* (272) from_db_opt ::= */ - -2, /* (273) from_db_opt ::= FROM db_name */ - 0, /* (274) tag_list_opt ::= */ - -1, /* (275) tag_list_opt ::= tag_item */ - -3, /* (276) tag_list_opt ::= tag_list_opt NK_COMMA tag_item */ - -1, /* (277) tag_item ::= TBNAME */ - -1, /* (278) tag_item ::= QTAGS */ - -1, /* (279) tag_item ::= column_name */ - -2, /* (280) tag_item ::= column_name column_alias */ - -3, /* (281) tag_item ::= column_name AS column_alias */ - -8, /* (282) cmd ::= CREATE SMA INDEX not_exists_opt full_index_name ON full_table_name index_options */ - -9, /* (283) cmd ::= CREATE INDEX not_exists_opt full_index_name ON full_table_name NK_LP col_name_list NK_RP */ - -4, /* (284) cmd ::= DROP INDEX exists_opt full_index_name */ - -1, /* (285) full_index_name ::= index_name */ - -3, /* (286) full_index_name ::= db_name NK_DOT index_name */ - -10, /* (287) index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_RP sliding_opt sma_stream_opt */ - -12, /* (288) index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_COMMA duration_literal NK_RP sliding_opt sma_stream_opt */ - -1, /* (289) func_list ::= func */ - -3, /* (290) func_list ::= func_list NK_COMMA func */ - -4, /* (291) func ::= sma_func_name NK_LP expression_list NK_RP */ - -1, /* (292) sma_func_name ::= function_name */ - -1, /* (293) sma_func_name ::= COUNT */ - -1, /* (294) sma_func_name ::= FIRST */ - -1, /* (295) sma_func_name ::= LAST */ - -1, /* (296) sma_func_name ::= LAST_ROW */ - 0, /* (297) sma_stream_opt ::= */ - -3, /* (298) sma_stream_opt ::= sma_stream_opt WATERMARK duration_literal */ - -3, /* (299) sma_stream_opt ::= sma_stream_opt MAX_DELAY duration_literal */ - -3, /* (300) sma_stream_opt ::= sma_stream_opt DELETE_MARK duration_literal */ - -6, /* (301) cmd ::= CREATE TOPIC not_exists_opt topic_name AS query_or_subquery */ - -7, /* (302) cmd ::= CREATE TOPIC not_exists_opt topic_name AS DATABASE db_name */ - -9, /* (303) cmd ::= CREATE TOPIC not_exists_opt topic_name WITH META AS DATABASE db_name */ - -7, /* (304) cmd ::= CREATE TOPIC not_exists_opt topic_name AS STABLE full_table_name */ - -9, /* (305) cmd ::= CREATE TOPIC not_exists_opt topic_name WITH META AS STABLE full_table_name */ - -4, /* (306) cmd ::= DROP TOPIC exists_opt topic_name */ - -7, /* (307) cmd ::= DROP CONSUMER GROUP exists_opt cgroup_name ON topic_name */ - -2, /* (308) cmd ::= DESC full_table_name */ - -2, /* (309) cmd ::= DESCRIBE full_table_name */ - -3, /* (310) cmd ::= RESET QUERY CACHE */ - -4, /* (311) cmd ::= EXPLAIN analyze_opt explain_options query_or_subquery */ - -4, /* (312) cmd ::= EXPLAIN analyze_opt explain_options insert_query */ - 0, /* (313) analyze_opt ::= */ - -1, /* (314) analyze_opt ::= ANALYZE */ - 0, /* (315) explain_options ::= */ - -3, /* (316) explain_options ::= explain_options VERBOSE NK_BOOL */ - -3, /* (317) explain_options ::= explain_options RATIO NK_FLOAT */ - -12, /* (318) cmd ::= CREATE or_replace_opt agg_func_opt FUNCTION not_exists_opt function_name AS NK_STRING OUTPUTTYPE type_name bufsize_opt language_opt */ - -4, /* (319) cmd ::= DROP FUNCTION exists_opt function_name */ - 0, /* (320) agg_func_opt ::= */ - -1, /* (321) agg_func_opt ::= AGGREGATE */ - 0, /* (322) bufsize_opt ::= */ - -2, /* (323) bufsize_opt ::= BUFSIZE NK_INTEGER */ - 0, /* (324) language_opt ::= */ - -2, /* (325) language_opt ::= LANGUAGE NK_STRING */ - 0, /* (326) or_replace_opt ::= */ - -2, /* (327) or_replace_opt ::= OR REPLACE */ - -12, /* (328) cmd ::= CREATE STREAM not_exists_opt stream_name stream_options INTO full_table_name col_list_opt tag_def_or_ref_opt subtable_opt AS query_or_subquery */ - -4, /* (329) cmd ::= DROP STREAM exists_opt stream_name */ - -4, /* (330) cmd ::= PAUSE STREAM exists_opt stream_name */ - -5, /* (331) cmd ::= RESUME STREAM exists_opt ignore_opt stream_name */ - 0, /* (332) col_list_opt ::= */ - -3, /* (333) col_list_opt ::= NK_LP col_name_list NK_RP */ - 0, /* (334) tag_def_or_ref_opt ::= */ - -1, /* (335) tag_def_or_ref_opt ::= tags_def */ - -4, /* (336) tag_def_or_ref_opt ::= TAGS NK_LP col_name_list NK_RP */ - 0, /* (337) stream_options ::= */ - -3, /* (338) stream_options ::= stream_options TRIGGER AT_ONCE */ - -3, /* (339) stream_options ::= stream_options TRIGGER WINDOW_CLOSE */ - -4, /* (340) stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal */ - -3, /* (341) stream_options ::= stream_options WATERMARK duration_literal */ - -4, /* (342) stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER */ - -3, /* (343) stream_options ::= stream_options FILL_HISTORY NK_INTEGER */ - -3, /* (344) stream_options ::= stream_options DELETE_MARK duration_literal */ - -4, /* (345) stream_options ::= stream_options IGNORE UPDATE NK_INTEGER */ - 0, /* (346) subtable_opt ::= */ - -4, /* (347) subtable_opt ::= SUBTABLE NK_LP expression NK_RP */ - 0, /* (348) ignore_opt ::= */ - -2, /* (349) ignore_opt ::= IGNORE UNTREATED */ - -3, /* (350) cmd ::= KILL CONNECTION NK_INTEGER */ - -3, /* (351) cmd ::= KILL QUERY NK_STRING */ - -3, /* (352) cmd ::= KILL TRANSACTION NK_INTEGER */ - -2, /* (353) cmd ::= BALANCE VGROUP */ - -3, /* (354) cmd ::= BALANCE VGROUP LEADER */ - -4, /* (355) cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER */ - -4, /* (356) cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list */ - -3, /* (357) cmd ::= SPLIT VGROUP NK_INTEGER */ - -2, /* (358) dnode_list ::= DNODE NK_INTEGER */ - -3, /* (359) dnode_list ::= dnode_list DNODE NK_INTEGER */ - -4, /* (360) cmd ::= DELETE FROM full_table_name where_clause_opt */ - -1, /* (361) cmd ::= query_or_subquery */ - -1, /* (362) cmd ::= insert_query */ - -7, /* (363) insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery */ - -4, /* (364) insert_query ::= INSERT INTO full_table_name query_or_subquery */ - -1, /* (365) literal ::= NK_INTEGER */ - -1, /* (366) literal ::= NK_FLOAT */ - -1, /* (367) literal ::= NK_STRING */ - -1, /* (368) literal ::= NK_BOOL */ - -2, /* (369) literal ::= TIMESTAMP NK_STRING */ - -1, /* (370) literal ::= duration_literal */ - -1, /* (371) literal ::= NULL */ - -1, /* (372) literal ::= NK_QUESTION */ - -1, /* (373) duration_literal ::= NK_VARIABLE */ - -1, /* (374) signed ::= NK_INTEGER */ - -2, /* (375) signed ::= NK_PLUS NK_INTEGER */ - -2, /* (376) signed ::= NK_MINUS NK_INTEGER */ - -1, /* (377) signed ::= NK_FLOAT */ - -2, /* (378) signed ::= NK_PLUS NK_FLOAT */ - -2, /* (379) signed ::= NK_MINUS NK_FLOAT */ - -1, /* (380) signed_literal ::= signed */ - -1, /* (381) signed_literal ::= NK_STRING */ - -1, /* (382) signed_literal ::= NK_BOOL */ - -2, /* (383) signed_literal ::= TIMESTAMP NK_STRING */ - -1, /* (384) signed_literal ::= duration_literal */ - -1, /* (385) signed_literal ::= NULL */ - -1, /* (386) signed_literal ::= literal_func */ - -1, /* (387) signed_literal ::= NK_QUESTION */ - -1, /* (388) literal_list ::= signed_literal */ - -3, /* (389) literal_list ::= literal_list NK_COMMA signed_literal */ - -1, /* (390) db_name ::= NK_ID */ - -1, /* (391) table_name ::= NK_ID */ - -1, /* (392) column_name ::= NK_ID */ - -1, /* (393) function_name ::= NK_ID */ - -1, /* (394) table_alias ::= NK_ID */ - -1, /* (395) column_alias ::= NK_ID */ - -1, /* (396) user_name ::= NK_ID */ - -1, /* (397) topic_name ::= NK_ID */ - -1, /* (398) stream_name ::= NK_ID */ - -1, /* (399) cgroup_name ::= NK_ID */ - -1, /* (400) index_name ::= NK_ID */ - -1, /* (401) expr_or_subquery ::= expression */ - -1, /* (402) expression ::= literal */ - -1, /* (403) expression ::= pseudo_column */ - -1, /* (404) expression ::= column_reference */ - -1, /* (405) expression ::= function_expression */ - -1, /* (406) expression ::= case_when_expression */ - -3, /* (407) expression ::= NK_LP expression NK_RP */ - -2, /* (408) expression ::= NK_PLUS expr_or_subquery */ - -2, /* (409) expression ::= NK_MINUS expr_or_subquery */ - -3, /* (410) expression ::= expr_or_subquery NK_PLUS expr_or_subquery */ - -3, /* (411) expression ::= expr_or_subquery NK_MINUS expr_or_subquery */ - -3, /* (412) expression ::= expr_or_subquery NK_STAR expr_or_subquery */ - -3, /* (413) expression ::= expr_or_subquery NK_SLASH expr_or_subquery */ - -3, /* (414) expression ::= expr_or_subquery NK_REM expr_or_subquery */ - -3, /* (415) expression ::= column_reference NK_ARROW NK_STRING */ - -3, /* (416) expression ::= expr_or_subquery NK_BITAND expr_or_subquery */ - -3, /* (417) expression ::= expr_or_subquery NK_BITOR expr_or_subquery */ - -1, /* (418) expression_list ::= expr_or_subquery */ - -3, /* (419) expression_list ::= expression_list NK_COMMA expr_or_subquery */ - -1, /* (420) column_reference ::= column_name */ - -3, /* (421) column_reference ::= table_name NK_DOT column_name */ - -1, /* (422) pseudo_column ::= ROWTS */ - -1, /* (423) pseudo_column ::= TBNAME */ - -3, /* (424) pseudo_column ::= table_name NK_DOT TBNAME */ - -1, /* (425) pseudo_column ::= QSTART */ - -1, /* (426) pseudo_column ::= QEND */ - -1, /* (427) pseudo_column ::= QDURATION */ - -1, /* (428) pseudo_column ::= WSTART */ - -1, /* (429) pseudo_column ::= WEND */ - -1, /* (430) pseudo_column ::= WDURATION */ - -1, /* (431) pseudo_column ::= IROWTS */ - -1, /* (432) pseudo_column ::= ISFILLED */ - -1, /* (433) pseudo_column ::= QTAGS */ - -4, /* (434) function_expression ::= function_name NK_LP expression_list NK_RP */ - -4, /* (435) function_expression ::= star_func NK_LP star_func_para_list NK_RP */ - -6, /* (436) function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP */ - -1, /* (437) function_expression ::= literal_func */ - -3, /* (438) literal_func ::= noarg_func NK_LP NK_RP */ - -1, /* (439) literal_func ::= NOW */ - -1, /* (440) noarg_func ::= NOW */ - -1, /* (441) noarg_func ::= TODAY */ - -1, /* (442) noarg_func ::= TIMEZONE */ - -1, /* (443) noarg_func ::= DATABASE */ - -1, /* (444) noarg_func ::= CLIENT_VERSION */ - -1, /* (445) noarg_func ::= SERVER_VERSION */ - -1, /* (446) noarg_func ::= SERVER_STATUS */ - -1, /* (447) noarg_func ::= CURRENT_USER */ - -1, /* (448) noarg_func ::= USER */ - -1, /* (449) star_func ::= COUNT */ - -1, /* (450) star_func ::= FIRST */ - -1, /* (451) star_func ::= LAST */ - -1, /* (452) star_func ::= LAST_ROW */ - -1, /* (453) star_func_para_list ::= NK_STAR */ - -1, /* (454) star_func_para_list ::= other_para_list */ - -1, /* (455) other_para_list ::= star_func_para */ - -3, /* (456) other_para_list ::= other_para_list NK_COMMA star_func_para */ - -1, /* (457) star_func_para ::= expr_or_subquery */ - -3, /* (458) star_func_para ::= table_name NK_DOT NK_STAR */ - -4, /* (459) case_when_expression ::= CASE when_then_list case_when_else_opt END */ - -5, /* (460) case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END */ - -1, /* (461) when_then_list ::= when_then_expr */ - -2, /* (462) when_then_list ::= when_then_list when_then_expr */ - -4, /* (463) when_then_expr ::= WHEN common_expression THEN common_expression */ - 0, /* (464) case_when_else_opt ::= */ - -2, /* (465) case_when_else_opt ::= ELSE common_expression */ - -3, /* (466) predicate ::= expr_or_subquery compare_op expr_or_subquery */ - -5, /* (467) predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery */ - -6, /* (468) predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery */ - -3, /* (469) predicate ::= expr_or_subquery IS NULL */ - -4, /* (470) predicate ::= expr_or_subquery IS NOT NULL */ - -3, /* (471) predicate ::= expr_or_subquery in_op in_predicate_value */ - -1, /* (472) compare_op ::= NK_LT */ - -1, /* (473) compare_op ::= NK_GT */ - -1, /* (474) compare_op ::= NK_LE */ - -1, /* (475) compare_op ::= NK_GE */ - -1, /* (476) compare_op ::= NK_NE */ - -1, /* (477) compare_op ::= NK_EQ */ - -1, /* (478) compare_op ::= LIKE */ - -2, /* (479) compare_op ::= NOT LIKE */ - -1, /* (480) compare_op ::= MATCH */ - -1, /* (481) compare_op ::= NMATCH */ - -1, /* (482) compare_op ::= CONTAINS */ - -1, /* (483) in_op ::= IN */ - -2, /* (484) in_op ::= NOT IN */ - -3, /* (485) in_predicate_value ::= NK_LP literal_list NK_RP */ - -1, /* (486) boolean_value_expression ::= boolean_primary */ - -2, /* (487) boolean_value_expression ::= NOT boolean_primary */ - -3, /* (488) boolean_value_expression ::= boolean_value_expression OR boolean_value_expression */ - -3, /* (489) boolean_value_expression ::= boolean_value_expression AND boolean_value_expression */ - -1, /* (490) boolean_primary ::= predicate */ - -3, /* (491) boolean_primary ::= NK_LP boolean_value_expression NK_RP */ - -1, /* (492) common_expression ::= expr_or_subquery */ - -1, /* (493) common_expression ::= boolean_value_expression */ - 0, /* (494) from_clause_opt ::= */ - -2, /* (495) from_clause_opt ::= FROM table_reference_list */ - -1, /* (496) table_reference_list ::= table_reference */ - -3, /* (497) table_reference_list ::= table_reference_list NK_COMMA table_reference */ - -1, /* (498) table_reference ::= table_primary */ - -1, /* (499) table_reference ::= joined_table */ - -2, /* (500) table_primary ::= table_name alias_opt */ - -4, /* (501) table_primary ::= db_name NK_DOT table_name alias_opt */ - -2, /* (502) table_primary ::= subquery alias_opt */ - -1, /* (503) table_primary ::= parenthesized_joined_table */ - 0, /* (504) alias_opt ::= */ - -1, /* (505) alias_opt ::= table_alias */ - -2, /* (506) alias_opt ::= AS table_alias */ - -3, /* (507) parenthesized_joined_table ::= NK_LP joined_table NK_RP */ - -3, /* (508) parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP */ - -6, /* (509) joined_table ::= table_reference join_type JOIN table_reference ON search_condition */ - 0, /* (510) join_type ::= */ - -1, /* (511) join_type ::= INNER */ - -12, /* (512) query_specification ::= SELECT set_quantifier_opt select_list from_clause_opt where_clause_opt partition_by_clause_opt range_opt every_opt fill_opt twindow_clause_opt group_by_clause_opt having_clause_opt */ - 0, /* (513) set_quantifier_opt ::= */ - -1, /* (514) set_quantifier_opt ::= DISTINCT */ - -1, /* (515) set_quantifier_opt ::= ALL */ - -1, /* (516) select_list ::= select_item */ - -3, /* (517) select_list ::= select_list NK_COMMA select_item */ - -1, /* (518) select_item ::= NK_STAR */ - -1, /* (519) select_item ::= common_expression */ - -2, /* (520) select_item ::= common_expression column_alias */ - -3, /* (521) select_item ::= common_expression AS column_alias */ - -3, /* (522) select_item ::= table_name NK_DOT NK_STAR */ - 0, /* (523) where_clause_opt ::= */ - -2, /* (524) where_clause_opt ::= WHERE search_condition */ - 0, /* (525) partition_by_clause_opt ::= */ - -3, /* (526) partition_by_clause_opt ::= PARTITION BY partition_list */ - -1, /* (527) partition_list ::= partition_item */ - -3, /* (528) partition_list ::= partition_list NK_COMMA partition_item */ - -1, /* (529) partition_item ::= expr_or_subquery */ - -2, /* (530) partition_item ::= expr_or_subquery column_alias */ - -3, /* (531) partition_item ::= expr_or_subquery AS column_alias */ - 0, /* (532) twindow_clause_opt ::= */ - -6, /* (533) twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA duration_literal NK_RP */ - -4, /* (534) twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP */ - -6, /* (535) twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_RP sliding_opt fill_opt */ - -8, /* (536) twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_COMMA duration_literal NK_RP sliding_opt fill_opt */ - -7, /* (537) twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition */ - 0, /* (538) sliding_opt ::= */ - -4, /* (539) sliding_opt ::= SLIDING NK_LP duration_literal NK_RP */ - 0, /* (540) fill_opt ::= */ - -4, /* (541) fill_opt ::= FILL NK_LP fill_mode NK_RP */ - -6, /* (542) fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP */ - -6, /* (543) fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP */ - -1, /* (544) fill_mode ::= NONE */ - -1, /* (545) fill_mode ::= PREV */ - -1, /* (546) fill_mode ::= NULL */ - -1, /* (547) fill_mode ::= NULL_F */ - -1, /* (548) fill_mode ::= LINEAR */ - -1, /* (549) fill_mode ::= NEXT */ - 0, /* (550) group_by_clause_opt ::= */ - -3, /* (551) group_by_clause_opt ::= GROUP BY group_by_list */ - -1, /* (552) group_by_list ::= expr_or_subquery */ - -3, /* (553) group_by_list ::= group_by_list NK_COMMA expr_or_subquery */ - 0, /* (554) having_clause_opt ::= */ - -2, /* (555) having_clause_opt ::= HAVING search_condition */ - 0, /* (556) range_opt ::= */ - -6, /* (557) range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP */ - 0, /* (558) every_opt ::= */ - -4, /* (559) every_opt ::= EVERY NK_LP duration_literal NK_RP */ - -4, /* (560) query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt */ - -1, /* (561) query_simple ::= query_specification */ - -1, /* (562) query_simple ::= union_query_expression */ - -4, /* (563) union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery */ - -3, /* (564) union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery */ - -1, /* (565) query_simple_or_subquery ::= query_simple */ - -1, /* (566) query_simple_or_subquery ::= subquery */ - -1, /* (567) query_or_subquery ::= query_expression */ - -1, /* (568) query_or_subquery ::= subquery */ - 0, /* (569) order_by_clause_opt ::= */ - -3, /* (570) order_by_clause_opt ::= ORDER BY sort_specification_list */ - 0, /* (571) slimit_clause_opt ::= */ - -2, /* (572) slimit_clause_opt ::= SLIMIT NK_INTEGER */ - -4, /* (573) slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER */ - -4, /* (574) slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER */ - 0, /* (575) limit_clause_opt ::= */ - -2, /* (576) limit_clause_opt ::= LIMIT NK_INTEGER */ - -4, /* (577) limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER */ - -4, /* (578) limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER */ - -3, /* (579) subquery ::= NK_LP query_expression NK_RP */ - -3, /* (580) subquery ::= NK_LP subquery NK_RP */ - -1, /* (581) search_condition ::= common_expression */ - -1, /* (582) sort_specification_list ::= sort_specification */ - -3, /* (583) sort_specification_list ::= sort_specification_list NK_COMMA sort_specification */ - -3, /* (584) sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt */ - 0, /* (585) ordering_specification_opt ::= */ - -1, /* (586) ordering_specification_opt ::= ASC */ - -1, /* (587) ordering_specification_opt ::= DESC */ - 0, /* (588) null_ordering_opt ::= */ - -2, /* (589) null_ordering_opt ::= NULLS FIRST */ - -2, /* (590) null_ordering_opt ::= NULLS LAST */ + -4, /* (50) cmd ::= DROP DNODE NK_INTEGER unsafe_opt */ + -4, /* (51) cmd ::= DROP DNODE dnode_endpoint unsafe_opt */ + -4, /* (52) cmd ::= ALTER DNODE NK_INTEGER NK_STRING */ + -5, /* (53) cmd ::= ALTER DNODE NK_INTEGER NK_STRING NK_STRING */ + -4, /* (54) cmd ::= ALTER ALL DNODES NK_STRING */ + -5, /* (55) cmd ::= ALTER ALL DNODES NK_STRING NK_STRING */ + -3, /* (56) cmd ::= RESTORE DNODE NK_INTEGER */ + -1, /* (57) dnode_endpoint ::= NK_STRING */ + -1, /* (58) dnode_endpoint ::= NK_ID */ + -1, /* (59) dnode_endpoint ::= NK_IPTOKEN */ + 0, /* (60) force_opt ::= */ + -1, /* (61) force_opt ::= FORCE */ + -1, /* (62) unsafe_opt ::= UNSAFE */ + -3, /* (63) cmd ::= ALTER LOCAL NK_STRING */ + -4, /* (64) cmd ::= ALTER LOCAL NK_STRING NK_STRING */ + -5, /* (65) cmd ::= CREATE QNODE ON DNODE NK_INTEGER */ + -5, /* (66) cmd ::= DROP QNODE ON DNODE NK_INTEGER */ + -5, /* (67) cmd ::= RESTORE QNODE ON DNODE NK_INTEGER */ + -5, /* (68) cmd ::= CREATE BNODE ON DNODE NK_INTEGER */ + -5, /* (69) cmd ::= DROP BNODE ON DNODE NK_INTEGER */ + -5, /* (70) cmd ::= CREATE SNODE ON DNODE NK_INTEGER */ + -5, /* (71) cmd ::= DROP SNODE ON DNODE NK_INTEGER */ + -5, /* (72) cmd ::= CREATE MNODE ON DNODE NK_INTEGER */ + -5, /* (73) cmd ::= DROP MNODE ON DNODE NK_INTEGER */ + -5, /* (74) cmd ::= RESTORE MNODE ON DNODE NK_INTEGER */ + -5, /* (75) cmd ::= RESTORE VNODE ON DNODE NK_INTEGER */ + -5, /* (76) cmd ::= CREATE DATABASE not_exists_opt db_name db_options */ + -4, /* (77) cmd ::= DROP DATABASE exists_opt db_name */ + -2, /* (78) cmd ::= USE db_name */ + -4, /* (79) cmd ::= ALTER DATABASE db_name alter_db_options */ + -3, /* (80) cmd ::= FLUSH DATABASE db_name */ + -4, /* (81) cmd ::= TRIM DATABASE db_name speed_opt */ + -5, /* (82) cmd ::= COMPACT DATABASE db_name start_opt end_opt */ + -3, /* (83) not_exists_opt ::= IF NOT EXISTS */ + 0, /* (84) not_exists_opt ::= */ + -2, /* (85) exists_opt ::= IF EXISTS */ + 0, /* (86) exists_opt ::= */ + 0, /* (87) db_options ::= */ + -3, /* (88) db_options ::= db_options BUFFER NK_INTEGER */ + -3, /* (89) db_options ::= db_options CACHEMODEL NK_STRING */ + -3, /* (90) db_options ::= db_options CACHESIZE NK_INTEGER */ + -3, /* (91) db_options ::= db_options COMP NK_INTEGER */ + -3, /* (92) db_options ::= db_options DURATION NK_INTEGER */ + -3, /* (93) db_options ::= db_options DURATION NK_VARIABLE */ + -3, /* (94) db_options ::= db_options MAXROWS NK_INTEGER */ + -3, /* (95) db_options ::= db_options MINROWS NK_INTEGER */ + -3, /* (96) db_options ::= db_options KEEP integer_list */ + -3, /* (97) db_options ::= db_options KEEP variable_list */ + -3, /* (98) db_options ::= db_options PAGES NK_INTEGER */ + -3, /* (99) db_options ::= db_options PAGESIZE NK_INTEGER */ + -3, /* (100) db_options ::= db_options TSDB_PAGESIZE NK_INTEGER */ + -3, /* (101) db_options ::= db_options PRECISION NK_STRING */ + -3, /* (102) db_options ::= db_options REPLICA NK_INTEGER */ + -3, /* (103) db_options ::= db_options VGROUPS NK_INTEGER */ + -3, /* (104) db_options ::= db_options SINGLE_STABLE NK_INTEGER */ + -3, /* (105) db_options ::= db_options RETENTIONS retention_list */ + -3, /* (106) db_options ::= db_options SCHEMALESS NK_INTEGER */ + -3, /* (107) db_options ::= db_options WAL_LEVEL NK_INTEGER */ + -3, /* (108) db_options ::= db_options WAL_FSYNC_PERIOD NK_INTEGER */ + -3, /* (109) db_options ::= db_options WAL_RETENTION_PERIOD NK_INTEGER */ + -4, /* (110) db_options ::= db_options WAL_RETENTION_PERIOD NK_MINUS NK_INTEGER */ + -3, /* (111) db_options ::= db_options WAL_RETENTION_SIZE NK_INTEGER */ + -4, /* (112) db_options ::= db_options WAL_RETENTION_SIZE NK_MINUS NK_INTEGER */ + -3, /* (113) db_options ::= db_options WAL_ROLL_PERIOD NK_INTEGER */ + -3, /* (114) db_options ::= db_options WAL_SEGMENT_SIZE NK_INTEGER */ + -3, /* (115) db_options ::= db_options STT_TRIGGER NK_INTEGER */ + -3, /* (116) db_options ::= db_options TABLE_PREFIX signed */ + -3, /* (117) db_options ::= db_options TABLE_SUFFIX signed */ + -1, /* (118) alter_db_options ::= alter_db_option */ + -2, /* (119) alter_db_options ::= alter_db_options alter_db_option */ + -2, /* (120) alter_db_option ::= BUFFER NK_INTEGER */ + -2, /* (121) alter_db_option ::= CACHEMODEL NK_STRING */ + -2, /* (122) alter_db_option ::= CACHESIZE NK_INTEGER */ + -2, /* (123) alter_db_option ::= WAL_FSYNC_PERIOD NK_INTEGER */ + -2, /* (124) alter_db_option ::= KEEP integer_list */ + -2, /* (125) alter_db_option ::= KEEP variable_list */ + -2, /* (126) alter_db_option ::= PAGES NK_INTEGER */ + -2, /* (127) alter_db_option ::= REPLICA NK_INTEGER */ + -2, /* (128) alter_db_option ::= WAL_LEVEL NK_INTEGER */ + -2, /* (129) alter_db_option ::= STT_TRIGGER NK_INTEGER */ + -2, /* (130) alter_db_option ::= MINROWS NK_INTEGER */ + -2, /* (131) alter_db_option ::= WAL_RETENTION_PERIOD NK_INTEGER */ + -3, /* (132) alter_db_option ::= WAL_RETENTION_PERIOD NK_MINUS NK_INTEGER */ + -2, /* (133) alter_db_option ::= WAL_RETENTION_SIZE NK_INTEGER */ + -3, /* (134) alter_db_option ::= WAL_RETENTION_SIZE NK_MINUS NK_INTEGER */ + -1, /* (135) integer_list ::= NK_INTEGER */ + -3, /* (136) integer_list ::= integer_list NK_COMMA NK_INTEGER */ + -1, /* (137) variable_list ::= NK_VARIABLE */ + -3, /* (138) variable_list ::= variable_list NK_COMMA NK_VARIABLE */ + -1, /* (139) retention_list ::= retention */ + -3, /* (140) retention_list ::= retention_list NK_COMMA retention */ + -3, /* (141) retention ::= NK_VARIABLE NK_COLON NK_VARIABLE */ + 0, /* (142) speed_opt ::= */ + -2, /* (143) speed_opt ::= MAX_SPEED NK_INTEGER */ + 0, /* (144) start_opt ::= */ + -3, /* (145) start_opt ::= START WITH NK_INTEGER */ + -3, /* (146) start_opt ::= START WITH NK_STRING */ + -4, /* (147) start_opt ::= START WITH TIMESTAMP NK_STRING */ + 0, /* (148) end_opt ::= */ + -3, /* (149) end_opt ::= END WITH NK_INTEGER */ + -3, /* (150) end_opt ::= END WITH NK_STRING */ + -4, /* (151) end_opt ::= END WITH TIMESTAMP NK_STRING */ + -9, /* (152) cmd ::= CREATE TABLE not_exists_opt full_table_name NK_LP column_def_list NK_RP tags_def_opt table_options */ + -3, /* (153) cmd ::= CREATE TABLE multi_create_clause */ + -9, /* (154) cmd ::= CREATE STABLE not_exists_opt full_table_name NK_LP column_def_list NK_RP tags_def table_options */ + -3, /* (155) cmd ::= DROP TABLE multi_drop_clause */ + -4, /* (156) cmd ::= DROP STABLE exists_opt full_table_name */ + -3, /* (157) cmd ::= ALTER TABLE alter_table_clause */ + -3, /* (158) cmd ::= ALTER STABLE alter_table_clause */ + -2, /* (159) alter_table_clause ::= full_table_name alter_table_options */ + -5, /* (160) alter_table_clause ::= full_table_name ADD COLUMN column_name type_name */ + -4, /* (161) alter_table_clause ::= full_table_name DROP COLUMN column_name */ + -5, /* (162) alter_table_clause ::= full_table_name MODIFY COLUMN column_name type_name */ + -5, /* (163) alter_table_clause ::= full_table_name RENAME COLUMN column_name column_name */ + -5, /* (164) alter_table_clause ::= full_table_name ADD TAG column_name type_name */ + -4, /* (165) alter_table_clause ::= full_table_name DROP TAG column_name */ + -5, /* (166) alter_table_clause ::= full_table_name MODIFY TAG column_name type_name */ + -5, /* (167) alter_table_clause ::= full_table_name RENAME TAG column_name column_name */ + -6, /* (168) alter_table_clause ::= full_table_name SET TAG column_name NK_EQ signed_literal */ + -1, /* (169) multi_create_clause ::= create_subtable_clause */ + -2, /* (170) multi_create_clause ::= multi_create_clause create_subtable_clause */ + -10, /* (171) create_subtable_clause ::= not_exists_opt full_table_name USING full_table_name specific_cols_opt TAGS NK_LP expression_list NK_RP table_options */ + -1, /* (172) multi_drop_clause ::= drop_table_clause */ + -3, /* (173) multi_drop_clause ::= multi_drop_clause NK_COMMA drop_table_clause */ + -2, /* (174) drop_table_clause ::= exists_opt full_table_name */ + 0, /* (175) specific_cols_opt ::= */ + -3, /* (176) specific_cols_opt ::= NK_LP col_name_list NK_RP */ + -1, /* (177) full_table_name ::= table_name */ + -3, /* (178) full_table_name ::= db_name NK_DOT table_name */ + -1, /* (179) column_def_list ::= column_def */ + -3, /* (180) column_def_list ::= column_def_list NK_COMMA column_def */ + -2, /* (181) column_def ::= column_name type_name */ + -1, /* (182) type_name ::= BOOL */ + -1, /* (183) type_name ::= TINYINT */ + -1, /* (184) type_name ::= SMALLINT */ + -1, /* (185) type_name ::= INT */ + -1, /* (186) type_name ::= INTEGER */ + -1, /* (187) type_name ::= BIGINT */ + -1, /* (188) type_name ::= FLOAT */ + -1, /* (189) type_name ::= DOUBLE */ + -4, /* (190) type_name ::= BINARY NK_LP NK_INTEGER NK_RP */ + -1, /* (191) type_name ::= TIMESTAMP */ + -4, /* (192) type_name ::= NCHAR NK_LP NK_INTEGER NK_RP */ + -2, /* (193) type_name ::= TINYINT UNSIGNED */ + -2, /* (194) type_name ::= SMALLINT UNSIGNED */ + -2, /* (195) type_name ::= INT UNSIGNED */ + -2, /* (196) type_name ::= BIGINT UNSIGNED */ + -1, /* (197) type_name ::= JSON */ + -4, /* (198) type_name ::= VARCHAR NK_LP NK_INTEGER NK_RP */ + -1, /* (199) type_name ::= MEDIUMBLOB */ + -1, /* (200) type_name ::= BLOB */ + -4, /* (201) type_name ::= VARBINARY NK_LP NK_INTEGER NK_RP */ + -1, /* (202) type_name ::= DECIMAL */ + -4, /* (203) type_name ::= DECIMAL NK_LP NK_INTEGER NK_RP */ + -6, /* (204) type_name ::= DECIMAL NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP */ + 0, /* (205) tags_def_opt ::= */ + -1, /* (206) tags_def_opt ::= tags_def */ + -4, /* (207) tags_def ::= TAGS NK_LP column_def_list NK_RP */ + 0, /* (208) table_options ::= */ + -3, /* (209) table_options ::= table_options COMMENT NK_STRING */ + -3, /* (210) table_options ::= table_options MAX_DELAY duration_list */ + -3, /* (211) table_options ::= table_options WATERMARK duration_list */ + -5, /* (212) table_options ::= table_options ROLLUP NK_LP rollup_func_list NK_RP */ + -3, /* (213) table_options ::= table_options TTL NK_INTEGER */ + -5, /* (214) table_options ::= table_options SMA NK_LP col_name_list NK_RP */ + -3, /* (215) table_options ::= table_options DELETE_MARK duration_list */ + -1, /* (216) alter_table_options ::= alter_table_option */ + -2, /* (217) alter_table_options ::= alter_table_options alter_table_option */ + -2, /* (218) alter_table_option ::= COMMENT NK_STRING */ + -2, /* (219) alter_table_option ::= TTL NK_INTEGER */ + -1, /* (220) duration_list ::= duration_literal */ + -3, /* (221) duration_list ::= duration_list NK_COMMA duration_literal */ + -1, /* (222) rollup_func_list ::= rollup_func_name */ + -3, /* (223) rollup_func_list ::= rollup_func_list NK_COMMA rollup_func_name */ + -1, /* (224) rollup_func_name ::= function_name */ + -1, /* (225) rollup_func_name ::= FIRST */ + -1, /* (226) rollup_func_name ::= LAST */ + -1, /* (227) col_name_list ::= col_name */ + -3, /* (228) col_name_list ::= col_name_list NK_COMMA col_name */ + -1, /* (229) col_name ::= column_name */ + -2, /* (230) cmd ::= SHOW DNODES */ + -2, /* (231) cmd ::= SHOW USERS */ + -3, /* (232) cmd ::= SHOW USER PRIVILEGES */ + -2, /* (233) cmd ::= SHOW DATABASES */ + -4, /* (234) cmd ::= SHOW db_name_cond_opt TABLES like_pattern_opt */ + -4, /* (235) cmd ::= SHOW db_name_cond_opt STABLES like_pattern_opt */ + -3, /* (236) cmd ::= SHOW db_name_cond_opt VGROUPS */ + -2, /* (237) cmd ::= SHOW MNODES */ + -2, /* (238) cmd ::= SHOW QNODES */ + -2, /* (239) cmd ::= SHOW FUNCTIONS */ + -5, /* (240) cmd ::= SHOW INDEXES FROM table_name_cond from_db_opt */ + -2, /* (241) cmd ::= SHOW STREAMS */ + -2, /* (242) cmd ::= SHOW ACCOUNTS */ + -2, /* (243) cmd ::= SHOW APPS */ + -2, /* (244) cmd ::= SHOW CONNECTIONS */ + -2, /* (245) cmd ::= SHOW LICENCES */ + -2, /* (246) cmd ::= SHOW GRANTS */ + -4, /* (247) cmd ::= SHOW CREATE DATABASE db_name */ + -4, /* (248) cmd ::= SHOW CREATE TABLE full_table_name */ + -4, /* (249) cmd ::= SHOW CREATE STABLE full_table_name */ + -2, /* (250) cmd ::= SHOW QUERIES */ + -2, /* (251) cmd ::= SHOW SCORES */ + -2, /* (252) cmd ::= SHOW TOPICS */ + -2, /* (253) cmd ::= SHOW VARIABLES */ + -3, /* (254) cmd ::= SHOW CLUSTER VARIABLES */ + -3, /* (255) cmd ::= SHOW LOCAL VARIABLES */ + -5, /* (256) cmd ::= SHOW DNODE NK_INTEGER VARIABLES like_pattern_opt */ + -2, /* (257) cmd ::= SHOW BNODES */ + -2, /* (258) cmd ::= SHOW SNODES */ + -2, /* (259) cmd ::= SHOW CLUSTER */ + -2, /* (260) cmd ::= SHOW TRANSACTIONS */ + -4, /* (261) cmd ::= SHOW TABLE DISTRIBUTED full_table_name */ + -2, /* (262) cmd ::= SHOW CONSUMERS */ + -2, /* (263) cmd ::= SHOW SUBSCRIPTIONS */ + -5, /* (264) cmd ::= SHOW TAGS FROM table_name_cond from_db_opt */ + -7, /* (265) cmd ::= SHOW TABLE TAGS tag_list_opt FROM table_name_cond from_db_opt */ + -3, /* (266) cmd ::= SHOW VNODES NK_INTEGER */ + -3, /* (267) cmd ::= SHOW VNODES NK_STRING */ + -3, /* (268) cmd ::= SHOW db_name_cond_opt ALIVE */ + -3, /* (269) cmd ::= SHOW CLUSTER ALIVE */ + 0, /* (270) db_name_cond_opt ::= */ + -2, /* (271) db_name_cond_opt ::= db_name NK_DOT */ + 0, /* (272) like_pattern_opt ::= */ + -2, /* (273) like_pattern_opt ::= LIKE NK_STRING */ + -1, /* (274) table_name_cond ::= table_name */ + 0, /* (275) from_db_opt ::= */ + -2, /* (276) from_db_opt ::= FROM db_name */ + 0, /* (277) tag_list_opt ::= */ + -1, /* (278) tag_list_opt ::= tag_item */ + -3, /* (279) tag_list_opt ::= tag_list_opt NK_COMMA tag_item */ + -1, /* (280) tag_item ::= TBNAME */ + -1, /* (281) tag_item ::= QTAGS */ + -1, /* (282) tag_item ::= column_name */ + -2, /* (283) tag_item ::= column_name column_alias */ + -3, /* (284) tag_item ::= column_name AS column_alias */ + -8, /* (285) cmd ::= CREATE SMA INDEX not_exists_opt full_index_name ON full_table_name index_options */ + -9, /* (286) cmd ::= CREATE INDEX not_exists_opt full_index_name ON full_table_name NK_LP col_name_list NK_RP */ + -4, /* (287) cmd ::= DROP INDEX exists_opt full_index_name */ + -1, /* (288) full_index_name ::= index_name */ + -3, /* (289) full_index_name ::= db_name NK_DOT index_name */ + -10, /* (290) index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_RP sliding_opt sma_stream_opt */ + -12, /* (291) index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_COMMA duration_literal NK_RP sliding_opt sma_stream_opt */ + -1, /* (292) func_list ::= func */ + -3, /* (293) func_list ::= func_list NK_COMMA func */ + -4, /* (294) func ::= sma_func_name NK_LP expression_list NK_RP */ + -1, /* (295) sma_func_name ::= function_name */ + -1, /* (296) sma_func_name ::= COUNT */ + -1, /* (297) sma_func_name ::= FIRST */ + -1, /* (298) sma_func_name ::= LAST */ + -1, /* (299) sma_func_name ::= LAST_ROW */ + 0, /* (300) sma_stream_opt ::= */ + -3, /* (301) sma_stream_opt ::= sma_stream_opt WATERMARK duration_literal */ + -3, /* (302) sma_stream_opt ::= sma_stream_opt MAX_DELAY duration_literal */ + -3, /* (303) sma_stream_opt ::= sma_stream_opt DELETE_MARK duration_literal */ + -6, /* (304) cmd ::= CREATE TOPIC not_exists_opt topic_name AS query_or_subquery */ + -7, /* (305) cmd ::= CREATE TOPIC not_exists_opt topic_name AS DATABASE db_name */ + -9, /* (306) cmd ::= CREATE TOPIC not_exists_opt topic_name WITH META AS DATABASE db_name */ + -7, /* (307) cmd ::= CREATE TOPIC not_exists_opt topic_name AS STABLE full_table_name */ + -9, /* (308) cmd ::= CREATE TOPIC not_exists_opt topic_name WITH META AS STABLE full_table_name */ + -4, /* (309) cmd ::= DROP TOPIC exists_opt topic_name */ + -7, /* (310) cmd ::= DROP CONSUMER GROUP exists_opt cgroup_name ON topic_name */ + -2, /* (311) cmd ::= DESC full_table_name */ + -2, /* (312) cmd ::= DESCRIBE full_table_name */ + -3, /* (313) cmd ::= RESET QUERY CACHE */ + -4, /* (314) cmd ::= EXPLAIN analyze_opt explain_options query_or_subquery */ + -4, /* (315) cmd ::= EXPLAIN analyze_opt explain_options insert_query */ + 0, /* (316) analyze_opt ::= */ + -1, /* (317) analyze_opt ::= ANALYZE */ + 0, /* (318) explain_options ::= */ + -3, /* (319) explain_options ::= explain_options VERBOSE NK_BOOL */ + -3, /* (320) explain_options ::= explain_options RATIO NK_FLOAT */ + -12, /* (321) cmd ::= CREATE or_replace_opt agg_func_opt FUNCTION not_exists_opt function_name AS NK_STRING OUTPUTTYPE type_name bufsize_opt language_opt */ + -4, /* (322) cmd ::= DROP FUNCTION exists_opt function_name */ + 0, /* (323) agg_func_opt ::= */ + -1, /* (324) agg_func_opt ::= AGGREGATE */ + 0, /* (325) bufsize_opt ::= */ + -2, /* (326) bufsize_opt ::= BUFSIZE NK_INTEGER */ + 0, /* (327) language_opt ::= */ + -2, /* (328) language_opt ::= LANGUAGE NK_STRING */ + 0, /* (329) or_replace_opt ::= */ + -2, /* (330) or_replace_opt ::= OR REPLACE */ + -12, /* (331) cmd ::= CREATE STREAM not_exists_opt stream_name stream_options INTO full_table_name col_list_opt tag_def_or_ref_opt subtable_opt AS query_or_subquery */ + -4, /* (332) cmd ::= DROP STREAM exists_opt stream_name */ + -4, /* (333) cmd ::= PAUSE STREAM exists_opt stream_name */ + -5, /* (334) cmd ::= RESUME STREAM exists_opt ignore_opt stream_name */ + 0, /* (335) col_list_opt ::= */ + -3, /* (336) col_list_opt ::= NK_LP col_name_list NK_RP */ + 0, /* (337) tag_def_or_ref_opt ::= */ + -1, /* (338) tag_def_or_ref_opt ::= tags_def */ + -4, /* (339) tag_def_or_ref_opt ::= TAGS NK_LP col_name_list NK_RP */ + 0, /* (340) stream_options ::= */ + -3, /* (341) stream_options ::= stream_options TRIGGER AT_ONCE */ + -3, /* (342) stream_options ::= stream_options TRIGGER WINDOW_CLOSE */ + -4, /* (343) stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal */ + -3, /* (344) stream_options ::= stream_options WATERMARK duration_literal */ + -4, /* (345) stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER */ + -3, /* (346) stream_options ::= stream_options FILL_HISTORY NK_INTEGER */ + -3, /* (347) stream_options ::= stream_options DELETE_MARK duration_literal */ + -4, /* (348) stream_options ::= stream_options IGNORE UPDATE NK_INTEGER */ + 0, /* (349) subtable_opt ::= */ + -4, /* (350) subtable_opt ::= SUBTABLE NK_LP expression NK_RP */ + 0, /* (351) ignore_opt ::= */ + -2, /* (352) ignore_opt ::= IGNORE UNTREATED */ + -3, /* (353) cmd ::= KILL CONNECTION NK_INTEGER */ + -3, /* (354) cmd ::= KILL QUERY NK_STRING */ + -3, /* (355) cmd ::= KILL TRANSACTION NK_INTEGER */ + -2, /* (356) cmd ::= BALANCE VGROUP */ + -3, /* (357) cmd ::= BALANCE VGROUP LEADER */ + -4, /* (358) cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER */ + -4, /* (359) cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list */ + -3, /* (360) cmd ::= SPLIT VGROUP NK_INTEGER */ + -2, /* (361) dnode_list ::= DNODE NK_INTEGER */ + -3, /* (362) dnode_list ::= dnode_list DNODE NK_INTEGER */ + -4, /* (363) cmd ::= DELETE FROM full_table_name where_clause_opt */ + -1, /* (364) cmd ::= query_or_subquery */ + -1, /* (365) cmd ::= insert_query */ + -7, /* (366) insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery */ + -4, /* (367) insert_query ::= INSERT INTO full_table_name query_or_subquery */ + -1, /* (368) literal ::= NK_INTEGER */ + -1, /* (369) literal ::= NK_FLOAT */ + -1, /* (370) literal ::= NK_STRING */ + -1, /* (371) literal ::= NK_BOOL */ + -2, /* (372) literal ::= TIMESTAMP NK_STRING */ + -1, /* (373) literal ::= duration_literal */ + -1, /* (374) literal ::= NULL */ + -1, /* (375) literal ::= NK_QUESTION */ + -1, /* (376) duration_literal ::= NK_VARIABLE */ + -1, /* (377) signed ::= NK_INTEGER */ + -2, /* (378) signed ::= NK_PLUS NK_INTEGER */ + -2, /* (379) signed ::= NK_MINUS NK_INTEGER */ + -1, /* (380) signed ::= NK_FLOAT */ + -2, /* (381) signed ::= NK_PLUS NK_FLOAT */ + -2, /* (382) signed ::= NK_MINUS NK_FLOAT */ + -1, /* (383) signed_literal ::= signed */ + -1, /* (384) signed_literal ::= NK_STRING */ + -1, /* (385) signed_literal ::= NK_BOOL */ + -2, /* (386) signed_literal ::= TIMESTAMP NK_STRING */ + -1, /* (387) signed_literal ::= duration_literal */ + -1, /* (388) signed_literal ::= NULL */ + -1, /* (389) signed_literal ::= literal_func */ + -1, /* (390) signed_literal ::= NK_QUESTION */ + -1, /* (391) literal_list ::= signed_literal */ + -3, /* (392) literal_list ::= literal_list NK_COMMA signed_literal */ + -1, /* (393) db_name ::= NK_ID */ + -1, /* (394) table_name ::= NK_ID */ + -1, /* (395) column_name ::= NK_ID */ + -1, /* (396) function_name ::= NK_ID */ + -1, /* (397) table_alias ::= NK_ID */ + -1, /* (398) column_alias ::= NK_ID */ + -1, /* (399) user_name ::= NK_ID */ + -1, /* (400) topic_name ::= NK_ID */ + -1, /* (401) stream_name ::= NK_ID */ + -1, /* (402) cgroup_name ::= NK_ID */ + -1, /* (403) index_name ::= NK_ID */ + -1, /* (404) expr_or_subquery ::= expression */ + -1, /* (405) expression ::= literal */ + -1, /* (406) expression ::= pseudo_column */ + -1, /* (407) expression ::= column_reference */ + -1, /* (408) expression ::= function_expression */ + -1, /* (409) expression ::= case_when_expression */ + -3, /* (410) expression ::= NK_LP expression NK_RP */ + -2, /* (411) expression ::= NK_PLUS expr_or_subquery */ + -2, /* (412) expression ::= NK_MINUS expr_or_subquery */ + -3, /* (413) expression ::= expr_or_subquery NK_PLUS expr_or_subquery */ + -3, /* (414) expression ::= expr_or_subquery NK_MINUS expr_or_subquery */ + -3, /* (415) expression ::= expr_or_subquery NK_STAR expr_or_subquery */ + -3, /* (416) expression ::= expr_or_subquery NK_SLASH expr_or_subquery */ + -3, /* (417) expression ::= expr_or_subquery NK_REM expr_or_subquery */ + -3, /* (418) expression ::= column_reference NK_ARROW NK_STRING */ + -3, /* (419) expression ::= expr_or_subquery NK_BITAND expr_or_subquery */ + -3, /* (420) expression ::= expr_or_subquery NK_BITOR expr_or_subquery */ + -1, /* (421) expression_list ::= expr_or_subquery */ + -3, /* (422) expression_list ::= expression_list NK_COMMA expr_or_subquery */ + -1, /* (423) column_reference ::= column_name */ + -3, /* (424) column_reference ::= table_name NK_DOT column_name */ + -1, /* (425) pseudo_column ::= ROWTS */ + -1, /* (426) pseudo_column ::= TBNAME */ + -3, /* (427) pseudo_column ::= table_name NK_DOT TBNAME */ + -1, /* (428) pseudo_column ::= QSTART */ + -1, /* (429) pseudo_column ::= QEND */ + -1, /* (430) pseudo_column ::= QDURATION */ + -1, /* (431) pseudo_column ::= WSTART */ + -1, /* (432) pseudo_column ::= WEND */ + -1, /* (433) pseudo_column ::= WDURATION */ + -1, /* (434) pseudo_column ::= IROWTS */ + -1, /* (435) pseudo_column ::= ISFILLED */ + -1, /* (436) pseudo_column ::= QTAGS */ + -4, /* (437) function_expression ::= function_name NK_LP expression_list NK_RP */ + -4, /* (438) function_expression ::= star_func NK_LP star_func_para_list NK_RP */ + -6, /* (439) function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP */ + -1, /* (440) function_expression ::= literal_func */ + -3, /* (441) literal_func ::= noarg_func NK_LP NK_RP */ + -1, /* (442) literal_func ::= NOW */ + -1, /* (443) noarg_func ::= NOW */ + -1, /* (444) noarg_func ::= TODAY */ + -1, /* (445) noarg_func ::= TIMEZONE */ + -1, /* (446) noarg_func ::= DATABASE */ + -1, /* (447) noarg_func ::= CLIENT_VERSION */ + -1, /* (448) noarg_func ::= SERVER_VERSION */ + -1, /* (449) noarg_func ::= SERVER_STATUS */ + -1, /* (450) noarg_func ::= CURRENT_USER */ + -1, /* (451) noarg_func ::= USER */ + -1, /* (452) star_func ::= COUNT */ + -1, /* (453) star_func ::= FIRST */ + -1, /* (454) star_func ::= LAST */ + -1, /* (455) star_func ::= LAST_ROW */ + -1, /* (456) star_func_para_list ::= NK_STAR */ + -1, /* (457) star_func_para_list ::= other_para_list */ + -1, /* (458) other_para_list ::= star_func_para */ + -3, /* (459) other_para_list ::= other_para_list NK_COMMA star_func_para */ + -1, /* (460) star_func_para ::= expr_or_subquery */ + -3, /* (461) star_func_para ::= table_name NK_DOT NK_STAR */ + -4, /* (462) case_when_expression ::= CASE when_then_list case_when_else_opt END */ + -5, /* (463) case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END */ + -1, /* (464) when_then_list ::= when_then_expr */ + -2, /* (465) when_then_list ::= when_then_list when_then_expr */ + -4, /* (466) when_then_expr ::= WHEN common_expression THEN common_expression */ + 0, /* (467) case_when_else_opt ::= */ + -2, /* (468) case_when_else_opt ::= ELSE common_expression */ + -3, /* (469) predicate ::= expr_or_subquery compare_op expr_or_subquery */ + -5, /* (470) predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery */ + -6, /* (471) predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery */ + -3, /* (472) predicate ::= expr_or_subquery IS NULL */ + -4, /* (473) predicate ::= expr_or_subquery IS NOT NULL */ + -3, /* (474) predicate ::= expr_or_subquery in_op in_predicate_value */ + -1, /* (475) compare_op ::= NK_LT */ + -1, /* (476) compare_op ::= NK_GT */ + -1, /* (477) compare_op ::= NK_LE */ + -1, /* (478) compare_op ::= NK_GE */ + -1, /* (479) compare_op ::= NK_NE */ + -1, /* (480) compare_op ::= NK_EQ */ + -1, /* (481) compare_op ::= LIKE */ + -2, /* (482) compare_op ::= NOT LIKE */ + -1, /* (483) compare_op ::= MATCH */ + -1, /* (484) compare_op ::= NMATCH */ + -1, /* (485) compare_op ::= CONTAINS */ + -1, /* (486) in_op ::= IN */ + -2, /* (487) in_op ::= NOT IN */ + -3, /* (488) in_predicate_value ::= NK_LP literal_list NK_RP */ + -1, /* (489) boolean_value_expression ::= boolean_primary */ + -2, /* (490) boolean_value_expression ::= NOT boolean_primary */ + -3, /* (491) boolean_value_expression ::= boolean_value_expression OR boolean_value_expression */ + -3, /* (492) boolean_value_expression ::= boolean_value_expression AND boolean_value_expression */ + -1, /* (493) boolean_primary ::= predicate */ + -3, /* (494) boolean_primary ::= NK_LP boolean_value_expression NK_RP */ + -1, /* (495) common_expression ::= expr_or_subquery */ + -1, /* (496) common_expression ::= boolean_value_expression */ + 0, /* (497) from_clause_opt ::= */ + -2, /* (498) from_clause_opt ::= FROM table_reference_list */ + -1, /* (499) table_reference_list ::= table_reference */ + -3, /* (500) table_reference_list ::= table_reference_list NK_COMMA table_reference */ + -1, /* (501) table_reference ::= table_primary */ + -1, /* (502) table_reference ::= joined_table */ + -2, /* (503) table_primary ::= table_name alias_opt */ + -4, /* (504) table_primary ::= db_name NK_DOT table_name alias_opt */ + -2, /* (505) table_primary ::= subquery alias_opt */ + -1, /* (506) table_primary ::= parenthesized_joined_table */ + 0, /* (507) alias_opt ::= */ + -1, /* (508) alias_opt ::= table_alias */ + -2, /* (509) alias_opt ::= AS table_alias */ + -3, /* (510) parenthesized_joined_table ::= NK_LP joined_table NK_RP */ + -3, /* (511) parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP */ + -6, /* (512) joined_table ::= table_reference join_type JOIN table_reference ON search_condition */ + 0, /* (513) join_type ::= */ + -1, /* (514) join_type ::= INNER */ + -12, /* (515) query_specification ::= SELECT set_quantifier_opt select_list from_clause_opt where_clause_opt partition_by_clause_opt range_opt every_opt fill_opt twindow_clause_opt group_by_clause_opt having_clause_opt */ + 0, /* (516) set_quantifier_opt ::= */ + -1, /* (517) set_quantifier_opt ::= DISTINCT */ + -1, /* (518) set_quantifier_opt ::= ALL */ + -1, /* (519) select_list ::= select_item */ + -3, /* (520) select_list ::= select_list NK_COMMA select_item */ + -1, /* (521) select_item ::= NK_STAR */ + -1, /* (522) select_item ::= common_expression */ + -2, /* (523) select_item ::= common_expression column_alias */ + -3, /* (524) select_item ::= common_expression AS column_alias */ + -3, /* (525) select_item ::= table_name NK_DOT NK_STAR */ + 0, /* (526) where_clause_opt ::= */ + -2, /* (527) where_clause_opt ::= WHERE search_condition */ + 0, /* (528) partition_by_clause_opt ::= */ + -3, /* (529) partition_by_clause_opt ::= PARTITION BY partition_list */ + -1, /* (530) partition_list ::= partition_item */ + -3, /* (531) partition_list ::= partition_list NK_COMMA partition_item */ + -1, /* (532) partition_item ::= expr_or_subquery */ + -2, /* (533) partition_item ::= expr_or_subquery column_alias */ + -3, /* (534) partition_item ::= expr_or_subquery AS column_alias */ + 0, /* (535) twindow_clause_opt ::= */ + -6, /* (536) twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA duration_literal NK_RP */ + -4, /* (537) twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP */ + -6, /* (538) twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_RP sliding_opt fill_opt */ + -8, /* (539) twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_COMMA duration_literal NK_RP sliding_opt fill_opt */ + -7, /* (540) twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition */ + 0, /* (541) sliding_opt ::= */ + -4, /* (542) sliding_opt ::= SLIDING NK_LP duration_literal NK_RP */ + 0, /* (543) fill_opt ::= */ + -4, /* (544) fill_opt ::= FILL NK_LP fill_mode NK_RP */ + -6, /* (545) fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP */ + -6, /* (546) fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP */ + -1, /* (547) fill_mode ::= NONE */ + -1, /* (548) fill_mode ::= PREV */ + -1, /* (549) fill_mode ::= NULL */ + -1, /* (550) fill_mode ::= NULL_F */ + -1, /* (551) fill_mode ::= LINEAR */ + -1, /* (552) fill_mode ::= NEXT */ + 0, /* (553) group_by_clause_opt ::= */ + -3, /* (554) group_by_clause_opt ::= GROUP BY group_by_list */ + -1, /* (555) group_by_list ::= expr_or_subquery */ + -3, /* (556) group_by_list ::= group_by_list NK_COMMA expr_or_subquery */ + 0, /* (557) having_clause_opt ::= */ + -2, /* (558) having_clause_opt ::= HAVING search_condition */ + 0, /* (559) range_opt ::= */ + -6, /* (560) range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP */ + 0, /* (561) every_opt ::= */ + -4, /* (562) every_opt ::= EVERY NK_LP duration_literal NK_RP */ + -4, /* (563) query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt */ + -1, /* (564) query_simple ::= query_specification */ + -1, /* (565) query_simple ::= union_query_expression */ + -4, /* (566) union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery */ + -3, /* (567) union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery */ + -1, /* (568) query_simple_or_subquery ::= query_simple */ + -1, /* (569) query_simple_or_subquery ::= subquery */ + -1, /* (570) query_or_subquery ::= query_expression */ + -1, /* (571) query_or_subquery ::= subquery */ + 0, /* (572) order_by_clause_opt ::= */ + -3, /* (573) order_by_clause_opt ::= ORDER BY sort_specification_list */ + 0, /* (574) slimit_clause_opt ::= */ + -2, /* (575) slimit_clause_opt ::= SLIMIT NK_INTEGER */ + -4, /* (576) slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER */ + -4, /* (577) slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER */ + 0, /* (578) limit_clause_opt ::= */ + -2, /* (579) limit_clause_opt ::= LIMIT NK_INTEGER */ + -4, /* (580) limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER */ + -4, /* (581) limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER */ + -3, /* (582) subquery ::= NK_LP query_expression NK_RP */ + -3, /* (583) subquery ::= NK_LP subquery NK_RP */ + -1, /* (584) search_condition ::= common_expression */ + -1, /* (585) sort_specification_list ::= sort_specification */ + -3, /* (586) sort_specification_list ::= sort_specification_list NK_COMMA sort_specification */ + -3, /* (587) sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt */ + 0, /* (588) ordering_specification_opt ::= */ + -1, /* (589) ordering_specification_opt ::= ASC */ + -1, /* (590) ordering_specification_opt ::= DESC */ + 0, /* (591) null_ordering_opt ::= */ + -2, /* (592) null_ordering_opt ::= NULLS FIRST */ + -2, /* (593) null_ordering_opt ::= NULLS LAST */ }; static void yy_accept(yyParser*); /* Forward Declaration */ @@ -4503,21 +4482,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 4509 "sql.c" - yy_destructor(yypParser,336,&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 4515 "sql.c" yy_destructor(yypParser,337,&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,338,&yymsp[0].minor); + break; case 2: /* account_options ::= */ -#line 55 "sql.y" { } -#line 4521 "sql.c" break; case 3: /* account_options ::= account_options PPS literal */ case 4: /* account_options ::= account_options TSERIES literal */ yytestcase(yyruleno==4); @@ -4528,26 +4501,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,336,&yymsp[-2].minor); -#line 56 "sql.y" +{ yy_destructor(yypParser,337,&yymsp[-2].minor); { } -#line 4535 "sql.c" - yy_destructor(yypParser,338,&yymsp[0].minor); + yy_destructor(yypParser,339,&yymsp[0].minor); } break; case 12: /* alter_account_options ::= alter_account_option */ -{ yy_destructor(yypParser,339,&yymsp[0].minor); -#line 68 "sql.y" +{ yy_destructor(yypParser,340,&yymsp[0].minor); { } -#line 4543 "sql.c" } break; case 13: /* alter_account_options ::= alter_account_options alter_account_option */ -{ yy_destructor(yypParser,337,&yymsp[-1].minor); -#line 69 "sql.y" +{ yy_destructor(yypParser,338,&yymsp[-1].minor); { } -#line 4550 "sql.c" - yy_destructor(yypParser,339,&yymsp[0].minor); + yy_destructor(yypParser,340,&yymsp[0].minor); } break; case 14: /* alter_account_option ::= PASS literal */ @@ -4560,2405 +4527,1628 @@ 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 4566 "sql.c" - yy_destructor(yypParser,338,&yymsp[0].minor); + yy_destructor(yypParser,339,&yymsp[0].minor); break; case 24: /* cmd ::= CREATE USER user_name PASS NK_STRING sysinfo_opt */ -#line 85 "sql.y" -{ pCxt->pRootNode = createCreateUserStmt(pCxt, &yymsp[-3].minor.yy497, &yymsp[-1].minor.yy0, yymsp[0].minor.yy563); } -#line 4572 "sql.c" +{ pCxt->pRootNode = createCreateUserStmt(pCxt, &yymsp[-3].minor.yy833, &yymsp[-1].minor.yy0, yymsp[0].minor.yy575); } break; case 25: /* cmd ::= ALTER USER user_name PASS NK_STRING */ -#line 86 "sql.y" -{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy497, TSDB_ALTER_USER_PASSWD, &yymsp[0].minor.yy0); } -#line 4577 "sql.c" +{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy833, TSDB_ALTER_USER_PASSWD, &yymsp[0].minor.yy0); } break; case 26: /* cmd ::= ALTER USER user_name ENABLE NK_INTEGER */ -#line 87 "sql.y" -{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy497, TSDB_ALTER_USER_ENABLE, &yymsp[0].minor.yy0); } -#line 4582 "sql.c" +{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy833, TSDB_ALTER_USER_ENABLE, &yymsp[0].minor.yy0); } break; case 27: /* cmd ::= ALTER USER user_name SYSINFO NK_INTEGER */ -#line 88 "sql.y" -{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy497, TSDB_ALTER_USER_SYSINFO, &yymsp[0].minor.yy0); } -#line 4587 "sql.c" +{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy833, TSDB_ALTER_USER_SYSINFO, &yymsp[0].minor.yy0); } break; case 28: /* cmd ::= DROP USER user_name */ -#line 89 "sql.y" -{ pCxt->pRootNode = createDropUserStmt(pCxt, &yymsp[0].minor.yy497); } -#line 4592 "sql.c" +{ pCxt->pRootNode = createDropUserStmt(pCxt, &yymsp[0].minor.yy833); } break; case 29: /* sysinfo_opt ::= */ -#line 93 "sql.y" -{ yymsp[1].minor.yy563 = 1; } -#line 4597 "sql.c" +{ yymsp[1].minor.yy575 = 1; } break; case 30: /* sysinfo_opt ::= SYSINFO NK_INTEGER */ -#line 94 "sql.y" -{ yymsp[-1].minor.yy563 = taosStr2Int8(yymsp[0].minor.yy0.z, NULL, 10); } -#line 4602 "sql.c" +{ yymsp[-1].minor.yy575 = taosStr2Int8(yymsp[0].minor.yy0.z, NULL, 10); } break; case 31: /* cmd ::= GRANT privileges ON priv_level with_opt TO user_name */ -#line 97 "sql.y" -{ pCxt->pRootNode = createGrantStmt(pCxt, yymsp[-5].minor.yy693, &yymsp[-3].minor.yy953, &yymsp[0].minor.yy497, yymsp[-2].minor.yy164); } -#line 4607 "sql.c" +{ pCxt->pRootNode = createGrantStmt(pCxt, yymsp[-5].minor.yy85, &yymsp[-3].minor.yy489, &yymsp[0].minor.yy833, yymsp[-2].minor.yy520); } break; case 32: /* cmd ::= REVOKE privileges ON priv_level with_opt FROM user_name */ -#line 98 "sql.y" -{ pCxt->pRootNode = createRevokeStmt(pCxt, yymsp[-5].minor.yy693, &yymsp[-3].minor.yy953, &yymsp[0].minor.yy497, yymsp[-2].minor.yy164); } -#line 4612 "sql.c" +{ pCxt->pRootNode = createRevokeStmt(pCxt, yymsp[-5].minor.yy85, &yymsp[-3].minor.yy489, &yymsp[0].minor.yy833, yymsp[-2].minor.yy520); } break; case 33: /* privileges ::= ALL */ -#line 102 "sql.y" -{ yymsp[0].minor.yy693 = PRIVILEGE_TYPE_ALL; } -#line 4617 "sql.c" +{ yymsp[0].minor.yy85 = PRIVILEGE_TYPE_ALL; } break; case 34: /* privileges ::= priv_type_list */ case 36: /* priv_type_list ::= priv_type */ yytestcase(yyruleno==36); -#line 103 "sql.y" -{ yylhsminor.yy693 = yymsp[0].minor.yy693; } -#line 4623 "sql.c" - yymsp[0].minor.yy693 = yylhsminor.yy693; +{ yylhsminor.yy85 = yymsp[0].minor.yy85; } + yymsp[0].minor.yy85 = yylhsminor.yy85; break; case 35: /* privileges ::= SUBSCRIBE */ -#line 104 "sql.y" -{ yymsp[0].minor.yy693 = PRIVILEGE_TYPE_SUBSCRIBE; } -#line 4629 "sql.c" +{ yymsp[0].minor.yy85 = PRIVILEGE_TYPE_SUBSCRIBE; } break; case 37: /* priv_type_list ::= priv_type_list NK_COMMA priv_type */ -#line 109 "sql.y" -{ yylhsminor.yy693 = yymsp[-2].minor.yy693 | yymsp[0].minor.yy693; } -#line 4634 "sql.c" - yymsp[-2].minor.yy693 = yylhsminor.yy693; +{ yylhsminor.yy85 = yymsp[-2].minor.yy85 | yymsp[0].minor.yy85; } + yymsp[-2].minor.yy85 = yylhsminor.yy85; break; case 38: /* priv_type ::= READ */ -#line 113 "sql.y" -{ yymsp[0].minor.yy693 = PRIVILEGE_TYPE_READ; } -#line 4640 "sql.c" +{ yymsp[0].minor.yy85 = PRIVILEGE_TYPE_READ; } break; case 39: /* priv_type ::= WRITE */ -#line 114 "sql.y" -{ yymsp[0].minor.yy693 = PRIVILEGE_TYPE_WRITE; } -#line 4645 "sql.c" +{ yymsp[0].minor.yy85 = PRIVILEGE_TYPE_WRITE; } break; case 40: /* priv_level ::= NK_STAR NK_DOT NK_STAR */ -#line 118 "sql.y" -{ yylhsminor.yy953.first = yymsp[-2].minor.yy0; yylhsminor.yy953.second = yymsp[0].minor.yy0; } -#line 4650 "sql.c" - yymsp[-2].minor.yy953 = yylhsminor.yy953; +{ yylhsminor.yy489.first = yymsp[-2].minor.yy0; yylhsminor.yy489.second = yymsp[0].minor.yy0; } + yymsp[-2].minor.yy489 = yylhsminor.yy489; break; case 41: /* priv_level ::= db_name NK_DOT NK_STAR */ -#line 119 "sql.y" -{ yylhsminor.yy953.first = yymsp[-2].minor.yy497; yylhsminor.yy953.second = yymsp[0].minor.yy0; } -#line 4656 "sql.c" - yymsp[-2].minor.yy953 = yylhsminor.yy953; +{ yylhsminor.yy489.first = yymsp[-2].minor.yy833; yylhsminor.yy489.second = yymsp[0].minor.yy0; } + yymsp[-2].minor.yy489 = yylhsminor.yy489; break; case 42: /* priv_level ::= db_name NK_DOT table_name */ -#line 120 "sql.y" -{ yylhsminor.yy953.first = yymsp[-2].minor.yy497; yylhsminor.yy953.second = yymsp[0].minor.yy497; } -#line 4662 "sql.c" - yymsp[-2].minor.yy953 = yylhsminor.yy953; +{ yylhsminor.yy489.first = yymsp[-2].minor.yy833; yylhsminor.yy489.second = yymsp[0].minor.yy833; } + yymsp[-2].minor.yy489 = yylhsminor.yy489; break; case 43: /* priv_level ::= topic_name */ -#line 121 "sql.y" -{ yylhsminor.yy953.first = yymsp[0].minor.yy497; yylhsminor.yy953.second = nil_token; } -#line 4668 "sql.c" - yymsp[0].minor.yy953 = yylhsminor.yy953; +{ yylhsminor.yy489.first = yymsp[0].minor.yy833; yylhsminor.yy489.second = nil_token; } + yymsp[0].minor.yy489 = yylhsminor.yy489; break; case 44: /* with_opt ::= */ - case 141: /* start_opt ::= */ yytestcase(yyruleno==141); - case 145: /* end_opt ::= */ yytestcase(yyruleno==145); - case 269: /* like_pattern_opt ::= */ yytestcase(yyruleno==269); - case 346: /* subtable_opt ::= */ yytestcase(yyruleno==346); - case 464: /* case_when_else_opt ::= */ yytestcase(yyruleno==464); - case 494: /* from_clause_opt ::= */ yytestcase(yyruleno==494); - case 523: /* where_clause_opt ::= */ yytestcase(yyruleno==523); - case 532: /* twindow_clause_opt ::= */ yytestcase(yyruleno==532); - case 538: /* sliding_opt ::= */ yytestcase(yyruleno==538); - case 540: /* fill_opt ::= */ yytestcase(yyruleno==540); - case 554: /* having_clause_opt ::= */ yytestcase(yyruleno==554); - case 556: /* range_opt ::= */ yytestcase(yyruleno==556); - case 558: /* every_opt ::= */ yytestcase(yyruleno==558); - case 571: /* slimit_clause_opt ::= */ yytestcase(yyruleno==571); - case 575: /* limit_clause_opt ::= */ yytestcase(yyruleno==575); -#line 123 "sql.y" -{ yymsp[1].minor.yy164 = NULL; } -#line 4689 "sql.c" + case 144: /* start_opt ::= */ yytestcase(yyruleno==144); + case 148: /* end_opt ::= */ yytestcase(yyruleno==148); + case 272: /* like_pattern_opt ::= */ yytestcase(yyruleno==272); + case 349: /* subtable_opt ::= */ yytestcase(yyruleno==349); + case 467: /* case_when_else_opt ::= */ yytestcase(yyruleno==467); + case 497: /* from_clause_opt ::= */ yytestcase(yyruleno==497); + case 526: /* where_clause_opt ::= */ yytestcase(yyruleno==526); + case 535: /* twindow_clause_opt ::= */ yytestcase(yyruleno==535); + case 541: /* sliding_opt ::= */ yytestcase(yyruleno==541); + case 543: /* fill_opt ::= */ yytestcase(yyruleno==543); + case 557: /* having_clause_opt ::= */ yytestcase(yyruleno==557); + case 559: /* range_opt ::= */ yytestcase(yyruleno==559); + case 561: /* every_opt ::= */ yytestcase(yyruleno==561); + case 574: /* slimit_clause_opt ::= */ yytestcase(yyruleno==574); + case 578: /* limit_clause_opt ::= */ yytestcase(yyruleno==578); +{ yymsp[1].minor.yy520 = NULL; } break; case 45: /* with_opt ::= WITH search_condition */ - case 495: /* from_clause_opt ::= FROM table_reference_list */ yytestcase(yyruleno==495); - case 524: /* where_clause_opt ::= WHERE search_condition */ yytestcase(yyruleno==524); - case 555: /* having_clause_opt ::= HAVING search_condition */ yytestcase(yyruleno==555); -#line 124 "sql.y" -{ yymsp[-1].minor.yy164 = yymsp[0].minor.yy164; } -#line 4697 "sql.c" + case 498: /* from_clause_opt ::= FROM table_reference_list */ yytestcase(yyruleno==498); + case 527: /* where_clause_opt ::= WHERE search_condition */ yytestcase(yyruleno==527); + case 558: /* having_clause_opt ::= HAVING search_condition */ yytestcase(yyruleno==558); +{ yymsp[-1].minor.yy520 = yymsp[0].minor.yy520; } break; case 46: /* cmd ::= CREATE DNODE dnode_endpoint */ -#line 127 "sql.y" -{ pCxt->pRootNode = createCreateDnodeStmt(pCxt, &yymsp[0].minor.yy497, NULL); } -#line 4702 "sql.c" +{ pCxt->pRootNode = createCreateDnodeStmt(pCxt, &yymsp[0].minor.yy833, NULL); } break; case 47: /* cmd ::= CREATE DNODE dnode_endpoint PORT NK_INTEGER */ -#line 128 "sql.y" -{ pCxt->pRootNode = createCreateDnodeStmt(pCxt, &yymsp[-2].minor.yy497, &yymsp[0].minor.yy0); } -#line 4707 "sql.c" +{ pCxt->pRootNode = createCreateDnodeStmt(pCxt, &yymsp[-2].minor.yy833, &yymsp[0].minor.yy0); } break; case 48: /* cmd ::= DROP DNODE NK_INTEGER force_opt */ -#line 129 "sql.y" -{ pCxt->pRootNode = createDropDnodeStmt(pCxt, &yymsp[-1].minor.yy0, yymsp[0].minor.yy441); } -#line 4712 "sql.c" +{ pCxt->pRootNode = createDropDnodeStmt(pCxt, &yymsp[-1].minor.yy0, yymsp[0].minor.yy537, false); } break; case 49: /* cmd ::= DROP DNODE dnode_endpoint force_opt */ -#line 130 "sql.y" -{ pCxt->pRootNode = createDropDnodeStmt(pCxt, &yymsp[-1].minor.yy497, yymsp[0].minor.yy441); } -#line 4717 "sql.c" +{ pCxt->pRootNode = createDropDnodeStmt(pCxt, &yymsp[-1].minor.yy833, yymsp[0].minor.yy537, false); } break; - case 50: /* cmd ::= ALTER DNODE NK_INTEGER NK_STRING */ -#line 131 "sql.y" + case 50: /* cmd ::= DROP DNODE NK_INTEGER unsafe_opt */ +{ pCxt->pRootNode = createDropDnodeStmt(pCxt, &yymsp[-1].minor.yy0, false, yymsp[0].minor.yy537); } + break; + case 51: /* cmd ::= DROP DNODE dnode_endpoint unsafe_opt */ +{ pCxt->pRootNode = createDropDnodeStmt(pCxt, &yymsp[-1].minor.yy833, false, yymsp[0].minor.yy537); } + break; + case 52: /* cmd ::= ALTER DNODE NK_INTEGER NK_STRING */ { pCxt->pRootNode = createAlterDnodeStmt(pCxt, &yymsp[-1].minor.yy0, &yymsp[0].minor.yy0, NULL); } -#line 4722 "sql.c" break; - case 51: /* cmd ::= ALTER DNODE NK_INTEGER NK_STRING NK_STRING */ -#line 132 "sql.y" + case 53: /* cmd ::= ALTER DNODE NK_INTEGER NK_STRING NK_STRING */ { pCxt->pRootNode = createAlterDnodeStmt(pCxt, &yymsp[-2].minor.yy0, &yymsp[-1].minor.yy0, &yymsp[0].minor.yy0); } -#line 4727 "sql.c" break; - case 52: /* cmd ::= ALTER ALL DNODES NK_STRING */ -#line 133 "sql.y" + case 54: /* cmd ::= ALTER ALL DNODES NK_STRING */ { pCxt->pRootNode = createAlterDnodeStmt(pCxt, NULL, &yymsp[0].minor.yy0, NULL); } -#line 4732 "sql.c" break; - case 53: /* cmd ::= ALTER ALL DNODES NK_STRING NK_STRING */ -#line 134 "sql.y" + case 55: /* cmd ::= ALTER ALL DNODES NK_STRING NK_STRING */ { pCxt->pRootNode = createAlterDnodeStmt(pCxt, NULL, &yymsp[-1].minor.yy0, &yymsp[0].minor.yy0); } -#line 4737 "sql.c" break; - case 54: /* cmd ::= RESTORE DNODE NK_INTEGER */ -#line 135 "sql.y" + case 56: /* cmd ::= RESTORE DNODE NK_INTEGER */ { pCxt->pRootNode = createRestoreComponentNodeStmt(pCxt, QUERY_NODE_RESTORE_DNODE_STMT, &yymsp[0].minor.yy0); } -#line 4742 "sql.c" break; - case 55: /* dnode_endpoint ::= NK_STRING */ - case 56: /* dnode_endpoint ::= NK_ID */ yytestcase(yyruleno==56); - case 57: /* dnode_endpoint ::= NK_IPTOKEN */ yytestcase(yyruleno==57); - case 293: /* sma_func_name ::= COUNT */ yytestcase(yyruleno==293); - case 294: /* sma_func_name ::= FIRST */ yytestcase(yyruleno==294); - case 295: /* sma_func_name ::= LAST */ yytestcase(yyruleno==295); - case 296: /* sma_func_name ::= LAST_ROW */ yytestcase(yyruleno==296); - case 390: /* db_name ::= NK_ID */ yytestcase(yyruleno==390); - case 391: /* table_name ::= NK_ID */ yytestcase(yyruleno==391); - case 392: /* column_name ::= NK_ID */ yytestcase(yyruleno==392); - case 393: /* function_name ::= NK_ID */ yytestcase(yyruleno==393); - case 394: /* table_alias ::= NK_ID */ yytestcase(yyruleno==394); - case 395: /* column_alias ::= NK_ID */ yytestcase(yyruleno==395); - case 396: /* user_name ::= NK_ID */ yytestcase(yyruleno==396); - case 397: /* topic_name ::= NK_ID */ yytestcase(yyruleno==397); - case 398: /* stream_name ::= NK_ID */ yytestcase(yyruleno==398); - case 399: /* cgroup_name ::= NK_ID */ yytestcase(yyruleno==399); - case 400: /* index_name ::= NK_ID */ yytestcase(yyruleno==400); - case 440: /* noarg_func ::= NOW */ yytestcase(yyruleno==440); - case 441: /* noarg_func ::= TODAY */ yytestcase(yyruleno==441); - case 442: /* noarg_func ::= TIMEZONE */ yytestcase(yyruleno==442); - case 443: /* noarg_func ::= DATABASE */ yytestcase(yyruleno==443); - case 444: /* noarg_func ::= CLIENT_VERSION */ yytestcase(yyruleno==444); - case 445: /* noarg_func ::= SERVER_VERSION */ yytestcase(yyruleno==445); - case 446: /* noarg_func ::= SERVER_STATUS */ yytestcase(yyruleno==446); - case 447: /* noarg_func ::= CURRENT_USER */ yytestcase(yyruleno==447); - case 448: /* noarg_func ::= USER */ yytestcase(yyruleno==448); - case 449: /* star_func ::= COUNT */ yytestcase(yyruleno==449); - case 450: /* star_func ::= FIRST */ yytestcase(yyruleno==450); - case 451: /* star_func ::= LAST */ yytestcase(yyruleno==451); - case 452: /* star_func ::= LAST_ROW */ yytestcase(yyruleno==452); -#line 139 "sql.y" -{ yylhsminor.yy497 = yymsp[0].minor.yy0; } -#line 4777 "sql.c" - yymsp[0].minor.yy497 = yylhsminor.yy497; + case 57: /* dnode_endpoint ::= NK_STRING */ + case 58: /* dnode_endpoint ::= NK_ID */ yytestcase(yyruleno==58); + case 59: /* dnode_endpoint ::= NK_IPTOKEN */ yytestcase(yyruleno==59); + case 296: /* sma_func_name ::= COUNT */ yytestcase(yyruleno==296); + case 297: /* sma_func_name ::= FIRST */ yytestcase(yyruleno==297); + case 298: /* sma_func_name ::= LAST */ yytestcase(yyruleno==298); + case 299: /* sma_func_name ::= LAST_ROW */ yytestcase(yyruleno==299); + case 393: /* db_name ::= NK_ID */ yytestcase(yyruleno==393); + case 394: /* table_name ::= NK_ID */ yytestcase(yyruleno==394); + case 395: /* column_name ::= NK_ID */ yytestcase(yyruleno==395); + case 396: /* function_name ::= NK_ID */ yytestcase(yyruleno==396); + case 397: /* table_alias ::= NK_ID */ yytestcase(yyruleno==397); + case 398: /* column_alias ::= NK_ID */ yytestcase(yyruleno==398); + case 399: /* user_name ::= NK_ID */ yytestcase(yyruleno==399); + case 400: /* topic_name ::= NK_ID */ yytestcase(yyruleno==400); + case 401: /* stream_name ::= NK_ID */ yytestcase(yyruleno==401); + case 402: /* cgroup_name ::= NK_ID */ yytestcase(yyruleno==402); + case 403: /* index_name ::= NK_ID */ yytestcase(yyruleno==403); + case 443: /* noarg_func ::= NOW */ yytestcase(yyruleno==443); + case 444: /* noarg_func ::= TODAY */ yytestcase(yyruleno==444); + case 445: /* noarg_func ::= TIMEZONE */ yytestcase(yyruleno==445); + case 446: /* noarg_func ::= DATABASE */ yytestcase(yyruleno==446); + case 447: /* noarg_func ::= CLIENT_VERSION */ yytestcase(yyruleno==447); + case 448: /* noarg_func ::= SERVER_VERSION */ yytestcase(yyruleno==448); + case 449: /* noarg_func ::= SERVER_STATUS */ yytestcase(yyruleno==449); + case 450: /* noarg_func ::= CURRENT_USER */ yytestcase(yyruleno==450); + case 451: /* noarg_func ::= USER */ yytestcase(yyruleno==451); + case 452: /* star_func ::= COUNT */ yytestcase(yyruleno==452); + case 453: /* star_func ::= FIRST */ yytestcase(yyruleno==453); + case 454: /* star_func ::= LAST */ yytestcase(yyruleno==454); + case 455: /* star_func ::= LAST_ROW */ yytestcase(yyruleno==455); +{ yylhsminor.yy833 = yymsp[0].minor.yy0; } + yymsp[0].minor.yy833 = yylhsminor.yy833; break; - case 58: /* force_opt ::= */ - case 81: /* not_exists_opt ::= */ yytestcase(yyruleno==81); - case 83: /* exists_opt ::= */ yytestcase(yyruleno==83); - case 313: /* analyze_opt ::= */ yytestcase(yyruleno==313); - case 320: /* agg_func_opt ::= */ yytestcase(yyruleno==320); - case 326: /* or_replace_opt ::= */ yytestcase(yyruleno==326); - case 348: /* ignore_opt ::= */ yytestcase(yyruleno==348); - case 513: /* set_quantifier_opt ::= */ yytestcase(yyruleno==513); -#line 145 "sql.y" -{ yymsp[1].minor.yy441 = false; } -#line 4790 "sql.c" + case 60: /* force_opt ::= */ + case 84: /* not_exists_opt ::= */ yytestcase(yyruleno==84); + case 86: /* exists_opt ::= */ yytestcase(yyruleno==86); + case 316: /* analyze_opt ::= */ yytestcase(yyruleno==316); + case 323: /* agg_func_opt ::= */ yytestcase(yyruleno==323); + case 329: /* or_replace_opt ::= */ yytestcase(yyruleno==329); + case 351: /* ignore_opt ::= */ yytestcase(yyruleno==351); + case 516: /* set_quantifier_opt ::= */ yytestcase(yyruleno==516); +{ yymsp[1].minor.yy537 = false; } break; - case 59: /* force_opt ::= FORCE */ - case 314: /* analyze_opt ::= ANALYZE */ yytestcase(yyruleno==314); - case 321: /* agg_func_opt ::= AGGREGATE */ yytestcase(yyruleno==321); - case 514: /* set_quantifier_opt ::= DISTINCT */ yytestcase(yyruleno==514); -#line 146 "sql.y" -{ yymsp[0].minor.yy441 = true; } -#line 4798 "sql.c" + case 61: /* force_opt ::= FORCE */ + case 62: /* unsafe_opt ::= UNSAFE */ yytestcase(yyruleno==62); + case 317: /* analyze_opt ::= ANALYZE */ yytestcase(yyruleno==317); + case 324: /* agg_func_opt ::= AGGREGATE */ yytestcase(yyruleno==324); + case 517: /* set_quantifier_opt ::= DISTINCT */ yytestcase(yyruleno==517); +{ yymsp[0].minor.yy537 = true; } break; - case 60: /* cmd ::= ALTER LOCAL NK_STRING */ -#line 149 "sql.y" + case 63: /* cmd ::= ALTER LOCAL NK_STRING */ { pCxt->pRootNode = createAlterLocalStmt(pCxt, &yymsp[0].minor.yy0, NULL); } -#line 4803 "sql.c" break; - case 61: /* cmd ::= ALTER LOCAL NK_STRING NK_STRING */ -#line 150 "sql.y" + case 64: /* cmd ::= ALTER LOCAL NK_STRING NK_STRING */ { pCxt->pRootNode = createAlterLocalStmt(pCxt, &yymsp[-1].minor.yy0, &yymsp[0].minor.yy0); } -#line 4808 "sql.c" break; - case 62: /* cmd ::= CREATE QNODE ON DNODE NK_INTEGER */ -#line 153 "sql.y" + case 65: /* cmd ::= CREATE QNODE ON DNODE NK_INTEGER */ { pCxt->pRootNode = createCreateComponentNodeStmt(pCxt, QUERY_NODE_CREATE_QNODE_STMT, &yymsp[0].minor.yy0); } -#line 4813 "sql.c" break; - case 63: /* cmd ::= DROP QNODE ON DNODE NK_INTEGER */ -#line 154 "sql.y" + case 66: /* cmd ::= DROP QNODE ON DNODE NK_INTEGER */ { pCxt->pRootNode = createDropComponentNodeStmt(pCxt, QUERY_NODE_DROP_QNODE_STMT, &yymsp[0].minor.yy0); } -#line 4818 "sql.c" break; - case 64: /* cmd ::= RESTORE QNODE ON DNODE NK_INTEGER */ -#line 155 "sql.y" + case 67: /* cmd ::= RESTORE QNODE ON DNODE NK_INTEGER */ { pCxt->pRootNode = createRestoreComponentNodeStmt(pCxt, QUERY_NODE_RESTORE_QNODE_STMT, &yymsp[0].minor.yy0); } -#line 4823 "sql.c" break; - case 65: /* cmd ::= CREATE BNODE ON DNODE NK_INTEGER */ -#line 158 "sql.y" + case 68: /* cmd ::= CREATE BNODE ON DNODE NK_INTEGER */ { pCxt->pRootNode = createCreateComponentNodeStmt(pCxt, QUERY_NODE_CREATE_BNODE_STMT, &yymsp[0].minor.yy0); } -#line 4828 "sql.c" break; - case 66: /* cmd ::= DROP BNODE ON DNODE NK_INTEGER */ -#line 159 "sql.y" + case 69: /* cmd ::= DROP BNODE ON DNODE NK_INTEGER */ { pCxt->pRootNode = createDropComponentNodeStmt(pCxt, QUERY_NODE_DROP_BNODE_STMT, &yymsp[0].minor.yy0); } -#line 4833 "sql.c" break; - case 67: /* cmd ::= CREATE SNODE ON DNODE NK_INTEGER */ -#line 162 "sql.y" + case 70: /* cmd ::= CREATE SNODE ON DNODE NK_INTEGER */ { pCxt->pRootNode = createCreateComponentNodeStmt(pCxt, QUERY_NODE_CREATE_SNODE_STMT, &yymsp[0].minor.yy0); } -#line 4838 "sql.c" break; - case 68: /* cmd ::= DROP SNODE ON DNODE NK_INTEGER */ -#line 163 "sql.y" + case 71: /* cmd ::= DROP SNODE ON DNODE NK_INTEGER */ { pCxt->pRootNode = createDropComponentNodeStmt(pCxt, QUERY_NODE_DROP_SNODE_STMT, &yymsp[0].minor.yy0); } -#line 4843 "sql.c" break; - case 69: /* cmd ::= CREATE MNODE ON DNODE NK_INTEGER */ -#line 166 "sql.y" + case 72: /* cmd ::= CREATE MNODE ON DNODE NK_INTEGER */ { pCxt->pRootNode = createCreateComponentNodeStmt(pCxt, QUERY_NODE_CREATE_MNODE_STMT, &yymsp[0].minor.yy0); } -#line 4848 "sql.c" break; - case 70: /* cmd ::= DROP MNODE ON DNODE NK_INTEGER */ -#line 167 "sql.y" + case 73: /* cmd ::= DROP MNODE ON DNODE NK_INTEGER */ { pCxt->pRootNode = createDropComponentNodeStmt(pCxt, QUERY_NODE_DROP_MNODE_STMT, &yymsp[0].minor.yy0); } -#line 4853 "sql.c" break; - case 71: /* cmd ::= RESTORE MNODE ON DNODE NK_INTEGER */ -#line 168 "sql.y" + case 74: /* cmd ::= RESTORE MNODE ON DNODE NK_INTEGER */ { pCxt->pRootNode = createRestoreComponentNodeStmt(pCxt, QUERY_NODE_RESTORE_MNODE_STMT, &yymsp[0].minor.yy0); } -#line 4858 "sql.c" break; - case 72: /* cmd ::= RESTORE VNODE ON DNODE NK_INTEGER */ -#line 171 "sql.y" + case 75: /* cmd ::= RESTORE VNODE ON DNODE NK_INTEGER */ { pCxt->pRootNode = createRestoreComponentNodeStmt(pCxt, QUERY_NODE_RESTORE_VNODE_STMT, &yymsp[0].minor.yy0); } -#line 4863 "sql.c" break; - case 73: /* cmd ::= CREATE DATABASE not_exists_opt db_name db_options */ -#line 174 "sql.y" -{ pCxt->pRootNode = createCreateDatabaseStmt(pCxt, yymsp[-2].minor.yy441, &yymsp[-1].minor.yy497, yymsp[0].minor.yy164); } -#line 4868 "sql.c" + case 76: /* cmd ::= CREATE DATABASE not_exists_opt db_name db_options */ +{ pCxt->pRootNode = createCreateDatabaseStmt(pCxt, yymsp[-2].minor.yy537, &yymsp[-1].minor.yy833, yymsp[0].minor.yy520); } break; - case 74: /* cmd ::= DROP DATABASE exists_opt db_name */ -#line 175 "sql.y" -{ pCxt->pRootNode = createDropDatabaseStmt(pCxt, yymsp[-1].minor.yy441, &yymsp[0].minor.yy497); } -#line 4873 "sql.c" + case 77: /* cmd ::= DROP DATABASE exists_opt db_name */ +{ pCxt->pRootNode = createDropDatabaseStmt(pCxt, yymsp[-1].minor.yy537, &yymsp[0].minor.yy833); } break; - case 75: /* cmd ::= USE db_name */ -#line 176 "sql.y" -{ pCxt->pRootNode = createUseDatabaseStmt(pCxt, &yymsp[0].minor.yy497); } -#line 4878 "sql.c" + case 78: /* cmd ::= USE db_name */ +{ pCxt->pRootNode = createUseDatabaseStmt(pCxt, &yymsp[0].minor.yy833); } break; - case 76: /* cmd ::= ALTER DATABASE db_name alter_db_options */ -#line 177 "sql.y" -{ pCxt->pRootNode = createAlterDatabaseStmt(pCxt, &yymsp[-1].minor.yy497, yymsp[0].minor.yy164); } -#line 4883 "sql.c" + case 79: /* cmd ::= ALTER DATABASE db_name alter_db_options */ +{ pCxt->pRootNode = createAlterDatabaseStmt(pCxt, &yymsp[-1].minor.yy833, yymsp[0].minor.yy520); } break; - case 77: /* cmd ::= FLUSH DATABASE db_name */ -#line 178 "sql.y" -{ pCxt->pRootNode = createFlushDatabaseStmt(pCxt, &yymsp[0].minor.yy497); } -#line 4888 "sql.c" + case 80: /* cmd ::= FLUSH DATABASE db_name */ +{ pCxt->pRootNode = createFlushDatabaseStmt(pCxt, &yymsp[0].minor.yy833); } break; - case 78: /* cmd ::= TRIM DATABASE db_name speed_opt */ -#line 179 "sql.y" -{ pCxt->pRootNode = createTrimDatabaseStmt(pCxt, &yymsp[-1].minor.yy497, yymsp[0].minor.yy560); } -#line 4893 "sql.c" + case 81: /* cmd ::= TRIM DATABASE db_name speed_opt */ +{ pCxt->pRootNode = createTrimDatabaseStmt(pCxt, &yymsp[-1].minor.yy833, yymsp[0].minor.yy860); } break; - case 79: /* cmd ::= COMPACT DATABASE db_name start_opt end_opt */ -#line 180 "sql.y" -{ pCxt->pRootNode = createCompactStmt(pCxt, &yymsp[-2].minor.yy497, yymsp[-1].minor.yy164, yymsp[0].minor.yy164); } -#line 4898 "sql.c" + case 82: /* cmd ::= COMPACT DATABASE db_name start_opt end_opt */ +{ pCxt->pRootNode = createCompactStmt(pCxt, &yymsp[-2].minor.yy833, yymsp[-1].minor.yy520, yymsp[0].minor.yy520); } break; - case 80: /* not_exists_opt ::= IF NOT EXISTS */ -#line 184 "sql.y" -{ yymsp[-2].minor.yy441 = true; } -#line 4903 "sql.c" + case 83: /* not_exists_opt ::= IF NOT EXISTS */ +{ yymsp[-2].minor.yy537 = true; } break; - case 82: /* exists_opt ::= IF EXISTS */ - case 327: /* or_replace_opt ::= OR REPLACE */ yytestcase(yyruleno==327); - case 349: /* ignore_opt ::= IGNORE UNTREATED */ yytestcase(yyruleno==349); -#line 189 "sql.y" -{ yymsp[-1].minor.yy441 = true; } -#line 4910 "sql.c" + case 85: /* exists_opt ::= IF EXISTS */ + case 330: /* or_replace_opt ::= OR REPLACE */ yytestcase(yyruleno==330); + case 352: /* ignore_opt ::= IGNORE UNTREATED */ yytestcase(yyruleno==352); +{ yymsp[-1].minor.yy537 = true; } break; - case 84: /* db_options ::= */ -#line 192 "sql.y" -{ yymsp[1].minor.yy164 = createDefaultDatabaseOptions(pCxt); } -#line 4915 "sql.c" + case 87: /* db_options ::= */ +{ yymsp[1].minor.yy520 = createDefaultDatabaseOptions(pCxt); } break; - case 85: /* db_options ::= db_options BUFFER NK_INTEGER */ -#line 193 "sql.y" -{ yylhsminor.yy164 = setDatabaseOption(pCxt, yymsp[-2].minor.yy164, DB_OPTION_BUFFER, &yymsp[0].minor.yy0); } -#line 4920 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 88: /* db_options ::= db_options BUFFER NK_INTEGER */ +{ yylhsminor.yy520 = setDatabaseOption(pCxt, yymsp[-2].minor.yy520, DB_OPTION_BUFFER, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 86: /* db_options ::= db_options CACHEMODEL NK_STRING */ -#line 194 "sql.y" -{ yylhsminor.yy164 = setDatabaseOption(pCxt, yymsp[-2].minor.yy164, DB_OPTION_CACHEMODEL, &yymsp[0].minor.yy0); } -#line 4926 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 89: /* db_options ::= db_options CACHEMODEL NK_STRING */ +{ yylhsminor.yy520 = setDatabaseOption(pCxt, yymsp[-2].minor.yy520, DB_OPTION_CACHEMODEL, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 87: /* db_options ::= db_options CACHESIZE NK_INTEGER */ -#line 195 "sql.y" -{ yylhsminor.yy164 = setDatabaseOption(pCxt, yymsp[-2].minor.yy164, DB_OPTION_CACHESIZE, &yymsp[0].minor.yy0); } -#line 4932 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 90: /* db_options ::= db_options CACHESIZE NK_INTEGER */ +{ yylhsminor.yy520 = setDatabaseOption(pCxt, yymsp[-2].minor.yy520, DB_OPTION_CACHESIZE, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 88: /* db_options ::= db_options COMP NK_INTEGER */ -#line 196 "sql.y" -{ yylhsminor.yy164 = setDatabaseOption(pCxt, yymsp[-2].minor.yy164, DB_OPTION_COMP, &yymsp[0].minor.yy0); } -#line 4938 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 91: /* db_options ::= db_options COMP NK_INTEGER */ +{ yylhsminor.yy520 = setDatabaseOption(pCxt, yymsp[-2].minor.yy520, DB_OPTION_COMP, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 89: /* db_options ::= db_options DURATION NK_INTEGER */ - case 90: /* db_options ::= db_options DURATION NK_VARIABLE */ yytestcase(yyruleno==90); -#line 197 "sql.y" -{ yylhsminor.yy164 = setDatabaseOption(pCxt, yymsp[-2].minor.yy164, DB_OPTION_DAYS, &yymsp[0].minor.yy0); } -#line 4945 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 92: /* db_options ::= db_options DURATION NK_INTEGER */ + case 93: /* db_options ::= db_options DURATION NK_VARIABLE */ yytestcase(yyruleno==93); +{ yylhsminor.yy520 = setDatabaseOption(pCxt, yymsp[-2].minor.yy520, DB_OPTION_DAYS, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 91: /* db_options ::= db_options MAXROWS NK_INTEGER */ -#line 199 "sql.y" -{ yylhsminor.yy164 = setDatabaseOption(pCxt, yymsp[-2].minor.yy164, DB_OPTION_MAXROWS, &yymsp[0].minor.yy0); } -#line 4951 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 94: /* db_options ::= db_options MAXROWS NK_INTEGER */ +{ yylhsminor.yy520 = setDatabaseOption(pCxt, yymsp[-2].minor.yy520, DB_OPTION_MAXROWS, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 92: /* db_options ::= db_options MINROWS NK_INTEGER */ -#line 200 "sql.y" -{ yylhsminor.yy164 = setDatabaseOption(pCxt, yymsp[-2].minor.yy164, DB_OPTION_MINROWS, &yymsp[0].minor.yy0); } -#line 4957 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 95: /* db_options ::= db_options MINROWS NK_INTEGER */ +{ yylhsminor.yy520 = setDatabaseOption(pCxt, yymsp[-2].minor.yy520, DB_OPTION_MINROWS, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 93: /* db_options ::= db_options KEEP integer_list */ - case 94: /* db_options ::= db_options KEEP variable_list */ yytestcase(yyruleno==94); -#line 201 "sql.y" -{ yylhsminor.yy164 = setDatabaseOption(pCxt, yymsp[-2].minor.yy164, DB_OPTION_KEEP, yymsp[0].minor.yy72); } -#line 4964 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 96: /* db_options ::= db_options KEEP integer_list */ + case 97: /* db_options ::= db_options KEEP variable_list */ yytestcase(yyruleno==97); +{ yylhsminor.yy520 = setDatabaseOption(pCxt, yymsp[-2].minor.yy520, DB_OPTION_KEEP, yymsp[0].minor.yy904); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 95: /* db_options ::= db_options PAGES NK_INTEGER */ -#line 203 "sql.y" -{ yylhsminor.yy164 = setDatabaseOption(pCxt, yymsp[-2].minor.yy164, DB_OPTION_PAGES, &yymsp[0].minor.yy0); } -#line 4970 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 98: /* db_options ::= db_options PAGES NK_INTEGER */ +{ yylhsminor.yy520 = setDatabaseOption(pCxt, yymsp[-2].minor.yy520, DB_OPTION_PAGES, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 96: /* db_options ::= db_options PAGESIZE NK_INTEGER */ -#line 204 "sql.y" -{ yylhsminor.yy164 = setDatabaseOption(pCxt, yymsp[-2].minor.yy164, DB_OPTION_PAGESIZE, &yymsp[0].minor.yy0); } -#line 4976 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 99: /* db_options ::= db_options PAGESIZE NK_INTEGER */ +{ yylhsminor.yy520 = setDatabaseOption(pCxt, yymsp[-2].minor.yy520, DB_OPTION_PAGESIZE, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 97: /* db_options ::= db_options TSDB_PAGESIZE NK_INTEGER */ -#line 205 "sql.y" -{ yylhsminor.yy164 = setDatabaseOption(pCxt, yymsp[-2].minor.yy164, DB_OPTION_TSDB_PAGESIZE, &yymsp[0].minor.yy0); } -#line 4982 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 100: /* db_options ::= db_options TSDB_PAGESIZE NK_INTEGER */ +{ yylhsminor.yy520 = setDatabaseOption(pCxt, yymsp[-2].minor.yy520, DB_OPTION_TSDB_PAGESIZE, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 98: /* db_options ::= db_options PRECISION NK_STRING */ -#line 206 "sql.y" -{ yylhsminor.yy164 = setDatabaseOption(pCxt, yymsp[-2].minor.yy164, DB_OPTION_PRECISION, &yymsp[0].minor.yy0); } -#line 4988 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 101: /* db_options ::= db_options PRECISION NK_STRING */ +{ yylhsminor.yy520 = setDatabaseOption(pCxt, yymsp[-2].minor.yy520, DB_OPTION_PRECISION, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 99: /* db_options ::= db_options REPLICA NK_INTEGER */ -#line 207 "sql.y" -{ yylhsminor.yy164 = setDatabaseOption(pCxt, yymsp[-2].minor.yy164, DB_OPTION_REPLICA, &yymsp[0].minor.yy0); } -#line 4994 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 102: /* db_options ::= db_options REPLICA NK_INTEGER */ +{ yylhsminor.yy520 = setDatabaseOption(pCxt, yymsp[-2].minor.yy520, DB_OPTION_REPLICA, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 100: /* db_options ::= db_options VGROUPS NK_INTEGER */ -#line 209 "sql.y" -{ yylhsminor.yy164 = setDatabaseOption(pCxt, yymsp[-2].minor.yy164, DB_OPTION_VGROUPS, &yymsp[0].minor.yy0); } -#line 5000 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 103: /* db_options ::= db_options VGROUPS NK_INTEGER */ +{ yylhsminor.yy520 = setDatabaseOption(pCxt, yymsp[-2].minor.yy520, DB_OPTION_VGROUPS, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 101: /* db_options ::= db_options SINGLE_STABLE NK_INTEGER */ -#line 210 "sql.y" -{ yylhsminor.yy164 = setDatabaseOption(pCxt, yymsp[-2].minor.yy164, DB_OPTION_SINGLE_STABLE, &yymsp[0].minor.yy0); } -#line 5006 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 104: /* db_options ::= db_options SINGLE_STABLE NK_INTEGER */ +{ yylhsminor.yy520 = setDatabaseOption(pCxt, yymsp[-2].minor.yy520, DB_OPTION_SINGLE_STABLE, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 102: /* db_options ::= db_options RETENTIONS retention_list */ -#line 211 "sql.y" -{ yylhsminor.yy164 = setDatabaseOption(pCxt, yymsp[-2].minor.yy164, DB_OPTION_RETENTIONS, yymsp[0].minor.yy72); } -#line 5012 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 105: /* db_options ::= db_options RETENTIONS retention_list */ +{ yylhsminor.yy520 = setDatabaseOption(pCxt, yymsp[-2].minor.yy520, DB_OPTION_RETENTIONS, yymsp[0].minor.yy904); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 103: /* db_options ::= db_options SCHEMALESS NK_INTEGER */ -#line 212 "sql.y" -{ yylhsminor.yy164 = setDatabaseOption(pCxt, yymsp[-2].minor.yy164, DB_OPTION_SCHEMALESS, &yymsp[0].minor.yy0); } -#line 5018 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 106: /* db_options ::= db_options SCHEMALESS NK_INTEGER */ +{ yylhsminor.yy520 = setDatabaseOption(pCxt, yymsp[-2].minor.yy520, DB_OPTION_SCHEMALESS, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 104: /* db_options ::= db_options WAL_LEVEL NK_INTEGER */ -#line 213 "sql.y" -{ yylhsminor.yy164 = setDatabaseOption(pCxt, yymsp[-2].minor.yy164, DB_OPTION_WAL, &yymsp[0].minor.yy0); } -#line 5024 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 107: /* db_options ::= db_options WAL_LEVEL NK_INTEGER */ +{ yylhsminor.yy520 = setDatabaseOption(pCxt, yymsp[-2].minor.yy520, DB_OPTION_WAL, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 105: /* db_options ::= db_options WAL_FSYNC_PERIOD NK_INTEGER */ -#line 214 "sql.y" -{ yylhsminor.yy164 = setDatabaseOption(pCxt, yymsp[-2].minor.yy164, DB_OPTION_FSYNC, &yymsp[0].minor.yy0); } -#line 5030 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 108: /* db_options ::= db_options WAL_FSYNC_PERIOD NK_INTEGER */ +{ yylhsminor.yy520 = setDatabaseOption(pCxt, yymsp[-2].minor.yy520, DB_OPTION_FSYNC, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 106: /* db_options ::= db_options WAL_RETENTION_PERIOD NK_INTEGER */ -#line 215 "sql.y" -{ yylhsminor.yy164 = setDatabaseOption(pCxt, yymsp[-2].minor.yy164, DB_OPTION_WAL_RETENTION_PERIOD, &yymsp[0].minor.yy0); } -#line 5036 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 109: /* db_options ::= db_options WAL_RETENTION_PERIOD NK_INTEGER */ +{ yylhsminor.yy520 = setDatabaseOption(pCxt, yymsp[-2].minor.yy520, DB_OPTION_WAL_RETENTION_PERIOD, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 107: /* db_options ::= db_options WAL_RETENTION_PERIOD NK_MINUS NK_INTEGER */ -#line 216 "sql.y" + case 110: /* db_options ::= db_options WAL_RETENTION_PERIOD 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.yy164 = setDatabaseOption(pCxt, yymsp[-3].minor.yy164, DB_OPTION_WAL_RETENTION_PERIOD, &t); + yylhsminor.yy520 = setDatabaseOption(pCxt, yymsp[-3].minor.yy520, DB_OPTION_WAL_RETENTION_PERIOD, &t); } -#line 5046 "sql.c" - yymsp[-3].minor.yy164 = yylhsminor.yy164; + yymsp[-3].minor.yy520 = yylhsminor.yy520; break; - case 108: /* db_options ::= db_options WAL_RETENTION_SIZE NK_INTEGER */ -#line 221 "sql.y" -{ yylhsminor.yy164 = setDatabaseOption(pCxt, yymsp[-2].minor.yy164, DB_OPTION_WAL_RETENTION_SIZE, &yymsp[0].minor.yy0); } -#line 5052 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 111: /* db_options ::= db_options WAL_RETENTION_SIZE NK_INTEGER */ +{ yylhsminor.yy520 = setDatabaseOption(pCxt, yymsp[-2].minor.yy520, DB_OPTION_WAL_RETENTION_SIZE, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 109: /* db_options ::= db_options WAL_RETENTION_SIZE NK_MINUS NK_INTEGER */ -#line 222 "sql.y" + case 112: /* db_options ::= db_options WAL_RETENTION_SIZE 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.yy164 = setDatabaseOption(pCxt, yymsp[-3].minor.yy164, DB_OPTION_WAL_RETENTION_SIZE, &t); + yylhsminor.yy520 = setDatabaseOption(pCxt, yymsp[-3].minor.yy520, DB_OPTION_WAL_RETENTION_SIZE, &t); } -#line 5062 "sql.c" - yymsp[-3].minor.yy164 = yylhsminor.yy164; + yymsp[-3].minor.yy520 = yylhsminor.yy520; break; - case 110: /* db_options ::= db_options WAL_ROLL_PERIOD NK_INTEGER */ -#line 227 "sql.y" -{ yylhsminor.yy164 = setDatabaseOption(pCxt, yymsp[-2].minor.yy164, DB_OPTION_WAL_ROLL_PERIOD, &yymsp[0].minor.yy0); } -#line 5068 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 113: /* db_options ::= db_options WAL_ROLL_PERIOD NK_INTEGER */ +{ yylhsminor.yy520 = setDatabaseOption(pCxt, yymsp[-2].minor.yy520, DB_OPTION_WAL_ROLL_PERIOD, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 111: /* db_options ::= db_options WAL_SEGMENT_SIZE NK_INTEGER */ -#line 228 "sql.y" -{ yylhsminor.yy164 = setDatabaseOption(pCxt, yymsp[-2].minor.yy164, DB_OPTION_WAL_SEGMENT_SIZE, &yymsp[0].minor.yy0); } -#line 5074 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 114: /* db_options ::= db_options WAL_SEGMENT_SIZE NK_INTEGER */ +{ yylhsminor.yy520 = setDatabaseOption(pCxt, yymsp[-2].minor.yy520, DB_OPTION_WAL_SEGMENT_SIZE, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 112: /* db_options ::= db_options STT_TRIGGER NK_INTEGER */ -#line 229 "sql.y" -{ yylhsminor.yy164 = setDatabaseOption(pCxt, yymsp[-2].minor.yy164, DB_OPTION_STT_TRIGGER, &yymsp[0].minor.yy0); } -#line 5080 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 115: /* db_options ::= db_options STT_TRIGGER NK_INTEGER */ +{ yylhsminor.yy520 = setDatabaseOption(pCxt, yymsp[-2].minor.yy520, DB_OPTION_STT_TRIGGER, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 113: /* db_options ::= db_options TABLE_PREFIX signed */ -#line 230 "sql.y" -{ yylhsminor.yy164 = setDatabaseOption(pCxt, yymsp[-2].minor.yy164, DB_OPTION_TABLE_PREFIX, yymsp[0].minor.yy164); } -#line 5086 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 116: /* db_options ::= db_options TABLE_PREFIX signed */ +{ yylhsminor.yy520 = setDatabaseOption(pCxt, yymsp[-2].minor.yy520, DB_OPTION_TABLE_PREFIX, yymsp[0].minor.yy520); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 114: /* db_options ::= db_options TABLE_SUFFIX signed */ -#line 231 "sql.y" -{ yylhsminor.yy164 = setDatabaseOption(pCxt, yymsp[-2].minor.yy164, DB_OPTION_TABLE_SUFFIX, yymsp[0].minor.yy164); } -#line 5092 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 117: /* db_options ::= db_options TABLE_SUFFIX signed */ +{ yylhsminor.yy520 = setDatabaseOption(pCxt, yymsp[-2].minor.yy520, DB_OPTION_TABLE_SUFFIX, yymsp[0].minor.yy520); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 115: /* alter_db_options ::= alter_db_option */ -#line 233 "sql.y" -{ yylhsminor.yy164 = createAlterDatabaseOptions(pCxt); yylhsminor.yy164 = setAlterDatabaseOption(pCxt, yylhsminor.yy164, &yymsp[0].minor.yy761); } -#line 5098 "sql.c" - yymsp[0].minor.yy164 = yylhsminor.yy164; + case 118: /* alter_db_options ::= alter_db_option */ +{ yylhsminor.yy520 = createAlterDatabaseOptions(pCxt); yylhsminor.yy520 = setAlterDatabaseOption(pCxt, yylhsminor.yy520, &yymsp[0].minor.yy805); } + yymsp[0].minor.yy520 = yylhsminor.yy520; break; - case 116: /* alter_db_options ::= alter_db_options alter_db_option */ -#line 234 "sql.y" -{ yylhsminor.yy164 = setAlterDatabaseOption(pCxt, yymsp[-1].minor.yy164, &yymsp[0].minor.yy761); } -#line 5104 "sql.c" - yymsp[-1].minor.yy164 = yylhsminor.yy164; + case 119: /* alter_db_options ::= alter_db_options alter_db_option */ +{ yylhsminor.yy520 = setAlterDatabaseOption(pCxt, yymsp[-1].minor.yy520, &yymsp[0].minor.yy805); } + yymsp[-1].minor.yy520 = yylhsminor.yy520; break; - case 117: /* alter_db_option ::= BUFFER NK_INTEGER */ -#line 238 "sql.y" -{ yymsp[-1].minor.yy761.type = DB_OPTION_BUFFER; yymsp[-1].minor.yy761.val = yymsp[0].minor.yy0; } -#line 5110 "sql.c" + case 120: /* alter_db_option ::= BUFFER NK_INTEGER */ +{ yymsp[-1].minor.yy805.type = DB_OPTION_BUFFER; yymsp[-1].minor.yy805.val = yymsp[0].minor.yy0; } break; - case 118: /* alter_db_option ::= CACHEMODEL NK_STRING */ -#line 239 "sql.y" -{ yymsp[-1].minor.yy761.type = DB_OPTION_CACHEMODEL; yymsp[-1].minor.yy761.val = yymsp[0].minor.yy0; } -#line 5115 "sql.c" + case 121: /* alter_db_option ::= CACHEMODEL NK_STRING */ +{ yymsp[-1].minor.yy805.type = DB_OPTION_CACHEMODEL; yymsp[-1].minor.yy805.val = yymsp[0].minor.yy0; } break; - case 119: /* alter_db_option ::= CACHESIZE NK_INTEGER */ -#line 240 "sql.y" -{ yymsp[-1].minor.yy761.type = DB_OPTION_CACHESIZE; yymsp[-1].minor.yy761.val = yymsp[0].minor.yy0; } -#line 5120 "sql.c" + case 122: /* alter_db_option ::= CACHESIZE NK_INTEGER */ +{ yymsp[-1].minor.yy805.type = DB_OPTION_CACHESIZE; yymsp[-1].minor.yy805.val = yymsp[0].minor.yy0; } break; - case 120: /* alter_db_option ::= WAL_FSYNC_PERIOD NK_INTEGER */ -#line 241 "sql.y" -{ yymsp[-1].minor.yy761.type = DB_OPTION_FSYNC; yymsp[-1].minor.yy761.val = yymsp[0].minor.yy0; } -#line 5125 "sql.c" + case 123: /* alter_db_option ::= WAL_FSYNC_PERIOD NK_INTEGER */ +{ yymsp[-1].minor.yy805.type = DB_OPTION_FSYNC; yymsp[-1].minor.yy805.val = yymsp[0].minor.yy0; } break; - case 121: /* alter_db_option ::= KEEP integer_list */ - case 122: /* alter_db_option ::= KEEP variable_list */ yytestcase(yyruleno==122); -#line 242 "sql.y" -{ yymsp[-1].minor.yy761.type = DB_OPTION_KEEP; yymsp[-1].minor.yy761.pList = yymsp[0].minor.yy72; } -#line 5131 "sql.c" + case 124: /* alter_db_option ::= KEEP integer_list */ + case 125: /* alter_db_option ::= KEEP variable_list */ yytestcase(yyruleno==125); +{ yymsp[-1].minor.yy805.type = DB_OPTION_KEEP; yymsp[-1].minor.yy805.pList = yymsp[0].minor.yy904; } break; - case 123: /* alter_db_option ::= PAGES NK_INTEGER */ -#line 244 "sql.y" -{ yymsp[-1].minor.yy761.type = DB_OPTION_PAGES; yymsp[-1].minor.yy761.val = yymsp[0].minor.yy0; } -#line 5136 "sql.c" + case 126: /* alter_db_option ::= PAGES NK_INTEGER */ +{ yymsp[-1].minor.yy805.type = DB_OPTION_PAGES; yymsp[-1].minor.yy805.val = yymsp[0].minor.yy0; } break; - case 124: /* alter_db_option ::= REPLICA NK_INTEGER */ -#line 245 "sql.y" -{ yymsp[-1].minor.yy761.type = DB_OPTION_REPLICA; yymsp[-1].minor.yy761.val = yymsp[0].minor.yy0; } -#line 5141 "sql.c" + case 127: /* alter_db_option ::= REPLICA NK_INTEGER */ +{ yymsp[-1].minor.yy805.type = DB_OPTION_REPLICA; yymsp[-1].minor.yy805.val = yymsp[0].minor.yy0; } break; - case 125: /* alter_db_option ::= WAL_LEVEL NK_INTEGER */ -#line 247 "sql.y" -{ yymsp[-1].minor.yy761.type = DB_OPTION_WAL; yymsp[-1].minor.yy761.val = yymsp[0].minor.yy0; } -#line 5146 "sql.c" + case 128: /* alter_db_option ::= WAL_LEVEL NK_INTEGER */ +{ yymsp[-1].minor.yy805.type = DB_OPTION_WAL; yymsp[-1].minor.yy805.val = yymsp[0].minor.yy0; } break; - case 126: /* alter_db_option ::= STT_TRIGGER NK_INTEGER */ -#line 248 "sql.y" -{ yymsp[-1].minor.yy761.type = DB_OPTION_STT_TRIGGER; yymsp[-1].minor.yy761.val = yymsp[0].minor.yy0; } -#line 5151 "sql.c" + case 129: /* alter_db_option ::= STT_TRIGGER NK_INTEGER */ +{ yymsp[-1].minor.yy805.type = DB_OPTION_STT_TRIGGER; yymsp[-1].minor.yy805.val = yymsp[0].minor.yy0; } break; - case 127: /* alter_db_option ::= MINROWS NK_INTEGER */ -#line 249 "sql.y" -{ yymsp[-1].minor.yy761.type = DB_OPTION_MINROWS; yymsp[-1].minor.yy761.val = yymsp[0].minor.yy0; } -#line 5156 "sql.c" + case 130: /* alter_db_option ::= MINROWS NK_INTEGER */ +{ yymsp[-1].minor.yy805.type = DB_OPTION_MINROWS; yymsp[-1].minor.yy805.val = yymsp[0].minor.yy0; } break; - case 128: /* alter_db_option ::= WAL_RETENTION_PERIOD NK_INTEGER */ -#line 250 "sql.y" -{ yymsp[-1].minor.yy761.type = DB_OPTION_WAL_RETENTION_PERIOD; yymsp[-1].minor.yy761.val = yymsp[0].minor.yy0; } -#line 5161 "sql.c" + case 131: /* alter_db_option ::= WAL_RETENTION_PERIOD NK_INTEGER */ +{ yymsp[-1].minor.yy805.type = DB_OPTION_WAL_RETENTION_PERIOD; yymsp[-1].minor.yy805.val = yymsp[0].minor.yy0; } break; - case 129: /* alter_db_option ::= WAL_RETENTION_PERIOD NK_MINUS NK_INTEGER */ -#line 251 "sql.y" + case 132: /* alter_db_option ::= WAL_RETENTION_PERIOD 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; - yymsp[-2].minor.yy761.type = DB_OPTION_WAL_RETENTION_PERIOD; yymsp[-2].minor.yy761.val = t; + yymsp[-2].minor.yy805.type = DB_OPTION_WAL_RETENTION_PERIOD; yymsp[-2].minor.yy805.val = t; } -#line 5170 "sql.c" break; - case 130: /* alter_db_option ::= WAL_RETENTION_SIZE NK_INTEGER */ -#line 256 "sql.y" -{ yymsp[-1].minor.yy761.type = DB_OPTION_WAL_RETENTION_SIZE; yymsp[-1].minor.yy761.val = yymsp[0].minor.yy0; } -#line 5175 "sql.c" + case 133: /* alter_db_option ::= WAL_RETENTION_SIZE NK_INTEGER */ +{ yymsp[-1].minor.yy805.type = DB_OPTION_WAL_RETENTION_SIZE; yymsp[-1].minor.yy805.val = yymsp[0].minor.yy0; } break; - case 131: /* alter_db_option ::= WAL_RETENTION_SIZE NK_MINUS NK_INTEGER */ -#line 257 "sql.y" + case 134: /* alter_db_option ::= WAL_RETENTION_SIZE 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; - yymsp[-2].minor.yy761.type = DB_OPTION_WAL_RETENTION_SIZE; yymsp[-2].minor.yy761.val = t; + yymsp[-2].minor.yy805.type = DB_OPTION_WAL_RETENTION_SIZE; yymsp[-2].minor.yy805.val = t; } -#line 5184 "sql.c" - break; - case 132: /* integer_list ::= NK_INTEGER */ -#line 265 "sql.y" -{ yylhsminor.yy72 = createNodeList(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0)); } -#line 5189 "sql.c" - yymsp[0].minor.yy72 = yylhsminor.yy72; - break; - case 133: /* integer_list ::= integer_list NK_COMMA NK_INTEGER */ - case 359: /* dnode_list ::= dnode_list DNODE NK_INTEGER */ yytestcase(yyruleno==359); -#line 266 "sql.y" -{ yylhsminor.yy72 = addNodeToList(pCxt, yymsp[-2].minor.yy72, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0)); } -#line 5196 "sql.c" - yymsp[-2].minor.yy72 = yylhsminor.yy72; - break; - case 134: /* variable_list ::= NK_VARIABLE */ -#line 270 "sql.y" -{ yylhsminor.yy72 = createNodeList(pCxt, createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } -#line 5202 "sql.c" - yymsp[0].minor.yy72 = yylhsminor.yy72; - break; - case 135: /* variable_list ::= variable_list NK_COMMA NK_VARIABLE */ -#line 271 "sql.y" -{ yylhsminor.yy72 = addNodeToList(pCxt, yymsp[-2].minor.yy72, createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } -#line 5208 "sql.c" - yymsp[-2].minor.yy72 = yylhsminor.yy72; - break; - case 136: /* retention_list ::= retention */ - case 166: /* multi_create_clause ::= create_subtable_clause */ yytestcase(yyruleno==166); - case 169: /* multi_drop_clause ::= drop_table_clause */ yytestcase(yyruleno==169); - case 176: /* column_def_list ::= column_def */ yytestcase(yyruleno==176); - case 219: /* rollup_func_list ::= rollup_func_name */ yytestcase(yyruleno==219); - case 224: /* col_name_list ::= col_name */ yytestcase(yyruleno==224); - case 275: /* tag_list_opt ::= tag_item */ yytestcase(yyruleno==275); - case 289: /* func_list ::= func */ yytestcase(yyruleno==289); - case 388: /* literal_list ::= signed_literal */ yytestcase(yyruleno==388); - case 455: /* other_para_list ::= star_func_para */ yytestcase(yyruleno==455); - case 461: /* when_then_list ::= when_then_expr */ yytestcase(yyruleno==461); - case 516: /* select_list ::= select_item */ yytestcase(yyruleno==516); - case 527: /* partition_list ::= partition_item */ yytestcase(yyruleno==527); - case 582: /* sort_specification_list ::= sort_specification */ yytestcase(yyruleno==582); -#line 275 "sql.y" -{ yylhsminor.yy72 = createNodeList(pCxt, yymsp[0].minor.yy164); } -#line 5227 "sql.c" - yymsp[0].minor.yy72 = yylhsminor.yy72; - break; - case 137: /* retention_list ::= retention_list NK_COMMA retention */ - case 170: /* multi_drop_clause ::= multi_drop_clause NK_COMMA drop_table_clause */ yytestcase(yyruleno==170); - case 177: /* column_def_list ::= column_def_list NK_COMMA column_def */ yytestcase(yyruleno==177); - case 220: /* rollup_func_list ::= rollup_func_list NK_COMMA rollup_func_name */ yytestcase(yyruleno==220); - case 225: /* col_name_list ::= col_name_list NK_COMMA col_name */ yytestcase(yyruleno==225); - case 276: /* tag_list_opt ::= tag_list_opt NK_COMMA tag_item */ yytestcase(yyruleno==276); - case 290: /* func_list ::= func_list NK_COMMA func */ yytestcase(yyruleno==290); - case 389: /* literal_list ::= literal_list NK_COMMA signed_literal */ yytestcase(yyruleno==389); - case 456: /* other_para_list ::= other_para_list NK_COMMA star_func_para */ yytestcase(yyruleno==456); - case 517: /* select_list ::= select_list NK_COMMA select_item */ yytestcase(yyruleno==517); - case 528: /* partition_list ::= partition_list NK_COMMA partition_item */ yytestcase(yyruleno==528); - case 583: /* sort_specification_list ::= sort_specification_list NK_COMMA sort_specification */ yytestcase(yyruleno==583); -#line 276 "sql.y" -{ yylhsminor.yy72 = addNodeToList(pCxt, yymsp[-2].minor.yy72, yymsp[0].minor.yy164); } -#line 5244 "sql.c" - yymsp[-2].minor.yy72 = yylhsminor.yy72; - break; - case 138: /* retention ::= NK_VARIABLE NK_COLON NK_VARIABLE */ -#line 278 "sql.y" -{ yylhsminor.yy164 = createNodeListNodeEx(pCxt, createDurationValueNode(pCxt, &yymsp[-2].minor.yy0), createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } -#line 5250 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; - break; - case 139: /* speed_opt ::= */ - case 322: /* bufsize_opt ::= */ yytestcase(yyruleno==322); -#line 282 "sql.y" -{ yymsp[1].minor.yy560 = 0; } -#line 5257 "sql.c" - break; - case 140: /* speed_opt ::= MAX_SPEED NK_INTEGER */ - case 323: /* bufsize_opt ::= BUFSIZE NK_INTEGER */ yytestcase(yyruleno==323); -#line 283 "sql.y" -{ yymsp[-1].minor.yy560 = taosStr2Int32(yymsp[0].minor.yy0.z, NULL, 10); } -#line 5263 "sql.c" - break; - case 142: /* start_opt ::= START WITH NK_INTEGER */ - case 146: /* end_opt ::= END WITH NK_INTEGER */ yytestcase(yyruleno==146); -#line 286 "sql.y" -{ yymsp[-2].minor.yy164 = createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0); } -#line 5269 "sql.c" - break; - case 143: /* start_opt ::= START WITH NK_STRING */ - case 147: /* end_opt ::= END WITH NK_STRING */ yytestcase(yyruleno==147); -#line 287 "sql.y" -{ yymsp[-2].minor.yy164 = createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0); } -#line 5275 "sql.c" - break; - case 144: /* start_opt ::= START WITH TIMESTAMP NK_STRING */ - case 148: /* end_opt ::= END WITH TIMESTAMP NK_STRING */ yytestcase(yyruleno==148); -#line 288 "sql.y" -{ yymsp[-3].minor.yy164 = createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0); } -#line 5281 "sql.c" - break; - case 149: /* cmd ::= CREATE TABLE not_exists_opt full_table_name NK_LP column_def_list NK_RP tags_def_opt table_options */ - case 151: /* cmd ::= CREATE STABLE not_exists_opt full_table_name NK_LP column_def_list NK_RP tags_def table_options */ yytestcase(yyruleno==151); -#line 297 "sql.y" -{ pCxt->pRootNode = createCreateTableStmt(pCxt, yymsp[-6].minor.yy441, yymsp[-5].minor.yy164, yymsp[-3].minor.yy72, yymsp[-1].minor.yy72, yymsp[0].minor.yy164); } -#line 5287 "sql.c" - break; - case 150: /* cmd ::= CREATE TABLE multi_create_clause */ -#line 298 "sql.y" -{ pCxt->pRootNode = createCreateMultiTableStmt(pCxt, yymsp[0].minor.yy72); } -#line 5292 "sql.c" - break; - case 152: /* cmd ::= DROP TABLE multi_drop_clause */ -#line 301 "sql.y" -{ pCxt->pRootNode = createDropTableStmt(pCxt, yymsp[0].minor.yy72); } -#line 5297 "sql.c" - break; - case 153: /* cmd ::= DROP STABLE exists_opt full_table_name */ -#line 302 "sql.y" -{ pCxt->pRootNode = createDropSuperTableStmt(pCxt, yymsp[-1].minor.yy441, yymsp[0].minor.yy164); } -#line 5302 "sql.c" - break; - case 154: /* cmd ::= ALTER TABLE alter_table_clause */ - case 361: /* cmd ::= query_or_subquery */ yytestcase(yyruleno==361); - case 362: /* cmd ::= insert_query */ yytestcase(yyruleno==362); -#line 304 "sql.y" -{ pCxt->pRootNode = yymsp[0].minor.yy164; } -#line 5309 "sql.c" - break; - case 155: /* cmd ::= ALTER STABLE alter_table_clause */ -#line 305 "sql.y" -{ pCxt->pRootNode = setAlterSuperTableType(yymsp[0].minor.yy164); } -#line 5314 "sql.c" - break; - case 156: /* alter_table_clause ::= full_table_name alter_table_options */ -#line 307 "sql.y" -{ yylhsminor.yy164 = createAlterTableModifyOptions(pCxt, yymsp[-1].minor.yy164, yymsp[0].minor.yy164); } -#line 5319 "sql.c" - yymsp[-1].minor.yy164 = yylhsminor.yy164; - break; - case 157: /* alter_table_clause ::= full_table_name ADD COLUMN column_name type_name */ -#line 309 "sql.y" -{ yylhsminor.yy164 = createAlterTableAddModifyCol(pCxt, yymsp[-4].minor.yy164, TSDB_ALTER_TABLE_ADD_COLUMN, &yymsp[-1].minor.yy497, yymsp[0].minor.yy700); } -#line 5325 "sql.c" - yymsp[-4].minor.yy164 = yylhsminor.yy164; - break; - case 158: /* alter_table_clause ::= full_table_name DROP COLUMN column_name */ -#line 310 "sql.y" -{ yylhsminor.yy164 = createAlterTableDropCol(pCxt, yymsp[-3].minor.yy164, TSDB_ALTER_TABLE_DROP_COLUMN, &yymsp[0].minor.yy497); } -#line 5331 "sql.c" - yymsp[-3].minor.yy164 = yylhsminor.yy164; - break; - case 159: /* alter_table_clause ::= full_table_name MODIFY COLUMN column_name type_name */ -#line 312 "sql.y" -{ yylhsminor.yy164 = createAlterTableAddModifyCol(pCxt, yymsp[-4].minor.yy164, TSDB_ALTER_TABLE_UPDATE_COLUMN_BYTES, &yymsp[-1].minor.yy497, yymsp[0].minor.yy700); } -#line 5337 "sql.c" - yymsp[-4].minor.yy164 = yylhsminor.yy164; - break; - case 160: /* alter_table_clause ::= full_table_name RENAME COLUMN column_name column_name */ -#line 314 "sql.y" -{ yylhsminor.yy164 = createAlterTableRenameCol(pCxt, yymsp[-4].minor.yy164, TSDB_ALTER_TABLE_UPDATE_COLUMN_NAME, &yymsp[-1].minor.yy497, &yymsp[0].minor.yy497); } -#line 5343 "sql.c" - yymsp[-4].minor.yy164 = yylhsminor.yy164; - break; - case 161: /* alter_table_clause ::= full_table_name ADD TAG column_name type_name */ -#line 316 "sql.y" -{ yylhsminor.yy164 = createAlterTableAddModifyCol(pCxt, yymsp[-4].minor.yy164, TSDB_ALTER_TABLE_ADD_TAG, &yymsp[-1].minor.yy497, yymsp[0].minor.yy700); } -#line 5349 "sql.c" - yymsp[-4].minor.yy164 = yylhsminor.yy164; - break; - case 162: /* alter_table_clause ::= full_table_name DROP TAG column_name */ -#line 317 "sql.y" -{ yylhsminor.yy164 = createAlterTableDropCol(pCxt, yymsp[-3].minor.yy164, TSDB_ALTER_TABLE_DROP_TAG, &yymsp[0].minor.yy497); } -#line 5355 "sql.c" - yymsp[-3].minor.yy164 = yylhsminor.yy164; - break; - case 163: /* alter_table_clause ::= full_table_name MODIFY TAG column_name type_name */ -#line 319 "sql.y" -{ yylhsminor.yy164 = createAlterTableAddModifyCol(pCxt, yymsp[-4].minor.yy164, TSDB_ALTER_TABLE_UPDATE_TAG_BYTES, &yymsp[-1].minor.yy497, yymsp[0].minor.yy700); } -#line 5361 "sql.c" - yymsp[-4].minor.yy164 = yylhsminor.yy164; - break; - case 164: /* alter_table_clause ::= full_table_name RENAME TAG column_name column_name */ -#line 321 "sql.y" -{ yylhsminor.yy164 = createAlterTableRenameCol(pCxt, yymsp[-4].minor.yy164, TSDB_ALTER_TABLE_UPDATE_TAG_NAME, &yymsp[-1].minor.yy497, &yymsp[0].minor.yy497); } -#line 5367 "sql.c" - yymsp[-4].minor.yy164 = yylhsminor.yy164; - break; - case 165: /* alter_table_clause ::= full_table_name SET TAG column_name NK_EQ signed_literal */ -#line 323 "sql.y" -{ yylhsminor.yy164 = createAlterTableSetTag(pCxt, yymsp[-5].minor.yy164, &yymsp[-2].minor.yy497, yymsp[0].minor.yy164); } -#line 5373 "sql.c" - yymsp[-5].minor.yy164 = yylhsminor.yy164; - break; - case 167: /* multi_create_clause ::= multi_create_clause create_subtable_clause */ - case 462: /* when_then_list ::= when_then_list when_then_expr */ yytestcase(yyruleno==462); -#line 328 "sql.y" -{ yylhsminor.yy72 = addNodeToList(pCxt, yymsp[-1].minor.yy72, yymsp[0].minor.yy164); } -#line 5380 "sql.c" - yymsp[-1].minor.yy72 = yylhsminor.yy72; - break; - case 168: /* create_subtable_clause ::= not_exists_opt full_table_name USING full_table_name specific_cols_opt TAGS NK_LP expression_list NK_RP table_options */ -#line 332 "sql.y" -{ yylhsminor.yy164 = createCreateSubTableClause(pCxt, yymsp[-9].minor.yy441, yymsp[-8].minor.yy164, yymsp[-6].minor.yy164, yymsp[-5].minor.yy72, yymsp[-2].minor.yy72, yymsp[0].minor.yy164); } -#line 5386 "sql.c" - yymsp[-9].minor.yy164 = yylhsminor.yy164; - break; - case 171: /* drop_table_clause ::= exists_opt full_table_name */ -#line 339 "sql.y" -{ yylhsminor.yy164 = createDropTableClause(pCxt, yymsp[-1].minor.yy441, yymsp[0].minor.yy164); } -#line 5392 "sql.c" - yymsp[-1].minor.yy164 = yylhsminor.yy164; - break; - case 172: /* specific_cols_opt ::= */ - case 202: /* tags_def_opt ::= */ yytestcase(yyruleno==202); - case 274: /* tag_list_opt ::= */ yytestcase(yyruleno==274); - case 332: /* col_list_opt ::= */ yytestcase(yyruleno==332); - case 334: /* tag_def_or_ref_opt ::= */ yytestcase(yyruleno==334); - case 525: /* partition_by_clause_opt ::= */ yytestcase(yyruleno==525); - case 550: /* group_by_clause_opt ::= */ yytestcase(yyruleno==550); - case 569: /* order_by_clause_opt ::= */ yytestcase(yyruleno==569); -#line 343 "sql.y" -{ yymsp[1].minor.yy72 = NULL; } -#line 5405 "sql.c" - break; - case 173: /* specific_cols_opt ::= NK_LP col_name_list NK_RP */ - case 333: /* col_list_opt ::= NK_LP col_name_list NK_RP */ yytestcase(yyruleno==333); -#line 344 "sql.y" -{ yymsp[-2].minor.yy72 = yymsp[-1].minor.yy72; } -#line 5411 "sql.c" - break; - case 174: /* full_table_name ::= table_name */ -#line 346 "sql.y" -{ yylhsminor.yy164 = createRealTableNode(pCxt, NULL, &yymsp[0].minor.yy497, NULL); } -#line 5416 "sql.c" - yymsp[0].minor.yy164 = yylhsminor.yy164; - break; - case 175: /* full_table_name ::= db_name NK_DOT table_name */ -#line 347 "sql.y" -{ yylhsminor.yy164 = createRealTableNode(pCxt, &yymsp[-2].minor.yy497, &yymsp[0].minor.yy497, NULL); } -#line 5422 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; - break; - case 178: /* column_def ::= column_name type_name */ -#line 354 "sql.y" -{ yylhsminor.yy164 = createColumnDefNode(pCxt, &yymsp[-1].minor.yy497, yymsp[0].minor.yy700, NULL); } -#line 5428 "sql.c" - yymsp[-1].minor.yy164 = yylhsminor.yy164; - break; - case 179: /* type_name ::= BOOL */ -#line 359 "sql.y" -{ yymsp[0].minor.yy700 = createDataType(TSDB_DATA_TYPE_BOOL); } -#line 5434 "sql.c" - break; - case 180: /* type_name ::= TINYINT */ -#line 360 "sql.y" -{ yymsp[0].minor.yy700 = createDataType(TSDB_DATA_TYPE_TINYINT); } -#line 5439 "sql.c" - break; - case 181: /* type_name ::= SMALLINT */ -#line 361 "sql.y" -{ yymsp[0].minor.yy700 = createDataType(TSDB_DATA_TYPE_SMALLINT); } -#line 5444 "sql.c" - break; - case 182: /* type_name ::= INT */ - case 183: /* type_name ::= INTEGER */ yytestcase(yyruleno==183); -#line 362 "sql.y" -{ yymsp[0].minor.yy700 = createDataType(TSDB_DATA_TYPE_INT); } -#line 5450 "sql.c" - break; - case 184: /* type_name ::= BIGINT */ -#line 364 "sql.y" -{ yymsp[0].minor.yy700 = createDataType(TSDB_DATA_TYPE_BIGINT); } -#line 5455 "sql.c" - break; - case 185: /* type_name ::= FLOAT */ -#line 365 "sql.y" -{ yymsp[0].minor.yy700 = createDataType(TSDB_DATA_TYPE_FLOAT); } -#line 5460 "sql.c" - break; - case 186: /* type_name ::= DOUBLE */ -#line 366 "sql.y" -{ yymsp[0].minor.yy700 = createDataType(TSDB_DATA_TYPE_DOUBLE); } -#line 5465 "sql.c" - break; - case 187: /* type_name ::= BINARY NK_LP NK_INTEGER NK_RP */ -#line 367 "sql.y" -{ yymsp[-3].minor.yy700 = createVarLenDataType(TSDB_DATA_TYPE_BINARY, &yymsp[-1].minor.yy0); } -#line 5470 "sql.c" - break; - case 188: /* type_name ::= TIMESTAMP */ -#line 368 "sql.y" -{ yymsp[0].minor.yy700 = createDataType(TSDB_DATA_TYPE_TIMESTAMP); } -#line 5475 "sql.c" - break; - case 189: /* type_name ::= NCHAR NK_LP NK_INTEGER NK_RP */ -#line 369 "sql.y" -{ yymsp[-3].minor.yy700 = createVarLenDataType(TSDB_DATA_TYPE_NCHAR, &yymsp[-1].minor.yy0); } -#line 5480 "sql.c" - break; - case 190: /* type_name ::= TINYINT UNSIGNED */ -#line 370 "sql.y" -{ yymsp[-1].minor.yy700 = createDataType(TSDB_DATA_TYPE_UTINYINT); } -#line 5485 "sql.c" - break; - case 191: /* type_name ::= SMALLINT UNSIGNED */ -#line 371 "sql.y" -{ yymsp[-1].minor.yy700 = createDataType(TSDB_DATA_TYPE_USMALLINT); } -#line 5490 "sql.c" - break; - case 192: /* type_name ::= INT UNSIGNED */ -#line 372 "sql.y" -{ yymsp[-1].minor.yy700 = createDataType(TSDB_DATA_TYPE_UINT); } -#line 5495 "sql.c" - break; - case 193: /* type_name ::= BIGINT UNSIGNED */ -#line 373 "sql.y" -{ yymsp[-1].minor.yy700 = createDataType(TSDB_DATA_TYPE_UBIGINT); } -#line 5500 "sql.c" - break; - case 194: /* type_name ::= JSON */ -#line 374 "sql.y" -{ yymsp[0].minor.yy700 = createDataType(TSDB_DATA_TYPE_JSON); } -#line 5505 "sql.c" - break; - case 195: /* type_name ::= VARCHAR NK_LP NK_INTEGER NK_RP */ -#line 375 "sql.y" -{ yymsp[-3].minor.yy700 = createVarLenDataType(TSDB_DATA_TYPE_VARCHAR, &yymsp[-1].minor.yy0); } -#line 5510 "sql.c" - break; - case 196: /* type_name ::= MEDIUMBLOB */ -#line 376 "sql.y" -{ yymsp[0].minor.yy700 = createDataType(TSDB_DATA_TYPE_MEDIUMBLOB); } -#line 5515 "sql.c" - break; - case 197: /* type_name ::= BLOB */ -#line 377 "sql.y" -{ yymsp[0].minor.yy700 = createDataType(TSDB_DATA_TYPE_BLOB); } -#line 5520 "sql.c" - break; - case 198: /* type_name ::= VARBINARY NK_LP NK_INTEGER NK_RP */ -#line 378 "sql.y" -{ yymsp[-3].minor.yy700 = createVarLenDataType(TSDB_DATA_TYPE_VARBINARY, &yymsp[-1].minor.yy0); } -#line 5525 "sql.c" - break; - case 199: /* type_name ::= DECIMAL */ -#line 379 "sql.y" -{ yymsp[0].minor.yy700 = createDataType(TSDB_DATA_TYPE_DECIMAL); } -#line 5530 "sql.c" - break; - case 200: /* type_name ::= DECIMAL NK_LP NK_INTEGER NK_RP */ -#line 380 "sql.y" -{ yymsp[-3].minor.yy700 = createDataType(TSDB_DATA_TYPE_DECIMAL); } -#line 5535 "sql.c" - break; - case 201: /* type_name ::= DECIMAL NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP */ -#line 381 "sql.y" -{ yymsp[-5].minor.yy700 = createDataType(TSDB_DATA_TYPE_DECIMAL); } -#line 5540 "sql.c" - break; - case 203: /* tags_def_opt ::= tags_def */ - case 335: /* tag_def_or_ref_opt ::= tags_def */ yytestcase(yyruleno==335); - case 454: /* star_func_para_list ::= other_para_list */ yytestcase(yyruleno==454); -#line 386 "sql.y" -{ yylhsminor.yy72 = yymsp[0].minor.yy72; } -#line 5547 "sql.c" - yymsp[0].minor.yy72 = yylhsminor.yy72; - break; - case 204: /* tags_def ::= TAGS NK_LP column_def_list NK_RP */ - case 336: /* tag_def_or_ref_opt ::= TAGS NK_LP col_name_list NK_RP */ yytestcase(yyruleno==336); -#line 390 "sql.y" -{ yymsp[-3].minor.yy72 = yymsp[-1].minor.yy72; } -#line 5554 "sql.c" - break; - case 205: /* table_options ::= */ -#line 392 "sql.y" -{ yymsp[1].minor.yy164 = createDefaultTableOptions(pCxt); } -#line 5559 "sql.c" - break; - case 206: /* table_options ::= table_options COMMENT NK_STRING */ -#line 393 "sql.y" -{ yylhsminor.yy164 = setTableOption(pCxt, yymsp[-2].minor.yy164, TABLE_OPTION_COMMENT, &yymsp[0].minor.yy0); } -#line 5564 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; - break; - case 207: /* table_options ::= table_options MAX_DELAY duration_list */ -#line 394 "sql.y" -{ yylhsminor.yy164 = setTableOption(pCxt, yymsp[-2].minor.yy164, TABLE_OPTION_MAXDELAY, yymsp[0].minor.yy72); } -#line 5570 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; - break; - case 208: /* table_options ::= table_options WATERMARK duration_list */ -#line 395 "sql.y" -{ yylhsminor.yy164 = setTableOption(pCxt, yymsp[-2].minor.yy164, TABLE_OPTION_WATERMARK, yymsp[0].minor.yy72); } -#line 5576 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; - break; - case 209: /* table_options ::= table_options ROLLUP NK_LP rollup_func_list NK_RP */ -#line 396 "sql.y" -{ yylhsminor.yy164 = setTableOption(pCxt, yymsp[-4].minor.yy164, TABLE_OPTION_ROLLUP, yymsp[-1].minor.yy72); } -#line 5582 "sql.c" - yymsp[-4].minor.yy164 = yylhsminor.yy164; - break; - case 210: /* table_options ::= table_options TTL NK_INTEGER */ -#line 397 "sql.y" -{ yylhsminor.yy164 = setTableOption(pCxt, yymsp[-2].minor.yy164, TABLE_OPTION_TTL, &yymsp[0].minor.yy0); } -#line 5588 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; - break; - case 211: /* table_options ::= table_options SMA NK_LP col_name_list NK_RP */ -#line 398 "sql.y" -{ yylhsminor.yy164 = setTableOption(pCxt, yymsp[-4].minor.yy164, TABLE_OPTION_SMA, yymsp[-1].minor.yy72); } -#line 5594 "sql.c" - yymsp[-4].minor.yy164 = yylhsminor.yy164; - break; - case 212: /* table_options ::= table_options DELETE_MARK duration_list */ -#line 399 "sql.y" -{ yylhsminor.yy164 = setTableOption(pCxt, yymsp[-2].minor.yy164, TABLE_OPTION_DELETE_MARK, yymsp[0].minor.yy72); } -#line 5600 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; - break; - case 213: /* alter_table_options ::= alter_table_option */ -#line 401 "sql.y" -{ yylhsminor.yy164 = createAlterTableOptions(pCxt); yylhsminor.yy164 = setTableOption(pCxt, yylhsminor.yy164, yymsp[0].minor.yy761.type, &yymsp[0].minor.yy761.val); } -#line 5606 "sql.c" - yymsp[0].minor.yy164 = yylhsminor.yy164; - break; - case 214: /* alter_table_options ::= alter_table_options alter_table_option */ -#line 402 "sql.y" -{ yylhsminor.yy164 = setTableOption(pCxt, yymsp[-1].minor.yy164, yymsp[0].minor.yy761.type, &yymsp[0].minor.yy761.val); } -#line 5612 "sql.c" - yymsp[-1].minor.yy164 = yylhsminor.yy164; - break; - case 215: /* alter_table_option ::= COMMENT NK_STRING */ -#line 406 "sql.y" -{ yymsp[-1].minor.yy761.type = TABLE_OPTION_COMMENT; yymsp[-1].minor.yy761.val = yymsp[0].minor.yy0; } -#line 5618 "sql.c" - break; - case 216: /* alter_table_option ::= TTL NK_INTEGER */ -#line 407 "sql.y" -{ yymsp[-1].minor.yy761.type = TABLE_OPTION_TTL; yymsp[-1].minor.yy761.val = yymsp[0].minor.yy0; } -#line 5623 "sql.c" - break; - case 217: /* duration_list ::= duration_literal */ - case 418: /* expression_list ::= expr_or_subquery */ yytestcase(yyruleno==418); -#line 411 "sql.y" -{ yylhsminor.yy72 = createNodeList(pCxt, releaseRawExprNode(pCxt, yymsp[0].minor.yy164)); } -#line 5629 "sql.c" - yymsp[0].minor.yy72 = yylhsminor.yy72; - break; - case 218: /* duration_list ::= duration_list NK_COMMA duration_literal */ - case 419: /* expression_list ::= expression_list NK_COMMA expr_or_subquery */ yytestcase(yyruleno==419); -#line 412 "sql.y" -{ yylhsminor.yy72 = addNodeToList(pCxt, yymsp[-2].minor.yy72, releaseRawExprNode(pCxt, yymsp[0].minor.yy164)); } -#line 5636 "sql.c" - yymsp[-2].minor.yy72 = yylhsminor.yy72; - break; - case 221: /* rollup_func_name ::= function_name */ -#line 419 "sql.y" -{ yylhsminor.yy164 = createFunctionNode(pCxt, &yymsp[0].minor.yy497, NULL); } -#line 5642 "sql.c" - yymsp[0].minor.yy164 = yylhsminor.yy164; - break; - case 222: /* rollup_func_name ::= FIRST */ - case 223: /* rollup_func_name ::= LAST */ yytestcase(yyruleno==223); - case 278: /* tag_item ::= QTAGS */ yytestcase(yyruleno==278); -#line 420 "sql.y" -{ yylhsminor.yy164 = createFunctionNode(pCxt, &yymsp[0].minor.yy0, NULL); } -#line 5650 "sql.c" - yymsp[0].minor.yy164 = yylhsminor.yy164; - break; - case 226: /* col_name ::= column_name */ - case 279: /* tag_item ::= column_name */ yytestcase(yyruleno==279); -#line 428 "sql.y" -{ yylhsminor.yy164 = createColumnNode(pCxt, NULL, &yymsp[0].minor.yy497); } -#line 5657 "sql.c" - yymsp[0].minor.yy164 = yylhsminor.yy164; - break; - case 227: /* cmd ::= SHOW DNODES */ -#line 431 "sql.y" + break; + case 135: /* integer_list ::= NK_INTEGER */ +{ yylhsminor.yy904 = createNodeList(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy904 = yylhsminor.yy904; + break; + case 136: /* integer_list ::= integer_list NK_COMMA NK_INTEGER */ + case 362: /* dnode_list ::= dnode_list DNODE NK_INTEGER */ yytestcase(yyruleno==362); +{ yylhsminor.yy904 = addNodeToList(pCxt, yymsp[-2].minor.yy904, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0)); } + yymsp[-2].minor.yy904 = yylhsminor.yy904; + break; + case 137: /* variable_list ::= NK_VARIABLE */ +{ yylhsminor.yy904 = createNodeList(pCxt, createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy904 = yylhsminor.yy904; + break; + case 138: /* variable_list ::= variable_list NK_COMMA NK_VARIABLE */ +{ yylhsminor.yy904 = addNodeToList(pCxt, yymsp[-2].minor.yy904, createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } + yymsp[-2].minor.yy904 = yylhsminor.yy904; + break; + case 139: /* retention_list ::= retention */ + case 169: /* multi_create_clause ::= create_subtable_clause */ yytestcase(yyruleno==169); + case 172: /* multi_drop_clause ::= drop_table_clause */ yytestcase(yyruleno==172); + case 179: /* column_def_list ::= column_def */ yytestcase(yyruleno==179); + case 222: /* rollup_func_list ::= rollup_func_name */ yytestcase(yyruleno==222); + case 227: /* col_name_list ::= col_name */ yytestcase(yyruleno==227); + case 278: /* tag_list_opt ::= tag_item */ yytestcase(yyruleno==278); + case 292: /* func_list ::= func */ yytestcase(yyruleno==292); + case 391: /* literal_list ::= signed_literal */ yytestcase(yyruleno==391); + case 458: /* other_para_list ::= star_func_para */ yytestcase(yyruleno==458); + case 464: /* when_then_list ::= when_then_expr */ yytestcase(yyruleno==464); + case 519: /* select_list ::= select_item */ yytestcase(yyruleno==519); + case 530: /* partition_list ::= partition_item */ yytestcase(yyruleno==530); + case 585: /* sort_specification_list ::= sort_specification */ yytestcase(yyruleno==585); +{ yylhsminor.yy904 = createNodeList(pCxt, yymsp[0].minor.yy520); } + yymsp[0].minor.yy904 = yylhsminor.yy904; + break; + case 140: /* retention_list ::= retention_list NK_COMMA retention */ + case 173: /* multi_drop_clause ::= multi_drop_clause NK_COMMA drop_table_clause */ yytestcase(yyruleno==173); + case 180: /* column_def_list ::= column_def_list NK_COMMA column_def */ yytestcase(yyruleno==180); + case 223: /* rollup_func_list ::= rollup_func_list NK_COMMA rollup_func_name */ yytestcase(yyruleno==223); + case 228: /* col_name_list ::= col_name_list NK_COMMA col_name */ yytestcase(yyruleno==228); + case 279: /* tag_list_opt ::= tag_list_opt NK_COMMA tag_item */ yytestcase(yyruleno==279); + case 293: /* func_list ::= func_list NK_COMMA func */ yytestcase(yyruleno==293); + case 392: /* literal_list ::= literal_list NK_COMMA signed_literal */ yytestcase(yyruleno==392); + case 459: /* other_para_list ::= other_para_list NK_COMMA star_func_para */ yytestcase(yyruleno==459); + case 520: /* select_list ::= select_list NK_COMMA select_item */ yytestcase(yyruleno==520); + case 531: /* partition_list ::= partition_list NK_COMMA partition_item */ yytestcase(yyruleno==531); + case 586: /* sort_specification_list ::= sort_specification_list NK_COMMA sort_specification */ yytestcase(yyruleno==586); +{ yylhsminor.yy904 = addNodeToList(pCxt, yymsp[-2].minor.yy904, yymsp[0].minor.yy520); } + yymsp[-2].minor.yy904 = yylhsminor.yy904; + break; + case 141: /* retention ::= NK_VARIABLE NK_COLON NK_VARIABLE */ +{ yylhsminor.yy520 = createNodeListNodeEx(pCxt, createDurationValueNode(pCxt, &yymsp[-2].minor.yy0), createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; + break; + case 142: /* speed_opt ::= */ + case 325: /* bufsize_opt ::= */ yytestcase(yyruleno==325); +{ yymsp[1].minor.yy860 = 0; } + break; + case 143: /* speed_opt ::= MAX_SPEED NK_INTEGER */ + case 326: /* bufsize_opt ::= BUFSIZE NK_INTEGER */ yytestcase(yyruleno==326); +{ yymsp[-1].minor.yy860 = taosStr2Int32(yymsp[0].minor.yy0.z, NULL, 10); } + break; + case 145: /* start_opt ::= START WITH NK_INTEGER */ + case 149: /* end_opt ::= END WITH NK_INTEGER */ yytestcase(yyruleno==149); +{ yymsp[-2].minor.yy520 = createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0); } + break; + case 146: /* start_opt ::= START WITH NK_STRING */ + case 150: /* end_opt ::= END WITH NK_STRING */ yytestcase(yyruleno==150); +{ yymsp[-2].minor.yy520 = createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0); } + break; + case 147: /* start_opt ::= START WITH TIMESTAMP NK_STRING */ + case 151: /* end_opt ::= END WITH TIMESTAMP NK_STRING */ yytestcase(yyruleno==151); +{ yymsp[-3].minor.yy520 = createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0); } + break; + case 152: /* cmd ::= CREATE TABLE not_exists_opt full_table_name NK_LP column_def_list NK_RP tags_def_opt table_options */ + case 154: /* cmd ::= CREATE STABLE not_exists_opt full_table_name NK_LP column_def_list NK_RP tags_def table_options */ yytestcase(yyruleno==154); +{ pCxt->pRootNode = createCreateTableStmt(pCxt, yymsp[-6].minor.yy537, yymsp[-5].minor.yy520, yymsp[-3].minor.yy904, yymsp[-1].minor.yy904, yymsp[0].minor.yy520); } + break; + case 153: /* cmd ::= CREATE TABLE multi_create_clause */ +{ pCxt->pRootNode = createCreateMultiTableStmt(pCxt, yymsp[0].minor.yy904); } + break; + case 155: /* cmd ::= DROP TABLE multi_drop_clause */ +{ pCxt->pRootNode = createDropTableStmt(pCxt, yymsp[0].minor.yy904); } + break; + case 156: /* cmd ::= DROP STABLE exists_opt full_table_name */ +{ pCxt->pRootNode = createDropSuperTableStmt(pCxt, yymsp[-1].minor.yy537, yymsp[0].minor.yy520); } + break; + case 157: /* cmd ::= ALTER TABLE alter_table_clause */ + case 364: /* cmd ::= query_or_subquery */ yytestcase(yyruleno==364); + case 365: /* cmd ::= insert_query */ yytestcase(yyruleno==365); +{ pCxt->pRootNode = yymsp[0].minor.yy520; } + break; + case 158: /* cmd ::= ALTER STABLE alter_table_clause */ +{ pCxt->pRootNode = setAlterSuperTableType(yymsp[0].minor.yy520); } + break; + case 159: /* alter_table_clause ::= full_table_name alter_table_options */ +{ yylhsminor.yy520 = createAlterTableModifyOptions(pCxt, yymsp[-1].minor.yy520, yymsp[0].minor.yy520); } + yymsp[-1].minor.yy520 = yylhsminor.yy520; + break; + case 160: /* alter_table_clause ::= full_table_name ADD COLUMN column_name type_name */ +{ yylhsminor.yy520 = createAlterTableAddModifyCol(pCxt, yymsp[-4].minor.yy520, TSDB_ALTER_TABLE_ADD_COLUMN, &yymsp[-1].minor.yy833, yymsp[0].minor.yy840); } + yymsp[-4].minor.yy520 = yylhsminor.yy520; + break; + case 161: /* alter_table_clause ::= full_table_name DROP COLUMN column_name */ +{ yylhsminor.yy520 = createAlterTableDropCol(pCxt, yymsp[-3].minor.yy520, TSDB_ALTER_TABLE_DROP_COLUMN, &yymsp[0].minor.yy833); } + yymsp[-3].minor.yy520 = yylhsminor.yy520; + break; + case 162: /* alter_table_clause ::= full_table_name MODIFY COLUMN column_name type_name */ +{ yylhsminor.yy520 = createAlterTableAddModifyCol(pCxt, yymsp[-4].minor.yy520, TSDB_ALTER_TABLE_UPDATE_COLUMN_BYTES, &yymsp[-1].minor.yy833, yymsp[0].minor.yy840); } + yymsp[-4].minor.yy520 = yylhsminor.yy520; + break; + case 163: /* alter_table_clause ::= full_table_name RENAME COLUMN column_name column_name */ +{ yylhsminor.yy520 = createAlterTableRenameCol(pCxt, yymsp[-4].minor.yy520, TSDB_ALTER_TABLE_UPDATE_COLUMN_NAME, &yymsp[-1].minor.yy833, &yymsp[0].minor.yy833); } + yymsp[-4].minor.yy520 = yylhsminor.yy520; + break; + case 164: /* alter_table_clause ::= full_table_name ADD TAG column_name type_name */ +{ yylhsminor.yy520 = createAlterTableAddModifyCol(pCxt, yymsp[-4].minor.yy520, TSDB_ALTER_TABLE_ADD_TAG, &yymsp[-1].minor.yy833, yymsp[0].minor.yy840); } + yymsp[-4].minor.yy520 = yylhsminor.yy520; + break; + case 165: /* alter_table_clause ::= full_table_name DROP TAG column_name */ +{ yylhsminor.yy520 = createAlterTableDropCol(pCxt, yymsp[-3].minor.yy520, TSDB_ALTER_TABLE_DROP_TAG, &yymsp[0].minor.yy833); } + yymsp[-3].minor.yy520 = yylhsminor.yy520; + break; + case 166: /* alter_table_clause ::= full_table_name MODIFY TAG column_name type_name */ +{ yylhsminor.yy520 = createAlterTableAddModifyCol(pCxt, yymsp[-4].minor.yy520, TSDB_ALTER_TABLE_UPDATE_TAG_BYTES, &yymsp[-1].minor.yy833, yymsp[0].minor.yy840); } + yymsp[-4].minor.yy520 = yylhsminor.yy520; + break; + case 167: /* alter_table_clause ::= full_table_name RENAME TAG column_name column_name */ +{ yylhsminor.yy520 = createAlterTableRenameCol(pCxt, yymsp[-4].minor.yy520, TSDB_ALTER_TABLE_UPDATE_TAG_NAME, &yymsp[-1].minor.yy833, &yymsp[0].minor.yy833); } + yymsp[-4].minor.yy520 = yylhsminor.yy520; + break; + case 168: /* alter_table_clause ::= full_table_name SET TAG column_name NK_EQ signed_literal */ +{ yylhsminor.yy520 = createAlterTableSetTag(pCxt, yymsp[-5].minor.yy520, &yymsp[-2].minor.yy833, yymsp[0].minor.yy520); } + yymsp[-5].minor.yy520 = yylhsminor.yy520; + break; + case 170: /* multi_create_clause ::= multi_create_clause create_subtable_clause */ + case 465: /* when_then_list ::= when_then_list when_then_expr */ yytestcase(yyruleno==465); +{ yylhsminor.yy904 = addNodeToList(pCxt, yymsp[-1].minor.yy904, yymsp[0].minor.yy520); } + yymsp[-1].minor.yy904 = yylhsminor.yy904; + break; + case 171: /* create_subtable_clause ::= not_exists_opt full_table_name USING full_table_name specific_cols_opt TAGS NK_LP expression_list NK_RP table_options */ +{ yylhsminor.yy520 = createCreateSubTableClause(pCxt, yymsp[-9].minor.yy537, yymsp[-8].minor.yy520, yymsp[-6].minor.yy520, yymsp[-5].minor.yy904, yymsp[-2].minor.yy904, yymsp[0].minor.yy520); } + yymsp[-9].minor.yy520 = yylhsminor.yy520; + break; + case 174: /* drop_table_clause ::= exists_opt full_table_name */ +{ yylhsminor.yy520 = createDropTableClause(pCxt, yymsp[-1].minor.yy537, yymsp[0].minor.yy520); } + yymsp[-1].minor.yy520 = yylhsminor.yy520; + break; + case 175: /* specific_cols_opt ::= */ + case 205: /* tags_def_opt ::= */ yytestcase(yyruleno==205); + case 277: /* tag_list_opt ::= */ yytestcase(yyruleno==277); + case 335: /* col_list_opt ::= */ yytestcase(yyruleno==335); + case 337: /* tag_def_or_ref_opt ::= */ yytestcase(yyruleno==337); + case 528: /* partition_by_clause_opt ::= */ yytestcase(yyruleno==528); + case 553: /* group_by_clause_opt ::= */ yytestcase(yyruleno==553); + case 572: /* order_by_clause_opt ::= */ yytestcase(yyruleno==572); +{ yymsp[1].minor.yy904 = NULL; } + break; + case 176: /* specific_cols_opt ::= NK_LP col_name_list NK_RP */ + case 336: /* col_list_opt ::= NK_LP col_name_list NK_RP */ yytestcase(yyruleno==336); +{ yymsp[-2].minor.yy904 = yymsp[-1].minor.yy904; } + break; + case 177: /* full_table_name ::= table_name */ +{ yylhsminor.yy520 = createRealTableNode(pCxt, NULL, &yymsp[0].minor.yy833, NULL); } + yymsp[0].minor.yy520 = yylhsminor.yy520; + break; + case 178: /* full_table_name ::= db_name NK_DOT table_name */ +{ yylhsminor.yy520 = createRealTableNode(pCxt, &yymsp[-2].minor.yy833, &yymsp[0].minor.yy833, NULL); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; + break; + case 181: /* column_def ::= column_name type_name */ +{ yylhsminor.yy520 = createColumnDefNode(pCxt, &yymsp[-1].minor.yy833, yymsp[0].minor.yy840, NULL); } + yymsp[-1].minor.yy520 = yylhsminor.yy520; + break; + case 182: /* type_name ::= BOOL */ +{ yymsp[0].minor.yy840 = createDataType(TSDB_DATA_TYPE_BOOL); } + break; + case 183: /* type_name ::= TINYINT */ +{ yymsp[0].minor.yy840 = createDataType(TSDB_DATA_TYPE_TINYINT); } + break; + case 184: /* type_name ::= SMALLINT */ +{ yymsp[0].minor.yy840 = createDataType(TSDB_DATA_TYPE_SMALLINT); } + break; + case 185: /* type_name ::= INT */ + case 186: /* type_name ::= INTEGER */ yytestcase(yyruleno==186); +{ yymsp[0].minor.yy840 = createDataType(TSDB_DATA_TYPE_INT); } + break; + case 187: /* type_name ::= BIGINT */ +{ yymsp[0].minor.yy840 = createDataType(TSDB_DATA_TYPE_BIGINT); } + break; + case 188: /* type_name ::= FLOAT */ +{ yymsp[0].minor.yy840 = createDataType(TSDB_DATA_TYPE_FLOAT); } + break; + case 189: /* type_name ::= DOUBLE */ +{ yymsp[0].minor.yy840 = createDataType(TSDB_DATA_TYPE_DOUBLE); } + break; + case 190: /* type_name ::= BINARY NK_LP NK_INTEGER NK_RP */ +{ yymsp[-3].minor.yy840 = createVarLenDataType(TSDB_DATA_TYPE_BINARY, &yymsp[-1].minor.yy0); } + break; + case 191: /* type_name ::= TIMESTAMP */ +{ yymsp[0].minor.yy840 = createDataType(TSDB_DATA_TYPE_TIMESTAMP); } + break; + case 192: /* type_name ::= NCHAR NK_LP NK_INTEGER NK_RP */ +{ yymsp[-3].minor.yy840 = createVarLenDataType(TSDB_DATA_TYPE_NCHAR, &yymsp[-1].minor.yy0); } + break; + case 193: /* type_name ::= TINYINT UNSIGNED */ +{ yymsp[-1].minor.yy840 = createDataType(TSDB_DATA_TYPE_UTINYINT); } + break; + case 194: /* type_name ::= SMALLINT UNSIGNED */ +{ yymsp[-1].minor.yy840 = createDataType(TSDB_DATA_TYPE_USMALLINT); } + break; + case 195: /* type_name ::= INT UNSIGNED */ +{ yymsp[-1].minor.yy840 = createDataType(TSDB_DATA_TYPE_UINT); } + break; + case 196: /* type_name ::= BIGINT UNSIGNED */ +{ yymsp[-1].minor.yy840 = createDataType(TSDB_DATA_TYPE_UBIGINT); } + break; + case 197: /* type_name ::= JSON */ +{ yymsp[0].minor.yy840 = createDataType(TSDB_DATA_TYPE_JSON); } + break; + case 198: /* type_name ::= VARCHAR NK_LP NK_INTEGER NK_RP */ +{ yymsp[-3].minor.yy840 = createVarLenDataType(TSDB_DATA_TYPE_VARCHAR, &yymsp[-1].minor.yy0); } + break; + case 199: /* type_name ::= MEDIUMBLOB */ +{ yymsp[0].minor.yy840 = createDataType(TSDB_DATA_TYPE_MEDIUMBLOB); } + break; + case 200: /* type_name ::= BLOB */ +{ yymsp[0].minor.yy840 = createDataType(TSDB_DATA_TYPE_BLOB); } + break; + case 201: /* type_name ::= VARBINARY NK_LP NK_INTEGER NK_RP */ +{ yymsp[-3].minor.yy840 = createVarLenDataType(TSDB_DATA_TYPE_VARBINARY, &yymsp[-1].minor.yy0); } + break; + case 202: /* type_name ::= DECIMAL */ +{ yymsp[0].minor.yy840 = createDataType(TSDB_DATA_TYPE_DECIMAL); } + break; + case 203: /* type_name ::= DECIMAL NK_LP NK_INTEGER NK_RP */ +{ yymsp[-3].minor.yy840 = createDataType(TSDB_DATA_TYPE_DECIMAL); } + break; + case 204: /* type_name ::= DECIMAL NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP */ +{ yymsp[-5].minor.yy840 = createDataType(TSDB_DATA_TYPE_DECIMAL); } + break; + case 206: /* tags_def_opt ::= tags_def */ + case 338: /* tag_def_or_ref_opt ::= tags_def */ yytestcase(yyruleno==338); + case 457: /* star_func_para_list ::= other_para_list */ yytestcase(yyruleno==457); +{ yylhsminor.yy904 = yymsp[0].minor.yy904; } + yymsp[0].minor.yy904 = yylhsminor.yy904; + break; + case 207: /* tags_def ::= TAGS NK_LP column_def_list NK_RP */ + case 339: /* tag_def_or_ref_opt ::= TAGS NK_LP col_name_list NK_RP */ yytestcase(yyruleno==339); +{ yymsp[-3].minor.yy904 = yymsp[-1].minor.yy904; } + break; + case 208: /* table_options ::= */ +{ yymsp[1].minor.yy520 = createDefaultTableOptions(pCxt); } + break; + case 209: /* table_options ::= table_options COMMENT NK_STRING */ +{ yylhsminor.yy520 = setTableOption(pCxt, yymsp[-2].minor.yy520, TABLE_OPTION_COMMENT, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; + break; + case 210: /* table_options ::= table_options MAX_DELAY duration_list */ +{ yylhsminor.yy520 = setTableOption(pCxt, yymsp[-2].minor.yy520, TABLE_OPTION_MAXDELAY, yymsp[0].minor.yy904); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; + break; + case 211: /* table_options ::= table_options WATERMARK duration_list */ +{ yylhsminor.yy520 = setTableOption(pCxt, yymsp[-2].minor.yy520, TABLE_OPTION_WATERMARK, yymsp[0].minor.yy904); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; + break; + case 212: /* table_options ::= table_options ROLLUP NK_LP rollup_func_list NK_RP */ +{ yylhsminor.yy520 = setTableOption(pCxt, yymsp[-4].minor.yy520, TABLE_OPTION_ROLLUP, yymsp[-1].minor.yy904); } + yymsp[-4].minor.yy520 = yylhsminor.yy520; + break; + case 213: /* table_options ::= table_options TTL NK_INTEGER */ +{ yylhsminor.yy520 = setTableOption(pCxt, yymsp[-2].minor.yy520, TABLE_OPTION_TTL, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; + break; + case 214: /* table_options ::= table_options SMA NK_LP col_name_list NK_RP */ +{ yylhsminor.yy520 = setTableOption(pCxt, yymsp[-4].minor.yy520, TABLE_OPTION_SMA, yymsp[-1].minor.yy904); } + yymsp[-4].minor.yy520 = yylhsminor.yy520; + break; + case 215: /* table_options ::= table_options DELETE_MARK duration_list */ +{ yylhsminor.yy520 = setTableOption(pCxt, yymsp[-2].minor.yy520, TABLE_OPTION_DELETE_MARK, yymsp[0].minor.yy904); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; + break; + case 216: /* alter_table_options ::= alter_table_option */ +{ yylhsminor.yy520 = createAlterTableOptions(pCxt); yylhsminor.yy520 = setTableOption(pCxt, yylhsminor.yy520, yymsp[0].minor.yy805.type, &yymsp[0].minor.yy805.val); } + yymsp[0].minor.yy520 = yylhsminor.yy520; + break; + case 217: /* alter_table_options ::= alter_table_options alter_table_option */ +{ yylhsminor.yy520 = setTableOption(pCxt, yymsp[-1].minor.yy520, yymsp[0].minor.yy805.type, &yymsp[0].minor.yy805.val); } + yymsp[-1].minor.yy520 = yylhsminor.yy520; + break; + case 218: /* alter_table_option ::= COMMENT NK_STRING */ +{ yymsp[-1].minor.yy805.type = TABLE_OPTION_COMMENT; yymsp[-1].minor.yy805.val = yymsp[0].minor.yy0; } + break; + case 219: /* alter_table_option ::= TTL NK_INTEGER */ +{ yymsp[-1].minor.yy805.type = TABLE_OPTION_TTL; yymsp[-1].minor.yy805.val = yymsp[0].minor.yy0; } + break; + case 220: /* duration_list ::= duration_literal */ + case 421: /* expression_list ::= expr_or_subquery */ yytestcase(yyruleno==421); +{ yylhsminor.yy904 = createNodeList(pCxt, releaseRawExprNode(pCxt, yymsp[0].minor.yy520)); } + yymsp[0].minor.yy904 = yylhsminor.yy904; + break; + case 221: /* duration_list ::= duration_list NK_COMMA duration_literal */ + case 422: /* expression_list ::= expression_list NK_COMMA expr_or_subquery */ yytestcase(yyruleno==422); +{ yylhsminor.yy904 = addNodeToList(pCxt, yymsp[-2].minor.yy904, releaseRawExprNode(pCxt, yymsp[0].minor.yy520)); } + yymsp[-2].minor.yy904 = yylhsminor.yy904; + break; + case 224: /* rollup_func_name ::= function_name */ +{ yylhsminor.yy520 = createFunctionNode(pCxt, &yymsp[0].minor.yy833, NULL); } + yymsp[0].minor.yy520 = yylhsminor.yy520; + break; + case 225: /* rollup_func_name ::= FIRST */ + case 226: /* rollup_func_name ::= LAST */ yytestcase(yyruleno==226); + case 281: /* tag_item ::= QTAGS */ yytestcase(yyruleno==281); +{ yylhsminor.yy520 = createFunctionNode(pCxt, &yymsp[0].minor.yy0, NULL); } + yymsp[0].minor.yy520 = yylhsminor.yy520; + break; + case 229: /* col_name ::= column_name */ + case 282: /* tag_item ::= column_name */ yytestcase(yyruleno==282); +{ yylhsminor.yy520 = createColumnNode(pCxt, NULL, &yymsp[0].minor.yy833); } + yymsp[0].minor.yy520 = yylhsminor.yy520; + break; + case 230: /* cmd ::= SHOW DNODES */ { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_DNODES_STMT); } -#line 5663 "sql.c" break; - case 228: /* cmd ::= SHOW USERS */ -#line 432 "sql.y" + case 231: /* cmd ::= SHOW USERS */ { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_USERS_STMT); } -#line 5668 "sql.c" break; - case 229: /* cmd ::= SHOW USER PRIVILEGES */ -#line 433 "sql.y" + case 232: /* cmd ::= SHOW USER PRIVILEGES */ { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_USER_PRIVILEGES_STMT); } -#line 5673 "sql.c" break; - case 230: /* cmd ::= SHOW DATABASES */ -#line 434 "sql.y" + case 233: /* cmd ::= SHOW DATABASES */ { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_DATABASES_STMT); } -#line 5678 "sql.c" break; - case 231: /* cmd ::= SHOW db_name_cond_opt TABLES like_pattern_opt */ -#line 435 "sql.y" -{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_TABLES_STMT, yymsp[-2].minor.yy164, yymsp[0].minor.yy164, OP_TYPE_LIKE); } -#line 5683 "sql.c" + case 234: /* cmd ::= SHOW db_name_cond_opt TABLES like_pattern_opt */ +{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_TABLES_STMT, yymsp[-2].minor.yy520, yymsp[0].minor.yy520, OP_TYPE_LIKE); } break; - case 232: /* cmd ::= SHOW db_name_cond_opt STABLES like_pattern_opt */ -#line 436 "sql.y" -{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_STABLES_STMT, yymsp[-2].minor.yy164, yymsp[0].minor.yy164, OP_TYPE_LIKE); } -#line 5688 "sql.c" + case 235: /* cmd ::= SHOW db_name_cond_opt STABLES like_pattern_opt */ +{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_STABLES_STMT, yymsp[-2].minor.yy520, yymsp[0].minor.yy520, OP_TYPE_LIKE); } break; - case 233: /* cmd ::= SHOW db_name_cond_opt VGROUPS */ -#line 437 "sql.y" -{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_VGROUPS_STMT, yymsp[-1].minor.yy164, NULL, OP_TYPE_LIKE); } -#line 5693 "sql.c" + case 236: /* cmd ::= SHOW db_name_cond_opt VGROUPS */ +{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_VGROUPS_STMT, yymsp[-1].minor.yy520, NULL, OP_TYPE_LIKE); } break; - case 234: /* cmd ::= SHOW MNODES */ -#line 438 "sql.y" + case 237: /* cmd ::= SHOW MNODES */ { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_MNODES_STMT); } -#line 5698 "sql.c" break; - case 235: /* cmd ::= SHOW QNODES */ -#line 440 "sql.y" + case 238: /* cmd ::= SHOW QNODES */ { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_QNODES_STMT); } -#line 5703 "sql.c" break; - case 236: /* cmd ::= SHOW FUNCTIONS */ -#line 441 "sql.y" + case 239: /* cmd ::= SHOW FUNCTIONS */ { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_FUNCTIONS_STMT); } -#line 5708 "sql.c" break; - case 237: /* cmd ::= SHOW INDEXES FROM table_name_cond from_db_opt */ -#line 442 "sql.y" -{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_INDEXES_STMT, yymsp[0].minor.yy164, yymsp[-1].minor.yy164, OP_TYPE_EQUAL); } -#line 5713 "sql.c" + case 240: /* cmd ::= SHOW INDEXES FROM table_name_cond from_db_opt */ +{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_INDEXES_STMT, yymsp[0].minor.yy520, yymsp[-1].minor.yy520, OP_TYPE_EQUAL); } break; - case 238: /* cmd ::= SHOW STREAMS */ -#line 443 "sql.y" + case 241: /* cmd ::= SHOW STREAMS */ { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_STREAMS_STMT); } -#line 5718 "sql.c" break; - case 239: /* cmd ::= SHOW ACCOUNTS */ -#line 444 "sql.y" + case 242: /* cmd ::= SHOW ACCOUNTS */ { pCxt->errCode = generateSyntaxErrMsg(&pCxt->msgBuf, TSDB_CODE_PAR_EXPRIE_STATEMENT); } -#line 5723 "sql.c" break; - case 240: /* cmd ::= SHOW APPS */ -#line 445 "sql.y" + case 243: /* cmd ::= SHOW APPS */ { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_APPS_STMT); } -#line 5728 "sql.c" break; - case 241: /* cmd ::= SHOW CONNECTIONS */ -#line 446 "sql.y" + case 244: /* cmd ::= SHOW CONNECTIONS */ { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_CONNECTIONS_STMT); } -#line 5733 "sql.c" break; - case 242: /* cmd ::= SHOW LICENCES */ - case 243: /* cmd ::= SHOW GRANTS */ yytestcase(yyruleno==243); -#line 447 "sql.y" + case 245: /* cmd ::= SHOW LICENCES */ + case 246: /* cmd ::= SHOW GRANTS */ yytestcase(yyruleno==246); { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_LICENCES_STMT); } -#line 5739 "sql.c" break; - case 244: /* cmd ::= SHOW CREATE DATABASE db_name */ -#line 449 "sql.y" -{ pCxt->pRootNode = createShowCreateDatabaseStmt(pCxt, &yymsp[0].minor.yy497); } -#line 5744 "sql.c" + case 247: /* cmd ::= SHOW CREATE DATABASE db_name */ +{ pCxt->pRootNode = createShowCreateDatabaseStmt(pCxt, &yymsp[0].minor.yy833); } break; - case 245: /* cmd ::= SHOW CREATE TABLE full_table_name */ -#line 450 "sql.y" -{ pCxt->pRootNode = createShowCreateTableStmt(pCxt, QUERY_NODE_SHOW_CREATE_TABLE_STMT, yymsp[0].minor.yy164); } -#line 5749 "sql.c" + case 248: /* cmd ::= SHOW CREATE TABLE full_table_name */ +{ pCxt->pRootNode = createShowCreateTableStmt(pCxt, QUERY_NODE_SHOW_CREATE_TABLE_STMT, yymsp[0].minor.yy520); } break; - case 246: /* cmd ::= SHOW CREATE STABLE full_table_name */ -#line 451 "sql.y" -{ pCxt->pRootNode = createShowCreateTableStmt(pCxt, QUERY_NODE_SHOW_CREATE_STABLE_STMT, yymsp[0].minor.yy164); } -#line 5754 "sql.c" + case 249: /* cmd ::= SHOW CREATE STABLE full_table_name */ +{ pCxt->pRootNode = createShowCreateTableStmt(pCxt, QUERY_NODE_SHOW_CREATE_STABLE_STMT, yymsp[0].minor.yy520); } break; - case 247: /* cmd ::= SHOW QUERIES */ -#line 452 "sql.y" + case 250: /* cmd ::= SHOW QUERIES */ { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_QUERIES_STMT); } -#line 5759 "sql.c" break; - case 248: /* cmd ::= SHOW SCORES */ -#line 453 "sql.y" + case 251: /* cmd ::= SHOW SCORES */ { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_SCORES_STMT); } -#line 5764 "sql.c" break; - case 249: /* cmd ::= SHOW TOPICS */ -#line 454 "sql.y" + case 252: /* cmd ::= SHOW TOPICS */ { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_TOPICS_STMT); } -#line 5769 "sql.c" break; - case 250: /* cmd ::= SHOW VARIABLES */ - case 251: /* cmd ::= SHOW CLUSTER VARIABLES */ yytestcase(yyruleno==251); -#line 455 "sql.y" + case 253: /* cmd ::= SHOW VARIABLES */ + case 254: /* cmd ::= SHOW CLUSTER VARIABLES */ yytestcase(yyruleno==254); { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_VARIABLES_STMT); } -#line 5775 "sql.c" break; - case 252: /* cmd ::= SHOW LOCAL VARIABLES */ -#line 457 "sql.y" + case 255: /* cmd ::= SHOW LOCAL VARIABLES */ { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_LOCAL_VARIABLES_STMT); } -#line 5780 "sql.c" break; - case 253: /* cmd ::= SHOW DNODE NK_INTEGER VARIABLES like_pattern_opt */ -#line 458 "sql.y" -{ pCxt->pRootNode = createShowDnodeVariablesStmt(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[-2].minor.yy0), yymsp[0].minor.yy164); } -#line 5785 "sql.c" + case 256: /* cmd ::= SHOW DNODE NK_INTEGER VARIABLES like_pattern_opt */ +{ pCxt->pRootNode = createShowDnodeVariablesStmt(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[-2].minor.yy0), yymsp[0].minor.yy520); } break; - case 254: /* cmd ::= SHOW BNODES */ -#line 459 "sql.y" + case 257: /* cmd ::= SHOW BNODES */ { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_BNODES_STMT); } -#line 5790 "sql.c" break; - case 255: /* cmd ::= SHOW SNODES */ -#line 460 "sql.y" + case 258: /* cmd ::= SHOW SNODES */ { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_SNODES_STMT); } -#line 5795 "sql.c" break; - case 256: /* cmd ::= SHOW CLUSTER */ -#line 461 "sql.y" + case 259: /* cmd ::= SHOW CLUSTER */ { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_CLUSTER_STMT); } -#line 5800 "sql.c" break; - case 257: /* cmd ::= SHOW TRANSACTIONS */ -#line 462 "sql.y" + case 260: /* cmd ::= SHOW TRANSACTIONS */ { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_TRANSACTIONS_STMT); } -#line 5805 "sql.c" break; - case 258: /* cmd ::= SHOW TABLE DISTRIBUTED full_table_name */ -#line 463 "sql.y" -{ pCxt->pRootNode = createShowTableDistributedStmt(pCxt, yymsp[0].minor.yy164); } -#line 5810 "sql.c" + case 261: /* cmd ::= SHOW TABLE DISTRIBUTED full_table_name */ +{ pCxt->pRootNode = createShowTableDistributedStmt(pCxt, yymsp[0].minor.yy520); } break; - case 259: /* cmd ::= SHOW CONSUMERS */ -#line 464 "sql.y" + case 262: /* cmd ::= SHOW CONSUMERS */ { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_CONSUMERS_STMT); } -#line 5815 "sql.c" break; - case 260: /* cmd ::= SHOW SUBSCRIPTIONS */ -#line 465 "sql.y" + case 263: /* cmd ::= SHOW SUBSCRIPTIONS */ { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_SUBSCRIPTIONS_STMT); } -#line 5820 "sql.c" break; - case 261: /* cmd ::= SHOW TAGS FROM table_name_cond from_db_opt */ -#line 466 "sql.y" -{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_TAGS_STMT, yymsp[0].minor.yy164, yymsp[-1].minor.yy164, OP_TYPE_EQUAL); } -#line 5825 "sql.c" + case 264: /* cmd ::= SHOW TAGS FROM table_name_cond from_db_opt */ +{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_TAGS_STMT, yymsp[0].minor.yy520, yymsp[-1].minor.yy520, OP_TYPE_EQUAL); } break; - case 262: /* cmd ::= SHOW TABLE TAGS tag_list_opt FROM table_name_cond from_db_opt */ -#line 467 "sql.y" -{ pCxt->pRootNode = createShowTableTagsStmt(pCxt, yymsp[-1].minor.yy164, yymsp[0].minor.yy164, yymsp[-3].minor.yy72); } -#line 5830 "sql.c" + case 265: /* cmd ::= SHOW TABLE TAGS tag_list_opt FROM table_name_cond from_db_opt */ +{ pCxt->pRootNode = createShowTableTagsStmt(pCxt, yymsp[-1].minor.yy520, yymsp[0].minor.yy520, yymsp[-3].minor.yy904); } break; - case 263: /* cmd ::= SHOW VNODES NK_INTEGER */ -#line 468 "sql.y" + case 266: /* cmd ::= SHOW VNODES NK_INTEGER */ { pCxt->pRootNode = createShowVnodesStmt(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0), NULL); } -#line 5835 "sql.c" break; - case 264: /* cmd ::= SHOW VNODES NK_STRING */ -#line 469 "sql.y" + case 267: /* cmd ::= SHOW VNODES NK_STRING */ { pCxt->pRootNode = createShowVnodesStmt(pCxt, NULL, createValueNode(pCxt, TSDB_DATA_TYPE_VARCHAR, &yymsp[0].minor.yy0)); } -#line 5840 "sql.c" break; - case 265: /* cmd ::= SHOW db_name_cond_opt ALIVE */ -#line 471 "sql.y" -{ pCxt->pRootNode = createShowAliveStmt(pCxt, yymsp[-1].minor.yy164, QUERY_NODE_SHOW_DB_ALIVE_STMT); } -#line 5845 "sql.c" + case 268: /* cmd ::= SHOW db_name_cond_opt ALIVE */ +{ pCxt->pRootNode = createShowAliveStmt(pCxt, yymsp[-1].minor.yy520, QUERY_NODE_SHOW_DB_ALIVE_STMT); } break; - case 266: /* cmd ::= SHOW CLUSTER ALIVE */ -#line 472 "sql.y" + case 269: /* cmd ::= SHOW CLUSTER ALIVE */ { pCxt->pRootNode = createShowAliveStmt(pCxt, NULL, QUERY_NODE_SHOW_CLUSTER_ALIVE_STMT); } -#line 5850 "sql.c" break; - case 267: /* db_name_cond_opt ::= */ - case 272: /* from_db_opt ::= */ yytestcase(yyruleno==272); -#line 474 "sql.y" -{ yymsp[1].minor.yy164 = createDefaultDatabaseCondValue(pCxt); } -#line 5856 "sql.c" + case 270: /* db_name_cond_opt ::= */ + case 275: /* from_db_opt ::= */ yytestcase(yyruleno==275); +{ yymsp[1].minor.yy520 = createDefaultDatabaseCondValue(pCxt); } break; - case 268: /* db_name_cond_opt ::= db_name NK_DOT */ -#line 475 "sql.y" -{ yylhsminor.yy164 = createIdentifierValueNode(pCxt, &yymsp[-1].minor.yy497); } -#line 5861 "sql.c" - yymsp[-1].minor.yy164 = yylhsminor.yy164; + case 271: /* db_name_cond_opt ::= db_name NK_DOT */ +{ yylhsminor.yy520 = createIdentifierValueNode(pCxt, &yymsp[-1].minor.yy833); } + yymsp[-1].minor.yy520 = yylhsminor.yy520; break; - case 270: /* like_pattern_opt ::= LIKE NK_STRING */ -#line 478 "sql.y" -{ yymsp[-1].minor.yy164 = createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0); } -#line 5867 "sql.c" + case 273: /* like_pattern_opt ::= LIKE NK_STRING */ +{ yymsp[-1].minor.yy520 = createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0); } break; - case 271: /* table_name_cond ::= table_name */ -#line 480 "sql.y" -{ yylhsminor.yy164 = createIdentifierValueNode(pCxt, &yymsp[0].minor.yy497); } -#line 5872 "sql.c" - yymsp[0].minor.yy164 = yylhsminor.yy164; + case 274: /* table_name_cond ::= table_name */ +{ yylhsminor.yy520 = createIdentifierValueNode(pCxt, &yymsp[0].minor.yy833); } + yymsp[0].minor.yy520 = yylhsminor.yy520; break; - case 273: /* from_db_opt ::= FROM db_name */ -#line 483 "sql.y" -{ yymsp[-1].minor.yy164 = createIdentifierValueNode(pCxt, &yymsp[0].minor.yy497); } -#line 5878 "sql.c" + case 276: /* from_db_opt ::= FROM db_name */ +{ yymsp[-1].minor.yy520 = createIdentifierValueNode(pCxt, &yymsp[0].minor.yy833); } break; - case 277: /* tag_item ::= TBNAME */ -#line 491 "sql.y" -{ yylhsminor.yy164 = setProjectionAlias(pCxt, createFunctionNode(pCxt, &yymsp[0].minor.yy0, NULL), &yymsp[0].minor.yy0); } -#line 5883 "sql.c" - yymsp[0].minor.yy164 = yylhsminor.yy164; + case 280: /* tag_item ::= TBNAME */ +{ yylhsminor.yy520 = setProjectionAlias(pCxt, createFunctionNode(pCxt, &yymsp[0].minor.yy0, NULL), &yymsp[0].minor.yy0); } + yymsp[0].minor.yy520 = yylhsminor.yy520; break; - case 280: /* tag_item ::= column_name column_alias */ -#line 494 "sql.y" -{ yylhsminor.yy164 = setProjectionAlias(pCxt, createColumnNode(pCxt, NULL, &yymsp[-1].minor.yy497), &yymsp[0].minor.yy497); } -#line 5889 "sql.c" - yymsp[-1].minor.yy164 = yylhsminor.yy164; + case 283: /* tag_item ::= column_name column_alias */ +{ yylhsminor.yy520 = setProjectionAlias(pCxt, createColumnNode(pCxt, NULL, &yymsp[-1].minor.yy833), &yymsp[0].minor.yy833); } + yymsp[-1].minor.yy520 = yylhsminor.yy520; break; - case 281: /* tag_item ::= column_name AS column_alias */ -#line 495 "sql.y" -{ yylhsminor.yy164 = setProjectionAlias(pCxt, createColumnNode(pCxt, NULL, &yymsp[-2].minor.yy497), &yymsp[0].minor.yy497); } -#line 5895 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 284: /* tag_item ::= column_name AS column_alias */ +{ yylhsminor.yy520 = setProjectionAlias(pCxt, createColumnNode(pCxt, NULL, &yymsp[-2].minor.yy833), &yymsp[0].minor.yy833); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 282: /* cmd ::= CREATE SMA INDEX not_exists_opt full_index_name ON full_table_name index_options */ -#line 499 "sql.y" -{ pCxt->pRootNode = createCreateIndexStmt(pCxt, INDEX_TYPE_SMA, yymsp[-4].minor.yy441, yymsp[-3].minor.yy164, yymsp[-1].minor.yy164, NULL, yymsp[0].minor.yy164); } -#line 5901 "sql.c" + case 285: /* cmd ::= CREATE SMA INDEX not_exists_opt full_index_name ON full_table_name index_options */ +{ pCxt->pRootNode = createCreateIndexStmt(pCxt, INDEX_TYPE_SMA, yymsp[-4].minor.yy537, yymsp[-3].minor.yy520, yymsp[-1].minor.yy520, NULL, yymsp[0].minor.yy520); } break; - case 283: /* cmd ::= CREATE INDEX not_exists_opt full_index_name ON full_table_name NK_LP col_name_list NK_RP */ -#line 501 "sql.y" -{ pCxt->pRootNode = createCreateIndexStmt(pCxt, INDEX_TYPE_NORMAL, yymsp[-6].minor.yy441, yymsp[-5].minor.yy164, yymsp[-3].minor.yy164, yymsp[-1].minor.yy72, NULL); } -#line 5906 "sql.c" + case 286: /* cmd ::= CREATE INDEX not_exists_opt full_index_name ON full_table_name NK_LP col_name_list NK_RP */ +{ pCxt->pRootNode = createCreateIndexStmt(pCxt, INDEX_TYPE_NORMAL, yymsp[-6].minor.yy537, yymsp[-5].minor.yy520, yymsp[-3].minor.yy520, yymsp[-1].minor.yy904, NULL); } break; - case 284: /* cmd ::= DROP INDEX exists_opt full_index_name */ -#line 502 "sql.y" -{ pCxt->pRootNode = createDropIndexStmt(pCxt, yymsp[-1].minor.yy441, yymsp[0].minor.yy164); } -#line 5911 "sql.c" + case 287: /* cmd ::= DROP INDEX exists_opt full_index_name */ +{ pCxt->pRootNode = createDropIndexStmt(pCxt, yymsp[-1].minor.yy537, yymsp[0].minor.yy520); } break; - case 285: /* full_index_name ::= index_name */ -#line 504 "sql.y" -{ yylhsminor.yy164 = createRealTableNodeForIndexName(pCxt, NULL, &yymsp[0].minor.yy497); } -#line 5916 "sql.c" - yymsp[0].minor.yy164 = yylhsminor.yy164; + case 288: /* full_index_name ::= index_name */ +{ yylhsminor.yy520 = createRealTableNodeForIndexName(pCxt, NULL, &yymsp[0].minor.yy833); } + yymsp[0].minor.yy520 = yylhsminor.yy520; break; - case 286: /* full_index_name ::= db_name NK_DOT index_name */ -#line 505 "sql.y" -{ yylhsminor.yy164 = createRealTableNodeForIndexName(pCxt, &yymsp[-2].minor.yy497, &yymsp[0].minor.yy497); } -#line 5922 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 289: /* full_index_name ::= db_name NK_DOT index_name */ +{ yylhsminor.yy520 = createRealTableNodeForIndexName(pCxt, &yymsp[-2].minor.yy833, &yymsp[0].minor.yy833); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 287: /* index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_RP sliding_opt sma_stream_opt */ -#line 508 "sql.y" -{ yymsp[-9].minor.yy164 = createIndexOption(pCxt, yymsp[-7].minor.yy72, releaseRawExprNode(pCxt, yymsp[-3].minor.yy164), NULL, yymsp[-1].minor.yy164, yymsp[0].minor.yy164); } -#line 5928 "sql.c" + case 290: /* index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_RP sliding_opt sma_stream_opt */ +{ yymsp[-9].minor.yy520 = createIndexOption(pCxt, yymsp[-7].minor.yy904, releaseRawExprNode(pCxt, yymsp[-3].minor.yy520), NULL, yymsp[-1].minor.yy520, yymsp[0].minor.yy520); } break; - case 288: /* 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 511 "sql.y" -{ yymsp[-11].minor.yy164 = createIndexOption(pCxt, yymsp[-9].minor.yy72, releaseRawExprNode(pCxt, yymsp[-5].minor.yy164), releaseRawExprNode(pCxt, yymsp[-3].minor.yy164), yymsp[-1].minor.yy164, yymsp[0].minor.yy164); } -#line 5933 "sql.c" + case 291: /* index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_COMMA duration_literal NK_RP sliding_opt sma_stream_opt */ +{ yymsp[-11].minor.yy520 = createIndexOption(pCxt, yymsp[-9].minor.yy904, releaseRawExprNode(pCxt, yymsp[-5].minor.yy520), releaseRawExprNode(pCxt, yymsp[-3].minor.yy520), yymsp[-1].minor.yy520, yymsp[0].minor.yy520); } break; - case 291: /* func ::= sma_func_name NK_LP expression_list NK_RP */ -#line 518 "sql.y" -{ yylhsminor.yy164 = createFunctionNode(pCxt, &yymsp[-3].minor.yy497, yymsp[-1].minor.yy72); } -#line 5938 "sql.c" - yymsp[-3].minor.yy164 = yylhsminor.yy164; + case 294: /* func ::= sma_func_name NK_LP expression_list NK_RP */ +{ yylhsminor.yy520 = createFunctionNode(pCxt, &yymsp[-3].minor.yy833, yymsp[-1].minor.yy904); } + yymsp[-3].minor.yy520 = yylhsminor.yy520; break; - case 292: /* sma_func_name ::= function_name */ - case 505: /* alias_opt ::= table_alias */ yytestcase(yyruleno==505); -#line 522 "sql.y" -{ yylhsminor.yy497 = yymsp[0].minor.yy497; } -#line 5945 "sql.c" - yymsp[0].minor.yy497 = yylhsminor.yy497; + case 295: /* sma_func_name ::= function_name */ + case 508: /* alias_opt ::= table_alias */ yytestcase(yyruleno==508); +{ yylhsminor.yy833 = yymsp[0].minor.yy833; } + yymsp[0].minor.yy833 = yylhsminor.yy833; break; - case 297: /* sma_stream_opt ::= */ - case 337: /* stream_options ::= */ yytestcase(yyruleno==337); -#line 528 "sql.y" -{ yymsp[1].minor.yy164 = createStreamOptions(pCxt); } -#line 5952 "sql.c" + case 300: /* sma_stream_opt ::= */ + case 340: /* stream_options ::= */ yytestcase(yyruleno==340); +{ yymsp[1].minor.yy520 = createStreamOptions(pCxt); } break; - case 298: /* sma_stream_opt ::= sma_stream_opt WATERMARK duration_literal */ -#line 529 "sql.y" -{ ((SStreamOptions*)yymsp[-2].minor.yy164)->pWatermark = releaseRawExprNode(pCxt, yymsp[0].minor.yy164); yylhsminor.yy164 = yymsp[-2].minor.yy164; } -#line 5957 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 301: /* sma_stream_opt ::= sma_stream_opt WATERMARK duration_literal */ +{ ((SStreamOptions*)yymsp[-2].minor.yy520)->pWatermark = releaseRawExprNode(pCxt, yymsp[0].minor.yy520); yylhsminor.yy520 = yymsp[-2].minor.yy520; } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 299: /* sma_stream_opt ::= sma_stream_opt MAX_DELAY duration_literal */ -#line 530 "sql.y" -{ ((SStreamOptions*)yymsp[-2].minor.yy164)->pDelay = releaseRawExprNode(pCxt, yymsp[0].minor.yy164); yylhsminor.yy164 = yymsp[-2].minor.yy164; } -#line 5963 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 302: /* sma_stream_opt ::= sma_stream_opt MAX_DELAY duration_literal */ +{ ((SStreamOptions*)yymsp[-2].minor.yy520)->pDelay = releaseRawExprNode(pCxt, yymsp[0].minor.yy520); yylhsminor.yy520 = yymsp[-2].minor.yy520; } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 300: /* sma_stream_opt ::= sma_stream_opt DELETE_MARK duration_literal */ -#line 531 "sql.y" -{ ((SStreamOptions*)yymsp[-2].minor.yy164)->pDeleteMark = releaseRawExprNode(pCxt, yymsp[0].minor.yy164); yylhsminor.yy164 = yymsp[-2].minor.yy164; } -#line 5969 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 303: /* sma_stream_opt ::= sma_stream_opt DELETE_MARK duration_literal */ +{ ((SStreamOptions*)yymsp[-2].minor.yy520)->pDeleteMark = releaseRawExprNode(pCxt, yymsp[0].minor.yy520); yylhsminor.yy520 = yymsp[-2].minor.yy520; } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 301: /* cmd ::= CREATE TOPIC not_exists_opt topic_name AS query_or_subquery */ -#line 534 "sql.y" -{ pCxt->pRootNode = createCreateTopicStmtUseQuery(pCxt, yymsp[-3].minor.yy441, &yymsp[-2].minor.yy497, yymsp[0].minor.yy164); } -#line 5975 "sql.c" + case 304: /* cmd ::= CREATE TOPIC not_exists_opt topic_name AS query_or_subquery */ +{ pCxt->pRootNode = createCreateTopicStmtUseQuery(pCxt, yymsp[-3].minor.yy537, &yymsp[-2].minor.yy833, yymsp[0].minor.yy520); } break; - case 302: /* cmd ::= CREATE TOPIC not_exists_opt topic_name AS DATABASE db_name */ -#line 535 "sql.y" -{ pCxt->pRootNode = createCreateTopicStmtUseDb(pCxt, yymsp[-4].minor.yy441, &yymsp[-3].minor.yy497, &yymsp[0].minor.yy497, false); } -#line 5980 "sql.c" + case 305: /* cmd ::= CREATE TOPIC not_exists_opt topic_name AS DATABASE db_name */ +{ pCxt->pRootNode = createCreateTopicStmtUseDb(pCxt, yymsp[-4].minor.yy537, &yymsp[-3].minor.yy833, &yymsp[0].minor.yy833, false); } break; - case 303: /* cmd ::= CREATE TOPIC not_exists_opt topic_name WITH META AS DATABASE db_name */ -#line 537 "sql.y" -{ pCxt->pRootNode = createCreateTopicStmtUseDb(pCxt, yymsp[-6].minor.yy441, &yymsp[-5].minor.yy497, &yymsp[0].minor.yy497, true); } -#line 5985 "sql.c" + case 306: /* cmd ::= CREATE TOPIC not_exists_opt topic_name WITH META AS DATABASE db_name */ +{ pCxt->pRootNode = createCreateTopicStmtUseDb(pCxt, yymsp[-6].minor.yy537, &yymsp[-5].minor.yy833, &yymsp[0].minor.yy833, true); } break; - case 304: /* cmd ::= CREATE TOPIC not_exists_opt topic_name AS STABLE full_table_name */ -#line 539 "sql.y" -{ pCxt->pRootNode = createCreateTopicStmtUseTable(pCxt, yymsp[-4].minor.yy441, &yymsp[-3].minor.yy497, yymsp[0].minor.yy164, false); } -#line 5990 "sql.c" + case 307: /* cmd ::= CREATE TOPIC not_exists_opt topic_name AS STABLE full_table_name */ +{ pCxt->pRootNode = createCreateTopicStmtUseTable(pCxt, yymsp[-4].minor.yy537, &yymsp[-3].minor.yy833, yymsp[0].minor.yy520, false); } break; - case 305: /* cmd ::= CREATE TOPIC not_exists_opt topic_name WITH META AS STABLE full_table_name */ -#line 541 "sql.y" -{ pCxt->pRootNode = createCreateTopicStmtUseTable(pCxt, yymsp[-6].minor.yy441, &yymsp[-5].minor.yy497, yymsp[0].minor.yy164, true); } -#line 5995 "sql.c" + case 308: /* cmd ::= CREATE TOPIC not_exists_opt topic_name WITH META AS STABLE full_table_name */ +{ pCxt->pRootNode = createCreateTopicStmtUseTable(pCxt, yymsp[-6].minor.yy537, &yymsp[-5].minor.yy833, yymsp[0].minor.yy520, true); } break; - case 306: /* cmd ::= DROP TOPIC exists_opt topic_name */ -#line 542 "sql.y" -{ pCxt->pRootNode = createDropTopicStmt(pCxt, yymsp[-1].minor.yy441, &yymsp[0].minor.yy497); } -#line 6000 "sql.c" + case 309: /* cmd ::= DROP TOPIC exists_opt topic_name */ +{ pCxt->pRootNode = createDropTopicStmt(pCxt, yymsp[-1].minor.yy537, &yymsp[0].minor.yy833); } break; - case 307: /* cmd ::= DROP CONSUMER GROUP exists_opt cgroup_name ON topic_name */ -#line 543 "sql.y" -{ pCxt->pRootNode = createDropCGroupStmt(pCxt, yymsp[-3].minor.yy441, &yymsp[-2].minor.yy497, &yymsp[0].minor.yy497); } -#line 6005 "sql.c" + case 310: /* cmd ::= DROP CONSUMER GROUP exists_opt cgroup_name ON topic_name */ +{ pCxt->pRootNode = createDropCGroupStmt(pCxt, yymsp[-3].minor.yy537, &yymsp[-2].minor.yy833, &yymsp[0].minor.yy833); } break; - case 308: /* cmd ::= DESC full_table_name */ - case 309: /* cmd ::= DESCRIBE full_table_name */ yytestcase(yyruleno==309); -#line 546 "sql.y" -{ pCxt->pRootNode = createDescribeStmt(pCxt, yymsp[0].minor.yy164); } -#line 6011 "sql.c" + case 311: /* cmd ::= DESC full_table_name */ + case 312: /* cmd ::= DESCRIBE full_table_name */ yytestcase(yyruleno==312); +{ pCxt->pRootNode = createDescribeStmt(pCxt, yymsp[0].minor.yy520); } break; - case 310: /* cmd ::= RESET QUERY CACHE */ -#line 550 "sql.y" + case 313: /* cmd ::= RESET QUERY CACHE */ { pCxt->pRootNode = createResetQueryCacheStmt(pCxt); } -#line 6016 "sql.c" break; - case 311: /* cmd ::= EXPLAIN analyze_opt explain_options query_or_subquery */ - case 312: /* cmd ::= EXPLAIN analyze_opt explain_options insert_query */ yytestcase(yyruleno==312); -#line 553 "sql.y" -{ pCxt->pRootNode = createExplainStmt(pCxt, yymsp[-2].minor.yy441, yymsp[-1].minor.yy164, yymsp[0].minor.yy164); } -#line 6022 "sql.c" + case 314: /* cmd ::= EXPLAIN analyze_opt explain_options query_or_subquery */ + case 315: /* cmd ::= EXPLAIN analyze_opt explain_options insert_query */ yytestcase(yyruleno==315); +{ pCxt->pRootNode = createExplainStmt(pCxt, yymsp[-2].minor.yy537, yymsp[-1].minor.yy520, yymsp[0].minor.yy520); } break; - case 315: /* explain_options ::= */ -#line 561 "sql.y" -{ yymsp[1].minor.yy164 = createDefaultExplainOptions(pCxt); } -#line 6027 "sql.c" + case 318: /* explain_options ::= */ +{ yymsp[1].minor.yy520 = createDefaultExplainOptions(pCxt); } break; - case 316: /* explain_options ::= explain_options VERBOSE NK_BOOL */ -#line 562 "sql.y" -{ yylhsminor.yy164 = setExplainVerbose(pCxt, yymsp[-2].minor.yy164, &yymsp[0].minor.yy0); } -#line 6032 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 319: /* explain_options ::= explain_options VERBOSE NK_BOOL */ +{ yylhsminor.yy520 = setExplainVerbose(pCxt, yymsp[-2].minor.yy520, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 317: /* explain_options ::= explain_options RATIO NK_FLOAT */ -#line 563 "sql.y" -{ yylhsminor.yy164 = setExplainRatio(pCxt, yymsp[-2].minor.yy164, &yymsp[0].minor.yy0); } -#line 6038 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 320: /* explain_options ::= explain_options RATIO NK_FLOAT */ +{ yylhsminor.yy520 = setExplainRatio(pCxt, yymsp[-2].minor.yy520, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 318: /* 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 568 "sql.y" -{ pCxt->pRootNode = createCreateFunctionStmt(pCxt, yymsp[-7].minor.yy441, yymsp[-9].minor.yy441, &yymsp[-6].minor.yy497, &yymsp[-4].minor.yy0, yymsp[-2].minor.yy700, yymsp[-1].minor.yy560, &yymsp[0].minor.yy497, yymsp[-10].minor.yy441); } -#line 6044 "sql.c" + case 321: /* cmd ::= CREATE or_replace_opt agg_func_opt FUNCTION not_exists_opt function_name AS NK_STRING OUTPUTTYPE type_name bufsize_opt language_opt */ +{ pCxt->pRootNode = createCreateFunctionStmt(pCxt, yymsp[-7].minor.yy537, yymsp[-9].minor.yy537, &yymsp[-6].minor.yy833, &yymsp[-4].minor.yy0, yymsp[-2].minor.yy840, yymsp[-1].minor.yy860, &yymsp[0].minor.yy833, yymsp[-10].minor.yy537); } break; - case 319: /* cmd ::= DROP FUNCTION exists_opt function_name */ -#line 569 "sql.y" -{ pCxt->pRootNode = createDropFunctionStmt(pCxt, yymsp[-1].minor.yy441, &yymsp[0].minor.yy497); } -#line 6049 "sql.c" + case 322: /* cmd ::= DROP FUNCTION exists_opt function_name */ +{ pCxt->pRootNode = createDropFunctionStmt(pCxt, yymsp[-1].minor.yy537, &yymsp[0].minor.yy833); } break; - case 324: /* language_opt ::= */ -#line 583 "sql.y" -{ yymsp[1].minor.yy497 = nil_token; } -#line 6054 "sql.c" + case 327: /* language_opt ::= */ +{ yymsp[1].minor.yy833 = nil_token; } break; - case 325: /* language_opt ::= LANGUAGE NK_STRING */ -#line 584 "sql.y" -{ yymsp[-1].minor.yy497 = yymsp[0].minor.yy0; } -#line 6059 "sql.c" + case 328: /* language_opt ::= LANGUAGE NK_STRING */ +{ yymsp[-1].minor.yy833 = yymsp[0].minor.yy0; } break; - case 328: /* 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 594 "sql.y" -{ pCxt->pRootNode = createCreateStreamStmt(pCxt, yymsp[-9].minor.yy441, &yymsp[-8].minor.yy497, yymsp[-5].minor.yy164, yymsp[-7].minor.yy164, yymsp[-3].minor.yy72, yymsp[-2].minor.yy164, yymsp[0].minor.yy164, yymsp[-4].minor.yy72); } -#line 6064 "sql.c" + case 331: /* cmd ::= CREATE STREAM not_exists_opt stream_name stream_options INTO full_table_name col_list_opt tag_def_or_ref_opt subtable_opt AS query_or_subquery */ +{ pCxt->pRootNode = createCreateStreamStmt(pCxt, yymsp[-9].minor.yy537, &yymsp[-8].minor.yy833, yymsp[-5].minor.yy520, yymsp[-7].minor.yy520, yymsp[-3].minor.yy904, yymsp[-2].minor.yy520, yymsp[0].minor.yy520, yymsp[-4].minor.yy904); } break; - case 329: /* cmd ::= DROP STREAM exists_opt stream_name */ -#line 595 "sql.y" -{ pCxt->pRootNode = createDropStreamStmt(pCxt, yymsp[-1].minor.yy441, &yymsp[0].minor.yy497); } -#line 6069 "sql.c" + case 332: /* cmd ::= DROP STREAM exists_opt stream_name */ +{ pCxt->pRootNode = createDropStreamStmt(pCxt, yymsp[-1].minor.yy537, &yymsp[0].minor.yy833); } break; - case 330: /* cmd ::= PAUSE STREAM exists_opt stream_name */ -#line 596 "sql.y" -{ pCxt->pRootNode = createPauseStreamStmt(pCxt, yymsp[-1].minor.yy441, &yymsp[0].minor.yy497); } -#line 6074 "sql.c" + case 333: /* cmd ::= PAUSE STREAM exists_opt stream_name */ +{ pCxt->pRootNode = createPauseStreamStmt(pCxt, yymsp[-1].minor.yy537, &yymsp[0].minor.yy833); } break; - case 331: /* cmd ::= RESUME STREAM exists_opt ignore_opt stream_name */ -#line 597 "sql.y" -{ pCxt->pRootNode = createResumeStreamStmt(pCxt, yymsp[-2].minor.yy441, yymsp[-1].minor.yy441, &yymsp[0].minor.yy497); } -#line 6079 "sql.c" + case 334: /* cmd ::= RESUME STREAM exists_opt ignore_opt stream_name */ +{ pCxt->pRootNode = createResumeStreamStmt(pCxt, yymsp[-2].minor.yy537, yymsp[-1].minor.yy537, &yymsp[0].minor.yy833); } break; - case 338: /* stream_options ::= stream_options TRIGGER AT_ONCE */ - case 339: /* stream_options ::= stream_options TRIGGER WINDOW_CLOSE */ yytestcase(yyruleno==339); -#line 611 "sql.y" -{ yylhsminor.yy164 = setStreamOptions(pCxt, yymsp[-2].minor.yy164, SOPT_TRIGGER_TYPE_SET, &yymsp[0].minor.yy0, NULL); } -#line 6085 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 341: /* stream_options ::= stream_options TRIGGER AT_ONCE */ + case 342: /* stream_options ::= stream_options TRIGGER WINDOW_CLOSE */ yytestcase(yyruleno==342); +{ yylhsminor.yy520 = setStreamOptions(pCxt, yymsp[-2].minor.yy520, SOPT_TRIGGER_TYPE_SET, &yymsp[0].minor.yy0, NULL); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 340: /* stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal */ -#line 613 "sql.y" -{ yylhsminor.yy164 = setStreamOptions(pCxt, yymsp[-3].minor.yy164, SOPT_TRIGGER_TYPE_SET, &yymsp[-1].minor.yy0, releaseRawExprNode(pCxt, yymsp[0].minor.yy164)); } -#line 6091 "sql.c" - yymsp[-3].minor.yy164 = yylhsminor.yy164; + case 343: /* stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal */ +{ yylhsminor.yy520 = setStreamOptions(pCxt, yymsp[-3].minor.yy520, SOPT_TRIGGER_TYPE_SET, &yymsp[-1].minor.yy0, releaseRawExprNode(pCxt, yymsp[0].minor.yy520)); } + yymsp[-3].minor.yy520 = yylhsminor.yy520; break; - case 341: /* stream_options ::= stream_options WATERMARK duration_literal */ -#line 614 "sql.y" -{ yylhsminor.yy164 = setStreamOptions(pCxt, yymsp[-2].minor.yy164, SOPT_WATERMARK_SET, NULL, releaseRawExprNode(pCxt, yymsp[0].minor.yy164)); } -#line 6097 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 344: /* stream_options ::= stream_options WATERMARK duration_literal */ +{ yylhsminor.yy520 = setStreamOptions(pCxt, yymsp[-2].minor.yy520, SOPT_WATERMARK_SET, NULL, releaseRawExprNode(pCxt, yymsp[0].minor.yy520)); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 342: /* stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER */ -#line 615 "sql.y" -{ yylhsminor.yy164 = setStreamOptions(pCxt, yymsp[-3].minor.yy164, SOPT_IGNORE_EXPIRED_SET, &yymsp[0].minor.yy0, NULL); } -#line 6103 "sql.c" - yymsp[-3].minor.yy164 = yylhsminor.yy164; + case 345: /* stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER */ +{ yylhsminor.yy520 = setStreamOptions(pCxt, yymsp[-3].minor.yy520, SOPT_IGNORE_EXPIRED_SET, &yymsp[0].minor.yy0, NULL); } + yymsp[-3].minor.yy520 = yylhsminor.yy520; break; - case 343: /* stream_options ::= stream_options FILL_HISTORY NK_INTEGER */ -#line 616 "sql.y" -{ yylhsminor.yy164 = setStreamOptions(pCxt, yymsp[-2].minor.yy164, SOPT_FILL_HISTORY_SET, &yymsp[0].minor.yy0, NULL); } -#line 6109 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 346: /* stream_options ::= stream_options FILL_HISTORY NK_INTEGER */ +{ yylhsminor.yy520 = setStreamOptions(pCxt, yymsp[-2].minor.yy520, SOPT_FILL_HISTORY_SET, &yymsp[0].minor.yy0, NULL); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 344: /* stream_options ::= stream_options DELETE_MARK duration_literal */ -#line 617 "sql.y" -{ yylhsminor.yy164 = setStreamOptions(pCxt, yymsp[-2].minor.yy164, SOPT_DELETE_MARK_SET, NULL, releaseRawExprNode(pCxt, yymsp[0].minor.yy164)); } -#line 6115 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 347: /* stream_options ::= stream_options DELETE_MARK duration_literal */ +{ yylhsminor.yy520 = setStreamOptions(pCxt, yymsp[-2].minor.yy520, SOPT_DELETE_MARK_SET, NULL, releaseRawExprNode(pCxt, yymsp[0].minor.yy520)); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 345: /* stream_options ::= stream_options IGNORE UPDATE NK_INTEGER */ -#line 618 "sql.y" -{ yylhsminor.yy164 = setStreamOptions(pCxt, yymsp[-3].minor.yy164, SOPT_IGNORE_UPDATE_SET, &yymsp[0].minor.yy0, NULL); } -#line 6121 "sql.c" - yymsp[-3].minor.yy164 = yylhsminor.yy164; + case 348: /* stream_options ::= stream_options IGNORE UPDATE NK_INTEGER */ +{ yylhsminor.yy520 = setStreamOptions(pCxt, yymsp[-3].minor.yy520, SOPT_IGNORE_UPDATE_SET, &yymsp[0].minor.yy0, NULL); } + yymsp[-3].minor.yy520 = yylhsminor.yy520; break; - case 347: /* subtable_opt ::= SUBTABLE NK_LP expression NK_RP */ - case 539: /* sliding_opt ::= SLIDING NK_LP duration_literal NK_RP */ yytestcase(yyruleno==539); - case 559: /* every_opt ::= EVERY NK_LP duration_literal NK_RP */ yytestcase(yyruleno==559); -#line 621 "sql.y" -{ yymsp[-3].minor.yy164 = releaseRawExprNode(pCxt, yymsp[-1].minor.yy164); } -#line 6129 "sql.c" + case 350: /* subtable_opt ::= SUBTABLE NK_LP expression NK_RP */ + case 542: /* sliding_opt ::= SLIDING NK_LP duration_literal NK_RP */ yytestcase(yyruleno==542); + case 562: /* every_opt ::= EVERY NK_LP duration_literal NK_RP */ yytestcase(yyruleno==562); +{ yymsp[-3].minor.yy520 = releaseRawExprNode(pCxt, yymsp[-1].minor.yy520); } break; - case 350: /* cmd ::= KILL CONNECTION NK_INTEGER */ -#line 629 "sql.y" + case 353: /* cmd ::= KILL CONNECTION NK_INTEGER */ { pCxt->pRootNode = createKillStmt(pCxt, QUERY_NODE_KILL_CONNECTION_STMT, &yymsp[0].minor.yy0); } -#line 6134 "sql.c" break; - case 351: /* cmd ::= KILL QUERY NK_STRING */ -#line 630 "sql.y" + case 354: /* cmd ::= KILL QUERY NK_STRING */ { pCxt->pRootNode = createKillQueryStmt(pCxt, &yymsp[0].minor.yy0); } -#line 6139 "sql.c" break; - case 352: /* cmd ::= KILL TRANSACTION NK_INTEGER */ -#line 631 "sql.y" + case 355: /* cmd ::= KILL TRANSACTION NK_INTEGER */ { pCxt->pRootNode = createKillStmt(pCxt, QUERY_NODE_KILL_TRANSACTION_STMT, &yymsp[0].minor.yy0); } -#line 6144 "sql.c" break; - case 353: /* cmd ::= BALANCE VGROUP */ -#line 634 "sql.y" + case 356: /* cmd ::= BALANCE VGROUP */ { pCxt->pRootNode = createBalanceVgroupStmt(pCxt); } -#line 6149 "sql.c" break; - case 354: /* cmd ::= BALANCE VGROUP LEADER */ -#line 635 "sql.y" + case 357: /* cmd ::= BALANCE VGROUP LEADER */ { pCxt->pRootNode = createBalanceVgroupLeaderStmt(pCxt); } -#line 6154 "sql.c" break; - case 355: /* cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER */ -#line 636 "sql.y" + case 358: /* cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER */ { pCxt->pRootNode = createMergeVgroupStmt(pCxt, &yymsp[-1].minor.yy0, &yymsp[0].minor.yy0); } -#line 6159 "sql.c" break; - case 356: /* cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list */ -#line 637 "sql.y" -{ pCxt->pRootNode = createRedistributeVgroupStmt(pCxt, &yymsp[-1].minor.yy0, yymsp[0].minor.yy72); } -#line 6164 "sql.c" + case 359: /* cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list */ +{ pCxt->pRootNode = createRedistributeVgroupStmt(pCxt, &yymsp[-1].minor.yy0, yymsp[0].minor.yy904); } break; - case 357: /* cmd ::= SPLIT VGROUP NK_INTEGER */ -#line 638 "sql.y" + case 360: /* cmd ::= SPLIT VGROUP NK_INTEGER */ { pCxt->pRootNode = createSplitVgroupStmt(pCxt, &yymsp[0].minor.yy0); } -#line 6169 "sql.c" break; - case 358: /* dnode_list ::= DNODE NK_INTEGER */ -#line 642 "sql.y" -{ yymsp[-1].minor.yy72 = createNodeList(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0)); } -#line 6174 "sql.c" + case 361: /* dnode_list ::= DNODE NK_INTEGER */ +{ yymsp[-1].minor.yy904 = createNodeList(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0)); } break; - case 360: /* cmd ::= DELETE FROM full_table_name where_clause_opt */ -#line 649 "sql.y" -{ pCxt->pRootNode = createDeleteStmt(pCxt, yymsp[-1].minor.yy164, yymsp[0].minor.yy164); } -#line 6179 "sql.c" + case 363: /* cmd ::= DELETE FROM full_table_name where_clause_opt */ +{ pCxt->pRootNode = createDeleteStmt(pCxt, yymsp[-1].minor.yy520, yymsp[0].minor.yy520); } break; - case 363: /* insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery */ -#line 658 "sql.y" -{ yymsp[-6].minor.yy164 = createInsertStmt(pCxt, yymsp[-4].minor.yy164, yymsp[-2].minor.yy72, yymsp[0].minor.yy164); } -#line 6184 "sql.c" + case 366: /* insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery */ +{ yymsp[-6].minor.yy520 = createInsertStmt(pCxt, yymsp[-4].minor.yy520, yymsp[-2].minor.yy904, yymsp[0].minor.yy520); } break; - case 364: /* insert_query ::= INSERT INTO full_table_name query_or_subquery */ -#line 659 "sql.y" -{ yymsp[-3].minor.yy164 = createInsertStmt(pCxt, yymsp[-1].minor.yy164, NULL, yymsp[0].minor.yy164); } -#line 6189 "sql.c" + case 367: /* insert_query ::= INSERT INTO full_table_name query_or_subquery */ +{ yymsp[-3].minor.yy520 = createInsertStmt(pCxt, yymsp[-1].minor.yy520, NULL, yymsp[0].minor.yy520); } break; - case 365: /* literal ::= NK_INTEGER */ -#line 662 "sql.y" -{ yylhsminor.yy164 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &yymsp[0].minor.yy0)); } -#line 6194 "sql.c" - yymsp[0].minor.yy164 = yylhsminor.yy164; + case 368: /* literal ::= NK_INTEGER */ +{ yylhsminor.yy520 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy520 = yylhsminor.yy520; break; - case 366: /* literal ::= NK_FLOAT */ -#line 663 "sql.y" -{ yylhsminor.yy164 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &yymsp[0].minor.yy0)); } -#line 6200 "sql.c" - yymsp[0].minor.yy164 = yylhsminor.yy164; + case 369: /* literal ::= NK_FLOAT */ +{ yylhsminor.yy520 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy520 = yylhsminor.yy520; break; - case 367: /* literal ::= NK_STRING */ -#line 664 "sql.y" -{ yylhsminor.yy164 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0)); } -#line 6206 "sql.c" - yymsp[0].minor.yy164 = yylhsminor.yy164; + case 370: /* literal ::= NK_STRING */ +{ yylhsminor.yy520 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy520 = yylhsminor.yy520; break; - case 368: /* literal ::= NK_BOOL */ -#line 665 "sql.y" -{ yylhsminor.yy164 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_BOOL, &yymsp[0].minor.yy0)); } -#line 6212 "sql.c" - yymsp[0].minor.yy164 = yylhsminor.yy164; + case 371: /* literal ::= NK_BOOL */ +{ yylhsminor.yy520 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_BOOL, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy520 = yylhsminor.yy520; break; - case 369: /* literal ::= TIMESTAMP NK_STRING */ -#line 666 "sql.y" -{ yylhsminor.yy164 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0)); } -#line 6218 "sql.c" - yymsp[-1].minor.yy164 = yylhsminor.yy164; + case 372: /* literal ::= TIMESTAMP NK_STRING */ +{ yylhsminor.yy520 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0)); } + yymsp[-1].minor.yy520 = yylhsminor.yy520; break; - case 370: /* literal ::= duration_literal */ - case 380: /* signed_literal ::= signed */ yytestcase(yyruleno==380); - case 401: /* expr_or_subquery ::= expression */ yytestcase(yyruleno==401); - case 402: /* expression ::= literal */ yytestcase(yyruleno==402); - case 403: /* expression ::= pseudo_column */ yytestcase(yyruleno==403); - case 404: /* expression ::= column_reference */ yytestcase(yyruleno==404); - case 405: /* expression ::= function_expression */ yytestcase(yyruleno==405); - case 406: /* expression ::= case_when_expression */ yytestcase(yyruleno==406); - case 437: /* function_expression ::= literal_func */ yytestcase(yyruleno==437); - case 486: /* boolean_value_expression ::= boolean_primary */ yytestcase(yyruleno==486); - case 490: /* boolean_primary ::= predicate */ yytestcase(yyruleno==490); - case 492: /* common_expression ::= expr_or_subquery */ yytestcase(yyruleno==492); - case 493: /* common_expression ::= boolean_value_expression */ yytestcase(yyruleno==493); - case 496: /* table_reference_list ::= table_reference */ yytestcase(yyruleno==496); - case 498: /* table_reference ::= table_primary */ yytestcase(yyruleno==498); - case 499: /* table_reference ::= joined_table */ yytestcase(yyruleno==499); - case 503: /* table_primary ::= parenthesized_joined_table */ yytestcase(yyruleno==503); - case 561: /* query_simple ::= query_specification */ yytestcase(yyruleno==561); - case 562: /* query_simple ::= union_query_expression */ yytestcase(yyruleno==562); - case 565: /* query_simple_or_subquery ::= query_simple */ yytestcase(yyruleno==565); - case 567: /* query_or_subquery ::= query_expression */ yytestcase(yyruleno==567); -#line 667 "sql.y" -{ yylhsminor.yy164 = yymsp[0].minor.yy164; } -#line 6244 "sql.c" - yymsp[0].minor.yy164 = yylhsminor.yy164; + case 373: /* literal ::= duration_literal */ + case 383: /* signed_literal ::= signed */ yytestcase(yyruleno==383); + case 404: /* expr_or_subquery ::= expression */ yytestcase(yyruleno==404); + case 405: /* expression ::= literal */ yytestcase(yyruleno==405); + case 406: /* expression ::= pseudo_column */ yytestcase(yyruleno==406); + case 407: /* expression ::= column_reference */ yytestcase(yyruleno==407); + case 408: /* expression ::= function_expression */ yytestcase(yyruleno==408); + case 409: /* expression ::= case_when_expression */ yytestcase(yyruleno==409); + case 440: /* function_expression ::= literal_func */ yytestcase(yyruleno==440); + case 489: /* boolean_value_expression ::= boolean_primary */ yytestcase(yyruleno==489); + case 493: /* boolean_primary ::= predicate */ yytestcase(yyruleno==493); + case 495: /* common_expression ::= expr_or_subquery */ yytestcase(yyruleno==495); + case 496: /* common_expression ::= boolean_value_expression */ yytestcase(yyruleno==496); + case 499: /* table_reference_list ::= table_reference */ yytestcase(yyruleno==499); + case 501: /* table_reference ::= table_primary */ yytestcase(yyruleno==501); + case 502: /* table_reference ::= joined_table */ yytestcase(yyruleno==502); + case 506: /* table_primary ::= parenthesized_joined_table */ yytestcase(yyruleno==506); + case 564: /* query_simple ::= query_specification */ yytestcase(yyruleno==564); + case 565: /* query_simple ::= union_query_expression */ yytestcase(yyruleno==565); + case 568: /* query_simple_or_subquery ::= query_simple */ yytestcase(yyruleno==568); + case 570: /* query_or_subquery ::= query_expression */ yytestcase(yyruleno==570); +{ yylhsminor.yy520 = yymsp[0].minor.yy520; } + yymsp[0].minor.yy520 = yylhsminor.yy520; break; - case 371: /* literal ::= NULL */ -#line 668 "sql.y" -{ yylhsminor.yy164 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_NULL, &yymsp[0].minor.yy0)); } -#line 6250 "sql.c" - yymsp[0].minor.yy164 = yylhsminor.yy164; + case 374: /* literal ::= NULL */ +{ yylhsminor.yy520 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_NULL, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy520 = yylhsminor.yy520; break; - case 372: /* literal ::= NK_QUESTION */ -#line 669 "sql.y" -{ yylhsminor.yy164 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createPlaceholderValueNode(pCxt, &yymsp[0].minor.yy0)); } -#line 6256 "sql.c" - yymsp[0].minor.yy164 = yylhsminor.yy164; + case 375: /* literal ::= NK_QUESTION */ +{ yylhsminor.yy520 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createPlaceholderValueNode(pCxt, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy520 = yylhsminor.yy520; break; - case 373: /* duration_literal ::= NK_VARIABLE */ -#line 671 "sql.y" -{ yylhsminor.yy164 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } -#line 6262 "sql.c" - yymsp[0].minor.yy164 = yylhsminor.yy164; + case 376: /* duration_literal ::= NK_VARIABLE */ +{ yylhsminor.yy520 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy520 = yylhsminor.yy520; break; - case 374: /* signed ::= NK_INTEGER */ -#line 673 "sql.y" -{ yylhsminor.yy164 = createValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &yymsp[0].minor.yy0); } -#line 6268 "sql.c" - yymsp[0].minor.yy164 = yylhsminor.yy164; + case 377: /* signed ::= NK_INTEGER */ +{ yylhsminor.yy520 = createValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &yymsp[0].minor.yy0); } + yymsp[0].minor.yy520 = yylhsminor.yy520; break; - case 375: /* signed ::= NK_PLUS NK_INTEGER */ -#line 674 "sql.y" -{ yymsp[-1].minor.yy164 = createValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &yymsp[0].minor.yy0); } -#line 6274 "sql.c" + case 378: /* signed ::= NK_PLUS NK_INTEGER */ +{ yymsp[-1].minor.yy520 = createValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &yymsp[0].minor.yy0); } break; - case 376: /* signed ::= NK_MINUS NK_INTEGER */ -#line 675 "sql.y" + case 379: /* 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.yy164 = createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &t); + yylhsminor.yy520 = createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &t); } -#line 6283 "sql.c" - yymsp[-1].minor.yy164 = yylhsminor.yy164; + yymsp[-1].minor.yy520 = yylhsminor.yy520; break; - case 377: /* signed ::= NK_FLOAT */ -#line 680 "sql.y" -{ yylhsminor.yy164 = createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &yymsp[0].minor.yy0); } -#line 6289 "sql.c" - yymsp[0].minor.yy164 = yylhsminor.yy164; + case 380: /* signed ::= NK_FLOAT */ +{ yylhsminor.yy520 = createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &yymsp[0].minor.yy0); } + yymsp[0].minor.yy520 = yylhsminor.yy520; break; - case 378: /* signed ::= NK_PLUS NK_FLOAT */ -#line 681 "sql.y" -{ yymsp[-1].minor.yy164 = createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &yymsp[0].minor.yy0); } -#line 6295 "sql.c" + case 381: /* signed ::= NK_PLUS NK_FLOAT */ +{ yymsp[-1].minor.yy520 = createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &yymsp[0].minor.yy0); } break; - case 379: /* signed ::= NK_MINUS NK_FLOAT */ -#line 682 "sql.y" + case 382: /* 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.yy164 = createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &t); + yylhsminor.yy520 = createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &t); } -#line 6304 "sql.c" - yymsp[-1].minor.yy164 = yylhsminor.yy164; + yymsp[-1].minor.yy520 = yylhsminor.yy520; break; - case 381: /* signed_literal ::= NK_STRING */ -#line 689 "sql.y" -{ yylhsminor.yy164 = createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0); } -#line 6310 "sql.c" - yymsp[0].minor.yy164 = yylhsminor.yy164; + case 384: /* signed_literal ::= NK_STRING */ +{ yylhsminor.yy520 = createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0); } + yymsp[0].minor.yy520 = yylhsminor.yy520; break; - case 382: /* signed_literal ::= NK_BOOL */ -#line 690 "sql.y" -{ yylhsminor.yy164 = createValueNode(pCxt, TSDB_DATA_TYPE_BOOL, &yymsp[0].minor.yy0); } -#line 6316 "sql.c" - yymsp[0].minor.yy164 = yylhsminor.yy164; + case 385: /* signed_literal ::= NK_BOOL */ +{ yylhsminor.yy520 = createValueNode(pCxt, TSDB_DATA_TYPE_BOOL, &yymsp[0].minor.yy0); } + yymsp[0].minor.yy520 = yylhsminor.yy520; break; - case 383: /* signed_literal ::= TIMESTAMP NK_STRING */ -#line 691 "sql.y" -{ yymsp[-1].minor.yy164 = createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0); } -#line 6322 "sql.c" + case 386: /* signed_literal ::= TIMESTAMP NK_STRING */ +{ yymsp[-1].minor.yy520 = createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0); } break; - case 384: /* signed_literal ::= duration_literal */ - case 386: /* signed_literal ::= literal_func */ yytestcase(yyruleno==386); - case 457: /* star_func_para ::= expr_or_subquery */ yytestcase(yyruleno==457); - case 519: /* select_item ::= common_expression */ yytestcase(yyruleno==519); - case 529: /* partition_item ::= expr_or_subquery */ yytestcase(yyruleno==529); - case 566: /* query_simple_or_subquery ::= subquery */ yytestcase(yyruleno==566); - case 568: /* query_or_subquery ::= subquery */ yytestcase(yyruleno==568); - case 581: /* search_condition ::= common_expression */ yytestcase(yyruleno==581); -#line 692 "sql.y" -{ yylhsminor.yy164 = releaseRawExprNode(pCxt, yymsp[0].minor.yy164); } -#line 6334 "sql.c" - yymsp[0].minor.yy164 = yylhsminor.yy164; + case 387: /* signed_literal ::= duration_literal */ + case 389: /* signed_literal ::= literal_func */ yytestcase(yyruleno==389); + case 460: /* star_func_para ::= expr_or_subquery */ yytestcase(yyruleno==460); + case 522: /* select_item ::= common_expression */ yytestcase(yyruleno==522); + case 532: /* partition_item ::= expr_or_subquery */ yytestcase(yyruleno==532); + case 569: /* query_simple_or_subquery ::= subquery */ yytestcase(yyruleno==569); + case 571: /* query_or_subquery ::= subquery */ yytestcase(yyruleno==571); + case 584: /* search_condition ::= common_expression */ yytestcase(yyruleno==584); +{ yylhsminor.yy520 = releaseRawExprNode(pCxt, yymsp[0].minor.yy520); } + yymsp[0].minor.yy520 = yylhsminor.yy520; break; - case 385: /* signed_literal ::= NULL */ -#line 693 "sql.y" -{ yylhsminor.yy164 = createValueNode(pCxt, TSDB_DATA_TYPE_NULL, &yymsp[0].minor.yy0); } -#line 6340 "sql.c" - yymsp[0].minor.yy164 = yylhsminor.yy164; + case 388: /* signed_literal ::= NULL */ +{ yylhsminor.yy520 = createValueNode(pCxt, TSDB_DATA_TYPE_NULL, &yymsp[0].minor.yy0); } + yymsp[0].minor.yy520 = yylhsminor.yy520; break; - case 387: /* signed_literal ::= NK_QUESTION */ -#line 695 "sql.y" -{ yylhsminor.yy164 = createPlaceholderValueNode(pCxt, &yymsp[0].minor.yy0); } -#line 6346 "sql.c" - yymsp[0].minor.yy164 = yylhsminor.yy164; + case 390: /* signed_literal ::= NK_QUESTION */ +{ yylhsminor.yy520 = createPlaceholderValueNode(pCxt, &yymsp[0].minor.yy0); } + yymsp[0].minor.yy520 = yylhsminor.yy520; break; - case 407: /* expression ::= NK_LP expression NK_RP */ - case 491: /* boolean_primary ::= NK_LP boolean_value_expression NK_RP */ yytestcase(yyruleno==491); - case 580: /* subquery ::= NK_LP subquery NK_RP */ yytestcase(yyruleno==580); -#line 756 "sql.y" -{ yylhsminor.yy164 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0, releaseRawExprNode(pCxt, yymsp[-1].minor.yy164)); } -#line 6354 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 410: /* expression ::= NK_LP expression NK_RP */ + case 494: /* boolean_primary ::= NK_LP boolean_value_expression NK_RP */ yytestcase(yyruleno==494); + case 583: /* subquery ::= NK_LP subquery NK_RP */ yytestcase(yyruleno==583); +{ yylhsminor.yy520 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0, releaseRawExprNode(pCxt, yymsp[-1].minor.yy520)); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 408: /* expression ::= NK_PLUS expr_or_subquery */ -#line 757 "sql.y" + case 411: /* expression ::= NK_PLUS expr_or_subquery */ { - SToken t = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy164); - yylhsminor.yy164 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &t, releaseRawExprNode(pCxt, yymsp[0].minor.yy164)); + SToken t = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy520); + yylhsminor.yy520 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &t, releaseRawExprNode(pCxt, yymsp[0].minor.yy520)); } -#line 6363 "sql.c" - yymsp[-1].minor.yy164 = yylhsminor.yy164; + yymsp[-1].minor.yy520 = yylhsminor.yy520; break; - case 409: /* expression ::= NK_MINUS expr_or_subquery */ -#line 761 "sql.y" + case 412: /* expression ::= NK_MINUS expr_or_subquery */ { - SToken t = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy164); - yylhsminor.yy164 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &t, createOperatorNode(pCxt, OP_TYPE_MINUS, releaseRawExprNode(pCxt, yymsp[0].minor.yy164), NULL)); + SToken t = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy520); + yylhsminor.yy520 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &t, createOperatorNode(pCxt, OP_TYPE_MINUS, releaseRawExprNode(pCxt, yymsp[0].minor.yy520), NULL)); } -#line 6372 "sql.c" - yymsp[-1].minor.yy164 = yylhsminor.yy164; + yymsp[-1].minor.yy520 = yylhsminor.yy520; break; - case 410: /* expression ::= expr_or_subquery NK_PLUS expr_or_subquery */ -#line 765 "sql.y" + case 413: /* expression ::= expr_or_subquery NK_PLUS expr_or_subquery */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy164); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy164); - yylhsminor.yy164 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_ADD, releaseRawExprNode(pCxt, yymsp[-2].minor.yy164), releaseRawExprNode(pCxt, yymsp[0].minor.yy164))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy520); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy520); + yylhsminor.yy520 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_ADD, releaseRawExprNode(pCxt, yymsp[-2].minor.yy520), releaseRawExprNode(pCxt, yymsp[0].minor.yy520))); } -#line 6382 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 411: /* expression ::= expr_or_subquery NK_MINUS expr_or_subquery */ -#line 770 "sql.y" + case 414: /* expression ::= expr_or_subquery NK_MINUS expr_or_subquery */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy164); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy164); - yylhsminor.yy164 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_SUB, releaseRawExprNode(pCxt, yymsp[-2].minor.yy164), releaseRawExprNode(pCxt, yymsp[0].minor.yy164))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy520); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy520); + yylhsminor.yy520 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_SUB, releaseRawExprNode(pCxt, yymsp[-2].minor.yy520), releaseRawExprNode(pCxt, yymsp[0].minor.yy520))); } -#line 6392 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 412: /* expression ::= expr_or_subquery NK_STAR expr_or_subquery */ -#line 775 "sql.y" + case 415: /* expression ::= expr_or_subquery NK_STAR expr_or_subquery */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy164); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy164); - yylhsminor.yy164 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_MULTI, releaseRawExprNode(pCxt, yymsp[-2].minor.yy164), releaseRawExprNode(pCxt, yymsp[0].minor.yy164))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy520); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy520); + yylhsminor.yy520 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_MULTI, releaseRawExprNode(pCxt, yymsp[-2].minor.yy520), releaseRawExprNode(pCxt, yymsp[0].minor.yy520))); } -#line 6402 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 413: /* expression ::= expr_or_subquery NK_SLASH expr_or_subquery */ -#line 780 "sql.y" + case 416: /* expression ::= expr_or_subquery NK_SLASH expr_or_subquery */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy164); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy164); - yylhsminor.yy164 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_DIV, releaseRawExprNode(pCxt, yymsp[-2].minor.yy164), releaseRawExprNode(pCxt, yymsp[0].minor.yy164))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy520); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy520); + yylhsminor.yy520 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_DIV, releaseRawExprNode(pCxt, yymsp[-2].minor.yy520), releaseRawExprNode(pCxt, yymsp[0].minor.yy520))); } -#line 6412 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 414: /* expression ::= expr_or_subquery NK_REM expr_or_subquery */ -#line 785 "sql.y" + case 417: /* expression ::= expr_or_subquery NK_REM expr_or_subquery */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy164); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy164); - yylhsminor.yy164 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_REM, releaseRawExprNode(pCxt, yymsp[-2].minor.yy164), releaseRawExprNode(pCxt, yymsp[0].minor.yy164))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy520); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy520); + yylhsminor.yy520 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_REM, releaseRawExprNode(pCxt, yymsp[-2].minor.yy520), releaseRawExprNode(pCxt, yymsp[0].minor.yy520))); } -#line 6422 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 415: /* expression ::= column_reference NK_ARROW NK_STRING */ -#line 790 "sql.y" + case 418: /* expression ::= column_reference NK_ARROW NK_STRING */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy164); - yylhsminor.yy164 = createRawExprNodeExt(pCxt, &s, &yymsp[0].minor.yy0, createOperatorNode(pCxt, OP_TYPE_JSON_GET_VALUE, releaseRawExprNode(pCxt, yymsp[-2].minor.yy164), createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy520); + yylhsminor.yy520 = createRawExprNodeExt(pCxt, &s, &yymsp[0].minor.yy0, createOperatorNode(pCxt, OP_TYPE_JSON_GET_VALUE, releaseRawExprNode(pCxt, yymsp[-2].minor.yy520), createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0))); } -#line 6431 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 416: /* expression ::= expr_or_subquery NK_BITAND expr_or_subquery */ -#line 794 "sql.y" + case 419: /* expression ::= expr_or_subquery NK_BITAND expr_or_subquery */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy164); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy164); - yylhsminor.yy164 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_BIT_AND, releaseRawExprNode(pCxt, yymsp[-2].minor.yy164), releaseRawExprNode(pCxt, yymsp[0].minor.yy164))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy520); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy520); + yylhsminor.yy520 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_BIT_AND, releaseRawExprNode(pCxt, yymsp[-2].minor.yy520), releaseRawExprNode(pCxt, yymsp[0].minor.yy520))); } -#line 6441 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 417: /* expression ::= expr_or_subquery NK_BITOR expr_or_subquery */ -#line 799 "sql.y" + case 420: /* expression ::= expr_or_subquery NK_BITOR expr_or_subquery */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy164); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy164); - yylhsminor.yy164 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_BIT_OR, releaseRawExprNode(pCxt, yymsp[-2].minor.yy164), releaseRawExprNode(pCxt, yymsp[0].minor.yy164))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy520); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy520); + yylhsminor.yy520 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_BIT_OR, releaseRawExprNode(pCxt, yymsp[-2].minor.yy520), releaseRawExprNode(pCxt, yymsp[0].minor.yy520))); } -#line 6451 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 420: /* column_reference ::= column_name */ -#line 810 "sql.y" -{ yylhsminor.yy164 = createRawExprNode(pCxt, &yymsp[0].minor.yy497, createColumnNode(pCxt, NULL, &yymsp[0].minor.yy497)); } -#line 6457 "sql.c" - yymsp[0].minor.yy164 = yylhsminor.yy164; + case 423: /* column_reference ::= column_name */ +{ yylhsminor.yy520 = createRawExprNode(pCxt, &yymsp[0].minor.yy833, createColumnNode(pCxt, NULL, &yymsp[0].minor.yy833)); } + yymsp[0].minor.yy520 = yylhsminor.yy520; break; - case 421: /* column_reference ::= table_name NK_DOT column_name */ -#line 811 "sql.y" -{ yylhsminor.yy164 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy497, &yymsp[0].minor.yy497, createColumnNode(pCxt, &yymsp[-2].minor.yy497, &yymsp[0].minor.yy497)); } -#line 6463 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 424: /* column_reference ::= table_name NK_DOT column_name */ +{ yylhsminor.yy520 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy833, &yymsp[0].minor.yy833, createColumnNode(pCxt, &yymsp[-2].minor.yy833, &yymsp[0].minor.yy833)); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 422: /* pseudo_column ::= ROWTS */ - case 423: /* pseudo_column ::= TBNAME */ yytestcase(yyruleno==423); - case 425: /* pseudo_column ::= QSTART */ yytestcase(yyruleno==425); - case 426: /* pseudo_column ::= QEND */ yytestcase(yyruleno==426); - case 427: /* pseudo_column ::= QDURATION */ yytestcase(yyruleno==427); - case 428: /* pseudo_column ::= WSTART */ yytestcase(yyruleno==428); - case 429: /* pseudo_column ::= WEND */ yytestcase(yyruleno==429); - case 430: /* pseudo_column ::= WDURATION */ yytestcase(yyruleno==430); - case 431: /* pseudo_column ::= IROWTS */ yytestcase(yyruleno==431); - case 432: /* pseudo_column ::= ISFILLED */ yytestcase(yyruleno==432); - case 433: /* pseudo_column ::= QTAGS */ yytestcase(yyruleno==433); - case 439: /* literal_func ::= NOW */ yytestcase(yyruleno==439); -#line 813 "sql.y" -{ yylhsminor.yy164 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[0].minor.yy0, NULL)); } -#line 6480 "sql.c" - yymsp[0].minor.yy164 = yylhsminor.yy164; + case 425: /* pseudo_column ::= ROWTS */ + case 426: /* pseudo_column ::= TBNAME */ yytestcase(yyruleno==426); + case 428: /* pseudo_column ::= QSTART */ yytestcase(yyruleno==428); + case 429: /* pseudo_column ::= QEND */ yytestcase(yyruleno==429); + case 430: /* pseudo_column ::= QDURATION */ yytestcase(yyruleno==430); + case 431: /* pseudo_column ::= WSTART */ yytestcase(yyruleno==431); + case 432: /* pseudo_column ::= WEND */ yytestcase(yyruleno==432); + case 433: /* pseudo_column ::= WDURATION */ yytestcase(yyruleno==433); + case 434: /* pseudo_column ::= IROWTS */ yytestcase(yyruleno==434); + case 435: /* pseudo_column ::= ISFILLED */ yytestcase(yyruleno==435); + case 436: /* pseudo_column ::= QTAGS */ yytestcase(yyruleno==436); + case 442: /* literal_func ::= NOW */ yytestcase(yyruleno==442); +{ yylhsminor.yy520 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[0].minor.yy0, NULL)); } + yymsp[0].minor.yy520 = yylhsminor.yy520; break; - case 424: /* pseudo_column ::= table_name NK_DOT TBNAME */ -#line 815 "sql.y" -{ yylhsminor.yy164 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy497, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[0].minor.yy0, createNodeList(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[-2].minor.yy497)))); } -#line 6486 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 427: /* pseudo_column ::= table_name NK_DOT TBNAME */ +{ yylhsminor.yy520 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy833, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[0].minor.yy0, createNodeList(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[-2].minor.yy833)))); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 434: /* function_expression ::= function_name NK_LP expression_list NK_RP */ - case 435: /* function_expression ::= star_func NK_LP star_func_para_list NK_RP */ yytestcase(yyruleno==435); -#line 826 "sql.y" -{ yylhsminor.yy164 = createRawExprNodeExt(pCxt, &yymsp[-3].minor.yy497, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[-3].minor.yy497, yymsp[-1].minor.yy72)); } -#line 6493 "sql.c" - yymsp[-3].minor.yy164 = yylhsminor.yy164; + case 437: /* function_expression ::= function_name NK_LP expression_list NK_RP */ + case 438: /* function_expression ::= star_func NK_LP star_func_para_list NK_RP */ yytestcase(yyruleno==438); +{ yylhsminor.yy520 = createRawExprNodeExt(pCxt, &yymsp[-3].minor.yy833, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[-3].minor.yy833, yymsp[-1].minor.yy904)); } + yymsp[-3].minor.yy520 = yylhsminor.yy520; break; - case 436: /* function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP */ -#line 829 "sql.y" -{ yylhsminor.yy164 = createRawExprNodeExt(pCxt, &yymsp[-5].minor.yy0, &yymsp[0].minor.yy0, createCastFunctionNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy164), yymsp[-1].minor.yy700)); } -#line 6499 "sql.c" - yymsp[-5].minor.yy164 = yylhsminor.yy164; + case 439: /* function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP */ +{ yylhsminor.yy520 = createRawExprNodeExt(pCxt, &yymsp[-5].minor.yy0, &yymsp[0].minor.yy0, createCastFunctionNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy520), yymsp[-1].minor.yy840)); } + yymsp[-5].minor.yy520 = yylhsminor.yy520; break; - case 438: /* literal_func ::= noarg_func NK_LP NK_RP */ -#line 832 "sql.y" -{ yylhsminor.yy164 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy497, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[-2].minor.yy497, NULL)); } -#line 6505 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 441: /* literal_func ::= noarg_func NK_LP NK_RP */ +{ yylhsminor.yy520 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy833, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[-2].minor.yy833, NULL)); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 453: /* star_func_para_list ::= NK_STAR */ -#line 856 "sql.y" -{ yylhsminor.yy72 = createNodeList(pCxt, createColumnNode(pCxt, NULL, &yymsp[0].minor.yy0)); } -#line 6511 "sql.c" - yymsp[0].minor.yy72 = yylhsminor.yy72; + case 456: /* star_func_para_list ::= NK_STAR */ +{ yylhsminor.yy904 = createNodeList(pCxt, createColumnNode(pCxt, NULL, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy904 = yylhsminor.yy904; break; - case 458: /* star_func_para ::= table_name NK_DOT NK_STAR */ - case 522: /* select_item ::= table_name NK_DOT NK_STAR */ yytestcase(yyruleno==522); -#line 865 "sql.y" -{ yylhsminor.yy164 = createColumnNode(pCxt, &yymsp[-2].minor.yy497, &yymsp[0].minor.yy0); } -#line 6518 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 461: /* star_func_para ::= table_name NK_DOT NK_STAR */ + case 525: /* select_item ::= table_name NK_DOT NK_STAR */ yytestcase(yyruleno==525); +{ yylhsminor.yy520 = createColumnNode(pCxt, &yymsp[-2].minor.yy833, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 459: /* case_when_expression ::= CASE when_then_list case_when_else_opt END */ -#line 868 "sql.y" -{ yylhsminor.yy164 = createRawExprNodeExt(pCxt, &yymsp[-3].minor.yy0, &yymsp[0].minor.yy0, createCaseWhenNode(pCxt, NULL, yymsp[-2].minor.yy72, yymsp[-1].minor.yy164)); } -#line 6524 "sql.c" - yymsp[-3].minor.yy164 = yylhsminor.yy164; + case 462: /* case_when_expression ::= CASE when_then_list case_when_else_opt END */ +{ yylhsminor.yy520 = createRawExprNodeExt(pCxt, &yymsp[-3].minor.yy0, &yymsp[0].minor.yy0, createCaseWhenNode(pCxt, NULL, yymsp[-2].minor.yy904, yymsp[-1].minor.yy520)); } + yymsp[-3].minor.yy520 = yylhsminor.yy520; break; - case 460: /* case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END */ -#line 870 "sql.y" -{ yylhsminor.yy164 = createRawExprNodeExt(pCxt, &yymsp[-4].minor.yy0, &yymsp[0].minor.yy0, createCaseWhenNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy164), yymsp[-2].minor.yy72, yymsp[-1].minor.yy164)); } -#line 6530 "sql.c" - yymsp[-4].minor.yy164 = yylhsminor.yy164; + case 463: /* case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END */ +{ yylhsminor.yy520 = createRawExprNodeExt(pCxt, &yymsp[-4].minor.yy0, &yymsp[0].minor.yy0, createCaseWhenNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy520), yymsp[-2].minor.yy904, yymsp[-1].minor.yy520)); } + yymsp[-4].minor.yy520 = yylhsminor.yy520; break; - case 463: /* when_then_expr ::= WHEN common_expression THEN common_expression */ -#line 877 "sql.y" -{ yymsp[-3].minor.yy164 = createWhenThenNode(pCxt, releaseRawExprNode(pCxt, yymsp[-2].minor.yy164), releaseRawExprNode(pCxt, yymsp[0].minor.yy164)); } -#line 6536 "sql.c" + case 466: /* when_then_expr ::= WHEN common_expression THEN common_expression */ +{ yymsp[-3].minor.yy520 = createWhenThenNode(pCxt, releaseRawExprNode(pCxt, yymsp[-2].minor.yy520), releaseRawExprNode(pCxt, yymsp[0].minor.yy520)); } break; - case 465: /* case_when_else_opt ::= ELSE common_expression */ -#line 880 "sql.y" -{ yymsp[-1].minor.yy164 = releaseRawExprNode(pCxt, yymsp[0].minor.yy164); } -#line 6541 "sql.c" + case 468: /* case_when_else_opt ::= ELSE common_expression */ +{ yymsp[-1].minor.yy520 = releaseRawExprNode(pCxt, yymsp[0].minor.yy520); } break; - case 466: /* predicate ::= expr_or_subquery compare_op expr_or_subquery */ - case 471: /* predicate ::= expr_or_subquery in_op in_predicate_value */ yytestcase(yyruleno==471); -#line 883 "sql.y" + case 469: /* predicate ::= expr_or_subquery compare_op expr_or_subquery */ + case 474: /* predicate ::= expr_or_subquery in_op in_predicate_value */ yytestcase(yyruleno==474); { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy164); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy164); - yylhsminor.yy164 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, yymsp[-1].minor.yy796, releaseRawExprNode(pCxt, yymsp[-2].minor.yy164), releaseRawExprNode(pCxt, yymsp[0].minor.yy164))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy520); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy520); + yylhsminor.yy520 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, yymsp[-1].minor.yy524, releaseRawExprNode(pCxt, yymsp[-2].minor.yy520), releaseRawExprNode(pCxt, yymsp[0].minor.yy520))); } -#line 6551 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 467: /* predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery */ -#line 890 "sql.y" + case 470: /* predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-4].minor.yy164); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy164); - yylhsminor.yy164 = createRawExprNodeExt(pCxt, &s, &e, createBetweenAnd(pCxt, releaseRawExprNode(pCxt, yymsp[-4].minor.yy164), releaseRawExprNode(pCxt, yymsp[-2].minor.yy164), releaseRawExprNode(pCxt, yymsp[0].minor.yy164))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-4].minor.yy520); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy520); + yylhsminor.yy520 = createRawExprNodeExt(pCxt, &s, &e, createBetweenAnd(pCxt, releaseRawExprNode(pCxt, yymsp[-4].minor.yy520), releaseRawExprNode(pCxt, yymsp[-2].minor.yy520), releaseRawExprNode(pCxt, yymsp[0].minor.yy520))); } -#line 6561 "sql.c" - yymsp[-4].minor.yy164 = yylhsminor.yy164; + yymsp[-4].minor.yy520 = yylhsminor.yy520; break; - case 468: /* predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery */ -#line 896 "sql.y" + case 471: /* predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-5].minor.yy164); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy164); - yylhsminor.yy164 = createRawExprNodeExt(pCxt, &s, &e, createNotBetweenAnd(pCxt, releaseRawExprNode(pCxt, yymsp[-5].minor.yy164), releaseRawExprNode(pCxt, yymsp[-2].minor.yy164), releaseRawExprNode(pCxt, yymsp[0].minor.yy164))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-5].minor.yy520); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy520); + yylhsminor.yy520 = createRawExprNodeExt(pCxt, &s, &e, createNotBetweenAnd(pCxt, releaseRawExprNode(pCxt, yymsp[-5].minor.yy520), releaseRawExprNode(pCxt, yymsp[-2].minor.yy520), releaseRawExprNode(pCxt, yymsp[0].minor.yy520))); } -#line 6571 "sql.c" - yymsp[-5].minor.yy164 = yylhsminor.yy164; + yymsp[-5].minor.yy520 = yylhsminor.yy520; break; - case 469: /* predicate ::= expr_or_subquery IS NULL */ -#line 901 "sql.y" + case 472: /* predicate ::= expr_or_subquery IS NULL */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy164); - yylhsminor.yy164 = createRawExprNodeExt(pCxt, &s, &yymsp[0].minor.yy0, createOperatorNode(pCxt, OP_TYPE_IS_NULL, releaseRawExprNode(pCxt, yymsp[-2].minor.yy164), NULL)); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy520); + yylhsminor.yy520 = createRawExprNodeExt(pCxt, &s, &yymsp[0].minor.yy0, createOperatorNode(pCxt, OP_TYPE_IS_NULL, releaseRawExprNode(pCxt, yymsp[-2].minor.yy520), NULL)); } -#line 6580 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 470: /* predicate ::= expr_or_subquery IS NOT NULL */ -#line 905 "sql.y" + case 473: /* predicate ::= expr_or_subquery IS NOT NULL */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-3].minor.yy164); - yylhsminor.yy164 = createRawExprNodeExt(pCxt, &s, &yymsp[0].minor.yy0, createOperatorNode(pCxt, OP_TYPE_IS_NOT_NULL, releaseRawExprNode(pCxt, yymsp[-3].minor.yy164), NULL)); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-3].minor.yy520); + yylhsminor.yy520 = createRawExprNodeExt(pCxt, &s, &yymsp[0].minor.yy0, createOperatorNode(pCxt, OP_TYPE_IS_NOT_NULL, releaseRawExprNode(pCxt, yymsp[-3].minor.yy520), NULL)); } -#line 6589 "sql.c" - yymsp[-3].minor.yy164 = yylhsminor.yy164; + yymsp[-3].minor.yy520 = yylhsminor.yy520; break; - case 472: /* compare_op ::= NK_LT */ -#line 917 "sql.y" -{ yymsp[0].minor.yy796 = OP_TYPE_LOWER_THAN; } -#line 6595 "sql.c" + case 475: /* compare_op ::= NK_LT */ +{ yymsp[0].minor.yy524 = OP_TYPE_LOWER_THAN; } break; - case 473: /* compare_op ::= NK_GT */ -#line 918 "sql.y" -{ yymsp[0].minor.yy796 = OP_TYPE_GREATER_THAN; } -#line 6600 "sql.c" + case 476: /* compare_op ::= NK_GT */ +{ yymsp[0].minor.yy524 = OP_TYPE_GREATER_THAN; } break; - case 474: /* compare_op ::= NK_LE */ -#line 919 "sql.y" -{ yymsp[0].minor.yy796 = OP_TYPE_LOWER_EQUAL; } -#line 6605 "sql.c" + case 477: /* compare_op ::= NK_LE */ +{ yymsp[0].minor.yy524 = OP_TYPE_LOWER_EQUAL; } break; - case 475: /* compare_op ::= NK_GE */ -#line 920 "sql.y" -{ yymsp[0].minor.yy796 = OP_TYPE_GREATER_EQUAL; } -#line 6610 "sql.c" + case 478: /* compare_op ::= NK_GE */ +{ yymsp[0].minor.yy524 = OP_TYPE_GREATER_EQUAL; } break; - case 476: /* compare_op ::= NK_NE */ -#line 921 "sql.y" -{ yymsp[0].minor.yy796 = OP_TYPE_NOT_EQUAL; } -#line 6615 "sql.c" + case 479: /* compare_op ::= NK_NE */ +{ yymsp[0].minor.yy524 = OP_TYPE_NOT_EQUAL; } break; - case 477: /* compare_op ::= NK_EQ */ -#line 922 "sql.y" -{ yymsp[0].minor.yy796 = OP_TYPE_EQUAL; } -#line 6620 "sql.c" + case 480: /* compare_op ::= NK_EQ */ +{ yymsp[0].minor.yy524 = OP_TYPE_EQUAL; } break; - case 478: /* compare_op ::= LIKE */ -#line 923 "sql.y" -{ yymsp[0].minor.yy796 = OP_TYPE_LIKE; } -#line 6625 "sql.c" + case 481: /* compare_op ::= LIKE */ +{ yymsp[0].minor.yy524 = OP_TYPE_LIKE; } break; - case 479: /* compare_op ::= NOT LIKE */ -#line 924 "sql.y" -{ yymsp[-1].minor.yy796 = OP_TYPE_NOT_LIKE; } -#line 6630 "sql.c" + case 482: /* compare_op ::= NOT LIKE */ +{ yymsp[-1].minor.yy524 = OP_TYPE_NOT_LIKE; } break; - case 480: /* compare_op ::= MATCH */ -#line 925 "sql.y" -{ yymsp[0].minor.yy796 = OP_TYPE_MATCH; } -#line 6635 "sql.c" + case 483: /* compare_op ::= MATCH */ +{ yymsp[0].minor.yy524 = OP_TYPE_MATCH; } break; - case 481: /* compare_op ::= NMATCH */ -#line 926 "sql.y" -{ yymsp[0].minor.yy796 = OP_TYPE_NMATCH; } -#line 6640 "sql.c" + case 484: /* compare_op ::= NMATCH */ +{ yymsp[0].minor.yy524 = OP_TYPE_NMATCH; } break; - case 482: /* compare_op ::= CONTAINS */ -#line 927 "sql.y" -{ yymsp[0].minor.yy796 = OP_TYPE_JSON_CONTAINS; } -#line 6645 "sql.c" + case 485: /* compare_op ::= CONTAINS */ +{ yymsp[0].minor.yy524 = OP_TYPE_JSON_CONTAINS; } break; - case 483: /* in_op ::= IN */ -#line 931 "sql.y" -{ yymsp[0].minor.yy796 = OP_TYPE_IN; } -#line 6650 "sql.c" + case 486: /* in_op ::= IN */ +{ yymsp[0].minor.yy524 = OP_TYPE_IN; } break; - case 484: /* in_op ::= NOT IN */ -#line 932 "sql.y" -{ yymsp[-1].minor.yy796 = OP_TYPE_NOT_IN; } -#line 6655 "sql.c" + case 487: /* in_op ::= NOT IN */ +{ yymsp[-1].minor.yy524 = OP_TYPE_NOT_IN; } break; - case 485: /* in_predicate_value ::= NK_LP literal_list NK_RP */ -#line 934 "sql.y" -{ yylhsminor.yy164 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0, createNodeListNode(pCxt, yymsp[-1].minor.yy72)); } -#line 6660 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 488: /* in_predicate_value ::= NK_LP literal_list NK_RP */ +{ yylhsminor.yy520 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0, createNodeListNode(pCxt, yymsp[-1].minor.yy904)); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 487: /* boolean_value_expression ::= NOT boolean_primary */ -#line 938 "sql.y" + case 490: /* boolean_value_expression ::= NOT boolean_primary */ { - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy164); - yylhsminor.yy164 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &e, createLogicConditionNode(pCxt, LOGIC_COND_TYPE_NOT, releaseRawExprNode(pCxt, yymsp[0].minor.yy164), NULL)); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy520); + yylhsminor.yy520 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &e, createLogicConditionNode(pCxt, LOGIC_COND_TYPE_NOT, releaseRawExprNode(pCxt, yymsp[0].minor.yy520), NULL)); } -#line 6669 "sql.c" - yymsp[-1].minor.yy164 = yylhsminor.yy164; + yymsp[-1].minor.yy520 = yylhsminor.yy520; break; - case 488: /* boolean_value_expression ::= boolean_value_expression OR boolean_value_expression */ -#line 943 "sql.y" + case 491: /* boolean_value_expression ::= boolean_value_expression OR boolean_value_expression */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy164); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy164); - yylhsminor.yy164 = createRawExprNodeExt(pCxt, &s, &e, createLogicConditionNode(pCxt, LOGIC_COND_TYPE_OR, releaseRawExprNode(pCxt, yymsp[-2].minor.yy164), releaseRawExprNode(pCxt, yymsp[0].minor.yy164))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy520); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy520); + yylhsminor.yy520 = createRawExprNodeExt(pCxt, &s, &e, createLogicConditionNode(pCxt, LOGIC_COND_TYPE_OR, releaseRawExprNode(pCxt, yymsp[-2].minor.yy520), releaseRawExprNode(pCxt, yymsp[0].minor.yy520))); } -#line 6679 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 489: /* boolean_value_expression ::= boolean_value_expression AND boolean_value_expression */ -#line 949 "sql.y" + case 492: /* boolean_value_expression ::= boolean_value_expression AND boolean_value_expression */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy164); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy164); - yylhsminor.yy164 = createRawExprNodeExt(pCxt, &s, &e, createLogicConditionNode(pCxt, LOGIC_COND_TYPE_AND, releaseRawExprNode(pCxt, yymsp[-2].minor.yy164), releaseRawExprNode(pCxt, yymsp[0].minor.yy164))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy520); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy520); + yylhsminor.yy520 = createRawExprNodeExt(pCxt, &s, &e, createLogicConditionNode(pCxt, LOGIC_COND_TYPE_AND, releaseRawExprNode(pCxt, yymsp[-2].minor.yy520), releaseRawExprNode(pCxt, yymsp[0].minor.yy520))); } -#line 6689 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 497: /* table_reference_list ::= table_reference_list NK_COMMA table_reference */ -#line 967 "sql.y" -{ yylhsminor.yy164 = createJoinTableNode(pCxt, JOIN_TYPE_INNER, yymsp[-2].minor.yy164, yymsp[0].minor.yy164, NULL); } -#line 6695 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 500: /* table_reference_list ::= table_reference_list NK_COMMA table_reference */ +{ yylhsminor.yy520 = createJoinTableNode(pCxt, JOIN_TYPE_INNER, yymsp[-2].minor.yy520, yymsp[0].minor.yy520, NULL); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 500: /* table_primary ::= table_name alias_opt */ -#line 973 "sql.y" -{ yylhsminor.yy164 = createRealTableNode(pCxt, NULL, &yymsp[-1].minor.yy497, &yymsp[0].minor.yy497); } -#line 6701 "sql.c" - yymsp[-1].minor.yy164 = yylhsminor.yy164; + case 503: /* table_primary ::= table_name alias_opt */ +{ yylhsminor.yy520 = createRealTableNode(pCxt, NULL, &yymsp[-1].minor.yy833, &yymsp[0].minor.yy833); } + yymsp[-1].minor.yy520 = yylhsminor.yy520; break; - case 501: /* table_primary ::= db_name NK_DOT table_name alias_opt */ -#line 974 "sql.y" -{ yylhsminor.yy164 = createRealTableNode(pCxt, &yymsp[-3].minor.yy497, &yymsp[-1].minor.yy497, &yymsp[0].minor.yy497); } -#line 6707 "sql.c" - yymsp[-3].minor.yy164 = yylhsminor.yy164; + case 504: /* table_primary ::= db_name NK_DOT table_name alias_opt */ +{ yylhsminor.yy520 = createRealTableNode(pCxt, &yymsp[-3].minor.yy833, &yymsp[-1].minor.yy833, &yymsp[0].minor.yy833); } + yymsp[-3].minor.yy520 = yylhsminor.yy520; break; - case 502: /* table_primary ::= subquery alias_opt */ -#line 975 "sql.y" -{ yylhsminor.yy164 = createTempTableNode(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy164), &yymsp[0].minor.yy497); } -#line 6713 "sql.c" - yymsp[-1].minor.yy164 = yylhsminor.yy164; + case 505: /* table_primary ::= subquery alias_opt */ +{ yylhsminor.yy520 = createTempTableNode(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy520), &yymsp[0].minor.yy833); } + yymsp[-1].minor.yy520 = yylhsminor.yy520; break; - case 504: /* alias_opt ::= */ -#line 980 "sql.y" -{ yymsp[1].minor.yy497 = nil_token; } -#line 6719 "sql.c" + case 507: /* alias_opt ::= */ +{ yymsp[1].minor.yy833 = nil_token; } break; - case 506: /* alias_opt ::= AS table_alias */ -#line 982 "sql.y" -{ yymsp[-1].minor.yy497 = yymsp[0].minor.yy497; } -#line 6724 "sql.c" + case 509: /* alias_opt ::= AS table_alias */ +{ yymsp[-1].minor.yy833 = yymsp[0].minor.yy833; } break; - case 507: /* parenthesized_joined_table ::= NK_LP joined_table NK_RP */ - case 508: /* parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP */ yytestcase(yyruleno==508); -#line 984 "sql.y" -{ yymsp[-2].minor.yy164 = yymsp[-1].minor.yy164; } -#line 6730 "sql.c" + case 510: /* parenthesized_joined_table ::= NK_LP joined_table NK_RP */ + case 511: /* parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP */ yytestcase(yyruleno==511); +{ yymsp[-2].minor.yy520 = yymsp[-1].minor.yy520; } break; - case 509: /* joined_table ::= table_reference join_type JOIN table_reference ON search_condition */ -#line 989 "sql.y" -{ yylhsminor.yy164 = createJoinTableNode(pCxt, yymsp[-4].minor.yy196, yymsp[-5].minor.yy164, yymsp[-2].minor.yy164, yymsp[0].minor.yy164); } -#line 6735 "sql.c" - yymsp[-5].minor.yy164 = yylhsminor.yy164; + case 512: /* joined_table ::= table_reference join_type JOIN table_reference ON search_condition */ +{ yylhsminor.yy520 = createJoinTableNode(pCxt, yymsp[-4].minor.yy596, yymsp[-5].minor.yy520, yymsp[-2].minor.yy520, yymsp[0].minor.yy520); } + yymsp[-5].minor.yy520 = yylhsminor.yy520; break; - case 510: /* join_type ::= */ -#line 993 "sql.y" -{ yymsp[1].minor.yy196 = JOIN_TYPE_INNER; } -#line 6741 "sql.c" + case 513: /* join_type ::= */ +{ yymsp[1].minor.yy596 = JOIN_TYPE_INNER; } break; - case 511: /* join_type ::= INNER */ -#line 994 "sql.y" -{ yymsp[0].minor.yy196 = JOIN_TYPE_INNER; } -#line 6746 "sql.c" + case 514: /* join_type ::= INNER */ +{ yymsp[0].minor.yy596 = JOIN_TYPE_INNER; } break; - case 512: /* query_specification ::= SELECT set_quantifier_opt select_list from_clause_opt where_clause_opt partition_by_clause_opt range_opt every_opt fill_opt twindow_clause_opt group_by_clause_opt having_clause_opt */ -#line 1000 "sql.y" + case 515: /* query_specification ::= SELECT set_quantifier_opt select_list from_clause_opt where_clause_opt partition_by_clause_opt range_opt every_opt fill_opt twindow_clause_opt group_by_clause_opt having_clause_opt */ { - yymsp[-11].minor.yy164 = createSelectStmt(pCxt, yymsp[-10].minor.yy441, yymsp[-9].minor.yy72, yymsp[-8].minor.yy164); - yymsp[-11].minor.yy164 = addWhereClause(pCxt, yymsp[-11].minor.yy164, yymsp[-7].minor.yy164); - yymsp[-11].minor.yy164 = addPartitionByClause(pCxt, yymsp[-11].minor.yy164, yymsp[-6].minor.yy72); - yymsp[-11].minor.yy164 = addWindowClauseClause(pCxt, yymsp[-11].minor.yy164, yymsp[-2].minor.yy164); - yymsp[-11].minor.yy164 = addGroupByClause(pCxt, yymsp[-11].minor.yy164, yymsp[-1].minor.yy72); - yymsp[-11].minor.yy164 = addHavingClause(pCxt, yymsp[-11].minor.yy164, yymsp[0].minor.yy164); - yymsp[-11].minor.yy164 = addRangeClause(pCxt, yymsp[-11].minor.yy164, yymsp[-5].minor.yy164); - yymsp[-11].minor.yy164 = addEveryClause(pCxt, yymsp[-11].minor.yy164, yymsp[-4].minor.yy164); - yymsp[-11].minor.yy164 = addFillClause(pCxt, yymsp[-11].minor.yy164, yymsp[-3].minor.yy164); + yymsp[-11].minor.yy520 = createSelectStmt(pCxt, yymsp[-10].minor.yy537, yymsp[-9].minor.yy904, yymsp[-8].minor.yy520); + yymsp[-11].minor.yy520 = addWhereClause(pCxt, yymsp[-11].minor.yy520, yymsp[-7].minor.yy520); + yymsp[-11].minor.yy520 = addPartitionByClause(pCxt, yymsp[-11].minor.yy520, yymsp[-6].minor.yy904); + yymsp[-11].minor.yy520 = addWindowClauseClause(pCxt, yymsp[-11].minor.yy520, yymsp[-2].minor.yy520); + yymsp[-11].minor.yy520 = addGroupByClause(pCxt, yymsp[-11].minor.yy520, yymsp[-1].minor.yy904); + yymsp[-11].minor.yy520 = addHavingClause(pCxt, yymsp[-11].minor.yy520, yymsp[0].minor.yy520); + yymsp[-11].minor.yy520 = addRangeClause(pCxt, yymsp[-11].minor.yy520, yymsp[-5].minor.yy520); + yymsp[-11].minor.yy520 = addEveryClause(pCxt, yymsp[-11].minor.yy520, yymsp[-4].minor.yy520); + yymsp[-11].minor.yy520 = addFillClause(pCxt, yymsp[-11].minor.yy520, yymsp[-3].minor.yy520); } -#line 6761 "sql.c" break; - case 515: /* set_quantifier_opt ::= ALL */ -#line 1016 "sql.y" -{ yymsp[0].minor.yy441 = false; } -#line 6766 "sql.c" + case 518: /* set_quantifier_opt ::= ALL */ +{ yymsp[0].minor.yy537 = false; } break; - case 518: /* select_item ::= NK_STAR */ -#line 1023 "sql.y" -{ yylhsminor.yy164 = createColumnNode(pCxt, NULL, &yymsp[0].minor.yy0); } -#line 6771 "sql.c" - yymsp[0].minor.yy164 = yylhsminor.yy164; + case 521: /* select_item ::= NK_STAR */ +{ yylhsminor.yy520 = createColumnNode(pCxt, NULL, &yymsp[0].minor.yy0); } + yymsp[0].minor.yy520 = yylhsminor.yy520; break; - case 520: /* select_item ::= common_expression column_alias */ - case 530: /* partition_item ::= expr_or_subquery column_alias */ yytestcase(yyruleno==530); -#line 1025 "sql.y" -{ yylhsminor.yy164 = setProjectionAlias(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy164), &yymsp[0].minor.yy497); } -#line 6778 "sql.c" - yymsp[-1].minor.yy164 = yylhsminor.yy164; + case 523: /* select_item ::= common_expression column_alias */ + case 533: /* partition_item ::= expr_or_subquery column_alias */ yytestcase(yyruleno==533); +{ yylhsminor.yy520 = setProjectionAlias(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy520), &yymsp[0].minor.yy833); } + yymsp[-1].minor.yy520 = yylhsminor.yy520; break; - case 521: /* select_item ::= common_expression AS column_alias */ - case 531: /* partition_item ::= expr_or_subquery AS column_alias */ yytestcase(yyruleno==531); -#line 1026 "sql.y" -{ yylhsminor.yy164 = setProjectionAlias(pCxt, releaseRawExprNode(pCxt, yymsp[-2].minor.yy164), &yymsp[0].minor.yy497); } -#line 6785 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 524: /* select_item ::= common_expression AS column_alias */ + case 534: /* partition_item ::= expr_or_subquery AS column_alias */ yytestcase(yyruleno==534); +{ yylhsminor.yy520 = setProjectionAlias(pCxt, releaseRawExprNode(pCxt, yymsp[-2].minor.yy520), &yymsp[0].minor.yy833); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 526: /* partition_by_clause_opt ::= PARTITION BY partition_list */ - case 551: /* group_by_clause_opt ::= GROUP BY group_by_list */ yytestcase(yyruleno==551); - case 570: /* order_by_clause_opt ::= ORDER BY sort_specification_list */ yytestcase(yyruleno==570); -#line 1035 "sql.y" -{ yymsp[-2].minor.yy72 = yymsp[0].minor.yy72; } -#line 6793 "sql.c" + case 529: /* partition_by_clause_opt ::= PARTITION BY partition_list */ + case 554: /* group_by_clause_opt ::= GROUP BY group_by_list */ yytestcase(yyruleno==554); + case 573: /* order_by_clause_opt ::= ORDER BY sort_specification_list */ yytestcase(yyruleno==573); +{ yymsp[-2].minor.yy904 = yymsp[0].minor.yy904; } break; - case 533: /* twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA duration_literal NK_RP */ -#line 1048 "sql.y" -{ yymsp[-5].minor.yy164 = createSessionWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy164), releaseRawExprNode(pCxt, yymsp[-1].minor.yy164)); } -#line 6798 "sql.c" + case 536: /* twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA duration_literal NK_RP */ +{ yymsp[-5].minor.yy520 = createSessionWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy520), releaseRawExprNode(pCxt, yymsp[-1].minor.yy520)); } break; - case 534: /* twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP */ -#line 1049 "sql.y" -{ yymsp[-3].minor.yy164 = createStateWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy164)); } -#line 6803 "sql.c" + case 537: /* twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP */ +{ yymsp[-3].minor.yy520 = createStateWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy520)); } break; - case 535: /* twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_RP sliding_opt fill_opt */ -#line 1051 "sql.y" -{ yymsp[-5].minor.yy164 = createIntervalWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy164), NULL, yymsp[-1].minor.yy164, yymsp[0].minor.yy164); } -#line 6808 "sql.c" + case 538: /* twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_RP sliding_opt fill_opt */ +{ yymsp[-5].minor.yy520 = createIntervalWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy520), NULL, yymsp[-1].minor.yy520, yymsp[0].minor.yy520); } break; - case 536: /* twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_COMMA duration_literal NK_RP sliding_opt fill_opt */ -#line 1054 "sql.y" -{ yymsp[-7].minor.yy164 = createIntervalWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-5].minor.yy164), releaseRawExprNode(pCxt, yymsp[-3].minor.yy164), yymsp[-1].minor.yy164, yymsp[0].minor.yy164); } -#line 6813 "sql.c" + case 539: /* twindow_clause_opt ::= INTERVAL NK_LP duration_literal NK_COMMA duration_literal NK_RP sliding_opt fill_opt */ +{ yymsp[-7].minor.yy520 = createIntervalWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-5].minor.yy520), releaseRawExprNode(pCxt, yymsp[-3].minor.yy520), yymsp[-1].minor.yy520, yymsp[0].minor.yy520); } break; - case 537: /* twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition */ -#line 1056 "sql.y" -{ yymsp[-6].minor.yy164 = createEventWindowNode(pCxt, yymsp[-3].minor.yy164, yymsp[0].minor.yy164); } -#line 6818 "sql.c" + case 540: /* twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition */ +{ yymsp[-6].minor.yy520 = createEventWindowNode(pCxt, yymsp[-3].minor.yy520, yymsp[0].minor.yy520); } break; - case 541: /* fill_opt ::= FILL NK_LP fill_mode NK_RP */ -#line 1062 "sql.y" -{ yymsp[-3].minor.yy164 = createFillNode(pCxt, yymsp[-1].minor.yy446, NULL); } -#line 6823 "sql.c" + case 544: /* fill_opt ::= FILL NK_LP fill_mode NK_RP */ +{ yymsp[-3].minor.yy520 = createFillNode(pCxt, yymsp[-1].minor.yy470, NULL); } break; - case 542: /* fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP */ -#line 1063 "sql.y" -{ yymsp[-5].minor.yy164 = createFillNode(pCxt, FILL_MODE_VALUE, createNodeListNode(pCxt, yymsp[-1].minor.yy72)); } -#line 6828 "sql.c" + case 545: /* fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP */ +{ yymsp[-5].minor.yy520 = createFillNode(pCxt, FILL_MODE_VALUE, createNodeListNode(pCxt, yymsp[-1].minor.yy904)); } break; - case 543: /* fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP */ -#line 1064 "sql.y" -{ yymsp[-5].minor.yy164 = createFillNode(pCxt, FILL_MODE_VALUE_F, createNodeListNode(pCxt, yymsp[-1].minor.yy72)); } -#line 6833 "sql.c" + case 546: /* fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP */ +{ yymsp[-5].minor.yy520 = createFillNode(pCxt, FILL_MODE_VALUE_F, createNodeListNode(pCxt, yymsp[-1].minor.yy904)); } break; - case 544: /* fill_mode ::= NONE */ -#line 1068 "sql.y" -{ yymsp[0].minor.yy446 = FILL_MODE_NONE; } -#line 6838 "sql.c" + case 547: /* fill_mode ::= NONE */ +{ yymsp[0].minor.yy470 = FILL_MODE_NONE; } break; - case 545: /* fill_mode ::= PREV */ -#line 1069 "sql.y" -{ yymsp[0].minor.yy446 = FILL_MODE_PREV; } -#line 6843 "sql.c" + case 548: /* fill_mode ::= PREV */ +{ yymsp[0].minor.yy470 = FILL_MODE_PREV; } break; - case 546: /* fill_mode ::= NULL */ -#line 1070 "sql.y" -{ yymsp[0].minor.yy446 = FILL_MODE_NULL; } -#line 6848 "sql.c" + case 549: /* fill_mode ::= NULL */ +{ yymsp[0].minor.yy470 = FILL_MODE_NULL; } break; - case 547: /* fill_mode ::= NULL_F */ -#line 1071 "sql.y" -{ yymsp[0].minor.yy446 = FILL_MODE_NULL_F; } -#line 6853 "sql.c" + case 550: /* fill_mode ::= NULL_F */ +{ yymsp[0].minor.yy470 = FILL_MODE_NULL_F; } break; - case 548: /* fill_mode ::= LINEAR */ -#line 1072 "sql.y" -{ yymsp[0].minor.yy446 = FILL_MODE_LINEAR; } -#line 6858 "sql.c" + case 551: /* fill_mode ::= LINEAR */ +{ yymsp[0].minor.yy470 = FILL_MODE_LINEAR; } break; - case 549: /* fill_mode ::= NEXT */ -#line 1073 "sql.y" -{ yymsp[0].minor.yy446 = FILL_MODE_NEXT; } -#line 6863 "sql.c" + case 552: /* fill_mode ::= NEXT */ +{ yymsp[0].minor.yy470 = FILL_MODE_NEXT; } break; - case 552: /* group_by_list ::= expr_or_subquery */ -#line 1082 "sql.y" -{ yylhsminor.yy72 = createNodeList(pCxt, createGroupingSetNode(pCxt, releaseRawExprNode(pCxt, yymsp[0].minor.yy164))); } -#line 6868 "sql.c" - yymsp[0].minor.yy72 = yylhsminor.yy72; + case 555: /* group_by_list ::= expr_or_subquery */ +{ yylhsminor.yy904 = createNodeList(pCxt, createGroupingSetNode(pCxt, releaseRawExprNode(pCxt, yymsp[0].minor.yy520))); } + yymsp[0].minor.yy904 = yylhsminor.yy904; break; - case 553: /* group_by_list ::= group_by_list NK_COMMA expr_or_subquery */ -#line 1083 "sql.y" -{ yylhsminor.yy72 = addNodeToList(pCxt, yymsp[-2].minor.yy72, createGroupingSetNode(pCxt, releaseRawExprNode(pCxt, yymsp[0].minor.yy164))); } -#line 6874 "sql.c" - yymsp[-2].minor.yy72 = yylhsminor.yy72; + case 556: /* group_by_list ::= group_by_list NK_COMMA expr_or_subquery */ +{ yylhsminor.yy904 = addNodeToList(pCxt, yymsp[-2].minor.yy904, createGroupingSetNode(pCxt, releaseRawExprNode(pCxt, yymsp[0].minor.yy520))); } + yymsp[-2].minor.yy904 = yylhsminor.yy904; break; - case 557: /* range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP */ -#line 1090 "sql.y" -{ yymsp[-5].minor.yy164 = createInterpTimeRange(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy164), releaseRawExprNode(pCxt, yymsp[-1].minor.yy164)); } -#line 6880 "sql.c" + case 560: /* range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP */ +{ yymsp[-5].minor.yy520 = createInterpTimeRange(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy520), releaseRawExprNode(pCxt, yymsp[-1].minor.yy520)); } break; - case 560: /* query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt */ -#line 1097 "sql.y" + case 563: /* query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt */ { - yylhsminor.yy164 = addOrderByClause(pCxt, yymsp[-3].minor.yy164, yymsp[-2].minor.yy72); - yylhsminor.yy164 = addSlimitClause(pCxt, yylhsminor.yy164, yymsp[-1].minor.yy164); - yylhsminor.yy164 = addLimitClause(pCxt, yylhsminor.yy164, yymsp[0].minor.yy164); + yylhsminor.yy520 = addOrderByClause(pCxt, yymsp[-3].minor.yy520, yymsp[-2].minor.yy904); + yylhsminor.yy520 = addSlimitClause(pCxt, yylhsminor.yy520, yymsp[-1].minor.yy520); + yylhsminor.yy520 = addLimitClause(pCxt, yylhsminor.yy520, yymsp[0].minor.yy520); } -#line 6889 "sql.c" - yymsp[-3].minor.yy164 = yylhsminor.yy164; + yymsp[-3].minor.yy520 = yylhsminor.yy520; break; - case 563: /* union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery */ -#line 1107 "sql.y" -{ yylhsminor.yy164 = createSetOperator(pCxt, SET_OP_TYPE_UNION_ALL, yymsp[-3].minor.yy164, yymsp[0].minor.yy164); } -#line 6895 "sql.c" - yymsp[-3].minor.yy164 = yylhsminor.yy164; + case 566: /* union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery */ +{ yylhsminor.yy520 = createSetOperator(pCxt, SET_OP_TYPE_UNION_ALL, yymsp[-3].minor.yy520, yymsp[0].minor.yy520); } + yymsp[-3].minor.yy520 = yylhsminor.yy520; break; - case 564: /* union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery */ -#line 1109 "sql.y" -{ yylhsminor.yy164 = createSetOperator(pCxt, SET_OP_TYPE_UNION, yymsp[-2].minor.yy164, yymsp[0].minor.yy164); } -#line 6901 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 567: /* union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery */ +{ yylhsminor.yy520 = createSetOperator(pCxt, SET_OP_TYPE_UNION, yymsp[-2].minor.yy520, yymsp[0].minor.yy520); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 572: /* slimit_clause_opt ::= SLIMIT NK_INTEGER */ - case 576: /* limit_clause_opt ::= LIMIT NK_INTEGER */ yytestcase(yyruleno==576); -#line 1123 "sql.y" -{ yymsp[-1].minor.yy164 = createLimitNode(pCxt, &yymsp[0].minor.yy0, NULL); } -#line 6908 "sql.c" + case 575: /* slimit_clause_opt ::= SLIMIT NK_INTEGER */ + case 579: /* limit_clause_opt ::= LIMIT NK_INTEGER */ yytestcase(yyruleno==579); +{ yymsp[-1].minor.yy520 = createLimitNode(pCxt, &yymsp[0].minor.yy0, NULL); } break; - case 573: /* slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER */ - case 577: /* limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER */ yytestcase(yyruleno==577); -#line 1124 "sql.y" -{ yymsp[-3].minor.yy164 = createLimitNode(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0); } -#line 6914 "sql.c" + case 576: /* slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER */ + case 580: /* limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER */ yytestcase(yyruleno==580); +{ yymsp[-3].minor.yy520 = createLimitNode(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0); } break; - case 574: /* slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER */ - case 578: /* limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER */ yytestcase(yyruleno==578); -#line 1125 "sql.y" -{ yymsp[-3].minor.yy164 = createLimitNode(pCxt, &yymsp[0].minor.yy0, &yymsp[-2].minor.yy0); } -#line 6920 "sql.c" + case 577: /* slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER */ + case 581: /* limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER */ yytestcase(yyruleno==581); +{ yymsp[-3].minor.yy520 = createLimitNode(pCxt, &yymsp[0].minor.yy0, &yymsp[-2].minor.yy0); } break; - case 579: /* subquery ::= NK_LP query_expression NK_RP */ -#line 1133 "sql.y" -{ yylhsminor.yy164 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0, yymsp[-1].minor.yy164); } -#line 6925 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 582: /* subquery ::= NK_LP query_expression NK_RP */ +{ yylhsminor.yy520 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0, yymsp[-1].minor.yy520); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 584: /* sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt */ -#line 1147 "sql.y" -{ yylhsminor.yy164 = createOrderByExprNode(pCxt, releaseRawExprNode(pCxt, yymsp[-2].minor.yy164), yymsp[-1].minor.yy550, yymsp[0].minor.yy517); } -#line 6931 "sql.c" - yymsp[-2].minor.yy164 = yylhsminor.yy164; + case 587: /* sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt */ +{ yylhsminor.yy520 = createOrderByExprNode(pCxt, releaseRawExprNode(pCxt, yymsp[-2].minor.yy520), yymsp[-1].minor.yy906, yymsp[0].minor.yy257); } + yymsp[-2].minor.yy520 = yylhsminor.yy520; break; - case 585: /* ordering_specification_opt ::= */ -#line 1151 "sql.y" -{ yymsp[1].minor.yy550 = ORDER_ASC; } -#line 6937 "sql.c" + case 588: /* ordering_specification_opt ::= */ +{ yymsp[1].minor.yy906 = ORDER_ASC; } break; - case 586: /* ordering_specification_opt ::= ASC */ -#line 1152 "sql.y" -{ yymsp[0].minor.yy550 = ORDER_ASC; } -#line 6942 "sql.c" + case 589: /* ordering_specification_opt ::= ASC */ +{ yymsp[0].minor.yy906 = ORDER_ASC; } break; - case 587: /* ordering_specification_opt ::= DESC */ -#line 1153 "sql.y" -{ yymsp[0].minor.yy550 = ORDER_DESC; } -#line 6947 "sql.c" + case 590: /* ordering_specification_opt ::= DESC */ +{ yymsp[0].minor.yy906 = ORDER_DESC; } break; - case 588: /* null_ordering_opt ::= */ -#line 1157 "sql.y" -{ yymsp[1].minor.yy517 = NULL_ORDER_DEFAULT; } -#line 6952 "sql.c" + case 591: /* null_ordering_opt ::= */ +{ yymsp[1].minor.yy257 = NULL_ORDER_DEFAULT; } break; - case 589: /* null_ordering_opt ::= NULLS FIRST */ -#line 1158 "sql.y" -{ yymsp[-1].minor.yy517 = NULL_ORDER_FIRST; } -#line 6957 "sql.c" + case 592: /* null_ordering_opt ::= NULLS FIRST */ +{ yymsp[-1].minor.yy257 = NULL_ORDER_FIRST; } break; - case 590: /* null_ordering_opt ::= NULLS LAST */ -#line 1159 "sql.y" -{ yymsp[-1].minor.yy517 = NULL_ORDER_LAST; } -#line 6962 "sql.c" + case 593: /* null_ordering_opt ::= NULLS LAST */ +{ yymsp[-1].minor.yy257 = NULL_ORDER_LAST; } break; default: break; @@ -7020,7 +6210,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) { @@ -7031,7 +6220,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 7035 "sql.c" /************ End %syntax_error code ******************************************/ ParseARG_STORE /* Suppress warning about unused %extra_argument variable */ ParseCTX_STORE diff --git a/source/libs/parser/test/parInitialDTest.cpp b/source/libs/parser/test/parInitialDTest.cpp index 203156ec88..cddd2aa8f7 100644 --- a/source/libs/parser/test/parInitialDTest.cpp +++ b/source/libs/parser/test/parInitialDTest.cpp @@ -94,15 +94,17 @@ TEST_F(ParserInitialDTest, dropDnode) { auto clearDropDnodeReq = [&]() { memset(&expect, 0, sizeof(SDropDnodeReq)); }; - auto setDropDnodeReqById = [&](int32_t dnodeId, bool force = false) { + auto setDropDnodeReqById = [&](int32_t dnodeId, bool force = false, bool unsafe = false) { expect.dnodeId = dnodeId; expect.force = force; + expect.unsafe = unsafe; }; - auto setDropDnodeReqByEndpoint = [&](const char* pFqdn, int32_t port = tsServerPort, bool force = false) { + auto setDropDnodeReqByEndpoint = [&](const char* pFqdn, int32_t port = tsServerPort, bool force = false, bool unsafe = false) { strcpy(expect.fqdn, pFqdn); expect.port = port; expect.force = force; + expect.unsafe = unsafe; }; setCheckDdlFunc([&](const SQuery* pQuery, ParserStage stage) { @@ -114,6 +116,7 @@ TEST_F(ParserInitialDTest, dropDnode) { ASSERT_EQ(std::string(req.fqdn), std::string(expect.fqdn)); ASSERT_EQ(req.port, expect.port); ASSERT_EQ(req.force, expect.force); + ASSERT_EQ(req.unsafe, expect.unsafe); }); setDropDnodeReqById(1); @@ -124,6 +127,10 @@ TEST_F(ParserInitialDTest, dropDnode) { run("DROP DNODE 2 FORCE"); clearDropDnodeReq(); + setDropDnodeReqById(2, false, true); + run("DROP DNODE 2 UNSAFE"); + clearDropDnodeReq(); + setDropDnodeReqByEndpoint("host1", 7030); run("DROP DNODE 'host1:7030'"); clearDropDnodeReq(); @@ -132,6 +139,10 @@ TEST_F(ParserInitialDTest, dropDnode) { run("DROP DNODE 'host2:8030' FORCE"); clearDropDnodeReq(); + setDropDnodeReqByEndpoint("host2", 8030, false, true); + run("DROP DNODE 'host2:8030' UNSAFE"); + clearDropDnodeReq(); + setDropDnodeReqByEndpoint("host1"); run("DROP DNODE host1"); clearDropDnodeReq(); @@ -139,6 +150,10 @@ TEST_F(ParserInitialDTest, dropDnode) { setDropDnodeReqByEndpoint("host2", tsServerPort, true); run("DROP DNODE host2 FORCE"); clearDropDnodeReq(); + + setDropDnodeReqByEndpoint("host2", tsServerPort, false, true); + run("DROP DNODE host2 UNSAFE"); + clearDropDnodeReq(); } // todo DROP function From cbb02458ed4653192484dfe52429cd27ee71b4c5 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 16 May 2023 11:19:11 +0800 Subject: [PATCH 12/59] fix(stream): fix the seek condition. --- source/dnode/vnode/src/tq/tqRestore.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/tq/tqRestore.c b/source/dnode/vnode/src/tq/tqRestore.c index 091b8b9ef8..63d17dd6c5 100644 --- a/source/dnode/vnode/src/tq/tqRestore.c +++ b/source/dnode/vnode/src/tq/tqRestore.c @@ -124,7 +124,7 @@ int32_t createStreamRunReq(SStreamMeta* pStreamMeta, bool* pScanIdle) { tqDebug("vgId:%d s-task:%s wal reader seek to ver:%" PRId64, vgId, pTask->id.idStr, pTask->chkInfo.currentVer); } else { int64_t currentVer = walReaderGetCurrentVer(pTask->exec.pWalReader); - if (currentVer != -1 && currentVer != pTask->chkInfo.currentVer) { + if (currentVer == -1) { int32_t code = walReaderSeekVer(pTask->exec.pWalReader, pTask->chkInfo.currentVer); if (code != TSDB_CODE_SUCCESS) { // no data in wal, quit streamMetaReleaseTask(pStreamMeta, pTask); From 8e9b253574e906f886ab6f7fc4ffe8f3c7f9a4ef Mon Sep 17 00:00:00 2001 From: dmchen Date: Tue, 16 May 2023 11:31:23 +0800 Subject: [PATCH 13/59] status resp return null when only one ep --- source/dnode/mgmt/node_mgmt/src/dmTransport.c | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/source/dnode/mgmt/node_mgmt/src/dmTransport.c b/source/dnode/mgmt/node_mgmt/src/dmTransport.c index ea46b70693..7ecf2fd234 100644 --- a/source/dnode/mgmt/node_mgmt/src/dmTransport.c +++ b/source/dnode/mgmt/node_mgmt/src/dmTransport.c @@ -23,6 +23,10 @@ static inline void dmBuildMnodeRedirectRsp(SDnode *pDnode, SRpcMsg *pMsg) { SEpSet epSet = {0}; dmGetMnodeEpSetForRedirect(&pDnode->data, pMsg, &epSet); + if (epSet.numOfEps == 1) { + return; + } + const int32_t contLen = tSerializeSEpSet(NULL, 0, &epSet); pMsg->pCont = rpcMallocCont(contLen); if (pMsg->pCont == NULL) { From 9ff56a4e2b252b0c22dfc341bc7fab04d8f0cdc0 Mon Sep 17 00:00:00 2001 From: dmchen Date: Tue, 16 May 2023 14:02:54 +0800 Subject: [PATCH 14/59] unsafe command --- include/util/taoserror.h | 1 + source/common/src/tmsg.c | 7 ++++++- source/dnode/mnode/impl/inc/mndVgroup.h | 2 +- source/dnode/mnode/impl/src/mndDnode.c | 17 ++++++++++++----- source/dnode/mnode/impl/src/mndVgroup.c | 11 ++++++++--- source/util/src/terror.c | 1 + 6 files changed, 29 insertions(+), 10 deletions(-) diff --git a/include/util/taoserror.h b/include/util/taoserror.h index 8bc6ed2ac8..afa827bcb0 100644 --- a/include/util/taoserror.h +++ b/include/util/taoserror.h @@ -445,6 +445,7 @@ int32_t* taosGetErrno(); #define TSDB_CODE_VND_NOT_CATCH_UP TAOS_DEF_ERROR_CODE(0, 0x0532) // internal #define TSDB_CODE_VND_ALREADY_IS_VOTER TAOS_DEF_ERROR_CODE(0, 0x0533) // internal #define TSDB_CODE_VND_DIR_ALREADY_EXIST TAOS_DEF_ERROR_CODE(0, 0x0534) +#define TSDB_CODE_VND_META_DATA_UNSAFE_DELETE TAOS_DEF_ERROR_CODE(0, 0x0535) // tsdb #define TSDB_CODE_TDB_INVALID_TABLE_ID TAOS_DEF_ERROR_CODE(0, 0x0600) diff --git a/source/common/src/tmsg.c b/source/common/src/tmsg.c index dbf7b88ea8..0e4aeb74c9 100644 --- a/source/common/src/tmsg.c +++ b/source/common/src/tmsg.c @@ -1718,7 +1718,12 @@ int32_t tDeserializeSDropDnodeReq(void *buf, int32_t bufLen, SDropDnodeReq *pReq if (tDecodeCStrTo(&decoder, pReq->fqdn) < 0) return -1; if (tDecodeI32(&decoder, &pReq->port) < 0) return -1; if (tDecodeI8(&decoder, &pReq->force) < 0) return -1; - if (tDecodeI8(&decoder, &pReq->unsafe) < 0) return -1; + if (!tDecodeIsEnd(&decoder)) { + if (tDecodeI8(&decoder, &pReq->unsafe) < 0) return -1; + } else { + pReq->unsafe = false; + } + tEndDecode(&decoder); tDecoderClear(&decoder); diff --git a/source/dnode/mnode/impl/inc/mndVgroup.h b/source/dnode/mnode/impl/inc/mndVgroup.h index 2ece0da5eb..0cd1228f25 100644 --- a/source/dnode/mnode/impl/inc/mndVgroup.h +++ b/source/dnode/mnode/impl/inc/mndVgroup.h @@ -40,7 +40,7 @@ int32_t mndAddCreateVnodeAction(SMnode *, STrans *pTrans, SDbObj *pDb, SVgObj *p int32_t mndAddAlterVnodeConfirmAction(SMnode *, STrans *pTrans, SDbObj *pDb, SVgObj *pVgroup); int32_t mndAddAlterVnodeAction(SMnode *, STrans *pTrans, SDbObj *pDb, SVgObj *pVgroup, tmsg_t msgType); int32_t mndAddDropVnodeAction(SMnode *, STrans *pTrans, SDbObj *pDb, SVgObj *pVgroup, SVnodeGid *pVgid, bool isRedo); -int32_t mndSetMoveVgroupsInfoToTrans(SMnode *, STrans *pTrans, int32_t dropDnodeId, bool force); +int32_t mndSetMoveVgroupsInfoToTrans(SMnode *, STrans *pTrans, int32_t dropDnodeId, bool force, bool unsafe); int32_t mndBuildAlterVgroupAction(SMnode *pMnode, STrans *pTrans, SDbObj *pOldDb, SDbObj *pNewDb, SVgObj *pVgroup, SArray *pArray); int32_t mndBuildCompactVgroupAction(SMnode *pMnode, STrans *pTrans, SDbObj *pDb, SVgObj *pVgroup, int64_t compactTs, diff --git a/source/dnode/mnode/impl/src/mndDnode.c b/source/dnode/mnode/impl/src/mndDnode.c index 002407ce8a..cf331fbed8 100644 --- a/source/dnode/mnode/impl/src/mndDnode.c +++ b/source/dnode/mnode/impl/src/mndDnode.c @@ -804,7 +804,7 @@ int32_t mndProcessRestoreDnodeReqImpl(SRpcMsg *pReq){ #endif static int32_t mndDropDnode(SMnode *pMnode, SRpcMsg *pReq, SDnodeObj *pDnode, SMnodeObj *pMObj, SQnodeObj *pQObj, - SSnodeObj *pSObj, int32_t numOfVnodes, bool force) { + SSnodeObj *pSObj, int32_t numOfVnodes, bool force, bool unsafe) { int32_t code = -1; SSdbRaw *pRaw = NULL; STrans *pTrans = NULL; @@ -844,7 +844,7 @@ static int32_t mndDropDnode(SMnode *pMnode, SRpcMsg *pReq, SDnodeObj *pDnode, SM if (numOfVnodes > 0) { mInfo("trans:%d, %d vnodes on dnode:%d will be dropped", pTrans->id, numOfVnodes, pDnode->id); - if (mndSetMoveVgroupsInfoToTrans(pMnode, pTrans, pDnode->id, force) != 0) goto _OVER; + if (mndSetMoveVgroupsInfoToTrans(pMnode, pTrans, pDnode->id, force, unsafe) != 0) goto _OVER; } if (mndTransPrepare(pMnode, pTrans) != 0) goto _OVER; @@ -871,11 +871,18 @@ static int32_t mndProcessDropDnodeReq(SRpcMsg *pReq) { goto _OVER; } - mInfo("dnode:%d, start to drop, ep:%s:%d", dropReq.dnodeId, dropReq.fqdn, dropReq.port); + mInfo("dnode:%d, start to drop, ep:%s:%d, force:%s, unsafe:%s", + dropReq.dnodeId, dropReq.fqdn, dropReq.port, dropReq.force?"true":"false", dropReq.unsafe?"true":"false"); if (mndCheckOperPrivilege(pMnode, pReq->info.conn.user, MND_OPER_DROP_MNODE) != 0) { goto _OVER; } + bool force = dropReq.force; + if(dropReq.unsafe) + { + force = true; + } + pDnode = mndAcquireDnode(pMnode, dropReq.dnodeId); if (pDnode == NULL) { int32_t err = terrno; @@ -903,7 +910,7 @@ static int32_t mndProcessDropDnodeReq(SRpcMsg *pReq) { } int32_t numOfVnodes = mndGetVnodesNum(pMnode, pDnode->id); - if ((numOfVnodes > 0 || pMObj != NULL || pSObj != NULL || pQObj != NULL) && !dropReq.force) { + if ((numOfVnodes > 0 || pMObj != NULL || pSObj != NULL || pQObj != NULL) && !force) { if (!mndIsDnodeOnline(pDnode, taosGetTimestampMs())) { terrno = TSDB_CODE_DNODE_OFFLINE; mError("dnode:%d, failed to drop since %s, vnodes:%d mnode:%d qnode:%d snode:%d", pDnode->id, terrstr(), @@ -912,7 +919,7 @@ static int32_t mndProcessDropDnodeReq(SRpcMsg *pReq) { } } - code = mndDropDnode(pMnode, pReq, pDnode, pMObj, pQObj, pSObj, numOfVnodes, dropReq.force); + code = mndDropDnode(pMnode, pReq, pDnode, pMObj, pQObj, pSObj, numOfVnodes, force, dropReq.unsafe); if (code == 0) code = TSDB_CODE_ACTION_IN_PROGRESS; _OVER: diff --git a/source/dnode/mnode/impl/src/mndVgroup.c b/source/dnode/mnode/impl/src/mndVgroup.c index 5cf286354c..0e93555e80 100644 --- a/source/dnode/mnode/impl/src/mndVgroup.c +++ b/source/dnode/mnode/impl/src/mndVgroup.c @@ -1379,7 +1379,7 @@ int32_t mndAddDropVnodeAction(SMnode *pMnode, STrans *pTrans, SDbObj *pDb, SVgOb } int32_t mndSetMoveVgroupInfoToTrans(SMnode *pMnode, STrans *pTrans, SDbObj *pDb, SVgObj *pVgroup, int32_t vnIndex, - SArray *pArray, bool force) { + SArray *pArray, bool force, bool unsafe) { SVgObj newVg = {0}; memcpy(&newVg, pVgroup, sizeof(SVgObj)); @@ -1478,6 +1478,11 @@ int32_t mndSetMoveVgroupInfoToTrans(SMnode *pMnode, STrans *pTrans, SDbObj *pDb, if (mndAddAlterVnodeConfirmAction(pMnode, pTrans, pDb, &newVg) != 0) return -1; if(newVg.replica == 1){ + if(force && !unsafe){ + terrno = TSDB_CODE_VND_META_DATA_UNSAFE_DELETE; + return -1; + } + SSdb *pSdb = pMnode->pSdb; void *pIter = NULL; @@ -1518,7 +1523,7 @@ int32_t mndSetMoveVgroupInfoToTrans(SMnode *pMnode, STrans *pTrans, SDbObj *pDb, return 0; } -int32_t mndSetMoveVgroupsInfoToTrans(SMnode *pMnode, STrans *pTrans, int32_t delDnodeId, bool force) { +int32_t mndSetMoveVgroupsInfoToTrans(SMnode *pMnode, STrans *pTrans, int32_t delDnodeId, bool force, bool unsafe) { int32_t code = 0; SArray *pArray = mndBuildDnodesArray(pMnode, delDnodeId); if (pArray == NULL) return -1; @@ -1541,7 +1546,7 @@ int32_t mndSetMoveVgroupsInfoToTrans(SMnode *pMnode, STrans *pTrans, int32_t del if (vnIndex != -1) { mInfo("vgId:%d, vnode:%d will be removed from dnode:%d, force:%d", pVgroup->vgId, vnIndex, delDnodeId, force); SDbObj *pDb = mndAcquireDb(pMnode, pVgroup->dbName); - code = mndSetMoveVgroupInfoToTrans(pMnode, pTrans, pDb, pVgroup, vnIndex, pArray, force); + code = mndSetMoveVgroupInfoToTrans(pMnode, pTrans, pDb, pVgroup, vnIndex, pArray, force, unsafe); mndReleaseDb(pMnode, pDb); } diff --git a/source/util/src/terror.c b/source/util/src/terror.c index 58c7300bee..1d1a2dea27 100644 --- a/source/util/src/terror.c +++ b/source/util/src/terror.c @@ -342,6 +342,7 @@ TAOS_DEFINE_ERROR(TSDB_CODE_VND_QUERY_BUSY, "Query busy") TAOS_DEFINE_ERROR(TSDB_CODE_VND_NOT_CATCH_UP, "Vnode didn't catch up its leader") TAOS_DEFINE_ERROR(TSDB_CODE_VND_ALREADY_IS_VOTER, "Vnode already is a voter") TAOS_DEFINE_ERROR(TSDB_CODE_VND_DIR_ALREADY_EXIST, "Vnode directory already exist") +TAOS_DEFINE_ERROR(TSDB_CODE_VND_META_DATA_UNSAFE_DELETE, "Single replica vnode data will lost permanently after this operation, if you make sure this, please use drop dnode unsafe to execute") // tsdb TAOS_DEFINE_ERROR(TSDB_CODE_TDB_INVALID_TABLE_ID, "Invalid table ID") From 3b48d1aac13eae26c8cfb0f01d0b55fee2941d2c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Chappyguoxy=E2=80=9D?= <“happy_guoxy@163.com”> Date: Tue, 16 May 2023 14:17:15 +0800 Subject: [PATCH 15/59] test: rowlength64k --- .../1-insert/rowlength64k_benchmark.py | 192 ++++++++++++++++++ 1 file changed, 192 insertions(+) create mode 100755 tests/system-test/1-insert/rowlength64k_benchmark.py diff --git a/tests/system-test/1-insert/rowlength64k_benchmark.py b/tests/system-test/1-insert/rowlength64k_benchmark.py new file mode 100755 index 0000000000..e95f35fc7f --- /dev/null +++ b/tests/system-test/1-insert/rowlength64k_benchmark.py @@ -0,0 +1,192 @@ +################################################################### +# Copyright (c) 2016 by TAOS Technologies, Inc. +# All rights reserved. +# +# This file is proprietary and confidential to TAOS Technologies. +# No part of this file may be reproduced, stored, transmitted, +# disclosed or used in any form or by any means other than as +# expressly provided by the written permission from Jianhui Tao +# +################################################################### + +# -*- coding: utf-8 -*- + +import random +import os +import time +import taos +import subprocess +import string +from faker import Faker +from util.log import tdLog +from util.cases import tdCases +from util.sql import tdSql +from util.dnodes import tdDnodes +from util.dnodes import * + +class TDTestCase: + updatecfgDict = {'maxSQLLength':1048576,'debugFlag': 143 ,"querySmaOptimize":1} + + def init(self, conn, logSql, replicaVar): + tdLog.debug("start to execute %s" % __file__) + tdSql.init(conn.cursor(), logSql) + + self.testcasePath = os.path.split(__file__)[0] + self.testcaseFilename = os.path.split(__file__)[-1] + os.system("rm -rf %s/%s.sql" % (self.testcasePath,self.testcaseFilename)) + + now = time.time() + self.ts = int(round(now * 1000)) + self.num = 100 + + def getBuildPath(self): + selfPath = os.path.dirname(os.path.realpath(__file__)) + + if ("community" in selfPath): + projPath = selfPath[:selfPath.find("community")] + else: + projPath = selfPath[:selfPath.find("tests")] + + for root, dirs, files in os.walk(projPath): + if ("taosd" in files): + rootRealPath = os.path.dirname(os.path.realpath(root)) + if ("packaging" not in rootRealPath): + buildPath = root[:len(root)-len("/build/bin")] + break + return buildPath + + def run(self): + tdSql.prepare() + + startTime_all = time.time() + + buildPath = self.getBuildPath() + if (buildPath == ""): + tdLog.exit("taosd not found!") + else: + tdLog.info("taosd found in %s" % buildPath) + binPath = buildPath+ "/build/bin/" + + #-N:regular table -d:database name -t:table num -n:rows num per table -l:col num -y:force + #regular old && new + startTime = time.time() + os.system("%staosBenchmark -N -d regular_old -t 1 -n 10 -l 1023 -y" % binPath) + tdSql.execute("use regular_old") + tdSql.query("show tables;") + tdSql.checkRows(1) + tdSql.query("select * from meters;") + tdSql.checkCols(1024) + tdSql.query("describe meters;") + tdSql.checkRows(1024) + + os.system("%staosBenchmark -N -d regular_new -t 1 -n 10 -l 4095 -y" % binPath) + tdSql.execute("use regular_new") + tdSql.query("show tables;") + tdSql.checkRows(1) + tdSql.query("select * from meters;") + tdSql.checkCols(4096) + tdSql.query("describe meters;") + tdSql.checkRows(4096) + + #super table -d:database name -t:table num -n:rows num per table -l:col num -y:force + os.system("%staosBenchmark -d super_old -t 1 -n 10 -l 1021 -y" % binPath) + tdSql.execute("use super_old") + tdSql.query("show tables;") + tdSql.checkRows(1) + tdSql.query("select * from meters;") + tdSql.checkCols(1024) + tdSql.query("select * from d0;") + tdSql.checkCols(1022) + tdSql.query("describe meters;") + tdSql.checkRows(1024) + tdSql.query("describe d0;") + tdSql.checkRows(1024) + + os.system("%staosBenchmark -d super_new -t 1 -n 10 -l 4093 -y" % binPath) + tdSql.execute("use super_new") + tdSql.query("show tables;") + tdSql.checkRows(1) + tdSql.query("select * from meters;") + tdSql.checkCols(4096) + tdSql.query("select * from d0;") + tdSql.checkCols(4094) + tdSql.query("describe meters;") + tdSql.checkRows(4096) + tdSql.query("describe d0;") + tdSql.checkRows(4096) + tdSql.execute("create table stb_new1_1 using meters tags(1,2)") + tdSql.query("select * from stb_new1_1") + tdSql.checkCols(4094) + tdSql.query("describe stb_new1_1;") + tdSql.checkRows(4096) + + # insert: create one or mutiple tables per sql and insert multiple rows per sql + os.system("%staosBenchmark -f %s/rowlength64k.json -y " % (binPath,self.testcasePath)) + tdSql.execute("use json_test") + tdSql.query("select count (tbname) from stb_old") + tdSql.checkData(0, 0, 10) + + tdSql.query("select * from stb_old") + tdSql.checkRows(10) + tdSql.checkCols(1024) + + tdSql.query("select count (tbname) from stb_new") + tdSql.checkData(0, 0, 10) + + tdSql.query("select * from stb_new") + tdSql.checkRows(10) + tdSql.checkCols(4096) + tdSql.query("describe stb_new;") + tdSql.checkRows(4096) + tdSql.query("select * from stb_new_0") + tdSql.checkRows(10) + tdSql.checkCols(4091) + tdSql.query("describe stb_new_0;") + tdSql.checkRows(4096) + tdSql.execute("create table stb_new1_1 using stb_new tags(1,2,3,4,5)") + tdSql.query("select * from stb_new1_1") + tdSql.checkCols(4091) + tdSql.query("describe stb_new1_1;") + tdSql.checkRows(4096) + + tdSql.query("select count (tbname) from stb_mix") + tdSql.checkData(0, 0, 10) + + tdSql.query("select * from stb_mix") + tdSql.checkRows(10) + tdSql.checkCols(4096) + tdSql.query("describe stb_mix;") + tdSql.checkRows(4096) + tdSql.query("select * from stb_mix_0") + tdSql.checkRows(10) + tdSql.checkCols(4092) + tdSql.query("describe stb_mix_0;") + tdSql.checkRows(4096) + + tdSql.query("select count (tbname) from stb_excel") + tdSql.checkData(0, 0, 10) + + tdSql.query("select * from stb_excel") + tdSql.checkRows(10) + tdSql.checkCols(4096) + tdSql.query("describe stb_excel;") + tdSql.checkRows(4096) + tdSql.query("select * from stb_excel_0") + tdSql.checkRows(10) + tdSql.checkCols(4092) + tdSql.query("describe stb_excel_0;") + tdSql.checkRows(4096) + endTime = time.time() + print("total time %ds" % (endTime - startTime)) + + endTime_all = time.time() + print("total time %ds" % (endTime_all - startTime_all)) + + + def stop(self): + tdSql.close() + tdLog.success("%s successfully executed" % __file__) + + +tdCases.addWindows(__file__, TDTestCase()) +tdCases.addLinux(__file__, TDTestCase()) \ No newline at end of file From df32ceff1b6f747833a16c68887483afa1716fba Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Chappyguoxy=E2=80=9D?= <“happy_guoxy@163.com”> Date: Tue, 16 May 2023 14:17:39 +0800 Subject: [PATCH 16/59] test: rowlength64k --- tests/system-test/1-insert/rowlength64k.py | 1261 ++++++++++++++++++++ 1 file changed, 1261 insertions(+) create mode 100755 tests/system-test/1-insert/rowlength64k.py diff --git a/tests/system-test/1-insert/rowlength64k.py b/tests/system-test/1-insert/rowlength64k.py new file mode 100755 index 0000000000..1e7c262b29 --- /dev/null +++ b/tests/system-test/1-insert/rowlength64k.py @@ -0,0 +1,1261 @@ +################################################################### +# Copyright (c) 2016 by TAOS Technologies, Inc. +# All rights reserved. +# +# This file is proprietary and confidential to TAOS Technologies. +# No part of this file may be reproduced, stored, transmitted, +# disclosed or used in any form or by any means other than as +# expressly provided by the written permission from Jianhui Tao +# +################################################################### + +# -*- coding: utf-8 -*- + +import random +import os +import time +import taos +import subprocess +import string +from faker import Faker +from util.log import tdLog +from util.cases import tdCases +from util.sql import tdSql +from util.dnodes import tdDnodes +from util.dnodes import * + +class TDTestCase: + updatecfgDict = {'maxSQLLength':1048576,'debugFlag': 143 ,"querySmaOptimize":1} + + def init(self, conn, logSql, replicaVar): + tdLog.debug("start to execute %s" % __file__) + tdSql.init(conn.cursor(), logSql) + + self.testcasePath = os.path.split(__file__)[0] + self.testcaseFilename = os.path.split(__file__)[-1] + os.system("rm -rf %s/%s.sql" % (self.testcasePath,self.testcaseFilename)) + + now = time.time() + self.ts = int(round(now * 1000)) + self.num = 100 + + def get_random_string(self, length): + letters = string.ascii_lowercase + result_str = ''.join(random.choice(letters) for i in range(length)) + return result_str + + def ins_query(self): + sql = 'select * from information_schema.ins_tables where table_name match "table"' + tdSql.query(sql) + + self.stable_query() + + def stable_query(self): + # select * from stable_1 where loc match '[a-z]'; + show_sql = "show db.stables;" + tdSql.query(show_sql) + queryRows = len(tdSql.queryResult) + for i in range(queryRows): + show_sql = "show db.stables;" + tdSql.query(show_sql) + stable_name = tdSql.queryResult[i][0] + + stable_sql = "select * from db.%s where loc match '[a-z]'" %stable_name + tdSql.query(stable_sql) + + def run(self): + tdSql.prepare() + + startTime_all = time.time() + + print("==============step8,stable table , mix data type==============") + sql = "create stable db.stable_16(ts timestamp, " + sql += "col4090 int ," + sql += "col4091 binary(65517))" + sql += " tags (loc binary(16370),tag_1 int,tag_2 int,tag_3 int) " + tdLog.info(len(sql)) + tdSql.execute(sql) + sql = '''create table db.table_160 using db.stable_16 + tags('%s' , '1' , '2' , '3' );'''% self.get_random_string(16370) + tdSql.execute(sql) + + for i in range(self.num): + sql = "insert into db.table_160 values(%d, " + sql += "'%d'," % i + sql += "'%s')" % self.get_random_string(65517) + tdSql.execute(sql % (self.ts + i)) + time.sleep(1) + tdSql.query("select count(*) from db.table_160") + tdSql.checkData(0, 0, self.num) + tdSql.query("select * from db.table_160") + tdSql.checkRows(self.num) + tdSql.checkCols(3) + + self.ins_query() + + #insert null value + tdLog.info('test insert null value') + sql = '''create table db.table_161 using db.stable_16 + tags('table_61' , '1' , '2' , '3' );''' + tdSql.execute(sql) + + for i in range(self.num): + sql = "insert into db.table_161(ts) values(%d) " + tdSql.execute(sql % (self.ts + i)) + time.sleep(1) + tdSql.query("select count(*) from db.table_161") + tdSql.checkData(0, 0, self.num) + tdSql.query("select * from db.table_161") + tdSql.checkRows(self.num) + tdSql.checkCols(3) + + #define TSDB_MAX_BYTES_PER_ROW 65531 TSDB_MAX_TAGS_LEN 16384 + #ts:8\int:4\smallint:2\bigint:8\bool:1\float:4\tinyint:1\nchar:4*()+2[offset]\binary:1*()+2[offset] + tdLog.info('test super table max bytes per row 65531') + sql = "create table db.stable_17(ts timestamp, " + sql += "col4090 int," + sql += "col4091 binary(65517))" + sql += " tags (loc binary(16370),tag_1 int,tag_2 int,tag_3 int) " #4*3+16370+2 + tdSql.execute(sql) + sql = '''create table db.table_170 using db.stable_17 + tags('%s' , '1' , '2' , '3' );'''% self.get_random_string(16370) + tdSql.execute(sql) + tdSql.query("select * from db.table_170") + tdSql.checkCols(3) + tdSql.query("describe db.table_170") + tdSql.checkRows(7) + + self.ins_query() + + tdLog.info('test super table drop and add column or tag') + sql = "alter stable db.stable_17 drop column col4091; " + tdSql.execute(sql) + sql = "select * from db.stable_17; " + tdSql.query(sql) + tdSql.checkCols(6) + sql = "alter table db.stable_17 add column col4091 binary(65518); " + tdSql.error(sql) + sql = "alter table db.stable_17 add column col4091 binary(65517); " + tdSql.execute(sql) + sql = "select * from db.stable_17; " + tdSql.query(sql) + tdSql.checkCols(7) + + self.ins_query() + + sql = "alter stable db.stable_17 drop tag loc; " + tdSql.execute(sql) + sql = "select * from db.stable_17; " + tdSql.query(sql) + tdSql.checkCols(6) + sql = "alter table db.stable_17 add tag loc binary(16371); " + tdSql.error(sql) + sql = "alter table db.stable_17 add tag loc binary(16370); " + tdSql.execute(sql) + sql = "select * from db.stable_17; " + tdSql.query(sql) + tdSql.checkCols(7) + + sql = "alter stable db.stable_17 drop tag tag_1; " + tdSql.execute(sql) + sql = "select * from db.stable_17; " + tdSql.query(sql) + tdSql.checkCols(6) + sql = "alter table db.stable_17 add tag tag_1 int; " + tdSql.execute(sql) + sql = "select * from db.stable_17; " + tdSql.query(sql) + tdSql.checkCols(7) + sql = "alter table db.stable_17 add tag loc1 nchar(10); " + tdSql.error(sql) + + tdLog.info('test super table max bytes per row 65531') + sql = "create table db.stable_18(ts timestamp, " + sql += "col4091 binary(65518))" + sql += " tags (loc nchar(10),tag_1 int,tag_2 int,tag_3 int) " + tdSql.error(sql) + + tdLog.info('test super table max bytes per row tag 16384') + sql = "create table db.stable_18(ts timestamp, " + sql += "col4091 binary(65517))" + sql += " tags (loc binary(16371),tag_1 int,tag_2 int,tag_3 int) " + tdSql.error(sql) + + self.ins_query() + + print("==============step9,stable table , mix data type==============") + sql = "create stable db.stable_26(ts timestamp, " + sql += "col4090 int ," + sql += "col4091 binary(65517))" + sql += " tags (loc nchar(4092),tag_1 int,tag_2 int,tag_3 int) " + tdLog.info(len(sql)) + tdSql.execute(sql) + sql = '''create table db.table_260 using db.stable_26 + tags('%s' , '1' , '2' , '3' );'''% self.get_random_string(4092) + tdSql.execute(sql) + + for i in range(self.num): + sql = "insert into db.table_260 values(%d, " + sql += "'%d'," % i + sql += "'%s')" % self.get_random_string(65517) + tdSql.execute(sql % (self.ts + i)) + time.sleep(1) + tdSql.query("select count(*) from db.table_260") + tdSql.checkData(0, 0, self.num) + tdSql.query("select * from db.table_260") + tdSql.checkRows(self.num) + tdSql.checkCols(3) + + self.ins_query() + + #insert null value + tdLog.info('test insert null value') + sql = '''create table db.table_261 using db.stable_26 + tags('table_261' , '1' , '2' , '3' );''' + tdSql.execute(sql) + + for i in range(self.num): + sql = "insert into db.table_261(ts) values(%d) " + tdSql.execute(sql % (self.ts + i)) + time.sleep(1) + tdSql.query("select count(*) from db.table_261") + tdSql.checkData(0, 0, self.num) + tdSql.query("select * from db.table_261") + tdSql.checkRows(self.num) + tdSql.checkCols(3) + + #define TSDB_MAX_BYTES_PER_ROW 65531 TSDB_MAX_TAGS_LEN 16384 + #ts:8\int:4\smallint:2\bigint:8\bool:1\float:4\tinyint:1\nchar:4*()+2[offset]\binary:1*()+2[offset] + tdLog.info('test super table max bytes per row 65531') + sql = "create table db.stable_27(ts timestamp, " + sql += "col4090 int," + sql += "col4091 binary(65517))" + sql += " tags (loc nchar(4092),tag_1 int,tag_2 int,tag_3 int) " #4*3+16370+2 + tdSql.execute(sql) + sql = '''create table db.table_270 using db.stable_27 + tags('%s' , '1' , '2' , '3' );'''% self.get_random_string(4092) + tdSql.execute(sql) + tdSql.query("select * from db.table_270") + tdSql.checkCols(3) + tdSql.query("describe db.table_270") + tdSql.checkRows(7) + + self.ins_query() + + tdLog.info('test super table drop and add column or tag') + sql = "alter stable db.stable_27 drop column col4091; " + tdSql.execute(sql) + sql = "select * from db.stable_27; " + tdSql.query(sql) + tdSql.checkCols(6) + sql = "alter table db.stable_27 add column col4091 binary(65518); " + tdSql.error(sql) + sql = "alter table db.stable_27 add column col4091 binary(65517); " + tdSql.execute(sql) + sql = "select * from db.stable_27; " + tdSql.query(sql) + tdSql.checkCols(7) + + self.ins_query() + + sql = "alter stable db.stable_27 drop tag loc; " + tdSql.execute(sql) + sql = "select * from db.stable_27; " + tdSql.query(sql) + tdSql.checkCols(6) + sql = "alter table db.stable_27 add tag loc binary(16371); " + tdSql.error(sql) + sql = "alter table db.stable_27 add tag loc binary(16370); " + tdSql.execute(sql) + sql = "select * from db.stable_27; " + tdSql.query(sql) + tdSql.checkCols(7) + + sql = "alter stable db.stable_27 drop tag tag_1; " + tdSql.execute(sql) + sql = "select * from db.stable_27; " + tdSql.query(sql) + tdSql.checkCols(6) + sql = "alter table db.stable_27 add tag tag_1 int; " + tdSql.execute(sql) + sql = "select * from db.stable_27; " + tdSql.query(sql) + tdSql.checkCols(7) + sql = "alter table db.stable_27 add tag loc1 nchar(10); " + tdSql.error(sql) + + tdLog.info('test super table max bytes per row 65531') + sql = "create table db.stable_28(ts timestamp, " + sql += "col4091 binary(65518))" + sql += " tags (loc nchar(10),tag_1 int,tag_2 int,tag_3 int) " + tdSql.error(sql) + + tdLog.info('test super table max bytes per row tag 16384') + sql = "create table db.stable_28(ts timestamp, " + sql += "col4091 binary(65517))" + sql += " tags (loc binary(16371),tag_1 int,tag_2 int,tag_3 int) " + tdSql.error(sql) + + self.ins_query() + + print("==============step1, regular table, 1 ts + 4094 cols + 1 binary==============") + startTime = time.time() + sql = "create table db.regular_table_1(ts timestamp, " + for i in range(4094): + sql += "col%d int, " % (i + 1) + sql += "col4095 binary(22))" + tdLog.info(len(sql)) + tdSql.execute(sql) + + for i in range(self.num): + sql = "insert into db.regular_table_1 values(%d, " + for j in range(4094): + str = "'%s', " % random.randint(0,1000) + sql += str + sql += "'%s')" % self.get_random_string(22) + tdSql.execute(sql % (self.ts + i)) + time.sleep(1) + tdSql.query("select count(*) from db.regular_table_1") + tdSql.checkData(0, 0, self.num) + tdSql.query("select * from db.regular_table_1") + tdSql.checkRows(self.num) + tdSql.checkCols(4096) + + self.ins_query() + + endTime = time.time() + print("total time %ds" % (endTime - startTime)) + + #insert in order + tdLog.info('test insert in order') + for i in range(self.num): + sql = "insert into db.regular_table_1 (ts,col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col4095) values(%d, " + for j in range(10): + str = "'%s', " % random.randint(0,1000) + sql += str + sql += "'%s')" % self.get_random_string(22) + tdSql.execute(sql % (self.ts + i + 1000)) + time.sleep(1) + tdSql.query("select count(*) from db.regular_table_1") + tdSql.checkData(0, 0, 2*self.num) + tdSql.query("select * from db.regular_table_1") + tdSql.checkRows(2*self.num) + tdSql.checkCols(4096) + + #insert out of order + tdLog.info('test insert out of order') + for i in range(self.num): + sql = "insert into db.regular_table_1 (ts,col123,col2213,col331,col41,col523,col236,col71,col813,col912,col1320,col4095) values(%d, " + for j in range(10): + str = "'%s', " % random.randint(0,1000) + sql += str + sql += "'%s')" % self.get_random_string(22) + tdSql.execute(sql % (self.ts + i + 2000)) + time.sleep(1) + tdSql.query("select count(*) from db.regular_table_1") + tdSql.checkData(0, 0, 3*self.num) + tdSql.query("select * from db.regular_table_1") + tdSql.checkRows(3*self.num) + tdSql.checkCols(4096) + + self.ins_query() + + + print("==============step2,regular table error col or value==============") + tdLog.info('test regular table exceeds row num') + # column > 4096 + sql = "create table db.regular_table_2(ts timestamp, " + for i in range(4095): + sql += "col%d int, " % (i + 1) + sql += "col4096 binary(22))" + tdLog.info(len(sql)) + tdSql.error(sql) + + self.ins_query() + + # column > 4096 + sql = "insert into db.regular_table_1 values(%d, " + for j in range(4095): + str = "'%s', " % random.randint(0,1000) + sql += str + sql += "'%s')" % self.get_random_string(22) + tdSql.error(sql) + + # insert column < 4096 + sql = "insert into db.regular_table_1 values(%d, " + for j in range(4092): + str = "'%s', " % random.randint(0,1000) + sql += str + sql += "'%s')" % self.get_random_string(22) + tdSql.error(sql) + + # alter column > 4096 + sql = "alter table db.regular_table_1 add column max int; " + tdSql.error(sql) + + self.ins_query() + + print("==============step3,regular table , mix data type==============") + startTime = time.time() + sql = "create table db.regular_table_3(ts timestamp, " + for i in range(2000): + sql += "col%d int, " % (i + 1) + for i in range(2000,4094): + sql += "col%d bigint, " % (i + 1) + sql += "col4095 binary(22))" + tdLog.info(len(sql)) + tdSql.execute(sql) + + for i in range(self.num): + sql = "insert into db.regular_table_3 values(%d, " + for j in range(4094): + str = "'%s', " % random.randint(0,1000) + sql += str + sql += "'%s')" % self.get_random_string(22) + tdSql.execute(sql % (self.ts + i)) + time.sleep(1) + tdSql.query("select count(*) from db.regular_table_3") + tdSql.checkData(0, 0, self.num) + tdSql.query("select * from db.regular_table_3") + tdSql.checkRows(self.num) + tdSql.checkCols(4096) + + self.ins_query() + + endTime = time.time() + print("total time %ds" % (endTime - startTime)) + + sql = "create table db.regular_table_4(ts timestamp, " + for i in range(500): + sql += "int_%d int, " % (i + 1) + for i in range(500,1000): + sql += "smallint_%d smallint, " % (i + 1) + for i in range(1000,1500): + sql += "tinyint_%d tinyint, " % (i + 1) + for i in range(1500,2000): + sql += "double_%d double, " % (i + 1) + for i in range(2000,2500): + sql += "float_%d float, " % (i + 1) + for i in range(2500,3000): + sql += "bool_%d bool, " % (i + 1) + for i in range(3000,3500): + sql += "bigint_%d bigint, " % (i + 1) + for i in range(3500,3800): + sql += "nchar_%d nchar(4), " % (i + 1) + for i in range(3800,4090): + sql += "binary_%d binary(10), " % (i + 1) + for i in range(4090,4094): + sql += "timestamp_%d timestamp, " % (i + 1) + sql += "col4095 binary(22))" + tdLog.info(len(sql)) + tdSql.execute(sql) + + self.ins_query() + + for i in range(self.num): + sql = "insert into db.regular_table_4 values(%d, " + for j in range(500): + str = "'%s', " % random.randint(-2147483647,2147483647) + sql += str + for j in range(500,1000): + str = "'%s', " % random.randint(-32767,32767 ) + sql += str + for j in range(1000,1500): + str = "'%s', " % random.randint(-127,127) + sql += str + for j in range(1500,2000): + str = "'%s', " % random.randint(-922337203685477580700,922337203685477580700) + sql += str + for j in range(2000,2500): + str = "'%s', " % random.randint(-92233720368547758070,92233720368547758070) + sql += str + for j in range(2500,3000): + str = "'%s', " % random.choice(['true','false']) + sql += str + for j in range(3000,3500): + str = "'%s', " % random.randint(-9223372036854775807,9223372036854775807) + sql += str + for j in range(3500,3800): + str = "'%s', " % self.get_random_string(4) + sql += str + for j in range(3800,4090): + str = "'%s', " % self.get_random_string(10) + sql += str + for j in range(4090,4094): + str = "%s, " % (self.ts + j) + sql += str + sql += "'%s')" % self.get_random_string(22) + tdSql.execute(sql % (self.ts + i)) + time.sleep(1) + tdSql.query("select count(*) from db.regular_table_4") + tdSql.checkData(0, 0, self.num) + tdSql.query("select * from db.regular_table_4") + tdSql.checkRows(self.num) + tdSql.checkCols(4096) + tdLog.info("end ,now new one") + + self.ins_query() + + #insert null value + tdLog.info('test insert null value') + for i in range(self.num): + sql = "insert into db.regular_table_4 values(%d, " + for j in range(2500): + str = "'%s', " % random.choice(['NULL' ,'NULL' ,'NULL' ,1 , 10 ,100 ,-100 ,-10, 88 ,66 ,'NULL' ,'NULL' ,'NULL' ]) + sql += str + for j in range(2500,3000): + str = "'%s', " % random.choice(['true' ,'false']) + sql += str + for j in range(3000,3500): + str = "'%s', " % random.randint(-9223372036854775807,9223372036854775807) + sql += str + for j in range(3500,3800): + str = "'%s', " % self.get_random_string(4) + sql += str + for j in range(3800,4090): + str = "'%s', " % self.get_random_string(10) + sql += str + for j in range(4090,4094): + str = "%s, " % (self.ts + j) + sql += str + sql += "'%s')" % self.get_random_string(22) + tdSql.execute(sql % (self.ts + i + 10000)) + time.sleep(1) + tdSql.query("select count(*) from db.regular_table_4") + tdSql.checkData(0, 0, 2*self.num) + tdSql.query("select * from db.regular_table_4") + tdSql.checkRows(2*self.num) + tdSql.checkCols(4096) + + #insert in order + tdLog.info('test insert in order') + for i in range(self.num): + sql = "insert into db.regular_table_4 (ts,int_2,int_22,int_169,smallint_537,smallint_607,tinyint_1030,tinyint_1491,double_1629,double_1808,float_2075,col4095) values(%d, " + for j in range(10): + str = "'%s', " % random.randint(0,100) + sql += str + sql += "'%s')" % self.get_random_string(22) + tdSql.execute(sql % (self.ts + i + 1000)) + time.sleep(1) + tdSql.query("select count(*) from db.regular_table_4") + tdSql.checkData(0, 0, 3*self.num) + tdSql.query("select * from db.regular_table_4") + tdSql.checkRows(3*self.num) + tdSql.checkCols(4096) + + self.ins_query() + + #insert out of order + tdLog.info('test insert out of order') + for i in range(self.num): + sql = "insert into db.regular_table_4 (ts,int_169,float_2075,int_369,tinyint_1491,tinyint_1030,float_2360,smallint_537,double_1808,double_1608,double_1629,col4095) values(%d, " + for j in range(10): + str = "'%s', " % random.randint(0,100) + sql += str + sql += "'%s')" % self.get_random_string(22) + tdSql.execute(sql % (self.ts + i + 2000)) + time.sleep(1) + tdSql.query("select count(*) from db.regular_table_4") + tdSql.checkData(0, 0, 4*self.num) + tdSql.query("select * from db.regular_table_4") + tdSql.checkRows(4*self.num) + tdSql.checkCols(4096) + + #define TSDB_MAX_BYTES_PER_ROW 49151[old:1024 && 16384] + # 3.0 old: tag binary max is 16384, col+ts binary max 49151 + # 3.0 new: tag binary max is 16384-2, col+ts binary max 65531 + #ts:8\int:4\smallint:2\bigint:8\bool:1\float:4\tinyint:1\nchar:4*()+2[offset]\binary:1*()+2[offset] + tdLog.info('test regular_table max bytes per row 65531') + sql = "create table db.regular_table_5(ts timestamp, " #1*8 sum=8 + for i in range(500): + sql += "int_%d int, " % (i + 1) #500*4=2000 sum=2008 + for i in range(500,1000): + sql += "smallint_%d smallint, " % (i + 1) #500*2=1000 sum=3008 + for i in range(1000,1500): + sql += "tinyint_%d tinyint, " % (i + 1) #500*1=500 sum=3508 + for i in range(1500,2000): + sql += "double_%d double, " % (i + 1) #500*8=4000 sum=7508 + for i in range(2000,2500): + sql += "float_%d float, " % (i + 1) #500*4=2000 sum=9508 + for i in range(2500,3000): + sql += "bool_%d bool, " % (i + 1) #500*1=500 sum=10008 + for i in range(3000,3500): + sql += "bigint_%d bigint, " % (i + 1) #500*8=4000 sum=14008 + for i in range(3500,3800): + sql += "nchar_%d nchar(32), " % (i + 1) #300*(32*4+2)=39000 sum=53008 + for i in range(3800,4090): + sql += "binary_%d binary(40), " % (i + 1) #290*(40+2)=12180 sum=65188 + for i in range(4090,4094): + sql += "timestamp_%d timestamp, " % (i + 1) #4*8=32 sum=65220 + sql += "col4095 binary(309))" #309+2=311 sum=65531 + tdSql.execute(sql) + tdSql.query("select * from db.regular_table_5") + tdSql.checkCols(4096) + + sql = "alter table db.regular_table_5 modify column col4095 binary(310); " + tdSql.error(sql) + + self.ins_query() + + # drop and add + sql = "alter table db.regular_table_5 drop column col4095; " + tdSql.execute(sql) + sql = "select * from db.regular_table_5; " + tdSql.query(sql) + tdSql.checkCols(4095) + sql = "alter table db.regular_table_5 add column col4095 binary(310); " + tdSql.error(sql) + sql = "alter table db.regular_table_5 add column col4095 binary(309); " + tdSql.execute(sql) + sql = "select * from db.regular_table_5; " + tdSql.query(sql) + tdSql.checkCols(4096) + + #out TSDB_MAX_BYTES_PER_ROW 65531 + tdLog.info('test regular_table max bytes per row out 65531') + sql = "create table db.regular_table_6(ts timestamp, " + for i in range(500): + sql += "int_%d int, " % (i + 1) + for i in range(500,1000): + sql += "smallint_%d smallint, " % (i + 1) + for i in range(1000,1500): + sql += "tinyint_%d tinyint, " % (i + 1) + for i in range(1500,2000): + sql += "double_%d double, " % (i + 1) + for i in range(2000,2500): + sql += "float_%d float, " % (i + 1) + for i in range(2500,3000): + sql += "bool_%d bool, " % (i + 1) + for i in range(3000,3500): + sql += "bigint_%d bigint, " % (i + 1) + for i in range(3500,3800): + sql += "nchar_%d nchar(32), " % (i + 1) + for i in range(3800,4090): + sql += "binary_%d binary(40), " % (i + 1) + for i in range(4090,4094): + sql += "timestamp_%d timestamp, " % (i + 1) + sql += "col4095 binary(310))" + tdLog.info(len(sql)) + tdSql.error(sql) + + self.ins_query() + + + print("==============step4, super table , 1 ts + 4090 cols + 4 tags ==============") + startTime = time.time() + sql = "create stable db.stable_1(ts timestamp, " + for i in range(4090): + sql += "col%d int, " % (i + 1) + sql += "col4091 binary(22))" + sql += " tags (loc nchar(10),tag_1 int,tag_2 int,tag_3 int) " + tdLog.info(len(sql)) + tdSql.execute(sql) + sql = '''create table db.table_0 using db.stable_1 + tags('%s' , '1' , '2' , '3' );'''% self.get_random_string(10) + tdSql.execute(sql) + + for i in range(self.num): + sql = "insert into db.table_0 values(%d, " + for j in range(4090): + str = "'%s', " % random.randint(0,1000) + sql += str + sql += "'%s')" % self.get_random_string(22) + tdSql.execute(sql % (self.ts + i)) + time.sleep(1) + tdSql.query("select count(*) from db.table_0") + tdSql.checkData(0, 0, self.num) + tdSql.query("select * from db.table_0") + tdSql.checkRows(self.num) + tdSql.checkCols(4092) + + sql = '''create table db.table_1 using db.stable_1 + tags('%s' , '1' , '2' , '3' );'''% self.get_random_string(10) + tdSql.execute(sql) + + self.ins_query() + + for i in range(self.num): + sql = "insert into db.table_1 values(%d, " + for j in range(2080): + sql += "'%d', " % random.randint(0,1000) + for j in range(2080,4080): + sql += "'%s', " % 'NULL' + for j in range(4080,4090): + sql += "'%s', " % random.randint(0,10000) + sql += "'%s')" % self.get_random_string(22) + tdSql.execute(sql % (self.ts + i)) + time.sleep(1) + tdSql.query("select count(*) from db.table_1") + tdSql.checkData(0, 0, self.num) + tdSql.query("select * from db.table_1") + tdSql.checkRows(self.num) + tdSql.checkCols(4092) + + endTime = time.time() + print("total time %ds" % (endTime - startTime)) + + #insert in order + tdLog.info('test insert in order') + for i in range(self.num): + sql = "insert into db.table_1 (ts,col1,col2,col3,col4,col5,col6,col7,col8,col9,col10,col4091) values(%d, " + for j in range(10): + str = "'%s', " % random.randint(0,1000) + sql += str + sql += "'%s')" % self.get_random_string(22) + tdSql.execute(sql % (self.ts + i + 1000)) + time.sleep(1) + tdSql.query("select count(*) from db.table_1") + tdSql.checkData(0, 0, 2*self.num) + tdSql.query("select * from db.table_1") + tdSql.checkRows(2*self.num) + tdSql.checkCols(4092) + + #insert out of order + tdLog.info('test insert out of order') + for i in range(self.num): + sql = "insert into db.table_1 (ts,col123,col2213,col331,col41,col523,col236,col71,col813,col912,col1320,col4091) values(%d, " + for j in range(10): + str = "'%s', " % random.randint(0,1000) + sql += str + sql += "'%s')" % self.get_random_string(22) + tdSql.execute(sql % (self.ts + i + 2000)) + time.sleep(1) + tdSql.query("select count(*) from db.table_1") + tdSql.checkData(0, 0, 3*self.num) + tdSql.query("select * from db.table_1") + tdSql.checkRows(3*self.num) + tdSql.checkCols(4092) + + self.ins_query() + + print("==============step5,stable table , mix data type==============") + sql = "create stable db.stable_3(ts timestamp, " + for i in range(500): + sql += "int_%d int, " % (i + 1) + for i in range(500,1000): + sql += "smallint_%d smallint, " % (i + 1) + for i in range(1000,1500): + sql += "tinyint_%d tinyint, " % (i + 1) + for i in range(1500,2000): + sql += "double_%d double, " % (i + 1) + for i in range(2000,2500): + sql += "float_%d float, " % (i + 1) + for i in range(2500,3000): + sql += "bool_%d bool, " % (i + 1) + for i in range(3000,3500): + sql += "bigint_%d bigint, " % (i + 1) + for i in range(3500,3800): + sql += "nchar_%d nchar(4), " % (i + 1) + for i in range(3800,4090): + sql += "binary_%d binary(10), " % (i + 1) + sql += "col4091 binary(22))" + sql += " tags (loc binary(16370),tag_1 int,tag_2 int,tag_3 int) " + tdLog.info(len(sql)) + tdSql.execute(sql) + sql = '''create table db.table_30 using db.stable_3 + tags('%s' , '1' , '2' , '3' );'''%self.get_random_string(16370) + tdSql.execute(sql) + + self.ins_query() + + for i in range(self.num): + sql = "insert into db.table_30 values(%d, " + for j in range(500): + str = "'%s', " % random.randint(-2147483647,2147483647) + sql += str + for j in range(500,1000): + str = "'%s', " % random.randint(-32767,32767 ) + sql += str + for j in range(1000,1500): + str = "'%s', " % random.randint(-127,127) + sql += str + for j in range(1500,2000): + str = "'%s', " % random.randint(-922337203685477580700,922337203685477580700) + sql += str + for j in range(2000,2500): + str = "'%s', " % random.randint(-92233720368547758070,92233720368547758070) + sql += str + for j in range(2500,3000): + str = "'%s', " % random.choice(['true','false']) + sql += str + for j in range(3000,3500): + str = "'%s', " % random.randint(-9223372036854775807,9223372036854775807) + sql += str + for j in range(3500,3800): + str = "'%s', " % self.get_random_string(4) + sql += str + for j in range(3800,4090): + str = "'%s', " % self.get_random_string(10) + sql += str + sql += "'%s')" % self.get_random_string(22) + tdSql.execute(sql % (self.ts + i)) + time.sleep(1) + tdSql.query("select count(*) from db.table_30") + tdSql.checkData(0, 0, self.num) + tdSql.query("select * from db.table_30") + tdSql.checkRows(self.num) + tdSql.checkCols(4092) + + #insert null value + tdLog.info('test insert null value') + sql = '''create table db.table_31 using db.stable_3 + tags('%s' , '1' , '2' , '3' );'''%self.get_random_string(16370) + tdSql.execute(sql) + + for i in range(self.num): + sql = "insert into db.table_31 values(%d, " + for j in range(2500): + str = "'%s', " % random.choice(['NULL' ,'NULL' ,'NULL' ,1 , 10 ,100 ,-100 ,-10, 88 ,66 ,'NULL' ,'NULL' ,'NULL' ]) + sql += str + for j in range(2500,3000): + str = "'%s', " % random.choice(['true' ,'false']) + sql += str + for j in range(3000,3500): + str = "'%s', " % random.randint(-9223372036854775807,9223372036854775807) + sql += str + for j in range(3500,3800): + str = "'%s', " % self.get_random_string(4) + sql += str + for j in range(3800,4090): + str = "'%s', " % self.get_random_string(10) + sql += str + sql += "'%s')" % self.get_random_string(22) + tdSql.execute(sql % (self.ts + i)) + time.sleep(1) + tdSql.query("select count(*) from db.table_31") + tdSql.checkData(0, 0, self.num) + tdSql.query("select * from db.table_31") + tdSql.checkRows(self.num) + tdSql.checkCols(4092) + + self.ins_query() + + #insert in order + tdLog.info('test insert in order') + for i in range(self.num): + sql = "insert into db.table_31 (ts,int_2,int_22,int_169,smallint_537,smallint_607,tinyint_1030,tinyint_1491,double_1629,double_1808,float_2075,col4091) values(%d, " + for j in range(10): + str = "'%s', " % random.randint(0,100) + sql += str + sql += "'%s')" % self.get_random_string(22) + tdSql.execute(sql % (self.ts + i + 1000)) + time.sleep(1) + tdSql.query("select count(*) from db.table_31") + tdSql.checkData(0, 0, 2*self.num) + tdSql.query("select * from db.table_31") + tdSql.checkRows(2*self.num) + tdSql.checkCols(4092) + + #insert out of order + tdLog.info('test insert out of order') + for i in range(self.num): + sql = "insert into db.table_31 (ts,int_169,float_2075,int_369,tinyint_1491,tinyint_1030,float_2360,smallint_537,double_1808,double_1608,double_1629,col4091) values(%d, " + for j in range(10): + str = "'%s', " % random.randint(0,100) + sql += str + sql += "'%s')" % self.get_random_string(22) + tdSql.execute(sql % (self.ts + i + 2000)) + time.sleep(1) + tdSql.query("select count(*) from db.table_31") + tdSql.checkData(0, 0, 3*self.num) + tdSql.query("select * from db.table_31") + tdSql.checkRows(3*self.num) + tdSql.checkCols(4092) + + #define TSDB_MAX_BYTES_PER_ROW 65531 TSDB_MAX_TAGS_LEN 16384 + #ts:8\int:4\smallint:2\bigint:8\bool:1\float:4\tinyint:1\nchar:4*()+2[offset]\binary:1*()+2[offset] + tdLog.info('test super table max bytes per row 65531') + sql = "create table db.stable_4(ts timestamp, " #1*8 sum=8 + for i in range(500): + sql += "int_%d int, " % (i + 1) #500*4=2000 sum=2008 + for i in range(500,1000): + sql += "smallint_%d smallint, " % (i + 1) #500*2=1000 sum=3008 + for i in range(1000,1500): + sql += "tinyint_%d tinyint, " % (i + 1) #500*1=500 sum=3508 + for i in range(1500,2000): + sql += "double_%d double, " % (i + 1) #500*8=4000 sum=7508 + for i in range(2000,2500): + sql += "float_%d float, " % (i + 1) #500*4=2000 sum=9508 + for i in range(2500,3000): + sql += "bool_%d bool, " % (i + 1) #500*1=500 sum=10008 + for i in range(3000,3500): + sql += "bigint_%d bigint, " % (i + 1) #500*8=4000 sum=14008 + for i in range(3500,3800): + sql += "nchar_%d nchar(32), " % (i + 1) #300*(32*4+2)=39000 sum=53008 + for i in range(3800,4090): + sql += "binary_%d binary(40), " % (i + 1) #290*(40+2)=12180 sum=65188 + sql += "col4091 binary(341))" #341+2=343 sum=65531 + sql += " tags (loc binary(16370),tag_1 int,tag_2 int,tag_3 int) " + tdSql.execute(sql) + sql = '''create table db.table_40 using db.stable_4 + tags('%s' , '1' , '2' , '3' );'''%self.get_random_string(16370) + tdSql.execute(sql) + tdSql.query("select * from db.table_40") + tdSql.checkCols(4092) + tdSql.query("describe db.table_40") + tdSql.checkRows(4096) + + tdLog.info('test super table drop and add column or tag') + sql = "alter stable db.stable_4 drop column col4091; " + tdSql.execute(sql) + sql = "select * from db.stable_4; " + tdSql.query(sql) + tdSql.checkCols(4095) + sql = "alter table db.stable_4 add column col4091 binary(342); " + tdSql.error(sql) + sql = "alter table db.stable_4 add column col4091 binary(341); " + tdSql.execute(sql) + sql = "select * from db.stable_4; " + tdSql.query(sql) + tdSql.checkCols(4096) + + self.ins_query() + + sql = "alter stable db.stable_4 drop tag tag_1; " + tdSql.execute(sql) + sql = "select * from db.stable_4; " + tdSql.query(sql) + tdSql.checkCols(4095) + sql = "alter table db.stable_4 add tag tag_1 int; " + tdSql.execute(sql) + sql = "select * from db.stable_4; " + tdSql.query(sql) + tdSql.checkCols(4096) + sql = "alter table db.stable_4 add tag loc1 nchar(10); " + tdSql.error(sql) + + tdLog.info('test super table max bytes per row 65531') + sql = "create table db.stable_5(ts timestamp, " + for i in range(500): + sql += "int_%d int, " % (i + 1) + for i in range(500,1000): + sql += "smallint_%d smallint, " % (i + 1) + for i in range(1000,1500): + sql += "tinyint_%d tinyint, " % (i + 1) + for i in range(1500,2000): + sql += "double_%d double, " % (i + 1) + for i in range(2000,2500): + sql += "float_%d float, " % (i + 1) + for i in range(2500,3000): + sql += "bool_%d bool, " % (i + 1) + for i in range(3000,3500): + sql += "bigint_%d bigint, " % (i + 1) + for i in range(3500,3800): + sql += "nchar_%d nchar(32), " % (i + 1) + for i in range(3800,4090): + sql += "binary_%d binary(40), " % (i + 1) + sql += "col4091 binary(342))" + sql += " tags (loc nchar(10),tag_1 int,tag_2 int,tag_3 int) " + tdSql.error(sql) + + self.ins_query() + + print("==============step6,stable table , mix data type==============") + sql = "create stable db.stable_6(ts timestamp, " + for i in range(500): + sql += "int_%d int, " % (i + 1) + for i in range(500,1000): + sql += "smallint_%d smallint, " % (i + 1) + for i in range(1000,1500): + sql += "tinyint_%d tinyint, " % (i + 1) + for i in range(1500,2000): + sql += "double_%d double, " % (i + 1) + for i in range(2000,2500): + sql += "float_%d float, " % (i + 1) + for i in range(2500,3000): + sql += "bool_%d bool, " % (i + 1) + for i in range(3000,3500): + sql += "bigint_%d bigint, " % (i + 1) + for i in range(3500,3800): + sql += "nchar_%d nchar(4), " % (i + 1) + for i in range(3800,4090): + sql += "binary_%d binary(10), " % (i + 1) + sql += "col4091 binary(22))" + sql += " tags (loc binary(16370),tag_1 int,tag_2 int,tag_3 int) " + tdLog.info(len(sql)) + tdSql.execute(sql) + sql = '''create table db.table_60 using db.stable_6 + tags('%s' , '1' , '2' , '3' );'''%self.get_random_string(16370) + tdSql.execute(sql) + + self.ins_query() + + for i in range(self.num): + sql = "insert into db.table_60 values(%d, " + for j in range(500): + str = "'%s', " % random.randint(-2147483647,2147483647) + sql += str + for j in range(500,1000): + str = "'%s', " % random.randint(-32767,32767 ) + sql += str + for j in range(1000,1500): + str = "'%s', " % random.randint(-127,127) + sql += str + for j in range(1500,2000): + str = "'%s', " % random.randint(-922337203685477580700,922337203685477580700) + sql += str + for j in range(2000,2500): + str = "'%s', " % random.randint(-92233720368547758070,92233720368547758070) + sql += str + for j in range(2500,3000): + str = "'%s', " % random.choice(['true','false']) + sql += str + for j in range(3000,3500): + str = "'%s', " % random.randint(-9223372036854775807,9223372036854775807) + sql += str + for j in range(3500,3800): + str = "'%s', " % self.get_random_string(4) + sql += str + for j in range(3800,4090): + str = "'%s', " % self.get_random_string(10) + sql += str + sql += "'%s')" % self.get_random_string(22) + tdSql.execute(sql % (self.ts + i)) + time.sleep(1) + tdSql.query("select count(*) from db.table_60") + tdSql.checkData(0, 0, self.num) + tdSql.query("select * from db.table_60") + tdSql.checkRows(self.num) + tdSql.checkCols(4092) + + #insert null value + tdLog.info('test insert null value') + sql = '''create table db.table_61 using db.stable_6 + tags('%s' , '1' , '2' , '3' );'''%self.get_random_string(16370) + tdSql.execute(sql) + + self.ins_query() + + for i in range(self.num): + sql = "insert into db.table_61 values(%d, " + for j in range(2500): + str = "'%s', " % random.choice(['NULL' ,'NULL' ,'NULL' ,1 , 10 ,100 ,-100 ,-10, 88 ,66 ,'NULL' ,'NULL' ,'NULL' ]) + sql += str + for j in range(2500,3000): + str = "'%s', " % random.choice(['true' ,'false']) + sql += str + for j in range(3000,3500): + str = "'%s', " % random.randint(-9223372036854775807,9223372036854775807) + sql += str + for j in range(3500,3800): + str = "'%s', " % self.get_random_string(4) + sql += str + for j in range(3800,4090): + str = "'%s', " % self.get_random_string(10) + sql += str + sql += "'%s')" % self.get_random_string(22) + tdSql.execute(sql % (self.ts + i)) + time.sleep(1) + tdSql.query("select count(*) from db.table_61") + tdSql.checkData(0, 0, self.num) + tdSql.query("select * from db.table_61") + tdSql.checkRows(self.num) + tdSql.checkCols(4092) + + #insert in order + tdLog.info('test insert in order') + for i in range(self.num): + sql = "insert into db.table_61 (ts,int_2,int_22,int_169,smallint_537,smallint_607,tinyint_1030,tinyint_1491,double_1629,double_1808,float_2075,col4091) values(%d, " + for j in range(10): + str = "'%s', " % random.randint(0,100) + sql += str + sql += "'%s')" % self.get_random_string(22) + tdSql.execute(sql % (self.ts + i + 1000)) + time.sleep(1) + tdSql.query("select count(*) from db.table_61") + tdSql.checkData(0, 0, 2*self.num) + tdSql.query("select * from db.table_61") + tdSql.checkRows(2*self.num) + tdSql.checkCols(4092) + + #insert out of order + tdLog.info('test insert out of order') + for i in range(self.num): + sql = "insert into db.table_61 (ts,int_169,float_2075,int_369,tinyint_1491,tinyint_1030,float_2360,smallint_537,double_1808,double_1608,double_1629,col4091) values(%d, " + for j in range(10): + str = "'%s', " % random.randint(0,100) + sql += str + sql += "'%s')" % self.get_random_string(22) + tdSql.execute(sql % (self.ts + i + 2000)) + time.sleep(1) + tdSql.query("select count(*) from db.table_61") + tdSql.checkData(0, 0, 3*self.num) + tdSql.query("select * from db.table_61") + tdSql.checkRows(3*self.num) + tdSql.checkCols(4092) + + self.ins_query() + + #define TSDB_MAX_BYTES_PER_ROW 65531 TSDB_MAX_TAGS_LEN 16384 + #ts:8\int:4\smallint:2\bigint:8\bool:1\float:4\tinyint:1\nchar:4*()+2[offset]\binary:1*()+2[offset] + tdLog.info('test super table max bytes per row 65531') + sql = "create table db.stable_7(ts timestamp, " #1*8 sum=8 + for i in range(500): + sql += "int_%d int, " % (i + 1) #500*4=2000 sum=2008 + for i in range(500,1000): + sql += "smallint_%d smallint, " % (i + 1) #500*2=1000 sum=3008 + for i in range(1000,1500): + sql += "tinyint_%d tinyint, " % (i + 1) #500*1=500 sum=3508 + for i in range(1500,2000): + sql += "double_%d double, " % (i + 1) #500*8=4000 sum=7508 + for i in range(2000,2500): + sql += "float_%d float, " % (i + 1) #500*4=2000 sum=9508 + for i in range(2500,3000): + sql += "bool_%d bool, " % (i + 1) #500*1=500 sum=10008 + for i in range(3000,3500): + sql += "bigint_%d bigint, " % (i + 1) #500*8=4000 sum=14008 + for i in range(3500,3800): + sql += "nchar_%d nchar(32), " % (i + 1) #300*(32*4+2)=39000 sum=53008 + for i in range(3800,4090): + sql += "binary_%d binary(40), " % (i + 1) #290*(40+2)=12180 sum=65188 + sql += "col4091 binary(341))" #341+2=343 sum=65531 + sql += " tags (loc binary(16370),tag_1 int,tag_2 int,tag_3 int) " #4*3+16370+2 + tdSql.execute(sql) + sql = '''create table db.table_70 using db.stable_7 + tags('%s' , '1' , '2' , '3' );'''%self.get_random_string(16370) + tdSql.execute(sql) + tdSql.query("select * from db.table_70") + tdSql.checkCols(4092) + tdSql.query("describe db.table_70") + tdSql.checkRows(4096) + + tdLog.info('test super table drop and add column or tag') + sql = "alter stable db.stable_7 drop column col4091; " + tdSql.execute(sql) + sql = "select * from db.stable_7; " + tdSql.query(sql) + tdSql.checkCols(4095) + sql = "alter table db.stable_7 add column col4091 binary(342); " + tdSql.error(sql) + sql = "alter table db.stable_7 add column col4091 binary(341); " + tdSql.execute(sql) + sql = "select * from db.stable_7; " + tdSql.query(sql) + tdSql.checkCols(4096) + + sql = "alter stable db.stable_7 drop tag loc; " + tdSql.execute(sql) + sql = "select * from db.stable_7; " + tdSql.query(sql) + tdSql.checkCols(4095) + sql = "alter table db.stable_7 add tag loc binary(16371); " + tdSql.error(sql) + sql = "alter table db.stable_7 add tag loc binary(16370); " + tdSql.execute(sql) + sql = "select * from db.stable_7; " + tdSql.query(sql) + tdSql.checkCols(4096) + + sql = "alter stable db.stable_7 drop tag tag_1; " + tdSql.execute(sql) + sql = "select * from db.stable_7; " + tdSql.query(sql) + tdSql.checkCols(4095) + sql = "alter table db.stable_7 add tag tag_1 int; " + tdSql.execute(sql) + sql = "select * from db.stable_7; " + tdSql.query(sql) + tdSql.checkCols(4096) + sql = "alter table db.stable_7 add tag loc1 nchar(10); " + tdSql.error(sql) + + self.ins_query() + + tdLog.info('test super table max bytes per row 65531') + sql = "create table db.stable_8(ts timestamp, " + for i in range(500): + sql += "int_%d int, " % (i + 1) + for i in range(500,1000): + sql += "smallint_%d smallint, " % (i + 1) + for i in range(1000,1500): + sql += "tinyint_%d tinyint, " % (i + 1) + for i in range(1500,2000): + sql += "double_%d double, " % (i + 1) + for i in range(2000,2500): + sql += "float_%d float, " % (i + 1) + for i in range(2500,3000): + sql += "bool_%d bool, " % (i + 1) + for i in range(3000,3500): + sql += "bigint_%d bigint, " % (i + 1) + for i in range(3500,3800): + sql += "nchar_%d nchar(32), " % (i + 1) + for i in range(3800,4090): + sql += "binary_%d binary(40), " % (i + 1) + sql += "col4091 binary(342))" + sql += " tags (loc nchar(10),tag_1 int,tag_2 int,tag_3 int) " + tdSql.error(sql) + + tdLog.info('test super table max bytes per row tag 16384') + sql = "create table db.stable_8(ts timestamp, " + for i in range(500): + sql += "int_%d int, " % (i + 1) + for i in range(500,1000): + sql += "smallint_%d smallint, " % (i + 1) + for i in range(1000,1500): + sql += "tinyint_%d tinyint, " % (i + 1) + for i in range(1500,2000): + sql += "double_%d double, " % (i + 1) + for i in range(2000,2500): + sql += "float_%d float, " % (i + 1) + for i in range(2500,3000): + sql += "bool_%d bool, " % (i + 1) + for i in range(3000,3500): + sql += "bigint_%d bigint, " % (i + 1) + for i in range(3500,3800): + sql += "nchar_%d nchar(32), " % (i + 1) + for i in range(3800,4090): + sql += "binary_%d binary(40), " % (i + 1) + sql += "col4091 binary(341))" + sql += " tags (loc binary(16371),tag_1 int,tag_2 int,tag_3 int) " + tdSql.error(sql) + + print("==============step7, super table error col ==============") + tdLog.info('test exceeds row num') + # column + tag > 4096 + sql = "create stable db.stable_2(ts timestamp, " + for i in range(4091): + sql += "col%d int, " % (i + 1) + sql += "col4092 binary(22))" + sql += " tags (loc nchar(10),tag_1 int,tag_2 int,tag_3 int) " + tdLog.info(len(sql)) + tdSql.error(sql) + + self.ins_query() + + # column + tag > 4096 + sql = "create stable db.stable_2(ts timestamp, " + for i in range(4090): + sql += "col%d int, " % (i + 1) + sql += "col4091 binary(22))" + sql += " tags (loc nchar(10),tag_1 int,tag_2 int,tag_3 int,tag_4 int) " + tdLog.info(len(sql)) + tdSql.error(sql) + + # alter column + tag > 4096 + sql = "alter table db.stable_1 add column max int; " + tdSql.error(sql) + + sql = "alter table db.stable_1 add tag max int; " + tdSql.error(sql) + + sql = "alter table db.stable_4 modify column col4091 binary(102); " + tdSql.error(sql) + sql = "alter table db.stable_4 modify tag loc nchar(20); " + tdSql.query("select * from db.table_40") + tdSql.checkCols(4092) + tdSql.query("describe db.table_40") + tdSql.checkRows(4096) + + self.ins_query() + + endTime_all = time.time() + print("total time %ds" % (endTime_all - startTime_all)) + + + def stop(self): + tdSql.close() + tdLog.success("%s successfully executed" % __file__) + + +tdCases.addWindows(__file__, TDTestCase()) +tdCases.addLinux(__file__, TDTestCase()) \ No newline at end of file From 5cdc119a717287da542e0da153a3b20e03c9fc7d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Chappyguoxy=E2=80=9D?= <“happy_guoxy@163.com”> Date: Tue, 16 May 2023 14:17:56 +0800 Subject: [PATCH 17/59] test: rowlength64k --- tests/system-test/1-insert/rowlength64k.json | 128 +++++++++++++++++++ 1 file changed, 128 insertions(+) create mode 100755 tests/system-test/1-insert/rowlength64k.json diff --git a/tests/system-test/1-insert/rowlength64k.json b/tests/system-test/1-insert/rowlength64k.json new file mode 100755 index 0000000000..a0a3e7522c --- /dev/null +++ b/tests/system-test/1-insert/rowlength64k.json @@ -0,0 +1,128 @@ +{ + "filetype": "insert", + "cfgdir": "/etc/taos", + "host": "127.0.0.1", + "port": 6030, + "user": "root", + "password": "taosdata", + "thread_count": 10, + "thread_count_create_tbl": 10, + "result_file": "./insert_res.txt", + "confirm_parameter_prompt": "no", + "insert_interval": 0, + "interlace_rows": 10, + "num_of_records_per_req": 1, + "max_sql_len": 102400000, + "databases": [{ + "dbinfo": { + "name": "json_test", + "drop": "yes", + "replica": 1, + "precision": "ms", + "keep": 36500, + "minRows": 100, + "maxRows": 4096 + }, + "super_tables": [{ + "name": "stb_old", + "child_table_exists":"no", + "childtable_count": 1, + "childtable_prefix": "stb_old_", + "auto_create_table": "no", + "batch_create_tbl_num": 5, + "data_source": "rand", + "insert_mode": "taosc", + "insert_rows": 10, + "childtable_limit": 0, + "childtable_offset":0, + "multi_thread_write_one_tbl": "no", + "interlace_rows": 0, + "insert_interval":0, + "max_sql_len": 1024000, + "disorder_ratio": 0, + "disorder_range": 1000, + "timestamp_step": 1, + "start_timestamp": "2020-10-01 00:00:00.000", + "sample_format": "sample", + "sample_file": "./tools/taosdemoAllTest/TD-5213/insertSigcolumnsNum4096.csv", + "tags_file": "", + "columns": [{"type": "INT","count":1000}, {"type": "BINARY", "len": 16, "count":20}], + "tags": [{"type": "TINYINT", "count":2}, {"type": "BINARY", "len": 16, "count":1}] + },{ + "name": "stb_new", + "child_table_exists":"no", + "childtable_count": 1, + "childtable_prefix": "stb_new_", + "auto_create_table": "no", + "batch_create_tbl_num": 5, + "data_source": "rand", + "insert_mode": "taosc", + "insert_rows": 10, + "childtable_limit": 0, + "childtable_offset":0, + "multi_thread_write_one_tbl": "no", + "interlace_rows": 0, + "insert_interval":0, + "max_sql_len": 1024000, + "disorder_ratio": 0, + "disorder_range": 1000, + "timestamp_step": 1, + "start_timestamp": "2020-10-01 00:00:00.000", + "sample_format": "csv", + "sample_file": "./sample.csv", + "tags_file": "", + "columns": [{"type": "INT","count":4000}, {"type": "BINARY", "len": 16, "count":90}], + "tags": [{"type": "TINYINT", "count":2}, {"type": "BINARY", "len": 16, "count":3}] + },{ + "name": "stb_mix", + "child_table_exists":"no", + "childtable_count": 1, + "childtable_prefix": "stb_mix_", + "auto_create_table": "no", + "batch_create_tbl_num": 5, + "data_source": "rand", + "insert_mode": "taosc", + "insert_rows": 10, + "childtable_limit": 0, + "childtable_offset":0, + "multi_thread_write_one_tbl": "no", + "interlace_rows": 0, + "insert_interval":0, + "max_sql_len": 1024000, + "disorder_ratio": 0, + "disorder_range": 1000, + "timestamp_step": 1, + "start_timestamp": "2020-10-01 00:00:00.000", + "sample_format": "csv", + "sample_file": "./sample.csv", + "tags_file": "", + "columns": [{"type": "INT","count":500},{"type": "SMALLINT","count":500},{"type": "TINYINT","count":500},{"type": "DOUBLE","count":500},{"type": "FLOAT","count":500},{"type": "BOOL","count":500},{"type": "BIGINT","count":500},{"type": "NCHAR","len": 20,"count":300},{"type": "BINARY","len": 34,"count":290},{"type": "BINARY","len": 101,"count":1}], + "tags": [{"type": "INT", "count":3}, {"type": "NCHAR", "len": 10, "count":1}] + },{ + "name": "stb_excel", + "child_table_exists":"no", + "childtable_count": 1, + "childtable_prefix": "stb_excel_", + "auto_create_table": "no", + "batch_create_tbl_num": 5, + "data_source": "rand", + "insert_mode": "taosc", + "insert_rows": 10, + "childtable_limit": 0, + "childtable_offset":0, + "multi_thread_write_one_tbl": "no", + "interlace_rows": 0, + "insert_interval":0, + "max_sql_len": 1024000, + "disorder_ratio": 0, + "disorder_range": 1000, + "timestamp_step": 1, + "start_timestamp": "2020-10-01 00:00:00.000", + "sample_format": "csv", + "sample_file": "./sample.csv", + "tags_file": "", + "columns": [{"type": "INT","count":500},{"type": "SMALLINT","count":500},{"type": "SMALLINT","count":500},{"type": "DOUBLE","count":500},{"type": "FLOAT","count":500},{"type": "BOOL","count":500},{"type": "BIGINT","count":500},{"type": "NCHAR","len": 30,"count":300},{"type": "BINARY","len": 47,"count":290},{"type": "BINARY","len": 211,"count":1}], + "tags": [{"type": "INT", "count":3}, {"type": "NCHAR", "len": 10, "count":1}] + }] + }] +} \ No newline at end of file From 7278d931131db0b58d64500235b11764406eb6d9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Chappyguoxy=E2=80=9D?= <“happy_guoxy@163.com”> Date: Tue, 16 May 2023 14:20:19 +0800 Subject: [PATCH 18/59] test: rowlength64k --- tests/parallel_test/cases.task | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tests/parallel_test/cases.task b/tests/parallel_test/cases.task index 3e656c0302..66eeb9e35f 100644 --- a/tests/parallel_test/cases.task +++ b/tests/parallel_test/cases.task @@ -582,6 +582,12 @@ ,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/db_tb_name_check.py ,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/InsertFuturets.py ,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/insert_wide_column.py +,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k_benchmark.py +,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k.py +,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k.py -R +,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k.py -Q 2 +,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k.py -Q 3 +,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k.py -Q 4 ,,y,system-test,./pytest.sh python3 ./test.py -f 0-others/show.py ,,y,system-test,./pytest.sh python3 ./test.py -f 0-others/information_schema.py ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/abs.py From 523b86f792a4d90c44e4e797a88536b93338334d Mon Sep 17 00:00:00 2001 From: dmchen Date: Tue, 16 May 2023 16:13:31 +0800 Subject: [PATCH 19/59] fix test case --- tests/script/tsim/dnode/drop_dnode_force.sim | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/script/tsim/dnode/drop_dnode_force.sim b/tests/script/tsim/dnode/drop_dnode_force.sim index bce51a9888..dbe34ab529 100644 --- a/tests/script/tsim/dnode/drop_dnode_force.sim +++ b/tests/script/tsim/dnode/drop_dnode_force.sim @@ -209,7 +209,8 @@ endi print =============== step5a: drop dnode 3 sql_error drop dnode 3 -sql drop dnode 3 force +sql_error drop dnode 3 force +sql drop dnode 3 unsafe print select * from information_schema.ins_dnodes; sql select * from information_schema.ins_dnodes; From 756fefaa292c22bd294c50017acc7e38807cc7cd Mon Sep 17 00:00:00 2001 From: Minglei Jin Date: Tue, 16 May 2023 18:56:05 +0800 Subject: [PATCH 20/59] cache/batchread: load columns in one trip --- source/dnode/vnode/src/inc/tsdb.h | 24 +- source/dnode/vnode/src/tsdb/tsdbCache.c | 676 ++++++++++++++++---- source/dnode/vnode/src/tsdb/tsdbCacheRead.c | 24 +- 3 files changed, 567 insertions(+), 157 deletions(-) diff --git a/source/dnode/vnode/src/inc/tsdb.h b/source/dnode/vnode/src/inc/tsdb.h index b2bc9abf33..486becdf96 100644 --- a/source/dnode/vnode/src/inc/tsdb.h +++ b/source/dnode/vnode/src/inc/tsdb.h @@ -346,13 +346,17 @@ struct STsdbFS { }; typedef struct { - rocksdb_t *db; - rocksdb_options_t *options; - rocksdb_flushoptions_t *flushoptions; - rocksdb_writeoptions_t *writeoptions; - rocksdb_readoptions_t *readoptions; - rocksdb_writebatch_t *writebatch; - TdThreadMutex rMutex; + rocksdb_t *db; + rocksdb_comparator_t *my_comparator; + rocksdb_cache_t *blockcache; + rocksdb_block_based_table_options_t *tableoptions; + rocksdb_options_t *options; + rocksdb_flushoptions_t *flushoptions; + rocksdb_writeoptions_t *writeoptions; + rocksdb_readoptions_t *readoptions; + rocksdb_writebatch_t *writebatch; + TdThreadMutex rMutex; + STSchema *pTSchema; } SRocksCache; struct STsdb { @@ -782,7 +786,7 @@ typedef struct SLDataIter { #define tMergeTreeGetRow(_t) (&((_t)->pIter->rInfo.row)) int32_t tMergeTreeOpen(SMergeTree *pMTree, int8_t backward, SDataFReader *pFReader, uint64_t suid, uint64_t uid, STimeWindow *pTimeWindow, SVersionRange *pVerRange, SSttBlockLoadInfo *pBlockLoadInfo, - bool destroyLoadInfo, const char *idStr, bool strictTimeRange, SLDataIter* pLDataIter); + bool destroyLoadInfo, const char *idStr, bool strictTimeRange, SLDataIter *pLDataIter); void tMergeTreeAddIter(SMergeTree *pMTree, SLDataIter *pIter); bool tMergeTreeNext(SMergeTree *pMTree); bool tMergeTreeIgnoreEarlierTs(SMergeTree *pMTree); @@ -822,13 +826,15 @@ typedef struct SCacheRowsReader { typedef struct { TSKEY ts; + int8_t dirty; SColVal colVal; } SLastCol; int32_t tsdbOpenCache(STsdb *pTsdb); void tsdbCloseCache(STsdb *pTsdb); int32_t tsdbCacheUpdate(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSDBROW *row); -int32_t tsdbCacheGet(STsdb *pTsdb, tb_uid_t uid, SArray *pLastArray, SCacheRowsReader *pr, int32_t ltype); +int32_t tsdbCacheGetBatch(STsdb *pTsdb, tb_uid_t uid, SArray *pLastArray, SCacheRowsReader *pr, int8_t ltype); +int32_t tsdbCacheGet(STsdb *pTsdb, tb_uid_t uid, SArray *pLastArray, SCacheRowsReader *pr, int8_t ltype); int32_t tsdbCacheDel(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSKEY sKey, TSKEY eKey); int32_t tsdbCacheInsertLast(SLRUCache *pCache, tb_uid_t uid, TSDBROW *row, STsdb *pTsdb); diff --git a/source/dnode/vnode/src/tsdb/tsdbCache.c b/source/dnode/vnode/src/tsdb/tsdbCache.c index c0a8de5743..a637ba4968 100644 --- a/source/dnode/vnode/src/tsdb/tsdbCache.c +++ b/source/dnode/vnode/src/tsdb/tsdbCache.c @@ -46,7 +46,13 @@ static void tsdbCloseBICache(STsdb *pTsdb) { } } -#define ROCKS_KEY_LEN 64 +#define ROCKS_KEY_LEN (sizeof(tb_uid_t) + sizeof(int16_t) + sizeof(int8_t)) + +typedef struct { + tb_uid_t uid; + int16_t cid; + int8_t ltype; +} SLastKey; static void tsdbGetRocksPath(STsdb *pTsdb, char *path) { SVnode *pVnode = pTsdb->pVnode; @@ -62,9 +68,56 @@ static void tsdbGetRocksPath(STsdb *pTsdb, char *path) { } } +static const char *myCmpName(void *state) { + (void)state; + return "myCmp"; +} + +static void myCmpDestroy(void *state) { (void)state; } + +static int myCmp(void *state, const char *a, size_t alen, const char *b, size_t blen) { + (void)state; + (void)alen; + (void)blen; + SLastKey *lhs = (SLastKey *)a; + SLastKey *rhs = (SLastKey *)b; + + if (lhs->uid < rhs->uid) { + return -1; + } else if (lhs->uid > rhs->uid) { + return 1; + } + + if (lhs->cid < rhs->cid) { + return -1; + } else if (lhs->cid > rhs->cid) { + return 1; + } + + if (lhs->ltype < rhs->ltype) { + return -1; + } else if (lhs->ltype > rhs->ltype) { + return 1; + } + + return 0; +} + static int32_t tsdbOpenRocksCache(STsdb *pTsdb) { int32_t code = 0; + rocksdb_comparator_t *cmp = rocksdb_comparator_create(NULL, myCmpDestroy, myCmp, myCmpName); + if (NULL == cmp) { + code = TSDB_CODE_OUT_OF_MEMORY; + return code; + } + + rocksdb_cache_t *cache = rocksdb_cache_create_lru(5 * 1024 * 1024); + pTsdb->rCache.blockcache = cache; + + rocksdb_block_based_table_options_t *tableoptions = rocksdb_block_based_options_create(); + pTsdb->rCache.tableoptions = tableoptions; + rocksdb_options_t *options = rocksdb_options_create(); if (NULL == options) { code = TSDB_CODE_OUT_OF_MEMORY; @@ -72,6 +125,9 @@ static int32_t tsdbOpenRocksCache(STsdb *pTsdb) { } rocksdb_options_set_create_if_missing(options, 1); + rocksdb_options_set_comparator(options, cmp); + rocksdb_block_based_options_set_block_cache(tableoptions, cache); + rocksdb_options_set_block_based_table_factory(options, tableoptions); // rocksdb_options_set_inplace_update_support(options, 1); // rocksdb_options_set_allow_concurrent_memtable_write(options, 0); @@ -80,12 +136,12 @@ static int32_t tsdbOpenRocksCache(STsdb *pTsdb) { code = TSDB_CODE_OUT_OF_MEMORY; goto _err2; } - // rocksdb_writeoptions_disable_WAL(writeoptions, 1); + rocksdb_writeoptions_disable_WAL(writeoptions, 1); rocksdb_readoptions_t *readoptions = rocksdb_readoptions_create(); if (NULL == readoptions) { code = TSDB_CODE_OUT_OF_MEMORY; - goto _err2; + goto _err3; } char *err = NULL; @@ -94,19 +150,23 @@ static int32_t tsdbOpenRocksCache(STsdb *pTsdb) { rocksdb_t *db = rocksdb_open(options, cachePath, &err); if (NULL == db) { - code = -1; - goto _err3; + tsdbError("vgId:%d, %s failed at line %d since %s", TD_VID(pTsdb->pVnode), __func__, __LINE__, err); + rocksdb_free(err); + + code = TSDB_CODE_OUT_OF_MEMORY; + goto _err4; } rocksdb_flushoptions_t *flushoptions = rocksdb_flushoptions_create(); if (NULL == flushoptions) { code = TSDB_CODE_OUT_OF_MEMORY; - goto _err4; + goto _err5; } rocksdb_writebatch_t *writebatch = rocksdb_writebatch_create(); pTsdb->rCache.writebatch = writebatch; + pTsdb->rCache.my_comparator = cmp; pTsdb->rCache.options = options; pTsdb->rCache.writeoptions = writeoptions; pTsdb->rCache.readoptions = readoptions; @@ -115,15 +175,22 @@ static int32_t tsdbOpenRocksCache(STsdb *pTsdb) { taosThreadMutexInit(&pTsdb->rCache.rMutex, NULL); + pTsdb->rCache.pTSchema = NULL; + return code; +_err5: + rocksdb_close(pTsdb->rCache.db); _err4: rocksdb_readoptions_destroy(readoptions); _err3: rocksdb_writeoptions_destroy(writeoptions); _err2: rocksdb_options_destroy(options); + rocksdb_block_based_options_destroy(tableoptions); + rocksdb_cache_destroy(cache); _err: + rocksdb_comparator_destroy(cmp); return code; } @@ -134,7 +201,11 @@ static void tsdbCloseRocksCache(STsdb *pTsdb) { rocksdb_readoptions_destroy(pTsdb->rCache.readoptions); rocksdb_writeoptions_destroy(pTsdb->rCache.writeoptions); rocksdb_options_destroy(pTsdb->rCache.options); + rocksdb_block_based_options_destroy(pTsdb->rCache.tableoptions); + rocksdb_cache_destroy(pTsdb->rCache.blockcache); + rocksdb_comparator_destroy(pTsdb->rCache.my_comparator); taosThreadMutexDestroy(&pTsdb->rCache.rMutex); + taosMemoryFree(pTsdb->rCache.pTSchema); } int32_t tsdbCacheCommit(STsdb *pTsdb) { @@ -191,15 +262,15 @@ void tsdbCacheSerialize(SLastCol *pLastCol, char **value, size_t *size) { *size = length; } -static SLastCol *tsdbCacheLookup(STsdb *pTsdb, tb_uid_t uid, int16_t cid, char const *lstring) { +static SLastCol *tsdbCacheLookup(STsdb *pTsdb, tb_uid_t uid, int16_t cid, int8_t ltype) { SLastCol *pLastCol = NULL; - char *err = NULL; - size_t vlen = 0; - char key[ROCKS_KEY_LEN]; - size_t klen = snprintf(key, ROCKS_KEY_LEN, "%" PRIi64 ":%" PRIi16 ":%s", uid, cid, lstring); - char *value = NULL; - value = rocksdb_get(pTsdb->rCache.db, pTsdb->rCache.readoptions, key, klen, &vlen, &err); + char *err = NULL; + size_t vlen = 0; + SLastKey *key = &(SLastKey){.ltype = ltype, .uid = uid, .cid = cid}; + size_t klen = ROCKS_KEY_LEN; + char *value = NULL; + value = rocksdb_get(pTsdb->rCache.db, pTsdb->rCache.readoptions, (char *)key, klen, &vlen, &err); if (NULL != err) { tsdbError("vgId:%d, %s failed at line %d since %s", TD_VID(pTsdb->pVnode), __func__, __LINE__, err); rocksdb_free(err); @@ -210,18 +281,40 @@ static SLastCol *tsdbCacheLookup(STsdb *pTsdb, tb_uid_t uid, int16_t cid, char c return pLastCol; } +static void rocksMayWrite(STsdb *pTsdb) { + rocksdb_writebatch_t *wb = pTsdb->rCache.writebatch; + + int count = rocksdb_writebatch_count(wb); + if (count >= 1024) { + char *err = NULL; + rocksdb_write(pTsdb->rCache.db, pTsdb->rCache.writeoptions, wb, &err); + if (NULL != err) { + tsdbError("vgId:%d, %s failed at line %d since %s", TD_VID(pTsdb->pVnode), __func__, __LINE__, err); + rocksdb_free(err); + } + + rocksdb_writebatch_clear(wb); + } +} + int32_t tsdbCacheUpdate(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSDBROW *pRow) { int32_t code = 0; // 1, fetch schema - STSchema *pTSchema = NULL; + STSchema *pTSchema = pTsdb->rCache.pTSchema; int32_t sver = TSDBROW_SVERSION(pRow); - code = metaGetTbTSchemaEx(pTsdb->pVnode->pMeta, suid, uid, sver, &pTSchema); - if (code != TSDB_CODE_SUCCESS) { - terrno = code; - return -1; - } + if (!pTSchema || sver != pTSchema->version) { + if (pTSchema) { + taosMemoryFree(pTSchema); + } + code = metaGetTbTSchemaEx(pTsdb->pVnode->pMeta, suid, uid, sver, &pTSchema); + if (code != TSDB_CODE_SUCCESS) { + terrno = code; + return -1; + } + pTsdb->rCache.pTSchema = pTSchema; + } // 2, iterate col values into array SArray *aColVal = taosArrayInit(32, sizeof(SColVal)); @@ -229,22 +322,6 @@ int32_t tsdbCacheUpdate(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSDBROW *pRow tsdbRowIterOpen(&iter, pRow, pTSchema); for (SColVal *pColVal = tsdbRowIterNext(&iter); pColVal; pColVal = tsdbRowIterNext(&iter)) { - /* - if (IS_VAR_DATA_TYPE(pColVal->type)) { - uint8_t *pVal = pColVal->value.pData; - - pColVal->value.pData = NULL; - code = tRealloc(&pColVal->value.pData, pColVal->value.nData); - if (code) { - code = TSDB_CODE_OUT_OF_MEMORY; - goto _exit; - } - - if (pColVal->value.nData) { - memcpy(pColVal->value.pData, pVal, pColVal->value.nData); - } - } - */ taosArrayPush(aColVal, pColVal); } @@ -254,23 +331,18 @@ int32_t tsdbCacheUpdate(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSDBROW *pRow int num_keys = TARRAY_SIZE(aColVal); char **keys_list = taosMemoryCalloc(num_keys * 2, sizeof(char *)); size_t *keys_list_sizes = taosMemoryCalloc(num_keys * 2, sizeof(size_t)); + char *key_list = taosMemoryMalloc(num_keys * ROCKS_KEY_LEN * 2); for (int i = 0; i < num_keys; ++i) { SColVal *pColVal = (SColVal *)taosArrayGet(aColVal, i); int16_t cid = pColVal->cid; - char *keys = taosMemoryCalloc(2, ROCKS_KEY_LEN); - int last_key_len = snprintf(keys, ROCKS_KEY_LEN, "%" PRIi64 ":%" PRIi16 ":last", uid, cid); - if (last_key_len >= ROCKS_KEY_LEN) { - tsdbError("vgId:%d, %s failed at line %d since %s", TD_VID(pTsdb->pVnode), __func__, __LINE__, tstrerror(code)); - } - int lr_key_len = snprintf(keys + ROCKS_KEY_LEN, ROCKS_KEY_LEN, "%" PRIi64 ":%" PRIi16 ":last_row", uid, cid); - if (lr_key_len >= ROCKS_KEY_LEN) { - tsdbError("vgId:%d, %s failed at line %d since %s", TD_VID(pTsdb->pVnode), __func__, __LINE__, tstrerror(code)); - } - keys_list[i] = keys; - keys_list[num_keys + i] = keys + ROCKS_KEY_LEN; - keys_list_sizes[i] = last_key_len; - keys_list_sizes[num_keys + i] = lr_key_len; + memcpy(key_list + i * ROCKS_KEY_LEN, &(SLastKey){.ltype = 1, .uid = uid, .cid = cid}, ROCKS_KEY_LEN); + memcpy(key_list + i * ROCKS_KEY_LEN + num_keys * ROCKS_KEY_LEN, &(SLastKey){.ltype = 0, .uid = uid, .cid = cid}, + ROCKS_KEY_LEN); + keys_list[i] = key_list + i * ROCKS_KEY_LEN; + keys_list[num_keys + i] = key_list + i * ROCKS_KEY_LEN + num_keys * ROCKS_KEY_LEN; + keys_list_sizes[i] = ROCKS_KEY_LEN; + keys_list_sizes[num_keys + i] = ROCKS_KEY_LEN; } char **values_list = taosMemoryCalloc(num_keys * 2, sizeof(char *)); size_t *values_list_sizes = taosMemoryCalloc(num_keys * 2, sizeof(size_t)); @@ -278,12 +350,10 @@ int32_t tsdbCacheUpdate(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSDBROW *pRow taosThreadMutexLock(&pTsdb->rCache.rMutex); rocksdb_multi_get(pTsdb->rCache.db, pTsdb->rCache.readoptions, num_keys * 2, (const char *const *)keys_list, keys_list_sizes, values_list, values_list_sizes, errs); - for (int i = 0; i < num_keys; ++i) { - taosMemoryFree(keys_list[i]); - } for (int i = 0; i < num_keys * 2; ++i) { rocksdb_free(errs[i]); } + taosMemoryFree(key_list); taosMemoryFree(keys_list); taosMemoryFree(keys_list_sizes); taosMemoryFree(errs); @@ -292,19 +362,6 @@ int32_t tsdbCacheUpdate(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSDBROW *pRow rocksdb_writebatch_t *wb = pTsdb->rCache.writebatch; for (int i = 0; i < num_keys; ++i) { SColVal *pColVal = (SColVal *)taosArrayGet(aColVal, i); - if (COL_VAL_IS_VALUE(pColVal)) { - SLastCol *pLastCol = tsdbCacheDeserialize(values_list[i]); - - if (NULL == pLastCol || pLastCol->ts <= keyTs) { - char *value = NULL; - size_t vlen = 0; - tsdbCacheSerialize(&(SLastCol){.ts = keyTs, .colVal = *pColVal}, &value, &vlen); - char key[ROCKS_KEY_LEN]; - size_t klen = snprintf(key, ROCKS_KEY_LEN, "%" PRIi64 ":%" PRIi16 ":last", uid, pColVal->cid); - rocksdb_writebatch_put(wb, key, klen, value, vlen); - taosMemoryFree(value); - } - } if (!COL_VAL_IS_NONE(pColVal)) { SLastCol *pLastCol = tsdbCacheDeserialize(values_list[i + num_keys]); @@ -313,11 +370,26 @@ int32_t tsdbCacheUpdate(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSDBROW *pRow char *value = NULL; size_t vlen = 0; tsdbCacheSerialize(&(SLastCol){.ts = keyTs, .colVal = *pColVal}, &value, &vlen); - char key[ROCKS_KEY_LEN]; - size_t klen = snprintf(key, ROCKS_KEY_LEN, "%" PRIi64 ":%" PRIi16 ":last_row", uid, pColVal->cid); - rocksdb_writebatch_put(wb, key, klen, value, vlen); + SLastKey key = (SLastKey){.ltype = 0, .uid = uid, .cid = pColVal->cid}; + size_t klen = ROCKS_KEY_LEN; + rocksdb_writebatch_put(wb, (char *)&key, klen, value, vlen); taosMemoryFree(value); } + + if (COL_VAL_IS_VALUE(pColVal)) { + SLastCol *pLastCol = tsdbCacheDeserialize(values_list[i]); + + if (NULL == pLastCol || pLastCol->ts <= keyTs) { + char *value = NULL; + size_t vlen = 0; + tsdbCacheSerialize(&(SLastCol){.ts = keyTs, .colVal = *pColVal}, &value, &vlen); + SLastKey key = (SLastKey){.ltype = 1, .uid = uid, .cid = pColVal->cid}; + size_t klen = ROCKS_KEY_LEN; + + rocksdb_writebatch_put(wb, (char *)&key, klen, value, vlen); + taosMemoryFree(value); + } + } } rocksdb_free(values_list[i]); @@ -326,18 +398,12 @@ int32_t tsdbCacheUpdate(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSDBROW *pRow taosMemoryFree(values_list); taosMemoryFree(values_list_sizes); - char *err = NULL; - rocksdb_write(pTsdb->rCache.db, pTsdb->rCache.writeoptions, wb, &err); - if (NULL != err) { - tsdbError("vgId:%d, %s failed at line %d since %s", TD_VID(pTsdb->pVnode), __func__, __LINE__, err); - rocksdb_free(err); - } + rocksMayWrite(pTsdb); taosThreadMutexUnlock(&pTsdb->rCache.rMutex); - rocksdb_writebatch_clear(wb); _exit: taosArrayDestroy(aColVal); - taosMemoryFree(pTSchema); + // taosMemoryFree(pTSchema); return code; } @@ -356,38 +422,36 @@ static int32_t mergeLastCid(tb_uid_t uid, STsdb *pTsdb, SArray **ppLastArray, SC static int32_t mergeLastRowCid(tb_uid_t uid, STsdb *pTsdb, SArray **ppLastArray, SCacheRowsReader *pr, int16_t *aCols, int nCols, int16_t *slotIds); - -int32_t tsdbCacheGet(STsdb *pTsdb, tb_uid_t uid, SArray *pLastArray, SCacheRowsReader *pr, int32_t ltype) { - static char const *alstring[2] = {"last_row", "last"}; - char const *lstring = alstring[ltype]; +#if 1 +int32_t tsdbCacheGetSlow(STsdb *pTsdb, tb_uid_t uid, SArray *pLastArray, SCacheRowsReader *pr, int8_t ltype) { rocksdb_writebatch_t *wb = NULL; int32_t code = 0; SArray *pCidList = pr->pCidList; int num_keys = TARRAY_SIZE(pCidList); - char **keys_list = taosMemoryCalloc(num_keys, sizeof(char *)); - size_t *keys_list_sizes = taosMemoryCalloc(num_keys, sizeof(size_t)); + + char **keys_list = taosMemoryMalloc(num_keys * sizeof(char *)); + size_t *keys_list_sizes = taosMemoryMalloc(num_keys * sizeof(size_t)); + char *key_list = taosMemoryMalloc(num_keys * ROCKS_KEY_LEN); for (int i = 0; i < num_keys; ++i) { int16_t cid = *(int16_t *)taosArrayGet(pCidList, i); - char *keys = taosMemoryCalloc(2, ROCKS_KEY_LEN); - int last_key_len = snprintf(keys, ROCKS_KEY_LEN, "%" PRIi64 ":%" PRIi16 ":%s", uid, cid, lstring); - if (last_key_len >= ROCKS_KEY_LEN) { - tsdbError("vgId:%d, %s failed at line %d since %s", TD_VID(pTsdb->pVnode), __func__, __LINE__, tstrerror(code)); - } - - keys_list[i] = keys; - keys_list_sizes[i] = last_key_len; + memcpy(key_list + i * ROCKS_KEY_LEN, &(SLastKey){.ltype = ltype, .uid = uid, .cid = cid}, ROCKS_KEY_LEN); + keys_list[i] = key_list + i * ROCKS_KEY_LEN; + keys_list_sizes[i] = ROCKS_KEY_LEN; } + char **values_list = taosMemoryCalloc(num_keys, sizeof(char *)); size_t *values_list_sizes = taosMemoryCalloc(num_keys, sizeof(size_t)); - char **errs = taosMemoryCalloc(num_keys, sizeof(char *)); + char **errs = taosMemoryMalloc(num_keys * sizeof(char *)); rocksdb_multi_get(pTsdb->rCache.db, pTsdb->rCache.readoptions, num_keys, (const char *const *)keys_list, keys_list_sizes, values_list, values_list_sizes, errs); for (int i = 0; i < num_keys; ++i) { - taosMemoryFree(keys_list[i]); - rocksdb_free(errs[i]); + if (errs[i]) { + rocksdb_free(errs[i]); + } } + taosMemoryFree(key_list); taosMemoryFree(keys_list); taosMemoryFree(keys_list_sizes); taosMemoryFree(errs); @@ -403,7 +467,7 @@ int32_t tsdbCacheGet(STsdb *pTsdb, tb_uid_t uid, SArray *pLastArray, SCacheRowsR } else { taosThreadMutexLock(&pTsdb->rCache.rMutex); - pLastCol = tsdbCacheLookup(pTsdb, uid, cid, lstring); + pLastCol = tsdbCacheLookup(pTsdb, uid, cid, ltype); if (!pLastCol) { // recalc: load from tsdb int16_t aCols[1] = {cid}; @@ -432,9 +496,10 @@ int32_t tsdbCacheGet(STsdb *pTsdb, tb_uid_t uid, SArray *pLastArray, SCacheRowsR char *value = NULL; size_t vlen = 0; tsdbCacheSerialize(pLastCol, &value, &vlen); - char key[ROCKS_KEY_LEN]; - size_t klen = snprintf(key, ROCKS_KEY_LEN, "%" PRIi64 ":%" PRIi16 ":%s", uid, pLastCol->colVal.cid, lstring); - rocksdb_writebatch_put(wb, key, klen, value, vlen); + + SLastKey *key = &(SLastKey){.ltype = ltype, .uid = uid, .cid = pLastCol->colVal.cid}; + size_t klen = ROCKS_KEY_LEN; + rocksdb_writebatch_put(wb, (char *)key, klen, value, vlen); taosMemoryFree(value); } else { @@ -442,21 +507,13 @@ int32_t tsdbCacheGet(STsdb *pTsdb, tb_uid_t uid, SArray *pLastArray, SCacheRowsR } if (wb) { - char *err = NULL; - rocksdb_write(pTsdb->rCache.db, pTsdb->rCache.writeoptions, wb, &err); - if (NULL != err) { - tsdbError("vgId:%d, %s failed at line %d since %s", TD_VID(pTsdb->pVnode), __func__, __LINE__, err); - rocksdb_free(err); - } - - rocksdb_writebatch_clear(wb); + rocksMayWrite(pTsdb); } taosThreadMutexUnlock(&pTsdb->rCache.rMutex); } taosArrayPush(pLastArray, pLastCol); - taosArrayDestroy(pTmpColArray); if (freeCol) { taosMemoryFree(pLastCol); @@ -467,6 +524,356 @@ int32_t tsdbCacheGet(STsdb *pTsdb, tb_uid_t uid, SArray *pLastArray, SCacheRowsR return code; } +#endif + +static SLastCol *tsdbCacheLoadCol(STsdb *pTsdb, SCacheRowsReader *pr, int16_t slotid, tb_uid_t uid, int16_t cid, + int8_t ltype) { + SLastCol *pLastCol = tsdbCacheLookup(pTsdb, uid, cid, ltype); + if (!pLastCol) { + rocksdb_writebatch_t *wb = NULL; + + taosThreadMutexLock(&pTsdb->rCache.rMutex); + pLastCol = tsdbCacheLookup(pTsdb, uid, cid, ltype); + if (!pLastCol) { + // recalc: load from tsdb + int16_t aCols[1] = {cid}; + int16_t slotIds[1] = {slotid}; + SArray *pTmpColArray = NULL; + + if (ltype) { + mergeLastCid(uid, pTsdb, &pTmpColArray, pr, aCols, 1, slotIds); + } else { + mergeLastRowCid(uid, pTsdb, &pTmpColArray, pr, aCols, 1, slotIds); + } + + if (pTmpColArray && TARRAY_SIZE(pTmpColArray) >= 1) { + pLastCol = taosArrayGet(pTmpColArray, 0); + } + + // still null, then make up a none col value + SLastCol noneCol = {.ts = TSKEY_MIN, .colVal = COL_VAL_NONE(cid, pr->pSchema->columns[slotid].type)}; + if (!pLastCol) { + pLastCol = &noneCol; + } + + // store result back to rocks cache + wb = pTsdb->rCache.writebatch; + char *value = NULL; + size_t vlen = 0; + tsdbCacheSerialize(pLastCol, &value, &vlen); + + SLastKey *key = &(SLastKey){.ltype = ltype, .uid = uid, .cid = pLastCol->colVal.cid}; + size_t klen = ROCKS_KEY_LEN; + rocksdb_writebatch_put(wb, (char *)key, klen, value, vlen); + taosMemoryFree(value); + + SLastCol *pTmpLastCol = taosMemoryCalloc(1, sizeof(SLastCol)); + *pTmpLastCol = *pLastCol; + pLastCol = pTmpLastCol; + + taosArrayDestroy(pTmpColArray); + } + + if (wb) { + rocksMayWrite(pTsdb); + } + + taosThreadMutexUnlock(&pTsdb->rCache.rMutex); + } + + return pLastCol; +} + +static void tsdbCacheDeleter(const void *key, size_t keyLen, void *value) { + SLastCol *pLastCol = (SLastCol *)value; + + // TODO: add dirty flag to SLastCol + if (pLastCol->dirty) { + // TODO: queue into dirty list, free it after save to backstore + } else { + if (IS_VAR_DATA_TYPE(pLastCol->colVal.type) && pLastCol->colVal.value.nData > 0) { + taosMemoryFree(pLastCol->colVal.value.pData); + } + + taosMemoryFree(value); + } +} + +typedef struct { + int idx; + SLastKey key; +} SIdxKey; + +static int32_t tsdbCacheLoadFromRaw(STsdb *pTsdb, tb_uid_t uid, SArray *pLastArray, SArray *remainCols, + SCacheRowsReader *pr, int8_t ltype) { + int32_t code = 0; + rocksdb_writebatch_t *wb = NULL; + SArray *pTmpColArray = NULL; + int num_keys = TARRAY_SIZE(remainCols); + int16_t *aCols = taosMemoryMalloc(num_keys * sizeof(int16_t)); + int16_t *slotIds = taosMemoryMalloc(num_keys * sizeof(int16_t)); + + for (int i = 0; i < num_keys; ++i) { + SIdxKey *idxKey = taosArrayGet(remainCols, i); + aCols[i] = idxKey->key.cid; + slotIds[i] = pr->pSlotIds[idxKey->idx]; + } + + if (ltype) { + mergeLastCid(uid, pTsdb, &pTmpColArray, pr, aCols, num_keys, slotIds); + } else { + mergeLastRowCid(uid, pTsdb, &pTmpColArray, pr, aCols, num_keys, slotIds); + } + + SLRUCache *pCache = pTsdb->lruCache; + for (int i = 0; i < num_keys; ++i) { + SIdxKey *idxKey = taosArrayGet(remainCols, i); + SLastCol *pLastCol = NULL; + + if (pTmpColArray && TARRAY_SIZE(pTmpColArray) >= i + 1) { + pLastCol = taosArrayGet(pTmpColArray, i); + } + + // still null, then make up a none col value + SLastCol noneCol = {.ts = TSKEY_MIN, + .colVal = COL_VAL_NONE(idxKey->key.cid, pr->pSchema->columns[slotIds[i]].type)}; + if (!pLastCol) { + pLastCol = &noneCol; + } + + SLastCol *pTmpLastCol = taosMemoryCalloc(1, sizeof(SLastCol)); + *pTmpLastCol = *pLastCol; + pLastCol = pTmpLastCol; + + size_t charge = sizeof(*pLastCol); + if (IS_VAR_DATA_TYPE(pLastCol->colVal.type)) { + charge += pLastCol->colVal.value.nData; + } + + LRUStatus status = taosLRUCacheInsert(pCache, &idxKey->key, ROCKS_KEY_LEN, pLastCol, charge, tsdbCacheDeleter, NULL, + TAOS_LRU_PRIORITY_LOW); + if (status != TAOS_LRU_STATUS_OK) { + code = -1; + } + + // store result back to rocks cache + wb = pTsdb->rCache.writebatch; + char *value = NULL; + size_t vlen = 0; + tsdbCacheSerialize(pLastCol, &value, &vlen); + + SLastKey *key = &idxKey->key; + size_t klen = ROCKS_KEY_LEN; + rocksdb_writebatch_put(wb, (char *)key, klen, value, vlen); + taosMemoryFree(value); + + taosArraySet(pLastArray, idxKey->idx, pLastCol); + // taosArrayRemove(remainCols, i); + } + + if (wb) { + rocksMayWrite(pTsdb); + } + + taosArrayDestroy(pTmpColArray); + + taosMemoryFree(aCols); + taosMemoryFree(slotIds); + + return code; +} + +static int32_t tsdbCacheLoadFromRocks(STsdb *pTsdb, tb_uid_t uid, SArray *pLastArray, SArray *remainCols, + SCacheRowsReader *pr, int8_t ltype) { + int32_t code = 0; + int num_keys = TARRAY_SIZE(remainCols); + char **keys_list = taosMemoryMalloc(num_keys * sizeof(char *)); + size_t *keys_list_sizes = taosMemoryMalloc(num_keys * sizeof(size_t)); + char *key_list = taosMemoryMalloc(num_keys * ROCKS_KEY_LEN); + for (int i = 0; i < num_keys; ++i) { + int16_t cid = *(int16_t *)taosArrayGet(remainCols, i); + + memcpy(key_list + i * ROCKS_KEY_LEN, &((SIdxKey *)taosArrayGet(remainCols, i))->key, ROCKS_KEY_LEN); + keys_list[i] = key_list + i * ROCKS_KEY_LEN; + keys_list_sizes[i] = ROCKS_KEY_LEN; + } + + char **values_list = taosMemoryCalloc(num_keys, sizeof(char *)); + size_t *values_list_sizes = taosMemoryCalloc(num_keys, sizeof(size_t)); + char **errs = taosMemoryMalloc(num_keys * sizeof(char *)); + rocksdb_multi_get(pTsdb->rCache.db, pTsdb->rCache.readoptions, num_keys, (const char *const *)keys_list, + keys_list_sizes, values_list, values_list_sizes, errs); + for (int i = 0; i < num_keys; ++i) { + if (errs[i]) { + rocksdb_free(errs[i]); + } + } + taosMemoryFree(key_list); + taosMemoryFree(keys_list); + taosMemoryFree(keys_list_sizes); + taosMemoryFree(errs); + + SLRUCache *pCache = pTsdb->lruCache; + for (int i = 0, j = 0; i < num_keys && j < TARRAY_SIZE(remainCols); ++i) { + SLastCol *pLastCol = tsdbCacheDeserialize(values_list[i]); + SIdxKey *idxKey = taosArrayGet(remainCols, j); + int16_t cid = idxKey->key.cid; + SLastCol noneCol = {.ts = TSKEY_MIN, .colVal = COL_VAL_NONE(cid, pr->pSchema->columns[pr->pSlotIds[i]].type)}; + if (pLastCol) { + SLastCol *pTmpLastCol = taosMemoryCalloc(1, sizeof(SLastCol)); + *pTmpLastCol = *pLastCol; + pLastCol = pTmpLastCol; + + reallocVarData(&pLastCol->colVal); + size_t charge = sizeof(*pLastCol); + if (IS_VAR_DATA_TYPE(pLastCol->colVal.type)) { + charge += pLastCol->colVal.value.nData; + } + + LRUStatus status = taosLRUCacheInsert(pCache, &idxKey->key, ROCKS_KEY_LEN, pLastCol, charge, tsdbCacheDeleter, + NULL, TAOS_LRU_PRIORITY_LOW); + if (status != TAOS_LRU_STATUS_OK) { + code = -1; + } + + SLastCol lastCol = *pLastCol; + reallocVarData(&lastCol.colVal); + + taosArraySet(pLastArray, idxKey->idx, &lastCol); + taosArrayRemove(remainCols, j); + + taosMemoryFree(values_list[i]); + } else { + ++j; + } + } + taosMemoryFree(values_list); + taosMemoryFree(values_list_sizes); + + if (TARRAY_SIZE(remainCols) > 0) { + code = tsdbCacheLoadFromRaw(pTsdb, uid, pLastArray, remainCols, pr, ltype); + } + + return code; +} + +int32_t tsdbCacheGetBatch(STsdb *pTsdb, tb_uid_t uid, SArray *pLastArray, SCacheRowsReader *pr, int8_t ltype) { + int32_t code = 0; + SLRUCache *pCache = pTsdb->lruCache; + SArray *pCidList = pr->pCidList; + int num_keys = TARRAY_SIZE(pCidList); + + SArray *remainCols = NULL; + + for (int i = 0; i < num_keys; ++i) { + int16_t cid = *(int16_t *)taosArrayGet(pCidList, i); + + SLastKey *key = &(SLastKey){.ltype = ltype, .uid = uid, .cid = cid}; + + LRUHandle *h = taosLRUCacheLookup(pCache, key, ROCKS_KEY_LEN); + if (h) { + SLastCol *pLastCol = (SLastCol *)taosLRUCacheValue(pCache, h); + + SLastCol lastCol = *pLastCol; + reallocVarData(&lastCol.colVal); + taosArrayPush(pLastArray, &lastCol); + + if (h) { + taosLRUCacheRelease(pCache, h, false); + } + } else { + SLastCol noneCol = {.ts = TSKEY_MIN, .colVal = COL_VAL_NONE(cid, pr->pSchema->columns[pr->pSlotIds[i]].type)}; + + taosArrayPush(pLastArray, &noneCol); + + if (!remainCols) { + remainCols = taosArrayInit(num_keys, sizeof(SIdxKey)); + } + taosArrayPush(remainCols, &(SIdxKey){i, *key}); + } + } + + if (remainCols && TARRAY_SIZE(remainCols) > 0) { + taosThreadMutexLock(&pTsdb->lruMutex); + for (int i = 0; i < TARRAY_SIZE(remainCols);) { + SIdxKey *idxKey = taosArrayGet(remainCols, i); + LRUHandle *h = taosLRUCacheLookup(pCache, &idxKey->key, ROCKS_KEY_LEN); + if (h) { + SLastCol *pLastCol = (SLastCol *)taosLRUCacheValue(pCache, h); + + SLastCol lastCol = *pLastCol; + reallocVarData(&lastCol.colVal); + + taosArraySet(pLastArray, idxKey->idx, &lastCol); + if (h) { + taosLRUCacheRelease(pCache, h, false); + } + + taosArrayRemove(remainCols, i); + } else { + ++i; + } + } + + code = tsdbCacheLoadFromRocks(pTsdb, uid, pLastArray, remainCols, pr, ltype); + + taosThreadMutexUnlock(&pTsdb->lruMutex); + } + + if (remainCols) { + taosArrayDestroy(remainCols); + } + + return code; +} + +int32_t tsdbCacheGet(STsdb *pTsdb, tb_uid_t uid, SArray *pLastArray, SCacheRowsReader *pr, int8_t ltype) { + int32_t code = 0; + SLRUCache *pCache = pTsdb->lruCache; + SArray *pCidList = pr->pCidList; + int num_keys = TARRAY_SIZE(pCidList); + + for (int i = 0; i < num_keys; ++i) { + SLastCol *pLastCol = NULL; + int16_t cid = *(int16_t *)taosArrayGet(pCidList, i); + + SLastKey *key = &(SLastKey){.ltype = ltype, .uid = uid, .cid = cid}; + LRUHandle *h = taosLRUCacheLookup(pCache, key, ROCKS_KEY_LEN); + if (!h) { + taosThreadMutexLock(&pTsdb->lruMutex); + h = taosLRUCacheLookup(pCache, key, ROCKS_KEY_LEN); + if (!h) { + pLastCol = tsdbCacheLoadCol(pTsdb, pr, pr->pSlotIds[i], uid, cid, ltype); + + size_t charge = sizeof(*pLastCol); + if (IS_VAR_DATA_TYPE(pLastCol->colVal.type)) { + charge += pLastCol->colVal.value.nData; + } + + LRUStatus status = taosLRUCacheInsert(pCache, key, ROCKS_KEY_LEN, pLastCol, charge, tsdbCacheDeleter, &h, + TAOS_LRU_PRIORITY_LOW); + if (status != TAOS_LRU_STATUS_OK) { + code = -1; + } + } + + taosThreadMutexUnlock(&pTsdb->lruMutex); + } + + pLastCol = (SLastCol *)taosLRUCacheValue(pCache, h); + + SLastCol lastCol = *pLastCol; + reallocVarData(&lastCol.colVal); + + if (h) { + taosLRUCacheRelease(pCache, h, false); + } + + taosArrayPush(pLastArray, &lastCol); + } + + return code; +} int32_t tsdbCacheDel(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSKEY sKey, TSKEY eKey) { int32_t code = 0; @@ -486,19 +893,15 @@ int32_t tsdbCacheDel(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSKEY sKey, TSKE for (int i = 0; i < num_keys; ++i) { int16_t cid = pTSchema->columns[i].colId; - char *keys = taosMemoryCalloc(2, ROCKS_KEY_LEN); - int last_key_len = snprintf(keys, ROCKS_KEY_LEN, "%" PRIi64 ":%" PRIi16 ":last", uid, cid); - if (last_key_len >= ROCKS_KEY_LEN) { - tsdbError("vgId:%d, %s failed at line %d since %s", TD_VID(pTsdb->pVnode), __func__, __LINE__, tstrerror(code)); - } - int lr_key_len = snprintf(keys + ROCKS_KEY_LEN, ROCKS_KEY_LEN, "%" PRIi64 ":%" PRIi16 ":last_row", uid, cid); - if (lr_key_len >= ROCKS_KEY_LEN) { - tsdbError("vgId:%d, %s failed at line %d since %s", TD_VID(pTsdb->pVnode), __func__, __LINE__, tstrerror(code)); - } + size_t klen = ROCKS_KEY_LEN; + char *keys = taosMemoryCalloc(2, klen); + ((SLastKey *)keys)[0] = (SLastKey){.ltype = 1, .uid = uid, .cid = cid}; + ((SLastKey *)keys)[1] = (SLastKey){.ltype = 0, .uid = uid, .cid = cid}; + keys_list[i] = keys; - keys_list[num_keys + i] = keys + ROCKS_KEY_LEN; - keys_list_sizes[i] = last_key_len; - keys_list_sizes[num_keys + i] = lr_key_len; + keys_list[num_keys + i] = keys + klen; + keys_list_sizes[i] = klen; + keys_list_sizes[num_keys + i] = klen; } char **values_list = taosMemoryCalloc(num_keys * 2, sizeof(char *)); size_t *values_list_sizes = taosMemoryCalloc(num_keys * 2, sizeof(size_t)); @@ -520,16 +923,18 @@ int32_t tsdbCacheDel(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSKEY sKey, TSKE for (int i = 0; i < num_keys; ++i) { SLastCol *pLastCol = tsdbCacheDeserialize(values_list[i]); if (NULL != pLastCol && (pLastCol->ts <= eKey && pLastCol->ts >= sKey)) { - char key[ROCKS_KEY_LEN]; - size_t klen = snprintf(key, ROCKS_KEY_LEN, "%" PRIi64 ":%" PRIi16 ":last", uid, pLastCol->colVal.cid); - rocksdb_writebatch_delete(wb, key, klen); + SLastKey *key = &(SLastKey){.ltype = 1, .uid = uid, .cid = pLastCol->colVal.cid}; + size_t klen = sizeof(*key); + + rocksdb_writebatch_delete(wb, (char *)key, klen); } pLastCol = tsdbCacheDeserialize(values_list[i + num_keys]); if (NULL != pLastCol && (pLastCol->ts <= eKey && pLastCol->ts >= sKey)) { - char key[ROCKS_KEY_LEN]; - size_t klen = snprintf(key, ROCKS_KEY_LEN, "%" PRIi64 ":%" PRIi16 ":last_row", uid, pLastCol->colVal.cid); - rocksdb_writebatch_delete(wb, key, klen); + SLastKey *key = &(SLastKey){.ltype = 0, .uid = uid, .cid = pLastCol->colVal.cid}; + size_t klen = sizeof(*key); + + rocksdb_writebatch_delete(wb, (char *)key, klen); } rocksdb_free(values_list[i]); @@ -538,14 +943,8 @@ int32_t tsdbCacheDel(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSKEY sKey, TSKE taosMemoryFree(values_list); taosMemoryFree(values_list_sizes); - char *err = NULL; - rocksdb_write(pTsdb->rCache.db, pTsdb->rCache.writeoptions, wb, &err); - if (NULL != err) { - tsdbError("vgId:%d, %s failed at line %d since %s", TD_VID(pTsdb->pVnode), __func__, __LINE__, err); - rocksdb_free(err); - } + rocksMayWrite(pTsdb); taosThreadMutexUnlock(&pTsdb->rCache.rMutex); - rocksdb_writebatch_clear(wb); _exit: taosMemoryFree(pTSchema); @@ -1111,7 +1510,7 @@ typedef struct { SMergeTree mergeTree; SMergeTree *pMergeTree; SSttBlockLoadInfo *pLoadInfo; - SLDataIter* pDataIter; + SLDataIter *pDataIter; int64_t lastTs; } SFSLastNextRowIter; @@ -1159,7 +1558,8 @@ static int32_t getNextRowFromFSLast(void *iter, TSDBROW **ppRow, bool *pIgnoreEa } tMergeTreeOpen(&state->mergeTree, 1, *state->pDataFReader, state->suid, state->uid, &(STimeWindow){.skey = state->lastTs, .ekey = TSKEY_MAX}, - &(SVersionRange){.minVer = 0, .maxVer = UINT64_MAX}, state->pLoadInfo, false, NULL, true, state->pDataIter); + &(SVersionRange){.minVer = 0, .maxVer = UINT64_MAX}, state->pLoadInfo, false, NULL, true, + state->pDataIter); state->pMergeTree = &state->mergeTree; state->state = SFSLASTNEXTROW_BLOCKROW; } @@ -1394,11 +1794,13 @@ static int32_t getNextRowFromFS(void *iter, TSDBROW **ppRow, bool *pIgnoreEarlie tBlockDataReset(state->pBlockData); TABLEID tid = {.suid = state->suid, .uid = state->uid}; int nTmpCols = nCols; - if (aCols[0] == PRIMARYKEY_TIMESTAMP_COL_ID && nCols == 1) { - nTmpCols = 0; + bool hasTs = false; + if (aCols[0] == PRIMARYKEY_TIMESTAMP_COL_ID) { + --nTmpCols; skipBlock = false; + hasTs = true; } - code = tBlockDataInit(state->pBlockData, &tid, state->pTSchema, aCols, nTmpCols); + code = tBlockDataInit(state->pBlockData, &tid, state->pTSchema, hasTs ? aCols + 1 : aCols, nTmpCols); if (code) goto _err; code = tsdbReadDataBlock(*state->pDataFReader, &block, state->pBlockData); @@ -1730,8 +2132,8 @@ typedef struct { } CacheNextRowIter; static int32_t nextRowIterOpen(CacheNextRowIter *pIter, tb_uid_t uid, STsdb *pTsdb, STSchema *pTSchema, tb_uid_t suid, - SSttBlockLoadInfo *pLoadInfo, SLDataIter* pLDataIter, STsdbReadSnap *pReadSnap, SDataFReader **pDataFReader, - SDataFReader **pDataFReaderLast, int64_t lastTs) { + SSttBlockLoadInfo *pLoadInfo, SLDataIter *pLDataIter, STsdbReadSnap *pReadSnap, + SDataFReader **pDataFReader, SDataFReader **pDataFReaderLast, int64_t lastTs) { int code = 0; STbData *pMem = NULL; diff --git a/source/dnode/vnode/src/tsdb/tsdbCacheRead.c b/source/dnode/vnode/src/tsdb/tsdbCacheRead.c index 53103e9fbb..f6e37d9427 100644 --- a/source/dnode/vnode/src/tsdb/tsdbCacheRead.c +++ b/source/dnode/vnode/src/tsdb/tsdbCacheRead.c @@ -168,8 +168,7 @@ int32_t tsdbCacherowsReaderOpen(void* pVnode, int32_t type, void* pTableIdList, } SVnodeCfg* pCfg = &((SVnode*)pVnode)->config; - - int32_t numOfStt = pCfg->sttTrigger; + int32_t numOfStt = pCfg->sttTrigger; p->pLoadInfo = tCreateLastBlockLoadInfo(p->pSchema, NULL, 0, numOfStt); if (p->pLoadInfo == NULL) { tsdbCacherowsReaderClose(p); @@ -203,7 +202,7 @@ void* tsdbCacherowsReaderClose(void* pReader) { taosMemoryFree(p->pSchema); } - taosMemoryFreeClear(p->pDataIter); + taosMemoryFree(p->pDataIter); taosMemoryFree(p->pCurrSchema); destroyLastBlockLoadInfo(p->pLoadInfo); @@ -294,21 +293,23 @@ int32_t tsdbRetrieveCacheRows(void* pReader, SSDataBlock* pResBlock, const int32 pr->pDataFReader = NULL; pr->pDataFReaderLast = NULL; - int32_t ltype = (pr->type & CACHESCAN_RETRIEVE_LAST) >> 3; + int8_t ltype = (pr->type & CACHESCAN_RETRIEVE_LAST) >> 3; // retrieve the only one last row of all tables in the uid list. if (HASTYPE(pr->type, CACHESCAN_RETRIEVE_TYPE_SINGLE)) { int64_t st = taosGetTimestampUs(); int64_t totalLastTs = INT64_MAX; + for (int32_t i = 0; i < pr->numOfTables; ++i) { STableKeyInfo* pKeyInfo = &pr->pTableList[i]; - tsdbCacheGet(pr->pTsdb, pKeyInfo->uid, pRow, pr, ltype); + tsdbCacheGetBatch(pr->pTsdb, pKeyInfo->uid, pRow, pr, ltype); + // tsdbCacheGet(pr->pTsdb, pKeyInfo->uid, pRow, pr, ltype); if (TARRAY_SIZE(pRow) <= 0) { taosArrayClearEx(pRow, freeItem); continue; } - SLastCol* pColVal = (SLastCol*)taosArrayGet(pRow, 0); + SLastCol* pColVal = taosArrayGet(pRow, 0); if (COL_VAL_IS_NONE(&pColVal->colVal)) { taosArrayClearEx(pRow, freeItem); continue; @@ -361,7 +362,7 @@ int32_t tsdbRetrieveCacheRows(void* pReader, SSDataBlock* pResBlock, const int32 } } - if (taosArrayGetSize(pTableUidList) == 0) { + if (TARRAY_SIZE(pTableUidList) == 0) { taosArrayPush(pTableUidList, &pKeyInfo->uid); } else { taosArraySet(pTableUidList, 0, &pKeyInfo->uid); @@ -375,9 +376,9 @@ int32_t tsdbRetrieveCacheRows(void* pReader, SSDataBlock* pResBlock, const int32 } } else if (HASTYPE(pr->type, CACHESCAN_RETRIEVE_TYPE_ALL)) { for (int32_t i = pr->tableIndex; i < pr->numOfTables; ++i) { - STableKeyInfo* pKeyInfo = &pr->pTableList[i]; + tb_uid_t uid = pr->pTableList[i].uid; - tsdbCacheGet(pr->pTsdb, pKeyInfo->uid, pRow, pr, ltype); + tsdbCacheGetBatch(pr->pTsdb, uid, pRow, pr, ltype); if (TARRAY_SIZE(pRow) <= 0) { taosArrayClearEx(pRow, freeItem); continue; @@ -391,9 +392,9 @@ int32_t tsdbRetrieveCacheRows(void* pReader, SSDataBlock* pResBlock, const int32 saveOneRow(pRow, pResBlock, pr, slotIds, dstSlotIds, pRes, pr->idstr); taosArrayClearEx(pRow, freeItem); - taosArrayPush(pTableUidList, &pKeyInfo->uid); + taosArrayPush(pTableUidList, &uid); - pr->tableIndex += 1; + ++pr->tableIndex; if (pResBlock->info.rows >= pResBlock->info.capacity) { goto _end; } @@ -419,5 +420,6 @@ _end: taosMemoryFree(pRes); taosArrayDestroyEx(pRow, freeItem); taosArrayDestroyEx(pLastCols, freeItem); + return code; } From 0099e1d9da263855cef98d7cfee64e7dc49a8935 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Chappyguoxy=E2=80=9D?= <“happy_guoxy@163.com”> Date: Wed, 17 May 2023 10:53:35 +0800 Subject: [PATCH 21/59] test: rowlength64k --- tests/parallel_test/cases.task | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/tests/parallel_test/cases.task b/tests/parallel_test/cases.task index 66eeb9e35f..8e786f7bc5 100644 --- a/tests/parallel_test/cases.task +++ b/tests/parallel_test/cases.task @@ -588,6 +588,16 @@ ,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k.py -Q 2 ,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k.py -Q 3 ,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k.py -Q 4 +,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k_1.py +,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k_1.py -R +,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k_1.py -Q 2 +,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k_1.py -Q 3 +,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k_1.py -Q 4 +,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k_2.py +,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k_2.py -R +,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k_2.py -Q 2 +,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k_2.py -Q 3 +,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k_2.py -Q 4 ,,y,system-test,./pytest.sh python3 ./test.py -f 0-others/show.py ,,y,system-test,./pytest.sh python3 ./test.py -f 0-others/information_schema.py ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/abs.py From 1d0148414d1e708fbec5d61d8ed87ef3a585dc27 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Chappyguoxy=E2=80=9D?= <“happy_guoxy@163.com”> Date: Wed, 17 May 2023 10:53:58 +0800 Subject: [PATCH 22/59] test: rowlength64k --- tests/system-test/1-insert/rowlength64k.py | 46 ++++++++++++++++++---- 1 file changed, 38 insertions(+), 8 deletions(-) diff --git a/tests/system-test/1-insert/rowlength64k.py b/tests/system-test/1-insert/rowlength64k.py index 1e7c262b29..3e6b5def7c 100755 --- a/tests/system-test/1-insert/rowlength64k.py +++ b/tests/system-test/1-insert/rowlength64k.py @@ -63,10 +63,7 @@ class TDTestCase: stable_sql = "select * from db.%s where loc match '[a-z]'" %stable_name tdSql.query(stable_sql) - def run(self): - tdSql.prepare() - - startTime_all = time.time() + def run_8(self): print("==============step8,stable table , mix data type==============") sql = "create stable db.stable_16(ts timestamp, " @@ -183,6 +180,8 @@ class TDTestCase: self.ins_query() + def run_9(self): + print("==============step9,stable table , mix data type==============") sql = "create stable db.stable_26(ts timestamp, " sql += "col4090 int ," @@ -297,6 +296,9 @@ class TDTestCase: tdSql.error(sql) self.ins_query() + + def run_1(self): + print("==============step1, regular table, 1 ts + 4094 cols + 1 binary==============") startTime = time.time() @@ -359,7 +361,8 @@ class TDTestCase: tdSql.checkCols(4096) self.ins_query() - + + def run_2(self): print("==============step2,regular table error col or value==============") tdLog.info('test regular table exceeds row num') @@ -394,6 +397,9 @@ class TDTestCase: tdSql.error(sql) self.ins_query() + + def run_3(self): + print("==============step3,regular table , mix data type==============") startTime = time.time() @@ -639,8 +645,9 @@ class TDTestCase: tdSql.error(sql) self.ins_query() - - + + def run_4(self): + print("==============step4, super table , 1 ts + 4090 cols + 4 tags ==============") startTime = time.time() sql = "create stable db.stable_1(ts timestamp, " @@ -727,7 +734,9 @@ class TDTestCase: tdSql.checkCols(4092) self.ins_query() - + + def run_5(self): + print("==============step5,stable table , mix data type==============") sql = "create stable db.stable_3(ts timestamp, " for i in range(500): @@ -950,6 +959,9 @@ class TDTestCase: self.ins_query() + def run_6(self): + + print("==============step6,stable table , mix data type==============") sql = "create stable db.stable_6(ts timestamp, " for i in range(500): @@ -1208,6 +1220,9 @@ class TDTestCase: sql += "col4091 binary(341))" sql += " tags (loc binary(16371),tag_1 int,tag_2 int,tag_3 int) " tdSql.error(sql) + + def run_7(self): + print("==============step7, super table error col ==============") tdLog.info('test exceeds row num') @@ -1248,6 +1263,21 @@ class TDTestCase: self.ins_query() + + def run(self): + tdSql.prepare() + + startTime_all = time.time() + self.run_8() + self.run_9() + self.run_1() + self.run_2() + # self.run_3() + # self.run_4() + # self.run_5() + # self.run_6() + # self.run_7() + endTime_all = time.time() print("total time %ds" % (endTime_all - startTime_all)) From 7af9e7d7cca22298d63360002f2269e21ce286b8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Chappyguoxy=E2=80=9D?= <“happy_guoxy@163.com”> Date: Wed, 17 May 2023 10:54:10 +0800 Subject: [PATCH 23/59] test: rowlength64k --- tests/system-test/1-insert/rowlength64k_1.py | 40 ++++++++++++++++++++ 1 file changed, 40 insertions(+) create mode 100755 tests/system-test/1-insert/rowlength64k_1.py diff --git a/tests/system-test/1-insert/rowlength64k_1.py b/tests/system-test/1-insert/rowlength64k_1.py new file mode 100755 index 0000000000..fe676cd71a --- /dev/null +++ b/tests/system-test/1-insert/rowlength64k_1.py @@ -0,0 +1,40 @@ +################################################################### +# Copyright (c) 2016 by TAOS Technologies, Inc. +# All rights reserved. +# +# This file is proprietary and confidential to TAOS Technologies. +# No part of this file may be reproduced, stored, transmitted, +# disclosed or used in any form or by any means other than as +# expressly provided by the written permission from Jianhui Tao +# +################################################################### + +# -*- coding: utf-8 -*- +from util.cases import tdCases +from .rowlength64k import * + +class TDTestCase(TDTestCase): + + + def run(self): + tdSql.prepare() + + startTime_all = time.time() + #self.run_1() + # self.run_2() + self.run_3() + self.run_4() + + endTime_all = time.time() + print("total time %ds" % (endTime_all - startTime_all)) + + + + + def stop(self): + tdSql.close() + tdLog.success("%s successfully executed" % __file__) + + +tdCases.addWindows(__file__, TDTestCase()) +tdCases.addLinux(__file__, TDTestCase()) From ed703c650b01a34a7b729493aba6d4aa0d01d584 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Chappyguoxy=E2=80=9D?= <“happy_guoxy@163.com”> Date: Wed, 17 May 2023 10:54:17 +0800 Subject: [PATCH 24/59] test: rowlength64k --- tests/system-test/1-insert/rowlength64k_2.py | 43 ++++++++++++++++++++ 1 file changed, 43 insertions(+) create mode 100755 tests/system-test/1-insert/rowlength64k_2.py diff --git a/tests/system-test/1-insert/rowlength64k_2.py b/tests/system-test/1-insert/rowlength64k_2.py new file mode 100755 index 0000000000..becae4daa4 --- /dev/null +++ b/tests/system-test/1-insert/rowlength64k_2.py @@ -0,0 +1,43 @@ +################################################################### +# Copyright (c) 2016 by TAOS Technologies, Inc. +# All rights reserved. +# +# This file is proprietary and confidential to TAOS Technologies. +# No part of this file may be reproduced, stored, transmitted, +# disclosed or used in any form or by any means other than as +# expressly provided by the written permission from Jianhui Tao +# +################################################################### + +# -*- coding: utf-8 -*- +from util.cases import tdCases +from .rowlength64k import * + +class TDTestCase(TDTestCase): + + + def run(self): + tdSql.prepare() + + startTime_all = time.time() + # self.run_1() + # self.run_2() + # self.run_3() + # self.run_4() + self.run_5() + self.run_6() + self.run_7() + + endTime_all = time.time() + print("total time %ds" % (endTime_all - startTime_all)) + + + + + def stop(self): + tdSql.close() + tdLog.success("%s successfully executed" % __file__) + + +tdCases.addWindows(__file__, TDTestCase()) +tdCases.addLinux(__file__, TDTestCase()) From 86e3c1d20d8eb8413cf5022e9aed40546c2c0180 Mon Sep 17 00:00:00 2001 From: Minglei Jin Date: Wed, 17 May 2023 14:05:28 +0800 Subject: [PATCH 25/59] cache/stt: remove ts from merge tree loading --- source/dnode/vnode/src/tsdb/tsdbCache.c | 24 ++++++++++++--------- source/dnode/vnode/src/tsdb/tsdbCacheRead.c | 21 ++++++++++++------ 2 files changed, 28 insertions(+), 17 deletions(-) diff --git a/source/dnode/vnode/src/tsdb/tsdbCache.c b/source/dnode/vnode/src/tsdb/tsdbCache.c index a637ba4968..3890e3d1f7 100644 --- a/source/dnode/vnode/src/tsdb/tsdbCache.c +++ b/source/dnode/vnode/src/tsdb/tsdbCache.c @@ -716,9 +716,7 @@ static int32_t tsdbCacheLoadFromRocks(STsdb *pTsdb, tb_uid_t uid, SArray *pLastA SLRUCache *pCache = pTsdb->lruCache; for (int i = 0, j = 0; i < num_keys && j < TARRAY_SIZE(remainCols); ++i) { SLastCol *pLastCol = tsdbCacheDeserialize(values_list[i]); - SIdxKey *idxKey = taosArrayGet(remainCols, j); - int16_t cid = idxKey->key.cid; - SLastCol noneCol = {.ts = TSKEY_MIN, .colVal = COL_VAL_NONE(cid, pr->pSchema->columns[pr->pSlotIds[i]].type)}; + SIdxKey *idxKey = &((SIdxKey *)TARRAY_DATA(remainCols))[j]; if (pLastCol) { SLastCol *pTmpLastCol = taosMemoryCalloc(1, sizeof(SLastCol)); *pTmpLastCol = *pLastCol; @@ -747,6 +745,7 @@ static int32_t tsdbCacheLoadFromRocks(STsdb *pTsdb, tb_uid_t uid, SArray *pLastA ++j; } } + taosMemoryFree(values_list); taosMemoryFree(values_list_sizes); @@ -759,14 +758,13 @@ static int32_t tsdbCacheLoadFromRocks(STsdb *pTsdb, tb_uid_t uid, SArray *pLastA int32_t tsdbCacheGetBatch(STsdb *pTsdb, tb_uid_t uid, SArray *pLastArray, SCacheRowsReader *pr, int8_t ltype) { int32_t code = 0; + SArray *remainCols = NULL; SLRUCache *pCache = pTsdb->lruCache; SArray *pCidList = pr->pCidList; int num_keys = TARRAY_SIZE(pCidList); - SArray *remainCols = NULL; - for (int i = 0; i < num_keys; ++i) { - int16_t cid = *(int16_t *)taosArrayGet(pCidList, i); + int16_t cid = ((int16_t *)TARRAY_DATA(pCidList))[i]; SLastKey *key = &(SLastKey){.ltype = ltype, .uid = uid, .cid = cid}; @@ -775,7 +773,7 @@ int32_t tsdbCacheGetBatch(STsdb *pTsdb, tb_uid_t uid, SArray *pLastArray, SCache SLastCol *pLastCol = (SLastCol *)taosLRUCacheValue(pCache, h); SLastCol lastCol = *pLastCol; - reallocVarData(&lastCol.colVal); + // reallocVarData(&lastCol.colVal); taosArrayPush(pLastArray, &lastCol); if (h) { @@ -796,7 +794,7 @@ int32_t tsdbCacheGetBatch(STsdb *pTsdb, tb_uid_t uid, SArray *pLastArray, SCache if (remainCols && TARRAY_SIZE(remainCols) > 0) { taosThreadMutexLock(&pTsdb->lruMutex); for (int i = 0; i < TARRAY_SIZE(remainCols);) { - SIdxKey *idxKey = taosArrayGet(remainCols, i); + SIdxKey *idxKey = &((SIdxKey *)TARRAY_DATA(remainCols))[i]; LRUHandle *h = taosLRUCacheLookup(pCache, &idxKey->key, ROCKS_KEY_LEN); if (h) { SLastCol *pLastCol = (SLastCol *)taosLRUCacheValue(pCache, h); @@ -1551,9 +1549,15 @@ static int32_t getNextRowFromFSLast(void *iter, TSDBROW **ppRow, bool *pIgnoreEa if (code) goto _err; } + int nTmpCols = nCols; + bool hasTs = false; + if (aCols[0] == PRIMARYKEY_TIMESTAMP_COL_ID) { + --nTmpCols; + hasTs = true; + } for (int i = 0; i < state->pLoadInfo->numOfStt; ++i) { - state->pLoadInfo[i].colIds = aCols; - state->pLoadInfo[i].numOfCols = nCols; + state->pLoadInfo[i].colIds = hasTs ? aCols + 1 : aCols; + state->pLoadInfo[i].numOfCols = nTmpCols; state->pLoadInfo[i].isLast = isLast; } tMergeTreeOpen(&state->mergeTree, 1, *state->pDataFReader, state->suid, state->uid, diff --git a/source/dnode/vnode/src/tsdb/tsdbCacheRead.c b/source/dnode/vnode/src/tsdb/tsdbCacheRead.c index f6e37d9427..d15eb0a911 100644 --- a/source/dnode/vnode/src/tsdb/tsdbCacheRead.c +++ b/source/dnode/vnode/src/tsdb/tsdbCacheRead.c @@ -306,12 +306,14 @@ int32_t tsdbRetrieveCacheRows(void* pReader, SSDataBlock* pResBlock, const int32 tsdbCacheGetBatch(pr->pTsdb, pKeyInfo->uid, pRow, pr, ltype); // tsdbCacheGet(pr->pTsdb, pKeyInfo->uid, pRow, pr, ltype); if (TARRAY_SIZE(pRow) <= 0) { - taosArrayClearEx(pRow, freeItem); + // taosArrayClearEx(pRow, freeItem); + taosArrayClear(pRow); continue; } SLastCol* pColVal = taosArrayGet(pRow, 0); if (COL_VAL_IS_NONE(&pColVal->colVal)) { - taosArrayClearEx(pRow, freeItem); + // taosArrayClearEx(pRow, freeItem); + taosArrayClear(pRow); continue; } @@ -368,7 +370,8 @@ int32_t tsdbRetrieveCacheRows(void* pReader, SSDataBlock* pResBlock, const int32 taosArraySet(pTableUidList, 0, &pKeyInfo->uid); } - taosArrayClearEx(pRow, freeItem); + // taosArrayClearEx(pRow, freeItem); + taosArrayClear(pRow); } if (hasRes) { @@ -380,17 +383,20 @@ int32_t tsdbRetrieveCacheRows(void* pReader, SSDataBlock* pResBlock, const int32 tsdbCacheGetBatch(pr->pTsdb, uid, pRow, pr, ltype); if (TARRAY_SIZE(pRow) <= 0) { - taosArrayClearEx(pRow, freeItem); + // taosArrayClearEx(pRow, freeItem); + taosArrayClear(pRow); continue; } SLastCol* pColVal = (SLastCol*)taosArrayGet(pRow, 0); if (COL_VAL_IS_NONE(&pColVal->colVal)) { - taosArrayClearEx(pRow, freeItem); + // taosArrayClearEx(pRow, freeItem); + taosArrayClear(pRow); continue; } saveOneRow(pRow, pResBlock, pr, slotIds, dstSlotIds, pRes, pr->idstr); - taosArrayClearEx(pRow, freeItem); + // taosArrayClearEx(pRow, freeItem); + taosArrayClear(pRow); taosArrayPush(pTableUidList, &uid); @@ -418,7 +424,8 @@ _end: } taosMemoryFree(pRes); - taosArrayDestroyEx(pRow, freeItem); + // taosArrayDestroyEx(pRow, freeItem); + taosArrayDestroy(pRow); taosArrayDestroyEx(pLastCols, freeItem); return code; From e95613ec10910c74a52e1d66de820f3ccd87b5c5 Mon Sep 17 00:00:00 2001 From: Minglei Jin Date: Wed, 17 May 2023 14:32:02 +0800 Subject: [PATCH 26/59] cache/commit: update & commit cache if cache not off --- source/dnode/vnode/src/tsdb/tsdbMemTable.c | 9 ++++++--- source/dnode/vnode/src/vnd/vnodeCommit.c | 6 ++++-- 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/source/dnode/vnode/src/tsdb/tsdbMemTable.c b/source/dnode/vnode/src/tsdb/tsdbMemTable.c index 97b648201c..803a4f6c2d 100644 --- a/source/dnode/vnode/src/tsdb/tsdbMemTable.c +++ b/source/dnode/vnode/src/tsdb/tsdbMemTable.c @@ -302,12 +302,12 @@ int64_t tsdbCountTbDataRows(STbData *pTbData) { return rowsNum; } -void tsdbMemTableCountRows(SMemTable *pMemTable, SSHashObj* pTableMap, int64_t *rowsNum) { +void tsdbMemTableCountRows(SMemTable *pMemTable, SSHashObj *pTableMap, int64_t *rowsNum) { taosRLockLatch(&pMemTable->latch); for (int32_t i = 0; i < pMemTable->nBucket; ++i) { STbData *pTbData = pMemTable->aBucket[i]; while (pTbData) { - void* p = tSimpleHashGet(pTableMap, &pTbData->uid, sizeof(pTbData->uid)); + void *p = tSimpleHashGet(pTableMap, &pTbData->uid, sizeof(pTbData->uid)); if (p == NULL) { pTbData = pTbData->next; continue; @@ -673,7 +673,10 @@ static int32_t tsdbInsertColDataToTable(SMemTable *pMemTable, STbData *pTbData, if (key.ts >= pTbData->maxKey) { pTbData->maxKey = key.ts; } - tsdbCacheUpdate(pMemTable->pTsdb, pTbData->suid, pTbData->uid, &lRow); + + if (!TSDB_CACHE_NO(pMemTable->pTsdb->pVnode->config)) { + tsdbCacheUpdate(pMemTable->pTsdb, pTbData->suid, pTbData->uid, &lRow); + } // SMemTable pMemTable->minKey = TMIN(pMemTable->minKey, pTbData->minKey); diff --git a/source/dnode/vnode/src/vnd/vnodeCommit.c b/source/dnode/vnode/src/vnd/vnodeCommit.c index 74168591d2..77453fd894 100644 --- a/source/dnode/vnode/src/vnd/vnodeCommit.c +++ b/source/dnode/vnode/src/vnd/vnodeCommit.c @@ -439,8 +439,10 @@ static int vnodeCommitImpl(SCommitInfo *pInfo) { code = tsdbCommit(pVnode->pTsdb, pInfo); TSDB_CHECK_CODE(code, lino, _exit); - code = tsdbCacheCommit(pVnode->pTsdb); - TSDB_CHECK_CODE(code, lino, _exit); + if (!TSDB_CACHE_NO(pVnode->config)) { + code = tsdbCacheCommit(pVnode->pTsdb); + TSDB_CHECK_CODE(code, lino, _exit); + } if (VND_IS_RSMA(pVnode)) { code = smaCommit(pVnode->pSma, pInfo); From b892b1fdfe521f46415deddad405afaea5e28e35 Mon Sep 17 00:00:00 2001 From: Minglei Jin Date: Wed, 17 May 2023 15:26:10 +0800 Subject: [PATCH 27/59] cache/flag: fix cache flag compiling issue --- source/dnode/vnode/src/inc/tsdb.h | 4 --- source/dnode/vnode/src/inc/vnodeInt.h | 6 +++- source/dnode/vnode/src/tsdb/tsdbCache.c | 48 ++++++++++++------------- 3 files changed, 29 insertions(+), 29 deletions(-) diff --git a/source/dnode/vnode/src/inc/tsdb.h b/source/dnode/vnode/src/inc/tsdb.h index 486becdf96..9baf38bca8 100644 --- a/source/dnode/vnode/src/inc/tsdb.h +++ b/source/dnode/vnode/src/inc/tsdb.h @@ -305,10 +305,6 @@ void tsdbUntakeReadSnap(STsdbReader *pReader, STsdbReadSnap *pSnap, bool proa // tsdbMerge.c ============================================================================================== int32_t tsdbMerge(STsdb *pTsdb); -#define TSDB_CACHE_NO(c) ((c).cacheLast == 0) -#define TSDB_CACHE_LAST_ROW(c) (((c).cacheLast & 1) > 0) -#define TSDB_CACHE_LAST(c) (((c).cacheLast & 2) > 0) - // tsdbDiskData ============================================================================================== int32_t tDiskDataBuilderCreate(SDiskDataBuilder **ppBuilder); void *tDiskDataBuilderDestroy(SDiskDataBuilder *pBuilder); diff --git a/source/dnode/vnode/src/inc/vnodeInt.h b/source/dnode/vnode/src/inc/vnodeInt.h index d7f0ef041a..c92cdd32b0 100644 --- a/source/dnode/vnode/src/inc/vnodeInt.h +++ b/source/dnode/vnode/src/inc/vnodeInt.h @@ -197,7 +197,7 @@ void tqClose(STQ*); int tqPushMsg(STQ*, void* msg, int32_t msgLen, tmsg_t msgType, int64_t ver); int tqRegisterPushHandle(STQ* pTq, void* handle, SRpcMsg* pMsg); int tqUnregisterPushHandle(STQ* pTq, void* pHandle); -int tqStartStreamTasks(STQ* pTq); // restore all stream tasks after vnode launching completed. +int tqStartStreamTasks(STQ* pTq); // restore all stream tasks after vnode launching completed. int tqCommit(STQ*); int32_t tqUpdateTbUidList(STQ* pTq, const SArray* tbUidList, bool isAdd); @@ -405,6 +405,10 @@ struct SVnode { #define VND_IS_RSMA(v) ((v)->config.isRsma == 1) #define VND_IS_TSMA(v) ((v)->config.isTsma == 1) +#define TSDB_CACHE_NO(c) ((c).cacheLast == 0) +#define TSDB_CACHE_LAST_ROW(c) (((c).cacheLast & 1) > 0) +#define TSDB_CACHE_LAST(c) (((c).cacheLast & 2) > 0) + struct STbUidStore { tb_uid_t suid; SArray* tbUids; diff --git a/source/dnode/vnode/src/tsdb/tsdbCache.c b/source/dnode/vnode/src/tsdb/tsdbCache.c index 3890e3d1f7..8f0d541d49 100644 --- a/source/dnode/vnode/src/tsdb/tsdbCache.c +++ b/source/dnode/vnode/src/tsdb/tsdbCache.c @@ -208,10 +208,26 @@ static void tsdbCloseRocksCache(STsdb *pTsdb) { taosMemoryFree(pTsdb->rCache.pTSchema); } +static void rocksMayWrite(STsdb *pTsdb, bool force) { + rocksdb_writebatch_t *wb = pTsdb->rCache.writebatch; + + if (force || rocksdb_writebatch_count(wb) >= 1024) { + char *err = NULL; + rocksdb_write(pTsdb->rCache.db, pTsdb->rCache.writeoptions, wb, &err); + if (NULL != err) { + tsdbError("vgId:%d, %s failed at line %d since %s", TD_VID(pTsdb->pVnode), __func__, __LINE__, err); + rocksdb_free(err); + } + + rocksdb_writebatch_clear(wb); + } +} + int32_t tsdbCacheCommit(STsdb *pTsdb) { int32_t code = 0; char *err = NULL; + rocksMayWrite(pTsdb, true); rocksdb_flush(pTsdb->rCache.db, pTsdb->rCache.flushoptions, &err); if (NULL != err) { tsdbError("vgId:%d, %s failed at line %d since %s", TD_VID(pTsdb->pVnode), __func__, __LINE__, err); @@ -281,22 +297,6 @@ static SLastCol *tsdbCacheLookup(STsdb *pTsdb, tb_uid_t uid, int16_t cid, int8_t return pLastCol; } -static void rocksMayWrite(STsdb *pTsdb) { - rocksdb_writebatch_t *wb = pTsdb->rCache.writebatch; - - int count = rocksdb_writebatch_count(wb); - if (count >= 1024) { - char *err = NULL; - rocksdb_write(pTsdb->rCache.db, pTsdb->rCache.writeoptions, wb, &err); - if (NULL != err) { - tsdbError("vgId:%d, %s failed at line %d since %s", TD_VID(pTsdb->pVnode), __func__, __LINE__, err); - rocksdb_free(err); - } - - rocksdb_writebatch_clear(wb); - } -} - int32_t tsdbCacheUpdate(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSDBROW *pRow) { int32_t code = 0; @@ -398,7 +398,7 @@ int32_t tsdbCacheUpdate(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSDBROW *pRow taosMemoryFree(values_list); taosMemoryFree(values_list_sizes); - rocksMayWrite(pTsdb); + rocksMayWrite(pTsdb, false); taosThreadMutexUnlock(&pTsdb->rCache.rMutex); _exit: @@ -507,7 +507,7 @@ int32_t tsdbCacheGetSlow(STsdb *pTsdb, tb_uid_t uid, SArray *pLastArray, SCacheR } if (wb) { - rocksMayWrite(pTsdb); + rocksMayWrite(pTsdb, false); } taosThreadMutexUnlock(&pTsdb->rCache.rMutex); @@ -575,7 +575,7 @@ static SLastCol *tsdbCacheLoadCol(STsdb *pTsdb, SCacheRowsReader *pr, int16_t sl } if (wb) { - rocksMayWrite(pTsdb); + rocksMayWrite(pTsdb, false); } taosThreadMutexUnlock(&pTsdb->rCache.rMutex); @@ -672,7 +672,7 @@ static int32_t tsdbCacheLoadFromRaw(STsdb *pTsdb, tb_uid_t uid, SArray *pLastArr } if (wb) { - rocksMayWrite(pTsdb); + rocksMayWrite(pTsdb, false); } taosArrayDestroy(pTmpColArray); @@ -734,10 +734,10 @@ static int32_t tsdbCacheLoadFromRocks(STsdb *pTsdb, tb_uid_t uid, SArray *pLastA code = -1; } - SLastCol lastCol = *pLastCol; - reallocVarData(&lastCol.colVal); + // SLastCol lastCol = *pLastCol; + // reallocVarData(&lastCol.colVal); - taosArraySet(pLastArray, idxKey->idx, &lastCol); + taosArraySet(pLastArray, idxKey->idx, pLastCol); taosArrayRemove(remainCols, j); taosMemoryFree(values_list[i]); @@ -941,7 +941,7 @@ int32_t tsdbCacheDel(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSKEY sKey, TSKE taosMemoryFree(values_list); taosMemoryFree(values_list_sizes); - rocksMayWrite(pTsdb); + rocksMayWrite(pTsdb, false); taosThreadMutexUnlock(&pTsdb->rCache.rMutex); _exit: From f428a68ec1c585f22d02784dea9c4ef437f98007 Mon Sep 17 00:00:00 2001 From: Minglei Jin Date: Wed, 17 May 2023 16:08:48 +0800 Subject: [PATCH 28/59] cache/update: update lru when deleting or updating --- source/dnode/vnode/src/tsdb/tsdbCache.c | 113 ++++++++++++++++-------- 1 file changed, 75 insertions(+), 38 deletions(-) diff --git a/source/dnode/vnode/src/tsdb/tsdbCache.c b/source/dnode/vnode/src/tsdb/tsdbCache.c index 8f0d541d49..2eddc29c0e 100644 --- a/source/dnode/vnode/src/tsdb/tsdbCache.c +++ b/source/dnode/vnode/src/tsdb/tsdbCache.c @@ -297,6 +297,31 @@ static SLastCol *tsdbCacheLookup(STsdb *pTsdb, tb_uid_t uid, int16_t cid, int8_t return pLastCol; } +static void reallocVarData(SColVal *pColVal) { + if (IS_VAR_DATA_TYPE(pColVal->type)) { + uint8_t *pVal = pColVal->value.pData; + pColVal->value.pData = taosMemoryMalloc(pColVal->value.nData); + if (pColVal->value.nData) { + memcpy(pColVal->value.pData, pVal, pColVal->value.nData); + } + } +} + +static void tsdbCacheDeleter(const void *key, size_t keyLen, void *value) { + SLastCol *pLastCol = (SLastCol *)value; + + // TODO: add dirty flag to SLastCol + if (pLastCol->dirty) { + // TODO: queue into dirty list, free it after save to backstore + } else { + if (IS_VAR_DATA_TYPE(pLastCol->colVal.type) && pLastCol->colVal.value.nData > 0) { + taosMemoryFree(pLastCol->colVal.value.pData); + } + + taosMemoryFree(value); + } +} + int32_t tsdbCacheUpdate(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSDBROW *pRow) { int32_t code = 0; @@ -373,6 +398,24 @@ int32_t tsdbCacheUpdate(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSDBROW *pRow SLastKey key = (SLastKey){.ltype = 0, .uid = uid, .cid = pColVal->cid}; size_t klen = ROCKS_KEY_LEN; rocksdb_writebatch_put(wb, (char *)&key, klen, value, vlen); + + pLastCol = (SLastCol *)value; + SLastCol *pTmpLastCol = taosMemoryCalloc(1, sizeof(SLastCol)); + *pTmpLastCol = *pLastCol; + pLastCol = pTmpLastCol; + + reallocVarData(&pLastCol->colVal); + size_t charge = sizeof(*pLastCol); + if (IS_VAR_DATA_TYPE(pLastCol->colVal.type)) { + charge += pLastCol->colVal.value.nData; + } + + LRUStatus status = taosLRUCacheInsert(pTsdb->lruCache, &key, ROCKS_KEY_LEN, pLastCol, charge, tsdbCacheDeleter, + NULL, TAOS_LRU_PRIORITY_LOW); + if (status != TAOS_LRU_STATUS_OK) { + code = -1; + } + taosMemoryFree(value); } @@ -384,9 +427,26 @@ int32_t tsdbCacheUpdate(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSDBROW *pRow size_t vlen = 0; tsdbCacheSerialize(&(SLastCol){.ts = keyTs, .colVal = *pColVal}, &value, &vlen); SLastKey key = (SLastKey){.ltype = 1, .uid = uid, .cid = pColVal->cid}; - size_t klen = ROCKS_KEY_LEN; - rocksdb_writebatch_put(wb, (char *)&key, klen, value, vlen); + rocksdb_writebatch_put(wb, (char *)&key, ROCKS_KEY_LEN, value, vlen); + + pLastCol = (SLastCol *)value; + SLastCol *pTmpLastCol = taosMemoryCalloc(1, sizeof(SLastCol)); + *pTmpLastCol = *pLastCol; + pLastCol = pTmpLastCol; + + reallocVarData(&pLastCol->colVal); + size_t charge = sizeof(*pLastCol); + if (IS_VAR_DATA_TYPE(pLastCol->colVal.type)) { + charge += pLastCol->colVal.value.nData; + } + + LRUStatus status = taosLRUCacheInsert(pTsdb->lruCache, &key, ROCKS_KEY_LEN, pLastCol, charge, + tsdbCacheDeleter, NULL, TAOS_LRU_PRIORITY_LOW); + if (status != TAOS_LRU_STATUS_OK) { + code = -1; + } + taosMemoryFree(value); } } @@ -407,16 +467,6 @@ _exit: return code; } -static void reallocVarData(SColVal *pColVal) { - if (IS_VAR_DATA_TYPE(pColVal->type)) { - uint8_t *pVal = pColVal->value.pData; - pColVal->value.pData = taosMemoryMalloc(pColVal->value.nData); - if (pColVal->value.nData) { - memcpy(pColVal->value.pData, pVal, pColVal->value.nData); - } - } -} - static int32_t mergeLastCid(tb_uid_t uid, STsdb *pTsdb, SArray **ppLastArray, SCacheRowsReader *pr, int16_t *aCols, int nCols, int16_t *slotIds); @@ -584,21 +634,6 @@ static SLastCol *tsdbCacheLoadCol(STsdb *pTsdb, SCacheRowsReader *pr, int16_t sl return pLastCol; } -static void tsdbCacheDeleter(const void *key, size_t keyLen, void *value) { - SLastCol *pLastCol = (SLastCol *)value; - - // TODO: add dirty flag to SLastCol - if (pLastCol->dirty) { - // TODO: queue into dirty list, free it after save to backstore - } else { - if (IS_VAR_DATA_TYPE(pLastCol->colVal.type) && pLastCol->colVal.value.nData > 0) { - taosMemoryFree(pLastCol->colVal.value.pData); - } - - taosMemoryFree(value); - } -} - typedef struct { int idx; SLastKey key; @@ -734,9 +769,6 @@ static int32_t tsdbCacheLoadFromRocks(STsdb *pTsdb, tb_uid_t uid, SArray *pLastA code = -1; } - // SLastCol lastCol = *pLastCol; - // reallocVarData(&lastCol.colVal); - taosArraySet(pLastArray, idxKey->idx, pLastCol); taosArrayRemove(remainCols, j); @@ -875,16 +907,19 @@ int32_t tsdbCacheGet(STsdb *pTsdb, tb_uid_t uid, SArray *pLastArray, SCacheRowsR int32_t tsdbCacheDel(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSKEY sKey, TSKEY eKey) { int32_t code = 0; - // 1, fetch schema - STSchema *pTSchema = NULL; - int32_t sver = -1; - code = metaGetTbTSchemaEx(pTsdb->pVnode->pMeta, suid, uid, sver, &pTSchema); - if (code != TSDB_CODE_SUCCESS) { - terrno = code; - return -1; + // fetch schema + STSchema *pTSchema = pTsdb->rCache.pTSchema; + if (!pTSchema) { + code = metaGetTbTSchemaEx(pTsdb->pVnode->pMeta, suid, uid, -1, &pTSchema); + if (code != TSDB_CODE_SUCCESS) { + terrno = code; + return -1; + } + + pTsdb->rCache.pTSchema = pTSchema; } - // 3, build keys & multi get from rocks + // build keys & multi get from rocks int num_keys = pTSchema->numOfCols; char **keys_list = taosMemoryCalloc(num_keys * 2, sizeof(char *)); size_t *keys_list_sizes = taosMemoryCalloc(num_keys * 2, sizeof(size_t)); @@ -925,6 +960,7 @@ int32_t tsdbCacheDel(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSKEY sKey, TSKE size_t klen = sizeof(*key); rocksdb_writebatch_delete(wb, (char *)key, klen); + taosLRUCacheErase(pTsdb->lruCache, key, klen); } pLastCol = tsdbCacheDeserialize(values_list[i + num_keys]); @@ -933,6 +969,7 @@ int32_t tsdbCacheDel(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSKEY sKey, TSKE size_t klen = sizeof(*key); rocksdb_writebatch_delete(wb, (char *)key, klen); + taosLRUCacheErase(pTsdb->lruCache, key, klen); } rocksdb_free(values_list[i]); From 0fdeb5fbae56e06835771c6870ef7bc2d35d92a2 Mon Sep 17 00:00:00 2001 From: dmchen Date: Wed, 17 May 2023 17:28:20 +0800 Subject: [PATCH 29/59] restore learner stuck --- source/libs/sync/src/syncMain.c | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/source/libs/sync/src/syncMain.c b/source/libs/sync/src/syncMain.c index 499df4a98b..f64c2a9560 100644 --- a/source/libs/sync/src/syncMain.c +++ b/source/libs/sync/src/syncMain.c @@ -2413,6 +2413,11 @@ int32_t syncNodeOnHeartbeat(SSyncNode* ths, const SRpcMsg* pRpcMsg) { "vgId:%d, heartbeat msg from dnode:%d, cluster:%d, Msgterm:%" PRId64 " currentTerm:%" PRId64, ths->vgId, DID(&(pMsg->srcId)), CID(&(pMsg->srcId)), pMsg->term, currentTerm); + if(pMsg->term > currentTerm && ths->state == TAOS_SYNC_STATE_LEARNER){ + raftStoreSetTerm(ths, pMsg->term); + currentTerm = pMsg->term; + } + if (pMsg->term == currentTerm && ths->state != TAOS_SYNC_STATE_LEADER) { syncIndexMgrSetRecvTime(ths->pNextIndex, &(pMsg->srcId), tsMs); resetElect = true; From c7911c664067c8bb2ce68f2ee5e9fe1390b192dd Mon Sep 17 00:00:00 2001 From: dmchen Date: Wed, 17 May 2023 18:33:35 +0800 Subject: [PATCH 30/59] fix ci --- source/dnode/mnode/impl/src/mndVgroup.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/dnode/mnode/impl/src/mndVgroup.c b/source/dnode/mnode/impl/src/mndVgroup.c index 0e93555e80..7691f0f8f4 100644 --- a/source/dnode/mnode/impl/src/mndVgroup.c +++ b/source/dnode/mnode/impl/src/mndVgroup.c @@ -1456,7 +1456,7 @@ int32_t mndSetMoveVgroupInfoToTrans(SMnode *pMnode, STrans *pTrans, SDbObj *pDb, mInfo("vgId:%d, will add 1 vnode and force remove 1 vnode", pVgroup->vgId); if (mndAddVnodeToVgroup(pMnode, pTrans, &newVg, pArray) != 0) return -1; newVg.replica--; - SVnodeGid del = newVg.vnodeGid[vnIndex]; + //SVnodeGid del = newVg.vnodeGid[vnIndex]; newVg.vnodeGid[vnIndex] = newVg.vnodeGid[newVg.replica]; memset(&newVg.vnodeGid[newVg.replica], 0, sizeof(SVnodeGid)); { From e9ce88aed04c50bc644e619fc4c5c5c78df702f4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Chappyguoxy=E2=80=9D?= <“happy_guoxy@163.com”> Date: Wed, 17 May 2023 18:39:20 +0800 Subject: [PATCH 31/59] test: rowlength64k --- tests/system-test/1-insert/rowlength64k_2.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/system-test/1-insert/rowlength64k_2.py b/tests/system-test/1-insert/rowlength64k_2.py index becae4daa4..81a57cd33d 100755 --- a/tests/system-test/1-insert/rowlength64k_2.py +++ b/tests/system-test/1-insert/rowlength64k_2.py @@ -23,10 +23,11 @@ class TDTestCase(TDTestCase): # self.run_1() # self.run_2() # self.run_3() - # self.run_4() + self.run_4() self.run_5() - self.run_6() - self.run_7() + # self.run_5() + # self.run_6() + # self.run_7() endTime_all = time.time() print("total time %ds" % (endTime_all - startTime_all)) From 1cec381e14a92fd9dd0655d250562f04d4001c07 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Chappyguoxy=E2=80=9D?= <“happy_guoxy@163.com”> Date: Wed, 17 May 2023 18:39:24 +0800 Subject: [PATCH 32/59] test: rowlength64k --- tests/system-test/1-insert/rowlength64k_3.py | 42 ++++++++++++++++++++ 1 file changed, 42 insertions(+) create mode 100755 tests/system-test/1-insert/rowlength64k_3.py diff --git a/tests/system-test/1-insert/rowlength64k_3.py b/tests/system-test/1-insert/rowlength64k_3.py new file mode 100755 index 0000000000..ebcc207de6 --- /dev/null +++ b/tests/system-test/1-insert/rowlength64k_3.py @@ -0,0 +1,42 @@ +################################################################### +# Copyright (c) 2016 by TAOS Technologies, Inc. +# All rights reserved. +# +# This file is proprietary and confidential to TAOS Technologies. +# No part of this file may be reproduced, stored, transmitted, +# disclosed or used in any form or by any means other than as +# expressly provided by the written permission from Jianhui Tao +# +################################################################### + +# -*- coding: utf-8 -*- +from util.cases import tdCases +from .rowlength64k import * + +class TDTestCase(TDTestCase): + + + def run(self): + tdSql.prepare() + + startTime_all = time.time() + # self.run_1() + # self.run_2() + # self.run_3() + # self.run_4() + self.run_6() + self.run_7() + + endTime_all = time.time() + print("total time %ds" % (endTime_all - startTime_all)) + + + + + def stop(self): + tdSql.close() + tdLog.success("%s successfully executed" % __file__) + + +tdCases.addWindows(__file__, TDTestCase()) +tdCases.addLinux(__file__, TDTestCase()) From 2e5a092ab6e682bc83c9888b7122b0a268bd8eb9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Chappyguoxy=E2=80=9D?= <“happy_guoxy@163.com”> Date: Wed, 17 May 2023 18:39:35 +0800 Subject: [PATCH 33/59] test: rowlength64k --- tests/system-test/1-insert/rowlength64k_1.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/system-test/1-insert/rowlength64k_1.py b/tests/system-test/1-insert/rowlength64k_1.py index fe676cd71a..b641cfd58a 100755 --- a/tests/system-test/1-insert/rowlength64k_1.py +++ b/tests/system-test/1-insert/rowlength64k_1.py @@ -23,7 +23,7 @@ class TDTestCase(TDTestCase): #self.run_1() # self.run_2() self.run_3() - self.run_4() + #self.run_4() endTime_all = time.time() print("total time %ds" % (endTime_all - startTime_all)) From 45647f26044e53a6745494e5c096af838b22cca0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Chappyguoxy=E2=80=9D?= <“happy_guoxy@163.com”> Date: Wed, 17 May 2023 18:39:45 +0800 Subject: [PATCH 34/59] test: rowlength64k --- tests/system-test/1-insert/rowlength64k.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/system-test/1-insert/rowlength64k.py b/tests/system-test/1-insert/rowlength64k.py index 3e6b5def7c..6e36d66e77 100755 --- a/tests/system-test/1-insert/rowlength64k.py +++ b/tests/system-test/1-insert/rowlength64k.py @@ -1256,9 +1256,9 @@ class TDTestCase: sql = "alter table db.stable_4 modify column col4091 binary(102); " tdSql.error(sql) sql = "alter table db.stable_4 modify tag loc nchar(20); " - tdSql.query("select * from db.table_40") + tdSql.query("select * from db.table_70") tdSql.checkCols(4092) - tdSql.query("describe db.table_40") + tdSql.query("describe db.table_70") tdSql.checkRows(4096) self.ins_query() From b625cfc3ec2aef8a5493a1ccb1810292de8d2d91 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Chappyguoxy=E2=80=9D?= <“happy_guoxy@163.com”> Date: Wed, 17 May 2023 18:40:02 +0800 Subject: [PATCH 35/59] test: rowlength64k --- tests/parallel_test/cases.task | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/parallel_test/cases.task b/tests/parallel_test/cases.task index 8e786f7bc5..130299ebce 100644 --- a/tests/parallel_test/cases.task +++ b/tests/parallel_test/cases.task @@ -598,6 +598,11 @@ ,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k_2.py -Q 2 ,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k_2.py -Q 3 ,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k_2.py -Q 4 +,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k_3.py +,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k_3.py -R +,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k_3.py -Q 2 +,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k_3.py -Q 3 +,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k_3.py -Q 4 ,,y,system-test,./pytest.sh python3 ./test.py -f 0-others/show.py ,,y,system-test,./pytest.sh python3 ./test.py -f 0-others/information_schema.py ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/abs.py From 765cc72618b13fee47baedaa9eb2f5d263785c23 Mon Sep 17 00:00:00 2001 From: Minglei Jin Date: Wed, 17 May 2023 18:56:03 +0800 Subject: [PATCH 36/59] cache/memtable: update cache when flag is on --- source/dnode/vnode/src/tsdb/tsdbCache.c | 2 +- source/dnode/vnode/src/tsdb/tsdbMemTable.c | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/source/dnode/vnode/src/tsdb/tsdbCache.c b/source/dnode/vnode/src/tsdb/tsdbCache.c index 2eddc29c0e..fe7e808db3 100644 --- a/source/dnode/vnode/src/tsdb/tsdbCache.c +++ b/source/dnode/vnode/src/tsdb/tsdbCache.c @@ -314,7 +314,7 @@ static void tsdbCacheDeleter(const void *key, size_t keyLen, void *value) { if (pLastCol->dirty) { // TODO: queue into dirty list, free it after save to backstore } else { - if (IS_VAR_DATA_TYPE(pLastCol->colVal.type) && pLastCol->colVal.value.nData > 0) { + if (IS_VAR_DATA_TYPE(pLastCol->colVal.type) /* && pLastCol->colVal.value.nData > 0*/) { taosMemoryFree(pLastCol->colVal.value.pData); } diff --git a/source/dnode/vnode/src/tsdb/tsdbMemTable.c b/source/dnode/vnode/src/tsdb/tsdbMemTable.c index 803a4f6c2d..80967a906f 100644 --- a/source/dnode/vnode/src/tsdb/tsdbMemTable.c +++ b/source/dnode/vnode/src/tsdb/tsdbMemTable.c @@ -737,7 +737,9 @@ static int32_t tsdbInsertRowDataToTable(SMemTable *pMemTable, STbData *pTbData, if (key.ts >= pTbData->maxKey) { pTbData->maxKey = key.ts; } - tsdbCacheUpdate(pMemTable->pTsdb, pTbData->suid, pTbData->uid, &lRow); + if (!TSDB_CACHE_NO(pMemTable->pTsdb->pVnode->config)) { + tsdbCacheUpdate(pMemTable->pTsdb, pTbData->suid, pTbData->uid, &lRow); + } // SMemTable pMemTable->minKey = TMIN(pMemTable->minKey, pTbData->minKey); From e494c9983975ce3d9ee53697d878553c5f72f8ca Mon Sep 17 00:00:00 2001 From: dapan1121 Date: Wed, 17 May 2023 19:35:57 +0800 Subject: [PATCH 37/59] fix: fill window end issue --- source/libs/executor/src/tfill.c | 1 + 1 file changed, 1 insertion(+) diff --git a/source/libs/executor/src/tfill.c b/source/libs/executor/src/tfill.c index fc4e82b57f..6b49d235f4 100644 --- a/source/libs/executor/src/tfill.c +++ b/source/libs/executor/src/tfill.c @@ -520,6 +520,7 @@ void taosFillSetStartInfo(SFillInfo* pFillInfo, int32_t numOfRows, TSKEY endKey) pFillInfo->end = endKey; if (!FILL_IS_ASC_FILL(pFillInfo)) { pFillInfo->end = taosTimeTruncate(endKey, &pFillInfo->interval, pFillInfo->interval.precision); + pFillInfo->end = taosTimeAdd(pFillInfo->end, pFillInfo->interval.interval, pFillInfo->interval.intervalUnit,pFillInfo->interval.precision); } pFillInfo->index = 0; From 32c6ef4b6ecefc7ad0e6670e825698cf21170132 Mon Sep 17 00:00:00 2001 From: dmchen Date: Wed, 17 May 2023 23:09:16 +0800 Subject: [PATCH 38/59] fix ci --- source/dnode/mnode/impl/src/mndStb.c | 6 ------ tests/pytest/util/dnodes.py | 12 ------------ tests/system-test/forcedrop | 1 + 3 files changed, 1 insertion(+), 18 deletions(-) create mode 120000 tests/system-test/forcedrop diff --git a/source/dnode/mnode/impl/src/mndStb.c b/source/dnode/mnode/impl/src/mndStb.c index ef72f66a7a..531521ae9b 100644 --- a/source/dnode/mnode/impl/src/mndStb.c +++ b/source/dnode/mnode/impl/src/mndStb.c @@ -689,13 +689,10 @@ static int32_t mndSetCreateStbRedoActions(SMnode *pMnode, STrans *pTrans, SDbObj int32_t mndSetForceDropCreateStbRedoActions(SMnode *pMnode, STrans *pTrans, SVgObj *pVgroup, SStbObj *pStb) { SSdb *pSdb = pMnode->pSdb; - void *pIter = NULL; int32_t contLen; void *pReq = mndBuildVCreateStbReq(pMnode, pVgroup, pStb, &contLen, NULL, 0); if (pReq == NULL) { - sdbCancelFetch(pSdb, pIter); - sdbRelease(pSdb, pVgroup); return -1; } @@ -709,11 +706,8 @@ int32_t mndSetForceDropCreateStbRedoActions(SMnode *pMnode, STrans *pTrans, SVgO action.retryCode = TSDB_CODE_TDB_STB_NOT_EXIST; if (mndTransAppendRedoAction(pTrans, &action) != 0) { taosMemoryFree(pReq); - sdbCancelFetch(pSdb, pIter); - sdbRelease(pSdb, pVgroup); return -1; } - sdbRelease(pSdb, pVgroup); return 0; } diff --git a/tests/pytest/util/dnodes.py b/tests/pytest/util/dnodes.py index 80ab5baa9a..058c652990 100644 --- a/tests/pytest/util/dnodes.py +++ b/tests/pytest/util/dnodes.py @@ -130,18 +130,6 @@ class TDDnode: "locale": "en_US.UTF-8", "charset": "UTF-8", "asyncLog": "0", - "mDebugFlag": "143", - "dDebugFlag": "143", - "vDebugFlag": "143", - "tqDebugFlag": "143", - "cDebugFlag": "143", - "jniDebugFlag": "143", - "qDebugFlag": "143", - "rpcDebugFlag": "143", - "tmrDebugFlag": "131", - "uDebugFlag": "143", - "sDebugFlag": "143", - "wDebugFlag": "143", "numOfLogLines": "100000000", "statusInterval": "1", "enableQueryHb": "1", diff --git a/tests/system-test/forcedrop b/tests/system-test/forcedrop new file mode 120000 index 0000000000..d2a16bf505 --- /dev/null +++ b/tests/system-test/forcedrop @@ -0,0 +1 @@ +/home/ubuntu/Documents/github/cadem/emptydebug/forcedrop/ \ No newline at end of file From 56a05dc0567dbbcad264221544612ddaf99f2d70 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Chappyguoxy=E2=80=9D?= <“happy_guoxy@163.com”> Date: Thu, 18 May 2023 10:22:08 +0800 Subject: [PATCH 39/59] test: rowlength64k --- tests/parallel_test/cases.task | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/parallel_test/cases.task b/tests/parallel_test/cases.task index 709e789977..2c970376ff 100644 --- a/tests/parallel_test/cases.task +++ b/tests/parallel_test/cases.task @@ -604,6 +604,11 @@ ,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k_3.py -Q 2 ,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k_3.py -Q 3 ,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k_3.py -Q 4 +,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k_4.py +,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k_4.py -R +,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k_4.py -Q 2 +,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k_4.py -Q 3 +,,y,system-test,./pytest.sh python3 ./test.py -f 1-insert/rowlength64k_4.py -Q 4 ,,y,system-test,./pytest.sh python3 ./test.py -f 0-others/show.py ,,y,system-test,./pytest.sh python3 ./test.py -f 0-others/information_schema.py ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/abs.py From cd362505a0e2b5c7f552a2a4029f89e5ef017138 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Chappyguoxy=E2=80=9D?= <“happy_guoxy@163.com”> Date: Thu, 18 May 2023 10:22:20 +0800 Subject: [PATCH 40/59] test: rowlength64k --- tests/system-test/1-insert/rowlength64k_2.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/system-test/1-insert/rowlength64k_2.py b/tests/system-test/1-insert/rowlength64k_2.py index 81a57cd33d..897419c242 100755 --- a/tests/system-test/1-insert/rowlength64k_2.py +++ b/tests/system-test/1-insert/rowlength64k_2.py @@ -24,7 +24,7 @@ class TDTestCase(TDTestCase): # self.run_2() # self.run_3() self.run_4() - self.run_5() + #self.run_5() # self.run_5() # self.run_6() # self.run_7() From f680cab3145240a80a16ed5512972a2d07b9e235 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Chappyguoxy=E2=80=9D?= <“happy_guoxy@163.com”> Date: Thu, 18 May 2023 10:22:35 +0800 Subject: [PATCH 41/59] test: rowlength64k --- tests/system-test/1-insert/rowlength64k_4.py | 44 ++++++++++++++++++++ 1 file changed, 44 insertions(+) create mode 100755 tests/system-test/1-insert/rowlength64k_4.py diff --git a/tests/system-test/1-insert/rowlength64k_4.py b/tests/system-test/1-insert/rowlength64k_4.py new file mode 100755 index 0000000000..a5abe3e268 --- /dev/null +++ b/tests/system-test/1-insert/rowlength64k_4.py @@ -0,0 +1,44 @@ +################################################################### +# Copyright (c) 2016 by TAOS Technologies, Inc. +# All rights reserved. +# +# This file is proprietary and confidential to TAOS Technologies. +# No part of this file may be reproduced, stored, transmitted, +# disclosed or used in any form or by any means other than as +# expressly provided by the written permission from Jianhui Tao +# +################################################################### + +# -*- coding: utf-8 -*- +from util.cases import tdCases +from .rowlength64k import * + +class TDTestCase(TDTestCase): + + + def run(self): + tdSql.prepare() + + startTime_all = time.time() + # self.run_1() + # self.run_2() + # self.run_3() + #self.run_4() + self.run_5() + # self.run_5() + # self.run_6() + # self.run_7() + + endTime_all = time.time() + print("total time %ds" % (endTime_all - startTime_all)) + + + + + def stop(self): + tdSql.close() + tdLog.success("%s successfully executed" % __file__) + + +tdCases.addWindows(__file__, TDTestCase()) +tdCases.addLinux(__file__, TDTestCase()) From 81f821a6fcb0f8a6167e0c12c07eaa993253d4d1 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 18 May 2023 10:51:43 +0800 Subject: [PATCH 42/59] refactor: do some internal refactor. --- include/common/tcommon.h | 5 - source/dnode/vnode/inc/vnode.h | 2 +- source/dnode/vnode/src/inc/vnodeInt.h | 2 - source/dnode/vnode/src/tq/tq.c | 153 -------------------------- source/dnode/vnode/src/tq/tqPush.c | 38 ++++++- source/dnode/vnode/src/tq/tqRead.c | 22 ++-- source/dnode/vnode/src/tq/tqRestore.c | 73 ++++++------ 7 files changed, 89 insertions(+), 206 deletions(-) diff --git a/include/common/tcommon.h b/include/common/tcommon.h index 0544247da2..4faea6e5e3 100644 --- a/include/common/tcommon.h +++ b/include/common/tcommon.h @@ -208,11 +208,6 @@ typedef struct SSDataBlock { SDataBlockInfo info; } SSDataBlock; -enum { - FETCH_TYPE__DATA = 0, - FETCH_TYPE__NONE, -}; - typedef struct SVarColAttr { int32_t* offset; // start position for each entry in the list uint32_t length; // used buffer size that contain the valid data diff --git a/source/dnode/vnode/inc/vnode.h b/source/dnode/vnode/inc/vnode.h index 549173b3f7..a2fe0bef21 100644 --- a/source/dnode/vnode/inc/vnode.h +++ b/source/dnode/vnode/inc/vnode.h @@ -267,7 +267,7 @@ int32_t tqReaderAddTbUidList(STqReader *pReader, const SArray *pTableUidList); int32_t tqReaderRemoveTbUidList(STqReader *pReader, const SArray *tbUidList); int32_t tqSeekVer(STqReader *pReader, int64_t ver, const char *id); -int32_t tqNextBlockInWal(STqReader* pReader); +bool tqNextBlockInWal(STqReader* pReader, const char* idstr); bool tqNextBlockImpl(STqReader *pReader, const char* idstr); int32_t extractMsgFromWal(SWalReader* pReader, void** pItem, const char* id); diff --git a/source/dnode/vnode/src/inc/vnodeInt.h b/source/dnode/vnode/src/inc/vnodeInt.h index 861235a5c3..230e8b8f88 100644 --- a/source/dnode/vnode/src/inc/vnodeInt.h +++ b/source/dnode/vnode/src/inc/vnodeInt.h @@ -219,8 +219,6 @@ int32_t tqProcessTaskPauseReq(STQ* pTq, int64_t version, char* msg, int32_t msgL int32_t tqProcessTaskResumeReq(STQ* pTq, int64_t version, char* msg, int32_t msgLen); int32_t tqProcessStreamTaskCheckReq(STQ* pTq, SRpcMsg* pMsg); int32_t tqProcessStreamTaskCheckRsp(STQ* pTq, int64_t version, char* msg, int32_t msgLen); -int32_t tqProcessSubmitReqForSubscribe(STQ* pTq); -int32_t tqProcessDeleteDataReq(STQ* pTq, void* pReq, int32_t len, int64_t ver); int32_t tqProcessTaskRunReq(STQ* pTq, SRpcMsg* pMsg); int32_t tqProcessTaskDispatchReq(STQ* pTq, SRpcMsg* pMsg, bool exec); int32_t tqProcessTaskDispatchRsp(STQ* pTq, SRpcMsg* pMsg); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 0179480011..75e25b26b1 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1195,159 +1195,6 @@ int32_t extractDelDataBlock(const void* pData, int32_t len, int64_t ver, SStream return TSDB_CODE_SUCCESS; } -int32_t tqProcessDeleteDataReq(STQ* pTq, void* pReq, int32_t len, int64_t ver) { - bool failed = false; - SDecoder* pCoder = &(SDecoder){0}; - SDeleteRes* pRes = &(SDeleteRes){0}; - - pRes->uidList = taosArrayInit(0, sizeof(tb_uid_t)); - if (pRes->uidList == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - failed = true; - } - - tDecoderInit(pCoder, pReq, len); - tDecodeDeleteRes(pCoder, pRes); - tDecoderClear(pCoder); - - int32_t sz = taosArrayGetSize(pRes->uidList); - if (sz == 0 || pRes->affectedRows == 0) { - taosArrayDestroy(pRes->uidList); - return 0; - } - - SSDataBlock* pDelBlock = createSpecialDataBlock(STREAM_DELETE_DATA); - blockDataEnsureCapacity(pDelBlock, sz); - pDelBlock->info.rows = sz; - pDelBlock->info.version = ver; - - for (int32_t i = 0; i < sz; i++) { - // start key column - SColumnInfoData* pStartCol = taosArrayGet(pDelBlock->pDataBlock, START_TS_COLUMN_INDEX); - colDataSetVal(pStartCol, i, (const char*)&pRes->skey, false); // end key column - SColumnInfoData* pEndCol = taosArrayGet(pDelBlock->pDataBlock, END_TS_COLUMN_INDEX); - colDataSetVal(pEndCol, i, (const char*)&pRes->ekey, false); - // uid column - SColumnInfoData* pUidCol = taosArrayGet(pDelBlock->pDataBlock, UID_COLUMN_INDEX); - int64_t* pUid = taosArrayGet(pRes->uidList, i); - colDataSetVal(pUidCol, i, (const char*)pUid, false); - - colDataSetNULL(taosArrayGet(pDelBlock->pDataBlock, GROUPID_COLUMN_INDEX), i); - colDataSetNULL(taosArrayGet(pDelBlock->pDataBlock, CALCULATE_START_TS_COLUMN_INDEX), i); - colDataSetNULL(taosArrayGet(pDelBlock->pDataBlock, CALCULATE_END_TS_COLUMN_INDEX), i); - } - - taosArrayDestroy(pRes->uidList); - - int32_t* pRef = taosMemoryMalloc(sizeof(int32_t)); - *pRef = 1; - - taosWLockLatch(&pTq->pStreamMeta->lock); - - void* pIter = NULL; - while (1) { - pIter = taosHashIterate(pTq->pStreamMeta->pTasks, pIter); - if (pIter == NULL) { - break; - } - - SStreamTask* pTask = *(SStreamTask**)pIter; - if (pTask->taskLevel != TASK_LEVEL__SOURCE) { - continue; - } - - qDebug("s-task:%s delete req enqueue, ver: %" PRId64, pTask->id.idStr, ver); - - if (!failed) { - SStreamRefDataBlock* pRefBlock = taosAllocateQitem(sizeof(SStreamRefDataBlock), DEF_QITEM, 0); - pRefBlock->type = STREAM_INPUT__REF_DATA_BLOCK; - pRefBlock->pBlock = pDelBlock; - - if (tAppendDataToInputQueue(pTask, (SStreamQueueItem*)pRefBlock) < 0) { - atomic_sub_fetch_32(pRef, 1); - taosFreeQitem(pRefBlock); - continue; - } - - if (streamSchedExec(pTask) < 0) { - qError("s-task:%s stream task launch failed", pTask->id.idStr); - continue; - } - - } else { - streamTaskInputFail(pTask); - } - } - - taosWUnLockLatch(&pTq->pStreamMeta->lock); - - int32_t ref = atomic_sub_fetch_32(pRef, 1); - if (ref == 0) { - blockDataDestroy(pDelBlock); - taosMemoryFree(pRef); - } - -#if 0 - SStreamDataBlock* pStreamBlock = taosAllocateQitem(sizeof(SStreamDataBlock), DEF_QITEM, 0); - pStreamBlock->type = STREAM_INPUT__DATA_BLOCK; - pStreamBlock->blocks = taosArrayInit(0, sizeof(SSDataBlock)); - SSDataBlock block = {0}; - assignOneDataBlock(&block, pDelBlock); - block.info.type = STREAM_DELETE_DATA; - taosArrayPush(pStreamBlock->blocks, &block); - - if (!failed) { - if (tAppendDataToInputQueue(pTask, (SStreamQueueItem*)pStreamBlock) < 0) { - qError("stream task input del failed, task id %d", pTask->id.taskId); - continue; - } - - if (streamSchedExec(pTask) < 0) { - qError("stream task launch failed, task id %d", pTask->id.taskId); - continue; - } - } else { - streamTaskInputFail(pTask); - } - } - blockDataDestroy(pDelBlock); -#endif - return 0; -} - -int32_t tqProcessSubmitReqForSubscribe(STQ* pTq) { - int32_t vgId = TD_VID(pTq->pVnode); - - taosWLockLatch(&pTq->lock); - - if (taosHashGetSize(pTq->pPushMgr) > 0) { - void* pIter = taosHashIterate(pTq->pPushMgr, NULL); - - while (pIter) { - STqHandle* pHandle = *(STqHandle**)pIter; - tqDebug("vgId:%d start set submit for pHandle:%p, consumer:0x%" PRIx64, vgId, pHandle, pHandle->consumerId); - - if (ASSERT(pHandle->msg != NULL)) { - tqError("pHandle->msg should not be null"); - break; - }else{ - SRpcMsg msg = {.msgType = TDMT_VND_TMQ_CONSUME, .pCont = pHandle->msg->pCont, .contLen = pHandle->msg->contLen, .info = pHandle->msg->info}; - tmsgPutToQueue(&pTq->pVnode->msgCb, QUERY_QUEUE, &msg); - taosMemoryFree(pHandle->msg); - pHandle->msg = NULL; - } - - pIter = taosHashIterate(pTq->pPushMgr, pIter); - } - - taosHashClear(pTq->pPushMgr); - } - - // unlock - taosWUnLockLatch(&pTq->lock); - return 0; -} - int32_t tqProcessTaskRunReq(STQ* pTq, SRpcMsg* pMsg) { SStreamTaskRunReq* pReq = pMsg->pCont; diff --git a/source/dnode/vnode/src/tq/tqPush.c b/source/dnode/vnode/src/tq/tqPush.c index 82a76657fd..7f4fe48b8e 100644 --- a/source/dnode/vnode/src/tq/tqPush.c +++ b/source/dnode/vnode/src/tq/tqPush.c @@ -16,8 +16,40 @@ #include "tq.h" #include "vnd.h" -int32_t tqPushMsg(STQ* pTq, void* msg, int32_t msgLen, tmsg_t msgType, int64_t ver) { +int32_t tqProcessSubmitReqForSubscribe(STQ* pTq) { + int32_t vgId = TD_VID(pTq->pVnode); + taosWLockLatch(&pTq->lock); + + if (taosHashGetSize(pTq->pPushMgr) > 0) { + void* pIter = taosHashIterate(pTq->pPushMgr, NULL); + + while (pIter) { + STqHandle* pHandle = *(STqHandle**)pIter; + tqDebug("vgId:%d start set submit for pHandle:%p, consumer:0x%" PRIx64, vgId, pHandle, pHandle->consumerId); + + if (ASSERT(pHandle->msg != NULL)) { + tqError("pHandle->msg should not be null"); + break; + }else{ + SRpcMsg msg = {.msgType = TDMT_VND_TMQ_CONSUME, .pCont = pHandle->msg->pCont, .contLen = pHandle->msg->contLen, .info = pHandle->msg->info}; + tmsgPutToQueue(&pTq->pVnode->msgCb, QUERY_QUEUE, &msg); + taosMemoryFree(pHandle->msg); + pHandle->msg = NULL; + } + + pIter = taosHashIterate(pTq->pPushMgr, pIter); + } + + taosHashClear(pTq->pPushMgr); + } + + // unlock + taosWUnLockLatch(&pTq->lock); + return 0; +} + +int32_t tqPushMsg(STQ* pTq, void* msg, int32_t msgLen, tmsg_t msgType, int64_t ver) { if (msgType == TDMT_VND_SUBMIT) { tqProcessSubmitReqForSubscribe(pTq); } @@ -37,10 +69,6 @@ int32_t tqPushMsg(STQ* pTq, void* msg, int32_t msgLen, tmsg_t msgType, int64_t v if (msgType == TDMT_VND_SUBMIT || msgType == TDMT_VND_DELETE) { tqStartStreamTasks(pTq); } - - if (msgType == TDMT_VND_DELETE) { -// tqProcessDeleteDataReq(pTq, POINTER_SHIFT(msg, sizeof(SMsgHead)), msgLen - sizeof(SMsgHead), ver); - } } return 0; diff --git a/source/dnode/vnode/src/tq/tqRead.c b/source/dnode/vnode/src/tq/tqRead.c index 6bf09b703b..9e245fb45d 100644 --- a/source/dnode/vnode/src/tq/tqRead.c +++ b/source/dnode/vnode/src/tq/tqRead.c @@ -344,7 +344,7 @@ int32_t extractMsgFromWal(SWalReader* pReader, void** pItem, const char* id) { } // todo ignore the error in wal? -int32_t tqNextBlockInWal(STqReader* pReader) { +bool tqNextBlockInWal(STqReader* pReader, const char* id) { SWalReader* pWalReader = pReader->pWalReader; while (1) { @@ -353,7 +353,7 @@ int32_t tqNextBlockInWal(STqReader* pReader) { // try next message in wal file // todo always retry to avoid read failure caused by wal file deletion if (walNextValidMsg(pWalReader) < 0) { - return FETCH_TYPE__NONE; + return false; } void* pBody = POINTER_SHIFT(pWalReader->pHead->head.body, sizeof(SSubmitReq2Msg)); @@ -379,7 +379,7 @@ int32_t tqNextBlockInWal(STqReader* pReader) { if (tDecodeSubmitReq(&decoder, &pReader->submit) < 0) { tDecoderClear(&decoder); tqError("decode wal file error, msgLen:%d, ver:%" PRId64, bodyLen, ver); - return FETCH_TYPE__NONE; + return false; } tDecoderClear(&decoder); @@ -388,15 +388,15 @@ int32_t tqNextBlockInWal(STqReader* pReader) { size_t numOfBlocks = taosArrayGetSize(pReader->submit.aSubmitTbData); while (pReader->nextBlk < numOfBlocks) { - tqDebug("tq reader next data block %p, %d %" PRId64 " %d", pReader->msg.msgStr, pReader->msg.msgLen, - pReader->msg.ver, pReader->nextBlk); + tqDebug("tq reader next data block %d/%d, len:%d %" PRId64 " %d", pReader->nextBlk, + numOfBlocks, pReader->msg.msgLen, pReader->msg.ver, pReader->nextBlk); SSubmitTbData* pSubmitTbData = taosArrayGet(pReader->submit.aSubmitTbData, pReader->nextBlk); if (pReader->tbIdHash == NULL) { int32_t code = tqRetrieveDataBlock(pReader, NULL); if (code == TSDB_CODE_SUCCESS && pReader->pResBlock->info.rows > 0) { - return FETCH_TYPE__DATA; + return true; } } @@ -406,7 +406,7 @@ int32_t tqNextBlockInWal(STqReader* pReader) { int32_t code = tqRetrieveDataBlock(pReader, NULL); if (code == TSDB_CODE_SUCCESS && pReader->pResBlock->info.rows > 0) { - return FETCH_TYPE__DATA; + return true; } } else { pReader->nextBlk += 1; @@ -414,7 +414,9 @@ int32_t tqNextBlockInWal(STqReader* pReader) { } } + qDebug("stream scan return empty, all %d submit blocks consumed, %s", numOfBlocks, id); tDestroySubmitReq(&pReader->submit, TSDB_MSG_FLG_DECODE); + pReader->msg.msgStr = NULL; } } @@ -729,13 +731,17 @@ int32_t tqRetrieveDataBlock(STqReader* pReader, const char* id) { return 0; } +// todo refactor: int32_t tqRetrieveTaosxBlock(STqReader* pReader, SArray* blocks, SArray* schemas, SSubmitTbData** pSubmitTbDataRet) { tqDebug("tq reader retrieve data block %p, %d", pReader->msg.msgStr, pReader->nextBlk); SSubmitTbData* pSubmitTbData = taosArrayGet(pReader->submit.aSubmitTbData, pReader->nextBlk); pReader->nextBlk++; - if (pSubmitTbDataRet) *pSubmitTbDataRet = pSubmitTbData; + if (pSubmitTbDataRet) { + *pSubmitTbDataRet = pSubmitTbData; + } + int32_t sversion = pSubmitTbData->sver; int64_t suid = pSubmitTbData->suid; int64_t uid = pSubmitTbData->uid; diff --git a/source/dnode/vnode/src/tq/tqRestore.c b/source/dnode/vnode/src/tq/tqRestore.c index 09e4ca4093..9acf2454af 100644 --- a/source/dnode/vnode/src/tq/tqRestore.c +++ b/source/dnode/vnode/src/tq/tqRestore.c @@ -15,7 +15,7 @@ #include "tq.h" -static int32_t createStreamRunReq(SStreamMeta* pStreamMeta, bool* pScanIdle); +static int32_t createStreamTaskRunReq(SStreamMeta* pStreamMeta, bool* pScanIdle); // this function should be executed by stream threads. // extract submit block from WAL, and add them into the input queue for the sources tasks. @@ -30,7 +30,7 @@ int32_t tqStreamTasksScanWal(STQ* pTq) { // check all restore tasks bool shouldIdle = true; - createStreamRunReq(pTq->pStreamMeta, &shouldIdle); + createStreamTaskRunReq(pTq->pStreamMeta, &shouldIdle); int32_t times = 0; @@ -57,7 +57,39 @@ int32_t tqStreamTasksScanWal(STQ* pTq) { return 0; } -int32_t createStreamRunReq(SStreamMeta* pStreamMeta, bool* pScanIdle) { +static int32_t doSetOffsetForWalReader(SStreamTask *pTask, int32_t vgId) { + // seek the stored version and extract data from WAL + int64_t firstVer = walReaderGetValidFirstVer(pTask->exec.pWalReader); + if (pTask->chkInfo.currentVer < firstVer) { + pTask->chkInfo.currentVer = firstVer; + tqWarn("vgId:%d s-task:%s ver earlier than the first ver of wal range %" PRId64 ", forward to %" PRId64, vgId, + pTask->id.idStr, firstVer, pTask->chkInfo.currentVer); + + // todo need retry if failed + int32_t code = walReaderSeekVer(pTask->exec.pWalReader, pTask->chkInfo.currentVer); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + + // append the data for the stream + tqDebug("vgId:%d s-task:%s wal reader seek to ver:%" PRId64, vgId, pTask->id.idStr, pTask->chkInfo.currentVer); + } else { + int64_t currentVer = walReaderGetCurrentVer(pTask->exec.pWalReader); + if (currentVer == -1) { // we only seek the read for the first time + int32_t code = walReaderSeekVer(pTask->exec.pWalReader, pTask->chkInfo.currentVer); + if (code != TSDB_CODE_SUCCESS) { // no data in wal, quit + return code; + } + + // append the data for the stream + tqDebug("vgId:%d s-task:%s wal reader initial seek to ver:%" PRId64, vgId, pTask->id.idStr, pTask->chkInfo.currentVer); + } + } + + return TSDB_CODE_SUCCESS; +} + +int32_t createStreamTaskRunReq(SStreamMeta* pStreamMeta, bool* pScanIdle) { *pScanIdle = true; bool noNewDataInWal = true; int32_t vgId = pStreamMeta->vgId; @@ -67,6 +99,7 @@ int32_t createStreamRunReq(SStreamMeta* pStreamMeta, bool* pScanIdle) { return TSDB_CODE_SUCCESS; } + // clone the task list, to avoid the task update during scan wal files SArray* pTaskList = NULL; taosWLockLatch(&pStreamMeta->lock); pTaskList = taosArrayDup(pStreamMeta->pTaskList, NULL); @@ -107,38 +140,15 @@ int32_t createStreamRunReq(SStreamMeta* pStreamMeta, bool* pScanIdle) { *pScanIdle = false; // seek the stored version and extract data from WAL - int64_t firstVer = walReaderGetValidFirstVer(pTask->exec.pWalReader); - if (pTask->chkInfo.currentVer < firstVer) { - pTask->chkInfo.currentVer = firstVer; - tqWarn("vgId:%d s-task:%s ver earlier than the first ver of wal range %" PRId64 ", forward to %" PRId64, vgId, - pTask->id.idStr, firstVer, pTask->chkInfo.currentVer); - - // todo need retry if failed - int32_t code = walReaderSeekVer(pTask->exec.pWalReader, pTask->chkInfo.currentVer); - if (code != TSDB_CODE_SUCCESS) { - streamMetaReleaseTask(pStreamMeta, pTask); - continue; - } - - // append the data for the stream - tqDebug("vgId:%d s-task:%s wal reader seek to ver:%" PRId64, vgId, pTask->id.idStr, pTask->chkInfo.currentVer); - } else { - int64_t currentVer = walReaderGetCurrentVer(pTask->exec.pWalReader); - if (currentVer == -1) { - int32_t code = walReaderSeekVer(pTask->exec.pWalReader, pTask->chkInfo.currentVer); - if (code != TSDB_CODE_SUCCESS) { // no data in wal, quit - streamMetaReleaseTask(pStreamMeta, pTask); - continue; - } - - // append the data for the stream - tqDebug("vgId:%d s-task:%s wal reader initial seek to ver:%" PRId64, vgId, pTask->id.idStr, pTask->chkInfo.currentVer); - } + int32_t code = doSetOffsetForWalReader(pTask, vgId); + if (code != TSDB_CODE_SUCCESS) { + streamMetaReleaseTask(pStreamMeta, pTask); + continue; } // append the data for the stream SStreamQueueItem* pItem = NULL; - int32_t code = extractMsgFromWal(pTask->exec.pWalReader, (void**) &pItem, pTask->id.idStr); + code = extractMsgFromWal(pTask->exec.pWalReader, (void**) &pItem, pTask->id.idStr); if (code != TSDB_CODE_SUCCESS) { // failed, continue streamMetaReleaseTask(pStreamMeta, pTask); continue; @@ -161,7 +171,6 @@ int32_t createStreamRunReq(SStreamMeta* pStreamMeta, bool* pScanIdle) { tqError("s-task:%s append input queue failed, ver:%" PRId64, pTask->id.idStr, pTask->chkInfo.currentVer); } - streamMetaReleaseTask(pStreamMeta, pTask); } From c3491dae16f77706e53ffd2a1a2b5386c7a44671 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 18 May 2023 10:56:40 +0800 Subject: [PATCH 43/59] refactor: do some interal refactor. --- source/dnode/vnode/inc/vnode.h | 3 --- 1 file changed, 3 deletions(-) diff --git a/source/dnode/vnode/inc/vnode.h b/source/dnode/vnode/inc/vnode.h index a2fe0bef21..da5d6f8b3c 100644 --- a/source/dnode/vnode/inc/vnode.h +++ b/source/dnode/vnode/inc/vnode.h @@ -185,7 +185,6 @@ typedef struct STsdbReader STsdbReader; int32_t tsdbReaderOpen(SVnode *pVnode, SQueryTableDataCond *pCond, void *pTableList, int32_t numOfTables, SSDataBlock *pResBlock, STsdbReader **ppReader, const char *idstr, bool countOnly, SHashObj** pIgnoreTables); int32_t tsdbSetTableList(STsdbReader *pReader, const void *pTableList, int32_t num); - void tsdbReaderSetId(STsdbReader *pReader, const char *idstr); void tsdbReaderClose(STsdbReader *pReader); int32_t tsdbNextDataBlock(STsdbReader *pReader, bool *hasNext); @@ -198,8 +197,6 @@ int64_t tsdbGetNumOfRowsInMemTable(STsdbReader *pHandle); void *tsdbGetIdx(SMeta *pMeta); void *tsdbGetIvtIdx(SMeta *pMeta); uint64_t tsdbGetReaderMaxVersion(STsdbReader *pReader); -int32_t tsdbSetTableList(STsdbReader *pReader, const void *pTableList, int32_t num); -void tsdbReaderSetId(STsdbReader *pReader, const char *idstr); void tsdbReaderSetCloseFlag(STsdbReader *pReader); int32_t tsdbReuseCacherowsReader(void* pReader, void* pTableIdList, int32_t numOfTables); From ce61b2ab00a6cff7b7fcd2904a40828963b8b73f Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 18 May 2023 11:35:45 +0800 Subject: [PATCH 44/59] fix: fix syntax error in tqRead --- source/dnode/vnode/src/tq/tqRead.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/tq/tqRead.c b/source/dnode/vnode/src/tq/tqRead.c index 9e245fb45d..53c2da42c6 100644 --- a/source/dnode/vnode/src/tq/tqRead.c +++ b/source/dnode/vnode/src/tq/tqRead.c @@ -386,7 +386,7 @@ bool tqNextBlockInWal(STqReader* pReader, const char* id) { pReader->nextBlk = 0; } - size_t numOfBlocks = taosArrayGetSize(pReader->submit.aSubmitTbData); + int32_t numOfBlocks = taosArrayGetSize(pReader->submit.aSubmitTbData); while (pReader->nextBlk < numOfBlocks) { tqDebug("tq reader next data block %d/%d, len:%d %" PRId64 " %d", pReader->nextBlk, numOfBlocks, pReader->msg.msgLen, pReader->msg.ver, pReader->nextBlk); From 5d5b2bb16ab39021901b319a6dcf8bf2cadeabb3 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 18 May 2023 12:59:19 +0800 Subject: [PATCH 45/59] refactor: do some internal refactor. --- include/libs/executor/executor.h | 2 - include/libs/wal/wal.h | 5 +- source/dnode/vnode/src/tq/tqUtil.c | 2 +- source/libs/executor/src/executor.c | 13 +- source/libs/executor/src/scanoperator.c | 153 ++++++++++++++++-------- source/libs/wal/src/walRead.c | 11 ++ 6 files changed, 118 insertions(+), 68 deletions(-) diff --git a/include/libs/executor/executor.h b/include/libs/executor/executor.h index b7e6c42e3b..1fb00e743f 100644 --- a/include/libs/executor/executor.h +++ b/include/libs/executor/executor.h @@ -190,8 +190,6 @@ STimeWindow getAlignQueryTimeWindow(SInterval* pInterval, int32_t precision, int SArray* qGetQueriedTableListInfo(qTaskInfo_t tinfo); -void verifyOffset(void *pWalReader, STqOffsetVal* pOffset); - int32_t qStreamPrepareScan(qTaskInfo_t tinfo, STqOffsetVal* pOffset, int8_t subType); void qStreamSetOpen(qTaskInfo_t tinfo); diff --git a/include/libs/wal/wal.h b/include/libs/wal/wal.h index 862a1da9d0..35a6838b2e 100644 --- a/include/libs/wal/wal.h +++ b/include/libs/wal/wal.h @@ -142,7 +142,7 @@ typedef struct { typedef struct SWalReader SWalReader; // todo hide this struct -typedef struct SWalReader { +struct SWalReader { SWal *pWal; int64_t readerId; TdFilePtr pLogFile; @@ -154,7 +154,7 @@ typedef struct SWalReader { SWalFilterCond cond; // TODO remove it SWalCkHead *pHead; -} SWalReader; +}; // module initialization int32_t walInit(); @@ -201,6 +201,7 @@ int32_t walNextValidMsg(SWalReader *pRead); int64_t walReaderGetCurrentVer(const SWalReader *pReader); int64_t walReaderGetValidFirstVer(const SWalReader *pReader); void walReaderValidVersionRange(SWalReader *pReader, int64_t *sver, int64_t *ever); +void walReaderVerifyOffset(SWalReader *pWalReader, STqOffsetVal* pOffset); // only for tq usage void walSetReaderCapacity(SWalReader *pRead, int32_t capacity); diff --git a/source/dnode/vnode/src/tq/tqUtil.c b/source/dnode/vnode/src/tq/tqUtil.c index 8a0faa9d18..704f4695fb 100644 --- a/source/dnode/vnode/src/tq/tqUtil.c +++ b/source/dnode/vnode/src/tq/tqUtil.c @@ -245,7 +245,7 @@ static int32_t extractDataAndRspForDbStbSubscribe(STQ* pTq, STqHandle* pHandle, } if (offset->type == TMQ_OFFSET__LOG) { - verifyOffset(pHandle->pWalReader, offset); + walReaderVerifyOffset(pHandle->pWalReader, offset); int64_t fetchVer = offset->version + 1; pCkHead = taosMemoryMalloc(sizeof(SWalCkHead) + 2048); if (pCkHead == NULL) { diff --git a/source/libs/executor/src/executor.c b/source/libs/executor/src/executor.c index a825f56639..b6d4615997 100644 --- a/source/libs/executor/src/executor.c +++ b/source/libs/executor/src/executor.c @@ -1058,17 +1058,6 @@ void qStreamSetOpen(qTaskInfo_t tinfo) { pOperator->status = OP_NOT_OPENED; } -void verifyOffset(void *pWalReader, STqOffsetVal* pOffset){ - // if offset version is small than first version , let's seek to first version - taosThreadMutexLock(&((SWalReader*)pWalReader)->pWal->mutex); - int64_t firstVer = walGetFirstVer(((SWalReader*)pWalReader)->pWal); - taosThreadMutexUnlock(&((SWalReader*)pWalReader)->pWal->mutex); - - if (pOffset->version + 1 < firstVer){ - pOffset->version = firstVer - 1; - } -} - int32_t qStreamPrepareScan(qTaskInfo_t tinfo, STqOffsetVal* pOffset, int8_t subType) { SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo; SOperatorInfo* pOperator = pTaskInfo->pRoot; @@ -1095,7 +1084,7 @@ int32_t qStreamPrepareScan(qTaskInfo_t tinfo, STqOffsetVal* pOffset, int8_t subT tsdbReaderClose(pScanBaseInfo->dataReader); pScanBaseInfo->dataReader = NULL; - verifyOffset(pInfo->tqReader->pWalReader, pOffset); + walReaderVerifyOffset(pInfo->tqReader->pWalReader, pOffset); if (tqSeekVer(pInfo->tqReader, pOffset->version + 1, id) < 0) { qError("tqSeekVer failed ver:%" PRId64 ", %s", pOffset->version + 1, id); return -1; diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 1aa60ea392..0c93365c1c 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -1697,13 +1697,13 @@ static SSDataBlock* doQueueScan(SOperatorInfo* pOperator) { if (pTaskInfo->streamInfo.currentOffset.type == TMQ_OFFSET__LOG) { while (1) { - int32_t type = tqNextBlockInWal(pInfo->tqReader); + bool hasResult = tqNextBlockInWal(pInfo->tqReader, id); SSDataBlock* pRes = pInfo->tqReader->pResBlock; // curVersion move to next, so currentOffset = curVersion - 1 tqOffsetResetToLog(&pTaskInfo->streamInfo.currentOffset, pInfo->tqReader->pWalReader->curVersion - 1); - if (type == FETCH_TYPE__DATA) { + if (hasResult) { qDebug("doQueueScan get data from log %" PRId64 " rows, version:%" PRId64, pRes->info.rows, pTaskInfo->streamInfo.currentOffset.version); blockDataCleanup(pInfo->pRes); @@ -1711,7 +1711,7 @@ static SSDataBlock* doQueueScan(SOperatorInfo* pOperator) { if (pInfo->pRes->info.rows > 0) { return pInfo->pRes; } - } else if (type == FETCH_TYPE__NONE) { + } else { qDebug("doQueueScan get none from log, return, version:%" PRId64, pTaskInfo->streamInfo.currentOffset.version); return NULL; } @@ -2074,66 +2074,43 @@ FETCH_NEXT_BLOCK: return pInfo->pUpdateRes; } - const char* id = GET_TASKID(pTaskInfo); - SDataBlockInfo* pBlockInfo = &pInfo->pRes->info; + const char* id = GET_TASKID(pTaskInfo); + SSDataBlock* pBlock = pInfo->pRes; + SDataBlockInfo* pBlockInfo = &pBlock->info; int32_t totalBlocks = taosArrayGetSize(pInfo->pBlockLists); NEXT_SUBMIT_BLK: + while (1) { - if (pInfo->tqReader->msg.msgStr == NULL) { - if (pInfo->validBlockIndex >= totalBlocks) { - updateInfoDestoryColseWinSBF(pInfo->pUpdateInfo); - doClearBufferedBlocks(pInfo); + bool hasResult = tqNextBlockInWal(pInfo->tqReader, id); + SSDataBlock* pRes = pInfo->tqReader->pResBlock; - qDebug("stream scan return empty, all %d submit blocks consumed, %s", totalBlocks, id); - void* buff = NULL; - // int32_t len = streamScanOperatorEncode(pInfo, &buff); - // if (len > 0) { - // streamStateSaveInfo(pInfo->pState, STREAM_SCAN_OP_NAME, strlen(STREAM_SCAN_OP_NAME), buff, len); - // } - taosMemoryFreeClear(buff); - return NULL; - } + blockDataCleanup(pBlock); - int32_t current = pInfo->validBlockIndex++; - SPackedData* pSubmit = taosArrayGet(pInfo->pBlockLists, current); - - qDebug("set %d/%d as the input submit block, %s", current, totalBlocks, id); - if (tqReaderSetSubmitMsg(pInfo->tqReader, pSubmit->msgStr, pSubmit->msgLen, pSubmit->ver) < 0) { - qError("submit msg messed up when initializing stream submit block %p, current %d/%d, %s", pSubmit, current, totalBlocks, id); - continue; - } - } - - blockDataCleanup(pInfo->pRes); - - while (tqNextBlockImpl(pInfo->tqReader, id)) { - int32_t code = tqRetrieveDataBlock(pInfo->tqReader, id); - if (code != TSDB_CODE_SUCCESS || pInfo->tqReader->pResBlock->info.rows == 0) { - continue; - } - - setBlockIntoRes(pInfo, pInfo->tqReader->pResBlock, false); + if (hasResult) { + qDebug("stream scan get data from log %" PRId64 " rows, version:%" PRId64, pRes->info.rows, + pTaskInfo->streamInfo.currentOffset.version); + setBlockIntoRes(pInfo, pRes, true); if (pInfo->pCreateTbRes->info.rows > 0) { pInfo->scanMode = STREAM_SCAN_FROM_RES; return pInfo->pCreateTbRes; } - doCheckUpdate(pInfo, pBlockInfo->window.ekey, pInfo->pRes); - doFilter(pInfo->pRes, pOperator->exprSupp.pFilterInfo, NULL); - pInfo->pRes->info.dataLoad = 1; - blockDataUpdateTsWindow(pInfo->pRes, pInfo->primaryTsIndex); + doCheckUpdate(pInfo, pBlockInfo->window.ekey, pBlock); + doFilter(pBlock, pOperator->exprSupp.pFilterInfo, NULL); + pBlock->info.dataLoad = 1; + blockDataUpdateTsWindow(pBlock, pInfo->primaryTsIndex); + } else { + updateInfoDestoryColseWinSBF(pInfo->pUpdateInfo); + doClearBufferedBlocks(pInfo); - if (pBlockInfo->rows > 0 || pInfo->pUpdateDataRes->info.rows > 0) { - break; - } + qDebug("stream scan return empty, all %d submit blocks consumed, %s", totalBlocks, id); + return NULL; } if (pBlockInfo->rows > 0 || pInfo->pUpdateDataRes->info.rows > 0) { break; - } else { - continue; } } @@ -2141,9 +2118,9 @@ FETCH_NEXT_BLOCK: pInfo->numOfExec++; pOperator->resultInfo.totalRows += pBlockInfo->rows; - qDebug("stream scan get source rows:%" PRId64", %s", pBlockInfo->rows, id); + qDebug("stream scan get source rows:%" PRId64 ", %s", pBlockInfo->rows, id); if (pBlockInfo->rows > 0) { - return pInfo->pRes; + return pBlock; } if (pInfo->pUpdateDataRes->info.rows > 0) { @@ -2151,10 +2128,84 @@ FETCH_NEXT_BLOCK: } goto NEXT_SUBMIT_BLK; - } else { - ASSERT(0); - return NULL; + + // } else { + // qDebug("stream scan get none from log, return, version:%" PRId64, + // pTaskInfo->streamInfo.currentOffset.version); return NULL; + // } + + // while (1) { + // if (pInfo->tqReader->msg.msgStr == NULL) { + // if (pInfo->validBlockIndex >= totalBlocks) { + // updateInfoDestoryColseWinSBF(pInfo->pUpdateInfo); + // doClearBufferedBlocks(pInfo); + // + // qDebug("stream scan return empty, all %d submit blocks consumed, %s", totalBlocks, id); + // return NULL; + // } + // + // int32_t current = pInfo->validBlockIndex++; + // SPackedData* pSubmit = taosArrayGet(pInfo->pBlockLists, current); + // + // qDebug("set %d/%d as the input submit block, %s", current, totalBlocks, id); + // if (tqReaderSetSubmitMsg(pInfo->tqReader, pSubmit->msgStr, pSubmit->msgLen, pSubmit->ver) < 0) { + // qError("submit msg messed up when initializing stream submit block %p, current %d/%d, %s", pSubmit, + // current, totalBlocks, id); continue; + // } + // } + // + // blockDataCleanup(pInfo->pRes); + // + // while (tqNextBlockImpl(pInfo->tqReader, id)) { + // int32_t code = tqRetrieveDataBlock(pInfo->tqReader, id); + // if (code != TSDB_CODE_SUCCESS || pInfo->tqReader->pResBlock->info.rows == 0) { + // continue; + // } + // + // setBlockIntoRes(pInfo, pInfo->tqReader->pResBlock, false); + // + // if (pInfo->pCreateTbRes->info.rows > 0) { + // pInfo->scanMode = STREAM_SCAN_FROM_RES; + // return pInfo->pCreateTbRes; + // } + // + // doCheckUpdate(pInfo, pBlockInfo->window.ekey, pInfo->pRes); + // doFilter(pInfo->pRes, pOperator->exprSupp.pFilterInfo, NULL); + // pInfo->pRes->info.dataLoad = 1; + // blockDataUpdateTsWindow(pInfo->pRes, pInfo->primaryTsIndex); + // + // if (pBlockInfo->rows > 0 || pInfo->pUpdateDataRes->info.rows > 0) { + // break; + // } + // } + // + // if (pBlockInfo->rows > 0 || pInfo->pUpdateDataRes->info.rows > 0) { + // break; + // } else { + // continue; + // } + // } + // + // // record the scan action. + // pInfo->numOfExec++; + // pOperator->resultInfo.totalRows += pBlockInfo->rows; + // + // qDebug("stream scan get source rows:%" PRId64", %s", pBlockInfo->rows, id); + // if (pBlockInfo->rows > 0) { + // return pInfo->pRes; + // } + // + // if (pInfo->pUpdateDataRes->info.rows > 0) { + // goto FETCH_NEXT_BLOCK; + // } + // + // goto NEXT_SUBMIT_BLK; + // } else { + // ASSERT(0); + // return NULL; + // } } + return NULL; } static SArray* extractTableIdList(const STableListInfo* pTableListInfo) { diff --git a/source/libs/wal/src/walRead.c b/source/libs/wal/src/walRead.c index 627529d956..c29d82bcf3 100644 --- a/source/libs/wal/src/walRead.c +++ b/source/libs/wal/src/walRead.c @@ -116,6 +116,17 @@ void walReaderValidVersionRange(SWalReader *pReader, int64_t *sver, int64_t *eve *ever = pReader->cond.scanUncommited ? lastVer : committedVer; } +void walReaderVerifyOffset(SWalReader *pWalReader, STqOffsetVal* pOffset){ + // if offset version is small than first version , let's seek to first version + taosThreadMutexLock(&pWalReader->pWal->mutex); + int64_t firstVer = walGetFirstVer((pWalReader)->pWal); + taosThreadMutexUnlock(&pWalReader->pWal->mutex); + + if (pOffset->version + 1 < firstVer){ + pOffset->version = firstVer - 1; + } +} + static int64_t walReadSeekFilePos(SWalReader *pReader, int64_t fileFirstVer, int64_t ver) { int64_t ret = 0; From 0f8b7c87f7ba1733dfe17f488f12bc68d435c2f7 Mon Sep 17 00:00:00 2001 From: Minglei Jin Date: Thu, 18 May 2023 13:15:25 +0800 Subject: [PATCH 46/59] cache/delete: fix rocks key length --- source/dnode/vnode/src/tsdb/tsdbCache.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/tsdb/tsdbCache.c b/source/dnode/vnode/src/tsdb/tsdbCache.c index fe7e808db3..5d7df048f8 100644 --- a/source/dnode/vnode/src/tsdb/tsdbCache.c +++ b/source/dnode/vnode/src/tsdb/tsdbCache.c @@ -927,7 +927,7 @@ int32_t tsdbCacheDel(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSKEY sKey, TSKE int16_t cid = pTSchema->columns[i].colId; size_t klen = ROCKS_KEY_LEN; - char *keys = taosMemoryCalloc(2, klen); + char *keys = taosMemoryCalloc(2, sizeof(SLastKey)); ((SLastKey *)keys)[0] = (SLastKey){.ltype = 1, .uid = uid, .cid = cid}; ((SLastKey *)keys)[1] = (SLastKey){.ltype = 0, .uid = uid, .cid = cid}; From 3b316051bfb73c922c83ea823a5866da1e7e2695 Mon Sep 17 00:00:00 2001 From: Minglei Jin Date: Thu, 18 May 2023 13:51:07 +0800 Subject: [PATCH 47/59] cache/schema: fix schema double free --- source/dnode/vnode/src/tsdb/tsdbCache.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/tsdb/tsdbCache.c b/source/dnode/vnode/src/tsdb/tsdbCache.c index 5d7df048f8..bc79cacc4b 100644 --- a/source/dnode/vnode/src/tsdb/tsdbCache.c +++ b/source/dnode/vnode/src/tsdb/tsdbCache.c @@ -982,7 +982,7 @@ int32_t tsdbCacheDel(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSKEY sKey, TSKE taosThreadMutexUnlock(&pTsdb->rCache.rMutex); _exit: - taosMemoryFree(pTSchema); + // taosMemoryFree(pTSchema); return code; } From 178a3038326526be13725d02a46e99679c41893c Mon Sep 17 00:00:00 2001 From: dmchen Date: Thu, 18 May 2023 14:43:10 +0800 Subject: [PATCH 48/59] change restore prompt message --- include/util/taoserror.h | 1 + source/util/src/terror.c | 1 + 2 files changed, 2 insertions(+) diff --git a/include/util/taoserror.h b/include/util/taoserror.h index 8bc6ed2ac8..a2cf0d0cac 100644 --- a/include/util/taoserror.h +++ b/include/util/taoserror.h @@ -408,6 +408,7 @@ int32_t* taosGetErrno(); #define TSDB_CODE_MNODE_NOT_CATCH_UP TAOS_DEF_ERROR_CODE(0, 0x0412) // internal #define TSDB_CODE_MNODE_ALREADY_IS_VOTER TAOS_DEF_ERROR_CODE(0, 0x0413) // internal #define TSDB_CODE_MNODE_ONLY_TWO_MNODE TAOS_DEF_ERROR_CODE(0, 0x0414) // internal +#define TSDB_CODE_MNODE_NO_NEED_RESTORE TAOS_DEF_ERROR_CODE(0, 0x0415) // internal // vnode // #define TSDB_CODE_VND_ACTION_IN_PROGRESS TAOS_DEF_ERROR_CODE(0, 0x0500) // 2.x diff --git a/source/util/src/terror.c b/source/util/src/terror.c index 58c7300bee..54459329e7 100644 --- a/source/util/src/terror.c +++ b/source/util/src/terror.c @@ -324,6 +324,7 @@ TAOS_DEFINE_ERROR(TSDB_CODE_SNODE_NOT_DEPLOYED, "Snode not deployed") TAOS_DEFINE_ERROR(TSDB_CODE_MNODE_NOT_CATCH_UP, "Mnode didn't catch the leader") TAOS_DEFINE_ERROR(TSDB_CODE_MNODE_ALREADY_IS_VOTER, "Mnode already is a leader") TAOS_DEFINE_ERROR(TSDB_CODE_MNODE_ONLY_TWO_MNODE, "Only two mnodes exist") +TAOS_DEFINE_ERROR(TSDB_CODE_MNODE_NO_NEED_RESTORE, "No need restore on this dnode") // vnode TAOS_DEFINE_ERROR(TSDB_CODE_VND_INVALID_VGROUP_ID, "Vnode is closed or removed") From 4c6dd31006a579ca43bb85fde27b58d5c0f9e6f1 Mon Sep 17 00:00:00 2001 From: chenhaoran Date: Thu, 18 May 2023 14:48:08 +0800 Subject: [PATCH 49/59] test: modify building process in ci --- tests/parallel_test/container_build.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/parallel_test/container_build.sh b/tests/parallel_test/container_build.sh index 0fc29c241b..48fdbce16a 100755 --- a/tests/parallel_test/container_build.sh +++ b/tests/parallel_test/container_build.sh @@ -68,7 +68,7 @@ docker run \ -v ${REP_REAL_PATH}/community/contrib/libuv/:${REP_DIR}/community/contrib/libuv \ -v ${REP_REAL_PATH}/community/contrib/lz4/:${REP_DIR}/community/contrib/lz4 \ -v ${REP_REAL_PATH}/community/contrib/zlib/:${REP_DIR}/community/contrib/zlib \ - --rm --ulimit core=-1 taos_test:v1.0 sh -c "pip uninstall taospy -y;pip3 install taospy==2.7.2;cd $REP_DIR;rm -rf debug;mkdir -p debug;cd debug;cmake .. -DBUILD_HTTP=false -DBUILD_TOOLS=true -DBUILD_TEST=true -DWEBSOCKET=true -DBUILD_TAOSX=true -DJEMALLOC_ENABLED=0;make -j || exit 1" + --rm --ulimit core=-1 taos_test:v1.0 sh -c "pip uninstall taospy -y;pip3 install taospy==2.7.2;cd $REP_DIR;rm -rf debug;mkdir -p debug;cd debug;cmake .. -DBUILD_HTTP=false -DBUILD_TOOLS=true -DBUILD_TEST=true -DWEBSOCKET=true -DBUILD_TAOSX=true -DJEMALLOC_ENABLED=0;make -j 10 || exit 1" # -v ${REP_REAL_PATH}/community/contrib/jemalloc/:${REP_DIR}/community/contrib/jemalloc \ if [[ -d ${WORKDIR}/debugNoSan ]] ;then @@ -97,7 +97,7 @@ docker run \ -v ${REP_REAL_PATH}/community/contrib/lz4/:${REP_DIR}/community/contrib/lz4 \ -v ${REP_REAL_PATH}/community/contrib/zlib/:${REP_DIR}/community/contrib/zlib \ -v ${REP_REAL_PATH}/community/contrib/jemalloc/:${REP_DIR}/community/contrib/jemalloc \ - --rm --ulimit core=-1 taos_test:v1.0 sh -c "pip uninstall taospy -y;pip3 install taospy==2.7.2;cd $REP_DIR;rm -rf debug;mkdir -p debug;cd debug;cmake .. -DBUILD_HTTP=false -DBUILD_TOOLS=true -DBUILD_TEST=true -DWEBSOCKET=true -DBUILD_SANITIZER=1 -DTOOLS_SANITIZE=true -DTOOLS_BUILD_TYPE=Debug -DBUILD_TAOSX=true -DJEMALLOC_ENABLED=0;make -j || exit 1 " + --rm --ulimit core=-1 taos_test:v1.0 sh -c "pip uninstall taospy -y;pip3 install taospy==2.7.2;cd $REP_DIR;rm -rf debug;mkdir -p debug;cd debug;cmake .. -DBUILD_HTTP=false -DBUILD_TOOLS=true -DBUILD_TEST=true -DWEBSOCKET=true -DBUILD_SANITIZER=1 -DTOOLS_SANITIZE=true -DTOOLS_BUILD_TYPE=Debug -DBUILD_TAOSX=true -DJEMALLOC_ENABLED=0;make -j 10 || exit 1 " mv ${REP_REAL_PATH}/debug ${WORKDIR}/debugSan From f95867d2558b014fd965aaa2f32323ed4bbe8b71 Mon Sep 17 00:00:00 2001 From: Minglei Jin Date: Thu, 18 May 2023 14:48:08 +0800 Subject: [PATCH 50/59] cache/del: fix lru key length --- source/dnode/vnode/src/tsdb/tsdbCache.c | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/source/dnode/vnode/src/tsdb/tsdbCache.c b/source/dnode/vnode/src/tsdb/tsdbCache.c index bc79cacc4b..505bb4b906 100644 --- a/source/dnode/vnode/src/tsdb/tsdbCache.c +++ b/source/dnode/vnode/src/tsdb/tsdbCache.c @@ -932,7 +932,7 @@ int32_t tsdbCacheDel(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSKEY sKey, TSKE ((SLastKey *)keys)[1] = (SLastKey){.ltype = 0, .uid = uid, .cid = cid}; keys_list[i] = keys; - keys_list[num_keys + i] = keys + klen; + keys_list[num_keys + i] = keys + sizeof(SLastKey); keys_list_sizes[i] = klen; keys_list_sizes[num_keys + i] = klen; } @@ -940,6 +940,7 @@ int32_t tsdbCacheDel(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSKEY sKey, TSKE size_t *values_list_sizes = taosMemoryCalloc(num_keys * 2, sizeof(size_t)); char **errs = taosMemoryCalloc(num_keys * 2, sizeof(char *)); taosThreadMutexLock(&pTsdb->rCache.rMutex); + rocksMayWrite(pTsdb, true); rocksdb_multi_get(pTsdb->rCache.db, pTsdb->rCache.readoptions, num_keys * 2, (const char *const *)keys_list, keys_list_sizes, values_list, values_list_sizes, errs); for (int i = 0; i < num_keys; ++i) { @@ -957,7 +958,7 @@ int32_t tsdbCacheDel(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSKEY sKey, TSKE SLastCol *pLastCol = tsdbCacheDeserialize(values_list[i]); if (NULL != pLastCol && (pLastCol->ts <= eKey && pLastCol->ts >= sKey)) { SLastKey *key = &(SLastKey){.ltype = 1, .uid = uid, .cid = pLastCol->colVal.cid}; - size_t klen = sizeof(*key); + size_t klen = ROCKS_KEY_LEN; rocksdb_writebatch_delete(wb, (char *)key, klen); taosLRUCacheErase(pTsdb->lruCache, key, klen); @@ -966,7 +967,7 @@ int32_t tsdbCacheDel(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSKEY sKey, TSKE pLastCol = tsdbCacheDeserialize(values_list[i + num_keys]); if (NULL != pLastCol && (pLastCol->ts <= eKey && pLastCol->ts >= sKey)) { SLastKey *key = &(SLastKey){.ltype = 0, .uid = uid, .cid = pLastCol->colVal.cid}; - size_t klen = sizeof(*key); + size_t klen = ROCKS_KEY_LEN; rocksdb_writebatch_delete(wb, (char *)key, klen); taosLRUCacheErase(pTsdb->lruCache, key, klen); @@ -978,7 +979,7 @@ int32_t tsdbCacheDel(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSKEY sKey, TSKE taosMemoryFree(values_list); taosMemoryFree(values_list_sizes); - rocksMayWrite(pTsdb, false); + rocksMayWrite(pTsdb, true); taosThreadMutexUnlock(&pTsdb->rCache.rMutex); _exit: From da3fc5111e30826388ea1f767e3dd8efd1315625 Mon Sep 17 00:00:00 2001 From: liuyao <54liuyao@163.com> Date: Thu, 18 May 2023 14:59:51 +0800 Subject: [PATCH 51/59] fix pause agg task --- source/dnode/mnode/impl/src/mndStream.c | 4 +- source/dnode/vnode/src/tq/tq.c | 6 ++- source/libs/stream/src/streamExec.c | 3 ++ tests/script/tsim/stream/pauseAndResume.sim | 59 +++++++++++++++++++++ 4 files changed, 69 insertions(+), 3 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 68b8dd7201..0713150b48 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1306,7 +1306,7 @@ int32_t mndPauseAllStreamTasks(STrans *pTrans, SStreamObj *pStream) { int32_t sz = taosArrayGetSize(pTasks); for (int32_t j = 0; j < sz; j++) { SStreamTask *pTask = taosArrayGetP(pTasks, j); - if (pTask->taskLevel == TASK_LEVEL__SOURCE && mndPauseStreamTask(pTrans, pTask) < 0) { + if (pTask->taskLevel != TASK_LEVEL__SINK && mndPauseStreamTask(pTrans, pTask) < 0) { return -1; } } @@ -1430,7 +1430,7 @@ int32_t mndResumeAllStreamTasks(STrans *pTrans, SStreamObj *pStream, int8_t igUn int32_t sz = taosArrayGetSize(pTasks); for (int32_t j = 0; j < sz; j++) { SStreamTask *pTask = taosArrayGetP(pTasks, j); - if (pTask->taskLevel == TASK_LEVEL__SOURCE && mndResumeStreamTask(pTrans, pTask, igUntreated) < 0) { + if (pTask->taskLevel != TASK_LEVEL__SINK && mndResumeStreamTask(pTrans, pTask, igUntreated) < 0) { return -1; } } diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 0179480011..e61cc2fdb1 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1449,7 +1449,11 @@ int32_t tqProcessTaskResumeReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms } streamMetaReleaseTask(pTq->pStreamMeta, pTask); - tqStartStreamTasks(pTq); + if (pTask->taskLevel == TASK_LEVEL__SOURCE) { + tqStartStreamTasks(pTq); + } else { + streamSchedExec(pTask); + } } return 0; diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index e05660da32..f64b24ed7e 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -269,6 +269,9 @@ int32_t streamExecForAll(SStreamTask* pTask) { qDebug("s-task:%s start to extract data block from inputQ", pTask->id.idStr); while (1) { + if (streamTaskShouldPause(&pTask->status)) { + return 0; + } SStreamQueueItem* qItem = streamQueueNextItem(pTask->inputQueue); if (qItem == NULL) { if (pTask->taskLevel == TASK_LEVEL__SOURCE && batchSize < MIN_STREAM_EXEC_BATCH_NUM && times < 5) { diff --git a/tests/script/tsim/stream/pauseAndResume.sim b/tests/script/tsim/stream/pauseAndResume.sim index fa7be19310..402e0086f7 100644 --- a/tests/script/tsim/stream/pauseAndResume.sim +++ b/tests/script/tsim/stream/pauseAndResume.sim @@ -307,4 +307,63 @@ sql resume stream IF EXISTS streams66666666; print ===== step 4 over +print ===== step5 +sql drop stream if exists streams6; +sql drop database if exists test6; +sql create database test6 vgroups 10; +sql use test6; +sql create stable st(ts timestamp, a int, b int , c int, d double) tags(ta int,tb int,tc int); +sql create table ts1 using st tags(1,1,1); +sql create table ts2 using st tags(2,2,2); +sql create table ts3 using st tags(3,2,2); +sql create table ts4 using st tags(4,2,2); +sql create stream streams6 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 watermark 1d into streamt6 as select _wstart, count(*) c1 from st interval(10s); + +sql insert into ts1 values(1648791213001,1,12,3,1.0); +sql insert into ts2 values(1648791213001,1,12,3,1.0); + +sql insert into ts3 values(1648791213001,1,12,3,1.0); +sql insert into ts4 values(1648791213001,1,12,3,1.0); + +sleep 1000 + +sql pause stream streams6; + +sleep 1000 + + +sql insert into ts1 values(1648791223001,1,12,3,1.0); +sql insert into ts2 values(1648791233001,1,12,3,1.0); + +sql resume stream streams6; + +sql insert into ts3 values(1648791243001,1,12,3,1.0); +sql insert into ts4 values(1648791253001,1,12,3,1.0); + +$loop_count = 0 +loop6: + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +sleep 500 + +print 2 select * from streamt6; +sql select * from streamt6; + +if $rows != 5 then + print =====rows=$rows + print $data00 $data01 $data02 + print $data10 $data11 $data12 + print $data20 $data21 $data22 + print $data30 $data31 $data32 + print $data40 $data41 $data42 + print $data50 $data51 $data52 + goto loop6 +endi + +print ===== step5 over + system sh/stop_dnodes.sh From db2ac73b8ca8672832aa13d797a28eb7bb5dc99c Mon Sep 17 00:00:00 2001 From: haoranchen Date: Thu, 18 May 2023 16:15:16 +0800 Subject: [PATCH 52/59] Update container_build.sh --- tests/parallel_test/container_build.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/parallel_test/container_build.sh b/tests/parallel_test/container_build.sh index 0fc29c241b..5ae061072a 100755 --- a/tests/parallel_test/container_build.sh +++ b/tests/parallel_test/container_build.sh @@ -68,7 +68,7 @@ docker run \ -v ${REP_REAL_PATH}/community/contrib/libuv/:${REP_DIR}/community/contrib/libuv \ -v ${REP_REAL_PATH}/community/contrib/lz4/:${REP_DIR}/community/contrib/lz4 \ -v ${REP_REAL_PATH}/community/contrib/zlib/:${REP_DIR}/community/contrib/zlib \ - --rm --ulimit core=-1 taos_test:v1.0 sh -c "pip uninstall taospy -y;pip3 install taospy==2.7.2;cd $REP_DIR;rm -rf debug;mkdir -p debug;cd debug;cmake .. -DBUILD_HTTP=false -DBUILD_TOOLS=true -DBUILD_TEST=true -DWEBSOCKET=true -DBUILD_TAOSX=true -DJEMALLOC_ENABLED=0;make -j || exit 1" + --rm --ulimit core=-1 taos_test:v1.0 sh -c "pip uninstall taospy -y;pip3 install taospy==2.7.2;cd $REP_DIR;rm -rf debug;mkdir -p debug;cd debug;cmake .. -DBUILD_HTTP=false -DBUILD_TOOLS=true -DBUILD_TEST=true -DWEBSOCKET=true -DBUILD_TAOSX=true -DJEMALLOC_ENABLED=0;make -j 10|| exit 1" # -v ${REP_REAL_PATH}/community/contrib/jemalloc/:${REP_DIR}/community/contrib/jemalloc \ if [[ -d ${WORKDIR}/debugNoSan ]] ;then @@ -97,7 +97,7 @@ docker run \ -v ${REP_REAL_PATH}/community/contrib/lz4/:${REP_DIR}/community/contrib/lz4 \ -v ${REP_REAL_PATH}/community/contrib/zlib/:${REP_DIR}/community/contrib/zlib \ -v ${REP_REAL_PATH}/community/contrib/jemalloc/:${REP_DIR}/community/contrib/jemalloc \ - --rm --ulimit core=-1 taos_test:v1.0 sh -c "pip uninstall taospy -y;pip3 install taospy==2.7.2;cd $REP_DIR;rm -rf debug;mkdir -p debug;cd debug;cmake .. -DBUILD_HTTP=false -DBUILD_TOOLS=true -DBUILD_TEST=true -DWEBSOCKET=true -DBUILD_SANITIZER=1 -DTOOLS_SANITIZE=true -DTOOLS_BUILD_TYPE=Debug -DBUILD_TAOSX=true -DJEMALLOC_ENABLED=0;make -j || exit 1 " + --rm --ulimit core=-1 taos_test:v1.0 sh -c "pip uninstall taospy -y;pip3 install taospy==2.7.2;cd $REP_DIR;rm -rf debug;mkdir -p debug;cd debug;cmake .. -DBUILD_HTTP=false -DBUILD_TOOLS=true -DBUILD_TEST=true -DWEBSOCKET=true -DBUILD_SANITIZER=1 -DTOOLS_SANITIZE=true -DTOOLS_BUILD_TYPE=Debug -DBUILD_TAOSX=true -DJEMALLOC_ENABLED=0;make -j 10|| exit 1 " mv ${REP_REAL_PATH}/debug ${WORKDIR}/debugSan From 8a328b6636699c4cc91404a8c14e87d91d90de00 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 18 May 2023 16:59:18 +0800 Subject: [PATCH 53/59] fix(stream): fix error in stream read. --- source/libs/executor/src/scanoperator.c | 159 +++++------------------- source/libs/stream/src/streamQueue.c | 1 + 2 files changed, 33 insertions(+), 127 deletions(-) diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 0c93365c1c..0537702a75 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -2080,17 +2080,35 @@ FETCH_NEXT_BLOCK: int32_t totalBlocks = taosArrayGetSize(pInfo->pBlockLists); NEXT_SUBMIT_BLK: - while (1) { - bool hasResult = tqNextBlockInWal(pInfo->tqReader, id); - SSDataBlock* pRes = pInfo->tqReader->pResBlock; + if (pInfo->tqReader->msg.msgStr == NULL) { + if (pInfo->validBlockIndex >= totalBlocks) { + updateInfoDestoryColseWinSBF(pInfo->pUpdateInfo); + doClearBufferedBlocks(pInfo); + + qDebug("stream scan return empty, all %d submit blocks consumed, %s", totalBlocks, id); + return NULL; + } + + int32_t current = pInfo->validBlockIndex++; + SPackedData* pSubmit = taosArrayGet(pInfo->pBlockLists, current); + + qDebug("set %d/%d as the input submit block, %s", current, totalBlocks, id); + if (tqReaderSetSubmitMsg(pInfo->tqReader, pSubmit->msgStr, pSubmit->msgLen, pSubmit->ver) < 0) { + qError("submit msg messed up when initializing stream submit block %p, current %d/%d, %s", pSubmit, current, totalBlocks, id); + continue; + } + } blockDataCleanup(pBlock); - if (hasResult) { - qDebug("stream scan get data from log %" PRId64 " rows, version:%" PRId64, pRes->info.rows, - pTaskInfo->streamInfo.currentOffset.version); - setBlockIntoRes(pInfo, pRes, true); + while (tqNextBlockImpl(pInfo->tqReader, id)) { + int32_t code = tqRetrieveDataBlock(pInfo->tqReader, id); + if (code != TSDB_CODE_SUCCESS || pInfo->tqReader->pResBlock->info.rows == 0) { + continue; + } + + setBlockIntoRes(pInfo, pInfo->tqReader->pResBlock, false); if (pInfo->pCreateTbRes->info.rows > 0) { pInfo->scanMode = STREAM_SCAN_FROM_RES; @@ -2101,16 +2119,16 @@ FETCH_NEXT_BLOCK: doFilter(pBlock, pOperator->exprSupp.pFilterInfo, NULL); pBlock->info.dataLoad = 1; blockDataUpdateTsWindow(pBlock, pInfo->primaryTsIndex); - } else { - updateInfoDestoryColseWinSBF(pInfo->pUpdateInfo); - doClearBufferedBlocks(pInfo); - qDebug("stream scan return empty, all %d submit blocks consumed, %s", totalBlocks, id); - return NULL; + if (pBlockInfo->rows > 0 || pInfo->pUpdateDataRes->info.rows > 0) { + break; + } } if (pBlockInfo->rows > 0 || pInfo->pUpdateDataRes->info.rows > 0) { break; + } else { + continue; } } @@ -2118,7 +2136,7 @@ FETCH_NEXT_BLOCK: pInfo->numOfExec++; pOperator->resultInfo.totalRows += pBlockInfo->rows; - qDebug("stream scan get source rows:%" PRId64 ", %s", pBlockInfo->rows, id); + qDebug("stream scan get source rows:%" PRId64", %s", pBlockInfo->rows, id); if (pBlockInfo->rows > 0) { return pBlock; } @@ -2128,83 +2146,8 @@ FETCH_NEXT_BLOCK: } goto NEXT_SUBMIT_BLK; - - // } else { - // qDebug("stream scan get none from log, return, version:%" PRId64, - // pTaskInfo->streamInfo.currentOffset.version); return NULL; - // } - - // while (1) { - // if (pInfo->tqReader->msg.msgStr == NULL) { - // if (pInfo->validBlockIndex >= totalBlocks) { - // updateInfoDestoryColseWinSBF(pInfo->pUpdateInfo); - // doClearBufferedBlocks(pInfo); - // - // qDebug("stream scan return empty, all %d submit blocks consumed, %s", totalBlocks, id); - // return NULL; - // } - // - // int32_t current = pInfo->validBlockIndex++; - // SPackedData* pSubmit = taosArrayGet(pInfo->pBlockLists, current); - // - // qDebug("set %d/%d as the input submit block, %s", current, totalBlocks, id); - // if (tqReaderSetSubmitMsg(pInfo->tqReader, pSubmit->msgStr, pSubmit->msgLen, pSubmit->ver) < 0) { - // qError("submit msg messed up when initializing stream submit block %p, current %d/%d, %s", pSubmit, - // current, totalBlocks, id); continue; - // } - // } - // - // blockDataCleanup(pInfo->pRes); - // - // while (tqNextBlockImpl(pInfo->tqReader, id)) { - // int32_t code = tqRetrieveDataBlock(pInfo->tqReader, id); - // if (code != TSDB_CODE_SUCCESS || pInfo->tqReader->pResBlock->info.rows == 0) { - // continue; - // } - // - // setBlockIntoRes(pInfo, pInfo->tqReader->pResBlock, false); - // - // if (pInfo->pCreateTbRes->info.rows > 0) { - // pInfo->scanMode = STREAM_SCAN_FROM_RES; - // return pInfo->pCreateTbRes; - // } - // - // doCheckUpdate(pInfo, pBlockInfo->window.ekey, pInfo->pRes); - // doFilter(pInfo->pRes, pOperator->exprSupp.pFilterInfo, NULL); - // pInfo->pRes->info.dataLoad = 1; - // blockDataUpdateTsWindow(pInfo->pRes, pInfo->primaryTsIndex); - // - // if (pBlockInfo->rows > 0 || pInfo->pUpdateDataRes->info.rows > 0) { - // break; - // } - // } - // - // if (pBlockInfo->rows > 0 || pInfo->pUpdateDataRes->info.rows > 0) { - // break; - // } else { - // continue; - // } - // } - // - // // record the scan action. - // pInfo->numOfExec++; - // pOperator->resultInfo.totalRows += pBlockInfo->rows; - // - // qDebug("stream scan get source rows:%" PRId64", %s", pBlockInfo->rows, id); - // if (pBlockInfo->rows > 0) { - // return pInfo->pRes; - // } - // - // if (pInfo->pUpdateDataRes->info.rows > 0) { - // goto FETCH_NEXT_BLOCK; - // } - // - // goto NEXT_SUBMIT_BLK; - // } else { - // ASSERT(0); - // return NULL; - // } } + return NULL; } @@ -2293,44 +2236,6 @@ static SSDataBlock* doRawScan(SOperatorInfo* pOperator) { return NULL; } - // else if (pTaskInfo->streamInfo.prepareStatus.type == TMQ_OFFSET__LOG) { - // int64_t fetchVer = pTaskInfo->streamInfo.prepareStatus.version + 1; - // - // while(1){ - // if (tqFetchLog(pInfo->tqReader->pWalReader, pInfo->sContext->withMeta, &fetchVer, &pInfo->pCkHead) < 0) { - // qDebug("tmqsnap tmq poll: consumer log end. offset %" PRId64, fetchVer); - // pTaskInfo->streamInfo.lastStatus.version = fetchVer; - // pTaskInfo->streamInfo.lastStatus.type = TMQ_OFFSET__LOG; - // return NULL; - // } - // SWalCont* pHead = &pInfo->pCkHead->head; - // qDebug("tmqsnap tmq poll: consumer log offset %" PRId64 " msgType %d", fetchVer, pHead->msgType); - // - // if (pHead->msgType == TDMT_VND_SUBMIT) { - // SSubmitReq* pCont = (SSubmitReq*)&pHead->body; - // tqReaderSetDataMsg(pInfo->tqReader, pCont, 0); - // SSDataBlock* block = tqLogScanExec(pInfo->sContext->subType, pInfo->tqReader, pInfo->pFilterOutTbUid, - // &pInfo->pRes); if(block){ - // pTaskInfo->streamInfo.lastStatus.type = TMQ_OFFSET__LOG; - // pTaskInfo->streamInfo.lastStatus.version = fetchVer; - // qDebug("tmqsnap fetch data msg, ver:%" PRId64 ", type:%d", pHead->version, pHead->msgType); - // return block; - // }else{ - // fetchVer++; - // } - // } else{ - // ASSERT(pInfo->sContext->withMeta); - // ASSERT(IS_META_MSG(pHead->msgType)); - // qDebug("tmqsnap fetch meta msg, ver:%" PRId64 ", type:%d", pHead->version, pHead->msgType); - // pTaskInfo->streamInfo.metaRsp.rspOffset.version = fetchVer; - // pTaskInfo->streamInfo.metaRsp.rspOffset.type = TMQ_OFFSET__LOG; - // pTaskInfo->streamInfo.metaRsp.resMsgType = pHead->msgType; - // pTaskInfo->streamInfo.metaRsp.metaRspLen = pHead->bodyLen; - // pTaskInfo->streamInfo.metaRsp.metaRsp = taosMemoryMalloc(pHead->bodyLen); - // memcpy(pTaskInfo->streamInfo.metaRsp.metaRsp, pHead->body, pHead->bodyLen); - // return NULL; - // } - // } return NULL; } diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 20abcca197..d2b2c7d840 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -27,6 +27,7 @@ SStreamQueue* streamQueueOpen(int64_t cap) { taosSetQueueCapacity(pQueue->queue, cap); taosSetQueueMemoryCapacity(pQueue->queue, cap * 1024); return pQueue; + FAIL: if (pQueue->queue) taosCloseQueue(pQueue->queue); if (pQueue->qall) taosFreeQall(pQueue->qall); From 35f74ce2e8ffc439eccc574361238065459ce857 Mon Sep 17 00:00:00 2001 From: Minglei Jin Date: Thu, 18 May 2023 18:35:06 +0800 Subject: [PATCH 54/59] cache/schema: not cache schema --- source/dnode/vnode/src/tsdb/tsdbCache.c | 36 ++++++++++--------------- 1 file changed, 14 insertions(+), 22 deletions(-) diff --git a/source/dnode/vnode/src/tsdb/tsdbCache.c b/source/dnode/vnode/src/tsdb/tsdbCache.c index 505bb4b906..845fd2f304 100644 --- a/source/dnode/vnode/src/tsdb/tsdbCache.c +++ b/source/dnode/vnode/src/tsdb/tsdbCache.c @@ -326,20 +326,15 @@ int32_t tsdbCacheUpdate(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSDBROW *pRow int32_t code = 0; // 1, fetch schema - STSchema *pTSchema = pTsdb->rCache.pTSchema; + STSchema *pTSchema = NULL; int32_t sver = TSDBROW_SVERSION(pRow); - if (!pTSchema || sver != pTSchema->version) { - if (pTSchema) { - taosMemoryFree(pTSchema); - } - code = metaGetTbTSchemaEx(pTsdb->pVnode->pMeta, suid, uid, sver, &pTSchema); - if (code != TSDB_CODE_SUCCESS) { - terrno = code; - return -1; - } - pTsdb->rCache.pTSchema = pTSchema; + code = metaGetTbTSchemaEx(pTsdb->pVnode->pMeta, suid, uid, sver, &pTSchema); + if (code != TSDB_CODE_SUCCESS) { + terrno = code; + return -1; } + // 2, iterate col values into array SArray *aColVal = taosArrayInit(32, sizeof(SColVal)); @@ -463,7 +458,7 @@ int32_t tsdbCacheUpdate(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSDBROW *pRow _exit: taosArrayDestroy(aColVal); - // taosMemoryFree(pTSchema); + taosMemoryFree(pTSchema); return code; } @@ -908,15 +903,12 @@ int32_t tsdbCacheGet(STsdb *pTsdb, tb_uid_t uid, SArray *pLastArray, SCacheRowsR int32_t tsdbCacheDel(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSKEY sKey, TSKEY eKey) { int32_t code = 0; // fetch schema - STSchema *pTSchema = pTsdb->rCache.pTSchema; - if (!pTSchema) { - code = metaGetTbTSchemaEx(pTsdb->pVnode->pMeta, suid, uid, -1, &pTSchema); - if (code != TSDB_CODE_SUCCESS) { - terrno = code; - return -1; - } - - pTsdb->rCache.pTSchema = pTSchema; + STSchema *pTSchema = NULL; + int sver = -1; + code = metaGetTbTSchemaEx(pTsdb->pVnode->pMeta, suid, uid, sver, &pTSchema); + if (code != TSDB_CODE_SUCCESS) { + terrno = code; + return -1; } // build keys & multi get from rocks @@ -983,7 +975,7 @@ int32_t tsdbCacheDel(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSKEY sKey, TSKE taosThreadMutexUnlock(&pTsdb->rCache.rMutex); _exit: - // taosMemoryFree(pTSchema); + taosMemoryFree(pTSchema); return code; } From 52e85cb85a156be5d14e93975e95a23b33fe4730 Mon Sep 17 00:00:00 2001 From: chenhaoran Date: Thu, 18 May 2023 18:53:16 +0800 Subject: [PATCH 55/59] test: increase timeouts of checking db ready --- tests/script/tsim/sync/oneReplica5VgElect.sim | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/script/tsim/sync/oneReplica5VgElect.sim b/tests/script/tsim/sync/oneReplica5VgElect.sim index e42ef5fb82..225093ed8f 100644 --- a/tests/script/tsim/sync/oneReplica5VgElect.sim +++ b/tests/script/tsim/sync/oneReplica5VgElect.sim @@ -355,7 +355,7 @@ system sh/exec.sh -n dnode4 -s start $loop_cnt = 0 check_dnode_ready_2: $loop_cnt = $loop_cnt + 1 - sleep 200 + sleep 1500 if $loop_cnt == 10 then print ====> dnode not ready! return -1 From f3bdc855cb2c85c3393fd21ecb7ef8a0aa9fc61d Mon Sep 17 00:00:00 2001 From: wade zhang <95411902+gccgdb1234@users.noreply.github.com> Date: Thu, 18 May 2023 19:09:54 +0800 Subject: [PATCH 56/59] refine an error messsage --- source/util/src/terror.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/util/src/terror.c b/source/util/src/terror.c index 54459329e7..bd3f8657a8 100644 --- a/source/util/src/terror.c +++ b/source/util/src/terror.c @@ -324,7 +324,7 @@ TAOS_DEFINE_ERROR(TSDB_CODE_SNODE_NOT_DEPLOYED, "Snode not deployed") TAOS_DEFINE_ERROR(TSDB_CODE_MNODE_NOT_CATCH_UP, "Mnode didn't catch the leader") TAOS_DEFINE_ERROR(TSDB_CODE_MNODE_ALREADY_IS_VOTER, "Mnode already is a leader") TAOS_DEFINE_ERROR(TSDB_CODE_MNODE_ONLY_TWO_MNODE, "Only two mnodes exist") -TAOS_DEFINE_ERROR(TSDB_CODE_MNODE_NO_NEED_RESTORE, "No need restore on this dnode") +TAOS_DEFINE_ERROR(TSDB_CODE_MNODE_NO_NEED_RESTORE, "No need to restore on this dnode") // vnode TAOS_DEFINE_ERROR(TSDB_CODE_VND_INVALID_VGROUP_ID, "Vnode is closed or removed") From 3cfdbf2f0cef54a06c41c418985ca9db935f5c86 Mon Sep 17 00:00:00 2001 From: gccgdb1234 Date: Fri, 19 May 2023 08:18:29 +0800 Subject: [PATCH 57/59] doc: refine Kafka source connector configuration parameters --- docs/en/20-third-party/11-kafka.md | 13 ++++++++----- docs/zh/20-third-party/11-kafka.md | 8 +++++--- 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/docs/en/20-third-party/11-kafka.md b/docs/en/20-third-party/11-kafka.md index 3b0de6c349..71d8c41173 100644 --- a/docs/en/20-third-party/11-kafka.md +++ b/docs/en/20-third-party/11-kafka.md @@ -424,11 +424,14 @@ The following configuration items apply to TDengine Sink Connector and TDengine ### TDengine Source Connector specific configuration 1. `connection.database`: source database name, no default value. -2. `topic.prefix`: topic name prefix after data is imported into kafka. Use `topic.prefix` + `connection.database` name as the full topic name. Defaults to the empty string "". -3. `timestamp.initial`: Data synchronization start time. The format is 'yyyy-MM-dd HH:mm:ss'. Default "1970-01-01 00:00:00". -4. `poll.interval.ms`: Pull data interval, the unit is ms. Default is 1000. -5. `fetch.max.rows`: The maximum number of rows retrieved when retrieving the database. Default is 100. -6. `out.format`: The data format. The value could be line or json. The line represents the InfluxDB Line protocol format, and json represents the OpenTSDB JSON format. Default is `line`. +2. `topic.prefix`: topic name prefix used when importing data into kafka. Its defaults value is empty string "". +3. `timestamp.initial`: Data synchronization start time. The format is 'yyyy-MM-dd HH:mm:ss'. If it is not set, the data importing to Kafka will be started from the first/oldest row in the database. +4. `poll.interval.ms`: The time interval for checking newly created tables or removed tables, default value is 1000. +5. `fetch.max.rows`: The maximum number of rows retrieved when retrieving the database, default is 100. +6. `out.format`: The data format. The value could be `line`, which represents the InfluxDB Line protocol format. +7. 7. `query.interval.ms`: The time range of reading data from TDengine each time, its unit is millisecond. It should be adjusted according to the data flow in rate, the default value is 1000. +8. `topic.per.stable`: If it's set to true, it means one super table in TDengine corresponds to a topic in Kafka, the topic naming rule is `--`; if it's set to false, it means the whole DB corresponds to a topic in Kafka, the topic naming rule is `-`. + ## Other notes diff --git a/docs/zh/20-third-party/11-kafka.md b/docs/zh/20-third-party/11-kafka.md index 75d8deebb1..2470bf7c9a 100644 --- a/docs/zh/20-third-party/11-kafka.md +++ b/docs/zh/20-third-party/11-kafka.md @@ -435,10 +435,12 @@ confluent local services connect connector unload TDengineSourceConnector 1. `connection.database`: 源数据库名称,无缺省值。 2. `topic.prefix`: 数据导入 kafka 后 topic 名称前缀。 使用 `topic.prefix` + `connection.database` 名称作为完整 topic 名。默认为空字符串 ""。 -3. `timestamp.initial`: 数据同步起始时间。格式为'yyyy-MM-dd HH:mm:ss'。默认为 "1970-01-01 00:00:00"。 -4. `poll.interval.ms`: 拉取数据间隔,单位为 ms。默认为 1000。 +3. `timestamp.initial`: 数据同步起始时间。格式为'yyyy-MM-dd HH:mm:ss',若未指定则从指定 DB 中最早的一条记录开始。 +4. `poll.interval.ms`: 检查是否有新建或删除的表的时间间隔,单位为 ms。默认为 1000。 5. `fetch.max.rows` : 检索数据库时最大检索条数。 默认为 100。 -6. `out.format`: 数据格式。取值 line 或 json。line 表示 InfluxDB Line 协议格式, json 表示 OpenTSDB JSON 格式。默认为 line。 +6. `out.format`: 数据格式。取值为 `line`, 表示 InfluxDB Line 协议格式 +7. `query.interval.ms`: 从 TDengine 一次读取数据的时间跨度,需要根据表中的数据特征合理配置,避免一次查询的数据量过大或过小;在具体的环境中建议通过测试设置一个较优值,默认值为 1000. +8. `topic.per.stable`: 如果设置为true,表示一个超级表对应一个 Kafka topic,topic的命名规则 `--`;如果设置为 false,则指定的 DB 中的所有数据进入一个 Kafka topic,topic 的命名规则为 `-` ## 其他说明 From 84f7c12b8dffa898a7dcbe24085766ef3a4537c9 Mon Sep 17 00:00:00 2001 From: Shuduo Sang Date: Fri, 19 May 2023 09:37:35 +0800 Subject: [PATCH 58/59] docs: fix zh/02-intro (#21380) * docs: fix broken links * docs: fix zh/02-intro.md --- docs/zh/02-intro.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/02-intro.md b/docs/zh/02-intro.md index 525c8aa5c9..888f779966 100644 --- a/docs/zh/02-intro.md +++ b/docs/zh/02-intro.md @@ -92,7 +92,7 @@ TDengine 的主要功能如下: ## 典型适用场景 -作为一个高性能、分布式、支持 SQL 的时序数据库(Database),TDengine 的典型适用场景包括但不限于 IoT、工业互联网、车联网、IT 运维、能源、金融证券等领域。需要指出的是,TDengine 是针对时序数据场景设计的专用数据库和专用大数据处理工具,因其充分利用了时序大数据的特点,它无法用来处理网络爬虫、微博、微信、电商、ERP、CRM 等通用型数据。下面本文将对适用场景做更多详细的分析。 +作为一个高性能、分布式、支持 SQL 的时序数据库(Time-series Database),TDengine 的典型适用场景包括但不限于 IoT、工业互联网、车联网、IT 运维、能源、金融证券等领域。需要指出的是,TDengine 是针对时序数据场景设计的专用数据库和专用大数据处理工具,因其充分利用了时序大数据的特点,它无法用来处理网络爬虫、微博、微信、电商、ERP、CRM 等通用型数据。下面本文将对适用场景做更多详细的分析。 ### 数据源特点和需求 From 6724640003d307d67087e2c2149a477a5f07b7bd Mon Sep 17 00:00:00 2001 From: wade zhang <95411902+gccgdb1234@users.noreply.github.com> Date: Fri, 19 May 2023 10:29:07 +0800 Subject: [PATCH 59/59] Update 11-kafka.md --- docs/zh/20-third-party/11-kafka.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/20-third-party/11-kafka.md b/docs/zh/20-third-party/11-kafka.md index 2470bf7c9a..44ee528d9b 100644 --- a/docs/zh/20-third-party/11-kafka.md +++ b/docs/zh/20-third-party/11-kafka.md @@ -434,7 +434,7 @@ confluent local services connect connector unload TDengineSourceConnector ### TDengine Source Connector 特有的配置 1. `connection.database`: 源数据库名称,无缺省值。 -2. `topic.prefix`: 数据导入 kafka 后 topic 名称前缀。 使用 `topic.prefix` + `connection.database` 名称作为完整 topic 名。默认为空字符串 ""。 +2. `topic.prefix`: 数据导入 kafka 时使用的 topic 名称的前缀。默认为空字符串 ""。 3. `timestamp.initial`: 数据同步起始时间。格式为'yyyy-MM-dd HH:mm:ss',若未指定则从指定 DB 中最早的一条记录开始。 4. `poll.interval.ms`: 检查是否有新建或删除的表的时间间隔,单位为 ms。默认为 1000。 5. `fetch.max.rows` : 检索数据库时最大检索条数。 默认为 100。