diff --git a/docs/zh/06-advanced/03-stream.md b/docs/zh/06-advanced/03-stream.md index a219f86750..7486b3b043 100644 --- a/docs/zh/06-advanced/03-stream.md +++ b/docs/zh/06-advanced/03-stream.md @@ -116,10 +116,11 @@ create stream if not exists count_history_s fill_history 1 into count_history as ### 流计算的触发模式 -在创建流时,可以通过 TRIGGER 指令指定流计算的触发模式。对于非窗口计算,流计算的触发是实时的,对于窗口计算,目前提供 3 种触发模式,默认为 WINDOW_CLOSE。 +在创建流时,可以通过 TRIGGER 指令指定流计算的触发模式。对于非窗口计算,流计算的触发是实时的,对于窗口计算,目前提供 4 种触发模式,默认为 WINDOW_CLOSE。 1. AT_ONCE:写入立即触发。 2. WINDOW_CLOSE:窗口关闭时触发(窗口关闭由事件时间决定,可配合 watermark 使用)。 3. MAX_DELAY time:若窗口关闭,则触发计算。若窗口未关闭,且未关闭时长超过 max delay 指定的时间,则触发计算。 +4. FORCE_WINDOW_CLOSE:以操作系统当前时间为准,只计算当前关闭窗口的结果,并推送出去。窗口只会在被关闭的时刻计算一次,后续不会再重复计算。该模式当前只支持 INTERVAL 窗口(不支持滑动);FILL_HISTORY必须为 0,IGNORE EXPIRED 必须为 1,IGNORE UPDATE 必须为 1;FILL 只支持 PREV 、NULL、 NONE、VALUE。 窗口关闭是由事件时间决定的,如事件流中断、或持续延迟,此时事件时间无法更新,可能导致无法得到最新的计算结果。 diff --git a/docs/zh/14-reference/03-taos-sql/10-function.md b/docs/zh/14-reference/03-taos-sql/10-function.md index 8c882b3237..dd58adfeed 100644 --- a/docs/zh/14-reference/03-taos-sql/10-function.md +++ b/docs/zh/14-reference/03-taos-sql/10-function.md @@ -1569,7 +1569,7 @@ COUNT({* | expr}) ELAPSED(ts_primary_key [, time_unit]) ``` -**功能说明**:elapsed函数表达了统计周期内连续的时间长度,和twa函数配合使用可以计算统计曲线下的面积。在通过INTERVAL子句指定窗口的情况下,统计在给定时间范围内的每个窗口内有数据覆盖的时间范围;如果没有INTERVAL子句,则返回整个给定时间范围内的有数据覆盖的时间范围。注意,ELAPSED返回的并不是时间范围的绝对值,而是绝对值除以time_unit所得到的单位个数。 +**功能说明**:elapsed函数表达了统计周期内连续的时间长度,和twa函数配合使用可以计算统计曲线下的面积。在通过INTERVAL子句指定窗口的情况下,统计在给定时间范围内的每个窗口内有数据覆盖的时间范围;如果没有INTERVAL子句,则返回整个给定时间范围内的有数据覆盖的时间范围。注意,ELAPSED返回的并不是时间范围的绝对值,而是绝对值除以time_unit所得到的单位个数。流计算仅在FORCE_WINDOW_CLOSE模式下支持该函数。 **返回结果类型**:DOUBLE。 @@ -1829,7 +1829,7 @@ ignore_null_values: { - INTERP 用于在指定时间断面获取指定列的记录值,如果该时间断面不存在符合条件的行数据,那么会根据 FILL 参数的设定进行插值。 - INTERP 的输入数据为指定列的数据,可以通过条件语句(where 子句)来对原始列数据进行过滤,如果没有指定过滤条件则输入为全部数据。 -- INTERP 需要同时与 RANGE,EVERY 和 FILL 关键字一起使用。 +- INTERP SQL查询需要同时与 RANGE,EVERY 和 FILL 关键字一起使用;流计算不能使用RANGE,需要EVERY 和 FILL 关键字一起使用。 - INTERP 的输出时间范围根据 RANGE(timestamp1, timestamp2)字段来指定,需满足 timestamp1 \<= timestamp2。其中 timestamp1 为输出时间范围的起始值,即如果 timestamp1 时刻符合插值条件则 timestamp1 为输出的第一条记录,timestamp2 为输出时间范围的结束值,即输出的最后一条记录的 timestamp 不能大于 timestamp2。 - INTERP 根据 EVERY(time_unit) 字段来确定输出时间范围内的结果条数,即从 timestamp1 开始每隔固定长度的时间(time_unit 值)进行插值,time_unit 可取值时间单位:1a(毫秒),1s(秒),1m(分),1h(小时),1d(天),1w(周)。例如 EVERY(500a) 将对于指定数据每500毫秒间隔进行一次插值. - INTERP 根据 FILL 字段来决定在每个符合输出条件的时刻如何进行插值。关于 FILL 子句如何使用请参考 [FILL 子句](../distinguished/#fill-子句) @@ -2180,7 +2180,7 @@ STATEDURATION(expr, oper, val, unit) TWA(expr) ``` -**功能说明**:时间加权平均函数。统计表中某列在一段时间内的时间加权平均。对于存在复合主键的表的查询,若时间戳相同的数据存在多条,则只有对应的复合主键最小的数据参与运算。 +**功能说明**:时间加权平均函数。统计表中某列在一段时间内的时间加权平均。对于存在复合主键的表的查询,若时间戳相同的数据存在多条,则只有对应的复合主键最小的数据参与运算。流计算仅在FORCE_WINDOW_CLOSE模式下支持该函数。 **返回数据类型**:DOUBLE。 diff --git a/docs/zh/14-reference/03-taos-sql/14-stream.md b/docs/zh/14-reference/03-taos-sql/14-stream.md index dc0f404fcf..25a11ecdcb 100644 --- a/docs/zh/14-reference/03-taos-sql/14-stream.md +++ b/docs/zh/14-reference/03-taos-sql/14-stream.md @@ -143,13 +143,14 @@ SELECT * from information_schema.`ins_streams`; 在创建流时,可以通过 TRIGGER 指令指定流式计算的触发模式。 -对于非窗口计算,流式计算的触发是实时的;对于窗口计算,目前提供 3 种触发模式,默认为 WINDOW_CLOSE: +对于非窗口计算,流式计算的触发是实时的;对于窗口计算,目前提供 4 种触发模式,默认为 WINDOW_CLOSE: 1. AT_ONCE:写入立即触发 2. WINDOW_CLOSE:窗口关闭时触发(窗口关闭由事件时间决定,可配合 watermark 使用) 3. MAX_DELAY time:若窗口关闭,则触发计算。若窗口未关闭,且未关闭时长超过 max delay 指定的时间,则触发计算。 +4. FORCE_WINDOW_CLOSE:以操作系统当前时间为准,只计算当前关闭窗口的结果,并推送出去。窗口只会在被关闭的时刻计算一次,后续不会再重复计算。该模式当前只支持 INTERVAL 窗口(不支持滑动);FILL_HISTORY必须为 0,IGNORE EXPIRED 必须为 1,IGNORE UPDATE 必须为 1;FILL 只支持PREV 、NULL、 NONE、VALUE。 由于窗口关闭是由事件时间决定的,如事件流中断、或持续延迟,则事件时间无法更新,可能导致无法得到最新的计算结果。 @@ -248,11 +249,8 @@ T = 最新事件时间 - DELETE_MARK - [percentile](../function/#percentile) - [top](../function/#top) - [bottom](../function/#bottom) -- [elapsed](../function/#elapsed) -- [interp](../function/#interp) - [derivative](../function/#derivative) - [irate](../function/#irate) -- [twa](../function/#twa) - [histogram](../function/#histogram) - [diff](../function/#diff) - [statecount](../function/#statecount) diff --git a/include/common/tcommon.h b/include/common/tcommon.h index 3910ea6745..ea764e6760 100644 --- a/include/common/tcommon.h +++ b/include/common/tcommon.h @@ -154,6 +154,7 @@ typedef enum EStreamType { STREAM_TRANS_STATE, STREAM_MID_RETRIEVE, STREAM_PARTITION_DELETE_DATA, + STREAM_GET_RESULT, } EStreamType; #pragma pack(push, 1) @@ -383,6 +384,10 @@ typedef struct STUidTagInfo { #define TABLE_NAME_COLUMN_INDEX 6 #define PRIMARY_KEY_COLUMN_INDEX 7 +//steam get result block column +#define DATA_TS_COLUMN_INDEX 0 +#define DATA_VERSION_COLUMN_INDEX 1 + // stream create table block column #define UD_TABLE_NAME_COLUMN_INDEX 0 #define UD_GROUPID_COLUMN_INDEX 1 diff --git a/include/common/tmsg.h b/include/common/tmsg.h index c498d691d9..7ff70b243a 100644 --- a/include/common/tmsg.h +++ b/include/common/tmsg.h @@ -467,9 +467,11 @@ typedef enum ENodeType { QUERY_NODE_PHYSICAL_PLAN_MERGE_COUNT, QUERY_NODE_PHYSICAL_PLAN_STREAM_COUNT, QUERY_NODE_PHYSICAL_PLAN_STREAM_MID_INTERVAL, + QUERY_NODE_PHYSICAL_PLAN_STREAM_CONTINUE_INTERVAL, QUERY_NODE_PHYSICAL_PLAN_MERGE_ANOMALY, QUERY_NODE_PHYSICAL_PLAN_STREAM_ANOMALY, QUERY_NODE_PHYSICAL_PLAN_FORECAST_FUNC, + QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERP_FUNC, } ENodeType; typedef struct { @@ -2827,9 +2829,11 @@ typedef struct { int32_t code; } STaskDropRsp; -#define STREAM_TRIGGER_AT_ONCE 1 -#define STREAM_TRIGGER_WINDOW_CLOSE 2 -#define STREAM_TRIGGER_MAX_DELAY 3 +#define STREAM_TRIGGER_AT_ONCE 1 +#define STREAM_TRIGGER_WINDOW_CLOSE 2 +#define STREAM_TRIGGER_MAX_DELAY 3 +#define STREAM_TRIGGER_FORCE_WINDOW_CLOSE 4 + #define STREAM_DEFAULT_IGNORE_EXPIRED 1 #define STREAM_FILL_HISTORY_ON 1 #define STREAM_FILL_HISTORY_OFF 0 diff --git a/include/common/ttokendef.h b/include/common/ttokendef.h index 3b2a0a0094..e123b93f5c 100644 --- a/include/common/ttokendef.h +++ b/include/common/ttokendef.h @@ -16,394 +16,396 @@ #ifndef _TD_COMMON_TOKEN_H_ #define _TD_COMMON_TOKEN_H_ -#define TK_OR 1 -#define TK_AND 2 -#define TK_UNION 3 -#define TK_ALL 4 -#define TK_MINUS 5 -#define TK_EXCEPT 6 -#define TK_INTERSECT 7 -#define TK_NK_BITAND 8 -#define TK_NK_BITOR 9 -#define TK_NK_LSHIFT 10 -#define TK_NK_RSHIFT 11 -#define TK_NK_PLUS 12 -#define TK_NK_MINUS 13 -#define TK_NK_STAR 14 -#define TK_NK_SLASH 15 -#define TK_NK_REM 16 -#define TK_NK_CONCAT 17 -#define TK_CREATE 18 -#define TK_ACCOUNT 19 -#define TK_NK_ID 20 -#define TK_PASS 21 -#define TK_NK_STRING 22 -#define TK_ALTER 23 -#define TK_PPS 24 -#define TK_TSERIES 25 -#define TK_STORAGE 26 -#define TK_STREAMS 27 -#define TK_QTIME 28 -#define TK_DBS 29 -#define TK_USERS 30 -#define TK_CONNS 31 -#define TK_STATE 32 -#define TK_NK_COMMA 33 -#define TK_HOST 34 -#define TK_IS_IMPORT 35 -#define TK_NK_INTEGER 36 -#define TK_CREATEDB 37 -#define TK_USER 38 -#define TK_ENABLE 39 -#define TK_SYSINFO 40 -#define TK_ADD 41 -#define TK_DROP 42 -#define TK_GRANT 43 -#define TK_ON 44 -#define TK_TO 45 -#define TK_REVOKE 46 -#define TK_FROM 47 -#define TK_SUBSCRIBE 48 -#define TK_READ 49 -#define TK_WRITE 50 -#define TK_NK_DOT 51 -#define TK_WITH 52 -#define TK_ENCRYPT_KEY 53 -#define TK_ANODE 54 -#define TK_UPDATE 55 -#define TK_ANODES 56 -#define TK_DNODE 57 -#define TK_PORT 58 -#define TK_DNODES 59 -#define TK_RESTORE 60 -#define TK_NK_IPTOKEN 61 -#define TK_FORCE 62 -#define TK_UNSAFE 63 -#define TK_CLUSTER 64 -#define TK_LOCAL 65 -#define TK_QNODE 66 -#define TK_BNODE 67 -#define TK_SNODE 68 -#define TK_MNODE 69 -#define TK_VNODE 70 -#define TK_DATABASE 71 -#define TK_USE 72 -#define TK_FLUSH 73 -#define TK_TRIM 74 -#define TK_S3MIGRATE 75 -#define TK_COMPACT 76 -#define TK_IF 77 -#define TK_NOT 78 -#define TK_EXISTS 79 -#define TK_BUFFER 80 -#define TK_CACHEMODEL 81 -#define TK_CACHESIZE 82 -#define TK_COMP 83 -#define TK_DURATION 84 -#define TK_NK_VARIABLE 85 -#define TK_MAXROWS 86 -#define TK_MINROWS 87 -#define TK_KEEP 88 -#define TK_PAGES 89 -#define TK_PAGESIZE 90 -#define TK_TSDB_PAGESIZE 91 -#define TK_PRECISION 92 -#define TK_REPLICA 93 -#define TK_VGROUPS 94 -#define TK_SINGLE_STABLE 95 -#define TK_RETENTIONS 96 -#define TK_SCHEMALESS 97 -#define TK_WAL_LEVEL 98 -#define TK_WAL_FSYNC_PERIOD 99 -#define TK_WAL_RETENTION_PERIOD 100 -#define TK_WAL_RETENTION_SIZE 101 -#define TK_WAL_ROLL_PERIOD 102 -#define TK_WAL_SEGMENT_SIZE 103 -#define TK_STT_TRIGGER 104 -#define TK_TABLE_PREFIX 105 -#define TK_TABLE_SUFFIX 106 -#define TK_S3_CHUNKPAGES 107 -#define TK_S3_KEEPLOCAL 108 -#define TK_S3_COMPACT 109 -#define TK_KEEP_TIME_OFFSET 110 -#define TK_ENCRYPT_ALGORITHM 111 -#define TK_NK_COLON 112 -#define TK_BWLIMIT 113 -#define TK_START 114 -#define TK_TIMESTAMP 115 -#define TK_END 116 -#define TK_TABLE 117 -#define TK_NK_LP 118 -#define TK_NK_RP 119 -#define TK_USING 120 -#define TK_FILE 121 -#define TK_STABLE 122 -#define TK_COLUMN 123 -#define TK_MODIFY 124 -#define TK_RENAME 125 -#define TK_TAG 126 -#define TK_SET 127 -#define TK_NK_EQ 128 -#define TK_TAGS 129 -#define TK_BOOL 130 -#define TK_TINYINT 131 -#define TK_SMALLINT 132 -#define TK_INT 133 -#define TK_INTEGER 134 -#define TK_BIGINT 135 -#define TK_FLOAT 136 -#define TK_DOUBLE 137 -#define TK_BINARY 138 -#define TK_NCHAR 139 -#define TK_UNSIGNED 140 -#define TK_JSON 141 -#define TK_VARCHAR 142 -#define TK_MEDIUMBLOB 143 -#define TK_BLOB 144 -#define TK_VARBINARY 145 -#define TK_GEOMETRY 146 -#define TK_DECIMAL 147 -#define TK_COMMENT 148 -#define TK_MAX_DELAY 149 -#define TK_WATERMARK 150 -#define TK_ROLLUP 151 -#define TK_TTL 152 -#define TK_SMA 153 -#define TK_DELETE_MARK 154 -#define TK_FIRST 155 -#define TK_LAST 156 -#define TK_SHOW 157 -#define TK_FULL 158 -#define TK_PRIVILEGES 159 -#define TK_DATABASES 160 -#define TK_TABLES 161 -#define TK_STABLES 162 -#define TK_MNODES 163 -#define TK_QNODES 164 -#define TK_ARBGROUPS 165 -#define TK_FUNCTIONS 166 -#define TK_INDEXES 167 -#define TK_ACCOUNTS 168 -#define TK_APPS 169 -#define TK_CONNECTIONS 170 -#define TK_LICENCES 171 -#define TK_GRANTS 172 -#define TK_LOGS 173 -#define TK_MACHINES 174 -#define TK_ENCRYPTIONS 175 -#define TK_QUERIES 176 -#define TK_SCORES 177 -#define TK_TOPICS 178 -#define TK_VARIABLES 179 -#define TK_BNODES 180 -#define TK_SNODES 181 -#define TK_TRANSACTIONS 182 -#define TK_DISTRIBUTED 183 -#define TK_CONSUMERS 184 -#define TK_SUBSCRIPTIONS 185 -#define TK_VNODES 186 -#define TK_ALIVE 187 -#define TK_VIEWS 188 -#define TK_VIEW 189 -#define TK_COMPACTS 190 -#define TK_NORMAL 191 -#define TK_CHILD 192 -#define TK_LIKE 193 -#define TK_TBNAME 194 -#define TK_QTAGS 195 -#define TK_AS 196 -#define TK_SYSTEM 197 -#define TK_TSMA 198 -#define TK_INTERVAL 199 -#define TK_RECURSIVE 200 -#define TK_TSMAS 201 -#define TK_FUNCTION 202 -#define TK_INDEX 203 -#define TK_COUNT 204 -#define TK_LAST_ROW 205 -#define TK_META 206 -#define TK_ONLY 207 -#define TK_TOPIC 208 -#define TK_CONSUMER 209 -#define TK_GROUP 210 -#define TK_DESC 211 -#define TK_DESCRIBE 212 -#define TK_RESET 213 -#define TK_QUERY 214 -#define TK_CACHE 215 -#define TK_EXPLAIN 216 -#define TK_ANALYZE 217 -#define TK_VERBOSE 218 -#define TK_NK_BOOL 219 -#define TK_RATIO 220 -#define TK_NK_FLOAT 221 -#define TK_OUTPUTTYPE 222 -#define TK_AGGREGATE 223 -#define TK_BUFSIZE 224 -#define TK_LANGUAGE 225 -#define TK_REPLACE 226 -#define TK_STREAM 227 -#define TK_INTO 228 -#define TK_PAUSE 229 -#define TK_RESUME 230 -#define TK_PRIMARY 231 -#define TK_KEY 232 -#define TK_TRIGGER 233 -#define TK_AT_ONCE 234 -#define TK_WINDOW_CLOSE 235 -#define TK_IGNORE 236 -#define TK_EXPIRED 237 -#define TK_FILL_HISTORY 238 -#define TK_SUBTABLE 239 -#define TK_UNTREATED 240 -#define TK_KILL 241 -#define TK_CONNECTION 242 -#define TK_TRANSACTION 243 -#define TK_BALANCE 244 -#define TK_VGROUP 245 -#define TK_LEADER 246 -#define TK_MERGE 247 -#define TK_REDISTRIBUTE 248 -#define TK_SPLIT 249 -#define TK_DELETE 250 -#define TK_INSERT 251 -#define TK_NK_BIN 252 -#define TK_NK_HEX 253 -#define TK_NULL 254 -#define TK_NK_QUESTION 255 -#define TK_NK_ALIAS 256 -#define TK_NK_ARROW 257 -#define TK_ROWTS 258 -#define TK_QSTART 259 -#define TK_QEND 260 -#define TK_QDURATION 261 -#define TK_WSTART 262 -#define TK_WEND 263 -#define TK_WDURATION 264 -#define TK_IROWTS 265 -#define TK_ISFILLED 266 -#define TK_FLOW 267 -#define TK_FHIGH 268 -#define TK_FROWTS 269 -#define TK_CAST 270 -#define TK_POSITION 271 -#define TK_IN 272 -#define TK_FOR 273 -#define TK_NOW 274 -#define TK_TODAY 275 -#define TK_RAND 276 -#define TK_SUBSTR 277 -#define TK_SUBSTRING 278 -#define TK_BOTH 279 -#define TK_TRAILING 280 -#define TK_LEADING 281 -#define TK_TIMEZONE 282 -#define TK_CLIENT_VERSION 283 -#define TK_SERVER_VERSION 284 -#define TK_SERVER_STATUS 285 -#define TK_CURRENT_USER 286 -#define TK_PI 287 -#define TK_CASE 288 -#define TK_WHEN 289 -#define TK_THEN 290 -#define TK_ELSE 291 -#define TK_BETWEEN 292 -#define TK_IS 293 -#define TK_NK_LT 294 -#define TK_NK_GT 295 -#define TK_NK_LE 296 -#define TK_NK_GE 297 -#define TK_NK_NE 298 -#define TK_MATCH 299 -#define TK_NMATCH 300 -#define TK_CONTAINS 301 -#define TK_JOIN 302 -#define TK_INNER 303 -#define TK_LEFT 304 -#define TK_RIGHT 305 -#define TK_OUTER 306 -#define TK_SEMI 307 -#define TK_ANTI 308 -#define TK_ASOF 309 -#define TK_WINDOW 310 -#define TK_WINDOW_OFFSET 311 -#define TK_JLIMIT 312 -#define TK_SELECT 313 -#define TK_NK_HINT 314 -#define TK_DISTINCT 315 -#define TK_WHERE 316 -#define TK_PARTITION 317 -#define TK_BY 318 -#define TK_SESSION 319 -#define TK_STATE_WINDOW 320 -#define TK_EVENT_WINDOW 321 -#define TK_COUNT_WINDOW 322 -#define TK_ANOMALY_WINDOW 323 -#define TK_SLIDING 324 -#define TK_FILL 325 -#define TK_VALUE 326 -#define TK_VALUE_F 327 -#define TK_NONE 328 -#define TK_PREV 329 -#define TK_NULL_F 330 -#define TK_LINEAR 331 -#define TK_NEXT 332 -#define TK_HAVING 333 -#define TK_RANGE 334 -#define TK_EVERY 335 -#define TK_ORDER 336 -#define TK_SLIMIT 337 -#define TK_SOFFSET 338 -#define TK_LIMIT 339 -#define TK_OFFSET 340 -#define TK_ASC 341 -#define TK_NULLS 342 -#define TK_ABORT 343 -#define TK_AFTER 344 -#define TK_ATTACH 345 -#define TK_BEFORE 346 -#define TK_BEGIN 347 -#define TK_BITAND 348 -#define TK_BITNOT 349 -#define TK_BITOR 350 -#define TK_BLOCKS 351 -#define TK_CHANGE 352 -#define TK_COMMA 353 -#define TK_CONCAT 354 -#define TK_CONFLICT 355 -#define TK_COPY 356 -#define TK_DEFERRED 357 -#define TK_DELIMITERS 358 -#define TK_DETACH 359 -#define TK_DIVIDE 360 -#define TK_DOT 361 -#define TK_EACH 362 -#define TK_FAIL 363 -#define TK_GLOB 364 -#define TK_ID 365 -#define TK_IMMEDIATE 366 -#define TK_IMPORT 367 -#define TK_INITIALLY 368 -#define TK_INSTEAD 369 -#define TK_ISNULL 370 -#define TK_MODULES 371 -#define TK_NK_BITNOT 372 -#define TK_NK_SEMI 373 -#define TK_NOTNULL 374 -#define TK_OF 375 -#define TK_PLUS 376 -#define TK_PRIVILEGE 377 -#define TK_RAISE 378 -#define TK_RESTRICT 379 -#define TK_ROW 380 -#define TK_STAR 381 -#define TK_STATEMENT 382 -#define TK_STRICT 383 -#define TK_STRING 384 -#define TK_TIMES 385 -#define TK_VALUES 386 -#define TK_VARIABLE 387 -#define TK_WAL 388 +#define TK_OR 1 +#define TK_AND 2 +#define TK_UNION 3 +#define TK_ALL 4 +#define TK_MINUS 5 +#define TK_EXCEPT 6 +#define TK_INTERSECT 7 +#define TK_NK_BITAND 8 +#define TK_NK_BITOR 9 +#define TK_NK_LSHIFT 10 +#define TK_NK_RSHIFT 11 +#define TK_NK_PLUS 12 +#define TK_NK_MINUS 13 +#define TK_NK_STAR 14 +#define TK_NK_SLASH 15 +#define TK_NK_REM 16 +#define TK_NK_CONCAT 17 +#define TK_CREATE 18 +#define TK_ACCOUNT 19 +#define TK_NK_ID 20 +#define TK_PASS 21 +#define TK_NK_STRING 22 +#define TK_ALTER 23 +#define TK_PPS 24 +#define TK_TSERIES 25 +#define TK_STORAGE 26 +#define TK_STREAMS 27 +#define TK_QTIME 28 +#define TK_DBS 29 +#define TK_USERS 30 +#define TK_CONNS 31 +#define TK_STATE 32 +#define TK_NK_COMMA 33 +#define TK_HOST 34 +#define TK_IS_IMPORT 35 +#define TK_NK_INTEGER 36 +#define TK_CREATEDB 37 +#define TK_USER 38 +#define TK_ENABLE 39 +#define TK_SYSINFO 40 +#define TK_ADD 41 +#define TK_DROP 42 +#define TK_GRANT 43 +#define TK_ON 44 +#define TK_TO 45 +#define TK_REVOKE 46 +#define TK_FROM 47 +#define TK_SUBSCRIBE 48 +#define TK_READ 49 +#define TK_WRITE 50 +#define TK_NK_DOT 51 +#define TK_WITH 52 +#define TK_ENCRYPT_KEY 53 +#define TK_ANODE 54 +#define TK_UPDATE 55 +#define TK_ANODES 56 +#define TK_DNODE 57 +#define TK_PORT 58 +#define TK_DNODES 59 +#define TK_RESTORE 60 +#define TK_NK_IPTOKEN 61 +#define TK_FORCE 62 +#define TK_UNSAFE 63 +#define TK_CLUSTER 64 +#define TK_LOCAL 65 +#define TK_QNODE 66 +#define TK_BNODE 67 +#define TK_SNODE 68 +#define TK_MNODE 69 +#define TK_VNODE 70 +#define TK_DATABASE 71 +#define TK_USE 72 +#define TK_FLUSH 73 +#define TK_TRIM 74 +#define TK_S3MIGRATE 75 +#define TK_COMPACT 76 +#define TK_IF 77 +#define TK_NOT 78 +#define TK_EXISTS 79 +#define TK_BUFFER 80 +#define TK_CACHEMODEL 81 +#define TK_CACHESIZE 82 +#define TK_COMP 83 +#define TK_DURATION 84 +#define TK_NK_VARIABLE 85 +#define TK_MAXROWS 86 +#define TK_MINROWS 87 +#define TK_KEEP 88 +#define TK_PAGES 89 +#define TK_PAGESIZE 90 +#define TK_TSDB_PAGESIZE 91 +#define TK_PRECISION 92 +#define TK_REPLICA 93 +#define TK_VGROUPS 94 +#define TK_SINGLE_STABLE 95 +#define TK_RETENTIONS 96 +#define TK_SCHEMALESS 97 +#define TK_WAL_LEVEL 98 +#define TK_WAL_FSYNC_PERIOD 99 +#define TK_WAL_RETENTION_PERIOD 100 +#define TK_WAL_RETENTION_SIZE 101 +#define TK_WAL_ROLL_PERIOD 102 +#define TK_WAL_SEGMENT_SIZE 103 +#define TK_STT_TRIGGER 104 +#define TK_TABLE_PREFIX 105 +#define TK_TABLE_SUFFIX 106 +#define TK_S3_CHUNKPAGES 107 +#define TK_S3_KEEPLOCAL 108 +#define TK_S3_COMPACT 109 +#define TK_KEEP_TIME_OFFSET 110 +#define TK_ENCRYPT_ALGORITHM 111 +#define TK_NK_COLON 112 +#define TK_BWLIMIT 113 +#define TK_START 114 +#define TK_TIMESTAMP 115 +#define TK_END 116 +#define TK_TABLE 117 +#define TK_NK_LP 118 +#define TK_NK_RP 119 +#define TK_USING 120 +#define TK_FILE 121 +#define TK_STABLE 122 +#define TK_COLUMN 123 +#define TK_MODIFY 124 +#define TK_RENAME 125 +#define TK_TAG 126 +#define TK_SET 127 +#define TK_NK_EQ 128 +#define TK_TAGS 129 +#define TK_BOOL 130 +#define TK_TINYINT 131 +#define TK_SMALLINT 132 +#define TK_INT 133 +#define TK_INTEGER 134 +#define TK_BIGINT 135 +#define TK_FLOAT 136 +#define TK_DOUBLE 137 +#define TK_BINARY 138 +#define TK_NCHAR 139 +#define TK_UNSIGNED 140 +#define TK_JSON 141 +#define TK_VARCHAR 142 +#define TK_MEDIUMBLOB 143 +#define TK_BLOB 144 +#define TK_VARBINARY 145 +#define TK_GEOMETRY 146 +#define TK_DECIMAL 147 +#define TK_COMMENT 148 +#define TK_MAX_DELAY 149 +#define TK_WATERMARK 150 +#define TK_ROLLUP 151 +#define TK_TTL 152 +#define TK_SMA 153 +#define TK_DELETE_MARK 154 +#define TK_FIRST 155 +#define TK_LAST 156 +#define TK_SHOW 157 +#define TK_FULL 158 +#define TK_PRIVILEGES 159 +#define TK_DATABASES 160 +#define TK_TABLES 161 +#define TK_STABLES 162 +#define TK_MNODES 163 +#define TK_QNODES 164 +#define TK_ARBGROUPS 165 +#define TK_FUNCTIONS 166 +#define TK_INDEXES 167 +#define TK_ACCOUNTS 168 +#define TK_APPS 169 +#define TK_CONNECTIONS 170 +#define TK_LICENCES 171 +#define TK_GRANTS 172 +#define TK_LOGS 173 +#define TK_MACHINES 174 +#define TK_ENCRYPTIONS 175 +#define TK_QUERIES 176 +#define TK_SCORES 177 +#define TK_TOPICS 178 +#define TK_VARIABLES 179 +#define TK_BNODES 180 +#define TK_SNODES 181 +#define TK_TRANSACTIONS 182 +#define TK_DISTRIBUTED 183 +#define TK_CONSUMERS 184 +#define TK_SUBSCRIPTIONS 185 +#define TK_VNODES 186 +#define TK_ALIVE 187 +#define TK_VIEWS 188 +#define TK_VIEW 189 +#define TK_COMPACTS 190 +#define TK_NORMAL 191 +#define TK_CHILD 192 +#define TK_LIKE 193 +#define TK_TBNAME 194 +#define TK_QTAGS 195 +#define TK_AS 196 +#define TK_SYSTEM 197 +#define TK_TSMA 198 +#define TK_INTERVAL 199 +#define TK_RECURSIVE 200 +#define TK_TSMAS 201 +#define TK_FUNCTION 202 +#define TK_INDEX 203 +#define TK_COUNT 204 +#define TK_LAST_ROW 205 +#define TK_META 206 +#define TK_ONLY 207 +#define TK_TOPIC 208 +#define TK_CONSUMER 209 +#define TK_GROUP 210 +#define TK_DESC 211 +#define TK_DESCRIBE 212 +#define TK_RESET 213 +#define TK_QUERY 214 +#define TK_CACHE 215 +#define TK_EXPLAIN 216 +#define TK_ANALYZE 217 +#define TK_VERBOSE 218 +#define TK_NK_BOOL 219 +#define TK_RATIO 220 +#define TK_NK_FLOAT 221 +#define TK_OUTPUTTYPE 222 +#define TK_AGGREGATE 223 +#define TK_BUFSIZE 224 +#define TK_LANGUAGE 225 +#define TK_REPLACE 226 +#define TK_STREAM 227 +#define TK_INTO 228 +#define TK_PAUSE 229 +#define TK_RESUME 230 +#define TK_PRIMARY 231 +#define TK_KEY 232 +#define TK_TRIGGER 233 +#define TK_AT_ONCE 234 +#define TK_WINDOW_CLOSE 235 +#define TK_FORCE_WINDOW_CLOSE 236 +#define TK_IGNORE 237 +#define TK_EXPIRED 238 +#define TK_FILL_HISTORY 239 +#define TK_SUBTABLE 240 +#define TK_UNTREATED 241 +#define TK_KILL 242 +#define TK_CONNECTION 243 +#define TK_TRANSACTION 244 +#define TK_BALANCE 245 +#define TK_VGROUP 246 +#define TK_LEADER 247 +#define TK_MERGE 248 +#define TK_REDISTRIBUTE 249 +#define TK_SPLIT 250 +#define TK_DELETE 251 +#define TK_INSERT 252 +#define TK_NK_BIN 253 +#define TK_NK_HEX 254 +#define TK_NULL 255 +#define TK_NK_QUESTION 256 +#define TK_NK_ALIAS 257 +#define TK_NK_ARROW 258 +#define TK_ROWTS 259 +#define TK_QSTART 260 +#define TK_QEND 261 +#define TK_QDURATION 262 +#define TK_WSTART 263 +#define TK_WEND 264 +#define TK_WDURATION 265 +#define TK_IROWTS 266 +#define TK_ISFILLED 267 +#define TK_FLOW 268 +#define TK_FHIGH 269 +#define TK_FROWTS 270 +#define TK_CAST 271 +#define TK_POSITION 272 +#define TK_IN 273 +#define TK_FOR 274 +#define TK_NOW 275 +#define TK_TODAY 276 +#define TK_RAND 277 +#define TK_SUBSTR 278 +#define TK_SUBSTRING 279 +#define TK_BOTH 280 +#define TK_TRAILING 281 +#define TK_LEADING 282 +#define TK_TIMEZONE 283 +#define TK_CLIENT_VERSION 284 +#define TK_SERVER_VERSION 285 +#define TK_SERVER_STATUS 286 +#define TK_CURRENT_USER 287 +#define TK_PI 288 +#define TK_CASE 289 +#define TK_WHEN 290 +#define TK_THEN 291 +#define TK_ELSE 292 +#define TK_BETWEEN 293 +#define TK_IS 294 +#define TK_NK_LT 295 +#define TK_NK_GT 296 +#define TK_NK_LE 297 +#define TK_NK_GE 298 +#define TK_NK_NE 299 +#define TK_MATCH 300 +#define TK_NMATCH 301 +#define TK_CONTAINS 302 +#define TK_JOIN 303 +#define TK_INNER 304 +#define TK_LEFT 305 +#define TK_RIGHT 306 +#define TK_OUTER 307 +#define TK_SEMI 308 +#define TK_ANTI 309 +#define TK_ASOF 310 +#define TK_WINDOW 311 +#define TK_WINDOW_OFFSET 312 +#define TK_JLIMIT 313 +#define TK_SELECT 314 +#define TK_NK_HINT 315 +#define TK_DISTINCT 316 +#define TK_WHERE 317 +#define TK_PARTITION 318 +#define TK_BY 319 +#define TK_SESSION 320 +#define TK_STATE_WINDOW 321 +#define TK_EVENT_WINDOW 322 +#define TK_COUNT_WINDOW 323 +#define TK_ANOMALY_WINDOW 324 +#define TK_SLIDING 325 +#define TK_FILL 326 +#define TK_VALUE 327 +#define TK_VALUE_F 328 +#define TK_NONE 329 +#define TK_PREV 330 +#define TK_NULL_F 331 +#define TK_LINEAR 332 +#define TK_NEXT 333 +#define TK_HAVING 334 +#define TK_RANGE 335 +#define TK_EVERY 336 +#define TK_ORDER 337 +#define TK_SLIMIT 338 +#define TK_SOFFSET 339 +#define TK_LIMIT 340 +#define TK_OFFSET 341 +#define TK_ASC 342 +#define TK_NULLS 343 +#define TK_ABORT 344 +#define TK_AFTER 345 +#define TK_ATTACH 346 +#define TK_BEFORE 347 +#define TK_BEGIN 348 +#define TK_BITAND 349 +#define TK_BITNOT 350 +#define TK_BITOR 351 +#define TK_BLOCKS 352 +#define TK_CHANGE 353 +#define TK_COMMA 354 +#define TK_CONCAT 355 +#define TK_CONFLICT 356 +#define TK_COPY 357 +#define TK_DEFERRED 358 +#define TK_DELIMITERS 359 +#define TK_DETACH 360 +#define TK_DIVIDE 361 +#define TK_DOT 362 +#define TK_EACH 363 +#define TK_FAIL 364 +#define TK_GLOB 365 +#define TK_ID 366 +#define TK_IMMEDIATE 367 +#define TK_IMPORT 368 +#define TK_INITIALLY 369 +#define TK_INSTEAD 370 +#define TK_ISNULL 371 +#define TK_MODULES 372 +#define TK_NK_BITNOT 373 +#define TK_NK_SEMI 374 +#define TK_NOTNULL 375 +#define TK_OF 376 +#define TK_PLUS 377 +#define TK_PRIVILEGE 378 +#define TK_RAISE 379 +#define TK_RESTRICT 380 +#define TK_ROW 381 +#define TK_STAR 382 +#define TK_STATEMENT 383 +#define TK_STRICT 384 +#define TK_STRING 385 +#define TK_TIMES 386 +#define TK_VALUES 387 +#define TK_VARIABLE 388 +#define TK_WAL 389 + #define TK_NK_SPACE 600 #define TK_NK_COMMENT 601 diff --git a/include/libs/executor/executor.h b/include/libs/executor/executor.h index 1c04da08be..d955a7b3b9 100644 --- a/include/libs/executor/executor.h +++ b/include/libs/executor/executor.h @@ -222,8 +222,8 @@ int32_t qStreamSourceScanParamForHistoryScanStep2(qTaskInfo_t tinfo, SVersionRan int32_t qStreamRecoverFinish(qTaskInfo_t tinfo); bool qStreamScanhistoryFinished(qTaskInfo_t tinfo); int32_t qStreamInfoResetTimewindowFilter(qTaskInfo_t tinfo); -void resetTaskInfo(qTaskInfo_t tinfo); - +void qResetTaskInfoCode(qTaskInfo_t tinfo); +int32_t qGetStreamIntervalExecInfo(qTaskInfo_t tinfo, int64_t* pWaterMark, SInterval* pInterval, STimeWindow* pLastWindow); int32_t qStreamOperatorReleaseState(qTaskInfo_t tInfo); int32_t qStreamOperatorReloadState(qTaskInfo_t tInfo); diff --git a/include/libs/executor/storageapi.h b/include/libs/executor/storageapi.h index 8e88a1a278..db0d6339c8 100644 --- a/include/libs/executor/storageapi.h +++ b/include/libs/executor/storageapi.h @@ -39,8 +39,10 @@ extern "C" { #define META_READER_LOCK 0x0 #define META_READER_NOLOCK 0x1 -#define STREAM_STATE_BUFF_HASH 1 -#define STREAM_STATE_BUFF_SORT 2 +#define STREAM_STATE_BUFF_HASH 1 +#define STREAM_STATE_BUFF_SORT 2 +#define STREAM_STATE_BUFF_HASH_SORT 3 +#define STREAM_STATE_BUFF_HASH_SEARCH 4 typedef struct SMeta SMeta; typedef TSKEY (*GetTsFun)(void*); @@ -325,6 +327,9 @@ typedef struct { int64_t number; void* pStreamFileState; int32_t buffIndex; + int32_t hashIter; + void* pHashData; + int64_t minGpId; } SStreamStateCur; typedef struct SStateStore { @@ -337,6 +342,8 @@ typedef struct SStateStore { void (*streamStateReleaseBuf)(SStreamState* pState, void* pVal, bool used); void (*streamStateClearBuff)(SStreamState* pState, void* pVal); void (*streamStateFreeVal)(void* val); + int32_t (*streamStateGetPrev)(SStreamState* pState, const SWinKey* pKey, SWinKey* pResKey, void** pVal, + int32_t* pVLen, int32_t* pWinCode); 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, int32_t* pWinCode); @@ -349,8 +356,15 @@ typedef struct SStateStore { int32_t (*streamStateGetInfo)(SStreamState* pState, void* pKey, int32_t keyLen, void** pVal, int32_t* pLen); int32_t (*streamStateFillPut)(SStreamState* pState, const SWinKey* key, const void* value, int32_t vLen); - int32_t (*streamStateFillGet)(SStreamState* pState, const SWinKey* key, void** pVal, int32_t* pVLen); + int32_t (*streamStateFillGet)(SStreamState* pState, const SWinKey* key, void** pVal, int32_t* pVLen, + int32_t* pWinCode); + int32_t (*streamStateFillAddIfNotExist)(SStreamState* pState, const SWinKey* key, void** pVal, int32_t* pVLen, + int32_t* pWinCode); void (*streamStateFillDel)(SStreamState* pState, const SWinKey* key); + int32_t (*streamStateFillGetNext)(SStreamState* pState, const SWinKey* pKey, SWinKey* pResKey, void** pVal, + int32_t* pVLen, int32_t* pWinCode); + int32_t (*streamStateFillGetPrev)(SStreamState* pState, const SWinKey* pKey, SWinKey* pResKey, void** pVal, + int32_t* pVLen, int32_t* pWinCode); void (*streamStateCurNext)(SStreamState* pState, SStreamStateCur* pCur); void (*streamStateCurPrev)(SStreamState* pState, SStreamStateCur* pCur); @@ -361,9 +375,12 @@ typedef struct SStateStore { SStreamStateCur* (*streamStateFillSeekKeyPrev)(SStreamState* pState, const SWinKey* key); void (*streamStateFreeCur)(SStreamStateCur* pCur); - int32_t (*streamStateGetGroupKVByCur)(SStreamStateCur* pCur, SWinKey* pKey, const void** pVal, int32_t* pVLen); + int32_t (*streamStateFillGetGroupKVByCur)(SStreamStateCur* pCur, SWinKey* pKey, const void** pVal, int32_t* pVLen); int32_t (*streamStateGetKVByCur)(SStreamStateCur* pCur, SWinKey* pKey, const void** pVal, int32_t* pVLen); + void (*streamStateSetFillInfo)(SStreamState* pState); + void (*streamStateClearExpiredState)(SStreamState* pState); + int32_t (*streamStateSessionAddIfNotExist)(SStreamState* pState, SSessionKey* key, TSKEY gap, void** pVal, int32_t* pVLen, int32_t* pWinCode); int32_t (*streamStateSessionPut)(SStreamState* pState, const SSessionKey* key, void* value, int32_t vLen); @@ -400,8 +417,8 @@ typedef struct SStateStore { SUpdateInfo** ppInfo); void (*updateInfoAddCloseWindowSBF)(SUpdateInfo* pInfo); void (*updateInfoDestoryColseWinSBF)(SUpdateInfo* pInfo); - int32_t (*updateInfoSerialize)(void* buf, int32_t bufLen, const SUpdateInfo* pInfo, int32_t* pLen); - int32_t (*updateInfoDeserialize)(void* buf, int32_t bufLen, SUpdateInfo* pInfo); + int32_t (*updateInfoSerialize)(SEncoder* pEncoder, const SUpdateInfo* pInfo); + int32_t (*updateInfoDeserialize)(SDecoder* pDeCoder, SUpdateInfo* pInfo); SStreamStateCur* (*streamStateSessionSeekKeyNext)(SStreamState* pState, const SSessionKey* key); SStreamStateCur* (*streamStateCountSeekKeyPrev)(SStreamState* pState, const SSessionKey* pKey, COUNT_TYPE count); @@ -411,6 +428,11 @@ typedef struct SStateStore { int32_t (*streamFileStateInit)(int64_t memSize, uint32_t keySize, uint32_t rowSize, uint32_t selectRowSize, GetTsFun fp, void* pFile, TSKEY delMark, const char* id, int64_t ckId, int8_t type, struct SStreamFileState** ppFileState); + + int32_t (*streamStateGroupPut)(SStreamState* pState, int64_t groupId, void* value, int32_t vLen); + SStreamStateCur* (*streamStateGroupGetCur)(SStreamState* pState); + void (*streamStateGroupCurNext)(SStreamStateCur* pCur); + int32_t (*streamStateGroupGetKVByCur)(SStreamStateCur* pCur, int64_t* pKey, void** pVal, int32_t* pVLen); void (*streamFileStateDestroy)(struct SStreamFileState* pFileState); void (*streamFileStateClear)(struct SStreamFileState* pFileState); diff --git a/include/libs/nodes/plannodes.h b/include/libs/nodes/plannodes.h index 6d09ef6d88..cfd9c1a422 100644 --- a/include/libs/nodes/plannodes.h +++ b/include/libs/nodes/plannodes.h @@ -194,14 +194,26 @@ typedef struct SIndefRowsFuncLogicNode { bool isTimeLineFunc; } SIndefRowsFuncLogicNode; +typedef struct SStreamNodeOption { + int8_t triggerType; + int64_t watermark; + int64_t deleteMark; + int8_t igExpired; + int8_t igCheckUpdate; + int8_t destHasPrimaryKey; +} SStreamNodeOption; + typedef struct SInterpFuncLogicNode { - SLogicNode node; - SNodeList* pFuncs; - STimeWindow timeRange; - int64_t interval; - EFillMode fillMode; - SNode* pFillValues; // SNodeListNode - SNode* pTimeSeries; // SColumnNode + SLogicNode node; + SNodeList* pFuncs; + STimeWindow timeRange; + int64_t interval; + int8_t intervalUnit; + int8_t precision; + EFillMode fillMode; + SNode* pFillValues; // SNodeListNode + SNode* pTimeSeries; // SColumnNode + SStreamNodeOption streamNodeOption; } SInterpFuncLogicNode; typedef struct SForecastFuncLogicNode { @@ -505,17 +517,21 @@ typedef struct SIndefRowsFuncPhysiNode { } SIndefRowsFuncPhysiNode; typedef struct SInterpFuncPhysiNode { - SPhysiNode node; - SNodeList* pExprs; - SNodeList* pFuncs; - STimeWindow timeRange; - int64_t interval; - int8_t intervalUnit; - EFillMode fillMode; - SNode* pFillValues; // SNodeListNode - SNode* pTimeSeries; // SColumnNode + SPhysiNode node; + SNodeList* pExprs; + SNodeList* pFuncs; + STimeWindow timeRange; + int64_t interval; + int8_t intervalUnit; + int8_t precision; + EFillMode fillMode; + SNode* pFillValues; // SNodeListNode + SNode* pTimeSeries; // SColumnNode + SStreamNodeOption streamNodeOption; } SInterpFuncPhysiNode; +typedef SInterpFuncPhysiNode SStreamInterpFuncPhysiNode; + typedef struct SForecastFuncPhysiNode { SPhysiNode node; SNodeList* pExprs; @@ -650,7 +666,7 @@ typedef struct SWindowPhysiNode { int64_t watermark; int64_t deleteMark; int8_t igExpired; - int8_t destHasPrimayKey; + int8_t destHasPrimaryKey; bool mergeDataBlock; } SWindowPhysiNode; diff --git a/include/libs/nodes/querynodes.h b/include/libs/nodes/querynodes.h index 4763077ed9..763882ab3a 100644 --- a/include/libs/nodes/querynodes.h +++ b/include/libs/nodes/querynodes.h @@ -457,6 +457,7 @@ typedef struct SSelectStmt { bool hasCountFunc; bool hasUdaf; bool hasStateKey; + bool hasTwaOrElapsedFunc; bool onlyHasKeepOrderFunc; bool groupSort; bool tagScan; diff --git a/include/libs/stream/streamState.h b/include/libs/stream/streamState.h index f9469a449d..a50451c3eb 100644 --- a/include/libs/stream/streamState.h +++ b/include/libs/stream/streamState.h @@ -49,6 +49,8 @@ void streamStateClear(SStreamState* pState); void streamStateSetNumber(SStreamState* pState, int32_t number, int32_t tsIdex); void streamStateSaveInfo(SStreamState* pState, void* pKey, int32_t keyLen, void* pVal, int32_t vLen); int32_t streamStateGetInfo(SStreamState* pState, void* pKey, int32_t keyLen, void** pVal, int32_t* pLen); +int32_t streamStateGetPrev(SStreamState* pState, const SWinKey* pKey, SWinKey* pResKey, void** pVal, int32_t* pVLen, + int32_t* pWinCode); // session window int32_t streamStateSessionAddIfNotExist(SStreamState* pState, SSessionKey* key, TSKEY gap, void** pVal, int32_t* pVLen, @@ -75,8 +77,14 @@ int32_t streamStateStateAddIfNotExist(SStreamState* pState, SSessionKey* key, ch // fill int32_t streamStateFillPut(SStreamState* pState, const SWinKey* key, const void* value, int32_t vLen); -int32_t streamStateFillGet(SStreamState* pState, const SWinKey* key, void** pVal, int32_t* pVLen); +int32_t streamStateFillGet(SStreamState* pState, const SWinKey* key, void** pVal, int32_t* pVLen, int32_t* pWinCode); +int32_t streamStateFillAddIfNotExist(SStreamState* pState, const SWinKey* key, void** pVal, int32_t* pVLen, + int32_t* pWinCode); void streamStateFillDel(SStreamState* pState, const SWinKey* key); +int32_t streamStateFillGetNext(SStreamState* pState, const SWinKey* pKey, SWinKey* pResKey, void** pVal, int32_t* pVLen, + int32_t* pWinCode); +int32_t streamStateFillGetPrev(SStreamState* pState, const SWinKey* pKey, SWinKey* pResKey, void** pVal, int32_t* pVLen, + int32_t* pWinCode); int32_t streamStateAddIfNotExist(SStreamState* pState, const SWinKey* key, void** pVal, int32_t* pVLen, int32_t* pWinCode); @@ -96,15 +104,25 @@ SStreamStateCur* streamStateFillSeekKeyPrev(SStreamState* pState, const SWinKey* void streamStateFreeCur(SStreamStateCur* pCur); void streamStateResetCur(SStreamStateCur* pCur); -int32_t streamStateGetGroupKVByCur(SStreamStateCur* pCur, SWinKey* pKey, const void** pVal, int32_t* pVLen); +int32_t streamStateFillGetGroupKVByCur(SStreamStateCur* pCur, SWinKey* pKey, const void** pVal, int32_t* pVLen); int32_t streamStateGetKVByCur(SStreamStateCur* pCur, SWinKey* pKey, const void** pVal, int32_t* pVLen); +// twa +void streamStateSetFillInfo(SStreamState* pState); +void streamStateClearExpiredState(SStreamState* pState); + void streamStateCurNext(SStreamState* pState, SStreamStateCur* pCur); void streamStateCurPrev(SStreamState* pState, SStreamStateCur* pCur); int32_t streamStatePutParName(SStreamState* pState, int64_t groupId, const char* tbname); int32_t streamStateGetParName(SStreamState* pState, int64_t groupId, void** pVal, bool onlyCache, int32_t* pWinCode); +// group id +int32_t streamStateGroupPut(SStreamState* pState, int64_t groupId, void* value, int32_t vLen); +SStreamStateCur* streamStateGroupGetCur(SStreamState* pState); +void streamStateGroupCurNext(SStreamStateCur* pCur); +int32_t streamStateGroupGetKVByCur(SStreamStateCur* pCur, int64_t* pKey, void** pVal, int32_t* pVLen); + void streamStateReloadInfo(SStreamState* pState, TSKEY ts); void streamStateCopyBackend(SStreamState* src, SStreamState* dst); diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 81be70e35f..de10d6844e 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -114,7 +114,7 @@ enum { enum { TASK_TRIGGER_STATUS__INACTIVE = 1, - TASK_TRIGGER_STATUS__ACTIVE, + TASK_TRIGGER_STATUS__MAY_ACTIVE, }; typedef enum { @@ -295,9 +295,10 @@ typedef struct SStreamStatus { int32_t schedIdleTime; // idle time before invoke again int64_t lastExecTs; // last exec time stamp int32_t inScanHistorySentinel; - bool appendTranstateBlock; // has append the transfer state data block already + bool appendTranstateBlock; // has appended the transfer state data block already bool removeBackendFiles; // remove backend files on disk when free stream tasks SConsenChkptInfo consenChkptInfo; + STimeWindow latestForceWindow; // latest generated time window, only valid in } SStreamStatus; typedef struct SDataRange { @@ -306,14 +307,16 @@ typedef struct SDataRange { } SDataRange; typedef struct SSTaskBasicInfo { - int32_t nodeId; // vgroup id or snode id - SEpSet epSet; - SEpSet mnodeEpset; // mnode epset for send heartbeat - int32_t selfChildId; - int32_t totalLevel; - int8_t taskLevel; - int8_t fillHistory; // is fill history task or not - int64_t delaySchedParam; // in msec + int32_t nodeId; // vgroup id or snode id + SEpSet epSet; + SEpSet mnodeEpset; // mnode epset for send heartbeat + int32_t selfChildId; + int32_t trigger; + int8_t taskLevel; + int8_t fillHistory; // is fill history task or not + int64_t delaySchedParam; // in msec + int64_t watermark; // extracted from operators + SInterval interval; } SSTaskBasicInfo; typedef struct SStreamRetrieveReq SStreamRetrieveReq; @@ -544,8 +547,9 @@ typedef struct STaskUpdateEntry { typedef int32_t (*__state_trans_user_fn)(SStreamTask*, void* param); -int32_t tNewStreamTask(int64_t streamId, int8_t taskLevel, SEpSet* pEpset, bool fillHistory, int64_t triggerParam, - SArray* pTaskList, bool hasFillhistory, int8_t subtableWithoutMd5, SStreamTask** pTask); +int32_t tNewStreamTask(int64_t streamId, int8_t taskLevel, SEpSet* pEpset, bool fillHistory, int32_t trigger, + int64_t triggerParam, SArray* pTaskList, bool hasFillhistory, int8_t subtableWithoutMd5, + SStreamTask** pTask); void tFreeStreamTask(void* pTask); int32_t tEncodeStreamTask(SEncoder* pEncoder, const SStreamTask* pTask); int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask); diff --git a/include/libs/stream/tstreamFileState.h b/include/libs/stream/tstreamFileState.h index a265ae7e60..4a696d9798 100644 --- a/include/libs/stream/tstreamFileState.h +++ b/include/libs/stream/tstreamFileState.h @@ -16,8 +16,6 @@ #ifndef _STREAM_FILE_STATE_H_ #define _STREAM_FILE_STATE_H_ -#include "os.h" - #include "storageapi.h" #include "tarray.h" #include "tdef.h" @@ -37,7 +35,7 @@ typedef void (*_state_buff_cleanup_fn)(void* pRowBuff); typedef void* (*_state_buff_create_statekey_fn)(SRowBuffPos* pPos, int64_t num); typedef int32_t (*_state_file_remove_fn)(SStreamFileState* pFileState, const void* pKey); -typedef int32_t (*_state_file_get_fn)(SStreamFileState* pFileState, void* pKey, void* data, int32_t* pDataLen); +typedef int32_t (*_state_file_get_fn)(SStreamFileState* pFileState, void* pKey, void** data, int32_t* pDataLen); typedef int32_t (*_state_file_clear_fn)(SStreamState* pState); typedef int32_t (*_state_fun_get_fn)(SStreamFileState* pFileState, void* pKey, int32_t keyLen, void** pVal, @@ -45,6 +43,8 @@ typedef int32_t (*_state_fun_get_fn)(SStreamFileState* pFileState, void* pKey, i typedef int32_t (*range_cmpr_fn)(const SSessionKey* pWin1, const SSessionKey* pWin2); +typedef int (*__session_compare_fn_t)(const void* pWin, const void* pDatas, int pos); + int32_t streamFileStateInit(int64_t memSize, uint32_t keySize, uint32_t rowSize, uint32_t selectRowSize, GetTsFun fp, void* pFile, TSKEY delMark, const char* taskId, int64_t checkpointId, int8_t type, struct SStreamFileState** ppFileState); @@ -54,6 +54,8 @@ bool needClearDiskBuff(SStreamFileState* pFileState); void streamFileStateReleaseBuff(SStreamFileState* pFileState, SRowBuffPos* pPos, bool used); void streamFileStateClearBuff(SStreamFileState* pFileState, SRowBuffPos* pPos); +int32_t addRowBuffIfNotExist(SStreamFileState* pFileState, void* pKey, int32_t keyLen, void** pVal, int32_t* pVLen, + int32_t* pWinCode); int32_t getRowBuff(SStreamFileState* pFileState, void* pKey, int32_t keyLen, void** pVal, int32_t* pVLen, int32_t* pWinCode); void deleteRowBuff(SStreamFileState* pFileState, const void* pKey, int32_t keyLen); @@ -71,9 +73,11 @@ int32_t streamFileStateGetSelectRowSize(SStreamFileState* pFileState); void streamFileStateReloadInfo(SStreamFileState* pFileState, TSKEY ts); void* getRowStateBuff(SStreamFileState* pFileState); +void* getSearchBuff(SStreamFileState* pFileState); void* getStateFileStore(SStreamFileState* pFileState); bool isDeteled(SStreamFileState* pFileState, TSKEY ts); bool isFlushedState(SStreamFileState* pFileState, TSKEY ts, TSKEY gap); +TSKEY getFlushMark(SStreamFileState* pFileState); SRowBuffPos* getNewRowPosForWrite(SStreamFileState* pFileState); int32_t getRowStateRowSize(SStreamFileState* pFileState); @@ -94,6 +98,7 @@ int32_t recoverSesssion(SStreamFileState* pFileState, int64_t ckId); void sessionWinStateClear(SStreamFileState* pFileState); void sessionWinStateCleanup(void* pBuff); +SStreamStateCur* createStateCursor(SStreamFileState* pFileState); SStreamStateCur* sessionWinStateSeekKeyCurrentPrev(SStreamFileState* pFileState, const SSessionKey* pWinKey); SStreamStateCur* sessionWinStateSeekKeyCurrentNext(SStreamFileState* pFileState, const SSessionKey* pWinKey); SStreamStateCur* sessionWinStateSeekKeyNext(SStreamFileState* pFileState, const SSessionKey* pWinKey); @@ -103,6 +108,8 @@ void sessionWinStateMoveToNext(SStreamStateCur* pCur); int32_t sessionWinStateGetKeyByRange(SStreamFileState* pFileState, const SSessionKey* key, SSessionKey* curKey, range_cmpr_fn cmpFn); +int32_t binarySearch(void* keyList, int num, const void* key, __session_compare_fn_t cmpFn); + // state window int32_t getStateWinResultBuff(SStreamFileState* pFileState, SSessionKey* key, char* pKeyData, int32_t keyDataLen, state_key_cmpr_fn fn, void** pVal, int32_t* pVLen, int32_t* pWinCode); @@ -117,6 +124,34 @@ int32_t getSessionRowBuff(SStreamFileState* pFileState, void* pKey, int32_t keyL int32_t* pWinCode); int32_t getFunctionRowBuff(SStreamFileState* pFileState, void* pKey, int32_t keyLen, void** pVal, int32_t* pVLen); +// time slice +int32_t getHashSortRowBuff(SStreamFileState* pFileState, const SWinKey* pKey, void** pVal, int32_t* pVLen, + int32_t* pWinCode); +int32_t hashSortFileGetFn(SStreamFileState* pFileState, void* pKey, void** data, int32_t* pDataLen); +int32_t hashSortFileRemoveFn(SStreamFileState* pFileState, const void* pKey); +void clearSearchBuff(SStreamFileState* pFileState); +int32_t getHashSortNextRow(SStreamFileState* pFileState, const SWinKey* pKey, SWinKey* pResKey, void** pVal, + int32_t* pVLen, int32_t* pWinCode); +int32_t getHashSortPrevRow(SStreamFileState* pFileState, const SWinKey* pKey, SWinKey* pResKey, void** ppVal, + int32_t* pVLen, int32_t* pWinCode); +int32_t recoverFillSnapshot(SStreamFileState* pFileState, int64_t ckId); +void deleteHashSortRowBuff(SStreamFileState* pFileState, const SWinKey* pKey); + +//group +int32_t streamFileStateGroupPut(SStreamFileState* pFileState, int64_t groupId, void* value, int32_t vLen); +void streamFileStateGroupCurNext(SStreamStateCur* pCur); +int32_t streamFileStateGroupGetKVByCur(SStreamStateCur* pCur, int64_t* pKey, void** pVal, int32_t* pVLen); +SSHashObj* getGroupIdCache(SStreamFileState* pFileState); +int fillStateKeyCompare(const void* pWin1, const void* pDatas, int pos); +int32_t getRowStatePrevRow(SStreamFileState* pFileState, const SWinKey* pKey, SWinKey* pResKey, void** ppVal, + int32_t* pVLen, int32_t* pWinCode); +int32_t addSearchItem(SStreamFileState* pFileState, SArray* pWinStates, const SWinKey* pKey); + +//twa +void setFillInfo(SStreamFileState* pFileState); +void clearExpiredState(SStreamFileState* pFileState); +int32_t addArrayBuffIfNotExist(SSHashObj* pSearchBuff, uint64_t groupId, SArray** ppResStates); + #ifdef __cplusplus } #endif diff --git a/include/libs/stream/tstreamUpdate.h b/include/libs/stream/tstreamUpdate.h index 06465e79e5..32712736c2 100644 --- a/include/libs/stream/tstreamUpdate.h +++ b/include/libs/stream/tstreamUpdate.h @@ -36,8 +36,8 @@ bool updateInfoIsTableInserted(SUpdateInfo* pInfo, int64_t tbUid); void updateInfoDestroy(SUpdateInfo* pInfo); void updateInfoAddCloseWindowSBF(SUpdateInfo* pInfo); void updateInfoDestoryColseWinSBF(SUpdateInfo* pInfo); -int32_t updateInfoSerialize(void* buf, int32_t bufLen, const SUpdateInfo* pInfo, int32_t* pLen); -int32_t updateInfoDeserialize(void* buf, int32_t bufLen, SUpdateInfo* pInfo); +int32_t updateInfoSerialize(SEncoder* pEncoder, const SUpdateInfo* pInfo); +int32_t updateInfoDeserialize(SDecoder* pDeCoder, SUpdateInfo* pInfo); void windowSBfDelete(SUpdateInfo* pInfo, uint64_t count); int32_t windowSBfAdd(SUpdateInfo* pInfo, uint64_t count); bool isIncrementalTimeStamp(SUpdateInfo* pInfo, uint64_t tableId, TSKEY ts, void* pPkVal, int32_t len); diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index 8d33e61733..e67e4a963b 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -242,7 +242,7 @@ static int32_t doAddSinkTask(SStreamObj* pStream, SMnode* pMnode, SVgObj* pVgrou SArray** pTaskList = (isFillhistory) ? taosArrayGetLast(pStream->pHTasksList) : taosArrayGetLast(pStream->tasks); SStreamTask* pTask = NULL; - int32_t code = tNewStreamTask(uid, TASK_LEVEL__SINK, pEpset, isFillhistory, 0, *pTaskList, pStream->conf.fillHistory, + int32_t code = tNewStreamTask(uid, TASK_LEVEL__SINK, pEpset, isFillhistory, 0, 0, *pTaskList, pStream->conf.fillHistory, pStream->subTableWithoutMd5, &pTask); if (code != 0) { return code; @@ -356,8 +356,9 @@ static int32_t buildSourceTask(SStreamObj* pStream, SEpSet* pEpset, bool isFillh uint64_t uid = (isFillhistory) ? pStream->hTaskUid : pStream->uid; SArray** pTaskList = (isFillhistory) ? taosArrayGetLast(pStream->pHTasksList) : taosArrayGetLast(pStream->tasks); - int32_t code = tNewStreamTask(uid, TASK_LEVEL__SOURCE, pEpset, isFillhistory, useTriggerParam ? pStream->conf.triggerParam : 0, - *pTaskList, pStream->conf.fillHistory, pStream->subTableWithoutMd5, pTask); + int32_t code = tNewStreamTask(uid, TASK_LEVEL__SOURCE, pEpset, isFillhistory, pStream->conf.trigger, + useTriggerParam ? pStream->conf.triggerParam : 0, *pTaskList, pStream->conf.fillHistory, + pStream->subTableWithoutMd5, pTask); return code; } @@ -396,18 +397,18 @@ static void setHTasksId(SStreamObj* pStream) { } static int32_t doAddSourceTask(SMnode* pMnode, SSubplan* plan, SStreamObj* pStream, SEpSet* pEpset, int64_t skey, - SArray* pVerList, SVgObj* pVgroup, bool isFillhistory, bool useTriggerParam) { + SArray* pVerList, SVgObj* pVgroup, bool isHistoryTask, bool useTriggerParam) { SStreamTask* pTask = NULL; - int32_t code = buildSourceTask(pStream, pEpset, isFillhistory, useTriggerParam, &pTask); + int32_t code = buildSourceTask(pStream, pEpset, isHistoryTask, useTriggerParam, &pTask); if (code != TSDB_CODE_SUCCESS) { return code; } mDebug("doAddSourceTask taskId:%s, %p vgId:%d, isFillHistory:%d", pTask->id.idStr, pTask, pVgroup->vgId, - isFillhistory); + isHistoryTask); if (pStream->conf.fillHistory) { - haltInitialTaskStatus(pTask, plan, isFillhistory); + haltInitialTaskStatus(pTask, plan, isHistoryTask); } streamTaskSetDataRange(pTask, skey, pVerList, pVgroup->vgId); @@ -453,10 +454,12 @@ static SSubplan* getAggSubPlan(const SQueryPlan* pPlan, int index) { static int32_t addSourceTask(SMnode* pMnode, SSubplan* plan, SStreamObj* pStream, SEpSet* pEpset, int64_t nextWindowSkey, SArray* pVerList, bool useTriggerParam) { + void* pIter = NULL; + int32_t code = 0; + SSdb* pSdb = pMnode->pSdb; + addNewTaskList(pStream); - void* pIter = NULL; - SSdb* pSdb = pMnode->pSdb; while (1) { SVgObj* pVgroup; pIter = sdbFetch(pSdb, SDB_VGROUP, pIter, (void**)&pVgroup); @@ -469,10 +472,9 @@ static int32_t addSourceTask(SMnode* pMnode, SSubplan* plan, SStreamObj* pStream continue; } - int code = - doAddSourceTask(pMnode, plan, pStream, pEpset, nextWindowSkey, pVerList, pVgroup, false, useTriggerParam); + code = doAddSourceTask(pMnode, plan, pStream, pEpset, nextWindowSkey, pVerList, pVgroup, false, useTriggerParam); if (code != 0) { - mError("create stream task, code:%s", tstrerror(code)); + mError("failed to create stream task, code:%s", tstrerror(code)); // todo drop the added source tasks. sdbRelease(pSdb, pVgroup); @@ -504,9 +506,9 @@ static int32_t buildAggTask(SStreamObj* pStream, SEpSet* pEpset, bool isFillhist uint64_t uid = (isFillhistory) ? pStream->hTaskUid : pStream->uid; SArray** pTaskList = (isFillhistory) ? taosArrayGetLast(pStream->pHTasksList) : taosArrayGetLast(pStream->tasks); - int32_t code = - tNewStreamTask(uid, TASK_LEVEL__AGG, pEpset, isFillhistory, useTriggerParam ? pStream->conf.triggerParam : 0, - *pTaskList, pStream->conf.fillHistory, pStream->subTableWithoutMd5, pAggTask); + int32_t code = tNewStreamTask(uid, TASK_LEVEL__AGG, pEpset, isFillhistory, pStream->conf.trigger, + useTriggerParam ? pStream->conf.triggerParam : 0, *pTaskList, pStream->conf.fillHistory, + pStream->subTableWithoutMd5, pAggTask); return code; } @@ -695,7 +697,8 @@ static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* if (terrno != 0) code = terrno; TAOS_RETURN(code); } - code = addSourceTask(pMnode, plan, pStream, pEpset, skey, pVerList, numOfPlanLevel == 1); + + code = addSourceTask(pMnode, plan, pStream, pEpset, skey, pVerList, (numOfPlanLevel == 1)); if (code != TSDB_CODE_SUCCESS) { return code; } diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 8abe739f25..81db427afd 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -454,17 +454,16 @@ static int32_t mndBuildStreamObjFromCreateReq(SMnode *pMnode, SStreamObj *pObj, pObj->outputSchema.pSchema = pFullSchema; } - bool hasKey = hasDestPrimaryKey(&pObj->outputSchema); SPlanContext cxt = { .pAstRoot = pAst, .topicQuery = false, .streamQuery = true, - .triggerType = pObj->conf.trigger == STREAM_TRIGGER_MAX_DELAY ? STREAM_TRIGGER_WINDOW_CLOSE : pObj->conf.trigger, + .triggerType = (pObj->conf.trigger == STREAM_TRIGGER_MAX_DELAY)? STREAM_TRIGGER_WINDOW_CLOSE : pObj->conf.trigger, .watermark = pObj->conf.watermark, .igExpired = pObj->conf.igExpired, .deleteMark = pObj->deleteMark, .igCheckUpdate = pObj->igCheckUpdate, - .destHasPrimaryKey = hasKey, + .destHasPrimaryKey = hasDestPrimaryKey(&pObj->outputSchema), }; // using ast and param to build physical plan diff --git a/source/dnode/mnode/impl/src/mndStreamUtil.c b/source/dnode/mnode/impl/src/mndStreamUtil.c index 423d9df4b6..f9b7644af4 100644 --- a/source/dnode/mnode/impl/src/mndStreamUtil.c +++ b/source/dnode/mnode/impl/src/mndStreamUtil.c @@ -877,6 +877,8 @@ static void mndShowStreamTrigger(char *dst, SStreamObj *pStream) { strcpy(dst, "window close"); } else if (trigger == STREAM_TRIGGER_MAX_DELAY) { strcpy(dst, "max delay"); + } else if (trigger == STREAM_TRIGGER_FORCE_WINDOW_CLOSE) { + strcpy(dst, "force window close"); } } diff --git a/source/dnode/snode/src/snodeInitApi.c b/source/dnode/snode/src/snodeInitApi.c index 196fa56c99..680a2fd83c 100644 --- a/source/dnode/snode/src/snodeInitApi.c +++ b/source/dnode/snode/src/snodeInitApi.c @@ -46,10 +46,14 @@ void initStateStoreAPI(SStateStore* pStore) { pStore->streamStateSaveInfo = streamStateSaveInfo; pStore->streamStateGetInfo = streamStateGetInfo; pStore->streamStateSetNumber = streamStateSetNumber; + pStore->streamStateGetPrev = streamStateGetPrev; pStore->streamStateFillPut = streamStateFillPut; pStore->streamStateFillGet = streamStateFillGet; + pStore->streamStateFillAddIfNotExist = streamStateFillAddIfNotExist; pStore->streamStateFillDel = streamStateFillDel; + pStore->streamStateFillGetNext = streamStateFillGetNext; + pStore->streamStateFillGetPrev = streamStateFillGetPrev; pStore->streamStateCurNext = streamStateCurNext; pStore->streamStateCurPrev = streamStateCurPrev; @@ -60,9 +64,12 @@ void initStateStoreAPI(SStateStore* pStore) { pStore->streamStateFillSeekKeyPrev = streamStateFillSeekKeyPrev; pStore->streamStateFreeCur = streamStateFreeCur; - pStore->streamStateGetGroupKVByCur = streamStateGetGroupKVByCur; + pStore->streamStateFillGetGroupKVByCur = streamStateFillGetGroupKVByCur; pStore->streamStateGetKVByCur = streamStateGetKVByCur; + pStore->streamStateSetFillInfo = streamStateSetFillInfo; + pStore->streamStateClearExpiredState = streamStateClearExpiredState; + pStore->streamStateSessionAddIfNotExist = streamStateSessionAddIfNotExist; pStore->streamStateSessionPut = streamStateSessionPut; pStore->streamStateSessionGet = streamStateSessionGet; @@ -75,11 +82,6 @@ void initStateStoreAPI(SStateStore* pStore) { pStore->streamStateCountGetKeyByRange = streamStateCountGetKeyByRange; pStore->streamStateSessionAllocWinBuffByNextPosition = streamStateSessionAllocWinBuffByNextPosition; -//void initStreamStateAPI(SStorageAPI* pAPI) { -// initStateStoreAPI(&pAPI->stateStore); -// initFunctionStateStore(&pAPI->functionStore); -//} - pStore->updateInfoInit = updateInfoInit; pStore->updateInfoFillBlockData = updateInfoFillBlockData; pStore->updateInfoIsUpdated = updateInfoIsUpdated; @@ -100,6 +102,11 @@ void initStateStoreAPI(SStateStore* pStore) { pStore->streamStateSessionSeekKeyCurrentPrev = streamStateSessionSeekKeyCurrentPrev; pStore->streamStateSessionSeekKeyCurrentNext = streamStateSessionSeekKeyCurrentNext; + pStore->streamStateGroupPut = streamStateGroupPut; + pStore->streamStateGroupGetCur = streamStateGroupGetCur; + pStore->streamStateGroupCurNext = streamStateGroupCurNext; + pStore->streamStateGroupGetKVByCur = streamStateGroupGetKVByCur; + pStore->streamFileStateDestroy = streamFileStateDestroy; pStore->streamFileStateClear = streamFileStateClear; pStore->needClearDiskBuff = needClearDiskBuff; diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index ec7ac1054c..bd78f62cae 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -713,8 +713,10 @@ end: static void freePtr(void* ptr) { taosMemoryFree(*(void**)ptr); } int32_t tqBuildStreamTask(void* pTqObj, SStreamTask* pTask, int64_t nextProcessVer) { - STQ* pTq = (STQ*)pTqObj; - int32_t vgId = TD_VID(pTq->pVnode); + STQ* pTq = (STQ*)pTqObj; + int32_t vgId = TD_VID(pTq->pVnode); + SCheckpointInfo* pChkInfo = NULL; + tqDebug("s-task:0x%x start to build task", pTask->id.taskId); int32_t code = streamTaskInit(pTask, pTq->pStreamMeta, &pTq->pVnode->msgCb, nextProcessVer); @@ -766,7 +768,7 @@ int32_t tqBuildStreamTask(void* pTqObj, SStreamTask* pTask, int64_t nextProcessV streamTaskResetUpstreamStageInfo(pTask); - SCheckpointInfo* pChkInfo = &pTask->chkInfo; + pChkInfo = &pTask->chkInfo; tqSetRestoreVersionInfo(pTask); char* p = streamTaskGetStatus(pTask).name; @@ -886,13 +888,14 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { SStreamScanHistoryReq* pReq = (SStreamScanHistoryReq*)pMsg->pCont; SStreamMeta* pMeta = pTq->pStreamMeta; int32_t code = TSDB_CODE_SUCCESS; + SStreamTask* pTask = NULL; + SStreamTask* pStreamTask = NULL; - SStreamTask* pTask = NULL; code = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId, &pTask); if (pTask == NULL) { tqError("vgId:%d failed to acquire stream task:0x%x during scan history data, task may have been destroyed", pMeta->vgId, pReq->taskId); - return -1; + return code; } // do recovery step1 @@ -957,11 +960,11 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { ETaskStatus s = p.state; if (s == TASK_STATUS__PAUSE) { - tqDebug("s-task:%s is paused in the step1, elapsed time:%.2fs total:%.2fs, sched-status:%d", pTask->id.idStr, - el, pTask->execInfo.step1El, status); + tqDebug("s-task:%s is paused in the step1, elapsed time:%.2fs total:%.2fs, sched-status:%d", id, el, + pTask->execInfo.step1El, status); } else if (s == TASK_STATUS__STOP || s == TASK_STATUS__DROPPING) { - tqDebug("s-task:%s status:%p not continue scan-history data, total elapsed time:%.2fs quit", pTask->id.idStr, - p.name, pTask->execInfo.step1El); + tqDebug("s-task:%s status:%p not continue scan-history data, total elapsed time:%.2fs quit", id, p.name, + pTask->execInfo.step1El); } } @@ -978,7 +981,6 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { } // 1. get the related stream task - SStreamTask* pStreamTask = NULL; code = streamMetaAcquireTask(pMeta, pTask->streamTaskId.streamId, pTask->streamTaskId.taskId, &pStreamTask); if (pStreamTask == NULL) { tqError("failed to find s-task:0x%" PRIx64 ", it may have been destroyed, drop related fill-history task:%s", @@ -989,15 +991,15 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { atomic_store_32(&pTask->status.inScanHistorySentinel, 0); streamMetaReleaseTask(pMeta, pTask); - return code; // todo: handle failure + return code; } if (pStreamTask->info.taskLevel != TASK_LEVEL__SOURCE) { tqError("s-task:%s fill-history task related stream task level:%d, unexpected", id, pStreamTask->info.taskLevel); return TSDB_CODE_STREAM_INTERNAL_ERROR; } - code = streamTaskHandleEventAsync(pStreamTask->status.pSM, TASK_EVENT_HALT, handleStep2Async, pTq); + code = streamTaskHandleEventAsync(pStreamTask->status.pSM, TASK_EVENT_HALT, handleStep2Async, pTq); streamMetaReleaseTask(pMeta, pStreamTask); atomic_store_32(&pTask->status.inScanHistorySentinel, 0); diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 24c892de8b..29372c5da7 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -270,7 +270,12 @@ bool handleFillhistoryScanComplete(SStreamTask* pTask, int64_t ver) { bool taskReadyForDataFromWal(SStreamTask* pTask) { // non-source or fill-history tasks don't need to response the WAL scan action. - if ((pTask->info.taskLevel != TASK_LEVEL__SOURCE) || (pTask->status.downstreamReady == 0)) { + SSTaskBasicInfo* pInfo = &pTask->info; + if ((pInfo->taskLevel != TASK_LEVEL__SOURCE) || (pTask->status.downstreamReady == 0)) { + return false; + } + + if (pInfo->taskLevel == TASK_LEVEL__SOURCE && pInfo->trigger == STREAM_TRIGGER_FORCE_WINDOW_CLOSE) { return false; } @@ -282,7 +287,7 @@ bool taskReadyForDataFromWal(SStreamTask* pTask) { } // fill-history task has entered into the last phase, no need to anything - if ((pTask->info.fillHistory == 1) && pTask->status.appendTranstateBlock) { + if ((pInfo->fillHistory == 1) && pTask->status.appendTranstateBlock) { // the maximum version of data in the WAL has reached already, the step2 is done tqDebug("s-task:%s fill-history reach the maximum ver:%" PRId64 ", not scan wal anymore", pTask->id.idStr, pTask->dataRange.range.maxVer); @@ -419,9 +424,9 @@ int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle) { streamMutexLock(&pTask->lock); - SStreamTaskState pState = streamTaskGetStatus(pTask); - if (pState.state != TASK_STATUS__READY) { - tqDebug("s-task:%s not ready for submit block from wal, status:%s", pTask->id.idStr, pState.name); + SStreamTaskState state = streamTaskGetStatus(pTask); + if (state.state != TASK_STATUS__READY) { + tqDebug("s-task:%s not ready for submit block from wal, status:%s", pTask->id.idStr, state.name); streamMutexUnlock(&pTask->lock); streamMetaReleaseTask(pStreamMeta, pTask); continue; diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index 59e739d1ff..f31dd28847 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -88,6 +88,8 @@ int32_t tqExpandStreamTask(SStreamTask* pTask) { } } + streamSetupScheduleTrigger(pTask); + double el = (taosGetTimestampMs() - st) / 1000.0; tqDebug("s-task:%s vgId:%d expand stream task completed, elapsed time:%.2fsec", pTask->id.idStr, vgId, el); @@ -614,6 +616,7 @@ int32_t tqStreamTaskProcessDeployReq(SStreamMeta* pMeta, SMsgCb* cb, int64_t sve int32_t taskId = -1; int64_t streamId = -1; bool added = false; + int32_t size = sizeof(SStreamTask); if (tsDisableStream) { tqInfo("vgId:%d stream disabled, not deploy stream tasks", vgId); @@ -623,7 +626,6 @@ int32_t tqStreamTaskProcessDeployReq(SStreamMeta* pMeta, SMsgCb* cb, int64_t sve tqDebug("vgId:%d receive new stream task deploy msg, start to build stream task", vgId); // 1.deserialize msg and build task - int32_t size = sizeof(SStreamTask); SStreamTask* pTask = taosMemoryCalloc(1, size); if (pTask == NULL) { tqError("vgId:%d failed to create stream task due to out of memory, alloc size:%d", vgId, size); diff --git a/source/dnode/vnode/src/vnd/vnodeInitApi.c b/source/dnode/vnode/src/vnd/vnodeInitApi.c index 59a129cac8..d688d1323d 100644 --- a/source/dnode/vnode/src/vnd/vnodeInitApi.c +++ b/source/dnode/vnode/src/vnd/vnodeInitApi.c @@ -162,10 +162,14 @@ void initStateStoreAPI(SStateStore* pStore) { pStore->streamStateSaveInfo = streamStateSaveInfo; pStore->streamStateGetInfo = streamStateGetInfo; pStore->streamStateSetNumber = streamStateSetNumber; + pStore->streamStateGetPrev = streamStateGetPrev; pStore->streamStateFillPut = streamStateFillPut; pStore->streamStateFillGet = streamStateFillGet; + pStore->streamStateFillAddIfNotExist = streamStateFillAddIfNotExist; pStore->streamStateFillDel = streamStateFillDel; + pStore->streamStateFillGetNext = streamStateFillGetNext; + pStore->streamStateFillGetPrev = streamStateFillGetPrev; pStore->streamStateCurNext = streamStateCurNext; pStore->streamStateCurPrev = streamStateCurPrev; @@ -176,9 +180,12 @@ void initStateStoreAPI(SStateStore* pStore) { pStore->streamStateFillSeekKeyPrev = streamStateFillSeekKeyPrev; pStore->streamStateFreeCur = streamStateFreeCur; - pStore->streamStateGetGroupKVByCur = streamStateGetGroupKVByCur; + pStore->streamStateFillGetGroupKVByCur = streamStateFillGetGroupKVByCur; pStore->streamStateGetKVByCur = streamStateGetKVByCur; + pStore->streamStateSetFillInfo = streamStateSetFillInfo; + pStore->streamStateClearExpiredState = streamStateClearExpiredState; + pStore->streamStateSessionAddIfNotExist = streamStateSessionAddIfNotExist; pStore->streamStateSessionPut = streamStateSessionPut; pStore->streamStateSessionGet = streamStateSessionGet; @@ -214,6 +221,11 @@ void initStateStoreAPI(SStateStore* pStore) { pStore->streamStateSessionSeekKeyCurrentPrev = streamStateSessionSeekKeyCurrentPrev; pStore->streamStateSessionSeekKeyCurrentNext = streamStateSessionSeekKeyCurrentNext; + pStore->streamStateGroupPut = streamStateGroupPut; + pStore->streamStateGroupGetCur = streamStateGroupGetCur; + pStore->streamStateGroupCurNext = streamStateGroupCurNext; + pStore->streamStateGroupGetKVByCur = streamStateGroupGetKVByCur; + pStore->streamFileStateDestroy = streamFileStateDestroy; pStore->streamFileStateClear = streamFileStateClear; pStore->needClearDiskBuff = needClearDiskBuff; diff --git a/source/libs/executor/inc/executorInt.h b/source/libs/executor/inc/executorInt.h index 98d3fefada..039c0fa68b 100644 --- a/source/libs/executor/inc/executorInt.h +++ b/source/libs/executor/inc/executorInt.h @@ -454,6 +454,29 @@ typedef struct SSteamOpBasicInfo { bool updateOperatorInfo; } SSteamOpBasicInfo; +typedef struct SStreamFillSupporter { + int32_t type; // fill type + SInterval interval; + SResultRowData prev; + TSKEY prevOriginKey; + SResultRowData cur; + SResultRowData next; + TSKEY nextOriginKey; + SResultRowData nextNext; + SFillColInfo* pAllColInfo; // fill exprs and not fill exprs + SExprSupp notFillExprSup; + int32_t numOfAllCols; // number of all exprs, including the tags columns + int32_t numOfFillCols; + int32_t numOfNotFillCols; + int32_t rowSize; + SSHashObj* pResMap; + bool hasDelete; + SStorageAPI* pAPI; + STimeWindow winRange; + int32_t pkColBytes; + __compar_fn_t comparePkColFn; +} SStreamFillSupporter; + typedef struct SStreamScanInfo { SSteamOpBasicInfo basic; SExprInfo* pPseudoExpr; @@ -477,6 +500,7 @@ typedef struct SStreamScanInfo { STqReader* tqReader; uint64_t groupId; + bool igCheckGroupId; struct SUpdateInfo* pUpdateInfo; EStreamScanMode scanMode; @@ -493,6 +517,7 @@ typedef struct SStreamScanInfo { STimeWindow updateWin; STimeWindowAggSupp twAggSup; SSDataBlock* pUpdateDataRes; + SStreamFillSupporter* pFillSup; // status for tmq SNodeList* pGroupTags; SNode* pTagCond; @@ -511,6 +536,8 @@ typedef struct SStreamScanInfo { SSDataBlock* pCheckpointRes; int8_t pkColType; int32_t pkColLen; + bool useGetResultRange; + STimeWindow lastScanRange; } SStreamScanInfo; typedef struct { @@ -781,25 +808,6 @@ typedef struct SStreamPartitionOperatorInfo { SSDataBlock* pCreateTbRes; } SStreamPartitionOperatorInfo; -typedef struct SStreamFillSupporter { - int32_t type; // fill type - SInterval interval; - SResultRowData prev; - SResultRowData cur; - SResultRowData next; - SResultRowData nextNext; - SFillColInfo* pAllColInfo; // fill exprs and not fill exprs - SExprSupp notFillExprSup; - int32_t numOfAllCols; // number of all exprs, including the tags columns - int32_t numOfFillCols; - int32_t numOfNotFillCols; - int32_t rowSize; - SSHashObj* pResMap; - bool hasDelete; - SStorageAPI* pAPI; - STimeWindow winRange; -} SStreamFillSupporter; - typedef struct SStreamFillOperatorInfo { SSteamOpBasicInfo basic; SStreamFillSupporter* pFillSup; @@ -813,8 +821,70 @@ typedef struct SStreamFillOperatorInfo { int32_t primaryTsCol; int32_t primarySrcSlotId; SStreamFillInfo* pFillInfo; + SStreamAggSupporter* pStreamAggSup; + SArray* pCloseTs; + SArray* pUpdated; + SGroupResInfo groupResInfo; } SStreamFillOperatorInfo; +typedef struct SStreamTimeSliceOperatorInfo { + SSteamOpBasicInfo basic; + STimeWindowAggSupp twAggSup; + SStreamAggSupporter streamAggSup; + SStreamFillSupporter* pFillSup; + SStreamFillInfo* pFillInfo; + SSDataBlock* pRes; + SSDataBlock* pDelRes; + bool recvCkBlock; + SSDataBlock* pCheckpointRes; + int32_t fillType; + SResultRowData leftRow; + SResultRowData valueRow; + SResultRowData rightRow; + int32_t primaryTsIndex; + SExprSupp scalarSup; // scalar calculation + bool ignoreExpiredData; + bool ignoreExpiredDataSaved; + bool destHasPrimaryKey; + SArray* historyPoints; + SArray* pUpdated; // SWinKey + SArray* historyWins; + SSHashObj* pUpdatedMap; + int32_t delIndex; + SArray* pDelWins; // SWinKey + SSHashObj* pDeletedMap; + uint64_t numOfDatapack; + SGroupResInfo groupResInfo; + bool ignoreNull; + bool isHistoryOp; + SArray* pCloseTs; + struct SOperatorInfo* pOperator; +} SStreamTimeSliceOperatorInfo; + +typedef struct SStreamIntervalSliceOperatorInfo { + SSteamOpBasicInfo basic; + SOptrBasicInfo binfo; + STimeWindowAggSupp twAggSup; + SStreamAggSupporter streamAggSup; + SExprSupp scalarSup; + SInterval interval; + bool recvCkBlock; + SSDataBlock* pCheckpointRes; + int32_t primaryTsIndex; + SSHashObj* pUpdatedMap; // SWinKey + SArray* pUpdated; // SWinKey + SSHashObj* pDeletedMap; + SArray* pDelWins; + SSDataBlock* pDelRes; + int32_t delIndex; + bool destHasPrimaryKey; + int64_t endTs; + SGroupResInfo groupResInfo; + struct SOperatorInfo* pOperator; + bool hasFill; + bool hasInterpoFunc; +} SStreamIntervalSliceOperatorInfo; + #define OPTR_IS_OPENED(_optr) (((_optr)->status & OP_OPENED) == OP_OPENED) #define OPTR_SET_OPENED(_optr) ((_optr)->status |= OP_OPENED) @@ -937,54 +1007,57 @@ int32_t copyResultrowToDataBlock(SExprInfo* pExprInfo, int32_t numOfExprs, SResu void doUpdateNumOfRows(SqlFunctionCtx* pCtx, SResultRow* pRow, int32_t numOfExprs, const int32_t* rowEntryOffset); void doClearBufferedBlocks(SStreamScanInfo* pInfo); -void streamOpReleaseState(struct SOperatorInfo* pOperator); -void streamOpReloadState(struct SOperatorInfo* pOperator); -void destroyStreamAggSupporter(SStreamAggSupporter* pSup); -void clearGroupResInfo(SGroupResInfo* pGroupResInfo); -int32_t initBasicInfoEx(SOptrBasicInfo* pBasicInfo, SExprSupp* pSup, SExprInfo* pExprInfo, int32_t numOfCols, - SSDataBlock* pResultBlock, SFunctionStateStore* pStore); -int32_t initStreamAggSupporter(SStreamAggSupporter* pSup, SExprSupp* pExpSup, int32_t numOfOutput, int64_t gap, - SStreamState* pState, int32_t keySize, int16_t keyType, SStateStore* pStore, - SReadHandle* pHandle, STimeWindowAggSupp* pTwAggSup, const char* taskIdStr, - SStorageAPI* pApi, int32_t tsIndex); -int32_t initDownStream(struct SOperatorInfo* downstream, SStreamAggSupporter* pAggSup, uint16_t type, - int32_t tsColIndex, STimeWindowAggSupp* pTwSup, struct SSteamOpBasicInfo* pBasic); -int32_t getMaxTsWins(const SArray* pAllWins, SArray* pMaxWins); -void initGroupResInfoFromArrayList(SGroupResInfo* pGroupResInfo, SArray* pArrayList); -void getSessionHashKey(const SSessionKey* pKey, SSessionKey* pHashKey); -int32_t deleteSessionWinState(SStreamAggSupporter* pAggSup, SSDataBlock* pBlock, SSHashObj* pMapUpdate, - SSHashObj* pMapDelete, SSHashObj* pPkDelete, bool needAdd); -int32_t getAllSessionWindow(SSHashObj* pHashMap, SSHashObj* pStUpdated); -int32_t closeSessionWindow(SSHashObj* pHashMap, STimeWindowAggSupp* pTwSup, SSHashObj* pClosed); -int32_t copyUpdateResult(SSHashObj** ppWinUpdated, SArray* pUpdated, __compar_fn_t compar); -int32_t sessionKeyCompareAsc(const void* pKey1, const void* pKey2); -void removeSessionDeleteResults(SSHashObj* pHashMap, SArray* pWins); -int32_t doOneWindowAggImpl(SColumnInfoData* pTimeWindowData, SResultWindowInfo* pCurWin, SResultRow** pResult, - int32_t startIndex, int32_t winRows, int32_t rows, int32_t numOutput, - struct SOperatorInfo* pOperator, int64_t winDelta); -void setSessionWinOutputInfo(SSHashObj* pStUpdated, SResultWindowInfo* pWinInfo); -int32_t saveSessionOutputBuf(SStreamAggSupporter* pAggSup, SResultWindowInfo* pWinInfo); -int32_t saveResult(SResultWindowInfo winInfo, SSHashObj* pStUpdated); -int32_t saveDeleteRes(SSHashObj* pStDelete, SSessionKey key); -void removeSessionResult(SStreamAggSupporter* pAggSup, SSHashObj* pHashMap, SSHashObj* pResMap, SSessionKey* pKey); -void doBuildDeleteDataBlock(struct SOperatorInfo* pOp, SSHashObj* pStDeleted, SSDataBlock* pBlock, void** Ite); -void doBuildSessionResult(struct SOperatorInfo* pOperator, void* pState, SGroupResInfo* pGroupResInfo, - SSDataBlock* pBlock); -int32_t getSessionWindowInfoByKey(SStreamAggSupporter* pAggSup, SSessionKey* pKey, SResultWindowInfo* pWinInfo); -void getNextSessionWinInfo(SStreamAggSupporter* pAggSup, SSHashObj* pStUpdated, SResultWindowInfo* pCurWin, - SResultWindowInfo* pNextWin); -int32_t compactTimeWindow(SExprSupp* pSup, SStreamAggSupporter* pAggSup, STimeWindowAggSupp* pTwAggSup, - SExecTaskInfo* pTaskInfo, SResultWindowInfo* pCurWin, SResultWindowInfo* pNextWin, - SSHashObj* pStUpdated, SSHashObj* pStDeleted, bool addGap); -void releaseOutputBuf(void* pState, SRowBuffPos* pPos, SStateStore* pAPI); -void resetWinRange(STimeWindow* winRange); -bool checkExpiredData(SStateStore* pAPI, SUpdateInfo* pUpdateInfo, STimeWindowAggSupp* pTwSup, uint64_t tableId, - TSKEY ts, void* pPkVal, int32_t len); -int64_t getDeleteMark(SWindowPhysiNode* pWinPhyNode, int64_t interval); -void resetUnCloseSessionWinInfo(SSHashObj* winMap); -void setStreamOperatorCompleted(struct SOperatorInfo* pOperator); -void reloadAggSupFromDownStream(struct SOperatorInfo* downstream, SStreamAggSupporter* pAggSup); -void destroyFlusedPos(void* pRes); +void streamOpReleaseState(struct SOperatorInfo* pOperator); +void streamOpReloadState(struct SOperatorInfo* pOperator); +void destroyStreamAggSupporter(SStreamAggSupporter* pSup); +void clearGroupResInfo(SGroupResInfo* pGroupResInfo); +int32_t initBasicInfoEx(SOptrBasicInfo* pBasicInfo, SExprSupp* pSup, SExprInfo* pExprInfo, int32_t numOfCols, + SSDataBlock* pResultBlock, SFunctionStateStore* pStore); +int32_t initStreamAggSupporter(SStreamAggSupporter* pSup, SExprSupp* pExpSup, int32_t numOfOutput, int64_t gap, + SStreamState* pState, int32_t keySize, int16_t keyType, SStateStore* pStore, + SReadHandle* pHandle, STimeWindowAggSupp* pTwAggSup, const char* taskIdStr, + SStorageAPI* pApi, int32_t tsIndex, int8_t stateType, int32_t ratio); +int32_t initDownStream(struct SOperatorInfo* downstream, SStreamAggSupporter* pAggSup, uint16_t type, + int32_t tsColIndex, STimeWindowAggSupp* pTwSup, struct SSteamOpBasicInfo* pBasic); +int32_t getMaxTsWins(const SArray* pAllWins, SArray* pMaxWins); +void initGroupResInfoFromArrayList(SGroupResInfo* pGroupResInfo, SArray* pArrayList); +void getSessionHashKey(const SSessionKey* pKey, SSessionKey* pHashKey); +int32_t deleteSessionWinState(SStreamAggSupporter* pAggSup, SSDataBlock* pBlock, SSHashObj* pMapUpdate, + SSHashObj* pMapDelete, SSHashObj* pPkDelete, bool needAdd); +int32_t getAllSessionWindow(SSHashObj* pHashMap, SSHashObj* pStUpdated); +int32_t closeSessionWindow(SSHashObj* pHashMap, STimeWindowAggSupp* pTwSup, SSHashObj* pClosed); +int32_t copyUpdateResult(SSHashObj** ppWinUpdated, SArray* pUpdated, __compar_fn_t compar); +int32_t sessionKeyCompareAsc(const void* pKey1, const void* pKey2); +void removeSessionDeleteResults(SSHashObj* pHashMap, SArray* pWins); +int32_t doOneWindowAggImpl(SColumnInfoData* pTimeWindowData, SResultWindowInfo* pCurWin, SResultRow** pResult, + int32_t startIndex, int32_t winRows, int32_t rows, int32_t numOutput, + struct SOperatorInfo* pOperator, int64_t winDelta); +void setSessionWinOutputInfo(SSHashObj* pStUpdated, SResultWindowInfo* pWinInfo); +int32_t saveSessionOutputBuf(SStreamAggSupporter* pAggSup, SResultWindowInfo* pWinInfo); +int32_t saveResult(SResultWindowInfo winInfo, SSHashObj* pStUpdated); +int32_t saveDeleteRes(SSHashObj* pStDelete, SSessionKey key); +void removeSessionResult(SStreamAggSupporter* pAggSup, SSHashObj* pHashMap, SSHashObj* pResMap, SSessionKey* pKey); +void doBuildDeleteDataBlock(struct SOperatorInfo* pOp, SSHashObj* pStDeleted, SSDataBlock* pBlock, void** Ite); +void doBuildSessionResult(struct SOperatorInfo* pOperator, void* pState, SGroupResInfo* pGroupResInfo, + SSDataBlock* pBlock); +int32_t getSessionWindowInfoByKey(SStreamAggSupporter* pAggSup, SSessionKey* pKey, SResultWindowInfo* pWinInfo); +void getNextSessionWinInfo(SStreamAggSupporter* pAggSup, SSHashObj* pStUpdated, SResultWindowInfo* pCurWin, + SResultWindowInfo* pNextWin); +int32_t compactTimeWindow(SExprSupp* pSup, SStreamAggSupporter* pAggSup, STimeWindowAggSupp* pTwAggSup, + SExecTaskInfo* pTaskInfo, SResultWindowInfo* pCurWin, SResultWindowInfo* pNextWin, + SSHashObj* pStUpdated, SSHashObj* pStDeleted, bool addGap); +void releaseOutputBuf(void* pState, SRowBuffPos* pPos, SStateStore* pAPI); +void resetWinRange(STimeWindow* winRange); +bool checkExpiredData(SStateStore* pAPI, SUpdateInfo* pUpdateInfo, STimeWindowAggSupp* pTwSup, uint64_t tableId, + TSKEY ts, void* pPkVal, int32_t len); +int64_t getDeleteMark(SWindowPhysiNode* pWinPhyNode, int64_t interval); +void resetUnCloseSessionWinInfo(SSHashObj* winMap); +void setStreamOperatorCompleted(struct SOperatorInfo* pOperator); +void reloadAggSupFromDownStream(struct SOperatorInfo* downstream, SStreamAggSupporter* pAggSup); +void destroyFlusedPos(void* pRes); +bool isIrowtsPseudoColumn(SExprInfo* pExprInfo); +bool isIsfilledPseudoColumn(SExprInfo* pExprInfo); +bool isInterpFunc(SExprInfo* pExprInfo); int32_t encodeSSessionKey(void** buf, SSessionKey* key); void* decodeSSessionKey(void* buf, SSessionKey* key); @@ -1018,6 +1091,8 @@ int32_t doDeleteTimeWindows(SStreamAggSupporter* pAggSup, SSDataBlock* pBlock, S int32_t getNextQualifiedWindow(SInterval* pInterval, STimeWindow* pNext, SDataBlockInfo* pDataBlockInfo, TSKEY* primaryKeys, int32_t prevPosition, int32_t order); int32_t extractQualifiedTupleByFilterResult(SSDataBlock* pBlock, const SColumnInfoData* p, int32_t status); +bool getIgoreNullRes(SExprSupp* pExprSup); +bool checkNullRow(SExprSupp* pExprSup, SSDataBlock* pSrcBlock, int32_t index, bool ignoreNull); #ifdef __cplusplus } diff --git a/source/libs/executor/inc/operator.h b/source/libs/executor/inc/operator.h index b2035ecfde..91aef93452 100644 --- a/source/libs/executor/inc/operator.h +++ b/source/libs/executor/inc/operator.h @@ -167,6 +167,8 @@ int32_t createAnomalywindowOperatorInfo(SOperatorInfo* downstream, SPhysiNode* p int32_t createDynQueryCtrlOperatorInfo(SOperatorInfo** pDownstream, int32_t numOfDownstream, SDynQueryCtrlPhysiNode* pPhyciNode, SExecTaskInfo* pTaskInfo, SOperatorInfo** pInfo); +int32_t createStreamTimeSliceOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo, SReadHandle* pHandle, SOperatorInfo** ppOptInfo); + // clang-format on SOperatorFpSet createOperatorFpSet(__optr_open_fn_t openFn, __optr_fn_t nextFn, __optr_fn_t cleanup, diff --git a/source/libs/executor/inc/streamexecutorInt.h b/source/libs/executor/inc/streamexecutorInt.h index ab00dceb20..27686b0081 100644 --- a/source/libs/executor/inc/streamexecutorInt.h +++ b/source/libs/executor/inc/streamexecutorInt.h @@ -22,11 +22,87 @@ extern "C" { #include "executorInt.h" #include "tutil.h" +#define FILL_POS_INVALID 0 +#define FILL_POS_START 1 +#define FILL_POS_MID 2 +#define FILL_POS_END 3 + +#define HAS_NON_ROW_DATA(pRowData) (pRowData->key == INT64_MIN) +#define HAS_ROW_DATA(pRowData) (pRowData && pRowData->key != INT64_MIN) + +#define IS_INVALID_WIN_KEY(ts) ((ts) == INT64_MIN) +#define IS_VALID_WIN_KEY(ts) ((ts) != INT64_MIN) +#define SET_WIN_KEY_INVALID(ts) ((ts) = INT64_MIN) + +#define IS_NORMAL_INTERVAL_OP(op) \ + ((op)->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL || \ + (op)->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL) + +#define IS_CONTINUE_INTERVAL_OP(op) ((op)->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_CONTINUE_INTERVAL) + +#define IS_FILL_CONST_VALUE(type) ((type == TSDB_FILL_NULL || type == TSDB_FILL_NULL_F || type == TSDB_FILL_SET_VALUE || type == TSDB_FILL_SET_VALUE_F)) + +typedef struct SSliceRowData { + TSKEY key; + char pRowVal[]; +} SSliceRowData; + +typedef struct SSlicePoint { + SWinKey key; + SSliceRowData* pLeftRow; + SSliceRowData* pRightRow; + SRowBuffPos* pResPos; +} SSlicePoint; + void setStreamOperatorState(SSteamOpBasicInfo* pBasicInfo, EStreamType type); bool needSaveStreamOperatorInfo(SSteamOpBasicInfo* pBasicInfo); void saveStreamOperatorStateComplete(SSteamOpBasicInfo* pBasicInfo); +void initStreamBasicInfo(SSteamOpBasicInfo* pBasicInfo); -void reuseOutputBuf(void* pState, SRowBuffPos* pPos, SStateStore* pAPI); +int64_t getDeleteMarkFromOption(SStreamNodeOption* pOption); +void removeDeleteResults(SSHashObj* pUpdatedMap, SArray* pDelWins); +int32_t copyIntervalDeleteKey(SSHashObj* pMap, SArray* pWins); +bool hasSrcPrimaryKeyCol(SSteamOpBasicInfo* pInfo); +int32_t getNexWindowPos(SInterval* pInterval, SDataBlockInfo* pBlockInfo, TSKEY* tsCols, int32_t startPos, TSKEY eKey, + STimeWindow* pNextWin); +int32_t saveWinResult(SWinKey* pKey, SRowBuffPos* pPos, SSHashObj* pUpdatedMap); +void doBuildDeleteResultImpl(SStateStore* pAPI, SStreamState* pState, SArray* pWins, int32_t* index, + SSDataBlock* pBlock); + +SStreamFillInfo* initStreamFillInfo(SStreamFillSupporter* pFillSup, SSDataBlock* pRes); +SResultCellData* getResultCell(SResultRowData* pRaw, int32_t index); + +void destroyStreamFillSupporter(SStreamFillSupporter* pFillSup); +bool hasCurWindow(SStreamFillSupporter* pFillSup); +bool hasPrevWindow(SStreamFillSupporter* pFillSup); +bool hasNextWindow(SStreamFillSupporter* pFillSup); +void copyNotFillExpData(SStreamFillSupporter* pFillSup, SStreamFillInfo* pFillInfo); +int32_t setRowCell(SColumnInfoData* pCol, int32_t rowId, const SResultCellData* pCell); +bool hasRemainCalc(SStreamFillInfo* pFillInfo); +void destroySPoint(void* ptr); +void destroyStreamFillInfo(SStreamFillInfo* pFillInfo); +int32_t checkResult(SStreamFillSupporter* pFillSup, TSKEY ts, uint64_t groupId, bool* pRes); +void resetStreamFillSup(SStreamFillSupporter* pFillSup); +void setPointBuff(SSlicePoint* pPoint, SStreamFillSupporter* pFillSup); + +int32_t saveTimeSliceWinResult(SWinKey* pKey, SSHashObj* pUpdatedMap); + +int winPosCmprImpl(const void* pKey1, const void* pKey2); + +void reuseOutputBuf(void* pState, SRowBuffPos* pPos, SStateStore* pAPI); +SResultCellData* getSliceResultCell(SResultCellData* pRowVal, int32_t index); +int32_t getDownstreamRes(struct SOperatorInfo* downstream, SSDataBlock** ppRes, SColumnInfo** ppPkCol); +void destroyFlusedppPos(void* ppRes); +void doBuildStreamIntervalResult(struct SOperatorInfo* pOperator, void* pState, SSDataBlock* pBlock, + SGroupResInfo* pGroupResInfo); +void transBlockToSliceResultRow(const SSDataBlock* pBlock, int32_t rowId, TSKEY ts, SSliceRowData* pRowVal, + int32_t rowSize, void* pPkData, SColumnInfoData* pPkCol); +int32_t getQualifiedRowNumDesc(SExprSupp* pExprSup, SSDataBlock* pBlock, TSKEY* tsCols, int32_t rowId, bool ignoreNull); + +int32_t createStreamIntervalSliceOperatorInfo(struct SOperatorInfo* downstream, SPhysiNode* pPhyNode, + SExecTaskInfo* pTaskInfo, SReadHandle* pHandle, + struct SOperatorInfo** ppOptInfo); +int32_t buildAllResultKey(SStreamAggSupporter* pAggSup, TSKEY ts, SArray* pUpdated); #ifdef __cplusplus } diff --git a/source/libs/executor/inc/tfill.h b/source/libs/executor/inc/tfill.h index 31ac5689f6..6072063bbf 100644 --- a/source/libs/executor/inc/tfill.h +++ b/source/libs/executor/inc/tfill.h @@ -107,7 +107,9 @@ typedef struct SStreamFillInfo { TSKEY end; // endKey for fill TSKEY current; // current Key for fill TSKEY preRowKey; + TSKEY prePointKey; TSKEY nextRowKey; + TSKEY nextPointKey; SResultRowData* pResRow; SStreamFillLinearInfo* pLinearInfo; bool needFill; @@ -116,6 +118,8 @@ typedef struct SStreamFillInfo { SArray* delRanges; int32_t delIndex; uint64_t curGroupId; + bool hasNext; + SResultRowData* pNonFillRow; } SStreamFillInfo; int64_t getNumOfResultsAfterFillGap(SFillInfo* pFillInfo, int64_t ekey, int32_t maxNumOfRows); diff --git a/source/libs/executor/src/executil.c b/source/libs/executor/src/executil.c index 8cd573e971..814833fca8 100644 --- a/source/libs/executor/src/executil.c +++ b/source/libs/executor/src/executil.c @@ -450,8 +450,8 @@ int32_t prepareDataBlockBuf(SSDataBlock* pDataBlock, SColMatchInfo* pMatchInfo) EDealRes doTranslateTagExpr(SNode** pNode, void* pContext) { STransTagExprCtx* pCtx = pContext; - SMetaReader* mr = pCtx->pReader; - bool isTagCol = false, isTbname = false; + SMetaReader* mr = pCtx->pReader; + bool isTagCol = false, isTbname = false; if (nodeType(*pNode) == QUERY_NODE_COLUMN) { SColumnNode* pCol = (SColumnNode*)*pNode; if (pCol->colType == COLUMN_TYPE_TBNAME) @@ -573,7 +573,7 @@ int32_t isQualifiedTable(STableKeyInfo* info, SNode* pTagCond, void* metaHandle, static EDealRes getColumn(SNode** pNode, void* pContext) { tagFilterAssist* pData = (tagFilterAssist*)pContext; - SColumnNode* pSColumnNode = NULL; + SColumnNode* pSColumnNode = NULL; if (QUERY_NODE_COLUMN == nodeType((*pNode))) { pSColumnNode = *(SColumnNode**)pNode; } else if (QUERY_NODE_FUNCTION == nodeType((*pNode))) { @@ -596,7 +596,7 @@ static EDealRes getColumn(SNode** pNode, void* pContext) { return DEAL_RES_CONTINUE; } - void* data = taosHashGet(pData->colHash, &pSColumnNode->colId, sizeof(pSColumnNode->colId)); + void* data = taosHashGet(pData->colHash, &pSColumnNode->colId, sizeof(pSColumnNode->colId)); if (!data) { int32_t tempRes = taosHashPut(pData->colHash, &pSColumnNode->colId, sizeof(pSColumnNode->colId), pNode, sizeof((*pNode))); @@ -1637,7 +1637,7 @@ int32_t getGroupIdFromTagsVal(void* pVnode, uint64_t uid, SNodeList* pGroupNode, } SNodeList* groupNew = NULL; - int32_t code = nodesCloneList(pGroupNode, &groupNew); + int32_t code = nodesCloneList(pGroupNode, &groupNew); if (TSDB_CODE_SUCCESS != code) { pAPI->metaReaderFn.clearReader(&mr); return code; @@ -1919,7 +1919,7 @@ int32_t createExprFromOneNode(SExprInfo* pExp, SNode* pNode, int16_t slotId) { if (!pFuncNode->pParameterList && (memcmp(pExprNode->_function.functionName, name, len) == 0) && pExprNode->_function.functionName[len] == 0) { pFuncNode->pParameterList = NULL; - int32_t code = nodesMakeList(&pFuncNode->pParameterList); + int32_t code = nodesMakeList(&pFuncNode->pParameterList); SValueNode* res = NULL; if (TSDB_CODE_SUCCESS == code) { code = nodesMakeNode(QUERY_NODE_VALUE, (SNode**)&res); @@ -2925,6 +2925,10 @@ char* getStreamOpName(uint16_t opType) { return "stream event"; case QUERY_NODE_PHYSICAL_PLAN_STREAM_COUNT: return "stream count"; + case QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERP_FUNC: + return "stream interp"; + case QUERY_NODE_PHYSICAL_PLAN_STREAM_CONTINUE_INTERVAL: + return "interval continue"; } return ""; } @@ -2952,7 +2956,9 @@ void printSpecDataBlock(SSDataBlock* pBlock, const char* flag, const char* opStr qDebug("%s===stream===%s %s: Block is Null", taskIdStr, flag, opStr); return; } else if (pBlock->info.rows == 0) { - qDebug("%s===stream===%s %s: Block is Empty. block type %d", taskIdStr, flag, opStr, pBlock->info.type); + qDebug("%s===stream===%s %s: Block is Empty. block type %d.skey:%" PRId64 ",ekey:%" PRId64 ",version%" PRId64, + taskIdStr, flag, opStr, pBlock->info.type, pBlock->info.window.skey, pBlock->info.window.ekey, + pBlock->info.version); return; } if (qDebugFlag & DEBUG_DEBUG) { diff --git a/source/libs/executor/src/executor.c b/source/libs/executor/src/executor.c index ba0f55f825..a0e31a8a3e 100644 --- a/source/libs/executor/src/executor.c +++ b/source/libs/executor/src/executor.c @@ -131,7 +131,7 @@ static void clearStreamBlock(SOperatorInfo* pOperator) { } } -void resetTaskInfo(qTaskInfo_t tinfo) { +void qResetTaskInfoCode(qTaskInfo_t tinfo) { SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo; pTaskInfo->code = 0; clearStreamBlock(pTaskInfo->pRoot); @@ -1098,6 +1098,23 @@ _end: return code; } +static int32_t getOpratorIntervalInfo(SOperatorInfo* pOperator, int64_t* pWaterMark, SInterval* pInterval, STimeWindow* pLastWindow) { + if (pOperator->operatorType != QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN) { + return getOpratorIntervalInfo(pOperator->pDownstream[0], pWaterMark, pInterval, pLastWindow); + } + SStreamScanInfo* pScanOp = (SStreamScanInfo*) pOperator->info; + *pWaterMark = pScanOp->twAggSup.waterMark; + *pInterval = pScanOp->interval; + *pLastWindow = pScanOp->lastScanRange; + return TSDB_CODE_SUCCESS; +} + +int32_t qGetStreamIntervalExecInfo(qTaskInfo_t tinfo, int64_t* pWaterMark, SInterval* pInterval, STimeWindow* pLastWindow) { + SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo; + SOperatorInfo* pOperator = pTaskInfo->pRoot; + return getOpratorIntervalInfo(pOperator, pWaterMark, pInterval, pLastWindow); +} + int32_t qSetStreamOperatorOptionForScanHistory(qTaskInfo_t tinfo) { SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo; SOperatorInfo* pOperator = pTaskInfo->pRoot; @@ -1162,6 +1179,19 @@ int32_t qSetStreamOperatorOptionForScanHistory(qTaskInfo_t tinfo) { qInfo("save stream param for state: %d, %" PRId64, pSup->calTrigger, pSup->deleteMark); + pSup->calTriggerSaved = pSup->calTrigger; + pSup->deleteMarkSaved = pSup->deleteMark; + pSup->calTrigger = STREAM_TRIGGER_AT_ONCE; + pSup->deleteMark = INT64_MAX; + pInfo->ignoreExpiredDataSaved = pInfo->ignoreExpiredData; + pInfo->ignoreExpiredData = false; + qInfo("save stream task:%s, param for state: %d", GET_TASKID(pTaskInfo), pInfo->ignoreExpiredData); + } else if (type == QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERP_FUNC) { + SStreamTimeSliceOperatorInfo* pInfo = pOperator->info; + STimeWindowAggSupp* pSup = &pInfo->twAggSup; + + qInfo("save stream param for state: %d, %" PRId64, pSup->calTrigger, pSup->deleteMark); + pSup->calTriggerSaved = pSup->calTrigger; pSup->deleteMarkSaved = pSup->deleteMark; pSup->calTrigger = STREAM_TRIGGER_AT_ONCE; diff --git a/source/libs/executor/src/groupoperator.c b/source/libs/executor/src/groupoperator.c index e4db766a6a..fec35c3371 100644 --- a/source/libs/executor/src/groupoperator.c +++ b/source/libs/executor/src/groupoperator.c @@ -1508,6 +1508,7 @@ static int32_t doStreamHashPartitionNext(SOperatorInfo* pOperator, SSDataBlock** case STREAM_CREATE_CHILD_TABLE: case STREAM_RETRIEVE: case STREAM_CHECKPOINT: + case STREAM_GET_RESULT: case STREAM_GET_ALL: { (*ppRes) = pBlock; return code; diff --git a/source/libs/executor/src/operator.c b/source/libs/executor/src/operator.c index f88b544b0f..3b10dce63f 100644 --- a/source/libs/executor/src/operator.c +++ b/source/libs/executor/src/operator.c @@ -631,6 +631,8 @@ int32_t createOperator(SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo, SReadHand code = createStreamCountAggOperatorInfo(ops[0], pPhyNode, pTaskInfo, pHandle, &pOptr); } else if (QUERY_NODE_PHYSICAL_PLAN_MERGE_COUNT == type) { code = createCountwindowOperatorInfo(ops[0], pPhyNode, pTaskInfo, &pOptr); + } else if (QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERP_FUNC == type) { + code = createStreamTimeSliceOperatorInfo(ops[0], pPhyNode, pTaskInfo, pHandle, &pOptr); } else if (QUERY_NODE_PHYSICAL_PLAN_MERGE_ANOMALY == type) { code = createAnomalywindowOperatorInfo(ops[0], pPhyNode, pTaskInfo, &pOptr); } else { diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 727aede9f4..eac95e1a25 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -25,6 +25,7 @@ #include "tdatablock.h" #include "tmsg.h" +#include "ttime.h" #include "operator.h" #include "query.h" @@ -1649,10 +1650,18 @@ static bool isCountWindow(SStreamScanInfo* pInfo) { return pInfo->windowSup.parentType == QUERY_NODE_PHYSICAL_PLAN_STREAM_COUNT; } +static bool isTimeSlice(SStreamScanInfo* pInfo) { + return pInfo->windowSup.parentType == QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERP_FUNC; +} + static void setGroupId(SStreamScanInfo* pInfo, SSDataBlock* pBlock, int32_t groupColIndex, int32_t rowIndex) { SColumnInfoData* pColInfo = taosArrayGet(pBlock->pDataBlock, groupColIndex); uint64_t* groupCol = (uint64_t*)pColInfo->pData; - pInfo->groupId = groupCol[rowIndex]; + if (colDataIsNull_s(pColInfo, rowIndex)) { + pInfo->igCheckGroupId = true; + } else { + pInfo->groupId = groupCol[rowIndex]; + } } void resetTableScanInfo(STableScanInfo* pTableScanInfo, STimeWindow* pWin, uint64_t ver) { @@ -1926,6 +1935,12 @@ static int32_t doRangeScan(SStreamScanInfo* pInfo, SSDataBlock* pSDB, int32_t ts continue; } + if (pInfo->igCheckGroupId == true) { + pResult->info.calWin = pInfo->updateWin; + (*ppRes) = pResult; + goto _end; + } + if (pInfo->partitionSup.needCalc) { SSDataBlock* tmpBlock = NULL; code = createOneDataBlock(pResult, true, &tmpBlock); @@ -2000,10 +2015,10 @@ int32_t appendOneRowToSpecialBlockImpl(SSDataBlock* pBlock, TSKEY* pStartTs, TSK code = colDataSetVal(pEndTsCol, pBlock->info.rows, (const char*)pEndTs, false); QUERY_CHECK_CODE(code, lino, _end); - code = colDataSetVal(pUidCol, pBlock->info.rows, (const char*)pUid, false); + code = colDataSetVal(pUidCol, pBlock->info.rows, (const char*)pUid, pUid == NULL); QUERY_CHECK_CODE(code, lino, _end); - code = colDataSetVal(pGpCol, pBlock->info.rows, (const char*)pGp, false); + code = colDataSetVal(pGpCol, pBlock->info.rows, (const char*)pGp, pGp == NULL); QUERY_CHECK_CODE(code, lino, _end); code = colDataSetVal(pCalStartCol, pBlock->info.rows, (const char*)pCalStartTs, false); @@ -2247,6 +2262,167 @@ _end: return code; } +static int32_t setDelRangeEndKey(SStreamAggSupporter* pAggSup, SStreamFillSupporter* pFillSup, SWinKey* pEndKey, STimeWindow* pScanRange, bool* pRes) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + SSlicePoint nextPoint = {.key.groupId = pEndKey->groupId}; + int32_t vLen = 0; + int32_t winCode = TSDB_CODE_SUCCESS; + code = pAggSup->stateStore.streamStateFillGetNext(pAggSup->pState, pEndKey, &nextPoint.key, (void**)&nextPoint.pResPos, &vLen, &winCode); + QUERY_CHECK_CODE(code, lino, _end); + if (winCode == TSDB_CODE_SUCCESS) { + setPointBuff(&nextPoint, pFillSup); + if (HAS_ROW_DATA(nextPoint.pLeftRow) && pEndKey->ts < nextPoint.pLeftRow->key) { + pScanRange->ekey = nextPoint.pLeftRow->key; + *pRes = true; + } else if (pEndKey->ts < nextPoint.pRightRow->key) { + pScanRange->ekey = nextPoint.pRightRow->key; + *pRes = true; + } else { + *pEndKey = nextPoint.key; + pScanRange->ekey = TMAX(nextPoint.pRightRow->key, nextPoint.key.ts); + *pRes = false; + } + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +int32_t getTimeSliceWinRange(SStreamAggSupporter* pAggSup, SStreamFillSupporter* pFillSup, SInterval* pInterval, TSKEY start, TSKEY end, + int64_t groupId, STimeWindow* pScanRange) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + int32_t winCode = TSDB_CODE_SUCCESS; + SResultRowInfo dumyInfo = {0}; + dumyInfo.cur.pageId = -1; + STimeWindow sWin = getActiveTimeWindow(NULL, &dumyInfo, start, pInterval, TSDB_ORDER_ASC); + SWinKey startKey = {.groupId = groupId, .ts = sWin.skey}; + + sWin = getActiveTimeWindow(NULL, &dumyInfo, end, pInterval, TSDB_ORDER_ASC); + SWinKey endKey = {.groupId = groupId, .ts = sWin.ekey}; + + SSlicePoint prevPoint = {.key.groupId = groupId}; + SSlicePoint nextPoint = {.key.groupId = groupId}; + int32_t vLen = 0; + code = pAggSup->stateStore.streamStateFillGetPrev(pAggSup->pState, &startKey, &prevPoint.key, (void**)&prevPoint.pResPos, &vLen, &winCode); + QUERY_CHECK_CODE(code, lino, _end); + if (winCode == TSDB_CODE_SUCCESS) { + setPointBuff(&prevPoint, pFillSup); + if (HAS_ROW_DATA(prevPoint.pRightRow)) { + pScanRange->skey = prevPoint.pRightRow->key; + } else { + pScanRange->skey = prevPoint.pLeftRow->key; + } + } else { + pScanRange->skey = startKey.ts; + } + + bool res = false; + SWinKey curKey = endKey; + code = setDelRangeEndKey(pAggSup, pFillSup, &curKey, pScanRange, &res); + QUERY_CHECK_CODE(code, lino, _end); + if (res == false) { + code = setDelRangeEndKey(pAggSup, pFillSup, &curKey, pScanRange, &res); + QUERY_CHECK_CODE(code, lino, _end); + } + if (res == false) { + pScanRange->ekey = TMAX(endKey.ts, pScanRange->ekey); + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +static int32_t generateTimeSliceScanRange(SStreamScanInfo* pInfo, SSDataBlock* pSrcBlock, SSDataBlock* pDestBlock, + EStreamType mode) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + blockDataCleanup(pDestBlock); + if (pSrcBlock->info.rows == 0) { + return TSDB_CODE_SUCCESS; + } + SExecTaskInfo* pTaskInfo = pInfo->pStreamScanOp->pTaskInfo; + SColumnInfoData* pStartTsCol = taosArrayGet(pSrcBlock->pDataBlock, START_TS_COLUMN_INDEX); + TSKEY* startData = (TSKEY*)pStartTsCol->pData; + SColumnInfoData* pEndTsCol = taosArrayGet(pSrcBlock->pDataBlock, END_TS_COLUMN_INDEX); + TSKEY* endData = (TSKEY*)pEndTsCol->pData; + SColumnInfoData* pUidCol = taosArrayGet(pSrcBlock->pDataBlock, UID_COLUMN_INDEX); + uint64_t* uidCol = (uint64_t*)pUidCol->pData; + SColumnInfoData* pGpCol = taosArrayGet(pSrcBlock->pDataBlock, GROUPID_COLUMN_INDEX); + uint64_t* pSrcGp = (uint64_t*)pGpCol->pData; + SColumnInfoData* pSrcPkCol = NULL; + if (taosArrayGetSize(pSrcBlock->pDataBlock) > PRIMARY_KEY_COLUMN_INDEX) { + pSrcPkCol = taosArrayGet(pSrcBlock->pDataBlock, PRIMARY_KEY_COLUMN_INDEX); + } + int64_t ver = pSrcBlock->info.version - 1; + + if (pInfo->partitionSup.needCalc && + (startData[0] != endData[0] || (hasPrimaryKeyCol(pInfo) && mode == STREAM_DELETE_DATA))) { + code = getPreVersionDataBlock(uidCol[0], startData[0], endData[0], ver, GET_TASKID(pTaskInfo), pInfo, pSrcBlock); + QUERY_CHECK_CODE(code, lino, _end); + startData = (TSKEY*)pStartTsCol->pData; + endData = (TSKEY*)pEndTsCol->pData; + uidCol = (uint64_t*)pUidCol->pData; + pSrcGp = (uint64_t*)pGpCol->pData; + } + + code = blockDataEnsureCapacity(pDestBlock, pSrcBlock->info.rows); + QUERY_CHECK_CODE(code, lino, _end); + + SColumnInfoData* pDestStartCol = taosArrayGet(pDestBlock->pDataBlock, START_TS_COLUMN_INDEX); + SColumnInfoData* pDestEndCol = taosArrayGet(pDestBlock->pDataBlock, END_TS_COLUMN_INDEX); + SColumnInfoData* pDestUidCol = taosArrayGet(pDestBlock->pDataBlock, UID_COLUMN_INDEX); + SColumnInfoData* pDestGpCol = taosArrayGet(pDestBlock->pDataBlock, GROUPID_COLUMN_INDEX); + SColumnInfoData* pDestCalStartTsCol = taosArrayGet(pDestBlock->pDataBlock, CALCULATE_START_TS_COLUMN_INDEX); + SColumnInfoData* pDestCalEndTsCol = taosArrayGet(pDestBlock->pDataBlock, CALCULATE_END_TS_COLUMN_INDEX); + for (int32_t i = 0; i < pSrcBlock->info.rows; i++) { + uint64_t groupId = pSrcGp[i]; + if (groupId == 0) { + void* pVal = NULL; + if (hasPrimaryKeyCol(pInfo) && pSrcPkCol) { + pVal = colDataGetData(pSrcPkCol, i); + } + groupId = getGroupIdByData(pInfo, uidCol[i], startData[i], ver, pVal); + } + + STimeWindow scanRange = {0}; + code = getTimeSliceWinRange(pInfo->windowSup.pStreamAggSup, pInfo->pFillSup, &pInfo->interval, startData[i], endData[i], groupId, + &scanRange); + QUERY_CHECK_CODE(code, lino, _end); + + code = colDataSetVal(pDestStartCol, i, (const char*)&scanRange.skey, false); + QUERY_CHECK_CODE(code, lino, _end); + + code = colDataSetVal(pDestEndCol, i, (const char*)&scanRange.ekey, false); + QUERY_CHECK_CODE(code, lino, _end); + + colDataSetNULL(pDestUidCol, i); + code = colDataSetVal(pDestGpCol, i, (const char*)&groupId, false); + QUERY_CHECK_CODE(code, lino, _end); + + code = colDataSetVal(pDestCalStartTsCol, i, (const char*)&scanRange.skey, false); + QUERY_CHECK_CODE(code, lino, _end); + + code = colDataSetVal(pDestCalEndTsCol, i, (const char*)&scanRange.ekey, false); + QUERY_CHECK_CODE(code, lino, _end); + + pDestBlock->info.rows++; + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + static int32_t generateIntervalScanRange(SStreamScanInfo* pInfo, SSDataBlock* pSrcBlock, SSDataBlock* pDestBlock, EStreamType mode) { int32_t code = TSDB_CODE_SUCCESS; @@ -2504,6 +2680,9 @@ static int32_t generateScanRange(SStreamScanInfo* pInfo, SSDataBlock* pSrcBlock, } else if (isCountWindow(pInfo)) { code = generateCountScanRange(pInfo, pSrcBlock, pDestBlock, type); QUERY_CHECK_CODE(code, lino, _end); + } else if (isTimeSlice(pInfo)) { + code = generateTimeSliceScanRange(pInfo, pSrcBlock, pDestBlock, type); + QUERY_CHECK_CODE(code, lino, _end); } else { code = generateDeleteResultBlock(pInfo, pSrcBlock, pDestBlock); QUERY_CHECK_CODE(code, lino, _end); @@ -3155,31 +3334,78 @@ _end: } int32_t streamScanOperatorEncode(SStreamScanInfo* pInfo, void** pBuff, int32_t* pLen) { - int32_t code = TSDB_CODE_SUCCESS; - int32_t lino = 0; - int32_t len = 0; - code = pInfo->stateStore.updateInfoSerialize(NULL, 0, pInfo->pUpdateInfo, &len); - QUERY_CHECK_CODE(code, lino, _end); + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + int32_t len = 0; + SEncoder* pEnCoder = NULL; + SEncoder* pScanEnCoder = NULL; len += encodeSTimeWindowAggSupp(NULL, &pInfo->twAggSup); + SEncoder encoder = {0}; + pEnCoder = &encoder; + tEncoderInit(pEnCoder, NULL, 0); + if (tStartEncode(pEnCoder) != 0) { + code = TSDB_CODE_STREAM_INTERNAL_ERROR; + QUERY_CHECK_CODE(code, lino, _end); + } + code = pInfo->stateStore.updateInfoSerialize(pEnCoder, pInfo->pUpdateInfo); + QUERY_CHECK_CODE(code, lino, _end); + + if (tEncodeI64(pEnCoder, pInfo->lastScanRange.skey) < 0) { + code = TSDB_CODE_STREAM_INTERNAL_ERROR; + QUERY_CHECK_CODE(code, lino, _end); + } + if (tEncodeI64(pEnCoder, pInfo->lastScanRange.ekey) < 0) { + code = TSDB_CODE_STREAM_INTERNAL_ERROR; + QUERY_CHECK_CODE(code, lino, _end); + } + + tEndEncode(pEnCoder); + len += encoder.pos; + tEncoderClear(pEnCoder); + pEnCoder = NULL; + *pBuff = taosMemoryCalloc(1, len); if (!(*pBuff)) { code = terrno; QUERY_CHECK_CODE(code, lino, _end); } void* buf = *pBuff; - (void)encodeSTimeWindowAggSupp(&buf, &pInfo->twAggSup); + int32_t stwLen = encodeSTimeWindowAggSupp(&buf, &pInfo->twAggSup); - int32_t tmp = 0; - code = pInfo->stateStore.updateInfoSerialize(buf, len, pInfo->pUpdateInfo, &tmp); + SEncoder scanEncoder = {0}; + pScanEnCoder = &scanEncoder; + tEncoderInit(pScanEnCoder, buf, len - stwLen); + if (tStartEncode(pScanEnCoder) != 0) { + code = TSDB_CODE_FAILED; + QUERY_CHECK_CODE(code, lino, _end); + } + code = pInfo->stateStore.updateInfoSerialize(pScanEnCoder, pInfo->pUpdateInfo); QUERY_CHECK_CODE(code, lino, _end); + if (tEncodeI64(pScanEnCoder, pInfo->lastScanRange.skey) < 0) { + code = TSDB_CODE_STREAM_INTERNAL_ERROR; + QUERY_CHECK_CODE(code, lino, _end); + } + if (tEncodeI64(pScanEnCoder, pInfo->lastScanRange.ekey) < 0) { + code = TSDB_CODE_STREAM_INTERNAL_ERROR; + QUERY_CHECK_CODE(code, lino, _end); + } + *pLen = len; _end: if (code != TSDB_CODE_SUCCESS) { qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); } + if (pEnCoder != NULL) { + tEndEncode(pEnCoder); + tEncoderClear(pEnCoder); + } + if (pScanEnCoder != NULL) { + tEndEncode(pScanEnCoder); + tEncoderClear(pScanEnCoder); + } return code; } @@ -3208,35 +3434,86 @@ _end: // other properties are recovered from the execution plan void streamScanOperatorDecode(void* pBuff, int32_t len, SStreamScanInfo* pInfo) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + SDecoder* pDeCoder = NULL; if (!pBuff || len == 0) { - return; + lino = __LINE__; + goto _end; } void* buf = pBuff; buf = decodeSTimeWindowAggSupp(buf, &pInfo->twAggSup); int32_t tlen = len - encodeSTimeWindowAggSupp(NULL, &pInfo->twAggSup); if (tlen == 0) { - return; + lino = __LINE__; + goto _end; } void* pUpInfo = taosMemoryCalloc(1, sizeof(SUpdateInfo)); if (!pUpInfo) { - return; + lino = __LINE__; + goto _end; } - int32_t code = pInfo->stateStore.updateInfoDeserialize(buf, tlen, pUpInfo); + SDecoder decoder = {0}; + pDeCoder = &decoder; + tDecoderInit(pDeCoder, buf, tlen); + if (tStartDecode(pDeCoder) < 0) { + lino = __LINE__; + goto _end; + } + + code = pInfo->stateStore.updateInfoDeserialize(pDeCoder, pUpInfo); if (code == TSDB_CODE_SUCCESS) { pInfo->stateStore.updateInfoDestroy(pInfo->pUpdateInfo); pInfo->pUpdateInfo = pUpInfo; } else { taosMemoryFree(pUpInfo); + lino = __LINE__; + goto _end; } + + if (tDecodeIsEnd(pDeCoder)) { + lino = __LINE__; + goto _end; + } + + SET_WIN_KEY_INVALID(pInfo->lastScanRange.skey); + SET_WIN_KEY_INVALID(pInfo->lastScanRange.ekey); + + if (tDecodeI64(pDeCoder, &pInfo->lastScanRange.skey) < 0) { + lino = __LINE__; + goto _end; + } + + if (tDecodeI64(pDeCoder, &pInfo->lastScanRange.ekey) < 0) { + lino = __LINE__; + goto _end; + } + +_end: + if (pDeCoder != NULL) { + tEndDecode(pDeCoder); + tDecoderClear(pDeCoder); + } + qInfo("%s end at line %d", __func__, lino); } + static bool hasScanRange(SStreamScanInfo* pInfo) { SStreamAggSupporter* pSup = pInfo->windowSup.pStreamAggSup; return pSup && pSup->pScanBlock->info.rows > 0 && (isStateWindow(pInfo) || isCountWindow(pInfo)); } static bool isStreamWindow(SStreamScanInfo* pInfo) { - return isIntervalWindow(pInfo) || isSessionWindow(pInfo) || isStateWindow(pInfo) || isCountWindow(pInfo); + return isIntervalWindow(pInfo) || isSessionWindow(pInfo) || isStateWindow(pInfo) || isCountWindow(pInfo) || + isTimeSlice(pInfo); +} + +static int32_t copyGetResultBlock(SSDataBlock* dest, TSKEY start, TSKEY end) { + int32_t code = blockDataEnsureCapacity(dest, 1); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + return appendDataToSpecialBlock(dest, &start, &end, NULL, NULL, NULL); } static int32_t doStreamScanNext(SOperatorInfo* pOperator, SSDataBlock** ppRes) { @@ -3464,13 +3741,27 @@ FETCH_NEXT_BLOCK: } } } break; + case STREAM_GET_RESULT: { + pInfo->blockType = STREAM_INPUT__DATA_SUBMIT; + pInfo->updateResIndex = 0; + pInfo->lastScanRange = pBlock->info.window; + TSKEY endKey = taosTimeGetIntervalEnd(pBlock->info.window.skey, &pInfo->interval); + if (pInfo->useGetResultRange == true) { + endKey = pBlock->info.window.ekey; + } + code = copyGetResultBlock(pInfo->pUpdateRes, pBlock->info.window.skey, endKey); + QUERY_CHECK_CODE(code, lino, _end); + pInfo->pUpdateInfo->maxDataVersion = -1; + prepareRangeScan(pInfo, pInfo->pUpdateRes, &pInfo->updateResIndex, NULL); + pInfo->scanMode = STREAM_SCAN_FROM_DATAREADER_RANGE; + } break; case STREAM_CHECKPOINT: { qError("stream check point error. msg type: STREAM_INPUT__DATA_BLOCK"); } break; default: break; } - printDataBlock(pBlock, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); + printSpecDataBlock(pBlock, getStreamOpName(pOperator->operatorType), "block recv", GET_TASKID(pTaskInfo)); setStreamOperatorState(&pInfo->basic, pBlock->info.type); (*ppRes) = pBlock; return code; @@ -3964,6 +4255,8 @@ void streamScanReleaseState(SOperatorInfo* pOperator) { int32_t lino = 0; SStreamScanInfo* pInfo = pOperator->info; void* pBuff = NULL; + SEncoder* pEnCoder = NULL; + SEncoder* pScanEnCoder = NULL; if (!pInfo->pState) { return; } @@ -3971,32 +4264,63 @@ void streamScanReleaseState(SOperatorInfo* pOperator) { qDebug("stask:%s streamScanReleaseState cancel", GET_TASKID(pOperator->pTaskInfo)); return; } - int32_t len = 0; - code = pInfo->stateStore.updateInfoSerialize(NULL, 0, pInfo->pUpdateInfo, &len); + int32_t len = 0; + SEncoder encoder = {0}; + pEnCoder = &encoder; + tEncoderInit(pEnCoder, NULL, 0); + if (tStartEncode(pEnCoder) != 0) { + code = TSDB_CODE_FAILED; + QUERY_CHECK_CODE(code, lino, _end); + } + code = pInfo->stateStore.updateInfoSerialize(pEnCoder, pInfo->pUpdateInfo); QUERY_CHECK_CODE(code, lino, _end); + tEndEncode(pEnCoder); + len += encoder.pos; + tEncoderClear(pEnCoder); + pEnCoder = NULL; + pBuff = taosMemoryCalloc(1, len); if (!pBuff) { code = terrno; QUERY_CHECK_CODE(code, lino, _end); } - int32_t tmp = 0; - code = pInfo->stateStore.updateInfoSerialize(pBuff, len, pInfo->pUpdateInfo, &tmp); + SEncoder scanEncoder = {0}; + pScanEnCoder = &scanEncoder; + tEncoderInit(pScanEnCoder, pBuff, len); + if (tStartEncode(pScanEnCoder) != 0) { + code = TSDB_CODE_FAILED; + QUERY_CHECK_CODE(code, lino, _end); + } + code = pInfo->stateStore.updateInfoSerialize(pScanEnCoder, pInfo->pUpdateInfo); QUERY_CHECK_CODE(code, lino, _end); + tEndEncode(pScanEnCoder); + tEncoderClear(pScanEnCoder); + pScanEnCoder = NULL; + pInfo->stateStore.streamStateSaveInfo(pInfo->pState, STREAM_SCAN_OP_STATE_NAME, strlen(STREAM_SCAN_OP_STATE_NAME), pBuff, len); _end: if (code != TSDB_CODE_SUCCESS) { qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); } + if (pEnCoder != NULL) { + tEndEncode(pEnCoder); + tEncoderClear(pEnCoder); + } + if (pScanEnCoder != NULL) { + tEndEncode(pScanEnCoder); + tEncoderClear(pScanEnCoder); + } taosMemoryFree(pBuff); } void streamScanReloadState(SOperatorInfo* pOperator) { int32_t code = TSDB_CODE_SUCCESS; int32_t lino = 0; + SDecoder* pDeCoder = NULL; SStreamScanInfo* pInfo = pOperator->info; if (!pInfo->pState) { return; @@ -4017,7 +4341,14 @@ void streamScanReloadState(SOperatorInfo* pOperator) { QUERY_CHECK_CODE(code, lino, _end); } - int32_t winCode = pInfo->stateStore.updateInfoDeserialize(pBuff, len, pUpInfo); + SDecoder decoder = {0}; + pDeCoder = &decoder; + tDecoderInit(pDeCoder, pBuff, len); + if (tStartDecode(pDeCoder) < 0) { + lino = __LINE__; + goto _end; + } + int32_t winCode = pInfo->stateStore.updateInfoDeserialize(pDeCoder, pUpInfo); taosMemoryFree(pBuff); if (winCode == TSDB_CODE_SUCCESS && pInfo->pUpdateInfo) { if (pInfo->pUpdateInfo->minTS < 0) { @@ -4054,6 +4385,10 @@ void streamScanReloadState(SOperatorInfo* pOperator) { } _end: + if (pDeCoder != NULL) { + tEndDecode(pDeCoder); + tDecoderClear(pDeCoder); + } if (code != TSDB_CODE_SUCCESS) { qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); } @@ -4244,6 +4579,7 @@ int32_t createStreamScanOperatorInfo(SReadHandle* pHandle, STableScanPhysiNode* pInfo->scanMode = STREAM_SCAN_FROM_READERHANDLE; pInfo->windowSup = (SWindowSupporter){.pStreamAggSup = NULL, .gap = -1, .parentType = QUERY_NODE_PHYSICAL_PLAN}; pInfo->groupId = 0; + pInfo->igCheckGroupId = false; pInfo->pStreamScanOp = pOperator; pInfo->deleteDataIndex = 0; code = createSpecialDataBlock(STREAM_DELETE_DATA, &pInfo->pDeleteDataRes); @@ -4269,10 +4605,14 @@ int32_t createStreamScanOperatorInfo(SReadHandle* pHandle, STableScanPhysiNode* pInfo->pState = pTaskInfo->streamInfo.pState; pInfo->stateStore = pTaskInfo->storageAPI.stateStore; pInfo->readerFn = pTaskInfo->storageAPI.tqReaderFn; + pInfo->pFillSup = NULL; + pInfo->useGetResultRange = false; code = createSpecialDataBlock(STREAM_CHECKPOINT, &pInfo->pCheckpointRes); QUERY_CHECK_CODE(code, lino, _error); + SET_WIN_KEY_INVALID(pInfo->lastScanRange.skey); + SET_WIN_KEY_INVALID(pInfo->lastScanRange.ekey); // for stream if (pTaskInfo->streamInfo.pState) { void* buff = NULL; diff --git a/source/libs/executor/src/streamcountwindowoperator.c b/source/libs/executor/src/streamcountwindowoperator.c index 33b3e7748c..b8c3ec90f9 100644 --- a/source/libs/executor/src/streamcountwindowoperator.c +++ b/source/libs/executor/src/streamcountwindowoperator.c @@ -818,8 +818,8 @@ _end: } } -int32_t createStreamCountAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, - SExecTaskInfo* pTaskInfo, SReadHandle* pHandle, SOperatorInfo** pOptrInfo) { +int32_t createStreamCountAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo, + SReadHandle* pHandle, SOperatorInfo** pOptrInfo) { QRY_PARAM_CHECK(pOptrInfo); SCountWinodwPhysiNode* pCountNode = (SCountWinodwPhysiNode*)pPhyNode; @@ -869,7 +869,8 @@ int32_t createStreamCountAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pInfo->primaryTsIndex = ((SColumnNode*)pCountNode->window.pTspk)->slotId; code = initStreamAggSupporter(&pInfo->streamAggSup, pExpSup, numOfCols, 0, pTaskInfo->streamInfo.pState, sizeof(COUNT_TYPE), 0, &pTaskInfo->storageAPI.stateStore, pHandle, &pInfo->twAggSup, - GET_TASKID(pTaskInfo), &pTaskInfo->storageAPI, pInfo->primaryTsIndex); + GET_TASKID(pTaskInfo), &pTaskInfo->storageAPI, pInfo->primaryTsIndex, + STREAM_STATE_BUFF_SORT, 1); QUERY_CHECK_CODE(code, lino, _error); pInfo->streamAggSup.windowCount = pCountNode->windowCount; @@ -903,7 +904,7 @@ int32_t createStreamCountAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pInfo->recvGetAll = false; pInfo->pPkDeleted = tSimpleHashInit(64, hashFn); QUERY_CHECK_NULL(pInfo->pPkDeleted, code, lino, _error, terrno); - pInfo->destHasPrimaryKey = pCountNode->window.destHasPrimayKey; + pInfo->destHasPrimaryKey = pCountNode->window.destHasPrimaryKey; pOperator->operatorType = QUERY_NODE_PHYSICAL_PLAN_STREAM_COUNT; setOperatorInfo(pOperator, getStreamOpName(pOperator->operatorType), QUERY_NODE_PHYSICAL_PLAN_STREAM_COUNT, true, diff --git a/source/libs/executor/src/streameventwindowoperator.c b/source/libs/executor/src/streameventwindowoperator.c index 8a706f6d4e..29b3f473ba 100644 --- a/source/libs/executor/src/streameventwindowoperator.c +++ b/source/libs/executor/src/streameventwindowoperator.c @@ -525,8 +525,8 @@ int32_t doStreamEventDecodeOpState(void* buf, int32_t len, SOperatorInfo* pOpera QUERY_CHECK_CODE(code, lino, _end); buf = decodeSResultWindowInfo(buf, &winfo, pInfo->streamAggSup.resultRowSize); - code = - tSimpleHashPut(pInfo->streamAggSup.pResultRows, &winfo.sessionWin, sizeof(SSessionKey), &winfo, sizeof(SResultWindowInfo)); + code = tSimpleHashPut(pInfo->streamAggSup.pResultRows, &winfo.sessionWin, sizeof(SSessionKey), &winfo, + sizeof(SResultWindowInfo)); QUERY_CHECK_CODE(code, lino, _end); } @@ -917,7 +917,8 @@ int32_t createStreamEventAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pInfo->primaryTsIndex = tsSlotId; code = initStreamAggSupporter(&pInfo->streamAggSup, pExpSup, numOfCols, 0, pTaskInfo->streamInfo.pState, sizeof(bool) + sizeof(bool), 0, &pTaskInfo->storageAPI.stateStore, pHandle, - &pInfo->twAggSup, GET_TASKID(pTaskInfo), &pTaskInfo->storageAPI, pInfo->primaryTsIndex); + &pInfo->twAggSup, GET_TASKID(pTaskInfo), &pTaskInfo->storageAPI, pInfo->primaryTsIndex, + STREAM_STATE_BUFF_SORT, 1); QUERY_CHECK_CODE(code, lino, _error); _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); @@ -955,7 +956,7 @@ int32_t createStreamEventAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pInfo->recvGetAll = false; pInfo->pPkDeleted = tSimpleHashInit(64, hashFn); QUERY_CHECK_NULL(pInfo->pPkDeleted, code, lino, _error, terrno); - pInfo->destHasPrimaryKey = pEventNode->window.destHasPrimayKey; + pInfo->destHasPrimaryKey = pEventNode->window.destHasPrimaryKey; pInfo->pOperator = pOperator; setOperatorInfo(pOperator, "StreamEventAggOperator", QUERY_NODE_PHYSICAL_PLAN_STREAM_EVENT, true, OP_NOT_OPENED, diff --git a/source/libs/executor/src/streamexecutorInt.c b/source/libs/executor/src/streamexecutorInt.c index 875ae00350..b94798934c 100644 --- a/source/libs/executor/src/streamexecutorInt.c +++ b/source/libs/executor/src/streamexecutorInt.c @@ -28,3 +28,8 @@ bool needSaveStreamOperatorInfo(SSteamOpBasicInfo* pBasicInfo) { void saveStreamOperatorStateComplete(SSteamOpBasicInfo* pBasicInfo) { pBasicInfo->updateOperatorInfo = false; } + +void initStreamBasicInfo(SSteamOpBasicInfo* pBasicInfo) { + pBasicInfo->primaryPkIndex = -1; + pBasicInfo->updateOperatorInfo = false; +} diff --git a/source/libs/executor/src/streamfilloperator.c b/source/libs/executor/src/streamfilloperator.c index b7061fad97..ccf1f7c9e5 100644 --- a/source/libs/executor/src/streamfilloperator.c +++ b/source/libs/executor/src/streamfilloperator.c @@ -21,6 +21,7 @@ #include "ttypes.h" #include "executorInt.h" +#include "streamexecutorInt.h" #include "tcommon.h" #include "thash.h" #include "ttime.h" @@ -77,12 +78,17 @@ void* destroyFillColumnInfo(SFillColInfo* pFillCol, int32_t start, int32_t end) destroyExprInfo(pFillCol[i].pExpr, 1); taosVariantDestroy(&pFillCol[i].fillVal); } - taosMemoryFreeClear(pFillCol[start].pExpr); + if (start < end) { + taosMemoryFreeClear(pFillCol[start].pExpr); + } taosMemoryFree(pFillCol); return NULL; } void destroyStreamFillSupporter(SStreamFillSupporter* pFillSup) { + if (pFillSup == NULL) { + return; + } pFillSup->pAllColInfo = destroyFillColumnInfo(pFillSup->pAllColInfo, pFillSup->numOfFillCols, pFillSup->numOfAllCols); tSimpleHashCleanup(pFillSup->pResMap); pFillSup->pResMap = NULL; @@ -109,10 +115,15 @@ void destroyStreamFillLinearInfo(SStreamFillLinearInfo* pFillLinear) { } void destroyStreamFillInfo(SStreamFillInfo* pFillInfo) { + if (pFillInfo == NULL) { + return; + } if (pFillInfo->type == TSDB_FILL_SET_VALUE || pFillInfo->type == TSDB_FILL_SET_VALUE_F || pFillInfo->type == TSDB_FILL_NULL || pFillInfo->type == TSDB_FILL_NULL_F) { taosMemoryFreeClear(pFillInfo->pResRow->pRowVal); taosMemoryFreeClear(pFillInfo->pResRow); + taosMemoryFreeClear(pFillInfo->pNonFillRow->pRowVal); + taosMemoryFreeClear(pFillInfo->pNonFillRow); } destroyStreamFillLinearInfo(pFillInfo->pLinearInfo); pFillInfo->pLinearInfo = NULL; @@ -133,6 +144,10 @@ static void destroyStreamFillOperatorInfo(void* param) { pInfo->pDelRes = NULL; taosArrayDestroy(pInfo->matchInfo.pList); pInfo->matchInfo.pList = NULL; + taosArrayDestroy(pInfo->pUpdated); + clearGroupResInfo(&pInfo->groupResInfo); + taosArrayDestroy(pInfo->pCloseTs); + taosMemoryFree(pInfo); } @@ -141,7 +156,7 @@ static void resetFillWindow(SResultRowData* pRowData) { taosMemoryFreeClear(pRowData->pRowVal); } -void resetPrevAndNextWindow(SStreamFillSupporter* pFillSup, void* pState, SStorageAPI* pAPI) { +static void resetPrevAndNextWindow(SStreamFillSupporter* pFillSup) { if (pFillSup->cur.pRowVal != pFillSup->prev.pRowVal && pFillSup->cur.pRowVal != pFillSup->next.pRowVal) { resetFillWindow(&pFillSup->cur); } else { @@ -156,13 +171,13 @@ void resetPrevAndNextWindow(SStreamFillSupporter* pFillSup, void* pState, SStora void getWindowFromDiscBuf(SOperatorInfo* pOperator, TSKEY ts, uint64_t groupId, SStreamFillSupporter* pFillSup) { SStorageAPI* pAPI = &pOperator->pTaskInfo->storageAPI; void* pState = pOperator->pTaskInfo->streamInfo.pState; - resetPrevAndNextWindow(pFillSup, pState, pAPI); + resetPrevAndNextWindow(pFillSup); SWinKey key = {.ts = ts, .groupId = groupId}; void* curVal = NULL; int32_t curVLen = 0; bool hasCurKey = true; - int32_t code = pAPI->stateStore.streamStateFillGet(pState, &key, (void**)&curVal, &curVLen); + int32_t code = pAPI->stateStore.streamStateFillGet(pState, &key, (void**)&curVal, &curVLen, NULL); if (code == TSDB_CODE_SUCCESS) { pFillSup->cur.key = key.ts; pFillSup->cur.pRowVal = curVal; @@ -177,7 +192,7 @@ void getWindowFromDiscBuf(SOperatorInfo* pOperator, TSKEY ts, uint64_t groupId, SWinKey preKey = {.ts = INT64_MIN, .groupId = groupId}; void* preVal = NULL; int32_t preVLen = 0; - code = pAPI->stateStore.streamStateGetGroupKVByCur(pCur, &preKey, (const void**)&preVal, &preVLen); + code = pAPI->stateStore.streamStateFillGetGroupKVByCur(pCur, &preKey, (const void**)&preVal, &preVLen); if (code == TSDB_CODE_SUCCESS) { pFillSup->prev.key = preKey.ts; @@ -196,7 +211,7 @@ void getWindowFromDiscBuf(SOperatorInfo* pOperator, TSKEY ts, uint64_t groupId, SWinKey nextKey = {.ts = INT64_MIN, .groupId = groupId}; void* nextVal = NULL; int32_t nextVLen = 0; - code = pAPI->stateStore.streamStateGetGroupKVByCur(pCur, &nextKey, (const void**)&nextVal, &nextVLen); + code = pAPI->stateStore.streamStateFillGetGroupKVByCur(pCur, &nextKey, (const void**)&nextVal, &nextVLen); if (code == TSDB_CODE_SUCCESS) { pFillSup->next.key = nextKey.ts; pFillSup->next.pRowVal = nextVal; @@ -205,7 +220,7 @@ void getWindowFromDiscBuf(SOperatorInfo* pOperator, TSKEY ts, uint64_t groupId, SWinKey nextNextKey = {.groupId = groupId}; void* nextNextVal = NULL; int32_t nextNextVLen = 0; - code = pAPI->stateStore.streamStateGetGroupKVByCur(pCur, &nextNextKey, (const void**)&nextNextVal, &nextNextVLen); + code = pAPI->stateStore.streamStateFillGetGroupKVByCur(pCur, &nextNextKey, (const void**)&nextNextVal, &nextNextVLen); if (code == TSDB_CODE_SUCCESS) { pFillSup->nextNext.key = nextNextKey.ts; pFillSup->nextNext.pRowVal = nextNextVal; @@ -215,12 +230,10 @@ void getWindowFromDiscBuf(SOperatorInfo* pOperator, TSKEY ts, uint64_t groupId, pAPI->stateStore.streamStateFreeCur(pCur); } -static bool hasPrevWindow(SStreamFillSupporter* pFillSup) { return pFillSup->prev.key != INT64_MIN; } -static bool hasNextWindow(SStreamFillSupporter* pFillSup) { return pFillSup->next.key != INT64_MIN; } -static bool hasNextNextWindow(SStreamFillSupporter* pFillSup) { - return pFillSup->nextNext.key != INT64_MIN; - return false; -} +bool hasCurWindow(SStreamFillSupporter* pFillSup) { return pFillSup->cur.key != INT64_MIN; } +bool hasPrevWindow(SStreamFillSupporter* pFillSup) { return pFillSup->prev.key != INT64_MIN; } +bool hasNextWindow(SStreamFillSupporter* pFillSup) { return pFillSup->next.key != INT64_MIN; } +static bool hasNextNextWindow(SStreamFillSupporter* pFillSup) { return pFillSup->nextNext.key != INT64_MIN; } static void transBlockToResultRow(const SSDataBlock* pBlock, int32_t rowId, TSKEY ts, SResultRowData* pRowVal) { int32_t numOfCols = taosArrayGetSize(pBlock->pDataBlock); @@ -244,29 +257,6 @@ static void transBlockToResultRow(const SSDataBlock* pBlock, int32_t rowId, TSKE pRowVal->key = ts; } -static void calcDeltaData(SSDataBlock* pBlock, int32_t rowId, SResultRowData* pRowVal, SArray* pDelta, - SFillColInfo* pFillCol, int32_t numOfCol, int32_t winCount, int32_t order) { - for (int32_t i = 0; i < numOfCol; i++) { - if (!pFillCol[i].notFillCol) { - int32_t slotId = GET_DEST_SLOT_ID(pFillCol + i); - SColumnInfoData* pCol = taosArrayGet(pBlock->pDataBlock, slotId); - char* var = colDataGetData(pCol, rowId); - double start = 0; - GET_TYPED_DATA(start, double, pCol->info.type, var); - SResultCellData* pCell = getResultCell(pRowVal, slotId); - double end = 0; - GET_TYPED_DATA(end, double, pCell->type, pCell->pData); - double delta = 0; - if (order == TSDB_ORDER_ASC) { - delta = (end - start) / winCount; - } else { - delta = (start - end) / winCount; - } - taosArraySet(pDelta, slotId, &delta); - } - } -} - static void calcRowDeltaData(SResultRowData* pEndRow, SArray* pEndPoins, SFillColInfo* pFillCol, int32_t numOfCol) { for (int32_t i = 0; i < numOfCol; i++) { if (!pFillCol[i].notFillCol) { @@ -461,7 +451,7 @@ void setFillValueInfo(SSDataBlock* pBlock, TSKEY ts, int32_t rowId, SStreamFillS } } -static int32_t checkResult(SStreamFillSupporter* pFillSup, TSKEY ts, uint64_t groupId, bool* pRes) { +int32_t checkResult(SStreamFillSupporter* pFillSup, TSKEY ts, uint64_t groupId, bool* pRes) { int32_t code = TSDB_CODE_SUCCESS; int32_t lino = 0; SWinKey key = {.groupId = groupId, .ts = ts}; @@ -523,7 +513,7 @@ _end: return code; } -static bool hasRemainCalc(SStreamFillInfo* pFillInfo) { +bool hasRemainCalc(SStreamFillInfo* pFillInfo) { if (pFillInfo->current != INT64_MIN && pFillInfo->current <= pFillInfo->end) { return true; } @@ -736,7 +726,6 @@ static void doStreamFillImpl(SOperatorInfo* pOperator) { } while (pInfo->srcRowIndex < pBlock->info.rows) { - TSKEY ts = tsCol[pInfo->srcRowIndex]; code = keepBlockRowInDiscBuf(pOperator, pFillInfo, pBlock, tsCol, pInfo->srcRowIndex, groupId, pFillSup->rowSize); QUERY_CHECK_CODE(code, lino, _end); doFillResults(pOperator, pFillSup, pFillInfo, pBlock, tsCol, pInfo->srcRowIndex - 1, pRes); @@ -874,12 +863,12 @@ static void getWindowInfoByKey(SStorageAPI* pAPI, void* pState, TSKEY ts, int64_ SWinKey key = {.ts = ts, .groupId = groupId}; void* val = NULL; int32_t len = 0; - int32_t code = pAPI->stateStore.streamStateFillGet(pState, &key, (void**)&val, &len); + int32_t code = pAPI->stateStore.streamStateFillGet(pState, &key, (void**)&val, &len, NULL); if (code != TSDB_CODE_SUCCESS) { qDebug("get window info by key failed, Data may be deleted, try next window. ts:%" PRId64 ", groupId:%" PRId64, ts, groupId); SStreamStateCur* pCur = pAPI->stateStore.streamStateFillSeekKeyNext(pState, &key); - code = pAPI->stateStore.streamStateGetGroupKVByCur(pCur, &key, (const void**)&val, &len); + code = pAPI->stateStore.streamStateFillGetGroupKVByCur(pCur, &key, (const void**)&val, &len); pAPI->stateStore.streamStateFreeCur(pCur); qDebug("get window info by key ts:%" PRId64 ", groupId:%" PRId64 ", res%d", ts, groupId, code); } @@ -957,7 +946,7 @@ static int32_t doDeleteFillResult(SOperatorInfo* pOperator) { SWinKey delKey = {.groupId = delGroupId, .ts = delTs}; if (delTs == nextKey.ts) { pAPI->stateStore.streamStateCurNext(pOperator->pTaskInfo->streamInfo.pState, pCur); - winCode = pAPI->stateStore.streamStateGetGroupKVByCur(pCur, &nextKey, NULL, NULL); + winCode = pAPI->stateStore.streamStateFillGetGroupKVByCur(pCur, &nextKey, NULL, NULL); // ts will be deleted later if (delTs != ts) { pAPI->stateStore.streamStateFillDel(pOperator->pTaskInfo->streamInfo.pState, &delKey); @@ -986,9 +975,12 @@ _end: return code; } -static void resetStreamFillInfo(SStreamFillOperatorInfo* pInfo) { - tSimpleHashClear(pInfo->pFillSup->pResMap); - pInfo->pFillSup->hasDelete = false; +void resetStreamFillSup(SStreamFillSupporter* pFillSup) { + tSimpleHashClear(pFillSup->pResMap); + pFillSup->hasDelete = false; +} +void resetStreamFillInfo(SStreamFillOperatorInfo* pInfo) { + resetStreamFillSup(pInfo->pFillSup); taosArrayClear(pInfo->pFillInfo->delRanges); pInfo->pFillInfo->delIndex = 0; } @@ -1165,6 +1157,273 @@ _end: return code; } +void doBuildForceFillResultImpl(SOperatorInfo* pOperator, SStreamFillSupporter* pFillSup, + SStreamFillInfo* pFillInfo, SSDataBlock* pBlock, SGroupResInfo* pGroupResInfo) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + SStorageAPI* pAPI = &pOperator->pTaskInfo->storageAPI; + void* pState = pOperator->pTaskInfo->streamInfo.pState; + bool res = false; + int32_t numOfRows = getNumOfTotalRes(pGroupResInfo); + for (; pGroupResInfo->index < numOfRows; pGroupResInfo->index++) { + SWinKey* pKey = (SWinKey*)taosArrayGet(pGroupResInfo->pRows, pGroupResInfo->index); + if (pBlock->info.id.groupId == 0) { + pBlock->info.id.groupId = pKey->groupId; + } else if (pBlock->info.id.groupId != pKey->groupId) { + break; + } + void* val = NULL; + int32_t len = 0; + int32_t winCode = pAPI->stateStore.streamStateFillGet(pOperator->pTaskInfo->streamInfo.pState, pKey, (void**)&val, &len, NULL); + qDebug("===stream=== build force fill res. key:%" PRId64 ",groupId:%" PRId64".res:%d", pKey->ts, pKey->groupId, winCode); + if (winCode == TSDB_CODE_SUCCESS) { + pFillSup->cur.key = pKey->ts; + pFillSup->cur.pRowVal = val; + code = buildFillResult(&pFillSup->cur, pFillSup, pKey->ts, pBlock, &res); + QUERY_CHECK_CODE(code, lino, _end); + resetFillWindow(&pFillSup->cur); + } else { + SStreamStateCur* pCur = pAPI->stateStore.streamStateFillSeekKeyPrev(pState, pKey); + SWinKey preKey = {.ts = INT64_MIN, .groupId = pKey->groupId}; + void* preVal = NULL; + int32_t preVLen = 0; + winCode = pAPI->stateStore.streamStateFillGetGroupKVByCur(pCur, &preKey, (const void**)&preVal, &preVLen); + if (winCode == TSDB_CODE_SUCCESS) { + pFillSup->cur.key = pKey->ts; + pFillSup->cur.pRowVal = preVal; + if (pFillInfo->type == TSDB_FILL_PREV) { + code = buildFillResult(&pFillSup->cur, pFillSup, pKey->ts, pBlock, &res); + QUERY_CHECK_CODE(code, lino, _end); + } else { + copyNotFillExpData(pFillSup, pFillInfo); + pFillInfo->pResRow->key = pKey->ts; + code = buildFillResult(pFillInfo->pResRow, pFillSup, pKey->ts, pBlock, &res); + QUERY_CHECK_CODE(code, lino, _end); + } + resetFillWindow(&pFillSup->cur); + } + pAPI->stateStore.streamStateFreeCur(pCur); + } + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } +} + +void doBuildForceFillResult(SOperatorInfo* pOperator, SStreamFillSupporter* pFillSup, SStreamFillInfo* pFillInfo, + SSDataBlock* pBlock, SGroupResInfo* pGroupResInfo) { + blockDataCleanup(pBlock); + if (!hasRemainResults(pGroupResInfo)) { + return; + } + + // clear the existed group id + pBlock->info.id.groupId = 0; + doBuildForceFillResultImpl(pOperator, pFillSup, pFillInfo, pBlock, pGroupResInfo); +} + +static int32_t buildForceFillResult(SOperatorInfo* pOperator, SSDataBlock** ppRes) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + SStreamFillOperatorInfo* pInfo = pOperator->info; + uint16_t opType = pOperator->operatorType; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + + doBuildForceFillResult(pOperator, pInfo->pFillSup, pInfo->pFillInfo, pInfo->pRes, &pInfo->groupResInfo); + if (pInfo->pRes->info.rows != 0) { + printDataBlock(pInfo->pRes, getStreamOpName(opType), GET_TASKID(pTaskInfo)); + (*ppRes) = pInfo->pRes; + goto _end; + } + + (*ppRes) = NULL; + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +// force window close impl +static int32_t doStreamForceFillImpl(SOperatorInfo* pOperator) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + SStreamFillOperatorInfo* pInfo = pOperator->info; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + SStreamFillSupporter* pFillSup = pInfo->pFillSup; + SStreamFillInfo* pFillInfo = pInfo->pFillInfo; + SSDataBlock* pBlock = pInfo->pSrcBlock; + uint64_t groupId = pBlock->info.id.groupId; + SStreamAggSupporter* pAggSup = pInfo->pStreamAggSup; + SColumnInfoData* pTsCol = taosArrayGet(pInfo->pSrcBlock->pDataBlock, pInfo->primaryTsCol); + TSKEY* tsCol = (TSKEY*)pTsCol->pData; + for (int32_t i = 0; i < pBlock->info.rows; i++){ + code = keepBlockRowInDiscBuf(pOperator, pFillInfo, pBlock, tsCol, i, groupId, pFillSup->rowSize); + QUERY_CHECK_CODE(code, lino, _end); + + int32_t size = taosArrayGetSize(pInfo->pCloseTs); + if (size > 0) { + TSKEY* pTs = (TSKEY*) taosArrayGet(pInfo->pCloseTs, 0); + TSKEY resTs = tsCol[i]; + while (resTs < (*pTs)) { + SWinKey key = {.groupId = groupId, .ts = resTs}; + void* pPushRes = taosArrayPush(pInfo->pUpdated, &key); + QUERY_CHECK_NULL(pPushRes, code, lino, _end, terrno); + + if (IS_FILL_CONST_VALUE(pFillSup->type)) { + break; + } + resTs = taosTimeAdd(resTs, pFillSup->interval.sliding, pFillSup->interval.slidingUnit, + pFillSup->interval.precision); + } + } + } + code = pAggSup->stateStore.streamStateGroupPut(pAggSup->pState, groupId, NULL, 0); + QUERY_CHECK_CODE(code, lino, _end); + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s. task:%s", __func__, lino, tstrerror(code), GET_TASKID(pTaskInfo)); + } + return code; +} + +int32_t buildAllResultKey(SStreamAggSupporter* pAggSup, TSKEY ts, SArray* pUpdated) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + int64_t groupId = 0; + SStreamStateCur* pCur = pAggSup->stateStore.streamStateGroupGetCur(pAggSup->pState); + while (1) { + int32_t winCode = pAggSup->stateStore.streamStateGroupGetKVByCur(pCur, &groupId, NULL, NULL); + if (winCode != TSDB_CODE_SUCCESS) { + break; + } + SWinKey key = {.ts = ts, .groupId = groupId}; + void* pPushRes = taosArrayPush(pUpdated, &key); + QUERY_CHECK_NULL(pPushRes, code, lino, _end, terrno); + + pAggSup->stateStore.streamStateGroupCurNext(pCur); + } + pAggSup->stateStore.streamStateFreeCur(pCur); + pCur = NULL; + +_end: + if (code != TSDB_CODE_SUCCESS) { + pAggSup->stateStore.streamStateFreeCur(pCur); + pCur = NULL; + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +static void removeDuplicateResult(SArray* pTsArrray, __compar_fn_t fn) { + taosArraySort(pTsArrray, fn); + taosArrayRemoveDuplicate(pTsArrray, fn, NULL); +} + +// force window close +static int32_t doStreamForceFillNext(SOperatorInfo* pOperator, SSDataBlock** ppRes) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + SStreamFillOperatorInfo* pInfo = pOperator->info; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + + if (pOperator->status == OP_EXEC_DONE) { + (*ppRes) = NULL; + return code; + } + + if (pOperator->status == OP_RES_TO_RETURN) { + SSDataBlock* resBlock = NULL; + code = buildForceFillResult(pOperator, &resBlock); + QUERY_CHECK_CODE(code, lino, _end); + + if (resBlock != NULL) { + (*ppRes) = resBlock; + goto _end; + } + pInfo->pStreamAggSup->stateStore.streamStateClearExpiredState(pInfo->pStreamAggSup->pState); + setStreamOperatorCompleted(pOperator); + (*ppRes) = NULL; + goto _end; + } + + SSDataBlock* fillResult = NULL; + SOperatorInfo* downstream = pOperator->pDownstream[0]; + while (1) { + SSDataBlock* pBlock = getNextBlockFromDownstream(pOperator, 0); + if (pBlock == NULL) { + pOperator->status = OP_RES_TO_RETURN; + qDebug("===stream===return data:%s.", getStreamOpName(pOperator->operatorType)); + break; + } + printSpecDataBlock(pBlock, getStreamOpName(pOperator->operatorType), "recv", GET_TASKID(pTaskInfo)); + setStreamOperatorState(&pInfo->basic, pBlock->info.type); + + switch (pBlock->info.type) { + case STREAM_NORMAL: + case STREAM_INVALID: { + code = doApplyStreamScalarCalculation(pOperator, pBlock, pInfo->pSrcBlock); + QUERY_CHECK_CODE(code, lino, _end); + + memcpy(pInfo->pSrcBlock->info.parTbName, pBlock->info.parTbName, TSDB_TABLE_NAME_LEN); + pInfo->srcRowIndex = -1; + } break; + case STREAM_CHECKPOINT: + case STREAM_CREATE_CHILD_TABLE: { + (*ppRes) = pBlock; + goto _end; + } break; + case STREAM_GET_RESULT: { + void* pPushRes = taosArrayPush(pInfo->pCloseTs, &pBlock->info.window.skey); + QUERY_CHECK_NULL(pPushRes, code, lino, _end, terrno); + continue; + } + default: + code = TSDB_CODE_QRY_EXECUTOR_INTERNAL_ERROR; + QUERY_CHECK_CODE(code, lino, _end); + } + + code = doStreamForceFillImpl(pOperator); + QUERY_CHECK_CODE(code, lino, _end); + } + + for (int32_t i = 0; i < taosArrayGetSize(pInfo->pCloseTs); i++) { + TSKEY ts = *(TSKEY*) taosArrayGet(pInfo->pCloseTs, i); + code = buildAllResultKey(pInfo->pStreamAggSup, ts, pInfo->pUpdated); + QUERY_CHECK_CODE(code, lino, _end); + } + taosArrayClear(pInfo->pCloseTs); + removeDuplicateResult(pInfo->pUpdated, winKeyCmprImpl); + + initMultiResInfoFromArrayList(&pInfo->groupResInfo, pInfo->pUpdated); + pInfo->groupResInfo.freeItem = false; + + pInfo->pUpdated = taosArrayInit(1024, sizeof(SWinKey)); + QUERY_CHECK_NULL(pInfo->pUpdated, code, lino, _end, terrno); + + code = blockDataEnsureCapacity(pInfo->pRes, pOperator->resultInfo.capacity); + QUERY_CHECK_CODE(code, lino, _end); + + code = buildForceFillResult(pOperator, ppRes); + QUERY_CHECK_CODE(code, lino, _end); + + if ((*ppRes) == NULL) { + pInfo->pStreamAggSup->stateStore.streamStateClearExpiredState(pInfo->pStreamAggSup->pState); + setStreamOperatorCompleted(pOperator); + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s. task:%s", __func__, lino, tstrerror(code), GET_TASKID(pTaskInfo)); + pTaskInfo->code = code; + } + return code; +} + static int32_t initResultBuf(SSDataBlock* pInputRes, SStreamFillSupporter* pFillSup) { int32_t numOfCols = taosArrayGetSize(pInputRes->pDataBlock); pFillSup->rowSize = sizeof(SResultCellData) * numOfCols; @@ -1277,42 +1536,40 @@ SStreamFillInfo* initStreamFillInfo(SStreamFillSupporter* pFillSup, SSDataBlock* for (int32_t i = 0; i < pFillSup->numOfAllCols; i++) { SColumnInfoData* pColData = taosArrayGet(pRes->pDataBlock, i); - SPoint value = {0}; - value.val = taosMemoryCalloc(1, pColData->info.bytes); - if (!value.val) { - code = terrno; - QUERY_CHECK_CODE(code, lino, _end); + if (pColData == NULL) { + SPoint dummy = {0}; + dummy.val = taosMemoryCalloc(1, 1); + void* tmpRes = taosArrayPush(pFillInfo->pLinearInfo->pEndPoints, &dummy); + QUERY_CHECK_NULL(tmpRes, code, lino, _end, terrno); + + dummy.val = taosMemoryCalloc(1, 1); + tmpRes = taosArrayPush(pFillInfo->pLinearInfo->pNextEndPoints, &dummy); + QUERY_CHECK_NULL(tmpRes, code, lino, _end, terrno); + + continue; } + SPoint value = {0}; + value.val = taosMemoryCalloc(1, pColData->info.bytes); + QUERY_CHECK_NULL(value.val, code, lino, _end, terrno); void* tmpRes = taosArrayPush(pFillInfo->pLinearInfo->pEndPoints, &value); - if (!tmpRes) { - code = terrno; - QUERY_CHECK_CODE(code, lino, _end); - } + QUERY_CHECK_NULL(tmpRes, code, lino, _end, terrno); value.val = taosMemoryCalloc(1, pColData->info.bytes); - if (!value.val) { - code = terrno; - QUERY_CHECK_CODE(code, lino, _end); - } + QUERY_CHECK_NULL(value.val, code, lino, _end, terrno); tmpRes = taosArrayPush(pFillInfo->pLinearInfo->pNextEndPoints, &value); - if (!tmpRes) { - code = terrno; - QUERY_CHECK_CODE(code, lino, _end); - } + QUERY_CHECK_NULL(tmpRes, code, lino, _end, terrno); } } pFillInfo->pLinearInfo->winIndex = 0; + pFillInfo->pNonFillRow = NULL; pFillInfo->pResRow = NULL; if (pFillSup->type == TSDB_FILL_SET_VALUE || pFillSup->type == TSDB_FILL_SET_VALUE_F || pFillSup->type == TSDB_FILL_NULL || pFillSup->type == TSDB_FILL_NULL_F) { pFillInfo->pResRow = taosMemoryCalloc(1, sizeof(SResultRowData)); - if (!pFillInfo->pResRow) { - code = terrno; - QUERY_CHECK_CODE(code, lino, _end); - } + QUERY_CHECK_NULL(pFillInfo->pResRow, code, lino, _end, terrno); pFillInfo->pResRow->key = INT64_MIN; pFillInfo->pResRow->pRowVal = taosMemoryCalloc(1, pFillSup->rowSize); @@ -1324,9 +1581,20 @@ SStreamFillInfo* initStreamFillInfo(SStreamFillSupporter* pFillSup, SSDataBlock* for (int32_t i = 0; i < pFillSup->numOfAllCols; ++i) { SColumnInfoData* pColData = taosArrayGet(pRes->pDataBlock, i); SResultCellData* pCell = getResultCell(pFillInfo->pResRow, i); + if (pColData == NULL) { + pCell->bytes = 1; + pCell->type = 4; + continue; + } pCell->bytes = pColData->info.bytes; pCell->type = pColData->info.type; } + + pFillInfo->pNonFillRow = taosMemoryCalloc(1, sizeof(SResultRowData)); + QUERY_CHECK_NULL(pFillInfo->pNonFillRow, code, lino, _end, terrno); + pFillInfo->pNonFillRow->key = INT64_MIN; + pFillInfo->pNonFillRow->pRowVal = taosMemoryCalloc(1, pFillSup->rowSize); + memcpy(pFillInfo->pNonFillRow->pRowVal, pFillInfo->pResRow->pRowVal, pFillSup->rowSize); } pFillInfo->type = pFillSup->type; @@ -1338,6 +1606,7 @@ SStreamFillInfo* initStreamFillInfo(SStreamFillSupporter* pFillSup, SSDataBlock* pFillInfo->delIndex = 0; pFillInfo->curGroupId = 0; + pFillInfo->hasNext = false; return pFillInfo; _end: @@ -1348,8 +1617,68 @@ _end: return NULL; } +static void setValueForFillInfo(SStreamFillSupporter* pFillSup, SStreamFillInfo* pFillInfo) { + if (pFillInfo->type == TSDB_FILL_SET_VALUE || pFillInfo->type == TSDB_FILL_SET_VALUE_F) { + for (int32_t i = 0; i < pFillSup->numOfAllCols; ++i) { + SFillColInfo* pFillCol = pFillSup->pAllColInfo + i; + int32_t slotId = GET_DEST_SLOT_ID(pFillCol); + SResultCellData* pCell = getResultCell(pFillInfo->pResRow, slotId); + SVariant* pVar = &(pFillCol->fillVal); + if (pCell->type == TSDB_DATA_TYPE_FLOAT) { + float v = 0; + GET_TYPED_DATA(v, float, pVar->nType, &pVar->i); + SET_TYPED_DATA(pCell->pData, pCell->type, v); + } else if (IS_FLOAT_TYPE(pCell->type)) { + double v = 0; + GET_TYPED_DATA(v, double, pVar->nType, &pVar->i); + SET_TYPED_DATA(pCell->pData, pCell->type, v); + } else if (IS_INTEGER_TYPE(pCell->type)) { + int64_t v = 0; + GET_TYPED_DATA(v, int64_t, pVar->nType, &pVar->i); + SET_TYPED_DATA(pCell->pData, pCell->type, v); + } else { + pCell->isNull = true; + } + } + } else if (pFillInfo->type == TSDB_FILL_NULL || pFillInfo->type == TSDB_FILL_NULL_F) { + for (int32_t i = 0; i < pFillSup->numOfAllCols; ++i) { + SFillColInfo* pFillCol = pFillSup->pAllColInfo + i; + int32_t slotId = GET_DEST_SLOT_ID(pFillCol); + SResultCellData* pCell = getResultCell(pFillInfo->pResRow, slotId); + pCell->isNull = true; + } + } +} + +int32_t getDownStreamInfo(SOperatorInfo* downstream, int8_t* triggerType, SInterval* pInterval, SStreamAggSupporter** ppAggSup) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + if (IS_NORMAL_INTERVAL_OP(downstream)) { + SStreamIntervalOperatorInfo* pInfo = downstream->info; + *triggerType = pInfo->twAggSup.calTrigger; + *pInterval = pInfo->interval; + (*ppAggSup) = NULL; + } else if (IS_CONTINUE_INTERVAL_OP(downstream)) { + SStreamIntervalSliceOperatorInfo* pInfo = downstream->info; + *triggerType = pInfo->twAggSup.calTrigger; + *pInterval = pInfo->interval; + pInfo->hasFill = true; + (*ppAggSup) = &pInfo->streamAggSup; + pInfo->streamAggSup.stateStore.streamStateSetFillInfo(pInfo->streamAggSup.pState); + } else { + code = TSDB_CODE_STREAM_INTERNAL_ERROR; + } + QUERY_CHECK_CODE(code, lino, _end); + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + int32_t createStreamFillOperatorInfo(SOperatorInfo* downstream, SStreamFillPhysiNode* pPhyFillNode, - SExecTaskInfo* pTaskInfo, SOperatorInfo** pOptrInfo) { + SExecTaskInfo* pTaskInfo, SOperatorInfo** pOptrInfo) { QRY_PARAM_CHECK(pOptrInfo); int32_t code = TSDB_CODE_SUCCESS; @@ -1361,7 +1690,6 @@ int32_t createStreamFillOperatorInfo(SOperatorInfo* downstream, SStreamFillPhysi QUERY_CHECK_CODE(code, lino, _error); } - SInterval* pInterval = &((SStreamIntervalOperatorInfo*)downstream->info)->interval; int32_t numOfFillCols = 0; SExprInfo* pFillExprInfo = NULL; @@ -1374,7 +1702,12 @@ int32_t createStreamFillOperatorInfo(SOperatorInfo* downstream, SStreamFillPhysi pInfo->pSrcBlock = createDataBlockFromDescNode(pPhyFillNode->node.pOutputDataBlockDesc); QUERY_CHECK_NULL(pInfo->pSrcBlock, code, lino, _error, terrno); - pInfo->pFillSup = initStreamFillSup(pPhyFillNode, pInterval, pFillExprInfo, numOfFillCols, &pTaskInfo->storageAPI, + int8_t triggerType = 0; + SInterval interval = {0}; + code = getDownStreamInfo(downstream, &triggerType, &interval, &pInfo->pStreamAggSup); + QUERY_CHECK_CODE(code, lino, _error); + + pInfo->pFillSup = initStreamFillSup(pPhyFillNode, &interval, pFillExprInfo, numOfFillCols, &pTaskInfo->storageAPI, pInfo->pSrcBlock); if (!pInfo->pFillSup) { code = TSDB_CODE_FAILED; @@ -1396,36 +1729,7 @@ int32_t createStreamFillOperatorInfo(SOperatorInfo* downstream, SStreamFillPhysi goto _error; } - if (pInfo->pFillInfo->type == TSDB_FILL_SET_VALUE || pInfo->pFillInfo->type == TSDB_FILL_SET_VALUE_F) { - for (int32_t i = 0; i < pInfo->pFillSup->numOfAllCols; ++i) { - SFillColInfo* pFillCol = pInfo->pFillSup->pAllColInfo + i; - int32_t slotId = GET_DEST_SLOT_ID(pFillCol); - SResultCellData* pCell = getResultCell(pInfo->pFillInfo->pResRow, slotId); - SVariant* pVar = &(pFillCol->fillVal); - if (pCell->type == TSDB_DATA_TYPE_FLOAT) { - float v = 0; - GET_TYPED_DATA(v, float, pVar->nType, &pVar->i); - SET_TYPED_DATA(pCell->pData, pCell->type, v); - } else if (IS_FLOAT_TYPE(pCell->type)) { - double v = 0; - GET_TYPED_DATA(v, double, pVar->nType, &pVar->i); - SET_TYPED_DATA(pCell->pData, pCell->type, v); - } else if (IS_INTEGER_TYPE(pCell->type)) { - int64_t v = 0; - GET_TYPED_DATA(v, int64_t, pVar->nType, &pVar->i); - SET_TYPED_DATA(pCell->pData, pCell->type, v); - } else { - pCell->isNull = true; - } - } - } else if (pInfo->pFillInfo->type == TSDB_FILL_NULL || pInfo->pFillInfo->type == TSDB_FILL_NULL_F) { - for (int32_t i = 0; i < pInfo->pFillSup->numOfAllCols; ++i) { - SFillColInfo* pFillCol = pInfo->pFillSup->pAllColInfo + i; - int32_t slotId = GET_DEST_SLOT_ID(pFillCol); - SResultCellData* pCell = getResultCell(pInfo->pFillInfo->pResRow, slotId); - pCell->isNull = true; - } - } + setValueForFillInfo(pInfo->pFillSup, pInfo->pFillInfo); code = createSpecialDataBlock(STREAM_DELETE_RESULT, &pInfo->pDelRes); QUERY_CHECK_CODE(code, lino, _error); @@ -1433,6 +1737,12 @@ int32_t createStreamFillOperatorInfo(SOperatorInfo* downstream, SStreamFillPhysi code = blockDataEnsureCapacity(pInfo->pDelRes, pOperator->resultInfo.capacity); QUERY_CHECK_CODE(code, lino, _error); + pInfo->pUpdated = taosArrayInit(1024, sizeof(SWinKey)); + QUERY_CHECK_NULL(pInfo->pUpdated, code, lino, _error, terrno); + + pInfo->pCloseTs = taosArrayInit(1024, sizeof(TSKEY)); + QUERY_CHECK_NULL(pInfo->pCloseTs, code, lino, _error, terrno); + pInfo->primaryTsCol = ((STargetNode*)pPhyFillNode->pWStartTs)->slotId; pInfo->primarySrcSlotId = ((SColumnNode*)((STargetNode*)pPhyFillNode->pWStartTs)->pExpr)->slotId; @@ -1447,8 +1757,14 @@ int32_t createStreamFillOperatorInfo(SOperatorInfo* downstream, SStreamFillPhysi pInfo->srcRowIndex = -1; setOperatorInfo(pOperator, "StreamFillOperator", QUERY_NODE_PHYSICAL_PLAN_STREAM_FILL, false, OP_NOT_OPENED, pInfo, pTaskInfo); - pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, doStreamFillNext, NULL, destroyStreamFillOperatorInfo, - optrDefaultBufFn, NULL, optrDefaultGetNextExtFn, NULL); + + if (triggerType == STREAM_TRIGGER_FORCE_WINDOW_CLOSE) { + pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, doStreamForceFillNext, NULL, destroyStreamFillOperatorInfo, + optrDefaultBufFn, NULL, optrDefaultGetNextExtFn, NULL); + } else { + pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, doStreamFillNext, NULL, destroyStreamFillOperatorInfo, + optrDefaultBufFn, NULL, optrDefaultGetNextExtFn, NULL); + } setOperatorStreamStateFn(pOperator, streamOpReleaseState, streamOpReloadState); code = appendDownstream(pOperator, &downstream, 1); diff --git a/source/libs/executor/src/streamintervalsliceoperator.c b/source/libs/executor/src/streamintervalsliceoperator.c new file mode 100644 index 0000000000..bc35b58a99 --- /dev/null +++ b/source/libs/executor/src/streamintervalsliceoperator.c @@ -0,0 +1,643 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * This program is free software: you can use, redistribute, and/or modify + * it under the terms of the GNU Affero General Public License, version 3 + * or later ("AGPL"), as published by the Free Software Foundation. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +#include "executorInt.h" +#include "functionMgt.h" +#include "operator.h" +#include "querytask.h" +#include "storageapi.h" +#include "streamexecutorInt.h" +#include "tcommon.h" +#include "tcompare.h" +#include "tdatablock.h" +#include "ttime.h" + +#define STREAM_INTERVAL_SLICE_OP_CHECKPOINT_NAME "StreamIntervalSliceOperator_Checkpoint" + +typedef struct SInervalSlicePoint { + SSessionKey winKey; + bool *pFinished; + SSliceRowData* pLastRow; + SRowBuffPos* pResPos; +} SInervalSlicePoint; + +typedef enum SIntervalSliceType { + INTERVAL_SLICE_START = 1, + INTERVAL_SLICE_END = 2, +} SIntervalSliceType; + +void streamIntervalSliceReleaseState(SOperatorInfo* pOperator) { +} + +void streamIntervalSliceReloadState(SOperatorInfo* pOperator) { +} + +void destroyStreamIntervalSliceOperatorInfo(void* param) { + SStreamIntervalSliceOperatorInfo* pInfo = (SStreamIntervalSliceOperatorInfo*)param; + if (param == NULL) { + return; + } + cleanupBasicInfo(&pInfo->binfo); + if (pInfo->pOperator) { + cleanupResultInfoInStream(pInfo->pOperator->pTaskInfo, pInfo->streamAggSup.pState, &pInfo->pOperator->exprSupp, + &pInfo->groupResInfo); + pInfo->pOperator = NULL; + } + + clearGroupResInfo(&pInfo->groupResInfo); + taosArrayDestroyP(pInfo->pUpdated, destroyFlusedPos); + pInfo->pUpdated = NULL; + + if (pInfo->pUpdatedMap != NULL) { + tSimpleHashSetFreeFp(pInfo->pUpdatedMap, destroyFlusedppPos); + tSimpleHashCleanup(pInfo->pUpdatedMap); + pInfo->pUpdatedMap = NULL; + } + destroyStreamAggSupporter(&pInfo->streamAggSup); + + colDataDestroy(&pInfo->twAggSup.timeWindowData); + cleanupExprSupp(&pInfo->scalarSup); + + tSimpleHashCleanup(pInfo->pDeletedMap); + taosArrayDestroy(pInfo->pDelWins); + blockDataDestroy(pInfo->pDelRes); + + blockDataDestroy(pInfo->pCheckpointRes); + + taosMemoryFreeClear(param); +} + +static int32_t buildIntervalSliceResult(SOperatorInfo* pOperator, SSDataBlock** ppRes) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + SStreamIntervalSliceOperatorInfo* pInfo = pOperator->info; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + uint16_t opType = pOperator->operatorType; + SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; + + doBuildDeleteResultImpl(&pInfo->streamAggSup.stateStore, pInfo->streamAggSup.pState, pInfo->pDelWins, &pInfo->delIndex, + pInfo->pDelRes); + if (pInfo->pDelRes->info.rows != 0) { + // process the rest of the data + printDataBlock(pInfo->pDelRes, getStreamOpName(opType), GET_TASKID(pTaskInfo)); + (*ppRes) = pInfo->pDelRes; + return code; + } + + doBuildStreamIntervalResult(pOperator, pInfo->streamAggSup.pState, pInfo->binfo.pRes, &pInfo->groupResInfo); + if (pInfo->binfo.pRes->info.rows != 0) { + printDataBlock(pInfo->binfo.pRes, getStreamOpName(opType), GET_TASKID(pTaskInfo)); + (*ppRes) = pInfo->binfo.pRes; + goto _end; + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +// static void doStreamIntervalSliceSaveCheckpoint(SOperatorInfo* pOperator) { +// } + +void initIntervalSlicePoint(SStreamAggSupporter* pAggSup, STimeWindow* pTWin, int64_t groupId, SInervalSlicePoint* pPoint) { + pPoint->winKey.groupId = groupId; + pPoint->winKey.win = *pTWin; + pPoint->pFinished = POINTER_SHIFT(pPoint->pResPos->pRowBuff, pAggSup->resultRowSize - pAggSup->stateKeySize); + pPoint->pLastRow = POINTER_SHIFT(pPoint->pFinished, sizeof(bool)); +} + +static int32_t getIntervalSliceCurStateBuf(SStreamAggSupporter* pAggSup, SInterval* pInterval, bool needPrev, STimeWindow* pTWin, int64_t groupId, + SInervalSlicePoint* pCurPoint, SInervalSlicePoint* pPrevPoint, int32_t* pWinCode) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + SWinKey curKey = {.ts = pTWin->skey, .groupId = groupId}; + int32_t curVLen = 0; + code = pAggSup->stateStore.streamStateAddIfNotExist(pAggSup->pState, &curKey, (void**)&pCurPoint->pResPos, + &curVLen, pWinCode); + QUERY_CHECK_CODE(code, lino, _end); + + qDebug("===stream=== set stream twa cur point buf.ts:%" PRId64 ", groupId:%" PRIu64 ", res:%d", + curKey.ts, curKey.groupId, *pWinCode); + + initIntervalSlicePoint(pAggSup, pTWin, groupId, pCurPoint); + + if (needPrev) { + SWinKey prevKey = {.groupId = groupId}; + SET_WIN_KEY_INVALID(prevKey.ts); + int32_t prevVLen = 0; + int32_t prevWinCode = TSDB_CODE_SUCCESS; + code = pAggSup->stateStore.streamStateGetPrev(pAggSup->pState, &curKey, &prevKey, (void**)&pPrevPoint->pResPos, + &prevVLen, &prevWinCode); + QUERY_CHECK_CODE(code, lino, _end); + + if (prevWinCode == TSDB_CODE_SUCCESS) { + STimeWindow prevSTW = {.skey = prevKey.ts}; + prevSTW.ekey = taosTimeGetIntervalEnd(prevSTW.skey, pInterval); + initIntervalSlicePoint(pAggSup, &prevSTW, groupId, pPrevPoint); + qDebug("===stream=== set stream twa prev point buf.ts:%" PRId64 ", groupId:%" PRIu64 ", res:%d", pPrevPoint->winKey.win.skey, + pPrevPoint->winKey.groupId, prevWinCode); + } else { + SET_WIN_KEY_INVALID(pPrevPoint->winKey.win.skey); + SET_WIN_KEY_INVALID(pPrevPoint->winKey.win.ekey); + } + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +void doStreamSliceInterpolation(SSliceRowData* pPrevWinVal, TSKEY winKey, TSKEY curTs, SSDataBlock* pDataBlock, + int32_t curRowIndex, SExprSupp* pSup, SIntervalSliceType type) { + SqlFunctionCtx* pCtx = pSup->pCtx; + for (int32_t k = 0; k < pSup->numOfExprs; ++k) { + if (!fmIsIntervalInterpoFunc(pCtx[k].functionId)) { + pCtx[k].start.key = INT64_MIN; + continue; + } + + SFunctParam* pParam = &pCtx[k].param[0]; + SColumnInfoData* pColInfo = taosArrayGet(pDataBlock->pDataBlock, pParam->pCol->slotId); + + double prevVal = 0, curVal = 0, winVal = 0; + SResultCellData* pCell = getSliceResultCell((SResultCellData*)pPrevWinVal->pRowVal, pParam->pCol->slotId); + GET_TYPED_DATA(prevVal, double, pCell->type, pCell->pData); + GET_TYPED_DATA(curVal, double, pColInfo->info.type, colDataGetData(pColInfo, curRowIndex)); + + SPoint point1 = (SPoint){.key = pPrevWinVal->key, .val = &prevVal}; + SPoint point2 = (SPoint){.key = curTs, .val = &curVal}; + SPoint point = (SPoint){.key = winKey, .val = &winVal}; + + if (!fmIsElapsedFunc(pCtx[k].functionId)) { + taosGetLinearInterpolationVal(&point, TSDB_DATA_TYPE_DOUBLE, &point1, &point2, TSDB_DATA_TYPE_DOUBLE); + } + + if (type == INTERVAL_SLICE_START) { + pCtx[k].start.key = point.key; + pCtx[k].start.val = winVal; + } else { + pCtx[k].end.key = point.key; + pCtx[k].end.val = winVal; + } + } +} + +void doSetElapsedEndKey(TSKEY winKey, SExprSupp* pSup) { + SqlFunctionCtx* pCtx = pSup->pCtx; + for (int32_t k = 0; k < pSup->numOfExprs; ++k) { + if (fmIsElapsedFunc(pCtx[k].functionId)) { + pCtx[k].end.key = winKey; + pCtx[k].end.val = 0; + } + } +} + +static void resetIntervalSliceFunctionKey(SqlFunctionCtx* pCtx, int32_t numOfOutput) { + for (int32_t k = 0; k < numOfOutput; ++k) { + pCtx[k].start.key = INT64_MIN; + pCtx[k].end.key = INT64_MIN; + } +} + +int32_t setIntervalSliceOutputBuf(SInervalSlicePoint* pPoint, SqlFunctionCtx* pCtx, int32_t numOfOutput, + int32_t* rowEntryInfoOffset) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + SResultRow* res = pPoint->pResPos->pRowBuff; + + // set time window for current result + res->win = pPoint->winKey.win; + code = setResultRowInitCtx(res, pCtx, numOfOutput, rowEntryInfoOffset); + QUERY_CHECK_CODE(code, lino, _end); + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +static void setInterpoWindowFinished(SInervalSlicePoint* pPoint) { + (*pPoint->pFinished) = true; +} + +static bool isInterpoWindowFinished(SInervalSlicePoint* pPoint) { + return *pPoint->pFinished; +} + +static int32_t doStreamIntervalSliceAggImpl(SOperatorInfo* pOperator, SSDataBlock* pBlock, SSHashObj* pUpdatedMap, + SSHashObj* pDeletedMap) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + SStreamIntervalSliceOperatorInfo* pInfo = (SStreamIntervalSliceOperatorInfo*)pOperator->info; + SResultRowInfo* pResultRowInfo = &(pInfo->binfo.resultRowInfo); + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + SExprSupp* pSup = &pOperator->exprSupp; + int32_t numOfOutput = pSup->numOfExprs; + TSKEY* tsCols = NULL; + int64_t groupId = pBlock->info.id.groupId; + SResultRow* pResult = NULL; + int32_t forwardRows = 0; + + SColumnInfoData* pColDataInfo = taosArrayGet(pBlock->pDataBlock, pInfo->primaryTsIndex); + tsCols = (int64_t*)pColDataInfo->pData; + + int32_t startPos = 0; + TSKEY curTs = getStartTsKey(&pBlock->info.window, tsCols); + SInervalSlicePoint curPoint = {0}; + SInervalSlicePoint prevPoint = {0}; + STimeWindow curWin = + getActiveTimeWindow(NULL, pResultRowInfo, curTs, &pInfo->interval, TSDB_ORDER_ASC); + while (1) { + if (curTs > pInfo->endTs) { + break; + } + + int32_t winCode = TSDB_CODE_SUCCESS; + code = getIntervalSliceCurStateBuf(&pInfo->streamAggSup, &pInfo->interval, pInfo->hasInterpoFunc, &curWin, groupId, &curPoint, &prevPoint, &winCode); + QUERY_CHECK_CODE(code, lino, _end); + + if (pInfo->hasInterpoFunc && IS_VALID_WIN_KEY(prevPoint.winKey.win.skey) && isInterpoWindowFinished(&prevPoint) == false) { + code = setIntervalSliceOutputBuf(&prevPoint, pSup->pCtx, numOfOutput, pSup->rowEntryInfoOffset); + QUERY_CHECK_CODE(code, lino, _end); + + resetIntervalSliceFunctionKey(pSup->pCtx, numOfOutput); + doSetElapsedEndKey(prevPoint.winKey.win.ekey, &pOperator->exprSupp); + doStreamSliceInterpolation(prevPoint.pLastRow, prevPoint.winKey.win.ekey, curTs, pBlock, startPos, &pOperator->exprSupp, INTERVAL_SLICE_END); + updateTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &prevPoint.winKey.win, 1); + code = applyAggFunctionOnPartialTuples(pTaskInfo, pSup->pCtx, &pInfo->twAggSup.timeWindowData, startPos, + 0, pBlock->info.rows, numOfOutput); + QUERY_CHECK_CODE(code, lino, _end); + SWinKey prevKey = {.ts = prevPoint.winKey.win.skey, .groupId = prevPoint.winKey.groupId}; + code = saveWinResult(&prevKey, prevPoint.pResPos, pInfo->pUpdatedMap); + QUERY_CHECK_CODE(code, lino, _end); + setInterpoWindowFinished(&prevPoint); + } + + code = setIntervalSliceOutputBuf(&curPoint, pSup->pCtx, numOfOutput, pSup->rowEntryInfoOffset); + QUERY_CHECK_CODE(code, lino, _end); + + resetIntervalSliceFunctionKey(pSup->pCtx, numOfOutput); + if (pInfo->hasInterpoFunc && IS_VALID_WIN_KEY(prevPoint.winKey.win.skey) && curPoint.winKey.win.skey != curTs) { + doStreamSliceInterpolation(prevPoint.pLastRow, curPoint.winKey.win.skey, curTs, pBlock, startPos, &pOperator->exprSupp, INTERVAL_SLICE_START); + } + forwardRows = getNumOfRowsInTimeWindow(&pBlock->info, tsCols, startPos, curWin.ekey, binarySearchForKey, NULL, + TSDB_ORDER_ASC); + int32_t prevEndPos = (forwardRows - 1) + startPos; + if (pInfo->hasInterpoFunc && winCode != TSDB_CODE_SUCCESS) { + int32_t endRowId = getQualifiedRowNumDesc(pSup, pBlock, tsCols, prevEndPos, false); + TSKEY endRowTs = tsCols[endRowId]; + transBlockToSliceResultRow(pBlock, endRowId, endRowTs, curPoint.pLastRow, 0, NULL, NULL); + } + SWinKey curKey = {.ts = curPoint.winKey.win.skey, .groupId = curPoint.winKey.groupId}; + if (pInfo->destHasPrimaryKey && winCode == TSDB_CODE_SUCCESS) { + code = tSimpleHashPut(pDeletedMap, &curKey, sizeof(SWinKey), NULL, 0); + QUERY_CHECK_CODE(code, lino, _end); + } + + code = saveWinResult(&curKey, curPoint.pResPos, pInfo->pUpdatedMap); + QUERY_CHECK_CODE(code, lino, _end); + + updateTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &curPoint.winKey.win, 1); + code = applyAggFunctionOnPartialTuples(pTaskInfo, pSup->pCtx, &pInfo->twAggSup.timeWindowData, startPos, + forwardRows, pBlock->info.rows, numOfOutput); + QUERY_CHECK_CODE(code, lino, _end); + + if (curPoint.pLastRow->key == curPoint.winKey.win.ekey) { + setInterpoWindowFinished(&curPoint); + } + + startPos = getNextQualifiedWindow(&pInfo->interval, &curWin, &pBlock->info, tsCols, prevEndPos, TSDB_ORDER_ASC); + if (startPos < 0) { + break; + } + curTs = tsCols[startPos]; + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s. task:%s", __func__, lino, tstrerror(code), GET_TASKID(pTaskInfo)); + } + return code; +} + +static int32_t doStreamIntervalSliceNext(SOperatorInfo* pOperator, SSDataBlock** ppRes) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + SStreamIntervalSliceOperatorInfo* pInfo = pOperator->info; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; + + qDebug("stask:%s %s status: %d", GET_TASKID(pTaskInfo), getStreamOpName(pOperator->operatorType), pOperator->status); + + if (pOperator->status == OP_EXEC_DONE) { + (*ppRes) = NULL; + goto _end; + } + + if (pOperator->status == OP_RES_TO_RETURN) { + SSDataBlock* resBlock = NULL; + code = buildIntervalSliceResult(pOperator, &resBlock); + QUERY_CHECK_CODE(code, lino, _end); + if (resBlock != NULL) { + (*ppRes) = resBlock; + return code; + } + + if (pInfo->hasFill == false) { + pAggSup->stateStore.streamStateClearExpiredState(pAggSup->pState); + } + setStreamOperatorCompleted(pOperator); + (*ppRes) = NULL; + return code; + } + + SOperatorInfo* downstream = pOperator->pDownstream[0]; + int32_t numOfDatapack = 0; + + while (1) { + SSDataBlock* pBlock = NULL; + code = downstream->fpSet.getNextFn(downstream, &pBlock); + QUERY_CHECK_CODE(code, lino, _end); + + if (pBlock == NULL) { + pOperator->status = OP_RES_TO_RETURN; + break; + } + + switch (pBlock->info.type) { + case STREAM_NORMAL: + case STREAM_INVALID: { + SExprSupp* pExprSup = &pInfo->scalarSup; + if (pExprSup->pExprInfo != NULL) { + code = projectApplyFunctions(pExprSup->pExprInfo, pBlock, pBlock, pExprSup->pCtx, pExprSup->numOfExprs, NULL); + QUERY_CHECK_CODE(code, lino, _end); + } + } break; + case STREAM_CHECKPOINT: { + pInfo->recvCkBlock = true; + pAggSup->stateStore.streamStateCommit(pAggSup->pState); + // doStreamIntervalSliceSaveCheckpoint(pOperator); + pInfo->recvCkBlock = true; + code = copyDataBlock(pInfo->pCheckpointRes, pBlock); + QUERY_CHECK_CODE(code, lino, _end); + continue; + } break; + case STREAM_CREATE_CHILD_TABLE: { + (*ppRes) = pBlock; + goto _end; + } break; + case STREAM_GET_RESULT: { + pInfo->endTs = taosTimeGetIntervalEnd(pBlock->info.window.skey, &pInfo->interval); + if (pInfo->hasFill) { + (*ppRes) = pBlock; + goto _end; + } else { + continue; + } + } + default: + code = TSDB_CODE_QRY_EXECUTOR_INTERNAL_ERROR; + QUERY_CHECK_CODE(code, lino, _end); + } + + code = setInputDataBlock(&pOperator->exprSupp, pBlock, TSDB_ORDER_ASC, MAIN_SCAN, true); + QUERY_CHECK_CODE(code, lino, _end); + code = doStreamIntervalSliceAggImpl(pOperator, pBlock, pInfo->pUpdatedMap, NULL); + QUERY_CHECK_CODE(code, lino, _end); + + } + + if (!pInfo->destHasPrimaryKey) { + removeDeleteResults(pInfo->pUpdatedMap, pInfo->pDelWins); + } + + if (pInfo->destHasPrimaryKey) { + code = copyIntervalDeleteKey(pInfo->pDeletedMap, pInfo->pDelWins); + QUERY_CHECK_CODE(code, lino, _end); + } + + code = copyUpdateResult(&pInfo->pUpdatedMap, pInfo->pUpdated, winPosCmprImpl); + QUERY_CHECK_CODE(code, lino, _end); + + _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); + pInfo->pUpdatedMap = tSimpleHashInit(1024, hashFn); + QUERY_CHECK_NULL(pInfo->pUpdatedMap, code, lino, _end, terrno); + + initMultiResInfoFromArrayList(&pInfo->groupResInfo, pInfo->pUpdated); + pInfo->pUpdated = taosArrayInit(1024, POINTER_BYTES); + QUERY_CHECK_NULL(pInfo->pUpdated, code, lino, _end, terrno); + + code = blockDataEnsureCapacity(pInfo->binfo.pRes, pOperator->resultInfo.capacity); + QUERY_CHECK_CODE(code, lino, _end); + + (*ppRes) = NULL; + code = buildIntervalSliceResult(pOperator, ppRes); + QUERY_CHECK_CODE(code, lino, _end); + + if ((*ppRes) == NULL) { + if (pInfo->hasFill == false) { + pAggSup->stateStore.streamStateClearExpiredState(pAggSup->pState); + } + setStreamOperatorCompleted(pOperator); + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +int32_t initIntervalSliceDownStream(SOperatorInfo* downstream, SStreamAggSupporter* pAggSup, uint16_t type, + int32_t tsColIndex, STimeWindowAggSupp* pTwSup, struct SSteamOpBasicInfo* pBasic, + SInterval* pInterval, bool hasInterpoFunc) { + SExecTaskInfo* pTaskInfo = downstream->pTaskInfo; + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + if (downstream->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_PARTITION) { + SStreamPartitionOperatorInfo* pPartionInfo = downstream->info; + pPartionInfo->tsColIndex = tsColIndex; + pBasic->primaryPkIndex = pPartionInfo->basic.primaryPkIndex; + } + + if (downstream->operatorType != QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN) { + code = + initIntervalSliceDownStream(downstream->pDownstream[0], pAggSup, type, tsColIndex, pTwSup, pBasic, pInterval, hasInterpoFunc); + return code; + } + SStreamScanInfo* pScanInfo = downstream->info; + pScanInfo->useGetResultRange = hasInterpoFunc; + pScanInfo->igCheckUpdate = true; + pScanInfo->windowSup = (SWindowSupporter){.pStreamAggSup = pAggSup, .gap = pAggSup->gap, .parentType = type}; + pScanInfo->pState = pAggSup->pState; + if (!pScanInfo->pUpdateInfo) { + code = pAggSup->stateStore.updateInfoInit(60000, TSDB_TIME_PRECISION_MILLI, pTwSup->waterMark, + pScanInfo->igCheckUpdate, pScanInfo->pkColType, pScanInfo->pkColLen, + &pScanInfo->pUpdateInfo); + QUERY_CHECK_CODE(code, lino, _end); + } + pScanInfo->twAggSup = *pTwSup; + pScanInfo->interval = *pInterval; + pAggSup->pUpdateInfo = pScanInfo->pUpdateInfo; + if (!hasSrcPrimaryKeyCol(pBasic)) { + pBasic->primaryPkIndex = pScanInfo->basic.primaryPkIndex; + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s. task:%s", __func__, lino, tstrerror(code), GET_TASKID(pTaskInfo)); + } + return code; +} + +static bool windowinterpNeeded(SqlFunctionCtx* pCtx, int32_t numOfCols) { + bool needed = false; + for (int32_t i = 0; i < numOfCols; ++i) { + SExprInfo* pExpr = pCtx[i].pExpr; + if (fmIsIntervalInterpoFunc(pCtx[i].functionId)) { + needed = true; + break; + } + } + return needed; +} + +int32_t createStreamIntervalSliceOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo, + SReadHandle* pHandle, SOperatorInfo** ppOptInfo) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + SStreamIntervalSliceOperatorInfo* pInfo = taosMemoryCalloc(1, sizeof(SStreamIntervalSliceOperatorInfo)); + QUERY_CHECK_NULL(pInfo, code, lino, _error, terrno); + + SOperatorInfo* pOperator = taosMemoryCalloc(1, sizeof(SOperatorInfo)); + QUERY_CHECK_NULL(pOperator, code, lino, _error, terrno) + + pInfo->pUpdated = taosArrayInit(1024, POINTER_BYTES); + QUERY_CHECK_NULL(pInfo->pUpdated, code, lino, _error, terrno); + + _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); + pInfo->pUpdatedMap = tSimpleHashInit(1024, hashFn); + QUERY_CHECK_NULL(pInfo->pUpdatedMap, code, lino, _error, terrno); + + pInfo->pDeletedMap = tSimpleHashInit(1024, hashFn); + QUERY_CHECK_NULL(pInfo->pDeletedMap, code, lino, _error, terrno); + + pInfo->delIndex = 0; + pInfo->pDelWins = taosArrayInit(4, sizeof(SWinKey)); + QUERY_CHECK_NULL(pInfo->pDelWins, code, lino, _error, terrno); + + code = createSpecialDataBlock(STREAM_DELETE_RESULT, &pInfo->pDelRes); + QUERY_CHECK_CODE(code, lino, _error); + + SSDataBlock* pResBlock = createDataBlockFromDescNode(pPhyNode->pOutputDataBlockDesc); + QUERY_CHECK_NULL(pResBlock, code, lino, _error, terrno); + initBasicInfo(&pInfo->binfo, pResBlock); + + code = createSpecialDataBlock(STREAM_CHECKPOINT, &pInfo->pCheckpointRes); + QUERY_CHECK_CODE(code, lino, _error); + pInfo->recvCkBlock = false; + + SStreamIntervalPhysiNode* pIntervalPhyNode = (SStreamIntervalPhysiNode*)pPhyNode; + pOperator->pTaskInfo = pTaskInfo; + initResultSizeInfo(&pOperator->resultInfo, 4096); + SExprSupp* pExpSup = &pOperator->exprSupp; + int32_t numOfExprs = 0; + SExprInfo* pExprInfo = NULL; + code = createExprInfo(pIntervalPhyNode->window.pFuncs, NULL, &pExprInfo, &numOfExprs); + QUERY_CHECK_CODE(code, lino, _error); + + code = initExprSupp(pExpSup, pExprInfo, numOfExprs, &pTaskInfo->storageAPI.functionStore); + QUERY_CHECK_CODE(code, lino, _error); + + pInfo->interval = (SInterval){.interval = pIntervalPhyNode->interval, + .sliding = pIntervalPhyNode->sliding, + .intervalUnit = pIntervalPhyNode->intervalUnit, + .slidingUnit = pIntervalPhyNode->slidingUnit, + .offset = pIntervalPhyNode->offset, + .precision = ((SColumnNode*)pIntervalPhyNode->window.pTspk)->node.resType.precision}; + + pInfo->twAggSup = + (STimeWindowAggSupp){.waterMark = pIntervalPhyNode->window.watermark, + .calTrigger = pIntervalPhyNode->window.triggerType, + .maxTs = INT64_MIN, + .minTs = INT64_MAX, + .deleteMark = getDeleteMark(&pIntervalPhyNode->window, pIntervalPhyNode->interval)}; + code = initExecTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &pTaskInfo->window); + QUERY_CHECK_CODE(code, lino, _error); + + if (pIntervalPhyNode->window.pExprs != NULL) { + int32_t numOfScalar = 0; + SExprInfo* pScalarExprInfo = NULL; + code = createExprInfo(pIntervalPhyNode->window.pExprs, NULL, &pScalarExprInfo, &numOfScalar); + QUERY_CHECK_CODE(code, lino, _error); + + code = initExprSupp(&pInfo->scalarSup, pScalarExprInfo, numOfScalar, &pTaskInfo->storageAPI.functionStore); + QUERY_CHECK_CODE(code, lino, _error); + } + + SSDataBlock* pDownRes = NULL; + SColumnInfo* pPkCol = NULL; + code = getDownstreamRes(downstream, &pDownRes, &pPkCol); + QUERY_CHECK_CODE(code, lino, _error); + + int32_t keyBytes = sizeof(TSKEY); + keyBytes += blockDataGetRowSize(pDownRes) + sizeof(SResultCellData) * taosArrayGetSize(pDownRes->pDataBlock) + sizeof(bool); + if (pPkCol) { + keyBytes += pPkCol->bytes; + } + code = initStreamAggSupporter(&pInfo->streamAggSup, pExpSup, numOfExprs, 0, pTaskInfo->streamInfo.pState, keyBytes, 0, + &pTaskInfo->storageAPI.stateStore, pHandle, &pInfo->twAggSup, GET_TASKID(pTaskInfo), + &pTaskInfo->storageAPI, pInfo->primaryTsIndex, STREAM_STATE_BUFF_HASH_SEARCH, 1); + + pInfo->destHasPrimaryKey = pIntervalPhyNode->window.destHasPrimaryKey; + pInfo->pOperator = pOperator; + pInfo->hasFill = false; + pInfo->hasInterpoFunc = windowinterpNeeded(pExpSup->pCtx, numOfExprs); + + setOperatorInfo(pOperator, "StreamIntervalSliceOperator", QUERY_NODE_PHYSICAL_PLAN_STREAM_CONTINUE_INTERVAL, true, OP_NOT_OPENED, + pInfo, pTaskInfo); + pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, doStreamIntervalSliceNext, NULL, destroyStreamIntervalSliceOperatorInfo, + optrDefaultBufFn, NULL, optrDefaultGetNextExtFn, NULL); + setOperatorStreamStateFn(pOperator, streamIntervalSliceReleaseState, streamIntervalSliceReloadState); + + initStreamBasicInfo(&pInfo->basic); + if (downstream) { + code = initIntervalSliceDownStream(downstream, &pInfo->streamAggSup, pPhyNode->type, pInfo->primaryTsIndex, + &pInfo->twAggSup, &pInfo->basic, &pInfo->interval, pInfo->hasInterpoFunc); + QUERY_CHECK_CODE(code, lino, _error); + + code = appendDownstream(pOperator, &downstream, 1); + QUERY_CHECK_CODE(code, lino, _error); + } + + (*ppOptInfo) = pOperator; + return code; + +_error: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + if (pInfo != NULL) { + destroyStreamIntervalSliceOperatorInfo(pInfo); + } + destroyOperatorAndDownstreams(pOperator, &downstream, 1); + pTaskInfo->code = code; + (*ppOptInfo) = NULL; + return code; +} diff --git a/source/libs/executor/src/streamtimesliceoperator.c b/source/libs/executor/src/streamtimesliceoperator.c new file mode 100644 index 0000000000..b120bb6374 --- /dev/null +++ b/source/libs/executor/src/streamtimesliceoperator.c @@ -0,0 +1,2187 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * This program is free software: you can use, redistribute, and/or modify + * it under the terms of the GNU Affero General Public License, version 3 + * or later ("AGPL"), as published by the Free Software Foundation. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +#include "executorInt.h" +#include "filter.h" +#include "function.h" +#include "functionMgt.h" +#include "operator.h" +#include "querytask.h" +#include "storageapi.h" +#include "streamexecutorInt.h" +#include "tchecksum.h" +#include "tcommon.h" +#include "tcompare.h" +#include "tdatablock.h" +#include "tfill.h" +#include "ttime.h" + +#define STREAM_TIME_SLICE_OP_STATE_NAME "StreamTimeSliceHistoryState" +#define STREAM_TIME_SLICE_OP_CHECKPOINT_NAME "StreamTimeSliceOperator_Checkpoint" + + +int32_t saveTimeSliceWinResult(SWinKey* pKey, SSHashObj* pUpdatedMap) { + return tSimpleHashPut(pUpdatedMap, pKey, sizeof(SWinKey), NULL, 0); +} + +void streamTimeSliceReleaseState(SOperatorInfo* pOperator) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + SStreamTimeSliceOperatorInfo* pInfo = pOperator->info; + int32_t winNum = taosArrayGetSize(pInfo->historyWins); + + int32_t winSize = winNum * sizeof(SWinKey); + int32_t resSize = winSize + sizeof(TSKEY); + char* pBuff = taosMemoryCalloc(1, resSize); + QUERY_CHECK_NULL(pBuff, code, lino, _end, terrno); + + if (winNum > 0) { + memcpy(pBuff, pInfo->historyWins->pData, winSize); + } + memcpy(pBuff + winSize, &pInfo->twAggSup.maxTs, sizeof(TSKEY)); + qDebug("===stream=== time slice operator relase state. save result count:%d", winNum); + pInfo->streamAggSup.stateStore.streamStateSaveInfo(pInfo->streamAggSup.pState, STREAM_TIME_SLICE_OP_STATE_NAME, + strlen(STREAM_TIME_SLICE_OP_STATE_NAME), pBuff, resSize); + pInfo->streamAggSup.stateStore.streamStateCommit(pInfo->streamAggSup.pState); + taosMemoryFreeClear(pBuff); + + SOperatorInfo* downstream = pOperator->pDownstream[0]; + if (downstream->fpSet.releaseStreamStateFn) { + downstream->fpSet.releaseStreamStateFn(downstream); + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } +} + +void streamTimeSliceReloadState(SOperatorInfo* pOperator) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + SStreamTimeSliceOperatorInfo* pInfo = pOperator->info; + SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + SStreamFillSupporter* pFillSup = pInfo->pFillSup; + resetWinRange(&pAggSup->winRange); + + int32_t size = 0; + void* pBuf = NULL; + code = pAggSup->stateStore.streamStateGetInfo(pAggSup->pState, STREAM_TIME_SLICE_OP_STATE_NAME, + strlen(STREAM_TIME_SLICE_OP_STATE_NAME), &pBuf, &size); + QUERY_CHECK_CODE(code, lino, _end); + + int32_t num = (size - sizeof(TSKEY)) / sizeof(SWinKey); + qDebug("===stream=== time slice operator reload state. get result count:%d", num); + SWinKey* pKeyBuf = (SWinKey*)pBuf; + QUERY_CHECK_CONDITION((size == num * sizeof(SWinKey) + sizeof(TSKEY)), code, lino, _end, TSDB_CODE_QRY_EXECUTOR_INTERNAL_ERROR); + + TSKEY ts = *(TSKEY*)((char*)pBuf + size - sizeof(TSKEY)); + pInfo->twAggSup.maxTs = TMAX(pInfo->twAggSup.maxTs, ts); + pAggSup->stateStore.streamStateReloadInfo(pAggSup->pState, ts); + qDebug("===stream=== reload state. reload ts:%" PRId64, ts); + + if (!pInfo->pUpdatedMap && num > 0) { + _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); + pInfo->pUpdatedMap = tSimpleHashInit(64, hashFn); + QUERY_CHECK_NULL(pInfo->pUpdatedMap, code, lino, _end, terrno); + } + + int32_t tmpRes = TSDB_CODE_SUCCESS; + for (int32_t i = 0; i < num; i++) { + SWinKey* pKey = pKeyBuf + i; + SWinKey resKey = {.groupId = pKey->groupId}; + if (pFillSup->type != TSDB_FILL_PREV && pFillSup->type != TSDB_FILL_LINEAR) { + code = pAggSup->stateStore.streamStateFillGetNext(pAggSup->pState, pKey, &resKey, NULL, NULL, &tmpRes); + QUERY_CHECK_CODE(code, lino, _end); + + if (tmpRes != TSDB_CODE_SUCCESS) { + continue; + } + } else { + resKey = *pKey; + } + qDebug("===stream=== reload state. try process result %" PRId64 ", %" PRIu64 ", index:%d", resKey.ts, + resKey.groupId, i); + code = saveTimeSliceWinResult(&resKey, pInfo->pUpdatedMap); + QUERY_CHECK_CODE(code, lino, _end); + } + taosMemoryFree(pBuf); + + SOperatorInfo* downstream = pOperator->pDownstream[0]; + if (downstream->fpSet.reloadStreamStateFn) { + downstream->fpSet.reloadStreamStateFn(downstream); + } + reloadAggSupFromDownStream(downstream, &pInfo->streamAggSup); + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s. task:%s", __func__, lino, tstrerror(code), GET_TASKID(pTaskInfo)); + } +} + +static void resetFillWindow(SResultRowData* pRowData) { + pRowData->key = INT64_MIN; + pRowData->pRowVal = NULL; +} + +static void resetPrevAndNextWindow(SStreamFillSupporter* pFillSup) { + resetFillWindow(&pFillSup->cur); + resetFillWindow(&pFillSup->prev); + resetFillWindow(&pFillSup->next); + resetFillWindow(&pFillSup->nextNext); +} + +void destroyStreamTimeSliceOperatorInfo(void* param) { + SStreamTimeSliceOperatorInfo* pInfo = (SStreamTimeSliceOperatorInfo*)param; + if (pInfo->pOperator) { + cleanupResultInfoInStream(pInfo->pOperator->pTaskInfo, pInfo->streamAggSup.pState, &pInfo->pOperator->exprSupp, + &pInfo->groupResInfo); + pInfo->pOperator = NULL; + } + colDataDestroy(&pInfo->twAggSup.timeWindowData); + destroyStreamAggSupporter(&pInfo->streamAggSup); + resetPrevAndNextWindow(pInfo->pFillSup); + destroyStreamFillSupporter(pInfo->pFillSup); + destroyStreamFillInfo(pInfo->pFillInfo); + blockDataDestroy(pInfo->pRes); + blockDataDestroy(pInfo->pDelRes); + blockDataDestroy(pInfo->pCheckpointRes); + + taosMemoryFreeClear(pInfo->leftRow.pRowVal); + taosMemoryFreeClear(pInfo->valueRow.pRowVal); + taosMemoryFreeClear(pInfo->rightRow.pRowVal); + + cleanupExprSupp(&pInfo->scalarSup); + taosArrayDestroy(pInfo->historyPoints); + + taosArrayDestroyP(pInfo->pUpdated, destroyFlusedPos); + pInfo->pUpdated = NULL; + + tSimpleHashCleanup(pInfo->pUpdatedMap); + pInfo->pUpdatedMap = NULL; + + taosArrayDestroy(pInfo->pDelWins); + tSimpleHashCleanup(pInfo->pDeletedMap); + clearGroupResInfo(&pInfo->groupResInfo); + + taosArrayDestroy(pInfo->historyWins); + + taosArrayDestroy(pInfo->pCloseTs); + + taosMemoryFreeClear(param); +} + +int32_t doStreamTimeSliceEncodeOpState(void** buf, int32_t len, SOperatorInfo* pOperator, int32_t* pLen) { + int32_t code = TSDB_CODE_SUCCESS; + SStreamTimeSliceOperatorInfo* pInfo = pOperator->info; + if (!pInfo) { + return TSDB_CODE_FAILED; + } + + void* pData = (buf == NULL) ? NULL : *buf; + + // 1.streamAggSup.pResultRows + int32_t tlen = 0; + int32_t mapSize = tSimpleHashGetSize(pInfo->streamAggSup.pResultRows); + tlen += taosEncodeFixedI32(buf, mapSize); + void* pIte = NULL; + size_t keyLen = 0; + int32_t iter = 0; + while ((pIte = tSimpleHashIterate(pInfo->streamAggSup.pResultRows, pIte, &iter)) != NULL) { + void* pKey = tSimpleHashGetKey(pIte, &keyLen); + tlen += encodeSSessionKey(buf, pKey); + tlen += encodeSResultWindowInfo(buf, pIte, pInfo->streamAggSup.resultRowSize); + } + + // 2.twAggSup + tlen += encodeSTimeWindowAggSupp(buf, &pInfo->twAggSup); + + // 3.checksum + if (buf) { + uint32_t cksum = taosCalcChecksum(0, pData, len - sizeof(uint32_t)); + tlen += taosEncodeFixedU32(buf, cksum); + } else { + tlen += sizeof(uint32_t); + } + + (*pLen) = tlen; + return code; +} + +int32_t doStreamTimeSliceDecodeOpState(void* buf, int32_t len, SOperatorInfo* pOperator) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + SStreamTimeSliceOperatorInfo* pInfo = pOperator->info; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + if (!pInfo) { + code = TSDB_CODE_FAILED; + QUERY_CHECK_CODE(code, lino, _end); + } + SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; + + // 3.checksum + int32_t dataLen = len - sizeof(uint32_t); + void* pCksum = POINTER_SHIFT(buf, dataLen); + if (taosCheckChecksum(buf, dataLen, *(uint32_t*)pCksum) != TSDB_CODE_SUCCESS) { + qError("stream event state is invalid"); + code = TSDB_CODE_FAILED; + QUERY_CHECK_CODE(code, lino, _end); + } + + // 1.streamAggSup.pResultRows + int32_t mapSize = 0; + buf = taosDecodeFixedI32(buf, &mapSize); + for (int32_t i = 0; i < mapSize; i++) { + SResultWindowInfo winfo = {0}; + buf = decodeSSessionKey(buf, &winfo.sessionWin); + int32_t winCode = TSDB_CODE_SUCCESS; + code = pAggSup->stateStore.streamStateSessionAddIfNotExist( + pAggSup->pState, &winfo.sessionWin, pAggSup->gap, (void**)&winfo.pStatePos, &pAggSup->resultRowSize, &winCode); + QUERY_CHECK_CODE(code, lino, _end); + + buf = decodeSResultWindowInfo(buf, &winfo, pInfo->streamAggSup.resultRowSize); + code = tSimpleHashPut(pInfo->streamAggSup.pResultRows, &winfo.sessionWin, sizeof(SSessionKey), &winfo, + sizeof(SResultWindowInfo)); + QUERY_CHECK_CODE(code, lino, _end); + } + + // 2.twAggSup + buf = decodeSTimeWindowAggSupp(buf, &pInfo->twAggSup); + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s. task:%s", __func__, lino, tstrerror(code), GET_TASKID(pTaskInfo)); + } + return code; +} + +static int32_t initTimeSliceResultBuf(SStreamFillSupporter* pFillSup, SExprSupp* pExpSup) { + pFillSup->rowSize = sizeof(TSKEY) + getResultRowSize(pExpSup->pCtx, pFillSup->numOfAllCols); + pFillSup->next.key = INT64_MIN; + pFillSup->nextNext.key = INT64_MIN; + pFillSup->prev.key = INT64_MIN; + pFillSup->cur.key = INT64_MIN; + pFillSup->next.pRowVal = NULL; + pFillSup->nextNext.pRowVal = NULL; + pFillSup->prev.pRowVal = NULL; + pFillSup->cur.pRowVal = NULL; + + return TSDB_CODE_SUCCESS; +} + +static int32_t initTimeSliceFillSup(SStreamInterpFuncPhysiNode* pPhyFillNode, SExprSupp* pExprSup, int32_t numOfExprs, SColumnInfo* pPkCol, + SStreamFillSupporter** ppResFillSup) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + SStreamFillSupporter* pFillSup = taosMemoryCalloc(1, sizeof(SStreamFillSupporter)); + QUERY_CHECK_NULL(pFillSup, code, lino, _end, terrno); + + pFillSup->numOfFillCols = numOfExprs; + int32_t numOfNotFillCols = 0; + pFillSup->pAllColInfo = createFillColInfo(pExprSup->pExprInfo, pFillSup->numOfFillCols, NULL, numOfNotFillCols, NULL, 0, + (const SNodeListNode*)(pPhyFillNode->pFillValues)); + QUERY_CHECK_NULL(pFillSup->pAllColInfo, code, lino, _end, terrno); + + pFillSup->type = convertFillType(pPhyFillNode->fillMode); + pFillSup->numOfAllCols = pFillSup->numOfFillCols + numOfNotFillCols; + pFillSup->interval.interval = pPhyFillNode->interval; + pFillSup->interval.intervalUnit = pPhyFillNode->intervalUnit; + pFillSup->interval.offset = 0; + pFillSup->interval.offsetUnit = pPhyFillNode->intervalUnit; + pFillSup->interval.precision = pPhyFillNode->precision; + pFillSup->interval.sliding = pPhyFillNode->interval; + pFillSup->interval.slidingUnit = pPhyFillNode->intervalUnit; + pFillSup->pAPI = NULL; + _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); + pFillSup->pResMap = tSimpleHashInit(16, hashFn); + QUERY_CHECK_NULL(pFillSup->pResMap, code, lino, _end, terrno); + + code = initTimeSliceResultBuf(pFillSup, pExprSup); + QUERY_CHECK_CODE(code, lino, _end); + + pFillSup->hasDelete = false; + if (pPkCol != NULL) { + pFillSup->pkColBytes = pPkCol->bytes; + pFillSup->comparePkColFn = getKeyComparFunc(pPkCol->type, TSDB_ORDER_ASC); + } else { + pFillSup->pkColBytes = 0; + pFillSup->comparePkColFn = NULL; + } + + (*ppResFillSup) = pFillSup; + +_end: + if (code != TSDB_CODE_SUCCESS) { + destroyStreamFillSupporter(pFillSup); + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +static void doStreamTimeSliceSaveCheckpoint(SOperatorInfo* pOperator) { + SStreamTimeSliceOperatorInfo* pInfo = pOperator->info; + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + void* buf = NULL; + if (needSaveStreamOperatorInfo(&pInfo->basic)) { + int32_t len = 0; + code = doStreamTimeSliceEncodeOpState(NULL, 0, pOperator, &len); + QUERY_CHECK_CODE(code, lino, _end); + + buf = taosMemoryCalloc(1, len); + QUERY_CHECK_NULL(buf, code, lino, _end, terrno); + + void* pBuf = buf; + code = doStreamTimeSliceEncodeOpState(&pBuf, len, pOperator, &len); + QUERY_CHECK_CODE(code, lino, _end); + + pInfo->streamAggSup.stateStore.streamStateSaveInfo(pInfo->streamAggSup.pState, STREAM_TIME_SLICE_OP_CHECKPOINT_NAME, + strlen(STREAM_TIME_SLICE_OP_CHECKPOINT_NAME), buf, len); + saveStreamOperatorStateComplete(&pInfo->basic); + } + +_end: + taosMemoryFreeClear(buf); + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } +} + +SResultCellData* getSliceResultCell(SResultCellData* pRowVal, int32_t index) { + if (!pRowVal) { + return NULL; + } + char* pData = (char*)pRowVal; + SResultCellData* pCell = pRowVal; + for (int32_t i = 0; i < index; i++) { + pData += (pCell->bytes + sizeof(SResultCellData)); + pCell = (SResultCellData*)pData; + } + return pCell; +} + +static bool isGroupKeyFunc(SExprInfo* pExprInfo) { + int32_t functionType = pExprInfo->pExpr->_function.functionType; + return (functionType == FUNCTION_TYPE_GROUP_KEY); +} + +static bool isSelectGroupConstValueFunc(SExprInfo* pExprInfo) { + int32_t functionType = pExprInfo->pExpr->_function.functionType; + return (functionType == FUNCTION_TYPE_GROUP_CONST_VALUE); +} + +static int32_t fillPointResult(SStreamFillSupporter* pFillSup, SResultRowData* pResRow, SResultRowData* pNonFillRow, TSKEY ts, SSDataBlock* pBlock, + bool* pRes, bool isFilled) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + if (pBlock->info.rows >= pBlock->info.capacity) { + (*pRes) = false; + goto _end; + } + + bool ckRes = true; + code = checkResult(pFillSup, ts, pBlock->info.id.groupId, &ckRes); + QUERY_CHECK_CODE(code, lino, _end); + if (!ckRes) { + (*pRes) = true; + goto _end; + } + + for (int32_t i = 0; i < pFillSup->numOfAllCols; i++) { + SFillColInfo* pFillCol = pFillSup->pAllColInfo + i; + int32_t dstSlotId = GET_DEST_SLOT_ID(pFillCol); + SColumnInfoData* pDstCol = taosArrayGet(pBlock->pDataBlock, dstSlotId); + + if (isIrowtsPseudoColumn(pFillCol->pExpr)) { + code = colDataSetVal(pDstCol, pBlock->info.rows, (char*)&ts, false); + QUERY_CHECK_CODE(code, lino, _end); + } else if (isIsfilledPseudoColumn(pFillCol->pExpr)) { + code = colDataSetVal(pDstCol, pBlock->info.rows, (char*)&isFilled, false); + QUERY_CHECK_CODE(code, lino, _end); + } else { + int32_t srcSlot = pFillCol->pExpr->base.pParam[0].pCol->slotId; + SResultCellData* pCell = NULL; + if (IS_FILL_CONST_VALUE(pFillSup->type) && (isGroupKeyFunc(pFillCol->pExpr) || isSelectGroupConstValueFunc(pFillCol->pExpr)) ) { + pCell = getSliceResultCell(pNonFillRow->pRowVal, srcSlot); + } else { + pCell = getSliceResultCell(pResRow->pRowVal, srcSlot); + } + code = setRowCell(pDstCol, pBlock->info.rows, pCell); + QUERY_CHECK_CODE(code, lino, _end); + } + } + + pBlock->info.rows++; + (*pRes) = true; + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +static void fillNormalRange(SStreamFillSupporter* pFillSup, SStreamFillInfo* pFillInfo, SSDataBlock* pBlock) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + while (hasRemainCalc(pFillInfo) && pBlock->info.rows < pBlock->info.capacity) { + STimeWindow st = {.skey = pFillInfo->current, .ekey = pFillInfo->current}; + // if (inWinRange(&pFillSup->winRange, &st)) { + bool res = true; + code = fillPointResult(pFillSup, pFillInfo->pResRow, pFillInfo->pNonFillRow, pFillInfo->current, pBlock, &res, true); + QUERY_CHECK_CODE(code, lino, _end); + // } + pFillInfo->current = taosTimeAdd(pFillInfo->current, pFillSup->interval.sliding, pFillSup->interval.slidingUnit, + pFillSup->interval.precision); + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } +} + +static void fillLinearRange(SStreamFillSupporter* pFillSup, SStreamFillInfo* pFillInfo, SSDataBlock* pBlock) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + while (hasRemainCalc(pFillInfo) && pBlock->info.rows < pBlock->info.capacity) { + bool ckRes = true; + code = checkResult(pFillSup, pFillInfo->current, pBlock->info.id.groupId, &ckRes); + QUERY_CHECK_CODE(code, lino, _end); + for (int32_t i = 0; i < pFillSup->numOfAllCols && ckRes; ++i) { + SFillColInfo* pFillCol = pFillSup->pAllColInfo + i; + int32_t dstSlotId = GET_DEST_SLOT_ID(pFillCol); + SColumnInfoData* pDstCol = taosArrayGet(pBlock->pDataBlock, dstSlotId); + int16_t type = pDstCol->info.type; + int32_t index = pBlock->info.rows; + if (isIrowtsPseudoColumn(pFillCol->pExpr)) { + code = colDataSetVal(pDstCol, pBlock->info.rows, (char*)&pFillInfo->current, false); + QUERY_CHECK_CODE(code, lino, _end); + } else if (isIsfilledPseudoColumn(pFillCol->pExpr)) { + bool isFilled = true; + code = colDataSetVal(pDstCol, pBlock->info.rows, (char*)&isFilled, false); + QUERY_CHECK_CODE(code, lino, _end); + } else if (isInterpFunc(pFillCol->pExpr)) { + int32_t srcSlot = pFillCol->pExpr->base.pParam[0].pCol->slotId; + SResultCellData* pCell = getSliceResultCell(pFillInfo->pResRow->pRowVal, srcSlot); + if (IS_VAR_DATA_TYPE(type) || type == TSDB_DATA_TYPE_BOOL || pCell->isNull) { + colDataSetNULL(pDstCol, index); + continue; + } + SPoint* pEnd = taosArrayGet(pFillInfo->pLinearInfo->pEndPoints, srcSlot); + double vCell = 0; + SPoint start = {0}; + start.key = pFillInfo->pResRow->key; + start.val = pCell->pData; + + SPoint cur = {0}; + cur.key = pFillInfo->current; + cur.val = taosMemoryCalloc(1, pCell->bytes); + QUERY_CHECK_NULL(cur.val, code, lino, _end, terrno); + + taosGetLinearInterpolationVal(&cur, pCell->type, &start, pEnd, pCell->type); + code = colDataSetVal(pDstCol, index, (const char*)cur.val, false); + QUERY_CHECK_CODE(code, lino, _end); + + destroySPoint(&cur); + } else { + int32_t srcSlot = pFillCol->pExpr->base.pParam[0].pCol->slotId; + SResultCellData* pCell = getSliceResultCell(pFillInfo->pResRow->pRowVal, srcSlot); + code = setRowCell(pDstCol, pBlock->info.rows, pCell); + QUERY_CHECK_CODE(code, lino, _end); + } + } + pFillInfo->current = taosTimeAdd(pFillInfo->current, pFillSup->interval.sliding, pFillSup->interval.slidingUnit, + pFillSup->interval.precision); + if (ckRes) { + pBlock->info.rows++; + } + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } +} + +static void setFillKeyInfo(TSKEY start, TSKEY end, SInterval* pInterval, SStreamFillInfo* pFillInfo) { + pFillInfo->start = start; + pFillInfo->current = pFillInfo->start; + pFillInfo->end = end; +} + +static TSKEY adustPrevTsKey(TSKEY pointTs, TSKEY rowTs, SInterval* pInterval) { + if (rowTs >= pointTs) { + pointTs = taosTimeAdd(pointTs, pInterval->sliding, pInterval->slidingUnit, pInterval->precision); + } + return pointTs; +} + +static TSKEY adustEndTsKey(TSKEY pointTs, TSKEY rowTs, SInterval* pInterval) { + if (rowTs <= pointTs) { + pointTs = taosTimeAdd(pointTs, pInterval->sliding * -1, pInterval->slidingUnit, pInterval->precision); + } + return pointTs; +} + +static void adjustFillResRow(SResultRowData** ppResRow, SStreamFillSupporter* pFillSup) { + if (pFillSup->type == TSDB_FILL_PREV) { + (*ppResRow) = &pFillSup->cur; + } else if (pFillSup->type == TSDB_FILL_NEXT){ + (*ppResRow) = &pFillSup->next; + } +} + +static void doStreamFillRange(SStreamFillSupporter* pFillSup, SStreamFillInfo* pFillInfo, SSDataBlock* pRes) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + bool res = true; + if (pFillInfo->needFill == false && pFillInfo->pos != FILL_POS_INVALID) { + code = fillPointResult(pFillSup, &pFillSup->cur, pFillInfo->pNonFillRow, pFillSup->cur.key, pRes, &res, false); + QUERY_CHECK_CODE(code, lino, _end); + return; + } + + if (pFillInfo->pos == FILL_POS_START) { + code = fillPointResult(pFillSup, &pFillSup->cur, pFillInfo->pNonFillRow, pFillSup->cur.key, pRes, &res, false); + QUERY_CHECK_CODE(code, lino, _end); + if (res) { + pFillInfo->pos = FILL_POS_INVALID; + } + } + if (pFillInfo->type != TSDB_FILL_LINEAR) { + fillNormalRange(pFillSup, pFillInfo, pRes); + + if (pFillInfo->pos == FILL_POS_MID) { + code = fillPointResult(pFillSup, &pFillSup->cur, pFillInfo->pNonFillRow, pFillSup->cur.key, pRes, &res, false); + QUERY_CHECK_CODE(code, lino, _end); + if (res) { + pFillInfo->pos = FILL_POS_INVALID; + } + } + if (pFillInfo->current > pFillInfo->end && pFillInfo->hasNext) { + pFillInfo->hasNext = false; + TSKEY startTs = adustPrevTsKey(pFillInfo->current, pFillSup->cur.key, &pFillSup->interval); + setFillKeyInfo(startTs, pFillSup->next.key, &pFillSup->interval, pFillInfo); + adjustFillResRow(&pFillInfo->pResRow, pFillSup); + fillNormalRange(pFillSup, pFillInfo, pRes); + } + + } else { + fillLinearRange(pFillSup, pFillInfo, pRes); + + if (pFillInfo->pos == FILL_POS_MID) { + code = fillPointResult(pFillSup, &pFillSup->cur, pFillInfo->pNonFillRow, pFillSup->cur.key, pRes, &res, false); + QUERY_CHECK_CODE(code, lino, _end); + if (res) { + pFillInfo->pos = FILL_POS_INVALID; + } + } + + if (pFillInfo->current > pFillInfo->end && pFillInfo->pLinearInfo->hasNext) { + pFillInfo->pLinearInfo->hasNext = false; + taosArraySwap(pFillInfo->pLinearInfo->pEndPoints, pFillInfo->pLinearInfo->pNextEndPoints); + pFillInfo->pResRow = &pFillSup->cur; + setFillKeyInfo(pFillSup->cur.key, pFillInfo->pLinearInfo->nextEnd, &pFillSup->interval, pFillInfo); + fillLinearRange(pFillSup, pFillInfo, pRes); + } + } + if (pFillInfo->pos == FILL_POS_END) { + code = fillPointResult(pFillSup, &pFillSup->cur, pFillInfo->pNonFillRow, pFillSup->cur.key, pRes, &res, false); + QUERY_CHECK_CODE(code, lino, _end); + if (res) { + pFillInfo->pos = FILL_POS_INVALID; + } + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } +} + +static int32_t getQualifiedRowNumAsc(SExprSupp* pExprSup, SSDataBlock* pBlock, int32_t rowId, bool ignoreNull) { + if (rowId >= pBlock->info.rows) { + return -1; + } + + if (!ignoreNull) { + return rowId; + } + + for (int32_t i = rowId; i < pBlock->info.rows; i++) { + if (!checkNullRow(pExprSup, pBlock, i, ignoreNull)) { + return i; + } + } + return -1; +} + +int32_t getQualifiedRowNumDesc(SExprSupp* pExprSup, SSDataBlock* pBlock, TSKEY* tsCols, int32_t rowId, + bool ignoreNull) { + TSKEY ts = tsCols[rowId]; + int32_t resRow = -1; + for (; rowId >= 0; rowId--) { + if (checkNullRow(pExprSup, pBlock, rowId, ignoreNull)) { + continue; + } + + if (ts != tsCols[rowId]) { + if (resRow >= 0) { + break; + } else { + ts = tsCols[rowId]; + } + } + resRow = rowId; + } + return resRow; +} + +static void setResultRowData(SSliceRowData** ppRowData, void* pBuff) { (*ppRowData) = (SSliceRowData*)pBuff; } + +void setPointBuff(SSlicePoint* pPoint, SStreamFillSupporter* pFillSup) { + if (pFillSup->type != TSDB_FILL_LINEAR) { + setResultRowData(&pPoint->pRightRow, pPoint->pResPos->pRowBuff); + pPoint->pLeftRow = pPoint->pRightRow; + } else { + setResultRowData(&pPoint->pLeftRow, pPoint->pResPos->pRowBuff); + void* pBuff = POINTER_SHIFT(pPoint->pResPos->pRowBuff, pFillSup->rowSize + pFillSup->pkColBytes); + setResultRowData(&pPoint->pRightRow, pBuff); + } +} + +static int32_t getLinearResultInfoFromState(SStreamAggSupporter* pAggSup, SStreamFillSupporter* pFillSup, TSKEY ts, + int64_t groupId, SSlicePoint* pCurPoint, SSlicePoint* pPrevPoint, + SSlicePoint* pNextPoint) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + int32_t tmpRes = TSDB_CODE_SUCCESS; + void* pState = pAggSup->pState; + resetPrevAndNextWindow(pFillSup); + pCurPoint->pResPos = NULL; + pPrevPoint->pResPos = NULL; + pNextPoint->pResPos = NULL; + + pCurPoint->key.groupId = groupId; + pCurPoint->key.ts = ts; + int32_t curVLen = 0; + code = + pAggSup->stateStore.streamStateFillGet(pState, &pCurPoint->key, (void**)&pCurPoint->pResPos, &curVLen, &tmpRes); + QUERY_CHECK_CODE(code, lino, _end); + + setPointBuff(pCurPoint, pFillSup); + + if (HAS_ROW_DATA(pCurPoint->pRightRow)) { + pFillSup->cur.key = pCurPoint->pRightRow->key; + pFillSup->cur.pRowVal = (SResultCellData*)pCurPoint->pRightRow->pRowVal; + if (HAS_NON_ROW_DATA(pCurPoint->pLeftRow)) { + pPrevPoint->key.groupId = groupId; + int32_t preVLen = 0; + code = pAggSup->stateStore.streamStateFillGetPrev(pState, &pCurPoint->key, &pPrevPoint->key, + (void**)&pPrevPoint->pResPos, &preVLen, &tmpRes); + QUERY_CHECK_CODE(code, lino, _end); + if (tmpRes == TSDB_CODE_SUCCESS) { + QUERY_CHECK_CONDITION(!IS_INVALID_WIN_KEY(pPrevPoint->key.ts), code, lino, _end, TSDB_CODE_QRY_EXECUTOR_INTERNAL_ERROR); + setPointBuff(pPrevPoint, pFillSup); + if (HAS_ROW_DATA(pPrevPoint->pRightRow)) { + pFillSup->prev.key = pPrevPoint->pRightRow->key; + pFillSup->prev.pRowVal = (SResultCellData*)pPrevPoint->pRightRow->pRowVal; + } else { + pFillSup->prev.key = pPrevPoint->pLeftRow->key; + pFillSup->prev.pRowVal = (SResultCellData*)pPrevPoint->pLeftRow->pRowVal; + } + pFillSup->prevOriginKey = pFillSup->prev.key; + pFillSup->prev.key = adustPrevTsKey(pPrevPoint->key.ts, pFillSup->prev.key, &pFillSup->interval); + } + goto _end; + } + } + + if (HAS_ROW_DATA(pCurPoint->pLeftRow)) { + pFillSup->prev.key = pCurPoint->pLeftRow->key; + pFillSup->prev.pRowVal = (SResultCellData*)pCurPoint->pLeftRow->pRowVal; + pFillSup->prevOriginKey = pFillSup->prev.key; + pFillSup->prev.key = adustPrevTsKey(pCurPoint->key.ts, pFillSup->prev.key, &pFillSup->interval); + if (HAS_NON_ROW_DATA(pCurPoint->pRightRow)) { + pNextPoint->key.groupId = groupId; + int32_t nextVLen = 0; + code = pAggSup->stateStore.streamStateFillGetNext(pState, &pCurPoint->key, &pNextPoint->key, + (void**)&pNextPoint->pResPos, &nextVLen, &tmpRes); + QUERY_CHECK_CODE(code, lino, _end); + if (tmpRes == TSDB_CODE_SUCCESS) { + QUERY_CHECK_CONDITION(!IS_INVALID_WIN_KEY(pNextPoint->key.ts), code, lino, _end, TSDB_CODE_QRY_EXECUTOR_INTERNAL_ERROR); + setPointBuff(pNextPoint, pFillSup); + if (HAS_ROW_DATA(pNextPoint->pLeftRow)) { + pFillSup->next.key = pNextPoint->pLeftRow->key; + pFillSup->next.pRowVal = (SResultCellData*)pNextPoint->pLeftRow->pRowVal; + } else { + pFillSup->next.key = pNextPoint->pRightRow->key; + pFillSup->next.pRowVal = (SResultCellData*)pNextPoint->pRightRow->pRowVal; + } + pFillSup->nextOriginKey = pFillSup->next.key; + pFillSup->next.key = adustEndTsKey(pNextPoint->key.ts, pFillSup->next.key, &pFillSup->interval); + } else { + resetFillWindow(&pFillSup->prev); + } + } + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +static int32_t getResultInfoFromState(SStreamAggSupporter* pAggSup, SStreamFillSupporter* pFillSup, TSKEY ts, + int64_t groupId, SSlicePoint* pCurPoint, SSlicePoint* pPrevPoint, + SSlicePoint* pNextPoint) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + int32_t tmpRes = TSDB_CODE_SUCCESS; + void* pState = pAggSup->pState; + resetPrevAndNextWindow(pFillSup); + pCurPoint->pResPos = NULL; + pPrevPoint->pResPos = NULL; + pNextPoint->pResPos = NULL; + + pCurPoint->key.groupId = groupId; + pCurPoint->key.ts = ts; + int32_t curVLen = 0; + code = + pAggSup->stateStore.streamStateFillGet(pState, &pCurPoint->key, (void**)&pCurPoint->pResPos, &curVLen, &tmpRes); + QUERY_CHECK_CODE(code, lino, _end); + + if (tmpRes == TSDB_CODE_SUCCESS) { + setPointBuff(pCurPoint, pFillSup); + pFillSup->cur.key = pCurPoint->pRightRow->key; + pFillSup->cur.pRowVal = (SResultCellData*)pCurPoint->pRightRow->pRowVal; + } else { + pFillSup->cur.key = pCurPoint->key.ts + 1; + } + + pPrevPoint->key.groupId = groupId; + int32_t preVLen = 0; + code = pAggSup->stateStore.streamStateFillGetPrev(pState, &pCurPoint->key, &pPrevPoint->key, + (void**)&pPrevPoint->pResPos, &preVLen, &tmpRes); + QUERY_CHECK_CODE(code, lino, _end); + qDebug("===stream=== set stream interp resutl prev buf.ts:%" PRId64 ", groupId:%" PRId64 ", res:%d", pPrevPoint->key.ts, pPrevPoint->key.groupId, tmpRes); + + if (tmpRes == TSDB_CODE_SUCCESS) { + QUERY_CHECK_CONDITION(!IS_INVALID_WIN_KEY(pPrevPoint->key.ts), code, lino, _end, TSDB_CODE_QRY_EXECUTOR_INTERNAL_ERROR); + setPointBuff(pPrevPoint, pFillSup); + if (HAS_ROW_DATA(pPrevPoint->pRightRow)) { + pFillSup->prev.key = pPrevPoint->pRightRow->key; + pFillSup->prev.pRowVal = (SResultCellData*)pPrevPoint->pRightRow->pRowVal; + } else { + pFillSup->prev.key = pPrevPoint->pLeftRow->key; + pFillSup->prev.pRowVal = (SResultCellData*)pPrevPoint->pLeftRow->pRowVal; + } + pFillSup->prev.key = adustPrevTsKey(pPrevPoint->key.ts, pFillSup->prev.key, &pFillSup->interval); + } + + pNextPoint->key.groupId = groupId; + int32_t nextVLen = 0; + code = pAggSup->stateStore.streamStateFillGetNext(pState, &pCurPoint->key, &pNextPoint->key, + (void**)&pNextPoint->pResPos, &nextVLen, &tmpRes); + QUERY_CHECK_CODE(code, lino, _end); + qDebug("===stream=== set stream interp resutl next buf.ts:%" PRId64 ", groupId:%" PRId64 ", res:%d", pNextPoint->key.ts, pNextPoint->key.groupId, tmpRes); + if (tmpRes == TSDB_CODE_SUCCESS) { + QUERY_CHECK_CONDITION(!IS_INVALID_WIN_KEY(pNextPoint->key.ts), code, lino, _end, TSDB_CODE_QRY_EXECUTOR_INTERNAL_ERROR); + setPointBuff(pNextPoint, pFillSup); + if (HAS_ROW_DATA(pNextPoint->pLeftRow)) { + pFillSup->next.key = pNextPoint->pLeftRow->key; + pFillSup->next.pRowVal = (SResultCellData*)pNextPoint->pLeftRow->pRowVal; + } else { + pFillSup->next.key = pNextPoint->pRightRow->key; + pFillSup->next.pRowVal = (SResultCellData*)pNextPoint->pRightRow->pRowVal; + } + pFillSup->next.key = adustEndTsKey(pNextPoint->key.ts, pFillSup->next.key, &pFillSup->interval); + + if (pFillSup->type == TSDB_FILL_PREV) { + int32_t nextNextVLen = 0; + int32_t tmpWinCode = TSDB_CODE_SUCCESS; + SSlicePoint nextNextPoint = {.key.groupId = pNextPoint->key.groupId}; + code = pAggSup->stateStore.streamStateFillGetNext(pState, &pNextPoint->key, &nextNextPoint.key, NULL, NULL, + &tmpWinCode); + QUERY_CHECK_CODE(code, lino, _end); + if (tmpWinCode == TSDB_CODE_SUCCESS) { + pFillSup->nextNext.key = nextNextPoint.key.ts; + } + } + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +static int32_t getPointInfoFromStateRight(SStreamAggSupporter* pAggSup, SStreamFillSupporter* pFillSup, TSKEY ts, + int64_t groupId, SSlicePoint* pCurPoint, SSlicePoint* pNextPoint, + int32_t* pWinCode) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + int32_t tmpRes = TSDB_CODE_SUCCESS; + void* pState = pAggSup->pState; + pCurPoint->pResPos = NULL; + pNextPoint->pResPos = NULL; + + pNextPoint->key.groupId = groupId; + STimeWindow stw = {.skey = ts, .ekey = ts}; + getNextTimeWindow(&pFillSup->interval, &stw, TSDB_ORDER_ASC); + pNextPoint->key.ts = stw.skey; + + int32_t curVLen = 0; + code = pAggSup->stateStore.streamStateFillAddIfNotExist(pState, &pNextPoint->key, (void**)&pNextPoint->pResPos, + &curVLen, pWinCode); + QUERY_CHECK_CODE(code, lino, _end); + + qDebug("===stream=== set stream interp next point buf.ts:%" PRId64 ", groupId:%" PRId64 ", res:%d", + pNextPoint->key.ts, pNextPoint->key.groupId, *pWinCode); + + setPointBuff(pNextPoint, pFillSup); + + if (*pWinCode != TSDB_CODE_SUCCESS) { + if (pNextPoint->pLeftRow) { + SET_WIN_KEY_INVALID(pNextPoint->pLeftRow->key); + } + if (pNextPoint->pRightRow) { + SET_WIN_KEY_INVALID(pNextPoint->pRightRow->key); + } + } + + SET_WIN_KEY_INVALID(pCurPoint->key.ts); + pCurPoint->key.groupId = groupId; + int32_t nextVLen = 0; + code = pAggSup->stateStore.streamStateFillGetPrev(pState, &pNextPoint->key, &pCurPoint->key, + (void**)&pCurPoint->pResPos, &nextVLen, &tmpRes); + QUERY_CHECK_CODE(code, lino, _end); + + qDebug("===stream=== set stream interp cur point buf.ts:%" PRId64 ", groupId:%" PRId64 ", res:%d", pCurPoint->key.ts, pCurPoint->key.groupId, tmpRes); + + if (tmpRes == TSDB_CODE_SUCCESS) { + setPointBuff(pCurPoint, pFillSup); + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +static int32_t getPointInfoFromState(SStreamAggSupporter* pAggSup, SStreamFillSupporter* pFillSup, TSKEY ts, + int64_t groupId, SSlicePoint* pCurPoint, SSlicePoint* pNextPoint, + int32_t* pWinCode) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + int32_t tmpRes = TSDB_CODE_SUCCESS; + void* pState = pAggSup->pState; + pCurPoint->pResPos = NULL; + pNextPoint->pResPos = NULL; + pCurPoint->key.groupId = groupId; + pCurPoint->key.ts = ts; + + int32_t curVLen = 0; + code = pAggSup->stateStore.streamStateFillAddIfNotExist(pState, &pCurPoint->key, (void**)&pCurPoint->pResPos, + &curVLen, pWinCode); + QUERY_CHECK_CODE(code, lino, _end); + + qDebug("===stream=== set stream interp buf.ts:%" PRId64 ", groupId:%" PRId64, pCurPoint->key.ts, pCurPoint->key.groupId); + + setPointBuff(pCurPoint, pFillSup); + + if (*pWinCode != TSDB_CODE_SUCCESS) { + if (pCurPoint->pLeftRow) { + SET_WIN_KEY_INVALID(pCurPoint->pLeftRow->key); + } + if (pCurPoint->pRightRow) { + SET_WIN_KEY_INVALID(pCurPoint->pRightRow->key); + } + } + + int32_t nextVLen = 0; + pNextPoint->key.groupId = groupId; + if (pFillSup->type != TSDB_FILL_LINEAR && pFillSup->type != TSDB_FILL_PREV) { + SET_WIN_KEY_INVALID(pNextPoint->key.ts); + code = pAggSup->stateStore.streamStateFillGetNext(pState, &pCurPoint->key, &pNextPoint->key, + (void**)&pNextPoint->pResPos, &nextVLen, &tmpRes); + QUERY_CHECK_CODE(code, lino, _end); + if (tmpRes == TSDB_CODE_SUCCESS) { + setPointBuff(pNextPoint, pFillSup); + } + } else { + pNextPoint->key.ts = taosTimeAdd(pCurPoint->key.ts, pFillSup->interval.sliding, pFillSup->interval.slidingUnit, + pFillSup->interval.precision); + code = pAggSup->stateStore.streamStateFillAddIfNotExist(pState, &pNextPoint->key, (void**)&pNextPoint->pResPos, + &nextVLen, &tmpRes); + QUERY_CHECK_CODE(code, lino, _end); + setPointBuff(pNextPoint, pFillSup); + if (tmpRes != TSDB_CODE_SUCCESS) { + SET_WIN_KEY_INVALID(pNextPoint->pLeftRow->key); + SET_WIN_KEY_INVALID(pNextPoint->pRightRow->key); + } + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +// partition key +static void copyNonFillValueInfo(SStreamFillSupporter* pFillSup, SStreamFillInfo* pFillInfo) { + for (int32_t i = 0; i < pFillSup->numOfAllCols; ++i) { + SFillColInfo* pFillCol = pFillSup->pAllColInfo + i; + if (!isInterpFunc(pFillCol->pExpr) && !isIrowtsPseudoColumn(pFillCol->pExpr) && + !isIsfilledPseudoColumn(pFillCol->pExpr)) { + int32_t srcSlot = pFillCol->pExpr->base.pParam[0].pCol->slotId; + SResultCellData* pSrcCell = getResultCell(&pFillSup->cur, srcSlot); + SResultCellData* pDestCell = getResultCell(pFillInfo->pNonFillRow, srcSlot); + pDestCell->isNull = pSrcCell->isNull; + if (!pDestCell->isNull) { + memcpy(pDestCell->pData, pSrcCell->pData, pSrcCell->bytes); + } + } + } +} + +static void copyCalcRowDeltaData(SResultRowData* pEndRow, SArray* pEndPoins, SFillColInfo* pFillCol, int32_t numOfCol) { + for (int32_t i = 0; i < numOfCol; i++) { + if (isInterpFunc(pFillCol[i].pExpr)) { + int32_t slotId = pFillCol[i].pExpr->base.pParam[0].pCol->slotId; + SResultCellData* pECell = getResultCell(pEndRow, slotId); + SPoint* pPoint = taosArrayGet(pEndPoins, slotId); + pPoint->key = pEndRow->key; + memcpy(pPoint->val, pECell->pData, pECell->bytes); + } + } +} + +static void setForceWindowCloseFillRule(SStreamFillSupporter* pFillSup, SStreamFillInfo* pFillInfo, TSKEY ts) { + qDebug("===stream=== set force window close rule.ts:%" PRId64 ",cur key:%" PRId64 ", has prev%d, has next:%d", ts, + pFillSup->cur.key, hasPrevWindow(pFillSup), hasNextWindow(pFillSup)); + pFillInfo->needFill = true; + pFillInfo->pos = FILL_POS_INVALID; + switch (pFillInfo->type) { + case TSDB_FILL_NULL: + case TSDB_FILL_NULL_F: + case TSDB_FILL_SET_VALUE: + case TSDB_FILL_SET_VALUE_F: { + if (ts == pFillSup->cur.key) { + pFillInfo->pos = FILL_POS_START; + pFillInfo->needFill = false; + } else { + pFillInfo->pos = FILL_POS_INVALID; + setFillKeyInfo(ts, ts + 1, &pFillSup->interval, pFillInfo); + } + if (pFillSup->cur.pRowVal != NULL) { + copyNonFillValueInfo(pFillSup, pFillInfo); + } + } break; + case TSDB_FILL_PREV: { + if (ts == pFillSup->cur.key) { + pFillInfo->pos = FILL_POS_START; + pFillInfo->needFill = false; + } else if (ts > pFillSup->cur.key) { + setFillKeyInfo(ts, ts + 1, &pFillSup->interval, pFillInfo); + pFillInfo->pResRow = &pFillSup->cur; + } else if (hasPrevWindow(pFillSup)) { + pFillInfo->pos = FILL_POS_INVALID; + setFillKeyInfo(ts, ts + 1, &pFillSup->interval, pFillInfo); + pFillInfo->pResRow = &pFillSup->prev; + } else { + pFillInfo->needFill = false; + pFillInfo->pos = FILL_POS_INVALID; + } + } break; + default: + qError("%s failed at line %d since invalid fill type", __func__, __LINE__); + break; + } +} + +static void setTimeSliceFillRule(SStreamFillSupporter* pFillSup, SStreamFillInfo* pFillInfo, TSKEY ts) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + if (IS_FILL_CONST_VALUE(pFillInfo->type)) { + copyNonFillValueInfo(pFillSup, pFillInfo); + } + if (!hasNextWindow(pFillSup) && !hasPrevWindow(pFillSup)) { + pFillInfo->needFill = false; + pFillInfo->pos = FILL_POS_START; + goto _end; + } + + TSKEY prevWKey = INT64_MIN; + TSKEY nextWKey = INT64_MIN; + if (hasPrevWindow(pFillSup)) { + prevWKey = pFillSup->prev.key; + } + if (hasNextWindow(pFillSup)) { + nextWKey = pFillSup->next.key; + } + TSKEY endTs = adustEndTsKey(ts, pFillSup->cur.key, &pFillSup->interval); + TSKEY startTs = adustPrevTsKey(ts, pFillSup->cur.key, &pFillSup->interval); + + pFillInfo->needFill = true; + pFillInfo->pos = FILL_POS_INVALID; + switch (pFillInfo->type) { + case TSDB_FILL_NULL: + case TSDB_FILL_NULL_F: + case TSDB_FILL_SET_VALUE: + case TSDB_FILL_SET_VALUE_F: { + if (hasPrevWindow(pFillSup) && hasNextWindow(pFillSup) && pFillInfo->preRowKey == pFillInfo->prePointKey && + pFillInfo->nextRowKey != pFillInfo->nextPointKey) { + setFillKeyInfo(prevWKey, endTs, &pFillSup->interval, pFillInfo); + pFillInfo->pos = FILL_POS_MID; + pFillInfo->hasNext = true; + } else if (hasPrevWindow(pFillSup)) { + setFillKeyInfo(prevWKey, endTs, &pFillSup->interval, pFillInfo); + pFillInfo->pos = FILL_POS_END; + } else { + setFillKeyInfo(startTs, nextWKey, &pFillSup->interval, pFillInfo); + pFillInfo->pos = FILL_POS_START; + } + // copyNonFillValueInfo(pFillSup, pFillInfo); + } break; + case TSDB_FILL_PREV: { + if (hasPrevWindow(pFillSup) && hasNextWindow(pFillSup) && pFillInfo->preRowKey != pFillInfo->prePointKey && + pFillInfo->nextRowKey == pFillInfo->nextPointKey) { + setFillKeyInfo(prevWKey, endTs, &pFillSup->interval, pFillInfo); + pFillInfo->pos = FILL_POS_MID; + pFillInfo->hasNext = true; + } else if (hasNextWindow(pFillSup)) { + setFillKeyInfo(startTs, nextWKey, &pFillSup->interval, pFillInfo); + pFillInfo->pos = FILL_POS_START; + resetFillWindow(&pFillSup->prev); + pFillSup->prev.key = ts; + pFillSup->prev.pRowVal = pFillSup->cur.pRowVal; + } else { + QUERY_CHECK_CONDITION(hasPrevWindow(pFillSup), code, lino, _end, TSDB_CODE_QRY_EXECUTOR_INTERNAL_ERROR); + setFillKeyInfo(prevWKey, endTs, &pFillSup->interval, pFillInfo); + pFillInfo->pos = FILL_POS_END; + } + pFillInfo->pResRow = &pFillSup->prev; + } break; + case TSDB_FILL_NEXT: { + if (hasPrevWindow(pFillSup) && hasNextWindow(pFillSup) && pFillInfo->preRowKey == pFillInfo->prePointKey && + pFillInfo->nextRowKey != pFillInfo->nextPointKey) { + setFillKeyInfo(prevWKey, endTs, &pFillSup->interval, pFillInfo); + pFillInfo->pos = FILL_POS_MID; + pFillInfo->hasNext = true; + pFillInfo->pResRow = &pFillSup->cur; + } else if (hasPrevWindow(pFillSup)) { + setFillKeyInfo(prevWKey, endTs, &pFillSup->interval, pFillInfo); + pFillInfo->pos = FILL_POS_END; + resetFillWindow(&pFillSup->next); + pFillSup->next.key = ts; + pFillSup->next.pRowVal = pFillSup->cur.pRowVal; + pFillInfo->pResRow = &pFillSup->next; + } else { + setFillKeyInfo(startTs, nextWKey, &pFillSup->interval, pFillInfo); + pFillInfo->pos = FILL_POS_START; + resetFillWindow(&pFillSup->prev); + pFillInfo->pResRow = &pFillSup->next; + } + } break; + case TSDB_FILL_LINEAR: { + if (hasPrevWindow(pFillSup) && hasNextWindow(pFillSup)) { + setFillKeyInfo(prevWKey, nextWKey, &pFillSup->interval, pFillInfo); + pFillInfo->pos = FILL_POS_INVALID; + SET_WIN_KEY_INVALID(pFillInfo->pLinearInfo->nextEnd); + pFillSup->next.key = pFillSup->nextOriginKey; + copyCalcRowDeltaData(&pFillSup->next, pFillInfo->pLinearInfo->pEndPoints, pFillSup->pAllColInfo, + pFillSup->numOfAllCols); + pFillSup->prev.key = pFillSup->prevOriginKey; + pFillInfo->pResRow = &pFillSup->prev; + pFillInfo->pLinearInfo->hasNext = false; + } else if (hasPrevWindow(pFillSup)) { + setFillKeyInfo(prevWKey, endTs, &pFillSup->interval, pFillInfo); + pFillInfo->pos = FILL_POS_END; + SET_WIN_KEY_INVALID(pFillInfo->pLinearInfo->nextEnd); + copyCalcRowDeltaData(&pFillSup->cur, pFillInfo->pLinearInfo->pEndPoints, pFillSup->pAllColInfo, + pFillSup->numOfAllCols); + pFillSup->prev.key = pFillSup->prevOriginKey; + pFillInfo->pResRow = &pFillSup->prev; + pFillInfo->pLinearInfo->hasNext = false; + } else { + QUERY_CHECK_CONDITION(hasNextWindow(pFillSup), code, lino, _end, TSDB_CODE_QRY_EXECUTOR_INTERNAL_ERROR); + setFillKeyInfo(startTs, nextWKey, &pFillSup->interval, pFillInfo); + pFillInfo->pos = FILL_POS_START; + SET_WIN_KEY_INVALID(pFillInfo->pLinearInfo->nextEnd); + pFillSup->next.key = pFillSup->nextOriginKey; + copyCalcRowDeltaData(&pFillSup->next, pFillInfo->pLinearInfo->pEndPoints, pFillSup->pAllColInfo, + pFillSup->numOfAllCols); + pFillInfo->pResRow = &pFillSup->cur; + pFillInfo->pLinearInfo->hasNext = false; + } + } break; + default: + qError("%s failed at line %d since invalid fill type", __func__, __LINE__); + break; + } + +_end: + if (ts != pFillSup->cur.key) { + pFillInfo->pos = FILL_POS_INVALID; + } + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } +} + +static int32_t comparePkVal(void* pLeft, void* pRight, SStreamFillSupporter* pFillSup) { + void* pTmpVal = POINTER_SHIFT(pLeft, pFillSup->rowSize); + return pFillSup->comparePkColFn(pTmpVal, pRight); +} + +static bool needAdjustValue(SSlicePoint* pPoint, TSKEY ts, void* pPkVal, SStreamFillSupporter* pFillSup, bool isLeft, + int32_t fillType) { + if (IS_INVALID_WIN_KEY(pPoint->key.ts)) { + return false; + } + + switch (fillType) { + case TSDB_FILL_NULL: + case TSDB_FILL_NULL_F: + case TSDB_FILL_SET_VALUE: + case TSDB_FILL_SET_VALUE_F: { + if (!isLeft) { + if (HAS_NON_ROW_DATA(pPoint->pRightRow)) { + return true; + } else { + if (pPoint->key.ts == ts) { + if (pFillSup->comparePkColFn == NULL || + comparePkVal(pPoint->pRightRow, pPkVal, pFillSup) >= 0) { + return true; + } + } + } + } + } break; + case TSDB_FILL_PREV: { + if (isLeft) { + if (HAS_NON_ROW_DATA(pPoint->pLeftRow)) { + return true; + } else { + if (pPoint->pLeftRow->key < ts) { + return true; + } else if (pPoint->pLeftRow->key == ts) { + if (pFillSup->comparePkColFn == NULL || comparePkVal(pPoint->pLeftRow, pPkVal, pFillSup) >= 0) { + return true; + } + } + } + } + + if (!isLeft && pPoint->key.ts == ts) { + if (HAS_NON_ROW_DATA(pPoint->pLeftRow) || pFillSup->comparePkColFn == NULL || + comparePkVal(pPoint->pLeftRow, pPkVal, pFillSup) >= 0) { + return true; + } + } + } break; + case TSDB_FILL_NEXT: { + if (!isLeft) { + if (HAS_NON_ROW_DATA(pPoint->pRightRow)) { + return true; + } else { + if (pPoint->pRightRow->key > ts) { + return true; + } else if (pPoint->pRightRow->key == ts) { + if (pFillSup->comparePkColFn == NULL || + comparePkVal(pPoint->pRightRow, pPkVal, pFillSup) >= 0) { + return true; + } + } + } + } + } break; + case TSDB_FILL_LINEAR: { + if (isLeft) { + if (HAS_NON_ROW_DATA(pPoint->pLeftRow)) { + return true; + } else { + if (pPoint->pLeftRow->key < ts) { + return true; + } else if (pPoint->pLeftRow->key == ts) { + if (pFillSup->comparePkColFn == NULL || comparePkVal(pPoint->pLeftRow, pPkVal, pFillSup) >= 0) { + return true; + } + } + } + } else { + if (HAS_NON_ROW_DATA(pPoint->pRightRow)) { + return true; + } else { + if (pPoint->pRightRow->key > ts) { + return true; + } else if (pPoint->pRightRow->key == ts) { + if (pFillSup->comparePkColFn == NULL || + comparePkVal(pPoint->pRightRow, pPkVal, pFillSup) >= 0) { + return true; + } + } + } + } + } break; + default: + qError("%s failed at line %d since invalid fill type", __func__, __LINE__); + } + return false; +} + +void transBlockToSliceResultRow(const SSDataBlock* pBlock, int32_t rowId, TSKEY ts, SSliceRowData* pRowVal, + int32_t rowSize, void* pPkData, SColumnInfoData* pPkCol) { + int32_t numOfCols = taosArrayGetSize(pBlock->pDataBlock); + for (int32_t i = 0; i < numOfCols; ++i) { + SColumnInfoData* pColData = taosArrayGet(pBlock->pDataBlock, i); + SResultCellData* pCell = getSliceResultCell((SResultCellData*)pRowVal->pRowVal, i); + if (!colDataIsNull_s(pColData, rowId)) { + pCell->isNull = false; + pCell->type = pColData->info.type; + pCell->bytes = pColData->info.bytes; + char* val = colDataGetData(pColData, rowId); + if (IS_VAR_DATA_TYPE(pCell->type)) { + memcpy(pCell->pData, val, varDataTLen(val)); + } else { + memcpy(pCell->pData, val, pCell->bytes); + } + } else { + pCell->isNull = true; + } + } + pRowVal->key = ts; + if (pPkData != NULL) { + void* pPkVal = POINTER_SHIFT(pRowVal, rowSize); + if (IS_VAR_DATA_TYPE(pPkCol->info.type)) { + memcpy(pPkVal, pPkData, varDataTLen(pPkData)); + } else { + memcpy(pPkVal, pPkData, pPkCol->info.bytes); + } + } +} + +static int32_t saveTimeSliceWinResultInfo(SStreamAggSupporter* pAggSup, STimeWindowAggSupp* pTwAggSup, SWinKey* pKey, + SSHashObj* pUpdatedMap, bool needDel, SSHashObj* pDeletedMap) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + + if (pTwAggSup->calTrigger == STREAM_TRIGGER_AT_ONCE) { + code = saveTimeSliceWinResult(pKey, pUpdatedMap); + QUERY_CHECK_CODE(code, lino, _end); + if (needDel) { + code = saveTimeSliceWinResult(pKey, pDeletedMap); + QUERY_CHECK_CODE(code, lino, _end); + } + } else if (pTwAggSup->calTrigger == STREAM_TRIGGER_FORCE_WINDOW_CLOSE) { + code = pAggSup->stateStore.streamStateGroupPut(pAggSup->pState, pKey->groupId, NULL, 0); + QUERY_CHECK_CODE(code, lino, _end); + } + pTwAggSup->maxTs = TMAX(pTwAggSup->maxTs, pKey->ts); + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +static void doStreamTimeSliceImpl(SOperatorInfo* pOperator, SSDataBlock* pBlock) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + int32_t winCode = TSDB_CODE_SUCCESS; + SStreamTimeSliceOperatorInfo* pInfo = (SStreamTimeSliceOperatorInfo*)pOperator->info; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; + SExprSupp* pExprSup = &pOperator->exprSupp; + int32_t numOfOutput = pExprSup->numOfExprs; + SColumnInfoData* pColDataInfo = taosArrayGet(pBlock->pDataBlock, pInfo->primaryTsIndex); + TSKEY* tsCols = (int64_t*)pColDataInfo->pData; + void* pPkVal = NULL; + int32_t pkLen = 0; + int64_t groupId = pBlock->info.id.groupId; + SColumnInfoData* pPkColDataInfo = NULL; + SStreamFillSupporter* pFillSup = pInfo->pFillSup; + SStreamFillInfo* pFillInfo = pInfo->pFillInfo; + if (hasSrcPrimaryKeyCol(&pInfo->basic)) { + pPkColDataInfo = taosArrayGet(pBlock->pDataBlock, pInfo->basic.primaryPkIndex); + } + + pFillSup->winRange = pTaskInfo->streamInfo.fillHistoryWindow; + if (pFillSup->winRange.ekey <= 0) { + pFillSup->winRange.ekey = INT64_MAX; + } + + int32_t startPos = 0; + for (; startPos < pBlock->info.rows; startPos++) { + if (hasSrcPrimaryKeyCol(&pInfo->basic) && pInfo->ignoreExpiredData) { + pPkVal = colDataGetData(pPkColDataInfo, startPos); + pkLen = colDataGetRowLength(pPkColDataInfo, startPos); + } + + if (pInfo->twAggSup.calTrigger != STREAM_TRIGGER_FORCE_WINDOW_CLOSE && pInfo->ignoreExpiredData && + checkExpiredData(&pAggSup->stateStore, pAggSup->pUpdateInfo, &pInfo->twAggSup, pBlock->info.id.uid, + tsCols[startPos], pPkVal, pkLen)) { + qDebug("===stream===ignore expired data, window end ts:%" PRId64 ", maxts - wartermak:%" PRId64, tsCols[startPos], + pInfo->twAggSup.maxTs - pInfo->twAggSup.waterMark); + continue; + } + + if (checkNullRow(pExprSup, pBlock, startPos, pInfo->ignoreNull)) { + continue; + } + break; + } + + if (startPos >= pBlock->info.rows) { + return; + } + + SResultRowInfo dumyInfo = {0}; + dumyInfo.cur.pageId = -1; + STimeWindow curWin = getActiveTimeWindow(NULL, &dumyInfo, tsCols[startPos], &pFillSup->interval, TSDB_ORDER_ASC); + SSlicePoint curPoint = {0}; + SSlicePoint nextPoint = {0}; + bool left = false; + bool right = false; + if (pFillSup->type != TSDB_FILL_PREV || curWin.skey == tsCols[startPos]) { + code = getPointInfoFromState(pAggSup, pFillSup, curWin.skey, groupId, &curPoint, &nextPoint, &winCode); + } else { + code = getPointInfoFromStateRight(pAggSup, pFillSup, curWin.skey, groupId, &curPoint, &nextPoint, &winCode); + } + QUERY_CHECK_CODE(code, lino, _end); + + if (hasSrcPrimaryKeyCol(&pInfo->basic)) { + pPkVal = colDataGetData(pPkColDataInfo, startPos); + } + right = needAdjustValue(&curPoint, tsCols[startPos], pPkVal, pFillSup, false, pFillSup->type); + if (right) { + transBlockToSliceResultRow(pBlock, startPos, tsCols[startPos], curPoint.pRightRow, pFillSup->rowSize, pPkVal, pPkColDataInfo); + bool needDel = pInfo->destHasPrimaryKey && winCode == TSDB_CODE_SUCCESS; + code = saveTimeSliceWinResultInfo(pAggSup, &pInfo->twAggSup, &curPoint.key, pInfo->pUpdatedMap, needDel, + pInfo->pDeletedMap); + QUERY_CHECK_CODE(code, lino, _end); + } + releaseOutputBuf(pAggSup->pState, curPoint.pResPos, &pAggSup->stateStore); + + while (startPos < pBlock->info.rows) { + int32_t numOfWin = getNumOfRowsInTimeWindow(&pBlock->info, tsCols, startPos, curWin.ekey, binarySearchForKey, NULL, + TSDB_ORDER_ASC); + startPos += numOfWin; + int32_t leftRowId = getQualifiedRowNumDesc(pExprSup, pBlock, tsCols, startPos - 1, pInfo->ignoreNull); + QUERY_CHECK_CONDITION((leftRowId >= 0), code, lino, _end, TSDB_CODE_QRY_EXECUTOR_INTERNAL_ERROR); + if (hasSrcPrimaryKeyCol(&pInfo->basic)) { + pPkVal = colDataGetData(pPkColDataInfo, leftRowId); + } + left = needAdjustValue(&nextPoint, tsCols[leftRowId], pPkVal, pFillSup, true, pFillSup->type); + if (left) { + transBlockToSliceResultRow(pBlock, leftRowId, tsCols[leftRowId], nextPoint.pLeftRow, pFillSup->rowSize, pPkVal, pPkColDataInfo); + bool needDel = pInfo->destHasPrimaryKey && winCode == TSDB_CODE_SUCCESS; + code = saveTimeSliceWinResultInfo(pAggSup, &pInfo->twAggSup, &nextPoint.key, pInfo->pUpdatedMap, + needDel, pInfo->pDeletedMap); + QUERY_CHECK_CODE(code, lino, _end); + } + releaseOutputBuf(pAggSup->pState, nextPoint.pResPos, &pAggSup->stateStore); + + startPos = getQualifiedRowNumAsc(pExprSup, pBlock, startPos, pInfo->ignoreNull); + if (startPos < 0) { + break; + } + curWin = getActiveTimeWindow(NULL, &dumyInfo, tsCols[startPos], &pFillSup->interval, TSDB_ORDER_ASC); + if (pFillSup->type != TSDB_FILL_PREV || curWin.skey == tsCols[startPos]) { + code = getPointInfoFromState(pAggSup, pFillSup, curWin.skey, groupId, &curPoint, &nextPoint, &winCode); + } else { + code = getPointInfoFromStateRight(pAggSup, pFillSup, curWin.skey, groupId, &curPoint, &nextPoint, &winCode); + } + QUERY_CHECK_CODE(code, lino, _end); + + if (hasSrcPrimaryKeyCol(&pInfo->basic)) { + pPkVal = colDataGetData(pPkColDataInfo, startPos); + } + right = needAdjustValue(&curPoint, tsCols[startPos], pPkVal, pFillSup, false, pFillSup->type); + if (right) { + transBlockToSliceResultRow(pBlock, startPos, tsCols[startPos], curPoint.pRightRow, pFillSup->rowSize, pPkVal, pPkColDataInfo); + bool needDel = pInfo->destHasPrimaryKey && winCode == TSDB_CODE_SUCCESS; + code = saveTimeSliceWinResultInfo(pAggSup, &pInfo->twAggSup, &curPoint.key, pInfo->pUpdatedMap, needDel, + pInfo->pDeletedMap); + QUERY_CHECK_CODE(code, lino, _end); + } + releaseOutputBuf(pAggSup->pState, curPoint.pResPos, &pAggSup->stateStore); + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } +} + +void getNextResKey(int64_t curGroupId, SArray* pKeyArray, int32_t curIndex, TSKEY* pNextKey) { + int32_t nextIndex = curIndex + 1; + if (nextIndex < taosArrayGetSize(pKeyArray)) { + SWinKey* pKey = (SWinKey*)taosArrayGet(pKeyArray, nextIndex); + if (pKey->groupId == curGroupId) { + *pNextKey = pKey->ts; + return; + } + } + *pNextKey = INT64_MIN; +} + +void getPrevResKey(int64_t curGroupId, SArray* pKeyArray, int32_t curIndex, TSKEY* pNextKey) { + int32_t prevIndex = curIndex - 1; + if (prevIndex >= 0) { + SWinKey* pKey = (SWinKey*)taosArrayGet(pKeyArray, prevIndex); + if (pKey->groupId == curGroupId) { + *pNextKey = pKey->ts; + return; + } + } + *pNextKey = INT64_MIN; +} + +void doBuildTimeSlicePointResult(SStreamAggSupporter* pAggSup, STimeWindowAggSupp* pTwSup, SStreamFillSupporter* pFillSup, + SStreamFillInfo* pFillInfo, SSDataBlock* pBlock, SGroupResInfo* pGroupResInfo) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + blockDataCleanup(pBlock); + if (!hasRemainResults(pGroupResInfo)) { + return; + } + + // clear the existed group id + pBlock->info.id.groupId = 0; + int32_t numOfRows = getNumOfTotalRes(pGroupResInfo); + for (; pGroupResInfo->index < numOfRows; pGroupResInfo->index++) { + SWinKey* pKey = (SWinKey*)taosArrayGet(pGroupResInfo->pRows, pGroupResInfo->index); + qDebug("===stream=== build interp res. key:%" PRId64 ",groupId:%" PRIu64, pKey->ts, pKey->groupId); + if (pBlock->info.id.groupId == 0) { + pBlock->info.id.groupId = pKey->groupId; + } else if (pBlock->info.id.groupId != pKey->groupId) { + if (pBlock->info.rows > 0) { + break; + } else { + pBlock->info.id.groupId = pKey->groupId; + } + } + SSlicePoint curPoint = {.key.ts = pKey->ts, .key.groupId = pKey->groupId}; + SSlicePoint prevPoint = {0}; + SSlicePoint nextPoint = {0}; + if (pFillSup->type != TSDB_FILL_LINEAR) { + code = getResultInfoFromState(pAggSup, pFillSup, pKey->ts, pKey->groupId, &curPoint, &prevPoint, &nextPoint); + } else { + code = + getLinearResultInfoFromState(pAggSup, pFillSup, pKey->ts, pKey->groupId, &curPoint, &prevPoint, &nextPoint); + } + QUERY_CHECK_CODE(code, lino, _end); + + if (pFillSup->type != TSDB_FILL_LINEAR) { + getPrevResKey(pKey->groupId, pGroupResInfo->pRows, pGroupResInfo->index, &pFillInfo->preRowKey); + if (hasPrevWindow(pFillSup)) { + pFillInfo->prePointKey = prevPoint.key.ts; + } + + getNextResKey(pKey->groupId, pGroupResInfo->pRows, pGroupResInfo->index, &pFillInfo->nextRowKey); + if (hasNextWindow(pFillSup)) { + pFillInfo->nextPointKey = nextPoint.key.ts; + } + } + + if (pTwSup->calTrigger == STREAM_TRIGGER_FORCE_WINDOW_CLOSE) { + setForceWindowCloseFillRule(pFillSup, pFillInfo, pKey->ts); + } else { + setTimeSliceFillRule(pFillSup, pFillInfo, pKey->ts); + } + doStreamFillRange(pFillSup, pFillInfo, pBlock); + releaseOutputBuf(pAggSup->pState, curPoint.pResPos, &pAggSup->stateStore); + releaseOutputBuf(pAggSup->pState, prevPoint.pResPos, &pAggSup->stateStore); + releaseOutputBuf(pAggSup->pState, nextPoint.pResPos, &pAggSup->stateStore); + if (pBlock->info.rows >= pBlock->info.capacity) { + pGroupResInfo->index++; + break; + } + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } +} + +static void doBuildTimeSliceDeleteResult(SStreamAggSupporter* pAggSup, SStreamFillSupporter* pFillSup, SArray* pWins, int32_t* index, SSDataBlock* pBlock) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + blockDataCleanup(pBlock); + int32_t size = taosArrayGetSize(pWins); + if (*index == size) { + *index = 0; + taosArrayClear(pWins); + goto _end; + } + code = blockDataEnsureCapacity(pBlock, size - *index); + QUERY_CHECK_CODE(code, lino, _end); + + uint64_t uid = 0; + for (int32_t i = *index; i < size; i++) { + SWinKey* pKey = taosArrayGet(pWins, i); + SSlicePoint curPoint = {.key.ts = pKey->ts, .key.groupId = pKey->groupId}; + SSlicePoint prevPoint = {0}; + SSlicePoint nextPoint = {0}; + STimeWindow tw = {0}; + if (pFillSup->type != TSDB_FILL_LINEAR) { + code = getResultInfoFromState(pAggSup, pFillSup, pKey->ts, pKey->groupId, &curPoint, &prevPoint, &nextPoint); + } else { + code = + getLinearResultInfoFromState(pAggSup, pFillSup, pKey->ts, pKey->groupId, &curPoint, &prevPoint, &nextPoint); + } + QUERY_CHECK_CODE(code, lino, _end); + + if (pFillSup->type == TSDB_FILL_PREV && hasNextWindow(pFillSup)) { + tw.skey = pFillSup->cur.key; + tw.ekey = pFillSup->next.key; + } else if (pFillSup->type == TSDB_FILL_NEXT && hasPrevWindow(pFillSup)) { + tw.skey = pFillSup->prev.key; + tw.ekey = pFillSup->cur.key; + } else if (pFillSup->type == TSDB_FILL_LINEAR) { + if (hasPrevWindow(pFillSup)) { + tw.skey = pFillSup->prev.key; + } else { + tw.skey = pFillSup->cur.key; + } + if (hasNextWindow(pFillSup)) { + tw.ekey = pFillSup->next.key; + } else { + tw.ekey = pFillSup->cur.key; + } + } else { + tw.skey = pFillSup->cur.key; + tw.ekey = pFillSup->cur.key; + } + + if (tw.skey == INT64_MIN || tw.ekey == INT64_MIN) { + continue; + } + + releaseOutputBuf(pAggSup->pState, curPoint.pResPos, &pAggSup->stateStore); + releaseOutputBuf(pAggSup->pState, prevPoint.pResPos, &pAggSup->stateStore); + releaseOutputBuf(pAggSup->pState, nextPoint.pResPos, &pAggSup->stateStore); + + void* tbname = NULL; + int32_t winCode = TSDB_CODE_SUCCESS; + + code = pAggSup->stateStore.streamStateGetParName(pAggSup->pState, pKey->groupId, &tbname, false, &winCode); + QUERY_CHECK_CODE(code, lino, _end); + + if (winCode != TSDB_CODE_SUCCESS) { + code = appendDataToSpecialBlock(pBlock, &tw.skey, &tw.ekey, &uid, &pKey->groupId, NULL); + QUERY_CHECK_CODE(code, lino, _end); + } else { + QUERY_CHECK_CONDITION((tbname), code, lino, _end, TSDB_CODE_QRY_EXECUTOR_INTERNAL_ERROR); + char parTbName[VARSTR_HEADER_SIZE + TSDB_TABLE_NAME_LEN]; + STR_WITH_MAXSIZE_TO_VARSTR(parTbName, tbname, sizeof(parTbName)); + code = appendDataToSpecialBlock(pBlock, &tw.skey, &tw.ekey, &uid, &pKey->groupId, parTbName); + QUERY_CHECK_CODE(code, lino, _end); + } + pAggSup->stateStore.streamStateFreeVal(tbname); + (*index)++; + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } +} + +static int32_t buildTimeSliceResult(SOperatorInfo* pOperator, SSDataBlock** ppRes) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + SStreamTimeSliceOperatorInfo* pInfo = pOperator->info; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + uint16_t opType = pOperator->operatorType; + SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; + + + doBuildTimeSliceDeleteResult(pAggSup, pInfo->pFillSup, pInfo->pDelWins, &pInfo->delIndex, pInfo->pDelRes); + if (pInfo->pDelRes->info.rows != 0) { + // process the rest of the data + printDataBlock(pInfo->pDelRes, getStreamOpName(opType), GET_TASKID(pTaskInfo)); + (*ppRes) = pInfo->pDelRes; + goto _end; + } + + doBuildTimeSlicePointResult(pAggSup, &pInfo->twAggSup, pInfo->pFillSup, pInfo->pFillInfo, pInfo->pRes, &pInfo->groupResInfo); + if (pInfo->pRes->info.rows != 0) { + printDataBlock(pInfo->pRes, getStreamOpName(opType), GET_TASKID(pTaskInfo)); + (*ppRes) = pInfo->pRes; + goto _end; + } + + (*ppRes) = NULL; + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +int32_t getSliceMaxTsWins(const SArray* pAllWins, SArray* pMaxWins) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + int32_t size = taosArrayGetSize(pAllWins); + if (size == 0) { + goto _end; + } + SWinKey* pKey = taosArrayGet(pAllWins, size - 1); + void* tmp = taosArrayPush(pMaxWins, pKey); + QUERY_CHECK_NULL(tmp, code, lino, _end, terrno); + + if (pKey->groupId == 0) { + goto _end; + } + uint64_t preGpId = pKey->groupId; + for (int32_t i = size - 2; i >= 0; i--) { + pKey = taosArrayGet(pAllWins, i); + if (preGpId != pKey->groupId) { + void* p = taosArrayPush(pMaxWins, pKey); + QUERY_CHECK_NULL(p, code, lino, _end, terrno); + preGpId = pKey->groupId; + } + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +static int32_t doDeleteTimeSliceResult(SStreamAggSupporter* pAggSup, SSDataBlock* pBlock, SSHashObj* pUpdatedMap) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + int32_t winCode = TSDB_CODE_SUCCESS; + + SColumnInfoData* pGroupCol = taosArrayGet(pBlock->pDataBlock, GROUPID_COLUMN_INDEX); + uint64_t* groupIds = (uint64_t*)pGroupCol->pData; + SColumnInfoData* pStartCol = taosArrayGet(pBlock->pDataBlock, START_TS_COLUMN_INDEX); + TSKEY* tsStarts = (TSKEY*)pStartCol->pData; + SColumnInfoData* pEndCol = taosArrayGet(pBlock->pDataBlock, END_TS_COLUMN_INDEX); + TSKEY* tsEnds = (TSKEY*)pEndCol->pData; + for (int32_t i = 0; i < pBlock->info.rows; i++) { + TSKEY ts = tsStarts[i]; + TSKEY endCalTs = tsEnds[i]; + uint64_t groupId = groupIds[i]; + SWinKey key = {.ts = ts, .groupId = groupId}; + while (1) { + SWinKey nextKey = {.groupId = groupId}; + code = pAggSup->stateStore.streamStateFillGetNext(pAggSup->pState, &key, &nextKey, NULL, NULL, &winCode); + QUERY_CHECK_CODE(code, lino, _end); + if (key.ts > endCalTs) { + break; + } + int32_t tmpRes = tSimpleHashRemove(pUpdatedMap, &key, sizeof(SWinKey)); + qTrace("%s delete stream interp result at line %d res: %s", __func__, __LINE__, tstrerror(tmpRes)); + + pAggSup->stateStore.streamStateDel(pAggSup->pState, &key); + if (winCode != TSDB_CODE_SUCCESS) { + break; + } + key = nextKey; + } + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +int32_t setAllResultKey(SStreamAggSupporter* pAggSup, TSKEY ts, SSHashObj* pUpdatedMap) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + int64_t groupId = 0; + SStreamStateCur* pCur = pAggSup->stateStore.streamStateGroupGetCur(pAggSup->pState); + while (1) { + int32_t winCode = pAggSup->stateStore.streamStateGroupGetKVByCur(pCur, &groupId, NULL, NULL); + if (winCode != TSDB_CODE_SUCCESS) { + break; + } + SWinKey key = {.ts = ts, .groupId = groupId}; + code = saveTimeSliceWinResult(&key, pUpdatedMap); + QUERY_CHECK_CODE(code, lino, _end); + + pAggSup->stateStore.streamStateGroupCurNext(pCur); + } + pAggSup->stateStore.streamStateFreeCur(pCur); + pCur = NULL; + +_end: + if (code != TSDB_CODE_SUCCESS) { + pAggSup->stateStore.streamStateFreeCur(pCur); + pCur = NULL; + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +static void removeDuplicateTs(SArray* pTsArrray) { + __compar_fn_t fn = getKeyComparFunc(TSDB_DATA_TYPE_TIMESTAMP, TSDB_ORDER_ASC); + taosArraySort(pTsArrray, fn); + taosArrayRemoveDuplicate(pTsArrray, fn, NULL); +} + +static int32_t doStreamTimeSliceNext(SOperatorInfo* pOperator, SSDataBlock** ppRes) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + SStreamTimeSliceOperatorInfo* pInfo = pOperator->info; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; + + if (pOperator->status == OP_EXEC_DONE) { + (*ppRes) = NULL; + goto _end; + } + + if (pOperator->status == OP_RES_TO_RETURN) { + if (hasRemainCalc(pInfo->pFillInfo) || + (pInfo->pFillInfo->pos != FILL_POS_INVALID && pInfo->pFillInfo->needFill == true)) { + blockDataCleanup(pInfo->pRes); + doStreamFillRange(pInfo->pFillSup, pInfo->pFillInfo, pInfo->pRes); + if (pInfo->pRes->info.rows > 0) { + printDataBlock(pInfo->pRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); + (*ppRes) = pInfo->pRes; + goto _end; + } + } + + SSDataBlock* resBlock = NULL; + code = buildTimeSliceResult(pOperator, &resBlock); + QUERY_CHECK_CODE(code, lino, _end); + + if (resBlock != NULL) { + (*ppRes) = resBlock; + goto _end; + } + + if (pInfo->recvCkBlock) { + pInfo->recvCkBlock = false; + printDataBlock(pInfo->pCheckpointRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); + (*ppRes) = pInfo->pCheckpointRes; + goto _end; + } + + if (pInfo->twAggSup.calTrigger == STREAM_TRIGGER_FORCE_WINDOW_CLOSE) { + pAggSup->stateStore.streamStateClearExpiredState(pAggSup->pState); + } + setStreamOperatorCompleted(pOperator); + resetStreamFillSup(pInfo->pFillSup); + (*ppRes) = NULL; + goto _end; + } + + SSDataBlock* fillResult = NULL; + SOperatorInfo* downstream = pOperator->pDownstream[0]; + while (1) { + SSDataBlock* pBlock = getNextBlockFromDownstream(pOperator, 0); + if (pBlock == NULL) { + pOperator->status = OP_RES_TO_RETURN; + qDebug("===stream===return data:%s. recv datablock num:%" PRIu64, getStreamOpName(pOperator->operatorType), + pInfo->numOfDatapack); + pInfo->numOfDatapack = 0; + break; + } + pInfo->numOfDatapack++; + printSpecDataBlock(pBlock, getStreamOpName(pOperator->operatorType), "recv", GET_TASKID(pTaskInfo)); + setStreamOperatorState(&pInfo->basic, pBlock->info.type); + + switch (pBlock->info.type) { + case STREAM_DELETE_RESULT: + case STREAM_DELETE_DATA: { + code = doDeleteTimeSliceResult(pAggSup, pBlock, pInfo->pUpdatedMap); + QUERY_CHECK_CODE(code, lino, _end); + code = copyDataBlock(pInfo->pDelRes, pBlock); + QUERY_CHECK_CODE(code, lino, _end); + pInfo->pDelRes->info.type = STREAM_DELETE_RESULT; + (*ppRes) = pInfo->pDelRes; + printDataBlock((*ppRes), getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); + goto _end; + } break; + case STREAM_NORMAL: + case STREAM_INVALID: { + SExprSupp* pExprSup = &pInfo->scalarSup; + if (pExprSup->pExprInfo != NULL) { + code = projectApplyFunctions(pExprSup->pExprInfo, pBlock, pBlock, pExprSup->pCtx, pExprSup->numOfExprs, NULL); + QUERY_CHECK_CODE(code, lino, _end); + } + } break; + case STREAM_CHECKPOINT: { + pInfo->recvCkBlock = true; + pAggSup->stateStore.streamStateCommit(pAggSup->pState); + doStreamTimeSliceSaveCheckpoint(pOperator); + code = copyDataBlock(pInfo->pCheckpointRes, pBlock); + QUERY_CHECK_CODE(code, lino, _end); + continue; + } break; + case STREAM_CREATE_CHILD_TABLE: { + (*ppRes) = pBlock; + goto _end; + } break; + case STREAM_GET_RESULT: { + void* pPushRes = taosArrayPush(pInfo->pCloseTs, &pBlock->info.window.skey); + QUERY_CHECK_NULL(pPushRes, code, lino, _end, terrno); + continue; + } + default: + code = TSDB_CODE_QRY_EXECUTOR_INTERNAL_ERROR; + QUERY_CHECK_CODE(code, lino, _end); + } + + doStreamTimeSliceImpl(pOperator, pBlock); + pInfo->twAggSup.maxTs = TMAX(pInfo->twAggSup.maxTs, pBlock->info.window.ekey); + } + + if (pInfo->destHasPrimaryKey) { + code = copyIntervalDeleteKey(pInfo->pDeletedMap, pInfo->pDelWins); + QUERY_CHECK_CODE(code, lino, _end); + } + + if (taosArrayGetSize(pInfo->pCloseTs) > 0) { + removeDuplicateTs(pInfo->pCloseTs); + int32_t size = taosArrayGetSize(pInfo->pCloseTs); + qDebug("===stream===build stream result, ts count:%d", size); + for (int32_t i = 0; i < size; i++) { + TSKEY ts = *(TSKEY*) taosArrayGet(pInfo->pCloseTs, i); + code = buildAllResultKey(&pInfo->streamAggSup, ts, pInfo->pUpdated); + QUERY_CHECK_CODE(code, lino, _end); + } + qDebug("===stream===build stream result, res count:%ld", taosArrayGetSize(pInfo->pUpdated)); + taosArrayClear(pInfo->pCloseTs); + if (size > 1024) { + taosArrayDestroy(pInfo->pCloseTs); + pInfo->pCloseTs = taosArrayInit(1024, sizeof(TSKEY)); + } + } else { + void* pIte = NULL; + int32_t iter = 0; + while ((pIte = tSimpleHashIterate(pInfo->pUpdatedMap, pIte, &iter)) != NULL) { + SWinKey* pKey = (SWinKey*)tSimpleHashGetKey(pIte, NULL); + void* tmp = taosArrayPush(pInfo->pUpdated, pKey); + QUERY_CHECK_NULL(tmp, code, lino, _end, terrno); + } + } + taosArraySort(pInfo->pUpdated, winKeyCmprImpl); + + if (pInfo->isHistoryOp) { + code = getSliceMaxTsWins(pInfo->pUpdated, pInfo->historyWins); + QUERY_CHECK_CODE(code, lino, _end); + } + + initMultiResInfoFromArrayList(&pInfo->groupResInfo, pInfo->pUpdated); + pInfo->groupResInfo.freeItem = false; + + pInfo->pUpdated = taosArrayInit(16, sizeof(SWinKey)); + QUERY_CHECK_NULL(pInfo->pUpdated, code, lino, _end, terrno); + + code = blockDataEnsureCapacity(pInfo->pRes, pOperator->resultInfo.capacity); + QUERY_CHECK_CODE(code, lino, _end); + + tSimpleHashCleanup(pInfo->pUpdatedMap); + _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); + pInfo->pUpdatedMap = tSimpleHashInit(1024, hashFn); + + code = buildTimeSliceResult(pOperator, ppRes); + QUERY_CHECK_CODE(code, lino, _end); + + if (!(*ppRes)) { + if (pInfo->twAggSup.calTrigger == STREAM_TRIGGER_FORCE_WINDOW_CLOSE) { + pAggSup->stateStore.streamStateClearExpiredState(pAggSup->pState); + } + setStreamOperatorCompleted(pOperator); + resetStreamFillSup(pInfo->pFillSup); + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +static void copyFillValueInfo(SStreamFillSupporter* pFillSup, SStreamFillInfo* pFillInfo) { + if (pFillInfo->type == TSDB_FILL_SET_VALUE || pFillInfo->type == TSDB_FILL_SET_VALUE_F) { + int32_t valueIndex = 0; + for (int32_t i = 0; i < pFillSup->numOfAllCols; ++i) { + SFillColInfo* pFillCol = pFillSup->pAllColInfo + i; + if (!isInterpFunc(pFillCol->pExpr)) { + continue; + } + int32_t srcSlot = pFillCol->pExpr->base.pParam[0].pCol->slotId; + SResultCellData* pCell = getResultCell(pFillInfo->pResRow, srcSlot); + SFillColInfo* pValueCol = pFillSup->pAllColInfo + valueIndex; + SVariant* pVar = &(pValueCol->fillVal); + if (pCell->type == TSDB_DATA_TYPE_FLOAT) { + float v = 0; + GET_TYPED_DATA(v, float, pVar->nType, &pVar->i); + SET_TYPED_DATA(pCell->pData, pCell->type, v); + } else if (IS_FLOAT_TYPE(pCell->type)) { + double v = 0; + GET_TYPED_DATA(v, double, pVar->nType, &pVar->i); + SET_TYPED_DATA(pCell->pData, pCell->type, v); + } else if (IS_INTEGER_TYPE(pCell->type)) { + int64_t v = 0; + GET_TYPED_DATA(v, int64_t, pVar->nType, &pVar->i); + SET_TYPED_DATA(pCell->pData, pCell->type, v); + } else { + pCell->isNull = true; + } + valueIndex++; + } + } else if (pFillInfo->type == TSDB_FILL_NULL || pFillInfo->type == TSDB_FILL_NULL_F) { + for (int32_t i = 0; i < pFillSup->numOfAllCols; ++i) { + SFillColInfo* pFillCol = pFillSup->pAllColInfo + i; + int32_t slotId = GET_DEST_SLOT_ID(pFillCol); + SResultCellData* pCell = getResultCell(pFillInfo->pResRow, slotId); + pCell->isNull = true; + } + } +} + +int32_t getDownstreamRes(SOperatorInfo* downstream, SSDataBlock** ppRes, SColumnInfo** ppPkCol) { + if (downstream->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN) { + SStreamScanInfo* pInfo = (SStreamScanInfo*)downstream->info; + *ppRes = pInfo->pRes; + if (hasSrcPrimaryKeyCol(&pInfo->basic)) { + SColumnInfoData* pPkColInfo = taosArrayGet(pInfo->pRes->pDataBlock, pInfo->basic.primaryPkIndex); + (*ppPkCol) = &pPkColInfo->info; + } + return TSDB_CODE_SUCCESS; + } else if (downstream->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_PARTITION) { + SStreamPartitionOperatorInfo* pInfo = (SStreamPartitionOperatorInfo*)downstream->info; + *ppRes = pInfo->binfo.pRes; + if (hasSrcPrimaryKeyCol(&pInfo->basic)) { + SColumnInfoData* pPkColInfo = taosArrayGet(pInfo->binfo.pRes->pDataBlock, pInfo->basic.primaryPkIndex); + (*ppPkCol) = &pPkColInfo->info; + } + return TSDB_CODE_SUCCESS; + } + qError("%s failed at line %d since %s", __func__, __LINE__, tstrerror(TSDB_CODE_FAILED)); + return TSDB_CODE_FAILED; +} + +int32_t initTimeSliceDownStream(SOperatorInfo* downstream, SStreamAggSupporter* pAggSup, uint16_t type, + int32_t tsColIndex, STimeWindowAggSupp* pTwSup, struct SSteamOpBasicInfo* pBasic, + SStreamFillSupporter* pFillSup) { + SExecTaskInfo* pTaskInfo = downstream->pTaskInfo; + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + if (downstream->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_PARTITION) { + SStreamPartitionOperatorInfo* pPartionInfo = downstream->info; + pPartionInfo->tsColIndex = tsColIndex; + pBasic->primaryPkIndex = pPartionInfo->basic.primaryPkIndex; + } + + if (downstream->operatorType != QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN) { + code = initTimeSliceDownStream(downstream->pDownstream[0], pAggSup, type, tsColIndex, pTwSup, pBasic, pFillSup); + return code; + } + SStreamScanInfo* pScanInfo = downstream->info; + pScanInfo->igCheckUpdate = true; + pScanInfo->windowSup = (SWindowSupporter){.pStreamAggSup = pAggSup, .gap = pAggSup->gap, .parentType = type}; + pScanInfo->pState = pAggSup->pState; + if (!pScanInfo->pUpdateInfo) { + code = pAggSup->stateStore.updateInfoInit(60000, TSDB_TIME_PRECISION_MILLI, pTwSup->waterMark, + pScanInfo->igCheckUpdate, pScanInfo->pkColType, pScanInfo->pkColLen, + &pScanInfo->pUpdateInfo); + QUERY_CHECK_CODE(code, lino, _end); + } + pScanInfo->twAggSup = *pTwSup; + pScanInfo->pFillSup = pFillSup; + pScanInfo->interval = pFillSup->interval; + pAggSup->pUpdateInfo = pScanInfo->pUpdateInfo; + if (!hasSrcPrimaryKeyCol(pBasic)) { + pBasic->primaryPkIndex = pScanInfo->basic.primaryPkIndex; + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s. task:%s", __func__, lino, tstrerror(code), GET_TASKID(pTaskInfo)); + } + return code; +} + +int32_t createStreamTimeSliceOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo, + SReadHandle* pHandle, SOperatorInfo** ppOptInfo) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + SStreamTimeSliceOperatorInfo* pInfo = taosMemoryCalloc(1, sizeof(SStreamTimeSliceOperatorInfo)); + QUERY_CHECK_NULL(pInfo, code, lino, _error, terrno); + + SOperatorInfo* pOperator = taosMemoryCalloc(1, sizeof(SOperatorInfo)); + QUERY_CHECK_NULL(pOperator, code, lino, _error, terrno); + + SStreamInterpFuncPhysiNode* pInterpPhyNode = (SStreamInterpFuncPhysiNode*)pPhyNode; + pOperator->pTaskInfo = pTaskInfo; + initResultSizeInfo(&pOperator->resultInfo, 4096); + SExprSupp* pExpSup = &pOperator->exprSupp; + int32_t numOfExprs = 0; + SExprInfo* pExprInfo = NULL; + code = createExprInfo(pInterpPhyNode->pFuncs, NULL, &pExprInfo, &numOfExprs); + QUERY_CHECK_CODE(code, lino, _error); + + code = initExprSupp(pExpSup, pExprInfo, numOfExprs, &pTaskInfo->storageAPI.functionStore); + QUERY_CHECK_CODE(code, lino, _error); + + if (pInterpPhyNode->pExprs != NULL) { + int32_t num = 0; + SExprInfo* pScalarExprInfo = NULL; + code = createExprInfo(pInterpPhyNode->pExprs, NULL, &pScalarExprInfo, &num); + QUERY_CHECK_CODE(code, lino, _error); + + code = initExprSupp(&pInfo->scalarSup, pScalarExprInfo, num, &pTaskInfo->storageAPI.functionStore); + QUERY_CHECK_CODE(code, lino, _error); + } + + code = filterInitFromNode((SNode*)pInterpPhyNode->node.pConditions, &pOperator->exprSupp.pFilterInfo, 0); + QUERY_CHECK_CODE(code, lino, _error); + + pInfo->twAggSup = (STimeWindowAggSupp){ + .waterMark = pInterpPhyNode->streamNodeOption.watermark, + .calTrigger = pInterpPhyNode->streamNodeOption.triggerType, + .maxTs = INT64_MIN, + .minTs = INT64_MAX, + .deleteMark = getDeleteMarkFromOption(&pInterpPhyNode->streamNodeOption), + }; + + pInfo->primaryTsIndex = ((SColumnNode*)pInterpPhyNode->pTimeSeries)->slotId; + + SSDataBlock* pDownRes = NULL; + SColumnInfo* pPkCol = NULL; + code = getDownstreamRes(downstream, &pDownRes, &pPkCol); + QUERY_CHECK_CODE(code, lino, _error); + + pInfo->pFillSup = NULL; + code = initTimeSliceFillSup(pInterpPhyNode, pExpSup, numOfExprs, pPkCol, &pInfo->pFillSup); + QUERY_CHECK_CODE(code, lino, _error); + + int32_t ratio = 1; + if (pInfo->pFillSup->type == TSDB_FILL_LINEAR) { + ratio = 2; + } + + int32_t keyBytes = sizeof(TSKEY); + keyBytes += blockDataGetRowSize(pDownRes) + sizeof(SResultCellData) * taosArrayGetSize(pDownRes->pDataBlock); + if (pPkCol) { + keyBytes += pPkCol->bytes; + } + code = initStreamAggSupporter(&pInfo->streamAggSup, pExpSup, numOfExprs, 0, pTaskInfo->streamInfo.pState, keyBytes, 0, + &pTaskInfo->storageAPI.stateStore, pHandle, &pInfo->twAggSup, GET_TASKID(pTaskInfo), + &pTaskInfo->storageAPI, pInfo->primaryTsIndex, STREAM_STATE_BUFF_HASH_SORT, ratio); + QUERY_CHECK_CODE(code, lino, _error); + + code = initExecTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &pTaskInfo->window); + QUERY_CHECK_CODE(code, lino, _error); + + pInfo->pRes = createDataBlockFromDescNode(pPhyNode->pOutputDataBlockDesc); + pInfo->delIndex = 0; + pInfo->pDelWins = taosArrayInit(4, sizeof(SWinKey)); + QUERY_CHECK_NULL(pInfo->pDelWins, code, lino, _error, terrno); + + pInfo->pDelRes = NULL; + code = createSpecialDataBlock(STREAM_DELETE_RESULT, &pInfo->pDelRes); + QUERY_CHECK_CODE(code, lino, _error); + + _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); + pInfo->pDeletedMap = tSimpleHashInit(1024, hashFn); + QUERY_CHECK_NULL(pInfo->pDeletedMap, code, lino, _error, terrno); + + pInfo->ignoreExpiredData = pInterpPhyNode->streamNodeOption.igExpired; + pInfo->ignoreExpiredDataSaved = false; + pInfo->pUpdated = taosArrayInit(64, sizeof(SWinKey)); + pInfo->pUpdatedMap = tSimpleHashInit(1024, hashFn); + pInfo->historyPoints = taosArrayInit(4, sizeof(SWinKey)); + QUERY_CHECK_NULL(pInfo->historyPoints, code, lino, _error, terrno); + + pInfo->recvCkBlock = false; + pInfo->pCheckpointRes = NULL; + code = createSpecialDataBlock(STREAM_CHECKPOINT, &pInfo->pCheckpointRes); + QUERY_CHECK_CODE(code, lino, _error); + + pInfo->destHasPrimaryKey = pInterpPhyNode->streamNodeOption.destHasPrimaryKey; + pInfo->numOfDatapack = 0; + + pInfo->pFillInfo = initStreamFillInfo(pInfo->pFillSup, pDownRes); + copyFillValueInfo(pInfo->pFillSup, pInfo->pFillInfo); + pInfo->ignoreNull = getIgoreNullRes(pExpSup); + + pInfo->historyWins = taosArrayInit(4, sizeof(SWinKey)); + QUERY_CHECK_NULL(pInfo->historyWins, code, lino, _error, terrno); + + if (pHandle) { + pInfo->isHistoryOp = pHandle->fillHistory; + } + + pInfo->pCloseTs = taosArrayInit(1024, sizeof(TSKEY)); + QUERY_CHECK_NULL(pInfo->pCloseTs, code, lino, _error, terrno); + + pInfo->pOperator = pOperator; + + pOperator->operatorType = QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERP_FUNC; + setOperatorInfo(pOperator, getStreamOpName(pOperator->operatorType), QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERP_FUNC, + true, OP_NOT_OPENED, pInfo, pTaskInfo); + // for stream + void* buff = NULL; + int32_t len = 0; + int32_t res = pTaskInfo->storageAPI.stateStore.streamStateGetInfo( + pTaskInfo->streamInfo.pState, STREAM_TIME_SLICE_OP_CHECKPOINT_NAME, strlen(STREAM_TIME_SLICE_OP_CHECKPOINT_NAME), + &buff, &len); + if (res == TSDB_CODE_SUCCESS) { + code = doStreamTimeSliceDecodeOpState(buff, len, pOperator); + taosMemoryFree(buff); + QUERY_CHECK_CODE(code, lino, _error); + } + pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, doStreamTimeSliceNext, NULL, destroyStreamTimeSliceOperatorInfo, + optrDefaultBufFn, NULL, optrDefaultGetNextExtFn, NULL); + setOperatorStreamStateFn(pOperator, streamTimeSliceReleaseState, streamTimeSliceReloadState); + + initStreamBasicInfo(&pInfo->basic); + if (downstream) { + code = initTimeSliceDownStream(downstream, &pInfo->streamAggSup, pOperator->operatorType, pInfo->primaryTsIndex, + &pInfo->twAggSup, &pInfo->basic, pInfo->pFillSup); + QUERY_CHECK_CODE(code, lino, _error); + + code = appendDownstream(pOperator, &downstream, 1); + QUERY_CHECK_CODE(code, lino, _error); + } + (*ppOptInfo) = pOperator; + return code; + +_error: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + if (pInfo != NULL) { + destroyStreamTimeSliceOperatorInfo(pInfo); + } + destroyOperatorAndDownstreams(pOperator, &downstream, 1); + pTaskInfo->code = code; + (*ppOptInfo) = NULL; + return code; +} diff --git a/source/libs/executor/src/streamtimewindowoperator.c b/source/libs/executor/src/streamtimewindowoperator.c index a7f8934c03..8fd00e9313 100644 --- a/source/libs/executor/src/streamtimewindowoperator.c +++ b/source/libs/executor/src/streamtimewindowoperator.c @@ -30,9 +30,6 @@ #define IS_FINAL_INTERVAL_OP(op) ((op)->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL) #define IS_MID_INTERVAL_OP(op) ((op)->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_MID_INTERVAL) -#define IS_NORMAL_INTERVAL_OP(op) \ - ((op)->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL || \ - (op)->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL) #define IS_FINAL_SESSION_OP(op) ((op)->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_SESSION) #define IS_NORMAL_SESSION_OP(op) \ @@ -171,7 +168,7 @@ int32_t saveResult(SResultWindowInfo winInfo, SSHashObj* pStUpdated) { return tSimpleHashPut(pStUpdated, &winInfo.sessionWin, sizeof(SSessionKey), &winInfo, sizeof(SResultWindowInfo)); } -static int32_t saveWinResult(SWinKey* pKey, SRowBuffPos* pPos, SSHashObj* pUpdatedMap) { +int32_t saveWinResult(SWinKey* pKey, SRowBuffPos* pPos, SSHashObj* pUpdatedMap) { if (tSimpleHashGetSize(pUpdatedMap) > MAX_STREAM_HISTORY_RESULT) { qError("%s failed at line %d since too many history result. ", __func__, __LINE__); return TSDB_CODE_STREAM_INTERNAL_ERROR; @@ -203,7 +200,7 @@ static int32_t compareWinKey(void* pKey, void* data, int32_t index) { return winKeyCmprImpl(pKey, pDataPos); } -static void removeDeleteResults(SSHashObj* pUpdatedMap, SArray* pDelWins) { +void removeDeleteResults(SSHashObj* pUpdatedMap, SArray* pDelWins) { taosArraySort(pDelWins, winKeyCmprImpl); taosArrayRemoveDuplicate(pDelWins, winKeyCmprImpl, NULL); int32_t delSize = taosArrayGetSize(pDelWins); @@ -400,6 +397,11 @@ STimeWindow getFinalTimeWindow(int64_t ts, SInterval* pInterval) { static void doBuildDeleteResult(SStreamIntervalOperatorInfo* pInfo, SArray* pWins, int32_t* index, SSDataBlock* pBlock) { + doBuildDeleteResultImpl(&pInfo->stateStore, pInfo->pState, pWins, index, pBlock); +} + +void doBuildDeleteResultImpl(SStateStore* pAPI, SStreamState* pState, SArray* pWins, int32_t* index, + SSDataBlock* pBlock) { int32_t code = TSDB_CODE_SUCCESS; int32_t lino = 0; blockDataCleanup(pBlock); @@ -417,7 +419,7 @@ static void doBuildDeleteResult(SStreamIntervalOperatorInfo* pInfo, SArray* pWin SWinKey* pWin = taosArrayGet(pWins, i); void* tbname = NULL; int32_t winCode = TSDB_CODE_SUCCESS; - code = pInfo->stateStore.streamStateGetParName(pInfo->pState, pWin->groupId, &tbname, false, &winCode); + code = pAPI->streamStateGetParName(pState, pWin->groupId, &tbname, false, &winCode); QUERY_CHECK_CODE(code, lino, _end); if (winCode != TSDB_CODE_SUCCESS) { @@ -430,7 +432,7 @@ static void doBuildDeleteResult(SStreamIntervalOperatorInfo* pInfo, SArray* pWin code = appendDataToSpecialBlock(pBlock, &pWin->ts, &pWin->ts, &uid, &pWin->groupId, parTbName); QUERY_CHECK_CODE(code, lino, _end); } - pInfo->stateStore.streamStateFreeVal(tbname); + pAPI->streamStateFreeVal(tbname); (*index)++; } @@ -1205,7 +1207,7 @@ _end: return code; } -static inline int winPosCmprImpl(const void* pKey1, const void* pKey2) { +int winPosCmprImpl(const void* pKey1, const void* pKey2) { SRowBuffPos* pos1 = *(SRowBuffPos**)pKey1; SRowBuffPos* pos2 = *(SRowBuffPos**)pKey2; SWinKey* pWin1 = (SWinKey*)pos1->pKey; @@ -1476,7 +1478,7 @@ void doStreamIntervalSaveCheckpoint(SOperatorInfo* pOperator) { } } -static int32_t copyIntervalDeleteKey(SSHashObj* pMap, SArray* pWins) { +int32_t copyIntervalDeleteKey(SSHashObj* pMap, SArray* pWins) { int32_t code = TSDB_CODE_SUCCESS; int32_t lino = 0; void* pIte = NULL; @@ -1824,6 +1826,14 @@ int64_t getDeleteMark(SWindowPhysiNode* pWinPhyNode, int64_t interval) { return deleteMark; } +int64_t getDeleteMarkFromOption(SStreamNodeOption* pOption) { + if (pOption->deleteMark <= 0) { + return DEAULT_DELETE_MARK; + } + int64_t deleteMark = TMAX(pOption->deleteMark, pOption->watermark); + return deleteMark; +} + static TSKEY compareTs(void* pKey) { SWinKey* pWinKey = (SWinKey*)pKey; return pWinKey->ts; @@ -2035,7 +2045,7 @@ int32_t createStreamFinalIntervalOperatorInfo(SOperatorInfo* downstream, SPhysiN QUERY_CHECK_NULL(pInfo->pMidPullDatas, code, lino, _error, terrno); pInfo->pDeletedMap = tSimpleHashInit(4096, hashFn); QUERY_CHECK_NULL(pInfo->pDeletedMap, code, lino, _error, terrno); - pInfo->destHasPrimaryKey = pIntervalPhyNode->window.destHasPrimayKey; + pInfo->destHasPrimaryKey = pIntervalPhyNode->window.destHasPrimaryKey; pInfo->pOperator = pOperator; pOperator->operatorType = pPhyNode->type; @@ -2206,8 +2216,8 @@ static TSKEY sesionTs(void* pKey) { int32_t initStreamAggSupporter(SStreamAggSupporter* pSup, SExprSupp* pExpSup, int32_t numOfOutput, int64_t gap, SStreamState* pState, int32_t keySize, int16_t keyType, SStateStore* pStore, SReadHandle* pHandle, STimeWindowAggSupp* pTwAggSup, const char* taskIdStr, - SStorageAPI* pApi, int32_t tsIndex) { - pSup->resultRowSize = keySize + getResultRowSize(pExpSup->pCtx, numOfOutput); + SStorageAPI* pApi, int32_t tsIndex, int8_t stateType, int32_t ratio) { + pSup->resultRowSize = (keySize + getResultRowSize(pExpSup->pCtx, numOfOutput)) * ratio; int32_t lino = 0; int32_t code = createSpecialDataBlock(STREAM_CLEAR, &pSup->pScanBlock); QUERY_CHECK_CODE(code, lino, _end); @@ -2228,16 +2238,24 @@ int32_t initStreamAggSupporter(SStreamAggSupporter* pSup, SExprSupp* pExpSup, in *(pSup->pState) = *pState; pSup->stateStore.streamStateSetNumber(pSup->pState, -1, tsIndex); int32_t funResSize = getMaxFunResSize(pExpSup, numOfOutput); + if (stateType != STREAM_STATE_BUFF_HASH_SORT) { + // used for backward compatibility of function's result info + pSup->pState->pResultRowStore.resultRowGet = getResultRowFromBuf; + pSup->pState->pResultRowStore.resultRowPut = putResultRowToBuf; + pSup->pState->pExprSupp = pExpSup; + } + + if (stateType == STREAM_STATE_BUFF_SORT) { pSup->pState->pFileState = NULL; - - // used for backward compatibility of function's result info - pSup->pState->pResultRowStore.resultRowGet = getResultRowFromBuf; - pSup->pState->pResultRowStore.resultRowPut = putResultRowToBuf; - pSup->pState->pExprSupp = pExpSup; - - code = pSup->stateStore.streamFileStateInit(tsStreamBufferSize, sizeof(SSessionKey), pSup->resultRowSize, funResSize, - sesionTs, pSup->pState, pTwAggSup->deleteMark, taskIdStr, - pHandle->checkpointId, STREAM_STATE_BUFF_SORT, &pSup->pState->pFileState); + code = pSup->stateStore.streamFileStateInit(tsStreamBufferSize, sizeof(SSessionKey), pSup->resultRowSize, + funResSize, sesionTs, pSup->pState, pTwAggSup->deleteMark, taskIdStr, + pHandle->checkpointId, stateType, &pSup->pState->pFileState); + } else if (stateType == STREAM_STATE_BUFF_HASH_SORT || stateType == STREAM_STATE_BUFF_HASH_SEARCH) { + pSup->pState->pFileState = NULL; + code = pSup->stateStore.streamFileStateInit(tsStreamBufferSize, sizeof(SWinKey), pSup->resultRowSize, funResSize, + compareTs, pSup->pState, pTwAggSup->deleteMark, taskIdStr, + pHandle->checkpointId, stateType, &pSup->pState->pFileState); + } QUERY_CHECK_CODE(code, lino, _end); _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); @@ -3837,9 +3855,10 @@ int32_t createStreamSessionAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode }; pInfo->primaryTsIndex = ((SColumnNode*)pSessionNode->window.pTspk)->slotId; - code = initStreamAggSupporter(&pInfo->streamAggSup, pExpSup, numOfCols, pSessionNode->gap, - pTaskInfo->streamInfo.pState, 0, 0, &pTaskInfo->storageAPI.stateStore, pHandle, - &pInfo->twAggSup, GET_TASKID(pTaskInfo), &pTaskInfo->storageAPI, pInfo->primaryTsIndex); + code = + initStreamAggSupporter(&pInfo->streamAggSup, pExpSup, numOfCols, pSessionNode->gap, pTaskInfo->streamInfo.pState, + 0, 0, &pTaskInfo->storageAPI.stateStore, pHandle, &pInfo->twAggSup, GET_TASKID(pTaskInfo), + &pTaskInfo->storageAPI, pInfo->primaryTsIndex, STREAM_STATE_BUFF_SORT, 1); if (code != TSDB_CODE_SUCCESS) { goto _error; } @@ -3879,7 +3898,7 @@ int32_t createStreamSessionAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode pInfo->clearState = false; pInfo->recvGetAll = false; - pInfo->destHasPrimaryKey = pSessionNode->window.destHasPrimayKey; + pInfo->destHasPrimaryKey = pSessionNode->window.destHasPrimaryKey; pInfo->pPkDeleted = tSimpleHashInit(64, hashFn); QUERY_CHECK_NULL(pInfo->pPkDeleted, code, lino, _error, terrno); pInfo->pOperator = pOperator; @@ -5030,9 +5049,10 @@ int32_t createStreamStateAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode* int32_t keySize = sizeof(SStateKeys) + pColNode->node.resType.bytes; int16_t type = pColNode->node.resType.type; pInfo->primaryTsIndex = tsSlotId; - code = initStreamAggSupporter(&pInfo->streamAggSup, pExpSup, numOfCols, 0, pTaskInfo->streamInfo.pState, keySize, - type, &pTaskInfo->storageAPI.stateStore, pHandle, &pInfo->twAggSup, - GET_TASKID(pTaskInfo), &pTaskInfo->storageAPI, pInfo->primaryTsIndex); + code = + initStreamAggSupporter(&pInfo->streamAggSup, pExpSup, numOfCols, 0, pTaskInfo->streamInfo.pState, keySize, type, + &pTaskInfo->storageAPI.stateStore, pHandle, &pInfo->twAggSup, GET_TASKID(pTaskInfo), + &pTaskInfo->storageAPI, pInfo->primaryTsIndex, STREAM_STATE_BUFF_SORT, 1); QUERY_CHECK_CODE(code, lino, _error); _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); @@ -5065,7 +5085,7 @@ int32_t createStreamStateAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pInfo->recvGetAll = false; pInfo->pPkDeleted = tSimpleHashInit(64, hashFn); QUERY_CHECK_NULL(pInfo->pPkDeleted, code, lino, _error, terrno); - pInfo->destHasPrimaryKey = pStateNode->window.destHasPrimayKey; + pInfo->destHasPrimaryKey = pStateNode->window.destHasPrimaryKey; pInfo->pOperator = pOperator; setOperatorInfo(pOperator, "StreamStateAggOperator", QUERY_NODE_PHYSICAL_PLAN_STREAM_STATE, true, OP_NOT_OPENED, @@ -5294,8 +5314,8 @@ _end: return code; } -int32_t createStreamIntervalOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo, - SReadHandle* pHandle, SOperatorInfo** pOptrInfo) { +static int32_t createStreamSingleIntervalOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo, + SReadHandle* pHandle, SOperatorInfo** pOptrInfo) { QRY_PARAM_CHECK(pOptrInfo); int32_t code = TSDB_CODE_SUCCESS; @@ -5424,7 +5444,7 @@ int32_t createStreamIntervalOperatorInfo(SOperatorInfo* downstream, SPhysiNode* _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); pInfo->pDeletedMap = tSimpleHashInit(4096, hashFn); QUERY_CHECK_NULL(pInfo->pDeletedMap, code, lino, _error, terrno); - pInfo->destHasPrimaryKey = pIntervalPhyNode->window.destHasPrimayKey; + pInfo->destHasPrimaryKey = pIntervalPhyNode->window.destHasPrimaryKey; // for stream void* buff = NULL; @@ -5453,6 +5473,17 @@ _error: return code; } +int32_t createStreamIntervalOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo, + SReadHandle* pHandle, SOperatorInfo** pOptrInfo) { + SStreamIntervalPhysiNode* pIntervalPhyNode = (SStreamIntervalPhysiNode*)pPhyNode; + if (pIntervalPhyNode->window.triggerType == STREAM_TRIGGER_FORCE_WINDOW_CLOSE) { + return createStreamIntervalSliceOperatorInfo(downstream, pPhyNode, pTaskInfo, pHandle, pOptrInfo); + } else { + return createStreamSingleIntervalOperatorInfo(downstream, pPhyNode, pTaskInfo, pHandle, pOptrInfo); + } + return TSDB_CODE_SUCCESS; +} + static void doStreamMidIntervalAggImpl(SOperatorInfo* pOperator, SSDataBlock* pSDataBlock, SSHashObj* pUpdatedMap) { int32_t code = TSDB_CODE_SUCCESS; int32_t lino = 0; @@ -5844,7 +5875,7 @@ static SSDataBlock* doStreamMidIntervalAgg(SOperatorInfo* pOperator) { } void setStreamOperatorCompleted(SOperatorInfo* pOperator) { - setOperatorCompleted(pOperator); qDebug("stask:%s %s status: %d. set completed", GET_TASKID(pOperator->pTaskInfo), getStreamOpName(pOperator->operatorType), pOperator->status); + setOperatorCompleted(pOperator); } diff --git a/source/libs/executor/src/timesliceoperator.c b/source/libs/executor/src/timesliceoperator.c index 88c1642654..f77aa8f34a 100644 --- a/source/libs/executor/src/timesliceoperator.c +++ b/source/libs/executor/src/timesliceoperator.c @@ -169,12 +169,12 @@ static FORCE_INLINE int32_t timeSliceEnsureBlockCapacity(STimeSliceOperatorInfo* return TSDB_CODE_SUCCESS; } -static bool isIrowtsPseudoColumn(SExprInfo* pExprInfo) { +bool isIrowtsPseudoColumn(SExprInfo* pExprInfo) { char* name = pExprInfo->pExpr->_function.functionName; return (IS_TIMESTAMP_TYPE(pExprInfo->base.resSchema.type) && strcasecmp(name, "_irowts") == 0); } -static bool isIsfilledPseudoColumn(SExprInfo* pExprInfo) { +bool isIsfilledPseudoColumn(SExprInfo* pExprInfo) { char* name = pExprInfo->pExpr->_function.functionName; return (IS_BOOLEAN_TYPE(pExprInfo->base.resSchema.type) && strcasecmp(name, "_isfilled") == 0); } @@ -224,7 +224,7 @@ static bool checkDuplicateTimestamps(STimeSliceOperatorInfo* pSliceInfo, SColumn return false; } -static bool isInterpFunc(SExprInfo* pExprInfo) { +bool isInterpFunc(SExprInfo* pExprInfo) { int32_t functionType = pExprInfo->pExpr->_function.functionType; return (functionType == FUNCTION_TYPE_INTERP); } @@ -239,7 +239,7 @@ static bool isSelectGroupConstValueFunc(SExprInfo* pExprInfo) { return (functionType == FUNCTION_TYPE_GROUP_CONST_VALUE); } -static bool getIgoreNullRes(SExprSupp* pExprSup) { +bool getIgoreNullRes(SExprSupp* pExprSup) { for (int32_t i = 0; i < pExprSup->numOfExprs; ++i) { SExprInfo* pExprInfo = &pExprSup->pExprInfo[i]; @@ -256,7 +256,7 @@ static bool getIgoreNullRes(SExprSupp* pExprSup) { return false; } -static bool checkNullRow(SExprSupp* pExprSup, SSDataBlock* pSrcBlock, int32_t index, bool ignoreNull) { +bool checkNullRow(SExprSupp* pExprSup, SSDataBlock* pSrcBlock, int32_t index, bool ignoreNull) { if (!ignoreNull) { return false; } diff --git a/source/libs/executor/test/queryPlanTests.cpp b/source/libs/executor/test/queryPlanTests.cpp index 2b0c0e1118..6710435aba 100755 --- a/source/libs/executor/test/queryPlanTests.cpp +++ b/source/libs/executor/test/queryPlanTests.cpp @@ -2384,7 +2384,7 @@ void qptCreateWindowPhysiNode(SWindowPhysiNode* pWindow) { pWindow->watermark = taosRand(); pWindow->deleteMark = taosRand(); pWindow->igExpired = taosRand(); - pWindow->destHasPrimayKey = taosRand(); + pWindow->destHasPrimaryKey = taosRand(); pWindow->mergeDataBlock = QPT_RAND_BOOL_V; } diff --git a/source/libs/function/inc/functionResInfoInt.h b/source/libs/function/inc/functionResInfoInt.h index 9ee1e884b3..f97d2e8024 100644 --- a/source/libs/function/inc/functionResInfoInt.h +++ b/source/libs/function/inc/functionResInfoInt.h @@ -237,6 +237,7 @@ typedef struct SElapsedInfo { } SElapsedInfo; typedef struct STwaInfo { + double dTwaRes; double dOutput; int64_t numOfElems; SPoint1 p; diff --git a/source/libs/function/src/builtins.c b/source/libs/function/src/builtins.c index 58f20cc398..552933dcad 100644 --- a/source/libs/function/src/builtins.c +++ b/source/libs/function/src/builtins.c @@ -2360,7 +2360,7 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { { .name = "elapsed", .type = FUNCTION_TYPE_ELAPSED, - .classification = FUNC_MGT_AGG_FUNC | FUNC_MGT_TIMELINE_FUNC | FUNC_MGT_INTERVAL_INTERPO_FUNC | FUNC_MGT_FORBID_STREAM_FUNC | + .classification = FUNC_MGT_AGG_FUNC | FUNC_MGT_TIMELINE_FUNC | FUNC_MGT_INTERVAL_INTERPO_FUNC | FUNC_MGT_FORBID_SYSTABLE_FUNC | FUNC_MGT_SPECIAL_DATA_REQUIRED, .parameters = {.minParamNum = 1, .maxParamNum = 2, @@ -2425,7 +2425,7 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .name = "interp", .type = FUNCTION_TYPE_INTERP, .classification = FUNC_MGT_TIMELINE_FUNC | FUNC_MGT_INTERVAL_INTERPO_FUNC | FUNC_MGT_IMPLICIT_TS_FUNC | - FUNC_MGT_FORBID_STREAM_FUNC | FUNC_MGT_FORBID_SYSTABLE_FUNC | FUNC_MGT_KEEP_ORDER_FUNC | FUNC_MGT_PRIMARY_KEY_FUNC, + FUNC_MGT_FORBID_SYSTABLE_FUNC | FUNC_MGT_KEEP_ORDER_FUNC | FUNC_MGT_PRIMARY_KEY_FUNC, .parameters = {.minParamNum = 1, .maxParamNum = 2, .paramInfoPattern = 1, @@ -2857,7 +2857,7 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { { .name = "twa", .type = FUNCTION_TYPE_TWA, - .classification = FUNC_MGT_AGG_FUNC | FUNC_MGT_TIMELINE_FUNC | FUNC_MGT_INTERVAL_INTERPO_FUNC | FUNC_MGT_FORBID_STREAM_FUNC | + .classification = FUNC_MGT_AGG_FUNC | FUNC_MGT_TIMELINE_FUNC | FUNC_MGT_INTERVAL_INTERPO_FUNC | FUNC_MGT_IMPLICIT_TS_FUNC | FUNC_MGT_FORBID_SYSTABLE_FUNC | FUNC_MGT_PRIMARY_KEY_FUNC, .parameters = {.minParamNum = 1, .maxParamNum = 1, diff --git a/source/libs/function/src/builtinsimpl.c b/source/libs/function/src/builtinsimpl.c index 983fccac1e..0aad1501ce 100644 --- a/source/libs/function/src/builtinsimpl.c +++ b/source/libs/function/src/builtinsimpl.c @@ -4266,6 +4266,10 @@ int32_t elapsedFunction(SqlFunctionCtx* pCtx) { numOfElems = pInput->numOfRows; // since this is the primary timestamp, no need to exclude NULL values if (numOfElems == 0) { + // for stream + if (pCtx->end.key != INT64_MIN) { + pInfo->max = pCtx->end.key + 1; + } goto _elapsed_over; } @@ -6207,11 +6211,11 @@ int32_t twaFinalize(struct SqlFunctionCtx* pCtx, SSDataBlock* pBlock) { pResInfo->numOfRes = 0; } else { if (pInfo->win.ekey == pInfo->win.skey) { - pInfo->dOutput = pInfo->p.val; + pInfo->dTwaRes = pInfo->p.val; } else if (pInfo->win.ekey == INT64_MAX || pInfo->win.skey == INT64_MIN) { // no data in timewindow - pInfo->dOutput = 0; + pInfo->dTwaRes = 0; } else { - pInfo->dOutput = pInfo->dOutput / (pInfo->win.ekey - pInfo->win.skey); + pInfo->dTwaRes = pInfo->dOutput / (pInfo->win.ekey - pInfo->win.skey); } pResInfo->numOfRes = 1; diff --git a/source/libs/nodes/src/nodesCloneFuncs.c b/source/libs/nodes/src/nodesCloneFuncs.c index 866b105ca1..1a5785190b 100644 --- a/source/libs/nodes/src/nodesCloneFuncs.c +++ b/source/libs/nodes/src/nodesCloneFuncs.c @@ -678,9 +678,12 @@ static int32_t logicInterpFuncCopy(const SInterpFuncLogicNode* pSrc, SInterpFunc CLONE_NODE_LIST_FIELD(pFuncs); COPY_OBJECT_FIELD(timeRange, sizeof(STimeWindow)); COPY_SCALAR_FIELD(interval); + COPY_SCALAR_FIELD(intervalUnit); + COPY_SCALAR_FIELD(precision); COPY_SCALAR_FIELD(fillMode); CLONE_NODE_FIELD(pFillValues); CLONE_NODE_FIELD(pTimeSeries); + COPY_OBJECT_FIELD(streamNodeOption, sizeof(SStreamNodeOption)); return TSDB_CODE_SUCCESS; } @@ -789,7 +792,7 @@ static int32_t physiWindowCopy(const SWindowPhysiNode* pSrc, SWindowPhysiNode* p COPY_SCALAR_FIELD(triggerType); COPY_SCALAR_FIELD(watermark); COPY_SCALAR_FIELD(igExpired); - COPY_SCALAR_FIELD(destHasPrimayKey); + COPY_SCALAR_FIELD(destHasPrimaryKey); return TSDB_CODE_SUCCESS; } diff --git a/source/libs/nodes/src/nodesCodeFuncs.c b/source/libs/nodes/src/nodesCodeFuncs.c index 723a38c2bd..3275cfd838 100644 --- a/source/libs/nodes/src/nodesCodeFuncs.c +++ b/source/libs/nodes/src/nodesCodeFuncs.c @@ -431,6 +431,8 @@ const char* nodesNodeName(ENodeType type) { return "PhysiIndefRowsFunc"; case QUERY_NODE_PHYSICAL_PLAN_INTERP_FUNC: return "PhysiInterpFunc"; + case QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERP_FUNC: + return "PhysiStreamInterpFunc"; case QUERY_NODE_PHYSICAL_PLAN_FORECAST_FUNC: return "PhysiForecastFunc"; case QUERY_NODE_PHYSICAL_PLAN_DISPATCH: @@ -1235,10 +1237,67 @@ static int32_t jsonToLogicIndefRowsFuncNode(const SJson* pJson, void* pObj) { return code; } +static const char* jkStreamOption_triggerType = "StreamOptionTriggerType"; +static const char* jkStreamOption_watermark = "StreamOptionWatermark"; +static const char* jkStreamOption_deleteMark = "StreamOptionDeleteMark"; +static const char* jkStreamOption_igExpired = "StreamOptionIgExpired"; +static const char* jkStreamOption_igCheckUpdate = "StreamOption_igCheckUpdate"; +static const char* jkStreamOption_destHasPrimaryKey = "StreamOptionDestHasPrimaryKey"; + +static int32_t streamNodeOptionToJson(const void* pObj, SJson* pJson) { + const SStreamNodeOption* pNode = (const SStreamNodeOption*)pObj; + int32_t code = tjsonAddIntegerToObject(pJson, jkStreamOption_triggerType, pNode->triggerType); + + if (TSDB_CODE_SUCCESS == code) { + code = tjsonAddIntegerToObject(pJson, jkStreamOption_watermark, pNode->watermark); + } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonAddIntegerToObject(pJson, jkStreamOption_deleteMark, pNode->deleteMark); + } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonAddIntegerToObject(pJson, jkStreamOption_igExpired, pNode->igExpired); + } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonAddIntegerToObject(pJson, jkStreamOption_igCheckUpdate, pNode->igCheckUpdate); + } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonAddIntegerToObject(pJson, jkStreamOption_destHasPrimaryKey, pNode->destHasPrimaryKey); + } + return code; +} + +static int32_t jsonToStreamNodeOption(const SJson* pJson, void* pObj) { + SStreamNodeOption* pNode = (SStreamNodeOption*)pObj; + int32_t code = tjsonGetTinyIntValue(pJson, jkStreamOption_triggerType, &pNode->triggerType); + + if (TSDB_CODE_SUCCESS == code) { + code = tjsonGetBigIntValue(pJson, jkStreamOption_watermark, &pNode->watermark); + } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonGetBigIntValue(pJson, jkStreamOption_deleteMark, &pNode->deleteMark); + } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonGetTinyIntValue(pJson, jkStreamOption_igExpired, &pNode->igExpired); + } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonGetTinyIntValue(pJson, jkStreamOption_igCheckUpdate, &pNode->igCheckUpdate); + } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonGetTinyIntValue(pJson, jkStreamOption_destHasPrimaryKey, &pNode->destHasPrimaryKey); + } + return code; +} + static const char* jkInterpFuncLogicPlanFuncs = "Funcs"; static const char* jkInterpFuncLogicPlanStartTime = "StartTime"; static const char* jkInterpFuncLogicPlanEndTime = "EndTime"; static const char* jkInterpFuncLogicPlanInterval = "Interval"; +static const char* jkInterpFuncLogicPlanIntervalUnit = "IntervalUnit"; +static const char* jkInterpFuncLogicPlanPrecision = "Precision"; +static const char* jkInterpFuncLogicPlanFillMode = "fillMode"; +static const char* jkInterpFuncLogicPlanFillValues = "FillValues"; +static const char* jkInterpFuncLogicPlanTimeSeries = "TimeSeries"; +static const char* jkInterpFuncLogicPlanStreamNodeOption = "StreamNodeOption"; static int32_t logicInterpFuncNodeToJson(const void* pObj, SJson* pJson) { const SInterpFuncLogicNode* pNode = (const SInterpFuncLogicNode*)pObj; @@ -1256,6 +1315,24 @@ static int32_t logicInterpFuncNodeToJson(const void* pObj, SJson* pJson) { if (TSDB_CODE_SUCCESS == code) { code = tjsonAddIntegerToObject(pJson, jkInterpFuncLogicPlanInterval, pNode->interval); } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonAddIntegerToObject(pJson, jkInterpFuncLogicPlanIntervalUnit, pNode->intervalUnit); + } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonAddIntegerToObject(pJson, jkInterpFuncLogicPlanPrecision, pNode->precision); + } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonAddIntegerToObject(pJson, jkInterpFuncLogicPlanFillMode, pNode->fillMode); + } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonAddObject(pJson, jkInterpFuncLogicPlanFillValues, nodeToJson, pNode->pFillValues); + } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonAddObject(pJson, jkInterpFuncLogicPlanTimeSeries, nodeToJson, pNode->pTimeSeries); + } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonAddObject(pJson, jkInterpFuncLogicPlanStreamNodeOption, streamNodeOptionToJson, &pNode->streamNodeOption); + } return code; } @@ -1276,6 +1353,24 @@ static int32_t jsonToLogicInterpFuncNode(const SJson* pJson, void* pObj) { if (TSDB_CODE_SUCCESS == code) { code = tjsonGetBigIntValue(pJson, jkInterpFuncLogicPlanInterval, &pNode->interval); } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonGetTinyIntValue(pJson, jkInterpFuncLogicPlanIntervalUnit, &pNode->intervalUnit); + } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonGetTinyIntValue(pJson, jkInterpFuncLogicPlanPrecision, &pNode->precision); + } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonGetTinyIntValue(pJson, jkInterpFuncLogicPlanFillMode, (int8_t*)&pNode->fillMode); + } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonToObject(pJson, jkInterpFuncLogicPlanFillValues, jsonToNode, pNode->pFillValues); + } + if (TSDB_CODE_SUCCESS == code) { + code = jsonToNodeObject(pJson, jkInterpFuncLogicPlanTimeSeries, &pNode->pTimeSeries); + } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonToObject(pJson, jkInterpFuncLogicPlanStreamNodeOption, jsonToStreamNodeOption, &pNode->streamNodeOption); + } return code; } @@ -2784,7 +2879,7 @@ static int32_t physiWindowNodeToJson(const void* pObj, SJson* pJson) { code = tjsonAddBoolToObject(pJson, jkWindowPhysiPlanMergeDataBlock, pNode->mergeDataBlock); } if (TSDB_CODE_SUCCESS == code) { - code = tjsonAddIntegerToObject(pJson, jkWindowPhysiPlanDestHasPrimaryKey, pNode->destHasPrimayKey); + code = tjsonAddIntegerToObject(pJson, jkWindowPhysiPlanDestHasPrimaryKey, pNode->destHasPrimaryKey); } return code; @@ -2822,7 +2917,7 @@ static int32_t jsonToPhysiWindowNode(const SJson* pJson, void* pObj) { code = tjsonGetBoolValue(pJson, jkWindowPhysiPlanMergeDataBlock, &pNode->mergeDataBlock); } if (TSDB_CODE_SUCCESS == code) { - code = tjsonGetTinyIntValue(pJson, jkWindowPhysiPlanDestHasPrimaryKey, &pNode->destHasPrimayKey); + code = tjsonGetTinyIntValue(pJson, jkWindowPhysiPlanDestHasPrimaryKey, &pNode->destHasPrimaryKey); } return code; @@ -3211,9 +3306,12 @@ static const char* jkInterpFuncPhysiPlanFuncs = "Funcs"; static const char* jkInterpFuncPhysiPlanStartTime = "StartTime"; static const char* jkInterpFuncPhysiPlanEndTime = "EndTime"; static const char* jkInterpFuncPhysiPlanInterval = "Interval"; +static const char* jkInterpFuncPhysiPlanIntervalUnit = "intervalUnit"; +static const char* jkInterpFuncPhysiPlanPrecision = "precision"; static const char* jkInterpFuncPhysiPlanFillMode = "FillMode"; static const char* jkInterpFuncPhysiPlanFillValues = "FillValues"; static const char* jkInterpFuncPhysiPlanTimeSeries = "TimeSeries"; +static const char* jkInterpFuncPhysiPlanStreamNodeOption = "StreamNodeOption"; static int32_t physiInterpFuncNodeToJson(const void* pObj, SJson* pJson) { const SInterpFuncPhysiNode* pNode = (const SInterpFuncPhysiNode*)pObj; @@ -3234,6 +3332,12 @@ static int32_t physiInterpFuncNodeToJson(const void* pObj, SJson* pJson) { if (TSDB_CODE_SUCCESS == code) { code = tjsonAddIntegerToObject(pJson, jkInterpFuncPhysiPlanInterval, pNode->interval); } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonAddIntegerToObject(pJson, jkInterpFuncPhysiPlanIntervalUnit, pNode->intervalUnit); + } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonAddIntegerToObject(pJson, jkInterpFuncPhysiPlanPrecision, pNode->precision); + } if (TSDB_CODE_SUCCESS == code) { code = tjsonAddIntegerToObject(pJson, jkInterpFuncPhysiPlanFillMode, pNode->fillMode); } @@ -3243,6 +3347,9 @@ static int32_t physiInterpFuncNodeToJson(const void* pObj, SJson* pJson) { if (TSDB_CODE_SUCCESS == code) { code = tjsonAddObject(pJson, jkInterpFuncPhysiPlanTimeSeries, nodeToJson, pNode->pTimeSeries); } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonAddObject(pJson, jkInterpFuncPhysiPlanStreamNodeOption, streamNodeOptionToJson, &pNode->streamNodeOption); + } return code; } @@ -3266,6 +3373,12 @@ static int32_t jsonToPhysiInterpFuncNode(const SJson* pJson, void* pObj) { if (TSDB_CODE_SUCCESS == code) { code = tjsonGetBigIntValue(pJson, jkInterpFuncPhysiPlanInterval, &pNode->interval); } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonGetTinyIntValue(pJson, jkInterpFuncPhysiPlanIntervalUnit, &pNode->intervalUnit); + } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonGetTinyIntValue(pJson, jkInterpFuncPhysiPlanPrecision, &pNode->precision); + } if (TSDB_CODE_SUCCESS == code) { tjsonGetNumberValue(pJson, jkInterpFuncPhysiPlanFillMode, pNode->fillMode, code); } @@ -3275,6 +3388,9 @@ static int32_t jsonToPhysiInterpFuncNode(const SJson* pJson, void* pObj) { if (TSDB_CODE_SUCCESS == code) { code = jsonToNodeObject(pJson, jkInterpFuncPhysiPlanTimeSeries, &pNode->pTimeSeries); } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonToObject(pJson, jkInterpFuncPhysiPlanStreamNodeOption, jsonToStreamNodeOption, &pNode->streamNodeOption); + } return code; } @@ -5771,6 +5887,10 @@ static const char* jkSelectStmtLimit = "Limit"; static const char* jkSelectStmtSlimit = "Slimit"; static const char* jkSelectStmtStmtName = "StmtName"; static const char* jkSelectStmtHasAggFuncs = "HasAggFuncs"; +static const char* jkSelectStmtInterpFuncs = "HasInterpFuncs"; +static const char* jkSelectStmtInterpFill = "InterpFill"; +static const char* jkSelectStmtInterpEvery = "InterpEvery"; +static const char* jkSelectStmtTwaOrElapsedFuncs = "HasTwaOrElapsedFuncs"; static int32_t selectStmtToJson(const void* pObj, SJson* pJson) { const SSelectStmt* pNode = (const SSelectStmt*)pObj; @@ -5818,6 +5938,18 @@ static int32_t selectStmtToJson(const void* pObj, SJson* pJson) { if (TSDB_CODE_SUCCESS == code) { code = tjsonAddBoolToObject(pJson, jkSelectStmtHasAggFuncs, pNode->hasAggFuncs); } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonAddBoolToObject(pJson, jkSelectStmtInterpFuncs, pNode->hasInterpFunc); + } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonAddBoolToObject(pJson, jkSelectStmtTwaOrElapsedFuncs, pNode->hasTwaOrElapsedFunc); + } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonAddObject(pJson, jkSelectStmtInterpFill, nodeToJson, pNode->pFill); + } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonAddObject(pJson, jkSelectStmtInterpEvery, nodeToJson, pNode->pEvery); + } return code; } @@ -5868,6 +6000,18 @@ static int32_t jsonToSelectStmt(const SJson* pJson, void* pObj) { if (TSDB_CODE_SUCCESS == code) { code = tjsonGetBoolValue(pJson, jkSelectStmtHasAggFuncs, &pNode->hasAggFuncs); } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonGetBoolValue(pJson, jkSelectStmtInterpFuncs, &pNode->hasInterpFunc); + } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonGetBoolValue(pJson, jkSelectStmtTwaOrElapsedFuncs, &pNode->hasTwaOrElapsedFunc); + } + if (TSDB_CODE_SUCCESS == code) { + code = jsonToNodeObject(pJson, jkSelectStmtInterpFill, &pNode->pFill); + } + if (TSDB_CODE_SUCCESS == code) { + code = jsonToNodeObject(pJson, jkSelectStmtInterpEvery, &pNode->pEvery); + } return code; } @@ -8009,6 +8153,7 @@ static int32_t specificNodeToJson(const void* pObj, SJson* pJson) { case QUERY_NODE_PHYSICAL_PLAN_INDEF_ROWS_FUNC: return physiIndefRowsFuncNodeToJson(pObj, pJson); case QUERY_NODE_PHYSICAL_PLAN_INTERP_FUNC: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERP_FUNC: return physiInterpFuncNodeToJson(pObj, pJson); case QUERY_NODE_PHYSICAL_PLAN_FORECAST_FUNC: return physiForecastFuncNodeToJson(pObj, pJson); @@ -8379,6 +8524,7 @@ static int32_t jsonToSpecificNode(const SJson* pJson, void* pObj) { case QUERY_NODE_PHYSICAL_PLAN_INDEF_ROWS_FUNC: return jsonToPhysiIndefRowsFuncNode(pJson, pObj); case QUERY_NODE_PHYSICAL_PLAN_INTERP_FUNC: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERP_FUNC: return jsonToPhysiInterpFuncNode(pJson, pObj); case QUERY_NODE_PHYSICAL_PLAN_FORECAST_FUNC: return jsonToPhysiForecastFuncNode(pJson, pObj); diff --git a/source/libs/nodes/src/nodesMsgFuncs.c b/source/libs/nodes/src/nodesMsgFuncs.c index 7e3e0e0806..28d0b9fbd4 100644 --- a/source/libs/nodes/src/nodesMsgFuncs.c +++ b/source/libs/nodes/src/nodesMsgFuncs.c @@ -3190,7 +3190,7 @@ static int32_t physiWindowNodeToMsg(const void* pObj, STlvEncoder* pEncoder) { code = tlvEncodeBool(pEncoder, PHY_WINDOW_CODE_MERGE_DATA_BLOCK, pNode->mergeDataBlock); } if (TSDB_CODE_SUCCESS == code) { - code = tlvEncodeI8(pEncoder, PHY_WINDOW_CODE_DEST_HAS_PRIMARY_KEY, pNode->destHasPrimayKey); + code = tlvEncodeI8(pEncoder, PHY_WINDOW_CODE_DEST_HAS_PRIMARY_KEY, pNode->destHasPrimaryKey); } return code; @@ -3234,7 +3234,7 @@ static int32_t msgToPhysiWindowNode(STlvDecoder* pDecoder, void* pObj) { code = tlvDecodeBool(pTlv, &pNode->mergeDataBlock); break; case PHY_WINDOW_CODE_DEST_HAS_PRIMARY_KEY: - code = tlvDecodeI8(pTlv, &pNode->destHasPrimayKey); + code = tlvDecodeI8(pTlv, &pNode->destHasPrimaryKey); break; default: break; @@ -4640,6 +4640,7 @@ static int32_t specificNodeToMsg(const void* pObj, STlvEncoder* pEncoder) { code = physiIndefRowsFuncNodeToMsg(pObj, pEncoder); break; case QUERY_NODE_PHYSICAL_PLAN_INTERP_FUNC: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERP_FUNC: code = physiInterpFuncNodeToMsg(pObj, pEncoder); break; case QUERY_NODE_PHYSICAL_PLAN_FORECAST_FUNC: @@ -4808,6 +4809,7 @@ static int32_t msgToSpecificNode(STlvDecoder* pDecoder, void* pObj) { code = msgToPhysiIndefRowsFuncNode(pDecoder, pObj); break; case QUERY_NODE_PHYSICAL_PLAN_INTERP_FUNC: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERP_FUNC: code = msgToPhysiInterpFuncNode(pDecoder, pObj); break; case QUERY_NODE_PHYSICAL_PLAN_FORECAST_FUNC: diff --git a/source/libs/nodes/src/nodesUtilFuncs.c b/source/libs/nodes/src/nodesUtilFuncs.c index af51eff8a9..a9d0aa2924 100644 --- a/source/libs/nodes/src/nodesUtilFuncs.c +++ b/source/libs/nodes/src/nodesUtilFuncs.c @@ -766,6 +766,8 @@ int32_t nodesMakeNode(ENodeType type, SNode** ppNodeOut) { code = makeNode(type, sizeof(SSubplan), &pNode); break; case QUERY_NODE_PHYSICAL_PLAN: code = makeNode(type, sizeof(SQueryPlan), &pNode); break; + case QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERP_FUNC: + code = makeNode(type, sizeof(SStreamInterpFuncPhysiNode), &pNode); break; default: break; } @@ -1725,7 +1727,8 @@ void nodesDestroyNode(SNode* pNode) { nodesDestroyList(pPhyNode->pFuncs); break; } - case QUERY_NODE_PHYSICAL_PLAN_INTERP_FUNC: { + case QUERY_NODE_PHYSICAL_PLAN_INTERP_FUNC: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERP_FUNC: { SInterpFuncPhysiNode* pPhyNode = (SInterpFuncPhysiNode*)pNode; destroyPhysiNode((SPhysiNode*)pPhyNode); nodesDestroyList(pPhyNode->pExprs); diff --git a/source/libs/parser/inc/sql.y b/source/libs/parser/inc/sql.y index 635e9f570f..e1c3456e3f 100644 --- a/source/libs/parser/inc/sql.y +++ b/source/libs/parser/inc/sql.y @@ -776,6 +776,7 @@ tag_def_or_ref_opt(A) ::= TAGS NK_LP column_stream_def_list(B) NK_RP. stream_options(A) ::= . { A = createStreamOptions(pCxt); } stream_options(A) ::= stream_options(B) TRIGGER AT_ONCE(C). { A = setStreamOptions(pCxt, B, SOPT_TRIGGER_TYPE_SET, &C, NULL); } stream_options(A) ::= stream_options(B) TRIGGER WINDOW_CLOSE(C). { A = setStreamOptions(pCxt, B, SOPT_TRIGGER_TYPE_SET, &C, NULL); } +stream_options(A) ::= stream_options(B) TRIGGER FORCE_WINDOW_CLOSE(C). { A = setStreamOptions(pCxt, B, SOPT_TRIGGER_TYPE_SET, &C, NULL); } stream_options(A) ::= stream_options(B) TRIGGER MAX_DELAY(C) duration_literal(D). { A = setStreamOptions(pCxt, B, SOPT_TRIGGER_TYPE_SET, &C, releaseRawExprNode(pCxt, D)); } stream_options(A) ::= stream_options(B) WATERMARK duration_literal(C). { A = setStreamOptions(pCxt, B, SOPT_WATERMARK_SET, NULL, releaseRawExprNode(pCxt, C)); } stream_options(A) ::= stream_options(B) IGNORE EXPIRED NK_INTEGER(C). { A = setStreamOptions(pCxt, B, SOPT_IGNORE_EXPIRED_SET, &C, NULL); } diff --git a/source/libs/parser/src/parAstCreater.c b/source/libs/parser/src/parAstCreater.c index 269e9e4a04..245346273f 100644 --- a/source/libs/parser/src/parAstCreater.c +++ b/source/libs/parser/src/parAstCreater.c @@ -3476,6 +3476,8 @@ static int8_t getTriggerType(uint32_t tokenType) { return STREAM_TRIGGER_WINDOW_CLOSE; case TK_MAX_DELAY: return STREAM_TRIGGER_MAX_DELAY; + case TK_FORCE_WINDOW_CLOSE: + return STREAM_TRIGGER_FORCE_WINDOW_CLOSE; default: break; } diff --git a/source/libs/parser/src/parTokenizer.c b/source/libs/parser/src/parTokenizer.c index 9b2815d2df..1db139b8d4 100644 --- a/source/libs/parser/src/parTokenizer.c +++ b/source/libs/parser/src/parTokenizer.c @@ -347,6 +347,7 @@ static SKeyword keywordTable[] = { {"KEEP_TIME_OFFSET", TK_KEEP_TIME_OFFSET}, {"ARBGROUPS", TK_ARBGROUPS}, {"IS_IMPORT", TK_IS_IMPORT}, + {"FORCE_WINDOW_CLOSE", TK_FORCE_WINDOW_CLOSE}, }; // clang-format on diff --git a/source/libs/parser/src/parTranslater.c b/source/libs/parser/src/parTranslater.c index 7aada2be12..02295b34da 100755 --- a/source/libs/parser/src/parTranslater.c +++ b/source/libs/parser/src/parTranslater.c @@ -2846,6 +2846,9 @@ static void setFuncClassification(STranslateContext* pCxt, SFunctionNode* pFunc) pSelect->hasUniqueFunc = pSelect->hasUniqueFunc ? true : (FUNCTION_TYPE_UNIQUE == pFunc->funcType); pSelect->hasTailFunc = pSelect->hasTailFunc ? true : (FUNCTION_TYPE_TAIL == pFunc->funcType); pSelect->hasInterpFunc = pSelect->hasInterpFunc ? true : (FUNCTION_TYPE_INTERP == pFunc->funcType); + pSelect->hasTwaOrElapsedFunc = pSelect->hasTwaOrElapsedFunc ? true + : (FUNCTION_TYPE_TWA == pFunc->funcType || + FUNCTION_TYPE_ELAPSED == pFunc->funcType); pSelect->hasInterpPseudoColFunc = pSelect->hasInterpPseudoColFunc ? true : fmIsInterpPseudoColumnFunc(pFunc->funcId); pSelect->hasForecastFunc = pSelect->hasForecastFunc ? true : (FUNCTION_TYPE_FORECAST == pFunc->funcType); @@ -6224,12 +6227,24 @@ static int32_t translateInterp(STranslateContext* pCxt, SSelectStmt* pSelect) { } } - if (NULL == pSelect->pRange || NULL == pSelect->pEvery || NULL == pSelect->pFill) { - if (pSelect->pRange != NULL && QUERY_NODE_OPERATOR == nodeType(pSelect->pRange) && pSelect->pEvery == NULL) { - // single point interp every can be omitted - } else { - return generateSyntaxErrMsgExt(&pCxt->msgBuf, TSDB_CODE_PAR_INVALID_INTERP_CLAUSE, - "Missing RANGE clause, EVERY clause or FILL clause"); + if (pCxt->createStream) { + if (NULL != pSelect->pRange) { + return generateSyntaxErrMsgExt(&pCxt->msgBuf, TSDB_CODE_PAR_INVALID_STREAM_QUERY, + "Stream Unsupported RANGE clause"); + } + + if (NULL == pSelect->pEvery || NULL == pSelect->pFill) { + return generateSyntaxErrMsgExt(&pCxt->msgBuf, TSDB_CODE_PAR_INVALID_STREAM_QUERY, + "Missing EVERY clause or FILL clause"); + } + } else { + if (NULL == pSelect->pRange || NULL == pSelect->pEvery || NULL == pSelect->pFill) { + if (pSelect->pRange != NULL && QUERY_NODE_OPERATOR == nodeType(pSelect->pRange) && pSelect->pEvery == NULL) { + // single point interp every can be omitted + } else { + return generateSyntaxErrMsgExt(&pCxt->msgBuf, TSDB_CODE_PAR_INVALID_INTERP_CLAUSE, + "Missing RANGE clause, EVERY clause or FILL clause"); + } } } @@ -10506,15 +10521,14 @@ static void getSourceDatabase(SNode* pStmt, int32_t acctId, char* pDbFName) { (void)tNameGetFullDbName(&name, pDbFName); } -static void getStreamQueryFirstProjectAliasName(SHashObj* pUserAliasSet, char* aliasName, int32_t len) { - if (NULL == taosHashGet(pUserAliasSet, "_wstart", strlen("_wstart"))) { - snprintf(aliasName, len, "%s", "_wstart"); - return; - } - if (NULL == taosHashGet(pUserAliasSet, "ts", strlen("ts"))) { - snprintf(aliasName, len, "%s", "ts"); - return; +static void getStreamQueryFirstProjectAliasName(SHashObj* pUserAliasSet, char* aliasName, int32_t len, char* defaultName[]) { + for (int32_t i = 0; defaultName[i] != NULL; i++) { + if (NULL == taosHashGet(pUserAliasSet, defaultName[i], strlen(defaultName[i]))) { + snprintf(aliasName, len, "%s", defaultName[i]); + return; + } } + do { taosRandStr(aliasName, len - 1); aliasName[len - 1] = '\0'; @@ -10533,6 +10547,46 @@ static int32_t setColumnDefNodePrimaryKey(SColumnDefNode* pNode, bool isPk) { return code; } +static int32_t addIrowTsToCreateStreamQueryImpl(STranslateContext* pCxt, SSelectStmt* pSelect, + SHashObj* pUserAliasSet, SNodeList* pCols, SCMCreateStreamReq* pReq) { + SNode* pProj = nodesListGetNode(pSelect->pProjectionList, 0); + if (!pSelect->hasInterpFunc || + (QUERY_NODE_FUNCTION == nodeType(pProj) && 0 == strcmp("_irowts", ((SFunctionNode*)pProj)->functionName))) { + return TSDB_CODE_SUCCESS; + } + SFunctionNode* pFunc = NULL; + int32_t code = nodesMakeNode(QUERY_NODE_FUNCTION, (SNode**)&pFunc); + if (NULL == pFunc) { + return code; + } + tstrncpy(pFunc->functionName, "_irowts", tListLen(pFunc->functionName)); + tstrncpy(pFunc->node.userAlias, "_irowts", tListLen(pFunc->node.userAlias)); + char* defaultName[] = {"_irowts", NULL}; + getStreamQueryFirstProjectAliasName(pUserAliasSet, pFunc->node.aliasName, sizeof(pFunc->node.aliasName), defaultName); + code = getFuncInfo(pCxt, pFunc); + if (TSDB_CODE_SUCCESS == code) { + code = nodesListPushFront(pSelect->pProjectionList, (SNode*)pFunc); + } + if (TSDB_CODE_SUCCESS != code) { + nodesDestroyNode((SNode*)pFunc); + } + + if (TSDB_CODE_SUCCESS == code && STREAM_CREATE_STABLE_TRUE == pReq->createStb) { + SColumnDefNode* pColDef = NULL; + code = nodesMakeNode(QUERY_NODE_COLUMN_DEF, (SNode**)&pColDef); + if (TSDB_CODE_SUCCESS == code) { + strcpy(pColDef->colName, pFunc->node.aliasName); + pColDef->dataType = pFunc->node.resType; + pColDef->sma = true; + code = setColumnDefNodePrimaryKey(pColDef, false); + } + if (TSDB_CODE_SUCCESS == code) code = nodesListPushFront(pCols, (SNode*)pColDef); + if (TSDB_CODE_SUCCESS != code) nodesDestroyNode((SNode*)pColDef); + } + + return code; +} + static int32_t addWstartTsToCreateStreamQueryImpl(STranslateContext* pCxt, SSelectStmt* pSelect, SHashObj* pUserAliasSet, SNodeList* pCols, SCMCreateStreamReq* pReq) { SNode* pProj = nodesListGetNode(pSelect->pProjectionList, 0); @@ -10541,12 +10595,14 @@ static int32_t addWstartTsToCreateStreamQueryImpl(STranslateContext* pCxt, SSele return TSDB_CODE_SUCCESS; } SFunctionNode* pFunc = NULL; - int32_t code = nodesMakeNode(QUERY_NODE_FUNCTION, (SNode**)&pFunc); + int32_t code = nodesMakeNode(QUERY_NODE_FUNCTION, (SNode**)&pFunc); if (NULL == pFunc) { return code; } strcpy(pFunc->functionName, "_wstart"); - getStreamQueryFirstProjectAliasName(pUserAliasSet, pFunc->node.aliasName, sizeof(pFunc->node.aliasName)); + strcpy(pFunc->node.userAlias, "_irowts"); + char* defaultName[] = {"_wstart", "ts", NULL}; + getStreamQueryFirstProjectAliasName(pUserAliasSet, pFunc->node.aliasName, sizeof(pFunc->node.aliasName), defaultName); code = getFuncInfo(pCxt, pFunc); if (TSDB_CODE_SUCCESS == code) { code = nodesListPushFront(pSelect->pProjectionList, (SNode*)pFunc); @@ -10570,7 +10626,7 @@ static int32_t addWstartTsToCreateStreamQueryImpl(STranslateContext* pCxt, SSele return code; } -static int32_t addWstartTsToCreateStreamQuery(STranslateContext* pCxt, SNode* pStmt, SNodeList* pCols, +static int32_t addTsKeyToCreateStreamQuery(STranslateContext* pCxt, SNode* pStmt, SNodeList* pCols, SCMCreateStreamReq* pReq) { SSelectStmt* pSelect = (SSelectStmt*)pStmt; SHashObj* pUserAliasSet = NULL; @@ -10578,6 +10634,9 @@ static int32_t addWstartTsToCreateStreamQuery(STranslateContext* pCxt, SNode* pS if (TSDB_CODE_SUCCESS == code) { code = addWstartTsToCreateStreamQueryImpl(pCxt, pSelect, pUserAliasSet, pCols, pReq); } + if (TSDB_CODE_SUCCESS == code) { + code = addIrowTsToCreateStreamQueryImpl(pCxt, pSelect, pUserAliasSet, pCols, pReq); + } taosHashCleanup(pUserAliasSet); return code; } @@ -10834,7 +10893,7 @@ static int32_t checkStreamQuery(STranslateContext* pCxt, SCreateStreamStmt* pStm "SUBTABLE expression must not has column when no partition by clause"); } - if (NULL == pSelect->pWindow && STREAM_TRIGGER_AT_ONCE != pStmt->pOptions->triggerType) { + if (NULL == pSelect->pWindow && !pSelect->hasInterpFunc && STREAM_TRIGGER_AT_ONCE != pStmt->pOptions->triggerType) { return generateSyntaxErrMsgExt(&pCxt->msgBuf, TSDB_CODE_PAR_INVALID_STREAM_QUERY, "The trigger mode of non window query can only be AT_ONCE"); } @@ -10870,6 +10929,104 @@ static int32_t checkStreamQuery(STranslateContext* pCxt, SCreateStreamStmt* pStm } } + if (pSelect->hasInterpFunc) { + // Temporary code + if (pStmt->pOptions->triggerType != STREAM_TRIGGER_FORCE_WINDOW_CLOSE) { + return generateSyntaxErrMsgExt(&pCxt->msgBuf, TSDB_CODE_PAR_INVALID_STREAM_QUERY, + "Stream interp function only support force window close"); + } + + if (pStmt->pOptions->triggerType == STREAM_TRIGGER_FORCE_WINDOW_CLOSE) { + if (pStmt->pOptions->fillHistory) { + return generateSyntaxErrMsgExt(&pCxt->msgBuf, TSDB_CODE_PAR_INVALID_STREAM_QUERY, + "When trigger was force window close, Stream interp unsupported Fill history"); + } else if (pSelect->pFill != NULL) { + EFillMode mode = ((SFillNode*)(pSelect->pFill))->mode; + if (mode == FILL_MODE_NEXT) { + return generateSyntaxErrMsgExt(&pCxt->msgBuf, TSDB_CODE_PAR_INVALID_STREAM_QUERY, + "When trigger was force window close, Stream interp unsupported Fill(Next)"); + } else if (mode == FILL_MODE_LINEAR) { + return generateSyntaxErrMsgExt(&pCxt->msgBuf, TSDB_CODE_PAR_INVALID_STREAM_QUERY, + "When trigger was force window close, Stream interp unsupported Fill(Linear)"); + } + } + } + + if (pStmt->pOptions->triggerType == STREAM_TRIGGER_WINDOW_CLOSE) { + return generateSyntaxErrMsgExt(&pCxt->msgBuf, TSDB_CODE_PAR_INVALID_STREAM_QUERY, + "Stream interp unsupported window close"); + } + + if (pStmt->pOptions->triggerType == STREAM_TRIGGER_MAX_DELAY) { + return generateSyntaxErrMsgExt(&pCxt->msgBuf, TSDB_CODE_PAR_INVALID_STREAM_QUERY, + "Stream interp unsupported max delay"); + } + + if ((SRealTableNode*)pSelect->pFromTable && ((SRealTableNode*)pSelect->pFromTable)->pMeta && + TSDB_SUPER_TABLE == ((SRealTableNode*)pSelect->pFromTable)->pMeta->tableType && + !hasTbnameFunction(pSelect->pPartitionByList)) { + return generateSyntaxErrMsgExt(&pCxt->msgBuf, TSDB_CODE_PAR_INVALID_STREAM_QUERY, + "Interp for stream on super table must patitioned by table name"); + } + } + + if (pSelect->hasTwaOrElapsedFunc) { + if (pStmt->pOptions->triggerType != STREAM_TRIGGER_FORCE_WINDOW_CLOSE) { + return generateSyntaxErrMsgExt(&pCxt->msgBuf, TSDB_CODE_PAR_INVALID_STREAM_QUERY, + "Stream twa or elapsed function only support force window close"); + } + if (pSelect->pWindow->type != QUERY_NODE_INTERVAL_WINDOW) { + return generateSyntaxErrMsgExt(&pCxt->msgBuf, TSDB_CODE_PAR_INVALID_STREAM_QUERY, + "Stream twa or elapsed function only support interval"); + } + + if ((SRealTableNode*)pSelect->pFromTable && ((SRealTableNode*)pSelect->pFromTable)->pMeta && + TSDB_SUPER_TABLE == ((SRealTableNode*)pSelect->pFromTable)->pMeta->tableType && + !hasTbnameFunction(pSelect->pPartitionByList)) { + return generateSyntaxErrMsgExt(&pCxt->msgBuf, TSDB_CODE_PAR_INVALID_STREAM_QUERY, + "twa or elapsed on super table must patitioned by table name"); + } + } + + if (pStmt->pOptions->triggerType == STREAM_TRIGGER_FORCE_WINDOW_CLOSE) { + if (pStmt->pOptions->fillHistory) { + return generateSyntaxErrMsgExt( + &pCxt->msgBuf, TSDB_CODE_PAR_INVALID_STREAM_QUERY, + "When trigger was force window close, Stream unsupported Fill history"); + } + + if (pStmt->pOptions->ignoreExpired != 1) { + return generateSyntaxErrMsgExt( + &pCxt->msgBuf, TSDB_CODE_PAR_INVALID_STREAM_QUERY, + "When trigger was force window close, Stream must not set ignore expired 0"); + } + + if (pStmt->pOptions->ignoreUpdate != 1) { + return generateSyntaxErrMsgExt( + &pCxt->msgBuf, TSDB_CODE_PAR_INVALID_STREAM_QUERY, + "When trigger was force window close, Stream must not set ignore update 0"); + } + + if (pSelect->pWindow != NULL && QUERY_NODE_INTERVAL_WINDOW == nodeType(pSelect->pWindow)) { + SIntervalWindowNode* pWindow = (SIntervalWindowNode*)pSelect->pWindow; + if (NULL != pWindow->pSliding) { + int64_t interval = ((SValueNode*)pWindow->pInterval)->datum.i; + int64_t sliding = ((SValueNode*)pWindow->pSliding)->datum.i; + if (interval != sliding) { + return generateSyntaxErrMsgExt(&pCxt->msgBuf, TSDB_CODE_PAR_INVALID_STREAM_QUERY, + "When trigger was force window close, Stream unsupported sliding"); + } + } + } + + if ((SRealTableNode*)pSelect->pFromTable && ((SRealTableNode*)pSelect->pFromTable)->pMeta && + TSDB_SUPER_TABLE == ((SRealTableNode*)pSelect->pFromTable)->pMeta->tableType && + !hasTbnameFunction(pSelect->pPartitionByList)) { + return generateSyntaxErrMsgExt(&pCxt->msgBuf, TSDB_CODE_PAR_INVALID_STREAM_QUERY, + "When trigger was force window close, Super table must patitioned by table name"); + } + } + if (NULL != pSelect->pGroupByList) { return generateSyntaxErrMsgExt(&pCxt->msgBuf, TSDB_CODE_PAR_INVALID_STREAM_QUERY, "Unsupported Group by"); } @@ -11434,7 +11591,7 @@ static int32_t checkAndAdjStreamDestTableSchema(STranslateContext* pCxt, SCreate .bytes = tDataTypes[TSDB_DATA_TYPE_TIMESTAMP].bytes}; } int32_t code = checkTableSchemaImpl(pCxt, pStmt->pTags, pStmt->pCols, NULL); - if (TSDB_CODE_SUCCESS == code && NULL == pSelect->pWindow && + if (TSDB_CODE_SUCCESS == code && NULL == pSelect->pWindow && !pSelect->hasInterpFunc && ((SRealTableNode*)pSelect->pFromTable && hasPkInTable(((SRealTableNode*)pSelect->pFromTable)->pMeta))) { if (1 >= LIST_LENGTH(pStmt->pCols) || 1 >= LIST_LENGTH(pSelect->pProjectionList)) { return generateSyntaxErrMsg(&pCxt->msgBuf, TSDB_CODE_PAR_INVALID_STREAM_QUERY); @@ -11475,7 +11632,7 @@ static int32_t buildCreateStreamQuery(STranslateContext* pCxt, SCreateStreamStmt code = addColsToCreateStreamQuery(pCxt, pStmt, pReq); } if (TSDB_CODE_SUCCESS == code) { - code = addWstartTsToCreateStreamQuery(pCxt, pStmt->pQuery, pStmt->pCols, pReq); + code = addTsKeyToCreateStreamQuery(pCxt, pStmt->pQuery, pStmt->pCols, pReq); } if (TSDB_CODE_SUCCESS == code) { code = checkStreamQuery(pCxt, pStmt); diff --git a/source/libs/parser/src/sql.c b/source/libs/parser/src/sql.c index a352b237b6..06b5350ed8 100644 --- a/source/libs/parser/src/sql.c +++ b/source/libs/parser/src/sql.c @@ -1,3 +1,5 @@ +/* This file is automatically generated by Lemon from input grammar +** source file "sql.y". */ /* ** 2000-05-29 ** @@ -22,10 +24,7 @@ ** The following is the concatenation of all %include directives from the ** input grammar file: */ -#include -#include /************ Begin %include sections from the grammar ************************/ - #include #include #include @@ -42,11 +41,400 @@ #define YYSTACKDEPTH 0 /**************** End of %include directives **********************************/ -/* These constants specify the various numeric values for terminal symbols -** in a format understandable to "makeheaders". This section is blank unless -** "lemon" is run with the "-m" command-line option. -***************** Begin makeheaders token definitions *************************/ -/**************** End makeheaders token definitions ***************************/ +/* These constants specify the various numeric values for terminal symbols. +***************** Begin token definitions *************************************/ +#ifndef TK_OR +#define TK_OR 1 +#define TK_AND 2 +#define TK_UNION 3 +#define TK_ALL 4 +#define TK_MINUS 5 +#define TK_EXCEPT 6 +#define TK_INTERSECT 7 +#define TK_NK_BITAND 8 +#define TK_NK_BITOR 9 +#define TK_NK_LSHIFT 10 +#define TK_NK_RSHIFT 11 +#define TK_NK_PLUS 12 +#define TK_NK_MINUS 13 +#define TK_NK_STAR 14 +#define TK_NK_SLASH 15 +#define TK_NK_REM 16 +#define TK_NK_CONCAT 17 +#define TK_CREATE 18 +#define TK_ACCOUNT 19 +#define TK_NK_ID 20 +#define TK_PASS 21 +#define TK_NK_STRING 22 +#define TK_ALTER 23 +#define TK_PPS 24 +#define TK_TSERIES 25 +#define TK_STORAGE 26 +#define TK_STREAMS 27 +#define TK_QTIME 28 +#define TK_DBS 29 +#define TK_USERS 30 +#define TK_CONNS 31 +#define TK_STATE 32 +#define TK_NK_COMMA 33 +#define TK_HOST 34 +#define TK_IS_IMPORT 35 +#define TK_NK_INTEGER 36 +#define TK_CREATEDB 37 +#define TK_USER 38 +#define TK_ENABLE 39 +#define TK_SYSINFO 40 +#define TK_ADD 41 +#define TK_DROP 42 +#define TK_GRANT 43 +#define TK_ON 44 +#define TK_TO 45 +#define TK_REVOKE 46 +#define TK_FROM 47 +#define TK_SUBSCRIBE 48 +#define TK_READ 49 +#define TK_WRITE 50 +#define TK_NK_DOT 51 +#define TK_WITH 52 +#define TK_ENCRYPT_KEY 53 +#define TK_ANODE 54 +#define TK_UPDATE 55 +#define TK_ANODES 56 +#define TK_DNODE 57 +#define TK_PORT 58 +#define TK_DNODES 59 +#define TK_RESTORE 60 +#define TK_NK_IPTOKEN 61 +#define TK_FORCE 62 +#define TK_UNSAFE 63 +#define TK_CLUSTER 64 +#define TK_LOCAL 65 +#define TK_QNODE 66 +#define TK_BNODE 67 +#define TK_SNODE 68 +#define TK_MNODE 69 +#define TK_VNODE 70 +#define TK_DATABASE 71 +#define TK_USE 72 +#define TK_FLUSH 73 +#define TK_TRIM 74 +#define TK_S3MIGRATE 75 +#define TK_COMPACT 76 +#define TK_IF 77 +#define TK_NOT 78 +#define TK_EXISTS 79 +#define TK_BUFFER 80 +#define TK_CACHEMODEL 81 +#define TK_CACHESIZE 82 +#define TK_COMP 83 +#define TK_DURATION 84 +#define TK_NK_VARIABLE 85 +#define TK_MAXROWS 86 +#define TK_MINROWS 87 +#define TK_KEEP 88 +#define TK_PAGES 89 +#define TK_PAGESIZE 90 +#define TK_TSDB_PAGESIZE 91 +#define TK_PRECISION 92 +#define TK_REPLICA 93 +#define TK_VGROUPS 94 +#define TK_SINGLE_STABLE 95 +#define TK_RETENTIONS 96 +#define TK_SCHEMALESS 97 +#define TK_WAL_LEVEL 98 +#define TK_WAL_FSYNC_PERIOD 99 +#define TK_WAL_RETENTION_PERIOD 100 +#define TK_WAL_RETENTION_SIZE 101 +#define TK_WAL_ROLL_PERIOD 102 +#define TK_WAL_SEGMENT_SIZE 103 +#define TK_STT_TRIGGER 104 +#define TK_TABLE_PREFIX 105 +#define TK_TABLE_SUFFIX 106 +#define TK_S3_CHUNKPAGES 107 +#define TK_S3_KEEPLOCAL 108 +#define TK_S3_COMPACT 109 +#define TK_KEEP_TIME_OFFSET 110 +#define TK_ENCRYPT_ALGORITHM 111 +#define TK_NK_COLON 112 +#define TK_BWLIMIT 113 +#define TK_START 114 +#define TK_TIMESTAMP 115 +#define TK_END 116 +#define TK_TABLE 117 +#define TK_NK_LP 118 +#define TK_NK_RP 119 +#define TK_USING 120 +#define TK_FILE 121 +#define TK_STABLE 122 +#define TK_COLUMN 123 +#define TK_MODIFY 124 +#define TK_RENAME 125 +#define TK_TAG 126 +#define TK_SET 127 +#define TK_NK_EQ 128 +#define TK_TAGS 129 +#define TK_BOOL 130 +#define TK_TINYINT 131 +#define TK_SMALLINT 132 +#define TK_INT 133 +#define TK_INTEGER 134 +#define TK_BIGINT 135 +#define TK_FLOAT 136 +#define TK_DOUBLE 137 +#define TK_BINARY 138 +#define TK_NCHAR 139 +#define TK_UNSIGNED 140 +#define TK_JSON 141 +#define TK_VARCHAR 142 +#define TK_MEDIUMBLOB 143 +#define TK_BLOB 144 +#define TK_VARBINARY 145 +#define TK_GEOMETRY 146 +#define TK_DECIMAL 147 +#define TK_COMMENT 148 +#define TK_MAX_DELAY 149 +#define TK_WATERMARK 150 +#define TK_ROLLUP 151 +#define TK_TTL 152 +#define TK_SMA 153 +#define TK_DELETE_MARK 154 +#define TK_FIRST 155 +#define TK_LAST 156 +#define TK_SHOW 157 +#define TK_FULL 158 +#define TK_PRIVILEGES 159 +#define TK_DATABASES 160 +#define TK_TABLES 161 +#define TK_STABLES 162 +#define TK_MNODES 163 +#define TK_QNODES 164 +#define TK_ARBGROUPS 165 +#define TK_FUNCTIONS 166 +#define TK_INDEXES 167 +#define TK_ACCOUNTS 168 +#define TK_APPS 169 +#define TK_CONNECTIONS 170 +#define TK_LICENCES 171 +#define TK_GRANTS 172 +#define TK_LOGS 173 +#define TK_MACHINES 174 +#define TK_ENCRYPTIONS 175 +#define TK_QUERIES 176 +#define TK_SCORES 177 +#define TK_TOPICS 178 +#define TK_VARIABLES 179 +#define TK_BNODES 180 +#define TK_SNODES 181 +#define TK_TRANSACTIONS 182 +#define TK_DISTRIBUTED 183 +#define TK_CONSUMERS 184 +#define TK_SUBSCRIPTIONS 185 +#define TK_VNODES 186 +#define TK_ALIVE 187 +#define TK_VIEWS 188 +#define TK_VIEW 189 +#define TK_COMPACTS 190 +#define TK_NORMAL 191 +#define TK_CHILD 192 +#define TK_LIKE 193 +#define TK_TBNAME 194 +#define TK_QTAGS 195 +#define TK_AS 196 +#define TK_SYSTEM 197 +#define TK_TSMA 198 +#define TK_INTERVAL 199 +#define TK_RECURSIVE 200 +#define TK_TSMAS 201 +#define TK_FUNCTION 202 +#define TK_INDEX 203 +#define TK_COUNT 204 +#define TK_LAST_ROW 205 +#define TK_META 206 +#define TK_ONLY 207 +#define TK_TOPIC 208 +#define TK_CONSUMER 209 +#define TK_GROUP 210 +#define TK_DESC 211 +#define TK_DESCRIBE 212 +#define TK_RESET 213 +#define TK_QUERY 214 +#define TK_CACHE 215 +#define TK_EXPLAIN 216 +#define TK_ANALYZE 217 +#define TK_VERBOSE 218 +#define TK_NK_BOOL 219 +#define TK_RATIO 220 +#define TK_NK_FLOAT 221 +#define TK_OUTPUTTYPE 222 +#define TK_AGGREGATE 223 +#define TK_BUFSIZE 224 +#define TK_LANGUAGE 225 +#define TK_REPLACE 226 +#define TK_STREAM 227 +#define TK_INTO 228 +#define TK_PAUSE 229 +#define TK_RESUME 230 +#define TK_PRIMARY 231 +#define TK_KEY 232 +#define TK_TRIGGER 233 +#define TK_AT_ONCE 234 +#define TK_WINDOW_CLOSE 235 +#define TK_FORCE_WINDOW_CLOSE 236 +#define TK_IGNORE 237 +#define TK_EXPIRED 238 +#define TK_FILL_HISTORY 239 +#define TK_SUBTABLE 240 +#define TK_UNTREATED 241 +#define TK_KILL 242 +#define TK_CONNECTION 243 +#define TK_TRANSACTION 244 +#define TK_BALANCE 245 +#define TK_VGROUP 246 +#define TK_LEADER 247 +#define TK_MERGE 248 +#define TK_REDISTRIBUTE 249 +#define TK_SPLIT 250 +#define TK_DELETE 251 +#define TK_INSERT 252 +#define TK_NK_BIN 253 +#define TK_NK_HEX 254 +#define TK_NULL 255 +#define TK_NK_QUESTION 256 +#define TK_NK_ALIAS 257 +#define TK_NK_ARROW 258 +#define TK_ROWTS 259 +#define TK_QSTART 260 +#define TK_QEND 261 +#define TK_QDURATION 262 +#define TK_WSTART 263 +#define TK_WEND 264 +#define TK_WDURATION 265 +#define TK_IROWTS 266 +#define TK_ISFILLED 267 +#define TK_FLOW 268 +#define TK_FHIGH 269 +#define TK_FROWTS 270 +#define TK_CAST 271 +#define TK_POSITION 272 +#define TK_IN 273 +#define TK_FOR 274 +#define TK_NOW 275 +#define TK_TODAY 276 +#define TK_RAND 277 +#define TK_SUBSTR 278 +#define TK_SUBSTRING 279 +#define TK_BOTH 280 +#define TK_TRAILING 281 +#define TK_LEADING 282 +#define TK_TIMEZONE 283 +#define TK_CLIENT_VERSION 284 +#define TK_SERVER_VERSION 285 +#define TK_SERVER_STATUS 286 +#define TK_CURRENT_USER 287 +#define TK_PI 288 +#define TK_CASE 289 +#define TK_WHEN 290 +#define TK_THEN 291 +#define TK_ELSE 292 +#define TK_BETWEEN 293 +#define TK_IS 294 +#define TK_NK_LT 295 +#define TK_NK_GT 296 +#define TK_NK_LE 297 +#define TK_NK_GE 298 +#define TK_NK_NE 299 +#define TK_MATCH 300 +#define TK_NMATCH 301 +#define TK_CONTAINS 302 +#define TK_JOIN 303 +#define TK_INNER 304 +#define TK_LEFT 305 +#define TK_RIGHT 306 +#define TK_OUTER 307 +#define TK_SEMI 308 +#define TK_ANTI 309 +#define TK_ASOF 310 +#define TK_WINDOW 311 +#define TK_WINDOW_OFFSET 312 +#define TK_JLIMIT 313 +#define TK_SELECT 314 +#define TK_NK_HINT 315 +#define TK_DISTINCT 316 +#define TK_WHERE 317 +#define TK_PARTITION 318 +#define TK_BY 319 +#define TK_SESSION 320 +#define TK_STATE_WINDOW 321 +#define TK_EVENT_WINDOW 322 +#define TK_COUNT_WINDOW 323 +#define TK_ANOMALY_WINDOW 324 +#define TK_SLIDING 325 +#define TK_FILL 326 +#define TK_VALUE 327 +#define TK_VALUE_F 328 +#define TK_NONE 329 +#define TK_PREV 330 +#define TK_NULL_F 331 +#define TK_LINEAR 332 +#define TK_NEXT 333 +#define TK_HAVING 334 +#define TK_RANGE 335 +#define TK_EVERY 336 +#define TK_ORDER 337 +#define TK_SLIMIT 338 +#define TK_SOFFSET 339 +#define TK_LIMIT 340 +#define TK_OFFSET 341 +#define TK_ASC 342 +#define TK_NULLS 343 +#define TK_ABORT 344 +#define TK_AFTER 345 +#define TK_ATTACH 346 +#define TK_BEFORE 347 +#define TK_BEGIN 348 +#define TK_BITAND 349 +#define TK_BITNOT 350 +#define TK_BITOR 351 +#define TK_BLOCKS 352 +#define TK_CHANGE 353 +#define TK_COMMA 354 +#define TK_CONCAT 355 +#define TK_CONFLICT 356 +#define TK_COPY 357 +#define TK_DEFERRED 358 +#define TK_DELIMITERS 359 +#define TK_DETACH 360 +#define TK_DIVIDE 361 +#define TK_DOT 362 +#define TK_EACH 363 +#define TK_FAIL 364 +#define TK_GLOB 365 +#define TK_ID 366 +#define TK_IMMEDIATE 367 +#define TK_IMPORT 368 +#define TK_INITIALLY 369 +#define TK_INSTEAD 370 +#define TK_ISNULL 371 +#define TK_MODULES 372 +#define TK_NK_BITNOT 373 +#define TK_NK_SEMI 374 +#define TK_NOTNULL 375 +#define TK_OF 376 +#define TK_PLUS 377 +#define TK_PRIVILEGE 378 +#define TK_RAISE 379 +#define TK_RESTRICT 380 +#define TK_ROW 381 +#define TK_STAR 382 +#define TK_STATEMENT 383 +#define TK_STRICT 384 +#define TK_STRING 385 +#define TK_TIMES 386 +#define TK_VALUES 387 +#define TK_VARIABLE 388 +#define TK_WAL 389 +#endif +/**************** End token definitions ***************************************/ /* The next sections is a series of control #defines. ** various aspects of the generated parser. @@ -104,31 +492,31 @@ #endif /************* Begin control #defines *****************************************/ #define YYCODETYPE unsigned short int -#define YYNOCODE 573 +#define YYNOCODE 574 #define YYACTIONTYPE unsigned short int #define ParseTOKENTYPE SToken typedef union { int yyinit; ParseTOKENTYPE yy0; - EFillMode yy102; - EOperatorType yy140; - ETrimType yy300; - ENullOrder yy307; - EOrder yy410; - SDataType yy424; - int64_t yy483; - SToken yy557; - bool yy569; - SShowTablesOption yy595; - SAlterOption yy683; - STokenPair yy723; - EShowKind yy741; - EJoinSubType yy744; - EJoinType yy792; - int8_t yy815; - int32_t yy904; - SNodeList* yy946; - SNode* yy974; + int8_t yy47; + EOrder yy274; + ENullOrder yy305; + STokenPair yy469; + SShowTablesOption yy513; + SAlterOption yy529; + SToken yy561; + EJoinSubType yy606; + SNodeList* yy628; + EJoinType yy652; + EOperatorType yy688; + EFillMode yy690; + int64_t yy781; + ETrimType yy840; + int32_t yy844; + SDataType yy896; + bool yy957; + SNode* yy980; + EShowKind yy1041; } YYMINORTYPE; #ifndef YYSTACKDEPTH #define YYSTACKDEPTH 100 @@ -145,17 +533,17 @@ typedef union { #define ParseCTX_STORE #define YYFALLBACK 1 #define YYNSTATE 1026 -#define YYNRULE 785 -#define YYNRULE_WITH_ACTION 785 -#define YYNTOKEN 389 +#define YYNRULE 786 +#define YYNRULE_WITH_ACTION 786 +#define YYNTOKEN 390 #define YY_MAX_SHIFT 1025 -#define YY_MIN_SHIFTREDUCE 1518 -#define YY_MAX_SHIFTREDUCE 2302 -#define YY_ERROR_ACTION 2303 -#define YY_ACCEPT_ACTION 2304 -#define YY_NO_ACTION 2305 -#define YY_MIN_REDUCE 2306 -#define YY_MAX_REDUCE 3090 +#define YY_MIN_SHIFTREDUCE 1519 +#define YY_MAX_SHIFTREDUCE 2304 +#define YY_ERROR_ACTION 2305 +#define YY_ACCEPT_ACTION 2306 +#define YY_NO_ACTION 2307 +#define YY_MIN_REDUCE 2308 +#define YY_MAX_REDUCE 3093 /************* End control #defines *******************************************/ #define YY_NLOOKAHEAD ((int)(sizeof(yy_lookahead)/sizeof(yy_lookahead[0]))) @@ -222,1204 +610,1252 @@ typedef union { ** yy_default[] Default action for each state. ** *********** Begin parsing tables **********************************************/ -#define YY_ACTTAB_COUNT (4475) +#define YY_ACTTAB_COUNT (4719) static const YYACTIONTYPE yy_action[] = { - /* 0 */ 839, 683, 3063, 2678, 684, 2354, 113, 2510, 3058, 678, - /* 10 */ 3058, 2820, 60, 58, 225, 59, 57, 56, 55, 54, - /* 20 */ 506, 2307, 2019, 2676, 901, 471, 838, 231, 224, 3062, - /* 30 */ 2820, 3059, 840, 3059, 3061, 2514, 2017, 2803, 2127, 2402, - /* 40 */ 523, 2824, 151, 2845, 2568, 150, 149, 148, 147, 146, - /* 50 */ 145, 144, 143, 142, 196, 851, 170, 914, 854, 680, - /* 60 */ 2824, 53, 52, 526, 2523, 59, 57, 56, 55, 54, - /* 70 */ 512, 2122, 151, 2048, 884, 150, 149, 148, 147, 146, - /* 80 */ 145, 144, 143, 142, 919, 2025, 914, 2678, 2863, 53, - /* 90 */ 52, 2826, 2829, 59, 57, 56, 55, 54, 957, 691, - /* 100 */ 512, 498, 684, 2354, 2810, 919, 896, 2675, 901, 2044, - /* 110 */ 2826, 2828, 501, 2763, 919, 1022, 851, 170, 61, 991, - /* 120 */ 990, 989, 988, 536, 919, 987, 986, 175, 981, 980, + /* 0 */ 839, 683, 3066, 691, 684, 2356, 684, 2356, 3061, 205, + /* 10 */ 3061, 332, 60, 58, 887, 59, 57, 56, 55, 54, + /* 20 */ 506, 2309, 2021, 225, 466, 886, 838, 231, 2046, 3065, + /* 30 */ 220, 3062, 840, 3062, 3064, 126, 2019, 528, 2129, 2404, + /* 40 */ 2576, 2578, 151, 2848, 389, 150, 149, 148, 147, 146, + /* 50 */ 145, 144, 143, 142, 914, 851, 170, 2653, 854, 14, + /* 60 */ 13, 53, 52, 2514, 510, 59, 57, 56, 55, 54, + /* 70 */ 220, 2124, 151, 2050, 884, 150, 149, 148, 147, 146, + /* 80 */ 145, 144, 143, 142, 2823, 2027, 53, 52, 2866, 772, + /* 90 */ 59, 57, 56, 55, 54, 53, 52, 2653, 957, 59, + /* 100 */ 57, 56, 55, 54, 2813, 766, 896, 770, 768, 303, + /* 110 */ 302, 34, 703, 914, 2827, 1022, 638, 914, 61, 991, + /* 120 */ 990, 989, 988, 536, 75, 987, 986, 175, 981, 980, /* 130 */ 979, 978, 977, 976, 975, 174, 969, 968, 967, 535, /* 140 */ 534, 964, 963, 962, 211, 210, 961, 531, 960, 959, - /* 150 */ 958, 2844, 2804, 2525, 2895, 2130, 2131, 127, 134, 2846, - /* 160 */ 900, 2848, 2849, 895, 45, 355, 883, 2896, 919, 914, - /* 170 */ 802, 772, 1594, 213, 1593, 2958, 2045, 251, 3058, 500, - /* 180 */ 2954, 207, 2966, 850, 63, 162, 849, 766, 166, 770, - /* 190 */ 768, 303, 302, 3058, 2080, 2090, 3064, 231, 829, 232, - /* 200 */ 785, 3059, 840, 3063, 2129, 2132, 528, 3005, 75, 2574, - /* 210 */ 2576, 838, 231, 1595, 782, 512, 3059, 840, 914, 2020, - /* 220 */ 2047, 2018, 824, 107, 197, 9, 882, 2223, 106, 919, - /* 230 */ 53, 52, 2452, 2044, 59, 57, 56, 55, 54, 676, - /* 240 */ 64, 853, 200, 2966, 2967, 699, 168, 2971, 674, 227, - /* 250 */ 318, 670, 666, 581, 2023, 2024, 2077, 2845, 2079, 2082, - /* 260 */ 2083, 2084, 2085, 2086, 2087, 2088, 2089, 2091, 2092, 2093, - /* 270 */ 892, 885, 894, 307, 917, 916, 881, 2114, 2115, 2116, - /* 280 */ 2117, 2118, 2121, 2123, 2124, 2125, 2126, 2128, 2, 60, - /* 290 */ 58, 2219, 2296, 2845, 445, 105, 2042, 506, 2306, 2019, - /* 300 */ 458, 118, 2863, 625, 644, 642, 469, 441, 897, 2049, - /* 310 */ 245, 1892, 1893, 2017, 646, 2127, 63, 79, 2810, 2289, - /* 320 */ 896, 2620, 160, 159, 158, 157, 156, 155, 154, 153, - /* 330 */ 152, 466, 468, 604, 2750, 648, 828, 220, 2863, 2194, - /* 340 */ 446, 606, 830, 825, 818, 814, 810, 2138, 2122, 2678, - /* 350 */ 2157, 884, 584, 2044, 2810, 19, 896, 3062, 1946, 1947, - /* 360 */ 1891, 1894, 2025, 508, 2651, 2844, 2863, 2329, 2895, 2675, - /* 370 */ 901, 2028, 436, 2846, 900, 2848, 2849, 895, 893, 339, - /* 380 */ 883, 2896, 919, 874, 2923, 954, 187, 186, 951, 950, - /* 390 */ 949, 184, 1022, 467, 2048, 15, 337, 2044, 851, 170, - /* 400 */ 1698, 2844, 2254, 339, 2895, 592, 518, 2081, 134, 2846, - /* 410 */ 900, 2848, 2849, 895, 1697, 2158, 883, 2896, 919, 558, - /* 420 */ 2603, 172, 2295, 181, 2929, 2958, 533, 532, 2810, 500, - /* 430 */ 2954, 75, 2130, 2131, 2077, 339, 2658, 2637, 827, 634, - /* 440 */ 633, 631, 630, 629, 624, 623, 622, 621, 450, 702, - /* 450 */ 2026, 611, 610, 609, 608, 607, 601, 600, 599, 510, - /* 460 */ 594, 593, 465, 2045, 706, 220, 585, 1853, 1854, 2078, - /* 470 */ 220, 2080, 2090, 1872, 821, 820, 2252, 2253, 2255, 2256, - /* 480 */ 2257, 2129, 2132, 2978, 2191, 2192, 2193, 2978, 2978, 2978, - /* 490 */ 2978, 2978, 2651, 649, 2163, 618, 2020, 2652, 2018, 41, - /* 500 */ 617, 688, 2044, 882, 956, 53, 52, 685, 616, 59, - /* 510 */ 57, 56, 55, 54, 44, 502, 2152, 2153, 2154, 2155, - /* 520 */ 2156, 2160, 2161, 2162, 140, 2966, 2967, 2081, 168, 2971, - /* 530 */ 2754, 2023, 2024, 2077, 2845, 2079, 2082, 2083, 2084, 2085, - /* 540 */ 2086, 2087, 2088, 2089, 2091, 2092, 2093, 892, 885, 854, - /* 550 */ 3063, 917, 916, 881, 2114, 2115, 2031, 562, 3058, 2121, - /* 560 */ 2123, 2124, 2125, 2126, 2128, 2, 60, 58, 2845, 2194, - /* 570 */ 2019, 1865, 1866, 75, 506, 700, 2019, 3062, 620, 2863, - /* 580 */ 222, 3059, 3060, 897, 2017, 564, 560, 546, 1750, 2078, - /* 590 */ 2017, 1988, 2127, 619, 205, 2810, 332, 896, 1789, 1790, - /* 600 */ 75, 2328, 1563, 1741, 946, 945, 944, 1745, 943, 1747, - /* 610 */ 1748, 942, 939, 2863, 1756, 936, 1758, 1759, 933, 930, - /* 620 */ 927, 1570, 517, 516, 786, 2122, 339, 887, 884, 2810, - /* 630 */ 2049, 896, 19, 2025, 137, 2029, 915, 2521, 886, 2025, - /* 640 */ 701, 2671, 2844, 526, 2523, 2895, 1565, 1568, 1569, 134, - /* 650 */ 2846, 900, 2848, 2849, 895, 12, 530, 883, 2896, 919, - /* 660 */ 915, 2521, 2810, 1022, 213, 117, 2958, 590, 2647, 1022, - /* 670 */ 500, 2954, 15, 572, 971, 571, 2844, 915, 2521, 2895, - /* 680 */ 161, 2845, 521, 135, 2846, 900, 2848, 2849, 895, 731, - /* 690 */ 842, 883, 2896, 919, 487, 2725, 897, 236, 3006, 1920, - /* 700 */ 2958, 2503, 473, 2194, 2957, 2954, 570, 693, 2717, 2130, - /* 710 */ 2131, 2575, 2576, 2978, 2191, 2192, 2193, 2978, 2978, 2978, - /* 720 */ 2978, 2978, 247, 53, 52, 783, 2863, 59, 57, 56, - /* 730 */ 55, 54, 752, 751, 750, 851, 170, 915, 2521, 742, - /* 740 */ 167, 746, 2810, 2025, 896, 745, 973, 2973, 2080, 2090, - /* 750 */ 744, 749, 480, 479, 915, 2521, 743, 161, 2129, 2132, - /* 760 */ 478, 739, 738, 737, 1597, 1598, 736, 2020, 339, 2018, - /* 770 */ 915, 2521, 43, 2020, 68, 2018, 2970, 2048, 53, 52, - /* 780 */ 882, 389, 59, 57, 56, 55, 54, 267, 2047, 2844, - /* 790 */ 578, 686, 2895, 2362, 2496, 339, 199, 2846, 900, 2848, - /* 800 */ 2849, 895, 2023, 2024, 883, 2896, 919, 2304, 2023, 2024, - /* 810 */ 2077, 2845, 2079, 2082, 2083, 2084, 2085, 2086, 2087, 2088, - /* 820 */ 2089, 2091, 2092, 2093, 892, 885, 897, 176, 917, 916, - /* 830 */ 881, 2114, 2115, 947, 2265, 2044, 2121, 2123, 2124, 2125, - /* 840 */ 2126, 2128, 2, 12, 60, 58, 803, 3016, 2191, 2192, - /* 850 */ 2193, 269, 506, 195, 2019, 686, 2863, 2362, 786, 1686, - /* 860 */ 1750, 201, 2966, 2967, 2527, 168, 2971, 1987, 2017, 198, - /* 870 */ 2127, 2318, 2810, 2508, 896, 1741, 946, 945, 944, 1745, - /* 880 */ 943, 1747, 1748, 891, 890, 2581, 1756, 889, 1758, 1759, - /* 890 */ 888, 930, 927, 464, 596, 2647, 2973, 539, 520, 519, - /* 900 */ 346, 347, 538, 2122, 2579, 345, 884, 638, 1688, 396, - /* 910 */ 19, 3019, 2196, 2197, 2198, 2199, 2200, 2025, 1681, 2844, - /* 920 */ 915, 2521, 2895, 627, 2647, 2969, 135, 2846, 900, 2848, - /* 930 */ 2849, 895, 915, 2521, 883, 2896, 919, 2506, 915, 2521, - /* 940 */ 579, 788, 2717, 2958, 2845, 915, 2521, 1022, 2955, 249, - /* 950 */ 15, 2327, 598, 226, 802, 2581, 915, 2521, 612, 897, - /* 960 */ 2764, 2364, 3058, 496, 337, 613, 190, 1682, 703, 486, - /* 970 */ 2725, 409, 533, 532, 2579, 256, 614, 1594, 254, 1593, - /* 980 */ 3064, 231, 2033, 2581, 839, 3059, 840, 2130, 2131, 2863, - /* 990 */ 407, 89, 3058, 395, 88, 2326, 2026, 12, 2127, 10, - /* 1000 */ 637, 255, 858, 470, 2495, 2810, 314, 896, 447, 2081, - /* 1010 */ 838, 231, 2810, 2049, 635, 3059, 840, 802, 1595, 402, - /* 1020 */ 265, 661, 659, 656, 654, 3058, 2080, 2090, 2665, 974, - /* 1030 */ 704, 2122, 2478, 1702, 53, 52, 2129, 2132, 59, 57, - /* 1040 */ 56, 55, 54, 3064, 231, 2025, 2581, 1701, 3059, 840, - /* 1050 */ 91, 2020, 2844, 2018, 511, 2895, 2810, 2048, 882, 134, - /* 1060 */ 2846, 900, 2848, 2849, 895, 2579, 75, 883, 2896, 919, - /* 1070 */ 875, 2078, 2930, 2325, 3078, 879, 2958, 56, 55, 54, - /* 1080 */ 500, 2954, 14, 13, 574, 1570, 2023, 2024, 2077, 573, - /* 1090 */ 2079, 2082, 2083, 2084, 2085, 2086, 2087, 2088, 2089, 2091, - /* 1100 */ 2092, 2093, 892, 885, 104, 76, 917, 916, 881, 2114, - /* 1110 */ 2115, 1568, 1569, 2324, 2121, 2123, 2124, 2125, 2126, 2128, - /* 1120 */ 2, 60, 58, 2133, 915, 2521, 651, 2323, 2525, 506, - /* 1130 */ 2322, 2019, 53, 52, 2810, 2370, 59, 57, 56, 55, - /* 1140 */ 54, 802, 306, 775, 705, 2017, 305, 2127, 171, 3058, - /* 1150 */ 759, 2929, 954, 187, 186, 951, 950, 949, 184, 101, - /* 1160 */ 100, 577, 915, 2521, 244, 773, 509, 3064, 231, 2492, - /* 1170 */ 2321, 40, 3059, 840, 2810, 2319, 195, 569, 567, 2034, - /* 1180 */ 2122, 2029, 2516, 884, 304, 725, 724, 2526, 2810, 444, - /* 1190 */ 2405, 2810, 556, 1009, 2025, 552, 548, 544, 541, 570, - /* 1200 */ 2171, 53, 52, 762, 2246, 59, 57, 56, 55, 54, - /* 1210 */ 756, 754, 915, 2521, 2037, 2039, 948, 301, 2247, 2572, - /* 1220 */ 116, 2845, 727, 726, 1022, 453, 34, 61, 485, 2581, - /* 1230 */ 774, 2810, 309, 234, 917, 916, 897, 527, 3013, 748, - /* 1240 */ 747, 2320, 2121, 2123, 2124, 2125, 2126, 2128, 2579, 915, - /* 1250 */ 2521, 53, 52, 915, 2521, 59, 57, 56, 55, 54, - /* 1260 */ 2216, 339, 2453, 85, 2130, 2131, 2863, 2245, 84, 317, - /* 1270 */ 752, 751, 750, 857, 2044, 205, 809, 742, 167, 746, - /* 1280 */ 97, 801, 2810, 745, 896, 915, 2521, 2317, 744, 749, - /* 1290 */ 480, 479, 822, 2049, 743, 915, 2521, 173, 478, 739, - /* 1300 */ 738, 737, 2810, 2080, 2090, 350, 915, 2521, 2511, 877, - /* 1310 */ 833, 2930, 2266, 2129, 2132, 871, 553, 954, 187, 186, - /* 1320 */ 951, 950, 949, 184, 126, 776, 357, 2316, 2020, 2844, - /* 1330 */ 2018, 2159, 2895, 985, 983, 882, 134, 2846, 900, 2848, - /* 1340 */ 2849, 895, 2581, 238, 883, 2896, 919, 952, 2810, 2592, - /* 1350 */ 2572, 3078, 2512, 2958, 2493, 2315, 403, 500, 2954, 2314, - /* 1360 */ 2558, 866, 2241, 2023, 2024, 2077, 2845, 2079, 2082, 2083, - /* 1370 */ 2084, 2085, 2086, 2087, 2088, 2089, 2091, 2092, 2093, 892, - /* 1380 */ 885, 897, 2205, 917, 916, 881, 2114, 2115, 2810, 915, - /* 1390 */ 2521, 2121, 2123, 2124, 2125, 2126, 2128, 2, 60, 58, - /* 1400 */ 2845, 915, 2521, 915, 2521, 2313, 506, 2312, 2019, 908, - /* 1410 */ 2164, 2863, 915, 2521, 2311, 897, 2810, 3026, 396, 2310, - /* 1420 */ 2810, 909, 2017, 913, 2127, 73, 2309, 2810, 185, 896, - /* 1430 */ 42, 956, 385, 53, 52, 2973, 799, 59, 57, 56, - /* 1440 */ 55, 54, 53, 52, 529, 2863, 59, 57, 56, 55, - /* 1450 */ 54, 2581, 953, 1572, 195, 2572, 2630, 2122, 735, 2043, - /* 1460 */ 884, 2810, 734, 896, 2968, 2526, 2810, 113, 2810, 2583, - /* 1470 */ 2580, 2025, 2731, 294, 2844, 2810, 292, 2895, 177, 2726, - /* 1480 */ 2810, 413, 2846, 900, 2848, 2849, 895, 2810, 32, 883, - /* 1490 */ 2896, 919, 196, 2334, 1015, 3, 2515, 333, 2845, 843, - /* 1500 */ 588, 1022, 2524, 846, 61, 2112, 922, 632, 2844, 66, - /* 1510 */ 2078, 2895, 2027, 897, 67, 134, 2846, 900, 2848, 2849, - /* 1520 */ 895, 163, 177, 883, 2896, 919, 296, 852, 298, 295, - /* 1530 */ 3078, 297, 2958, 185, 308, 103, 500, 2954, 2389, 740, - /* 1540 */ 741, 2130, 2131, 2863, 2104, 300, 2298, 2299, 299, 325, - /* 1550 */ 53, 52, 812, 2235, 59, 57, 56, 55, 54, 2810, - /* 1560 */ 753, 896, 1679, 1677, 1936, 14, 13, 165, 53, 52, - /* 1570 */ 880, 2215, 59, 57, 56, 55, 54, 2864, 53, 52, - /* 1580 */ 2080, 2090, 59, 57, 56, 55, 54, 2444, 1658, 47, - /* 1590 */ 2129, 2132, 53, 52, 62, 62, 59, 57, 56, 55, - /* 1600 */ 54, 132, 214, 129, 62, 2020, 2844, 2018, 1944, 2895, - /* 1610 */ 2387, 2378, 882, 134, 2846, 900, 2848, 2849, 895, 856, - /* 1620 */ 344, 883, 2896, 919, 364, 363, 2831, 647, 2933, 2443, - /* 1630 */ 2958, 2355, 755, 757, 500, 2954, 2376, 1659, 2239, 3009, - /* 1640 */ 2023, 2024, 2077, 819, 2079, 2082, 2083, 2084, 2085, 2086, - /* 1650 */ 2087, 2088, 2089, 2091, 2092, 2093, 892, 885, 760, 492, - /* 1660 */ 917, 916, 881, 2114, 2115, 488, 90, 62, 2121, 2123, - /* 1670 */ 2124, 2125, 2126, 2128, 2, 60, 58, 2845, 77, 2100, - /* 1680 */ 2251, 2250, 779, 506, 826, 2019, 62, 860, 323, 2102, - /* 1690 */ 2113, 62, 897, 2833, 816, 366, 365, 2030, 46, 2017, - /* 1700 */ 62, 2127, 2656, 2105, 90, 537, 348, 53, 52, 555, - /* 1710 */ 39, 59, 57, 56, 55, 54, 2361, 923, 2569, 53, - /* 1720 */ 52, 795, 2863, 59, 57, 56, 55, 54, 2845, 87, - /* 1730 */ 368, 367, 370, 369, 2122, 372, 371, 884, 2810, 802, - /* 1740 */ 896, 182, 163, 897, 3010, 185, 2149, 3058, 2025, 374, - /* 1750 */ 373, 965, 863, 2103, 376, 375, 378, 377, 48, 380, - /* 1760 */ 379, 382, 381, 925, 2165, 3064, 231, 384, 383, 183, - /* 1770 */ 3059, 840, 2106, 2863, 1650, 966, 185, 2094, 1022, 3020, - /* 1780 */ 335, 61, 834, 835, 1631, 2844, 1889, 330, 2895, 2810, - /* 1790 */ 1879, 896, 134, 2846, 900, 2848, 2849, 895, 1648, 164, - /* 1800 */ 883, 2896, 919, 2657, 844, 338, 2479, 3078, 540, 2958, - /* 1810 */ 847, 182, 5, 500, 2954, 401, 53, 52, 2130, 2131, - /* 1820 */ 59, 57, 56, 55, 54, 545, 462, 360, 912, 2042, - /* 1830 */ 2099, 1732, 554, 1632, 2052, 566, 2844, 565, 239, 2895, - /* 1840 */ 1913, 582, 240, 135, 2846, 900, 2848, 2849, 895, 1763, - /* 1850 */ 568, 883, 2896, 919, 394, 1771, 242, 2080, 2090, 2043, - /* 1860 */ 2958, 589, 1778, 253, 878, 2954, 591, 2129, 2132, 53, - /* 1870 */ 52, 597, 595, 59, 57, 56, 55, 54, 640, 602, - /* 1880 */ 615, 2649, 2020, 628, 2018, 1776, 53, 52, 626, 882, - /* 1890 */ 59, 57, 56, 55, 54, 636, 639, 188, 652, 477, - /* 1900 */ 475, 641, 650, 653, 259, 258, 657, 655, 658, 262, - /* 1910 */ 660, 662, 2050, 681, 4, 682, 689, 2023, 2024, 2077, - /* 1920 */ 692, 2079, 2082, 2083, 2084, 2085, 2086, 2087, 2088, 2089, - /* 1930 */ 2091, 2092, 2093, 892, 885, 690, 270, 917, 916, 881, - /* 1940 */ 2114, 2115, 2045, 787, 109, 2121, 2123, 2124, 2125, 2126, - /* 1950 */ 2128, 2, 60, 58, 694, 273, 2051, 695, 2053, 696, - /* 1960 */ 506, 276, 2019, 698, 855, 278, 2054, 2672, 110, 2055, - /* 1970 */ 1730, 49, 2666, 111, 2845, 730, 2017, 284, 2127, 707, - /* 1980 */ 2101, 112, 474, 472, 114, 733, 2740, 287, 763, 897, - /* 1990 */ 764, 3051, 2737, 732, 2509, 139, 291, 2098, 439, 778, - /* 2000 */ 802, 2736, 2505, 780, 138, 2845, 735, 115, 3058, 178, - /* 2010 */ 734, 2122, 2046, 397, 884, 293, 191, 136, 2507, 2863, - /* 2020 */ 897, 802, 2502, 192, 2718, 2025, 3064, 231, 193, 3058, - /* 2030 */ 310, 3059, 840, 791, 789, 2810, 790, 896, 794, 315, - /* 2040 */ 823, 313, 3025, 796, 797, 806, 861, 3064, 231, 3024, - /* 2050 */ 2863, 832, 3059, 840, 8, 1022, 2845, 2997, 61, 204, - /* 2060 */ 807, 320, 326, 322, 324, 805, 2810, 804, 896, 837, - /* 2070 */ 327, 897, 328, 2990, 329, 836, 493, 2977, 331, 848, - /* 2080 */ 334, 3081, 2844, 845, 3057, 2895, 169, 2047, 2213, 134, - /* 2090 */ 2846, 900, 2848, 2849, 895, 2130, 2131, 883, 2896, 919, - /* 2100 */ 2211, 2863, 2974, 217, 3078, 340, 2958, 179, 398, 859, - /* 2110 */ 500, 2954, 2686, 898, 1, 2685, 2895, 2810, 2684, 896, - /* 2120 */ 135, 2846, 900, 2848, 2849, 895, 497, 399, 883, 2896, - /* 2130 */ 919, 864, 180, 869, 2080, 2090, 865, 2958, 872, 353, - /* 2140 */ 74, 457, 2954, 233, 2129, 2132, 2939, 902, 906, 400, - /* 2150 */ 904, 2802, 907, 2801, 358, 2797, 2796, 2788, 125, 2020, - /* 2160 */ 2522, 2018, 128, 2787, 2844, 387, 882, 2895, 1542, 2779, - /* 2170 */ 921, 134, 2846, 900, 2848, 2849, 895, 2778, 404, 883, - /* 2180 */ 2896, 919, 2794, 1018, 1019, 2793, 3078, 782, 2958, 1017, - /* 2190 */ 1014, 406, 500, 2954, 2023, 2024, 2077, 189, 2079, 2082, - /* 2200 */ 2083, 2084, 2085, 2086, 2087, 2088, 2089, 2091, 2092, 2093, - /* 2210 */ 892, 885, 2785, 390, 917, 916, 881, 2114, 2115, 391, - /* 2220 */ 2784, 1021, 2121, 2123, 2124, 2125, 2126, 2128, 2, 60, - /* 2230 */ 58, 440, 2773, 2772, 2791, 476, 2790, 506, 2782, 2019, - /* 2240 */ 2781, 65, 2770, 2769, 2767, 2766, 427, 2573, 454, 455, - /* 2250 */ 438, 2845, 524, 2017, 428, 2127, 408, 2762, 2761, 2760, - /* 2260 */ 98, 2755, 542, 543, 1970, 547, 897, 1971, 237, 2753, - /* 2270 */ 549, 550, 551, 1969, 2752, 2751, 463, 2749, 557, 2748, - /* 2280 */ 559, 2747, 561, 2746, 563, 1957, 2722, 241, 2122, 2721, - /* 2290 */ 243, 884, 99, 1916, 1915, 2699, 2863, 2698, 2697, 575, - /* 2300 */ 576, 2696, 2025, 2695, 2639, 580, 1852, 2636, 583, 2635, - /* 2310 */ 2629, 2626, 2810, 587, 896, 246, 586, 2625, 102, 2624, - /* 2320 */ 2623, 2628, 2627, 2622, 248, 2621, 2619, 449, 448, 2618, - /* 2330 */ 2617, 250, 1022, 2845, 2616, 15, 603, 513, 605, 2614, - /* 2340 */ 2613, 2612, 2611, 2610, 2634, 2609, 2608, 2607, 897, 2632, - /* 2350 */ 2615, 522, 2606, 2127, 2605, 2604, 2602, 2601, 2600, 2844, - /* 2360 */ 2599, 2598, 2895, 2597, 252, 2596, 134, 2846, 900, 2848, - /* 2370 */ 2849, 895, 2130, 2131, 883, 2896, 919, 2595, 2863, 108, - /* 2380 */ 2594, 2931, 2593, 2958, 2591, 2590, 2122, 500, 2954, 2664, - /* 2390 */ 2633, 2631, 2589, 2588, 2810, 1858, 896, 2587, 2586, 257, - /* 2400 */ 643, 2585, 645, 2584, 2582, 1699, 2409, 260, 1703, 2408, - /* 2410 */ 1695, 2080, 2090, 2407, 451, 2406, 452, 2404, 2401, 261, - /* 2420 */ 665, 2129, 2132, 663, 263, 664, 2400, 668, 2393, 672, - /* 2430 */ 264, 667, 669, 2380, 2368, 671, 2020, 677, 2018, 2367, - /* 2440 */ 673, 2844, 2350, 882, 2895, 2830, 675, 679, 134, 2846, - /* 2450 */ 900, 2848, 2849, 895, 266, 212, 883, 2896, 919, 223, - /* 2460 */ 94, 1571, 2349, 876, 268, 2958, 2720, 2716, 95, 500, - /* 2470 */ 2954, 2023, 2024, 2077, 687, 2079, 2082, 2083, 2084, 2085, - /* 2480 */ 2086, 2087, 2088, 2089, 2091, 2092, 2093, 892, 885, 784, - /* 2490 */ 2706, 917, 916, 881, 2114, 2115, 2694, 275, 277, 2121, - /* 2500 */ 2123, 2124, 2125, 2126, 2128, 2, 2693, 1025, 280, 2670, - /* 2510 */ 282, 2663, 2497, 2403, 1624, 2845, 2399, 708, 709, 710, - /* 2520 */ 2397, 712, 713, 714, 2395, 716, 393, 717, 718, 2392, - /* 2530 */ 897, 720, 2375, 721, 2010, 722, 1986, 2373, 2374, 2372, - /* 2540 */ 2369, 1013, 1011, 728, 2346, 221, 2499, 290, 86, 1782, - /* 2550 */ 1783, 2498, 1685, 1684, 1007, 1003, 999, 995, 1683, 388, - /* 2560 */ 2863, 1680, 1678, 2390, 1676, 2388, 982, 515, 514, 2011, - /* 2570 */ 1675, 1674, 1673, 2379, 1667, 984, 2810, 1672, 896, 1669, - /* 2580 */ 481, 1668, 482, 1666, 483, 2377, 484, 2345, 2344, 917, - /* 2590 */ 916, 761, 2343, 765, 2342, 767, 2341, 2121, 2123, 2124, - /* 2600 */ 2125, 2126, 2128, 769, 2340, 133, 771, 758, 1951, 141, - /* 2610 */ 361, 1953, 1950, 1955, 2719, 33, 80, 312, 2715, 1922, - /* 2620 */ 1924, 2845, 2705, 2844, 792, 69, 2895, 70, 1926, 316, - /* 2630 */ 202, 2846, 900, 2848, 2849, 895, 897, 1941, 883, 2896, - /* 2640 */ 919, 867, 793, 2692, 2845, 781, 2691, 194, 1901, 798, - /* 2650 */ 1900, 800, 3063, 22, 808, 17, 23, 811, 6, 897, - /* 2660 */ 489, 7, 24, 216, 228, 2206, 2863, 35, 2831, 2268, - /* 2670 */ 38, 319, 229, 2242, 36, 78, 2208, 26, 2283, 336, - /* 2680 */ 2240, 2282, 2810, 494, 896, 817, 359, 813, 873, 2863, - /* 2690 */ 815, 342, 841, 3079, 2287, 203, 341, 321, 2249, 215, - /* 2700 */ 37, 2286, 495, 72, 96, 2810, 503, 896, 208, 2234, - /* 2710 */ 2204, 230, 2690, 2669, 2288, 311, 2289, 119, 25, 2188, - /* 2720 */ 2845, 120, 2187, 2668, 343, 121, 2244, 2662, 218, 2844, - /* 2730 */ 349, 122, 2895, 82, 71, 897, 437, 2846, 900, 2848, - /* 2740 */ 2849, 895, 27, 352, 883, 2896, 919, 13, 18, 209, - /* 2750 */ 868, 2140, 2844, 11, 2139, 2895, 2035, 2150, 219, 199, - /* 2760 */ 2846, 900, 2848, 2849, 895, 2863, 21, 883, 2896, 919, - /* 2770 */ 28, 2845, 29, 351, 20, 2070, 899, 50, 2661, 123, - /* 2780 */ 2097, 2810, 2096, 896, 129, 932, 897, 935, 2494, 938, - /* 2790 */ 910, 941, 51, 362, 2095, 2062, 16, 30, 31, 2302, - /* 2800 */ 862, 83, 1755, 903, 2845, 490, 356, 870, 124, 354, - /* 2810 */ 3017, 92, 911, 2908, 2907, 905, 2863, 924, 2109, 897, - /* 2820 */ 918, 81, 525, 920, 1764, 1761, 926, 928, 2844, 929, - /* 2830 */ 931, 2895, 2810, 1760, 896, 437, 2846, 900, 2848, 2849, - /* 2840 */ 895, 1757, 934, 883, 2896, 919, 937, 1751, 1749, 2863, - /* 2850 */ 940, 2301, 386, 1777, 130, 1754, 491, 1753, 1752, 131, - /* 2860 */ 1773, 93, 1622, 955, 1663, 2810, 1662, 896, 1661, 2845, - /* 2870 */ 1660, 1657, 1654, 1653, 1693, 1652, 1651, 1649, 1692, 2844, - /* 2880 */ 1640, 1647, 2895, 1646, 897, 1645, 437, 2846, 900, 2848, - /* 2890 */ 2849, 895, 2845, 235, 883, 2896, 919, 1643, 1642, 1641, - /* 2900 */ 1639, 1638, 970, 1637, 972, 1689, 1687, 894, 1634, 1633, - /* 2910 */ 1630, 1629, 2844, 1628, 2863, 2895, 1627, 2398, 992, 430, - /* 2920 */ 2846, 900, 2848, 2849, 895, 993, 2396, 883, 2896, 919, - /* 2930 */ 2810, 994, 896, 996, 997, 2394, 998, 2863, 1000, 1002, - /* 2940 */ 1001, 2391, 1004, 1005, 1006, 2371, 1008, 2366, 2365, 1010, - /* 2950 */ 1012, 1560, 2339, 2810, 1543, 896, 1016, 1548, 1550, 392, - /* 2960 */ 2845, 1020, 2305, 2305, 2021, 405, 2305, 1023, 2305, 1024, - /* 2970 */ 2305, 2305, 2305, 831, 2305, 897, 2305, 2844, 2305, 2305, - /* 2980 */ 2895, 2305, 2305, 2305, 202, 2846, 900, 2848, 2849, 895, - /* 2990 */ 2305, 2305, 883, 2896, 919, 2305, 2305, 2305, 2305, 2305, - /* 3000 */ 2844, 2305, 2305, 2895, 2305, 2863, 2305, 436, 2846, 900, - /* 3010 */ 2848, 2849, 895, 2305, 2305, 883, 2896, 919, 2305, 2924, - /* 3020 */ 2305, 2810, 2305, 896, 2305, 2845, 2305, 289, 2305, 2305, - /* 3030 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, - /* 3040 */ 897, 2305, 2305, 729, 2305, 504, 206, 3080, 2305, 2305, - /* 3050 */ 2305, 2305, 2845, 2305, 2305, 723, 719, 715, 711, 2305, - /* 3060 */ 288, 2305, 2305, 2305, 2305, 2305, 2305, 897, 2844, 2305, - /* 3070 */ 2863, 2895, 2305, 2305, 2305, 437, 2846, 900, 2848, 2849, - /* 3080 */ 895, 2305, 2305, 883, 2896, 919, 2810, 2305, 896, 2305, - /* 3090 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2863, 2305, 2305, - /* 3100 */ 2305, 2305, 2305, 2305, 2305, 2305, 286, 2305, 2305, 2305, - /* 3110 */ 499, 285, 2305, 2810, 2305, 896, 2305, 2305, 2305, 2305, - /* 3120 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, - /* 3130 */ 2305, 2305, 2305, 2844, 2305, 2305, 2895, 505, 2845, 2305, - /* 3140 */ 422, 2846, 900, 2848, 2849, 895, 2305, 2305, 883, 2896, - /* 3150 */ 919, 2305, 2305, 897, 2305, 2305, 2305, 2305, 2845, 2305, - /* 3160 */ 2844, 2305, 2305, 2895, 2305, 2305, 2305, 437, 2846, 900, - /* 3170 */ 2848, 2849, 895, 897, 2305, 883, 2896, 919, 272, 2845, - /* 3180 */ 2305, 2305, 2305, 2863, 2305, 2305, 2305, 283, 2305, 2305, - /* 3190 */ 2305, 274, 281, 2305, 897, 2305, 2305, 279, 697, 2810, - /* 3200 */ 2305, 896, 2305, 2863, 2305, 2305, 2305, 2305, 2305, 2305, - /* 3210 */ 2305, 2305, 2305, 2305, 2305, 2305, 271, 2305, 2305, 2810, - /* 3220 */ 2305, 896, 2305, 507, 2863, 2305, 2305, 2305, 2305, 2305, - /* 3230 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, - /* 3240 */ 2810, 2305, 896, 2305, 2305, 2305, 2844, 2305, 2305, 2895, - /* 3250 */ 2305, 2305, 2305, 437, 2846, 900, 2848, 2849, 895, 2305, - /* 3260 */ 2305, 883, 2896, 919, 2305, 2305, 2844, 2305, 2305, 2895, - /* 3270 */ 2305, 2305, 2305, 418, 2846, 900, 2848, 2849, 895, 2305, - /* 3280 */ 2305, 883, 2896, 919, 2305, 2845, 2305, 777, 2305, 2305, - /* 3290 */ 2895, 2305, 2305, 2305, 432, 2846, 900, 2848, 2849, 895, - /* 3300 */ 897, 2305, 883, 2896, 919, 2845, 2305, 2305, 2305, 2305, - /* 3310 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, - /* 3320 */ 897, 2305, 2305, 2305, 2305, 2845, 2305, 2305, 2305, 2305, - /* 3330 */ 2863, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, - /* 3340 */ 897, 2305, 2305, 2305, 2305, 2305, 2810, 2305, 896, 2305, - /* 3350 */ 2863, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, - /* 3360 */ 2305, 2305, 2305, 2305, 2305, 2305, 2810, 2305, 896, 2305, - /* 3370 */ 2863, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, - /* 3380 */ 2305, 2305, 2305, 2305, 2305, 2305, 2810, 2305, 896, 2305, - /* 3390 */ 2305, 2305, 2305, 2844, 2305, 2305, 2895, 2305, 2305, 2305, - /* 3400 */ 414, 2846, 900, 2848, 2849, 895, 2305, 2305, 883, 2896, - /* 3410 */ 919, 2305, 2845, 2844, 2305, 2305, 2895, 2305, 2305, 2305, - /* 3420 */ 410, 2846, 900, 2848, 2849, 895, 2305, 897, 883, 2896, - /* 3430 */ 919, 2305, 2305, 2844, 2305, 2305, 2895, 2305, 2845, 2305, - /* 3440 */ 411, 2846, 900, 2848, 2849, 895, 2305, 2305, 883, 2896, - /* 3450 */ 919, 2305, 2305, 897, 2305, 2305, 2305, 2863, 2305, 2305, - /* 3460 */ 2305, 2845, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, - /* 3470 */ 2305, 2305, 2305, 2810, 2305, 896, 897, 2305, 2305, 2305, - /* 3480 */ 2305, 2845, 2305, 2863, 2305, 2305, 2305, 2305, 2305, 2305, - /* 3490 */ 2305, 2305, 2305, 2305, 2305, 2305, 897, 2305, 2305, 2810, - /* 3500 */ 2305, 896, 2305, 2845, 2305, 2305, 2863, 2305, 2305, 2305, - /* 3510 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 897, 2305, - /* 3520 */ 2844, 2305, 2810, 2895, 896, 2305, 2863, 415, 2846, 900, - /* 3530 */ 2848, 2849, 895, 2305, 2305, 883, 2896, 919, 2305, 2305, - /* 3540 */ 2305, 2305, 2810, 2305, 896, 2305, 2844, 2305, 2863, 2895, - /* 3550 */ 2305, 2305, 2305, 429, 2846, 900, 2848, 2849, 895, 2305, - /* 3560 */ 2305, 883, 2896, 919, 2810, 2305, 896, 2305, 2845, 2844, - /* 3570 */ 2305, 2305, 2895, 2305, 2305, 2305, 416, 2846, 900, 2848, - /* 3580 */ 2849, 895, 2305, 897, 883, 2896, 919, 2845, 2305, 2844, - /* 3590 */ 2305, 2305, 2895, 2305, 2305, 2305, 417, 2846, 900, 2848, - /* 3600 */ 2849, 895, 897, 2305, 883, 2896, 919, 2305, 2305, 2845, - /* 3610 */ 2305, 2844, 2305, 2863, 2895, 2305, 2305, 2305, 433, 2846, - /* 3620 */ 900, 2848, 2849, 895, 897, 2305, 883, 2896, 919, 2810, - /* 3630 */ 2305, 896, 2863, 2305, 2305, 2305, 2305, 2305, 2305, 2305, - /* 3640 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2810, 2305, - /* 3650 */ 896, 2305, 2305, 2305, 2863, 2305, 2305, 2305, 2305, 2305, - /* 3660 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, - /* 3670 */ 2810, 2305, 896, 2305, 2845, 2305, 2844, 2305, 2305, 2895, - /* 3680 */ 2305, 2305, 2305, 419, 2846, 900, 2848, 2849, 895, 897, - /* 3690 */ 2305, 883, 2896, 919, 2305, 2844, 2305, 2305, 2895, 2305, - /* 3700 */ 2305, 2305, 434, 2846, 900, 2848, 2849, 895, 2305, 2305, - /* 3710 */ 883, 2896, 919, 2305, 2305, 2305, 2305, 2844, 2305, 2863, - /* 3720 */ 2895, 2305, 2305, 2845, 420, 2846, 900, 2848, 2849, 895, - /* 3730 */ 2305, 2305, 883, 2896, 919, 2810, 2305, 896, 897, 2305, - /* 3740 */ 2305, 2305, 2305, 2845, 2305, 2305, 2305, 2305, 2305, 2305, - /* 3750 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 897, 2305, - /* 3760 */ 2305, 2305, 2305, 2845, 2305, 2305, 2305, 2305, 2863, 2305, - /* 3770 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 897, 2305, - /* 3780 */ 2305, 2305, 2844, 2305, 2810, 2895, 896, 2305, 2863, 435, - /* 3790 */ 2846, 900, 2848, 2849, 895, 2305, 2305, 883, 2896, 919, - /* 3800 */ 2305, 2305, 2305, 2305, 2810, 2305, 896, 2305, 2863, 2305, - /* 3810 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, - /* 3820 */ 2305, 2305, 2305, 2305, 2810, 2305, 896, 2305, 2305, 2305, - /* 3830 */ 2305, 2844, 2305, 2305, 2895, 2305, 2305, 2305, 421, 2846, - /* 3840 */ 900, 2848, 2849, 895, 2305, 2305, 883, 2896, 919, 2305, - /* 3850 */ 2845, 2844, 2305, 2305, 2895, 2305, 2305, 2305, 412, 2846, - /* 3860 */ 900, 2848, 2849, 895, 2305, 897, 883, 2896, 919, 2845, - /* 3870 */ 2305, 2844, 2305, 2305, 2895, 2305, 2305, 2305, 423, 2846, - /* 3880 */ 900, 2848, 2849, 895, 897, 2305, 883, 2896, 919, 2845, - /* 3890 */ 2305, 2305, 2305, 2305, 2305, 2863, 2305, 2305, 2305, 2305, - /* 3900 */ 2305, 2305, 2305, 2305, 897, 2305, 2305, 2305, 2305, 2305, - /* 3910 */ 2305, 2810, 2305, 896, 2863, 2305, 2305, 2305, 2305, 2305, - /* 3920 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, - /* 3930 */ 2810, 2305, 896, 2305, 2863, 2305, 2305, 2305, 2305, 2305, - /* 3940 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, - /* 3950 */ 2810, 2305, 896, 2305, 2305, 2305, 2845, 2305, 2844, 2305, - /* 3960 */ 2305, 2895, 2305, 2305, 2305, 424, 2846, 900, 2848, 2849, - /* 3970 */ 895, 897, 2305, 883, 2896, 919, 2845, 2844, 2305, 2305, - /* 3980 */ 2895, 2305, 2305, 2305, 425, 2846, 900, 2848, 2849, 895, - /* 3990 */ 2305, 897, 883, 2896, 919, 2845, 2305, 2844, 2305, 2305, - /* 4000 */ 2895, 2863, 2305, 2305, 426, 2846, 900, 2848, 2849, 895, - /* 4010 */ 897, 2305, 883, 2896, 919, 2305, 2305, 2810, 2305, 896, - /* 4020 */ 2305, 2863, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, - /* 4030 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2810, 2305, 896, - /* 4040 */ 2863, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, - /* 4050 */ 2305, 2305, 2305, 2305, 2305, 2305, 2810, 2305, 896, 2305, - /* 4060 */ 2305, 2305, 2305, 2845, 2844, 2305, 2305, 2895, 2305, 2305, - /* 4070 */ 2305, 442, 2846, 900, 2848, 2849, 895, 2305, 897, 883, - /* 4080 */ 2896, 919, 2845, 2305, 2844, 2305, 2305, 2895, 2305, 2305, - /* 4090 */ 2305, 443, 2846, 900, 2848, 2849, 895, 897, 2305, 883, - /* 4100 */ 2896, 919, 2305, 2844, 2305, 2305, 2895, 2305, 2863, 2305, - /* 4110 */ 2857, 2846, 900, 2848, 2849, 895, 2305, 2305, 883, 2896, - /* 4120 */ 919, 2305, 2305, 2305, 2810, 2305, 896, 2863, 2305, 2305, - /* 4130 */ 2305, 2845, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, - /* 4140 */ 2305, 2305, 2305, 2810, 2305, 896, 897, 2305, 2305, 2305, - /* 4150 */ 2305, 2845, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, - /* 4160 */ 2305, 2305, 2305, 2305, 2305, 2305, 897, 2305, 2305, 2305, - /* 4170 */ 2305, 2844, 2305, 2305, 2895, 2305, 2863, 2305, 2856, 2846, - /* 4180 */ 900, 2848, 2849, 895, 2305, 2305, 883, 2896, 919, 2305, - /* 4190 */ 2844, 2305, 2810, 2895, 896, 2305, 2863, 2855, 2846, 900, - /* 4200 */ 2848, 2849, 895, 2305, 2305, 883, 2896, 919, 2305, 2305, - /* 4210 */ 2305, 2305, 2810, 2305, 896, 2305, 2305, 2305, 2305, 2845, - /* 4220 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, - /* 4230 */ 2305, 2305, 2305, 2305, 897, 2305, 2305, 2305, 2305, 2844, - /* 4240 */ 2305, 2305, 2895, 2845, 2305, 2305, 459, 2846, 900, 2848, - /* 4250 */ 2849, 895, 2305, 2305, 883, 2896, 919, 2305, 897, 2844, - /* 4260 */ 2305, 2305, 2895, 2305, 2863, 2305, 460, 2846, 900, 2848, - /* 4270 */ 2849, 895, 2305, 2305, 883, 2896, 919, 2305, 2305, 2305, - /* 4280 */ 2810, 2305, 896, 2305, 2305, 2305, 2305, 2305, 2863, 2305, - /* 4290 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, - /* 4300 */ 2305, 2305, 2305, 2305, 2810, 2305, 896, 2305, 2305, 2305, - /* 4310 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, - /* 4320 */ 2305, 2305, 2305, 2305, 2305, 2845, 2305, 2844, 2305, 2305, - /* 4330 */ 2895, 2305, 2305, 2305, 456, 2846, 900, 2848, 2849, 895, - /* 4340 */ 897, 2305, 883, 2896, 919, 2305, 2305, 2305, 2305, 2845, - /* 4350 */ 2305, 2844, 2305, 2305, 2895, 2305, 2305, 2305, 461, 2846, - /* 4360 */ 900, 2848, 2849, 895, 897, 2305, 883, 2896, 919, 2305, - /* 4370 */ 2863, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, - /* 4380 */ 2305, 2305, 2305, 2305, 2305, 2305, 2810, 2305, 896, 2305, - /* 4390 */ 2305, 2305, 2305, 2305, 2863, 2305, 2305, 2305, 2305, 2305, - /* 4400 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, - /* 4410 */ 2810, 2305, 896, 2305, 2305, 2305, 2305, 2305, 2305, 2305, - /* 4420 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, - /* 4430 */ 2305, 2305, 2305, 898, 2305, 2305, 2895, 2305, 2305, 2305, - /* 4440 */ 432, 2846, 900, 2848, 2849, 895, 2305, 2305, 883, 2896, - /* 4450 */ 919, 2305, 2305, 2305, 2305, 2305, 2305, 2844, 2305, 2305, - /* 4460 */ 2895, 2305, 2305, 2305, 431, 2846, 900, 2848, 2849, 895, - /* 4470 */ 2305, 2305, 883, 2896, 919, + /* 150 */ 958, 2847, 63, 2046, 2898, 2132, 2133, 2248, 134, 2849, + /* 160 */ 900, 2851, 2852, 895, 2829, 2832, 883, 2899, 919, 915, + /* 170 */ 2523, 2249, 2667, 213, 704, 2961, 2046, 2173, 919, 500, + /* 180 */ 2957, 207, 2969, 850, 256, 162, 849, 346, 347, 161, + /* 190 */ 914, 137, 345, 3061, 2082, 2092, 581, 2268, 731, 232, + /* 200 */ 526, 2525, 1866, 1867, 2131, 2134, 2106, 3008, 196, 637, + /* 210 */ 255, 838, 231, 127, 1751, 63, 3062, 840, 2526, 2022, + /* 220 */ 2247, 2020, 824, 635, 2196, 198, 882, 2320, 786, 1742, + /* 230 */ 946, 945, 944, 1746, 943, 1748, 1749, 942, 939, 64, + /* 240 */ 1757, 936, 1759, 1760, 933, 930, 927, 644, 642, 227, + /* 250 */ 441, 487, 2727, 245, 785, 2025, 2026, 2079, 2848, 2081, + /* 260 */ 2084, 2085, 2086, 2087, 2088, 2089, 2090, 2091, 2093, 2094, + /* 270 */ 2095, 892, 885, 894, 75, 917, 916, 881, 2116, 2117, + /* 280 */ 2118, 2119, 2120, 2123, 2125, 2126, 2127, 2128, 2130, 2, + /* 290 */ 60, 58, 2221, 676, 2848, 445, 706, 2044, 506, 2308, + /* 300 */ 2021, 117, 674, 2866, 625, 670, 666, 469, 196, 897, + /* 310 */ 2051, 693, 2719, 307, 2019, 646, 2129, 526, 2525, 2813, + /* 320 */ 339, 896, 2050, 160, 159, 158, 157, 156, 155, 154, + /* 330 */ 153, 152, 574, 468, 604, 1921, 648, 573, 2046, 2866, + /* 340 */ 2196, 446, 606, 830, 825, 818, 814, 810, 339, 2124, + /* 350 */ 851, 170, 884, 584, 267, 2813, 19, 896, 686, 3066, + /* 360 */ 2364, 46, 688, 2027, 75, 220, 2847, 3061, 685, 2898, + /* 370 */ 2193, 2194, 2195, 436, 2849, 900, 2851, 2852, 895, 893, + /* 380 */ 2047, 883, 2899, 919, 874, 2926, 3065, 2680, 2495, 802, + /* 390 */ 3062, 3063, 2654, 1022, 467, 699, 15, 3061, 1947, 1948, + /* 400 */ 1949, 498, 2847, 915, 2523, 2898, 592, 2677, 901, 134, + /* 410 */ 2849, 900, 2851, 2852, 895, 3067, 231, 883, 2899, 919, + /* 420 */ 3062, 840, 172, 161, 181, 2932, 2961, 533, 532, 509, + /* 430 */ 500, 2957, 736, 2132, 2133, 339, 2047, 2660, 2639, 195, + /* 440 */ 634, 633, 631, 630, 629, 624, 623, 622, 621, 451, + /* 450 */ 2528, 2028, 611, 610, 609, 608, 607, 601, 600, 599, + /* 460 */ 702, 594, 593, 465, 572, 956, 571, 585, 1854, 1855, + /* 470 */ 339, 396, 2082, 2092, 1873, 853, 200, 2969, 2970, 9, + /* 480 */ 168, 2974, 2131, 2134, 224, 2981, 2193, 2194, 2195, 2981, + /* 490 */ 2981, 2981, 2981, 2981, 3066, 41, 518, 2022, 570, 2020, + /* 500 */ 2570, 53, 52, 1687, 882, 59, 57, 56, 55, 54, + /* 510 */ 185, 2049, 2083, 53, 52, 1790, 1791, 59, 57, 56, + /* 520 */ 55, 54, 53, 52, 45, 355, 59, 57, 56, 55, + /* 530 */ 54, 486, 2727, 2025, 2026, 2079, 2848, 2081, 2084, 2085, + /* 540 */ 2086, 2087, 2088, 2089, 2090, 2091, 2093, 2094, 2095, 892, + /* 550 */ 885, 854, 1689, 917, 916, 881, 2116, 2117, 289, 2051, + /* 560 */ 339, 2123, 2125, 2126, 2127, 2128, 2130, 2, 60, 58, + /* 570 */ 783, 1893, 1894, 2049, 729, 2080, 506, 206, 2021, 477, + /* 580 */ 475, 2866, 197, 458, 2733, 1751, 723, 719, 715, 711, + /* 590 */ 2454, 288, 2019, 2196, 2129, 2407, 67, 2813, 2823, 896, + /* 600 */ 1742, 946, 945, 944, 1746, 943, 1748, 1749, 891, 890, + /* 610 */ 2291, 1757, 889, 1759, 1760, 888, 930, 927, 53, 52, + /* 620 */ 1892, 1895, 59, 57, 56, 55, 54, 2124, 2827, 1595, + /* 630 */ 884, 1594, 2766, 2159, 19, 2976, 2031, 286, 851, 170, + /* 640 */ 974, 2027, 285, 2480, 2847, 166, 308, 2898, 829, 79, + /* 650 */ 2583, 134, 2849, 900, 2851, 2852, 895, 40, 496, 883, + /* 660 */ 2899, 919, 474, 472, 2973, 733, 213, 2757, 2961, 2581, + /* 670 */ 1596, 1022, 500, 2957, 15, 752, 751, 750, 2829, 2831, + /* 680 */ 501, 1990, 742, 167, 746, 2512, 735, 337, 745, 802, + /* 690 */ 734, 2583, 919, 744, 749, 480, 479, 3061, 2160, 743, + /* 700 */ 3009, 2225, 2140, 478, 739, 738, 737, 2046, 2046, 272, + /* 710 */ 858, 2132, 2133, 517, 516, 3067, 231, 956, 283, 2527, + /* 720 */ 3062, 840, 274, 281, 546, 2298, 2505, 473, 279, 697, + /* 730 */ 190, 2806, 2680, 12, 523, 409, 915, 2523, 2981, 2193, + /* 740 */ 2194, 2195, 2981, 2981, 2981, 2981, 2981, 271, 512, 337, + /* 750 */ 2082, 2092, 2678, 901, 407, 89, 530, 1699, 88, 558, + /* 760 */ 2131, 2134, 919, 75, 140, 2969, 2970, 470, 168, 2974, + /* 770 */ 2632, 1698, 447, 2583, 2807, 2022, 948, 2020, 2165, 2574, + /* 780 */ 782, 464, 882, 2027, 265, 661, 659, 656, 654, 2256, + /* 790 */ 915, 2523, 2581, 700, 512, 2767, 2306, 2622, 44, 502, + /* 800 */ 2154, 2155, 2156, 2157, 2158, 2162, 2163, 2164, 919, 947, + /* 810 */ 236, 2025, 2026, 2079, 588, 2081, 2084, 2085, 2086, 2087, + /* 820 */ 2088, 2089, 2090, 2091, 2093, 2094, 2095, 892, 885, 2497, + /* 830 */ 75, 917, 916, 881, 2116, 2117, 12, 512, 10, 2123, + /* 840 */ 2125, 2126, 2127, 2128, 2130, 2, 12, 60, 58, 251, + /* 850 */ 649, 919, 802, 740, 2680, 506, 2297, 2021, 701, 2673, + /* 860 */ 3061, 821, 820, 2254, 2255, 2257, 2258, 2259, 508, 76, + /* 870 */ 2494, 2019, 839, 2129, 2677, 901, 1680, 2848, 3067, 231, + /* 880 */ 3061, 2083, 2083, 3062, 840, 2079, 539, 171, 915, 2523, + /* 890 */ 2932, 538, 897, 2583, 2366, 107, 880, 562, 838, 231, + /* 900 */ 106, 269, 828, 3062, 840, 686, 2124, 2364, 68, 884, + /* 910 */ 590, 2649, 866, 19, 2198, 2199, 2200, 2201, 2202, 875, + /* 920 */ 2027, 2933, 2866, 101, 100, 577, 564, 560, 244, 53, + /* 930 */ 52, 2372, 2866, 59, 57, 56, 55, 54, 2813, 2498, + /* 940 */ 896, 569, 567, 802, 2080, 2080, 915, 2523, 596, 2649, + /* 950 */ 1022, 3061, 2218, 15, 444, 627, 2649, 556, 786, 339, + /* 960 */ 552, 548, 544, 541, 570, 247, 578, 105, 47, 3067, + /* 970 */ 231, 618, 176, 2021, 3062, 840, 617, 954, 187, 186, + /* 980 */ 951, 950, 949, 184, 616, 2847, 2976, 2019, 2898, 1009, + /* 990 */ 2132, 2133, 134, 2849, 900, 2851, 2852, 895, 113, 2976, + /* 1000 */ 883, 2899, 919, 249, 827, 915, 2523, 3081, 971, 2961, + /* 1010 */ 254, 915, 2523, 500, 2957, 2972, 784, 471, 954, 187, + /* 1020 */ 186, 951, 950, 949, 184, 579, 339, 2516, 2971, 2082, + /* 1030 */ 2092, 598, 915, 2523, 1025, 2510, 2027, 915, 2523, 2131, + /* 1040 */ 2134, 788, 2719, 53, 52, 915, 2523, 59, 57, 56, + /* 1050 */ 55, 54, 612, 393, 2022, 2508, 2020, 613, 521, 915, + /* 1060 */ 2523, 882, 725, 724, 43, 614, 1022, 2331, 1013, 1011, + /* 1070 */ 53, 52, 221, 2151, 59, 57, 56, 55, 54, 705, + /* 1080 */ 973, 1007, 1003, 999, 995, 2161, 388, 2050, 113, 1571, + /* 1090 */ 2025, 2026, 2079, 395, 2081, 2084, 2085, 2086, 2087, 2088, + /* 1100 */ 2089, 2090, 2091, 2093, 2094, 2095, 892, 885, 851, 170, + /* 1110 */ 917, 916, 881, 2116, 2117, 1569, 1570, 2517, 2123, 2125, + /* 1120 */ 2126, 2127, 2128, 2130, 2, 60, 58, 2135, 2813, 2848, + /* 1130 */ 915, 2523, 133, 506, 1564, 2021, 779, 361, 915, 2523, + /* 1140 */ 2577, 2578, 759, 306, 897, 2583, 3016, 305, 403, 2019, + /* 1150 */ 2518, 2129, 2560, 1571, 2267, 53, 52, 773, 309, 59, + /* 1160 */ 57, 56, 55, 54, 2582, 2166, 2848, 2583, 867, 833, + /* 1170 */ 2022, 801, 2020, 553, 2866, 511, 304, 97, 1566, 1569, + /* 1180 */ 1570, 897, 318, 3029, 2124, 42, 2581, 884, 1598, 1599, + /* 1190 */ 2813, 2583, 896, 802, 173, 762, 915, 2523, 2027, 527, + /* 1200 */ 238, 3061, 756, 754, 3065, 2513, 2025, 2026, 3, 301, + /* 1210 */ 2581, 2866, 877, 359, 2933, 873, 317, 2330, 342, 3067, + /* 1220 */ 231, 116, 66, 341, 3062, 840, 448, 2813, 1022, 896, + /* 1230 */ 485, 61, 774, 118, 201, 2969, 2970, 2847, 168, 2974, + /* 1240 */ 2898, 2207, 311, 1989, 134, 2849, 900, 2851, 2852, 895, + /* 1250 */ 915, 2523, 883, 2899, 919, 85, 1595, 2455, 1594, 3081, + /* 1260 */ 84, 2961, 787, 226, 2217, 500, 2957, 2329, 2132, 2133, + /* 1270 */ 857, 56, 55, 54, 2847, 520, 519, 2898, 2813, 727, + /* 1280 */ 726, 134, 2849, 900, 2851, 2852, 895, 748, 747, 883, + /* 1290 */ 2899, 919, 752, 751, 750, 163, 3081, 1596, 2961, 742, + /* 1300 */ 167, 746, 500, 2957, 529, 745, 396, 2082, 2092, 103, + /* 1310 */ 744, 749, 480, 479, 195, 73, 743, 2131, 2134, 802, + /* 1320 */ 478, 739, 738, 737, 2051, 2528, 799, 3061, 2813, 985, + /* 1330 */ 983, 2050, 2022, 2753, 2020, 2046, 53, 52, 2605, 882, + /* 1340 */ 59, 57, 56, 55, 54, 3067, 231, 2321, 53, 52, + /* 1350 */ 3062, 840, 59, 57, 56, 55, 54, 2328, 53, 52, + /* 1360 */ 2327, 314, 59, 57, 56, 55, 54, 2728, 2025, 2026, + /* 1370 */ 2079, 2326, 2081, 2084, 2085, 2086, 2087, 2088, 2089, 2090, + /* 1380 */ 2091, 2093, 2094, 2095, 892, 885, 843, 48, 917, 916, + /* 1390 */ 881, 2116, 2117, 2336, 1015, 922, 2123, 2125, 2126, 2127, + /* 1400 */ 2128, 2130, 2, 60, 58, 915, 2523, 2848, 915, 2523, + /* 1410 */ 2325, 506, 1703, 2021, 952, 855, 953, 2574, 2813, 2574, + /* 1420 */ 49, 2813, 897, 402, 816, 350, 1702, 2019, 871, 2129, + /* 1430 */ 915, 2523, 2813, 2594, 2848, 954, 187, 186, 951, 950, + /* 1440 */ 949, 184, 1573, 2585, 915, 2523, 915, 2523, 2045, 897, + /* 1450 */ 357, 3054, 2866, 915, 2523, 915, 2523, 735, 2324, 2101, + /* 1460 */ 2527, 734, 2124, 195, 908, 884, 909, 846, 2813, 2237, + /* 1470 */ 896, 2813, 802, 913, 2529, 385, 2027, 53, 52, 2866, + /* 1480 */ 3061, 59, 57, 56, 55, 54, 53, 52, 2030, 741, + /* 1490 */ 59, 57, 56, 55, 54, 2813, 620, 896, 3067, 231, + /* 1500 */ 2323, 2322, 809, 3062, 840, 651, 1022, 234, 2319, 61, + /* 1510 */ 2114, 619, 1678, 2318, 2317, 2847, 2316, 2315, 2898, 2813, + /* 1520 */ 2848, 2314, 134, 2849, 900, 2851, 2852, 895, 2313, 2312, + /* 1530 */ 883, 2899, 919, 2311, 678, 897, 177, 3081, 842, 2961, + /* 1540 */ 775, 3022, 2847, 500, 2957, 2898, 2132, 2133, 91, 134, + /* 1550 */ 2849, 900, 2851, 2852, 895, 177, 1682, 883, 2899, 919, + /* 1560 */ 294, 2813, 2813, 292, 3081, 2866, 2961, 205, 2051, 2813, + /* 1570 */ 500, 2957, 2080, 296, 2813, 2813, 295, 2813, 2813, 812, + /* 1580 */ 222, 2813, 2813, 896, 680, 2082, 2092, 2391, 2243, 2813, + /* 1590 */ 2813, 632, 2389, 2380, 2813, 2131, 2134, 2102, 62, 333, + /* 1600 */ 298, 647, 104, 297, 300, 1683, 923, 299, 2378, 753, + /* 1610 */ 2022, 62, 2020, 214, 755, 757, 185, 882, 344, 53, + /* 1620 */ 52, 2834, 1937, 59, 57, 56, 55, 54, 2847, 822, + /* 1630 */ 760, 2898, 325, 14, 13, 199, 2849, 900, 2851, 2852, + /* 1640 */ 895, 1945, 2029, 883, 2899, 919, 2025, 2026, 2079, 90, + /* 1650 */ 2081, 2084, 2085, 2086, 2087, 2088, 2089, 2090, 2091, 2093, + /* 1660 */ 2094, 2095, 892, 885, 62, 2241, 917, 916, 881, 2116, + /* 1670 */ 2117, 2300, 2301, 2033, 2123, 2125, 2126, 2127, 2128, 2130, + /* 1680 */ 2, 60, 58, 2848, 2253, 803, 3019, 165, 2836, 506, + /* 1690 */ 77, 2021, 844, 364, 363, 852, 62, 2252, 897, 323, + /* 1700 */ 2993, 2867, 856, 62, 348, 2019, 62, 2129, 2446, 2445, + /* 1710 */ 62, 90, 366, 365, 53, 52, 39, 3012, 59, 57, + /* 1720 */ 56, 55, 54, 776, 2357, 182, 163, 132, 2866, 129, + /* 1730 */ 2104, 185, 368, 367, 819, 863, 370, 369, 372, 371, + /* 1740 */ 2124, 374, 373, 884, 2813, 87, 896, 925, 376, 375, + /* 1750 */ 2105, 378, 377, 492, 2027, 53, 52, 380, 379, 59, + /* 1760 */ 57, 56, 55, 54, 53, 52, 183, 185, 59, 57, + /* 1770 */ 56, 55, 54, 53, 52, 847, 2167, 59, 57, 56, + /* 1780 */ 55, 54, 2115, 164, 1022, 382, 381, 61, 1659, 2108, + /* 1790 */ 182, 2847, 2096, 965, 2898, 2848, 1890, 1880, 134, 2849, + /* 1800 */ 900, 2851, 2852, 895, 384, 383, 883, 2899, 919, 966, + /* 1810 */ 897, 360, 912, 3081, 826, 2961, 1651, 1733, 1632, 500, + /* 1820 */ 2957, 488, 2658, 537, 2132, 2133, 860, 2032, 555, 2363, + /* 1830 */ 2571, 401, 1649, 1764, 795, 3013, 3023, 1660, 834, 835, + /* 1840 */ 2866, 330, 335, 338, 2659, 5, 2481, 540, 545, 462, + /* 1850 */ 2044, 554, 1772, 1779, 2054, 566, 2813, 565, 896, 239, + /* 1860 */ 568, 240, 242, 2082, 2092, 1914, 2107, 1633, 394, 1777, + /* 1870 */ 582, 2045, 589, 2131, 2134, 2103, 188, 591, 253, 595, + /* 1880 */ 597, 640, 602, 626, 2100, 636, 615, 2651, 2022, 628, + /* 1890 */ 2020, 639, 641, 652, 653, 882, 259, 650, 655, 657, + /* 1900 */ 262, 258, 138, 2847, 658, 660, 2898, 662, 2052, 681, + /* 1910 */ 202, 2849, 900, 2851, 2852, 895, 4, 682, 883, 2899, + /* 1920 */ 919, 689, 690, 270, 2025, 2026, 2079, 692, 2081, 2084, + /* 1930 */ 2085, 2086, 2087, 2088, 2089, 2090, 2091, 2093, 2094, 2095, + /* 1940 */ 892, 885, 109, 2047, 917, 916, 881, 2116, 2117, 273, + /* 1950 */ 694, 2053, 2123, 2125, 2126, 2127, 2128, 2130, 2, 60, + /* 1960 */ 58, 695, 2055, 276, 2848, 696, 698, 506, 2056, 2021, + /* 1970 */ 278, 110, 841, 3082, 2674, 2057, 111, 2668, 112, 897, + /* 1980 */ 1731, 707, 284, 2019, 114, 2129, 730, 732, 287, 2511, + /* 1990 */ 763, 291, 764, 2743, 2507, 139, 293, 191, 439, 136, + /* 2000 */ 778, 2509, 2504, 115, 192, 193, 178, 780, 2048, 2866, + /* 2010 */ 790, 397, 789, 310, 2720, 315, 791, 313, 2124, 796, + /* 2020 */ 797, 884, 2740, 2739, 861, 2813, 3028, 896, 823, 794, + /* 2030 */ 806, 8, 2027, 832, 804, 837, 805, 3027, 2980, 807, + /* 2040 */ 836, 204, 326, 3000, 848, 533, 532, 324, 327, 320, + /* 2050 */ 331, 322, 328, 493, 169, 2035, 845, 2049, 2215, 2213, + /* 2060 */ 329, 2848, 1022, 3060, 217, 61, 340, 179, 859, 2028, + /* 2070 */ 864, 2129, 2847, 1, 398, 2898, 897, 3084, 233, 134, + /* 2080 */ 2849, 900, 2851, 2852, 895, 180, 399, 883, 2899, 919, + /* 2090 */ 334, 2977, 2688, 2687, 2936, 2686, 2961, 497, 865, 869, + /* 2100 */ 500, 2957, 2132, 2133, 2124, 353, 2866, 872, 74, 2942, + /* 2110 */ 904, 902, 906, 400, 907, 2805, 2524, 128, 2027, 358, + /* 2120 */ 2804, 125, 2813, 1543, 896, 1017, 2800, 1018, 2799, 2791, + /* 2130 */ 2790, 1019, 404, 2782, 65, 189, 2781, 2797, 2796, 2788, + /* 2140 */ 2787, 2082, 2092, 2776, 2775, 387, 1014, 390, 879, 2794, + /* 2150 */ 2793, 2131, 2134, 427, 391, 1021, 406, 440, 921, 476, + /* 2160 */ 782, 408, 454, 2785, 2765, 2784, 2022, 2773, 2020, 2847, + /* 2170 */ 438, 2772, 2898, 882, 2764, 2770, 134, 2849, 900, 2851, + /* 2180 */ 2852, 895, 524, 2763, 883, 2899, 919, 98, 428, 2769, + /* 2190 */ 2758, 2934, 2575, 2961, 542, 543, 1972, 500, 2957, 1973, + /* 2200 */ 547, 237, 2025, 2026, 2079, 2756, 2081, 2084, 2085, 2086, + /* 2210 */ 2087, 2088, 2089, 2090, 2091, 2093, 2094, 2095, 892, 885, + /* 2220 */ 549, 455, 917, 916, 881, 2116, 2117, 550, 551, 1971, + /* 2230 */ 2123, 2125, 2126, 2127, 2128, 2130, 2, 60, 58, 2755, + /* 2240 */ 2754, 463, 2752, 557, 2751, 506, 559, 2021, 2750, 561, + /* 2250 */ 2749, 563, 2036, 1959, 2031, 2724, 241, 2723, 243, 1917, + /* 2260 */ 99, 2019, 1916, 2129, 2701, 2700, 2699, 2848, 575, 576, + /* 2270 */ 2698, 2697, 2641, 580, 1853, 2638, 2637, 583, 2631, 586, + /* 2280 */ 2628, 587, 897, 246, 2627, 102, 2626, 2625, 2039, 2041, + /* 2290 */ 2630, 2629, 2624, 2623, 2621, 2620, 2124, 248, 2619, 884, + /* 2300 */ 250, 2618, 605, 603, 2616, 2615, 2614, 2613, 917, 916, + /* 2310 */ 2027, 2612, 2866, 2636, 2611, 2610, 2123, 2125, 2126, 2127, + /* 2320 */ 2128, 2130, 2609, 2634, 2617, 2608, 2607, 2606, 2813, 2604, + /* 2330 */ 896, 2603, 2602, 2601, 2600, 2599, 252, 2598, 2597, 2596, + /* 2340 */ 1022, 2595, 2593, 15, 108, 2592, 2666, 2635, 2633, 2591, + /* 2350 */ 2590, 2589, 1859, 2588, 2587, 257, 643, 645, 2586, 2584, + /* 2360 */ 452, 453, 1700, 2411, 2410, 1704, 2409, 260, 2408, 2406, + /* 2370 */ 261, 263, 1696, 264, 2403, 2847, 664, 2402, 2898, 2395, + /* 2380 */ 2132, 2133, 134, 2849, 900, 2851, 2852, 895, 663, 2382, + /* 2390 */ 883, 2899, 919, 667, 2370, 2369, 671, 876, 2352, 2961, + /* 2400 */ 665, 668, 1572, 500, 2957, 669, 673, 675, 679, 2351, + /* 2410 */ 2722, 677, 2848, 212, 266, 2718, 2708, 94, 2696, 2082, + /* 2420 */ 2092, 2833, 672, 268, 277, 2695, 223, 897, 95, 2131, + /* 2430 */ 2134, 275, 687, 280, 2672, 2665, 282, 2499, 2405, 2401, + /* 2440 */ 710, 708, 2399, 709, 2022, 714, 2020, 712, 713, 2397, + /* 2450 */ 716, 882, 718, 2394, 717, 720, 721, 2866, 2377, 2375, + /* 2460 */ 2376, 722, 1625, 2374, 2371, 728, 2348, 2501, 1784, 1783, + /* 2470 */ 2500, 758, 1686, 2813, 1668, 896, 1685, 1684, 982, 1681, + /* 2480 */ 2025, 2026, 2079, 481, 2081, 2084, 2085, 2086, 2087, 2088, + /* 2490 */ 2089, 2090, 2091, 2093, 2094, 2095, 892, 885, 1679, 2392, + /* 2500 */ 917, 916, 881, 2116, 2117, 1677, 984, 1676, 2123, 2125, + /* 2510 */ 2126, 2127, 2128, 2130, 2, 450, 449, 1675, 2848, 1674, + /* 2520 */ 2847, 86, 1673, 2898, 2390, 513, 290, 135, 2849, 900, + /* 2530 */ 2851, 2852, 895, 897, 1670, 883, 2899, 919, 1669, 522, + /* 2540 */ 1667, 2129, 482, 2381, 2961, 2848, 2379, 483, 2960, 2957, + /* 2550 */ 484, 761, 2347, 2346, 2345, 765, 2344, 767, 2343, 769, + /* 2560 */ 897, 2342, 771, 2866, 141, 1953, 1955, 1952, 2721, 312, + /* 2570 */ 1957, 2717, 33, 80, 2124, 1923, 2707, 1925, 2694, 2813, + /* 2580 */ 2693, 896, 792, 22, 3066, 2848, 69, 17, 808, 228, + /* 2590 */ 2866, 2270, 35, 319, 6, 2244, 36, 7, 2242, 70, + /* 2600 */ 897, 811, 793, 489, 817, 813, 2813, 815, 896, 316, + /* 2610 */ 23, 321, 2848, 2251, 203, 24, 1902, 1901, 215, 37, + /* 2620 */ 216, 2834, 38, 96, 1942, 2236, 2847, 897, 1927, 2898, + /* 2630 */ 2866, 25, 798, 135, 2849, 900, 2851, 2852, 895, 800, + /* 2640 */ 229, 883, 2899, 919, 781, 194, 2813, 2208, 896, 2210, + /* 2650 */ 2961, 2206, 230, 898, 878, 2957, 2898, 2866, 78, 26, + /* 2660 */ 135, 2849, 900, 2851, 2852, 895, 2290, 2291, 883, 2899, + /* 2670 */ 919, 2285, 2284, 2813, 494, 896, 2289, 2961, 2288, 495, + /* 2680 */ 336, 457, 2957, 2190, 72, 208, 2692, 2671, 2189, 120, + /* 2690 */ 119, 2670, 343, 2847, 2246, 2664, 2898, 218, 121, 27, + /* 2700 */ 413, 2849, 900, 2851, 2852, 895, 349, 32, 883, 2899, + /* 2710 */ 919, 862, 82, 351, 868, 352, 870, 122, 2142, 11, + /* 2720 */ 2847, 2141, 2012, 2898, 1988, 354, 13, 135, 2849, 900, + /* 2730 */ 2851, 2852, 895, 71, 2037, 883, 2899, 919, 21, 2152, + /* 2740 */ 28, 18, 29, 20, 2961, 50, 2663, 209, 932, 2958, + /* 2750 */ 2099, 2098, 2097, 219, 2072, 935, 515, 514, 2013, 938, + /* 2760 */ 123, 2496, 941, 910, 51, 16, 30, 899, 31, 2848, + /* 2770 */ 362, 2064, 903, 905, 83, 356, 124, 92, 917, 916, + /* 2780 */ 129, 2911, 911, 2910, 897, 918, 2123, 2125, 2126, 2127, + /* 2790 */ 2128, 2130, 2304, 2111, 81, 2303, 1765, 2848, 924, 920, + /* 2800 */ 525, 926, 928, 1762, 1761, 931, 929, 934, 1758, 1752, + /* 2810 */ 937, 1750, 897, 940, 2866, 1756, 130, 386, 1755, 1754, + /* 2820 */ 1753, 131, 1778, 93, 1774, 1664, 1623, 955, 1663, 1662, + /* 2830 */ 2813, 1661, 896, 1658, 1655, 1654, 1653, 1652, 1694, 970, + /* 2840 */ 1650, 1648, 2866, 1647, 1646, 1693, 972, 235, 1644, 1643, + /* 2850 */ 1641, 2400, 1642, 1640, 503, 1639, 1638, 1690, 2813, 1688, + /* 2860 */ 896, 1635, 1634, 1631, 1629, 1630, 1628, 992, 993, 2398, + /* 2870 */ 996, 2396, 1000, 2393, 2373, 997, 994, 2847, 1001, 998, + /* 2880 */ 2898, 2848, 1002, 1004, 437, 2849, 900, 2851, 2852, 895, + /* 2890 */ 1005, 1008, 883, 2899, 919, 2368, 897, 1006, 1010, 2367, + /* 2900 */ 1012, 1561, 2341, 1544, 1549, 2847, 1551, 392, 2898, 2848, + /* 2910 */ 1016, 1020, 199, 2849, 900, 2851, 2852, 895, 2023, 405, + /* 2920 */ 883, 2899, 919, 1024, 897, 1023, 2866, 2307, 2307, 2307, + /* 2930 */ 2848, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 2940 */ 2307, 2307, 2813, 2307, 896, 897, 2307, 2307, 2307, 2307, + /* 2950 */ 2307, 2307, 2307, 2307, 2866, 2307, 2307, 2307, 2307, 2307, + /* 2960 */ 2307, 2307, 2307, 3020, 2307, 2307, 490, 2307, 2307, 2307, + /* 2970 */ 2813, 2307, 896, 2307, 2307, 2866, 2307, 2307, 2307, 2307, + /* 2980 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2847, + /* 2990 */ 2307, 2813, 2898, 896, 491, 2307, 437, 2849, 900, 2851, + /* 3000 */ 2852, 895, 2307, 2307, 883, 2899, 919, 2307, 2307, 2307, + /* 3010 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2847, 2307, 2307, + /* 3020 */ 2898, 2848, 2307, 2307, 437, 2849, 900, 2851, 2852, 895, + /* 3030 */ 2307, 2307, 883, 2899, 919, 2307, 897, 2307, 2847, 2307, + /* 3040 */ 2307, 2898, 2848, 2307, 2307, 430, 2849, 900, 2851, 2852, + /* 3050 */ 895, 2307, 2307, 883, 2899, 919, 2307, 894, 2307, 2307, + /* 3060 */ 2307, 2307, 2848, 2307, 2307, 2307, 2866, 2307, 2307, 2307, + /* 3070 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 897, 2307, 2307, + /* 3080 */ 2307, 2307, 2813, 2307, 896, 2307, 2307, 2866, 2307, 2307, + /* 3090 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 831, + /* 3100 */ 2307, 2307, 2307, 2813, 2307, 896, 2307, 2866, 2307, 2307, + /* 3110 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 3120 */ 2307, 2307, 2307, 2813, 2307, 896, 2307, 2307, 2307, 2847, + /* 3130 */ 2307, 2307, 2898, 2307, 2307, 2307, 202, 2849, 900, 2851, + /* 3140 */ 2852, 895, 2307, 2307, 883, 2899, 919, 504, 2307, 2307, + /* 3150 */ 2847, 2307, 2307, 2898, 2307, 2307, 2307, 436, 2849, 900, + /* 3160 */ 2851, 2852, 895, 2307, 2307, 883, 2899, 919, 2307, 2927, + /* 3170 */ 2847, 2307, 2307, 2898, 2307, 2307, 2307, 437, 2849, 900, + /* 3180 */ 2851, 2852, 895, 2848, 2307, 883, 2899, 919, 2307, 2307, + /* 3190 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 897, 3083, + /* 3200 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2848, 2307, + /* 3210 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 3220 */ 2307, 2307, 2307, 897, 2307, 2307, 2307, 2307, 2866, 2307, + /* 3230 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 3240 */ 2307, 2307, 2307, 2307, 2813, 2307, 896, 2307, 2307, 2307, + /* 3250 */ 2307, 2307, 2848, 2866, 2307, 2307, 2307, 2307, 2307, 2307, + /* 3260 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 897, 499, 2813, + /* 3270 */ 2307, 896, 2307, 2307, 2307, 2848, 2307, 2307, 2307, 2307, + /* 3280 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 3290 */ 897, 2847, 2307, 505, 2898, 2307, 2307, 2866, 422, 2849, + /* 3300 */ 900, 2851, 2852, 895, 2307, 2307, 883, 2899, 919, 2307, + /* 3310 */ 2307, 2307, 2307, 2813, 2307, 896, 2847, 2307, 2307, 2898, + /* 3320 */ 2866, 2307, 2307, 437, 2849, 900, 2851, 2852, 895, 2307, + /* 3330 */ 2307, 883, 2899, 919, 2307, 2307, 2813, 507, 896, 2307, + /* 3340 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 3350 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 3360 */ 2847, 2307, 2307, 2898, 2848, 2307, 2307, 437, 2849, 900, + /* 3370 */ 2851, 2852, 895, 2307, 2307, 883, 2899, 919, 2307, 897, + /* 3380 */ 2307, 2307, 2307, 2847, 2307, 2307, 2898, 2848, 2307, 2307, + /* 3390 */ 418, 2849, 900, 2851, 2852, 895, 2307, 2307, 883, 2899, + /* 3400 */ 919, 2307, 897, 2307, 2307, 2307, 2307, 2307, 2307, 2866, + /* 3410 */ 2307, 2307, 2307, 2307, 2307, 2848, 2307, 2307, 2307, 2307, + /* 3420 */ 2307, 2307, 2307, 2307, 2307, 2813, 2307, 896, 2307, 2307, + /* 3430 */ 897, 2307, 2866, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 3440 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2813, 2307, + /* 3450 */ 896, 2307, 2307, 2307, 2307, 2307, 2848, 2307, 2307, 2307, + /* 3460 */ 2866, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 3470 */ 2307, 897, 777, 2307, 2307, 2898, 2813, 2307, 896, 432, + /* 3480 */ 2849, 900, 2851, 2852, 895, 2307, 2307, 883, 2899, 919, + /* 3490 */ 2307, 2307, 2307, 2307, 2307, 2847, 2307, 2307, 2898, 2848, + /* 3500 */ 2307, 2866, 414, 2849, 900, 2851, 2852, 895, 2307, 2307, + /* 3510 */ 883, 2899, 919, 2307, 897, 2307, 2307, 2813, 2307, 896, + /* 3520 */ 2307, 2307, 2307, 2847, 2307, 2307, 2898, 2307, 2307, 2307, + /* 3530 */ 410, 2849, 900, 2851, 2852, 895, 2307, 2307, 883, 2899, + /* 3540 */ 919, 2848, 2307, 2307, 2866, 2307, 2307, 2307, 2307, 2307, + /* 3550 */ 2307, 2307, 2307, 2307, 2307, 2307, 897, 2307, 2307, 2307, + /* 3560 */ 2813, 2307, 896, 2307, 2847, 2307, 2307, 2898, 2307, 2307, + /* 3570 */ 2307, 411, 2849, 900, 2851, 2852, 895, 2307, 2307, 883, + /* 3580 */ 2899, 919, 2307, 2307, 2307, 2307, 2866, 2307, 2307, 2307, + /* 3590 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 3600 */ 2307, 2307, 2813, 2307, 896, 2307, 2307, 2847, 2307, 2307, + /* 3610 */ 2898, 2307, 2307, 2307, 415, 2849, 900, 2851, 2852, 895, + /* 3620 */ 2307, 2307, 883, 2899, 919, 2307, 2307, 2307, 2307, 2307, + /* 3630 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2848, 2307, + /* 3640 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2847, + /* 3650 */ 2307, 2307, 2898, 897, 2307, 2307, 429, 2849, 900, 2851, + /* 3660 */ 2852, 895, 2848, 2307, 883, 2899, 919, 2307, 2307, 2307, + /* 3670 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 897, 2307, 2307, + /* 3680 */ 2307, 2307, 2307, 2866, 2307, 2307, 2307, 2848, 2307, 2307, + /* 3690 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2813, + /* 3700 */ 2307, 896, 897, 2307, 2307, 2307, 2307, 2866, 2307, 2307, + /* 3710 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 3720 */ 2307, 2307, 2307, 2813, 2307, 896, 2307, 2848, 2307, 2307, + /* 3730 */ 2307, 2307, 2866, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 3740 */ 2307, 2307, 897, 2307, 2307, 2307, 2847, 2307, 2813, 2898, + /* 3750 */ 896, 2307, 2307, 416, 2849, 900, 2851, 2852, 895, 2307, + /* 3760 */ 2307, 883, 2899, 919, 2307, 2307, 2307, 2307, 2307, 2848, + /* 3770 */ 2847, 2307, 2866, 2898, 2307, 2307, 2307, 417, 2849, 900, + /* 3780 */ 2851, 2852, 895, 2307, 897, 883, 2899, 919, 2813, 2307, + /* 3790 */ 896, 2307, 2307, 2307, 2848, 2847, 2307, 2307, 2898, 2307, + /* 3800 */ 2307, 2307, 433, 2849, 900, 2851, 2852, 895, 2307, 897, + /* 3810 */ 883, 2899, 919, 2307, 2866, 2307, 2307, 2307, 2307, 2307, + /* 3820 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 3830 */ 2813, 2307, 896, 2307, 2307, 2847, 2307, 2307, 2898, 2866, + /* 3840 */ 2307, 2307, 419, 2849, 900, 2851, 2852, 895, 2307, 2307, + /* 3850 */ 883, 2899, 919, 2307, 2307, 2813, 2307, 896, 2307, 2307, + /* 3860 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 3870 */ 2307, 2307, 2307, 2307, 2307, 2848, 2307, 2847, 2307, 2307, + /* 3880 */ 2898, 2307, 2307, 2307, 434, 2849, 900, 2851, 2852, 895, + /* 3890 */ 897, 2307, 883, 2899, 919, 2307, 2307, 2307, 2307, 2307, + /* 3900 */ 2848, 2307, 2847, 2307, 2307, 2898, 2307, 2307, 2307, 420, + /* 3910 */ 2849, 900, 2851, 2852, 895, 897, 2307, 883, 2899, 919, + /* 3920 */ 2866, 2307, 2307, 2307, 2848, 2307, 2307, 2307, 2307, 2307, + /* 3930 */ 2307, 2307, 2307, 2307, 2307, 2307, 2813, 2307, 896, 897, + /* 3940 */ 2307, 2307, 2307, 2307, 2307, 2866, 2307, 2307, 2307, 2307, + /* 3950 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 3960 */ 2307, 2813, 2307, 896, 2307, 2848, 2307, 2307, 2307, 2866, + /* 3970 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 3980 */ 897, 2307, 2307, 2847, 2307, 2813, 2898, 896, 2307, 2307, + /* 3990 */ 435, 2849, 900, 2851, 2852, 895, 2307, 2307, 883, 2899, + /* 4000 */ 919, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2847, 2307, + /* 4010 */ 2866, 2898, 2307, 2307, 2307, 421, 2849, 900, 2851, 2852, + /* 4020 */ 895, 2307, 2307, 883, 2899, 919, 2813, 2307, 896, 2307, + /* 4030 */ 2307, 2307, 2847, 2307, 2307, 2898, 2307, 2307, 2307, 412, + /* 4040 */ 2849, 900, 2851, 2852, 895, 2307, 2848, 883, 2899, 919, + /* 4050 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 4060 */ 2307, 897, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 4070 */ 2307, 2848, 2307, 2847, 2307, 2307, 2898, 2307, 2307, 2307, + /* 4080 */ 423, 2849, 900, 2851, 2852, 895, 897, 2307, 883, 2899, + /* 4090 */ 919, 2866, 2307, 2307, 2307, 2848, 2307, 2307, 2307, 2307, + /* 4100 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2813, 2307, 896, + /* 4110 */ 897, 2307, 2307, 2307, 2307, 2307, 2866, 2307, 2307, 2307, + /* 4120 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 4130 */ 2307, 2307, 2813, 2307, 896, 2307, 2848, 2307, 2307, 2307, + /* 4140 */ 2866, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 4150 */ 2307, 897, 2307, 2307, 2847, 2307, 2813, 2898, 896, 2307, + /* 4160 */ 2307, 424, 2849, 900, 2851, 2852, 895, 2307, 2307, 883, + /* 4170 */ 2899, 919, 2307, 2307, 2307, 2307, 2307, 2307, 2848, 2847, + /* 4180 */ 2307, 2866, 2898, 2307, 2307, 2307, 425, 2849, 900, 2851, + /* 4190 */ 2852, 895, 2307, 897, 883, 2899, 919, 2813, 2307, 896, + /* 4200 */ 2307, 2848, 2307, 2847, 2307, 2307, 2898, 2307, 2307, 2307, + /* 4210 */ 426, 2849, 900, 2851, 2852, 895, 897, 2307, 883, 2899, + /* 4220 */ 919, 2307, 2307, 2866, 2307, 2307, 2307, 2307, 2307, 2307, + /* 4230 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2813, + /* 4240 */ 2307, 896, 2307, 2307, 2847, 2307, 2866, 2898, 2307, 2307, + /* 4250 */ 2307, 442, 2849, 900, 2851, 2852, 895, 2307, 2307, 883, + /* 4260 */ 2899, 919, 2813, 2307, 896, 2307, 2307, 2307, 2307, 2848, + /* 4270 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 4280 */ 2307, 2307, 2307, 2307, 897, 2307, 2847, 2307, 2307, 2898, + /* 4290 */ 2307, 2307, 2307, 443, 2849, 900, 2851, 2852, 895, 2307, + /* 4300 */ 2307, 883, 2899, 919, 2307, 2307, 2307, 2848, 2307, 2847, + /* 4310 */ 2307, 2307, 2898, 2307, 2866, 2307, 2860, 2849, 900, 2851, + /* 4320 */ 2852, 895, 897, 2307, 883, 2899, 919, 2307, 2307, 2307, + /* 4330 */ 2813, 2307, 896, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 4340 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 4350 */ 2307, 2307, 2866, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 4360 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2813, 2307, + /* 4370 */ 896, 2307, 2307, 2307, 2307, 2307, 2307, 2847, 2307, 2307, + /* 4380 */ 2898, 2307, 2307, 2307, 2859, 2849, 900, 2851, 2852, 895, + /* 4390 */ 2307, 2307, 883, 2899, 919, 2307, 2307, 2307, 2307, 2307, + /* 4400 */ 2307, 2307, 2848, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 4410 */ 2307, 2307, 2307, 2307, 2307, 2847, 2307, 897, 2898, 2307, + /* 4420 */ 2307, 2307, 2858, 2849, 900, 2851, 2852, 895, 2307, 2848, + /* 4430 */ 883, 2899, 919, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 4440 */ 2307, 2307, 2307, 2307, 897, 2307, 2307, 2866, 2307, 2307, + /* 4450 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 4460 */ 2307, 2307, 2307, 2813, 2307, 896, 2307, 2848, 2307, 2307, + /* 4470 */ 2307, 2307, 2307, 2307, 2866, 2307, 2307, 2307, 2307, 2307, + /* 4480 */ 2307, 2307, 897, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 4490 */ 2813, 2307, 896, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 4500 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 4510 */ 2847, 2307, 2866, 2898, 2307, 2307, 2307, 459, 2849, 900, + /* 4520 */ 2851, 2852, 895, 2307, 2307, 883, 2899, 919, 2813, 2307, + /* 4530 */ 896, 2307, 2307, 2307, 2307, 2307, 2307, 2847, 2307, 2307, + /* 4540 */ 2898, 2307, 2307, 2307, 460, 2849, 900, 2851, 2852, 895, + /* 4550 */ 2848, 2307, 883, 2899, 919, 2307, 2307, 2307, 2307, 2307, + /* 4560 */ 2307, 2307, 2307, 2307, 2307, 897, 2307, 2307, 2307, 2307, + /* 4570 */ 2307, 2307, 2307, 2848, 2307, 2847, 2307, 2307, 2898, 2307, + /* 4580 */ 2307, 2307, 456, 2849, 900, 2851, 2852, 895, 897, 2307, + /* 4590 */ 883, 2899, 919, 2848, 2307, 2866, 2307, 2307, 2307, 2307, + /* 4600 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 897, 2307, + /* 4610 */ 2307, 2813, 2307, 896, 2307, 2307, 2307, 2307, 2866, 2307, + /* 4620 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 4630 */ 2307, 2307, 2307, 2307, 2813, 2307, 896, 2307, 2866, 2307, + /* 4640 */ 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, 2307, + /* 4650 */ 2307, 2307, 2307, 2307, 2813, 2307, 896, 2307, 2847, 2307, + /* 4660 */ 2307, 2898, 2307, 2307, 2307, 461, 2849, 900, 2851, 2852, + /* 4670 */ 895, 2307, 2307, 883, 2899, 919, 2307, 2307, 2307, 2307, + /* 4680 */ 2307, 898, 2307, 2307, 2898, 2307, 2307, 2307, 432, 2849, + /* 4690 */ 900, 2851, 2852, 895, 2307, 2307, 883, 2899, 919, 2307, + /* 4700 */ 2307, 2847, 2307, 2307, 2898, 2307, 2307, 2307, 431, 2849, + /* 4710 */ 900, 2851, 2852, 895, 2307, 2307, 883, 2899, 919, }; static const YYCODETYPE yy_lookahead[] = { - /* 0 */ 536, 401, 536, 455, 404, 405, 415, 440, 544, 4, - /* 10 */ 544, 423, 12, 13, 485, 12, 13, 14, 15, 16, - /* 20 */ 20, 0, 22, 475, 476, 434, 562, 563, 436, 563, - /* 30 */ 423, 567, 568, 567, 568, 444, 36, 440, 38, 0, - /* 40 */ 443, 453, 21, 392, 452, 24, 25, 26, 27, 28, - /* 50 */ 29, 30, 31, 32, 437, 406, 407, 20, 407, 54, - /* 60 */ 453, 8, 9, 446, 447, 12, 13, 14, 15, 16, - /* 70 */ 503, 71, 21, 20, 74, 24, 25, 26, 27, 28, - /* 80 */ 29, 30, 31, 32, 517, 85, 20, 455, 437, 8, - /* 90 */ 9, 503, 504, 12, 13, 14, 15, 16, 59, 401, - /* 100 */ 503, 469, 404, 405, 453, 517, 455, 475, 476, 20, - /* 110 */ 503, 504, 505, 479, 517, 115, 406, 407, 118, 80, - /* 120 */ 81, 82, 83, 84, 517, 86, 87, 88, 89, 90, + /* 0 */ 537, 402, 537, 402, 405, 406, 405, 406, 545, 539, + /* 10 */ 545, 541, 12, 13, 439, 12, 13, 14, 15, 16, + /* 20 */ 20, 0, 22, 486, 432, 450, 563, 564, 20, 564, + /* 30 */ 438, 568, 569, 568, 569, 414, 36, 452, 38, 0, + /* 40 */ 455, 456, 21, 393, 34, 24, 25, 26, 27, 28, + /* 50 */ 29, 30, 31, 32, 20, 407, 408, 465, 408, 1, + /* 60 */ 2, 8, 9, 442, 432, 12, 13, 14, 15, 16, + /* 70 */ 438, 71, 21, 20, 74, 24, 25, 26, 27, 28, + /* 80 */ 29, 30, 31, 32, 424, 85, 8, 9, 438, 21, + /* 90 */ 12, 13, 14, 15, 16, 8, 9, 465, 59, 12, + /* 100 */ 13, 14, 15, 16, 454, 37, 456, 39, 40, 41, + /* 110 */ 42, 33, 407, 20, 454, 115, 94, 20, 118, 80, + /* 120 */ 81, 82, 83, 84, 118, 86, 87, 88, 89, 90, /* 130 */ 91, 92, 93, 94, 95, 96, 97, 98, 99, 100, /* 140 */ 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, - /* 150 */ 111, 500, 440, 438, 503, 155, 156, 120, 507, 508, - /* 160 */ 509, 510, 511, 512, 525, 526, 515, 516, 517, 20, - /* 170 */ 536, 21, 20, 522, 22, 524, 20, 71, 544, 528, - /* 180 */ 529, 532, 533, 534, 118, 536, 537, 37, 36, 39, - /* 190 */ 40, 41, 42, 544, 194, 195, 562, 563, 20, 548, - /* 200 */ 20, 567, 568, 3, 204, 205, 451, 556, 118, 454, - /* 210 */ 455, 562, 563, 61, 499, 503, 567, 568, 20, 219, - /* 220 */ 20, 221, 199, 117, 416, 44, 226, 14, 122, 517, - /* 230 */ 8, 9, 424, 20, 12, 13, 14, 15, 16, 57, - /* 240 */ 118, 531, 532, 533, 534, 20, 536, 537, 66, 196, - /* 250 */ 71, 69, 70, 406, 254, 255, 256, 392, 258, 259, + /* 150 */ 111, 501, 118, 20, 504, 155, 156, 22, 508, 509, + /* 160 */ 510, 511, 512, 513, 504, 505, 516, 517, 518, 407, + /* 170 */ 408, 36, 467, 523, 469, 525, 20, 119, 518, 529, + /* 180 */ 530, 533, 534, 535, 162, 537, 538, 149, 150, 427, + /* 190 */ 20, 438, 154, 545, 194, 195, 407, 119, 436, 549, + /* 200 */ 447, 448, 194, 195, 204, 205, 119, 557, 438, 187, + /* 210 */ 188, 563, 564, 120, 115, 118, 568, 569, 448, 219, + /* 220 */ 85, 221, 199, 201, 158, 392, 226, 394, 407, 130, + /* 230 */ 131, 132, 133, 134, 135, 136, 137, 138, 139, 118, + /* 240 */ 141, 142, 143, 144, 145, 146, 147, 458, 459, 196, + /* 250 */ 461, 498, 499, 464, 20, 255, 256, 257, 393, 259, /* 260 */ 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, - /* 270 */ 270, 271, 407, 149, 274, 275, 276, 277, 278, 279, - /* 280 */ 280, 281, 282, 283, 284, 285, 286, 287, 288, 12, - /* 290 */ 13, 14, 211, 392, 18, 189, 20, 20, 0, 22, - /* 300 */ 78, 122, 437, 27, 457, 458, 30, 460, 407, 256, - /* 310 */ 463, 155, 156, 36, 38, 38, 118, 4, 453, 119, - /* 320 */ 455, 0, 24, 25, 26, 27, 28, 29, 30, 31, - /* 330 */ 32, 431, 56, 57, 0, 59, 407, 437, 437, 158, - /* 340 */ 64, 65, 319, 320, 321, 322, 323, 14, 71, 455, - /* 350 */ 128, 74, 76, 20, 453, 78, 455, 3, 234, 235, - /* 360 */ 204, 205, 85, 469, 464, 500, 437, 392, 503, 475, - /* 370 */ 476, 36, 507, 508, 509, 510, 511, 512, 513, 313, - /* 380 */ 515, 516, 517, 518, 519, 148, 149, 150, 151, 152, - /* 390 */ 153, 154, 115, 117, 20, 118, 196, 20, 406, 407, - /* 400 */ 22, 500, 254, 313, 503, 129, 36, 194, 507, 508, - /* 410 */ 509, 510, 511, 512, 36, 193, 515, 516, 517, 76, - /* 420 */ 0, 520, 341, 522, 523, 524, 12, 13, 453, 528, - /* 430 */ 529, 118, 155, 156, 256, 313, 160, 161, 509, 163, - /* 440 */ 164, 165, 166, 167, 168, 169, 170, 171, 172, 20, - /* 450 */ 36, 175, 176, 177, 178, 179, 180, 181, 182, 431, - /* 460 */ 184, 185, 186, 20, 77, 437, 190, 191, 192, 256, - /* 470 */ 437, 194, 195, 197, 326, 327, 328, 329, 330, 331, - /* 480 */ 332, 204, 205, 302, 303, 304, 305, 306, 307, 308, - /* 490 */ 309, 310, 464, 115, 272, 174, 219, 464, 221, 2, - /* 500 */ 179, 14, 20, 226, 77, 8, 9, 20, 187, 12, - /* 510 */ 13, 14, 15, 16, 292, 293, 294, 295, 296, 297, - /* 520 */ 298, 299, 300, 301, 532, 533, 534, 194, 536, 537, - /* 530 */ 0, 254, 255, 256, 392, 258, 259, 260, 261, 262, - /* 540 */ 263, 264, 265, 266, 267, 268, 269, 270, 271, 407, - /* 550 */ 536, 274, 275, 276, 277, 278, 221, 214, 544, 282, - /* 560 */ 283, 284, 285, 286, 287, 288, 12, 13, 392, 158, - /* 570 */ 22, 194, 195, 118, 20, 406, 22, 563, 158, 437, - /* 580 */ 246, 567, 568, 407, 36, 242, 243, 57, 115, 256, - /* 590 */ 36, 221, 38, 173, 538, 453, 540, 455, 155, 156, - /* 600 */ 118, 392, 4, 130, 131, 132, 133, 134, 135, 136, - /* 610 */ 137, 138, 139, 437, 141, 142, 143, 144, 145, 146, - /* 620 */ 147, 23, 252, 253, 406, 71, 313, 438, 74, 453, - /* 630 */ 256, 455, 78, 85, 437, 221, 406, 407, 449, 85, - /* 640 */ 471, 472, 500, 446, 447, 503, 48, 49, 50, 507, - /* 650 */ 508, 509, 510, 511, 512, 289, 426, 515, 516, 517, - /* 660 */ 406, 407, 453, 115, 522, 189, 524, 406, 407, 115, - /* 670 */ 528, 529, 118, 218, 13, 220, 500, 406, 407, 503, - /* 680 */ 426, 392, 36, 507, 508, 509, 510, 511, 512, 435, - /* 690 */ 336, 515, 516, 517, 497, 498, 407, 426, 556, 223, - /* 700 */ 524, 438, 439, 158, 528, 529, 251, 489, 490, 155, - /* 710 */ 156, 454, 455, 302, 303, 304, 305, 306, 307, 308, - /* 720 */ 309, 310, 461, 8, 9, 129, 437, 12, 13, 14, - /* 730 */ 15, 16, 80, 81, 82, 406, 407, 406, 407, 87, - /* 740 */ 88, 89, 453, 85, 455, 93, 85, 506, 194, 195, - /* 750 */ 98, 99, 100, 101, 406, 407, 104, 426, 204, 205, - /* 760 */ 108, 109, 110, 111, 62, 63, 435, 219, 313, 221, - /* 770 */ 406, 407, 2, 219, 426, 221, 535, 20, 8, 9, - /* 780 */ 226, 34, 12, 13, 14, 15, 16, 402, 20, 500, - /* 790 */ 426, 406, 503, 408, 0, 313, 507, 508, 509, 510, - /* 800 */ 511, 512, 254, 255, 515, 516, 517, 389, 254, 255, - /* 810 */ 256, 392, 258, 259, 260, 261, 262, 263, 264, 265, - /* 820 */ 266, 267, 268, 269, 270, 271, 407, 33, 274, 275, - /* 830 */ 276, 277, 278, 129, 119, 20, 282, 283, 284, 285, - /* 840 */ 286, 287, 288, 289, 12, 13, 557, 558, 303, 304, - /* 850 */ 305, 402, 20, 437, 22, 406, 437, 408, 406, 36, - /* 860 */ 115, 532, 533, 534, 448, 536, 537, 221, 36, 391, - /* 870 */ 38, 393, 453, 438, 455, 130, 131, 132, 133, 134, - /* 880 */ 135, 136, 137, 138, 139, 437, 141, 142, 143, 144, - /* 890 */ 145, 146, 147, 445, 406, 407, 506, 479, 252, 253, - /* 900 */ 149, 150, 484, 71, 456, 154, 74, 94, 85, 437, - /* 910 */ 78, 465, 306, 307, 308, 309, 310, 85, 36, 500, - /* 920 */ 406, 407, 503, 406, 407, 535, 507, 508, 509, 510, - /* 930 */ 511, 512, 406, 407, 515, 516, 517, 438, 406, 407, - /* 940 */ 426, 489, 490, 524, 392, 406, 407, 115, 529, 461, - /* 950 */ 118, 392, 426, 196, 536, 437, 406, 407, 426, 407, - /* 960 */ 479, 409, 544, 445, 196, 426, 18, 85, 406, 497, - /* 970 */ 498, 23, 12, 13, 456, 162, 426, 20, 461, 22, - /* 980 */ 562, 563, 22, 437, 536, 567, 568, 155, 156, 437, - /* 990 */ 42, 43, 544, 438, 46, 392, 36, 289, 38, 291, - /* 1000 */ 187, 188, 456, 55, 0, 453, 438, 455, 60, 194, - /* 1010 */ 562, 563, 453, 256, 201, 567, 568, 536, 61, 438, - /* 1020 */ 72, 73, 74, 75, 76, 544, 194, 195, 466, 422, - /* 1030 */ 468, 71, 425, 22, 8, 9, 204, 205, 12, 13, - /* 1040 */ 14, 15, 16, 562, 563, 85, 437, 36, 567, 568, - /* 1050 */ 129, 219, 500, 221, 445, 503, 453, 20, 226, 507, - /* 1060 */ 508, 509, 510, 511, 512, 456, 118, 515, 516, 517, - /* 1070 */ 521, 256, 523, 392, 522, 115, 524, 14, 15, 16, - /* 1080 */ 528, 529, 1, 2, 479, 23, 254, 255, 256, 484, - /* 1090 */ 258, 259, 260, 261, 262, 263, 264, 265, 266, 267, - /* 1100 */ 268, 269, 270, 271, 183, 157, 274, 275, 276, 277, - /* 1110 */ 278, 49, 50, 392, 282, 283, 284, 285, 286, 287, - /* 1120 */ 288, 12, 13, 14, 406, 407, 115, 392, 438, 20, - /* 1130 */ 392, 22, 8, 9, 453, 0, 12, 13, 14, 15, - /* 1140 */ 16, 536, 150, 55, 426, 36, 154, 38, 520, 544, - /* 1150 */ 4, 523, 148, 149, 150, 151, 152, 153, 154, 211, - /* 1160 */ 212, 213, 406, 407, 216, 19, 427, 562, 563, 0, - /* 1170 */ 392, 47, 567, 568, 453, 393, 437, 229, 230, 219, - /* 1180 */ 71, 221, 426, 74, 38, 411, 412, 448, 453, 241, - /* 1190 */ 0, 453, 244, 58, 85, 247, 248, 249, 250, 251, - /* 1200 */ 119, 8, 9, 57, 22, 12, 13, 14, 15, 16, - /* 1210 */ 64, 65, 406, 407, 254, 255, 450, 71, 36, 453, - /* 1220 */ 228, 392, 411, 412, 115, 233, 33, 118, 236, 437, - /* 1230 */ 238, 453, 426, 196, 274, 275, 407, 445, 409, 420, - /* 1240 */ 421, 392, 282, 283, 284, 285, 286, 287, 456, 406, - /* 1250 */ 407, 8, 9, 406, 407, 12, 13, 14, 15, 16, - /* 1260 */ 4, 313, 424, 117, 155, 156, 437, 85, 122, 426, - /* 1270 */ 80, 81, 82, 426, 20, 538, 33, 87, 88, 89, - /* 1280 */ 413, 52, 453, 93, 455, 406, 407, 392, 98, 99, - /* 1290 */ 100, 101, 560, 256, 104, 406, 407, 430, 108, 109, - /* 1300 */ 110, 111, 453, 194, 195, 426, 406, 407, 441, 521, - /* 1310 */ 13, 523, 119, 204, 205, 426, 44, 148, 149, 150, - /* 1320 */ 151, 152, 153, 154, 413, 237, 426, 392, 219, 500, - /* 1330 */ 221, 193, 503, 420, 421, 226, 507, 508, 509, 510, - /* 1340 */ 511, 512, 437, 71, 515, 516, 517, 450, 453, 0, - /* 1350 */ 453, 522, 441, 524, 0, 392, 428, 528, 529, 392, - /* 1360 */ 432, 456, 119, 254, 255, 256, 392, 258, 259, 260, - /* 1370 */ 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, - /* 1380 */ 271, 407, 85, 274, 275, 276, 277, 278, 453, 406, - /* 1390 */ 407, 282, 283, 284, 285, 286, 287, 288, 12, 13, - /* 1400 */ 392, 406, 407, 406, 407, 392, 20, 392, 22, 426, - /* 1410 */ 272, 437, 406, 407, 392, 407, 453, 409, 437, 392, - /* 1420 */ 453, 426, 36, 426, 38, 196, 392, 453, 33, 455, - /* 1430 */ 292, 77, 426, 8, 9, 506, 207, 12, 13, 14, - /* 1440 */ 15, 16, 8, 9, 427, 437, 12, 13, 14, 15, - /* 1450 */ 16, 437, 450, 14, 437, 453, 0, 71, 148, 20, - /* 1460 */ 74, 453, 152, 455, 535, 448, 453, 415, 453, 0, - /* 1470 */ 456, 85, 432, 123, 500, 453, 126, 503, 33, 498, - /* 1480 */ 453, 507, 508, 509, 510, 511, 512, 453, 514, 515, - /* 1490 */ 516, 517, 437, 395, 396, 33, 444, 571, 392, 33, - /* 1500 */ 44, 115, 447, 33, 118, 119, 20, 158, 500, 47, - /* 1510 */ 256, 503, 36, 407, 119, 507, 508, 509, 510, 511, - /* 1520 */ 512, 33, 33, 515, 516, 517, 123, 539, 123, 126, - /* 1530 */ 522, 126, 524, 33, 494, 47, 528, 529, 0, 13, - /* 1540 */ 13, 155, 156, 437, 119, 123, 155, 156, 126, 553, - /* 1550 */ 8, 9, 33, 119, 12, 13, 14, 15, 16, 453, - /* 1560 */ 22, 455, 36, 36, 119, 1, 2, 410, 8, 9, - /* 1570 */ 78, 315, 12, 13, 14, 15, 16, 437, 8, 9, - /* 1580 */ 194, 195, 12, 13, 14, 15, 16, 423, 36, 47, - /* 1590 */ 204, 205, 8, 9, 33, 33, 12, 13, 14, 15, - /* 1600 */ 16, 118, 33, 120, 33, 219, 500, 221, 119, 503, - /* 1610 */ 0, 0, 226, 507, 508, 509, 510, 511, 512, 119, - /* 1620 */ 33, 515, 516, 517, 12, 13, 51, 158, 522, 423, - /* 1630 */ 524, 405, 22, 22, 528, 529, 0, 85, 119, 465, - /* 1640 */ 254, 255, 256, 559, 258, 259, 260, 261, 262, 263, - /* 1650 */ 264, 265, 266, 267, 268, 269, 270, 271, 22, 559, - /* 1660 */ 274, 275, 276, 277, 278, 478, 33, 33, 282, 283, - /* 1670 */ 284, 285, 286, 287, 288, 12, 13, 392, 33, 119, - /* 1680 */ 119, 119, 479, 20, 559, 22, 33, 559, 119, 119, - /* 1690 */ 119, 33, 407, 118, 409, 12, 13, 221, 273, 36, - /* 1700 */ 33, 38, 465, 119, 33, 410, 119, 8, 9, 501, - /* 1710 */ 47, 12, 13, 14, 15, 16, 407, 231, 452, 8, - /* 1720 */ 9, 486, 437, 12, 13, 14, 15, 16, 392, 33, - /* 1730 */ 12, 13, 12, 13, 71, 12, 13, 74, 453, 536, - /* 1740 */ 455, 33, 33, 407, 465, 33, 254, 544, 85, 12, - /* 1750 */ 13, 13, 119, 119, 12, 13, 12, 13, 47, 12, - /* 1760 */ 13, 12, 13, 33, 119, 562, 563, 12, 13, 33, - /* 1770 */ 567, 568, 119, 437, 36, 13, 33, 119, 115, 465, - /* 1780 */ 564, 118, 543, 543, 36, 500, 119, 530, 503, 453, - /* 1790 */ 119, 455, 507, 508, 509, 510, 511, 512, 36, 33, - /* 1800 */ 515, 516, 517, 465, 338, 546, 425, 522, 480, 524, - /* 1810 */ 340, 33, 316, 528, 529, 119, 8, 9, 155, 156, - /* 1820 */ 12, 13, 14, 15, 16, 57, 502, 119, 119, 20, - /* 1830 */ 119, 119, 406, 85, 20, 491, 500, 236, 496, 503, - /* 1840 */ 217, 406, 415, 507, 508, 509, 510, 511, 512, 119, - /* 1850 */ 491, 515, 516, 517, 482, 119, 415, 194, 195, 20, - /* 1860 */ 524, 407, 119, 47, 528, 529, 462, 204, 205, 8, - /* 1870 */ 9, 462, 407, 12, 13, 14, 15, 16, 193, 459, - /* 1880 */ 406, 406, 219, 462, 221, 119, 8, 9, 407, 226, - /* 1890 */ 12, 13, 14, 15, 16, 459, 459, 119, 116, 41, - /* 1900 */ 42, 459, 114, 419, 406, 418, 113, 406, 417, 406, - /* 1910 */ 406, 406, 20, 399, 52, 403, 399, 254, 255, 256, - /* 1920 */ 491, 258, 259, 260, 261, 262, 263, 264, 265, 266, - /* 1930 */ 267, 268, 269, 270, 271, 403, 415, 274, 275, 276, - /* 1940 */ 277, 278, 20, 479, 415, 282, 283, 284, 285, 286, - /* 1950 */ 287, 288, 12, 13, 455, 415, 20, 408, 20, 481, - /* 1960 */ 20, 415, 22, 408, 479, 415, 20, 472, 415, 20, - /* 1970 */ 52, 272, 466, 415, 392, 399, 36, 433, 38, 406, - /* 1980 */ 119, 415, 124, 125, 433, 127, 453, 415, 395, 407, - /* 1990 */ 395, 409, 453, 437, 437, 406, 437, 119, 399, 239, - /* 2000 */ 536, 453, 437, 495, 196, 392, 148, 118, 544, 493, - /* 2010 */ 152, 71, 20, 491, 74, 437, 437, 437, 437, 437, - /* 2020 */ 407, 536, 437, 437, 490, 85, 562, 563, 437, 544, - /* 2030 */ 413, 567, 568, 488, 224, 453, 225, 455, 455, 413, - /* 2040 */ 325, 487, 552, 480, 406, 453, 324, 562, 563, 552, - /* 2050 */ 437, 210, 567, 568, 333, 115, 392, 555, 118, 552, - /* 2060 */ 335, 473, 551, 473, 554, 334, 453, 317, 455, 312, - /* 2070 */ 550, 407, 549, 409, 480, 311, 342, 542, 541, 339, - /* 2080 */ 565, 572, 500, 337, 566, 503, 407, 20, 129, 507, - /* 2090 */ 508, 509, 510, 511, 512, 155, 156, 515, 516, 517, - /* 2100 */ 314, 437, 506, 408, 522, 413, 524, 413, 473, 453, - /* 2110 */ 528, 529, 453, 500, 547, 453, 503, 453, 453, 455, - /* 2120 */ 507, 508, 509, 510, 511, 512, 453, 473, 515, 516, - /* 2130 */ 517, 202, 413, 453, 194, 195, 470, 524, 466, 413, - /* 2140 */ 118, 528, 529, 545, 204, 205, 527, 202, 467, 432, - /* 2150 */ 453, 453, 466, 453, 413, 453, 453, 453, 413, 219, - /* 2160 */ 407, 221, 118, 453, 500, 413, 226, 503, 22, 453, - /* 2170 */ 442, 507, 508, 509, 510, 511, 512, 453, 406, 515, - /* 2180 */ 516, 517, 453, 35, 37, 453, 522, 499, 524, 394, - /* 2190 */ 40, 414, 528, 529, 254, 255, 256, 397, 258, 259, - /* 2200 */ 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, - /* 2210 */ 270, 271, 453, 398, 274, 275, 276, 277, 278, 400, - /* 2220 */ 453, 399, 282, 283, 284, 285, 286, 287, 288, 12, - /* 2230 */ 13, 492, 453, 453, 453, 439, 453, 20, 453, 22, - /* 2240 */ 453, 483, 453, 453, 453, 453, 429, 453, 474, 474, - /* 2250 */ 429, 392, 439, 36, 429, 38, 390, 0, 0, 0, - /* 2260 */ 47, 0, 36, 245, 36, 245, 407, 36, 36, 0, - /* 2270 */ 36, 36, 245, 36, 0, 0, 245, 0, 36, 0, - /* 2280 */ 36, 0, 22, 0, 36, 240, 0, 227, 71, 0, - /* 2290 */ 227, 74, 228, 221, 219, 0, 437, 0, 0, 215, - /* 2300 */ 214, 0, 85, 0, 161, 51, 51, 0, 36, 0, - /* 2310 */ 0, 0, 453, 57, 455, 51, 36, 0, 47, 0, - /* 2320 */ 0, 0, 0, 0, 51, 0, 0, 12, 13, 0, - /* 2330 */ 0, 179, 115, 392, 0, 118, 36, 22, 179, 0, - /* 2340 */ 0, 0, 0, 0, 0, 0, 0, 0, 407, 0, - /* 2350 */ 0, 36, 0, 38, 0, 0, 0, 0, 0, 500, - /* 2360 */ 0, 0, 503, 0, 51, 0, 507, 508, 509, 510, - /* 2370 */ 511, 512, 155, 156, 515, 516, 517, 0, 437, 47, - /* 2380 */ 0, 522, 0, 524, 0, 0, 71, 528, 529, 0, - /* 2390 */ 0, 0, 0, 0, 453, 22, 455, 0, 0, 161, - /* 2400 */ 160, 0, 159, 0, 0, 22, 0, 71, 22, 0, - /* 2410 */ 36, 194, 195, 0, 52, 0, 52, 0, 0, 71, - /* 2420 */ 44, 204, 205, 36, 71, 57, 0, 57, 0, 57, - /* 2430 */ 71, 36, 44, 0, 0, 36, 219, 56, 221, 0, - /* 2440 */ 44, 500, 0, 226, 503, 51, 36, 36, 507, 508, - /* 2450 */ 509, 510, 511, 512, 47, 33, 515, 516, 517, 51, - /* 2460 */ 44, 14, 0, 522, 45, 524, 0, 0, 44, 528, - /* 2470 */ 529, 254, 255, 256, 51, 258, 259, 260, 261, 262, - /* 2480 */ 263, 264, 265, 266, 267, 268, 269, 270, 271, 1, - /* 2490 */ 0, 274, 275, 276, 277, 278, 0, 44, 210, 282, - /* 2500 */ 283, 284, 285, 286, 287, 288, 0, 19, 51, 0, - /* 2510 */ 51, 0, 0, 0, 79, 392, 0, 36, 57, 44, - /* 2520 */ 0, 36, 57, 44, 0, 36, 38, 57, 44, 0, - /* 2530 */ 407, 36, 0, 57, 219, 44, 221, 0, 0, 0, - /* 2540 */ 0, 53, 54, 36, 0, 57, 0, 126, 128, 22, - /* 2550 */ 36, 0, 22, 36, 66, 67, 68, 69, 36, 71, - /* 2560 */ 437, 36, 36, 0, 36, 0, 33, 252, 253, 254, - /* 2570 */ 36, 36, 36, 0, 22, 33, 453, 36, 455, 36, - /* 2580 */ 22, 36, 22, 36, 22, 0, 22, 0, 0, 274, - /* 2590 */ 275, 36, 0, 36, 0, 36, 0, 282, 283, 284, - /* 2600 */ 285, 286, 287, 36, 0, 117, 22, 59, 36, 20, - /* 2610 */ 122, 36, 36, 119, 0, 118, 118, 51, 0, 36, - /* 2620 */ 22, 392, 0, 500, 22, 196, 503, 196, 226, 202, - /* 2630 */ 507, 508, 509, 510, 511, 512, 407, 232, 515, 516, - /* 2640 */ 517, 153, 196, 0, 392, 231, 0, 222, 196, 206, - /* 2650 */ 196, 206, 3, 33, 22, 318, 33, 36, 52, 407, - /* 2660 */ 36, 52, 33, 33, 51, 85, 437, 118, 51, 119, - /* 2670 */ 33, 118, 33, 119, 118, 3, 36, 33, 36, 51, - /* 2680 */ 119, 36, 453, 36, 455, 114, 198, 118, 200, 437, - /* 2690 */ 116, 203, 569, 570, 36, 118, 208, 119, 119, 118, - /* 2700 */ 118, 36, 36, 33, 118, 453, 477, 455, 51, 119, - /* 2710 */ 119, 118, 0, 0, 119, 227, 119, 118, 318, 119, - /* 2720 */ 392, 44, 119, 0, 119, 44, 119, 0, 118, 500, - /* 2730 */ 118, 44, 503, 118, 302, 407, 507, 508, 509, 510, - /* 2740 */ 511, 512, 33, 118, 515, 516, 517, 2, 318, 51, - /* 2750 */ 119, 116, 500, 290, 116, 503, 22, 254, 51, 507, - /* 2760 */ 508, 509, 510, 511, 512, 437, 118, 515, 516, 517, - /* 2770 */ 118, 392, 118, 203, 118, 22, 257, 118, 0, 44, - /* 2780 */ 119, 453, 119, 455, 120, 118, 407, 118, 0, 118, - /* 2790 */ 22, 118, 118, 51, 119, 119, 118, 118, 118, 22, - /* 2800 */ 199, 118, 140, 119, 392, 477, 118, 199, 118, 198, - /* 2810 */ 558, 118, 121, 118, 118, 199, 437, 36, 119, 407, - /* 2820 */ 118, 118, 36, 129, 119, 119, 118, 36, 500, 118, - /* 2830 */ 36, 503, 453, 119, 455, 507, 508, 509, 510, 511, - /* 2840 */ 512, 119, 36, 515, 516, 517, 36, 119, 119, 437, - /* 2850 */ 36, 232, 33, 36, 118, 140, 477, 140, 140, 118, - /* 2860 */ 22, 118, 79, 78, 22, 453, 22, 455, 36, 392, - /* 2870 */ 36, 36, 36, 36, 85, 36, 36, 36, 85, 500, - /* 2880 */ 22, 36, 503, 36, 407, 36, 507, 508, 509, 510, - /* 2890 */ 511, 512, 392, 33, 515, 516, 517, 36, 36, 36, - /* 2900 */ 36, 36, 112, 36, 112, 85, 36, 407, 36, 36, - /* 2910 */ 36, 36, 500, 22, 437, 503, 36, 0, 36, 507, - /* 2920 */ 508, 509, 510, 511, 512, 57, 0, 515, 516, 517, - /* 2930 */ 453, 44, 455, 36, 57, 0, 44, 437, 36, 44, - /* 2940 */ 57, 0, 36, 57, 44, 0, 36, 0, 0, 22, - /* 2950 */ 22, 36, 0, 453, 22, 455, 33, 36, 36, 22, - /* 2960 */ 392, 21, 573, 573, 22, 22, 573, 21, 573, 20, - /* 2970 */ 573, 573, 573, 561, 573, 407, 573, 500, 573, 573, - /* 2980 */ 503, 573, 573, 573, 507, 508, 509, 510, 511, 512, - /* 2990 */ 573, 573, 515, 516, 517, 573, 573, 573, 573, 573, - /* 3000 */ 500, 573, 573, 503, 573, 437, 573, 507, 508, 509, - /* 3010 */ 510, 511, 512, 573, 573, 515, 516, 517, 573, 519, - /* 3020 */ 573, 453, 573, 455, 573, 392, 573, 38, 573, 573, - /* 3030 */ 573, 573, 573, 573, 573, 573, 573, 573, 573, 573, - /* 3040 */ 407, 573, 573, 54, 573, 477, 57, 570, 573, 573, - /* 3050 */ 573, 573, 392, 573, 573, 66, 67, 68, 69, 573, - /* 3060 */ 71, 573, 573, 573, 573, 573, 573, 407, 500, 573, - /* 3070 */ 437, 503, 573, 573, 573, 507, 508, 509, 510, 511, - /* 3080 */ 512, 573, 573, 515, 516, 517, 453, 573, 455, 573, - /* 3090 */ 573, 573, 573, 573, 573, 573, 573, 437, 573, 573, - /* 3100 */ 573, 573, 573, 573, 573, 573, 117, 573, 573, 573, - /* 3110 */ 477, 122, 573, 453, 573, 455, 573, 573, 573, 573, - /* 3120 */ 573, 573, 573, 573, 573, 573, 573, 573, 573, 573, - /* 3130 */ 573, 573, 573, 500, 573, 573, 503, 477, 392, 573, - /* 3140 */ 507, 508, 509, 510, 511, 512, 573, 573, 515, 516, - /* 3150 */ 517, 573, 573, 407, 573, 573, 573, 573, 392, 573, - /* 3160 */ 500, 573, 573, 503, 573, 573, 573, 507, 508, 509, - /* 3170 */ 510, 511, 512, 407, 573, 515, 516, 517, 189, 392, - /* 3180 */ 573, 573, 573, 437, 573, 573, 573, 198, 573, 573, - /* 3190 */ 573, 202, 203, 573, 407, 573, 573, 208, 209, 453, - /* 3200 */ 573, 455, 573, 437, 573, 573, 573, 573, 573, 573, - /* 3210 */ 573, 573, 573, 573, 573, 573, 227, 573, 573, 453, - /* 3220 */ 573, 455, 573, 477, 437, 573, 573, 573, 573, 573, - /* 3230 */ 573, 573, 573, 573, 573, 573, 573, 573, 573, 573, - /* 3240 */ 453, 573, 455, 573, 573, 573, 500, 573, 573, 503, - /* 3250 */ 573, 573, 573, 507, 508, 509, 510, 511, 512, 573, - /* 3260 */ 573, 515, 516, 517, 573, 573, 500, 573, 573, 503, - /* 3270 */ 573, 573, 573, 507, 508, 509, 510, 511, 512, 573, - /* 3280 */ 573, 515, 516, 517, 573, 392, 573, 500, 573, 573, - /* 3290 */ 503, 573, 573, 573, 507, 508, 509, 510, 511, 512, - /* 3300 */ 407, 573, 515, 516, 517, 392, 573, 573, 573, 573, - /* 3310 */ 573, 573, 573, 573, 573, 573, 573, 573, 573, 573, - /* 3320 */ 407, 573, 573, 573, 573, 392, 573, 573, 573, 573, - /* 3330 */ 437, 573, 573, 573, 573, 573, 573, 573, 573, 573, - /* 3340 */ 407, 573, 573, 573, 573, 573, 453, 573, 455, 573, - /* 3350 */ 437, 573, 573, 573, 573, 573, 573, 573, 573, 573, - /* 3360 */ 573, 573, 573, 573, 573, 573, 453, 573, 455, 573, - /* 3370 */ 437, 573, 573, 573, 573, 573, 573, 573, 573, 573, - /* 3380 */ 573, 573, 573, 573, 573, 573, 453, 573, 455, 573, - /* 3390 */ 573, 573, 573, 500, 573, 573, 503, 573, 573, 573, - /* 3400 */ 507, 508, 509, 510, 511, 512, 573, 573, 515, 516, - /* 3410 */ 517, 573, 392, 500, 573, 573, 503, 573, 573, 573, - /* 3420 */ 507, 508, 509, 510, 511, 512, 573, 407, 515, 516, - /* 3430 */ 517, 573, 573, 500, 573, 573, 503, 573, 392, 573, - /* 3440 */ 507, 508, 509, 510, 511, 512, 573, 573, 515, 516, - /* 3450 */ 517, 573, 573, 407, 573, 573, 573, 437, 573, 573, - /* 3460 */ 573, 392, 573, 573, 573, 573, 573, 573, 573, 573, - /* 3470 */ 573, 573, 573, 453, 573, 455, 407, 573, 573, 573, - /* 3480 */ 573, 392, 573, 437, 573, 573, 573, 573, 573, 573, - /* 3490 */ 573, 573, 573, 573, 573, 573, 407, 573, 573, 453, - /* 3500 */ 573, 455, 573, 392, 573, 573, 437, 573, 573, 573, - /* 3510 */ 573, 573, 573, 573, 573, 573, 573, 573, 407, 573, - /* 3520 */ 500, 573, 453, 503, 455, 573, 437, 507, 508, 509, - /* 3530 */ 510, 511, 512, 573, 573, 515, 516, 517, 573, 573, - /* 3540 */ 573, 573, 453, 573, 455, 573, 500, 573, 437, 503, - /* 3550 */ 573, 573, 573, 507, 508, 509, 510, 511, 512, 573, - /* 3560 */ 573, 515, 516, 517, 453, 573, 455, 573, 392, 500, - /* 3570 */ 573, 573, 503, 573, 573, 573, 507, 508, 509, 510, - /* 3580 */ 511, 512, 573, 407, 515, 516, 517, 392, 573, 500, - /* 3590 */ 573, 573, 503, 573, 573, 573, 507, 508, 509, 510, - /* 3600 */ 511, 512, 407, 573, 515, 516, 517, 573, 573, 392, - /* 3610 */ 573, 500, 573, 437, 503, 573, 573, 573, 507, 508, - /* 3620 */ 509, 510, 511, 512, 407, 573, 515, 516, 517, 453, - /* 3630 */ 573, 455, 437, 573, 573, 573, 573, 573, 573, 573, - /* 3640 */ 573, 573, 573, 573, 573, 573, 573, 573, 453, 573, - /* 3650 */ 455, 573, 573, 573, 437, 573, 573, 573, 573, 573, - /* 3660 */ 573, 573, 573, 573, 573, 573, 573, 573, 573, 573, - /* 3670 */ 453, 573, 455, 573, 392, 573, 500, 573, 573, 503, - /* 3680 */ 573, 573, 573, 507, 508, 509, 510, 511, 512, 407, - /* 3690 */ 573, 515, 516, 517, 573, 500, 573, 573, 503, 573, - /* 3700 */ 573, 573, 507, 508, 509, 510, 511, 512, 573, 573, - /* 3710 */ 515, 516, 517, 573, 573, 573, 573, 500, 573, 437, - /* 3720 */ 503, 573, 573, 392, 507, 508, 509, 510, 511, 512, - /* 3730 */ 573, 573, 515, 516, 517, 453, 573, 455, 407, 573, - /* 3740 */ 573, 573, 573, 392, 573, 573, 573, 573, 573, 573, - /* 3750 */ 573, 573, 573, 573, 573, 573, 573, 573, 407, 573, - /* 3760 */ 573, 573, 573, 392, 573, 573, 573, 573, 437, 573, - /* 3770 */ 573, 573, 573, 573, 573, 573, 573, 573, 407, 573, - /* 3780 */ 573, 573, 500, 573, 453, 503, 455, 573, 437, 507, - /* 3790 */ 508, 509, 510, 511, 512, 573, 573, 515, 516, 517, - /* 3800 */ 573, 573, 573, 573, 453, 573, 455, 573, 437, 573, - /* 3810 */ 573, 573, 573, 573, 573, 573, 573, 573, 573, 573, - /* 3820 */ 573, 573, 573, 573, 453, 573, 455, 573, 573, 573, - /* 3830 */ 573, 500, 573, 573, 503, 573, 573, 573, 507, 508, - /* 3840 */ 509, 510, 511, 512, 573, 573, 515, 516, 517, 573, - /* 3850 */ 392, 500, 573, 573, 503, 573, 573, 573, 507, 508, - /* 3860 */ 509, 510, 511, 512, 573, 407, 515, 516, 517, 392, - /* 3870 */ 573, 500, 573, 573, 503, 573, 573, 573, 507, 508, - /* 3880 */ 509, 510, 511, 512, 407, 573, 515, 516, 517, 392, - /* 3890 */ 573, 573, 573, 573, 573, 437, 573, 573, 573, 573, - /* 3900 */ 573, 573, 573, 573, 407, 573, 573, 573, 573, 573, - /* 3910 */ 573, 453, 573, 455, 437, 573, 573, 573, 573, 573, - /* 3920 */ 573, 573, 573, 573, 573, 573, 573, 573, 573, 573, - /* 3930 */ 453, 573, 455, 573, 437, 573, 573, 573, 573, 573, - /* 3940 */ 573, 573, 573, 573, 573, 573, 573, 573, 573, 573, - /* 3950 */ 453, 573, 455, 573, 573, 573, 392, 573, 500, 573, - /* 3960 */ 573, 503, 573, 573, 573, 507, 508, 509, 510, 511, - /* 3970 */ 512, 407, 573, 515, 516, 517, 392, 500, 573, 573, - /* 3980 */ 503, 573, 573, 573, 507, 508, 509, 510, 511, 512, - /* 3990 */ 573, 407, 515, 516, 517, 392, 573, 500, 573, 573, - /* 4000 */ 503, 437, 573, 573, 507, 508, 509, 510, 511, 512, - /* 4010 */ 407, 573, 515, 516, 517, 573, 573, 453, 573, 455, - /* 4020 */ 573, 437, 573, 573, 573, 573, 573, 573, 573, 573, - /* 4030 */ 573, 573, 573, 573, 573, 573, 573, 453, 573, 455, - /* 4040 */ 437, 573, 573, 573, 573, 573, 573, 573, 573, 573, - /* 4050 */ 573, 573, 573, 573, 573, 573, 453, 573, 455, 573, - /* 4060 */ 573, 573, 573, 392, 500, 573, 573, 503, 573, 573, - /* 4070 */ 573, 507, 508, 509, 510, 511, 512, 573, 407, 515, - /* 4080 */ 516, 517, 392, 573, 500, 573, 573, 503, 573, 573, - /* 4090 */ 573, 507, 508, 509, 510, 511, 512, 407, 573, 515, - /* 4100 */ 516, 517, 573, 500, 573, 573, 503, 573, 437, 573, - /* 4110 */ 507, 508, 509, 510, 511, 512, 573, 573, 515, 516, - /* 4120 */ 517, 573, 573, 573, 453, 573, 455, 437, 573, 573, - /* 4130 */ 573, 392, 573, 573, 573, 573, 573, 573, 573, 573, - /* 4140 */ 573, 573, 573, 453, 573, 455, 407, 573, 573, 573, - /* 4150 */ 573, 392, 573, 573, 573, 573, 573, 573, 573, 573, - /* 4160 */ 573, 573, 573, 573, 573, 573, 407, 573, 573, 573, - /* 4170 */ 573, 500, 573, 573, 503, 573, 437, 573, 507, 508, - /* 4180 */ 509, 510, 511, 512, 573, 573, 515, 516, 517, 573, - /* 4190 */ 500, 573, 453, 503, 455, 573, 437, 507, 508, 509, - /* 4200 */ 510, 511, 512, 573, 573, 515, 516, 517, 573, 573, - /* 4210 */ 573, 573, 453, 573, 455, 573, 573, 573, 573, 392, - /* 4220 */ 573, 573, 573, 573, 573, 573, 573, 573, 573, 573, - /* 4230 */ 573, 573, 573, 573, 407, 573, 573, 573, 573, 500, - /* 4240 */ 573, 573, 503, 392, 573, 573, 507, 508, 509, 510, - /* 4250 */ 511, 512, 573, 573, 515, 516, 517, 573, 407, 500, - /* 4260 */ 573, 573, 503, 573, 437, 573, 507, 508, 509, 510, - /* 4270 */ 511, 512, 573, 573, 515, 516, 517, 573, 573, 573, - /* 4280 */ 453, 573, 455, 573, 573, 573, 573, 573, 437, 573, - /* 4290 */ 573, 573, 573, 573, 573, 573, 573, 573, 573, 573, - /* 4300 */ 573, 573, 573, 573, 453, 573, 455, 573, 573, 573, - /* 4310 */ 573, 573, 573, 573, 573, 573, 573, 573, 573, 573, - /* 4320 */ 573, 573, 573, 573, 573, 392, 573, 500, 573, 573, - /* 4330 */ 503, 573, 573, 573, 507, 508, 509, 510, 511, 512, - /* 4340 */ 407, 573, 515, 516, 517, 573, 573, 573, 573, 392, - /* 4350 */ 573, 500, 573, 573, 503, 573, 573, 573, 507, 508, - /* 4360 */ 509, 510, 511, 512, 407, 573, 515, 516, 517, 573, - /* 4370 */ 437, 573, 573, 573, 573, 573, 573, 573, 573, 573, - /* 4380 */ 573, 573, 573, 573, 573, 573, 453, 573, 455, 573, - /* 4390 */ 573, 573, 573, 573, 437, 573, 573, 573, 573, 573, - /* 4400 */ 573, 573, 573, 573, 573, 573, 573, 573, 573, 573, - /* 4410 */ 453, 573, 455, 573, 573, 573, 573, 573, 573, 573, - /* 4420 */ 573, 573, 573, 573, 573, 573, 573, 573, 573, 573, - /* 4430 */ 573, 573, 573, 500, 573, 573, 503, 573, 573, 573, - /* 4440 */ 507, 508, 509, 510, 511, 512, 573, 573, 515, 516, - /* 4450 */ 517, 573, 573, 573, 573, 573, 573, 500, 573, 573, - /* 4460 */ 503, 573, 573, 573, 507, 508, 509, 510, 511, 512, - /* 4470 */ 573, 573, 515, 516, 517, 389, 389, 389, 389, 389, - /* 4480 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4490 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4500 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4510 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4520 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4530 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4540 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4550 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4560 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4570 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4580 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4590 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4600 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4610 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4620 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4630 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4640 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4650 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4660 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4670 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4680 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4690 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4700 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4710 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4720 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4730 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4740 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4750 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4760 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4770 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4780 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4790 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4800 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4810 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4820 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4830 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4840 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4850 */ 389, 389, 389, 389, 389, 389, 389, 389, 389, 389, - /* 4860 */ 389, 389, 389, 389, + /* 270 */ 270, 271, 272, 408, 118, 275, 276, 277, 278, 279, + /* 280 */ 280, 281, 282, 283, 284, 285, 286, 287, 288, 289, + /* 290 */ 12, 13, 14, 57, 393, 18, 77, 20, 20, 0, + /* 300 */ 22, 189, 66, 438, 27, 69, 70, 30, 438, 408, + /* 310 */ 257, 490, 491, 149, 36, 38, 38, 447, 448, 454, + /* 320 */ 314, 456, 20, 24, 25, 26, 27, 28, 29, 30, + /* 330 */ 31, 32, 480, 56, 57, 223, 59, 485, 20, 438, + /* 340 */ 158, 64, 65, 320, 321, 322, 323, 324, 314, 71, + /* 350 */ 407, 408, 74, 76, 403, 454, 78, 456, 407, 537, + /* 360 */ 409, 274, 14, 85, 118, 438, 501, 545, 20, 504, + /* 370 */ 304, 305, 306, 508, 509, 510, 511, 512, 513, 514, + /* 380 */ 20, 516, 517, 518, 519, 520, 564, 456, 0, 537, + /* 390 */ 568, 569, 465, 115, 117, 20, 118, 545, 234, 235, + /* 400 */ 236, 470, 501, 407, 408, 504, 129, 476, 477, 508, + /* 410 */ 509, 510, 511, 512, 513, 563, 564, 516, 517, 518, + /* 420 */ 568, 569, 521, 427, 523, 524, 525, 12, 13, 428, + /* 430 */ 529, 530, 436, 155, 156, 314, 20, 160, 161, 438, + /* 440 */ 163, 164, 165, 166, 167, 168, 169, 170, 171, 172, + /* 450 */ 449, 36, 175, 176, 177, 178, 179, 180, 181, 182, + /* 460 */ 20, 184, 185, 186, 218, 77, 220, 190, 191, 192, + /* 470 */ 314, 438, 194, 195, 197, 532, 533, 534, 535, 44, + /* 480 */ 537, 538, 204, 205, 437, 303, 304, 305, 306, 307, + /* 490 */ 308, 309, 310, 311, 3, 2, 36, 219, 252, 221, + /* 500 */ 453, 8, 9, 36, 226, 12, 13, 14, 15, 16, + /* 510 */ 33, 20, 194, 8, 9, 155, 156, 12, 13, 14, + /* 520 */ 15, 16, 8, 9, 526, 527, 12, 13, 14, 15, + /* 530 */ 16, 498, 499, 255, 256, 257, 393, 259, 260, 261, + /* 540 */ 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, + /* 550 */ 272, 408, 85, 275, 276, 277, 278, 279, 38, 257, + /* 560 */ 314, 283, 284, 285, 286, 287, 288, 289, 12, 13, + /* 570 */ 129, 155, 156, 20, 54, 257, 20, 57, 22, 41, + /* 580 */ 42, 438, 417, 78, 433, 115, 66, 67, 68, 69, + /* 590 */ 425, 71, 36, 158, 38, 0, 119, 454, 424, 456, + /* 600 */ 130, 131, 132, 133, 134, 135, 136, 137, 138, 139, + /* 610 */ 119, 141, 142, 143, 144, 145, 146, 147, 8, 9, + /* 620 */ 204, 205, 12, 13, 14, 15, 16, 71, 454, 20, + /* 630 */ 74, 22, 480, 128, 78, 507, 221, 117, 407, 408, + /* 640 */ 423, 85, 122, 426, 501, 36, 495, 504, 20, 4, + /* 650 */ 438, 508, 509, 510, 511, 512, 513, 47, 446, 516, + /* 660 */ 517, 518, 124, 125, 536, 127, 523, 0, 525, 457, + /* 670 */ 61, 115, 529, 530, 118, 80, 81, 82, 504, 505, + /* 680 */ 506, 221, 87, 88, 89, 441, 148, 196, 93, 537, + /* 690 */ 152, 438, 518, 98, 99, 100, 101, 545, 193, 104, + /* 700 */ 557, 14, 14, 108, 109, 110, 111, 20, 20, 189, + /* 710 */ 457, 155, 156, 253, 254, 563, 564, 77, 198, 439, + /* 720 */ 568, 569, 202, 203, 57, 211, 439, 440, 208, 209, + /* 730 */ 18, 441, 456, 290, 444, 23, 407, 408, 303, 304, + /* 740 */ 305, 306, 307, 308, 309, 310, 311, 227, 504, 196, + /* 750 */ 194, 195, 476, 477, 42, 43, 427, 22, 46, 76, + /* 760 */ 204, 205, 518, 118, 533, 534, 535, 55, 537, 538, + /* 770 */ 0, 36, 60, 438, 441, 219, 451, 221, 273, 454, + /* 780 */ 500, 446, 226, 85, 72, 73, 74, 75, 76, 255, + /* 790 */ 407, 408, 457, 407, 504, 480, 390, 0, 293, 294, + /* 800 */ 295, 296, 297, 298, 299, 300, 301, 302, 518, 129, + /* 810 */ 427, 255, 256, 257, 44, 259, 260, 261, 262, 263, + /* 820 */ 264, 265, 266, 267, 268, 269, 270, 271, 272, 0, + /* 830 */ 118, 275, 276, 277, 278, 279, 290, 504, 292, 283, + /* 840 */ 284, 285, 286, 287, 288, 289, 290, 12, 13, 71, + /* 850 */ 115, 518, 537, 13, 456, 20, 342, 22, 472, 473, + /* 860 */ 545, 327, 328, 329, 330, 331, 332, 333, 470, 157, + /* 870 */ 0, 36, 537, 38, 476, 477, 36, 393, 563, 564, + /* 880 */ 545, 194, 194, 568, 569, 257, 480, 521, 407, 408, + /* 890 */ 524, 485, 408, 438, 410, 117, 78, 214, 563, 564, + /* 900 */ 122, 403, 408, 568, 569, 407, 71, 409, 427, 74, + /* 910 */ 407, 408, 457, 78, 307, 308, 309, 310, 311, 522, + /* 920 */ 85, 524, 438, 211, 212, 213, 243, 244, 216, 8, + /* 930 */ 9, 0, 438, 12, 13, 14, 15, 16, 454, 0, + /* 940 */ 456, 229, 230, 537, 257, 257, 407, 408, 407, 408, + /* 950 */ 115, 545, 4, 118, 242, 407, 408, 245, 407, 314, + /* 960 */ 248, 249, 250, 251, 252, 462, 427, 189, 47, 563, + /* 970 */ 564, 174, 33, 22, 568, 569, 179, 148, 149, 150, + /* 980 */ 151, 152, 153, 154, 187, 501, 507, 36, 504, 58, + /* 990 */ 155, 156, 508, 509, 510, 511, 512, 513, 416, 507, + /* 1000 */ 516, 517, 518, 462, 510, 407, 408, 523, 13, 525, + /* 1010 */ 462, 407, 408, 529, 530, 536, 1, 435, 148, 149, + /* 1020 */ 150, 151, 152, 153, 154, 427, 314, 445, 536, 194, + /* 1030 */ 195, 427, 407, 408, 19, 439, 85, 407, 408, 204, + /* 1040 */ 205, 490, 491, 8, 9, 407, 408, 12, 13, 14, + /* 1050 */ 15, 16, 427, 38, 219, 439, 221, 427, 36, 407, + /* 1060 */ 408, 226, 412, 413, 2, 427, 115, 393, 53, 54, + /* 1070 */ 8, 9, 57, 255, 12, 13, 14, 15, 16, 427, + /* 1080 */ 85, 66, 67, 68, 69, 193, 71, 20, 416, 23, + /* 1090 */ 255, 256, 257, 439, 259, 260, 261, 262, 263, 264, + /* 1100 */ 265, 266, 267, 268, 269, 270, 271, 272, 407, 408, + /* 1110 */ 275, 276, 277, 278, 279, 49, 50, 445, 283, 284, + /* 1120 */ 285, 286, 287, 288, 289, 12, 13, 14, 454, 393, + /* 1130 */ 407, 408, 117, 20, 4, 22, 480, 122, 407, 408, + /* 1140 */ 455, 456, 4, 150, 408, 438, 410, 154, 429, 36, + /* 1150 */ 427, 38, 433, 23, 119, 8, 9, 19, 427, 12, + /* 1160 */ 13, 14, 15, 16, 457, 273, 393, 438, 153, 13, + /* 1170 */ 219, 52, 221, 44, 438, 446, 38, 414, 48, 49, + /* 1180 */ 50, 408, 71, 410, 71, 293, 457, 74, 62, 63, + /* 1190 */ 454, 438, 456, 537, 431, 57, 407, 408, 85, 446, + /* 1200 */ 71, 545, 64, 65, 3, 442, 255, 256, 33, 71, + /* 1210 */ 457, 438, 522, 198, 524, 200, 427, 393, 203, 563, + /* 1220 */ 564, 228, 47, 208, 568, 569, 233, 454, 115, 456, + /* 1230 */ 237, 118, 239, 122, 533, 534, 535, 501, 537, 538, + /* 1240 */ 504, 85, 227, 221, 508, 509, 510, 511, 512, 513, + /* 1250 */ 407, 408, 516, 517, 518, 117, 20, 425, 22, 523, + /* 1260 */ 122, 525, 480, 196, 316, 529, 530, 393, 155, 156, + /* 1270 */ 427, 14, 15, 16, 501, 253, 254, 504, 454, 412, + /* 1280 */ 413, 508, 509, 510, 511, 512, 513, 421, 422, 516, + /* 1290 */ 517, 518, 80, 81, 82, 33, 523, 61, 525, 87, + /* 1300 */ 88, 89, 529, 530, 428, 93, 438, 194, 195, 47, + /* 1310 */ 98, 99, 100, 101, 438, 196, 104, 204, 205, 537, + /* 1320 */ 108, 109, 110, 111, 257, 449, 207, 545, 454, 421, + /* 1330 */ 422, 20, 219, 0, 221, 20, 8, 9, 0, 226, + /* 1340 */ 12, 13, 14, 15, 16, 563, 564, 394, 8, 9, + /* 1350 */ 568, 569, 12, 13, 14, 15, 16, 393, 8, 9, + /* 1360 */ 393, 439, 12, 13, 14, 15, 16, 499, 255, 256, + /* 1370 */ 257, 393, 259, 260, 261, 262, 263, 264, 265, 266, + /* 1380 */ 267, 268, 269, 270, 271, 272, 33, 47, 275, 276, + /* 1390 */ 277, 278, 279, 396, 397, 20, 283, 284, 285, 286, + /* 1400 */ 287, 288, 289, 12, 13, 407, 408, 393, 407, 408, + /* 1410 */ 393, 20, 22, 22, 451, 480, 451, 454, 454, 454, + /* 1420 */ 273, 454, 408, 439, 410, 427, 36, 36, 427, 38, + /* 1430 */ 407, 408, 454, 0, 393, 148, 149, 150, 151, 152, + /* 1440 */ 153, 154, 14, 0, 407, 408, 407, 408, 20, 408, + /* 1450 */ 427, 410, 438, 407, 408, 407, 408, 148, 393, 119, + /* 1460 */ 439, 152, 71, 438, 427, 74, 427, 33, 454, 119, + /* 1470 */ 456, 454, 537, 427, 449, 427, 85, 8, 9, 438, + /* 1480 */ 545, 12, 13, 14, 15, 16, 8, 9, 36, 13, + /* 1490 */ 12, 13, 14, 15, 16, 454, 158, 456, 563, 564, + /* 1500 */ 393, 393, 33, 568, 569, 115, 115, 196, 393, 118, + /* 1510 */ 119, 173, 36, 393, 393, 501, 393, 393, 504, 454, + /* 1520 */ 393, 393, 508, 509, 510, 511, 512, 513, 393, 393, + /* 1530 */ 516, 517, 518, 393, 4, 408, 33, 523, 337, 525, + /* 1540 */ 55, 466, 501, 529, 530, 504, 155, 156, 129, 508, + /* 1550 */ 509, 510, 511, 512, 513, 33, 36, 516, 517, 518, + /* 1560 */ 123, 454, 454, 126, 523, 438, 525, 539, 257, 454, + /* 1570 */ 529, 530, 257, 123, 454, 454, 126, 454, 454, 33, + /* 1580 */ 247, 454, 454, 456, 54, 194, 195, 0, 119, 454, + /* 1590 */ 454, 158, 0, 0, 454, 204, 205, 119, 33, 572, + /* 1600 */ 123, 158, 183, 126, 123, 85, 231, 126, 0, 22, + /* 1610 */ 219, 33, 221, 33, 22, 22, 33, 226, 33, 8, + /* 1620 */ 9, 51, 119, 12, 13, 14, 15, 16, 501, 561, + /* 1630 */ 22, 504, 554, 1, 2, 508, 509, 510, 511, 512, + /* 1640 */ 513, 119, 36, 516, 517, 518, 255, 256, 257, 33, + /* 1650 */ 259, 260, 261, 262, 263, 264, 265, 266, 267, 268, + /* 1660 */ 269, 270, 271, 272, 33, 119, 275, 276, 277, 278, + /* 1670 */ 279, 155, 156, 221, 283, 284, 285, 286, 287, 288, + /* 1680 */ 289, 12, 13, 393, 119, 558, 559, 411, 118, 20, + /* 1690 */ 33, 22, 339, 12, 13, 540, 33, 119, 408, 119, + /* 1700 */ 410, 438, 119, 33, 119, 36, 33, 38, 424, 424, + /* 1710 */ 33, 33, 12, 13, 8, 9, 47, 466, 12, 13, + /* 1720 */ 14, 15, 16, 238, 406, 33, 33, 118, 438, 120, + /* 1730 */ 119, 33, 12, 13, 560, 119, 12, 13, 12, 13, + /* 1740 */ 71, 12, 13, 74, 454, 33, 456, 33, 12, 13, + /* 1750 */ 119, 12, 13, 560, 85, 8, 9, 12, 13, 12, + /* 1760 */ 13, 14, 15, 16, 8, 9, 33, 33, 12, 13, + /* 1770 */ 14, 15, 16, 8, 9, 341, 119, 12, 13, 14, + /* 1780 */ 15, 16, 119, 33, 115, 12, 13, 118, 36, 119, + /* 1790 */ 33, 501, 119, 13, 504, 393, 119, 119, 508, 509, + /* 1800 */ 510, 511, 512, 513, 12, 13, 516, 517, 518, 13, + /* 1810 */ 408, 119, 119, 523, 560, 525, 36, 119, 36, 529, + /* 1820 */ 530, 479, 466, 411, 155, 156, 560, 221, 502, 408, + /* 1830 */ 453, 119, 36, 119, 487, 466, 466, 85, 544, 544, + /* 1840 */ 438, 531, 565, 547, 466, 317, 426, 481, 57, 503, + /* 1850 */ 20, 407, 119, 119, 20, 492, 454, 237, 456, 497, + /* 1860 */ 492, 416, 416, 194, 195, 217, 119, 85, 483, 119, + /* 1870 */ 407, 20, 408, 204, 205, 119, 119, 463, 47, 408, + /* 1880 */ 463, 193, 460, 408, 119, 460, 407, 407, 219, 463, + /* 1890 */ 221, 460, 460, 116, 420, 226, 407, 114, 407, 113, + /* 1900 */ 407, 419, 196, 501, 418, 407, 504, 407, 20, 400, + /* 1910 */ 508, 509, 510, 511, 512, 513, 52, 404, 516, 517, + /* 1920 */ 518, 400, 404, 416, 255, 256, 257, 492, 259, 260, + /* 1930 */ 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, + /* 1940 */ 271, 272, 416, 20, 275, 276, 277, 278, 279, 416, + /* 1950 */ 456, 20, 283, 284, 285, 286, 287, 288, 289, 12, + /* 1960 */ 13, 409, 20, 416, 393, 482, 409, 20, 20, 22, + /* 1970 */ 416, 416, 570, 571, 473, 20, 416, 467, 416, 408, + /* 1980 */ 52, 407, 434, 36, 434, 38, 400, 438, 416, 438, + /* 1990 */ 396, 438, 396, 454, 438, 407, 438, 438, 400, 438, + /* 2000 */ 240, 438, 438, 118, 438, 438, 494, 496, 20, 438, + /* 2010 */ 225, 492, 224, 414, 491, 414, 489, 488, 71, 481, + /* 2020 */ 407, 74, 454, 454, 325, 454, 553, 456, 326, 456, + /* 2030 */ 454, 334, 85, 210, 318, 313, 335, 553, 543, 336, + /* 2040 */ 312, 553, 552, 556, 340, 12, 13, 555, 551, 474, + /* 2050 */ 542, 474, 550, 343, 408, 22, 338, 20, 129, 315, + /* 2060 */ 481, 393, 115, 567, 409, 118, 414, 414, 454, 36, + /* 2070 */ 202, 38, 501, 548, 474, 504, 408, 573, 546, 508, + /* 2080 */ 509, 510, 511, 512, 513, 414, 474, 516, 517, 518, + /* 2090 */ 566, 507, 454, 454, 523, 454, 525, 454, 471, 454, + /* 2100 */ 529, 530, 155, 156, 71, 414, 438, 467, 118, 528, + /* 2110 */ 454, 202, 468, 433, 467, 454, 408, 118, 85, 414, + /* 2120 */ 454, 414, 454, 22, 456, 395, 454, 35, 454, 454, + /* 2130 */ 454, 37, 407, 454, 484, 398, 454, 454, 454, 454, + /* 2140 */ 454, 194, 195, 454, 454, 414, 40, 399, 115, 454, + /* 2150 */ 454, 204, 205, 430, 401, 400, 415, 493, 443, 440, + /* 2160 */ 500, 391, 475, 454, 0, 454, 219, 454, 221, 501, + /* 2170 */ 430, 454, 504, 226, 0, 454, 508, 509, 510, 511, + /* 2180 */ 512, 513, 440, 0, 516, 517, 518, 47, 430, 454, + /* 2190 */ 0, 523, 454, 525, 36, 246, 36, 529, 530, 36, + /* 2200 */ 246, 36, 255, 256, 257, 0, 259, 260, 261, 262, + /* 2210 */ 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, + /* 2220 */ 36, 475, 275, 276, 277, 278, 279, 36, 246, 36, + /* 2230 */ 283, 284, 285, 286, 287, 288, 289, 12, 13, 0, + /* 2240 */ 0, 246, 0, 36, 0, 20, 36, 22, 0, 22, + /* 2250 */ 0, 36, 219, 241, 221, 0, 227, 0, 227, 221, + /* 2260 */ 228, 36, 219, 38, 0, 0, 0, 393, 215, 214, + /* 2270 */ 0, 0, 161, 51, 51, 0, 0, 36, 0, 36, + /* 2280 */ 0, 57, 408, 51, 0, 47, 0, 0, 255, 256, + /* 2290 */ 0, 0, 0, 0, 0, 0, 71, 51, 0, 74, + /* 2300 */ 179, 0, 179, 36, 0, 0, 0, 0, 275, 276, + /* 2310 */ 85, 0, 438, 0, 0, 0, 283, 284, 285, 286, + /* 2320 */ 287, 288, 0, 0, 0, 0, 0, 0, 454, 0, + /* 2330 */ 456, 0, 0, 0, 0, 0, 51, 0, 0, 0, + /* 2340 */ 115, 0, 0, 118, 47, 0, 0, 0, 0, 0, + /* 2350 */ 0, 0, 22, 0, 0, 161, 160, 159, 0, 0, + /* 2360 */ 52, 52, 22, 0, 0, 22, 0, 71, 0, 0, + /* 2370 */ 71, 71, 36, 71, 0, 501, 57, 0, 504, 0, + /* 2380 */ 155, 156, 508, 509, 510, 511, 512, 513, 36, 0, + /* 2390 */ 516, 517, 518, 36, 0, 0, 36, 523, 0, 525, + /* 2400 */ 44, 57, 14, 529, 530, 44, 44, 36, 36, 0, + /* 2410 */ 0, 56, 393, 33, 47, 0, 0, 44, 0, 194, + /* 2420 */ 195, 51, 57, 45, 210, 0, 51, 408, 44, 204, + /* 2430 */ 205, 44, 51, 51, 0, 0, 51, 0, 0, 0, + /* 2440 */ 44, 36, 0, 57, 219, 44, 221, 36, 57, 0, + /* 2450 */ 36, 226, 44, 0, 57, 36, 57, 438, 0, 0, + /* 2460 */ 0, 44, 79, 0, 0, 36, 0, 0, 36, 22, + /* 2470 */ 0, 59, 22, 454, 22, 456, 36, 36, 33, 36, + /* 2480 */ 255, 256, 257, 22, 259, 260, 261, 262, 263, 264, + /* 2490 */ 265, 266, 267, 268, 269, 270, 271, 272, 36, 0, + /* 2500 */ 275, 276, 277, 278, 279, 36, 33, 36, 283, 284, + /* 2510 */ 285, 286, 287, 288, 289, 12, 13, 36, 393, 36, + /* 2520 */ 501, 128, 36, 504, 0, 22, 126, 508, 509, 510, + /* 2530 */ 511, 512, 513, 408, 36, 516, 517, 518, 36, 36, + /* 2540 */ 36, 38, 22, 0, 525, 393, 0, 22, 529, 530, + /* 2550 */ 22, 36, 0, 0, 0, 36, 0, 36, 0, 36, + /* 2560 */ 408, 0, 22, 438, 20, 36, 36, 36, 0, 51, + /* 2570 */ 119, 0, 118, 118, 71, 36, 0, 22, 0, 454, + /* 2580 */ 0, 456, 22, 33, 3, 393, 196, 319, 22, 51, + /* 2590 */ 438, 119, 118, 118, 52, 119, 118, 52, 119, 196, + /* 2600 */ 408, 36, 196, 36, 114, 118, 454, 116, 456, 202, + /* 2610 */ 33, 119, 393, 119, 118, 33, 196, 196, 118, 118, + /* 2620 */ 33, 51, 33, 118, 232, 119, 501, 408, 226, 504, + /* 2630 */ 438, 319, 206, 508, 509, 510, 511, 512, 513, 206, + /* 2640 */ 33, 516, 517, 518, 231, 222, 454, 85, 456, 36, + /* 2650 */ 525, 119, 118, 501, 529, 530, 504, 438, 3, 33, + /* 2660 */ 508, 509, 510, 511, 512, 513, 119, 119, 516, 517, + /* 2670 */ 518, 36, 36, 454, 36, 456, 36, 525, 36, 36, + /* 2680 */ 51, 529, 530, 119, 33, 51, 0, 0, 119, 44, + /* 2690 */ 118, 0, 119, 501, 119, 0, 504, 118, 44, 33, + /* 2700 */ 508, 509, 510, 511, 512, 513, 118, 515, 516, 517, + /* 2710 */ 518, 199, 118, 203, 119, 118, 199, 44, 116, 291, + /* 2720 */ 501, 116, 219, 504, 221, 198, 2, 508, 509, 510, + /* 2730 */ 511, 512, 513, 303, 22, 516, 517, 518, 118, 255, + /* 2740 */ 118, 319, 118, 118, 525, 118, 0, 51, 118, 530, + /* 2750 */ 119, 119, 119, 51, 22, 118, 253, 254, 255, 118, + /* 2760 */ 44, 0, 118, 22, 118, 118, 118, 258, 118, 393, + /* 2770 */ 51, 119, 119, 199, 118, 118, 118, 118, 275, 276, + /* 2780 */ 120, 118, 121, 118, 408, 118, 283, 284, 285, 286, + /* 2790 */ 287, 288, 22, 119, 118, 232, 119, 393, 36, 129, + /* 2800 */ 36, 118, 36, 119, 119, 36, 118, 36, 119, 119, + /* 2810 */ 36, 119, 408, 36, 438, 140, 118, 33, 140, 140, + /* 2820 */ 140, 118, 36, 118, 22, 22, 79, 78, 22, 36, + /* 2830 */ 454, 36, 456, 36, 36, 36, 36, 36, 85, 112, + /* 2840 */ 36, 36, 438, 36, 36, 85, 112, 33, 36, 36, + /* 2850 */ 22, 0, 36, 36, 478, 36, 36, 85, 454, 36, + /* 2860 */ 456, 36, 36, 36, 22, 36, 36, 36, 57, 0, + /* 2870 */ 36, 0, 36, 0, 0, 57, 44, 501, 57, 44, + /* 2880 */ 504, 393, 44, 36, 508, 509, 510, 511, 512, 513, + /* 2890 */ 57, 36, 516, 517, 518, 0, 408, 44, 22, 0, + /* 2900 */ 22, 36, 0, 22, 36, 501, 36, 22, 504, 393, + /* 2910 */ 33, 21, 508, 509, 510, 511, 512, 513, 22, 22, + /* 2920 */ 516, 517, 518, 20, 408, 21, 438, 574, 574, 574, + /* 2930 */ 393, 574, 574, 574, 574, 574, 574, 574, 574, 574, + /* 2940 */ 574, 574, 454, 574, 456, 408, 574, 574, 574, 574, + /* 2950 */ 574, 574, 574, 574, 438, 574, 574, 574, 574, 574, + /* 2960 */ 574, 574, 574, 559, 574, 574, 478, 574, 574, 574, + /* 2970 */ 454, 574, 456, 574, 574, 438, 574, 574, 574, 574, + /* 2980 */ 574, 574, 574, 574, 574, 574, 574, 574, 574, 501, + /* 2990 */ 574, 454, 504, 456, 478, 574, 508, 509, 510, 511, + /* 3000 */ 512, 513, 574, 574, 516, 517, 518, 574, 574, 574, + /* 3010 */ 574, 574, 574, 574, 574, 574, 574, 501, 574, 574, + /* 3020 */ 504, 393, 574, 574, 508, 509, 510, 511, 512, 513, + /* 3030 */ 574, 574, 516, 517, 518, 574, 408, 574, 501, 574, + /* 3040 */ 574, 504, 393, 574, 574, 508, 509, 510, 511, 512, + /* 3050 */ 513, 574, 574, 516, 517, 518, 574, 408, 574, 574, + /* 3060 */ 574, 574, 393, 574, 574, 574, 438, 574, 574, 574, + /* 3070 */ 574, 574, 574, 574, 574, 574, 574, 408, 574, 574, + /* 3080 */ 574, 574, 454, 574, 456, 574, 574, 438, 574, 574, + /* 3090 */ 574, 574, 574, 574, 574, 574, 574, 574, 574, 562, + /* 3100 */ 574, 574, 574, 454, 574, 456, 574, 438, 574, 574, + /* 3110 */ 574, 574, 574, 574, 574, 574, 574, 574, 574, 574, + /* 3120 */ 574, 574, 574, 454, 574, 456, 574, 574, 574, 501, + /* 3130 */ 574, 574, 504, 574, 574, 574, 508, 509, 510, 511, + /* 3140 */ 512, 513, 574, 574, 516, 517, 518, 478, 574, 574, + /* 3150 */ 501, 574, 574, 504, 574, 574, 574, 508, 509, 510, + /* 3160 */ 511, 512, 513, 574, 574, 516, 517, 518, 574, 520, + /* 3170 */ 501, 574, 574, 504, 574, 574, 574, 508, 509, 510, + /* 3180 */ 511, 512, 513, 393, 574, 516, 517, 518, 574, 574, + /* 3190 */ 574, 574, 574, 574, 574, 574, 574, 574, 408, 571, + /* 3200 */ 574, 574, 574, 574, 574, 574, 574, 574, 393, 574, + /* 3210 */ 574, 574, 574, 574, 574, 574, 574, 574, 574, 574, + /* 3220 */ 574, 574, 574, 408, 574, 574, 574, 574, 438, 574, + /* 3230 */ 574, 574, 574, 574, 574, 574, 574, 574, 574, 574, + /* 3240 */ 574, 574, 574, 574, 454, 574, 456, 574, 574, 574, + /* 3250 */ 574, 574, 393, 438, 574, 574, 574, 574, 574, 574, + /* 3260 */ 574, 574, 574, 574, 574, 574, 574, 408, 478, 454, + /* 3270 */ 574, 456, 574, 574, 574, 393, 574, 574, 574, 574, + /* 3280 */ 574, 574, 574, 574, 574, 574, 574, 574, 574, 574, + /* 3290 */ 408, 501, 574, 478, 504, 574, 574, 438, 508, 509, + /* 3300 */ 510, 511, 512, 513, 574, 574, 516, 517, 518, 574, + /* 3310 */ 574, 574, 574, 454, 574, 456, 501, 574, 574, 504, + /* 3320 */ 438, 574, 574, 508, 509, 510, 511, 512, 513, 574, + /* 3330 */ 574, 516, 517, 518, 574, 574, 454, 478, 456, 574, + /* 3340 */ 574, 574, 574, 574, 574, 574, 574, 574, 574, 574, + /* 3350 */ 574, 574, 574, 574, 574, 574, 574, 574, 574, 574, + /* 3360 */ 501, 574, 574, 504, 393, 574, 574, 508, 509, 510, + /* 3370 */ 511, 512, 513, 574, 574, 516, 517, 518, 574, 408, + /* 3380 */ 574, 574, 574, 501, 574, 574, 504, 393, 574, 574, + /* 3390 */ 508, 509, 510, 511, 512, 513, 574, 574, 516, 517, + /* 3400 */ 518, 574, 408, 574, 574, 574, 574, 574, 574, 438, + /* 3410 */ 574, 574, 574, 574, 574, 393, 574, 574, 574, 574, + /* 3420 */ 574, 574, 574, 574, 574, 454, 574, 456, 574, 574, + /* 3430 */ 408, 574, 438, 574, 574, 574, 574, 574, 574, 574, + /* 3440 */ 574, 574, 574, 574, 574, 574, 574, 574, 454, 574, + /* 3450 */ 456, 574, 574, 574, 574, 574, 393, 574, 574, 574, + /* 3460 */ 438, 574, 574, 574, 574, 574, 574, 574, 574, 574, + /* 3470 */ 574, 408, 501, 574, 574, 504, 454, 574, 456, 508, + /* 3480 */ 509, 510, 511, 512, 513, 574, 574, 516, 517, 518, + /* 3490 */ 574, 574, 574, 574, 574, 501, 574, 574, 504, 393, + /* 3500 */ 574, 438, 508, 509, 510, 511, 512, 513, 574, 574, + /* 3510 */ 516, 517, 518, 574, 408, 574, 574, 454, 574, 456, + /* 3520 */ 574, 574, 574, 501, 574, 574, 504, 574, 574, 574, + /* 3530 */ 508, 509, 510, 511, 512, 513, 574, 574, 516, 517, + /* 3540 */ 518, 393, 574, 574, 438, 574, 574, 574, 574, 574, + /* 3550 */ 574, 574, 574, 574, 574, 574, 408, 574, 574, 574, + /* 3560 */ 454, 574, 456, 574, 501, 574, 574, 504, 574, 574, + /* 3570 */ 574, 508, 509, 510, 511, 512, 513, 574, 574, 516, + /* 3580 */ 517, 518, 574, 574, 574, 574, 438, 574, 574, 574, + /* 3590 */ 574, 574, 574, 574, 574, 574, 574, 574, 574, 574, + /* 3600 */ 574, 574, 454, 574, 456, 574, 574, 501, 574, 574, + /* 3610 */ 504, 574, 574, 574, 508, 509, 510, 511, 512, 513, + /* 3620 */ 574, 574, 516, 517, 518, 574, 574, 574, 574, 574, + /* 3630 */ 574, 574, 574, 574, 574, 574, 574, 574, 393, 574, + /* 3640 */ 574, 574, 574, 574, 574, 574, 574, 574, 574, 501, + /* 3650 */ 574, 574, 504, 408, 574, 574, 508, 509, 510, 511, + /* 3660 */ 512, 513, 393, 574, 516, 517, 518, 574, 574, 574, + /* 3670 */ 574, 574, 574, 574, 574, 574, 574, 408, 574, 574, + /* 3680 */ 574, 574, 574, 438, 574, 574, 574, 393, 574, 574, + /* 3690 */ 574, 574, 574, 574, 574, 574, 574, 574, 574, 454, + /* 3700 */ 574, 456, 408, 574, 574, 574, 574, 438, 574, 574, + /* 3710 */ 574, 574, 574, 574, 574, 574, 574, 574, 574, 574, + /* 3720 */ 574, 574, 574, 454, 574, 456, 574, 393, 574, 574, + /* 3730 */ 574, 574, 438, 574, 574, 574, 574, 574, 574, 574, + /* 3740 */ 574, 574, 408, 574, 574, 574, 501, 574, 454, 504, + /* 3750 */ 456, 574, 574, 508, 509, 510, 511, 512, 513, 574, + /* 3760 */ 574, 516, 517, 518, 574, 574, 574, 574, 574, 393, + /* 3770 */ 501, 574, 438, 504, 574, 574, 574, 508, 509, 510, + /* 3780 */ 511, 512, 513, 574, 408, 516, 517, 518, 454, 574, + /* 3790 */ 456, 574, 574, 574, 393, 501, 574, 574, 504, 574, + /* 3800 */ 574, 574, 508, 509, 510, 511, 512, 513, 574, 408, + /* 3810 */ 516, 517, 518, 574, 438, 574, 574, 574, 574, 574, + /* 3820 */ 574, 574, 574, 574, 574, 574, 574, 574, 574, 574, + /* 3830 */ 454, 574, 456, 574, 574, 501, 574, 574, 504, 438, + /* 3840 */ 574, 574, 508, 509, 510, 511, 512, 513, 574, 574, + /* 3850 */ 516, 517, 518, 574, 574, 454, 574, 456, 574, 574, + /* 3860 */ 574, 574, 574, 574, 574, 574, 574, 574, 574, 574, + /* 3870 */ 574, 574, 574, 574, 574, 393, 574, 501, 574, 574, + /* 3880 */ 504, 574, 574, 574, 508, 509, 510, 511, 512, 513, + /* 3890 */ 408, 574, 516, 517, 518, 574, 574, 574, 574, 574, + /* 3900 */ 393, 574, 501, 574, 574, 504, 574, 574, 574, 508, + /* 3910 */ 509, 510, 511, 512, 513, 408, 574, 516, 517, 518, + /* 3920 */ 438, 574, 574, 574, 393, 574, 574, 574, 574, 574, + /* 3930 */ 574, 574, 574, 574, 574, 574, 454, 574, 456, 408, + /* 3940 */ 574, 574, 574, 574, 574, 438, 574, 574, 574, 574, + /* 3950 */ 574, 574, 574, 574, 574, 574, 574, 574, 574, 574, + /* 3960 */ 574, 454, 574, 456, 574, 393, 574, 574, 574, 438, + /* 3970 */ 574, 574, 574, 574, 574, 574, 574, 574, 574, 574, + /* 3980 */ 408, 574, 574, 501, 574, 454, 504, 456, 574, 574, + /* 3990 */ 508, 509, 510, 511, 512, 513, 574, 574, 516, 517, + /* 4000 */ 518, 574, 574, 574, 574, 574, 574, 574, 501, 574, + /* 4010 */ 438, 504, 574, 574, 574, 508, 509, 510, 511, 512, + /* 4020 */ 513, 574, 574, 516, 517, 518, 454, 574, 456, 574, + /* 4030 */ 574, 574, 501, 574, 574, 504, 574, 574, 574, 508, + /* 4040 */ 509, 510, 511, 512, 513, 574, 393, 516, 517, 518, + /* 4050 */ 574, 574, 574, 574, 574, 574, 574, 574, 574, 574, + /* 4060 */ 574, 408, 574, 574, 574, 574, 574, 574, 574, 574, + /* 4070 */ 574, 393, 574, 501, 574, 574, 504, 574, 574, 574, + /* 4080 */ 508, 509, 510, 511, 512, 513, 408, 574, 516, 517, + /* 4090 */ 518, 438, 574, 574, 574, 393, 574, 574, 574, 574, + /* 4100 */ 574, 574, 574, 574, 574, 574, 574, 454, 574, 456, + /* 4110 */ 408, 574, 574, 574, 574, 574, 438, 574, 574, 574, + /* 4120 */ 574, 574, 574, 574, 574, 574, 574, 574, 574, 574, + /* 4130 */ 574, 574, 454, 574, 456, 574, 393, 574, 574, 574, + /* 4140 */ 438, 574, 574, 574, 574, 574, 574, 574, 574, 574, + /* 4150 */ 574, 408, 574, 574, 501, 574, 454, 504, 456, 574, + /* 4160 */ 574, 508, 509, 510, 511, 512, 513, 574, 574, 516, + /* 4170 */ 517, 518, 574, 574, 574, 574, 574, 574, 393, 501, + /* 4180 */ 574, 438, 504, 574, 574, 574, 508, 509, 510, 511, + /* 4190 */ 512, 513, 574, 408, 516, 517, 518, 454, 574, 456, + /* 4200 */ 574, 393, 574, 501, 574, 574, 504, 574, 574, 574, + /* 4210 */ 508, 509, 510, 511, 512, 513, 408, 574, 516, 517, + /* 4220 */ 518, 574, 574, 438, 574, 574, 574, 574, 574, 574, + /* 4230 */ 574, 574, 574, 574, 574, 574, 574, 574, 574, 454, + /* 4240 */ 574, 456, 574, 574, 501, 574, 438, 504, 574, 574, + /* 4250 */ 574, 508, 509, 510, 511, 512, 513, 574, 574, 516, + /* 4260 */ 517, 518, 454, 574, 456, 574, 574, 574, 574, 393, + /* 4270 */ 574, 574, 574, 574, 574, 574, 574, 574, 574, 574, + /* 4280 */ 574, 574, 574, 574, 408, 574, 501, 574, 574, 504, + /* 4290 */ 574, 574, 574, 508, 509, 510, 511, 512, 513, 574, + /* 4300 */ 574, 516, 517, 518, 574, 574, 574, 393, 574, 501, + /* 4310 */ 574, 574, 504, 574, 438, 574, 508, 509, 510, 511, + /* 4320 */ 512, 513, 408, 574, 516, 517, 518, 574, 574, 574, + /* 4330 */ 454, 574, 456, 574, 574, 574, 574, 574, 574, 574, + /* 4340 */ 574, 574, 574, 574, 574, 574, 574, 574, 574, 574, + /* 4350 */ 574, 574, 438, 574, 574, 574, 574, 574, 574, 574, + /* 4360 */ 574, 574, 574, 574, 574, 574, 574, 574, 454, 574, + /* 4370 */ 456, 574, 574, 574, 574, 574, 574, 501, 574, 574, + /* 4380 */ 504, 574, 574, 574, 508, 509, 510, 511, 512, 513, + /* 4390 */ 574, 574, 516, 517, 518, 574, 574, 574, 574, 574, + /* 4400 */ 574, 574, 393, 574, 574, 574, 574, 574, 574, 574, + /* 4410 */ 574, 574, 574, 574, 574, 501, 574, 408, 504, 574, + /* 4420 */ 574, 574, 508, 509, 510, 511, 512, 513, 574, 393, + /* 4430 */ 516, 517, 518, 574, 574, 574, 574, 574, 574, 574, + /* 4440 */ 574, 574, 574, 574, 408, 574, 574, 438, 574, 574, + /* 4450 */ 574, 574, 574, 574, 574, 574, 574, 574, 574, 574, + /* 4460 */ 574, 574, 574, 454, 574, 456, 574, 393, 574, 574, + /* 4470 */ 574, 574, 574, 574, 438, 574, 574, 574, 574, 574, + /* 4480 */ 574, 574, 408, 574, 574, 574, 574, 574, 574, 574, + /* 4490 */ 454, 574, 456, 574, 574, 574, 574, 574, 574, 574, + /* 4500 */ 574, 574, 574, 574, 574, 574, 574, 574, 574, 574, + /* 4510 */ 501, 574, 438, 504, 574, 574, 574, 508, 509, 510, + /* 4520 */ 511, 512, 513, 574, 574, 516, 517, 518, 454, 574, + /* 4530 */ 456, 574, 574, 574, 574, 574, 574, 501, 574, 574, + /* 4540 */ 504, 574, 574, 574, 508, 509, 510, 511, 512, 513, + /* 4550 */ 393, 574, 516, 517, 518, 574, 574, 574, 574, 574, + /* 4560 */ 574, 574, 574, 574, 574, 408, 574, 574, 574, 574, + /* 4570 */ 574, 574, 574, 393, 574, 501, 574, 574, 504, 574, + /* 4580 */ 574, 574, 508, 509, 510, 511, 512, 513, 408, 574, + /* 4590 */ 516, 517, 518, 393, 574, 438, 574, 574, 574, 574, + /* 4600 */ 574, 574, 574, 574, 574, 574, 574, 574, 408, 574, + /* 4610 */ 574, 454, 574, 456, 574, 574, 574, 574, 438, 574, + /* 4620 */ 574, 574, 574, 574, 574, 574, 574, 574, 574, 574, + /* 4630 */ 574, 574, 574, 574, 454, 574, 456, 574, 438, 574, + /* 4640 */ 574, 574, 574, 574, 574, 574, 574, 574, 574, 574, + /* 4650 */ 574, 574, 574, 574, 454, 574, 456, 574, 501, 574, + /* 4660 */ 574, 504, 574, 574, 574, 508, 509, 510, 511, 512, + /* 4670 */ 513, 574, 574, 516, 517, 518, 574, 574, 574, 574, + /* 4680 */ 574, 501, 574, 574, 504, 574, 574, 574, 508, 509, + /* 4690 */ 510, 511, 512, 513, 574, 574, 516, 517, 518, 574, + /* 4700 */ 574, 501, 574, 574, 504, 574, 574, 574, 508, 509, + /* 4710 */ 510, 511, 512, 513, 574, 574, 516, 517, 518, 390, + /* 4720 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 4730 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 4740 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 4750 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 4760 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 4770 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 4780 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 4790 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 4800 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 4810 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 4820 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 4830 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 4840 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 4850 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 4860 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 4870 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 4880 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 4890 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 4900 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 4910 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 4920 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 4930 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 4940 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 4950 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 4960 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 4970 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 4980 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 4990 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 5000 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 5010 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 5020 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 5030 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 5040 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 5050 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 5060 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 5070 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 5080 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 5090 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, + /* 5100 */ 390, 390, 390, 390, 390, 390, 390, 390, 390, }; #define YY_SHIFT_COUNT (1025) #define YY_SHIFT_MIN (0) -#define YY_SHIFT_MAX (2989) +#define YY_SHIFT_MAX (2904) static const unsigned short int yy_shift_ofst[] = { - /* 0 */ 948, 277, 554, 277, 832, 832, 832, 832, 832, 832, - /* 10 */ 832, 832, 832, 832, 832, 832, 1109, 1940, 1940, 2217, - /* 20 */ 0, 1386, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, - /* 30 */ 1940, 1940, 1663, 1940, 1940, 1940, 1940, 1940, 1940, 1940, - /* 40 */ 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, - /* 50 */ 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, - /* 60 */ 1940, 1940, 1940, 66, 482, 455, 198, 90, 122, 90, - /* 70 */ 90, 198, 198, 90, 960, 90, 276, 960, 313, 90, - /* 80 */ 89, 2315, 156, 156, 149, 149, 2315, 2315, 598, 598, - /* 90 */ 156, 377, 377, 443, 487, 487, 178, 37, 149, 149, - /* 100 */ 149, 149, 149, 149, 149, 149, 149, 149, 149, 180, - /* 110 */ 225, 429, 149, 149, 387, 89, 149, 180, 149, 89, - /* 120 */ 149, 149, 149, 149, 89, 149, 149, 149, 89, 149, - /* 130 */ 89, 89, 89, 427, 222, 222, 473, 473, 745, 652, - /* 140 */ 181, 51, 548, 548, 548, 548, 548, 548, 548, 548, - /* 150 */ 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, - /* 160 */ 548, 1858, 200, 377, 443, 702, 702, 823, 768, 768, - /* 170 */ 768, 708, 708, 1354, 661, 823, 387, 89, 596, 89, - /* 180 */ 89, 366, 89, 89, 658, 89, 658, 658, 704, 747, - /* 190 */ 2488, 473, 473, 473, 473, 473, 473, 1190, 21, 53, - /* 200 */ 411, 411, 81, 148, 23, 606, 152, 545, 213, 333, - /* 210 */ 414, 414, 1062, 757, 1182, 1182, 1182, 1229, 1182, 815, - /* 220 */ 1037, 957, 1272, 1439, 1310, 476, 374, 374, 1254, 1297, - /* 230 */ 1297, 354, 1462, 1256, 374, 661, 1496, 1768, 1809, 1814, - /* 240 */ 1601, 387, 1814, 387, 1623, 1809, 1839, 1816, 1839, 1816, - /* 250 */ 1685, 1809, 1839, 1809, 1816, 1685, 1685, 1685, 1782, 1788, - /* 260 */ 1809, 1809, 1793, 1809, 1809, 1809, 1892, 1862, 1892, 1862, - /* 270 */ 1814, 387, 387, 1922, 387, 1936, 1938, 387, 1936, 387, - /* 280 */ 1946, 387, 1949, 387, 387, 1918, 1918, 1809, 387, 1892, - /* 290 */ 89, 89, 89, 89, 89, 89, 89, 89, 89, 89, - /* 300 */ 89, 1809, 747, 747, 1892, 658, 658, 658, 1760, 1889, - /* 310 */ 1814, 427, 1992, 1811, 1810, 1922, 427, 1496, 1809, 658, - /* 320 */ 1715, 1722, 1715, 1722, 1721, 1841, 1715, 1725, 1731, 1750, - /* 330 */ 1496, 1757, 1764, 1734, 1740, 1746, 1839, 2067, 1959, 1786, - /* 340 */ 1936, 427, 427, 1722, 658, 658, 658, 658, 1722, 658, - /* 350 */ 1929, 427, 658, 1949, 427, 2022, 658, 1945, 1949, 427, - /* 360 */ 704, 427, 1839, 658, 658, 658, 658, 658, 658, 658, - /* 370 */ 658, 658, 658, 658, 658, 658, 658, 658, 658, 658, - /* 380 */ 658, 658, 658, 658, 658, 2044, 658, 1809, 427, 2146, - /* 390 */ 2148, 2147, 2150, 1892, 4475, 4475, 4475, 4475, 4475, 4475, - /* 400 */ 4475, 4475, 4475, 4475, 4475, 4475, 39, 2989, 298, 1146, - /* 410 */ 1193, 1243, 1425, 1711, 715, 1434, 1560, 1570, 1124, 497, - /* 420 */ 770, 1584, 1542, 1861, 1878, 1699, 1808, 1004, 1169, 1026, - /* 430 */ 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 237, 150, - /* 440 */ 992, 813, 3, 3, 343, 106, 321, 182, 370, 646, - /* 450 */ 420, 378, 1011, 124, 751, 751, 1063, 1081, 1138, 1063, - /* 460 */ 1063, 1063, 530, 334, 1395, 1456, 1488, 921, 1349, 1469, - /* 470 */ 5, 794, 1350, 1486, 1403, 1405, 1486, 1422, 882, 1526, - /* 480 */ 1527, 1538, 1610, 1611, 1636, 1088, 1445, 1489, 179, 1519, - /* 490 */ 1561, 1562, 1569, 1391, 1466, 1470, 1500, 1587, 1633, 1634, - /* 500 */ 1564, 1645, 1492, 1571, 1653, 1658, 1575, 1667, 1671, 1708, - /* 510 */ 1709, 1712, 1612, 1683, 1718, 1720, 1723, 1737, 1742, 1744, - /* 520 */ 1747, 1749, 1755, 1696, 1486, 1730, 1736, 1743, 1766, 1778, - /* 530 */ 1483, 1552, 335, 1476, 1738, 1762, 1748, 1135, 2257, 2258, - /* 540 */ 2259, 2213, 2261, 2226, 2018, 2228, 2231, 2232, 2020, 2269, - /* 550 */ 2234, 2235, 2027, 2237, 2274, 2275, 2031, 2277, 2242, 2279, - /* 560 */ 2244, 2281, 2260, 2283, 2248, 2045, 2286, 2060, 2289, 2063, - /* 570 */ 2064, 2072, 2075, 2295, 2297, 2298, 2084, 2086, 2301, 2303, - /* 580 */ 2143, 2254, 2255, 2307, 2272, 2309, 2310, 2280, 2256, 2311, - /* 590 */ 2264, 2317, 2271, 2319, 2320, 2321, 2273, 2322, 2323, 2325, - /* 600 */ 2326, 2329, 2330, 2152, 2300, 2334, 2159, 2339, 2340, 2341, - /* 610 */ 2342, 2343, 2344, 2345, 2346, 2347, 2349, 2350, 2352, 2354, - /* 620 */ 2355, 2356, 2357, 2358, 2360, 2361, 2363, 2313, 2365, 2332, - /* 630 */ 2377, 2380, 2382, 2384, 2385, 2389, 2390, 2391, 2392, 2393, - /* 640 */ 2373, 2397, 2238, 2398, 2240, 2401, 2243, 2403, 2404, 2383, - /* 650 */ 2362, 2386, 2364, 2406, 2336, 2409, 2348, 2374, 2413, 2353, - /* 660 */ 2415, 2359, 2417, 2418, 2387, 2368, 2376, 2426, 2395, 2370, - /* 670 */ 2388, 2428, 2399, 2372, 2396, 2433, 2410, 2434, 2381, 2439, - /* 680 */ 2411, 2442, 2407, 2416, 2422, 2394, 2408, 2447, 2423, 2462, - /* 690 */ 2419, 2424, 2466, 2467, 2490, 2496, 2453, 2288, 2506, 2394, - /* 700 */ 2457, 2509, 2394, 2459, 2511, 2512, 2435, 2513, 2516, 2481, - /* 710 */ 2461, 2475, 2520, 2485, 2465, 2479, 2524, 2489, 2470, 2484, - /* 720 */ 2529, 2495, 2476, 2491, 2532, 2537, 2538, 2539, 2540, 2507, - /* 730 */ 2544, 2546, 2420, 2421, 2514, 2527, 2551, 2530, 2517, 2522, - /* 740 */ 2525, 2526, 2528, 2534, 2535, 2536, 2541, 2533, 2542, 2543, - /* 750 */ 2545, 2552, 2547, 2563, 2558, 2565, 2560, 2573, 2562, 2548, - /* 760 */ 2585, 2564, 2555, 2587, 2588, 2592, 2557, 2594, 2559, 2596, - /* 770 */ 2567, 2604, 2584, 2589, 2572, 2575, 2576, 2494, 2497, 2614, - /* 780 */ 2429, 2405, 2414, 2498, 2402, 2394, 2566, 2618, 2431, 2583, - /* 790 */ 2598, 2622, 2425, 2602, 2446, 2427, 2643, 2646, 2452, 2443, - /* 800 */ 2454, 2445, 2649, 2620, 2337, 2549, 2550, 2553, 2554, 2632, - /* 810 */ 2556, 2561, 2621, 2624, 2569, 2606, 2574, 2609, 2571, 2578, - /* 820 */ 2623, 2629, 2579, 2577, 2581, 2582, 2590, 2630, 2613, 2617, - /* 830 */ 2586, 2637, 2400, 2580, 2591, 2639, 2593, 2640, 2595, 2597, - /* 840 */ 2672, 2644, 2430, 2642, 2645, 2647, 2658, 2665, 2666, 2600, - /* 850 */ 2603, 2628, 2432, 2670, 2657, 2712, 2713, 2599, 2677, 2605, - /* 860 */ 2607, 2610, 2612, 2601, 2615, 2723, 2681, 2570, 2727, 2631, - /* 870 */ 2625, 2608, 2687, 2611, 2709, 2635, 2463, 2638, 2745, 2734, - /* 880 */ 2503, 2648, 2652, 2654, 2656, 2659, 2661, 2663, 2667, 2669, - /* 890 */ 2671, 2673, 2674, 2675, 2698, 2678, 2679, 2707, 2676, 2753, - /* 900 */ 2519, 2680, 2683, 2778, 2684, 2688, 2616, 2735, 2690, 2664, - /* 910 */ 2788, 2768, 2691, 2693, 2394, 2742, 2695, 2696, 2699, 2702, - /* 920 */ 2703, 2694, 2777, 2619, 2705, 2781, 2786, 2708, 2706, 2791, - /* 930 */ 2711, 2714, 2794, 2667, 2722, 2806, 2669, 2728, 2810, 2671, - /* 940 */ 2729, 2814, 2673, 2662, 2715, 2717, 2718, 2736, 2819, 2741, - /* 950 */ 2817, 2743, 2819, 2819, 2838, 2783, 2785, 2842, 2844, 2832, - /* 960 */ 2834, 2835, 2836, 2837, 2839, 2840, 2841, 2845, 2847, 2849, - /* 970 */ 2789, 2790, 2793, 2792, 2860, 2861, 2862, 2863, 2858, 2864, - /* 980 */ 2865, 2867, 2820, 2533, 2870, 2542, 2872, 2873, 2874, 2875, - /* 990 */ 2891, 2880, 2917, 2882, 2868, 2887, 2926, 2897, 2877, 2892, - /* 1000 */ 2935, 2902, 2883, 2895, 2941, 2906, 2886, 2900, 2945, 2910, - /* 1010 */ 2947, 2927, 2948, 2928, 2915, 2952, 2932, 2923, 2921, 2922, - /* 1020 */ 2937, 2940, 2942, 2943, 2946, 2949, + /* 0 */ 712, 278, 556, 278, 835, 835, 835, 835, 835, 835, + /* 10 */ 835, 835, 835, 835, 835, 835, 1113, 1947, 1947, 2225, + /* 20 */ 0, 1391, 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, + /* 30 */ 1947, 1947, 1669, 1947, 1947, 1947, 1947, 1947, 1947, 1947, + /* 40 */ 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, + /* 50 */ 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, + /* 60 */ 1947, 1947, 1947, 34, 156, 246, 97, 6, 121, 6, + /* 70 */ 6, 97, 97, 6, 2033, 6, 277, 2033, 645, 6, + /* 80 */ 133, 2503, 416, 416, 170, 170, 2503, 2503, 1130, 1130, + /* 90 */ 416, 8, 8, 360, 348, 348, 628, 93, 170, 170, + /* 100 */ 170, 170, 170, 170, 170, 170, 170, 170, 170, 234, + /* 110 */ 375, 440, 170, 170, 219, 133, 170, 234, 170, 133, + /* 120 */ 170, 170, 170, 170, 133, 170, 170, 170, 133, 170, + /* 130 */ 133, 133, 133, 640, 505, 505, 99, 99, 470, 1212, + /* 140 */ 435, 51, 951, 951, 951, 951, 951, 951, 951, 951, + /* 150 */ 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, + /* 160 */ 951, 538, 491, 8, 360, 1126, 1126, 467, 553, 553, + /* 170 */ 553, 546, 546, 388, 995, 467, 219, 133, 441, 133, + /* 180 */ 133, 443, 133, 133, 698, 133, 698, 698, 680, 10, + /* 190 */ 1015, 99, 99, 99, 99, 99, 99, 595, 21, 53, + /* 200 */ 182, 182, 514, 534, 23, 607, 609, 66, 687, 688, + /* 210 */ 415, 415, 1066, 1067, 135, 135, 135, 1119, 135, 318, + /* 220 */ 1311, 1236, 1129, 1428, 1309, 112, 302, 302, 1315, 1156, + /* 230 */ 1156, 1201, 1175, 948, 302, 995, 1528, 1791, 1830, 1834, + /* 240 */ 1620, 219, 1834, 219, 1648, 1830, 1851, 1831, 1851, 1831, + /* 250 */ 1688, 1830, 1851, 1830, 1831, 1688, 1688, 1688, 1777, 1783, + /* 260 */ 1830, 1830, 1786, 1830, 1830, 1830, 1888, 1864, 1888, 1864, + /* 270 */ 1834, 219, 219, 1923, 219, 1931, 1942, 219, 1931, 219, + /* 280 */ 1948, 219, 1955, 219, 219, 1928, 1928, 1830, 219, 1888, + /* 290 */ 133, 133, 133, 133, 133, 133, 133, 133, 133, 133, + /* 300 */ 133, 1830, 10, 10, 1888, 698, 698, 698, 1760, 1885, + /* 310 */ 1834, 640, 1988, 1785, 1788, 1923, 640, 1528, 1830, 698, + /* 320 */ 1702, 1699, 1702, 1699, 1697, 1823, 1702, 1703, 1701, 1716, + /* 330 */ 1528, 1722, 1728, 1710, 1704, 1718, 1851, 2037, 1929, 1744, + /* 340 */ 1931, 640, 640, 1699, 698, 698, 698, 698, 1699, 698, + /* 350 */ 1868, 640, 698, 1955, 640, 1990, 698, 1909, 1955, 640, + /* 360 */ 680, 640, 1851, 698, 698, 698, 698, 698, 698, 698, + /* 370 */ 698, 698, 698, 698, 698, 698, 698, 698, 698, 698, + /* 380 */ 698, 698, 698, 698, 698, 1999, 698, 1830, 640, 2101, + /* 390 */ 2092, 2094, 2106, 1888, 4719, 4719, 4719, 4719, 4719, 4719, + /* 400 */ 4719, 4719, 4719, 4719, 4719, 4719, 39, 520, 299, 1138, + /* 410 */ 78, 1469, 87, 1340, 1035, 1350, 1478, 1611, 610, 493, + /* 420 */ 1062, 1747, 921, 1756, 1765, 1147, 1706, 829, 870, 1328, + /* 430 */ 1328, 1328, 1328, 1328, 1328, 1328, 1328, 1328, 1287, 68, + /* 440 */ 993, 22, 3, 3, 683, 778, 797, 236, 164, 460, + /* 450 */ 1022, 1338, 735, 1390, 38, 38, 1257, 58, 892, 1257, + /* 460 */ 1257, 1257, 667, 1333, 477, 770, 1262, 1419, 1433, 1443, + /* 470 */ 1530, 939, 1437, 1375, 1450, 1477, 1375, 1481, 1520, 840, + /* 480 */ 1476, 1587, 1592, 1593, 1608, 1485, 1503, 1522, 1111, 1546, + /* 490 */ 1565, 1578, 1580, 1516, 1353, 1434, 1583, 1585, 1616, 1631, + /* 500 */ 1632, 1657, 818, 1663, 1670, 1673, 1570, 1677, 1678, 1692, + /* 510 */ 1693, 1698, 1681, 1700, 1720, 1724, 1726, 1729, 1736, 1739, + /* 520 */ 1745, 1773, 1792, 1712, 1375, 1714, 1733, 1734, 1750, 1757, + /* 530 */ 1609, 1752, 1452, 1606, 1780, 1796, 1782, 931, 2164, 2174, + /* 540 */ 2183, 2140, 2190, 2158, 1949, 2160, 2163, 2165, 1954, 2205, + /* 550 */ 2184, 2191, 1982, 2193, 2239, 2240, 1995, 2242, 2207, 2244, + /* 560 */ 2210, 2248, 2227, 2250, 2215, 2012, 2255, 2029, 2257, 2031, + /* 570 */ 2032, 2038, 2043, 2264, 2265, 2266, 2053, 2055, 2270, 2271, + /* 580 */ 2111, 2222, 2223, 2275, 2241, 2276, 2278, 2243, 2224, 2280, + /* 590 */ 2232, 2284, 2238, 2286, 2287, 2290, 2246, 2291, 2292, 2293, + /* 600 */ 2294, 2295, 2298, 2121, 2267, 2301, 2123, 2304, 2305, 2306, + /* 610 */ 2307, 2311, 2313, 2314, 2315, 2322, 2323, 2324, 2325, 2326, + /* 620 */ 2327, 2329, 2331, 2332, 2333, 2334, 2335, 2285, 2337, 2297, + /* 630 */ 2338, 2339, 2341, 2342, 2345, 2346, 2347, 2348, 2349, 2350, + /* 640 */ 2330, 2351, 2194, 2353, 2196, 2354, 2198, 2358, 2359, 2340, + /* 650 */ 2308, 2343, 2309, 2363, 2296, 2364, 2299, 2336, 2366, 2300, + /* 660 */ 2368, 2302, 2369, 2374, 2352, 2319, 2356, 2377, 2357, 2344, + /* 670 */ 2361, 2379, 2360, 2365, 2362, 2389, 2371, 2394, 2355, 2395, + /* 680 */ 2372, 2398, 2367, 2373, 2380, 2370, 2375, 2388, 2381, 2409, + /* 690 */ 2378, 2384, 2410, 2415, 2416, 2418, 2387, 2214, 2425, 2370, + /* 700 */ 2382, 2434, 2370, 2385, 2435, 2437, 2383, 2438, 2439, 2405, + /* 710 */ 2386, 2396, 2442, 2411, 2391, 2401, 2449, 2414, 2397, 2408, + /* 720 */ 2453, 2419, 2399, 2417, 2458, 2459, 2460, 2463, 2464, 2429, + /* 730 */ 2466, 2467, 2393, 2400, 2432, 2447, 2470, 2450, 2440, 2441, + /* 740 */ 2443, 2462, 2469, 2471, 2481, 2483, 2486, 2445, 2473, 2498, + /* 750 */ 2502, 2452, 2504, 2499, 2461, 2524, 2520, 2543, 2525, 2412, + /* 760 */ 2546, 2528, 2515, 2552, 2553, 2554, 2519, 2556, 2521, 2558, + /* 770 */ 2523, 2561, 2540, 2544, 2529, 2530, 2531, 2451, 2454, 2568, + /* 780 */ 2390, 2392, 2413, 2455, 2402, 2370, 2518, 2571, 2403, 2539, + /* 790 */ 2555, 2576, 2423, 2560, 2406, 2407, 2578, 2580, 2420, 2426, + /* 800 */ 2421, 2433, 2581, 2550, 2268, 2474, 2472, 2475, 2476, 2566, + /* 810 */ 2478, 2479, 2565, 2567, 2487, 2542, 2491, 2545, 2490, 2492, + /* 820 */ 2577, 2582, 2494, 2496, 2500, 2501, 2506, 2587, 2538, 2570, + /* 830 */ 2505, 2589, 2312, 2562, 2532, 2607, 2534, 2613, 2547, 2548, + /* 840 */ 2655, 2626, 2422, 2635, 2636, 2638, 2640, 2642, 2643, 2564, + /* 850 */ 2569, 2629, 2430, 2651, 2634, 2686, 2687, 2572, 2645, 2573, + /* 860 */ 2575, 2579, 2588, 2512, 2594, 2691, 2654, 2510, 2695, 2595, + /* 870 */ 2597, 2517, 2673, 2527, 2666, 2602, 2428, 2605, 2724, 2712, + /* 880 */ 2484, 2620, 2622, 2624, 2625, 2627, 2631, 2632, 2630, 2637, + /* 890 */ 2641, 2644, 2646, 2633, 2696, 2647, 2648, 2702, 2652, 2732, + /* 900 */ 2509, 2650, 2656, 2746, 2653, 2657, 2574, 2716, 2658, 2660, + /* 910 */ 2761, 2741, 2661, 2659, 2370, 2719, 2663, 2665, 2674, 2667, + /* 920 */ 2676, 2670, 2770, 2563, 2677, 2762, 2764, 2683, 2684, 2766, + /* 930 */ 2688, 2685, 2769, 2630, 2689, 2771, 2637, 2690, 2774, 2641, + /* 940 */ 2692, 2777, 2644, 2675, 2678, 2679, 2680, 2698, 2784, 2703, + /* 950 */ 2786, 2705, 2784, 2784, 2802, 2747, 2749, 2803, 2806, 2793, + /* 960 */ 2795, 2797, 2798, 2799, 2800, 2801, 2804, 2805, 2807, 2808, + /* 970 */ 2753, 2727, 2760, 2734, 2814, 2812, 2813, 2816, 2828, 2817, + /* 980 */ 2819, 2820, 2772, 2445, 2823, 2473, 2825, 2826, 2827, 2829, + /* 990 */ 2842, 2830, 2851, 2831, 2811, 2832, 2869, 2834, 2818, 2835, + /* 1000 */ 2871, 2836, 2821, 2838, 2873, 2847, 2833, 2853, 2874, 2855, + /* 1010 */ 2895, 2876, 2899, 2878, 2865, 2902, 2881, 2877, 2868, 2870, + /* 1020 */ 2885, 2890, 2896, 2897, 2904, 2903, }; #define YY_REDUCE_COUNT (405) -#define YY_REDUCE_MIN (-536) -#define YY_REDUCE_MAX (3957) +#define YY_REDUCE_MIN (-537) +#define YY_REDUCE_MAX (4200) static const short yy_reduce_ofst[] = { - /* 0 */ 418, -349, -99, 142, 552, 829, 1008, 1285, 1582, 1664, - /* 10 */ 1106, 1859, 1941, 176, 1336, 1613, -135, 289, 2123, 419, - /* 20 */ 974, 2229, 2252, 2328, 2379, 2412, 2477, 2500, 2568, 2633, - /* 30 */ 2660, 2746, 2766, 2787, 2893, 2913, 2933, 3020, 3046, 3069, - /* 40 */ 3089, 3111, 3176, 3195, 3217, 3282, 3331, 3351, 3371, 3458, - /* 50 */ 3477, 3497, 3564, 3584, 3603, 3671, 3690, 3739, 3759, 3827, - /* 60 */ 3851, 3933, 3957, -351, 448, 605, -290, -366, 481, 1203, - /* 70 */ 1464, -8, 329, 1485, -393, -536, -153, -412, -534, 14, - /* 80 */ 197, -403, -368, -106, 254, 331, -433, -288, -400, -302, - /* 90 */ -452, -100, 28, -245, 385, 449, -71, 230, 271, 348, - /* 100 */ 364, 514, 261, 488, 526, 532, 539, 550, 517, 218, - /* 110 */ 169, 562, 718, 756, -409, 472, 806, 452, 843, 518, - /* 120 */ 847, 879, 889, 900, 739, 983, 995, 997, 609, 1006, - /* 130 */ -383, 792, 1017, 867, -361, -361, 263, -285, 189, -192, - /* 140 */ 56, 478, -25, 209, 559, 603, 681, 721, 735, 738, - /* 150 */ 778, 849, 895, 935, 963, 967, 1013, 1015, 1022, 1027, - /* 160 */ 1034, -408, 241, 33, 257, 774, 811, 819, 241, 390, - /* 170 */ 929, 549, 788, 911, 607, 913, 1052, 981, 1040, 546, - /* 180 */ 905, 628, 416, 1055, 766, 1014, 897, 1002, 928, 1098, - /* 190 */ -471, 435, 499, 555, 568, 581, 690, 838, 782, 446, - /* 200 */ 737, 737, 926, 732, 996, 988, 1157, 737, 1140, 1140, - /* 210 */ 1164, 1206, 1226, 1174, 1084, 1100, 1125, 1187, 1128, 1140, - /* 220 */ 1237, 1295, 1208, 1309, 1266, 1235, 1279, 1314, 1140, 1239, - /* 230 */ 1240, 1216, 1257, 1259, 1338, 1381, 1328, 1324, 1426, 1344, - /* 240 */ 1342, 1427, 1359, 1441, 1372, 1435, 1454, 1404, 1465, 1409, - /* 250 */ 1420, 1474, 1481, 1475, 1421, 1436, 1437, 1442, 1484, 1487, - /* 260 */ 1498, 1501, 1491, 1503, 1504, 1505, 1514, 1512, 1517, 1532, - /* 270 */ 1429, 1521, 1529, 1499, 1540, 1549, 1478, 1546, 1555, 1550, - /* 280 */ 1495, 1553, 1506, 1558, 1566, 1544, 1551, 1573, 1572, 1576, - /* 290 */ 1556, 1557, 1559, 1565, 1578, 1579, 1580, 1581, 1585, 1586, - /* 300 */ 1591, 1589, 1593, 1595, 1599, 1533, 1539, 1548, 1508, 1516, - /* 310 */ 1522, 1617, 1534, 1545, 1554, 1583, 1626, 1563, 1638, 1592, - /* 320 */ 1490, 1588, 1497, 1590, 1502, 1510, 1507, 1511, 1520, 1523, - /* 330 */ 1594, 1535, 1537, 1509, 1518, 1515, 1679, 1596, 1567, 1598, - /* 340 */ 1695, 1692, 1694, 1635, 1656, 1659, 1662, 1665, 1654, 1673, - /* 350 */ 1666, 1719, 1680, 1672, 1726, 1619, 1697, 1681, 1686, 1741, - /* 360 */ 1717, 1745, 1753, 1698, 1700, 1702, 1703, 1704, 1710, 1716, - /* 370 */ 1724, 1729, 1732, 1759, 1767, 1779, 1780, 1781, 1783, 1785, - /* 380 */ 1787, 1789, 1790, 1791, 1792, 1728, 1794, 1772, 1752, 1795, - /* 390 */ 1800, 1815, 1819, 1822, 1758, 1796, 1688, 1739, 1774, 1775, - /* 400 */ 1817, 1821, 1813, 1825, 1777, 1866, + /* 0 */ 406, -350, -99, 143, 484, 736, 773, 1014, 1041, 1290, + /* 10 */ 1571, 1668, 1874, 2019, 2125, 2152, -135, 1127, 1402, 2219, + /* 20 */ 2192, 2376, 2404, 2488, 2516, 2537, 2628, 2649, 2669, 2790, + /* 30 */ 2815, 2859, 2882, 2971, 2994, 3022, 3063, 3106, 3148, 3245, + /* 40 */ 3269, 3294, 3334, 3376, 3401, 3482, 3507, 3531, 3572, 3653, + /* 50 */ 3678, 3702, 3743, 3785, 3808, 3876, 3914, 4009, 4036, 4074, + /* 60 */ 4157, 4180, 4200, -352, 335, -148, -57, 152, 315, 656, + /* 70 */ 782, 231, 701, 935, 174, -537, -211, -340, -535, -178, + /* 80 */ -247, 290, -69, 398, -238, -4, 244, 333, -401, -399, + /* 90 */ 276, -408, -368, -415, -49, 498, 494, 329, 383, 481, + /* 100 */ 539, 598, 503, 541, 604, 625, 630, 638, 548, -179, + /* 110 */ 386, -295, 652, 723, 582, 33, 731, 551, 789, 212, + /* 120 */ 843, 998, 1001, 1023, 1, 1037, 1039, 1046, 729, 1048, + /* 130 */ -130, 753, 876, 763, -2, -2, 287, 280, -425, 165, + /* 140 */ -530, -167, 674, 824, 874, 964, 967, 978, 1017, 1065, + /* 150 */ 1107, 1108, 1115, 1120, 1121, 1123, 1124, 1128, 1135, 1136, + /* 160 */ 1140, 47, 128, -73, 685, 650, 867, 866, 128, 479, + /* 170 */ 492, 397, 690, -379, 217, 908, 672, 868, 151, 253, + /* 180 */ 455, 366, 1025, -230, 325, 707, 963, 965, 719, 997, + /* 190 */ -463, 596, 616, 654, 922, 984, 1021, 832, 953, 1075, + /* 200 */ 1028, 1028, 1027, 1068, 1078, 1155, 1276, 1028, 1263, 1263, + /* 210 */ 1284, 1285, 1318, 1251, 1174, 1193, 1254, 1342, 1266, 1263, + /* 220 */ 1356, 1412, 1326, 1421, 1377, 1347, 1369, 1370, 1263, 1294, + /* 230 */ 1295, 1277, 1310, 1296, 1378, 1420, 1366, 1346, 1444, 1363, + /* 240 */ 1362, 1445, 1368, 1446, 1385, 1463, 1464, 1414, 1471, 1417, + /* 250 */ 1422, 1479, 1475, 1480, 1426, 1425, 1431, 1432, 1474, 1482, + /* 260 */ 1489, 1491, 1486, 1493, 1498, 1500, 1509, 1513, 1521, 1518, + /* 270 */ 1435, 1507, 1526, 1494, 1533, 1552, 1483, 1547, 1557, 1554, + /* 280 */ 1501, 1555, 1510, 1560, 1562, 1548, 1550, 1574, 1572, 1586, + /* 290 */ 1549, 1551, 1553, 1556, 1558, 1559, 1561, 1563, 1564, 1566, + /* 300 */ 1567, 1588, 1594, 1596, 1598, 1539, 1568, 1569, 1511, 1512, + /* 310 */ 1519, 1599, 1523, 1527, 1529, 1573, 1601, 1538, 1613, 1576, + /* 320 */ 1473, 1575, 1484, 1577, 1487, 1492, 1488, 1490, 1497, 1502, + /* 330 */ 1579, 1495, 1508, 1504, 1496, 1524, 1646, 1584, 1525, 1532, + /* 340 */ 1655, 1652, 1653, 1600, 1614, 1638, 1639, 1641, 1612, 1643, + /* 350 */ 1627, 1671, 1645, 1640, 1691, 1581, 1656, 1644, 1647, 1705, + /* 360 */ 1680, 1707, 1708, 1661, 1666, 1672, 1674, 1675, 1676, 1679, + /* 370 */ 1682, 1683, 1684, 1685, 1686, 1689, 1690, 1695, 1696, 1709, + /* 380 */ 1711, 1713, 1717, 1721, 1735, 1715, 1738, 1725, 1731, 1730, + /* 390 */ 1737, 1748, 1753, 1755, 1650, 1719, 1660, 1664, 1687, 1746, + /* 400 */ 1723, 1740, 1742, 1758, 1741, 1770, }; static const YYACTIONTYPE yy_default[] = { - /* 0 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 10 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 20 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 30 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 40 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 50 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 60 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 70 */ 2303, 2303, 2303, 2687, 2303, 2303, 2643, 2303, 2303, 2303, - /* 80 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 90 */ 2303, 2650, 2650, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 100 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 110 */ 2303, 2303, 2303, 2303, 2413, 2303, 2303, 2303, 2303, 2303, - /* 120 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 130 */ 2303, 2303, 2303, 2411, 2960, 2303, 3088, 2728, 2303, 2303, - /* 140 */ 2989, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 150 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 160 */ 2303, 2303, 2972, 2303, 2303, 2384, 2384, 2303, 2972, 2972, - /* 170 */ 2972, 2932, 2932, 2411, 2303, 2303, 2413, 2303, 2730, 2303, - /* 180 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2557, 2333, - /* 190 */ 2713, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 3018, - /* 200 */ 2964, 2965, 3082, 2303, 3021, 2983, 2303, 2978, 2303, 2303, - /* 210 */ 2303, 2303, 2303, 3008, 2303, 2303, 2303, 2303, 2303, 2303, - /* 220 */ 2655, 2303, 2756, 2303, 2500, 2707, 2303, 2303, 2303, 2303, - /* 230 */ 2303, 3066, 2962, 3002, 2303, 2303, 3012, 2303, 2303, 2303, - /* 240 */ 2744, 2413, 2303, 2413, 2700, 2638, 2303, 2648, 2303, 2648, - /* 250 */ 2645, 2303, 2303, 2303, 2648, 2645, 2645, 2645, 2488, 2484, - /* 260 */ 2303, 2303, 2482, 2303, 2303, 2303, 2303, 2363, 2303, 2363, - /* 270 */ 2303, 2413, 2413, 2303, 2413, 2303, 2303, 2413, 2303, 2413, - /* 280 */ 2303, 2413, 2303, 2413, 2413, 2517, 2517, 2303, 2413, 2303, - /* 290 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 300 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2742, 2723, - /* 310 */ 2303, 2411, 2303, 2711, 2709, 2303, 2411, 3012, 2303, 2303, - /* 320 */ 3036, 3031, 3036, 3031, 3050, 3046, 3036, 3055, 3052, 3014, - /* 330 */ 3012, 2995, 2991, 3085, 3072, 3068, 2303, 2303, 3000, 2998, - /* 340 */ 2303, 2411, 2411, 3031, 2303, 2303, 2303, 2303, 3031, 2303, - /* 350 */ 2303, 2411, 2303, 2303, 2411, 2303, 2303, 2303, 2303, 2411, - /* 360 */ 2303, 2411, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 370 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 380 */ 2303, 2303, 2303, 2303, 2303, 2519, 2303, 2303, 2411, 2303, - /* 390 */ 2335, 2337, 2347, 2303, 2702, 3088, 2728, 2733, 2683, 2683, - /* 400 */ 2560, 2560, 3088, 2560, 2414, 2308, 2303, 2303, 2303, 2303, - /* 410 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 420 */ 2303, 2303, 2861, 2303, 2303, 2303, 2303, 2303, 2303, 3049, - /* 430 */ 3048, 2862, 2303, 2936, 2935, 2934, 2925, 2861, 2513, 2303, - /* 440 */ 2303, 2303, 2860, 2859, 2303, 2303, 2303, 2303, 2303, 2303, - /* 450 */ 2303, 2303, 2303, 2303, 2674, 2673, 2853, 2303, 2303, 2854, - /* 460 */ 2852, 2851, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 470 */ 2303, 2303, 2303, 2504, 2303, 2303, 2501, 2303, 2303, 2303, - /* 480 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 490 */ 2303, 2303, 2303, 2303, 3069, 3073, 2303, 2303, 2303, 2303, - /* 500 */ 2961, 2303, 2303, 2303, 2303, 2303, 2832, 2303, 2303, 2303, - /* 510 */ 2303, 2303, 2800, 2795, 2786, 2777, 2792, 2783, 2771, 2789, - /* 520 */ 2780, 2768, 2765, 2303, 2528, 2303, 2303, 2303, 2303, 2303, - /* 530 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 540 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 550 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 560 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 570 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 580 */ 2644, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 590 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 600 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 610 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 620 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 630 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 640 */ 2303, 2303, 2303, 2303, 2303, 2303, 2659, 2303, 2303, 2303, - /* 650 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 660 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 670 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 680 */ 2303, 2303, 2303, 2303, 2352, 2839, 2303, 2303, 2303, 2303, - /* 690 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2842, - /* 700 */ 2303, 2303, 2843, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 710 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 720 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 730 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 740 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2459, 2458, 2303, - /* 750 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 760 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 770 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2844, 2303, 2303, - /* 780 */ 2303, 2303, 2727, 2303, 2303, 2834, 2303, 2303, 2303, 2303, - /* 790 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 800 */ 2303, 2303, 3065, 3015, 2303, 2303, 2303, 2303, 2303, 2303, - /* 810 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 820 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2832, - /* 830 */ 2303, 3047, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 3063, - /* 840 */ 2303, 3067, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2971, - /* 850 */ 2967, 2303, 2303, 2963, 2303, 2303, 2303, 2303, 2303, 2303, - /* 860 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 870 */ 2303, 2303, 2303, 2303, 2922, 2303, 2303, 2303, 2956, 2303, - /* 880 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2556, 2555, - /* 890 */ 2554, 2553, 2303, 2303, 2303, 2303, 2303, 2303, 2844, 2303, - /* 900 */ 2847, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 910 */ 2303, 2303, 2303, 2303, 2831, 2303, 2899, 2898, 2303, 2303, - /* 920 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2550, 2303, 2303, - /* 930 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 940 */ 2303, 2303, 2303, 2534, 2532, 2531, 2530, 2303, 2567, 2303, - /* 950 */ 2303, 2303, 2563, 2562, 2303, 2303, 2303, 2303, 2303, 2303, - /* 960 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 970 */ 2303, 2303, 2303, 2303, 2432, 2303, 2303, 2303, 2303, 2303, - /* 980 */ 2303, 2303, 2303, 2424, 2303, 2423, 2303, 2303, 2303, 2303, - /* 990 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 1000 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2303, - /* 1010 */ 2303, 2303, 2303, 2303, 2303, 2303, 2303, 2332, 2303, 2303, - /* 1020 */ 2303, 2303, 2303, 2303, 2303, 2303, + /* 0 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 10 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 20 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 30 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 40 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 50 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 60 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 70 */ 2305, 2305, 2305, 2689, 2305, 2305, 2645, 2305, 2305, 2305, + /* 80 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 90 */ 2305, 2652, 2652, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 100 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 110 */ 2305, 2305, 2305, 2305, 2415, 2305, 2305, 2305, 2305, 2305, + /* 120 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 130 */ 2305, 2305, 2305, 2413, 2963, 2305, 3091, 2730, 2305, 2305, + /* 140 */ 2992, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 150 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 160 */ 2305, 2305, 2975, 2305, 2305, 2386, 2386, 2305, 2975, 2975, + /* 170 */ 2975, 2935, 2935, 2413, 2305, 2305, 2415, 2305, 2732, 2305, + /* 180 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2559, 2335, + /* 190 */ 2715, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 3021, + /* 200 */ 2967, 2968, 3085, 2305, 3024, 2986, 2305, 2981, 2305, 2305, + /* 210 */ 2305, 2305, 2305, 3011, 2305, 2305, 2305, 2305, 2305, 2305, + /* 220 */ 2657, 2305, 2759, 2305, 2502, 2709, 2305, 2305, 2305, 2305, + /* 230 */ 2305, 3069, 2965, 3005, 2305, 2305, 3015, 2305, 2305, 2305, + /* 240 */ 2747, 2415, 2305, 2415, 2702, 2640, 2305, 2650, 2305, 2650, + /* 250 */ 2647, 2305, 2305, 2305, 2650, 2647, 2647, 2647, 2490, 2486, + /* 260 */ 2305, 2305, 2484, 2305, 2305, 2305, 2305, 2365, 2305, 2365, + /* 270 */ 2305, 2415, 2415, 2305, 2415, 2305, 2305, 2415, 2305, 2415, + /* 280 */ 2305, 2415, 2305, 2415, 2415, 2519, 2519, 2305, 2415, 2305, + /* 290 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 300 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2745, 2725, + /* 310 */ 2305, 2413, 2305, 2713, 2711, 2305, 2413, 3015, 2305, 2305, + /* 320 */ 3039, 3034, 3039, 3034, 3053, 3049, 3039, 3058, 3055, 3017, + /* 330 */ 3015, 2998, 2994, 3088, 3075, 3071, 2305, 2305, 3003, 3001, + /* 340 */ 2305, 2413, 2413, 3034, 2305, 2305, 2305, 2305, 3034, 2305, + /* 350 */ 2305, 2413, 2305, 2305, 2413, 2305, 2305, 2305, 2305, 2413, + /* 360 */ 2305, 2413, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 370 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 380 */ 2305, 2305, 2305, 2305, 2305, 2521, 2305, 2305, 2413, 2305, + /* 390 */ 2337, 2339, 2349, 2305, 2704, 3091, 2730, 2735, 2685, 2685, + /* 400 */ 2562, 2562, 3091, 2562, 2416, 2310, 2305, 2305, 2305, 2305, + /* 410 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 420 */ 2305, 2305, 2864, 2305, 2305, 2305, 2305, 2305, 2305, 3052, + /* 430 */ 3051, 2865, 2305, 2939, 2938, 2937, 2928, 2864, 2515, 2305, + /* 440 */ 2305, 2305, 2863, 2862, 2305, 2305, 2305, 2305, 2305, 2305, + /* 450 */ 2305, 2305, 2305, 2305, 2676, 2675, 2856, 2305, 2305, 2857, + /* 460 */ 2855, 2854, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 470 */ 2305, 2305, 2305, 2506, 2305, 2305, 2503, 2305, 2305, 2305, + /* 480 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 490 */ 2305, 2305, 2305, 2305, 3072, 3076, 2305, 2305, 2305, 2305, + /* 500 */ 2964, 2305, 2305, 2305, 2305, 2305, 2835, 2305, 2305, 2305, + /* 510 */ 2305, 2305, 2803, 2798, 2789, 2780, 2795, 2786, 2774, 2792, + /* 520 */ 2783, 2771, 2768, 2305, 2530, 2305, 2305, 2305, 2305, 2305, + /* 530 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 540 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 550 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 560 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 570 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 580 */ 2646, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 590 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 600 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 610 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 620 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 630 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 640 */ 2305, 2305, 2305, 2305, 2305, 2305, 2661, 2305, 2305, 2305, + /* 650 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 660 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 670 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 680 */ 2305, 2305, 2305, 2305, 2354, 2842, 2305, 2305, 2305, 2305, + /* 690 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2845, + /* 700 */ 2305, 2305, 2846, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 710 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 720 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 730 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 740 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2461, 2460, 2305, + /* 750 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 760 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 770 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2847, 2305, 2305, + /* 780 */ 2305, 2305, 2729, 2305, 2305, 2837, 2305, 2305, 2305, 2305, + /* 790 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 800 */ 2305, 2305, 3068, 3018, 2305, 2305, 2305, 2305, 2305, 2305, + /* 810 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 820 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2835, + /* 830 */ 2305, 3050, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 3066, + /* 840 */ 2305, 3070, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2974, + /* 850 */ 2970, 2305, 2305, 2966, 2305, 2305, 2305, 2305, 2305, 2305, + /* 860 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 870 */ 2305, 2305, 2305, 2305, 2925, 2305, 2305, 2305, 2959, 2305, + /* 880 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2558, 2557, + /* 890 */ 2556, 2555, 2305, 2305, 2305, 2305, 2305, 2305, 2847, 2305, + /* 900 */ 2850, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 910 */ 2305, 2305, 2305, 2305, 2834, 2305, 2902, 2901, 2305, 2305, + /* 920 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2552, 2305, 2305, + /* 930 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 940 */ 2305, 2305, 2305, 2536, 2534, 2533, 2532, 2305, 2569, 2305, + /* 950 */ 2305, 2305, 2565, 2564, 2305, 2305, 2305, 2305, 2305, 2305, + /* 960 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 970 */ 2305, 2305, 2305, 2305, 2434, 2305, 2305, 2305, 2305, 2305, + /* 980 */ 2305, 2305, 2305, 2426, 2305, 2425, 2305, 2305, 2305, 2305, + /* 990 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 1000 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2305, + /* 1010 */ 2305, 2305, 2305, 2305, 2305, 2305, 2305, 2334, 2305, 2305, + /* 1020 */ 2305, 2305, 2305, 2305, 2305, 2305, }; /********** End of lemon-generated parsing tables *****************************/ @@ -1555,12 +1991,12 @@ static const YYCODETYPE yyFallback[] = { 0, /* BWLIMIT => nothing */ 0, /* START => nothing */ 0, /* TIMESTAMP => nothing */ - 343, /* END => ABORT */ + 344, /* END => ABORT */ 0, /* TABLE => nothing */ 0, /* NK_LP => nothing */ 0, /* NK_RP => nothing */ 0, /* USING => nothing */ - 343, /* FILE => ABORT */ + 344, /* FILE => ABORT */ 0, /* STABLE => nothing */ 0, /* COLUMN => nothing */ 0, /* MODIFY => nothing */ @@ -1628,7 +2064,7 @@ static const YYCODETYPE yyFallback[] = { 0, /* VNODES => nothing */ 0, /* ALIVE => nothing */ 0, /* VIEWS => nothing */ - 343, /* VIEW => ABORT */ + 344, /* VIEW => ABORT */ 0, /* COMPACTS => nothing */ 0, /* NORMAL => nothing */ 0, /* CHILD => nothing */ @@ -1671,10 +2107,11 @@ static const YYCODETYPE yyFallback[] = { 0, /* PAUSE => nothing */ 0, /* RESUME => nothing */ 0, /* PRIMARY => nothing */ - 343, /* KEY => ABORT */ + 344, /* KEY => ABORT */ 0, /* TRIGGER => nothing */ 0, /* AT_ONCE => nothing */ 0, /* WINDOW_CLOSE => nothing */ + 0, /* FORCE_WINDOW_CLOSE => nothing */ 0, /* IGNORE => nothing */ 0, /* EXPIRED => nothing */ 0, /* FILL_HISTORY => nothing */ @@ -1712,7 +2149,7 @@ static const YYCODETYPE yyFallback[] = { 0, /* CAST => nothing */ 0, /* POSITION => nothing */ 0, /* IN => nothing */ - 343, /* FOR => ABORT */ + 344, /* FOR => ABORT */ 0, /* NOW => nothing */ 0, /* TODAY => nothing */ 0, /* RAND => nothing */ @@ -1746,7 +2183,7 @@ static const YYCODETYPE yyFallback[] = { 0, /* LEFT => nothing */ 0, /* RIGHT => nothing */ 0, /* OUTER => nothing */ - 343, /* SEMI => ABORT */ + 344, /* SEMI => ABORT */ 0, /* ANTI => nothing */ 0, /* ASOF => nothing */ 0, /* WINDOW => nothing */ @@ -1783,51 +2220,51 @@ static const YYCODETYPE yyFallback[] = { 0, /* ASC => nothing */ 0, /* NULLS => nothing */ 0, /* ABORT => nothing */ - 343, /* AFTER => ABORT */ - 343, /* ATTACH => ABORT */ - 343, /* BEFORE => ABORT */ - 343, /* BEGIN => ABORT */ - 343, /* BITAND => ABORT */ - 343, /* BITNOT => ABORT */ - 343, /* BITOR => ABORT */ - 343, /* BLOCKS => ABORT */ - 343, /* CHANGE => ABORT */ - 343, /* COMMA => ABORT */ - 343, /* CONCAT => ABORT */ - 343, /* CONFLICT => ABORT */ - 343, /* COPY => ABORT */ - 343, /* DEFERRED => ABORT */ - 343, /* DELIMITERS => ABORT */ - 343, /* DETACH => ABORT */ - 343, /* DIVIDE => ABORT */ - 343, /* DOT => ABORT */ - 343, /* EACH => ABORT */ - 343, /* FAIL => ABORT */ - 343, /* GLOB => ABORT */ - 343, /* ID => ABORT */ - 343, /* IMMEDIATE => ABORT */ - 343, /* IMPORT => ABORT */ - 343, /* INITIALLY => ABORT */ - 343, /* INSTEAD => ABORT */ - 343, /* ISNULL => ABORT */ - 343, /* MODULES => ABORT */ - 343, /* NK_BITNOT => ABORT */ - 343, /* NK_SEMI => ABORT */ - 343, /* NOTNULL => ABORT */ - 343, /* OF => ABORT */ - 343, /* PLUS => ABORT */ - 343, /* PRIVILEGE => ABORT */ - 343, /* RAISE => ABORT */ - 343, /* RESTRICT => ABORT */ - 343, /* ROW => ABORT */ - 343, /* STAR => ABORT */ - 343, /* STATEMENT => ABORT */ - 343, /* STRICT => ABORT */ - 343, /* STRING => ABORT */ - 343, /* TIMES => ABORT */ - 343, /* VALUES => ABORT */ - 343, /* VARIABLE => ABORT */ - 343, /* WAL => ABORT */ + 344, /* AFTER => ABORT */ + 344, /* ATTACH => ABORT */ + 344, /* BEFORE => ABORT */ + 344, /* BEGIN => ABORT */ + 344, /* BITAND => ABORT */ + 344, /* BITNOT => ABORT */ + 344, /* BITOR => ABORT */ + 344, /* BLOCKS => ABORT */ + 344, /* CHANGE => ABORT */ + 344, /* COMMA => ABORT */ + 344, /* CONCAT => ABORT */ + 344, /* CONFLICT => ABORT */ + 344, /* COPY => ABORT */ + 344, /* DEFERRED => ABORT */ + 344, /* DELIMITERS => ABORT */ + 344, /* DETACH => ABORT */ + 344, /* DIVIDE => ABORT */ + 344, /* DOT => ABORT */ + 344, /* EACH => ABORT */ + 344, /* FAIL => ABORT */ + 344, /* GLOB => ABORT */ + 344, /* ID => ABORT */ + 344, /* IMMEDIATE => ABORT */ + 344, /* IMPORT => ABORT */ + 344, /* INITIALLY => ABORT */ + 344, /* INSTEAD => ABORT */ + 344, /* ISNULL => ABORT */ + 344, /* MODULES => ABORT */ + 344, /* NK_BITNOT => ABORT */ + 344, /* NK_SEMI => ABORT */ + 344, /* NOTNULL => ABORT */ + 344, /* OF => ABORT */ + 344, /* PLUS => ABORT */ + 344, /* PRIVILEGE => ABORT */ + 344, /* RAISE => ABORT */ + 344, /* RESTRICT => ABORT */ + 344, /* ROW => ABORT */ + 344, /* STAR => ABORT */ + 344, /* STATEMENT => ABORT */ + 344, /* STRICT => ABORT */ + 344, /* STRING => ABORT */ + 344, /* TIMES => ABORT */ + 344, /* VALUES => ABORT */ + 344, /* VARIABLE => ABORT */ + 344, /* WAL => ABORT */ }; #endif /* YYFALLBACK */ @@ -1879,6 +2316,7 @@ struct yyParser { }; typedef struct yyParser yyParser; +#include #ifndef NDEBUG #include static FILE *yyTraceFILE = 0; @@ -2151,343 +2589,344 @@ static const char *const yyTokenName[] = { /* 233 */ "TRIGGER", /* 234 */ "AT_ONCE", /* 235 */ "WINDOW_CLOSE", - /* 236 */ "IGNORE", - /* 237 */ "EXPIRED", - /* 238 */ "FILL_HISTORY", - /* 239 */ "SUBTABLE", - /* 240 */ "UNTREATED", - /* 241 */ "KILL", - /* 242 */ "CONNECTION", - /* 243 */ "TRANSACTION", - /* 244 */ "BALANCE", - /* 245 */ "VGROUP", - /* 246 */ "LEADER", - /* 247 */ "MERGE", - /* 248 */ "REDISTRIBUTE", - /* 249 */ "SPLIT", - /* 250 */ "DELETE", - /* 251 */ "INSERT", - /* 252 */ "NK_BIN", - /* 253 */ "NK_HEX", - /* 254 */ "NULL", - /* 255 */ "NK_QUESTION", - /* 256 */ "NK_ALIAS", - /* 257 */ "NK_ARROW", - /* 258 */ "ROWTS", - /* 259 */ "QSTART", - /* 260 */ "QEND", - /* 261 */ "QDURATION", - /* 262 */ "WSTART", - /* 263 */ "WEND", - /* 264 */ "WDURATION", - /* 265 */ "IROWTS", - /* 266 */ "ISFILLED", - /* 267 */ "FLOW", - /* 268 */ "FHIGH", - /* 269 */ "FROWTS", - /* 270 */ "CAST", - /* 271 */ "POSITION", - /* 272 */ "IN", - /* 273 */ "FOR", - /* 274 */ "NOW", - /* 275 */ "TODAY", - /* 276 */ "RAND", - /* 277 */ "SUBSTR", - /* 278 */ "SUBSTRING", - /* 279 */ "BOTH", - /* 280 */ "TRAILING", - /* 281 */ "LEADING", - /* 282 */ "TIMEZONE", - /* 283 */ "CLIENT_VERSION", - /* 284 */ "SERVER_VERSION", - /* 285 */ "SERVER_STATUS", - /* 286 */ "CURRENT_USER", - /* 287 */ "PI", - /* 288 */ "CASE", - /* 289 */ "WHEN", - /* 290 */ "THEN", - /* 291 */ "ELSE", - /* 292 */ "BETWEEN", - /* 293 */ "IS", - /* 294 */ "NK_LT", - /* 295 */ "NK_GT", - /* 296 */ "NK_LE", - /* 297 */ "NK_GE", - /* 298 */ "NK_NE", - /* 299 */ "MATCH", - /* 300 */ "NMATCH", - /* 301 */ "CONTAINS", - /* 302 */ "JOIN", - /* 303 */ "INNER", - /* 304 */ "LEFT", - /* 305 */ "RIGHT", - /* 306 */ "OUTER", - /* 307 */ "SEMI", - /* 308 */ "ANTI", - /* 309 */ "ASOF", - /* 310 */ "WINDOW", - /* 311 */ "WINDOW_OFFSET", - /* 312 */ "JLIMIT", - /* 313 */ "SELECT", - /* 314 */ "NK_HINT", - /* 315 */ "DISTINCT", - /* 316 */ "WHERE", - /* 317 */ "PARTITION", - /* 318 */ "BY", - /* 319 */ "SESSION", - /* 320 */ "STATE_WINDOW", - /* 321 */ "EVENT_WINDOW", - /* 322 */ "COUNT_WINDOW", - /* 323 */ "ANOMALY_WINDOW", - /* 324 */ "SLIDING", - /* 325 */ "FILL", - /* 326 */ "VALUE", - /* 327 */ "VALUE_F", - /* 328 */ "NONE", - /* 329 */ "PREV", - /* 330 */ "NULL_F", - /* 331 */ "LINEAR", - /* 332 */ "NEXT", - /* 333 */ "HAVING", - /* 334 */ "RANGE", - /* 335 */ "EVERY", - /* 336 */ "ORDER", - /* 337 */ "SLIMIT", - /* 338 */ "SOFFSET", - /* 339 */ "LIMIT", - /* 340 */ "OFFSET", - /* 341 */ "ASC", - /* 342 */ "NULLS", - /* 343 */ "ABORT", - /* 344 */ "AFTER", - /* 345 */ "ATTACH", - /* 346 */ "BEFORE", - /* 347 */ "BEGIN", - /* 348 */ "BITAND", - /* 349 */ "BITNOT", - /* 350 */ "BITOR", - /* 351 */ "BLOCKS", - /* 352 */ "CHANGE", - /* 353 */ "COMMA", - /* 354 */ "CONCAT", - /* 355 */ "CONFLICT", - /* 356 */ "COPY", - /* 357 */ "DEFERRED", - /* 358 */ "DELIMITERS", - /* 359 */ "DETACH", - /* 360 */ "DIVIDE", - /* 361 */ "DOT", - /* 362 */ "EACH", - /* 363 */ "FAIL", - /* 364 */ "GLOB", - /* 365 */ "ID", - /* 366 */ "IMMEDIATE", - /* 367 */ "IMPORT", - /* 368 */ "INITIALLY", - /* 369 */ "INSTEAD", - /* 370 */ "ISNULL", - /* 371 */ "MODULES", - /* 372 */ "NK_BITNOT", - /* 373 */ "NK_SEMI", - /* 374 */ "NOTNULL", - /* 375 */ "OF", - /* 376 */ "PLUS", - /* 377 */ "PRIVILEGE", - /* 378 */ "RAISE", - /* 379 */ "RESTRICT", - /* 380 */ "ROW", - /* 381 */ "STAR", - /* 382 */ "STATEMENT", - /* 383 */ "STRICT", - /* 384 */ "STRING", - /* 385 */ "TIMES", - /* 386 */ "VALUES", - /* 387 */ "VARIABLE", - /* 388 */ "WAL", - /* 389 */ "cmd", - /* 390 */ "account_options", - /* 391 */ "alter_account_options", - /* 392 */ "literal", - /* 393 */ "alter_account_option", - /* 394 */ "ip_range_list", - /* 395 */ "white_list", - /* 396 */ "white_list_opt", - /* 397 */ "is_import_opt", - /* 398 */ "is_createdb_opt", - /* 399 */ "user_name", - /* 400 */ "sysinfo_opt", - /* 401 */ "privileges", - /* 402 */ "priv_level", - /* 403 */ "with_clause_opt", - /* 404 */ "priv_type_list", - /* 405 */ "priv_type", - /* 406 */ "db_name", - /* 407 */ "table_name", - /* 408 */ "topic_name", - /* 409 */ "search_condition", - /* 410 */ "dnode_endpoint", - /* 411 */ "force_opt", - /* 412 */ "unsafe_opt", - /* 413 */ "not_exists_opt", - /* 414 */ "db_options", - /* 415 */ "exists_opt", - /* 416 */ "alter_db_options", - /* 417 */ "speed_opt", - /* 418 */ "start_opt", - /* 419 */ "end_opt", - /* 420 */ "integer_list", - /* 421 */ "variable_list", - /* 422 */ "retention_list", - /* 423 */ "signed", - /* 424 */ "alter_db_option", - /* 425 */ "retention", - /* 426 */ "full_table_name", - /* 427 */ "column_def_list", - /* 428 */ "tags_def_opt", - /* 429 */ "table_options", - /* 430 */ "multi_create_clause", - /* 431 */ "tag_list_opt", - /* 432 */ "tags_def", - /* 433 */ "with_opt", - /* 434 */ "multi_drop_clause", - /* 435 */ "alter_table_clause", - /* 436 */ "alter_table_options", - /* 437 */ "column_name", - /* 438 */ "type_name", - /* 439 */ "column_options", - /* 440 */ "tags_literal", - /* 441 */ "create_subtable_clause", - /* 442 */ "specific_cols_opt", - /* 443 */ "tags_literal_list", - /* 444 */ "drop_table_clause", - /* 445 */ "col_name_list", - /* 446 */ "tag_def_list", - /* 447 */ "tag_def", - /* 448 */ "column_def", - /* 449 */ "type_name_default_len", - /* 450 */ "duration_list", - /* 451 */ "rollup_func_list", - /* 452 */ "alter_table_option", - /* 453 */ "duration_literal", - /* 454 */ "rollup_func_name", - /* 455 */ "function_name", - /* 456 */ "col_name", - /* 457 */ "db_kind_opt", - /* 458 */ "table_kind_db_name_cond_opt", - /* 459 */ "like_pattern_opt", - /* 460 */ "db_name_cond_opt", - /* 461 */ "table_name_cond", - /* 462 */ "from_db_opt", - /* 463 */ "table_kind", - /* 464 */ "tag_item", - /* 465 */ "column_alias", - /* 466 */ "tsma_name", - /* 467 */ "tsma_func_list", - /* 468 */ "full_tsma_name", - /* 469 */ "func_list", - /* 470 */ "index_options", - /* 471 */ "full_index_name", - /* 472 */ "index_name", - /* 473 */ "sliding_opt", - /* 474 */ "sma_stream_opt", - /* 475 */ "func", - /* 476 */ "sma_func_name", - /* 477 */ "expression_list", - /* 478 */ "with_meta", - /* 479 */ "query_or_subquery", - /* 480 */ "where_clause_opt", - /* 481 */ "cgroup_name", - /* 482 */ "analyze_opt", - /* 483 */ "explain_options", - /* 484 */ "insert_query", - /* 485 */ "or_replace_opt", - /* 486 */ "agg_func_opt", - /* 487 */ "bufsize_opt", - /* 488 */ "language_opt", - /* 489 */ "full_view_name", - /* 490 */ "view_name", - /* 491 */ "stream_name", - /* 492 */ "stream_options", - /* 493 */ "col_list_opt", - /* 494 */ "tag_def_or_ref_opt", - /* 495 */ "subtable_opt", - /* 496 */ "ignore_opt", - /* 497 */ "column_stream_def_list", - /* 498 */ "column_stream_def", - /* 499 */ "stream_col_options", - /* 500 */ "expression", - /* 501 */ "on_vgroup_id", - /* 502 */ "dnode_list", - /* 503 */ "literal_func", - /* 504 */ "signed_literal", - /* 505 */ "literal_list", - /* 506 */ "table_alias", - /* 507 */ "expr_or_subquery", - /* 508 */ "pseudo_column", - /* 509 */ "column_reference", - /* 510 */ "function_expression", - /* 511 */ "case_when_expression", - /* 512 */ "star_func", - /* 513 */ "star_func_para_list", - /* 514 */ "trim_specification_type", - /* 515 */ "substr_func", - /* 516 */ "rand_func", - /* 517 */ "noarg_func", - /* 518 */ "other_para_list", - /* 519 */ "star_func_para", - /* 520 */ "when_then_list", - /* 521 */ "case_when_else_opt", - /* 522 */ "common_expression", - /* 523 */ "when_then_expr", - /* 524 */ "predicate", - /* 525 */ "compare_op", - /* 526 */ "in_op", - /* 527 */ "in_predicate_value", - /* 528 */ "boolean_value_expression", - /* 529 */ "boolean_primary", - /* 530 */ "from_clause_opt", - /* 531 */ "table_reference_list", - /* 532 */ "table_reference", - /* 533 */ "table_primary", - /* 534 */ "joined_table", - /* 535 */ "alias_opt", - /* 536 */ "subquery", - /* 537 */ "parenthesized_joined_table", - /* 538 */ "join_type", - /* 539 */ "join_subtype", - /* 540 */ "join_on_clause_opt", - /* 541 */ "window_offset_clause_opt", - /* 542 */ "jlimit_clause_opt", - /* 543 */ "window_offset_literal", - /* 544 */ "query_specification", - /* 545 */ "hint_list", - /* 546 */ "set_quantifier_opt", - /* 547 */ "tag_mode_opt", - /* 548 */ "select_list", - /* 549 */ "partition_by_clause_opt", - /* 550 */ "range_opt", - /* 551 */ "every_opt", - /* 552 */ "fill_opt", - /* 553 */ "twindow_clause_opt", - /* 554 */ "group_by_clause_opt", - /* 555 */ "having_clause_opt", - /* 556 */ "select_item", - /* 557 */ "partition_list", - /* 558 */ "partition_item", - /* 559 */ "interval_sliding_duration_literal", - /* 560 */ "fill_mode", - /* 561 */ "group_by_list", - /* 562 */ "query_expression", - /* 563 */ "query_simple", - /* 564 */ "order_by_clause_opt", - /* 565 */ "slimit_clause_opt", - /* 566 */ "limit_clause_opt", - /* 567 */ "union_query_expression", - /* 568 */ "query_simple_or_subquery", - /* 569 */ "sort_specification_list", - /* 570 */ "sort_specification", - /* 571 */ "ordering_specification_opt", - /* 572 */ "null_ordering_opt", + /* 236 */ "FORCE_WINDOW_CLOSE", + /* 237 */ "IGNORE", + /* 238 */ "EXPIRED", + /* 239 */ "FILL_HISTORY", + /* 240 */ "SUBTABLE", + /* 241 */ "UNTREATED", + /* 242 */ "KILL", + /* 243 */ "CONNECTION", + /* 244 */ "TRANSACTION", + /* 245 */ "BALANCE", + /* 246 */ "VGROUP", + /* 247 */ "LEADER", + /* 248 */ "MERGE", + /* 249 */ "REDISTRIBUTE", + /* 250 */ "SPLIT", + /* 251 */ "DELETE", + /* 252 */ "INSERT", + /* 253 */ "NK_BIN", + /* 254 */ "NK_HEX", + /* 255 */ "NULL", + /* 256 */ "NK_QUESTION", + /* 257 */ "NK_ALIAS", + /* 258 */ "NK_ARROW", + /* 259 */ "ROWTS", + /* 260 */ "QSTART", + /* 261 */ "QEND", + /* 262 */ "QDURATION", + /* 263 */ "WSTART", + /* 264 */ "WEND", + /* 265 */ "WDURATION", + /* 266 */ "IROWTS", + /* 267 */ "ISFILLED", + /* 268 */ "FLOW", + /* 269 */ "FHIGH", + /* 270 */ "FROWTS", + /* 271 */ "CAST", + /* 272 */ "POSITION", + /* 273 */ "IN", + /* 274 */ "FOR", + /* 275 */ "NOW", + /* 276 */ "TODAY", + /* 277 */ "RAND", + /* 278 */ "SUBSTR", + /* 279 */ "SUBSTRING", + /* 280 */ "BOTH", + /* 281 */ "TRAILING", + /* 282 */ "LEADING", + /* 283 */ "TIMEZONE", + /* 284 */ "CLIENT_VERSION", + /* 285 */ "SERVER_VERSION", + /* 286 */ "SERVER_STATUS", + /* 287 */ "CURRENT_USER", + /* 288 */ "PI", + /* 289 */ "CASE", + /* 290 */ "WHEN", + /* 291 */ "THEN", + /* 292 */ "ELSE", + /* 293 */ "BETWEEN", + /* 294 */ "IS", + /* 295 */ "NK_LT", + /* 296 */ "NK_GT", + /* 297 */ "NK_LE", + /* 298 */ "NK_GE", + /* 299 */ "NK_NE", + /* 300 */ "MATCH", + /* 301 */ "NMATCH", + /* 302 */ "CONTAINS", + /* 303 */ "JOIN", + /* 304 */ "INNER", + /* 305 */ "LEFT", + /* 306 */ "RIGHT", + /* 307 */ "OUTER", + /* 308 */ "SEMI", + /* 309 */ "ANTI", + /* 310 */ "ASOF", + /* 311 */ "WINDOW", + /* 312 */ "WINDOW_OFFSET", + /* 313 */ "JLIMIT", + /* 314 */ "SELECT", + /* 315 */ "NK_HINT", + /* 316 */ "DISTINCT", + /* 317 */ "WHERE", + /* 318 */ "PARTITION", + /* 319 */ "BY", + /* 320 */ "SESSION", + /* 321 */ "STATE_WINDOW", + /* 322 */ "EVENT_WINDOW", + /* 323 */ "COUNT_WINDOW", + /* 324 */ "ANOMALY_WINDOW", + /* 325 */ "SLIDING", + /* 326 */ "FILL", + /* 327 */ "VALUE", + /* 328 */ "VALUE_F", + /* 329 */ "NONE", + /* 330 */ "PREV", + /* 331 */ "NULL_F", + /* 332 */ "LINEAR", + /* 333 */ "NEXT", + /* 334 */ "HAVING", + /* 335 */ "RANGE", + /* 336 */ "EVERY", + /* 337 */ "ORDER", + /* 338 */ "SLIMIT", + /* 339 */ "SOFFSET", + /* 340 */ "LIMIT", + /* 341 */ "OFFSET", + /* 342 */ "ASC", + /* 343 */ "NULLS", + /* 344 */ "ABORT", + /* 345 */ "AFTER", + /* 346 */ "ATTACH", + /* 347 */ "BEFORE", + /* 348 */ "BEGIN", + /* 349 */ "BITAND", + /* 350 */ "BITNOT", + /* 351 */ "BITOR", + /* 352 */ "BLOCKS", + /* 353 */ "CHANGE", + /* 354 */ "COMMA", + /* 355 */ "CONCAT", + /* 356 */ "CONFLICT", + /* 357 */ "COPY", + /* 358 */ "DEFERRED", + /* 359 */ "DELIMITERS", + /* 360 */ "DETACH", + /* 361 */ "DIVIDE", + /* 362 */ "DOT", + /* 363 */ "EACH", + /* 364 */ "FAIL", + /* 365 */ "GLOB", + /* 366 */ "ID", + /* 367 */ "IMMEDIATE", + /* 368 */ "IMPORT", + /* 369 */ "INITIALLY", + /* 370 */ "INSTEAD", + /* 371 */ "ISNULL", + /* 372 */ "MODULES", + /* 373 */ "NK_BITNOT", + /* 374 */ "NK_SEMI", + /* 375 */ "NOTNULL", + /* 376 */ "OF", + /* 377 */ "PLUS", + /* 378 */ "PRIVILEGE", + /* 379 */ "RAISE", + /* 380 */ "RESTRICT", + /* 381 */ "ROW", + /* 382 */ "STAR", + /* 383 */ "STATEMENT", + /* 384 */ "STRICT", + /* 385 */ "STRING", + /* 386 */ "TIMES", + /* 387 */ "VALUES", + /* 388 */ "VARIABLE", + /* 389 */ "WAL", + /* 390 */ "cmd", + /* 391 */ "account_options", + /* 392 */ "alter_account_options", + /* 393 */ "literal", + /* 394 */ "alter_account_option", + /* 395 */ "ip_range_list", + /* 396 */ "white_list", + /* 397 */ "white_list_opt", + /* 398 */ "is_import_opt", + /* 399 */ "is_createdb_opt", + /* 400 */ "user_name", + /* 401 */ "sysinfo_opt", + /* 402 */ "privileges", + /* 403 */ "priv_level", + /* 404 */ "with_clause_opt", + /* 405 */ "priv_type_list", + /* 406 */ "priv_type", + /* 407 */ "db_name", + /* 408 */ "table_name", + /* 409 */ "topic_name", + /* 410 */ "search_condition", + /* 411 */ "dnode_endpoint", + /* 412 */ "force_opt", + /* 413 */ "unsafe_opt", + /* 414 */ "not_exists_opt", + /* 415 */ "db_options", + /* 416 */ "exists_opt", + /* 417 */ "alter_db_options", + /* 418 */ "speed_opt", + /* 419 */ "start_opt", + /* 420 */ "end_opt", + /* 421 */ "integer_list", + /* 422 */ "variable_list", + /* 423 */ "retention_list", + /* 424 */ "signed", + /* 425 */ "alter_db_option", + /* 426 */ "retention", + /* 427 */ "full_table_name", + /* 428 */ "column_def_list", + /* 429 */ "tags_def_opt", + /* 430 */ "table_options", + /* 431 */ "multi_create_clause", + /* 432 */ "tag_list_opt", + /* 433 */ "tags_def", + /* 434 */ "with_opt", + /* 435 */ "multi_drop_clause", + /* 436 */ "alter_table_clause", + /* 437 */ "alter_table_options", + /* 438 */ "column_name", + /* 439 */ "type_name", + /* 440 */ "column_options", + /* 441 */ "tags_literal", + /* 442 */ "create_subtable_clause", + /* 443 */ "specific_cols_opt", + /* 444 */ "tags_literal_list", + /* 445 */ "drop_table_clause", + /* 446 */ "col_name_list", + /* 447 */ "tag_def_list", + /* 448 */ "tag_def", + /* 449 */ "column_def", + /* 450 */ "type_name_default_len", + /* 451 */ "duration_list", + /* 452 */ "rollup_func_list", + /* 453 */ "alter_table_option", + /* 454 */ "duration_literal", + /* 455 */ "rollup_func_name", + /* 456 */ "function_name", + /* 457 */ "col_name", + /* 458 */ "db_kind_opt", + /* 459 */ "table_kind_db_name_cond_opt", + /* 460 */ "like_pattern_opt", + /* 461 */ "db_name_cond_opt", + /* 462 */ "table_name_cond", + /* 463 */ "from_db_opt", + /* 464 */ "table_kind", + /* 465 */ "tag_item", + /* 466 */ "column_alias", + /* 467 */ "tsma_name", + /* 468 */ "tsma_func_list", + /* 469 */ "full_tsma_name", + /* 470 */ "func_list", + /* 471 */ "index_options", + /* 472 */ "full_index_name", + /* 473 */ "index_name", + /* 474 */ "sliding_opt", + /* 475 */ "sma_stream_opt", + /* 476 */ "func", + /* 477 */ "sma_func_name", + /* 478 */ "expression_list", + /* 479 */ "with_meta", + /* 480 */ "query_or_subquery", + /* 481 */ "where_clause_opt", + /* 482 */ "cgroup_name", + /* 483 */ "analyze_opt", + /* 484 */ "explain_options", + /* 485 */ "insert_query", + /* 486 */ "or_replace_opt", + /* 487 */ "agg_func_opt", + /* 488 */ "bufsize_opt", + /* 489 */ "language_opt", + /* 490 */ "full_view_name", + /* 491 */ "view_name", + /* 492 */ "stream_name", + /* 493 */ "stream_options", + /* 494 */ "col_list_opt", + /* 495 */ "tag_def_or_ref_opt", + /* 496 */ "subtable_opt", + /* 497 */ "ignore_opt", + /* 498 */ "column_stream_def_list", + /* 499 */ "column_stream_def", + /* 500 */ "stream_col_options", + /* 501 */ "expression", + /* 502 */ "on_vgroup_id", + /* 503 */ "dnode_list", + /* 504 */ "literal_func", + /* 505 */ "signed_literal", + /* 506 */ "literal_list", + /* 507 */ "table_alias", + /* 508 */ "expr_or_subquery", + /* 509 */ "pseudo_column", + /* 510 */ "column_reference", + /* 511 */ "function_expression", + /* 512 */ "case_when_expression", + /* 513 */ "star_func", + /* 514 */ "star_func_para_list", + /* 515 */ "trim_specification_type", + /* 516 */ "substr_func", + /* 517 */ "rand_func", + /* 518 */ "noarg_func", + /* 519 */ "other_para_list", + /* 520 */ "star_func_para", + /* 521 */ "when_then_list", + /* 522 */ "case_when_else_opt", + /* 523 */ "common_expression", + /* 524 */ "when_then_expr", + /* 525 */ "predicate", + /* 526 */ "compare_op", + /* 527 */ "in_op", + /* 528 */ "in_predicate_value", + /* 529 */ "boolean_value_expression", + /* 530 */ "boolean_primary", + /* 531 */ "from_clause_opt", + /* 532 */ "table_reference_list", + /* 533 */ "table_reference", + /* 534 */ "table_primary", + /* 535 */ "joined_table", + /* 536 */ "alias_opt", + /* 537 */ "subquery", + /* 538 */ "parenthesized_joined_table", + /* 539 */ "join_type", + /* 540 */ "join_subtype", + /* 541 */ "join_on_clause_opt", + /* 542 */ "window_offset_clause_opt", + /* 543 */ "jlimit_clause_opt", + /* 544 */ "window_offset_literal", + /* 545 */ "query_specification", + /* 546 */ "hint_list", + /* 547 */ "set_quantifier_opt", + /* 548 */ "tag_mode_opt", + /* 549 */ "select_list", + /* 550 */ "partition_by_clause_opt", + /* 551 */ "range_opt", + /* 552 */ "every_opt", + /* 553 */ "fill_opt", + /* 554 */ "twindow_clause_opt", + /* 555 */ "group_by_clause_opt", + /* 556 */ "having_clause_opt", + /* 557 */ "select_item", + /* 558 */ "partition_list", + /* 559 */ "partition_item", + /* 560 */ "interval_sliding_duration_literal", + /* 561 */ "fill_mode", + /* 562 */ "group_by_list", + /* 563 */ "query_expression", + /* 564 */ "query_simple", + /* 565 */ "order_by_clause_opt", + /* 566 */ "slimit_clause_opt", + /* 567 */ "limit_clause_opt", + /* 568 */ "union_query_expression", + /* 569 */ "query_simple_or_subquery", + /* 570 */ "sort_specification_list", + /* 571 */ "sort_specification", + /* 572 */ "ordering_specification_opt", + /* 573 */ "null_ordering_opt", }; #endif /* defined(YYCOVERAGE) || !defined(NDEBUG) */ @@ -2925,361 +3364,362 @@ static const char *const yyRuleName[] = { /* 427 */ "stream_options ::=", /* 428 */ "stream_options ::= stream_options TRIGGER AT_ONCE", /* 429 */ "stream_options ::= stream_options TRIGGER WINDOW_CLOSE", - /* 430 */ "stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal", - /* 431 */ "stream_options ::= stream_options WATERMARK duration_literal", - /* 432 */ "stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER", - /* 433 */ "stream_options ::= stream_options FILL_HISTORY NK_INTEGER", - /* 434 */ "stream_options ::= stream_options DELETE_MARK duration_literal", - /* 435 */ "stream_options ::= stream_options IGNORE UPDATE NK_INTEGER", - /* 436 */ "subtable_opt ::=", - /* 437 */ "subtable_opt ::= SUBTABLE NK_LP expression NK_RP", - /* 438 */ "ignore_opt ::=", - /* 439 */ "ignore_opt ::= IGNORE UNTREATED", - /* 440 */ "cmd ::= KILL CONNECTION NK_INTEGER", - /* 441 */ "cmd ::= KILL QUERY NK_STRING", - /* 442 */ "cmd ::= KILL TRANSACTION NK_INTEGER", - /* 443 */ "cmd ::= KILL COMPACT NK_INTEGER", - /* 444 */ "cmd ::= BALANCE VGROUP", - /* 445 */ "cmd ::= BALANCE VGROUP LEADER on_vgroup_id", - /* 446 */ "cmd ::= BALANCE VGROUP LEADER DATABASE db_name", - /* 447 */ "cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER", - /* 448 */ "cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list", - /* 449 */ "cmd ::= SPLIT VGROUP NK_INTEGER", - /* 450 */ "on_vgroup_id ::=", - /* 451 */ "on_vgroup_id ::= ON NK_INTEGER", - /* 452 */ "dnode_list ::= DNODE NK_INTEGER", - /* 453 */ "dnode_list ::= dnode_list DNODE NK_INTEGER", - /* 454 */ "cmd ::= DELETE FROM full_table_name where_clause_opt", - /* 455 */ "cmd ::= query_or_subquery", - /* 456 */ "cmd ::= insert_query", - /* 457 */ "insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery", - /* 458 */ "insert_query ::= INSERT INTO full_table_name query_or_subquery", - /* 459 */ "tags_literal ::= NK_INTEGER", - /* 460 */ "tags_literal ::= NK_INTEGER NK_PLUS duration_literal", - /* 461 */ "tags_literal ::= NK_INTEGER NK_MINUS duration_literal", - /* 462 */ "tags_literal ::= NK_PLUS NK_INTEGER", - /* 463 */ "tags_literal ::= NK_PLUS NK_INTEGER NK_PLUS duration_literal", - /* 464 */ "tags_literal ::= NK_PLUS NK_INTEGER NK_MINUS duration_literal", - /* 465 */ "tags_literal ::= NK_MINUS NK_INTEGER", - /* 466 */ "tags_literal ::= NK_MINUS NK_INTEGER NK_PLUS duration_literal", - /* 467 */ "tags_literal ::= NK_MINUS NK_INTEGER NK_MINUS duration_literal", - /* 468 */ "tags_literal ::= NK_FLOAT", - /* 469 */ "tags_literal ::= NK_PLUS NK_FLOAT", - /* 470 */ "tags_literal ::= NK_MINUS NK_FLOAT", - /* 471 */ "tags_literal ::= NK_BIN", - /* 472 */ "tags_literal ::= NK_BIN NK_PLUS duration_literal", - /* 473 */ "tags_literal ::= NK_BIN NK_MINUS duration_literal", - /* 474 */ "tags_literal ::= NK_PLUS NK_BIN", - /* 475 */ "tags_literal ::= NK_PLUS NK_BIN NK_PLUS duration_literal", - /* 476 */ "tags_literal ::= NK_PLUS NK_BIN NK_MINUS duration_literal", - /* 477 */ "tags_literal ::= NK_MINUS NK_BIN", - /* 478 */ "tags_literal ::= NK_MINUS NK_BIN NK_PLUS duration_literal", - /* 479 */ "tags_literal ::= NK_MINUS NK_BIN NK_MINUS duration_literal", - /* 480 */ "tags_literal ::= NK_HEX", - /* 481 */ "tags_literal ::= NK_HEX NK_PLUS duration_literal", - /* 482 */ "tags_literal ::= NK_HEX NK_MINUS duration_literal", - /* 483 */ "tags_literal ::= NK_PLUS NK_HEX", - /* 484 */ "tags_literal ::= NK_PLUS NK_HEX NK_PLUS duration_literal", - /* 485 */ "tags_literal ::= NK_PLUS NK_HEX NK_MINUS duration_literal", - /* 486 */ "tags_literal ::= NK_MINUS NK_HEX", - /* 487 */ "tags_literal ::= NK_MINUS NK_HEX NK_PLUS duration_literal", - /* 488 */ "tags_literal ::= NK_MINUS NK_HEX NK_MINUS duration_literal", - /* 489 */ "tags_literal ::= NK_STRING", - /* 490 */ "tags_literal ::= NK_STRING NK_PLUS duration_literal", - /* 491 */ "tags_literal ::= NK_STRING NK_MINUS duration_literal", - /* 492 */ "tags_literal ::= NK_BOOL", - /* 493 */ "tags_literal ::= NULL", - /* 494 */ "tags_literal ::= literal_func", - /* 495 */ "tags_literal ::= literal_func NK_PLUS duration_literal", - /* 496 */ "tags_literal ::= literal_func NK_MINUS duration_literal", - /* 497 */ "tags_literal_list ::= tags_literal", - /* 498 */ "tags_literal_list ::= tags_literal_list NK_COMMA tags_literal", - /* 499 */ "literal ::= NK_INTEGER", - /* 500 */ "literal ::= NK_FLOAT", - /* 501 */ "literal ::= NK_STRING", - /* 502 */ "literal ::= NK_BOOL", - /* 503 */ "literal ::= TIMESTAMP NK_STRING", - /* 504 */ "literal ::= duration_literal", - /* 505 */ "literal ::= NULL", - /* 506 */ "literal ::= NK_QUESTION", - /* 507 */ "duration_literal ::= NK_VARIABLE", - /* 508 */ "signed ::= NK_INTEGER", - /* 509 */ "signed ::= NK_PLUS NK_INTEGER", - /* 510 */ "signed ::= NK_MINUS NK_INTEGER", - /* 511 */ "signed ::= NK_FLOAT", - /* 512 */ "signed ::= NK_PLUS NK_FLOAT", - /* 513 */ "signed ::= NK_MINUS NK_FLOAT", - /* 514 */ "signed_literal ::= signed", - /* 515 */ "signed_literal ::= NK_STRING", - /* 516 */ "signed_literal ::= NK_BOOL", - /* 517 */ "signed_literal ::= TIMESTAMP NK_STRING", - /* 518 */ "signed_literal ::= duration_literal", - /* 519 */ "signed_literal ::= NULL", - /* 520 */ "signed_literal ::= literal_func", - /* 521 */ "signed_literal ::= NK_QUESTION", - /* 522 */ "literal_list ::= signed_literal", - /* 523 */ "literal_list ::= literal_list NK_COMMA signed_literal", - /* 524 */ "db_name ::= NK_ID", - /* 525 */ "table_name ::= NK_ID", - /* 526 */ "column_name ::= NK_ID", - /* 527 */ "function_name ::= NK_ID", - /* 528 */ "view_name ::= NK_ID", - /* 529 */ "table_alias ::= NK_ID", - /* 530 */ "column_alias ::= NK_ID", - /* 531 */ "column_alias ::= NK_ALIAS", - /* 532 */ "user_name ::= NK_ID", - /* 533 */ "topic_name ::= NK_ID", - /* 534 */ "stream_name ::= NK_ID", - /* 535 */ "cgroup_name ::= NK_ID", - /* 536 */ "index_name ::= NK_ID", - /* 537 */ "tsma_name ::= NK_ID", - /* 538 */ "expr_or_subquery ::= expression", - /* 539 */ "expression ::= literal", - /* 540 */ "expression ::= pseudo_column", - /* 541 */ "expression ::= column_reference", - /* 542 */ "expression ::= function_expression", - /* 543 */ "expression ::= case_when_expression", - /* 544 */ "expression ::= NK_LP expression NK_RP", - /* 545 */ "expression ::= NK_PLUS expr_or_subquery", - /* 546 */ "expression ::= NK_MINUS expr_or_subquery", - /* 547 */ "expression ::= expr_or_subquery NK_PLUS expr_or_subquery", - /* 548 */ "expression ::= expr_or_subquery NK_MINUS expr_or_subquery", - /* 549 */ "expression ::= expr_or_subquery NK_STAR expr_or_subquery", - /* 550 */ "expression ::= expr_or_subquery NK_SLASH expr_or_subquery", - /* 551 */ "expression ::= expr_or_subquery NK_REM expr_or_subquery", - /* 552 */ "expression ::= column_reference NK_ARROW NK_STRING", - /* 553 */ "expression ::= expr_or_subquery NK_BITAND expr_or_subquery", - /* 554 */ "expression ::= expr_or_subquery NK_BITOR expr_or_subquery", - /* 555 */ "expression_list ::= expr_or_subquery", - /* 556 */ "expression_list ::= expression_list NK_COMMA expr_or_subquery", - /* 557 */ "column_reference ::= column_name", - /* 558 */ "column_reference ::= table_name NK_DOT column_name", - /* 559 */ "column_reference ::= NK_ALIAS", - /* 560 */ "column_reference ::= table_name NK_DOT NK_ALIAS", - /* 561 */ "pseudo_column ::= ROWTS", - /* 562 */ "pseudo_column ::= TBNAME", - /* 563 */ "pseudo_column ::= table_name NK_DOT TBNAME", - /* 564 */ "pseudo_column ::= QSTART", - /* 565 */ "pseudo_column ::= QEND", - /* 566 */ "pseudo_column ::= QDURATION", - /* 567 */ "pseudo_column ::= WSTART", - /* 568 */ "pseudo_column ::= WEND", - /* 569 */ "pseudo_column ::= WDURATION", - /* 570 */ "pseudo_column ::= IROWTS", - /* 571 */ "pseudo_column ::= ISFILLED", - /* 572 */ "pseudo_column ::= QTAGS", - /* 573 */ "pseudo_column ::= FLOW", - /* 574 */ "pseudo_column ::= FHIGH", - /* 575 */ "pseudo_column ::= FROWTS", - /* 576 */ "function_expression ::= function_name NK_LP expression_list NK_RP", - /* 577 */ "function_expression ::= star_func NK_LP star_func_para_list NK_RP", - /* 578 */ "function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP", - /* 579 */ "function_expression ::= CAST NK_LP expr_or_subquery AS type_name_default_len NK_RP", - /* 580 */ "function_expression ::= POSITION NK_LP expr_or_subquery IN expr_or_subquery NK_RP", - /* 581 */ "function_expression ::= TRIM NK_LP expr_or_subquery NK_RP", - /* 582 */ "function_expression ::= TRIM NK_LP trim_specification_type FROM expr_or_subquery NK_RP", - /* 583 */ "function_expression ::= TRIM NK_LP expr_or_subquery FROM expr_or_subquery NK_RP", - /* 584 */ "function_expression ::= TRIM NK_LP trim_specification_type expr_or_subquery FROM expr_or_subquery NK_RP", - /* 585 */ "function_expression ::= substr_func NK_LP expression_list NK_RP", - /* 586 */ "function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery NK_RP", - /* 587 */ "function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery FOR expr_or_subquery NK_RP", - /* 588 */ "function_expression ::= REPLACE NK_LP expression_list NK_RP", - /* 589 */ "function_expression ::= literal_func", - /* 590 */ "function_expression ::= rand_func", - /* 591 */ "literal_func ::= noarg_func NK_LP NK_RP", - /* 592 */ "literal_func ::= NOW", - /* 593 */ "literal_func ::= TODAY", - /* 594 */ "rand_func ::= RAND NK_LP NK_RP", - /* 595 */ "rand_func ::= RAND NK_LP expression_list NK_RP", - /* 596 */ "substr_func ::= SUBSTR", - /* 597 */ "substr_func ::= SUBSTRING", - /* 598 */ "trim_specification_type ::= BOTH", - /* 599 */ "trim_specification_type ::= TRAILING", - /* 600 */ "trim_specification_type ::= LEADING", - /* 601 */ "noarg_func ::= NOW", - /* 602 */ "noarg_func ::= TODAY", - /* 603 */ "noarg_func ::= TIMEZONE", - /* 604 */ "noarg_func ::= DATABASE", - /* 605 */ "noarg_func ::= CLIENT_VERSION", - /* 606 */ "noarg_func ::= SERVER_VERSION", - /* 607 */ "noarg_func ::= SERVER_STATUS", - /* 608 */ "noarg_func ::= CURRENT_USER", - /* 609 */ "noarg_func ::= USER", - /* 610 */ "noarg_func ::= PI", - /* 611 */ "star_func ::= COUNT", - /* 612 */ "star_func ::= FIRST", - /* 613 */ "star_func ::= LAST", - /* 614 */ "star_func ::= LAST_ROW", - /* 615 */ "star_func_para_list ::= NK_STAR", - /* 616 */ "star_func_para_list ::= other_para_list", - /* 617 */ "other_para_list ::= star_func_para", - /* 618 */ "other_para_list ::= other_para_list NK_COMMA star_func_para", - /* 619 */ "star_func_para ::= expr_or_subquery", - /* 620 */ "star_func_para ::= table_name NK_DOT NK_STAR", - /* 621 */ "case_when_expression ::= CASE when_then_list case_when_else_opt END", - /* 622 */ "case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END", - /* 623 */ "when_then_list ::= when_then_expr", - /* 624 */ "when_then_list ::= when_then_list when_then_expr", - /* 625 */ "when_then_expr ::= WHEN common_expression THEN common_expression", - /* 626 */ "case_when_else_opt ::=", - /* 627 */ "case_when_else_opt ::= ELSE common_expression", - /* 628 */ "predicate ::= expr_or_subquery compare_op expr_or_subquery", - /* 629 */ "predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery", - /* 630 */ "predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery", - /* 631 */ "predicate ::= expr_or_subquery IS NULL", - /* 632 */ "predicate ::= expr_or_subquery IS NOT NULL", - /* 633 */ "predicate ::= expr_or_subquery in_op in_predicate_value", - /* 634 */ "compare_op ::= NK_LT", - /* 635 */ "compare_op ::= NK_GT", - /* 636 */ "compare_op ::= NK_LE", - /* 637 */ "compare_op ::= NK_GE", - /* 638 */ "compare_op ::= NK_NE", - /* 639 */ "compare_op ::= NK_EQ", - /* 640 */ "compare_op ::= LIKE", - /* 641 */ "compare_op ::= NOT LIKE", - /* 642 */ "compare_op ::= MATCH", - /* 643 */ "compare_op ::= NMATCH", - /* 644 */ "compare_op ::= CONTAINS", - /* 645 */ "in_op ::= IN", - /* 646 */ "in_op ::= NOT IN", - /* 647 */ "in_predicate_value ::= NK_LP literal_list NK_RP", - /* 648 */ "boolean_value_expression ::= boolean_primary", - /* 649 */ "boolean_value_expression ::= NOT boolean_primary", - /* 650 */ "boolean_value_expression ::= boolean_value_expression OR boolean_value_expression", - /* 651 */ "boolean_value_expression ::= boolean_value_expression AND boolean_value_expression", - /* 652 */ "boolean_primary ::= predicate", - /* 653 */ "boolean_primary ::= NK_LP boolean_value_expression NK_RP", - /* 654 */ "common_expression ::= expr_or_subquery", - /* 655 */ "common_expression ::= boolean_value_expression", - /* 656 */ "from_clause_opt ::=", - /* 657 */ "from_clause_opt ::= FROM table_reference_list", - /* 658 */ "table_reference_list ::= table_reference", - /* 659 */ "table_reference_list ::= table_reference_list NK_COMMA table_reference", - /* 660 */ "table_reference ::= table_primary", - /* 661 */ "table_reference ::= joined_table", - /* 662 */ "table_primary ::= table_name alias_opt", - /* 663 */ "table_primary ::= db_name NK_DOT table_name alias_opt", - /* 664 */ "table_primary ::= subquery alias_opt", - /* 665 */ "table_primary ::= parenthesized_joined_table", - /* 666 */ "alias_opt ::=", - /* 667 */ "alias_opt ::= table_alias", - /* 668 */ "alias_opt ::= AS table_alias", - /* 669 */ "parenthesized_joined_table ::= NK_LP joined_table NK_RP", - /* 670 */ "parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP", - /* 671 */ "joined_table ::= table_reference join_type join_subtype JOIN table_reference join_on_clause_opt window_offset_clause_opt jlimit_clause_opt", - /* 672 */ "join_type ::=", - /* 673 */ "join_type ::= INNER", - /* 674 */ "join_type ::= LEFT", - /* 675 */ "join_type ::= RIGHT", - /* 676 */ "join_type ::= FULL", - /* 677 */ "join_subtype ::=", - /* 678 */ "join_subtype ::= OUTER", - /* 679 */ "join_subtype ::= SEMI", - /* 680 */ "join_subtype ::= ANTI", - /* 681 */ "join_subtype ::= ASOF", - /* 682 */ "join_subtype ::= WINDOW", - /* 683 */ "join_on_clause_opt ::=", - /* 684 */ "join_on_clause_opt ::= ON search_condition", - /* 685 */ "window_offset_clause_opt ::=", - /* 686 */ "window_offset_clause_opt ::= WINDOW_OFFSET NK_LP window_offset_literal NK_COMMA window_offset_literal NK_RP", - /* 687 */ "window_offset_literal ::= NK_VARIABLE", - /* 688 */ "window_offset_literal ::= NK_MINUS NK_VARIABLE", - /* 689 */ "jlimit_clause_opt ::=", - /* 690 */ "jlimit_clause_opt ::= JLIMIT NK_INTEGER", - /* 691 */ "query_specification ::= SELECT hint_list set_quantifier_opt tag_mode_opt select_list from_clause_opt where_clause_opt partition_by_clause_opt range_opt every_opt fill_opt twindow_clause_opt group_by_clause_opt having_clause_opt", - /* 692 */ "hint_list ::=", - /* 693 */ "hint_list ::= NK_HINT", - /* 694 */ "tag_mode_opt ::=", - /* 695 */ "tag_mode_opt ::= TAGS", - /* 696 */ "set_quantifier_opt ::=", - /* 697 */ "set_quantifier_opt ::= DISTINCT", - /* 698 */ "set_quantifier_opt ::= ALL", - /* 699 */ "select_list ::= select_item", - /* 700 */ "select_list ::= select_list NK_COMMA select_item", - /* 701 */ "select_item ::= NK_STAR", - /* 702 */ "select_item ::= common_expression", - /* 703 */ "select_item ::= common_expression column_alias", - /* 704 */ "select_item ::= common_expression AS column_alias", - /* 705 */ "select_item ::= table_name NK_DOT NK_STAR", - /* 706 */ "where_clause_opt ::=", - /* 707 */ "where_clause_opt ::= WHERE search_condition", - /* 708 */ "partition_by_clause_opt ::=", - /* 709 */ "partition_by_clause_opt ::= PARTITION BY partition_list", - /* 710 */ "partition_list ::= partition_item", - /* 711 */ "partition_list ::= partition_list NK_COMMA partition_item", - /* 712 */ "partition_item ::= expr_or_subquery", - /* 713 */ "partition_item ::= expr_or_subquery column_alias", - /* 714 */ "partition_item ::= expr_or_subquery AS column_alias", - /* 715 */ "twindow_clause_opt ::=", - /* 716 */ "twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA interval_sliding_duration_literal NK_RP", - /* 717 */ "twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP", - /* 718 */ "twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_RP sliding_opt fill_opt", - /* 719 */ "twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_COMMA interval_sliding_duration_literal NK_RP sliding_opt fill_opt", - /* 720 */ "twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition", - /* 721 */ "twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_RP", - /* 722 */ "twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP", - /* 723 */ "twindow_clause_opt ::= ANOMALY_WINDOW NK_LP expr_or_subquery NK_RP", - /* 724 */ "twindow_clause_opt ::= ANOMALY_WINDOW NK_LP expr_or_subquery NK_COMMA NK_STRING NK_RP", - /* 725 */ "sliding_opt ::=", - /* 726 */ "sliding_opt ::= SLIDING NK_LP interval_sliding_duration_literal NK_RP", - /* 727 */ "interval_sliding_duration_literal ::= NK_VARIABLE", - /* 728 */ "interval_sliding_duration_literal ::= NK_STRING", - /* 729 */ "interval_sliding_duration_literal ::= NK_INTEGER", - /* 730 */ "fill_opt ::=", - /* 731 */ "fill_opt ::= FILL NK_LP fill_mode NK_RP", - /* 732 */ "fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP", - /* 733 */ "fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP", - /* 734 */ "fill_mode ::= NONE", - /* 735 */ "fill_mode ::= PREV", - /* 736 */ "fill_mode ::= NULL", - /* 737 */ "fill_mode ::= NULL_F", - /* 738 */ "fill_mode ::= LINEAR", - /* 739 */ "fill_mode ::= NEXT", - /* 740 */ "group_by_clause_opt ::=", - /* 741 */ "group_by_clause_opt ::= GROUP BY group_by_list", - /* 742 */ "group_by_list ::= expr_or_subquery", - /* 743 */ "group_by_list ::= group_by_list NK_COMMA expr_or_subquery", - /* 744 */ "having_clause_opt ::=", - /* 745 */ "having_clause_opt ::= HAVING search_condition", - /* 746 */ "range_opt ::=", - /* 747 */ "range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP", - /* 748 */ "range_opt ::= RANGE NK_LP expr_or_subquery NK_RP", - /* 749 */ "every_opt ::=", - /* 750 */ "every_opt ::= EVERY NK_LP duration_literal NK_RP", - /* 751 */ "query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt", - /* 752 */ "query_simple ::= query_specification", - /* 753 */ "query_simple ::= union_query_expression", - /* 754 */ "union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery", - /* 755 */ "union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery", - /* 756 */ "query_simple_or_subquery ::= query_simple", - /* 757 */ "query_simple_or_subquery ::= subquery", - /* 758 */ "query_or_subquery ::= query_expression", - /* 759 */ "query_or_subquery ::= subquery", - /* 760 */ "order_by_clause_opt ::=", - /* 761 */ "order_by_clause_opt ::= ORDER BY sort_specification_list", - /* 762 */ "slimit_clause_opt ::=", - /* 763 */ "slimit_clause_opt ::= SLIMIT NK_INTEGER", - /* 764 */ "slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER", - /* 765 */ "slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER", - /* 766 */ "limit_clause_opt ::=", - /* 767 */ "limit_clause_opt ::= LIMIT NK_INTEGER", - /* 768 */ "limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER", - /* 769 */ "limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER", - /* 770 */ "subquery ::= NK_LP query_expression NK_RP", - /* 771 */ "subquery ::= NK_LP subquery NK_RP", - /* 772 */ "search_condition ::= common_expression", - /* 773 */ "sort_specification_list ::= sort_specification", - /* 774 */ "sort_specification_list ::= sort_specification_list NK_COMMA sort_specification", - /* 775 */ "sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt", - /* 776 */ "ordering_specification_opt ::=", - /* 777 */ "ordering_specification_opt ::= ASC", - /* 778 */ "ordering_specification_opt ::= DESC", - /* 779 */ "null_ordering_opt ::=", - /* 780 */ "null_ordering_opt ::= NULLS FIRST", - /* 781 */ "null_ordering_opt ::= NULLS LAST", - /* 782 */ "column_options ::=", - /* 783 */ "column_options ::= column_options PRIMARY KEY", - /* 784 */ "column_options ::= column_options NK_ID NK_STRING", + /* 430 */ "stream_options ::= stream_options TRIGGER FORCE_WINDOW_CLOSE", + /* 431 */ "stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal", + /* 432 */ "stream_options ::= stream_options WATERMARK duration_literal", + /* 433 */ "stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER", + /* 434 */ "stream_options ::= stream_options FILL_HISTORY NK_INTEGER", + /* 435 */ "stream_options ::= stream_options DELETE_MARK duration_literal", + /* 436 */ "stream_options ::= stream_options IGNORE UPDATE NK_INTEGER", + /* 437 */ "subtable_opt ::=", + /* 438 */ "subtable_opt ::= SUBTABLE NK_LP expression NK_RP", + /* 439 */ "ignore_opt ::=", + /* 440 */ "ignore_opt ::= IGNORE UNTREATED", + /* 441 */ "cmd ::= KILL CONNECTION NK_INTEGER", + /* 442 */ "cmd ::= KILL QUERY NK_STRING", + /* 443 */ "cmd ::= KILL TRANSACTION NK_INTEGER", + /* 444 */ "cmd ::= KILL COMPACT NK_INTEGER", + /* 445 */ "cmd ::= BALANCE VGROUP", + /* 446 */ "cmd ::= BALANCE VGROUP LEADER on_vgroup_id", + /* 447 */ "cmd ::= BALANCE VGROUP LEADER DATABASE db_name", + /* 448 */ "cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER", + /* 449 */ "cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list", + /* 450 */ "cmd ::= SPLIT VGROUP NK_INTEGER", + /* 451 */ "on_vgroup_id ::=", + /* 452 */ "on_vgroup_id ::= ON NK_INTEGER", + /* 453 */ "dnode_list ::= DNODE NK_INTEGER", + /* 454 */ "dnode_list ::= dnode_list DNODE NK_INTEGER", + /* 455 */ "cmd ::= DELETE FROM full_table_name where_clause_opt", + /* 456 */ "cmd ::= query_or_subquery", + /* 457 */ "cmd ::= insert_query", + /* 458 */ "insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery", + /* 459 */ "insert_query ::= INSERT INTO full_table_name query_or_subquery", + /* 460 */ "tags_literal ::= NK_INTEGER", + /* 461 */ "tags_literal ::= NK_INTEGER NK_PLUS duration_literal", + /* 462 */ "tags_literal ::= NK_INTEGER NK_MINUS duration_literal", + /* 463 */ "tags_literal ::= NK_PLUS NK_INTEGER", + /* 464 */ "tags_literal ::= NK_PLUS NK_INTEGER NK_PLUS duration_literal", + /* 465 */ "tags_literal ::= NK_PLUS NK_INTEGER NK_MINUS duration_literal", + /* 466 */ "tags_literal ::= NK_MINUS NK_INTEGER", + /* 467 */ "tags_literal ::= NK_MINUS NK_INTEGER NK_PLUS duration_literal", + /* 468 */ "tags_literal ::= NK_MINUS NK_INTEGER NK_MINUS duration_literal", + /* 469 */ "tags_literal ::= NK_FLOAT", + /* 470 */ "tags_literal ::= NK_PLUS NK_FLOAT", + /* 471 */ "tags_literal ::= NK_MINUS NK_FLOAT", + /* 472 */ "tags_literal ::= NK_BIN", + /* 473 */ "tags_literal ::= NK_BIN NK_PLUS duration_literal", + /* 474 */ "tags_literal ::= NK_BIN NK_MINUS duration_literal", + /* 475 */ "tags_literal ::= NK_PLUS NK_BIN", + /* 476 */ "tags_literal ::= NK_PLUS NK_BIN NK_PLUS duration_literal", + /* 477 */ "tags_literal ::= NK_PLUS NK_BIN NK_MINUS duration_literal", + /* 478 */ "tags_literal ::= NK_MINUS NK_BIN", + /* 479 */ "tags_literal ::= NK_MINUS NK_BIN NK_PLUS duration_literal", + /* 480 */ "tags_literal ::= NK_MINUS NK_BIN NK_MINUS duration_literal", + /* 481 */ "tags_literal ::= NK_HEX", + /* 482 */ "tags_literal ::= NK_HEX NK_PLUS duration_literal", + /* 483 */ "tags_literal ::= NK_HEX NK_MINUS duration_literal", + /* 484 */ "tags_literal ::= NK_PLUS NK_HEX", + /* 485 */ "tags_literal ::= NK_PLUS NK_HEX NK_PLUS duration_literal", + /* 486 */ "tags_literal ::= NK_PLUS NK_HEX NK_MINUS duration_literal", + /* 487 */ "tags_literal ::= NK_MINUS NK_HEX", + /* 488 */ "tags_literal ::= NK_MINUS NK_HEX NK_PLUS duration_literal", + /* 489 */ "tags_literal ::= NK_MINUS NK_HEX NK_MINUS duration_literal", + /* 490 */ "tags_literal ::= NK_STRING", + /* 491 */ "tags_literal ::= NK_STRING NK_PLUS duration_literal", + /* 492 */ "tags_literal ::= NK_STRING NK_MINUS duration_literal", + /* 493 */ "tags_literal ::= NK_BOOL", + /* 494 */ "tags_literal ::= NULL", + /* 495 */ "tags_literal ::= literal_func", + /* 496 */ "tags_literal ::= literal_func NK_PLUS duration_literal", + /* 497 */ "tags_literal ::= literal_func NK_MINUS duration_literal", + /* 498 */ "tags_literal_list ::= tags_literal", + /* 499 */ "tags_literal_list ::= tags_literal_list NK_COMMA tags_literal", + /* 500 */ "literal ::= NK_INTEGER", + /* 501 */ "literal ::= NK_FLOAT", + /* 502 */ "literal ::= NK_STRING", + /* 503 */ "literal ::= NK_BOOL", + /* 504 */ "literal ::= TIMESTAMP NK_STRING", + /* 505 */ "literal ::= duration_literal", + /* 506 */ "literal ::= NULL", + /* 507 */ "literal ::= NK_QUESTION", + /* 508 */ "duration_literal ::= NK_VARIABLE", + /* 509 */ "signed ::= NK_INTEGER", + /* 510 */ "signed ::= NK_PLUS NK_INTEGER", + /* 511 */ "signed ::= NK_MINUS NK_INTEGER", + /* 512 */ "signed ::= NK_FLOAT", + /* 513 */ "signed ::= NK_PLUS NK_FLOAT", + /* 514 */ "signed ::= NK_MINUS NK_FLOAT", + /* 515 */ "signed_literal ::= signed", + /* 516 */ "signed_literal ::= NK_STRING", + /* 517 */ "signed_literal ::= NK_BOOL", + /* 518 */ "signed_literal ::= TIMESTAMP NK_STRING", + /* 519 */ "signed_literal ::= duration_literal", + /* 520 */ "signed_literal ::= NULL", + /* 521 */ "signed_literal ::= literal_func", + /* 522 */ "signed_literal ::= NK_QUESTION", + /* 523 */ "literal_list ::= signed_literal", + /* 524 */ "literal_list ::= literal_list NK_COMMA signed_literal", + /* 525 */ "db_name ::= NK_ID", + /* 526 */ "table_name ::= NK_ID", + /* 527 */ "column_name ::= NK_ID", + /* 528 */ "function_name ::= NK_ID", + /* 529 */ "view_name ::= NK_ID", + /* 530 */ "table_alias ::= NK_ID", + /* 531 */ "column_alias ::= NK_ID", + /* 532 */ "column_alias ::= NK_ALIAS", + /* 533 */ "user_name ::= NK_ID", + /* 534 */ "topic_name ::= NK_ID", + /* 535 */ "stream_name ::= NK_ID", + /* 536 */ "cgroup_name ::= NK_ID", + /* 537 */ "index_name ::= NK_ID", + /* 538 */ "tsma_name ::= NK_ID", + /* 539 */ "expr_or_subquery ::= expression", + /* 540 */ "expression ::= literal", + /* 541 */ "expression ::= pseudo_column", + /* 542 */ "expression ::= column_reference", + /* 543 */ "expression ::= function_expression", + /* 544 */ "expression ::= case_when_expression", + /* 545 */ "expression ::= NK_LP expression NK_RP", + /* 546 */ "expression ::= NK_PLUS expr_or_subquery", + /* 547 */ "expression ::= NK_MINUS expr_or_subquery", + /* 548 */ "expression ::= expr_or_subquery NK_PLUS expr_or_subquery", + /* 549 */ "expression ::= expr_or_subquery NK_MINUS expr_or_subquery", + /* 550 */ "expression ::= expr_or_subquery NK_STAR expr_or_subquery", + /* 551 */ "expression ::= expr_or_subquery NK_SLASH expr_or_subquery", + /* 552 */ "expression ::= expr_or_subquery NK_REM expr_or_subquery", + /* 553 */ "expression ::= column_reference NK_ARROW NK_STRING", + /* 554 */ "expression ::= expr_or_subquery NK_BITAND expr_or_subquery", + /* 555 */ "expression ::= expr_or_subquery NK_BITOR expr_or_subquery", + /* 556 */ "expression_list ::= expr_or_subquery", + /* 557 */ "expression_list ::= expression_list NK_COMMA expr_or_subquery", + /* 558 */ "column_reference ::= column_name", + /* 559 */ "column_reference ::= table_name NK_DOT column_name", + /* 560 */ "column_reference ::= NK_ALIAS", + /* 561 */ "column_reference ::= table_name NK_DOT NK_ALIAS", + /* 562 */ "pseudo_column ::= ROWTS", + /* 563 */ "pseudo_column ::= TBNAME", + /* 564 */ "pseudo_column ::= table_name NK_DOT TBNAME", + /* 565 */ "pseudo_column ::= QSTART", + /* 566 */ "pseudo_column ::= QEND", + /* 567 */ "pseudo_column ::= QDURATION", + /* 568 */ "pseudo_column ::= WSTART", + /* 569 */ "pseudo_column ::= WEND", + /* 570 */ "pseudo_column ::= WDURATION", + /* 571 */ "pseudo_column ::= IROWTS", + /* 572 */ "pseudo_column ::= ISFILLED", + /* 573 */ "pseudo_column ::= QTAGS", + /* 574 */ "pseudo_column ::= FLOW", + /* 575 */ "pseudo_column ::= FHIGH", + /* 576 */ "pseudo_column ::= FROWTS", + /* 577 */ "function_expression ::= function_name NK_LP expression_list NK_RP", + /* 578 */ "function_expression ::= star_func NK_LP star_func_para_list NK_RP", + /* 579 */ "function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP", + /* 580 */ "function_expression ::= CAST NK_LP expr_or_subquery AS type_name_default_len NK_RP", + /* 581 */ "function_expression ::= POSITION NK_LP expr_or_subquery IN expr_or_subquery NK_RP", + /* 582 */ "function_expression ::= TRIM NK_LP expr_or_subquery NK_RP", + /* 583 */ "function_expression ::= TRIM NK_LP trim_specification_type FROM expr_or_subquery NK_RP", + /* 584 */ "function_expression ::= TRIM NK_LP expr_or_subquery FROM expr_or_subquery NK_RP", + /* 585 */ "function_expression ::= TRIM NK_LP trim_specification_type expr_or_subquery FROM expr_or_subquery NK_RP", + /* 586 */ "function_expression ::= substr_func NK_LP expression_list NK_RP", + /* 587 */ "function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery NK_RP", + /* 588 */ "function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery FOR expr_or_subquery NK_RP", + /* 589 */ "function_expression ::= REPLACE NK_LP expression_list NK_RP", + /* 590 */ "function_expression ::= literal_func", + /* 591 */ "function_expression ::= rand_func", + /* 592 */ "literal_func ::= noarg_func NK_LP NK_RP", + /* 593 */ "literal_func ::= NOW", + /* 594 */ "literal_func ::= TODAY", + /* 595 */ "rand_func ::= RAND NK_LP NK_RP", + /* 596 */ "rand_func ::= RAND NK_LP expression_list NK_RP", + /* 597 */ "substr_func ::= SUBSTR", + /* 598 */ "substr_func ::= SUBSTRING", + /* 599 */ "trim_specification_type ::= BOTH", + /* 600 */ "trim_specification_type ::= TRAILING", + /* 601 */ "trim_specification_type ::= LEADING", + /* 602 */ "noarg_func ::= NOW", + /* 603 */ "noarg_func ::= TODAY", + /* 604 */ "noarg_func ::= TIMEZONE", + /* 605 */ "noarg_func ::= DATABASE", + /* 606 */ "noarg_func ::= CLIENT_VERSION", + /* 607 */ "noarg_func ::= SERVER_VERSION", + /* 608 */ "noarg_func ::= SERVER_STATUS", + /* 609 */ "noarg_func ::= CURRENT_USER", + /* 610 */ "noarg_func ::= USER", + /* 611 */ "noarg_func ::= PI", + /* 612 */ "star_func ::= COUNT", + /* 613 */ "star_func ::= FIRST", + /* 614 */ "star_func ::= LAST", + /* 615 */ "star_func ::= LAST_ROW", + /* 616 */ "star_func_para_list ::= NK_STAR", + /* 617 */ "star_func_para_list ::= other_para_list", + /* 618 */ "other_para_list ::= star_func_para", + /* 619 */ "other_para_list ::= other_para_list NK_COMMA star_func_para", + /* 620 */ "star_func_para ::= expr_or_subquery", + /* 621 */ "star_func_para ::= table_name NK_DOT NK_STAR", + /* 622 */ "case_when_expression ::= CASE when_then_list case_when_else_opt END", + /* 623 */ "case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END", + /* 624 */ "when_then_list ::= when_then_expr", + /* 625 */ "when_then_list ::= when_then_list when_then_expr", + /* 626 */ "when_then_expr ::= WHEN common_expression THEN common_expression", + /* 627 */ "case_when_else_opt ::=", + /* 628 */ "case_when_else_opt ::= ELSE common_expression", + /* 629 */ "predicate ::= expr_or_subquery compare_op expr_or_subquery", + /* 630 */ "predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery", + /* 631 */ "predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery", + /* 632 */ "predicate ::= expr_or_subquery IS NULL", + /* 633 */ "predicate ::= expr_or_subquery IS NOT NULL", + /* 634 */ "predicate ::= expr_or_subquery in_op in_predicate_value", + /* 635 */ "compare_op ::= NK_LT", + /* 636 */ "compare_op ::= NK_GT", + /* 637 */ "compare_op ::= NK_LE", + /* 638 */ "compare_op ::= NK_GE", + /* 639 */ "compare_op ::= NK_NE", + /* 640 */ "compare_op ::= NK_EQ", + /* 641 */ "compare_op ::= LIKE", + /* 642 */ "compare_op ::= NOT LIKE", + /* 643 */ "compare_op ::= MATCH", + /* 644 */ "compare_op ::= NMATCH", + /* 645 */ "compare_op ::= CONTAINS", + /* 646 */ "in_op ::= IN", + /* 647 */ "in_op ::= NOT IN", + /* 648 */ "in_predicate_value ::= NK_LP literal_list NK_RP", + /* 649 */ "boolean_value_expression ::= boolean_primary", + /* 650 */ "boolean_value_expression ::= NOT boolean_primary", + /* 651 */ "boolean_value_expression ::= boolean_value_expression OR boolean_value_expression", + /* 652 */ "boolean_value_expression ::= boolean_value_expression AND boolean_value_expression", + /* 653 */ "boolean_primary ::= predicate", + /* 654 */ "boolean_primary ::= NK_LP boolean_value_expression NK_RP", + /* 655 */ "common_expression ::= expr_or_subquery", + /* 656 */ "common_expression ::= boolean_value_expression", + /* 657 */ "from_clause_opt ::=", + /* 658 */ "from_clause_opt ::= FROM table_reference_list", + /* 659 */ "table_reference_list ::= table_reference", + /* 660 */ "table_reference_list ::= table_reference_list NK_COMMA table_reference", + /* 661 */ "table_reference ::= table_primary", + /* 662 */ "table_reference ::= joined_table", + /* 663 */ "table_primary ::= table_name alias_opt", + /* 664 */ "table_primary ::= db_name NK_DOT table_name alias_opt", + /* 665 */ "table_primary ::= subquery alias_opt", + /* 666 */ "table_primary ::= parenthesized_joined_table", + /* 667 */ "alias_opt ::=", + /* 668 */ "alias_opt ::= table_alias", + /* 669 */ "alias_opt ::= AS table_alias", + /* 670 */ "parenthesized_joined_table ::= NK_LP joined_table NK_RP", + /* 671 */ "parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP", + /* 672 */ "joined_table ::= table_reference join_type join_subtype JOIN table_reference join_on_clause_opt window_offset_clause_opt jlimit_clause_opt", + /* 673 */ "join_type ::=", + /* 674 */ "join_type ::= INNER", + /* 675 */ "join_type ::= LEFT", + /* 676 */ "join_type ::= RIGHT", + /* 677 */ "join_type ::= FULL", + /* 678 */ "join_subtype ::=", + /* 679 */ "join_subtype ::= OUTER", + /* 680 */ "join_subtype ::= SEMI", + /* 681 */ "join_subtype ::= ANTI", + /* 682 */ "join_subtype ::= ASOF", + /* 683 */ "join_subtype ::= WINDOW", + /* 684 */ "join_on_clause_opt ::=", + /* 685 */ "join_on_clause_opt ::= ON search_condition", + /* 686 */ "window_offset_clause_opt ::=", + /* 687 */ "window_offset_clause_opt ::= WINDOW_OFFSET NK_LP window_offset_literal NK_COMMA window_offset_literal NK_RP", + /* 688 */ "window_offset_literal ::= NK_VARIABLE", + /* 689 */ "window_offset_literal ::= NK_MINUS NK_VARIABLE", + /* 690 */ "jlimit_clause_opt ::=", + /* 691 */ "jlimit_clause_opt ::= JLIMIT NK_INTEGER", + /* 692 */ "query_specification ::= SELECT hint_list set_quantifier_opt tag_mode_opt select_list from_clause_opt where_clause_opt partition_by_clause_opt range_opt every_opt fill_opt twindow_clause_opt group_by_clause_opt having_clause_opt", + /* 693 */ "hint_list ::=", + /* 694 */ "hint_list ::= NK_HINT", + /* 695 */ "tag_mode_opt ::=", + /* 696 */ "tag_mode_opt ::= TAGS", + /* 697 */ "set_quantifier_opt ::=", + /* 698 */ "set_quantifier_opt ::= DISTINCT", + /* 699 */ "set_quantifier_opt ::= ALL", + /* 700 */ "select_list ::= select_item", + /* 701 */ "select_list ::= select_list NK_COMMA select_item", + /* 702 */ "select_item ::= NK_STAR", + /* 703 */ "select_item ::= common_expression", + /* 704 */ "select_item ::= common_expression column_alias", + /* 705 */ "select_item ::= common_expression AS column_alias", + /* 706 */ "select_item ::= table_name NK_DOT NK_STAR", + /* 707 */ "where_clause_opt ::=", + /* 708 */ "where_clause_opt ::= WHERE search_condition", + /* 709 */ "partition_by_clause_opt ::=", + /* 710 */ "partition_by_clause_opt ::= PARTITION BY partition_list", + /* 711 */ "partition_list ::= partition_item", + /* 712 */ "partition_list ::= partition_list NK_COMMA partition_item", + /* 713 */ "partition_item ::= expr_or_subquery", + /* 714 */ "partition_item ::= expr_or_subquery column_alias", + /* 715 */ "partition_item ::= expr_or_subquery AS column_alias", + /* 716 */ "twindow_clause_opt ::=", + /* 717 */ "twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA interval_sliding_duration_literal NK_RP", + /* 718 */ "twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP", + /* 719 */ "twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_RP sliding_opt fill_opt", + /* 720 */ "twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_COMMA interval_sliding_duration_literal NK_RP sliding_opt fill_opt", + /* 721 */ "twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition", + /* 722 */ "twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_RP", + /* 723 */ "twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP", + /* 724 */ "twindow_clause_opt ::= ANOMALY_WINDOW NK_LP expr_or_subquery NK_RP", + /* 725 */ "twindow_clause_opt ::= ANOMALY_WINDOW NK_LP expr_or_subquery NK_COMMA NK_STRING NK_RP", + /* 726 */ "sliding_opt ::=", + /* 727 */ "sliding_opt ::= SLIDING NK_LP interval_sliding_duration_literal NK_RP", + /* 728 */ "interval_sliding_duration_literal ::= NK_VARIABLE", + /* 729 */ "interval_sliding_duration_literal ::= NK_STRING", + /* 730 */ "interval_sliding_duration_literal ::= NK_INTEGER", + /* 731 */ "fill_opt ::=", + /* 732 */ "fill_opt ::= FILL NK_LP fill_mode NK_RP", + /* 733 */ "fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP", + /* 734 */ "fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP", + /* 735 */ "fill_mode ::= NONE", + /* 736 */ "fill_mode ::= PREV", + /* 737 */ "fill_mode ::= NULL", + /* 738 */ "fill_mode ::= NULL_F", + /* 739 */ "fill_mode ::= LINEAR", + /* 740 */ "fill_mode ::= NEXT", + /* 741 */ "group_by_clause_opt ::=", + /* 742 */ "group_by_clause_opt ::= GROUP BY group_by_list", + /* 743 */ "group_by_list ::= expr_or_subquery", + /* 744 */ "group_by_list ::= group_by_list NK_COMMA expr_or_subquery", + /* 745 */ "having_clause_opt ::=", + /* 746 */ "having_clause_opt ::= HAVING search_condition", + /* 747 */ "range_opt ::=", + /* 748 */ "range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP", + /* 749 */ "range_opt ::= RANGE NK_LP expr_or_subquery NK_RP", + /* 750 */ "every_opt ::=", + /* 751 */ "every_opt ::= EVERY NK_LP duration_literal NK_RP", + /* 752 */ "query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt", + /* 753 */ "query_simple ::= query_specification", + /* 754 */ "query_simple ::= union_query_expression", + /* 755 */ "union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery", + /* 756 */ "union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery", + /* 757 */ "query_simple_or_subquery ::= query_simple", + /* 758 */ "query_simple_or_subquery ::= subquery", + /* 759 */ "query_or_subquery ::= query_expression", + /* 760 */ "query_or_subquery ::= subquery", + /* 761 */ "order_by_clause_opt ::=", + /* 762 */ "order_by_clause_opt ::= ORDER BY sort_specification_list", + /* 763 */ "slimit_clause_opt ::=", + /* 764 */ "slimit_clause_opt ::= SLIMIT NK_INTEGER", + /* 765 */ "slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER", + /* 766 */ "slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER", + /* 767 */ "limit_clause_opt ::=", + /* 768 */ "limit_clause_opt ::= LIMIT NK_INTEGER", + /* 769 */ "limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER", + /* 770 */ "limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER", + /* 771 */ "subquery ::= NK_LP query_expression NK_RP", + /* 772 */ "subquery ::= NK_LP subquery NK_RP", + /* 773 */ "search_condition ::= common_expression", + /* 774 */ "sort_specification_list ::= sort_specification", + /* 775 */ "sort_specification_list ::= sort_specification_list NK_COMMA sort_specification", + /* 776 */ "sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt", + /* 777 */ "ordering_specification_opt ::=", + /* 778 */ "ordering_specification_opt ::= ASC", + /* 779 */ "ordering_specification_opt ::= DESC", + /* 780 */ "null_ordering_opt ::=", + /* 781 */ "null_ordering_opt ::= NULLS FIRST", + /* 782 */ "null_ordering_opt ::= NULLS LAST", + /* 783 */ "column_options ::=", + /* 784 */ "column_options ::= column_options PRIMARY KEY", + /* 785 */ "column_options ::= column_options NK_ID NK_STRING", }; #endif /* NDEBUG */ @@ -3406,266 +3846,266 @@ static void yy_destructor( */ /********* Begin destructor definitions ***************************************/ /* Default NON-TERMINAL Destructor */ - case 389: /* cmd */ - case 392: /* literal */ - case 403: /* with_clause_opt */ - case 409: /* search_condition */ - case 414: /* db_options */ - case 416: /* alter_db_options */ - case 418: /* start_opt */ - case 419: /* end_opt */ - case 423: /* signed */ - case 425: /* retention */ - case 426: /* full_table_name */ - case 429: /* table_options */ - case 435: /* alter_table_clause */ - case 436: /* alter_table_options */ - case 439: /* column_options */ - case 440: /* tags_literal */ - case 441: /* create_subtable_clause */ - case 444: /* drop_table_clause */ - case 447: /* tag_def */ - case 448: /* column_def */ - case 453: /* duration_literal */ - case 454: /* rollup_func_name */ - case 456: /* col_name */ - case 459: /* like_pattern_opt */ - case 460: /* db_name_cond_opt */ - case 461: /* table_name_cond */ - case 462: /* from_db_opt */ - case 464: /* tag_item */ - case 468: /* full_tsma_name */ - case 470: /* index_options */ - case 471: /* full_index_name */ - case 473: /* sliding_opt */ - case 474: /* sma_stream_opt */ - case 475: /* func */ - case 479: /* query_or_subquery */ - case 480: /* where_clause_opt */ - case 483: /* explain_options */ - case 484: /* insert_query */ - case 489: /* full_view_name */ - case 492: /* stream_options */ - case 495: /* subtable_opt */ - case 498: /* column_stream_def */ - case 499: /* stream_col_options */ - case 500: /* expression */ - case 503: /* literal_func */ - case 504: /* signed_literal */ - case 507: /* expr_or_subquery */ - case 508: /* pseudo_column */ - case 509: /* column_reference */ - case 510: /* function_expression */ - case 511: /* case_when_expression */ - case 516: /* rand_func */ - case 519: /* star_func_para */ - case 521: /* case_when_else_opt */ - case 522: /* common_expression */ - case 523: /* when_then_expr */ - case 524: /* predicate */ - case 527: /* in_predicate_value */ - case 528: /* boolean_value_expression */ - case 529: /* boolean_primary */ - case 530: /* from_clause_opt */ - case 531: /* table_reference_list */ - case 532: /* table_reference */ - case 533: /* table_primary */ - case 534: /* joined_table */ - case 536: /* subquery */ - case 537: /* parenthesized_joined_table */ - case 540: /* join_on_clause_opt */ - case 541: /* window_offset_clause_opt */ - case 542: /* jlimit_clause_opt */ - case 543: /* window_offset_literal */ - case 544: /* query_specification */ - case 550: /* range_opt */ - case 551: /* every_opt */ - case 552: /* fill_opt */ - case 553: /* twindow_clause_opt */ - case 555: /* having_clause_opt */ - case 556: /* select_item */ - case 558: /* partition_item */ - case 559: /* interval_sliding_duration_literal */ - case 562: /* query_expression */ - case 563: /* query_simple */ - case 565: /* slimit_clause_opt */ - case 566: /* limit_clause_opt */ - case 567: /* union_query_expression */ - case 568: /* query_simple_or_subquery */ - case 570: /* sort_specification */ + case 390: /* cmd */ + case 393: /* literal */ + case 404: /* with_clause_opt */ + case 410: /* search_condition */ + case 415: /* db_options */ + case 417: /* alter_db_options */ + case 419: /* start_opt */ + case 420: /* end_opt */ + case 424: /* signed */ + case 426: /* retention */ + case 427: /* full_table_name */ + case 430: /* table_options */ + case 436: /* alter_table_clause */ + case 437: /* alter_table_options */ + case 440: /* column_options */ + case 441: /* tags_literal */ + case 442: /* create_subtable_clause */ + case 445: /* drop_table_clause */ + case 448: /* tag_def */ + case 449: /* column_def */ + case 454: /* duration_literal */ + case 455: /* rollup_func_name */ + case 457: /* col_name */ + case 460: /* like_pattern_opt */ + case 461: /* db_name_cond_opt */ + case 462: /* table_name_cond */ + case 463: /* from_db_opt */ + case 465: /* tag_item */ + case 469: /* full_tsma_name */ + case 471: /* index_options */ + case 472: /* full_index_name */ + case 474: /* sliding_opt */ + case 475: /* sma_stream_opt */ + case 476: /* func */ + case 480: /* query_or_subquery */ + case 481: /* where_clause_opt */ + case 484: /* explain_options */ + case 485: /* insert_query */ + case 490: /* full_view_name */ + case 493: /* stream_options */ + case 496: /* subtable_opt */ + case 499: /* column_stream_def */ + case 500: /* stream_col_options */ + case 501: /* expression */ + case 504: /* literal_func */ + case 505: /* signed_literal */ + case 508: /* expr_or_subquery */ + case 509: /* pseudo_column */ + case 510: /* column_reference */ + case 511: /* function_expression */ + case 512: /* case_when_expression */ + case 517: /* rand_func */ + case 520: /* star_func_para */ + case 522: /* case_when_else_opt */ + case 523: /* common_expression */ + case 524: /* when_then_expr */ + case 525: /* predicate */ + case 528: /* in_predicate_value */ + case 529: /* boolean_value_expression */ + case 530: /* boolean_primary */ + case 531: /* from_clause_opt */ + case 532: /* table_reference_list */ + case 533: /* table_reference */ + case 534: /* table_primary */ + case 535: /* joined_table */ + case 537: /* subquery */ + case 538: /* parenthesized_joined_table */ + case 541: /* join_on_clause_opt */ + case 542: /* window_offset_clause_opt */ + case 543: /* jlimit_clause_opt */ + case 544: /* window_offset_literal */ + case 545: /* query_specification */ + case 551: /* range_opt */ + case 552: /* every_opt */ + case 553: /* fill_opt */ + case 554: /* twindow_clause_opt */ + case 556: /* having_clause_opt */ + case 557: /* select_item */ + case 559: /* partition_item */ + case 560: /* interval_sliding_duration_literal */ + case 563: /* query_expression */ + case 564: /* query_simple */ + case 566: /* slimit_clause_opt */ + case 567: /* limit_clause_opt */ + case 568: /* union_query_expression */ + case 569: /* query_simple_or_subquery */ + case 571: /* sort_specification */ { - nodesDestroyNode((yypminor->yy974)); + nodesDestroyNode((yypminor->yy980)); } break; - case 390: /* account_options */ - case 391: /* alter_account_options */ - case 393: /* alter_account_option */ - case 417: /* speed_opt */ - case 478: /* with_meta */ - case 487: /* bufsize_opt */ + case 391: /* account_options */ + case 392: /* alter_account_options */ + case 394: /* alter_account_option */ + case 418: /* speed_opt */ + case 479: /* with_meta */ + case 488: /* bufsize_opt */ { } break; - case 394: /* ip_range_list */ - case 395: /* white_list */ - case 396: /* white_list_opt */ - case 420: /* integer_list */ - case 421: /* variable_list */ - case 422: /* retention_list */ - case 427: /* column_def_list */ - case 428: /* tags_def_opt */ - case 430: /* multi_create_clause */ - case 431: /* tag_list_opt */ - case 432: /* tags_def */ - case 434: /* multi_drop_clause */ - case 442: /* specific_cols_opt */ - case 443: /* tags_literal_list */ - case 445: /* col_name_list */ - case 446: /* tag_def_list */ - case 450: /* duration_list */ - case 451: /* rollup_func_list */ - case 469: /* func_list */ - case 477: /* expression_list */ - case 493: /* col_list_opt */ - case 494: /* tag_def_or_ref_opt */ - case 497: /* column_stream_def_list */ - case 502: /* dnode_list */ - case 505: /* literal_list */ - case 513: /* star_func_para_list */ - case 518: /* other_para_list */ - case 520: /* when_then_list */ - case 545: /* hint_list */ - case 548: /* select_list */ - case 549: /* partition_by_clause_opt */ - case 554: /* group_by_clause_opt */ - case 557: /* partition_list */ - case 561: /* group_by_list */ - case 564: /* order_by_clause_opt */ - case 569: /* sort_specification_list */ + case 395: /* ip_range_list */ + case 396: /* white_list */ + case 397: /* white_list_opt */ + case 421: /* integer_list */ + case 422: /* variable_list */ + case 423: /* retention_list */ + case 428: /* column_def_list */ + case 429: /* tags_def_opt */ + case 431: /* multi_create_clause */ + case 432: /* tag_list_opt */ + case 433: /* tags_def */ + case 435: /* multi_drop_clause */ + case 443: /* specific_cols_opt */ + case 444: /* tags_literal_list */ + case 446: /* col_name_list */ + case 447: /* tag_def_list */ + case 451: /* duration_list */ + case 452: /* rollup_func_list */ + case 470: /* func_list */ + case 478: /* expression_list */ + case 494: /* col_list_opt */ + case 495: /* tag_def_or_ref_opt */ + case 498: /* column_stream_def_list */ + case 503: /* dnode_list */ + case 506: /* literal_list */ + case 514: /* star_func_para_list */ + case 519: /* other_para_list */ + case 521: /* when_then_list */ + case 546: /* hint_list */ + case 549: /* select_list */ + case 550: /* partition_by_clause_opt */ + case 555: /* group_by_clause_opt */ + case 558: /* partition_list */ + case 562: /* group_by_list */ + case 565: /* order_by_clause_opt */ + case 570: /* sort_specification_list */ { - nodesDestroyList((yypminor->yy946)); + nodesDestroyList((yypminor->yy628)); } break; - case 397: /* is_import_opt */ - case 398: /* is_createdb_opt */ - case 400: /* sysinfo_opt */ + case 398: /* is_import_opt */ + case 399: /* is_createdb_opt */ + case 401: /* sysinfo_opt */ { } break; - case 399: /* user_name */ - case 406: /* db_name */ - case 407: /* table_name */ - case 408: /* topic_name */ - case 410: /* dnode_endpoint */ - case 437: /* column_name */ - case 455: /* function_name */ - case 465: /* column_alias */ - case 466: /* tsma_name */ - case 472: /* index_name */ - case 476: /* sma_func_name */ - case 481: /* cgroup_name */ - case 488: /* language_opt */ - case 490: /* view_name */ - case 491: /* stream_name */ - case 501: /* on_vgroup_id */ - case 506: /* table_alias */ - case 512: /* star_func */ - case 515: /* substr_func */ - case 517: /* noarg_func */ - case 535: /* alias_opt */ + case 400: /* user_name */ + case 407: /* db_name */ + case 408: /* table_name */ + case 409: /* topic_name */ + case 411: /* dnode_endpoint */ + case 438: /* column_name */ + case 456: /* function_name */ + case 466: /* column_alias */ + case 467: /* tsma_name */ + case 473: /* index_name */ + case 477: /* sma_func_name */ + case 482: /* cgroup_name */ + case 489: /* language_opt */ + case 491: /* view_name */ + case 492: /* stream_name */ + case 502: /* on_vgroup_id */ + case 507: /* table_alias */ + case 513: /* star_func */ + case 516: /* substr_func */ + case 518: /* noarg_func */ + case 536: /* alias_opt */ { } break; - case 401: /* privileges */ - case 404: /* priv_type_list */ - case 405: /* priv_type */ + case 402: /* privileges */ + case 405: /* priv_type_list */ + case 406: /* priv_type */ { } break; - case 402: /* priv_level */ + case 403: /* priv_level */ { } break; - case 411: /* force_opt */ - case 412: /* unsafe_opt */ - case 413: /* not_exists_opt */ - case 415: /* exists_opt */ - case 433: /* with_opt */ - case 482: /* analyze_opt */ - case 485: /* or_replace_opt */ - case 486: /* agg_func_opt */ - case 496: /* ignore_opt */ - case 546: /* set_quantifier_opt */ - case 547: /* tag_mode_opt */ + case 412: /* force_opt */ + case 413: /* unsafe_opt */ + case 414: /* not_exists_opt */ + case 416: /* exists_opt */ + case 434: /* with_opt */ + case 483: /* analyze_opt */ + case 486: /* or_replace_opt */ + case 487: /* agg_func_opt */ + case 497: /* ignore_opt */ + case 547: /* set_quantifier_opt */ + case 548: /* tag_mode_opt */ { } break; - case 424: /* alter_db_option */ - case 452: /* alter_table_option */ + case 425: /* alter_db_option */ + case 453: /* alter_table_option */ { } break; - case 438: /* type_name */ - case 449: /* type_name_default_len */ + case 439: /* type_name */ + case 450: /* type_name_default_len */ { } break; - case 457: /* db_kind_opt */ - case 463: /* table_kind */ + case 458: /* db_kind_opt */ + case 464: /* table_kind */ { } break; - case 458: /* table_kind_db_name_cond_opt */ + case 459: /* table_kind_db_name_cond_opt */ { } break; - case 467: /* tsma_func_list */ + case 468: /* tsma_func_list */ { - nodesDestroyNode((yypminor->yy974)); + nodesDestroyNode((yypminor->yy980)); } break; - case 514: /* trim_specification_type */ + case 515: /* trim_specification_type */ { } break; - case 525: /* compare_op */ - case 526: /* in_op */ + case 526: /* compare_op */ + case 527: /* in_op */ { } break; - case 538: /* join_type */ + case 539: /* join_type */ { } break; - case 539: /* join_subtype */ + case 540: /* join_subtype */ { } break; - case 560: /* fill_mode */ + case 561: /* fill_mode */ { } break; - case 571: /* ordering_specification_opt */ + case 572: /* ordering_specification_opt */ { } break; - case 572: /* null_ordering_opt */ + case 573: /* null_ordering_opt */ { } @@ -3834,7 +4274,7 @@ static YYACTIONTYPE yy_find_shift_action( #endif /* YYWILDCARD */ return yy_default[stateno]; }else{ - assert( i>=0 && i=0 && i<(int)(sizeof(yy_action)/sizeof(yy_action[0])) ); return yy_action[i]; } }while(1); @@ -3956,791 +4396,792 @@ static void yy_shift( /* For rule J, yyRuleInfoLhs[J] contains the symbol on the left-hand side ** of that rule */ static const YYCODETYPE yyRuleInfoLhs[] = { - 389, /* (0) cmd ::= CREATE ACCOUNT NK_ID PASS NK_STRING account_options */ - 389, /* (1) cmd ::= ALTER ACCOUNT NK_ID alter_account_options */ - 390, /* (2) account_options ::= */ - 390, /* (3) account_options ::= account_options PPS literal */ - 390, /* (4) account_options ::= account_options TSERIES literal */ - 390, /* (5) account_options ::= account_options STORAGE literal */ - 390, /* (6) account_options ::= account_options STREAMS literal */ - 390, /* (7) account_options ::= account_options QTIME literal */ - 390, /* (8) account_options ::= account_options DBS literal */ - 390, /* (9) account_options ::= account_options USERS literal */ - 390, /* (10) account_options ::= account_options CONNS literal */ - 390, /* (11) account_options ::= account_options STATE literal */ - 391, /* (12) alter_account_options ::= alter_account_option */ - 391, /* (13) alter_account_options ::= alter_account_options alter_account_option */ - 393, /* (14) alter_account_option ::= PASS literal */ - 393, /* (15) alter_account_option ::= PPS literal */ - 393, /* (16) alter_account_option ::= TSERIES literal */ - 393, /* (17) alter_account_option ::= STORAGE literal */ - 393, /* (18) alter_account_option ::= STREAMS literal */ - 393, /* (19) alter_account_option ::= QTIME literal */ - 393, /* (20) alter_account_option ::= DBS literal */ - 393, /* (21) alter_account_option ::= USERS literal */ - 393, /* (22) alter_account_option ::= CONNS literal */ - 393, /* (23) alter_account_option ::= STATE literal */ - 394, /* (24) ip_range_list ::= NK_STRING */ - 394, /* (25) ip_range_list ::= ip_range_list NK_COMMA NK_STRING */ - 395, /* (26) white_list ::= HOST ip_range_list */ - 396, /* (27) white_list_opt ::= */ - 396, /* (28) white_list_opt ::= white_list */ - 397, /* (29) is_import_opt ::= */ - 397, /* (30) is_import_opt ::= IS_IMPORT NK_INTEGER */ - 398, /* (31) is_createdb_opt ::= */ - 398, /* (32) is_createdb_opt ::= CREATEDB NK_INTEGER */ - 389, /* (33) cmd ::= CREATE USER user_name PASS NK_STRING sysinfo_opt is_createdb_opt is_import_opt white_list_opt */ - 389, /* (34) cmd ::= ALTER USER user_name PASS NK_STRING */ - 389, /* (35) cmd ::= ALTER USER user_name ENABLE NK_INTEGER */ - 389, /* (36) cmd ::= ALTER USER user_name SYSINFO NK_INTEGER */ - 389, /* (37) cmd ::= ALTER USER user_name CREATEDB NK_INTEGER */ - 389, /* (38) cmd ::= ALTER USER user_name ADD white_list */ - 389, /* (39) cmd ::= ALTER USER user_name DROP white_list */ - 389, /* (40) cmd ::= DROP USER user_name */ - 400, /* (41) sysinfo_opt ::= */ - 400, /* (42) sysinfo_opt ::= SYSINFO NK_INTEGER */ - 389, /* (43) cmd ::= GRANT privileges ON priv_level with_clause_opt TO user_name */ - 389, /* (44) cmd ::= REVOKE privileges ON priv_level with_clause_opt FROM user_name */ - 401, /* (45) privileges ::= ALL */ - 401, /* (46) privileges ::= priv_type_list */ - 401, /* (47) privileges ::= SUBSCRIBE */ - 404, /* (48) priv_type_list ::= priv_type */ - 404, /* (49) priv_type_list ::= priv_type_list NK_COMMA priv_type */ - 405, /* (50) priv_type ::= READ */ - 405, /* (51) priv_type ::= WRITE */ - 405, /* (52) priv_type ::= ALTER */ - 402, /* (53) priv_level ::= NK_STAR NK_DOT NK_STAR */ - 402, /* (54) priv_level ::= db_name NK_DOT NK_STAR */ - 402, /* (55) priv_level ::= db_name NK_DOT table_name */ - 402, /* (56) priv_level ::= topic_name */ - 403, /* (57) with_clause_opt ::= */ - 403, /* (58) with_clause_opt ::= WITH search_condition */ - 389, /* (59) cmd ::= CREATE ENCRYPT_KEY NK_STRING */ - 389, /* (60) cmd ::= CREATE ANODE NK_STRING */ - 389, /* (61) cmd ::= UPDATE ANODE NK_INTEGER */ - 389, /* (62) cmd ::= UPDATE ALL ANODES */ - 389, /* (63) cmd ::= DROP ANODE NK_INTEGER */ - 389, /* (64) cmd ::= CREATE DNODE dnode_endpoint */ - 389, /* (65) cmd ::= CREATE DNODE dnode_endpoint PORT NK_INTEGER */ - 389, /* (66) cmd ::= DROP DNODE NK_INTEGER force_opt */ - 389, /* (67) cmd ::= DROP DNODE dnode_endpoint force_opt */ - 389, /* (68) cmd ::= DROP DNODE NK_INTEGER unsafe_opt */ - 389, /* (69) cmd ::= DROP DNODE dnode_endpoint unsafe_opt */ - 389, /* (70) cmd ::= ALTER DNODE NK_INTEGER NK_STRING */ - 389, /* (71) cmd ::= ALTER DNODE NK_INTEGER NK_STRING NK_STRING */ - 389, /* (72) cmd ::= ALTER ALL DNODES NK_STRING */ - 389, /* (73) cmd ::= ALTER ALL DNODES NK_STRING NK_STRING */ - 389, /* (74) cmd ::= RESTORE DNODE NK_INTEGER */ - 410, /* (75) dnode_endpoint ::= NK_STRING */ - 410, /* (76) dnode_endpoint ::= NK_ID */ - 410, /* (77) dnode_endpoint ::= NK_IPTOKEN */ - 411, /* (78) force_opt ::= */ - 411, /* (79) force_opt ::= FORCE */ - 412, /* (80) unsafe_opt ::= UNSAFE */ - 389, /* (81) cmd ::= ALTER CLUSTER NK_STRING */ - 389, /* (82) cmd ::= ALTER CLUSTER NK_STRING NK_STRING */ - 389, /* (83) cmd ::= ALTER LOCAL NK_STRING */ - 389, /* (84) cmd ::= ALTER LOCAL NK_STRING NK_STRING */ - 389, /* (85) cmd ::= CREATE QNODE ON DNODE NK_INTEGER */ - 389, /* (86) cmd ::= DROP QNODE ON DNODE NK_INTEGER */ - 389, /* (87) cmd ::= RESTORE QNODE ON DNODE NK_INTEGER */ - 389, /* (88) cmd ::= CREATE BNODE ON DNODE NK_INTEGER */ - 389, /* (89) cmd ::= DROP BNODE ON DNODE NK_INTEGER */ - 389, /* (90) cmd ::= CREATE SNODE ON DNODE NK_INTEGER */ - 389, /* (91) cmd ::= DROP SNODE ON DNODE NK_INTEGER */ - 389, /* (92) cmd ::= CREATE MNODE ON DNODE NK_INTEGER */ - 389, /* (93) cmd ::= DROP MNODE ON DNODE NK_INTEGER */ - 389, /* (94) cmd ::= RESTORE MNODE ON DNODE NK_INTEGER */ - 389, /* (95) cmd ::= RESTORE VNODE ON DNODE NK_INTEGER */ - 389, /* (96) cmd ::= CREATE DATABASE not_exists_opt db_name db_options */ - 389, /* (97) cmd ::= DROP DATABASE exists_opt db_name */ - 389, /* (98) cmd ::= USE db_name */ - 389, /* (99) cmd ::= ALTER DATABASE db_name alter_db_options */ - 389, /* (100) cmd ::= FLUSH DATABASE db_name */ - 389, /* (101) cmd ::= TRIM DATABASE db_name speed_opt */ - 389, /* (102) cmd ::= S3MIGRATE DATABASE db_name */ - 389, /* (103) cmd ::= COMPACT DATABASE db_name start_opt end_opt */ - 413, /* (104) not_exists_opt ::= IF NOT EXISTS */ - 413, /* (105) not_exists_opt ::= */ - 415, /* (106) exists_opt ::= IF EXISTS */ - 415, /* (107) exists_opt ::= */ - 414, /* (108) db_options ::= */ - 414, /* (109) db_options ::= db_options BUFFER NK_INTEGER */ - 414, /* (110) db_options ::= db_options CACHEMODEL NK_STRING */ - 414, /* (111) db_options ::= db_options CACHESIZE NK_INTEGER */ - 414, /* (112) db_options ::= db_options COMP NK_INTEGER */ - 414, /* (113) db_options ::= db_options DURATION NK_INTEGER */ - 414, /* (114) db_options ::= db_options DURATION NK_VARIABLE */ - 414, /* (115) db_options ::= db_options MAXROWS NK_INTEGER */ - 414, /* (116) db_options ::= db_options MINROWS NK_INTEGER */ - 414, /* (117) db_options ::= db_options KEEP integer_list */ - 414, /* (118) db_options ::= db_options KEEP variable_list */ - 414, /* (119) db_options ::= db_options PAGES NK_INTEGER */ - 414, /* (120) db_options ::= db_options PAGESIZE NK_INTEGER */ - 414, /* (121) db_options ::= db_options TSDB_PAGESIZE NK_INTEGER */ - 414, /* (122) db_options ::= db_options PRECISION NK_STRING */ - 414, /* (123) db_options ::= db_options REPLICA NK_INTEGER */ - 414, /* (124) db_options ::= db_options VGROUPS NK_INTEGER */ - 414, /* (125) db_options ::= db_options SINGLE_STABLE NK_INTEGER */ - 414, /* (126) db_options ::= db_options RETENTIONS retention_list */ - 414, /* (127) db_options ::= db_options SCHEMALESS NK_INTEGER */ - 414, /* (128) db_options ::= db_options WAL_LEVEL NK_INTEGER */ - 414, /* (129) db_options ::= db_options WAL_FSYNC_PERIOD NK_INTEGER */ - 414, /* (130) db_options ::= db_options WAL_RETENTION_PERIOD NK_INTEGER */ - 414, /* (131) db_options ::= db_options WAL_RETENTION_PERIOD NK_MINUS NK_INTEGER */ - 414, /* (132) db_options ::= db_options WAL_RETENTION_SIZE NK_INTEGER */ - 414, /* (133) db_options ::= db_options WAL_RETENTION_SIZE NK_MINUS NK_INTEGER */ - 414, /* (134) db_options ::= db_options WAL_ROLL_PERIOD NK_INTEGER */ - 414, /* (135) db_options ::= db_options WAL_SEGMENT_SIZE NK_INTEGER */ - 414, /* (136) db_options ::= db_options STT_TRIGGER NK_INTEGER */ - 414, /* (137) db_options ::= db_options TABLE_PREFIX signed */ - 414, /* (138) db_options ::= db_options TABLE_SUFFIX signed */ - 414, /* (139) db_options ::= db_options S3_CHUNKPAGES NK_INTEGER */ - 414, /* (140) db_options ::= db_options S3_KEEPLOCAL NK_INTEGER */ - 414, /* (141) db_options ::= db_options S3_KEEPLOCAL NK_VARIABLE */ - 414, /* (142) db_options ::= db_options S3_COMPACT NK_INTEGER */ - 414, /* (143) db_options ::= db_options KEEP_TIME_OFFSET NK_INTEGER */ - 414, /* (144) db_options ::= db_options ENCRYPT_ALGORITHM NK_STRING */ - 414, /* (145) db_options ::= db_options DNODES NK_STRING */ - 416, /* (146) alter_db_options ::= alter_db_option */ - 416, /* (147) alter_db_options ::= alter_db_options alter_db_option */ - 424, /* (148) alter_db_option ::= BUFFER NK_INTEGER */ - 424, /* (149) alter_db_option ::= CACHEMODEL NK_STRING */ - 424, /* (150) alter_db_option ::= CACHESIZE NK_INTEGER */ - 424, /* (151) alter_db_option ::= WAL_FSYNC_PERIOD NK_INTEGER */ - 424, /* (152) alter_db_option ::= KEEP integer_list */ - 424, /* (153) alter_db_option ::= KEEP variable_list */ - 424, /* (154) alter_db_option ::= PAGES NK_INTEGER */ - 424, /* (155) alter_db_option ::= REPLICA NK_INTEGER */ - 424, /* (156) alter_db_option ::= WAL_LEVEL NK_INTEGER */ - 424, /* (157) alter_db_option ::= STT_TRIGGER NK_INTEGER */ - 424, /* (158) alter_db_option ::= MINROWS NK_INTEGER */ - 424, /* (159) alter_db_option ::= WAL_RETENTION_PERIOD NK_INTEGER */ - 424, /* (160) alter_db_option ::= WAL_RETENTION_PERIOD NK_MINUS NK_INTEGER */ - 424, /* (161) alter_db_option ::= WAL_RETENTION_SIZE NK_INTEGER */ - 424, /* (162) alter_db_option ::= WAL_RETENTION_SIZE NK_MINUS NK_INTEGER */ - 424, /* (163) alter_db_option ::= S3_KEEPLOCAL NK_INTEGER */ - 424, /* (164) alter_db_option ::= S3_KEEPLOCAL NK_VARIABLE */ - 424, /* (165) alter_db_option ::= S3_COMPACT NK_INTEGER */ - 424, /* (166) alter_db_option ::= KEEP_TIME_OFFSET NK_INTEGER */ - 424, /* (167) alter_db_option ::= ENCRYPT_ALGORITHM NK_STRING */ - 420, /* (168) integer_list ::= NK_INTEGER */ - 420, /* (169) integer_list ::= integer_list NK_COMMA NK_INTEGER */ - 421, /* (170) variable_list ::= NK_VARIABLE */ - 421, /* (171) variable_list ::= variable_list NK_COMMA NK_VARIABLE */ - 422, /* (172) retention_list ::= retention */ - 422, /* (173) retention_list ::= retention_list NK_COMMA retention */ - 425, /* (174) retention ::= NK_VARIABLE NK_COLON NK_VARIABLE */ - 425, /* (175) retention ::= NK_MINUS NK_COLON NK_VARIABLE */ - 417, /* (176) speed_opt ::= */ - 417, /* (177) speed_opt ::= BWLIMIT NK_INTEGER */ - 418, /* (178) start_opt ::= */ - 418, /* (179) start_opt ::= START WITH NK_INTEGER */ - 418, /* (180) start_opt ::= START WITH NK_STRING */ - 418, /* (181) start_opt ::= START WITH TIMESTAMP NK_STRING */ - 419, /* (182) end_opt ::= */ - 419, /* (183) end_opt ::= END WITH NK_INTEGER */ - 419, /* (184) end_opt ::= END WITH NK_STRING */ - 419, /* (185) end_opt ::= END WITH TIMESTAMP NK_STRING */ - 389, /* (186) cmd ::= CREATE TABLE not_exists_opt full_table_name NK_LP column_def_list NK_RP tags_def_opt table_options */ - 389, /* (187) cmd ::= CREATE TABLE multi_create_clause */ - 389, /* (188) cmd ::= CREATE TABLE not_exists_opt USING full_table_name NK_LP tag_list_opt NK_RP FILE NK_STRING */ - 389, /* (189) cmd ::= CREATE STABLE not_exists_opt full_table_name NK_LP column_def_list NK_RP tags_def table_options */ - 389, /* (190) cmd ::= DROP TABLE with_opt multi_drop_clause */ - 389, /* (191) cmd ::= DROP STABLE with_opt exists_opt full_table_name */ - 389, /* (192) cmd ::= ALTER TABLE alter_table_clause */ - 389, /* (193) cmd ::= ALTER STABLE alter_table_clause */ - 435, /* (194) alter_table_clause ::= full_table_name alter_table_options */ - 435, /* (195) alter_table_clause ::= full_table_name ADD COLUMN column_name type_name column_options */ - 435, /* (196) alter_table_clause ::= full_table_name DROP COLUMN column_name */ - 435, /* (197) alter_table_clause ::= full_table_name MODIFY COLUMN column_name type_name */ - 435, /* (198) alter_table_clause ::= full_table_name MODIFY COLUMN column_name column_options */ - 435, /* (199) alter_table_clause ::= full_table_name RENAME COLUMN column_name column_name */ - 435, /* (200) alter_table_clause ::= full_table_name ADD TAG column_name type_name */ - 435, /* (201) alter_table_clause ::= full_table_name DROP TAG column_name */ - 435, /* (202) alter_table_clause ::= full_table_name MODIFY TAG column_name type_name */ - 435, /* (203) alter_table_clause ::= full_table_name RENAME TAG column_name column_name */ - 435, /* (204) alter_table_clause ::= full_table_name SET TAG column_name NK_EQ tags_literal */ - 430, /* (205) multi_create_clause ::= create_subtable_clause */ - 430, /* (206) multi_create_clause ::= multi_create_clause create_subtable_clause */ - 441, /* (207) create_subtable_clause ::= not_exists_opt full_table_name USING full_table_name specific_cols_opt TAGS NK_LP tags_literal_list NK_RP table_options */ - 434, /* (208) multi_drop_clause ::= drop_table_clause */ - 434, /* (209) multi_drop_clause ::= multi_drop_clause NK_COMMA drop_table_clause */ - 444, /* (210) drop_table_clause ::= exists_opt full_table_name */ - 433, /* (211) with_opt ::= */ - 433, /* (212) with_opt ::= WITH */ - 442, /* (213) specific_cols_opt ::= */ - 442, /* (214) specific_cols_opt ::= NK_LP col_name_list NK_RP */ - 426, /* (215) full_table_name ::= table_name */ - 426, /* (216) full_table_name ::= db_name NK_DOT table_name */ - 446, /* (217) tag_def_list ::= tag_def */ - 446, /* (218) tag_def_list ::= tag_def_list NK_COMMA tag_def */ - 447, /* (219) tag_def ::= column_name type_name */ - 427, /* (220) column_def_list ::= column_def */ - 427, /* (221) column_def_list ::= column_def_list NK_COMMA column_def */ - 448, /* (222) column_def ::= column_name type_name column_options */ - 438, /* (223) type_name ::= BOOL */ - 438, /* (224) type_name ::= TINYINT */ - 438, /* (225) type_name ::= SMALLINT */ - 438, /* (226) type_name ::= INT */ - 438, /* (227) type_name ::= INTEGER */ - 438, /* (228) type_name ::= BIGINT */ - 438, /* (229) type_name ::= FLOAT */ - 438, /* (230) type_name ::= DOUBLE */ - 438, /* (231) type_name ::= BINARY NK_LP NK_INTEGER NK_RP */ - 438, /* (232) type_name ::= TIMESTAMP */ - 438, /* (233) type_name ::= NCHAR NK_LP NK_INTEGER NK_RP */ - 438, /* (234) type_name ::= TINYINT UNSIGNED */ - 438, /* (235) type_name ::= SMALLINT UNSIGNED */ - 438, /* (236) type_name ::= INT UNSIGNED */ - 438, /* (237) type_name ::= BIGINT UNSIGNED */ - 438, /* (238) type_name ::= JSON */ - 438, /* (239) type_name ::= VARCHAR NK_LP NK_INTEGER NK_RP */ - 438, /* (240) type_name ::= MEDIUMBLOB */ - 438, /* (241) type_name ::= BLOB */ - 438, /* (242) type_name ::= VARBINARY NK_LP NK_INTEGER NK_RP */ - 438, /* (243) type_name ::= GEOMETRY NK_LP NK_INTEGER NK_RP */ - 438, /* (244) type_name ::= DECIMAL */ - 438, /* (245) type_name ::= DECIMAL NK_LP NK_INTEGER NK_RP */ - 438, /* (246) type_name ::= DECIMAL NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP */ - 449, /* (247) type_name_default_len ::= BINARY */ - 449, /* (248) type_name_default_len ::= NCHAR */ - 449, /* (249) type_name_default_len ::= VARCHAR */ - 449, /* (250) type_name_default_len ::= VARBINARY */ - 428, /* (251) tags_def_opt ::= */ - 428, /* (252) tags_def_opt ::= tags_def */ - 432, /* (253) tags_def ::= TAGS NK_LP tag_def_list NK_RP */ - 429, /* (254) table_options ::= */ - 429, /* (255) table_options ::= table_options COMMENT NK_STRING */ - 429, /* (256) table_options ::= table_options MAX_DELAY duration_list */ - 429, /* (257) table_options ::= table_options WATERMARK duration_list */ - 429, /* (258) table_options ::= table_options ROLLUP NK_LP rollup_func_list NK_RP */ - 429, /* (259) table_options ::= table_options TTL NK_INTEGER */ - 429, /* (260) table_options ::= table_options SMA NK_LP col_name_list NK_RP */ - 429, /* (261) table_options ::= table_options DELETE_MARK duration_list */ - 436, /* (262) alter_table_options ::= alter_table_option */ - 436, /* (263) alter_table_options ::= alter_table_options alter_table_option */ - 452, /* (264) alter_table_option ::= COMMENT NK_STRING */ - 452, /* (265) alter_table_option ::= TTL NK_INTEGER */ - 450, /* (266) duration_list ::= duration_literal */ - 450, /* (267) duration_list ::= duration_list NK_COMMA duration_literal */ - 451, /* (268) rollup_func_list ::= rollup_func_name */ - 451, /* (269) rollup_func_list ::= rollup_func_list NK_COMMA rollup_func_name */ - 454, /* (270) rollup_func_name ::= function_name */ - 454, /* (271) rollup_func_name ::= FIRST */ - 454, /* (272) rollup_func_name ::= LAST */ - 445, /* (273) col_name_list ::= col_name */ - 445, /* (274) col_name_list ::= col_name_list NK_COMMA col_name */ - 456, /* (275) col_name ::= column_name */ - 389, /* (276) cmd ::= SHOW DNODES */ - 389, /* (277) cmd ::= SHOW USERS */ - 389, /* (278) cmd ::= SHOW USERS FULL */ - 389, /* (279) cmd ::= SHOW USER PRIVILEGES */ - 389, /* (280) cmd ::= SHOW db_kind_opt DATABASES */ - 389, /* (281) cmd ::= SHOW table_kind_db_name_cond_opt TABLES like_pattern_opt */ - 389, /* (282) cmd ::= SHOW db_name_cond_opt STABLES like_pattern_opt */ - 389, /* (283) cmd ::= SHOW db_name_cond_opt VGROUPS */ - 389, /* (284) cmd ::= SHOW MNODES */ - 389, /* (285) cmd ::= SHOW QNODES */ - 389, /* (286) cmd ::= SHOW ANODES */ - 389, /* (287) cmd ::= SHOW ANODES FULL */ - 389, /* (288) cmd ::= SHOW ARBGROUPS */ - 389, /* (289) cmd ::= SHOW FUNCTIONS */ - 389, /* (290) cmd ::= SHOW INDEXES FROM table_name_cond from_db_opt */ - 389, /* (291) cmd ::= SHOW INDEXES FROM db_name NK_DOT table_name */ - 389, /* (292) cmd ::= SHOW STREAMS */ - 389, /* (293) cmd ::= SHOW ACCOUNTS */ - 389, /* (294) cmd ::= SHOW APPS */ - 389, /* (295) cmd ::= SHOW CONNECTIONS */ - 389, /* (296) cmd ::= SHOW LICENCES */ - 389, /* (297) cmd ::= SHOW GRANTS */ - 389, /* (298) cmd ::= SHOW GRANTS FULL */ - 389, /* (299) cmd ::= SHOW GRANTS LOGS */ - 389, /* (300) cmd ::= SHOW CLUSTER MACHINES */ - 389, /* (301) cmd ::= SHOW CREATE DATABASE db_name */ - 389, /* (302) cmd ::= SHOW CREATE TABLE full_table_name */ - 389, /* (303) cmd ::= SHOW CREATE STABLE full_table_name */ - 389, /* (304) cmd ::= SHOW ENCRYPTIONS */ - 389, /* (305) cmd ::= SHOW QUERIES */ - 389, /* (306) cmd ::= SHOW SCORES */ - 389, /* (307) cmd ::= SHOW TOPICS */ - 389, /* (308) cmd ::= SHOW VARIABLES */ - 389, /* (309) cmd ::= SHOW CLUSTER VARIABLES */ - 389, /* (310) cmd ::= SHOW LOCAL VARIABLES */ - 389, /* (311) cmd ::= SHOW DNODE NK_INTEGER VARIABLES like_pattern_opt */ - 389, /* (312) cmd ::= SHOW BNODES */ - 389, /* (313) cmd ::= SHOW SNODES */ - 389, /* (314) cmd ::= SHOW CLUSTER */ - 389, /* (315) cmd ::= SHOW TRANSACTIONS */ - 389, /* (316) cmd ::= SHOW TABLE DISTRIBUTED full_table_name */ - 389, /* (317) cmd ::= SHOW CONSUMERS */ - 389, /* (318) cmd ::= SHOW SUBSCRIPTIONS */ - 389, /* (319) cmd ::= SHOW TAGS FROM table_name_cond from_db_opt */ - 389, /* (320) cmd ::= SHOW TAGS FROM db_name NK_DOT table_name */ - 389, /* (321) cmd ::= SHOW TABLE TAGS tag_list_opt FROM table_name_cond from_db_opt */ - 389, /* (322) cmd ::= SHOW TABLE TAGS tag_list_opt FROM db_name NK_DOT table_name */ - 389, /* (323) cmd ::= SHOW VNODES ON DNODE NK_INTEGER */ - 389, /* (324) cmd ::= SHOW VNODES */ - 389, /* (325) cmd ::= SHOW db_name_cond_opt ALIVE */ - 389, /* (326) cmd ::= SHOW CLUSTER ALIVE */ - 389, /* (327) cmd ::= SHOW db_name_cond_opt VIEWS like_pattern_opt */ - 389, /* (328) cmd ::= SHOW CREATE VIEW full_table_name */ - 389, /* (329) cmd ::= SHOW COMPACTS */ - 389, /* (330) cmd ::= SHOW COMPACT NK_INTEGER */ - 458, /* (331) table_kind_db_name_cond_opt ::= */ - 458, /* (332) table_kind_db_name_cond_opt ::= table_kind */ - 458, /* (333) table_kind_db_name_cond_opt ::= db_name NK_DOT */ - 458, /* (334) table_kind_db_name_cond_opt ::= table_kind db_name NK_DOT */ - 463, /* (335) table_kind ::= NORMAL */ - 463, /* (336) table_kind ::= CHILD */ - 460, /* (337) db_name_cond_opt ::= */ - 460, /* (338) db_name_cond_opt ::= db_name NK_DOT */ - 459, /* (339) like_pattern_opt ::= */ - 459, /* (340) like_pattern_opt ::= LIKE NK_STRING */ - 461, /* (341) table_name_cond ::= table_name */ - 462, /* (342) from_db_opt ::= */ - 462, /* (343) from_db_opt ::= FROM db_name */ - 431, /* (344) tag_list_opt ::= */ - 431, /* (345) tag_list_opt ::= tag_item */ - 431, /* (346) tag_list_opt ::= tag_list_opt NK_COMMA tag_item */ - 464, /* (347) tag_item ::= TBNAME */ - 464, /* (348) tag_item ::= QTAGS */ - 464, /* (349) tag_item ::= column_name */ - 464, /* (350) tag_item ::= column_name column_alias */ - 464, /* (351) tag_item ::= column_name AS column_alias */ - 457, /* (352) db_kind_opt ::= */ - 457, /* (353) db_kind_opt ::= USER */ - 457, /* (354) db_kind_opt ::= SYSTEM */ - 389, /* (355) cmd ::= CREATE TSMA not_exists_opt tsma_name ON full_table_name tsma_func_list INTERVAL NK_LP duration_literal NK_RP */ - 389, /* (356) cmd ::= CREATE RECURSIVE TSMA not_exists_opt tsma_name ON full_table_name INTERVAL NK_LP duration_literal NK_RP */ - 389, /* (357) cmd ::= DROP TSMA exists_opt full_tsma_name */ - 389, /* (358) cmd ::= SHOW db_name_cond_opt TSMAS */ - 468, /* (359) full_tsma_name ::= tsma_name */ - 468, /* (360) full_tsma_name ::= db_name NK_DOT tsma_name */ - 467, /* (361) tsma_func_list ::= FUNCTION NK_LP func_list NK_RP */ - 389, /* (362) cmd ::= CREATE SMA INDEX not_exists_opt col_name ON full_table_name index_options */ - 389, /* (363) cmd ::= CREATE INDEX not_exists_opt col_name ON full_table_name NK_LP col_name_list NK_RP */ - 389, /* (364) cmd ::= DROP INDEX exists_opt full_index_name */ - 471, /* (365) full_index_name ::= index_name */ - 471, /* (366) full_index_name ::= db_name NK_DOT index_name */ - 470, /* (367) index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_RP sliding_opt sma_stream_opt */ - 470, /* (368) 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 */ - 469, /* (369) func_list ::= func */ - 469, /* (370) func_list ::= func_list NK_COMMA func */ - 475, /* (371) func ::= sma_func_name NK_LP expression_list NK_RP */ - 476, /* (372) sma_func_name ::= function_name */ - 476, /* (373) sma_func_name ::= COUNT */ - 476, /* (374) sma_func_name ::= FIRST */ - 476, /* (375) sma_func_name ::= LAST */ - 476, /* (376) sma_func_name ::= LAST_ROW */ - 474, /* (377) sma_stream_opt ::= */ - 474, /* (378) sma_stream_opt ::= sma_stream_opt WATERMARK duration_literal */ - 474, /* (379) sma_stream_opt ::= sma_stream_opt MAX_DELAY duration_literal */ - 474, /* (380) sma_stream_opt ::= sma_stream_opt DELETE_MARK duration_literal */ - 478, /* (381) with_meta ::= AS */ - 478, /* (382) with_meta ::= WITH META AS */ - 478, /* (383) with_meta ::= ONLY META AS */ - 389, /* (384) cmd ::= CREATE TOPIC not_exists_opt topic_name AS query_or_subquery */ - 389, /* (385) cmd ::= CREATE TOPIC not_exists_opt topic_name with_meta DATABASE db_name */ - 389, /* (386) cmd ::= CREATE TOPIC not_exists_opt topic_name with_meta STABLE full_table_name where_clause_opt */ - 389, /* (387) cmd ::= DROP TOPIC exists_opt topic_name */ - 389, /* (388) cmd ::= DROP CONSUMER GROUP exists_opt cgroup_name ON topic_name */ - 389, /* (389) cmd ::= DESC full_table_name */ - 389, /* (390) cmd ::= DESCRIBE full_table_name */ - 389, /* (391) cmd ::= RESET QUERY CACHE */ - 389, /* (392) cmd ::= EXPLAIN analyze_opt explain_options query_or_subquery */ - 389, /* (393) cmd ::= EXPLAIN analyze_opt explain_options insert_query */ - 482, /* (394) analyze_opt ::= */ - 482, /* (395) analyze_opt ::= ANALYZE */ - 483, /* (396) explain_options ::= */ - 483, /* (397) explain_options ::= explain_options VERBOSE NK_BOOL */ - 483, /* (398) explain_options ::= explain_options RATIO NK_FLOAT */ - 389, /* (399) cmd ::= CREATE or_replace_opt agg_func_opt FUNCTION not_exists_opt function_name AS NK_STRING OUTPUTTYPE type_name bufsize_opt language_opt */ - 389, /* (400) cmd ::= DROP FUNCTION exists_opt function_name */ - 486, /* (401) agg_func_opt ::= */ - 486, /* (402) agg_func_opt ::= AGGREGATE */ - 487, /* (403) bufsize_opt ::= */ - 487, /* (404) bufsize_opt ::= BUFSIZE NK_INTEGER */ - 488, /* (405) language_opt ::= */ - 488, /* (406) language_opt ::= LANGUAGE NK_STRING */ - 485, /* (407) or_replace_opt ::= */ - 485, /* (408) or_replace_opt ::= OR REPLACE */ - 389, /* (409) cmd ::= CREATE or_replace_opt VIEW full_view_name AS query_or_subquery */ - 389, /* (410) cmd ::= DROP VIEW exists_opt full_view_name */ - 489, /* (411) full_view_name ::= view_name */ - 489, /* (412) full_view_name ::= db_name NK_DOT view_name */ - 389, /* (413) 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 */ - 389, /* (414) cmd ::= DROP STREAM exists_opt stream_name */ - 389, /* (415) cmd ::= PAUSE STREAM exists_opt stream_name */ - 389, /* (416) cmd ::= RESUME STREAM exists_opt ignore_opt stream_name */ - 493, /* (417) col_list_opt ::= */ - 493, /* (418) col_list_opt ::= NK_LP column_stream_def_list NK_RP */ - 497, /* (419) column_stream_def_list ::= column_stream_def */ - 497, /* (420) column_stream_def_list ::= column_stream_def_list NK_COMMA column_stream_def */ - 498, /* (421) column_stream_def ::= column_name stream_col_options */ - 499, /* (422) stream_col_options ::= */ - 499, /* (423) stream_col_options ::= stream_col_options PRIMARY KEY */ - 494, /* (424) tag_def_or_ref_opt ::= */ - 494, /* (425) tag_def_or_ref_opt ::= tags_def */ - 494, /* (426) tag_def_or_ref_opt ::= TAGS NK_LP column_stream_def_list NK_RP */ - 492, /* (427) stream_options ::= */ - 492, /* (428) stream_options ::= stream_options TRIGGER AT_ONCE */ - 492, /* (429) stream_options ::= stream_options TRIGGER WINDOW_CLOSE */ - 492, /* (430) stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal */ - 492, /* (431) stream_options ::= stream_options WATERMARK duration_literal */ - 492, /* (432) stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER */ - 492, /* (433) stream_options ::= stream_options FILL_HISTORY NK_INTEGER */ - 492, /* (434) stream_options ::= stream_options DELETE_MARK duration_literal */ - 492, /* (435) stream_options ::= stream_options IGNORE UPDATE NK_INTEGER */ - 495, /* (436) subtable_opt ::= */ - 495, /* (437) subtable_opt ::= SUBTABLE NK_LP expression NK_RP */ - 496, /* (438) ignore_opt ::= */ - 496, /* (439) ignore_opt ::= IGNORE UNTREATED */ - 389, /* (440) cmd ::= KILL CONNECTION NK_INTEGER */ - 389, /* (441) cmd ::= KILL QUERY NK_STRING */ - 389, /* (442) cmd ::= KILL TRANSACTION NK_INTEGER */ - 389, /* (443) cmd ::= KILL COMPACT NK_INTEGER */ - 389, /* (444) cmd ::= BALANCE VGROUP */ - 389, /* (445) cmd ::= BALANCE VGROUP LEADER on_vgroup_id */ - 389, /* (446) cmd ::= BALANCE VGROUP LEADER DATABASE db_name */ - 389, /* (447) cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER */ - 389, /* (448) cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list */ - 389, /* (449) cmd ::= SPLIT VGROUP NK_INTEGER */ - 501, /* (450) on_vgroup_id ::= */ - 501, /* (451) on_vgroup_id ::= ON NK_INTEGER */ - 502, /* (452) dnode_list ::= DNODE NK_INTEGER */ - 502, /* (453) dnode_list ::= dnode_list DNODE NK_INTEGER */ - 389, /* (454) cmd ::= DELETE FROM full_table_name where_clause_opt */ - 389, /* (455) cmd ::= query_or_subquery */ - 389, /* (456) cmd ::= insert_query */ - 484, /* (457) insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery */ - 484, /* (458) insert_query ::= INSERT INTO full_table_name query_or_subquery */ - 440, /* (459) tags_literal ::= NK_INTEGER */ - 440, /* (460) tags_literal ::= NK_INTEGER NK_PLUS duration_literal */ - 440, /* (461) tags_literal ::= NK_INTEGER NK_MINUS duration_literal */ - 440, /* (462) tags_literal ::= NK_PLUS NK_INTEGER */ - 440, /* (463) tags_literal ::= NK_PLUS NK_INTEGER NK_PLUS duration_literal */ - 440, /* (464) tags_literal ::= NK_PLUS NK_INTEGER NK_MINUS duration_literal */ - 440, /* (465) tags_literal ::= NK_MINUS NK_INTEGER */ - 440, /* (466) tags_literal ::= NK_MINUS NK_INTEGER NK_PLUS duration_literal */ - 440, /* (467) tags_literal ::= NK_MINUS NK_INTEGER NK_MINUS duration_literal */ - 440, /* (468) tags_literal ::= NK_FLOAT */ - 440, /* (469) tags_literal ::= NK_PLUS NK_FLOAT */ - 440, /* (470) tags_literal ::= NK_MINUS NK_FLOAT */ - 440, /* (471) tags_literal ::= NK_BIN */ - 440, /* (472) tags_literal ::= NK_BIN NK_PLUS duration_literal */ - 440, /* (473) tags_literal ::= NK_BIN NK_MINUS duration_literal */ - 440, /* (474) tags_literal ::= NK_PLUS NK_BIN */ - 440, /* (475) tags_literal ::= NK_PLUS NK_BIN NK_PLUS duration_literal */ - 440, /* (476) tags_literal ::= NK_PLUS NK_BIN NK_MINUS duration_literal */ - 440, /* (477) tags_literal ::= NK_MINUS NK_BIN */ - 440, /* (478) tags_literal ::= NK_MINUS NK_BIN NK_PLUS duration_literal */ - 440, /* (479) tags_literal ::= NK_MINUS NK_BIN NK_MINUS duration_literal */ - 440, /* (480) tags_literal ::= NK_HEX */ - 440, /* (481) tags_literal ::= NK_HEX NK_PLUS duration_literal */ - 440, /* (482) tags_literal ::= NK_HEX NK_MINUS duration_literal */ - 440, /* (483) tags_literal ::= NK_PLUS NK_HEX */ - 440, /* (484) tags_literal ::= NK_PLUS NK_HEX NK_PLUS duration_literal */ - 440, /* (485) tags_literal ::= NK_PLUS NK_HEX NK_MINUS duration_literal */ - 440, /* (486) tags_literal ::= NK_MINUS NK_HEX */ - 440, /* (487) tags_literal ::= NK_MINUS NK_HEX NK_PLUS duration_literal */ - 440, /* (488) tags_literal ::= NK_MINUS NK_HEX NK_MINUS duration_literal */ - 440, /* (489) tags_literal ::= NK_STRING */ - 440, /* (490) tags_literal ::= NK_STRING NK_PLUS duration_literal */ - 440, /* (491) tags_literal ::= NK_STRING NK_MINUS duration_literal */ - 440, /* (492) tags_literal ::= NK_BOOL */ - 440, /* (493) tags_literal ::= NULL */ - 440, /* (494) tags_literal ::= literal_func */ - 440, /* (495) tags_literal ::= literal_func NK_PLUS duration_literal */ - 440, /* (496) tags_literal ::= literal_func NK_MINUS duration_literal */ - 443, /* (497) tags_literal_list ::= tags_literal */ - 443, /* (498) tags_literal_list ::= tags_literal_list NK_COMMA tags_literal */ - 392, /* (499) literal ::= NK_INTEGER */ - 392, /* (500) literal ::= NK_FLOAT */ - 392, /* (501) literal ::= NK_STRING */ - 392, /* (502) literal ::= NK_BOOL */ - 392, /* (503) literal ::= TIMESTAMP NK_STRING */ - 392, /* (504) literal ::= duration_literal */ - 392, /* (505) literal ::= NULL */ - 392, /* (506) literal ::= NK_QUESTION */ - 453, /* (507) duration_literal ::= NK_VARIABLE */ - 423, /* (508) signed ::= NK_INTEGER */ - 423, /* (509) signed ::= NK_PLUS NK_INTEGER */ - 423, /* (510) signed ::= NK_MINUS NK_INTEGER */ - 423, /* (511) signed ::= NK_FLOAT */ - 423, /* (512) signed ::= NK_PLUS NK_FLOAT */ - 423, /* (513) signed ::= NK_MINUS NK_FLOAT */ - 504, /* (514) signed_literal ::= signed */ - 504, /* (515) signed_literal ::= NK_STRING */ - 504, /* (516) signed_literal ::= NK_BOOL */ - 504, /* (517) signed_literal ::= TIMESTAMP NK_STRING */ - 504, /* (518) signed_literal ::= duration_literal */ - 504, /* (519) signed_literal ::= NULL */ - 504, /* (520) signed_literal ::= literal_func */ - 504, /* (521) signed_literal ::= NK_QUESTION */ - 505, /* (522) literal_list ::= signed_literal */ - 505, /* (523) literal_list ::= literal_list NK_COMMA signed_literal */ - 406, /* (524) db_name ::= NK_ID */ - 407, /* (525) table_name ::= NK_ID */ - 437, /* (526) column_name ::= NK_ID */ - 455, /* (527) function_name ::= NK_ID */ - 490, /* (528) view_name ::= NK_ID */ - 506, /* (529) table_alias ::= NK_ID */ - 465, /* (530) column_alias ::= NK_ID */ - 465, /* (531) column_alias ::= NK_ALIAS */ - 399, /* (532) user_name ::= NK_ID */ - 408, /* (533) topic_name ::= NK_ID */ - 491, /* (534) stream_name ::= NK_ID */ - 481, /* (535) cgroup_name ::= NK_ID */ - 472, /* (536) index_name ::= NK_ID */ - 466, /* (537) tsma_name ::= NK_ID */ - 507, /* (538) expr_or_subquery ::= expression */ - 500, /* (539) expression ::= literal */ - 500, /* (540) expression ::= pseudo_column */ - 500, /* (541) expression ::= column_reference */ - 500, /* (542) expression ::= function_expression */ - 500, /* (543) expression ::= case_when_expression */ - 500, /* (544) expression ::= NK_LP expression NK_RP */ - 500, /* (545) expression ::= NK_PLUS expr_or_subquery */ - 500, /* (546) expression ::= NK_MINUS expr_or_subquery */ - 500, /* (547) expression ::= expr_or_subquery NK_PLUS expr_or_subquery */ - 500, /* (548) expression ::= expr_or_subquery NK_MINUS expr_or_subquery */ - 500, /* (549) expression ::= expr_or_subquery NK_STAR expr_or_subquery */ - 500, /* (550) expression ::= expr_or_subquery NK_SLASH expr_or_subquery */ - 500, /* (551) expression ::= expr_or_subquery NK_REM expr_or_subquery */ - 500, /* (552) expression ::= column_reference NK_ARROW NK_STRING */ - 500, /* (553) expression ::= expr_or_subquery NK_BITAND expr_or_subquery */ - 500, /* (554) expression ::= expr_or_subquery NK_BITOR expr_or_subquery */ - 477, /* (555) expression_list ::= expr_or_subquery */ - 477, /* (556) expression_list ::= expression_list NK_COMMA expr_or_subquery */ - 509, /* (557) column_reference ::= column_name */ - 509, /* (558) column_reference ::= table_name NK_DOT column_name */ - 509, /* (559) column_reference ::= NK_ALIAS */ - 509, /* (560) column_reference ::= table_name NK_DOT NK_ALIAS */ - 508, /* (561) pseudo_column ::= ROWTS */ - 508, /* (562) pseudo_column ::= TBNAME */ - 508, /* (563) pseudo_column ::= table_name NK_DOT TBNAME */ - 508, /* (564) pseudo_column ::= QSTART */ - 508, /* (565) pseudo_column ::= QEND */ - 508, /* (566) pseudo_column ::= QDURATION */ - 508, /* (567) pseudo_column ::= WSTART */ - 508, /* (568) pseudo_column ::= WEND */ - 508, /* (569) pseudo_column ::= WDURATION */ - 508, /* (570) pseudo_column ::= IROWTS */ - 508, /* (571) pseudo_column ::= ISFILLED */ - 508, /* (572) pseudo_column ::= QTAGS */ - 508, /* (573) pseudo_column ::= FLOW */ - 508, /* (574) pseudo_column ::= FHIGH */ - 508, /* (575) pseudo_column ::= FROWTS */ - 510, /* (576) function_expression ::= function_name NK_LP expression_list NK_RP */ - 510, /* (577) function_expression ::= star_func NK_LP star_func_para_list NK_RP */ - 510, /* (578) function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP */ - 510, /* (579) function_expression ::= CAST NK_LP expr_or_subquery AS type_name_default_len NK_RP */ - 510, /* (580) function_expression ::= POSITION NK_LP expr_or_subquery IN expr_or_subquery NK_RP */ - 510, /* (581) function_expression ::= TRIM NK_LP expr_or_subquery NK_RP */ - 510, /* (582) function_expression ::= TRIM NK_LP trim_specification_type FROM expr_or_subquery NK_RP */ - 510, /* (583) function_expression ::= TRIM NK_LP expr_or_subquery FROM expr_or_subquery NK_RP */ - 510, /* (584) function_expression ::= TRIM NK_LP trim_specification_type expr_or_subquery FROM expr_or_subquery NK_RP */ - 510, /* (585) function_expression ::= substr_func NK_LP expression_list NK_RP */ - 510, /* (586) function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery NK_RP */ - 510, /* (587) function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery FOR expr_or_subquery NK_RP */ - 510, /* (588) function_expression ::= REPLACE NK_LP expression_list NK_RP */ - 510, /* (589) function_expression ::= literal_func */ - 510, /* (590) function_expression ::= rand_func */ - 503, /* (591) literal_func ::= noarg_func NK_LP NK_RP */ - 503, /* (592) literal_func ::= NOW */ - 503, /* (593) literal_func ::= TODAY */ - 516, /* (594) rand_func ::= RAND NK_LP NK_RP */ - 516, /* (595) rand_func ::= RAND NK_LP expression_list NK_RP */ - 515, /* (596) substr_func ::= SUBSTR */ - 515, /* (597) substr_func ::= SUBSTRING */ - 514, /* (598) trim_specification_type ::= BOTH */ - 514, /* (599) trim_specification_type ::= TRAILING */ - 514, /* (600) trim_specification_type ::= LEADING */ - 517, /* (601) noarg_func ::= NOW */ - 517, /* (602) noarg_func ::= TODAY */ - 517, /* (603) noarg_func ::= TIMEZONE */ - 517, /* (604) noarg_func ::= DATABASE */ - 517, /* (605) noarg_func ::= CLIENT_VERSION */ - 517, /* (606) noarg_func ::= SERVER_VERSION */ - 517, /* (607) noarg_func ::= SERVER_STATUS */ - 517, /* (608) noarg_func ::= CURRENT_USER */ - 517, /* (609) noarg_func ::= USER */ - 517, /* (610) noarg_func ::= PI */ - 512, /* (611) star_func ::= COUNT */ - 512, /* (612) star_func ::= FIRST */ - 512, /* (613) star_func ::= LAST */ - 512, /* (614) star_func ::= LAST_ROW */ - 513, /* (615) star_func_para_list ::= NK_STAR */ - 513, /* (616) star_func_para_list ::= other_para_list */ - 518, /* (617) other_para_list ::= star_func_para */ - 518, /* (618) other_para_list ::= other_para_list NK_COMMA star_func_para */ - 519, /* (619) star_func_para ::= expr_or_subquery */ - 519, /* (620) star_func_para ::= table_name NK_DOT NK_STAR */ - 511, /* (621) case_when_expression ::= CASE when_then_list case_when_else_opt END */ - 511, /* (622) case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END */ - 520, /* (623) when_then_list ::= when_then_expr */ - 520, /* (624) when_then_list ::= when_then_list when_then_expr */ - 523, /* (625) when_then_expr ::= WHEN common_expression THEN common_expression */ - 521, /* (626) case_when_else_opt ::= */ - 521, /* (627) case_when_else_opt ::= ELSE common_expression */ - 524, /* (628) predicate ::= expr_or_subquery compare_op expr_or_subquery */ - 524, /* (629) predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery */ - 524, /* (630) predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery */ - 524, /* (631) predicate ::= expr_or_subquery IS NULL */ - 524, /* (632) predicate ::= expr_or_subquery IS NOT NULL */ - 524, /* (633) predicate ::= expr_or_subquery in_op in_predicate_value */ - 525, /* (634) compare_op ::= NK_LT */ - 525, /* (635) compare_op ::= NK_GT */ - 525, /* (636) compare_op ::= NK_LE */ - 525, /* (637) compare_op ::= NK_GE */ - 525, /* (638) compare_op ::= NK_NE */ - 525, /* (639) compare_op ::= NK_EQ */ - 525, /* (640) compare_op ::= LIKE */ - 525, /* (641) compare_op ::= NOT LIKE */ - 525, /* (642) compare_op ::= MATCH */ - 525, /* (643) compare_op ::= NMATCH */ - 525, /* (644) compare_op ::= CONTAINS */ - 526, /* (645) in_op ::= IN */ - 526, /* (646) in_op ::= NOT IN */ - 527, /* (647) in_predicate_value ::= NK_LP literal_list NK_RP */ - 528, /* (648) boolean_value_expression ::= boolean_primary */ - 528, /* (649) boolean_value_expression ::= NOT boolean_primary */ - 528, /* (650) boolean_value_expression ::= boolean_value_expression OR boolean_value_expression */ - 528, /* (651) boolean_value_expression ::= boolean_value_expression AND boolean_value_expression */ - 529, /* (652) boolean_primary ::= predicate */ - 529, /* (653) boolean_primary ::= NK_LP boolean_value_expression NK_RP */ - 522, /* (654) common_expression ::= expr_or_subquery */ - 522, /* (655) common_expression ::= boolean_value_expression */ - 530, /* (656) from_clause_opt ::= */ - 530, /* (657) from_clause_opt ::= FROM table_reference_list */ - 531, /* (658) table_reference_list ::= table_reference */ - 531, /* (659) table_reference_list ::= table_reference_list NK_COMMA table_reference */ - 532, /* (660) table_reference ::= table_primary */ - 532, /* (661) table_reference ::= joined_table */ - 533, /* (662) table_primary ::= table_name alias_opt */ - 533, /* (663) table_primary ::= db_name NK_DOT table_name alias_opt */ - 533, /* (664) table_primary ::= subquery alias_opt */ - 533, /* (665) table_primary ::= parenthesized_joined_table */ - 535, /* (666) alias_opt ::= */ - 535, /* (667) alias_opt ::= table_alias */ - 535, /* (668) alias_opt ::= AS table_alias */ - 537, /* (669) parenthesized_joined_table ::= NK_LP joined_table NK_RP */ - 537, /* (670) parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP */ - 534, /* (671) joined_table ::= table_reference join_type join_subtype JOIN table_reference join_on_clause_opt window_offset_clause_opt jlimit_clause_opt */ - 538, /* (672) join_type ::= */ - 538, /* (673) join_type ::= INNER */ - 538, /* (674) join_type ::= LEFT */ - 538, /* (675) join_type ::= RIGHT */ - 538, /* (676) join_type ::= FULL */ - 539, /* (677) join_subtype ::= */ - 539, /* (678) join_subtype ::= OUTER */ - 539, /* (679) join_subtype ::= SEMI */ - 539, /* (680) join_subtype ::= ANTI */ - 539, /* (681) join_subtype ::= ASOF */ - 539, /* (682) join_subtype ::= WINDOW */ - 540, /* (683) join_on_clause_opt ::= */ - 540, /* (684) join_on_clause_opt ::= ON search_condition */ - 541, /* (685) window_offset_clause_opt ::= */ - 541, /* (686) window_offset_clause_opt ::= WINDOW_OFFSET NK_LP window_offset_literal NK_COMMA window_offset_literal NK_RP */ - 543, /* (687) window_offset_literal ::= NK_VARIABLE */ - 543, /* (688) window_offset_literal ::= NK_MINUS NK_VARIABLE */ - 542, /* (689) jlimit_clause_opt ::= */ - 542, /* (690) jlimit_clause_opt ::= JLIMIT NK_INTEGER */ - 544, /* (691) query_specification ::= SELECT hint_list set_quantifier_opt tag_mode_opt select_list from_clause_opt where_clause_opt partition_by_clause_opt range_opt every_opt fill_opt twindow_clause_opt group_by_clause_opt having_clause_opt */ - 545, /* (692) hint_list ::= */ - 545, /* (693) hint_list ::= NK_HINT */ - 547, /* (694) tag_mode_opt ::= */ - 547, /* (695) tag_mode_opt ::= TAGS */ - 546, /* (696) set_quantifier_opt ::= */ - 546, /* (697) set_quantifier_opt ::= DISTINCT */ - 546, /* (698) set_quantifier_opt ::= ALL */ - 548, /* (699) select_list ::= select_item */ - 548, /* (700) select_list ::= select_list NK_COMMA select_item */ - 556, /* (701) select_item ::= NK_STAR */ - 556, /* (702) select_item ::= common_expression */ - 556, /* (703) select_item ::= common_expression column_alias */ - 556, /* (704) select_item ::= common_expression AS column_alias */ - 556, /* (705) select_item ::= table_name NK_DOT NK_STAR */ - 480, /* (706) where_clause_opt ::= */ - 480, /* (707) where_clause_opt ::= WHERE search_condition */ - 549, /* (708) partition_by_clause_opt ::= */ - 549, /* (709) partition_by_clause_opt ::= PARTITION BY partition_list */ - 557, /* (710) partition_list ::= partition_item */ - 557, /* (711) partition_list ::= partition_list NK_COMMA partition_item */ - 558, /* (712) partition_item ::= expr_or_subquery */ - 558, /* (713) partition_item ::= expr_or_subquery column_alias */ - 558, /* (714) partition_item ::= expr_or_subquery AS column_alias */ - 553, /* (715) twindow_clause_opt ::= */ - 553, /* (716) twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA interval_sliding_duration_literal NK_RP */ - 553, /* (717) twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP */ - 553, /* (718) twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_RP sliding_opt fill_opt */ - 553, /* (719) twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_COMMA interval_sliding_duration_literal NK_RP sliding_opt fill_opt */ - 553, /* (720) twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition */ - 553, /* (721) twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_RP */ - 553, /* (722) twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP */ - 553, /* (723) twindow_clause_opt ::= ANOMALY_WINDOW NK_LP expr_or_subquery NK_RP */ - 553, /* (724) twindow_clause_opt ::= ANOMALY_WINDOW NK_LP expr_or_subquery NK_COMMA NK_STRING NK_RP */ - 473, /* (725) sliding_opt ::= */ - 473, /* (726) sliding_opt ::= SLIDING NK_LP interval_sliding_duration_literal NK_RP */ - 559, /* (727) interval_sliding_duration_literal ::= NK_VARIABLE */ - 559, /* (728) interval_sliding_duration_literal ::= NK_STRING */ - 559, /* (729) interval_sliding_duration_literal ::= NK_INTEGER */ - 552, /* (730) fill_opt ::= */ - 552, /* (731) fill_opt ::= FILL NK_LP fill_mode NK_RP */ - 552, /* (732) fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP */ - 552, /* (733) fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP */ - 560, /* (734) fill_mode ::= NONE */ - 560, /* (735) fill_mode ::= PREV */ - 560, /* (736) fill_mode ::= NULL */ - 560, /* (737) fill_mode ::= NULL_F */ - 560, /* (738) fill_mode ::= LINEAR */ - 560, /* (739) fill_mode ::= NEXT */ - 554, /* (740) group_by_clause_opt ::= */ - 554, /* (741) group_by_clause_opt ::= GROUP BY group_by_list */ - 561, /* (742) group_by_list ::= expr_or_subquery */ - 561, /* (743) group_by_list ::= group_by_list NK_COMMA expr_or_subquery */ - 555, /* (744) having_clause_opt ::= */ - 555, /* (745) having_clause_opt ::= HAVING search_condition */ - 550, /* (746) range_opt ::= */ - 550, /* (747) range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP */ - 550, /* (748) range_opt ::= RANGE NK_LP expr_or_subquery NK_RP */ - 551, /* (749) every_opt ::= */ - 551, /* (750) every_opt ::= EVERY NK_LP duration_literal NK_RP */ - 562, /* (751) query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt */ - 563, /* (752) query_simple ::= query_specification */ - 563, /* (753) query_simple ::= union_query_expression */ - 567, /* (754) union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery */ - 567, /* (755) union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery */ - 568, /* (756) query_simple_or_subquery ::= query_simple */ - 568, /* (757) query_simple_or_subquery ::= subquery */ - 479, /* (758) query_or_subquery ::= query_expression */ - 479, /* (759) query_or_subquery ::= subquery */ - 564, /* (760) order_by_clause_opt ::= */ - 564, /* (761) order_by_clause_opt ::= ORDER BY sort_specification_list */ - 565, /* (762) slimit_clause_opt ::= */ - 565, /* (763) slimit_clause_opt ::= SLIMIT NK_INTEGER */ - 565, /* (764) slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER */ - 565, /* (765) slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER */ - 566, /* (766) limit_clause_opt ::= */ - 566, /* (767) limit_clause_opt ::= LIMIT NK_INTEGER */ - 566, /* (768) limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER */ - 566, /* (769) limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER */ - 536, /* (770) subquery ::= NK_LP query_expression NK_RP */ - 536, /* (771) subquery ::= NK_LP subquery NK_RP */ - 409, /* (772) search_condition ::= common_expression */ - 569, /* (773) sort_specification_list ::= sort_specification */ - 569, /* (774) sort_specification_list ::= sort_specification_list NK_COMMA sort_specification */ - 570, /* (775) sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt */ - 571, /* (776) ordering_specification_opt ::= */ - 571, /* (777) ordering_specification_opt ::= ASC */ - 571, /* (778) ordering_specification_opt ::= DESC */ - 572, /* (779) null_ordering_opt ::= */ - 572, /* (780) null_ordering_opt ::= NULLS FIRST */ - 572, /* (781) null_ordering_opt ::= NULLS LAST */ - 439, /* (782) column_options ::= */ - 439, /* (783) column_options ::= column_options PRIMARY KEY */ - 439, /* (784) column_options ::= column_options NK_ID NK_STRING */ + 390, /* (0) cmd ::= CREATE ACCOUNT NK_ID PASS NK_STRING account_options */ + 390, /* (1) cmd ::= ALTER ACCOUNT NK_ID alter_account_options */ + 391, /* (2) account_options ::= */ + 391, /* (3) account_options ::= account_options PPS literal */ + 391, /* (4) account_options ::= account_options TSERIES literal */ + 391, /* (5) account_options ::= account_options STORAGE literal */ + 391, /* (6) account_options ::= account_options STREAMS literal */ + 391, /* (7) account_options ::= account_options QTIME literal */ + 391, /* (8) account_options ::= account_options DBS literal */ + 391, /* (9) account_options ::= account_options USERS literal */ + 391, /* (10) account_options ::= account_options CONNS literal */ + 391, /* (11) account_options ::= account_options STATE literal */ + 392, /* (12) alter_account_options ::= alter_account_option */ + 392, /* (13) alter_account_options ::= alter_account_options alter_account_option */ + 394, /* (14) alter_account_option ::= PASS literal */ + 394, /* (15) alter_account_option ::= PPS literal */ + 394, /* (16) alter_account_option ::= TSERIES literal */ + 394, /* (17) alter_account_option ::= STORAGE literal */ + 394, /* (18) alter_account_option ::= STREAMS literal */ + 394, /* (19) alter_account_option ::= QTIME literal */ + 394, /* (20) alter_account_option ::= DBS literal */ + 394, /* (21) alter_account_option ::= USERS literal */ + 394, /* (22) alter_account_option ::= CONNS literal */ + 394, /* (23) alter_account_option ::= STATE literal */ + 395, /* (24) ip_range_list ::= NK_STRING */ + 395, /* (25) ip_range_list ::= ip_range_list NK_COMMA NK_STRING */ + 396, /* (26) white_list ::= HOST ip_range_list */ + 397, /* (27) white_list_opt ::= */ + 397, /* (28) white_list_opt ::= white_list */ + 398, /* (29) is_import_opt ::= */ + 398, /* (30) is_import_opt ::= IS_IMPORT NK_INTEGER */ + 399, /* (31) is_createdb_opt ::= */ + 399, /* (32) is_createdb_opt ::= CREATEDB NK_INTEGER */ + 390, /* (33) cmd ::= CREATE USER user_name PASS NK_STRING sysinfo_opt is_createdb_opt is_import_opt white_list_opt */ + 390, /* (34) cmd ::= ALTER USER user_name PASS NK_STRING */ + 390, /* (35) cmd ::= ALTER USER user_name ENABLE NK_INTEGER */ + 390, /* (36) cmd ::= ALTER USER user_name SYSINFO NK_INTEGER */ + 390, /* (37) cmd ::= ALTER USER user_name CREATEDB NK_INTEGER */ + 390, /* (38) cmd ::= ALTER USER user_name ADD white_list */ + 390, /* (39) cmd ::= ALTER USER user_name DROP white_list */ + 390, /* (40) cmd ::= DROP USER user_name */ + 401, /* (41) sysinfo_opt ::= */ + 401, /* (42) sysinfo_opt ::= SYSINFO NK_INTEGER */ + 390, /* (43) cmd ::= GRANT privileges ON priv_level with_clause_opt TO user_name */ + 390, /* (44) cmd ::= REVOKE privileges ON priv_level with_clause_opt FROM user_name */ + 402, /* (45) privileges ::= ALL */ + 402, /* (46) privileges ::= priv_type_list */ + 402, /* (47) privileges ::= SUBSCRIBE */ + 405, /* (48) priv_type_list ::= priv_type */ + 405, /* (49) priv_type_list ::= priv_type_list NK_COMMA priv_type */ + 406, /* (50) priv_type ::= READ */ + 406, /* (51) priv_type ::= WRITE */ + 406, /* (52) priv_type ::= ALTER */ + 403, /* (53) priv_level ::= NK_STAR NK_DOT NK_STAR */ + 403, /* (54) priv_level ::= db_name NK_DOT NK_STAR */ + 403, /* (55) priv_level ::= db_name NK_DOT table_name */ + 403, /* (56) priv_level ::= topic_name */ + 404, /* (57) with_clause_opt ::= */ + 404, /* (58) with_clause_opt ::= WITH search_condition */ + 390, /* (59) cmd ::= CREATE ENCRYPT_KEY NK_STRING */ + 390, /* (60) cmd ::= CREATE ANODE NK_STRING */ + 390, /* (61) cmd ::= UPDATE ANODE NK_INTEGER */ + 390, /* (62) cmd ::= UPDATE ALL ANODES */ + 390, /* (63) cmd ::= DROP ANODE NK_INTEGER */ + 390, /* (64) cmd ::= CREATE DNODE dnode_endpoint */ + 390, /* (65) cmd ::= CREATE DNODE dnode_endpoint PORT NK_INTEGER */ + 390, /* (66) cmd ::= DROP DNODE NK_INTEGER force_opt */ + 390, /* (67) cmd ::= DROP DNODE dnode_endpoint force_opt */ + 390, /* (68) cmd ::= DROP DNODE NK_INTEGER unsafe_opt */ + 390, /* (69) cmd ::= DROP DNODE dnode_endpoint unsafe_opt */ + 390, /* (70) cmd ::= ALTER DNODE NK_INTEGER NK_STRING */ + 390, /* (71) cmd ::= ALTER DNODE NK_INTEGER NK_STRING NK_STRING */ + 390, /* (72) cmd ::= ALTER ALL DNODES NK_STRING */ + 390, /* (73) cmd ::= ALTER ALL DNODES NK_STRING NK_STRING */ + 390, /* (74) cmd ::= RESTORE DNODE NK_INTEGER */ + 411, /* (75) dnode_endpoint ::= NK_STRING */ + 411, /* (76) dnode_endpoint ::= NK_ID */ + 411, /* (77) dnode_endpoint ::= NK_IPTOKEN */ + 412, /* (78) force_opt ::= */ + 412, /* (79) force_opt ::= FORCE */ + 413, /* (80) unsafe_opt ::= UNSAFE */ + 390, /* (81) cmd ::= ALTER CLUSTER NK_STRING */ + 390, /* (82) cmd ::= ALTER CLUSTER NK_STRING NK_STRING */ + 390, /* (83) cmd ::= ALTER LOCAL NK_STRING */ + 390, /* (84) cmd ::= ALTER LOCAL NK_STRING NK_STRING */ + 390, /* (85) cmd ::= CREATE QNODE ON DNODE NK_INTEGER */ + 390, /* (86) cmd ::= DROP QNODE ON DNODE NK_INTEGER */ + 390, /* (87) cmd ::= RESTORE QNODE ON DNODE NK_INTEGER */ + 390, /* (88) cmd ::= CREATE BNODE ON DNODE NK_INTEGER */ + 390, /* (89) cmd ::= DROP BNODE ON DNODE NK_INTEGER */ + 390, /* (90) cmd ::= CREATE SNODE ON DNODE NK_INTEGER */ + 390, /* (91) cmd ::= DROP SNODE ON DNODE NK_INTEGER */ + 390, /* (92) cmd ::= CREATE MNODE ON DNODE NK_INTEGER */ + 390, /* (93) cmd ::= DROP MNODE ON DNODE NK_INTEGER */ + 390, /* (94) cmd ::= RESTORE MNODE ON DNODE NK_INTEGER */ + 390, /* (95) cmd ::= RESTORE VNODE ON DNODE NK_INTEGER */ + 390, /* (96) cmd ::= CREATE DATABASE not_exists_opt db_name db_options */ + 390, /* (97) cmd ::= DROP DATABASE exists_opt db_name */ + 390, /* (98) cmd ::= USE db_name */ + 390, /* (99) cmd ::= ALTER DATABASE db_name alter_db_options */ + 390, /* (100) cmd ::= FLUSH DATABASE db_name */ + 390, /* (101) cmd ::= TRIM DATABASE db_name speed_opt */ + 390, /* (102) cmd ::= S3MIGRATE DATABASE db_name */ + 390, /* (103) cmd ::= COMPACT DATABASE db_name start_opt end_opt */ + 414, /* (104) not_exists_opt ::= IF NOT EXISTS */ + 414, /* (105) not_exists_opt ::= */ + 416, /* (106) exists_opt ::= IF EXISTS */ + 416, /* (107) exists_opt ::= */ + 415, /* (108) db_options ::= */ + 415, /* (109) db_options ::= db_options BUFFER NK_INTEGER */ + 415, /* (110) db_options ::= db_options CACHEMODEL NK_STRING */ + 415, /* (111) db_options ::= db_options CACHESIZE NK_INTEGER */ + 415, /* (112) db_options ::= db_options COMP NK_INTEGER */ + 415, /* (113) db_options ::= db_options DURATION NK_INTEGER */ + 415, /* (114) db_options ::= db_options DURATION NK_VARIABLE */ + 415, /* (115) db_options ::= db_options MAXROWS NK_INTEGER */ + 415, /* (116) db_options ::= db_options MINROWS NK_INTEGER */ + 415, /* (117) db_options ::= db_options KEEP integer_list */ + 415, /* (118) db_options ::= db_options KEEP variable_list */ + 415, /* (119) db_options ::= db_options PAGES NK_INTEGER */ + 415, /* (120) db_options ::= db_options PAGESIZE NK_INTEGER */ + 415, /* (121) db_options ::= db_options TSDB_PAGESIZE NK_INTEGER */ + 415, /* (122) db_options ::= db_options PRECISION NK_STRING */ + 415, /* (123) db_options ::= db_options REPLICA NK_INTEGER */ + 415, /* (124) db_options ::= db_options VGROUPS NK_INTEGER */ + 415, /* (125) db_options ::= db_options SINGLE_STABLE NK_INTEGER */ + 415, /* (126) db_options ::= db_options RETENTIONS retention_list */ + 415, /* (127) db_options ::= db_options SCHEMALESS NK_INTEGER */ + 415, /* (128) db_options ::= db_options WAL_LEVEL NK_INTEGER */ + 415, /* (129) db_options ::= db_options WAL_FSYNC_PERIOD NK_INTEGER */ + 415, /* (130) db_options ::= db_options WAL_RETENTION_PERIOD NK_INTEGER */ + 415, /* (131) db_options ::= db_options WAL_RETENTION_PERIOD NK_MINUS NK_INTEGER */ + 415, /* (132) db_options ::= db_options WAL_RETENTION_SIZE NK_INTEGER */ + 415, /* (133) db_options ::= db_options WAL_RETENTION_SIZE NK_MINUS NK_INTEGER */ + 415, /* (134) db_options ::= db_options WAL_ROLL_PERIOD NK_INTEGER */ + 415, /* (135) db_options ::= db_options WAL_SEGMENT_SIZE NK_INTEGER */ + 415, /* (136) db_options ::= db_options STT_TRIGGER NK_INTEGER */ + 415, /* (137) db_options ::= db_options TABLE_PREFIX signed */ + 415, /* (138) db_options ::= db_options TABLE_SUFFIX signed */ + 415, /* (139) db_options ::= db_options S3_CHUNKPAGES NK_INTEGER */ + 415, /* (140) db_options ::= db_options S3_KEEPLOCAL NK_INTEGER */ + 415, /* (141) db_options ::= db_options S3_KEEPLOCAL NK_VARIABLE */ + 415, /* (142) db_options ::= db_options S3_COMPACT NK_INTEGER */ + 415, /* (143) db_options ::= db_options KEEP_TIME_OFFSET NK_INTEGER */ + 415, /* (144) db_options ::= db_options ENCRYPT_ALGORITHM NK_STRING */ + 415, /* (145) db_options ::= db_options DNODES NK_STRING */ + 417, /* (146) alter_db_options ::= alter_db_option */ + 417, /* (147) alter_db_options ::= alter_db_options alter_db_option */ + 425, /* (148) alter_db_option ::= BUFFER NK_INTEGER */ + 425, /* (149) alter_db_option ::= CACHEMODEL NK_STRING */ + 425, /* (150) alter_db_option ::= CACHESIZE NK_INTEGER */ + 425, /* (151) alter_db_option ::= WAL_FSYNC_PERIOD NK_INTEGER */ + 425, /* (152) alter_db_option ::= KEEP integer_list */ + 425, /* (153) alter_db_option ::= KEEP variable_list */ + 425, /* (154) alter_db_option ::= PAGES NK_INTEGER */ + 425, /* (155) alter_db_option ::= REPLICA NK_INTEGER */ + 425, /* (156) alter_db_option ::= WAL_LEVEL NK_INTEGER */ + 425, /* (157) alter_db_option ::= STT_TRIGGER NK_INTEGER */ + 425, /* (158) alter_db_option ::= MINROWS NK_INTEGER */ + 425, /* (159) alter_db_option ::= WAL_RETENTION_PERIOD NK_INTEGER */ + 425, /* (160) alter_db_option ::= WAL_RETENTION_PERIOD NK_MINUS NK_INTEGER */ + 425, /* (161) alter_db_option ::= WAL_RETENTION_SIZE NK_INTEGER */ + 425, /* (162) alter_db_option ::= WAL_RETENTION_SIZE NK_MINUS NK_INTEGER */ + 425, /* (163) alter_db_option ::= S3_KEEPLOCAL NK_INTEGER */ + 425, /* (164) alter_db_option ::= S3_KEEPLOCAL NK_VARIABLE */ + 425, /* (165) alter_db_option ::= S3_COMPACT NK_INTEGER */ + 425, /* (166) alter_db_option ::= KEEP_TIME_OFFSET NK_INTEGER */ + 425, /* (167) alter_db_option ::= ENCRYPT_ALGORITHM NK_STRING */ + 421, /* (168) integer_list ::= NK_INTEGER */ + 421, /* (169) integer_list ::= integer_list NK_COMMA NK_INTEGER */ + 422, /* (170) variable_list ::= NK_VARIABLE */ + 422, /* (171) variable_list ::= variable_list NK_COMMA NK_VARIABLE */ + 423, /* (172) retention_list ::= retention */ + 423, /* (173) retention_list ::= retention_list NK_COMMA retention */ + 426, /* (174) retention ::= NK_VARIABLE NK_COLON NK_VARIABLE */ + 426, /* (175) retention ::= NK_MINUS NK_COLON NK_VARIABLE */ + 418, /* (176) speed_opt ::= */ + 418, /* (177) speed_opt ::= BWLIMIT NK_INTEGER */ + 419, /* (178) start_opt ::= */ + 419, /* (179) start_opt ::= START WITH NK_INTEGER */ + 419, /* (180) start_opt ::= START WITH NK_STRING */ + 419, /* (181) start_opt ::= START WITH TIMESTAMP NK_STRING */ + 420, /* (182) end_opt ::= */ + 420, /* (183) end_opt ::= END WITH NK_INTEGER */ + 420, /* (184) end_opt ::= END WITH NK_STRING */ + 420, /* (185) end_opt ::= END WITH TIMESTAMP NK_STRING */ + 390, /* (186) cmd ::= CREATE TABLE not_exists_opt full_table_name NK_LP column_def_list NK_RP tags_def_opt table_options */ + 390, /* (187) cmd ::= CREATE TABLE multi_create_clause */ + 390, /* (188) cmd ::= CREATE TABLE not_exists_opt USING full_table_name NK_LP tag_list_opt NK_RP FILE NK_STRING */ + 390, /* (189) cmd ::= CREATE STABLE not_exists_opt full_table_name NK_LP column_def_list NK_RP tags_def table_options */ + 390, /* (190) cmd ::= DROP TABLE with_opt multi_drop_clause */ + 390, /* (191) cmd ::= DROP STABLE with_opt exists_opt full_table_name */ + 390, /* (192) cmd ::= ALTER TABLE alter_table_clause */ + 390, /* (193) cmd ::= ALTER STABLE alter_table_clause */ + 436, /* (194) alter_table_clause ::= full_table_name alter_table_options */ + 436, /* (195) alter_table_clause ::= full_table_name ADD COLUMN column_name type_name column_options */ + 436, /* (196) alter_table_clause ::= full_table_name DROP COLUMN column_name */ + 436, /* (197) alter_table_clause ::= full_table_name MODIFY COLUMN column_name type_name */ + 436, /* (198) alter_table_clause ::= full_table_name MODIFY COLUMN column_name column_options */ + 436, /* (199) alter_table_clause ::= full_table_name RENAME COLUMN column_name column_name */ + 436, /* (200) alter_table_clause ::= full_table_name ADD TAG column_name type_name */ + 436, /* (201) alter_table_clause ::= full_table_name DROP TAG column_name */ + 436, /* (202) alter_table_clause ::= full_table_name MODIFY TAG column_name type_name */ + 436, /* (203) alter_table_clause ::= full_table_name RENAME TAG column_name column_name */ + 436, /* (204) alter_table_clause ::= full_table_name SET TAG column_name NK_EQ tags_literal */ + 431, /* (205) multi_create_clause ::= create_subtable_clause */ + 431, /* (206) multi_create_clause ::= multi_create_clause create_subtable_clause */ + 442, /* (207) create_subtable_clause ::= not_exists_opt full_table_name USING full_table_name specific_cols_opt TAGS NK_LP tags_literal_list NK_RP table_options */ + 435, /* (208) multi_drop_clause ::= drop_table_clause */ + 435, /* (209) multi_drop_clause ::= multi_drop_clause NK_COMMA drop_table_clause */ + 445, /* (210) drop_table_clause ::= exists_opt full_table_name */ + 434, /* (211) with_opt ::= */ + 434, /* (212) with_opt ::= WITH */ + 443, /* (213) specific_cols_opt ::= */ + 443, /* (214) specific_cols_opt ::= NK_LP col_name_list NK_RP */ + 427, /* (215) full_table_name ::= table_name */ + 427, /* (216) full_table_name ::= db_name NK_DOT table_name */ + 447, /* (217) tag_def_list ::= tag_def */ + 447, /* (218) tag_def_list ::= tag_def_list NK_COMMA tag_def */ + 448, /* (219) tag_def ::= column_name type_name */ + 428, /* (220) column_def_list ::= column_def */ + 428, /* (221) column_def_list ::= column_def_list NK_COMMA column_def */ + 449, /* (222) column_def ::= column_name type_name column_options */ + 439, /* (223) type_name ::= BOOL */ + 439, /* (224) type_name ::= TINYINT */ + 439, /* (225) type_name ::= SMALLINT */ + 439, /* (226) type_name ::= INT */ + 439, /* (227) type_name ::= INTEGER */ + 439, /* (228) type_name ::= BIGINT */ + 439, /* (229) type_name ::= FLOAT */ + 439, /* (230) type_name ::= DOUBLE */ + 439, /* (231) type_name ::= BINARY NK_LP NK_INTEGER NK_RP */ + 439, /* (232) type_name ::= TIMESTAMP */ + 439, /* (233) type_name ::= NCHAR NK_LP NK_INTEGER NK_RP */ + 439, /* (234) type_name ::= TINYINT UNSIGNED */ + 439, /* (235) type_name ::= SMALLINT UNSIGNED */ + 439, /* (236) type_name ::= INT UNSIGNED */ + 439, /* (237) type_name ::= BIGINT UNSIGNED */ + 439, /* (238) type_name ::= JSON */ + 439, /* (239) type_name ::= VARCHAR NK_LP NK_INTEGER NK_RP */ + 439, /* (240) type_name ::= MEDIUMBLOB */ + 439, /* (241) type_name ::= BLOB */ + 439, /* (242) type_name ::= VARBINARY NK_LP NK_INTEGER NK_RP */ + 439, /* (243) type_name ::= GEOMETRY NK_LP NK_INTEGER NK_RP */ + 439, /* (244) type_name ::= DECIMAL */ + 439, /* (245) type_name ::= DECIMAL NK_LP NK_INTEGER NK_RP */ + 439, /* (246) type_name ::= DECIMAL NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP */ + 450, /* (247) type_name_default_len ::= BINARY */ + 450, /* (248) type_name_default_len ::= NCHAR */ + 450, /* (249) type_name_default_len ::= VARCHAR */ + 450, /* (250) type_name_default_len ::= VARBINARY */ + 429, /* (251) tags_def_opt ::= */ + 429, /* (252) tags_def_opt ::= tags_def */ + 433, /* (253) tags_def ::= TAGS NK_LP tag_def_list NK_RP */ + 430, /* (254) table_options ::= */ + 430, /* (255) table_options ::= table_options COMMENT NK_STRING */ + 430, /* (256) table_options ::= table_options MAX_DELAY duration_list */ + 430, /* (257) table_options ::= table_options WATERMARK duration_list */ + 430, /* (258) table_options ::= table_options ROLLUP NK_LP rollup_func_list NK_RP */ + 430, /* (259) table_options ::= table_options TTL NK_INTEGER */ + 430, /* (260) table_options ::= table_options SMA NK_LP col_name_list NK_RP */ + 430, /* (261) table_options ::= table_options DELETE_MARK duration_list */ + 437, /* (262) alter_table_options ::= alter_table_option */ + 437, /* (263) alter_table_options ::= alter_table_options alter_table_option */ + 453, /* (264) alter_table_option ::= COMMENT NK_STRING */ + 453, /* (265) alter_table_option ::= TTL NK_INTEGER */ + 451, /* (266) duration_list ::= duration_literal */ + 451, /* (267) duration_list ::= duration_list NK_COMMA duration_literal */ + 452, /* (268) rollup_func_list ::= rollup_func_name */ + 452, /* (269) rollup_func_list ::= rollup_func_list NK_COMMA rollup_func_name */ + 455, /* (270) rollup_func_name ::= function_name */ + 455, /* (271) rollup_func_name ::= FIRST */ + 455, /* (272) rollup_func_name ::= LAST */ + 446, /* (273) col_name_list ::= col_name */ + 446, /* (274) col_name_list ::= col_name_list NK_COMMA col_name */ + 457, /* (275) col_name ::= column_name */ + 390, /* (276) cmd ::= SHOW DNODES */ + 390, /* (277) cmd ::= SHOW USERS */ + 390, /* (278) cmd ::= SHOW USERS FULL */ + 390, /* (279) cmd ::= SHOW USER PRIVILEGES */ + 390, /* (280) cmd ::= SHOW db_kind_opt DATABASES */ + 390, /* (281) cmd ::= SHOW table_kind_db_name_cond_opt TABLES like_pattern_opt */ + 390, /* (282) cmd ::= SHOW db_name_cond_opt STABLES like_pattern_opt */ + 390, /* (283) cmd ::= SHOW db_name_cond_opt VGROUPS */ + 390, /* (284) cmd ::= SHOW MNODES */ + 390, /* (285) cmd ::= SHOW QNODES */ + 390, /* (286) cmd ::= SHOW ANODES */ + 390, /* (287) cmd ::= SHOW ANODES FULL */ + 390, /* (288) cmd ::= SHOW ARBGROUPS */ + 390, /* (289) cmd ::= SHOW FUNCTIONS */ + 390, /* (290) cmd ::= SHOW INDEXES FROM table_name_cond from_db_opt */ + 390, /* (291) cmd ::= SHOW INDEXES FROM db_name NK_DOT table_name */ + 390, /* (292) cmd ::= SHOW STREAMS */ + 390, /* (293) cmd ::= SHOW ACCOUNTS */ + 390, /* (294) cmd ::= SHOW APPS */ + 390, /* (295) cmd ::= SHOW CONNECTIONS */ + 390, /* (296) cmd ::= SHOW LICENCES */ + 390, /* (297) cmd ::= SHOW GRANTS */ + 390, /* (298) cmd ::= SHOW GRANTS FULL */ + 390, /* (299) cmd ::= SHOW GRANTS LOGS */ + 390, /* (300) cmd ::= SHOW CLUSTER MACHINES */ + 390, /* (301) cmd ::= SHOW CREATE DATABASE db_name */ + 390, /* (302) cmd ::= SHOW CREATE TABLE full_table_name */ + 390, /* (303) cmd ::= SHOW CREATE STABLE full_table_name */ + 390, /* (304) cmd ::= SHOW ENCRYPTIONS */ + 390, /* (305) cmd ::= SHOW QUERIES */ + 390, /* (306) cmd ::= SHOW SCORES */ + 390, /* (307) cmd ::= SHOW TOPICS */ + 390, /* (308) cmd ::= SHOW VARIABLES */ + 390, /* (309) cmd ::= SHOW CLUSTER VARIABLES */ + 390, /* (310) cmd ::= SHOW LOCAL VARIABLES */ + 390, /* (311) cmd ::= SHOW DNODE NK_INTEGER VARIABLES like_pattern_opt */ + 390, /* (312) cmd ::= SHOW BNODES */ + 390, /* (313) cmd ::= SHOW SNODES */ + 390, /* (314) cmd ::= SHOW CLUSTER */ + 390, /* (315) cmd ::= SHOW TRANSACTIONS */ + 390, /* (316) cmd ::= SHOW TABLE DISTRIBUTED full_table_name */ + 390, /* (317) cmd ::= SHOW CONSUMERS */ + 390, /* (318) cmd ::= SHOW SUBSCRIPTIONS */ + 390, /* (319) cmd ::= SHOW TAGS FROM table_name_cond from_db_opt */ + 390, /* (320) cmd ::= SHOW TAGS FROM db_name NK_DOT table_name */ + 390, /* (321) cmd ::= SHOW TABLE TAGS tag_list_opt FROM table_name_cond from_db_opt */ + 390, /* (322) cmd ::= SHOW TABLE TAGS tag_list_opt FROM db_name NK_DOT table_name */ + 390, /* (323) cmd ::= SHOW VNODES ON DNODE NK_INTEGER */ + 390, /* (324) cmd ::= SHOW VNODES */ + 390, /* (325) cmd ::= SHOW db_name_cond_opt ALIVE */ + 390, /* (326) cmd ::= SHOW CLUSTER ALIVE */ + 390, /* (327) cmd ::= SHOW db_name_cond_opt VIEWS like_pattern_opt */ + 390, /* (328) cmd ::= SHOW CREATE VIEW full_table_name */ + 390, /* (329) cmd ::= SHOW COMPACTS */ + 390, /* (330) cmd ::= SHOW COMPACT NK_INTEGER */ + 459, /* (331) table_kind_db_name_cond_opt ::= */ + 459, /* (332) table_kind_db_name_cond_opt ::= table_kind */ + 459, /* (333) table_kind_db_name_cond_opt ::= db_name NK_DOT */ + 459, /* (334) table_kind_db_name_cond_opt ::= table_kind db_name NK_DOT */ + 464, /* (335) table_kind ::= NORMAL */ + 464, /* (336) table_kind ::= CHILD */ + 461, /* (337) db_name_cond_opt ::= */ + 461, /* (338) db_name_cond_opt ::= db_name NK_DOT */ + 460, /* (339) like_pattern_opt ::= */ + 460, /* (340) like_pattern_opt ::= LIKE NK_STRING */ + 462, /* (341) table_name_cond ::= table_name */ + 463, /* (342) from_db_opt ::= */ + 463, /* (343) from_db_opt ::= FROM db_name */ + 432, /* (344) tag_list_opt ::= */ + 432, /* (345) tag_list_opt ::= tag_item */ + 432, /* (346) tag_list_opt ::= tag_list_opt NK_COMMA tag_item */ + 465, /* (347) tag_item ::= TBNAME */ + 465, /* (348) tag_item ::= QTAGS */ + 465, /* (349) tag_item ::= column_name */ + 465, /* (350) tag_item ::= column_name column_alias */ + 465, /* (351) tag_item ::= column_name AS column_alias */ + 458, /* (352) db_kind_opt ::= */ + 458, /* (353) db_kind_opt ::= USER */ + 458, /* (354) db_kind_opt ::= SYSTEM */ + 390, /* (355) cmd ::= CREATE TSMA not_exists_opt tsma_name ON full_table_name tsma_func_list INTERVAL NK_LP duration_literal NK_RP */ + 390, /* (356) cmd ::= CREATE RECURSIVE TSMA not_exists_opt tsma_name ON full_table_name INTERVAL NK_LP duration_literal NK_RP */ + 390, /* (357) cmd ::= DROP TSMA exists_opt full_tsma_name */ + 390, /* (358) cmd ::= SHOW db_name_cond_opt TSMAS */ + 469, /* (359) full_tsma_name ::= tsma_name */ + 469, /* (360) full_tsma_name ::= db_name NK_DOT tsma_name */ + 468, /* (361) tsma_func_list ::= FUNCTION NK_LP func_list NK_RP */ + 390, /* (362) cmd ::= CREATE SMA INDEX not_exists_opt col_name ON full_table_name index_options */ + 390, /* (363) cmd ::= CREATE INDEX not_exists_opt col_name ON full_table_name NK_LP col_name_list NK_RP */ + 390, /* (364) cmd ::= DROP INDEX exists_opt full_index_name */ + 472, /* (365) full_index_name ::= index_name */ + 472, /* (366) full_index_name ::= db_name NK_DOT index_name */ + 471, /* (367) index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_RP sliding_opt sma_stream_opt */ + 471, /* (368) 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 */ + 470, /* (369) func_list ::= func */ + 470, /* (370) func_list ::= func_list NK_COMMA func */ + 476, /* (371) func ::= sma_func_name NK_LP expression_list NK_RP */ + 477, /* (372) sma_func_name ::= function_name */ + 477, /* (373) sma_func_name ::= COUNT */ + 477, /* (374) sma_func_name ::= FIRST */ + 477, /* (375) sma_func_name ::= LAST */ + 477, /* (376) sma_func_name ::= LAST_ROW */ + 475, /* (377) sma_stream_opt ::= */ + 475, /* (378) sma_stream_opt ::= sma_stream_opt WATERMARK duration_literal */ + 475, /* (379) sma_stream_opt ::= sma_stream_opt MAX_DELAY duration_literal */ + 475, /* (380) sma_stream_opt ::= sma_stream_opt DELETE_MARK duration_literal */ + 479, /* (381) with_meta ::= AS */ + 479, /* (382) with_meta ::= WITH META AS */ + 479, /* (383) with_meta ::= ONLY META AS */ + 390, /* (384) cmd ::= CREATE TOPIC not_exists_opt topic_name AS query_or_subquery */ + 390, /* (385) cmd ::= CREATE TOPIC not_exists_opt topic_name with_meta DATABASE db_name */ + 390, /* (386) cmd ::= CREATE TOPIC not_exists_opt topic_name with_meta STABLE full_table_name where_clause_opt */ + 390, /* (387) cmd ::= DROP TOPIC exists_opt topic_name */ + 390, /* (388) cmd ::= DROP CONSUMER GROUP exists_opt cgroup_name ON topic_name */ + 390, /* (389) cmd ::= DESC full_table_name */ + 390, /* (390) cmd ::= DESCRIBE full_table_name */ + 390, /* (391) cmd ::= RESET QUERY CACHE */ + 390, /* (392) cmd ::= EXPLAIN analyze_opt explain_options query_or_subquery */ + 390, /* (393) cmd ::= EXPLAIN analyze_opt explain_options insert_query */ + 483, /* (394) analyze_opt ::= */ + 483, /* (395) analyze_opt ::= ANALYZE */ + 484, /* (396) explain_options ::= */ + 484, /* (397) explain_options ::= explain_options VERBOSE NK_BOOL */ + 484, /* (398) explain_options ::= explain_options RATIO NK_FLOAT */ + 390, /* (399) cmd ::= CREATE or_replace_opt agg_func_opt FUNCTION not_exists_opt function_name AS NK_STRING OUTPUTTYPE type_name bufsize_opt language_opt */ + 390, /* (400) cmd ::= DROP FUNCTION exists_opt function_name */ + 487, /* (401) agg_func_opt ::= */ + 487, /* (402) agg_func_opt ::= AGGREGATE */ + 488, /* (403) bufsize_opt ::= */ + 488, /* (404) bufsize_opt ::= BUFSIZE NK_INTEGER */ + 489, /* (405) language_opt ::= */ + 489, /* (406) language_opt ::= LANGUAGE NK_STRING */ + 486, /* (407) or_replace_opt ::= */ + 486, /* (408) or_replace_opt ::= OR REPLACE */ + 390, /* (409) cmd ::= CREATE or_replace_opt VIEW full_view_name AS query_or_subquery */ + 390, /* (410) cmd ::= DROP VIEW exists_opt full_view_name */ + 490, /* (411) full_view_name ::= view_name */ + 490, /* (412) full_view_name ::= db_name NK_DOT view_name */ + 390, /* (413) 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 */ + 390, /* (414) cmd ::= DROP STREAM exists_opt stream_name */ + 390, /* (415) cmd ::= PAUSE STREAM exists_opt stream_name */ + 390, /* (416) cmd ::= RESUME STREAM exists_opt ignore_opt stream_name */ + 494, /* (417) col_list_opt ::= */ + 494, /* (418) col_list_opt ::= NK_LP column_stream_def_list NK_RP */ + 498, /* (419) column_stream_def_list ::= column_stream_def */ + 498, /* (420) column_stream_def_list ::= column_stream_def_list NK_COMMA column_stream_def */ + 499, /* (421) column_stream_def ::= column_name stream_col_options */ + 500, /* (422) stream_col_options ::= */ + 500, /* (423) stream_col_options ::= stream_col_options PRIMARY KEY */ + 495, /* (424) tag_def_or_ref_opt ::= */ + 495, /* (425) tag_def_or_ref_opt ::= tags_def */ + 495, /* (426) tag_def_or_ref_opt ::= TAGS NK_LP column_stream_def_list NK_RP */ + 493, /* (427) stream_options ::= */ + 493, /* (428) stream_options ::= stream_options TRIGGER AT_ONCE */ + 493, /* (429) stream_options ::= stream_options TRIGGER WINDOW_CLOSE */ + 493, /* (430) stream_options ::= stream_options TRIGGER FORCE_WINDOW_CLOSE */ + 493, /* (431) stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal */ + 493, /* (432) stream_options ::= stream_options WATERMARK duration_literal */ + 493, /* (433) stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER */ + 493, /* (434) stream_options ::= stream_options FILL_HISTORY NK_INTEGER */ + 493, /* (435) stream_options ::= stream_options DELETE_MARK duration_literal */ + 493, /* (436) stream_options ::= stream_options IGNORE UPDATE NK_INTEGER */ + 496, /* (437) subtable_opt ::= */ + 496, /* (438) subtable_opt ::= SUBTABLE NK_LP expression NK_RP */ + 497, /* (439) ignore_opt ::= */ + 497, /* (440) ignore_opt ::= IGNORE UNTREATED */ + 390, /* (441) cmd ::= KILL CONNECTION NK_INTEGER */ + 390, /* (442) cmd ::= KILL QUERY NK_STRING */ + 390, /* (443) cmd ::= KILL TRANSACTION NK_INTEGER */ + 390, /* (444) cmd ::= KILL COMPACT NK_INTEGER */ + 390, /* (445) cmd ::= BALANCE VGROUP */ + 390, /* (446) cmd ::= BALANCE VGROUP LEADER on_vgroup_id */ + 390, /* (447) cmd ::= BALANCE VGROUP LEADER DATABASE db_name */ + 390, /* (448) cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER */ + 390, /* (449) cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list */ + 390, /* (450) cmd ::= SPLIT VGROUP NK_INTEGER */ + 502, /* (451) on_vgroup_id ::= */ + 502, /* (452) on_vgroup_id ::= ON NK_INTEGER */ + 503, /* (453) dnode_list ::= DNODE NK_INTEGER */ + 503, /* (454) dnode_list ::= dnode_list DNODE NK_INTEGER */ + 390, /* (455) cmd ::= DELETE FROM full_table_name where_clause_opt */ + 390, /* (456) cmd ::= query_or_subquery */ + 390, /* (457) cmd ::= insert_query */ + 485, /* (458) insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery */ + 485, /* (459) insert_query ::= INSERT INTO full_table_name query_or_subquery */ + 441, /* (460) tags_literal ::= NK_INTEGER */ + 441, /* (461) tags_literal ::= NK_INTEGER NK_PLUS duration_literal */ + 441, /* (462) tags_literal ::= NK_INTEGER NK_MINUS duration_literal */ + 441, /* (463) tags_literal ::= NK_PLUS NK_INTEGER */ + 441, /* (464) tags_literal ::= NK_PLUS NK_INTEGER NK_PLUS duration_literal */ + 441, /* (465) tags_literal ::= NK_PLUS NK_INTEGER NK_MINUS duration_literal */ + 441, /* (466) tags_literal ::= NK_MINUS NK_INTEGER */ + 441, /* (467) tags_literal ::= NK_MINUS NK_INTEGER NK_PLUS duration_literal */ + 441, /* (468) tags_literal ::= NK_MINUS NK_INTEGER NK_MINUS duration_literal */ + 441, /* (469) tags_literal ::= NK_FLOAT */ + 441, /* (470) tags_literal ::= NK_PLUS NK_FLOAT */ + 441, /* (471) tags_literal ::= NK_MINUS NK_FLOAT */ + 441, /* (472) tags_literal ::= NK_BIN */ + 441, /* (473) tags_literal ::= NK_BIN NK_PLUS duration_literal */ + 441, /* (474) tags_literal ::= NK_BIN NK_MINUS duration_literal */ + 441, /* (475) tags_literal ::= NK_PLUS NK_BIN */ + 441, /* (476) tags_literal ::= NK_PLUS NK_BIN NK_PLUS duration_literal */ + 441, /* (477) tags_literal ::= NK_PLUS NK_BIN NK_MINUS duration_literal */ + 441, /* (478) tags_literal ::= NK_MINUS NK_BIN */ + 441, /* (479) tags_literal ::= NK_MINUS NK_BIN NK_PLUS duration_literal */ + 441, /* (480) tags_literal ::= NK_MINUS NK_BIN NK_MINUS duration_literal */ + 441, /* (481) tags_literal ::= NK_HEX */ + 441, /* (482) tags_literal ::= NK_HEX NK_PLUS duration_literal */ + 441, /* (483) tags_literal ::= NK_HEX NK_MINUS duration_literal */ + 441, /* (484) tags_literal ::= NK_PLUS NK_HEX */ + 441, /* (485) tags_literal ::= NK_PLUS NK_HEX NK_PLUS duration_literal */ + 441, /* (486) tags_literal ::= NK_PLUS NK_HEX NK_MINUS duration_literal */ + 441, /* (487) tags_literal ::= NK_MINUS NK_HEX */ + 441, /* (488) tags_literal ::= NK_MINUS NK_HEX NK_PLUS duration_literal */ + 441, /* (489) tags_literal ::= NK_MINUS NK_HEX NK_MINUS duration_literal */ + 441, /* (490) tags_literal ::= NK_STRING */ + 441, /* (491) tags_literal ::= NK_STRING NK_PLUS duration_literal */ + 441, /* (492) tags_literal ::= NK_STRING NK_MINUS duration_literal */ + 441, /* (493) tags_literal ::= NK_BOOL */ + 441, /* (494) tags_literal ::= NULL */ + 441, /* (495) tags_literal ::= literal_func */ + 441, /* (496) tags_literal ::= literal_func NK_PLUS duration_literal */ + 441, /* (497) tags_literal ::= literal_func NK_MINUS duration_literal */ + 444, /* (498) tags_literal_list ::= tags_literal */ + 444, /* (499) tags_literal_list ::= tags_literal_list NK_COMMA tags_literal */ + 393, /* (500) literal ::= NK_INTEGER */ + 393, /* (501) literal ::= NK_FLOAT */ + 393, /* (502) literal ::= NK_STRING */ + 393, /* (503) literal ::= NK_BOOL */ + 393, /* (504) literal ::= TIMESTAMP NK_STRING */ + 393, /* (505) literal ::= duration_literal */ + 393, /* (506) literal ::= NULL */ + 393, /* (507) literal ::= NK_QUESTION */ + 454, /* (508) duration_literal ::= NK_VARIABLE */ + 424, /* (509) signed ::= NK_INTEGER */ + 424, /* (510) signed ::= NK_PLUS NK_INTEGER */ + 424, /* (511) signed ::= NK_MINUS NK_INTEGER */ + 424, /* (512) signed ::= NK_FLOAT */ + 424, /* (513) signed ::= NK_PLUS NK_FLOAT */ + 424, /* (514) signed ::= NK_MINUS NK_FLOAT */ + 505, /* (515) signed_literal ::= signed */ + 505, /* (516) signed_literal ::= NK_STRING */ + 505, /* (517) signed_literal ::= NK_BOOL */ + 505, /* (518) signed_literal ::= TIMESTAMP NK_STRING */ + 505, /* (519) signed_literal ::= duration_literal */ + 505, /* (520) signed_literal ::= NULL */ + 505, /* (521) signed_literal ::= literal_func */ + 505, /* (522) signed_literal ::= NK_QUESTION */ + 506, /* (523) literal_list ::= signed_literal */ + 506, /* (524) literal_list ::= literal_list NK_COMMA signed_literal */ + 407, /* (525) db_name ::= NK_ID */ + 408, /* (526) table_name ::= NK_ID */ + 438, /* (527) column_name ::= NK_ID */ + 456, /* (528) function_name ::= NK_ID */ + 491, /* (529) view_name ::= NK_ID */ + 507, /* (530) table_alias ::= NK_ID */ + 466, /* (531) column_alias ::= NK_ID */ + 466, /* (532) column_alias ::= NK_ALIAS */ + 400, /* (533) user_name ::= NK_ID */ + 409, /* (534) topic_name ::= NK_ID */ + 492, /* (535) stream_name ::= NK_ID */ + 482, /* (536) cgroup_name ::= NK_ID */ + 473, /* (537) index_name ::= NK_ID */ + 467, /* (538) tsma_name ::= NK_ID */ + 508, /* (539) expr_or_subquery ::= expression */ + 501, /* (540) expression ::= literal */ + 501, /* (541) expression ::= pseudo_column */ + 501, /* (542) expression ::= column_reference */ + 501, /* (543) expression ::= function_expression */ + 501, /* (544) expression ::= case_when_expression */ + 501, /* (545) expression ::= NK_LP expression NK_RP */ + 501, /* (546) expression ::= NK_PLUS expr_or_subquery */ + 501, /* (547) expression ::= NK_MINUS expr_or_subquery */ + 501, /* (548) expression ::= expr_or_subquery NK_PLUS expr_or_subquery */ + 501, /* (549) expression ::= expr_or_subquery NK_MINUS expr_or_subquery */ + 501, /* (550) expression ::= expr_or_subquery NK_STAR expr_or_subquery */ + 501, /* (551) expression ::= expr_or_subquery NK_SLASH expr_or_subquery */ + 501, /* (552) expression ::= expr_or_subquery NK_REM expr_or_subquery */ + 501, /* (553) expression ::= column_reference NK_ARROW NK_STRING */ + 501, /* (554) expression ::= expr_or_subquery NK_BITAND expr_or_subquery */ + 501, /* (555) expression ::= expr_or_subquery NK_BITOR expr_or_subquery */ + 478, /* (556) expression_list ::= expr_or_subquery */ + 478, /* (557) expression_list ::= expression_list NK_COMMA expr_or_subquery */ + 510, /* (558) column_reference ::= column_name */ + 510, /* (559) column_reference ::= table_name NK_DOT column_name */ + 510, /* (560) column_reference ::= NK_ALIAS */ + 510, /* (561) column_reference ::= table_name NK_DOT NK_ALIAS */ + 509, /* (562) pseudo_column ::= ROWTS */ + 509, /* (563) pseudo_column ::= TBNAME */ + 509, /* (564) pseudo_column ::= table_name NK_DOT TBNAME */ + 509, /* (565) pseudo_column ::= QSTART */ + 509, /* (566) pseudo_column ::= QEND */ + 509, /* (567) pseudo_column ::= QDURATION */ + 509, /* (568) pseudo_column ::= WSTART */ + 509, /* (569) pseudo_column ::= WEND */ + 509, /* (570) pseudo_column ::= WDURATION */ + 509, /* (571) pseudo_column ::= IROWTS */ + 509, /* (572) pseudo_column ::= ISFILLED */ + 509, /* (573) pseudo_column ::= QTAGS */ + 509, /* (574) pseudo_column ::= FLOW */ + 509, /* (575) pseudo_column ::= FHIGH */ + 509, /* (576) pseudo_column ::= FROWTS */ + 511, /* (577) function_expression ::= function_name NK_LP expression_list NK_RP */ + 511, /* (578) function_expression ::= star_func NK_LP star_func_para_list NK_RP */ + 511, /* (579) function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP */ + 511, /* (580) function_expression ::= CAST NK_LP expr_or_subquery AS type_name_default_len NK_RP */ + 511, /* (581) function_expression ::= POSITION NK_LP expr_or_subquery IN expr_or_subquery NK_RP */ + 511, /* (582) function_expression ::= TRIM NK_LP expr_or_subquery NK_RP */ + 511, /* (583) function_expression ::= TRIM NK_LP trim_specification_type FROM expr_or_subquery NK_RP */ + 511, /* (584) function_expression ::= TRIM NK_LP expr_or_subquery FROM expr_or_subquery NK_RP */ + 511, /* (585) function_expression ::= TRIM NK_LP trim_specification_type expr_or_subquery FROM expr_or_subquery NK_RP */ + 511, /* (586) function_expression ::= substr_func NK_LP expression_list NK_RP */ + 511, /* (587) function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery NK_RP */ + 511, /* (588) function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery FOR expr_or_subquery NK_RP */ + 511, /* (589) function_expression ::= REPLACE NK_LP expression_list NK_RP */ + 511, /* (590) function_expression ::= literal_func */ + 511, /* (591) function_expression ::= rand_func */ + 504, /* (592) literal_func ::= noarg_func NK_LP NK_RP */ + 504, /* (593) literal_func ::= NOW */ + 504, /* (594) literal_func ::= TODAY */ + 517, /* (595) rand_func ::= RAND NK_LP NK_RP */ + 517, /* (596) rand_func ::= RAND NK_LP expression_list NK_RP */ + 516, /* (597) substr_func ::= SUBSTR */ + 516, /* (598) substr_func ::= SUBSTRING */ + 515, /* (599) trim_specification_type ::= BOTH */ + 515, /* (600) trim_specification_type ::= TRAILING */ + 515, /* (601) trim_specification_type ::= LEADING */ + 518, /* (602) noarg_func ::= NOW */ + 518, /* (603) noarg_func ::= TODAY */ + 518, /* (604) noarg_func ::= TIMEZONE */ + 518, /* (605) noarg_func ::= DATABASE */ + 518, /* (606) noarg_func ::= CLIENT_VERSION */ + 518, /* (607) noarg_func ::= SERVER_VERSION */ + 518, /* (608) noarg_func ::= SERVER_STATUS */ + 518, /* (609) noarg_func ::= CURRENT_USER */ + 518, /* (610) noarg_func ::= USER */ + 518, /* (611) noarg_func ::= PI */ + 513, /* (612) star_func ::= COUNT */ + 513, /* (613) star_func ::= FIRST */ + 513, /* (614) star_func ::= LAST */ + 513, /* (615) star_func ::= LAST_ROW */ + 514, /* (616) star_func_para_list ::= NK_STAR */ + 514, /* (617) star_func_para_list ::= other_para_list */ + 519, /* (618) other_para_list ::= star_func_para */ + 519, /* (619) other_para_list ::= other_para_list NK_COMMA star_func_para */ + 520, /* (620) star_func_para ::= expr_or_subquery */ + 520, /* (621) star_func_para ::= table_name NK_DOT NK_STAR */ + 512, /* (622) case_when_expression ::= CASE when_then_list case_when_else_opt END */ + 512, /* (623) case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END */ + 521, /* (624) when_then_list ::= when_then_expr */ + 521, /* (625) when_then_list ::= when_then_list when_then_expr */ + 524, /* (626) when_then_expr ::= WHEN common_expression THEN common_expression */ + 522, /* (627) case_when_else_opt ::= */ + 522, /* (628) case_when_else_opt ::= ELSE common_expression */ + 525, /* (629) predicate ::= expr_or_subquery compare_op expr_or_subquery */ + 525, /* (630) predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery */ + 525, /* (631) predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery */ + 525, /* (632) predicate ::= expr_or_subquery IS NULL */ + 525, /* (633) predicate ::= expr_or_subquery IS NOT NULL */ + 525, /* (634) predicate ::= expr_or_subquery in_op in_predicate_value */ + 526, /* (635) compare_op ::= NK_LT */ + 526, /* (636) compare_op ::= NK_GT */ + 526, /* (637) compare_op ::= NK_LE */ + 526, /* (638) compare_op ::= NK_GE */ + 526, /* (639) compare_op ::= NK_NE */ + 526, /* (640) compare_op ::= NK_EQ */ + 526, /* (641) compare_op ::= LIKE */ + 526, /* (642) compare_op ::= NOT LIKE */ + 526, /* (643) compare_op ::= MATCH */ + 526, /* (644) compare_op ::= NMATCH */ + 526, /* (645) compare_op ::= CONTAINS */ + 527, /* (646) in_op ::= IN */ + 527, /* (647) in_op ::= NOT IN */ + 528, /* (648) in_predicate_value ::= NK_LP literal_list NK_RP */ + 529, /* (649) boolean_value_expression ::= boolean_primary */ + 529, /* (650) boolean_value_expression ::= NOT boolean_primary */ + 529, /* (651) boolean_value_expression ::= boolean_value_expression OR boolean_value_expression */ + 529, /* (652) boolean_value_expression ::= boolean_value_expression AND boolean_value_expression */ + 530, /* (653) boolean_primary ::= predicate */ + 530, /* (654) boolean_primary ::= NK_LP boolean_value_expression NK_RP */ + 523, /* (655) common_expression ::= expr_or_subquery */ + 523, /* (656) common_expression ::= boolean_value_expression */ + 531, /* (657) from_clause_opt ::= */ + 531, /* (658) from_clause_opt ::= FROM table_reference_list */ + 532, /* (659) table_reference_list ::= table_reference */ + 532, /* (660) table_reference_list ::= table_reference_list NK_COMMA table_reference */ + 533, /* (661) table_reference ::= table_primary */ + 533, /* (662) table_reference ::= joined_table */ + 534, /* (663) table_primary ::= table_name alias_opt */ + 534, /* (664) table_primary ::= db_name NK_DOT table_name alias_opt */ + 534, /* (665) table_primary ::= subquery alias_opt */ + 534, /* (666) table_primary ::= parenthesized_joined_table */ + 536, /* (667) alias_opt ::= */ + 536, /* (668) alias_opt ::= table_alias */ + 536, /* (669) alias_opt ::= AS table_alias */ + 538, /* (670) parenthesized_joined_table ::= NK_LP joined_table NK_RP */ + 538, /* (671) parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP */ + 535, /* (672) joined_table ::= table_reference join_type join_subtype JOIN table_reference join_on_clause_opt window_offset_clause_opt jlimit_clause_opt */ + 539, /* (673) join_type ::= */ + 539, /* (674) join_type ::= INNER */ + 539, /* (675) join_type ::= LEFT */ + 539, /* (676) join_type ::= RIGHT */ + 539, /* (677) join_type ::= FULL */ + 540, /* (678) join_subtype ::= */ + 540, /* (679) join_subtype ::= OUTER */ + 540, /* (680) join_subtype ::= SEMI */ + 540, /* (681) join_subtype ::= ANTI */ + 540, /* (682) join_subtype ::= ASOF */ + 540, /* (683) join_subtype ::= WINDOW */ + 541, /* (684) join_on_clause_opt ::= */ + 541, /* (685) join_on_clause_opt ::= ON search_condition */ + 542, /* (686) window_offset_clause_opt ::= */ + 542, /* (687) window_offset_clause_opt ::= WINDOW_OFFSET NK_LP window_offset_literal NK_COMMA window_offset_literal NK_RP */ + 544, /* (688) window_offset_literal ::= NK_VARIABLE */ + 544, /* (689) window_offset_literal ::= NK_MINUS NK_VARIABLE */ + 543, /* (690) jlimit_clause_opt ::= */ + 543, /* (691) jlimit_clause_opt ::= JLIMIT NK_INTEGER */ + 545, /* (692) query_specification ::= SELECT hint_list set_quantifier_opt tag_mode_opt select_list from_clause_opt where_clause_opt partition_by_clause_opt range_opt every_opt fill_opt twindow_clause_opt group_by_clause_opt having_clause_opt */ + 546, /* (693) hint_list ::= */ + 546, /* (694) hint_list ::= NK_HINT */ + 548, /* (695) tag_mode_opt ::= */ + 548, /* (696) tag_mode_opt ::= TAGS */ + 547, /* (697) set_quantifier_opt ::= */ + 547, /* (698) set_quantifier_opt ::= DISTINCT */ + 547, /* (699) set_quantifier_opt ::= ALL */ + 549, /* (700) select_list ::= select_item */ + 549, /* (701) select_list ::= select_list NK_COMMA select_item */ + 557, /* (702) select_item ::= NK_STAR */ + 557, /* (703) select_item ::= common_expression */ + 557, /* (704) select_item ::= common_expression column_alias */ + 557, /* (705) select_item ::= common_expression AS column_alias */ + 557, /* (706) select_item ::= table_name NK_DOT NK_STAR */ + 481, /* (707) where_clause_opt ::= */ + 481, /* (708) where_clause_opt ::= WHERE search_condition */ + 550, /* (709) partition_by_clause_opt ::= */ + 550, /* (710) partition_by_clause_opt ::= PARTITION BY partition_list */ + 558, /* (711) partition_list ::= partition_item */ + 558, /* (712) partition_list ::= partition_list NK_COMMA partition_item */ + 559, /* (713) partition_item ::= expr_or_subquery */ + 559, /* (714) partition_item ::= expr_or_subquery column_alias */ + 559, /* (715) partition_item ::= expr_or_subquery AS column_alias */ + 554, /* (716) twindow_clause_opt ::= */ + 554, /* (717) twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA interval_sliding_duration_literal NK_RP */ + 554, /* (718) twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP */ + 554, /* (719) twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_RP sliding_opt fill_opt */ + 554, /* (720) twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_COMMA interval_sliding_duration_literal NK_RP sliding_opt fill_opt */ + 554, /* (721) twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition */ + 554, /* (722) twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_RP */ + 554, /* (723) twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP */ + 554, /* (724) twindow_clause_opt ::= ANOMALY_WINDOW NK_LP expr_or_subquery NK_RP */ + 554, /* (725) twindow_clause_opt ::= ANOMALY_WINDOW NK_LP expr_or_subquery NK_COMMA NK_STRING NK_RP */ + 474, /* (726) sliding_opt ::= */ + 474, /* (727) sliding_opt ::= SLIDING NK_LP interval_sliding_duration_literal NK_RP */ + 560, /* (728) interval_sliding_duration_literal ::= NK_VARIABLE */ + 560, /* (729) interval_sliding_duration_literal ::= NK_STRING */ + 560, /* (730) interval_sliding_duration_literal ::= NK_INTEGER */ + 553, /* (731) fill_opt ::= */ + 553, /* (732) fill_opt ::= FILL NK_LP fill_mode NK_RP */ + 553, /* (733) fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP */ + 553, /* (734) fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP */ + 561, /* (735) fill_mode ::= NONE */ + 561, /* (736) fill_mode ::= PREV */ + 561, /* (737) fill_mode ::= NULL */ + 561, /* (738) fill_mode ::= NULL_F */ + 561, /* (739) fill_mode ::= LINEAR */ + 561, /* (740) fill_mode ::= NEXT */ + 555, /* (741) group_by_clause_opt ::= */ + 555, /* (742) group_by_clause_opt ::= GROUP BY group_by_list */ + 562, /* (743) group_by_list ::= expr_or_subquery */ + 562, /* (744) group_by_list ::= group_by_list NK_COMMA expr_or_subquery */ + 556, /* (745) having_clause_opt ::= */ + 556, /* (746) having_clause_opt ::= HAVING search_condition */ + 551, /* (747) range_opt ::= */ + 551, /* (748) range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP */ + 551, /* (749) range_opt ::= RANGE NK_LP expr_or_subquery NK_RP */ + 552, /* (750) every_opt ::= */ + 552, /* (751) every_opt ::= EVERY NK_LP duration_literal NK_RP */ + 563, /* (752) query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt */ + 564, /* (753) query_simple ::= query_specification */ + 564, /* (754) query_simple ::= union_query_expression */ + 568, /* (755) union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery */ + 568, /* (756) union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery */ + 569, /* (757) query_simple_or_subquery ::= query_simple */ + 569, /* (758) query_simple_or_subquery ::= subquery */ + 480, /* (759) query_or_subquery ::= query_expression */ + 480, /* (760) query_or_subquery ::= subquery */ + 565, /* (761) order_by_clause_opt ::= */ + 565, /* (762) order_by_clause_opt ::= ORDER BY sort_specification_list */ + 566, /* (763) slimit_clause_opt ::= */ + 566, /* (764) slimit_clause_opt ::= SLIMIT NK_INTEGER */ + 566, /* (765) slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER */ + 566, /* (766) slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER */ + 567, /* (767) limit_clause_opt ::= */ + 567, /* (768) limit_clause_opt ::= LIMIT NK_INTEGER */ + 567, /* (769) limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER */ + 567, /* (770) limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER */ + 537, /* (771) subquery ::= NK_LP query_expression NK_RP */ + 537, /* (772) subquery ::= NK_LP subquery NK_RP */ + 410, /* (773) search_condition ::= common_expression */ + 570, /* (774) sort_specification_list ::= sort_specification */ + 570, /* (775) sort_specification_list ::= sort_specification_list NK_COMMA sort_specification */ + 571, /* (776) sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt */ + 572, /* (777) ordering_specification_opt ::= */ + 572, /* (778) ordering_specification_opt ::= ASC */ + 572, /* (779) ordering_specification_opt ::= DESC */ + 573, /* (780) null_ordering_opt ::= */ + 573, /* (781) null_ordering_opt ::= NULLS FIRST */ + 573, /* (782) null_ordering_opt ::= NULLS LAST */ + 440, /* (783) column_options ::= */ + 440, /* (784) column_options ::= column_options PRIMARY KEY */ + 440, /* (785) column_options ::= column_options NK_ID NK_STRING */ }; /* For rule J, yyRuleInfoNRhs[J] contains the negative of the number @@ -5176,361 +5617,362 @@ static const signed char yyRuleInfoNRhs[] = { 0, /* (427) stream_options ::= */ -3, /* (428) stream_options ::= stream_options TRIGGER AT_ONCE */ -3, /* (429) stream_options ::= stream_options TRIGGER WINDOW_CLOSE */ - -4, /* (430) stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal */ - -3, /* (431) stream_options ::= stream_options WATERMARK duration_literal */ - -4, /* (432) stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER */ - -3, /* (433) stream_options ::= stream_options FILL_HISTORY NK_INTEGER */ - -3, /* (434) stream_options ::= stream_options DELETE_MARK duration_literal */ - -4, /* (435) stream_options ::= stream_options IGNORE UPDATE NK_INTEGER */ - 0, /* (436) subtable_opt ::= */ - -4, /* (437) subtable_opt ::= SUBTABLE NK_LP expression NK_RP */ - 0, /* (438) ignore_opt ::= */ - -2, /* (439) ignore_opt ::= IGNORE UNTREATED */ - -3, /* (440) cmd ::= KILL CONNECTION NK_INTEGER */ - -3, /* (441) cmd ::= KILL QUERY NK_STRING */ - -3, /* (442) cmd ::= KILL TRANSACTION NK_INTEGER */ - -3, /* (443) cmd ::= KILL COMPACT NK_INTEGER */ - -2, /* (444) cmd ::= BALANCE VGROUP */ - -4, /* (445) cmd ::= BALANCE VGROUP LEADER on_vgroup_id */ - -5, /* (446) cmd ::= BALANCE VGROUP LEADER DATABASE db_name */ - -4, /* (447) cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER */ - -4, /* (448) cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list */ - -3, /* (449) cmd ::= SPLIT VGROUP NK_INTEGER */ - 0, /* (450) on_vgroup_id ::= */ - -2, /* (451) on_vgroup_id ::= ON NK_INTEGER */ - -2, /* (452) dnode_list ::= DNODE NK_INTEGER */ - -3, /* (453) dnode_list ::= dnode_list DNODE NK_INTEGER */ - -4, /* (454) cmd ::= DELETE FROM full_table_name where_clause_opt */ - -1, /* (455) cmd ::= query_or_subquery */ - -1, /* (456) cmd ::= insert_query */ - -7, /* (457) insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery */ - -4, /* (458) insert_query ::= INSERT INTO full_table_name query_or_subquery */ - -1, /* (459) tags_literal ::= NK_INTEGER */ - -3, /* (460) tags_literal ::= NK_INTEGER NK_PLUS duration_literal */ - -3, /* (461) tags_literal ::= NK_INTEGER NK_MINUS duration_literal */ - -2, /* (462) tags_literal ::= NK_PLUS NK_INTEGER */ - -4, /* (463) tags_literal ::= NK_PLUS NK_INTEGER NK_PLUS duration_literal */ - -4, /* (464) tags_literal ::= NK_PLUS NK_INTEGER NK_MINUS duration_literal */ - -2, /* (465) tags_literal ::= NK_MINUS NK_INTEGER */ - -4, /* (466) tags_literal ::= NK_MINUS NK_INTEGER NK_PLUS duration_literal */ - -4, /* (467) tags_literal ::= NK_MINUS NK_INTEGER NK_MINUS duration_literal */ - -1, /* (468) tags_literal ::= NK_FLOAT */ - -2, /* (469) tags_literal ::= NK_PLUS NK_FLOAT */ - -2, /* (470) tags_literal ::= NK_MINUS NK_FLOAT */ - -1, /* (471) tags_literal ::= NK_BIN */ - -3, /* (472) tags_literal ::= NK_BIN NK_PLUS duration_literal */ - -3, /* (473) tags_literal ::= NK_BIN NK_MINUS duration_literal */ - -2, /* (474) tags_literal ::= NK_PLUS NK_BIN */ - -4, /* (475) tags_literal ::= NK_PLUS NK_BIN NK_PLUS duration_literal */ - -4, /* (476) tags_literal ::= NK_PLUS NK_BIN NK_MINUS duration_literal */ - -2, /* (477) tags_literal ::= NK_MINUS NK_BIN */ - -4, /* (478) tags_literal ::= NK_MINUS NK_BIN NK_PLUS duration_literal */ - -4, /* (479) tags_literal ::= NK_MINUS NK_BIN NK_MINUS duration_literal */ - -1, /* (480) tags_literal ::= NK_HEX */ - -3, /* (481) tags_literal ::= NK_HEX NK_PLUS duration_literal */ - -3, /* (482) tags_literal ::= NK_HEX NK_MINUS duration_literal */ - -2, /* (483) tags_literal ::= NK_PLUS NK_HEX */ - -4, /* (484) tags_literal ::= NK_PLUS NK_HEX NK_PLUS duration_literal */ - -4, /* (485) tags_literal ::= NK_PLUS NK_HEX NK_MINUS duration_literal */ - -2, /* (486) tags_literal ::= NK_MINUS NK_HEX */ - -4, /* (487) tags_literal ::= NK_MINUS NK_HEX NK_PLUS duration_literal */ - -4, /* (488) tags_literal ::= NK_MINUS NK_HEX NK_MINUS duration_literal */ - -1, /* (489) tags_literal ::= NK_STRING */ - -3, /* (490) tags_literal ::= NK_STRING NK_PLUS duration_literal */ - -3, /* (491) tags_literal ::= NK_STRING NK_MINUS duration_literal */ - -1, /* (492) tags_literal ::= NK_BOOL */ - -1, /* (493) tags_literal ::= NULL */ - -1, /* (494) tags_literal ::= literal_func */ - -3, /* (495) tags_literal ::= literal_func NK_PLUS duration_literal */ - -3, /* (496) tags_literal ::= literal_func NK_MINUS duration_literal */ - -1, /* (497) tags_literal_list ::= tags_literal */ - -3, /* (498) tags_literal_list ::= tags_literal_list NK_COMMA tags_literal */ - -1, /* (499) literal ::= NK_INTEGER */ - -1, /* (500) literal ::= NK_FLOAT */ - -1, /* (501) literal ::= NK_STRING */ - -1, /* (502) literal ::= NK_BOOL */ - -2, /* (503) literal ::= TIMESTAMP NK_STRING */ - -1, /* (504) literal ::= duration_literal */ - -1, /* (505) literal ::= NULL */ - -1, /* (506) literal ::= NK_QUESTION */ - -1, /* (507) duration_literal ::= NK_VARIABLE */ - -1, /* (508) signed ::= NK_INTEGER */ - -2, /* (509) signed ::= NK_PLUS NK_INTEGER */ - -2, /* (510) signed ::= NK_MINUS NK_INTEGER */ - -1, /* (511) signed ::= NK_FLOAT */ - -2, /* (512) signed ::= NK_PLUS NK_FLOAT */ - -2, /* (513) signed ::= NK_MINUS NK_FLOAT */ - -1, /* (514) signed_literal ::= signed */ - -1, /* (515) signed_literal ::= NK_STRING */ - -1, /* (516) signed_literal ::= NK_BOOL */ - -2, /* (517) signed_literal ::= TIMESTAMP NK_STRING */ - -1, /* (518) signed_literal ::= duration_literal */ - -1, /* (519) signed_literal ::= NULL */ - -1, /* (520) signed_literal ::= literal_func */ - -1, /* (521) signed_literal ::= NK_QUESTION */ - -1, /* (522) literal_list ::= signed_literal */ - -3, /* (523) literal_list ::= literal_list NK_COMMA signed_literal */ - -1, /* (524) db_name ::= NK_ID */ - -1, /* (525) table_name ::= NK_ID */ - -1, /* (526) column_name ::= NK_ID */ - -1, /* (527) function_name ::= NK_ID */ - -1, /* (528) view_name ::= NK_ID */ - -1, /* (529) table_alias ::= NK_ID */ - -1, /* (530) column_alias ::= NK_ID */ - -1, /* (531) column_alias ::= NK_ALIAS */ - -1, /* (532) user_name ::= NK_ID */ - -1, /* (533) topic_name ::= NK_ID */ - -1, /* (534) stream_name ::= NK_ID */ - -1, /* (535) cgroup_name ::= NK_ID */ - -1, /* (536) index_name ::= NK_ID */ - -1, /* (537) tsma_name ::= NK_ID */ - -1, /* (538) expr_or_subquery ::= expression */ - -1, /* (539) expression ::= literal */ - -1, /* (540) expression ::= pseudo_column */ - -1, /* (541) expression ::= column_reference */ - -1, /* (542) expression ::= function_expression */ - -1, /* (543) expression ::= case_when_expression */ - -3, /* (544) expression ::= NK_LP expression NK_RP */ - -2, /* (545) expression ::= NK_PLUS expr_or_subquery */ - -2, /* (546) expression ::= NK_MINUS expr_or_subquery */ - -3, /* (547) expression ::= expr_or_subquery NK_PLUS expr_or_subquery */ - -3, /* (548) expression ::= expr_or_subquery NK_MINUS expr_or_subquery */ - -3, /* (549) expression ::= expr_or_subquery NK_STAR expr_or_subquery */ - -3, /* (550) expression ::= expr_or_subquery NK_SLASH expr_or_subquery */ - -3, /* (551) expression ::= expr_or_subquery NK_REM expr_or_subquery */ - -3, /* (552) expression ::= column_reference NK_ARROW NK_STRING */ - -3, /* (553) expression ::= expr_or_subquery NK_BITAND expr_or_subquery */ - -3, /* (554) expression ::= expr_or_subquery NK_BITOR expr_or_subquery */ - -1, /* (555) expression_list ::= expr_or_subquery */ - -3, /* (556) expression_list ::= expression_list NK_COMMA expr_or_subquery */ - -1, /* (557) column_reference ::= column_name */ - -3, /* (558) column_reference ::= table_name NK_DOT column_name */ - -1, /* (559) column_reference ::= NK_ALIAS */ - -3, /* (560) column_reference ::= table_name NK_DOT NK_ALIAS */ - -1, /* (561) pseudo_column ::= ROWTS */ - -1, /* (562) pseudo_column ::= TBNAME */ - -3, /* (563) pseudo_column ::= table_name NK_DOT TBNAME */ - -1, /* (564) pseudo_column ::= QSTART */ - -1, /* (565) pseudo_column ::= QEND */ - -1, /* (566) pseudo_column ::= QDURATION */ - -1, /* (567) pseudo_column ::= WSTART */ - -1, /* (568) pseudo_column ::= WEND */ - -1, /* (569) pseudo_column ::= WDURATION */ - -1, /* (570) pseudo_column ::= IROWTS */ - -1, /* (571) pseudo_column ::= ISFILLED */ - -1, /* (572) pseudo_column ::= QTAGS */ - -1, /* (573) pseudo_column ::= FLOW */ - -1, /* (574) pseudo_column ::= FHIGH */ - -1, /* (575) pseudo_column ::= FROWTS */ - -4, /* (576) function_expression ::= function_name NK_LP expression_list NK_RP */ - -4, /* (577) function_expression ::= star_func NK_LP star_func_para_list NK_RP */ - -6, /* (578) function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP */ - -6, /* (579) function_expression ::= CAST NK_LP expr_or_subquery AS type_name_default_len NK_RP */ - -6, /* (580) function_expression ::= POSITION NK_LP expr_or_subquery IN expr_or_subquery NK_RP */ - -4, /* (581) function_expression ::= TRIM NK_LP expr_or_subquery NK_RP */ - -6, /* (582) function_expression ::= TRIM NK_LP trim_specification_type FROM expr_or_subquery NK_RP */ - -6, /* (583) function_expression ::= TRIM NK_LP expr_or_subquery FROM expr_or_subquery NK_RP */ - -7, /* (584) function_expression ::= TRIM NK_LP trim_specification_type expr_or_subquery FROM expr_or_subquery NK_RP */ - -4, /* (585) function_expression ::= substr_func NK_LP expression_list NK_RP */ - -6, /* (586) function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery NK_RP */ - -8, /* (587) function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery FOR expr_or_subquery NK_RP */ - -4, /* (588) function_expression ::= REPLACE NK_LP expression_list NK_RP */ - -1, /* (589) function_expression ::= literal_func */ - -1, /* (590) function_expression ::= rand_func */ - -3, /* (591) literal_func ::= noarg_func NK_LP NK_RP */ - -1, /* (592) literal_func ::= NOW */ - -1, /* (593) literal_func ::= TODAY */ - -3, /* (594) rand_func ::= RAND NK_LP NK_RP */ - -4, /* (595) rand_func ::= RAND NK_LP expression_list NK_RP */ - -1, /* (596) substr_func ::= SUBSTR */ - -1, /* (597) substr_func ::= SUBSTRING */ - -1, /* (598) trim_specification_type ::= BOTH */ - -1, /* (599) trim_specification_type ::= TRAILING */ - -1, /* (600) trim_specification_type ::= LEADING */ - -1, /* (601) noarg_func ::= NOW */ - -1, /* (602) noarg_func ::= TODAY */ - -1, /* (603) noarg_func ::= TIMEZONE */ - -1, /* (604) noarg_func ::= DATABASE */ - -1, /* (605) noarg_func ::= CLIENT_VERSION */ - -1, /* (606) noarg_func ::= SERVER_VERSION */ - -1, /* (607) noarg_func ::= SERVER_STATUS */ - -1, /* (608) noarg_func ::= CURRENT_USER */ - -1, /* (609) noarg_func ::= USER */ - -1, /* (610) noarg_func ::= PI */ - -1, /* (611) star_func ::= COUNT */ - -1, /* (612) star_func ::= FIRST */ - -1, /* (613) star_func ::= LAST */ - -1, /* (614) star_func ::= LAST_ROW */ - -1, /* (615) star_func_para_list ::= NK_STAR */ - -1, /* (616) star_func_para_list ::= other_para_list */ - -1, /* (617) other_para_list ::= star_func_para */ - -3, /* (618) other_para_list ::= other_para_list NK_COMMA star_func_para */ - -1, /* (619) star_func_para ::= expr_or_subquery */ - -3, /* (620) star_func_para ::= table_name NK_DOT NK_STAR */ - -4, /* (621) case_when_expression ::= CASE when_then_list case_when_else_opt END */ - -5, /* (622) case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END */ - -1, /* (623) when_then_list ::= when_then_expr */ - -2, /* (624) when_then_list ::= when_then_list when_then_expr */ - -4, /* (625) when_then_expr ::= WHEN common_expression THEN common_expression */ - 0, /* (626) case_when_else_opt ::= */ - -2, /* (627) case_when_else_opt ::= ELSE common_expression */ - -3, /* (628) predicate ::= expr_or_subquery compare_op expr_or_subquery */ - -5, /* (629) predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery */ - -6, /* (630) predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery */ - -3, /* (631) predicate ::= expr_or_subquery IS NULL */ - -4, /* (632) predicate ::= expr_or_subquery IS NOT NULL */ - -3, /* (633) predicate ::= expr_or_subquery in_op in_predicate_value */ - -1, /* (634) compare_op ::= NK_LT */ - -1, /* (635) compare_op ::= NK_GT */ - -1, /* (636) compare_op ::= NK_LE */ - -1, /* (637) compare_op ::= NK_GE */ - -1, /* (638) compare_op ::= NK_NE */ - -1, /* (639) compare_op ::= NK_EQ */ - -1, /* (640) compare_op ::= LIKE */ - -2, /* (641) compare_op ::= NOT LIKE */ - -1, /* (642) compare_op ::= MATCH */ - -1, /* (643) compare_op ::= NMATCH */ - -1, /* (644) compare_op ::= CONTAINS */ - -1, /* (645) in_op ::= IN */ - -2, /* (646) in_op ::= NOT IN */ - -3, /* (647) in_predicate_value ::= NK_LP literal_list NK_RP */ - -1, /* (648) boolean_value_expression ::= boolean_primary */ - -2, /* (649) boolean_value_expression ::= NOT boolean_primary */ - -3, /* (650) boolean_value_expression ::= boolean_value_expression OR boolean_value_expression */ - -3, /* (651) boolean_value_expression ::= boolean_value_expression AND boolean_value_expression */ - -1, /* (652) boolean_primary ::= predicate */ - -3, /* (653) boolean_primary ::= NK_LP boolean_value_expression NK_RP */ - -1, /* (654) common_expression ::= expr_or_subquery */ - -1, /* (655) common_expression ::= boolean_value_expression */ - 0, /* (656) from_clause_opt ::= */ - -2, /* (657) from_clause_opt ::= FROM table_reference_list */ - -1, /* (658) table_reference_list ::= table_reference */ - -3, /* (659) table_reference_list ::= table_reference_list NK_COMMA table_reference */ - -1, /* (660) table_reference ::= table_primary */ - -1, /* (661) table_reference ::= joined_table */ - -2, /* (662) table_primary ::= table_name alias_opt */ - -4, /* (663) table_primary ::= db_name NK_DOT table_name alias_opt */ - -2, /* (664) table_primary ::= subquery alias_opt */ - -1, /* (665) table_primary ::= parenthesized_joined_table */ - 0, /* (666) alias_opt ::= */ - -1, /* (667) alias_opt ::= table_alias */ - -2, /* (668) alias_opt ::= AS table_alias */ - -3, /* (669) parenthesized_joined_table ::= NK_LP joined_table NK_RP */ - -3, /* (670) parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP */ - -8, /* (671) joined_table ::= table_reference join_type join_subtype JOIN table_reference join_on_clause_opt window_offset_clause_opt jlimit_clause_opt */ - 0, /* (672) join_type ::= */ - -1, /* (673) join_type ::= INNER */ - -1, /* (674) join_type ::= LEFT */ - -1, /* (675) join_type ::= RIGHT */ - -1, /* (676) join_type ::= FULL */ - 0, /* (677) join_subtype ::= */ - -1, /* (678) join_subtype ::= OUTER */ - -1, /* (679) join_subtype ::= SEMI */ - -1, /* (680) join_subtype ::= ANTI */ - -1, /* (681) join_subtype ::= ASOF */ - -1, /* (682) join_subtype ::= WINDOW */ - 0, /* (683) join_on_clause_opt ::= */ - -2, /* (684) join_on_clause_opt ::= ON search_condition */ - 0, /* (685) window_offset_clause_opt ::= */ - -6, /* (686) window_offset_clause_opt ::= WINDOW_OFFSET NK_LP window_offset_literal NK_COMMA window_offset_literal NK_RP */ - -1, /* (687) window_offset_literal ::= NK_VARIABLE */ - -2, /* (688) window_offset_literal ::= NK_MINUS NK_VARIABLE */ - 0, /* (689) jlimit_clause_opt ::= */ - -2, /* (690) jlimit_clause_opt ::= JLIMIT NK_INTEGER */ - -14, /* (691) query_specification ::= SELECT hint_list set_quantifier_opt tag_mode_opt select_list from_clause_opt where_clause_opt partition_by_clause_opt range_opt every_opt fill_opt twindow_clause_opt group_by_clause_opt having_clause_opt */ - 0, /* (692) hint_list ::= */ - -1, /* (693) hint_list ::= NK_HINT */ - 0, /* (694) tag_mode_opt ::= */ - -1, /* (695) tag_mode_opt ::= TAGS */ - 0, /* (696) set_quantifier_opt ::= */ - -1, /* (697) set_quantifier_opt ::= DISTINCT */ - -1, /* (698) set_quantifier_opt ::= ALL */ - -1, /* (699) select_list ::= select_item */ - -3, /* (700) select_list ::= select_list NK_COMMA select_item */ - -1, /* (701) select_item ::= NK_STAR */ - -1, /* (702) select_item ::= common_expression */ - -2, /* (703) select_item ::= common_expression column_alias */ - -3, /* (704) select_item ::= common_expression AS column_alias */ - -3, /* (705) select_item ::= table_name NK_DOT NK_STAR */ - 0, /* (706) where_clause_opt ::= */ - -2, /* (707) where_clause_opt ::= WHERE search_condition */ - 0, /* (708) partition_by_clause_opt ::= */ - -3, /* (709) partition_by_clause_opt ::= PARTITION BY partition_list */ - -1, /* (710) partition_list ::= partition_item */ - -3, /* (711) partition_list ::= partition_list NK_COMMA partition_item */ - -1, /* (712) partition_item ::= expr_or_subquery */ - -2, /* (713) partition_item ::= expr_or_subquery column_alias */ - -3, /* (714) partition_item ::= expr_or_subquery AS column_alias */ - 0, /* (715) twindow_clause_opt ::= */ - -6, /* (716) twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA interval_sliding_duration_literal NK_RP */ - -4, /* (717) twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP */ - -6, /* (718) twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_RP sliding_opt fill_opt */ - -8, /* (719) twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_COMMA interval_sliding_duration_literal NK_RP sliding_opt fill_opt */ - -7, /* (720) twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition */ - -4, /* (721) twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_RP */ - -6, /* (722) twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP */ - -4, /* (723) twindow_clause_opt ::= ANOMALY_WINDOW NK_LP expr_or_subquery NK_RP */ - -6, /* (724) twindow_clause_opt ::= ANOMALY_WINDOW NK_LP expr_or_subquery NK_COMMA NK_STRING NK_RP */ - 0, /* (725) sliding_opt ::= */ - -4, /* (726) sliding_opt ::= SLIDING NK_LP interval_sliding_duration_literal NK_RP */ - -1, /* (727) interval_sliding_duration_literal ::= NK_VARIABLE */ - -1, /* (728) interval_sliding_duration_literal ::= NK_STRING */ - -1, /* (729) interval_sliding_duration_literal ::= NK_INTEGER */ - 0, /* (730) fill_opt ::= */ - -4, /* (731) fill_opt ::= FILL NK_LP fill_mode NK_RP */ - -6, /* (732) fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP */ - -6, /* (733) fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP */ - -1, /* (734) fill_mode ::= NONE */ - -1, /* (735) fill_mode ::= PREV */ - -1, /* (736) fill_mode ::= NULL */ - -1, /* (737) fill_mode ::= NULL_F */ - -1, /* (738) fill_mode ::= LINEAR */ - -1, /* (739) fill_mode ::= NEXT */ - 0, /* (740) group_by_clause_opt ::= */ - -3, /* (741) group_by_clause_opt ::= GROUP BY group_by_list */ - -1, /* (742) group_by_list ::= expr_or_subquery */ - -3, /* (743) group_by_list ::= group_by_list NK_COMMA expr_or_subquery */ - 0, /* (744) having_clause_opt ::= */ - -2, /* (745) having_clause_opt ::= HAVING search_condition */ - 0, /* (746) range_opt ::= */ - -6, /* (747) range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP */ - -4, /* (748) range_opt ::= RANGE NK_LP expr_or_subquery NK_RP */ - 0, /* (749) every_opt ::= */ - -4, /* (750) every_opt ::= EVERY NK_LP duration_literal NK_RP */ - -4, /* (751) query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt */ - -1, /* (752) query_simple ::= query_specification */ - -1, /* (753) query_simple ::= union_query_expression */ - -4, /* (754) union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery */ - -3, /* (755) union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery */ - -1, /* (756) query_simple_or_subquery ::= query_simple */ - -1, /* (757) query_simple_or_subquery ::= subquery */ - -1, /* (758) query_or_subquery ::= query_expression */ - -1, /* (759) query_or_subquery ::= subquery */ - 0, /* (760) order_by_clause_opt ::= */ - -3, /* (761) order_by_clause_opt ::= ORDER BY sort_specification_list */ - 0, /* (762) slimit_clause_opt ::= */ - -2, /* (763) slimit_clause_opt ::= SLIMIT NK_INTEGER */ - -4, /* (764) slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER */ - -4, /* (765) slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER */ - 0, /* (766) limit_clause_opt ::= */ - -2, /* (767) limit_clause_opt ::= LIMIT NK_INTEGER */ - -4, /* (768) limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER */ - -4, /* (769) limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER */ - -3, /* (770) subquery ::= NK_LP query_expression NK_RP */ - -3, /* (771) subquery ::= NK_LP subquery NK_RP */ - -1, /* (772) search_condition ::= common_expression */ - -1, /* (773) sort_specification_list ::= sort_specification */ - -3, /* (774) sort_specification_list ::= sort_specification_list NK_COMMA sort_specification */ - -3, /* (775) sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt */ - 0, /* (776) ordering_specification_opt ::= */ - -1, /* (777) ordering_specification_opt ::= ASC */ - -1, /* (778) ordering_specification_opt ::= DESC */ - 0, /* (779) null_ordering_opt ::= */ - -2, /* (780) null_ordering_opt ::= NULLS FIRST */ - -2, /* (781) null_ordering_opt ::= NULLS LAST */ - 0, /* (782) column_options ::= */ - -3, /* (783) column_options ::= column_options PRIMARY KEY */ - -3, /* (784) column_options ::= column_options NK_ID NK_STRING */ + -3, /* (430) stream_options ::= stream_options TRIGGER FORCE_WINDOW_CLOSE */ + -4, /* (431) stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal */ + -3, /* (432) stream_options ::= stream_options WATERMARK duration_literal */ + -4, /* (433) stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER */ + -3, /* (434) stream_options ::= stream_options FILL_HISTORY NK_INTEGER */ + -3, /* (435) stream_options ::= stream_options DELETE_MARK duration_literal */ + -4, /* (436) stream_options ::= stream_options IGNORE UPDATE NK_INTEGER */ + 0, /* (437) subtable_opt ::= */ + -4, /* (438) subtable_opt ::= SUBTABLE NK_LP expression NK_RP */ + 0, /* (439) ignore_opt ::= */ + -2, /* (440) ignore_opt ::= IGNORE UNTREATED */ + -3, /* (441) cmd ::= KILL CONNECTION NK_INTEGER */ + -3, /* (442) cmd ::= KILL QUERY NK_STRING */ + -3, /* (443) cmd ::= KILL TRANSACTION NK_INTEGER */ + -3, /* (444) cmd ::= KILL COMPACT NK_INTEGER */ + -2, /* (445) cmd ::= BALANCE VGROUP */ + -4, /* (446) cmd ::= BALANCE VGROUP LEADER on_vgroup_id */ + -5, /* (447) cmd ::= BALANCE VGROUP LEADER DATABASE db_name */ + -4, /* (448) cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER */ + -4, /* (449) cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list */ + -3, /* (450) cmd ::= SPLIT VGROUP NK_INTEGER */ + 0, /* (451) on_vgroup_id ::= */ + -2, /* (452) on_vgroup_id ::= ON NK_INTEGER */ + -2, /* (453) dnode_list ::= DNODE NK_INTEGER */ + -3, /* (454) dnode_list ::= dnode_list DNODE NK_INTEGER */ + -4, /* (455) cmd ::= DELETE FROM full_table_name where_clause_opt */ + -1, /* (456) cmd ::= query_or_subquery */ + -1, /* (457) cmd ::= insert_query */ + -7, /* (458) insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery */ + -4, /* (459) insert_query ::= INSERT INTO full_table_name query_or_subquery */ + -1, /* (460) tags_literal ::= NK_INTEGER */ + -3, /* (461) tags_literal ::= NK_INTEGER NK_PLUS duration_literal */ + -3, /* (462) tags_literal ::= NK_INTEGER NK_MINUS duration_literal */ + -2, /* (463) tags_literal ::= NK_PLUS NK_INTEGER */ + -4, /* (464) tags_literal ::= NK_PLUS NK_INTEGER NK_PLUS duration_literal */ + -4, /* (465) tags_literal ::= NK_PLUS NK_INTEGER NK_MINUS duration_literal */ + -2, /* (466) tags_literal ::= NK_MINUS NK_INTEGER */ + -4, /* (467) tags_literal ::= NK_MINUS NK_INTEGER NK_PLUS duration_literal */ + -4, /* (468) tags_literal ::= NK_MINUS NK_INTEGER NK_MINUS duration_literal */ + -1, /* (469) tags_literal ::= NK_FLOAT */ + -2, /* (470) tags_literal ::= NK_PLUS NK_FLOAT */ + -2, /* (471) tags_literal ::= NK_MINUS NK_FLOAT */ + -1, /* (472) tags_literal ::= NK_BIN */ + -3, /* (473) tags_literal ::= NK_BIN NK_PLUS duration_literal */ + -3, /* (474) tags_literal ::= NK_BIN NK_MINUS duration_literal */ + -2, /* (475) tags_literal ::= NK_PLUS NK_BIN */ + -4, /* (476) tags_literal ::= NK_PLUS NK_BIN NK_PLUS duration_literal */ + -4, /* (477) tags_literal ::= NK_PLUS NK_BIN NK_MINUS duration_literal */ + -2, /* (478) tags_literal ::= NK_MINUS NK_BIN */ + -4, /* (479) tags_literal ::= NK_MINUS NK_BIN NK_PLUS duration_literal */ + -4, /* (480) tags_literal ::= NK_MINUS NK_BIN NK_MINUS duration_literal */ + -1, /* (481) tags_literal ::= NK_HEX */ + -3, /* (482) tags_literal ::= NK_HEX NK_PLUS duration_literal */ + -3, /* (483) tags_literal ::= NK_HEX NK_MINUS duration_literal */ + -2, /* (484) tags_literal ::= NK_PLUS NK_HEX */ + -4, /* (485) tags_literal ::= NK_PLUS NK_HEX NK_PLUS duration_literal */ + -4, /* (486) tags_literal ::= NK_PLUS NK_HEX NK_MINUS duration_literal */ + -2, /* (487) tags_literal ::= NK_MINUS NK_HEX */ + -4, /* (488) tags_literal ::= NK_MINUS NK_HEX NK_PLUS duration_literal */ + -4, /* (489) tags_literal ::= NK_MINUS NK_HEX NK_MINUS duration_literal */ + -1, /* (490) tags_literal ::= NK_STRING */ + -3, /* (491) tags_literal ::= NK_STRING NK_PLUS duration_literal */ + -3, /* (492) tags_literal ::= NK_STRING NK_MINUS duration_literal */ + -1, /* (493) tags_literal ::= NK_BOOL */ + -1, /* (494) tags_literal ::= NULL */ + -1, /* (495) tags_literal ::= literal_func */ + -3, /* (496) tags_literal ::= literal_func NK_PLUS duration_literal */ + -3, /* (497) tags_literal ::= literal_func NK_MINUS duration_literal */ + -1, /* (498) tags_literal_list ::= tags_literal */ + -3, /* (499) tags_literal_list ::= tags_literal_list NK_COMMA tags_literal */ + -1, /* (500) literal ::= NK_INTEGER */ + -1, /* (501) literal ::= NK_FLOAT */ + -1, /* (502) literal ::= NK_STRING */ + -1, /* (503) literal ::= NK_BOOL */ + -2, /* (504) literal ::= TIMESTAMP NK_STRING */ + -1, /* (505) literal ::= duration_literal */ + -1, /* (506) literal ::= NULL */ + -1, /* (507) literal ::= NK_QUESTION */ + -1, /* (508) duration_literal ::= NK_VARIABLE */ + -1, /* (509) signed ::= NK_INTEGER */ + -2, /* (510) signed ::= NK_PLUS NK_INTEGER */ + -2, /* (511) signed ::= NK_MINUS NK_INTEGER */ + -1, /* (512) signed ::= NK_FLOAT */ + -2, /* (513) signed ::= NK_PLUS NK_FLOAT */ + -2, /* (514) signed ::= NK_MINUS NK_FLOAT */ + -1, /* (515) signed_literal ::= signed */ + -1, /* (516) signed_literal ::= NK_STRING */ + -1, /* (517) signed_literal ::= NK_BOOL */ + -2, /* (518) signed_literal ::= TIMESTAMP NK_STRING */ + -1, /* (519) signed_literal ::= duration_literal */ + -1, /* (520) signed_literal ::= NULL */ + -1, /* (521) signed_literal ::= literal_func */ + -1, /* (522) signed_literal ::= NK_QUESTION */ + -1, /* (523) literal_list ::= signed_literal */ + -3, /* (524) literal_list ::= literal_list NK_COMMA signed_literal */ + -1, /* (525) db_name ::= NK_ID */ + -1, /* (526) table_name ::= NK_ID */ + -1, /* (527) column_name ::= NK_ID */ + -1, /* (528) function_name ::= NK_ID */ + -1, /* (529) view_name ::= NK_ID */ + -1, /* (530) table_alias ::= NK_ID */ + -1, /* (531) column_alias ::= NK_ID */ + -1, /* (532) column_alias ::= NK_ALIAS */ + -1, /* (533) user_name ::= NK_ID */ + -1, /* (534) topic_name ::= NK_ID */ + -1, /* (535) stream_name ::= NK_ID */ + -1, /* (536) cgroup_name ::= NK_ID */ + -1, /* (537) index_name ::= NK_ID */ + -1, /* (538) tsma_name ::= NK_ID */ + -1, /* (539) expr_or_subquery ::= expression */ + -1, /* (540) expression ::= literal */ + -1, /* (541) expression ::= pseudo_column */ + -1, /* (542) expression ::= column_reference */ + -1, /* (543) expression ::= function_expression */ + -1, /* (544) expression ::= case_when_expression */ + -3, /* (545) expression ::= NK_LP expression NK_RP */ + -2, /* (546) expression ::= NK_PLUS expr_or_subquery */ + -2, /* (547) expression ::= NK_MINUS expr_or_subquery */ + -3, /* (548) expression ::= expr_or_subquery NK_PLUS expr_or_subquery */ + -3, /* (549) expression ::= expr_or_subquery NK_MINUS expr_or_subquery */ + -3, /* (550) expression ::= expr_or_subquery NK_STAR expr_or_subquery */ + -3, /* (551) expression ::= expr_or_subquery NK_SLASH expr_or_subquery */ + -3, /* (552) expression ::= expr_or_subquery NK_REM expr_or_subquery */ + -3, /* (553) expression ::= column_reference NK_ARROW NK_STRING */ + -3, /* (554) expression ::= expr_or_subquery NK_BITAND expr_or_subquery */ + -3, /* (555) expression ::= expr_or_subquery NK_BITOR expr_or_subquery */ + -1, /* (556) expression_list ::= expr_or_subquery */ + -3, /* (557) expression_list ::= expression_list NK_COMMA expr_or_subquery */ + -1, /* (558) column_reference ::= column_name */ + -3, /* (559) column_reference ::= table_name NK_DOT column_name */ + -1, /* (560) column_reference ::= NK_ALIAS */ + -3, /* (561) column_reference ::= table_name NK_DOT NK_ALIAS */ + -1, /* (562) pseudo_column ::= ROWTS */ + -1, /* (563) pseudo_column ::= TBNAME */ + -3, /* (564) pseudo_column ::= table_name NK_DOT TBNAME */ + -1, /* (565) pseudo_column ::= QSTART */ + -1, /* (566) pseudo_column ::= QEND */ + -1, /* (567) pseudo_column ::= QDURATION */ + -1, /* (568) pseudo_column ::= WSTART */ + -1, /* (569) pseudo_column ::= WEND */ + -1, /* (570) pseudo_column ::= WDURATION */ + -1, /* (571) pseudo_column ::= IROWTS */ + -1, /* (572) pseudo_column ::= ISFILLED */ + -1, /* (573) pseudo_column ::= QTAGS */ + -1, /* (574) pseudo_column ::= FLOW */ + -1, /* (575) pseudo_column ::= FHIGH */ + -1, /* (576) pseudo_column ::= FROWTS */ + -4, /* (577) function_expression ::= function_name NK_LP expression_list NK_RP */ + -4, /* (578) function_expression ::= star_func NK_LP star_func_para_list NK_RP */ + -6, /* (579) function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP */ + -6, /* (580) function_expression ::= CAST NK_LP expr_or_subquery AS type_name_default_len NK_RP */ + -6, /* (581) function_expression ::= POSITION NK_LP expr_or_subquery IN expr_or_subquery NK_RP */ + -4, /* (582) function_expression ::= TRIM NK_LP expr_or_subquery NK_RP */ + -6, /* (583) function_expression ::= TRIM NK_LP trim_specification_type FROM expr_or_subquery NK_RP */ + -6, /* (584) function_expression ::= TRIM NK_LP expr_or_subquery FROM expr_or_subquery NK_RP */ + -7, /* (585) function_expression ::= TRIM NK_LP trim_specification_type expr_or_subquery FROM expr_or_subquery NK_RP */ + -4, /* (586) function_expression ::= substr_func NK_LP expression_list NK_RP */ + -6, /* (587) function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery NK_RP */ + -8, /* (588) function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery FOR expr_or_subquery NK_RP */ + -4, /* (589) function_expression ::= REPLACE NK_LP expression_list NK_RP */ + -1, /* (590) function_expression ::= literal_func */ + -1, /* (591) function_expression ::= rand_func */ + -3, /* (592) literal_func ::= noarg_func NK_LP NK_RP */ + -1, /* (593) literal_func ::= NOW */ + -1, /* (594) literal_func ::= TODAY */ + -3, /* (595) rand_func ::= RAND NK_LP NK_RP */ + -4, /* (596) rand_func ::= RAND NK_LP expression_list NK_RP */ + -1, /* (597) substr_func ::= SUBSTR */ + -1, /* (598) substr_func ::= SUBSTRING */ + -1, /* (599) trim_specification_type ::= BOTH */ + -1, /* (600) trim_specification_type ::= TRAILING */ + -1, /* (601) trim_specification_type ::= LEADING */ + -1, /* (602) noarg_func ::= NOW */ + -1, /* (603) noarg_func ::= TODAY */ + -1, /* (604) noarg_func ::= TIMEZONE */ + -1, /* (605) noarg_func ::= DATABASE */ + -1, /* (606) noarg_func ::= CLIENT_VERSION */ + -1, /* (607) noarg_func ::= SERVER_VERSION */ + -1, /* (608) noarg_func ::= SERVER_STATUS */ + -1, /* (609) noarg_func ::= CURRENT_USER */ + -1, /* (610) noarg_func ::= USER */ + -1, /* (611) noarg_func ::= PI */ + -1, /* (612) star_func ::= COUNT */ + -1, /* (613) star_func ::= FIRST */ + -1, /* (614) star_func ::= LAST */ + -1, /* (615) star_func ::= LAST_ROW */ + -1, /* (616) star_func_para_list ::= NK_STAR */ + -1, /* (617) star_func_para_list ::= other_para_list */ + -1, /* (618) other_para_list ::= star_func_para */ + -3, /* (619) other_para_list ::= other_para_list NK_COMMA star_func_para */ + -1, /* (620) star_func_para ::= expr_or_subquery */ + -3, /* (621) star_func_para ::= table_name NK_DOT NK_STAR */ + -4, /* (622) case_when_expression ::= CASE when_then_list case_when_else_opt END */ + -5, /* (623) case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END */ + -1, /* (624) when_then_list ::= when_then_expr */ + -2, /* (625) when_then_list ::= when_then_list when_then_expr */ + -4, /* (626) when_then_expr ::= WHEN common_expression THEN common_expression */ + 0, /* (627) case_when_else_opt ::= */ + -2, /* (628) case_when_else_opt ::= ELSE common_expression */ + -3, /* (629) predicate ::= expr_or_subquery compare_op expr_or_subquery */ + -5, /* (630) predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery */ + -6, /* (631) predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery */ + -3, /* (632) predicate ::= expr_or_subquery IS NULL */ + -4, /* (633) predicate ::= expr_or_subquery IS NOT NULL */ + -3, /* (634) predicate ::= expr_or_subquery in_op in_predicate_value */ + -1, /* (635) compare_op ::= NK_LT */ + -1, /* (636) compare_op ::= NK_GT */ + -1, /* (637) compare_op ::= NK_LE */ + -1, /* (638) compare_op ::= NK_GE */ + -1, /* (639) compare_op ::= NK_NE */ + -1, /* (640) compare_op ::= NK_EQ */ + -1, /* (641) compare_op ::= LIKE */ + -2, /* (642) compare_op ::= NOT LIKE */ + -1, /* (643) compare_op ::= MATCH */ + -1, /* (644) compare_op ::= NMATCH */ + -1, /* (645) compare_op ::= CONTAINS */ + -1, /* (646) in_op ::= IN */ + -2, /* (647) in_op ::= NOT IN */ + -3, /* (648) in_predicate_value ::= NK_LP literal_list NK_RP */ + -1, /* (649) boolean_value_expression ::= boolean_primary */ + -2, /* (650) boolean_value_expression ::= NOT boolean_primary */ + -3, /* (651) boolean_value_expression ::= boolean_value_expression OR boolean_value_expression */ + -3, /* (652) boolean_value_expression ::= boolean_value_expression AND boolean_value_expression */ + -1, /* (653) boolean_primary ::= predicate */ + -3, /* (654) boolean_primary ::= NK_LP boolean_value_expression NK_RP */ + -1, /* (655) common_expression ::= expr_or_subquery */ + -1, /* (656) common_expression ::= boolean_value_expression */ + 0, /* (657) from_clause_opt ::= */ + -2, /* (658) from_clause_opt ::= FROM table_reference_list */ + -1, /* (659) table_reference_list ::= table_reference */ + -3, /* (660) table_reference_list ::= table_reference_list NK_COMMA table_reference */ + -1, /* (661) table_reference ::= table_primary */ + -1, /* (662) table_reference ::= joined_table */ + -2, /* (663) table_primary ::= table_name alias_opt */ + -4, /* (664) table_primary ::= db_name NK_DOT table_name alias_opt */ + -2, /* (665) table_primary ::= subquery alias_opt */ + -1, /* (666) table_primary ::= parenthesized_joined_table */ + 0, /* (667) alias_opt ::= */ + -1, /* (668) alias_opt ::= table_alias */ + -2, /* (669) alias_opt ::= AS table_alias */ + -3, /* (670) parenthesized_joined_table ::= NK_LP joined_table NK_RP */ + -3, /* (671) parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP */ + -8, /* (672) joined_table ::= table_reference join_type join_subtype JOIN table_reference join_on_clause_opt window_offset_clause_opt jlimit_clause_opt */ + 0, /* (673) join_type ::= */ + -1, /* (674) join_type ::= INNER */ + -1, /* (675) join_type ::= LEFT */ + -1, /* (676) join_type ::= RIGHT */ + -1, /* (677) join_type ::= FULL */ + 0, /* (678) join_subtype ::= */ + -1, /* (679) join_subtype ::= OUTER */ + -1, /* (680) join_subtype ::= SEMI */ + -1, /* (681) join_subtype ::= ANTI */ + -1, /* (682) join_subtype ::= ASOF */ + -1, /* (683) join_subtype ::= WINDOW */ + 0, /* (684) join_on_clause_opt ::= */ + -2, /* (685) join_on_clause_opt ::= ON search_condition */ + 0, /* (686) window_offset_clause_opt ::= */ + -6, /* (687) window_offset_clause_opt ::= WINDOW_OFFSET NK_LP window_offset_literal NK_COMMA window_offset_literal NK_RP */ + -1, /* (688) window_offset_literal ::= NK_VARIABLE */ + -2, /* (689) window_offset_literal ::= NK_MINUS NK_VARIABLE */ + 0, /* (690) jlimit_clause_opt ::= */ + -2, /* (691) jlimit_clause_opt ::= JLIMIT NK_INTEGER */ + -14, /* (692) query_specification ::= SELECT hint_list set_quantifier_opt tag_mode_opt select_list from_clause_opt where_clause_opt partition_by_clause_opt range_opt every_opt fill_opt twindow_clause_opt group_by_clause_opt having_clause_opt */ + 0, /* (693) hint_list ::= */ + -1, /* (694) hint_list ::= NK_HINT */ + 0, /* (695) tag_mode_opt ::= */ + -1, /* (696) tag_mode_opt ::= TAGS */ + 0, /* (697) set_quantifier_opt ::= */ + -1, /* (698) set_quantifier_opt ::= DISTINCT */ + -1, /* (699) set_quantifier_opt ::= ALL */ + -1, /* (700) select_list ::= select_item */ + -3, /* (701) select_list ::= select_list NK_COMMA select_item */ + -1, /* (702) select_item ::= NK_STAR */ + -1, /* (703) select_item ::= common_expression */ + -2, /* (704) select_item ::= common_expression column_alias */ + -3, /* (705) select_item ::= common_expression AS column_alias */ + -3, /* (706) select_item ::= table_name NK_DOT NK_STAR */ + 0, /* (707) where_clause_opt ::= */ + -2, /* (708) where_clause_opt ::= WHERE search_condition */ + 0, /* (709) partition_by_clause_opt ::= */ + -3, /* (710) partition_by_clause_opt ::= PARTITION BY partition_list */ + -1, /* (711) partition_list ::= partition_item */ + -3, /* (712) partition_list ::= partition_list NK_COMMA partition_item */ + -1, /* (713) partition_item ::= expr_or_subquery */ + -2, /* (714) partition_item ::= expr_or_subquery column_alias */ + -3, /* (715) partition_item ::= expr_or_subquery AS column_alias */ + 0, /* (716) twindow_clause_opt ::= */ + -6, /* (717) twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA interval_sliding_duration_literal NK_RP */ + -4, /* (718) twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP */ + -6, /* (719) twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_RP sliding_opt fill_opt */ + -8, /* (720) twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_COMMA interval_sliding_duration_literal NK_RP sliding_opt fill_opt */ + -7, /* (721) twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition */ + -4, /* (722) twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_RP */ + -6, /* (723) twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP */ + -4, /* (724) twindow_clause_opt ::= ANOMALY_WINDOW NK_LP expr_or_subquery NK_RP */ + -6, /* (725) twindow_clause_opt ::= ANOMALY_WINDOW NK_LP expr_or_subquery NK_COMMA NK_STRING NK_RP */ + 0, /* (726) sliding_opt ::= */ + -4, /* (727) sliding_opt ::= SLIDING NK_LP interval_sliding_duration_literal NK_RP */ + -1, /* (728) interval_sliding_duration_literal ::= NK_VARIABLE */ + -1, /* (729) interval_sliding_duration_literal ::= NK_STRING */ + -1, /* (730) interval_sliding_duration_literal ::= NK_INTEGER */ + 0, /* (731) fill_opt ::= */ + -4, /* (732) fill_opt ::= FILL NK_LP fill_mode NK_RP */ + -6, /* (733) fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP */ + -6, /* (734) fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP */ + -1, /* (735) fill_mode ::= NONE */ + -1, /* (736) fill_mode ::= PREV */ + -1, /* (737) fill_mode ::= NULL */ + -1, /* (738) fill_mode ::= NULL_F */ + -1, /* (739) fill_mode ::= LINEAR */ + -1, /* (740) fill_mode ::= NEXT */ + 0, /* (741) group_by_clause_opt ::= */ + -3, /* (742) group_by_clause_opt ::= GROUP BY group_by_list */ + -1, /* (743) group_by_list ::= expr_or_subquery */ + -3, /* (744) group_by_list ::= group_by_list NK_COMMA expr_or_subquery */ + 0, /* (745) having_clause_opt ::= */ + -2, /* (746) having_clause_opt ::= HAVING search_condition */ + 0, /* (747) range_opt ::= */ + -6, /* (748) range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP */ + -4, /* (749) range_opt ::= RANGE NK_LP expr_or_subquery NK_RP */ + 0, /* (750) every_opt ::= */ + -4, /* (751) every_opt ::= EVERY NK_LP duration_literal NK_RP */ + -4, /* (752) query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt */ + -1, /* (753) query_simple ::= query_specification */ + -1, /* (754) query_simple ::= union_query_expression */ + -4, /* (755) union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery */ + -3, /* (756) union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery */ + -1, /* (757) query_simple_or_subquery ::= query_simple */ + -1, /* (758) query_simple_or_subquery ::= subquery */ + -1, /* (759) query_or_subquery ::= query_expression */ + -1, /* (760) query_or_subquery ::= subquery */ + 0, /* (761) order_by_clause_opt ::= */ + -3, /* (762) order_by_clause_opt ::= ORDER BY sort_specification_list */ + 0, /* (763) slimit_clause_opt ::= */ + -2, /* (764) slimit_clause_opt ::= SLIMIT NK_INTEGER */ + -4, /* (765) slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER */ + -4, /* (766) slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER */ + 0, /* (767) limit_clause_opt ::= */ + -2, /* (768) limit_clause_opt ::= LIMIT NK_INTEGER */ + -4, /* (769) limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER */ + -4, /* (770) limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER */ + -3, /* (771) subquery ::= NK_LP query_expression NK_RP */ + -3, /* (772) subquery ::= NK_LP subquery NK_RP */ + -1, /* (773) search_condition ::= common_expression */ + -1, /* (774) sort_specification_list ::= sort_specification */ + -3, /* (775) sort_specification_list ::= sort_specification_list NK_COMMA sort_specification */ + -3, /* (776) sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt */ + 0, /* (777) ordering_specification_opt ::= */ + -1, /* (778) ordering_specification_opt ::= ASC */ + -1, /* (779) ordering_specification_opt ::= DESC */ + 0, /* (780) null_ordering_opt ::= */ + -2, /* (781) null_ordering_opt ::= NULLS FIRST */ + -2, /* (782) null_ordering_opt ::= NULLS LAST */ + 0, /* (783) column_options ::= */ + -3, /* (784) column_options ::= column_options PRIMARY KEY */ + -3, /* (785) column_options ::= column_options NK_ID NK_STRING */ }; static void yy_accept(yyParser*); /* Forward Declaration */ @@ -5560,54 +6002,6 @@ static YYACTIONTYPE yy_reduce( (void)yyLookahead; (void)yyLookaheadToken; yymsp = yypParser->yytos; -#ifndef NDEBUG - if( yyTraceFILE && yyruleno<(int)(sizeof(yyRuleName)/sizeof(yyRuleName[0])) ){ - yysize = yyRuleInfoNRhs[yyruleno]; - if( yysize ){ - fprintf(yyTraceFILE, "%sReduce %d [%s]%s, pop back to state %d.\n", - yyTracePrompt, - yyruleno, yyRuleName[yyruleno], - yyrulenoyytos - yypParser->yystack)>yypParser->yyhwm ){ - yypParser->yyhwm++; - assert( yypParser->yyhwm == (int)(yypParser->yytos - yypParser->yystack)); - } -#endif -#if YYSTACKDEPTH>0 - if( yypParser->yytos>=yypParser->yystackEnd ){ - yyStackOverflow(yypParser); - /* The call to yyStackOverflow() above pops the stack until it is - ** empty, causing the main parser loop to exit. So the return value - ** is never used and does not matter. */ - return 0; - } -#else - if( yypParser->yytos>=&yypParser->yystack[yypParser->yystksz-1] ){ - if( yyGrowStack(yypParser) ){ - yyStackOverflow(yypParser); - /* The call to yyStackOverflow() above pops the stack until it is - ** empty, causing the main parser loop to exit. So the return value - ** is never used and does not matter. */ - return 0; - } - yymsp = yypParser->yytos; - } -#endif - } switch( yyruleno ){ /* Beginning here are the reduction cases. A typical example @@ -5622,11 +6016,11 @@ static YYACTIONTYPE yy_reduce( YYMINORTYPE yylhsminor; case 0: /* cmd ::= CREATE ACCOUNT NK_ID PASS NK_STRING account_options */ { pCxt->errCode = generateSyntaxErrMsg(&pCxt->msgBuf, TSDB_CODE_PAR_EXPRIE_STATEMENT); } - yy_destructor(yypParser,390,&yymsp[0].minor); + yy_destructor(yypParser,391,&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,391,&yymsp[0].minor); + yy_destructor(yypParser,392,&yymsp[0].minor); break; case 2: /* account_options ::= */ { } @@ -5640,20 +6034,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,390,&yymsp[-2].minor); +{ yy_destructor(yypParser,391,&yymsp[-2].minor); { } - yy_destructor(yypParser,392,&yymsp[0].minor); + yy_destructor(yypParser,393,&yymsp[0].minor); } break; case 12: /* alter_account_options ::= alter_account_option */ -{ yy_destructor(yypParser,393,&yymsp[0].minor); +{ yy_destructor(yypParser,394,&yymsp[0].minor); { } } break; case 13: /* alter_account_options ::= alter_account_options alter_account_option */ -{ yy_destructor(yypParser,391,&yymsp[-1].minor); +{ yy_destructor(yypParser,392,&yymsp[-1].minor); { } - yy_destructor(yypParser,393,&yymsp[0].minor); + yy_destructor(yypParser,394,&yymsp[0].minor); } break; case 14: /* alter_account_option ::= PASS literal */ @@ -5667,18 +6061,18 @@ static YYACTIONTYPE yy_reduce( case 22: /* alter_account_option ::= CONNS literal */ yytestcase(yyruleno==22); case 23: /* alter_account_option ::= STATE literal */ yytestcase(yyruleno==23); { } - yy_destructor(yypParser,392,&yymsp[0].minor); + yy_destructor(yypParser,393,&yymsp[0].minor); break; case 24: /* ip_range_list ::= NK_STRING */ -{ yylhsminor.yy946 = createNodeList(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0)); } - yymsp[0].minor.yy946 = yylhsminor.yy946; +{ yylhsminor.yy628 = createNodeList(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy628 = yylhsminor.yy628; break; case 25: /* ip_range_list ::= ip_range_list NK_COMMA NK_STRING */ -{ yylhsminor.yy946 = addNodeToList(pCxt, yymsp[-2].minor.yy946, createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0)); } - yymsp[-2].minor.yy946 = yylhsminor.yy946; +{ yylhsminor.yy628 = addNodeToList(pCxt, yymsp[-2].minor.yy628, createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0)); } + yymsp[-2].minor.yy628 = yylhsminor.yy628; break; case 26: /* white_list ::= HOST ip_range_list */ -{ yymsp[-1].minor.yy946 = yymsp[0].minor.yy946; } +{ yymsp[-1].minor.yy628 = yymsp[0].minor.yy628; } break; case 27: /* white_list_opt ::= */ case 213: /* specific_cols_opt ::= */ yytestcase(yyruleno==213); @@ -5686,130 +6080,130 @@ static YYACTIONTYPE yy_reduce( case 344: /* tag_list_opt ::= */ yytestcase(yyruleno==344); case 417: /* col_list_opt ::= */ yytestcase(yyruleno==417); case 424: /* tag_def_or_ref_opt ::= */ yytestcase(yyruleno==424); - case 708: /* partition_by_clause_opt ::= */ yytestcase(yyruleno==708); - case 740: /* group_by_clause_opt ::= */ yytestcase(yyruleno==740); - case 760: /* order_by_clause_opt ::= */ yytestcase(yyruleno==760); -{ yymsp[1].minor.yy946 = NULL; } + case 709: /* partition_by_clause_opt ::= */ yytestcase(yyruleno==709); + case 741: /* group_by_clause_opt ::= */ yytestcase(yyruleno==741); + case 761: /* order_by_clause_opt ::= */ yytestcase(yyruleno==761); +{ yymsp[1].minor.yy628 = NULL; } break; case 28: /* white_list_opt ::= white_list */ case 252: /* tags_def_opt ::= tags_def */ yytestcase(yyruleno==252); case 425: /* tag_def_or_ref_opt ::= tags_def */ yytestcase(yyruleno==425); - case 616: /* star_func_para_list ::= other_para_list */ yytestcase(yyruleno==616); -{ yylhsminor.yy946 = yymsp[0].minor.yy946; } - yymsp[0].minor.yy946 = yylhsminor.yy946; + case 617: /* star_func_para_list ::= other_para_list */ yytestcase(yyruleno==617); +{ yylhsminor.yy628 = yymsp[0].minor.yy628; } + yymsp[0].minor.yy628 = yylhsminor.yy628; break; case 29: /* is_import_opt ::= */ case 31: /* is_createdb_opt ::= */ yytestcase(yyruleno==31); -{ yymsp[1].minor.yy815 = 0; } +{ yymsp[1].minor.yy47 = 0; } break; case 30: /* is_import_opt ::= IS_IMPORT NK_INTEGER */ case 32: /* is_createdb_opt ::= CREATEDB NK_INTEGER */ yytestcase(yyruleno==32); case 42: /* sysinfo_opt ::= SYSINFO NK_INTEGER */ yytestcase(yyruleno==42); -{ yymsp[-1].minor.yy815 = taosStr2Int8(yymsp[0].minor.yy0.z, NULL, 10); } +{ yymsp[-1].minor.yy47 = taosStr2Int8(yymsp[0].minor.yy0.z, NULL, 10); } break; case 33: /* cmd ::= CREATE USER user_name PASS NK_STRING sysinfo_opt is_createdb_opt is_import_opt white_list_opt */ { - pCxt->pRootNode = createCreateUserStmt(pCxt, &yymsp[-6].minor.yy557, &yymsp[-4].minor.yy0, yymsp[-3].minor.yy815, yymsp[-1].minor.yy815, yymsp[-2].minor.yy815); - pCxt->pRootNode = addCreateUserStmtWhiteList(pCxt, pCxt->pRootNode, yymsp[0].minor.yy946); + pCxt->pRootNode = createCreateUserStmt(pCxt, &yymsp[-6].minor.yy561, &yymsp[-4].minor.yy0, yymsp[-3].minor.yy47, yymsp[-1].minor.yy47, yymsp[-2].minor.yy47); + pCxt->pRootNode = addCreateUserStmtWhiteList(pCxt, pCxt->pRootNode, yymsp[0].minor.yy628); } break; case 34: /* cmd ::= ALTER USER user_name PASS NK_STRING */ -{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy557, TSDB_ALTER_USER_PASSWD, &yymsp[0].minor.yy0); } +{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy561, TSDB_ALTER_USER_PASSWD, &yymsp[0].minor.yy0); } break; case 35: /* cmd ::= ALTER USER user_name ENABLE NK_INTEGER */ -{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy557, TSDB_ALTER_USER_ENABLE, &yymsp[0].minor.yy0); } +{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy561, TSDB_ALTER_USER_ENABLE, &yymsp[0].minor.yy0); } break; case 36: /* cmd ::= ALTER USER user_name SYSINFO NK_INTEGER */ -{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy557, TSDB_ALTER_USER_SYSINFO, &yymsp[0].minor.yy0); } +{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy561, TSDB_ALTER_USER_SYSINFO, &yymsp[0].minor.yy0); } break; case 37: /* cmd ::= ALTER USER user_name CREATEDB NK_INTEGER */ -{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy557, TSDB_ALTER_USER_CREATEDB, &yymsp[0].minor.yy0); } +{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy561, TSDB_ALTER_USER_CREATEDB, &yymsp[0].minor.yy0); } break; case 38: /* cmd ::= ALTER USER user_name ADD white_list */ -{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy557, TSDB_ALTER_USER_ADD_WHITE_LIST, yymsp[0].minor.yy946); } +{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy561, TSDB_ALTER_USER_ADD_WHITE_LIST, yymsp[0].minor.yy628); } break; case 39: /* cmd ::= ALTER USER user_name DROP white_list */ -{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy557, TSDB_ALTER_USER_DROP_WHITE_LIST, yymsp[0].minor.yy946); } +{ pCxt->pRootNode = createAlterUserStmt(pCxt, &yymsp[-2].minor.yy561, TSDB_ALTER_USER_DROP_WHITE_LIST, yymsp[0].minor.yy628); } break; case 40: /* cmd ::= DROP USER user_name */ -{ pCxt->pRootNode = createDropUserStmt(pCxt, &yymsp[0].minor.yy557); } +{ pCxt->pRootNode = createDropUserStmt(pCxt, &yymsp[0].minor.yy561); } break; case 41: /* sysinfo_opt ::= */ -{ yymsp[1].minor.yy815 = 1; } +{ yymsp[1].minor.yy47 = 1; } break; case 43: /* cmd ::= GRANT privileges ON priv_level with_clause_opt TO user_name */ -{ pCxt->pRootNode = createGrantStmt(pCxt, yymsp[-5].minor.yy483, &yymsp[-3].minor.yy723, &yymsp[0].minor.yy557, yymsp[-2].minor.yy974); } +{ pCxt->pRootNode = createGrantStmt(pCxt, yymsp[-5].minor.yy781, &yymsp[-3].minor.yy469, &yymsp[0].minor.yy561, yymsp[-2].minor.yy980); } break; case 44: /* cmd ::= REVOKE privileges ON priv_level with_clause_opt FROM user_name */ -{ pCxt->pRootNode = createRevokeStmt(pCxt, yymsp[-5].minor.yy483, &yymsp[-3].minor.yy723, &yymsp[0].minor.yy557, yymsp[-2].minor.yy974); } +{ pCxt->pRootNode = createRevokeStmt(pCxt, yymsp[-5].minor.yy781, &yymsp[-3].minor.yy469, &yymsp[0].minor.yy561, yymsp[-2].minor.yy980); } break; case 45: /* privileges ::= ALL */ -{ yymsp[0].minor.yy483 = PRIVILEGE_TYPE_ALL; } +{ yymsp[0].minor.yy781 = PRIVILEGE_TYPE_ALL; } break; case 46: /* privileges ::= priv_type_list */ case 48: /* priv_type_list ::= priv_type */ yytestcase(yyruleno==48); -{ yylhsminor.yy483 = yymsp[0].minor.yy483; } - yymsp[0].minor.yy483 = yylhsminor.yy483; +{ yylhsminor.yy781 = yymsp[0].minor.yy781; } + yymsp[0].minor.yy781 = yylhsminor.yy781; break; case 47: /* privileges ::= SUBSCRIBE */ -{ yymsp[0].minor.yy483 = PRIVILEGE_TYPE_SUBSCRIBE; } +{ yymsp[0].minor.yy781 = PRIVILEGE_TYPE_SUBSCRIBE; } break; case 49: /* priv_type_list ::= priv_type_list NK_COMMA priv_type */ -{ yylhsminor.yy483 = yymsp[-2].minor.yy483 | yymsp[0].minor.yy483; } - yymsp[-2].minor.yy483 = yylhsminor.yy483; +{ yylhsminor.yy781 = yymsp[-2].minor.yy781 | yymsp[0].minor.yy781; } + yymsp[-2].minor.yy781 = yylhsminor.yy781; break; case 50: /* priv_type ::= READ */ -{ yymsp[0].minor.yy483 = PRIVILEGE_TYPE_READ; } +{ yymsp[0].minor.yy781 = PRIVILEGE_TYPE_READ; } break; case 51: /* priv_type ::= WRITE */ -{ yymsp[0].minor.yy483 = PRIVILEGE_TYPE_WRITE; } +{ yymsp[0].minor.yy781 = PRIVILEGE_TYPE_WRITE; } break; case 52: /* priv_type ::= ALTER */ -{ yymsp[0].minor.yy483 = PRIVILEGE_TYPE_ALTER; } +{ yymsp[0].minor.yy781 = PRIVILEGE_TYPE_ALTER; } break; case 53: /* priv_level ::= NK_STAR NK_DOT NK_STAR */ -{ yylhsminor.yy723.first = yymsp[-2].minor.yy0; yylhsminor.yy723.second = yymsp[0].minor.yy0; } - yymsp[-2].minor.yy723 = yylhsminor.yy723; +{ yylhsminor.yy469.first = yymsp[-2].minor.yy0; yylhsminor.yy469.second = yymsp[0].minor.yy0; } + yymsp[-2].minor.yy469 = yylhsminor.yy469; break; case 54: /* priv_level ::= db_name NK_DOT NK_STAR */ -{ yylhsminor.yy723.first = yymsp[-2].minor.yy557; yylhsminor.yy723.second = yymsp[0].minor.yy0; } - yymsp[-2].minor.yy723 = yylhsminor.yy723; +{ yylhsminor.yy469.first = yymsp[-2].minor.yy561; yylhsminor.yy469.second = yymsp[0].minor.yy0; } + yymsp[-2].minor.yy469 = yylhsminor.yy469; break; case 55: /* priv_level ::= db_name NK_DOT table_name */ -{ yylhsminor.yy723.first = yymsp[-2].minor.yy557; yylhsminor.yy723.second = yymsp[0].minor.yy557; } - yymsp[-2].minor.yy723 = yylhsminor.yy723; +{ yylhsminor.yy469.first = yymsp[-2].minor.yy561; yylhsminor.yy469.second = yymsp[0].minor.yy561; } + yymsp[-2].minor.yy469 = yylhsminor.yy469; break; case 56: /* priv_level ::= topic_name */ -{ yylhsminor.yy723.first = yymsp[0].minor.yy557; yylhsminor.yy723.second = nil_token; } - yymsp[0].minor.yy723 = yylhsminor.yy723; +{ yylhsminor.yy469.first = yymsp[0].minor.yy561; yylhsminor.yy469.second = nil_token; } + yymsp[0].minor.yy469 = yylhsminor.yy469; break; case 57: /* with_clause_opt ::= */ case 178: /* start_opt ::= */ yytestcase(yyruleno==178); case 182: /* end_opt ::= */ yytestcase(yyruleno==182); case 339: /* like_pattern_opt ::= */ yytestcase(yyruleno==339); - case 436: /* subtable_opt ::= */ yytestcase(yyruleno==436); - case 626: /* case_when_else_opt ::= */ yytestcase(yyruleno==626); - case 656: /* from_clause_opt ::= */ yytestcase(yyruleno==656); - case 683: /* join_on_clause_opt ::= */ yytestcase(yyruleno==683); - case 685: /* window_offset_clause_opt ::= */ yytestcase(yyruleno==685); - case 689: /* jlimit_clause_opt ::= */ yytestcase(yyruleno==689); - case 706: /* where_clause_opt ::= */ yytestcase(yyruleno==706); - case 715: /* twindow_clause_opt ::= */ yytestcase(yyruleno==715); - case 725: /* sliding_opt ::= */ yytestcase(yyruleno==725); - case 730: /* fill_opt ::= */ yytestcase(yyruleno==730); - case 744: /* having_clause_opt ::= */ yytestcase(yyruleno==744); - case 746: /* range_opt ::= */ yytestcase(yyruleno==746); - case 749: /* every_opt ::= */ yytestcase(yyruleno==749); - case 762: /* slimit_clause_opt ::= */ yytestcase(yyruleno==762); - case 766: /* limit_clause_opt ::= */ yytestcase(yyruleno==766); -{ yymsp[1].minor.yy974 = NULL; } + case 437: /* subtable_opt ::= */ yytestcase(yyruleno==437); + case 627: /* case_when_else_opt ::= */ yytestcase(yyruleno==627); + case 657: /* from_clause_opt ::= */ yytestcase(yyruleno==657); + case 684: /* join_on_clause_opt ::= */ yytestcase(yyruleno==684); + case 686: /* window_offset_clause_opt ::= */ yytestcase(yyruleno==686); + case 690: /* jlimit_clause_opt ::= */ yytestcase(yyruleno==690); + case 707: /* where_clause_opt ::= */ yytestcase(yyruleno==707); + case 716: /* twindow_clause_opt ::= */ yytestcase(yyruleno==716); + case 726: /* sliding_opt ::= */ yytestcase(yyruleno==726); + case 731: /* fill_opt ::= */ yytestcase(yyruleno==731); + case 745: /* having_clause_opt ::= */ yytestcase(yyruleno==745); + case 747: /* range_opt ::= */ yytestcase(yyruleno==747); + case 750: /* every_opt ::= */ yytestcase(yyruleno==750); + case 763: /* slimit_clause_opt ::= */ yytestcase(yyruleno==763); + case 767: /* limit_clause_opt ::= */ yytestcase(yyruleno==767); +{ yymsp[1].minor.yy980 = NULL; } break; case 58: /* with_clause_opt ::= WITH search_condition */ - case 657: /* from_clause_opt ::= FROM table_reference_list */ yytestcase(yyruleno==657); - case 684: /* join_on_clause_opt ::= ON search_condition */ yytestcase(yyruleno==684); - case 707: /* where_clause_opt ::= WHERE search_condition */ yytestcase(yyruleno==707); - case 745: /* having_clause_opt ::= HAVING search_condition */ yytestcase(yyruleno==745); -{ yymsp[-1].minor.yy974 = yymsp[0].minor.yy974; } + case 658: /* from_clause_opt ::= FROM table_reference_list */ yytestcase(yyruleno==658); + case 685: /* join_on_clause_opt ::= ON search_condition */ yytestcase(yyruleno==685); + case 708: /* where_clause_opt ::= WHERE search_condition */ yytestcase(yyruleno==708); + case 746: /* having_clause_opt ::= HAVING search_condition */ yytestcase(yyruleno==746); +{ yymsp[-1].minor.yy980 = yymsp[0].minor.yy980; } break; case 59: /* cmd ::= CREATE ENCRYPT_KEY NK_STRING */ { pCxt->pRootNode = createEncryptKeyStmt(pCxt, &yymsp[0].minor.yy0); } @@ -5827,22 +6221,22 @@ static YYACTIONTYPE yy_reduce( { pCxt->pRootNode = createDropAnodeStmt(pCxt, &yymsp[0].minor.yy0); } break; case 64: /* cmd ::= CREATE DNODE dnode_endpoint */ -{ pCxt->pRootNode = createCreateDnodeStmt(pCxt, &yymsp[0].minor.yy557, NULL); } +{ pCxt->pRootNode = createCreateDnodeStmt(pCxt, &yymsp[0].minor.yy561, NULL); } break; case 65: /* cmd ::= CREATE DNODE dnode_endpoint PORT NK_INTEGER */ -{ pCxt->pRootNode = createCreateDnodeStmt(pCxt, &yymsp[-2].minor.yy557, &yymsp[0].minor.yy0); } +{ pCxt->pRootNode = createCreateDnodeStmt(pCxt, &yymsp[-2].minor.yy561, &yymsp[0].minor.yy0); } break; case 66: /* cmd ::= DROP DNODE NK_INTEGER force_opt */ -{ pCxt->pRootNode = createDropDnodeStmt(pCxt, &yymsp[-1].minor.yy0, yymsp[0].minor.yy569, false); } +{ pCxt->pRootNode = createDropDnodeStmt(pCxt, &yymsp[-1].minor.yy0, yymsp[0].minor.yy957, false); } break; case 67: /* cmd ::= DROP DNODE dnode_endpoint force_opt */ -{ pCxt->pRootNode = createDropDnodeStmt(pCxt, &yymsp[-1].minor.yy557, yymsp[0].minor.yy569, false); } +{ pCxt->pRootNode = createDropDnodeStmt(pCxt, &yymsp[-1].minor.yy561, yymsp[0].minor.yy957, false); } break; case 68: /* cmd ::= DROP DNODE NK_INTEGER unsafe_opt */ -{ pCxt->pRootNode = createDropDnodeStmt(pCxt, &yymsp[-1].minor.yy0, false, yymsp[0].minor.yy569); } +{ pCxt->pRootNode = createDropDnodeStmt(pCxt, &yymsp[-1].minor.yy0, false, yymsp[0].minor.yy957); } break; case 69: /* cmd ::= DROP DNODE dnode_endpoint unsafe_opt */ -{ pCxt->pRootNode = createDropDnodeStmt(pCxt, &yymsp[-1].minor.yy557, false, yymsp[0].minor.yy569); } +{ pCxt->pRootNode = createDropDnodeStmt(pCxt, &yymsp[-1].minor.yy561, false, yymsp[0].minor.yy957); } break; case 70: /* cmd ::= ALTER DNODE NK_INTEGER NK_STRING */ { pCxt->pRootNode = createAlterDnodeStmt(pCxt, &yymsp[-1].minor.yy0, &yymsp[0].minor.yy0, NULL); } @@ -5866,38 +6260,38 @@ static YYACTIONTYPE yy_reduce( case 374: /* sma_func_name ::= FIRST */ yytestcase(yyruleno==374); case 375: /* sma_func_name ::= LAST */ yytestcase(yyruleno==375); case 376: /* sma_func_name ::= LAST_ROW */ yytestcase(yyruleno==376); - case 524: /* db_name ::= NK_ID */ yytestcase(yyruleno==524); - case 525: /* table_name ::= NK_ID */ yytestcase(yyruleno==525); - case 526: /* column_name ::= NK_ID */ yytestcase(yyruleno==526); - case 527: /* function_name ::= NK_ID */ yytestcase(yyruleno==527); - case 528: /* view_name ::= NK_ID */ yytestcase(yyruleno==528); - case 529: /* table_alias ::= NK_ID */ yytestcase(yyruleno==529); - case 530: /* column_alias ::= NK_ID */ yytestcase(yyruleno==530); - case 531: /* column_alias ::= NK_ALIAS */ yytestcase(yyruleno==531); - case 532: /* user_name ::= NK_ID */ yytestcase(yyruleno==532); - case 533: /* topic_name ::= NK_ID */ yytestcase(yyruleno==533); - case 534: /* stream_name ::= NK_ID */ yytestcase(yyruleno==534); - case 535: /* cgroup_name ::= NK_ID */ yytestcase(yyruleno==535); - case 536: /* index_name ::= NK_ID */ yytestcase(yyruleno==536); - case 537: /* tsma_name ::= NK_ID */ yytestcase(yyruleno==537); - case 596: /* substr_func ::= SUBSTR */ yytestcase(yyruleno==596); - case 597: /* substr_func ::= SUBSTRING */ yytestcase(yyruleno==597); - case 601: /* noarg_func ::= NOW */ yytestcase(yyruleno==601); - case 602: /* noarg_func ::= TODAY */ yytestcase(yyruleno==602); - case 603: /* noarg_func ::= TIMEZONE */ yytestcase(yyruleno==603); - case 604: /* noarg_func ::= DATABASE */ yytestcase(yyruleno==604); - case 605: /* noarg_func ::= CLIENT_VERSION */ yytestcase(yyruleno==605); - case 606: /* noarg_func ::= SERVER_VERSION */ yytestcase(yyruleno==606); - case 607: /* noarg_func ::= SERVER_STATUS */ yytestcase(yyruleno==607); - case 608: /* noarg_func ::= CURRENT_USER */ yytestcase(yyruleno==608); - case 609: /* noarg_func ::= USER */ yytestcase(yyruleno==609); - case 610: /* noarg_func ::= PI */ yytestcase(yyruleno==610); - case 611: /* star_func ::= COUNT */ yytestcase(yyruleno==611); - case 612: /* star_func ::= FIRST */ yytestcase(yyruleno==612); - case 613: /* star_func ::= LAST */ yytestcase(yyruleno==613); - case 614: /* star_func ::= LAST_ROW */ yytestcase(yyruleno==614); -{ yylhsminor.yy557 = yymsp[0].minor.yy0; } - yymsp[0].minor.yy557 = yylhsminor.yy557; + case 525: /* db_name ::= NK_ID */ yytestcase(yyruleno==525); + case 526: /* table_name ::= NK_ID */ yytestcase(yyruleno==526); + case 527: /* column_name ::= NK_ID */ yytestcase(yyruleno==527); + case 528: /* function_name ::= NK_ID */ yytestcase(yyruleno==528); + case 529: /* view_name ::= NK_ID */ yytestcase(yyruleno==529); + case 530: /* table_alias ::= NK_ID */ yytestcase(yyruleno==530); + case 531: /* column_alias ::= NK_ID */ yytestcase(yyruleno==531); + case 532: /* column_alias ::= NK_ALIAS */ yytestcase(yyruleno==532); + case 533: /* user_name ::= NK_ID */ yytestcase(yyruleno==533); + case 534: /* topic_name ::= NK_ID */ yytestcase(yyruleno==534); + case 535: /* stream_name ::= NK_ID */ yytestcase(yyruleno==535); + case 536: /* cgroup_name ::= NK_ID */ yytestcase(yyruleno==536); + case 537: /* index_name ::= NK_ID */ yytestcase(yyruleno==537); + case 538: /* tsma_name ::= NK_ID */ yytestcase(yyruleno==538); + case 597: /* substr_func ::= SUBSTR */ yytestcase(yyruleno==597); + case 598: /* substr_func ::= SUBSTRING */ yytestcase(yyruleno==598); + case 602: /* noarg_func ::= NOW */ yytestcase(yyruleno==602); + case 603: /* noarg_func ::= TODAY */ yytestcase(yyruleno==603); + case 604: /* noarg_func ::= TIMEZONE */ yytestcase(yyruleno==604); + case 605: /* noarg_func ::= DATABASE */ yytestcase(yyruleno==605); + case 606: /* noarg_func ::= CLIENT_VERSION */ yytestcase(yyruleno==606); + case 607: /* noarg_func ::= SERVER_VERSION */ yytestcase(yyruleno==607); + case 608: /* noarg_func ::= SERVER_STATUS */ yytestcase(yyruleno==608); + case 609: /* noarg_func ::= CURRENT_USER */ yytestcase(yyruleno==609); + case 610: /* noarg_func ::= USER */ yytestcase(yyruleno==610); + case 611: /* noarg_func ::= PI */ yytestcase(yyruleno==611); + case 612: /* star_func ::= COUNT */ yytestcase(yyruleno==612); + case 613: /* star_func ::= FIRST */ yytestcase(yyruleno==613); + case 614: /* star_func ::= LAST */ yytestcase(yyruleno==614); + case 615: /* star_func ::= LAST_ROW */ yytestcase(yyruleno==615); +{ yylhsminor.yy561 = yymsp[0].minor.yy0; } + yymsp[0].minor.yy561 = yylhsminor.yy561; break; case 78: /* force_opt ::= */ case 105: /* not_exists_opt ::= */ yytestcase(yyruleno==105); @@ -5906,19 +6300,19 @@ static YYACTIONTYPE yy_reduce( case 394: /* analyze_opt ::= */ yytestcase(yyruleno==394); case 401: /* agg_func_opt ::= */ yytestcase(yyruleno==401); case 407: /* or_replace_opt ::= */ yytestcase(yyruleno==407); - case 438: /* ignore_opt ::= */ yytestcase(yyruleno==438); - case 694: /* tag_mode_opt ::= */ yytestcase(yyruleno==694); - case 696: /* set_quantifier_opt ::= */ yytestcase(yyruleno==696); -{ yymsp[1].minor.yy569 = false; } + case 439: /* ignore_opt ::= */ yytestcase(yyruleno==439); + case 695: /* tag_mode_opt ::= */ yytestcase(yyruleno==695); + case 697: /* set_quantifier_opt ::= */ yytestcase(yyruleno==697); +{ yymsp[1].minor.yy957 = false; } break; case 79: /* force_opt ::= FORCE */ case 80: /* unsafe_opt ::= UNSAFE */ yytestcase(yyruleno==80); case 212: /* with_opt ::= WITH */ yytestcase(yyruleno==212); case 395: /* analyze_opt ::= ANALYZE */ yytestcase(yyruleno==395); case 402: /* agg_func_opt ::= AGGREGATE */ yytestcase(yyruleno==402); - case 695: /* tag_mode_opt ::= TAGS */ yytestcase(yyruleno==695); - case 697: /* set_quantifier_opt ::= DISTINCT */ yytestcase(yyruleno==697); -{ yymsp[0].minor.yy569 = true; } + case 696: /* tag_mode_opt ::= TAGS */ yytestcase(yyruleno==696); + case 698: /* set_quantifier_opt ::= DISTINCT */ yytestcase(yyruleno==698); +{ yymsp[0].minor.yy957 = true; } break; case 81: /* cmd ::= ALTER CLUSTER NK_STRING */ { pCxt->pRootNode = createAlterClusterStmt(pCxt, &yymsp[0].minor.yy0, NULL); } @@ -5966,275 +6360,275 @@ static YYACTIONTYPE yy_reduce( { pCxt->pRootNode = createRestoreComponentNodeStmt(pCxt, QUERY_NODE_RESTORE_VNODE_STMT, &yymsp[0].minor.yy0); } break; case 96: /* cmd ::= CREATE DATABASE not_exists_opt db_name db_options */ -{ pCxt->pRootNode = createCreateDatabaseStmt(pCxt, yymsp[-2].minor.yy569, &yymsp[-1].minor.yy557, yymsp[0].minor.yy974); } +{ pCxt->pRootNode = createCreateDatabaseStmt(pCxt, yymsp[-2].minor.yy957, &yymsp[-1].minor.yy561, yymsp[0].minor.yy980); } break; case 97: /* cmd ::= DROP DATABASE exists_opt db_name */ -{ pCxt->pRootNode = createDropDatabaseStmt(pCxt, yymsp[-1].minor.yy569, &yymsp[0].minor.yy557); } +{ pCxt->pRootNode = createDropDatabaseStmt(pCxt, yymsp[-1].minor.yy957, &yymsp[0].minor.yy561); } break; case 98: /* cmd ::= USE db_name */ -{ pCxt->pRootNode = createUseDatabaseStmt(pCxt, &yymsp[0].minor.yy557); } +{ pCxt->pRootNode = createUseDatabaseStmt(pCxt, &yymsp[0].minor.yy561); } break; case 99: /* cmd ::= ALTER DATABASE db_name alter_db_options */ -{ pCxt->pRootNode = createAlterDatabaseStmt(pCxt, &yymsp[-1].minor.yy557, yymsp[0].minor.yy974); } +{ pCxt->pRootNode = createAlterDatabaseStmt(pCxt, &yymsp[-1].minor.yy561, yymsp[0].minor.yy980); } break; case 100: /* cmd ::= FLUSH DATABASE db_name */ -{ pCxt->pRootNode = createFlushDatabaseStmt(pCxt, &yymsp[0].minor.yy557); } +{ pCxt->pRootNode = createFlushDatabaseStmt(pCxt, &yymsp[0].minor.yy561); } break; case 101: /* cmd ::= TRIM DATABASE db_name speed_opt */ -{ pCxt->pRootNode = createTrimDatabaseStmt(pCxt, &yymsp[-1].minor.yy557, yymsp[0].minor.yy904); } +{ pCxt->pRootNode = createTrimDatabaseStmt(pCxt, &yymsp[-1].minor.yy561, yymsp[0].minor.yy844); } break; case 102: /* cmd ::= S3MIGRATE DATABASE db_name */ -{ pCxt->pRootNode = createS3MigrateDatabaseStmt(pCxt, &yymsp[0].minor.yy557); } +{ pCxt->pRootNode = createS3MigrateDatabaseStmt(pCxt, &yymsp[0].minor.yy561); } break; case 103: /* cmd ::= COMPACT DATABASE db_name start_opt end_opt */ -{ pCxt->pRootNode = createCompactStmt(pCxt, &yymsp[-2].minor.yy557, yymsp[-1].minor.yy974, yymsp[0].minor.yy974); } +{ pCxt->pRootNode = createCompactStmt(pCxt, &yymsp[-2].minor.yy561, yymsp[-1].minor.yy980, yymsp[0].minor.yy980); } break; case 104: /* not_exists_opt ::= IF NOT EXISTS */ -{ yymsp[-2].minor.yy569 = true; } +{ yymsp[-2].minor.yy957 = true; } break; case 106: /* exists_opt ::= IF EXISTS */ case 408: /* or_replace_opt ::= OR REPLACE */ yytestcase(yyruleno==408); - case 439: /* ignore_opt ::= IGNORE UNTREATED */ yytestcase(yyruleno==439); -{ yymsp[-1].minor.yy569 = true; } + case 440: /* ignore_opt ::= IGNORE UNTREATED */ yytestcase(yyruleno==440); +{ yymsp[-1].minor.yy957 = true; } break; case 108: /* db_options ::= */ -{ yymsp[1].minor.yy974 = createDefaultDatabaseOptions(pCxt); } +{ yymsp[1].minor.yy980 = createDefaultDatabaseOptions(pCxt); } break; case 109: /* db_options ::= db_options BUFFER NK_INTEGER */ -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_BUFFER, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_BUFFER, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 110: /* db_options ::= db_options CACHEMODEL NK_STRING */ -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_CACHEMODEL, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_CACHEMODEL, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 111: /* db_options ::= db_options CACHESIZE NK_INTEGER */ -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_CACHESIZE, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_CACHESIZE, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 112: /* db_options ::= db_options COMP NK_INTEGER */ -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_COMP, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_COMP, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 113: /* db_options ::= db_options DURATION NK_INTEGER */ case 114: /* db_options ::= db_options DURATION NK_VARIABLE */ yytestcase(yyruleno==114); -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_DAYS, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_DAYS, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 115: /* db_options ::= db_options MAXROWS NK_INTEGER */ -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_MAXROWS, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_MAXROWS, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 116: /* db_options ::= db_options MINROWS NK_INTEGER */ -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_MINROWS, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_MINROWS, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 117: /* db_options ::= db_options KEEP integer_list */ case 118: /* db_options ::= db_options KEEP variable_list */ yytestcase(yyruleno==118); -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_KEEP, yymsp[0].minor.yy946); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_KEEP, yymsp[0].minor.yy628); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 119: /* db_options ::= db_options PAGES NK_INTEGER */ -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_PAGES, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_PAGES, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 120: /* db_options ::= db_options PAGESIZE NK_INTEGER */ -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_PAGESIZE, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_PAGESIZE, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 121: /* db_options ::= db_options TSDB_PAGESIZE NK_INTEGER */ -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_TSDB_PAGESIZE, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_TSDB_PAGESIZE, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 122: /* db_options ::= db_options PRECISION NK_STRING */ -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_PRECISION, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_PRECISION, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 123: /* db_options ::= db_options REPLICA NK_INTEGER */ -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_REPLICA, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_REPLICA, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 124: /* db_options ::= db_options VGROUPS NK_INTEGER */ -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_VGROUPS, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_VGROUPS, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 125: /* db_options ::= db_options SINGLE_STABLE NK_INTEGER */ -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_SINGLE_STABLE, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_SINGLE_STABLE, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 126: /* db_options ::= db_options RETENTIONS retention_list */ -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_RETENTIONS, yymsp[0].minor.yy946); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_RETENTIONS, yymsp[0].minor.yy628); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 127: /* db_options ::= db_options SCHEMALESS NK_INTEGER */ -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_SCHEMALESS, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_SCHEMALESS, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 128: /* db_options ::= db_options WAL_LEVEL NK_INTEGER */ -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_WAL, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_WAL, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 129: /* db_options ::= db_options WAL_FSYNC_PERIOD NK_INTEGER */ -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_FSYNC, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_FSYNC, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 130: /* db_options ::= db_options WAL_RETENTION_PERIOD NK_INTEGER */ -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_WAL_RETENTION_PERIOD, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_WAL_RETENTION_PERIOD, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 131: /* 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.yy974 = setDatabaseOption(pCxt, yymsp[-3].minor.yy974, DB_OPTION_WAL_RETENTION_PERIOD, &t); + yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-3].minor.yy980, DB_OPTION_WAL_RETENTION_PERIOD, &t); } - yymsp[-3].minor.yy974 = yylhsminor.yy974; + yymsp[-3].minor.yy980 = yylhsminor.yy980; break; case 132: /* db_options ::= db_options WAL_RETENTION_SIZE NK_INTEGER */ -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_WAL_RETENTION_SIZE, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_WAL_RETENTION_SIZE, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 133: /* 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.yy974 = setDatabaseOption(pCxt, yymsp[-3].minor.yy974, DB_OPTION_WAL_RETENTION_SIZE, &t); + yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-3].minor.yy980, DB_OPTION_WAL_RETENTION_SIZE, &t); } - yymsp[-3].minor.yy974 = yylhsminor.yy974; + yymsp[-3].minor.yy980 = yylhsminor.yy980; break; case 134: /* db_options ::= db_options WAL_ROLL_PERIOD NK_INTEGER */ -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_WAL_ROLL_PERIOD, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_WAL_ROLL_PERIOD, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 135: /* db_options ::= db_options WAL_SEGMENT_SIZE NK_INTEGER */ -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_WAL_SEGMENT_SIZE, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_WAL_SEGMENT_SIZE, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 136: /* db_options ::= db_options STT_TRIGGER NK_INTEGER */ -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_STT_TRIGGER, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_STT_TRIGGER, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 137: /* db_options ::= db_options TABLE_PREFIX signed */ -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_TABLE_PREFIX, yymsp[0].minor.yy974); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_TABLE_PREFIX, yymsp[0].minor.yy980); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 138: /* db_options ::= db_options TABLE_SUFFIX signed */ -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_TABLE_SUFFIX, yymsp[0].minor.yy974); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_TABLE_SUFFIX, yymsp[0].minor.yy980); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 139: /* db_options ::= db_options S3_CHUNKPAGES NK_INTEGER */ -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_S3_CHUNKPAGES, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_S3_CHUNKPAGES, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 140: /* db_options ::= db_options S3_KEEPLOCAL NK_INTEGER */ case 141: /* db_options ::= db_options S3_KEEPLOCAL NK_VARIABLE */ yytestcase(yyruleno==141); -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_S3_KEEPLOCAL, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_S3_KEEPLOCAL, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 142: /* db_options ::= db_options S3_COMPACT NK_INTEGER */ -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_S3_COMPACT, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_S3_COMPACT, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 143: /* db_options ::= db_options KEEP_TIME_OFFSET NK_INTEGER */ -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_KEEP_TIME_OFFSET, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_KEEP_TIME_OFFSET, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 144: /* db_options ::= db_options ENCRYPT_ALGORITHM NK_STRING */ -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_ENCRYPT_ALGORITHM, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_ENCRYPT_ALGORITHM, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 145: /* db_options ::= db_options DNODES NK_STRING */ -{ yylhsminor.yy974 = setDatabaseOption(pCxt, yymsp[-2].minor.yy974, DB_OPTION_DNODES, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setDatabaseOption(pCxt, yymsp[-2].minor.yy980, DB_OPTION_DNODES, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 146: /* alter_db_options ::= alter_db_option */ -{ yylhsminor.yy974 = createAlterDatabaseOptions(pCxt); yylhsminor.yy974 = setAlterDatabaseOption(pCxt, yylhsminor.yy974, &yymsp[0].minor.yy683); } - yymsp[0].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createAlterDatabaseOptions(pCxt); yylhsminor.yy980 = setAlterDatabaseOption(pCxt, yylhsminor.yy980, &yymsp[0].minor.yy529); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; case 147: /* alter_db_options ::= alter_db_options alter_db_option */ -{ yylhsminor.yy974 = setAlterDatabaseOption(pCxt, yymsp[-1].minor.yy974, &yymsp[0].minor.yy683); } - yymsp[-1].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setAlterDatabaseOption(pCxt, yymsp[-1].minor.yy980, &yymsp[0].minor.yy529); } + yymsp[-1].minor.yy980 = yylhsminor.yy980; break; case 148: /* alter_db_option ::= BUFFER NK_INTEGER */ -{ yymsp[-1].minor.yy683.type = DB_OPTION_BUFFER; yymsp[-1].minor.yy683.val = yymsp[0].minor.yy0; } +{ yymsp[-1].minor.yy529.type = DB_OPTION_BUFFER; yymsp[-1].minor.yy529.val = yymsp[0].minor.yy0; } break; case 149: /* alter_db_option ::= CACHEMODEL NK_STRING */ -{ yymsp[-1].minor.yy683.type = DB_OPTION_CACHEMODEL; yymsp[-1].minor.yy683.val = yymsp[0].minor.yy0; } +{ yymsp[-1].minor.yy529.type = DB_OPTION_CACHEMODEL; yymsp[-1].minor.yy529.val = yymsp[0].minor.yy0; } break; case 150: /* alter_db_option ::= CACHESIZE NK_INTEGER */ -{ yymsp[-1].minor.yy683.type = DB_OPTION_CACHESIZE; yymsp[-1].minor.yy683.val = yymsp[0].minor.yy0; } +{ yymsp[-1].minor.yy529.type = DB_OPTION_CACHESIZE; yymsp[-1].minor.yy529.val = yymsp[0].minor.yy0; } break; case 151: /* alter_db_option ::= WAL_FSYNC_PERIOD NK_INTEGER */ -{ yymsp[-1].minor.yy683.type = DB_OPTION_FSYNC; yymsp[-1].minor.yy683.val = yymsp[0].minor.yy0; } +{ yymsp[-1].minor.yy529.type = DB_OPTION_FSYNC; yymsp[-1].minor.yy529.val = yymsp[0].minor.yy0; } break; case 152: /* alter_db_option ::= KEEP integer_list */ case 153: /* alter_db_option ::= KEEP variable_list */ yytestcase(yyruleno==153); -{ yymsp[-1].minor.yy683.type = DB_OPTION_KEEP; yymsp[-1].minor.yy683.pList = yymsp[0].minor.yy946; } +{ yymsp[-1].minor.yy529.type = DB_OPTION_KEEP; yymsp[-1].minor.yy529.pList = yymsp[0].minor.yy628; } break; case 154: /* alter_db_option ::= PAGES NK_INTEGER */ -{ yymsp[-1].minor.yy683.type = DB_OPTION_PAGES; yymsp[-1].minor.yy683.val = yymsp[0].minor.yy0; } +{ yymsp[-1].minor.yy529.type = DB_OPTION_PAGES; yymsp[-1].minor.yy529.val = yymsp[0].minor.yy0; } break; case 155: /* alter_db_option ::= REPLICA NK_INTEGER */ -{ yymsp[-1].minor.yy683.type = DB_OPTION_REPLICA; yymsp[-1].minor.yy683.val = yymsp[0].minor.yy0; } +{ yymsp[-1].minor.yy529.type = DB_OPTION_REPLICA; yymsp[-1].minor.yy529.val = yymsp[0].minor.yy0; } break; case 156: /* alter_db_option ::= WAL_LEVEL NK_INTEGER */ -{ yymsp[-1].minor.yy683.type = DB_OPTION_WAL; yymsp[-1].minor.yy683.val = yymsp[0].minor.yy0; } +{ yymsp[-1].minor.yy529.type = DB_OPTION_WAL; yymsp[-1].minor.yy529.val = yymsp[0].minor.yy0; } break; case 157: /* alter_db_option ::= STT_TRIGGER NK_INTEGER */ -{ yymsp[-1].minor.yy683.type = DB_OPTION_STT_TRIGGER; yymsp[-1].minor.yy683.val = yymsp[0].minor.yy0; } +{ yymsp[-1].minor.yy529.type = DB_OPTION_STT_TRIGGER; yymsp[-1].minor.yy529.val = yymsp[0].minor.yy0; } break; case 158: /* alter_db_option ::= MINROWS NK_INTEGER */ -{ yymsp[-1].minor.yy683.type = DB_OPTION_MINROWS; yymsp[-1].minor.yy683.val = yymsp[0].minor.yy0; } +{ yymsp[-1].minor.yy529.type = DB_OPTION_MINROWS; yymsp[-1].minor.yy529.val = yymsp[0].minor.yy0; } break; case 159: /* alter_db_option ::= WAL_RETENTION_PERIOD NK_INTEGER */ -{ yymsp[-1].minor.yy683.type = DB_OPTION_WAL_RETENTION_PERIOD; yymsp[-1].minor.yy683.val = yymsp[0].minor.yy0; } +{ yymsp[-1].minor.yy529.type = DB_OPTION_WAL_RETENTION_PERIOD; yymsp[-1].minor.yy529.val = yymsp[0].minor.yy0; } break; case 160: /* 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.yy683.type = DB_OPTION_WAL_RETENTION_PERIOD; yymsp[-2].minor.yy683.val = t; + yymsp[-2].minor.yy529.type = DB_OPTION_WAL_RETENTION_PERIOD; yymsp[-2].minor.yy529.val = t; } break; case 161: /* alter_db_option ::= WAL_RETENTION_SIZE NK_INTEGER */ -{ yymsp[-1].minor.yy683.type = DB_OPTION_WAL_RETENTION_SIZE; yymsp[-1].minor.yy683.val = yymsp[0].minor.yy0; } +{ yymsp[-1].minor.yy529.type = DB_OPTION_WAL_RETENTION_SIZE; yymsp[-1].minor.yy529.val = yymsp[0].minor.yy0; } break; case 162: /* 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.yy683.type = DB_OPTION_WAL_RETENTION_SIZE; yymsp[-2].minor.yy683.val = t; + yymsp[-2].minor.yy529.type = DB_OPTION_WAL_RETENTION_SIZE; yymsp[-2].minor.yy529.val = t; } break; case 163: /* alter_db_option ::= S3_KEEPLOCAL NK_INTEGER */ case 164: /* alter_db_option ::= S3_KEEPLOCAL NK_VARIABLE */ yytestcase(yyruleno==164); -{ yymsp[-1].minor.yy683.type = DB_OPTION_S3_KEEPLOCAL; yymsp[-1].minor.yy683.val = yymsp[0].minor.yy0; } +{ yymsp[-1].minor.yy529.type = DB_OPTION_S3_KEEPLOCAL; yymsp[-1].minor.yy529.val = yymsp[0].minor.yy0; } break; case 165: /* alter_db_option ::= S3_COMPACT NK_INTEGER */ -{ yymsp[-1].minor.yy683.type = DB_OPTION_S3_COMPACT, yymsp[-1].minor.yy683.val = yymsp[0].minor.yy0; } +{ yymsp[-1].minor.yy529.type = DB_OPTION_S3_COMPACT, yymsp[-1].minor.yy529.val = yymsp[0].minor.yy0; } break; case 166: /* alter_db_option ::= KEEP_TIME_OFFSET NK_INTEGER */ -{ yymsp[-1].minor.yy683.type = DB_OPTION_KEEP_TIME_OFFSET; yymsp[-1].minor.yy683.val = yymsp[0].minor.yy0; } +{ yymsp[-1].minor.yy529.type = DB_OPTION_KEEP_TIME_OFFSET; yymsp[-1].minor.yy529.val = yymsp[0].minor.yy0; } break; case 167: /* alter_db_option ::= ENCRYPT_ALGORITHM NK_STRING */ -{ yymsp[-1].minor.yy683.type = DB_OPTION_ENCRYPT_ALGORITHM; yymsp[-1].minor.yy683.val = yymsp[0].minor.yy0; } +{ yymsp[-1].minor.yy529.type = DB_OPTION_ENCRYPT_ALGORITHM; yymsp[-1].minor.yy529.val = yymsp[0].minor.yy0; } break; case 168: /* integer_list ::= NK_INTEGER */ -{ yylhsminor.yy946 = createNodeList(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0)); } - yymsp[0].minor.yy946 = yylhsminor.yy946; +{ yylhsminor.yy628 = createNodeList(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy628 = yylhsminor.yy628; break; case 169: /* integer_list ::= integer_list NK_COMMA NK_INTEGER */ - case 453: /* dnode_list ::= dnode_list DNODE NK_INTEGER */ yytestcase(yyruleno==453); -{ yylhsminor.yy946 = addNodeToList(pCxt, yymsp[-2].minor.yy946, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0)); } - yymsp[-2].minor.yy946 = yylhsminor.yy946; + case 454: /* dnode_list ::= dnode_list DNODE NK_INTEGER */ yytestcase(yyruleno==454); +{ yylhsminor.yy628 = addNodeToList(pCxt, yymsp[-2].minor.yy628, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0)); } + yymsp[-2].minor.yy628 = yylhsminor.yy628; break; case 170: /* variable_list ::= NK_VARIABLE */ -{ yylhsminor.yy946 = createNodeList(pCxt, createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } - yymsp[0].minor.yy946 = yylhsminor.yy946; +{ yylhsminor.yy628 = createNodeList(pCxt, createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy628 = yylhsminor.yy628; break; case 171: /* variable_list ::= variable_list NK_COMMA NK_VARIABLE */ -{ yylhsminor.yy946 = addNodeToList(pCxt, yymsp[-2].minor.yy946, createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } - yymsp[-2].minor.yy946 = yylhsminor.yy946; +{ yylhsminor.yy628 = addNodeToList(pCxt, yymsp[-2].minor.yy628, createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } + yymsp[-2].minor.yy628 = yylhsminor.yy628; break; case 172: /* retention_list ::= retention */ case 205: /* multi_create_clause ::= create_subtable_clause */ yytestcase(yyruleno==205); @@ -6246,15 +6640,15 @@ static YYACTIONTYPE yy_reduce( case 345: /* tag_list_opt ::= tag_item */ yytestcase(yyruleno==345); case 369: /* func_list ::= func */ yytestcase(yyruleno==369); case 419: /* column_stream_def_list ::= column_stream_def */ yytestcase(yyruleno==419); - case 497: /* tags_literal_list ::= tags_literal */ yytestcase(yyruleno==497); - case 522: /* literal_list ::= signed_literal */ yytestcase(yyruleno==522); - case 617: /* other_para_list ::= star_func_para */ yytestcase(yyruleno==617); - case 623: /* when_then_list ::= when_then_expr */ yytestcase(yyruleno==623); - case 699: /* select_list ::= select_item */ yytestcase(yyruleno==699); - case 710: /* partition_list ::= partition_item */ yytestcase(yyruleno==710); - case 773: /* sort_specification_list ::= sort_specification */ yytestcase(yyruleno==773); -{ yylhsminor.yy946 = createNodeList(pCxt, yymsp[0].minor.yy974); } - yymsp[0].minor.yy946 = yylhsminor.yy946; + case 498: /* tags_literal_list ::= tags_literal */ yytestcase(yyruleno==498); + case 523: /* literal_list ::= signed_literal */ yytestcase(yyruleno==523); + case 618: /* other_para_list ::= star_func_para */ yytestcase(yyruleno==618); + case 624: /* when_then_list ::= when_then_expr */ yytestcase(yyruleno==624); + case 700: /* select_list ::= select_item */ yytestcase(yyruleno==700); + case 711: /* partition_list ::= partition_item */ yytestcase(yyruleno==711); + case 774: /* sort_specification_list ::= sort_specification */ yytestcase(yyruleno==774); +{ yylhsminor.yy628 = createNodeList(pCxt, yymsp[0].minor.yy980); } + yymsp[0].minor.yy628 = yylhsminor.yy628; break; case 173: /* retention_list ::= retention_list NK_COMMA retention */ case 209: /* multi_drop_clause ::= multi_drop_clause NK_COMMA drop_table_clause */ yytestcase(yyruleno==209); @@ -6265,298 +6659,298 @@ static YYACTIONTYPE yy_reduce( case 346: /* tag_list_opt ::= tag_list_opt NK_COMMA tag_item */ yytestcase(yyruleno==346); case 370: /* func_list ::= func_list NK_COMMA func */ yytestcase(yyruleno==370); case 420: /* column_stream_def_list ::= column_stream_def_list NK_COMMA column_stream_def */ yytestcase(yyruleno==420); - case 498: /* tags_literal_list ::= tags_literal_list NK_COMMA tags_literal */ yytestcase(yyruleno==498); - case 523: /* literal_list ::= literal_list NK_COMMA signed_literal */ yytestcase(yyruleno==523); - case 618: /* other_para_list ::= other_para_list NK_COMMA star_func_para */ yytestcase(yyruleno==618); - case 700: /* select_list ::= select_list NK_COMMA select_item */ yytestcase(yyruleno==700); - case 711: /* partition_list ::= partition_list NK_COMMA partition_item */ yytestcase(yyruleno==711); - case 774: /* sort_specification_list ::= sort_specification_list NK_COMMA sort_specification */ yytestcase(yyruleno==774); -{ yylhsminor.yy946 = addNodeToList(pCxt, yymsp[-2].minor.yy946, yymsp[0].minor.yy974); } - yymsp[-2].minor.yy946 = yylhsminor.yy946; + case 499: /* tags_literal_list ::= tags_literal_list NK_COMMA tags_literal */ yytestcase(yyruleno==499); + case 524: /* literal_list ::= literal_list NK_COMMA signed_literal */ yytestcase(yyruleno==524); + case 619: /* other_para_list ::= other_para_list NK_COMMA star_func_para */ yytestcase(yyruleno==619); + case 701: /* select_list ::= select_list NK_COMMA select_item */ yytestcase(yyruleno==701); + case 712: /* partition_list ::= partition_list NK_COMMA partition_item */ yytestcase(yyruleno==712); + case 775: /* sort_specification_list ::= sort_specification_list NK_COMMA sort_specification */ yytestcase(yyruleno==775); +{ yylhsminor.yy628 = addNodeToList(pCxt, yymsp[-2].minor.yy628, yymsp[0].minor.yy980); } + yymsp[-2].minor.yy628 = yylhsminor.yy628; break; case 174: /* retention ::= NK_VARIABLE NK_COLON NK_VARIABLE */ case 175: /* retention ::= NK_MINUS NK_COLON NK_VARIABLE */ yytestcase(yyruleno==175); -{ yylhsminor.yy974 = createNodeListNodeEx(pCxt, createDurationValueNode(pCxt, &yymsp[-2].minor.yy0), createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createNodeListNodeEx(pCxt, createDurationValueNode(pCxt, &yymsp[-2].minor.yy0), createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 176: /* speed_opt ::= */ case 403: /* bufsize_opt ::= */ yytestcase(yyruleno==403); -{ yymsp[1].minor.yy904 = 0; } +{ yymsp[1].minor.yy844 = 0; } break; case 177: /* speed_opt ::= BWLIMIT NK_INTEGER */ case 404: /* bufsize_opt ::= BUFSIZE NK_INTEGER */ yytestcase(yyruleno==404); -{ yymsp[-1].minor.yy904 = taosStr2Int32(yymsp[0].minor.yy0.z, NULL, 10); } +{ yymsp[-1].minor.yy844 = taosStr2Int32(yymsp[0].minor.yy0.z, NULL, 10); } break; case 179: /* start_opt ::= START WITH NK_INTEGER */ case 183: /* end_opt ::= END WITH NK_INTEGER */ yytestcase(yyruleno==183); -{ yymsp[-2].minor.yy974 = createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0); } +{ yymsp[-2].minor.yy980 = createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0); } break; case 180: /* start_opt ::= START WITH NK_STRING */ case 184: /* end_opt ::= END WITH NK_STRING */ yytestcase(yyruleno==184); -{ yymsp[-2].minor.yy974 = createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0); } +{ yymsp[-2].minor.yy980 = createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0); } break; case 181: /* start_opt ::= START WITH TIMESTAMP NK_STRING */ case 185: /* end_opt ::= END WITH TIMESTAMP NK_STRING */ yytestcase(yyruleno==185); -{ yymsp[-3].minor.yy974 = createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0); } +{ yymsp[-3].minor.yy980 = createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0); } break; case 186: /* cmd ::= CREATE TABLE not_exists_opt full_table_name NK_LP column_def_list NK_RP tags_def_opt table_options */ case 189: /* cmd ::= CREATE STABLE not_exists_opt full_table_name NK_LP column_def_list NK_RP tags_def table_options */ yytestcase(yyruleno==189); -{ pCxt->pRootNode = createCreateTableStmt(pCxt, yymsp[-6].minor.yy569, yymsp[-5].minor.yy974, yymsp[-3].minor.yy946, yymsp[-1].minor.yy946, yymsp[0].minor.yy974); } +{ pCxt->pRootNode = createCreateTableStmt(pCxt, yymsp[-6].minor.yy957, yymsp[-5].minor.yy980, yymsp[-3].minor.yy628, yymsp[-1].minor.yy628, yymsp[0].minor.yy980); } break; case 187: /* cmd ::= CREATE TABLE multi_create_clause */ -{ pCxt->pRootNode = createCreateMultiTableStmt(pCxt, yymsp[0].minor.yy946); } +{ pCxt->pRootNode = createCreateMultiTableStmt(pCxt, yymsp[0].minor.yy628); } break; case 188: /* cmd ::= CREATE TABLE not_exists_opt USING full_table_name NK_LP tag_list_opt NK_RP FILE NK_STRING */ -{ pCxt->pRootNode = createCreateSubTableFromFileClause(pCxt, yymsp[-7].minor.yy569, yymsp[-5].minor.yy974, yymsp[-3].minor.yy946, &yymsp[0].minor.yy0); } +{ pCxt->pRootNode = createCreateSubTableFromFileClause(pCxt, yymsp[-7].minor.yy957, yymsp[-5].minor.yy980, yymsp[-3].minor.yy628, &yymsp[0].minor.yy0); } break; case 190: /* cmd ::= DROP TABLE with_opt multi_drop_clause */ -{ pCxt->pRootNode = createDropTableStmt(pCxt, yymsp[-1].minor.yy569, yymsp[0].minor.yy946); } +{ pCxt->pRootNode = createDropTableStmt(pCxt, yymsp[-1].minor.yy957, yymsp[0].minor.yy628); } break; case 191: /* cmd ::= DROP STABLE with_opt exists_opt full_table_name */ -{ pCxt->pRootNode = createDropSuperTableStmt(pCxt, yymsp[-2].minor.yy569, yymsp[-1].minor.yy569, yymsp[0].minor.yy974); } +{ pCxt->pRootNode = createDropSuperTableStmt(pCxt, yymsp[-2].minor.yy957, yymsp[-1].minor.yy957, yymsp[0].minor.yy980); } break; case 192: /* cmd ::= ALTER TABLE alter_table_clause */ - case 455: /* cmd ::= query_or_subquery */ yytestcase(yyruleno==455); - case 456: /* cmd ::= insert_query */ yytestcase(yyruleno==456); -{ pCxt->pRootNode = yymsp[0].minor.yy974; } + case 456: /* cmd ::= query_or_subquery */ yytestcase(yyruleno==456); + case 457: /* cmd ::= insert_query */ yytestcase(yyruleno==457); +{ pCxt->pRootNode = yymsp[0].minor.yy980; } break; case 193: /* cmd ::= ALTER STABLE alter_table_clause */ -{ pCxt->pRootNode = setAlterSuperTableType(yymsp[0].minor.yy974); } +{ pCxt->pRootNode = setAlterSuperTableType(yymsp[0].minor.yy980); } break; case 194: /* alter_table_clause ::= full_table_name alter_table_options */ -{ yylhsminor.yy974 = createAlterTableModifyOptions(pCxt, yymsp[-1].minor.yy974, yymsp[0].minor.yy974); } - yymsp[-1].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createAlterTableModifyOptions(pCxt, yymsp[-1].minor.yy980, yymsp[0].minor.yy980); } + yymsp[-1].minor.yy980 = yylhsminor.yy980; break; case 195: /* alter_table_clause ::= full_table_name ADD COLUMN column_name type_name column_options */ -{ yylhsminor.yy974 = createAlterTableAddModifyColOptions2(pCxt, yymsp[-5].minor.yy974, TSDB_ALTER_TABLE_ADD_COLUMN, &yymsp[-2].minor.yy557, yymsp[-1].minor.yy424, yymsp[0].minor.yy974); } - yymsp[-5].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createAlterTableAddModifyColOptions2(pCxt, yymsp[-5].minor.yy980, TSDB_ALTER_TABLE_ADD_COLUMN, &yymsp[-2].minor.yy561, yymsp[-1].minor.yy896, yymsp[0].minor.yy980); } + yymsp[-5].minor.yy980 = yylhsminor.yy980; break; case 196: /* alter_table_clause ::= full_table_name DROP COLUMN column_name */ -{ yylhsminor.yy974 = createAlterTableDropCol(pCxt, yymsp[-3].minor.yy974, TSDB_ALTER_TABLE_DROP_COLUMN, &yymsp[0].minor.yy557); } - yymsp[-3].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createAlterTableDropCol(pCxt, yymsp[-3].minor.yy980, TSDB_ALTER_TABLE_DROP_COLUMN, &yymsp[0].minor.yy561); } + yymsp[-3].minor.yy980 = yylhsminor.yy980; break; case 197: /* alter_table_clause ::= full_table_name MODIFY COLUMN column_name type_name */ -{ yylhsminor.yy974 = createAlterTableAddModifyCol(pCxt, yymsp[-4].minor.yy974, TSDB_ALTER_TABLE_UPDATE_COLUMN_BYTES, &yymsp[-1].minor.yy557, yymsp[0].minor.yy424); } - yymsp[-4].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createAlterTableAddModifyCol(pCxt, yymsp[-4].minor.yy980, TSDB_ALTER_TABLE_UPDATE_COLUMN_BYTES, &yymsp[-1].minor.yy561, yymsp[0].minor.yy896); } + yymsp[-4].minor.yy980 = yylhsminor.yy980; break; case 198: /* alter_table_clause ::= full_table_name MODIFY COLUMN column_name column_options */ -{ yylhsminor.yy974 = createAlterTableAddModifyColOptions(pCxt, yymsp[-4].minor.yy974, TSDB_ALTER_TABLE_UPDATE_COLUMN_COMPRESS, &yymsp[-1].minor.yy557, yymsp[0].minor.yy974); } - yymsp[-4].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createAlterTableAddModifyColOptions(pCxt, yymsp[-4].minor.yy980, TSDB_ALTER_TABLE_UPDATE_COLUMN_COMPRESS, &yymsp[-1].minor.yy561, yymsp[0].minor.yy980); } + yymsp[-4].minor.yy980 = yylhsminor.yy980; break; case 199: /* alter_table_clause ::= full_table_name RENAME COLUMN column_name column_name */ -{ yylhsminor.yy974 = createAlterTableRenameCol(pCxt, yymsp[-4].minor.yy974, TSDB_ALTER_TABLE_UPDATE_COLUMN_NAME, &yymsp[-1].minor.yy557, &yymsp[0].minor.yy557); } - yymsp[-4].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createAlterTableRenameCol(pCxt, yymsp[-4].minor.yy980, TSDB_ALTER_TABLE_UPDATE_COLUMN_NAME, &yymsp[-1].minor.yy561, &yymsp[0].minor.yy561); } + yymsp[-4].minor.yy980 = yylhsminor.yy980; break; case 200: /* alter_table_clause ::= full_table_name ADD TAG column_name type_name */ -{ yylhsminor.yy974 = createAlterTableAddModifyCol(pCxt, yymsp[-4].minor.yy974, TSDB_ALTER_TABLE_ADD_TAG, &yymsp[-1].minor.yy557, yymsp[0].minor.yy424); } - yymsp[-4].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createAlterTableAddModifyCol(pCxt, yymsp[-4].minor.yy980, TSDB_ALTER_TABLE_ADD_TAG, &yymsp[-1].minor.yy561, yymsp[0].minor.yy896); } + yymsp[-4].minor.yy980 = yylhsminor.yy980; break; case 201: /* alter_table_clause ::= full_table_name DROP TAG column_name */ -{ yylhsminor.yy974 = createAlterTableDropCol(pCxt, yymsp[-3].minor.yy974, TSDB_ALTER_TABLE_DROP_TAG, &yymsp[0].minor.yy557); } - yymsp[-3].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createAlterTableDropCol(pCxt, yymsp[-3].minor.yy980, TSDB_ALTER_TABLE_DROP_TAG, &yymsp[0].minor.yy561); } + yymsp[-3].minor.yy980 = yylhsminor.yy980; break; case 202: /* alter_table_clause ::= full_table_name MODIFY TAG column_name type_name */ -{ yylhsminor.yy974 = createAlterTableAddModifyCol(pCxt, yymsp[-4].minor.yy974, TSDB_ALTER_TABLE_UPDATE_TAG_BYTES, &yymsp[-1].minor.yy557, yymsp[0].minor.yy424); } - yymsp[-4].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createAlterTableAddModifyCol(pCxt, yymsp[-4].minor.yy980, TSDB_ALTER_TABLE_UPDATE_TAG_BYTES, &yymsp[-1].minor.yy561, yymsp[0].minor.yy896); } + yymsp[-4].minor.yy980 = yylhsminor.yy980; break; case 203: /* alter_table_clause ::= full_table_name RENAME TAG column_name column_name */ -{ yylhsminor.yy974 = createAlterTableRenameCol(pCxt, yymsp[-4].minor.yy974, TSDB_ALTER_TABLE_UPDATE_TAG_NAME, &yymsp[-1].minor.yy557, &yymsp[0].minor.yy557); } - yymsp[-4].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createAlterTableRenameCol(pCxt, yymsp[-4].minor.yy980, TSDB_ALTER_TABLE_UPDATE_TAG_NAME, &yymsp[-1].minor.yy561, &yymsp[0].minor.yy561); } + yymsp[-4].minor.yy980 = yylhsminor.yy980; break; case 204: /* alter_table_clause ::= full_table_name SET TAG column_name NK_EQ tags_literal */ -{ yylhsminor.yy974 = createAlterTableSetTag(pCxt, yymsp[-5].minor.yy974, &yymsp[-2].minor.yy557, yymsp[0].minor.yy974); } - yymsp[-5].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createAlterTableSetTag(pCxt, yymsp[-5].minor.yy980, &yymsp[-2].minor.yy561, yymsp[0].minor.yy980); } + yymsp[-5].minor.yy980 = yylhsminor.yy980; break; case 206: /* multi_create_clause ::= multi_create_clause create_subtable_clause */ - case 624: /* when_then_list ::= when_then_list when_then_expr */ yytestcase(yyruleno==624); -{ yylhsminor.yy946 = addNodeToList(pCxt, yymsp[-1].minor.yy946, yymsp[0].minor.yy974); } - yymsp[-1].minor.yy946 = yylhsminor.yy946; + case 625: /* when_then_list ::= when_then_list when_then_expr */ yytestcase(yyruleno==625); +{ yylhsminor.yy628 = addNodeToList(pCxt, yymsp[-1].minor.yy628, yymsp[0].minor.yy980); } + yymsp[-1].minor.yy628 = yylhsminor.yy628; break; case 207: /* create_subtable_clause ::= not_exists_opt full_table_name USING full_table_name specific_cols_opt TAGS NK_LP tags_literal_list NK_RP table_options */ -{ yylhsminor.yy974 = createCreateSubTableClause(pCxt, yymsp[-9].minor.yy569, yymsp[-8].minor.yy974, yymsp[-6].minor.yy974, yymsp[-5].minor.yy946, yymsp[-2].minor.yy946, yymsp[0].minor.yy974); } - yymsp[-9].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createCreateSubTableClause(pCxt, yymsp[-9].minor.yy957, yymsp[-8].minor.yy980, yymsp[-6].minor.yy980, yymsp[-5].minor.yy628, yymsp[-2].minor.yy628, yymsp[0].minor.yy980); } + yymsp[-9].minor.yy980 = yylhsminor.yy980; break; case 210: /* drop_table_clause ::= exists_opt full_table_name */ -{ yylhsminor.yy974 = createDropTableClause(pCxt, yymsp[-1].minor.yy569, yymsp[0].minor.yy974); } - yymsp[-1].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createDropTableClause(pCxt, yymsp[-1].minor.yy957, yymsp[0].minor.yy980); } + yymsp[-1].minor.yy980 = yylhsminor.yy980; break; case 214: /* specific_cols_opt ::= NK_LP col_name_list NK_RP */ case 418: /* col_list_opt ::= NK_LP column_stream_def_list NK_RP */ yytestcase(yyruleno==418); -{ yymsp[-2].minor.yy946 = yymsp[-1].minor.yy946; } +{ yymsp[-2].minor.yy628 = yymsp[-1].minor.yy628; } break; case 215: /* full_table_name ::= table_name */ case 359: /* full_tsma_name ::= tsma_name */ yytestcase(yyruleno==359); -{ yylhsminor.yy974 = createRealTableNode(pCxt, NULL, &yymsp[0].minor.yy557, NULL); } - yymsp[0].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createRealTableNode(pCxt, NULL, &yymsp[0].minor.yy561, NULL); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; case 216: /* full_table_name ::= db_name NK_DOT table_name */ case 360: /* full_tsma_name ::= db_name NK_DOT tsma_name */ yytestcase(yyruleno==360); -{ yylhsminor.yy974 = createRealTableNode(pCxt, &yymsp[-2].minor.yy557, &yymsp[0].minor.yy557, NULL); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createRealTableNode(pCxt, &yymsp[-2].minor.yy561, &yymsp[0].minor.yy561, NULL); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 219: /* tag_def ::= column_name type_name */ -{ yylhsminor.yy974 = createColumnDefNode(pCxt, &yymsp[-1].minor.yy557, yymsp[0].minor.yy424, NULL); } - yymsp[-1].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createColumnDefNode(pCxt, &yymsp[-1].minor.yy561, yymsp[0].minor.yy896, NULL); } + yymsp[-1].minor.yy980 = yylhsminor.yy980; break; case 222: /* column_def ::= column_name type_name column_options */ -{ yylhsminor.yy974 = createColumnDefNode(pCxt, &yymsp[-2].minor.yy557, yymsp[-1].minor.yy424, yymsp[0].minor.yy974); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createColumnDefNode(pCxt, &yymsp[-2].minor.yy561, yymsp[-1].minor.yy896, yymsp[0].minor.yy980); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 223: /* type_name ::= BOOL */ -{ yymsp[0].minor.yy424 = createDataType(TSDB_DATA_TYPE_BOOL); } +{ yymsp[0].minor.yy896 = createDataType(TSDB_DATA_TYPE_BOOL); } break; case 224: /* type_name ::= TINYINT */ -{ yymsp[0].minor.yy424 = createDataType(TSDB_DATA_TYPE_TINYINT); } +{ yymsp[0].minor.yy896 = createDataType(TSDB_DATA_TYPE_TINYINT); } break; case 225: /* type_name ::= SMALLINT */ -{ yymsp[0].minor.yy424 = createDataType(TSDB_DATA_TYPE_SMALLINT); } +{ yymsp[0].minor.yy896 = createDataType(TSDB_DATA_TYPE_SMALLINT); } break; case 226: /* type_name ::= INT */ case 227: /* type_name ::= INTEGER */ yytestcase(yyruleno==227); -{ yymsp[0].minor.yy424 = createDataType(TSDB_DATA_TYPE_INT); } +{ yymsp[0].minor.yy896 = createDataType(TSDB_DATA_TYPE_INT); } break; case 228: /* type_name ::= BIGINT */ -{ yymsp[0].minor.yy424 = createDataType(TSDB_DATA_TYPE_BIGINT); } +{ yymsp[0].minor.yy896 = createDataType(TSDB_DATA_TYPE_BIGINT); } break; case 229: /* type_name ::= FLOAT */ -{ yymsp[0].minor.yy424 = createDataType(TSDB_DATA_TYPE_FLOAT); } +{ yymsp[0].minor.yy896 = createDataType(TSDB_DATA_TYPE_FLOAT); } break; case 230: /* type_name ::= DOUBLE */ -{ yymsp[0].minor.yy424 = createDataType(TSDB_DATA_TYPE_DOUBLE); } +{ yymsp[0].minor.yy896 = createDataType(TSDB_DATA_TYPE_DOUBLE); } break; case 231: /* type_name ::= BINARY NK_LP NK_INTEGER NK_RP */ -{ yymsp[-3].minor.yy424 = createVarLenDataType(TSDB_DATA_TYPE_BINARY, &yymsp[-1].minor.yy0); } +{ yymsp[-3].minor.yy896 = createVarLenDataType(TSDB_DATA_TYPE_BINARY, &yymsp[-1].minor.yy0); } break; case 232: /* type_name ::= TIMESTAMP */ -{ yymsp[0].minor.yy424 = createDataType(TSDB_DATA_TYPE_TIMESTAMP); } +{ yymsp[0].minor.yy896 = createDataType(TSDB_DATA_TYPE_TIMESTAMP); } break; case 233: /* type_name ::= NCHAR NK_LP NK_INTEGER NK_RP */ -{ yymsp[-3].minor.yy424 = createVarLenDataType(TSDB_DATA_TYPE_NCHAR, &yymsp[-1].minor.yy0); } +{ yymsp[-3].minor.yy896 = createVarLenDataType(TSDB_DATA_TYPE_NCHAR, &yymsp[-1].minor.yy0); } break; case 234: /* type_name ::= TINYINT UNSIGNED */ -{ yymsp[-1].minor.yy424 = createDataType(TSDB_DATA_TYPE_UTINYINT); } +{ yymsp[-1].minor.yy896 = createDataType(TSDB_DATA_TYPE_UTINYINT); } break; case 235: /* type_name ::= SMALLINT UNSIGNED */ -{ yymsp[-1].minor.yy424 = createDataType(TSDB_DATA_TYPE_USMALLINT); } +{ yymsp[-1].minor.yy896 = createDataType(TSDB_DATA_TYPE_USMALLINT); } break; case 236: /* type_name ::= INT UNSIGNED */ -{ yymsp[-1].minor.yy424 = createDataType(TSDB_DATA_TYPE_UINT); } +{ yymsp[-1].minor.yy896 = createDataType(TSDB_DATA_TYPE_UINT); } break; case 237: /* type_name ::= BIGINT UNSIGNED */ -{ yymsp[-1].minor.yy424 = createDataType(TSDB_DATA_TYPE_UBIGINT); } +{ yymsp[-1].minor.yy896 = createDataType(TSDB_DATA_TYPE_UBIGINT); } break; case 238: /* type_name ::= JSON */ -{ yymsp[0].minor.yy424 = createDataType(TSDB_DATA_TYPE_JSON); } +{ yymsp[0].minor.yy896 = createDataType(TSDB_DATA_TYPE_JSON); } break; case 239: /* type_name ::= VARCHAR NK_LP NK_INTEGER NK_RP */ -{ yymsp[-3].minor.yy424 = createVarLenDataType(TSDB_DATA_TYPE_VARCHAR, &yymsp[-1].minor.yy0); } +{ yymsp[-3].minor.yy896 = createVarLenDataType(TSDB_DATA_TYPE_VARCHAR, &yymsp[-1].minor.yy0); } break; case 240: /* type_name ::= MEDIUMBLOB */ -{ yymsp[0].minor.yy424 = createDataType(TSDB_DATA_TYPE_MEDIUMBLOB); } +{ yymsp[0].minor.yy896 = createDataType(TSDB_DATA_TYPE_MEDIUMBLOB); } break; case 241: /* type_name ::= BLOB */ -{ yymsp[0].minor.yy424 = createDataType(TSDB_DATA_TYPE_BLOB); } +{ yymsp[0].minor.yy896 = createDataType(TSDB_DATA_TYPE_BLOB); } break; case 242: /* type_name ::= VARBINARY NK_LP NK_INTEGER NK_RP */ -{ yymsp[-3].minor.yy424 = createVarLenDataType(TSDB_DATA_TYPE_VARBINARY, &yymsp[-1].minor.yy0); } +{ yymsp[-3].minor.yy896 = createVarLenDataType(TSDB_DATA_TYPE_VARBINARY, &yymsp[-1].minor.yy0); } break; case 243: /* type_name ::= GEOMETRY NK_LP NK_INTEGER NK_RP */ -{ yymsp[-3].minor.yy424 = createVarLenDataType(TSDB_DATA_TYPE_GEOMETRY, &yymsp[-1].minor.yy0); } +{ yymsp[-3].minor.yy896 = createVarLenDataType(TSDB_DATA_TYPE_GEOMETRY, &yymsp[-1].minor.yy0); } break; case 244: /* type_name ::= DECIMAL */ -{ yymsp[0].minor.yy424 = createDataType(TSDB_DATA_TYPE_DECIMAL); } +{ yymsp[0].minor.yy896 = createDataType(TSDB_DATA_TYPE_DECIMAL); } break; case 245: /* type_name ::= DECIMAL NK_LP NK_INTEGER NK_RP */ -{ yymsp[-3].minor.yy424 = createDataType(TSDB_DATA_TYPE_DECIMAL); } +{ yymsp[-3].minor.yy896 = createDataType(TSDB_DATA_TYPE_DECIMAL); } break; case 246: /* type_name ::= DECIMAL NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP */ -{ yymsp[-5].minor.yy424 = createDataType(TSDB_DATA_TYPE_DECIMAL); } +{ yymsp[-5].minor.yy896 = createDataType(TSDB_DATA_TYPE_DECIMAL); } break; case 247: /* type_name_default_len ::= BINARY */ -{ yymsp[0].minor.yy424 = createVarLenDataType(TSDB_DATA_TYPE_BINARY, NULL); } +{ yymsp[0].minor.yy896 = createVarLenDataType(TSDB_DATA_TYPE_BINARY, NULL); } break; case 248: /* type_name_default_len ::= NCHAR */ -{ yymsp[0].minor.yy424 = createVarLenDataType(TSDB_DATA_TYPE_NCHAR, NULL); } +{ yymsp[0].minor.yy896 = createVarLenDataType(TSDB_DATA_TYPE_NCHAR, NULL); } break; case 249: /* type_name_default_len ::= VARCHAR */ -{ yymsp[0].minor.yy424 = createVarLenDataType(TSDB_DATA_TYPE_VARCHAR, NULL); } +{ yymsp[0].minor.yy896 = createVarLenDataType(TSDB_DATA_TYPE_VARCHAR, NULL); } break; case 250: /* type_name_default_len ::= VARBINARY */ -{ yymsp[0].minor.yy424 = createVarLenDataType(TSDB_DATA_TYPE_VARBINARY, NULL); } +{ yymsp[0].minor.yy896 = createVarLenDataType(TSDB_DATA_TYPE_VARBINARY, NULL); } break; case 253: /* tags_def ::= TAGS NK_LP tag_def_list NK_RP */ case 426: /* tag_def_or_ref_opt ::= TAGS NK_LP column_stream_def_list NK_RP */ yytestcase(yyruleno==426); -{ yymsp[-3].minor.yy946 = yymsp[-1].minor.yy946; } +{ yymsp[-3].minor.yy628 = yymsp[-1].minor.yy628; } break; case 254: /* table_options ::= */ -{ yymsp[1].minor.yy974 = createDefaultTableOptions(pCxt); } +{ yymsp[1].minor.yy980 = createDefaultTableOptions(pCxt); } break; case 255: /* table_options ::= table_options COMMENT NK_STRING */ -{ yylhsminor.yy974 = setTableOption(pCxt, yymsp[-2].minor.yy974, TABLE_OPTION_COMMENT, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setTableOption(pCxt, yymsp[-2].minor.yy980, TABLE_OPTION_COMMENT, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 256: /* table_options ::= table_options MAX_DELAY duration_list */ -{ yylhsminor.yy974 = setTableOption(pCxt, yymsp[-2].minor.yy974, TABLE_OPTION_MAXDELAY, yymsp[0].minor.yy946); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setTableOption(pCxt, yymsp[-2].minor.yy980, TABLE_OPTION_MAXDELAY, yymsp[0].minor.yy628); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 257: /* table_options ::= table_options WATERMARK duration_list */ -{ yylhsminor.yy974 = setTableOption(pCxt, yymsp[-2].minor.yy974, TABLE_OPTION_WATERMARK, yymsp[0].minor.yy946); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setTableOption(pCxt, yymsp[-2].minor.yy980, TABLE_OPTION_WATERMARK, yymsp[0].minor.yy628); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 258: /* table_options ::= table_options ROLLUP NK_LP rollup_func_list NK_RP */ -{ yylhsminor.yy974 = setTableOption(pCxt, yymsp[-4].minor.yy974, TABLE_OPTION_ROLLUP, yymsp[-1].minor.yy946); } - yymsp[-4].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setTableOption(pCxt, yymsp[-4].minor.yy980, TABLE_OPTION_ROLLUP, yymsp[-1].minor.yy628); } + yymsp[-4].minor.yy980 = yylhsminor.yy980; break; case 259: /* table_options ::= table_options TTL NK_INTEGER */ -{ yylhsminor.yy974 = setTableOption(pCxt, yymsp[-2].minor.yy974, TABLE_OPTION_TTL, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setTableOption(pCxt, yymsp[-2].minor.yy980, TABLE_OPTION_TTL, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 260: /* table_options ::= table_options SMA NK_LP col_name_list NK_RP */ -{ yylhsminor.yy974 = setTableOption(pCxt, yymsp[-4].minor.yy974, TABLE_OPTION_SMA, yymsp[-1].minor.yy946); } - yymsp[-4].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setTableOption(pCxt, yymsp[-4].minor.yy980, TABLE_OPTION_SMA, yymsp[-1].minor.yy628); } + yymsp[-4].minor.yy980 = yylhsminor.yy980; break; case 261: /* table_options ::= table_options DELETE_MARK duration_list */ -{ yylhsminor.yy974 = setTableOption(pCxt, yymsp[-2].minor.yy974, TABLE_OPTION_DELETE_MARK, yymsp[0].minor.yy946); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setTableOption(pCxt, yymsp[-2].minor.yy980, TABLE_OPTION_DELETE_MARK, yymsp[0].minor.yy628); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 262: /* alter_table_options ::= alter_table_option */ -{ yylhsminor.yy974 = createAlterTableOptions(pCxt); yylhsminor.yy974 = setTableOption(pCxt, yylhsminor.yy974, yymsp[0].minor.yy683.type, &yymsp[0].minor.yy683.val); } - yymsp[0].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createAlterTableOptions(pCxt); yylhsminor.yy980 = setTableOption(pCxt, yylhsminor.yy980, yymsp[0].minor.yy529.type, &yymsp[0].minor.yy529.val); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; case 263: /* alter_table_options ::= alter_table_options alter_table_option */ -{ yylhsminor.yy974 = setTableOption(pCxt, yymsp[-1].minor.yy974, yymsp[0].minor.yy683.type, &yymsp[0].minor.yy683.val); } - yymsp[-1].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setTableOption(pCxt, yymsp[-1].minor.yy980, yymsp[0].minor.yy529.type, &yymsp[0].minor.yy529.val); } + yymsp[-1].minor.yy980 = yylhsminor.yy980; break; case 264: /* alter_table_option ::= COMMENT NK_STRING */ -{ yymsp[-1].minor.yy683.type = TABLE_OPTION_COMMENT; yymsp[-1].minor.yy683.val = yymsp[0].minor.yy0; } +{ yymsp[-1].minor.yy529.type = TABLE_OPTION_COMMENT; yymsp[-1].minor.yy529.val = yymsp[0].minor.yy0; } break; case 265: /* alter_table_option ::= TTL NK_INTEGER */ -{ yymsp[-1].minor.yy683.type = TABLE_OPTION_TTL; yymsp[-1].minor.yy683.val = yymsp[0].minor.yy0; } +{ yymsp[-1].minor.yy529.type = TABLE_OPTION_TTL; yymsp[-1].minor.yy529.val = yymsp[0].minor.yy0; } break; case 266: /* duration_list ::= duration_literal */ - case 555: /* expression_list ::= expr_or_subquery */ yytestcase(yyruleno==555); -{ yylhsminor.yy946 = createNodeList(pCxt, releaseRawExprNode(pCxt, yymsp[0].minor.yy974)); } - yymsp[0].minor.yy946 = yylhsminor.yy946; + case 556: /* expression_list ::= expr_or_subquery */ yytestcase(yyruleno==556); +{ yylhsminor.yy628 = createNodeList(pCxt, releaseRawExprNode(pCxt, yymsp[0].minor.yy980)); } + yymsp[0].minor.yy628 = yylhsminor.yy628; break; case 267: /* duration_list ::= duration_list NK_COMMA duration_literal */ - case 556: /* expression_list ::= expression_list NK_COMMA expr_or_subquery */ yytestcase(yyruleno==556); -{ yylhsminor.yy946 = addNodeToList(pCxt, yymsp[-2].minor.yy946, releaseRawExprNode(pCxt, yymsp[0].minor.yy974)); } - yymsp[-2].minor.yy946 = yylhsminor.yy946; + case 557: /* expression_list ::= expression_list NK_COMMA expr_or_subquery */ yytestcase(yyruleno==557); +{ yylhsminor.yy628 = addNodeToList(pCxt, yymsp[-2].minor.yy628, releaseRawExprNode(pCxt, yymsp[0].minor.yy980)); } + yymsp[-2].minor.yy628 = yylhsminor.yy628; break; case 270: /* rollup_func_name ::= function_name */ -{ yylhsminor.yy974 = createFunctionNode(pCxt, &yymsp[0].minor.yy557, NULL); } - yymsp[0].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createFunctionNode(pCxt, &yymsp[0].minor.yy561, NULL); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; case 271: /* rollup_func_name ::= FIRST */ case 272: /* rollup_func_name ::= LAST */ yytestcase(yyruleno==272); case 348: /* tag_item ::= QTAGS */ yytestcase(yyruleno==348); -{ yylhsminor.yy974 = createFunctionNode(pCxt, &yymsp[0].minor.yy0, NULL); } - yymsp[0].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createFunctionNode(pCxt, &yymsp[0].minor.yy0, NULL); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; case 275: /* col_name ::= column_name */ case 349: /* tag_item ::= column_name */ yytestcase(yyruleno==349); -{ yylhsminor.yy974 = createColumnNode(pCxt, NULL, &yymsp[0].minor.yy557); } - yymsp[0].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createColumnNode(pCxt, NULL, &yymsp[0].minor.yy561); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; case 276: /* cmd ::= SHOW DNODES */ { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_DNODES_STMT); } @@ -6573,19 +6967,19 @@ static YYACTIONTYPE yy_reduce( case 280: /* cmd ::= SHOW db_kind_opt DATABASES */ { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_DATABASES_STMT); - (void)setShowKind(pCxt, pCxt->pRootNode, yymsp[-1].minor.yy741); + (void)setShowKind(pCxt, pCxt->pRootNode, yymsp[-1].minor.yy1041); } break; case 281: /* cmd ::= SHOW table_kind_db_name_cond_opt TABLES like_pattern_opt */ { - pCxt->pRootNode = createShowTablesStmt(pCxt, yymsp[-2].minor.yy595, yymsp[0].minor.yy974, OP_TYPE_LIKE); + pCxt->pRootNode = createShowTablesStmt(pCxt, yymsp[-2].minor.yy513, yymsp[0].minor.yy980, OP_TYPE_LIKE); } break; case 282: /* cmd ::= SHOW db_name_cond_opt STABLES like_pattern_opt */ -{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_STABLES_STMT, yymsp[-2].minor.yy974, yymsp[0].minor.yy974, OP_TYPE_LIKE); } +{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_STABLES_STMT, yymsp[-2].minor.yy980, yymsp[0].minor.yy980, OP_TYPE_LIKE); } break; case 283: /* cmd ::= SHOW db_name_cond_opt VGROUPS */ -{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_VGROUPS_STMT, yymsp[-1].minor.yy974, NULL, OP_TYPE_LIKE); } +{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_VGROUPS_STMT, yymsp[-1].minor.yy980, NULL, OP_TYPE_LIKE); } break; case 284: /* cmd ::= SHOW MNODES */ { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_MNODES_STMT); } @@ -6606,10 +7000,10 @@ static YYACTIONTYPE yy_reduce( { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_FUNCTIONS_STMT); } break; case 290: /* cmd ::= SHOW INDEXES FROM table_name_cond from_db_opt */ -{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_INDEXES_STMT, yymsp[0].minor.yy974, yymsp[-1].minor.yy974, OP_TYPE_EQUAL); } +{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_INDEXES_STMT, yymsp[0].minor.yy980, yymsp[-1].minor.yy980, OP_TYPE_EQUAL); } break; case 291: /* cmd ::= SHOW INDEXES FROM db_name NK_DOT table_name */ -{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_INDEXES_STMT, createIdentifierValueNode(pCxt, &yymsp[-2].minor.yy557), createIdentifierValueNode(pCxt, &yymsp[0].minor.yy557), OP_TYPE_EQUAL); } +{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_INDEXES_STMT, createIdentifierValueNode(pCxt, &yymsp[-2].minor.yy561), createIdentifierValueNode(pCxt, &yymsp[0].minor.yy561), OP_TYPE_EQUAL); } break; case 292: /* cmd ::= SHOW STREAMS */ { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_STREAMS_STMT); } @@ -6637,14 +7031,14 @@ static YYACTIONTYPE yy_reduce( { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_CLUSTER_MACHINES_STMT); } break; case 301: /* cmd ::= SHOW CREATE DATABASE db_name */ -{ pCxt->pRootNode = createShowCreateDatabaseStmt(pCxt, &yymsp[0].minor.yy557); } +{ pCxt->pRootNode = createShowCreateDatabaseStmt(pCxt, &yymsp[0].minor.yy561); } break; case 302: /* cmd ::= SHOW CREATE TABLE full_table_name */ -{ pCxt->pRootNode = createShowCreateTableStmt(pCxt, QUERY_NODE_SHOW_CREATE_TABLE_STMT, yymsp[0].minor.yy974); } +{ pCxt->pRootNode = createShowCreateTableStmt(pCxt, QUERY_NODE_SHOW_CREATE_TABLE_STMT, yymsp[0].minor.yy980); } break; case 303: /* cmd ::= SHOW CREATE STABLE full_table_name */ { pCxt->pRootNode = createShowCreateTableStmt(pCxt, QUERY_NODE_SHOW_CREATE_STABLE_STMT, -yymsp[0].minor.yy974); } +yymsp[0].minor.yy980); } break; case 304: /* cmd ::= SHOW ENCRYPTIONS */ { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_ENCRYPTIONS_STMT); } @@ -6666,7 +7060,7 @@ yymsp[0].minor.yy974); } { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_LOCAL_VARIABLES_STMT); } break; case 311: /* 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.yy974); } +{ pCxt->pRootNode = createShowDnodeVariablesStmt(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[-2].minor.yy0), yymsp[0].minor.yy980); } break; case 312: /* cmd ::= SHOW BNODES */ { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_BNODES_STMT); } @@ -6681,7 +7075,7 @@ yymsp[0].minor.yy974); } { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_TRANSACTIONS_STMT); } break; case 316: /* cmd ::= SHOW TABLE DISTRIBUTED full_table_name */ -{ pCxt->pRootNode = createShowTableDistributedStmt(pCxt, yymsp[0].minor.yy974); } +{ pCxt->pRootNode = createShowTableDistributedStmt(pCxt, yymsp[0].minor.yy980); } break; case 317: /* cmd ::= SHOW CONSUMERS */ { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_CONSUMERS_STMT); } @@ -6690,16 +7084,16 @@ yymsp[0].minor.yy974); } { pCxt->pRootNode = createShowStmt(pCxt, QUERY_NODE_SHOW_SUBSCRIPTIONS_STMT); } break; case 319: /* cmd ::= SHOW TAGS FROM table_name_cond from_db_opt */ -{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_TAGS_STMT, yymsp[0].minor.yy974, yymsp[-1].minor.yy974, OP_TYPE_EQUAL); } +{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_TAGS_STMT, yymsp[0].minor.yy980, yymsp[-1].minor.yy980, OP_TYPE_EQUAL); } break; case 320: /* cmd ::= SHOW TAGS FROM db_name NK_DOT table_name */ -{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_TAGS_STMT, createIdentifierValueNode(pCxt, &yymsp[-2].minor.yy557), createIdentifierValueNode(pCxt, &yymsp[0].minor.yy557), OP_TYPE_EQUAL); } +{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_TAGS_STMT, createIdentifierValueNode(pCxt, &yymsp[-2].minor.yy561), createIdentifierValueNode(pCxt, &yymsp[0].minor.yy561), OP_TYPE_EQUAL); } break; case 321: /* cmd ::= SHOW TABLE TAGS tag_list_opt FROM table_name_cond from_db_opt */ -{ pCxt->pRootNode = createShowTableTagsStmt(pCxt, yymsp[-1].minor.yy974, yymsp[0].minor.yy974, yymsp[-3].minor.yy946); } +{ pCxt->pRootNode = createShowTableTagsStmt(pCxt, yymsp[-1].minor.yy980, yymsp[0].minor.yy980, yymsp[-3].minor.yy628); } break; case 322: /* cmd ::= SHOW TABLE TAGS tag_list_opt FROM db_name NK_DOT table_name */ -{ pCxt->pRootNode = createShowTableTagsStmt(pCxt, createIdentifierValueNode(pCxt, &yymsp[0].minor.yy557), createIdentifierValueNode(pCxt, &yymsp[-2].minor.yy557), yymsp[-4].minor.yy946); } +{ pCxt->pRootNode = createShowTableTagsStmt(pCxt, createIdentifierValueNode(pCxt, &yymsp[0].minor.yy561), createIdentifierValueNode(pCxt, &yymsp[-2].minor.yy561), yymsp[-4].minor.yy628); } break; case 323: /* cmd ::= SHOW VNODES ON DNODE NK_INTEGER */ { pCxt->pRootNode = createShowVnodesStmt(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0), NULL); } @@ -6708,16 +7102,16 @@ yymsp[0].minor.yy974); } { pCxt->pRootNode = createShowVnodesStmt(pCxt, NULL, NULL); } break; case 325: /* cmd ::= SHOW db_name_cond_opt ALIVE */ -{ pCxt->pRootNode = createShowAliveStmt(pCxt, yymsp[-1].minor.yy974, QUERY_NODE_SHOW_DB_ALIVE_STMT); } +{ pCxt->pRootNode = createShowAliveStmt(pCxt, yymsp[-1].minor.yy980, QUERY_NODE_SHOW_DB_ALIVE_STMT); } break; case 326: /* cmd ::= SHOW CLUSTER ALIVE */ { pCxt->pRootNode = createShowAliveStmt(pCxt, NULL, QUERY_NODE_SHOW_CLUSTER_ALIVE_STMT); } break; case 327: /* cmd ::= SHOW db_name_cond_opt VIEWS like_pattern_opt */ -{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_VIEWS_STMT, yymsp[-2].minor.yy974, yymsp[0].minor.yy974, OP_TYPE_LIKE); } +{ pCxt->pRootNode = createShowStmtWithCond(pCxt, QUERY_NODE_SHOW_VIEWS_STMT, yymsp[-2].minor.yy980, yymsp[0].minor.yy980, OP_TYPE_LIKE); } break; case 328: /* cmd ::= SHOW CREATE VIEW full_table_name */ -{ pCxt->pRootNode = createShowCreateViewStmt(pCxt, QUERY_NODE_SHOW_CREATE_VIEW_STMT, yymsp[0].minor.yy974); } +{ pCxt->pRootNode = createShowCreateViewStmt(pCxt, QUERY_NODE_SHOW_CREATE_VIEW_STMT, yymsp[0].minor.yy980); } break; case 329: /* cmd ::= SHOW COMPACTS */ { pCxt->pRootNode = createShowCompactsStmt(pCxt, QUERY_NODE_SHOW_COMPACTS_STMT); } @@ -6726,1082 +7120,1083 @@ yymsp[0].minor.yy974); } { pCxt->pRootNode = createShowCompactDetailsStmt(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0)); } break; case 331: /* table_kind_db_name_cond_opt ::= */ -{ yymsp[1].minor.yy595.kind = SHOW_KIND_ALL; yymsp[1].minor.yy595.dbName = nil_token; } +{ yymsp[1].minor.yy513.kind = SHOW_KIND_ALL; yymsp[1].minor.yy513.dbName = nil_token; } break; case 332: /* table_kind_db_name_cond_opt ::= table_kind */ -{ yylhsminor.yy595.kind = yymsp[0].minor.yy741; yylhsminor.yy595.dbName = nil_token; } - yymsp[0].minor.yy595 = yylhsminor.yy595; +{ yylhsminor.yy513.kind = yymsp[0].minor.yy1041; yylhsminor.yy513.dbName = nil_token; } + yymsp[0].minor.yy513 = yylhsminor.yy513; break; case 333: /* table_kind_db_name_cond_opt ::= db_name NK_DOT */ -{ yylhsminor.yy595.kind = SHOW_KIND_ALL; yylhsminor.yy595.dbName = yymsp[-1].minor.yy557; } - yymsp[-1].minor.yy595 = yylhsminor.yy595; +{ yylhsminor.yy513.kind = SHOW_KIND_ALL; yylhsminor.yy513.dbName = yymsp[-1].minor.yy561; } + yymsp[-1].minor.yy513 = yylhsminor.yy513; break; case 334: /* table_kind_db_name_cond_opt ::= table_kind db_name NK_DOT */ -{ yylhsminor.yy595.kind = yymsp[-2].minor.yy741; yylhsminor.yy595.dbName = yymsp[-1].minor.yy557; } - yymsp[-2].minor.yy595 = yylhsminor.yy595; +{ yylhsminor.yy513.kind = yymsp[-2].minor.yy1041; yylhsminor.yy513.dbName = yymsp[-1].minor.yy561; } + yymsp[-2].minor.yy513 = yylhsminor.yy513; break; case 335: /* table_kind ::= NORMAL */ -{ yymsp[0].minor.yy741 = SHOW_KIND_TABLES_NORMAL; } +{ yymsp[0].minor.yy1041 = SHOW_KIND_TABLES_NORMAL; } break; case 336: /* table_kind ::= CHILD */ -{ yymsp[0].minor.yy741 = SHOW_KIND_TABLES_CHILD; } +{ yymsp[0].minor.yy1041 = SHOW_KIND_TABLES_CHILD; } break; case 337: /* db_name_cond_opt ::= */ case 342: /* from_db_opt ::= */ yytestcase(yyruleno==342); -{ yymsp[1].minor.yy974 = createDefaultDatabaseCondValue(pCxt); } +{ yymsp[1].minor.yy980 = createDefaultDatabaseCondValue(pCxt); } break; case 338: /* db_name_cond_opt ::= db_name NK_DOT */ -{ yylhsminor.yy974 = createIdentifierValueNode(pCxt, &yymsp[-1].minor.yy557); } - yymsp[-1].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createIdentifierValueNode(pCxt, &yymsp[-1].minor.yy561); } + yymsp[-1].minor.yy980 = yylhsminor.yy980; break; case 340: /* like_pattern_opt ::= LIKE NK_STRING */ -{ yymsp[-1].minor.yy974 = createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0); } +{ yymsp[-1].minor.yy980 = createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0); } break; case 341: /* table_name_cond ::= table_name */ -{ yylhsminor.yy974 = createIdentifierValueNode(pCxt, &yymsp[0].minor.yy557); } - yymsp[0].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createIdentifierValueNode(pCxt, &yymsp[0].minor.yy561); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; case 343: /* from_db_opt ::= FROM db_name */ -{ yymsp[-1].minor.yy974 = createIdentifierValueNode(pCxt, &yymsp[0].minor.yy557); } +{ yymsp[-1].minor.yy980 = createIdentifierValueNode(pCxt, &yymsp[0].minor.yy561); } break; case 347: /* tag_item ::= TBNAME */ -{ yylhsminor.yy974 = setProjectionAlias(pCxt, createFunctionNode(pCxt, &yymsp[0].minor.yy0, NULL), &yymsp[0].minor.yy0); } - yymsp[0].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setProjectionAlias(pCxt, createFunctionNode(pCxt, &yymsp[0].minor.yy0, NULL), &yymsp[0].minor.yy0); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; case 350: /* tag_item ::= column_name column_alias */ -{ yylhsminor.yy974 = setProjectionAlias(pCxt, createColumnNode(pCxt, NULL, &yymsp[-1].minor.yy557), &yymsp[0].minor.yy557); } - yymsp[-1].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setProjectionAlias(pCxt, createColumnNode(pCxt, NULL, &yymsp[-1].minor.yy561), &yymsp[0].minor.yy561); } + yymsp[-1].minor.yy980 = yylhsminor.yy980; break; case 351: /* tag_item ::= column_name AS column_alias */ -{ yylhsminor.yy974 = setProjectionAlias(pCxt, createColumnNode(pCxt, NULL, &yymsp[-2].minor.yy557), &yymsp[0].minor.yy557); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setProjectionAlias(pCxt, createColumnNode(pCxt, NULL, &yymsp[-2].minor.yy561), &yymsp[0].minor.yy561); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 352: /* db_kind_opt ::= */ -{ yymsp[1].minor.yy741 = SHOW_KIND_ALL; } +{ yymsp[1].minor.yy1041 = SHOW_KIND_ALL; } break; case 353: /* db_kind_opt ::= USER */ -{ yymsp[0].minor.yy741 = SHOW_KIND_DATABASES_USER; } +{ yymsp[0].minor.yy1041 = SHOW_KIND_DATABASES_USER; } break; case 354: /* db_kind_opt ::= SYSTEM */ -{ yymsp[0].minor.yy741 = SHOW_KIND_DATABASES_SYSTEM; } +{ yymsp[0].minor.yy1041 = SHOW_KIND_DATABASES_SYSTEM; } break; case 355: /* cmd ::= CREATE TSMA not_exists_opt tsma_name ON full_table_name tsma_func_list INTERVAL NK_LP duration_literal NK_RP */ -{ pCxt->pRootNode = createCreateTSMAStmt(pCxt, yymsp[-8].minor.yy569, &yymsp[-7].minor.yy557, yymsp[-4].minor.yy974, yymsp[-5].minor.yy974, releaseRawExprNode(pCxt, yymsp[-1].minor.yy974)); } +{ pCxt->pRootNode = createCreateTSMAStmt(pCxt, yymsp[-8].minor.yy957, &yymsp[-7].minor.yy561, yymsp[-4].minor.yy980, yymsp[-5].minor.yy980, releaseRawExprNode(pCxt, yymsp[-1].minor.yy980)); } break; case 356: /* cmd ::= CREATE RECURSIVE TSMA not_exists_opt tsma_name ON full_table_name INTERVAL NK_LP duration_literal NK_RP */ -{ pCxt->pRootNode = createCreateTSMAStmt(pCxt, yymsp[-7].minor.yy569, &yymsp[-6].minor.yy557, NULL, yymsp[-4].minor.yy974, releaseRawExprNode(pCxt, yymsp[-1].minor.yy974)); } +{ pCxt->pRootNode = createCreateTSMAStmt(pCxt, yymsp[-7].minor.yy957, &yymsp[-6].minor.yy561, NULL, yymsp[-4].minor.yy980, releaseRawExprNode(pCxt, yymsp[-1].minor.yy980)); } break; case 357: /* cmd ::= DROP TSMA exists_opt full_tsma_name */ -{ pCxt->pRootNode = createDropTSMAStmt(pCxt, yymsp[-1].minor.yy569, yymsp[0].minor.yy974); } +{ pCxt->pRootNode = createDropTSMAStmt(pCxt, yymsp[-1].minor.yy957, yymsp[0].minor.yy980); } break; case 358: /* cmd ::= SHOW db_name_cond_opt TSMAS */ -{ pCxt->pRootNode = createShowTSMASStmt(pCxt, yymsp[-1].minor.yy974); } +{ pCxt->pRootNode = createShowTSMASStmt(pCxt, yymsp[-1].minor.yy980); } break; case 361: /* tsma_func_list ::= FUNCTION NK_LP func_list NK_RP */ -{ yymsp[-3].minor.yy974 = createTSMAOptions(pCxt, yymsp[-1].minor.yy946); } +{ yymsp[-3].minor.yy980 = createTSMAOptions(pCxt, yymsp[-1].minor.yy628); } break; case 362: /* cmd ::= CREATE SMA INDEX not_exists_opt col_name ON full_table_name index_options */ -{ pCxt->pRootNode = createCreateIndexStmt(pCxt, INDEX_TYPE_SMA, yymsp[-4].minor.yy569, yymsp[-3].minor.yy974, yymsp[-1].minor.yy974, NULL, yymsp[0].minor.yy974); } +{ pCxt->pRootNode = createCreateIndexStmt(pCxt, INDEX_TYPE_SMA, yymsp[-4].minor.yy957, yymsp[-3].minor.yy980, yymsp[-1].minor.yy980, NULL, yymsp[0].minor.yy980); } break; case 363: /* cmd ::= CREATE INDEX not_exists_opt col_name ON full_table_name NK_LP col_name_list NK_RP */ -{ pCxt->pRootNode = createCreateIndexStmt(pCxt, INDEX_TYPE_NORMAL, yymsp[-6].minor.yy569, yymsp[-5].minor.yy974, yymsp[-3].minor.yy974, yymsp[-1].minor.yy946, NULL); } +{ pCxt->pRootNode = createCreateIndexStmt(pCxt, INDEX_TYPE_NORMAL, yymsp[-6].minor.yy957, yymsp[-5].minor.yy980, yymsp[-3].minor.yy980, yymsp[-1].minor.yy628, NULL); } break; case 364: /* cmd ::= DROP INDEX exists_opt full_index_name */ -{ pCxt->pRootNode = createDropIndexStmt(pCxt, yymsp[-1].minor.yy569, yymsp[0].minor.yy974); } +{ pCxt->pRootNode = createDropIndexStmt(pCxt, yymsp[-1].minor.yy957, yymsp[0].minor.yy980); } break; case 365: /* full_index_name ::= index_name */ -{ yylhsminor.yy974 = createRealTableNodeForIndexName(pCxt, NULL, &yymsp[0].minor.yy557); } - yymsp[0].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createRealTableNodeForIndexName(pCxt, NULL, &yymsp[0].minor.yy561); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; case 366: /* full_index_name ::= db_name NK_DOT index_name */ -{ yylhsminor.yy974 = createRealTableNodeForIndexName(pCxt, &yymsp[-2].minor.yy557, &yymsp[0].minor.yy557); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createRealTableNodeForIndexName(pCxt, &yymsp[-2].minor.yy561, &yymsp[0].minor.yy561); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 367: /* index_options ::= FUNCTION NK_LP func_list NK_RP INTERVAL NK_LP duration_literal NK_RP sliding_opt sma_stream_opt */ -{ yymsp[-9].minor.yy974 = createIndexOption(pCxt, yymsp[-7].minor.yy946, releaseRawExprNode(pCxt, yymsp[-3].minor.yy974), NULL, yymsp[-1].minor.yy974, yymsp[0].minor.yy974); } +{ yymsp[-9].minor.yy980 = createIndexOption(pCxt, yymsp[-7].minor.yy628, releaseRawExprNode(pCxt, yymsp[-3].minor.yy980), NULL, yymsp[-1].minor.yy980, yymsp[0].minor.yy980); } break; case 368: /* 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.yy974 = createIndexOption(pCxt, yymsp[-9].minor.yy946, releaseRawExprNode(pCxt, yymsp[-5].minor.yy974), releaseRawExprNode(pCxt, yymsp[-3].minor.yy974), yymsp[-1].minor.yy974, yymsp[0].minor.yy974); } +{ yymsp[-11].minor.yy980 = createIndexOption(pCxt, yymsp[-9].minor.yy628, releaseRawExprNode(pCxt, yymsp[-5].minor.yy980), releaseRawExprNode(pCxt, yymsp[-3].minor.yy980), yymsp[-1].minor.yy980, yymsp[0].minor.yy980); } break; case 371: /* func ::= sma_func_name NK_LP expression_list NK_RP */ -{ yylhsminor.yy974 = createFunctionNode(pCxt, &yymsp[-3].minor.yy557, yymsp[-1].minor.yy946); } - yymsp[-3].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createFunctionNode(pCxt, &yymsp[-3].minor.yy561, yymsp[-1].minor.yy628); } + yymsp[-3].minor.yy980 = yylhsminor.yy980; break; case 372: /* sma_func_name ::= function_name */ - case 667: /* alias_opt ::= table_alias */ yytestcase(yyruleno==667); -{ yylhsminor.yy557 = yymsp[0].minor.yy557; } - yymsp[0].minor.yy557 = yylhsminor.yy557; + case 668: /* alias_opt ::= table_alias */ yytestcase(yyruleno==668); +{ yylhsminor.yy561 = yymsp[0].minor.yy561; } + yymsp[0].minor.yy561 = yylhsminor.yy561; break; case 377: /* sma_stream_opt ::= */ case 427: /* stream_options ::= */ yytestcase(yyruleno==427); -{ yymsp[1].minor.yy974 = createStreamOptions(pCxt); } +{ yymsp[1].minor.yy980 = createStreamOptions(pCxt); } break; case 378: /* sma_stream_opt ::= sma_stream_opt WATERMARK duration_literal */ -{ ((SStreamOptions*)yymsp[-2].minor.yy974)->pWatermark = releaseRawExprNode(pCxt, yymsp[0].minor.yy974); yylhsminor.yy974 = yymsp[-2].minor.yy974; } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ ((SStreamOptions*)yymsp[-2].minor.yy980)->pWatermark = releaseRawExprNode(pCxt, yymsp[0].minor.yy980); yylhsminor.yy980 = yymsp[-2].minor.yy980; } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 379: /* sma_stream_opt ::= sma_stream_opt MAX_DELAY duration_literal */ -{ ((SStreamOptions*)yymsp[-2].minor.yy974)->pDelay = releaseRawExprNode(pCxt, yymsp[0].minor.yy974); yylhsminor.yy974 = yymsp[-2].minor.yy974; } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ ((SStreamOptions*)yymsp[-2].minor.yy980)->pDelay = releaseRawExprNode(pCxt, yymsp[0].minor.yy980); yylhsminor.yy980 = yymsp[-2].minor.yy980; } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 380: /* sma_stream_opt ::= sma_stream_opt DELETE_MARK duration_literal */ -{ ((SStreamOptions*)yymsp[-2].minor.yy974)->pDeleteMark = releaseRawExprNode(pCxt, yymsp[0].minor.yy974); yylhsminor.yy974 = yymsp[-2].minor.yy974; } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ ((SStreamOptions*)yymsp[-2].minor.yy980)->pDeleteMark = releaseRawExprNode(pCxt, yymsp[0].minor.yy980); yylhsminor.yy980 = yymsp[-2].minor.yy980; } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 381: /* with_meta ::= AS */ -{ yymsp[0].minor.yy904 = 0; } +{ yymsp[0].minor.yy844 = 0; } break; case 382: /* with_meta ::= WITH META AS */ -{ yymsp[-2].minor.yy904 = 1; } +{ yymsp[-2].minor.yy844 = 1; } break; case 383: /* with_meta ::= ONLY META AS */ -{ yymsp[-2].minor.yy904 = 2; } +{ yymsp[-2].minor.yy844 = 2; } break; case 384: /* cmd ::= CREATE TOPIC not_exists_opt topic_name AS query_or_subquery */ -{ pCxt->pRootNode = createCreateTopicStmtUseQuery(pCxt, yymsp[-3].minor.yy569, &yymsp[-2].minor.yy557, yymsp[0].minor.yy974); } +{ pCxt->pRootNode = createCreateTopicStmtUseQuery(pCxt, yymsp[-3].minor.yy957, &yymsp[-2].minor.yy561, yymsp[0].minor.yy980); } break; case 385: /* cmd ::= CREATE TOPIC not_exists_opt topic_name with_meta DATABASE db_name */ -{ pCxt->pRootNode = createCreateTopicStmtUseDb(pCxt, yymsp[-4].minor.yy569, &yymsp[-3].minor.yy557, &yymsp[0].minor.yy557, yymsp[-2].minor.yy904); } +{ pCxt->pRootNode = createCreateTopicStmtUseDb(pCxt, yymsp[-4].minor.yy957, &yymsp[-3].minor.yy561, &yymsp[0].minor.yy561, yymsp[-2].minor.yy844); } break; case 386: /* cmd ::= CREATE TOPIC not_exists_opt topic_name with_meta STABLE full_table_name where_clause_opt */ -{ pCxt->pRootNode = createCreateTopicStmtUseTable(pCxt, yymsp[-5].minor.yy569, &yymsp[-4].minor.yy557, yymsp[-1].minor.yy974, yymsp[-3].minor.yy904, yymsp[0].minor.yy974); } +{ pCxt->pRootNode = createCreateTopicStmtUseTable(pCxt, yymsp[-5].minor.yy957, &yymsp[-4].minor.yy561, yymsp[-1].minor.yy980, yymsp[-3].minor.yy844, yymsp[0].minor.yy980); } break; case 387: /* cmd ::= DROP TOPIC exists_opt topic_name */ -{ pCxt->pRootNode = createDropTopicStmt(pCxt, yymsp[-1].minor.yy569, &yymsp[0].minor.yy557); } +{ pCxt->pRootNode = createDropTopicStmt(pCxt, yymsp[-1].minor.yy957, &yymsp[0].minor.yy561); } break; case 388: /* cmd ::= DROP CONSUMER GROUP exists_opt cgroup_name ON topic_name */ -{ pCxt->pRootNode = createDropCGroupStmt(pCxt, yymsp[-3].minor.yy569, &yymsp[-2].minor.yy557, &yymsp[0].minor.yy557); } +{ pCxt->pRootNode = createDropCGroupStmt(pCxt, yymsp[-3].minor.yy957, &yymsp[-2].minor.yy561, &yymsp[0].minor.yy561); } break; case 389: /* cmd ::= DESC full_table_name */ case 390: /* cmd ::= DESCRIBE full_table_name */ yytestcase(yyruleno==390); -{ pCxt->pRootNode = createDescribeStmt(pCxt, yymsp[0].minor.yy974); } +{ pCxt->pRootNode = createDescribeStmt(pCxt, yymsp[0].minor.yy980); } break; case 391: /* cmd ::= RESET QUERY CACHE */ { pCxt->pRootNode = createResetQueryCacheStmt(pCxt); } break; case 392: /* cmd ::= EXPLAIN analyze_opt explain_options query_or_subquery */ case 393: /* cmd ::= EXPLAIN analyze_opt explain_options insert_query */ yytestcase(yyruleno==393); -{ pCxt->pRootNode = createExplainStmt(pCxt, yymsp[-2].minor.yy569, yymsp[-1].minor.yy974, yymsp[0].minor.yy974); } +{ pCxt->pRootNode = createExplainStmt(pCxt, yymsp[-2].minor.yy957, yymsp[-1].minor.yy980, yymsp[0].minor.yy980); } break; case 396: /* explain_options ::= */ -{ yymsp[1].minor.yy974 = createDefaultExplainOptions(pCxt); } +{ yymsp[1].minor.yy980 = createDefaultExplainOptions(pCxt); } break; case 397: /* explain_options ::= explain_options VERBOSE NK_BOOL */ -{ yylhsminor.yy974 = setExplainVerbose(pCxt, yymsp[-2].minor.yy974, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setExplainVerbose(pCxt, yymsp[-2].minor.yy980, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 398: /* explain_options ::= explain_options RATIO NK_FLOAT */ -{ yylhsminor.yy974 = setExplainRatio(pCxt, yymsp[-2].minor.yy974, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = setExplainRatio(pCxt, yymsp[-2].minor.yy980, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 399: /* 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.yy569, yymsp[-9].minor.yy569, &yymsp[-6].minor.yy557, &yymsp[-4].minor.yy0, yymsp[-2].minor.yy424, yymsp[-1].minor.yy904, &yymsp[0].minor.yy557, yymsp[-10].minor.yy569); } +{ pCxt->pRootNode = createCreateFunctionStmt(pCxt, yymsp[-7].minor.yy957, yymsp[-9].minor.yy957, &yymsp[-6].minor.yy561, &yymsp[-4].minor.yy0, yymsp[-2].minor.yy896, yymsp[-1].minor.yy844, &yymsp[0].minor.yy561, yymsp[-10].minor.yy957); } break; case 400: /* cmd ::= DROP FUNCTION exists_opt function_name */ -{ pCxt->pRootNode = createDropFunctionStmt(pCxt, yymsp[-1].minor.yy569, &yymsp[0].minor.yy557); } +{ pCxt->pRootNode = createDropFunctionStmt(pCxt, yymsp[-1].minor.yy957, &yymsp[0].minor.yy561); } break; case 405: /* language_opt ::= */ - case 450: /* on_vgroup_id ::= */ yytestcase(yyruleno==450); -{ yymsp[1].minor.yy557 = nil_token; } + case 451: /* on_vgroup_id ::= */ yytestcase(yyruleno==451); +{ yymsp[1].minor.yy561 = nil_token; } break; case 406: /* language_opt ::= LANGUAGE NK_STRING */ - case 451: /* on_vgroup_id ::= ON NK_INTEGER */ yytestcase(yyruleno==451); -{ yymsp[-1].minor.yy557 = yymsp[0].minor.yy0; } + case 452: /* on_vgroup_id ::= ON NK_INTEGER */ yytestcase(yyruleno==452); +{ yymsp[-1].minor.yy561 = yymsp[0].minor.yy0; } break; case 409: /* cmd ::= CREATE or_replace_opt VIEW full_view_name AS query_or_subquery */ -{ pCxt->pRootNode = createCreateViewStmt(pCxt, yymsp[-4].minor.yy569, yymsp[-2].minor.yy974, &yymsp[-1].minor.yy0, yymsp[0].minor.yy974); } +{ pCxt->pRootNode = createCreateViewStmt(pCxt, yymsp[-4].minor.yy957, yymsp[-2].minor.yy980, &yymsp[-1].minor.yy0, yymsp[0].minor.yy980); } break; case 410: /* cmd ::= DROP VIEW exists_opt full_view_name */ -{ pCxt->pRootNode = createDropViewStmt(pCxt, yymsp[-1].minor.yy569, yymsp[0].minor.yy974); } +{ pCxt->pRootNode = createDropViewStmt(pCxt, yymsp[-1].minor.yy957, yymsp[0].minor.yy980); } break; case 411: /* full_view_name ::= view_name */ -{ yylhsminor.yy974 = createViewNode(pCxt, NULL, &yymsp[0].minor.yy557); } - yymsp[0].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createViewNode(pCxt, NULL, &yymsp[0].minor.yy561); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; case 412: /* full_view_name ::= db_name NK_DOT view_name */ -{ yylhsminor.yy974 = createViewNode(pCxt, &yymsp[-2].minor.yy557, &yymsp[0].minor.yy557); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createViewNode(pCxt, &yymsp[-2].minor.yy561, &yymsp[0].minor.yy561); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 413: /* 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.yy569, &yymsp[-8].minor.yy557, yymsp[-5].minor.yy974, yymsp[-7].minor.yy974, yymsp[-3].minor.yy946, yymsp[-2].minor.yy974, yymsp[0].minor.yy974, yymsp[-4].minor.yy946); } +{ pCxt->pRootNode = createCreateStreamStmt(pCxt, yymsp[-9].minor.yy957, &yymsp[-8].minor.yy561, yymsp[-5].minor.yy980, yymsp[-7].minor.yy980, yymsp[-3].minor.yy628, yymsp[-2].minor.yy980, yymsp[0].minor.yy980, yymsp[-4].minor.yy628); } break; case 414: /* cmd ::= DROP STREAM exists_opt stream_name */ -{ pCxt->pRootNode = createDropStreamStmt(pCxt, yymsp[-1].minor.yy569, &yymsp[0].minor.yy557); } +{ pCxt->pRootNode = createDropStreamStmt(pCxt, yymsp[-1].minor.yy957, &yymsp[0].minor.yy561); } break; case 415: /* cmd ::= PAUSE STREAM exists_opt stream_name */ -{ pCxt->pRootNode = createPauseStreamStmt(pCxt, yymsp[-1].minor.yy569, &yymsp[0].minor.yy557); } +{ pCxt->pRootNode = createPauseStreamStmt(pCxt, yymsp[-1].minor.yy957, &yymsp[0].minor.yy561); } break; case 416: /* cmd ::= RESUME STREAM exists_opt ignore_opt stream_name */ -{ pCxt->pRootNode = createResumeStreamStmt(pCxt, yymsp[-2].minor.yy569, yymsp[-1].minor.yy569, &yymsp[0].minor.yy557); } +{ pCxt->pRootNode = createResumeStreamStmt(pCxt, yymsp[-2].minor.yy957, yymsp[-1].minor.yy957, &yymsp[0].minor.yy561); } break; case 421: /* column_stream_def ::= column_name stream_col_options */ -{ yylhsminor.yy974 = createColumnDefNode(pCxt, &yymsp[-1].minor.yy557, createDataType(TSDB_DATA_TYPE_NULL), yymsp[0].minor.yy974); } - yymsp[-1].minor.yy974 = yylhsminor.yy974; +{ yylhsminor.yy980 = createColumnDefNode(pCxt, &yymsp[-1].minor.yy561, createDataType(TSDB_DATA_TYPE_NULL), yymsp[0].minor.yy980); } + yymsp[-1].minor.yy980 = yylhsminor.yy980; break; case 422: /* stream_col_options ::= */ - case 782: /* column_options ::= */ yytestcase(yyruleno==782); -{ yymsp[1].minor.yy974 = createDefaultColumnOptions(pCxt); } + case 783: /* column_options ::= */ yytestcase(yyruleno==783); +{ yymsp[1].minor.yy980 = createDefaultColumnOptions(pCxt); } break; case 423: /* stream_col_options ::= stream_col_options PRIMARY KEY */ - case 783: /* column_options ::= column_options PRIMARY KEY */ yytestcase(yyruleno==783); -{ yylhsminor.yy974 = setColumnOptionsPK(pCxt, yymsp[-2].minor.yy974); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + case 784: /* column_options ::= column_options PRIMARY KEY */ yytestcase(yyruleno==784); +{ yylhsminor.yy980 = setColumnOptionsPK(pCxt, yymsp[-2].minor.yy980); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; case 428: /* stream_options ::= stream_options TRIGGER AT_ONCE */ case 429: /* stream_options ::= stream_options TRIGGER WINDOW_CLOSE */ yytestcase(yyruleno==429); -{ yylhsminor.yy974 = setStreamOptions(pCxt, yymsp[-2].minor.yy974, SOPT_TRIGGER_TYPE_SET, &yymsp[0].minor.yy0, NULL); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + case 430: /* stream_options ::= stream_options TRIGGER FORCE_WINDOW_CLOSE */ yytestcase(yyruleno==430); +{ yylhsminor.yy980 = setStreamOptions(pCxt, yymsp[-2].minor.yy980, SOPT_TRIGGER_TYPE_SET, &yymsp[0].minor.yy0, NULL); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 430: /* stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal */ -{ yylhsminor.yy974 = setStreamOptions(pCxt, yymsp[-3].minor.yy974, SOPT_TRIGGER_TYPE_SET, &yymsp[-1].minor.yy0, releaseRawExprNode(pCxt, yymsp[0].minor.yy974)); } - yymsp[-3].minor.yy974 = yylhsminor.yy974; + case 431: /* stream_options ::= stream_options TRIGGER MAX_DELAY duration_literal */ +{ yylhsminor.yy980 = setStreamOptions(pCxt, yymsp[-3].minor.yy980, SOPT_TRIGGER_TYPE_SET, &yymsp[-1].minor.yy0, releaseRawExprNode(pCxt, yymsp[0].minor.yy980)); } + yymsp[-3].minor.yy980 = yylhsminor.yy980; break; - case 431: /* stream_options ::= stream_options WATERMARK duration_literal */ -{ yylhsminor.yy974 = setStreamOptions(pCxt, yymsp[-2].minor.yy974, SOPT_WATERMARK_SET, NULL, releaseRawExprNode(pCxt, yymsp[0].minor.yy974)); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + case 432: /* stream_options ::= stream_options WATERMARK duration_literal */ +{ yylhsminor.yy980 = setStreamOptions(pCxt, yymsp[-2].minor.yy980, SOPT_WATERMARK_SET, NULL, releaseRawExprNode(pCxt, yymsp[0].minor.yy980)); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 432: /* stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER */ -{ yylhsminor.yy974 = setStreamOptions(pCxt, yymsp[-3].minor.yy974, SOPT_IGNORE_EXPIRED_SET, &yymsp[0].minor.yy0, NULL); } - yymsp[-3].minor.yy974 = yylhsminor.yy974; + case 433: /* stream_options ::= stream_options IGNORE EXPIRED NK_INTEGER */ +{ yylhsminor.yy980 = setStreamOptions(pCxt, yymsp[-3].minor.yy980, SOPT_IGNORE_EXPIRED_SET, &yymsp[0].minor.yy0, NULL); } + yymsp[-3].minor.yy980 = yylhsminor.yy980; break; - case 433: /* stream_options ::= stream_options FILL_HISTORY NK_INTEGER */ -{ yylhsminor.yy974 = setStreamOptions(pCxt, yymsp[-2].minor.yy974, SOPT_FILL_HISTORY_SET, &yymsp[0].minor.yy0, NULL); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + case 434: /* stream_options ::= stream_options FILL_HISTORY NK_INTEGER */ +{ yylhsminor.yy980 = setStreamOptions(pCxt, yymsp[-2].minor.yy980, SOPT_FILL_HISTORY_SET, &yymsp[0].minor.yy0, NULL); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 434: /* stream_options ::= stream_options DELETE_MARK duration_literal */ -{ yylhsminor.yy974 = setStreamOptions(pCxt, yymsp[-2].minor.yy974, SOPT_DELETE_MARK_SET, NULL, releaseRawExprNode(pCxt, yymsp[0].minor.yy974)); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + case 435: /* stream_options ::= stream_options DELETE_MARK duration_literal */ +{ yylhsminor.yy980 = setStreamOptions(pCxt, yymsp[-2].minor.yy980, SOPT_DELETE_MARK_SET, NULL, releaseRawExprNode(pCxt, yymsp[0].minor.yy980)); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 435: /* stream_options ::= stream_options IGNORE UPDATE NK_INTEGER */ -{ yylhsminor.yy974 = setStreamOptions(pCxt, yymsp[-3].minor.yy974, SOPT_IGNORE_UPDATE_SET, &yymsp[0].minor.yy0, NULL); } - yymsp[-3].minor.yy974 = yylhsminor.yy974; + case 436: /* stream_options ::= stream_options IGNORE UPDATE NK_INTEGER */ +{ yylhsminor.yy980 = setStreamOptions(pCxt, yymsp[-3].minor.yy980, SOPT_IGNORE_UPDATE_SET, &yymsp[0].minor.yy0, NULL); } + yymsp[-3].minor.yy980 = yylhsminor.yy980; break; - case 437: /* subtable_opt ::= SUBTABLE NK_LP expression NK_RP */ - case 726: /* sliding_opt ::= SLIDING NK_LP interval_sliding_duration_literal NK_RP */ yytestcase(yyruleno==726); - case 750: /* every_opt ::= EVERY NK_LP duration_literal NK_RP */ yytestcase(yyruleno==750); -{ yymsp[-3].minor.yy974 = releaseRawExprNode(pCxt, yymsp[-1].minor.yy974); } + case 438: /* subtable_opt ::= SUBTABLE NK_LP expression NK_RP */ + case 727: /* sliding_opt ::= SLIDING NK_LP interval_sliding_duration_literal NK_RP */ yytestcase(yyruleno==727); + case 751: /* every_opt ::= EVERY NK_LP duration_literal NK_RP */ yytestcase(yyruleno==751); +{ yymsp[-3].minor.yy980 = releaseRawExprNode(pCxt, yymsp[-1].minor.yy980); } break; - case 440: /* cmd ::= KILL CONNECTION NK_INTEGER */ + case 441: /* cmd ::= KILL CONNECTION NK_INTEGER */ { pCxt->pRootNode = createKillStmt(pCxt, QUERY_NODE_KILL_CONNECTION_STMT, &yymsp[0].minor.yy0); } break; - case 441: /* cmd ::= KILL QUERY NK_STRING */ + case 442: /* cmd ::= KILL QUERY NK_STRING */ { pCxt->pRootNode = createKillQueryStmt(pCxt, &yymsp[0].minor.yy0); } break; - case 442: /* cmd ::= KILL TRANSACTION NK_INTEGER */ + case 443: /* cmd ::= KILL TRANSACTION NK_INTEGER */ { pCxt->pRootNode = createKillStmt(pCxt, QUERY_NODE_KILL_TRANSACTION_STMT, &yymsp[0].minor.yy0); } break; - case 443: /* cmd ::= KILL COMPACT NK_INTEGER */ + case 444: /* cmd ::= KILL COMPACT NK_INTEGER */ { pCxt->pRootNode = createKillStmt(pCxt, QUERY_NODE_KILL_COMPACT_STMT, &yymsp[0].minor.yy0); } break; - case 444: /* cmd ::= BALANCE VGROUP */ + case 445: /* cmd ::= BALANCE VGROUP */ { pCxt->pRootNode = createBalanceVgroupStmt(pCxt); } break; - case 445: /* cmd ::= BALANCE VGROUP LEADER on_vgroup_id */ -{ pCxt->pRootNode = createBalanceVgroupLeaderStmt(pCxt, &yymsp[0].minor.yy557); } + case 446: /* cmd ::= BALANCE VGROUP LEADER on_vgroup_id */ +{ pCxt->pRootNode = createBalanceVgroupLeaderStmt(pCxt, &yymsp[0].minor.yy561); } break; - case 446: /* cmd ::= BALANCE VGROUP LEADER DATABASE db_name */ -{ pCxt->pRootNode = createBalanceVgroupLeaderDBNameStmt(pCxt, &yymsp[0].minor.yy557); } + case 447: /* cmd ::= BALANCE VGROUP LEADER DATABASE db_name */ +{ pCxt->pRootNode = createBalanceVgroupLeaderDBNameStmt(pCxt, &yymsp[0].minor.yy561); } break; - case 447: /* cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER */ + case 448: /* cmd ::= MERGE VGROUP NK_INTEGER NK_INTEGER */ { pCxt->pRootNode = createMergeVgroupStmt(pCxt, &yymsp[-1].minor.yy0, &yymsp[0].minor.yy0); } break; - case 448: /* cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list */ -{ pCxt->pRootNode = createRedistributeVgroupStmt(pCxt, &yymsp[-1].minor.yy0, yymsp[0].minor.yy946); } + case 449: /* cmd ::= REDISTRIBUTE VGROUP NK_INTEGER dnode_list */ +{ pCxt->pRootNode = createRedistributeVgroupStmt(pCxt, &yymsp[-1].minor.yy0, yymsp[0].minor.yy628); } break; - case 449: /* cmd ::= SPLIT VGROUP NK_INTEGER */ + case 450: /* cmd ::= SPLIT VGROUP NK_INTEGER */ { pCxt->pRootNode = createSplitVgroupStmt(pCxt, &yymsp[0].minor.yy0); } break; - case 452: /* dnode_list ::= DNODE NK_INTEGER */ -{ yymsp[-1].minor.yy946 = createNodeList(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0)); } + case 453: /* dnode_list ::= DNODE NK_INTEGER */ +{ yymsp[-1].minor.yy628 = createNodeList(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &yymsp[0].minor.yy0)); } break; - case 454: /* cmd ::= DELETE FROM full_table_name where_clause_opt */ -{ pCxt->pRootNode = createDeleteStmt(pCxt, yymsp[-1].minor.yy974, yymsp[0].minor.yy974); } + case 455: /* cmd ::= DELETE FROM full_table_name where_clause_opt */ +{ pCxt->pRootNode = createDeleteStmt(pCxt, yymsp[-1].minor.yy980, yymsp[0].minor.yy980); } break; - case 457: /* insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery */ -{ yymsp[-6].minor.yy974 = createInsertStmt(pCxt, yymsp[-4].minor.yy974, yymsp[-2].minor.yy946, yymsp[0].minor.yy974); } + case 458: /* insert_query ::= INSERT INTO full_table_name NK_LP col_name_list NK_RP query_or_subquery */ +{ yymsp[-6].minor.yy980 = createInsertStmt(pCxt, yymsp[-4].minor.yy980, yymsp[-2].minor.yy628, yymsp[0].minor.yy980); } break; - case 458: /* insert_query ::= INSERT INTO full_table_name query_or_subquery */ -{ yymsp[-3].minor.yy974 = createInsertStmt(pCxt, yymsp[-1].minor.yy974, NULL, yymsp[0].minor.yy974); } + case 459: /* insert_query ::= INSERT INTO full_table_name query_or_subquery */ +{ yymsp[-3].minor.yy980 = createInsertStmt(pCxt, yymsp[-1].minor.yy980, NULL, yymsp[0].minor.yy980); } break; - case 459: /* tags_literal ::= NK_INTEGER */ - case 471: /* tags_literal ::= NK_BIN */ yytestcase(yyruleno==471); - case 480: /* tags_literal ::= NK_HEX */ yytestcase(yyruleno==480); -{ yylhsminor.yy974 = createRawValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &yymsp[0].minor.yy0, NULL); } - yymsp[0].minor.yy974 = yylhsminor.yy974; + case 460: /* tags_literal ::= NK_INTEGER */ + case 472: /* tags_literal ::= NK_BIN */ yytestcase(yyruleno==472); + case 481: /* tags_literal ::= NK_HEX */ yytestcase(yyruleno==481); +{ yylhsminor.yy980 = createRawValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &yymsp[0].minor.yy0, NULL); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; - case 460: /* tags_literal ::= NK_INTEGER NK_PLUS duration_literal */ - case 461: /* tags_literal ::= NK_INTEGER NK_MINUS duration_literal */ yytestcase(yyruleno==461); - case 472: /* tags_literal ::= NK_BIN NK_PLUS duration_literal */ yytestcase(yyruleno==472); - case 473: /* tags_literal ::= NK_BIN NK_MINUS duration_literal */ yytestcase(yyruleno==473); - case 481: /* tags_literal ::= NK_HEX NK_PLUS duration_literal */ yytestcase(yyruleno==481); - case 482: /* tags_literal ::= NK_HEX NK_MINUS duration_literal */ yytestcase(yyruleno==482); - case 490: /* tags_literal ::= NK_STRING NK_PLUS duration_literal */ yytestcase(yyruleno==490); - case 491: /* tags_literal ::= NK_STRING NK_MINUS duration_literal */ yytestcase(yyruleno==491); + case 461: /* tags_literal ::= NK_INTEGER NK_PLUS duration_literal */ + case 462: /* tags_literal ::= NK_INTEGER NK_MINUS duration_literal */ yytestcase(yyruleno==462); + case 473: /* tags_literal ::= NK_BIN NK_PLUS duration_literal */ yytestcase(yyruleno==473); + case 474: /* tags_literal ::= NK_BIN NK_MINUS duration_literal */ yytestcase(yyruleno==474); + case 482: /* tags_literal ::= NK_HEX NK_PLUS duration_literal */ yytestcase(yyruleno==482); + case 483: /* tags_literal ::= NK_HEX NK_MINUS duration_literal */ yytestcase(yyruleno==483); + case 491: /* tags_literal ::= NK_STRING NK_PLUS duration_literal */ yytestcase(yyruleno==491); + case 492: /* tags_literal ::= NK_STRING NK_MINUS duration_literal */ yytestcase(yyruleno==492); { SToken l = yymsp[-2].minor.yy0; - SToken r = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy974); + SToken r = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy980); l.n = (r.z + r.n) - l.z; - yylhsminor.yy974 = createRawValueNodeExt(pCxt, TSDB_DATA_TYPE_BINARY, &l, NULL, yymsp[0].minor.yy974); + yylhsminor.yy980 = createRawValueNodeExt(pCxt, TSDB_DATA_TYPE_BINARY, &l, NULL, yymsp[0].minor.yy980); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 462: /* tags_literal ::= NK_PLUS NK_INTEGER */ - case 465: /* tags_literal ::= NK_MINUS NK_INTEGER */ yytestcase(yyruleno==465); - case 474: /* tags_literal ::= NK_PLUS NK_BIN */ yytestcase(yyruleno==474); - case 477: /* tags_literal ::= NK_MINUS NK_BIN */ yytestcase(yyruleno==477); - case 483: /* tags_literal ::= NK_PLUS NK_HEX */ yytestcase(yyruleno==483); - case 486: /* tags_literal ::= NK_MINUS NK_HEX */ yytestcase(yyruleno==486); + case 463: /* tags_literal ::= NK_PLUS NK_INTEGER */ + case 466: /* tags_literal ::= NK_MINUS NK_INTEGER */ yytestcase(yyruleno==466); + case 475: /* tags_literal ::= NK_PLUS NK_BIN */ yytestcase(yyruleno==475); + case 478: /* tags_literal ::= NK_MINUS NK_BIN */ yytestcase(yyruleno==478); + case 484: /* tags_literal ::= NK_PLUS NK_HEX */ yytestcase(yyruleno==484); + case 487: /* tags_literal ::= NK_MINUS NK_HEX */ yytestcase(yyruleno==487); { SToken t = yymsp[-1].minor.yy0; t.n = (yymsp[0].minor.yy0.z + yymsp[0].minor.yy0.n) - yymsp[-1].minor.yy0.z; - yylhsminor.yy974 = createRawValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &t, NULL); + yylhsminor.yy980 = createRawValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &t, NULL); } - yymsp[-1].minor.yy974 = yylhsminor.yy974; + yymsp[-1].minor.yy980 = yylhsminor.yy980; break; - case 463: /* tags_literal ::= NK_PLUS NK_INTEGER NK_PLUS duration_literal */ - case 464: /* tags_literal ::= NK_PLUS NK_INTEGER NK_MINUS duration_literal */ yytestcase(yyruleno==464); - case 466: /* tags_literal ::= NK_MINUS NK_INTEGER NK_PLUS duration_literal */ yytestcase(yyruleno==466); - case 467: /* tags_literal ::= NK_MINUS NK_INTEGER NK_MINUS duration_literal */ yytestcase(yyruleno==467); - case 475: /* tags_literal ::= NK_PLUS NK_BIN NK_PLUS duration_literal */ yytestcase(yyruleno==475); - case 476: /* tags_literal ::= NK_PLUS NK_BIN NK_MINUS duration_literal */ yytestcase(yyruleno==476); - case 478: /* tags_literal ::= NK_MINUS NK_BIN NK_PLUS duration_literal */ yytestcase(yyruleno==478); - case 479: /* tags_literal ::= NK_MINUS NK_BIN NK_MINUS duration_literal */ yytestcase(yyruleno==479); - case 484: /* tags_literal ::= NK_PLUS NK_HEX NK_PLUS duration_literal */ yytestcase(yyruleno==484); - case 485: /* tags_literal ::= NK_PLUS NK_HEX NK_MINUS duration_literal */ yytestcase(yyruleno==485); - case 487: /* tags_literal ::= NK_MINUS NK_HEX NK_PLUS duration_literal */ yytestcase(yyruleno==487); - case 488: /* tags_literal ::= NK_MINUS NK_HEX NK_MINUS duration_literal */ yytestcase(yyruleno==488); + case 464: /* tags_literal ::= NK_PLUS NK_INTEGER NK_PLUS duration_literal */ + case 465: /* tags_literal ::= NK_PLUS NK_INTEGER NK_MINUS duration_literal */ yytestcase(yyruleno==465); + case 467: /* tags_literal ::= NK_MINUS NK_INTEGER NK_PLUS duration_literal */ yytestcase(yyruleno==467); + case 468: /* tags_literal ::= NK_MINUS NK_INTEGER NK_MINUS duration_literal */ yytestcase(yyruleno==468); + case 476: /* tags_literal ::= NK_PLUS NK_BIN NK_PLUS duration_literal */ yytestcase(yyruleno==476); + case 477: /* tags_literal ::= NK_PLUS NK_BIN NK_MINUS duration_literal */ yytestcase(yyruleno==477); + case 479: /* tags_literal ::= NK_MINUS NK_BIN NK_PLUS duration_literal */ yytestcase(yyruleno==479); + case 480: /* tags_literal ::= NK_MINUS NK_BIN NK_MINUS duration_literal */ yytestcase(yyruleno==480); + case 485: /* tags_literal ::= NK_PLUS NK_HEX NK_PLUS duration_literal */ yytestcase(yyruleno==485); + case 486: /* tags_literal ::= NK_PLUS NK_HEX NK_MINUS duration_literal */ yytestcase(yyruleno==486); + case 488: /* tags_literal ::= NK_MINUS NK_HEX NK_PLUS duration_literal */ yytestcase(yyruleno==488); + case 489: /* tags_literal ::= NK_MINUS NK_HEX NK_MINUS duration_literal */ yytestcase(yyruleno==489); { SToken l = yymsp[-3].minor.yy0; - SToken r = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy974); + SToken r = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy980); l.n = (r.z + r.n) - l.z; - yylhsminor.yy974 = createRawValueNodeExt(pCxt, TSDB_DATA_TYPE_BINARY, &l, NULL, yymsp[0].minor.yy974); + yylhsminor.yy980 = createRawValueNodeExt(pCxt, TSDB_DATA_TYPE_BINARY, &l, NULL, yymsp[0].minor.yy980); } - yymsp[-3].minor.yy974 = yylhsminor.yy974; + yymsp[-3].minor.yy980 = yylhsminor.yy980; break; - case 468: /* tags_literal ::= NK_FLOAT */ -{ yylhsminor.yy974 = createRawValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &yymsp[0].minor.yy0, NULL); } - yymsp[0].minor.yy974 = yylhsminor.yy974; + case 469: /* tags_literal ::= NK_FLOAT */ +{ yylhsminor.yy980 = createRawValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &yymsp[0].minor.yy0, NULL); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; - case 469: /* tags_literal ::= NK_PLUS NK_FLOAT */ - case 470: /* tags_literal ::= NK_MINUS NK_FLOAT */ yytestcase(yyruleno==470); + case 470: /* tags_literal ::= NK_PLUS NK_FLOAT */ + case 471: /* tags_literal ::= NK_MINUS NK_FLOAT */ yytestcase(yyruleno==471); { SToken t = yymsp[-1].minor.yy0; t.n = (yymsp[0].minor.yy0.z + yymsp[0].minor.yy0.n) - yymsp[-1].minor.yy0.z; - yylhsminor.yy974 = createRawValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &t, NULL); + yylhsminor.yy980 = createRawValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &t, NULL); } - yymsp[-1].minor.yy974 = yylhsminor.yy974; + yymsp[-1].minor.yy980 = yylhsminor.yy980; break; - case 489: /* tags_literal ::= NK_STRING */ -{ yylhsminor.yy974 = createRawValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0, NULL); } - yymsp[0].minor.yy974 = yylhsminor.yy974; + case 490: /* tags_literal ::= NK_STRING */ +{ yylhsminor.yy980 = createRawValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0, NULL); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; - case 492: /* tags_literal ::= NK_BOOL */ -{ yylhsminor.yy974 = createRawValueNode(pCxt, TSDB_DATA_TYPE_BOOL, &yymsp[0].minor.yy0, NULL); } - yymsp[0].minor.yy974 = yylhsminor.yy974; + case 493: /* tags_literal ::= NK_BOOL */ +{ yylhsminor.yy980 = createRawValueNode(pCxt, TSDB_DATA_TYPE_BOOL, &yymsp[0].minor.yy0, NULL); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; - case 493: /* tags_literal ::= NULL */ -{ yylhsminor.yy974 = createRawValueNode(pCxt, TSDB_DATA_TYPE_NULL, &yymsp[0].minor.yy0, NULL); } - yymsp[0].minor.yy974 = yylhsminor.yy974; + case 494: /* tags_literal ::= NULL */ +{ yylhsminor.yy980 = createRawValueNode(pCxt, TSDB_DATA_TYPE_NULL, &yymsp[0].minor.yy0, NULL); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; - case 494: /* tags_literal ::= literal_func */ -{ yylhsminor.yy974 = createRawValueNode(pCxt, TSDB_DATA_TYPE_BINARY, NULL, yymsp[0].minor.yy974); } - yymsp[0].minor.yy974 = yylhsminor.yy974; + case 495: /* tags_literal ::= literal_func */ +{ yylhsminor.yy980 = createRawValueNode(pCxt, TSDB_DATA_TYPE_BINARY, NULL, yymsp[0].minor.yy980); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; - case 495: /* tags_literal ::= literal_func NK_PLUS duration_literal */ - case 496: /* tags_literal ::= literal_func NK_MINUS duration_literal */ yytestcase(yyruleno==496); + case 496: /* tags_literal ::= literal_func NK_PLUS duration_literal */ + case 497: /* tags_literal ::= literal_func NK_MINUS duration_literal */ yytestcase(yyruleno==497); { - SToken l = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy974); - SToken r = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy974); + SToken l = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy980); + SToken r = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy980); l.n = (r.z + r.n) - l.z; - yylhsminor.yy974 = createRawValueNodeExt(pCxt, TSDB_DATA_TYPE_BINARY, &l, yymsp[-2].minor.yy974, yymsp[0].minor.yy974); + yylhsminor.yy980 = createRawValueNodeExt(pCxt, TSDB_DATA_TYPE_BINARY, &l, yymsp[-2].minor.yy980, yymsp[0].minor.yy980); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 499: /* literal ::= NK_INTEGER */ -{ yylhsminor.yy974 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &yymsp[0].minor.yy0)); } - yymsp[0].minor.yy974 = yylhsminor.yy974; + case 500: /* literal ::= NK_INTEGER */ +{ yylhsminor.yy980 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; - case 500: /* literal ::= NK_FLOAT */ -{ yylhsminor.yy974 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &yymsp[0].minor.yy0)); } - yymsp[0].minor.yy974 = yylhsminor.yy974; + case 501: /* literal ::= NK_FLOAT */ +{ yylhsminor.yy980 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; - case 501: /* literal ::= NK_STRING */ -{ yylhsminor.yy974 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0)); } - yymsp[0].minor.yy974 = yylhsminor.yy974; + case 502: /* literal ::= NK_STRING */ +{ yylhsminor.yy980 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; - case 502: /* literal ::= NK_BOOL */ -{ yylhsminor.yy974 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_BOOL, &yymsp[0].minor.yy0)); } - yymsp[0].minor.yy974 = yylhsminor.yy974; + case 503: /* literal ::= NK_BOOL */ +{ yylhsminor.yy980 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_BOOL, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; - case 503: /* literal ::= TIMESTAMP NK_STRING */ -{ yylhsminor.yy974 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0)); } - yymsp[-1].minor.yy974 = yylhsminor.yy974; + case 504: /* literal ::= TIMESTAMP NK_STRING */ +{ yylhsminor.yy980 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0)); } + yymsp[-1].minor.yy980 = yylhsminor.yy980; break; - case 504: /* literal ::= duration_literal */ - case 514: /* signed_literal ::= signed */ yytestcase(yyruleno==514); - case 538: /* expr_or_subquery ::= expression */ yytestcase(yyruleno==538); - case 539: /* expression ::= literal */ yytestcase(yyruleno==539); - case 541: /* expression ::= column_reference */ yytestcase(yyruleno==541); - case 542: /* expression ::= function_expression */ yytestcase(yyruleno==542); - case 543: /* expression ::= case_when_expression */ yytestcase(yyruleno==543); - case 589: /* function_expression ::= literal_func */ yytestcase(yyruleno==589); - case 590: /* function_expression ::= rand_func */ yytestcase(yyruleno==590); - case 648: /* boolean_value_expression ::= boolean_primary */ yytestcase(yyruleno==648); - case 652: /* boolean_primary ::= predicate */ yytestcase(yyruleno==652); - case 654: /* common_expression ::= expr_or_subquery */ yytestcase(yyruleno==654); - case 655: /* common_expression ::= boolean_value_expression */ yytestcase(yyruleno==655); - case 658: /* table_reference_list ::= table_reference */ yytestcase(yyruleno==658); - case 660: /* table_reference ::= table_primary */ yytestcase(yyruleno==660); - case 661: /* table_reference ::= joined_table */ yytestcase(yyruleno==661); - case 665: /* table_primary ::= parenthesized_joined_table */ yytestcase(yyruleno==665); - case 752: /* query_simple ::= query_specification */ yytestcase(yyruleno==752); - case 753: /* query_simple ::= union_query_expression */ yytestcase(yyruleno==753); - case 756: /* query_simple_or_subquery ::= query_simple */ yytestcase(yyruleno==756); - case 758: /* query_or_subquery ::= query_expression */ yytestcase(yyruleno==758); -{ yylhsminor.yy974 = yymsp[0].minor.yy974; } - yymsp[0].minor.yy974 = yylhsminor.yy974; + case 505: /* literal ::= duration_literal */ + case 515: /* signed_literal ::= signed */ yytestcase(yyruleno==515); + case 539: /* expr_or_subquery ::= expression */ yytestcase(yyruleno==539); + case 540: /* expression ::= literal */ yytestcase(yyruleno==540); + case 542: /* expression ::= column_reference */ yytestcase(yyruleno==542); + case 543: /* expression ::= function_expression */ yytestcase(yyruleno==543); + case 544: /* expression ::= case_when_expression */ yytestcase(yyruleno==544); + case 590: /* function_expression ::= literal_func */ yytestcase(yyruleno==590); + case 591: /* function_expression ::= rand_func */ yytestcase(yyruleno==591); + case 649: /* boolean_value_expression ::= boolean_primary */ yytestcase(yyruleno==649); + case 653: /* boolean_primary ::= predicate */ yytestcase(yyruleno==653); + case 655: /* common_expression ::= expr_or_subquery */ yytestcase(yyruleno==655); + case 656: /* common_expression ::= boolean_value_expression */ yytestcase(yyruleno==656); + case 659: /* table_reference_list ::= table_reference */ yytestcase(yyruleno==659); + case 661: /* table_reference ::= table_primary */ yytestcase(yyruleno==661); + case 662: /* table_reference ::= joined_table */ yytestcase(yyruleno==662); + case 666: /* table_primary ::= parenthesized_joined_table */ yytestcase(yyruleno==666); + case 753: /* query_simple ::= query_specification */ yytestcase(yyruleno==753); + case 754: /* query_simple ::= union_query_expression */ yytestcase(yyruleno==754); + case 757: /* query_simple_or_subquery ::= query_simple */ yytestcase(yyruleno==757); + case 759: /* query_or_subquery ::= query_expression */ yytestcase(yyruleno==759); +{ yylhsminor.yy980 = yymsp[0].minor.yy980; } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; - case 505: /* literal ::= NULL */ -{ yylhsminor.yy974 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_NULL, &yymsp[0].minor.yy0)); } - yymsp[0].minor.yy974 = yylhsminor.yy974; + case 506: /* literal ::= NULL */ +{ yylhsminor.yy980 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createValueNode(pCxt, TSDB_DATA_TYPE_NULL, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; - case 506: /* literal ::= NK_QUESTION */ -{ yylhsminor.yy974 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createPlaceholderValueNode(pCxt, &yymsp[0].minor.yy0)); } - yymsp[0].minor.yy974 = yylhsminor.yy974; + case 507: /* literal ::= NK_QUESTION */ +{ yylhsminor.yy980 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createPlaceholderValueNode(pCxt, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; - case 507: /* duration_literal ::= NK_VARIABLE */ - case 727: /* interval_sliding_duration_literal ::= NK_VARIABLE */ yytestcase(yyruleno==727); - case 728: /* interval_sliding_duration_literal ::= NK_STRING */ yytestcase(yyruleno==728); - case 729: /* interval_sliding_duration_literal ::= NK_INTEGER */ yytestcase(yyruleno==729); -{ yylhsminor.yy974 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } - yymsp[0].minor.yy974 = yylhsminor.yy974; + case 508: /* duration_literal ::= NK_VARIABLE */ + case 728: /* interval_sliding_duration_literal ::= NK_VARIABLE */ yytestcase(yyruleno==728); + case 729: /* interval_sliding_duration_literal ::= NK_STRING */ yytestcase(yyruleno==729); + case 730: /* interval_sliding_duration_literal ::= NK_INTEGER */ yytestcase(yyruleno==730); +{ yylhsminor.yy980 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createDurationValueNode(pCxt, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; - case 508: /* signed ::= NK_INTEGER */ -{ yylhsminor.yy974 = createValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &yymsp[0].minor.yy0); } - yymsp[0].minor.yy974 = yylhsminor.yy974; + case 509: /* signed ::= NK_INTEGER */ +{ yylhsminor.yy980 = createValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &yymsp[0].minor.yy0); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; - case 509: /* signed ::= NK_PLUS NK_INTEGER */ -{ yymsp[-1].minor.yy974 = createValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &yymsp[0].minor.yy0); } + case 510: /* signed ::= NK_PLUS NK_INTEGER */ +{ yymsp[-1].minor.yy980 = createValueNode(pCxt, TSDB_DATA_TYPE_UBIGINT, &yymsp[0].minor.yy0); } break; - case 510: /* signed ::= NK_MINUS NK_INTEGER */ + case 511: /* 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.yy974 = createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &t); + yylhsminor.yy980 = createValueNode(pCxt, TSDB_DATA_TYPE_BIGINT, &t); } - yymsp[-1].minor.yy974 = yylhsminor.yy974; + yymsp[-1].minor.yy980 = yylhsminor.yy980; break; - case 511: /* signed ::= NK_FLOAT */ -{ yylhsminor.yy974 = createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &yymsp[0].minor.yy0); } - yymsp[0].minor.yy974 = yylhsminor.yy974; + case 512: /* signed ::= NK_FLOAT */ +{ yylhsminor.yy980 = createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &yymsp[0].minor.yy0); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; - case 512: /* signed ::= NK_PLUS NK_FLOAT */ -{ yymsp[-1].minor.yy974 = createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &yymsp[0].minor.yy0); } + case 513: /* signed ::= NK_PLUS NK_FLOAT */ +{ yymsp[-1].minor.yy980 = createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &yymsp[0].minor.yy0); } break; - case 513: /* signed ::= NK_MINUS NK_FLOAT */ + case 514: /* 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.yy974 = createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &t); + yylhsminor.yy980 = createValueNode(pCxt, TSDB_DATA_TYPE_DOUBLE, &t); } - yymsp[-1].minor.yy974 = yylhsminor.yy974; + yymsp[-1].minor.yy980 = yylhsminor.yy980; break; - case 515: /* signed_literal ::= NK_STRING */ -{ yylhsminor.yy974 = createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0); } - yymsp[0].minor.yy974 = yylhsminor.yy974; + case 516: /* signed_literal ::= NK_STRING */ +{ yylhsminor.yy980 = createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; - case 516: /* signed_literal ::= NK_BOOL */ -{ yylhsminor.yy974 = createValueNode(pCxt, TSDB_DATA_TYPE_BOOL, &yymsp[0].minor.yy0); } - yymsp[0].minor.yy974 = yylhsminor.yy974; + case 517: /* signed_literal ::= NK_BOOL */ +{ yylhsminor.yy980 = createValueNode(pCxt, TSDB_DATA_TYPE_BOOL, &yymsp[0].minor.yy0); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; - case 517: /* signed_literal ::= TIMESTAMP NK_STRING */ -{ yymsp[-1].minor.yy974 = createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0); } + case 518: /* signed_literal ::= TIMESTAMP NK_STRING */ +{ yymsp[-1].minor.yy980 = createValueNode(pCxt, TSDB_DATA_TYPE_TIMESTAMP, &yymsp[0].minor.yy0); } break; - case 518: /* signed_literal ::= duration_literal */ - case 520: /* signed_literal ::= literal_func */ yytestcase(yyruleno==520); - case 619: /* star_func_para ::= expr_or_subquery */ yytestcase(yyruleno==619); - case 702: /* select_item ::= common_expression */ yytestcase(yyruleno==702); - case 712: /* partition_item ::= expr_or_subquery */ yytestcase(yyruleno==712); - case 757: /* query_simple_or_subquery ::= subquery */ yytestcase(yyruleno==757); - case 759: /* query_or_subquery ::= subquery */ yytestcase(yyruleno==759); - case 772: /* search_condition ::= common_expression */ yytestcase(yyruleno==772); -{ yylhsminor.yy974 = releaseRawExprNode(pCxt, yymsp[0].minor.yy974); } - yymsp[0].minor.yy974 = yylhsminor.yy974; + case 519: /* signed_literal ::= duration_literal */ + case 521: /* signed_literal ::= literal_func */ yytestcase(yyruleno==521); + case 620: /* star_func_para ::= expr_or_subquery */ yytestcase(yyruleno==620); + case 703: /* select_item ::= common_expression */ yytestcase(yyruleno==703); + case 713: /* partition_item ::= expr_or_subquery */ yytestcase(yyruleno==713); + case 758: /* query_simple_or_subquery ::= subquery */ yytestcase(yyruleno==758); + case 760: /* query_or_subquery ::= subquery */ yytestcase(yyruleno==760); + case 773: /* search_condition ::= common_expression */ yytestcase(yyruleno==773); +{ yylhsminor.yy980 = releaseRawExprNode(pCxt, yymsp[0].minor.yy980); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; - case 519: /* signed_literal ::= NULL */ -{ yylhsminor.yy974 = createValueNode(pCxt, TSDB_DATA_TYPE_NULL, &yymsp[0].minor.yy0); } - yymsp[0].minor.yy974 = yylhsminor.yy974; + case 520: /* signed_literal ::= NULL */ +{ yylhsminor.yy980 = createValueNode(pCxt, TSDB_DATA_TYPE_NULL, &yymsp[0].minor.yy0); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; - case 521: /* signed_literal ::= NK_QUESTION */ -{ yylhsminor.yy974 = createPlaceholderValueNode(pCxt, &yymsp[0].minor.yy0); } - yymsp[0].minor.yy974 = yylhsminor.yy974; + case 522: /* signed_literal ::= NK_QUESTION */ +{ yylhsminor.yy980 = createPlaceholderValueNode(pCxt, &yymsp[0].minor.yy0); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; - case 540: /* expression ::= pseudo_column */ -{ yylhsminor.yy974 = yymsp[0].minor.yy974; (void)setRawExprNodeIsPseudoColumn(pCxt, yylhsminor.yy974, true); } - yymsp[0].minor.yy974 = yylhsminor.yy974; + case 541: /* expression ::= pseudo_column */ +{ yylhsminor.yy980 = yymsp[0].minor.yy980; (void)setRawExprNodeIsPseudoColumn(pCxt, yylhsminor.yy980, true); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; - case 544: /* expression ::= NK_LP expression NK_RP */ - case 653: /* boolean_primary ::= NK_LP boolean_value_expression NK_RP */ yytestcase(yyruleno==653); - case 771: /* subquery ::= NK_LP subquery NK_RP */ yytestcase(yyruleno==771); -{ yylhsminor.yy974 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0, releaseRawExprNode(pCxt, yymsp[-1].minor.yy974)); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + case 545: /* expression ::= NK_LP expression NK_RP */ + case 654: /* boolean_primary ::= NK_LP boolean_value_expression NK_RP */ yytestcase(yyruleno==654); + case 772: /* subquery ::= NK_LP subquery NK_RP */ yytestcase(yyruleno==772); +{ yylhsminor.yy980 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0, releaseRawExprNode(pCxt, yymsp[-1].minor.yy980)); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 545: /* expression ::= NK_PLUS expr_or_subquery */ + case 546: /* expression ::= NK_PLUS expr_or_subquery */ { - SToken t = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy974); - yylhsminor.yy974 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &t, releaseRawExprNode(pCxt, yymsp[0].minor.yy974)); + SToken t = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy980); + yylhsminor.yy980 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &t, releaseRawExprNode(pCxt, yymsp[0].minor.yy980)); } - yymsp[-1].minor.yy974 = yylhsminor.yy974; + yymsp[-1].minor.yy980 = yylhsminor.yy980; break; - case 546: /* expression ::= NK_MINUS expr_or_subquery */ + case 547: /* expression ::= NK_MINUS expr_or_subquery */ { - SToken t = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy974); - yylhsminor.yy974 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &t, createOperatorNode(pCxt, OP_TYPE_MINUS, releaseRawExprNode(pCxt, yymsp[0].minor.yy974), NULL)); + SToken t = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy980); + yylhsminor.yy980 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &t, createOperatorNode(pCxt, OP_TYPE_MINUS, releaseRawExprNode(pCxt, yymsp[0].minor.yy980), NULL)); } - yymsp[-1].minor.yy974 = yylhsminor.yy974; + yymsp[-1].minor.yy980 = yylhsminor.yy980; break; - case 547: /* expression ::= expr_or_subquery NK_PLUS expr_or_subquery */ + case 548: /* expression ::= expr_or_subquery NK_PLUS expr_or_subquery */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy974); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy974); - yylhsminor.yy974 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_ADD, releaseRawExprNode(pCxt, yymsp[-2].minor.yy974), releaseRawExprNode(pCxt, yymsp[0].minor.yy974))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy980); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy980); + yylhsminor.yy980 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_ADD, releaseRawExprNode(pCxt, yymsp[-2].minor.yy980), releaseRawExprNode(pCxt, yymsp[0].minor.yy980))); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 548: /* expression ::= expr_or_subquery NK_MINUS expr_or_subquery */ + case 549: /* expression ::= expr_or_subquery NK_MINUS expr_or_subquery */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy974); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy974); - yylhsminor.yy974 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_SUB, releaseRawExprNode(pCxt, yymsp[-2].minor.yy974), releaseRawExprNode(pCxt, yymsp[0].minor.yy974))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy980); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy980); + yylhsminor.yy980 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_SUB, releaseRawExprNode(pCxt, yymsp[-2].minor.yy980), releaseRawExprNode(pCxt, yymsp[0].minor.yy980))); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 549: /* expression ::= expr_or_subquery NK_STAR expr_or_subquery */ + case 550: /* expression ::= expr_or_subquery NK_STAR expr_or_subquery */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy974); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy974); - yylhsminor.yy974 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_MULTI, releaseRawExprNode(pCxt, yymsp[-2].minor.yy974), releaseRawExprNode(pCxt, yymsp[0].minor.yy974))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy980); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy980); + yylhsminor.yy980 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_MULTI, releaseRawExprNode(pCxt, yymsp[-2].minor.yy980), releaseRawExprNode(pCxt, yymsp[0].minor.yy980))); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 550: /* expression ::= expr_or_subquery NK_SLASH expr_or_subquery */ + case 551: /* expression ::= expr_or_subquery NK_SLASH expr_or_subquery */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy974); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy974); - yylhsminor.yy974 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_DIV, releaseRawExprNode(pCxt, yymsp[-2].minor.yy974), releaseRawExprNode(pCxt, yymsp[0].minor.yy974))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy980); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy980); + yylhsminor.yy980 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_DIV, releaseRawExprNode(pCxt, yymsp[-2].minor.yy980), releaseRawExprNode(pCxt, yymsp[0].minor.yy980))); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 551: /* expression ::= expr_or_subquery NK_REM expr_or_subquery */ + case 552: /* expression ::= expr_or_subquery NK_REM expr_or_subquery */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy974); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy974); - yylhsminor.yy974 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_REM, releaseRawExprNode(pCxt, yymsp[-2].minor.yy974), releaseRawExprNode(pCxt, yymsp[0].minor.yy974))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy980); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy980); + yylhsminor.yy980 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_REM, releaseRawExprNode(pCxt, yymsp[-2].minor.yy980), releaseRawExprNode(pCxt, yymsp[0].minor.yy980))); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 552: /* expression ::= column_reference NK_ARROW NK_STRING */ + case 553: /* expression ::= column_reference NK_ARROW NK_STRING */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy974); - yylhsminor.yy974 = createRawExprNodeExt(pCxt, &s, &yymsp[0].minor.yy0, createOperatorNode(pCxt, OP_TYPE_JSON_GET_VALUE, releaseRawExprNode(pCxt, yymsp[-2].minor.yy974), createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy980); + yylhsminor.yy980 = createRawExprNodeExt(pCxt, &s, &yymsp[0].minor.yy0, createOperatorNode(pCxt, OP_TYPE_JSON_GET_VALUE, releaseRawExprNode(pCxt, yymsp[-2].minor.yy980), createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[0].minor.yy0))); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 553: /* expression ::= expr_or_subquery NK_BITAND expr_or_subquery */ + case 554: /* expression ::= expr_or_subquery NK_BITAND expr_or_subquery */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy974); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy974); - yylhsminor.yy974 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_BIT_AND, releaseRawExprNode(pCxt, yymsp[-2].minor.yy974), releaseRawExprNode(pCxt, yymsp[0].minor.yy974))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy980); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy980); + yylhsminor.yy980 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_BIT_AND, releaseRawExprNode(pCxt, yymsp[-2].minor.yy980), releaseRawExprNode(pCxt, yymsp[0].minor.yy980))); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 554: /* expression ::= expr_or_subquery NK_BITOR expr_or_subquery */ + case 555: /* expression ::= expr_or_subquery NK_BITOR expr_or_subquery */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy974); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy974); - yylhsminor.yy974 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_BIT_OR, releaseRawExprNode(pCxt, yymsp[-2].minor.yy974), releaseRawExprNode(pCxt, yymsp[0].minor.yy974))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy980); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy980); + yylhsminor.yy980 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, OP_TYPE_BIT_OR, releaseRawExprNode(pCxt, yymsp[-2].minor.yy980), releaseRawExprNode(pCxt, yymsp[0].minor.yy980))); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 557: /* column_reference ::= column_name */ -{ yylhsminor.yy974 = createRawExprNode(pCxt, &yymsp[0].minor.yy557, createColumnNode(pCxt, NULL, &yymsp[0].minor.yy557)); } - yymsp[0].minor.yy974 = yylhsminor.yy974; + case 558: /* column_reference ::= column_name */ +{ yylhsminor.yy980 = createRawExprNode(pCxt, &yymsp[0].minor.yy561, createColumnNode(pCxt, NULL, &yymsp[0].minor.yy561)); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; - case 558: /* column_reference ::= table_name NK_DOT column_name */ -{ yylhsminor.yy974 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy557, &yymsp[0].minor.yy557, createColumnNode(pCxt, &yymsp[-2].minor.yy557, &yymsp[0].minor.yy557)); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + case 559: /* column_reference ::= table_name NK_DOT column_name */ +{ yylhsminor.yy980 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy561, &yymsp[0].minor.yy561, createColumnNode(pCxt, &yymsp[-2].minor.yy561, &yymsp[0].minor.yy561)); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 559: /* column_reference ::= NK_ALIAS */ -{ yylhsminor.yy974 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createColumnNode(pCxt, NULL, &yymsp[0].minor.yy0)); } - yymsp[0].minor.yy974 = yylhsminor.yy974; + case 560: /* column_reference ::= NK_ALIAS */ +{ yylhsminor.yy980 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createColumnNode(pCxt, NULL, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; - case 560: /* column_reference ::= table_name NK_DOT NK_ALIAS */ -{ yylhsminor.yy974 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy557, &yymsp[0].minor.yy0, createColumnNode(pCxt, &yymsp[-2].minor.yy557, &yymsp[0].minor.yy0)); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + case 561: /* column_reference ::= table_name NK_DOT NK_ALIAS */ +{ yylhsminor.yy980 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy561, &yymsp[0].minor.yy0, createColumnNode(pCxt, &yymsp[-2].minor.yy561, &yymsp[0].minor.yy0)); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 561: /* pseudo_column ::= ROWTS */ - case 562: /* pseudo_column ::= TBNAME */ yytestcase(yyruleno==562); - case 564: /* pseudo_column ::= QSTART */ yytestcase(yyruleno==564); - case 565: /* pseudo_column ::= QEND */ yytestcase(yyruleno==565); - case 566: /* pseudo_column ::= QDURATION */ yytestcase(yyruleno==566); - case 567: /* pseudo_column ::= WSTART */ yytestcase(yyruleno==567); - case 568: /* pseudo_column ::= WEND */ yytestcase(yyruleno==568); - case 569: /* pseudo_column ::= WDURATION */ yytestcase(yyruleno==569); - case 570: /* pseudo_column ::= IROWTS */ yytestcase(yyruleno==570); - case 571: /* pseudo_column ::= ISFILLED */ yytestcase(yyruleno==571); - case 572: /* pseudo_column ::= QTAGS */ yytestcase(yyruleno==572); - case 573: /* pseudo_column ::= FLOW */ yytestcase(yyruleno==573); - case 574: /* pseudo_column ::= FHIGH */ yytestcase(yyruleno==574); - case 575: /* pseudo_column ::= FROWTS */ yytestcase(yyruleno==575); - case 592: /* literal_func ::= NOW */ yytestcase(yyruleno==592); - case 593: /* literal_func ::= TODAY */ yytestcase(yyruleno==593); -{ yylhsminor.yy974 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[0].minor.yy0, NULL)); } - yymsp[0].minor.yy974 = yylhsminor.yy974; + case 562: /* pseudo_column ::= ROWTS */ + case 563: /* pseudo_column ::= TBNAME */ yytestcase(yyruleno==563); + case 565: /* pseudo_column ::= QSTART */ yytestcase(yyruleno==565); + case 566: /* pseudo_column ::= QEND */ yytestcase(yyruleno==566); + case 567: /* pseudo_column ::= QDURATION */ yytestcase(yyruleno==567); + case 568: /* pseudo_column ::= WSTART */ yytestcase(yyruleno==568); + case 569: /* pseudo_column ::= WEND */ yytestcase(yyruleno==569); + case 570: /* pseudo_column ::= WDURATION */ yytestcase(yyruleno==570); + case 571: /* pseudo_column ::= IROWTS */ yytestcase(yyruleno==571); + case 572: /* pseudo_column ::= ISFILLED */ yytestcase(yyruleno==572); + case 573: /* pseudo_column ::= QTAGS */ yytestcase(yyruleno==573); + case 574: /* pseudo_column ::= FLOW */ yytestcase(yyruleno==574); + case 575: /* pseudo_column ::= FHIGH */ yytestcase(yyruleno==575); + case 576: /* pseudo_column ::= FROWTS */ yytestcase(yyruleno==576); + case 593: /* literal_func ::= NOW */ yytestcase(yyruleno==593); + case 594: /* literal_func ::= TODAY */ yytestcase(yyruleno==594); +{ yylhsminor.yy980 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[0].minor.yy0, NULL)); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; - case 563: /* pseudo_column ::= table_name NK_DOT TBNAME */ -{ yylhsminor.yy974 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy557, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[0].minor.yy0, createNodeList(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[-2].minor.yy557)))); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + case 564: /* pseudo_column ::= table_name NK_DOT TBNAME */ +{ yylhsminor.yy980 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy561, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[0].minor.yy0, createNodeList(pCxt, createValueNode(pCxt, TSDB_DATA_TYPE_BINARY, &yymsp[-2].minor.yy561)))); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 576: /* function_expression ::= function_name NK_LP expression_list NK_RP */ - case 577: /* function_expression ::= star_func NK_LP star_func_para_list NK_RP */ yytestcase(yyruleno==577); - case 585: /* function_expression ::= substr_func NK_LP expression_list NK_RP */ yytestcase(yyruleno==585); -{ yylhsminor.yy974 = createRawExprNodeExt(pCxt, &yymsp[-3].minor.yy557, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[-3].minor.yy557, yymsp[-1].minor.yy946)); } - yymsp[-3].minor.yy974 = yylhsminor.yy974; + case 577: /* function_expression ::= function_name NK_LP expression_list NK_RP */ + case 578: /* function_expression ::= star_func NK_LP star_func_para_list NK_RP */ yytestcase(yyruleno==578); + case 586: /* function_expression ::= substr_func NK_LP expression_list NK_RP */ yytestcase(yyruleno==586); +{ yylhsminor.yy980 = createRawExprNodeExt(pCxt, &yymsp[-3].minor.yy561, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[-3].minor.yy561, yymsp[-1].minor.yy628)); } + yymsp[-3].minor.yy980 = yylhsminor.yy980; break; - case 578: /* function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP */ - case 579: /* function_expression ::= CAST NK_LP expr_or_subquery AS type_name_default_len NK_RP */ yytestcase(yyruleno==579); -{ yylhsminor.yy974 = createRawExprNodeExt(pCxt, &yymsp[-5].minor.yy0, &yymsp[0].minor.yy0, createCastFunctionNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy974), yymsp[-1].minor.yy424)); } - yymsp[-5].minor.yy974 = yylhsminor.yy974; + case 579: /* function_expression ::= CAST NK_LP expr_or_subquery AS type_name NK_RP */ + case 580: /* function_expression ::= CAST NK_LP expr_or_subquery AS type_name_default_len NK_RP */ yytestcase(yyruleno==580); +{ yylhsminor.yy980 = createRawExprNodeExt(pCxt, &yymsp[-5].minor.yy0, &yymsp[0].minor.yy0, createCastFunctionNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy980), yymsp[-1].minor.yy896)); } + yymsp[-5].minor.yy980 = yylhsminor.yy980; break; - case 580: /* function_expression ::= POSITION NK_LP expr_or_subquery IN expr_or_subquery NK_RP */ -{ yylhsminor.yy974 = createRawExprNodeExt(pCxt, &yymsp[-5].minor.yy0, &yymsp[0].minor.yy0, createPositionFunctionNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy974), releaseRawExprNode(pCxt, yymsp[-1].minor.yy974))); } - yymsp[-5].minor.yy974 = yylhsminor.yy974; + case 581: /* function_expression ::= POSITION NK_LP expr_or_subquery IN expr_or_subquery NK_RP */ +{ yylhsminor.yy980 = createRawExprNodeExt(pCxt, &yymsp[-5].minor.yy0, &yymsp[0].minor.yy0, createPositionFunctionNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy980), releaseRawExprNode(pCxt, yymsp[-1].minor.yy980))); } + yymsp[-5].minor.yy980 = yylhsminor.yy980; break; - case 581: /* function_expression ::= TRIM NK_LP expr_or_subquery NK_RP */ -{ yylhsminor.yy974 = createRawExprNodeExt(pCxt, &yymsp[-3].minor.yy0, &yymsp[0].minor.yy0, createTrimFunctionNode(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy974), TRIM_TYPE_BOTH)); } - yymsp[-3].minor.yy974 = yylhsminor.yy974; + case 582: /* function_expression ::= TRIM NK_LP expr_or_subquery NK_RP */ +{ yylhsminor.yy980 = createRawExprNodeExt(pCxt, &yymsp[-3].minor.yy0, &yymsp[0].minor.yy0, createTrimFunctionNode(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy980), TRIM_TYPE_BOTH)); } + yymsp[-3].minor.yy980 = yylhsminor.yy980; break; - case 582: /* function_expression ::= TRIM NK_LP trim_specification_type FROM expr_or_subquery NK_RP */ -{ yylhsminor.yy974 = createRawExprNodeExt(pCxt, &yymsp[-5].minor.yy0, &yymsp[0].minor.yy0, createTrimFunctionNode(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy974), yymsp[-3].minor.yy300)); } - yymsp[-5].minor.yy974 = yylhsminor.yy974; + case 583: /* function_expression ::= TRIM NK_LP trim_specification_type FROM expr_or_subquery NK_RP */ +{ yylhsminor.yy980 = createRawExprNodeExt(pCxt, &yymsp[-5].minor.yy0, &yymsp[0].minor.yy0, createTrimFunctionNode(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy980), yymsp[-3].minor.yy840)); } + yymsp[-5].minor.yy980 = yylhsminor.yy980; break; - case 583: /* function_expression ::= TRIM NK_LP expr_or_subquery FROM expr_or_subquery NK_RP */ -{ yylhsminor.yy974 = createRawExprNodeExt(pCxt, &yymsp[-5].minor.yy0, &yymsp[0].minor.yy0, createTrimFunctionNodeExt(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy974), releaseRawExprNode(pCxt, yymsp[-1].minor.yy974), TRIM_TYPE_BOTH)); } - yymsp[-5].minor.yy974 = yylhsminor.yy974; + case 584: /* function_expression ::= TRIM NK_LP expr_or_subquery FROM expr_or_subquery NK_RP */ +{ yylhsminor.yy980 = createRawExprNodeExt(pCxt, &yymsp[-5].minor.yy0, &yymsp[0].minor.yy0, createTrimFunctionNodeExt(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy980), releaseRawExprNode(pCxt, yymsp[-1].minor.yy980), TRIM_TYPE_BOTH)); } + yymsp[-5].minor.yy980 = yylhsminor.yy980; break; - case 584: /* function_expression ::= TRIM NK_LP trim_specification_type expr_or_subquery FROM expr_or_subquery NK_RP */ -{ yylhsminor.yy974 = createRawExprNodeExt(pCxt, &yymsp[-6].minor.yy0, &yymsp[0].minor.yy0, createTrimFunctionNodeExt(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy974), releaseRawExprNode(pCxt, yymsp[-1].minor.yy974), yymsp[-4].minor.yy300)); } - yymsp[-6].minor.yy974 = yylhsminor.yy974; + case 585: /* function_expression ::= TRIM NK_LP trim_specification_type expr_or_subquery FROM expr_or_subquery NK_RP */ +{ yylhsminor.yy980 = createRawExprNodeExt(pCxt, &yymsp[-6].minor.yy0, &yymsp[0].minor.yy0, createTrimFunctionNodeExt(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy980), releaseRawExprNode(pCxt, yymsp[-1].minor.yy980), yymsp[-4].minor.yy840)); } + yymsp[-6].minor.yy980 = yylhsminor.yy980; break; - case 586: /* function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery NK_RP */ -{ yylhsminor.yy974 = createRawExprNodeExt(pCxt, &yymsp[-5].minor.yy557, &yymsp[0].minor.yy0, createSubstrFunctionNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy974), releaseRawExprNode(pCxt, yymsp[-1].minor.yy974))); } - yymsp[-5].minor.yy974 = yylhsminor.yy974; + case 587: /* function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery NK_RP */ +{ yylhsminor.yy980 = createRawExprNodeExt(pCxt, &yymsp[-5].minor.yy561, &yymsp[0].minor.yy0, createSubstrFunctionNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy980), releaseRawExprNode(pCxt, yymsp[-1].minor.yy980))); } + yymsp[-5].minor.yy980 = yylhsminor.yy980; break; - case 587: /* function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery FOR expr_or_subquery NK_RP */ -{ yylhsminor.yy974 = createRawExprNodeExt(pCxt, &yymsp[-7].minor.yy557, &yymsp[0].minor.yy0, createSubstrFunctionNodeExt(pCxt, releaseRawExprNode(pCxt, yymsp[-5].minor.yy974), releaseRawExprNode(pCxt, yymsp[-3].minor.yy974), releaseRawExprNode(pCxt, yymsp[-1].minor.yy974))); } - yymsp[-7].minor.yy974 = yylhsminor.yy974; + case 588: /* function_expression ::= substr_func NK_LP expr_or_subquery FROM expr_or_subquery FOR expr_or_subquery NK_RP */ +{ yylhsminor.yy980 = createRawExprNodeExt(pCxt, &yymsp[-7].minor.yy561, &yymsp[0].minor.yy0, createSubstrFunctionNodeExt(pCxt, releaseRawExprNode(pCxt, yymsp[-5].minor.yy980), releaseRawExprNode(pCxt, yymsp[-3].minor.yy980), releaseRawExprNode(pCxt, yymsp[-1].minor.yy980))); } + yymsp[-7].minor.yy980 = yylhsminor.yy980; break; - case 588: /* function_expression ::= REPLACE NK_LP expression_list NK_RP */ - case 595: /* rand_func ::= RAND NK_LP expression_list NK_RP */ yytestcase(yyruleno==595); -{ yylhsminor.yy974 = createRawExprNodeExt(pCxt, &yymsp[-3].minor.yy0, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[-3].minor.yy0, yymsp[-1].minor.yy946)); } - yymsp[-3].minor.yy974 = yylhsminor.yy974; + case 589: /* function_expression ::= REPLACE NK_LP expression_list NK_RP */ + case 596: /* rand_func ::= RAND NK_LP expression_list NK_RP */ yytestcase(yyruleno==596); +{ yylhsminor.yy980 = createRawExprNodeExt(pCxt, &yymsp[-3].minor.yy0, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[-3].minor.yy0, yymsp[-1].minor.yy628)); } + yymsp[-3].minor.yy980 = yylhsminor.yy980; break; - case 591: /* literal_func ::= noarg_func NK_LP NK_RP */ -{ yylhsminor.yy974 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy557, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[-2].minor.yy557, NULL)); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + case 592: /* literal_func ::= noarg_func NK_LP NK_RP */ +{ yylhsminor.yy980 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy561, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[-2].minor.yy561, NULL)); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 594: /* rand_func ::= RAND NK_LP NK_RP */ -{ yylhsminor.yy974 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[-2].minor.yy0, NULL)); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + case 595: /* rand_func ::= RAND NK_LP NK_RP */ +{ yylhsminor.yy980 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0, createFunctionNode(pCxt, &yymsp[-2].minor.yy0, NULL)); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 598: /* trim_specification_type ::= BOTH */ -{ yymsp[0].minor.yy300 = TRIM_TYPE_BOTH; } + case 599: /* trim_specification_type ::= BOTH */ +{ yymsp[0].minor.yy840 = TRIM_TYPE_BOTH; } break; - case 599: /* trim_specification_type ::= TRAILING */ -{ yymsp[0].minor.yy300 = TRIM_TYPE_TRAILING; } + case 600: /* trim_specification_type ::= TRAILING */ +{ yymsp[0].minor.yy840 = TRIM_TYPE_TRAILING; } break; - case 600: /* trim_specification_type ::= LEADING */ -{ yymsp[0].minor.yy300 = TRIM_TYPE_LEADING; } + case 601: /* trim_specification_type ::= LEADING */ +{ yymsp[0].minor.yy840 = TRIM_TYPE_LEADING; } break; - case 615: /* star_func_para_list ::= NK_STAR */ -{ yylhsminor.yy946 = createNodeList(pCxt, createColumnNode(pCxt, NULL, &yymsp[0].minor.yy0)); } - yymsp[0].minor.yy946 = yylhsminor.yy946; + case 616: /* star_func_para_list ::= NK_STAR */ +{ yylhsminor.yy628 = createNodeList(pCxt, createColumnNode(pCxt, NULL, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy628 = yylhsminor.yy628; break; - case 620: /* star_func_para ::= table_name NK_DOT NK_STAR */ - case 705: /* select_item ::= table_name NK_DOT NK_STAR */ yytestcase(yyruleno==705); -{ yylhsminor.yy974 = createColumnNode(pCxt, &yymsp[-2].minor.yy557, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + case 621: /* star_func_para ::= table_name NK_DOT NK_STAR */ + case 706: /* select_item ::= table_name NK_DOT NK_STAR */ yytestcase(yyruleno==706); +{ yylhsminor.yy980 = createColumnNode(pCxt, &yymsp[-2].minor.yy561, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 621: /* case_when_expression ::= CASE when_then_list case_when_else_opt END */ -{ yylhsminor.yy974 = createRawExprNodeExt(pCxt, &yymsp[-3].minor.yy0, &yymsp[0].minor.yy0, createCaseWhenNode(pCxt, NULL, yymsp[-2].minor.yy946, yymsp[-1].minor.yy974)); } - yymsp[-3].minor.yy974 = yylhsminor.yy974; + case 622: /* case_when_expression ::= CASE when_then_list case_when_else_opt END */ +{ yylhsminor.yy980 = createRawExprNodeExt(pCxt, &yymsp[-3].minor.yy0, &yymsp[0].minor.yy0, createCaseWhenNode(pCxt, NULL, yymsp[-2].minor.yy628, yymsp[-1].minor.yy980)); } + yymsp[-3].minor.yy980 = yylhsminor.yy980; break; - case 622: /* case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END */ -{ yylhsminor.yy974 = createRawExprNodeExt(pCxt, &yymsp[-4].minor.yy0, &yymsp[0].minor.yy0, createCaseWhenNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy974), yymsp[-2].minor.yy946, yymsp[-1].minor.yy974)); } - yymsp[-4].minor.yy974 = yylhsminor.yy974; + case 623: /* case_when_expression ::= CASE common_expression when_then_list case_when_else_opt END */ +{ yylhsminor.yy980 = createRawExprNodeExt(pCxt, &yymsp[-4].minor.yy0, &yymsp[0].minor.yy0, createCaseWhenNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy980), yymsp[-2].minor.yy628, yymsp[-1].minor.yy980)); } + yymsp[-4].minor.yy980 = yylhsminor.yy980; break; - case 625: /* when_then_expr ::= WHEN common_expression THEN common_expression */ -{ yymsp[-3].minor.yy974 = createWhenThenNode(pCxt, releaseRawExprNode(pCxt, yymsp[-2].minor.yy974), releaseRawExprNode(pCxt, yymsp[0].minor.yy974)); } + case 626: /* when_then_expr ::= WHEN common_expression THEN common_expression */ +{ yymsp[-3].minor.yy980 = createWhenThenNode(pCxt, releaseRawExprNode(pCxt, yymsp[-2].minor.yy980), releaseRawExprNode(pCxt, yymsp[0].minor.yy980)); } break; - case 627: /* case_when_else_opt ::= ELSE common_expression */ -{ yymsp[-1].minor.yy974 = releaseRawExprNode(pCxt, yymsp[0].minor.yy974); } + case 628: /* case_when_else_opt ::= ELSE common_expression */ +{ yymsp[-1].minor.yy980 = releaseRawExprNode(pCxt, yymsp[0].minor.yy980); } break; - case 628: /* predicate ::= expr_or_subquery compare_op expr_or_subquery */ - case 633: /* predicate ::= expr_or_subquery in_op in_predicate_value */ yytestcase(yyruleno==633); + case 629: /* predicate ::= expr_or_subquery compare_op expr_or_subquery */ + case 634: /* predicate ::= expr_or_subquery in_op in_predicate_value */ yytestcase(yyruleno==634); { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy974); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy974); - yylhsminor.yy974 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, yymsp[-1].minor.yy140, releaseRawExprNode(pCxt, yymsp[-2].minor.yy974), releaseRawExprNode(pCxt, yymsp[0].minor.yy974))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy980); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy980); + yylhsminor.yy980 = createRawExprNodeExt(pCxt, &s, &e, createOperatorNode(pCxt, yymsp[-1].minor.yy688, releaseRawExprNode(pCxt, yymsp[-2].minor.yy980), releaseRawExprNode(pCxt, yymsp[0].minor.yy980))); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 629: /* predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery */ + case 630: /* predicate ::= expr_or_subquery BETWEEN expr_or_subquery AND expr_or_subquery */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-4].minor.yy974); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy974); - yylhsminor.yy974 = createRawExprNodeExt(pCxt, &s, &e, createBetweenAnd(pCxt, releaseRawExprNode(pCxt, yymsp[-4].minor.yy974), releaseRawExprNode(pCxt, yymsp[-2].minor.yy974), releaseRawExprNode(pCxt, yymsp[0].minor.yy974))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-4].minor.yy980); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy980); + yylhsminor.yy980 = createRawExprNodeExt(pCxt, &s, &e, createBetweenAnd(pCxt, releaseRawExprNode(pCxt, yymsp[-4].minor.yy980), releaseRawExprNode(pCxt, yymsp[-2].minor.yy980), releaseRawExprNode(pCxt, yymsp[0].minor.yy980))); } - yymsp[-4].minor.yy974 = yylhsminor.yy974; + yymsp[-4].minor.yy980 = yylhsminor.yy980; break; - case 630: /* predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery */ + case 631: /* predicate ::= expr_or_subquery NOT BETWEEN expr_or_subquery AND expr_or_subquery */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-5].minor.yy974); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy974); - yylhsminor.yy974 = createRawExprNodeExt(pCxt, &s, &e, createNotBetweenAnd(pCxt, releaseRawExprNode(pCxt, yymsp[-5].minor.yy974), releaseRawExprNode(pCxt, yymsp[-2].minor.yy974), releaseRawExprNode(pCxt, yymsp[0].minor.yy974))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-5].minor.yy980); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy980); + yylhsminor.yy980 = createRawExprNodeExt(pCxt, &s, &e, createNotBetweenAnd(pCxt, releaseRawExprNode(pCxt, yymsp[-5].minor.yy980), releaseRawExprNode(pCxt, yymsp[-2].minor.yy980), releaseRawExprNode(pCxt, yymsp[0].minor.yy980))); } - yymsp[-5].minor.yy974 = yylhsminor.yy974; + yymsp[-5].minor.yy980 = yylhsminor.yy980; break; - case 631: /* predicate ::= expr_or_subquery IS NULL */ + case 632: /* predicate ::= expr_or_subquery IS NULL */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy974); - yylhsminor.yy974 = createRawExprNodeExt(pCxt, &s, &yymsp[0].minor.yy0, createOperatorNode(pCxt, OP_TYPE_IS_NULL, releaseRawExprNode(pCxt, yymsp[-2].minor.yy974), NULL)); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy980); + yylhsminor.yy980 = createRawExprNodeExt(pCxt, &s, &yymsp[0].minor.yy0, createOperatorNode(pCxt, OP_TYPE_IS_NULL, releaseRawExprNode(pCxt, yymsp[-2].minor.yy980), NULL)); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 632: /* predicate ::= expr_or_subquery IS NOT NULL */ + case 633: /* predicate ::= expr_or_subquery IS NOT NULL */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-3].minor.yy974); - yylhsminor.yy974 = createRawExprNodeExt(pCxt, &s, &yymsp[0].minor.yy0, createOperatorNode(pCxt, OP_TYPE_IS_NOT_NULL, releaseRawExprNode(pCxt, yymsp[-3].minor.yy974), NULL)); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-3].minor.yy980); + yylhsminor.yy980 = createRawExprNodeExt(pCxt, &s, &yymsp[0].minor.yy0, createOperatorNode(pCxt, OP_TYPE_IS_NOT_NULL, releaseRawExprNode(pCxt, yymsp[-3].minor.yy980), NULL)); } - yymsp[-3].minor.yy974 = yylhsminor.yy974; + yymsp[-3].minor.yy980 = yylhsminor.yy980; break; - case 634: /* compare_op ::= NK_LT */ -{ yymsp[0].minor.yy140 = OP_TYPE_LOWER_THAN; } + case 635: /* compare_op ::= NK_LT */ +{ yymsp[0].minor.yy688 = OP_TYPE_LOWER_THAN; } break; - case 635: /* compare_op ::= NK_GT */ -{ yymsp[0].minor.yy140 = OP_TYPE_GREATER_THAN; } + case 636: /* compare_op ::= NK_GT */ +{ yymsp[0].minor.yy688 = OP_TYPE_GREATER_THAN; } break; - case 636: /* compare_op ::= NK_LE */ -{ yymsp[0].minor.yy140 = OP_TYPE_LOWER_EQUAL; } + case 637: /* compare_op ::= NK_LE */ +{ yymsp[0].minor.yy688 = OP_TYPE_LOWER_EQUAL; } break; - case 637: /* compare_op ::= NK_GE */ -{ yymsp[0].minor.yy140 = OP_TYPE_GREATER_EQUAL; } + case 638: /* compare_op ::= NK_GE */ +{ yymsp[0].minor.yy688 = OP_TYPE_GREATER_EQUAL; } break; - case 638: /* compare_op ::= NK_NE */ -{ yymsp[0].minor.yy140 = OP_TYPE_NOT_EQUAL; } + case 639: /* compare_op ::= NK_NE */ +{ yymsp[0].minor.yy688 = OP_TYPE_NOT_EQUAL; } break; - case 639: /* compare_op ::= NK_EQ */ -{ yymsp[0].minor.yy140 = OP_TYPE_EQUAL; } + case 640: /* compare_op ::= NK_EQ */ +{ yymsp[0].minor.yy688 = OP_TYPE_EQUAL; } break; - case 640: /* compare_op ::= LIKE */ -{ yymsp[0].minor.yy140 = OP_TYPE_LIKE; } + case 641: /* compare_op ::= LIKE */ +{ yymsp[0].minor.yy688 = OP_TYPE_LIKE; } break; - case 641: /* compare_op ::= NOT LIKE */ -{ yymsp[-1].minor.yy140 = OP_TYPE_NOT_LIKE; } + case 642: /* compare_op ::= NOT LIKE */ +{ yymsp[-1].minor.yy688 = OP_TYPE_NOT_LIKE; } break; - case 642: /* compare_op ::= MATCH */ -{ yymsp[0].minor.yy140 = OP_TYPE_MATCH; } + case 643: /* compare_op ::= MATCH */ +{ yymsp[0].minor.yy688 = OP_TYPE_MATCH; } break; - case 643: /* compare_op ::= NMATCH */ -{ yymsp[0].minor.yy140 = OP_TYPE_NMATCH; } + case 644: /* compare_op ::= NMATCH */ +{ yymsp[0].minor.yy688 = OP_TYPE_NMATCH; } break; - case 644: /* compare_op ::= CONTAINS */ -{ yymsp[0].minor.yy140 = OP_TYPE_JSON_CONTAINS; } + case 645: /* compare_op ::= CONTAINS */ +{ yymsp[0].minor.yy688 = OP_TYPE_JSON_CONTAINS; } break; - case 645: /* in_op ::= IN */ -{ yymsp[0].minor.yy140 = OP_TYPE_IN; } + case 646: /* in_op ::= IN */ +{ yymsp[0].minor.yy688 = OP_TYPE_IN; } break; - case 646: /* in_op ::= NOT IN */ -{ yymsp[-1].minor.yy140 = OP_TYPE_NOT_IN; } + case 647: /* in_op ::= NOT IN */ +{ yymsp[-1].minor.yy688 = OP_TYPE_NOT_IN; } break; - case 647: /* in_predicate_value ::= NK_LP literal_list NK_RP */ -{ yylhsminor.yy974 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0, createNodeListNode(pCxt, yymsp[-1].minor.yy946)); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + case 648: /* in_predicate_value ::= NK_LP literal_list NK_RP */ +{ yylhsminor.yy980 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0, createNodeListNode(pCxt, yymsp[-1].minor.yy628)); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 649: /* boolean_value_expression ::= NOT boolean_primary */ + case 650: /* boolean_value_expression ::= NOT boolean_primary */ { - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy974); - yylhsminor.yy974 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &e, createLogicConditionNode(pCxt, LOGIC_COND_TYPE_NOT, releaseRawExprNode(pCxt, yymsp[0].minor.yy974), NULL)); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy980); + yylhsminor.yy980 = createRawExprNodeExt(pCxt, &yymsp[-1].minor.yy0, &e, createLogicConditionNode(pCxt, LOGIC_COND_TYPE_NOT, releaseRawExprNode(pCxt, yymsp[0].minor.yy980), NULL)); } - yymsp[-1].minor.yy974 = yylhsminor.yy974; + yymsp[-1].minor.yy980 = yylhsminor.yy980; break; - case 650: /* boolean_value_expression ::= boolean_value_expression OR boolean_value_expression */ + case 651: /* boolean_value_expression ::= boolean_value_expression OR boolean_value_expression */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy974); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy974); - yylhsminor.yy974 = createRawExprNodeExt(pCxt, &s, &e, createLogicConditionNode(pCxt, LOGIC_COND_TYPE_OR, releaseRawExprNode(pCxt, yymsp[-2].minor.yy974), releaseRawExprNode(pCxt, yymsp[0].minor.yy974))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy980); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy980); + yylhsminor.yy980 = createRawExprNodeExt(pCxt, &s, &e, createLogicConditionNode(pCxt, LOGIC_COND_TYPE_OR, releaseRawExprNode(pCxt, yymsp[-2].minor.yy980), releaseRawExprNode(pCxt, yymsp[0].minor.yy980))); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 651: /* boolean_value_expression ::= boolean_value_expression AND boolean_value_expression */ + case 652: /* boolean_value_expression ::= boolean_value_expression AND boolean_value_expression */ { - SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy974); - SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy974); - yylhsminor.yy974 = createRawExprNodeExt(pCxt, &s, &e, createLogicConditionNode(pCxt, LOGIC_COND_TYPE_AND, releaseRawExprNode(pCxt, yymsp[-2].minor.yy974), releaseRawExprNode(pCxt, yymsp[0].minor.yy974))); + SToken s = getTokenFromRawExprNode(pCxt, yymsp[-2].minor.yy980); + SToken e = getTokenFromRawExprNode(pCxt, yymsp[0].minor.yy980); + yylhsminor.yy980 = createRawExprNodeExt(pCxt, &s, &e, createLogicConditionNode(pCxt, LOGIC_COND_TYPE_AND, releaseRawExprNode(pCxt, yymsp[-2].minor.yy980), releaseRawExprNode(pCxt, yymsp[0].minor.yy980))); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 659: /* table_reference_list ::= table_reference_list NK_COMMA table_reference */ -{ yylhsminor.yy974 = createJoinTableNode(pCxt, JOIN_TYPE_INNER, JOIN_STYPE_NONE, yymsp[-2].minor.yy974, yymsp[0].minor.yy974, NULL); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + case 660: /* table_reference_list ::= table_reference_list NK_COMMA table_reference */ +{ yylhsminor.yy980 = createJoinTableNode(pCxt, JOIN_TYPE_INNER, JOIN_STYPE_NONE, yymsp[-2].minor.yy980, yymsp[0].minor.yy980, NULL); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 662: /* table_primary ::= table_name alias_opt */ -{ yylhsminor.yy974 = createRealTableNode(pCxt, NULL, &yymsp[-1].minor.yy557, &yymsp[0].minor.yy557); } - yymsp[-1].minor.yy974 = yylhsminor.yy974; + case 663: /* table_primary ::= table_name alias_opt */ +{ yylhsminor.yy980 = createRealTableNode(pCxt, NULL, &yymsp[-1].minor.yy561, &yymsp[0].minor.yy561); } + yymsp[-1].minor.yy980 = yylhsminor.yy980; break; - case 663: /* table_primary ::= db_name NK_DOT table_name alias_opt */ -{ yylhsminor.yy974 = createRealTableNode(pCxt, &yymsp[-3].minor.yy557, &yymsp[-1].minor.yy557, &yymsp[0].minor.yy557); } - yymsp[-3].minor.yy974 = yylhsminor.yy974; + case 664: /* table_primary ::= db_name NK_DOT table_name alias_opt */ +{ yylhsminor.yy980 = createRealTableNode(pCxt, &yymsp[-3].minor.yy561, &yymsp[-1].minor.yy561, &yymsp[0].minor.yy561); } + yymsp[-3].minor.yy980 = yylhsminor.yy980; break; - case 664: /* table_primary ::= subquery alias_opt */ -{ yylhsminor.yy974 = createTempTableNode(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy974), &yymsp[0].minor.yy557); } - yymsp[-1].minor.yy974 = yylhsminor.yy974; + case 665: /* table_primary ::= subquery alias_opt */ +{ yylhsminor.yy980 = createTempTableNode(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy980), &yymsp[0].minor.yy561); } + yymsp[-1].minor.yy980 = yylhsminor.yy980; break; - case 666: /* alias_opt ::= */ -{ yymsp[1].minor.yy557 = nil_token; } + case 667: /* alias_opt ::= */ +{ yymsp[1].minor.yy561 = nil_token; } break; - case 668: /* alias_opt ::= AS table_alias */ -{ yymsp[-1].minor.yy557 = yymsp[0].minor.yy557; } + case 669: /* alias_opt ::= AS table_alias */ +{ yymsp[-1].minor.yy561 = yymsp[0].minor.yy561; } break; - case 669: /* parenthesized_joined_table ::= NK_LP joined_table NK_RP */ - case 670: /* parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP */ yytestcase(yyruleno==670); -{ yymsp[-2].minor.yy974 = yymsp[-1].minor.yy974; } + case 670: /* parenthesized_joined_table ::= NK_LP joined_table NK_RP */ + case 671: /* parenthesized_joined_table ::= NK_LP parenthesized_joined_table NK_RP */ yytestcase(yyruleno==671); +{ yymsp[-2].minor.yy980 = yymsp[-1].minor.yy980; } break; - case 671: /* joined_table ::= table_reference join_type join_subtype JOIN table_reference join_on_clause_opt window_offset_clause_opt jlimit_clause_opt */ + case 672: /* joined_table ::= table_reference join_type join_subtype JOIN table_reference join_on_clause_opt window_offset_clause_opt jlimit_clause_opt */ { - yylhsminor.yy974 = createJoinTableNode(pCxt, yymsp[-6].minor.yy792, yymsp[-5].minor.yy744, yymsp[-7].minor.yy974, yymsp[-3].minor.yy974, yymsp[-2].minor.yy974); - yylhsminor.yy974 = addWindowOffsetClause(pCxt, yylhsminor.yy974, yymsp[-1].minor.yy974); - yylhsminor.yy974 = addJLimitClause(pCxt, yylhsminor.yy974, yymsp[0].minor.yy974); + yylhsminor.yy980 = createJoinTableNode(pCxt, yymsp[-6].minor.yy652, yymsp[-5].minor.yy606, yymsp[-7].minor.yy980, yymsp[-3].minor.yy980, yymsp[-2].minor.yy980); + yylhsminor.yy980 = addWindowOffsetClause(pCxt, yylhsminor.yy980, yymsp[-1].minor.yy980); + yylhsminor.yy980 = addJLimitClause(pCxt, yylhsminor.yy980, yymsp[0].minor.yy980); } - yymsp[-7].minor.yy974 = yylhsminor.yy974; + yymsp[-7].minor.yy980 = yylhsminor.yy980; break; - case 672: /* join_type ::= */ -{ yymsp[1].minor.yy792 = JOIN_TYPE_INNER; } + case 673: /* join_type ::= */ +{ yymsp[1].minor.yy652 = JOIN_TYPE_INNER; } break; - case 673: /* join_type ::= INNER */ -{ yymsp[0].minor.yy792 = JOIN_TYPE_INNER; } + case 674: /* join_type ::= INNER */ +{ yymsp[0].minor.yy652 = JOIN_TYPE_INNER; } break; - case 674: /* join_type ::= LEFT */ -{ yymsp[0].minor.yy792 = JOIN_TYPE_LEFT; } + case 675: /* join_type ::= LEFT */ +{ yymsp[0].minor.yy652 = JOIN_TYPE_LEFT; } break; - case 675: /* join_type ::= RIGHT */ -{ yymsp[0].minor.yy792 = JOIN_TYPE_RIGHT; } + case 676: /* join_type ::= RIGHT */ +{ yymsp[0].minor.yy652 = JOIN_TYPE_RIGHT; } break; - case 676: /* join_type ::= FULL */ -{ yymsp[0].minor.yy792 = JOIN_TYPE_FULL; } + case 677: /* join_type ::= FULL */ +{ yymsp[0].minor.yy652 = JOIN_TYPE_FULL; } break; - case 677: /* join_subtype ::= */ -{ yymsp[1].minor.yy744 = JOIN_STYPE_NONE; } + case 678: /* join_subtype ::= */ +{ yymsp[1].minor.yy606 = JOIN_STYPE_NONE; } break; - case 678: /* join_subtype ::= OUTER */ -{ yymsp[0].minor.yy744 = JOIN_STYPE_OUTER; } + case 679: /* join_subtype ::= OUTER */ +{ yymsp[0].minor.yy606 = JOIN_STYPE_OUTER; } break; - case 679: /* join_subtype ::= SEMI */ -{ yymsp[0].minor.yy744 = JOIN_STYPE_SEMI; } + case 680: /* join_subtype ::= SEMI */ +{ yymsp[0].minor.yy606 = JOIN_STYPE_SEMI; } break; - case 680: /* join_subtype ::= ANTI */ -{ yymsp[0].minor.yy744 = JOIN_STYPE_ANTI; } + case 681: /* join_subtype ::= ANTI */ +{ yymsp[0].minor.yy606 = JOIN_STYPE_ANTI; } break; - case 681: /* join_subtype ::= ASOF */ -{ yymsp[0].minor.yy744 = JOIN_STYPE_ASOF; } + case 682: /* join_subtype ::= ASOF */ +{ yymsp[0].minor.yy606 = JOIN_STYPE_ASOF; } break; - case 682: /* join_subtype ::= WINDOW */ -{ yymsp[0].minor.yy744 = JOIN_STYPE_WIN; } + case 683: /* join_subtype ::= WINDOW */ +{ yymsp[0].minor.yy606 = JOIN_STYPE_WIN; } break; - case 686: /* window_offset_clause_opt ::= WINDOW_OFFSET NK_LP window_offset_literal NK_COMMA window_offset_literal NK_RP */ -{ yymsp[-5].minor.yy974 = createWindowOffsetNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy974), releaseRawExprNode(pCxt, yymsp[-1].minor.yy974)); } + case 687: /* window_offset_clause_opt ::= WINDOW_OFFSET NK_LP window_offset_literal NK_COMMA window_offset_literal NK_RP */ +{ yymsp[-5].minor.yy980 = createWindowOffsetNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy980), releaseRawExprNode(pCxt, yymsp[-1].minor.yy980)); } break; - case 687: /* window_offset_literal ::= NK_VARIABLE */ -{ yylhsminor.yy974 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createTimeOffsetValueNode(pCxt, &yymsp[0].minor.yy0)); } - yymsp[0].minor.yy974 = yylhsminor.yy974; + case 688: /* window_offset_literal ::= NK_VARIABLE */ +{ yylhsminor.yy980 = createRawExprNode(pCxt, &yymsp[0].minor.yy0, createTimeOffsetValueNode(pCxt, &yymsp[0].minor.yy0)); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; - case 688: /* window_offset_literal ::= NK_MINUS NK_VARIABLE */ + case 689: /* window_offset_literal ::= NK_MINUS NK_VARIABLE */ { SToken t = yymsp[-1].minor.yy0; t.n = (yymsp[0].minor.yy0.z + yymsp[0].minor.yy0.n) - yymsp[-1].minor.yy0.z; - yylhsminor.yy974 = createRawExprNode(pCxt, &t, createTimeOffsetValueNode(pCxt, &t)); + yylhsminor.yy980 = createRawExprNode(pCxt, &t, createTimeOffsetValueNode(pCxt, &t)); } - yymsp[-1].minor.yy974 = yylhsminor.yy974; + yymsp[-1].minor.yy980 = yylhsminor.yy980; break; - case 690: /* jlimit_clause_opt ::= JLIMIT NK_INTEGER */ - case 763: /* slimit_clause_opt ::= SLIMIT NK_INTEGER */ yytestcase(yyruleno==763); - case 767: /* limit_clause_opt ::= LIMIT NK_INTEGER */ yytestcase(yyruleno==767); -{ yymsp[-1].minor.yy974 = createLimitNode(pCxt, &yymsp[0].minor.yy0, NULL); } + case 691: /* jlimit_clause_opt ::= JLIMIT NK_INTEGER */ + case 764: /* slimit_clause_opt ::= SLIMIT NK_INTEGER */ yytestcase(yyruleno==764); + case 768: /* limit_clause_opt ::= LIMIT NK_INTEGER */ yytestcase(yyruleno==768); +{ yymsp[-1].minor.yy980 = createLimitNode(pCxt, &yymsp[0].minor.yy0, NULL); } break; - case 691: /* query_specification ::= SELECT hint_list set_quantifier_opt tag_mode_opt select_list from_clause_opt where_clause_opt partition_by_clause_opt range_opt every_opt fill_opt twindow_clause_opt group_by_clause_opt having_clause_opt */ + case 692: /* query_specification ::= SELECT hint_list set_quantifier_opt tag_mode_opt select_list from_clause_opt where_clause_opt partition_by_clause_opt range_opt every_opt fill_opt twindow_clause_opt group_by_clause_opt having_clause_opt */ { - yymsp[-13].minor.yy974 = createSelectStmt(pCxt, yymsp[-11].minor.yy569, yymsp[-9].minor.yy946, yymsp[-8].minor.yy974, yymsp[-12].minor.yy946); - yymsp[-13].minor.yy974 = setSelectStmtTagMode(pCxt, yymsp[-13].minor.yy974, yymsp[-10].minor.yy569); - yymsp[-13].minor.yy974 = addWhereClause(pCxt, yymsp[-13].minor.yy974, yymsp[-7].minor.yy974); - yymsp[-13].minor.yy974 = addPartitionByClause(pCxt, yymsp[-13].minor.yy974, yymsp[-6].minor.yy946); - yymsp[-13].minor.yy974 = addWindowClauseClause(pCxt, yymsp[-13].minor.yy974, yymsp[-2].minor.yy974); - yymsp[-13].minor.yy974 = addGroupByClause(pCxt, yymsp[-13].minor.yy974, yymsp[-1].minor.yy946); - yymsp[-13].minor.yy974 = addHavingClause(pCxt, yymsp[-13].minor.yy974, yymsp[0].minor.yy974); - yymsp[-13].minor.yy974 = addRangeClause(pCxt, yymsp[-13].minor.yy974, yymsp[-5].minor.yy974); - yymsp[-13].minor.yy974 = addEveryClause(pCxt, yymsp[-13].minor.yy974, yymsp[-4].minor.yy974); - yymsp[-13].minor.yy974 = addFillClause(pCxt, yymsp[-13].minor.yy974, yymsp[-3].minor.yy974); + yymsp[-13].minor.yy980 = createSelectStmt(pCxt, yymsp[-11].minor.yy957, yymsp[-9].minor.yy628, yymsp[-8].minor.yy980, yymsp[-12].minor.yy628); + yymsp[-13].minor.yy980 = setSelectStmtTagMode(pCxt, yymsp[-13].minor.yy980, yymsp[-10].minor.yy957); + yymsp[-13].minor.yy980 = addWhereClause(pCxt, yymsp[-13].minor.yy980, yymsp[-7].minor.yy980); + yymsp[-13].minor.yy980 = addPartitionByClause(pCxt, yymsp[-13].minor.yy980, yymsp[-6].minor.yy628); + yymsp[-13].minor.yy980 = addWindowClauseClause(pCxt, yymsp[-13].minor.yy980, yymsp[-2].minor.yy980); + yymsp[-13].minor.yy980 = addGroupByClause(pCxt, yymsp[-13].minor.yy980, yymsp[-1].minor.yy628); + yymsp[-13].minor.yy980 = addHavingClause(pCxt, yymsp[-13].minor.yy980, yymsp[0].minor.yy980); + yymsp[-13].minor.yy980 = addRangeClause(pCxt, yymsp[-13].minor.yy980, yymsp[-5].minor.yy980); + yymsp[-13].minor.yy980 = addEveryClause(pCxt, yymsp[-13].minor.yy980, yymsp[-4].minor.yy980); + yymsp[-13].minor.yy980 = addFillClause(pCxt, yymsp[-13].minor.yy980, yymsp[-3].minor.yy980); } break; - case 692: /* hint_list ::= */ -{ yymsp[1].minor.yy946 = createHintNodeList(pCxt, NULL); } + case 693: /* hint_list ::= */ +{ yymsp[1].minor.yy628 = createHintNodeList(pCxt, NULL); } break; - case 693: /* hint_list ::= NK_HINT */ -{ yylhsminor.yy946 = createHintNodeList(pCxt, &yymsp[0].minor.yy0); } - yymsp[0].minor.yy946 = yylhsminor.yy946; + case 694: /* hint_list ::= NK_HINT */ +{ yylhsminor.yy628 = createHintNodeList(pCxt, &yymsp[0].minor.yy0); } + yymsp[0].minor.yy628 = yylhsminor.yy628; break; - case 698: /* set_quantifier_opt ::= ALL */ -{ yymsp[0].minor.yy569 = false; } + case 699: /* set_quantifier_opt ::= ALL */ +{ yymsp[0].minor.yy957 = false; } break; - case 701: /* select_item ::= NK_STAR */ -{ yylhsminor.yy974 = createColumnNode(pCxt, NULL, &yymsp[0].minor.yy0); } - yymsp[0].minor.yy974 = yylhsminor.yy974; + case 702: /* select_item ::= NK_STAR */ +{ yylhsminor.yy980 = createColumnNode(pCxt, NULL, &yymsp[0].minor.yy0); } + yymsp[0].minor.yy980 = yylhsminor.yy980; break; - case 703: /* select_item ::= common_expression column_alias */ - case 713: /* partition_item ::= expr_or_subquery column_alias */ yytestcase(yyruleno==713); -{ yylhsminor.yy974 = setProjectionAlias(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy974), &yymsp[0].minor.yy557); } - yymsp[-1].minor.yy974 = yylhsminor.yy974; + case 704: /* select_item ::= common_expression column_alias */ + case 714: /* partition_item ::= expr_or_subquery column_alias */ yytestcase(yyruleno==714); +{ yylhsminor.yy980 = setProjectionAlias(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy980), &yymsp[0].minor.yy561); } + yymsp[-1].minor.yy980 = yylhsminor.yy980; break; - case 704: /* select_item ::= common_expression AS column_alias */ - case 714: /* partition_item ::= expr_or_subquery AS column_alias */ yytestcase(yyruleno==714); -{ yylhsminor.yy974 = setProjectionAlias(pCxt, releaseRawExprNode(pCxt, yymsp[-2].minor.yy974), &yymsp[0].minor.yy557); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + case 705: /* select_item ::= common_expression AS column_alias */ + case 715: /* partition_item ::= expr_or_subquery AS column_alias */ yytestcase(yyruleno==715); +{ yylhsminor.yy980 = setProjectionAlias(pCxt, releaseRawExprNode(pCxt, yymsp[-2].minor.yy980), &yymsp[0].minor.yy561); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 709: /* partition_by_clause_opt ::= PARTITION BY partition_list */ - case 741: /* group_by_clause_opt ::= GROUP BY group_by_list */ yytestcase(yyruleno==741); - case 761: /* order_by_clause_opt ::= ORDER BY sort_specification_list */ yytestcase(yyruleno==761); -{ yymsp[-2].minor.yy946 = yymsp[0].minor.yy946; } + case 710: /* partition_by_clause_opt ::= PARTITION BY partition_list */ + case 742: /* group_by_clause_opt ::= GROUP BY group_by_list */ yytestcase(yyruleno==742); + case 762: /* order_by_clause_opt ::= ORDER BY sort_specification_list */ yytestcase(yyruleno==762); +{ yymsp[-2].minor.yy628 = yymsp[0].minor.yy628; } break; - case 716: /* twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA interval_sliding_duration_literal NK_RP */ -{ yymsp[-5].minor.yy974 = createSessionWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy974), releaseRawExprNode(pCxt, yymsp[-1].minor.yy974)); } + case 717: /* twindow_clause_opt ::= SESSION NK_LP column_reference NK_COMMA interval_sliding_duration_literal NK_RP */ +{ yymsp[-5].minor.yy980 = createSessionWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy980), releaseRawExprNode(pCxt, yymsp[-1].minor.yy980)); } break; - case 717: /* twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP */ -{ yymsp[-3].minor.yy974 = createStateWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy974)); } + case 718: /* twindow_clause_opt ::= STATE_WINDOW NK_LP expr_or_subquery NK_RP */ +{ yymsp[-3].minor.yy980 = createStateWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy980)); } break; - case 718: /* twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_RP sliding_opt fill_opt */ -{ yymsp[-5].minor.yy974 = createIntervalWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy974), NULL, yymsp[-1].minor.yy974, yymsp[0].minor.yy974); } + case 719: /* twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_RP sliding_opt fill_opt */ +{ yymsp[-5].minor.yy980 = createIntervalWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy980), NULL, yymsp[-1].minor.yy980, yymsp[0].minor.yy980); } break; - case 719: /* twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_COMMA interval_sliding_duration_literal NK_RP sliding_opt fill_opt */ -{ yymsp[-7].minor.yy974 = createIntervalWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-5].minor.yy974), releaseRawExprNode(pCxt, yymsp[-3].minor.yy974), yymsp[-1].minor.yy974, yymsp[0].minor.yy974); } + case 720: /* twindow_clause_opt ::= INTERVAL NK_LP interval_sliding_duration_literal NK_COMMA interval_sliding_duration_literal NK_RP sliding_opt fill_opt */ +{ yymsp[-7].minor.yy980 = createIntervalWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-5].minor.yy980), releaseRawExprNode(pCxt, yymsp[-3].minor.yy980), yymsp[-1].minor.yy980, yymsp[0].minor.yy980); } break; - case 720: /* twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition */ -{ yymsp[-6].minor.yy974 = createEventWindowNode(pCxt, yymsp[-3].minor.yy974, yymsp[0].minor.yy974); } + case 721: /* twindow_clause_opt ::= EVENT_WINDOW START WITH search_condition END WITH search_condition */ +{ yymsp[-6].minor.yy980 = createEventWindowNode(pCxt, yymsp[-3].minor.yy980, yymsp[0].minor.yy980); } break; - case 721: /* twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_RP */ -{ yymsp[-3].minor.yy974 = createCountWindowNode(pCxt, &yymsp[-1].minor.yy0, &yymsp[-1].minor.yy0); } + case 722: /* twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_RP */ +{ yymsp[-3].minor.yy980 = createCountWindowNode(pCxt, &yymsp[-1].minor.yy0, &yymsp[-1].minor.yy0); } break; - case 722: /* twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP */ -{ yymsp[-5].minor.yy974 = createCountWindowNode(pCxt, &yymsp[-3].minor.yy0, &yymsp[-1].minor.yy0); } + case 723: /* twindow_clause_opt ::= COUNT_WINDOW NK_LP NK_INTEGER NK_COMMA NK_INTEGER NK_RP */ +{ yymsp[-5].minor.yy980 = createCountWindowNode(pCxt, &yymsp[-3].minor.yy0, &yymsp[-1].minor.yy0); } break; - case 723: /* twindow_clause_opt ::= ANOMALY_WINDOW NK_LP expr_or_subquery NK_RP */ -{ yymsp[-3].minor.yy974 = createAnomalyWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy974), NULL); } + case 724: /* twindow_clause_opt ::= ANOMALY_WINDOW NK_LP expr_or_subquery NK_RP */ +{ yymsp[-3].minor.yy980 = createAnomalyWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy980), NULL); } break; - case 724: /* twindow_clause_opt ::= ANOMALY_WINDOW NK_LP expr_or_subquery NK_COMMA NK_STRING NK_RP */ -{ yymsp[-5].minor.yy974 = createAnomalyWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy974), &yymsp[-1].minor.yy0); } + case 725: /* twindow_clause_opt ::= ANOMALY_WINDOW NK_LP expr_or_subquery NK_COMMA NK_STRING NK_RP */ +{ yymsp[-5].minor.yy980 = createAnomalyWindowNode(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy980), &yymsp[-1].minor.yy0); } break; - case 731: /* fill_opt ::= FILL NK_LP fill_mode NK_RP */ -{ yymsp[-3].minor.yy974 = createFillNode(pCxt, yymsp[-1].minor.yy102, NULL); } + case 732: /* fill_opt ::= FILL NK_LP fill_mode NK_RP */ +{ yymsp[-3].minor.yy980 = createFillNode(pCxt, yymsp[-1].minor.yy690, NULL); } break; - case 732: /* fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP */ -{ yymsp[-5].minor.yy974 = createFillNode(pCxt, FILL_MODE_VALUE, createNodeListNode(pCxt, yymsp[-1].minor.yy946)); } + case 733: /* fill_opt ::= FILL NK_LP VALUE NK_COMMA expression_list NK_RP */ +{ yymsp[-5].minor.yy980 = createFillNode(pCxt, FILL_MODE_VALUE, createNodeListNode(pCxt, yymsp[-1].minor.yy628)); } break; - case 733: /* fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP */ -{ yymsp[-5].minor.yy974 = createFillNode(pCxt, FILL_MODE_VALUE_F, createNodeListNode(pCxt, yymsp[-1].minor.yy946)); } + case 734: /* fill_opt ::= FILL NK_LP VALUE_F NK_COMMA expression_list NK_RP */ +{ yymsp[-5].minor.yy980 = createFillNode(pCxt, FILL_MODE_VALUE_F, createNodeListNode(pCxt, yymsp[-1].minor.yy628)); } break; - case 734: /* fill_mode ::= NONE */ -{ yymsp[0].minor.yy102 = FILL_MODE_NONE; } + case 735: /* fill_mode ::= NONE */ +{ yymsp[0].minor.yy690 = FILL_MODE_NONE; } break; - case 735: /* fill_mode ::= PREV */ -{ yymsp[0].minor.yy102 = FILL_MODE_PREV; } + case 736: /* fill_mode ::= PREV */ +{ yymsp[0].minor.yy690 = FILL_MODE_PREV; } break; - case 736: /* fill_mode ::= NULL */ -{ yymsp[0].minor.yy102 = FILL_MODE_NULL; } + case 737: /* fill_mode ::= NULL */ +{ yymsp[0].minor.yy690 = FILL_MODE_NULL; } break; - case 737: /* fill_mode ::= NULL_F */ -{ yymsp[0].minor.yy102 = FILL_MODE_NULL_F; } + case 738: /* fill_mode ::= NULL_F */ +{ yymsp[0].minor.yy690 = FILL_MODE_NULL_F; } break; - case 738: /* fill_mode ::= LINEAR */ -{ yymsp[0].minor.yy102 = FILL_MODE_LINEAR; } + case 739: /* fill_mode ::= LINEAR */ +{ yymsp[0].minor.yy690 = FILL_MODE_LINEAR; } break; - case 739: /* fill_mode ::= NEXT */ -{ yymsp[0].minor.yy102 = FILL_MODE_NEXT; } + case 740: /* fill_mode ::= NEXT */ +{ yymsp[0].minor.yy690 = FILL_MODE_NEXT; } break; - case 742: /* group_by_list ::= expr_or_subquery */ -{ yylhsminor.yy946 = createNodeList(pCxt, createGroupingSetNode(pCxt, releaseRawExprNode(pCxt, yymsp[0].minor.yy974))); } - yymsp[0].minor.yy946 = yylhsminor.yy946; + case 743: /* group_by_list ::= expr_or_subquery */ +{ yylhsminor.yy628 = createNodeList(pCxt, createGroupingSetNode(pCxt, releaseRawExprNode(pCxt, yymsp[0].minor.yy980))); } + yymsp[0].minor.yy628 = yylhsminor.yy628; break; - case 743: /* group_by_list ::= group_by_list NK_COMMA expr_or_subquery */ -{ yylhsminor.yy946 = addNodeToList(pCxt, yymsp[-2].minor.yy946, createGroupingSetNode(pCxt, releaseRawExprNode(pCxt, yymsp[0].minor.yy974))); } - yymsp[-2].minor.yy946 = yylhsminor.yy946; + case 744: /* group_by_list ::= group_by_list NK_COMMA expr_or_subquery */ +{ yylhsminor.yy628 = addNodeToList(pCxt, yymsp[-2].minor.yy628, createGroupingSetNode(pCxt, releaseRawExprNode(pCxt, yymsp[0].minor.yy980))); } + yymsp[-2].minor.yy628 = yylhsminor.yy628; break; - case 747: /* range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP */ -{ yymsp[-5].minor.yy974 = createInterpTimeRange(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy974), releaseRawExprNode(pCxt, yymsp[-1].minor.yy974)); } + case 748: /* range_opt ::= RANGE NK_LP expr_or_subquery NK_COMMA expr_or_subquery NK_RP */ +{ yymsp[-5].minor.yy980 = createInterpTimeRange(pCxt, releaseRawExprNode(pCxt, yymsp[-3].minor.yy980), releaseRawExprNode(pCxt, yymsp[-1].minor.yy980)); } break; - case 748: /* range_opt ::= RANGE NK_LP expr_or_subquery NK_RP */ -{ yymsp[-3].minor.yy974 = createInterpTimePoint(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy974)); } + case 749: /* range_opt ::= RANGE NK_LP expr_or_subquery NK_RP */ +{ yymsp[-3].minor.yy980 = createInterpTimePoint(pCxt, releaseRawExprNode(pCxt, yymsp[-1].minor.yy980)); } break; - case 751: /* query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt */ + case 752: /* query_expression ::= query_simple order_by_clause_opt slimit_clause_opt limit_clause_opt */ { - yylhsminor.yy974 = addOrderByClause(pCxt, yymsp[-3].minor.yy974, yymsp[-2].minor.yy946); - yylhsminor.yy974 = addSlimitClause(pCxt, yylhsminor.yy974, yymsp[-1].minor.yy974); - yylhsminor.yy974 = addLimitClause(pCxt, yylhsminor.yy974, yymsp[0].minor.yy974); + yylhsminor.yy980 = addOrderByClause(pCxt, yymsp[-3].minor.yy980, yymsp[-2].minor.yy628); + yylhsminor.yy980 = addSlimitClause(pCxt, yylhsminor.yy980, yymsp[-1].minor.yy980); + yylhsminor.yy980 = addLimitClause(pCxt, yylhsminor.yy980, yymsp[0].minor.yy980); } - yymsp[-3].minor.yy974 = yylhsminor.yy974; + yymsp[-3].minor.yy980 = yylhsminor.yy980; break; - case 754: /* union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery */ -{ yylhsminor.yy974 = createSetOperator(pCxt, SET_OP_TYPE_UNION_ALL, yymsp[-3].minor.yy974, yymsp[0].minor.yy974); } - yymsp[-3].minor.yy974 = yylhsminor.yy974; + case 755: /* union_query_expression ::= query_simple_or_subquery UNION ALL query_simple_or_subquery */ +{ yylhsminor.yy980 = createSetOperator(pCxt, SET_OP_TYPE_UNION_ALL, yymsp[-3].minor.yy980, yymsp[0].minor.yy980); } + yymsp[-3].minor.yy980 = yylhsminor.yy980; break; - case 755: /* union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery */ -{ yylhsminor.yy974 = createSetOperator(pCxt, SET_OP_TYPE_UNION, yymsp[-2].minor.yy974, yymsp[0].minor.yy974); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + case 756: /* union_query_expression ::= query_simple_or_subquery UNION query_simple_or_subquery */ +{ yylhsminor.yy980 = createSetOperator(pCxt, SET_OP_TYPE_UNION, yymsp[-2].minor.yy980, yymsp[0].minor.yy980); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 764: /* slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER */ - case 768: /* limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER */ yytestcase(yyruleno==768); -{ yymsp[-3].minor.yy974 = createLimitNode(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0); } + case 765: /* slimit_clause_opt ::= SLIMIT NK_INTEGER SOFFSET NK_INTEGER */ + case 769: /* limit_clause_opt ::= LIMIT NK_INTEGER OFFSET NK_INTEGER */ yytestcase(yyruleno==769); +{ yymsp[-3].minor.yy980 = createLimitNode(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0); } break; - case 765: /* slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER */ - case 769: /* limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER */ yytestcase(yyruleno==769); -{ yymsp[-3].minor.yy974 = createLimitNode(pCxt, &yymsp[0].minor.yy0, &yymsp[-2].minor.yy0); } + case 766: /* slimit_clause_opt ::= SLIMIT NK_INTEGER NK_COMMA NK_INTEGER */ + case 770: /* limit_clause_opt ::= LIMIT NK_INTEGER NK_COMMA NK_INTEGER */ yytestcase(yyruleno==770); +{ yymsp[-3].minor.yy980 = createLimitNode(pCxt, &yymsp[0].minor.yy0, &yymsp[-2].minor.yy0); } break; - case 770: /* subquery ::= NK_LP query_expression NK_RP */ -{ yylhsminor.yy974 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0, yymsp[-1].minor.yy974); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + case 771: /* subquery ::= NK_LP query_expression NK_RP */ +{ yylhsminor.yy980 = createRawExprNodeExt(pCxt, &yymsp[-2].minor.yy0, &yymsp[0].minor.yy0, yymsp[-1].minor.yy980); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 775: /* sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt */ -{ yylhsminor.yy974 = createOrderByExprNode(pCxt, releaseRawExprNode(pCxt, yymsp[-2].minor.yy974), yymsp[-1].minor.yy410, yymsp[0].minor.yy307); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + case 776: /* sort_specification ::= expr_or_subquery ordering_specification_opt null_ordering_opt */ +{ yylhsminor.yy980 = createOrderByExprNode(pCxt, releaseRawExprNode(pCxt, yymsp[-2].minor.yy980), yymsp[-1].minor.yy274, yymsp[0].minor.yy305); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; - case 776: /* ordering_specification_opt ::= */ -{ yymsp[1].minor.yy410 = ORDER_ASC; } + case 777: /* ordering_specification_opt ::= */ +{ yymsp[1].minor.yy274 = ORDER_ASC; } break; - case 777: /* ordering_specification_opt ::= ASC */ -{ yymsp[0].minor.yy410 = ORDER_ASC; } + case 778: /* ordering_specification_opt ::= ASC */ +{ yymsp[0].minor.yy274 = ORDER_ASC; } break; - case 778: /* ordering_specification_opt ::= DESC */ -{ yymsp[0].minor.yy410 = ORDER_DESC; } + case 779: /* ordering_specification_opt ::= DESC */ +{ yymsp[0].minor.yy274 = ORDER_DESC; } break; - case 779: /* null_ordering_opt ::= */ -{ yymsp[1].minor.yy307 = NULL_ORDER_DEFAULT; } + case 780: /* null_ordering_opt ::= */ +{ yymsp[1].minor.yy305 = NULL_ORDER_DEFAULT; } break; - case 780: /* null_ordering_opt ::= NULLS FIRST */ -{ yymsp[-1].minor.yy307 = NULL_ORDER_FIRST; } + case 781: /* null_ordering_opt ::= NULLS FIRST */ +{ yymsp[-1].minor.yy305 = NULL_ORDER_FIRST; } break; - case 781: /* null_ordering_opt ::= NULLS LAST */ -{ yymsp[-1].minor.yy307 = NULL_ORDER_LAST; } + case 782: /* null_ordering_opt ::= NULLS LAST */ +{ yymsp[-1].minor.yy305 = NULL_ORDER_LAST; } break; - case 784: /* column_options ::= column_options NK_ID NK_STRING */ -{ yylhsminor.yy974 = setColumnOptions(pCxt, yymsp[-2].minor.yy974, &yymsp[-1].minor.yy0, &yymsp[0].minor.yy0); } - yymsp[-2].minor.yy974 = yylhsminor.yy974; + case 785: /* column_options ::= column_options NK_ID NK_STRING */ +{ yylhsminor.yy980 = setColumnOptions(pCxt, yymsp[-2].minor.yy980, &yymsp[-1].minor.yy0, &yymsp[0].minor.yy0); } + yymsp[-2].minor.yy980 = yylhsminor.yy980; break; default: break; @@ -7958,12 +8353,56 @@ void Parse( } #endif - do{ + while(1){ /* Exit by "break" */ + assert( yypParser->yytos>=yypParser->yystack ); assert( yyact==yypParser->yytos->stateno ); yyact = yy_find_shift_action((YYCODETYPE)yymajor,yyact); if( yyact >= YY_MIN_REDUCE ){ - yyact = yy_reduce(yypParser,yyact-YY_MIN_REDUCE,yymajor, - yyminor ParseCTX_PARAM); + unsigned int yyruleno = yyact - YY_MIN_REDUCE; /* Reduce by this rule */ +#ifndef NDEBUG + assert( yyruleno<(int)(sizeof(yyRuleName)/sizeof(yyRuleName[0])) ); + if( yyTraceFILE ){ + int yysize = yyRuleInfoNRhs[yyruleno]; + if( yysize ){ + fprintf(yyTraceFILE, "%sReduce %d [%s]%s, pop back to state %d.\n", + yyTracePrompt, + yyruleno, yyRuleName[yyruleno], + yyrulenoyytos[yysize].stateno); + }else{ + fprintf(yyTraceFILE, "%sReduce %d [%s]%s.\n", + yyTracePrompt, yyruleno, yyRuleName[yyruleno], + yyrulenoyytos - yypParser->yystack)>yypParser->yyhwm ){ + yypParser->yyhwm++; + assert( yypParser->yyhwm == + (int)(yypParser->yytos - yypParser->yystack)); + } +#endif +#if YYSTACKDEPTH>0 + if( yypParser->yytos>=yypParser->yystackEnd ){ + yyStackOverflow(yypParser); + break; + } +#else + if( yypParser->yytos>=&yypParser->yystack[yypParser->yystksz-1] ){ + if( yyGrowStack(yypParser) ){ + yyStackOverflow(yypParser); + break; + } + } +#endif + } + yyact = yy_reduce(yypParser,yyruleno,yymajor,yyminor ParseCTX_PARAM); }else if( yyact <= YY_MAX_SHIFTREDUCE ){ yy_shift(yypParser,yyact,(YYCODETYPE)yymajor,yyminor); #ifndef YYNOERRORRECOVERY @@ -8019,14 +8458,13 @@ void Parse( yy_destructor(yypParser, (YYCODETYPE)yymajor, &yyminorunion); yymajor = YYNOCODE; }else{ - while( yypParser->yytos >= yypParser->yystack - && (yyact = yy_find_reduce_action( - yypParser->yytos->stateno, - YYERRORSYMBOL)) > YY_MAX_SHIFTREDUCE - ){ + while( yypParser->yytos > yypParser->yystack ){ + yyact = yy_find_reduce_action(yypParser->yytos->stateno, + YYERRORSYMBOL); + if( yyact<=YY_MAX_SHIFTREDUCE ) break; yy_pop_parser_stack(yypParser); } - if( yypParser->yytos < yypParser->yystack || yymajor==0 ){ + if( yypParser->yytos <= yypParser->yystack || yymajor==0 ){ yy_destructor(yypParser,(YYCODETYPE)yymajor,&yyminorunion); yy_parse_failed(yypParser); #ifndef YYNOERRORRECOVERY @@ -8076,7 +8514,7 @@ void Parse( break; #endif } - }while( yypParser->yytos>yypParser->yystack ); + } #ifndef NDEBUG if( yyTraceFILE ){ yyStackEntry *i; diff --git a/source/libs/planner/src/planLogicCreater.c b/source/libs/planner/src/planLogicCreater.c index c579b66511..34c83acee8 100644 --- a/source/libs/planner/src/planLogicCreater.c +++ b/source/libs/planner/src/planLogicCreater.c @@ -923,6 +923,15 @@ static bool isInterpFunc(int32_t funcId) { return fmIsInterpFunc(funcId) || fmIsInterpPseudoColumnFunc(funcId) || fmIsGroupKeyFunc(funcId) || fmisSelectGroupConstValueFunc(funcId); } +static void initStreamOption(SLogicPlanContext* pCxt, SStreamNodeOption* pOption) { + pOption->triggerType = pCxt->pPlanCxt->triggerType; + pOption->watermark = pCxt->pPlanCxt->watermark; + pOption->deleteMark = pCxt->pPlanCxt->deleteMark; + pOption->igExpired = pCxt->pPlanCxt->igExpired; + pOption->igCheckUpdate = pCxt->pPlanCxt->igCheckUpdate; + pOption->destHasPrimaryKey = pCxt->pPlanCxt->destHasPrimaryKey; +} + static int32_t createInterpFuncLogicNode(SLogicPlanContext* pCxt, SSelectStmt* pSelect, SLogicNode** pLogicNode) { if (!pSelect->hasInterpFunc) { return TSDB_CODE_SUCCESS; @@ -957,6 +966,8 @@ static int32_t createInterpFuncLogicNode(SLogicPlanContext* pCxt, SSelectStmt* p if (TSDB_CODE_SUCCESS == code && NULL != pSelect->pEvery) { pInterpFunc->interval = ((SValueNode*)pSelect->pEvery)->datum.i; + pInterpFunc->intervalUnit = ((SValueNode*)pSelect->pEvery)->unit; + pInterpFunc->precision = pSelect->precision; } // set the output @@ -964,6 +975,10 @@ static int32_t createInterpFuncLogicNode(SLogicPlanContext* pCxt, SSelectStmt* p code = createColumnByRewriteExprs(pInterpFunc->pFuncs, &pInterpFunc->node.pTargets); } + if (TSDB_CODE_SUCCESS == code) { + initStreamOption(pCxt, &pInterpFunc->streamNodeOption); + } + if (TSDB_CODE_SUCCESS == code) { *pLogicNode = (SLogicNode*)pInterpFunc; } else { diff --git a/source/libs/planner/src/planPhysiCreater.c b/source/libs/planner/src/planPhysiCreater.c index d36edf494b..347aeba95e 100644 --- a/source/libs/planner/src/planPhysiCreater.c +++ b/source/libs/planner/src/planPhysiCreater.c @@ -1924,8 +1924,9 @@ static int32_t createIndefRowsFuncPhysiNode(SPhysiPlanContext* pCxt, SNodeList* static int32_t createInterpFuncPhysiNode(SPhysiPlanContext* pCxt, SNodeList* pChildren, SInterpFuncLogicNode* pFuncLogicNode, SPhysiNode** pPhyNode) { - SInterpFuncPhysiNode* pInterpFunc = - (SInterpFuncPhysiNode*)makePhysiNode(pCxt, (SLogicNode*)pFuncLogicNode, QUERY_NODE_PHYSICAL_PLAN_INTERP_FUNC); + SInterpFuncPhysiNode* pInterpFunc = (SInterpFuncPhysiNode*)makePhysiNode( + pCxt, (SLogicNode*)pFuncLogicNode, + pCxt->pPlanCxt->streamQuery ? QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERP_FUNC : QUERY_NODE_PHYSICAL_PLAN_INTERP_FUNC); if (NULL == pInterpFunc) { return terrno; } @@ -1954,6 +1955,8 @@ static int32_t createInterpFuncPhysiNode(SPhysiPlanContext* pCxt, SNodeList* pCh pInterpFunc->timeRange = pFuncLogicNode->timeRange; pInterpFunc->interval = pFuncLogicNode->interval; pInterpFunc->fillMode = pFuncLogicNode->fillMode; + pInterpFunc->intervalUnit = pFuncLogicNode->intervalUnit; + pInterpFunc->precision = pFuncLogicNode->node.precision; pInterpFunc->pFillValues = NULL; code = nodesCloneNode(pFuncLogicNode->pFillValues, &pInterpFunc->pFillValues); if (TSDB_CODE_SUCCESS != code) { @@ -1969,6 +1972,10 @@ static int32_t createInterpFuncPhysiNode(SPhysiPlanContext* pCxt, SNodeList* pCh code = setConditionsSlotId(pCxt, (const SLogicNode*)pFuncLogicNode, (SPhysiNode*)pInterpFunc); } + if (pCxt->pPlanCxt->streamQuery) { + pInterpFunc->streamNodeOption = pFuncLogicNode->streamNodeOption; + } + if (TSDB_CODE_SUCCESS == code) { *pPhyNode = (SPhysiNode*)pInterpFunc; } else { @@ -2149,7 +2156,7 @@ static int32_t createWindowPhysiNodeFinalize(SPhysiPlanContext* pCxt, SNodeList* pWindow->deleteMark = pWindowLogicNode->deleteMark; pWindow->igExpired = pWindowLogicNode->igExpired; if (pCxt->pPlanCxt->streamQuery) { - pWindow->destHasPrimayKey = pCxt->pPlanCxt->destHasPrimaryKey; + pWindow->destHasPrimaryKey = pCxt->pPlanCxt->destHasPrimaryKey; } pWindow->mergeDataBlock = (GROUP_ACTION_KEEP == pWindowLogicNode->node.groupAction ? false : true); pWindow->node.inputTsOrder = pWindowLogicNode->node.inputTsOrder; diff --git a/source/libs/planner/src/planValidator.c b/source/libs/planner/src/planValidator.c index 4fcd064e56..6b7b46cfa7 100755 --- a/source/libs/planner/src/planValidator.c +++ b/source/libs/planner/src/planValidator.c @@ -118,6 +118,7 @@ int32_t doValidatePhysiNode(SValidatePlanContext* pCxt, SNode* pNode) { case QUERY_NODE_PHYSICAL_PLAN_HASH_JOIN: case QUERY_NODE_PHYSICAL_PLAN_GROUP_CACHE: case QUERY_NODE_PHYSICAL_PLAN_DYN_QUERY_CTRL: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERP_FUNC: break; case QUERY_NODE_PHYSICAL_SUBPLAN: return validateSubplanNode(pCxt, (SSubplan*)pNode); diff --git a/source/libs/stream/inc/streamBackendRocksdb.h b/source/libs/stream/inc/streamBackendRocksdb.h index 1e0801fb6b..d313acc61d 100644 --- a/source/libs/stream/inc/streamBackendRocksdb.h +++ b/source/libs/stream/inc/streamBackendRocksdb.h @@ -166,12 +166,13 @@ int32_t streamStateDel_rocksdb(SStreamState* pState, const SWinKey* key); int32_t streamStateClear_rocksdb(SStreamState* pState); void streamStateCurNext_rocksdb(SStreamStateCur* pCur); int32_t streamStateGetFirst_rocksdb(SStreamState* pState, SWinKey* key); -int32_t streamStateGetGroupKVByCur_rocksdb(SStreamStateCur* pCur, SWinKey* pKey, const void** pVal, int32_t* pVLen); +int32_t streamStateGetGroupKVByCur_rocksdb(SStreamState* pState, SStreamStateCur* pCur, SWinKey* pKey, const void** pVal, int32_t* pVLen); int32_t streamStateAddIfNotExist_rocksdb(SStreamState* pState, const SWinKey* key, void** pVal, int32_t* pVLen); void streamStateCurPrev_rocksdb(SStreamStateCur* pCur); int32_t streamStateGetKVByCur_rocksdb(SStreamState* pState, SStreamStateCur* pCur, SWinKey* pKey, const void** pVal, int32_t* pVLen); SStreamStateCur* streamStateGetAndCheckCur_rocksdb(SStreamState* pState, SWinKey* key); SStreamStateCur* streamStateSeekKeyNext_rocksdb(SStreamState* pState, const SWinKey* key); +SStreamStateCur* streamStateSeekKeyPrev_rocksdb(SStreamState* pState, const SWinKey* key); SStreamStateCur* streamStateSeekToLast_rocksdb(SStreamState* pState); SStreamStateCur* streamStateGetCur_rocksdb(SStreamState* pState, const SWinKey* key); @@ -210,10 +211,14 @@ SStreamStateCur* streamStateFillGetCur_rocksdb(SStreamState* pState, const SWinK int32_t streamStateFillGetKVByCur_rocksdb(SStreamStateCur* pCur, SWinKey* pKey, const void** pVal, int32_t* pVLen); SStreamStateCur* streamStateFillSeekKeyPrev_rocksdb(SStreamState* pState, const SWinKey* key); SStreamStateCur* streamStateFillSeekKeyNext_rocksdb(SStreamState* pState, const SWinKey* key); +SStreamStateCur* streamStateFillSeekToLast_rocksdb(SStreamState* pState); +int32_t streamStateFillGetGroupKVByCur_rocksdb(SStreamStateCur* pCur, SWinKey* pKey, const void** pVal, int32_t* pVLen); // partag cf int32_t streamStatePutParTag_rocksdb(SStreamState* pState, int64_t groupId, const void* tag, int32_t tagLen); int32_t streamStateGetParTag_rocksdb(SStreamState* pState, int64_t groupId, void** tagVal, int32_t* tagLen); +void streamStateParTagSeekKeyNext_rocksdb(SStreamState* pState, const int64_t groupId, SStreamStateCur* pCur); +int32_t streamStateParTagGetKVByCur_rocksdb(SStreamStateCur* pCur, int64_t* pGroupId, const void** pVal, int32_t* pVLen); // parname cf int32_t streamStatePutParName_rocksdb(SStreamState* pState, int64_t groupId, const char tbname[TSDB_TABLE_NAME_LEN]); diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index 93c8dc3b40..863bc76c79 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -241,7 +241,9 @@ int32_t initCheckpointReadyMsg(SStreamTask* pTask, int32_t upstreamNodeId, int32 int64_t checkpointId, SRpcMsg* pMsg); int32_t flushStateDataInExecutor(SStreamTask* pTask, SStreamQueueItem* pCheckpointBlock); - +int32_t streamCreateSinkResTrigger(SStreamTrigger** pTrigger); +int32_t streamCreateForcewindowTrigger(SStreamTrigger** pTrigger, int32_t trigger, SInterval* pInterval, + STimeWindow* pLatestWindow, const char* id); #ifdef __cplusplus } diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index b69e191059..09f4e95376 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -3386,7 +3386,7 @@ int32_t streamStateGetFirst_rocksdb(SStreamState* pState, SWinKey* key) { return streamStateDel_rocksdb(pState, &tmp); } -int32_t streamStateGetGroupKVByCur_rocksdb(SStreamStateCur* pCur, SWinKey* pKey, const void** pVal, int32_t* pVLen) { +int32_t streamStateFillGetGroupKVByCur_rocksdb(SStreamStateCur* pCur, SWinKey* pKey, const void** pVal, int32_t* pVLen) { if (!pCur) { return -1; } @@ -3478,7 +3478,7 @@ int32_t streamStateGetKVByCur_rocksdb(SStreamState* pState, SStreamStateCur* pCu SStreamStateCur* streamStateGetAndCheckCur_rocksdb(SStreamState* pState, SWinKey* key) { SStreamStateCur* pCur = streamStateFillGetCur_rocksdb(pState, key); if (pCur) { - int32_t code = streamStateGetGroupKVByCur_rocksdb(pCur, key, NULL, 0); + int32_t code = streamStateFillGetGroupKVByCur_rocksdb(pCur, key, NULL, 0); if (code == 0) return pCur; streamStateFreeCur(pCur); } @@ -3562,6 +3562,7 @@ SStreamStateCur* streamStateSeekToLast_rocksdb(SStreamState* pState) { STREAM_STATE_DEL_ROCKSDB(pState, "state", &maxStateKey); return pCur; } + SStreamStateCur* streamStateGetCur_rocksdb(SStreamState* pState, const SWinKey* key) { stDebug("streamStateGetCur_rocksdb"); STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; @@ -4126,6 +4127,12 @@ SStreamStateCur* streamStateFillSeekKeyPrev_rocksdb(SStreamState* pState, const streamStateFreeCur(pCur); return NULL; } + +SStreamStateCur* streamStateFillSeekToLast_rocksdb(SStreamState* pState) { + SWinKey key = {.groupId = UINT64_MAX, .ts = INT64_MAX}; + return streamStateFillSeekKeyNext_rocksdb(pState, &key); +} + #ifdef BUILD_NO_CALL int32_t streamStateSessionGetKeyByRange_rocksdb(SStreamState* pState, const SSessionKey* key, SSessionKey* curKey) { stDebug("streamStateSessionGetKeyByRange_rocksdb"); @@ -4316,25 +4323,87 @@ _end: return res; } -#ifdef BUILD_NO_CALL // partag cf int32_t streamStatePutParTag_rocksdb(SStreamState* pState, int64_t groupId, const void* tag, int32_t tagLen) { int code = 0; char* dst = NULL; size_t size = 0; - if (pState->pResultRowStore.resultRowPut == NULL || pState->pExprSupp == NULL) { + if (pState->pResultRowStore.resultRowPut == NULL || pState->pExprSupp == NULL || tag == NULL) { STREAM_STATE_PUT_ROCKSDB(pState, "partag", &groupId, tag, tagLen); return code; } - code = (pState->pResultRowStore.resultRowPut)(pState->pExprSupp, value, vLen, &dst, &size); + code = (pState->pResultRowStore.resultRowPut)(pState->pExprSupp, tag, tagLen, &dst, &size); if (code != 0) { return code; } - STREAM_STATE_PUT_ROCKSDB(pState, "partag", &groupId, dst, size); + STREAM_STATE_PUT_ROCKSDB(pState, "partag", &groupId, dst, (int32_t)size); taosMemoryFree(dst); return code; } +void streamStateParTagSeekKeyNext_rocksdb(SStreamState* pState, const int64_t groupId, SStreamStateCur* pCur) { + if (pCur == NULL) { + return ; + } + STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + pCur->number = pState->number; + pCur->db = wrapper->db; + pCur->iter = streamStateIterCreate(pState, "partag", (rocksdb_snapshot_t**)&pCur->snapshot, + (rocksdb_readoptions_t**)&pCur->readOpt); + int i = streamStateGetCfIdx(pState, "partag"); + if (i < 0) { + stError("streamState failed to put to cf name:%s", "partag"); + return ; + } + + char buf[128] = {0}; + int32_t klen = ginitDict[i].enFunc((void*)&groupId, buf); + if (!streamStateIterSeekAndValid(pCur->iter, buf, klen)) { + return ; + } + // skip ttl expired data + while (rocksdb_iter_valid(pCur->iter) && iterValueIsStale(pCur->iter)) { + rocksdb_iter_next(pCur->iter); + } + + if (rocksdb_iter_valid(pCur->iter)) { + int64_t curGroupId; + size_t kLen = 0; + char* keyStr = (char*)rocksdb_iter_key(pCur->iter, &kLen); + TAOS_UNUSED(parKeyDecode((void*)&curGroupId, keyStr)); + if (curGroupId > groupId) return ; + + rocksdb_iter_next(pCur->iter); + } +} + +int32_t streamStateParTagGetKVByCur_rocksdb(SStreamStateCur* pCur, int64_t* pGroupId, const void** pVal, int32_t* pVLen) { + stDebug("streamStateFillGetKVByCur_rocksdb"); + if (!pCur) { + return -1; + } + SWinKey winKey; + if (!rocksdb_iter_valid(pCur->iter) || iterValueIsStale(pCur->iter)) { + return -1; + } + + size_t klen, vlen; + char* keyStr = (char*)rocksdb_iter_key(pCur->iter, &klen); + (void)parKeyDecode(pGroupId, keyStr); + + if (pVal) { + const char* valStr = rocksdb_iter_value(pCur->iter, &vlen); + int32_t len = valueDecode((void*)valStr, vlen, NULL, (char**)pVal); + if (len < 0) { + return -1; + } + if (pVLen != NULL) *pVLen = len; + } + + return 0; +} + +#ifdef BUILD_NO_CALL int32_t streamStateGetParTag_rocksdb(SStreamState* pState, int64_t groupId, void** tagVal, int32_t* tagLen) { int code = 0; char* tVal; @@ -4538,7 +4607,9 @@ int32_t streamStatePutBatchOptimize(SStreamState* pState, int32_t cfIdx, rocksdb rocksdb_column_family_handle_t* pCf = wrapper->pCf[ginitDict[cfIdx].idx]; rocksdb_writebatch_put_cf((rocksdb_writebatch_t*)pBatch, pCf, buf, (size_t)klen, ttlV, (size_t)ttlVLen); - taosMemoryFree(dst); + if (pState->pResultRowStore.resultRowPut != NULL && pState->pExprSupp != NULL) { + taosMemoryFree(dst); + } if (tmpBuf == NULL) { taosMemoryFree(ttlV); @@ -5240,3 +5311,61 @@ int32_t bkdMgtDumpTo(SBkdMgt* bm, char* taskId, char* dname) { return code; } #endif + +SStreamStateCur* streamStateSeekKeyPrev_rocksdb(SStreamState* pState, const SWinKey* key) { + stDebug("streamStateSeekKeyPrev_rocksdb"); + STaskDbWrapper* wrapper = pState->pTdbState->pOwner->pBackend; + SStreamStateCur* pCur = createStreamStateCursor(); + if (pCur == NULL) { + return NULL; + } + + pCur->db = wrapper->db; + pCur->iter = streamStateIterCreate(pState, "state", (rocksdb_snapshot_t**)&pCur->snapshot, + (rocksdb_readoptions_t**)&pCur->readOpt); + pCur->number = pState->number; + + char buf[128] = {0}; + int len = winKeyEncode((void*)key, buf); + if (!streamStateIterSeekAndValid(pCur->iter, buf, len)) { + streamStateFreeCur(pCur); + return NULL; + } + while (rocksdb_iter_valid(pCur->iter) && iterValueIsStale(pCur->iter)) { + rocksdb_iter_prev(pCur->iter); + } + + if (rocksdb_iter_valid(pCur->iter)) { + SWinKey curKey; + size_t kLen = 0; + char* keyStr = (char*)rocksdb_iter_key(pCur->iter, &kLen); + TAOS_UNUSED(winKeyDecode((void*)&curKey, keyStr)); + if (winKeyCmpr(key, sizeof(*key), &curKey, sizeof(curKey)) > 0) { + return pCur; + } + rocksdb_iter_prev(pCur->iter); + return pCur; + } + + streamStateFreeCur(pCur); + return NULL; +} + +int32_t streamStateGetGroupKVByCur_rocksdb(SStreamState* pState, SStreamStateCur* pCur, SWinKey* pKey, const void** pVal, int32_t* pVLen) { + if (!pCur) { + return -1; + } + uint64_t groupId = pKey->groupId; + + int32_t code = streamStateGetKVByCur_rocksdb(pState, pCur, pKey, pVal, pVLen); + if (code == 0) { + if (pKey->groupId == groupId) { + return 0; + } + if (pVal != NULL) { + taosMemoryFree((void*)*pVal); + *pVal = NULL; + } + } + return -1; +} diff --git a/source/libs/stream/src/streamData.c b/source/libs/stream/src/streamData.c index 58826b2e99..306d6a0239 100644 --- a/source/libs/stream/src/streamData.c +++ b/source/libs/stream/src/streamData.c @@ -15,6 +15,42 @@ #include "streamInt.h" +static int32_t streamMergedSubmitNew(SStreamMergedSubmit** pSubmit) { + *pSubmit = NULL; + + int32_t code = taosAllocateQitem(sizeof(SStreamMergedSubmit), DEF_QITEM, 0, (void**)pSubmit); + if (code) { + return TSDB_CODE_OUT_OF_MEMORY; + } + + (*pSubmit)->submits = taosArrayInit(0, sizeof(SPackedData)); + if ((*pSubmit)->submits == NULL) { + taosFreeQitem(*pSubmit); + *pSubmit = NULL; + return terrno; + } + + (*pSubmit)->type = STREAM_INPUT__MERGED_SUBMIT; + return TSDB_CODE_SUCCESS; +} + +static int32_t streamMergeSubmit(SStreamMergedSubmit* pMerged, SStreamDataSubmit* pSubmit) { + void* p = taosArrayPush(pMerged->submits, &pSubmit->submit); + if (p == NULL) { + return terrno; + } + + if (pSubmit->ver > pMerged->ver) { + pMerged->ver = pSubmit->ver; + } + return 0; +} + +static void freeItems(void* param) { + SSDataBlock* pBlock = param; + taosArrayDestroy(pBlock->pDataBlock); +} + int32_t createStreamBlockFromDispatchMsg(const SStreamDispatchReq* pReq, int32_t blockType, int32_t srcVg, SStreamDataBlock** pRes) { SStreamDataBlock* pData = NULL; int32_t code = taosAllocateQitem(sizeof(SStreamDataBlock), DEF_QITEM, pReq->totalLen, (void**)&pData); @@ -179,37 +215,6 @@ void streamDataSubmitDestroy(SStreamDataSubmit* pDataSubmit) { } } -int32_t streamMergedSubmitNew(SStreamMergedSubmit** pSubmit) { - *pSubmit = NULL; - - int32_t code = taosAllocateQitem(sizeof(SStreamMergedSubmit), DEF_QITEM, 0, (void**)pSubmit); - if (code) { - return TSDB_CODE_OUT_OF_MEMORY; - } - - (*pSubmit)->submits = taosArrayInit(0, sizeof(SPackedData)); - if ((*pSubmit)->submits == NULL) { - taosFreeQitem(*pSubmit); - *pSubmit = NULL; - return terrno; - } - - (*pSubmit)->type = STREAM_INPUT__MERGED_SUBMIT; - return TSDB_CODE_SUCCESS; -} - -int32_t streamMergeSubmit(SStreamMergedSubmit* pMerged, SStreamDataSubmit* pSubmit) { - void* p = taosArrayPush(pMerged->submits, &pSubmit->submit); - if (p == NULL) { - return terrno; - } - - if (pSubmit->ver > pMerged->ver) { - pMerged->ver = pSubmit->ver; - } - return 0; -} - // todo handle memory error int32_t streamQueueMergeQueueItem(SStreamQueueItem* dst, SStreamQueueItem* pElem, SStreamQueueItem** pRes) { *pRes = NULL; @@ -267,11 +272,6 @@ int32_t streamQueueMergeQueueItem(SStreamQueueItem* dst, SStreamQueueItem* pElem } } -static void freeItems(void* param) { - SSDataBlock* pBlock = param; - taosArrayDestroy(pBlock->pDataBlock); -} - void streamFreeQitem(SStreamQueueItem* data) { int8_t type = data->type; if (type == STREAM_INPUT__GET_RES) { @@ -306,3 +306,68 @@ void streamFreeQitem(SStreamQueueItem* data) { taosFreeQitem(pBlock); } } + +int32_t streamCreateForcewindowTrigger(SStreamTrigger** pTrigger, int32_t trigger, SInterval* pInterval, STimeWindow* pLatestWindow, const char* id) { + QRY_PARAM_CHECK(pTrigger); + int64_t ts = INT64_MIN; + SStreamTrigger* p = NULL; + + int32_t code = taosAllocateQitem(sizeof(SStreamTrigger), DEF_QITEM, 0, (void**)&p); + if (code) { + return code; + } + + p->type = STREAM_INPUT__GET_RES; + p->pBlock = taosMemoryCalloc(1, sizeof(SSDataBlock)); + if (p->pBlock == NULL) { + taosFreeQitem(p); + return terrno; + } + + // let's calculate the previous time window + SInterval interval = {.interval = trigger, + .sliding = trigger, + .intervalUnit = pInterval->intervalUnit, + .slidingUnit = pInterval->slidingUnit}; + + ts = taosGetTimestampMs(); + + if (pLatestWindow->skey == INT64_MIN) { + STimeWindow window = getAlignQueryTimeWindow(&interval, ts - trigger); + + p->pBlock->info.window.skey = window.skey; + p->pBlock->info.window.ekey = TMAX(ts, window.ekey); + } else { + int64_t skey = pLatestWindow->skey + trigger; + p->pBlock->info.window.skey = skey; + p->pBlock->info.window.ekey = TMAX(ts, skey + trigger); + } + + p->pBlock->info.type = STREAM_GET_RESULT; + stDebug("s-task:%s force_window_close trigger block generated, window range:%" PRId64 "-%" PRId64, id, + p->pBlock->info.window.skey, p->pBlock->info.window.ekey); + + *pTrigger = p; + return code; +} + +int32_t streamCreateSinkResTrigger(SStreamTrigger** pTrigger) { + QRY_PARAM_CHECK(pTrigger); + SStreamTrigger* p = NULL; + + int32_t code = taosAllocateQitem(sizeof(SStreamTrigger), DEF_QITEM, 0, (void**)&p); + if (code) { + return code; + } + + p->type = STREAM_INPUT__GET_RES; + p->pBlock = taosMemoryCalloc(1, sizeof(SSDataBlock)); + if (p->pBlock == NULL) { + taosFreeQitem(p); + return terrno; + } + + p->pBlock->info.type = STREAM_GET_ALL; + *pTrigger = p; + return code; +} \ No newline at end of file diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 2e06813071..318720b5b0 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -154,7 +154,7 @@ int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, int64_t* if ((code = qExecTask(pExecutor, &output, &ts)) < 0) { if (code == TSDB_CODE_QRY_IN_EXEC) { - resetTaskInfo(pExecutor); + qResetTaskInfoCode(pExecutor); } if (code == TSDB_CODE_OUT_OF_MEMORY || code == TSDB_CODE_INVALID_PARA || code == TSDB_CODE_FILE_CORRUPTED) { @@ -188,15 +188,13 @@ int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, int64_t* continue; // checkpoint block not dispatch to downstream tasks } - SSDataBlock block = {0}; + SSDataBlock block = {.info.childId = pTask->info.selfChildId}; code = assignOneDataBlock(&block, output); if (code) { stError("s-task:%s failed to build result block due to out of memory", pTask->id.idStr); continue; } - block.info.childId = pTask->info.selfChildId; - size += blockDataGetSize(output) + sizeof(SSDataBlock) + sizeof(SColumnInfoData) * blockDataGetNumOfCols(&block); numOfBlocks += 1; diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index f5de719848..db6c841f5e 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -723,7 +723,7 @@ int32_t streamMetaRegisterTask(SStreamMeta* pMeta, int64_t ver, SStreamTask* pTa pTask->id.refId = refId = taosAddRef(streamTaskRefPool, pTask); code = taosHashPut(pMeta->pTasksMap, &id, sizeof(id), &pTask->id.refId, sizeof(int64_t)); - if (code) { + if (code) { // todo remove it from task list stError("s-task:0x%" PRIx64 " failed to register task into meta-list, code: out of memory", id.taskId); int32_t ret = taosRemoveRef(streamTaskRefPool, refId); @@ -754,9 +754,6 @@ int32_t streamMetaRegisterTask(SStreamMeta* pMeta, int64_t ver, SStreamTask* pTa int32_t val = atomic_add_fetch_32(&pMeta->numOfStreamTasks, 1); } - // enable the scheduler for stream tasks - streamSetupScheduleTrigger(pTask); - *pAdded = true; return code; } @@ -1159,9 +1156,6 @@ void streamMetaLoadAllTasks(SStreamMeta* pMeta) { continue; } - // enable the scheduler for stream tasks after acquire the task RefId. - streamSetupScheduleTrigger(pTask); - stInfo("s-task:0x%x vgId:%d set refId:%"PRId64, (int32_t) id.taskId, vgId, pTask->id.refId); if (pTask->info.fillHistory == 0) { int32_t val = atomic_add_fetch_32(&pMeta->numOfStreamTasks, 1); diff --git a/source/libs/stream/src/streamMsg.c b/source/libs/stream/src/streamMsg.c index 1c512888e7..193daa0cc4 100644 --- a/source/libs/stream/src/streamMsg.c +++ b/source/libs/stream/src/streamMsg.c @@ -613,7 +613,7 @@ int32_t tEncodeStreamTask(SEncoder* pEncoder, const SStreamTask* pTask) { TAOS_CHECK_EXIT(tEncodeI64(pEncoder, pTask->ver)); TAOS_CHECK_EXIT(tEncodeI64(pEncoder, pTask->id.streamId)); TAOS_CHECK_EXIT(tEncodeI32(pEncoder, pTask->id.taskId)); - TAOS_CHECK_EXIT(tEncodeI32(pEncoder, pTask->info.totalLevel)); + TAOS_CHECK_EXIT(tEncodeI32(pEncoder, pTask->info.trigger)); TAOS_CHECK_EXIT(tEncodeI8(pEncoder, pTask->info.taskLevel)); TAOS_CHECK_EXIT(tEncodeI8(pEncoder, pTask->outputInfo.type)); TAOS_CHECK_EXIT(tEncodeI16(pEncoder, pTask->msgInfo.msgType)); @@ -692,7 +692,7 @@ int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask) { TAOS_CHECK_EXIT(tDecodeI64(pDecoder, &pTask->id.streamId)); TAOS_CHECK_EXIT(tDecodeI32(pDecoder, &pTask->id.taskId)); - TAOS_CHECK_EXIT(tDecodeI32(pDecoder, &pTask->info.totalLevel)); + TAOS_CHECK_EXIT(tDecodeI32(pDecoder, &pTask->info.trigger)); TAOS_CHECK_EXIT(tDecodeI8(pDecoder, &pTask->info.taskLevel)); TAOS_CHECK_EXIT(tDecodeI8(pDecoder, &pTask->outputInfo.type)); TAOS_CHECK_EXIT(tDecodeI16(pDecoder, &pTask->msgInfo.msgType)); diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index bafd354360..20c3e5a6b9 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -351,7 +351,7 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) if (type != STREAM_INPUT__GET_RES && type != STREAM_INPUT__CHECKPOINT && type != STREAM_INPUT__CHECKPOINT_TRIGGER && (pTask->info.delaySchedParam != 0)) { (void)atomic_val_compare_exchange_8(&pTask->schedInfo.status, TASK_TRIGGER_STATUS__INACTIVE, - TASK_TRIGGER_STATUS__ACTIVE); + TASK_TRIGGER_STATUS__MAY_ACTIVE); stDebug("s-task:%s new data arrived, active the sched-trigger, triggerStatus:%d", pTask->id.idStr, pTask->schedInfo.status); } diff --git a/source/libs/stream/src/streamSched.c b/source/libs/stream/src/streamSched.c index c5c4a8cc34..8c79abfd02 100644 --- a/source/libs/stream/src/streamSched.c +++ b/source/libs/stream/src/streamSched.c @@ -13,6 +13,7 @@ * along with this program. If not, see . */ +#include "ttime.h" #include "streamInt.h" #include "ttimer.h" @@ -20,18 +21,54 @@ static void streamTaskResumeHelper(void* param, void* tmrId); static void streamTaskSchedHelper(void* param, void* tmrId); void streamSetupScheduleTrigger(SStreamTask* pTask) { - int64_t delayParam = pTask->info.delaySchedParam; - if (delayParam != 0 && pTask->info.fillHistory == 0) { - int64_t* pTaskRefId = NULL; - int32_t code = streamTaskAllocRefId(pTask, &pTaskRefId); - if (code == 0) { - stDebug("s-task:%s refId:%" PRId64 " enable the scheduler trigger, delay:%" PRId64, pTask->id.idStr, - pTask->id.refId, delayParam); + int64_t delay = 0; + int32_t code = 0; + const char* id = pTask->id.idStr; + int64_t* pTaskRefId = NULL; - streamTmrStart(streamTaskSchedHelper, (int32_t)delayParam, pTaskRefId, streamTimer, - &pTask->schedInfo.pDelayTimer, pTask->pMeta->vgId, "sched-tmr"); - pTask->schedInfo.status = TASK_TRIGGER_STATUS__INACTIVE; + if (pTask->info.fillHistory == 1) { + return; + } + + // dynamic set the trigger & triggerParam for STREAM_TRIGGER_FORCE_WINDOW_CLOSE + if ((pTask->info.trigger == STREAM_TRIGGER_FORCE_WINDOW_CLOSE) && (pTask->info.taskLevel == TASK_LEVEL__SOURCE)) { + int64_t waterMark = 0; + SInterval interval = {0}; + STimeWindow lastTimeWindow = {0}; + code = qGetStreamIntervalExecInfo(pTask->exec.pExecutor, &waterMark, &interval, &lastTimeWindow); + if (code) { + stError("s-task:%s failed to init scheduler info, code:%s", id, tstrerror(code)); + return; } + + pTask->status.latestForceWindow = lastTimeWindow; + pTask->info.delaySchedParam = interval.sliding; + pTask->info.watermark = waterMark; + pTask->info.interval = interval; + + // calculate the first start timestamp + int64_t now = taosGetTimestamp(interval.precision); + STimeWindow curWin = getAlignQueryTimeWindow(&pTask->info.interval, now); + delay = (curWin.ekey + 1) - now + waterMark; + + stInfo("s-task:%s extract interval info from executor, wm:%" PRId64 " interval:%" PRId64 " unit:%c sliding:%" PRId64 + " unit:%c, initial start after:%" PRId64, + id, waterMark, interval.interval, interval.intervalUnit, interval.sliding, interval.slidingUnit, delay); + } else { + delay = pTask->info.delaySchedParam; + if (delay == 0) { + return; + } + } + + code = streamTaskAllocRefId(pTask, &pTaskRefId); + if (code == 0) { + stDebug("s-task:%s refId:%" PRId64 " enable the scheduler trigger, delay:%" PRId64, pTask->id.idStr, + pTask->id.refId, delay); + + streamTmrStart(streamTaskSchedHelper, (int32_t)delay, pTaskRefId, streamTimer, + &pTask->schedInfo.pDelayTimer, pTask->pMeta->vgId, "sched-tmr"); + pTask->schedInfo.status = TASK_TRIGGER_STATUS__INACTIVE; } } @@ -142,6 +179,7 @@ void streamTaskSchedHelper(void* param, void* tmrId) { const char* id = pTask->id.idStr; int32_t nextTrigger = (int32_t)pTask->info.delaySchedParam; int32_t vgId = pTask->pMeta->vgId; + int32_t code = 0; int8_t status = atomic_load_8(&pTask->schedInfo.status); stTrace("s-task:%s in scheduler, trigger status:%d, next:%dms", id, status, nextTrigger); @@ -161,43 +199,76 @@ void streamTaskSchedHelper(void* param, void* tmrId) { return; } + if (streamTaskShouldPause(pTask)) { + stDebug("s-task:%s is paused, check in nextTrigger:%ds", id, nextTrigger/1000); + streamTmrStart(streamTaskSchedHelper, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pDelayTimer, vgId, + "sched-run-tmr"); + } + if (streamTaskGetStatus(pTask).state == TASK_STATUS__CK) { stDebug("s-task:%s in checkpoint procedure, not retrieve result, next:%dms", id, nextTrigger); } else { - if (status == TASK_TRIGGER_STATUS__ACTIVE) { - SStreamTrigger* pTrigger; + if (pTask->info.trigger == STREAM_TRIGGER_FORCE_WINDOW_CLOSE && pTask->info.taskLevel == TASK_LEVEL__SOURCE) { + SStreamTrigger* pTrigger = NULL; - int32_t code = taosAllocateQitem(sizeof(SStreamTrigger), DEF_QITEM, 0, (void**)&pTrigger); - if (code) { - stError("s-task:%s failed to prepare retrieve data trigger, code:%s, try again in %dms", id, "out of memory", - nextTrigger); - terrno = code; - goto _end; + while (1) { + code = streamCreateForcewindowTrigger(&pTrigger, pTask->info.delaySchedParam, &pTask->info.interval, + &pTask->status.latestForceWindow, id); + if (code != 0) { + stError("s-task:%s failed to prepare force window close trigger, code:%s, try again in %dms", id, + tstrerror(code), nextTrigger); + goto _end; + } + + // in the force window close model, status trigger does not matter. So we do not set the trigger model + code = streamTaskPutDataIntoInputQ(pTask, (SStreamQueueItem*)pTrigger); + if (code != TSDB_CODE_SUCCESS) { + stError("s-task:%s failed to put retrieve aggRes block into q, code:%s", pTask->id.idStr, tstrerror(code)); + goto _end; + } + + // check whether the time window gaps exist or not + int64_t now = taosGetTimestamp(pTask->info.interval.precision); + int64_t intervalEndTs = pTrigger->pBlock->info.window.skey + pTask->info.interval.interval; + + // there are gaps, needs to be filled + STimeWindow w = pTrigger->pBlock->info.window; + w.ekey = w.skey + pTask->info.interval.interval; + if (w.skey <= pTask->status.latestForceWindow.skey) { + stFatal("s-task:%s invalid new time window in force_window_close model, skey:%" PRId64 + " should be greater than latestForceWindow skey:%" PRId64, + pTask->id.idStr, w.skey, pTask->status.latestForceWindow.skey); + } + + pTask->status.latestForceWindow = w; + if (intervalEndTs + pTask->info.watermark + pTask->info.interval.interval > now) { + break; + } else { + stDebug("s-task:%s gap exist for force_window_close, current force_window_skey:%" PRId64, id, w.skey); + } } - pTrigger->type = STREAM_INPUT__GET_RES; - pTrigger->pBlock = taosMemoryCalloc(1, sizeof(SSDataBlock)); - if (pTrigger->pBlock == NULL) { - taosFreeQitem(pTrigger); - - stError("s-task:%s failed to build retrieve data trigger, code:out of memory, try again in %dms", id, + } else if (status == TASK_TRIGGER_STATUS__MAY_ACTIVE) { + SStreamTrigger* pTrigger = NULL; + code = streamCreateSinkResTrigger(&pTrigger); + if (code) { + stError("s-task:%s failed to prepare retrieve data trigger, code:%s, try again in %dms", id, tstrerror(code), nextTrigger); goto _end; } atomic_store_8(&pTask->schedInfo.status, TASK_TRIGGER_STATUS__INACTIVE); - pTrigger->pBlock->info.type = STREAM_GET_ALL; code = streamTaskPutDataIntoInputQ(pTask, (SStreamQueueItem*)pTrigger); if (code != TSDB_CODE_SUCCESS) { - stError("s-task:%s failed to put retrieve block into trigger, code:%s", pTask->id.idStr, tstrerror(code)); + stError("s-task:%s failed to put retrieve aggRes block into q, code:%s", pTask->id.idStr, tstrerror(code)); goto _end; } + } - code = streamTrySchedExec(pTask); - if (code != TSDB_CODE_SUCCESS) { - stError("s-task:%s failed to sched to run, wait for next time", pTask->id.idStr); - } + code = streamTrySchedExec(pTask); + if (code != TSDB_CODE_SUCCESS) { + stError("s-task:%s failed to sched to run, wait for next time", pTask->id.idStr); } } diff --git a/source/libs/stream/src/streamSessionState.c b/source/libs/stream/src/streamSessionState.c index 536636533f..d2d7c7b11b 100644 --- a/source/libs/stream/src/streamSessionState.c +++ b/source/libs/stream/src/streamSessionState.c @@ -20,12 +20,10 @@ #include "tcommon.h" #include "tsimplehash.h" -typedef int (*__session_compare_fn_t)(const SSessionKey* pWin, const void* pDatas, int pos); - -int sessionStateKeyCompare(const SSessionKey* pWin1, const void* pDatas, int pos) { +int sessionStateKeyCompare(const void* pWin1, const void* pDatas, int pos) { SRowBuffPos* pPos2 = taosArrayGetP(pDatas, pos); SSessionKey* pWin2 = (SSessionKey*)pPos2->pKey; - return sessionWinKeyCmpr(pWin1, pWin2); + return sessionWinKeyCmpr((SSessionKey*)pWin1, pWin2); } int sessionStateRangeKeyCompare(const SSessionKey* pWin1, const void* pDatas, int pos) { @@ -79,7 +77,7 @@ bool inSessionWindow(SSessionKey* pKey, TSKEY ts, int64_t gap) { return false; } -SStreamStateCur* createSessionStateCursor(SStreamFileState* pFileState) { +SStreamStateCur* createStateCursor(SStreamFileState* pFileState) { SStreamStateCur* pCur = createStreamStateCursor(); if (pCur == NULL) { return NULL; @@ -536,7 +534,7 @@ static SStreamStateCur* seekKeyCurrentPrev_buff(SStreamFileState* pFileState, co } if (index >= 0) { - pCur = createSessionStateCursor(pFileState); + pCur = createStateCursor(pFileState); if (pCur == NULL) { return NULL; } @@ -580,7 +578,7 @@ static void checkAndTransformCursor(SStreamFileState* pFileState, const uint64_t if (taosArrayGetSize(pWinStates) > 0 && (code == TSDB_CODE_FAILED || sessionStateKeyCompare(&key, pWinStates, 0) >= 0)) { if (!(*ppCur)) { - (*ppCur) = createSessionStateCursor(pFileState); + (*ppCur) = createStateCursor(pFileState); } transformCursor(pFileState, *ppCur); } else if (*ppCur) { @@ -640,7 +638,7 @@ SStreamStateCur* countWinStateSeekKeyPrev(SStreamFileState* pFileState, const SS } pBuffCur->buffIndex = 0; } else if (taosArrayGetSize(pWinStates) > 0) { - pBuffCur = createSessionStateCursor(pFileState); + pBuffCur = createStateCursor(pFileState); if (pBuffCur == NULL) { return NULL; } diff --git a/source/libs/stream/src/streamSliceState.c b/source/libs/stream/src/streamSliceState.c new file mode 100644 index 0000000000..238bff8afc --- /dev/null +++ b/source/libs/stream/src/streamSliceState.c @@ -0,0 +1,290 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * This program is free software: you can use, redistribute, and/or modify + * it under the terms of the GNU Affero General Public License, version 3 + * or later ("AGPL"), as published by the Free Software Foundation. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +#include "tstreamFileState.h" + +#include "query.h" +#include "streamBackendRocksdb.h" +#include "tcommon.h" +#include "tsimplehash.h" + +#define NUM_OF_CACHE_WIN 64 +#define MAX_NUM_OF_CACHE_WIN 128 + +int32_t getHashSortRowBuff(SStreamFileState* pFileState, const SWinKey* pKey, void** pVal, int32_t* pVLen, + int32_t* pWinCode) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + + code = addRowBuffIfNotExist(pFileState, (void*)pKey, sizeof(SWinKey), pVal, pVLen, pWinCode); + QUERY_CHECK_CODE(code, lino, _end); + + SArray* pWinStates = NULL; + SSHashObj* pSearchBuff = getSearchBuff(pFileState); + code = addArrayBuffIfNotExist(pSearchBuff, pKey->groupId, &pWinStates); + QUERY_CHECK_CODE(code, lino, _end); + + // recover + if (taosArrayGetSize(pWinStates) == 0 && needClearDiskBuff(pFileState)) { + TSKEY ts = getFlushMark(pFileState); + SWinKey start = {.groupId = pKey->groupId, .ts = INT64_MAX}; + void* pState = getStateFileStore(pFileState); + SStreamStateCur* pCur = streamStateFillSeekKeyPrev_rocksdb(pState, &start); + for (int32_t i = 0; i < NUM_OF_CACHE_WIN; i++) { + SWinKey tmpKey = {.groupId = pKey->groupId}; + int32_t tmpRes = streamStateFillGetGroupKVByCur_rocksdb(pCur, &tmpKey, NULL, 0); + if (tmpRes != TSDB_CODE_SUCCESS) { + break; + } + void* tmp = taosArrayPush(pWinStates, &tmpKey); + QUERY_CHECK_NULL(tmp, code, lino, _end, terrno); + streamStateCurPrev_rocksdb(pCur); + } + taosArraySort(pWinStates, winKeyCmprImpl); + streamStateFreeCur(pCur); + } + + code = addSearchItem(pFileState, pWinStates, pKey); + QUERY_CHECK_CODE(code, lino, _end); + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +int32_t hashSortFileGetFn(SStreamFileState* pFileState, void* pKey, void** data, int32_t* pDataLen) { + void* pState = getStateFileStore(pFileState); + return streamStateFillGet_rocksdb(pState, pKey, data, pDataLen); +} + +int32_t hashSortFileRemoveFn(SStreamFileState* pFileState, const void* pKey) { + void* pState = getStateFileStore(pFileState); + return streamStateFillDel_rocksdb(pState, pKey); +} + +void clearSearchBuff(SStreamFileState* pFileState) { + SSHashObj* pSearchBuff = getSearchBuff(pFileState); + if (!pSearchBuff) { + return; + } + TSKEY flushMark = getFlushMark(pFileState); + void* pIte = NULL; + int32_t iter = 0; + while ((pIte = tSimpleHashIterate(pSearchBuff, pIte, &iter)) != NULL) { + SArray* pWinStates = *((void**)pIte); + int32_t size = taosArrayGetSize(pWinStates); + if (size > 0) { + int64_t gpId = *(int64_t*)tSimpleHashGetKey(pIte, NULL); + SWinKey key = {.ts = flushMark, .groupId = gpId}; + int32_t num = binarySearch(pWinStates, size, &key, fillStateKeyCompare); + if (size > NUM_OF_CACHE_WIN) { + num = TMIN(num, size - NUM_OF_CACHE_WIN); + taosArrayRemoveBatch(pWinStates, 0, num, NULL); + } + } + } +} + +int32_t getStateFromRocksdbByCur(SStreamFileState* pFileState, SStreamStateCur* pCur, SWinKey* pResKey, SRowBuffPos** ppPos, int32_t* pVLen, int32_t* pWinCode) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + void* tmpVal = NULL; + int32_t len = 0; + (*pWinCode) = streamStateFillGetGroupKVByCur_rocksdb(pCur, pResKey, (const void**)&tmpVal, &len); + if ((*pWinCode) == TSDB_CODE_SUCCESS) { + SRowBuffPos* pNewPos = getNewRowPosForWrite(pFileState); + if (!pNewPos || !pNewPos->pRowBuff) { + code = TSDB_CODE_OUT_OF_MEMORY; + QUERY_CHECK_CODE(code, lino, _end); + } + memcpy(pNewPos->pRowBuff, tmpVal, len); + taosMemoryFreeClear(tmpVal); + *pVLen = getRowStateRowSize(pFileState); + (*ppPos) = pNewPos; + } +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +int32_t getHashSortNextRow(SStreamFileState* pFileState, const SWinKey* pKey, SWinKey* pResKey, void** ppVal, + int32_t* pVLen, int32_t* pWinCode) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + SArray* pWinStates = NULL; + SSHashObj* pSearchBuff = getSearchBuff(pFileState); + void* pState = getStateFileStore(pFileState); + void** ppBuff = tSimpleHashGet(pSearchBuff, &pKey->groupId, sizeof(uint64_t)); + if (ppBuff) { + pWinStates = (SArray*)(*ppBuff); + } else { + SStreamStateCur* pCur = streamStateFillSeekKeyNext_rocksdb(pState, pKey); + void* tmpVal = NULL; + int32_t len = 0; + (*pWinCode) = streamStateFillGetGroupKVByCur_rocksdb(pCur, pResKey, (const void**)&tmpVal, &len); + if ((*pWinCode) == TSDB_CODE_SUCCESS && ppVal != NULL) { + SRowBuffPos* pNewPos = getNewRowPosForWrite(pFileState); + if (!pNewPos || !pNewPos->pRowBuff) { + code = TSDB_CODE_OUT_OF_MEMORY; + QUERY_CHECK_CODE(code, lino, _end); + } + memcpy(pNewPos->pRowBuff, tmpVal, len); + *pVLen = getRowStateRowSize(pFileState); + (*ppVal) = pNewPos; + } + taosMemoryFreeClear(tmpVal); + streamStateFreeCur(pCur); + return code; + } + int32_t size = taosArrayGetSize(pWinStates); + int32_t index = binarySearch(pWinStates, size, pKey, fillStateKeyCompare); + if (index == -1) { + SStreamStateCur* pCur = streamStateFillSeekKeyNext_rocksdb(pState, pKey); + void* tmpVal = NULL; + int32_t len = 0; + (*pWinCode) = streamStateFillGetGroupKVByCur_rocksdb(pCur, pResKey, (const void**)&tmpVal, &len); + if ((*pWinCode) == TSDB_CODE_SUCCESS) { + if (ppVal != NULL) { + SRowBuffPos* pNewPos = getNewRowPosForWrite(pFileState); + if (!pNewPos || !pNewPos->pRowBuff) { + code = TSDB_CODE_OUT_OF_MEMORY; + QUERY_CHECK_CODE(code, lino, _end); + } + memcpy(pNewPos->pRowBuff, tmpVal, len); + *pVLen = getRowStateRowSize(pFileState); + (*ppVal) = pNewPos; + } + taosMemoryFreeClear(tmpVal); + streamStateFreeCur(pCur); + return code; + } + streamStateFreeCur(pCur); + } + + if (index == size - 1) { + (*pWinCode) = TSDB_CODE_FAILED; + return code; + } + SWinKey* pNext = taosArrayGet(pWinStates, index + 1); + *pResKey = *pNext; + if (ppVal == NULL) { + (*pWinCode) = TSDB_CODE_SUCCESS; + return code; + } + return getHashSortRowBuff(pFileState, pResKey, ppVal, pVLen, pWinCode); + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +int32_t getHashSortPrevRow(SStreamFileState* pFileState, const SWinKey* pKey, SWinKey* pResKey, void** ppVal, + int32_t* pVLen, int32_t* pWinCode) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + SArray* pWinStates = NULL; + SSHashObj* pSearchBuff = getSearchBuff(pFileState); + void* pState = getStateFileStore(pFileState); + void** ppBuff = (void**) tSimpleHashGet(pSearchBuff, &pKey->groupId, sizeof(uint64_t)); + if (ppBuff) { + pWinStates = (SArray*)(*ppBuff); + } else { + qDebug("===stream=== search buff is empty.group id:%" PRId64, pKey->groupId); + SStreamStateCur* pCur = streamStateFillSeekKeyPrev_rocksdb(pState, pKey); + void* tmpVal = NULL; + int32_t len = 0; + (*pWinCode) = streamStateFillGetGroupKVByCur_rocksdb(pCur, pResKey, (const void**)&tmpVal, &len); + if ((*pWinCode) == TSDB_CODE_SUCCESS) { + SRowBuffPos* pNewPos = getNewRowPosForWrite(pFileState); + if (!pNewPos || !pNewPos->pRowBuff) { + code = TSDB_CODE_OUT_OF_MEMORY; + QUERY_CHECK_CODE(code, lino, _end); + } + memcpy(pNewPos->pRowBuff, tmpVal, len); + taosMemoryFreeClear(tmpVal); + *pVLen = getRowStateRowSize(pFileState); + (*ppVal) = pNewPos; + } + streamStateFreeCur(pCur); + return code; + } + int32_t size = taosArrayGetSize(pWinStates); + int32_t index = binarySearch(pWinStates, size, pKey, fillStateKeyCompare); + if (index >= 0) { + SWinKey* pCurKey = taosArrayGet(pWinStates, index); + if (winKeyCmprImpl(pCurKey, pKey) == 0) { + index--; + } else { + qDebug("%s failed at line %d since do not find cur SWinKey. trigger may be force window close", __func__, __LINE__); + } + } + if (index == -1) { + SStreamStateCur* pCur = streamStateFillSeekKeyPrev_rocksdb(pState, pKey); + void* tmpVal = NULL; + int32_t len = 0; + (*pWinCode) = streamStateFillGetGroupKVByCur_rocksdb(pCur, pResKey, (const void**)&tmpVal, &len); + if ((*pWinCode) == TSDB_CODE_SUCCESS) { + SRowBuffPos* pNewPos = getNewRowPosForWrite(pFileState); + if (!pNewPos || !pNewPos->pRowBuff) { + code = TSDB_CODE_OUT_OF_MEMORY; + QUERY_CHECK_CODE(code, lino, _end); + } + memcpy(pNewPos->pRowBuff, tmpVal, len); + taosMemoryFreeClear(tmpVal); + *pVLen = getRowStateRowSize(pFileState); + (*ppVal) = pNewPos; + } + streamStateFreeCur(pCur); + return code; + } else { + SWinKey* pPrevKey = taosArrayGet(pWinStates, index); + *pResKey = *pPrevKey; + return getHashSortRowBuff(pFileState, pResKey, ppVal, pVLen, pWinCode); + } + (*pWinCode) = TSDB_CODE_FAILED; + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +void deleteHashSortRowBuff(SStreamFileState* pFileState, const SWinKey* pKey) { + SSHashObj* pSearchBuff = getSearchBuff(pFileState); + void** ppBuff = tSimpleHashGet(pSearchBuff, &pKey->groupId, sizeof(uint64_t)); + if (!ppBuff) { + return; + } + SArray* pWinStates = *ppBuff; + int32_t size = taosArrayGetSize(pWinStates); + if (!isFlushedState(pFileState, pKey->ts, 0)) { + // find the first position which is smaller than the pKey + int32_t index = binarySearch(pWinStates, size, pKey, fillStateKeyCompare); + if (index == -1) { + index = 0; + } + SWinKey* pTmpKey = taosArrayGet(pWinStates, index); + if (winKeyCmprImpl(pTmpKey, pKey) == 0) { + taosArrayRemove(pWinStates, index); + } + } +} diff --git a/source/libs/stream/src/streamState.c b/source/libs/stream/src/streamState.c index 0e2ff48fa5..794fc346bf 100644 --- a/source/libs/stream/src/streamState.c +++ b/source/libs/stream/src/streamState.c @@ -130,10 +130,8 @@ SStreamState* streamStateOpen(const char* path, void* pTask, int64_t streamId, i pState->pFileState = NULL; _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT); pState->parNameMap = tSimpleHashInit(1024, hashFn); - if (!pState->parNameMap) { - code = TSDB_CODE_OUT_OF_MEMORY; - QUERY_CHECK_CODE(code, lino, _end); - } + QUERY_CHECK_NULL(pState->parNameMap, code, lino, _end, terrno); + stInfo("open state %p on backend %p 0x%" PRIx64 "-%d succ", pState, pMeta->streamBackend, pState->streamId, pState->taskId); return pState; @@ -205,14 +203,10 @@ _end: return code; } -// todo refactor -int32_t streamStatePut(SStreamState* pState, const SWinKey* key, const void* value, int32_t vLen) { - return 0; - // return streamStatePut_rocksdb(pState, key, value, vLen); -} +int32_t streamStatePut(SStreamState* pState, const SWinKey* key, const void* value, int32_t vLen) { return 0; } int32_t streamStateGet(SStreamState* pState, const SWinKey* key, void** pVal, int32_t* pVLen, int32_t* pWinCode) { - return getRowBuff(pState->pFileState, (void*)key, sizeof(SWinKey), pVal, pVLen, pWinCode); + return addRowBuffIfNotExist(pState->pFileState, (void*)key, sizeof(SWinKey), pVal, pVLen, pWinCode); } bool streamStateCheck(SStreamState* pState, const SWinKey* key) { @@ -225,22 +219,36 @@ int32_t streamStateGetByPos(SStreamState* pState, void* pos, void** pVal) { return code; } -// todo refactor void streamStateDel(SStreamState* pState, const SWinKey* key) { deleteRowBuff(pState->pFileState, key, sizeof(SWinKey)); } -// todo refactor int32_t streamStateFillPut(SStreamState* pState, const SWinKey* key, const void* value, int32_t vLen) { return streamStateFillPut_rocksdb(pState, key, value, vLen); } -// todo refactor -int32_t streamStateFillGet(SStreamState* pState, const SWinKey* key, void** pVal, int32_t* pVLen) { +int32_t streamStateFillGet(SStreamState* pState, const SWinKey* key, void** pVal, int32_t* pVLen, int32_t* pWinCode) { + if (pState->pFileState) { + return getRowBuff(pState->pFileState, (void*)key, sizeof(SWinKey), pVal, pVLen, pWinCode); + } return streamStateFillGet_rocksdb(pState, key, pVal, pVLen); } -// todo refactor +int32_t streamStateFillAddIfNotExist(SStreamState* pState, const SWinKey* key, void** pVal, int32_t* pVLen, + int32_t* pWinCode) { + return getHashSortRowBuff(pState->pFileState, key, pVal, pVLen, pWinCode); +} + +int32_t streamStateFillGetNext(SStreamState* pState, const SWinKey* pKey, SWinKey* pResKey, void** pVal, int32_t* pVLen, + int32_t* pWinCode) { + return getHashSortNextRow(pState->pFileState, pKey, pResKey, pVal, pVLen, pWinCode); +} + +int32_t streamStateFillGetPrev(SStreamState* pState, const SWinKey* pKey, SWinKey* pResKey, void** pVal, int32_t* pVLen, + int32_t* pWinCode) { + return getHashSortPrevRow(pState->pFileState, pKey, pResKey, pVal, pVLen, pWinCode); +} + void streamStateFillDel(SStreamState* pState, const SWinKey* key) { int32_t code = streamStateFillDel_rocksdb(pState, key); qTrace("%s at line %d res %d", __func__, __LINE__, code); @@ -280,11 +288,27 @@ int32_t streamStateGetInfo(SStreamState* pState, void* pKey, int32_t keyLen, voi int32_t streamStateAddIfNotExist(SStreamState* pState, const SWinKey* key, void** pVal, int32_t* pVLen, int32_t* pWinCode) { - return streamStateGet(pState, key, pVal, pVLen, pWinCode); + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + code = streamStateGet(pState, key, pVal, pVLen, pWinCode); + QUERY_CHECK_CODE(code, lino, _end); + + SSHashObj* pSearchBuff = getSearchBuff(pState->pFileState); + if (pSearchBuff != NULL) { + SArray* pWinStates = NULL; + code = addArrayBuffIfNotExist(pSearchBuff, key->groupId, &pWinStates); + QUERY_CHECK_CODE(code, lino, _end); + code = addSearchItem(pState->pFileState, pWinStates, key); + QUERY_CHECK_CODE(code, lino, _end); + } +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; } void streamStateReleaseBuf(SStreamState* pState, void* pVal, bool used) { - // todo refactor if (!pVal) { return; } @@ -309,8 +333,8 @@ int32_t streamStateFillGetKVByCur(SStreamStateCur* pCur, SWinKey* pKey, const vo return streamStateFillGetKVByCur_rocksdb(pCur, pKey, pVal, pVLen); } -int32_t streamStateGetGroupKVByCur(SStreamStateCur* pCur, SWinKey* pKey, const void** pVal, int32_t* pVLen) { - return streamStateGetGroupKVByCur_rocksdb(pCur, pKey, pVal, pVLen); +int32_t streamStateFillGetGroupKVByCur(SStreamStateCur* pCur, SWinKey* pKey, const void** pVal, int32_t* pVLen) { + return streamStateFillGetGroupKVByCur_rocksdb(pCur, pKey, pVal, pVLen); } SStreamStateCur* streamStateSeekKeyNext(SStreamState* pState, const SWinKey* key) { @@ -447,7 +471,6 @@ int32_t streamStateSessionAddIfNotExist(SStreamState* pState, SSessionKey* key, int32_t streamStateStateAddIfNotExist(SStreamState* pState, SSessionKey* key, char* pKeyData, int32_t keyDataLen, state_key_cmpr_fn fn, void** pVal, int32_t* pVLen, int32_t* pWinCode) { - // todo refactor return getStateWinResultBuff(pState->pFileState, key, pKeyData, keyDataLen, fn, pVal, pVLen, pWinCode); } @@ -552,3 +575,44 @@ int32_t streamStateCountWinAddIfNotExist(SStreamState* pState, SSessionKey* pKey int32_t streamStateCountWinAdd(SStreamState* pState, SSessionKey* pKey, COUNT_TYPE winCount, void** pVal, int32_t* pVLen) { return createCountWinResultBuff(pState->pFileState, pKey, winCount, pVal, pVLen); } + +int32_t streamStateGroupPut(SStreamState* pState, int64_t groupId, void* value, int32_t vLen) { + return streamFileStateGroupPut(pState->pFileState, groupId, value, vLen); +} + +SStreamStateCur* streamStateGroupGetCur(SStreamState* pState) { + SStreamStateCur* pCur = createStateCursor(pState->pFileState); + pCur->hashIter = 0; + pCur->pHashData = NULL; + SSHashObj* pMap = getGroupIdCache(pState->pFileState); + pCur->pHashData = tSimpleHashIterate(pMap, pCur->pHashData, &pCur->hashIter); + if (pCur->pHashData == NULL) { + pCur->hashIter = -1; + streamStateParTagSeekKeyNext_rocksdb(pState, INT64_MIN, pCur); + } + return pCur; +} + +void streamStateGroupCurNext(SStreamStateCur* pCur) { + streamFileStateGroupCurNext(pCur); +} + +int32_t streamStateGroupGetKVByCur(SStreamStateCur* pCur, int64_t* pKey, void** pVal, int32_t* pVLen) { + if (pVal != NULL) { + return -1; + } + return streamFileStateGroupGetKVByCur(pCur, pKey, pVal, pVLen); +} + +void streamStateClearExpiredState(SStreamState* pState) { + clearExpiredState(pState->pFileState); +} + +void streamStateSetFillInfo(SStreamState* pState) { + setFillInfo(pState->pFileState); +} + +int32_t streamStateGetPrev(SStreamState* pState, const SWinKey* pKey, SWinKey* pResKey, void** pVal, int32_t* pVLen, + int32_t* pWinCode) { + return getRowStatePrevRow(pState->pFileState, pKey, pResKey, pVal, pVLen, pWinCode); +} diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 35cacc29dc..a044859b80 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -103,8 +103,9 @@ static SStreamUpstreamEpInfo* createStreamTaskEpInfo(const SStreamTask* pTask) { return pEpInfo; } -int32_t tNewStreamTask(int64_t streamId, int8_t taskLevel, SEpSet* pEpset, bool fillHistory, int64_t triggerParam, - SArray* pTaskList, bool hasFillhistory, int8_t subtableWithoutMd5, SStreamTask** p) { +int32_t tNewStreamTask(int64_t streamId, int8_t taskLevel, SEpSet* pEpset, bool fillHistory, int32_t trigger, + int64_t triggerParam, SArray* pTaskList, bool hasFillhistory, int8_t subtableWithoutMd5, + SStreamTask** p) { *p = NULL; SStreamTask* pTask = (SStreamTask*)taosMemoryCalloc(1, sizeof(SStreamTask)); @@ -120,6 +121,7 @@ int32_t tNewStreamTask(int64_t streamId, int8_t taskLevel, SEpSet* pEpset, bool pTask->info.taskLevel = taskLevel; pTask->info.fillHistory = fillHistory; + pTask->info.trigger = trigger; pTask->info.delaySchedParam = triggerParam; pTask->subtableWithoutMd5 = subtableWithoutMd5; diff --git a/source/libs/stream/src/streamUpdate.c b/source/libs/stream/src/streamUpdate.c index 60f2294c20..a3cfa00127 100644 --- a/source/libs/stream/src/streamUpdate.c +++ b/source/libs/stream/src/streamUpdate.c @@ -214,7 +214,6 @@ int32_t updateInfoInit(int64_t interval, int32_t precision, int64_t watermark, b if (pkLen != 0) { pInfo->comparePkRowFn = compareKeyTsAndPk; pInfo->comparePkCol = getKeyComparFunc(pkType, TSDB_ORDER_ASC); - ; } else { pInfo->comparePkRowFn = compareKeyTs; pInfo->comparePkCol = NULL; @@ -442,76 +441,69 @@ void updateInfoDestoryColseWinSBF(SUpdateInfo* pInfo) { pInfo->pCloseWinSBF = NULL; } -int32_t updateInfoSerialize(void* buf, int32_t bufLen, const SUpdateInfo* pInfo, int32_t* pLen) { +int32_t updateInfoSerialize(SEncoder* pEncoder, const SUpdateInfo* pInfo) { int32_t code = TSDB_CODE_SUCCESS; int32_t lino = 0; if (!pInfo) { return TSDB_CODE_SUCCESS; } - SEncoder encoder = {0}; - tEncoderInit(&encoder, buf, bufLen); - if (tStartEncode(&encoder) != 0) { - code = TSDB_CODE_FAILED; - QUERY_CHECK_CODE(code, lino, _end); - } - int32_t size = taosArrayGetSize(pInfo->pTsBuckets); - if (tEncodeI32(&encoder, size) < 0) { + if (tEncodeI32(pEncoder, size) < 0) { code = TSDB_CODE_FAILED; QUERY_CHECK_CODE(code, lino, _end); } for (int32_t i = 0; i < size; i++) { TSKEY* pTs = (TSKEY*)taosArrayGet(pInfo->pTsBuckets, i); - if (tEncodeI64(&encoder, *pTs) < 0) { + if (tEncodeI64(pEncoder, *pTs) < 0) { code = TSDB_CODE_FAILED; QUERY_CHECK_CODE(code, lino, _end); } } - if (tEncodeU64(&encoder, pInfo->numBuckets) < 0) { + if (tEncodeU64(pEncoder, pInfo->numBuckets) < 0) { code = TSDB_CODE_FAILED; QUERY_CHECK_CODE(code, lino, _end); } int32_t sBfSize = taosArrayGetSize(pInfo->pTsSBFs); - if (tEncodeI32(&encoder, sBfSize) < 0) { + if (tEncodeI32(pEncoder, sBfSize) < 0) { code = TSDB_CODE_FAILED; QUERY_CHECK_CODE(code, lino, _end); } for (int32_t i = 0; i < sBfSize; i++) { SScalableBf* pSBf = taosArrayGetP(pInfo->pTsSBFs, i); - if (tScalableBfEncode(pSBf, &encoder) < 0) { + if (tScalableBfEncode(pSBf, pEncoder) < 0) { code = TSDB_CODE_FAILED; QUERY_CHECK_CODE(code, lino, _end); } } - if (tEncodeU64(&encoder, pInfo->numSBFs) < 0) { + if (tEncodeU64(pEncoder, pInfo->numSBFs) < 0) { code = TSDB_CODE_FAILED; QUERY_CHECK_CODE(code, lino, _end); } - if (tEncodeI64(&encoder, pInfo->interval) < 0) { + if (tEncodeI64(pEncoder, pInfo->interval) < 0) { code = TSDB_CODE_FAILED; QUERY_CHECK_CODE(code, lino, _end); } - if (tEncodeI64(&encoder, pInfo->watermark) < 0) { + if (tEncodeI64(pEncoder, pInfo->watermark) < 0) { code = TSDB_CODE_FAILED; QUERY_CHECK_CODE(code, lino, _end); } - if (tEncodeI64(&encoder, pInfo->minTS) < 0) { + if (tEncodeI64(pEncoder, pInfo->minTS) < 0) { code = TSDB_CODE_FAILED; QUERY_CHECK_CODE(code, lino, _end); } - if (tScalableBfEncode(pInfo->pCloseWinSBF, &encoder) < 0) { + if (tScalableBfEncode(pInfo->pCloseWinSBF, pEncoder) < 0) { code = TSDB_CODE_FAILED; QUERY_CHECK_CODE(code, lino, _end); } int32_t mapSize = taosHashGetSize(pInfo->pMap); - if (tEncodeI32(&encoder, mapSize) < 0) { + if (tEncodeI32(pEncoder, mapSize) < 0) { code = TSDB_CODE_FAILED; QUERY_CHECK_CODE(code, lino, _end); } @@ -519,60 +511,51 @@ int32_t updateInfoSerialize(void* buf, int32_t bufLen, const SUpdateInfo* pInfo, size_t keyLen = 0; while ((pIte = taosHashIterate(pInfo->pMap, pIte)) != NULL) { void* key = taosHashGetKey(pIte, &keyLen); - if (tEncodeU64(&encoder, *(uint64_t*)key) < 0) { + if (tEncodeU64(pEncoder, *(uint64_t*)key) < 0) { code = TSDB_CODE_FAILED; QUERY_CHECK_CODE(code, lino, _end); } int32_t valueSize = taosHashGetValueSize(pIte); - if (tEncodeBinary(&encoder, (const uint8_t*)pIte, valueSize) < 0) { + if (tEncodeBinary(pEncoder, (const uint8_t*)pIte, valueSize) < 0) { code = TSDB_CODE_FAILED; QUERY_CHECK_CODE(code, lino, _end); } } - if (tEncodeU64(&encoder, pInfo->maxDataVersion) < 0) { + if (tEncodeU64(pEncoder, pInfo->maxDataVersion) < 0) { code = TSDB_CODE_FAILED; QUERY_CHECK_CODE(code, lino, _end); } - if (tEncodeI32(&encoder, pInfo->pkColLen) < 0) { + if (tEncodeI32(pEncoder, pInfo->pkColLen) < 0) { code = TSDB_CODE_FAILED; QUERY_CHECK_CODE(code, lino, _end); } - if (tEncodeI8(&encoder, pInfo->pkColType) < 0) { + if (tEncodeI8(pEncoder, pInfo->pkColType) < 0) { code = TSDB_CODE_FAILED; QUERY_CHECK_CODE(code, lino, _end); } - tEndEncode(&encoder); - - int32_t tlen = encoder.pos; - *pLen = tlen; - _end: - tEncoderClear(&encoder); if (code != TSDB_CODE_SUCCESS) { uError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); } return code; } -int32_t updateInfoDeserialize(void* buf, int32_t bufLen, SUpdateInfo* pInfo) { +int32_t updateInfoDeserialize(SDecoder* pDeCoder, SUpdateInfo* pInfo) { int32_t code = TSDB_CODE_SUCCESS; int32_t lino = 0; QUERY_CHECK_NULL(pInfo, code, lino, _error, TSDB_CODE_QRY_EXECUTOR_INTERNAL_ERROR); - SDecoder decoder = {0}; - tDecoderInit(&decoder, buf, bufLen); - if (tStartDecode(&decoder) < 0) return -1; - + int32_t size = 0; - if (tDecodeI32(&decoder, &size) < 0) return -1; + if (tDecodeI32(pDeCoder, &size) < 0) return -1; pInfo->pTsBuckets = taosArrayInit(size, sizeof(TSKEY)); QUERY_CHECK_NULL(pInfo->pTsBuckets, code, lino, _error, terrno); TSKEY ts = INT64_MIN; for (int32_t i = 0; i < size; i++) { - if (tDecodeI64(&decoder, &ts) < 0) return -1; + if (tDecodeI64(pDeCoder, &ts) < 0) return -1; void* tmp = taosArrayPush(pInfo->pTsBuckets, &ts); if (!tmp) { code = terrno; @@ -580,16 +563,16 @@ int32_t updateInfoDeserialize(void* buf, int32_t bufLen, SUpdateInfo* pInfo) { } } - if (tDecodeU64(&decoder, &pInfo->numBuckets) < 0) return -1; + if (tDecodeU64(pDeCoder, &pInfo->numBuckets) < 0) return -1; int32_t sBfSize = 0; - if (tDecodeI32(&decoder, &sBfSize) < 0) return -1; + if (tDecodeI32(pDeCoder, &sBfSize) < 0) return -1; pInfo->pTsSBFs = taosArrayInit(sBfSize, sizeof(void*)); QUERY_CHECK_NULL(pInfo->pTsSBFs, code, lino, _error, terrno); for (int32_t i = 0; i < sBfSize; i++) { SScalableBf* pSBf = NULL; - code = tScalableBfDecode(&decoder, &pSBf); + code = tScalableBfDecode(pDeCoder, &pSBf); QUERY_CHECK_CODE(code, lino, _error); void* tmp = taosArrayPush(pInfo->pTsSBFs, &pSBf); @@ -599,36 +582,36 @@ int32_t updateInfoDeserialize(void* buf, int32_t bufLen, SUpdateInfo* pInfo) { } } - if (tDecodeU64(&decoder, &pInfo->numSBFs) < 0) return -1; - if (tDecodeI64(&decoder, &pInfo->interval) < 0) return -1; - if (tDecodeI64(&decoder, &pInfo->watermark) < 0) return -1; - if (tDecodeI64(&decoder, &pInfo->minTS) < 0) return -1; + if (tDecodeU64(pDeCoder, &pInfo->numSBFs) < 0) return -1; + if (tDecodeI64(pDeCoder, &pInfo->interval) < 0) return -1; + if (tDecodeI64(pDeCoder, &pInfo->watermark) < 0) return -1; + if (tDecodeI64(pDeCoder, &pInfo->minTS) < 0) return -1; - code = tScalableBfDecode(&decoder, &pInfo->pCloseWinSBF); + code = tScalableBfDecode(pDeCoder, &pInfo->pCloseWinSBF); if (code != TSDB_CODE_SUCCESS) { pInfo->pCloseWinSBF = NULL; code = TSDB_CODE_SUCCESS; } int32_t mapSize = 0; - if (tDecodeI32(&decoder, &mapSize) < 0) return -1; + if (tDecodeI32(pDeCoder, &mapSize) < 0) return -1; _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_UBIGINT); pInfo->pMap = taosHashInit(mapSize, hashFn, true, HASH_NO_LOCK); uint64_t uid = 0; void* pVal = NULL; - int32_t valSize = 0; + uint32_t valSize = 0; for (int32_t i = 0; i < mapSize; i++) { - if (tDecodeU64(&decoder, &uid) < 0) return -1; - if (tDecodeBinary(&decoder, (uint8_t**)&pVal, &valSize) < 0) return -1; + if (tDecodeU64(pDeCoder, &uid) < 0) return -1; + if (tDecodeBinary(pDeCoder, (uint8_t**)&pVal, &valSize) < 0) return -1; code = taosHashPut(pInfo->pMap, &uid, sizeof(uint64_t), pVal, valSize); QUERY_CHECK_CODE(code, lino, _error); } QUERY_CHECK_CONDITION((mapSize == taosHashGetSize(pInfo->pMap)), code, lino, _error, TSDB_CODE_QRY_EXECUTOR_INTERNAL_ERROR); - if (tDecodeU64(&decoder, &pInfo->maxDataVersion) < 0) return -1; + if (tDecodeU64(pDeCoder, &pInfo->maxDataVersion) < 0) return -1; - if (tDecodeI32(&decoder, &pInfo->pkColLen) < 0) return -1; - if (tDecodeI8(&decoder, &pInfo->pkColType) < 0) return -1; + if (tDecodeI32(pDeCoder, &pInfo->pkColLen) < 0) return -1; + if (tDecodeI8(pDeCoder, &pInfo->pkColType) < 0) return -1; pInfo->pKeyBuff = taosMemoryCalloc(1, sizeof(TSKEY) + sizeof(int64_t) + pInfo->pkColLen); QUERY_CHECK_NULL(pInfo->pKeyBuff, code, lino, _error, terrno); @@ -644,10 +627,6 @@ int32_t updateInfoDeserialize(void* buf, int32_t bufLen, SUpdateInfo* pInfo) { pInfo->comparePkCol = NULL; } - tEndDecode(&decoder); - - tDecoderClear(&decoder); - _error: if (code != TSDB_CODE_SUCCESS) { uError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); diff --git a/source/libs/stream/src/tstreamFileState.c b/source/libs/stream/src/tstreamFileState.c index c630010598..dc4ca7c0e5 100644 --- a/source/libs/stream/src/tstreamFileState.c +++ b/source/libs/stream/src/tstreamFileState.c @@ -27,28 +27,36 @@ #define DEFAULT_MAX_STREAM_BUFFER_SIZE (128 * 1024 * 1024) #define MIN_NUM_OF_ROW_BUFF 10240 #define MIN_NUM_OF_RECOVER_ROW_BUFF 128 +#define MIN_NUM_SEARCH_BUCKET 128 +#define MAX_ARRAY_SIZE 1024 +#define MAX_GROUP_ID_NUM 200000 +#define NUM_OF_CACHE_WIN 64 +#define MAX_NUM_OF_CACHE_WIN 128 #define TASK_KEY "streamFileState" #define STREAM_STATE_INFO_NAME "StreamStateCheckPoint" struct SStreamFileState { - SList* usedBuffs; - SList* freeBuffs; - void* rowStateBuff; - void* pFileStore; - int32_t rowSize; - int32_t selectivityRowSize; - int32_t keyLen; - uint64_t preCheckPointVersion; - uint64_t checkPointVersion; - TSKEY maxTs; - TSKEY deleteMark; - TSKEY flushMark; - uint64_t maxRowCount; - uint64_t curRowCount; - GetTsFun getTs; - char* id; - char* cfName; + SList* usedBuffs; + SList* freeBuffs; + void* rowStateBuff; + void* pFileStore; + int32_t rowSize; + int32_t selectivityRowSize; + int32_t keyLen; + uint64_t preCheckPointVersion; + uint64_t checkPointVersion; + TSKEY maxTs; + TSKEY deleteMark; + TSKEY flushMark; + uint64_t maxRowCount; + uint64_t curRowCount; + GetTsFun getTs; + char* id; + char* cfName; + void* searchBuff; + SSHashObj* pGroupIdMap; + bool hasFillCatch; _state_buff_cleanup_fn stateBuffCleanupFn; _state_buff_remove_fn stateBuffRemoveFn; @@ -63,6 +71,11 @@ struct SStreamFileState { typedef SRowBuffPos SRowBuffInfo; +int fillStateKeyCompare(const void* pWin1, const void* pDatas, int pos) { + SWinKey* pWin2 = taosArrayGet(pDatas, pos); + return winKeyCmprImpl((SWinKey*)pWin1, pWin2); +} + int32_t stateHashBuffRemoveFn(void* pBuff, const void* pKey, size_t keyLen) { SRowBuffPos** pos = tSimpleHashGet(pBuff, pKey, keyLen); if (pos) { @@ -90,7 +103,7 @@ int32_t intervalFileRemoveFn(SStreamFileState* pFileState, const void* pKey) { return streamStateDel_rocksdb(pFileState->pFileStore, pKey); } -int32_t intervalFileGetFn(SStreamFileState* pFileState, void* pKey, void* data, int32_t* pDataLen) { +int32_t intervalFileGetFn(SStreamFileState* pFileState, void* pKey, void** data, int32_t* pDataLen) { return streamStateGet_rocksdb(pFileState->pFileStore, pKey, data, pDataLen); } @@ -106,11 +119,22 @@ void* intervalCreateStateKey(SRowBuffPos* pPos, int64_t num) { return pStateKey; } +void* defaultCreateStateKey(SRowBuffPos* pPos, int64_t num) { + SWinKey* pStateKey = taosMemoryCalloc(1, sizeof(SWinKey)); + if (pStateKey == NULL) { + qError("%s failed at line %d since %s", __func__, __LINE__, tstrerror(terrno)); + return NULL; + } + SWinKey* pWinKey = pPos->pKey; + *pStateKey = *pWinKey; + return pStateKey; +} + int32_t sessionFileRemoveFn(SStreamFileState* pFileState, const void* pKey) { return streamStateSessionDel_rocksdb(pFileState->pFileStore, pKey); } -int32_t sessionFileGetFn(SStreamFileState* pFileState, void* pKey, void* data, int32_t* pDataLen) { +int32_t sessionFileGetFn(SStreamFileState* pFileState, void* pKey, void** data, int32_t* pDataLen) { return streamStateSessionGet_rocksdb(pFileState->pFileStore, pKey, data, pDataLen); } @@ -150,23 +174,23 @@ int32_t streamFileStateInit(int64_t memSize, uint32_t keySize, uint32_t rowSize, } if (rowSize == 0) { code = TSDB_CODE_INVALID_PARA; - goto _error; + QUERY_CHECK_CODE(code, lino, _end); } SStreamFileState* pFileState = taosMemoryCalloc(1, sizeof(SStreamFileState)); - QUERY_CHECK_NULL(pFileState, code, lino, _error, terrno); + QUERY_CHECK_NULL(pFileState, code, lino, _end, terrno); rowSize += selectRowSize; pFileState->maxRowCount = TMAX((uint64_t)memSize / rowSize, FLUSH_NUM * 2); pFileState->usedBuffs = tdListNew(POINTER_BYTES); - QUERY_CHECK_NULL(pFileState->usedBuffs, code, lino, _error, terrno); + QUERY_CHECK_NULL(pFileState->usedBuffs, code, lino, _end, terrno); pFileState->freeBuffs = tdListNew(POINTER_BYTES); - QUERY_CHECK_NULL(pFileState->freeBuffs, code, lino, _error, terrno); + QUERY_CHECK_NULL(pFileState->freeBuffs, code, lino, _end, terrno); _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); int32_t cap = TMIN(MIN_NUM_OF_ROW_BUFF, pFileState->maxRowCount); - if (type == STREAM_STATE_BUFF_HASH) { + if (type == STREAM_STATE_BUFF_HASH || type == STREAM_STATE_BUFF_HASH_SEARCH) { pFileState->rowStateBuff = tSimpleHashInit(cap, hashFn); pFileState->stateBuffCleanupFn = stateHashBuffCleanupFn; pFileState->stateBuffRemoveFn = stateHashBuffRemoveFn; @@ -176,8 +200,8 @@ int32_t streamFileStateInit(int64_t memSize, uint32_t keySize, uint32_t rowSize, pFileState->stateFileRemoveFn = intervalFileRemoveFn; pFileState->stateFileGetFn = intervalFileGetFn; pFileState->cfName = taosStrdup("state"); - pFileState->stateFunctionGetFn = getRowBuff; - } else { + pFileState->stateFunctionGetFn = addRowBuffIfNotExist; + } else if (type == STREAM_STATE_BUFF_SORT) { pFileState->rowStateBuff = tSimpleHashInit(cap, hashFn); pFileState->stateBuffCleanupFn = sessionWinStateCleanup; pFileState->stateBuffRemoveFn = deleteSessionWinStateBuffFn; @@ -188,9 +212,30 @@ int32_t streamFileStateInit(int64_t memSize, uint32_t keySize, uint32_t rowSize, pFileState->stateFileGetFn = sessionFileGetFn; pFileState->cfName = taosStrdup("sess"); pFileState->stateFunctionGetFn = getSessionRowBuff; + } else if (type == STREAM_STATE_BUFF_HASH_SORT) { + pFileState->rowStateBuff = tSimpleHashInit(cap, hashFn); + pFileState->searchBuff = tSimpleHashInit(MIN_NUM_SEARCH_BUCKET, hashFn); + QUERY_CHECK_NULL(pFileState->searchBuff, code, lino, _end, terrno); + pFileState->stateBuffCleanupFn = stateHashBuffCleanupFn; + pFileState->stateBuffRemoveFn = stateHashBuffRemoveFn; + pFileState->stateBuffRemoveByPosFn = stateHashBuffRemoveByPosFn; + pFileState->stateBuffCreateStateKeyFn = defaultCreateStateKey; + + pFileState->stateFileRemoveFn = hashSortFileRemoveFn; + pFileState->stateFileGetFn = hashSortFileGetFn; + pFileState->cfName = taosStrdup("fill"); + pFileState->stateFunctionGetFn = NULL; + } + + QUERY_CHECK_NULL(pFileState->usedBuffs, code, lino, _end, terrno); + QUERY_CHECK_NULL(pFileState->freeBuffs, code, lino, _end, terrno); + QUERY_CHECK_NULL(pFileState->rowStateBuff, code, lino, _end, terrno); + QUERY_CHECK_NULL(pFileState->cfName, code, lino, _end, terrno); + + if (type == STREAM_STATE_BUFF_HASH_SEARCH) { + pFileState->searchBuff = tSimpleHashInit(MIN_NUM_SEARCH_BUCKET, hashFn); + QUERY_CHECK_NULL(pFileState->searchBuff, code, lino, _end, terrno); } - QUERY_CHECK_NULL(pFileState->rowStateBuff, code, lino, _error, terrno); - QUERY_CHECK_NULL(pFileState->cfName, code, lino, _error, terrno); pFileState->keyLen = keySize; pFileState->rowSize = rowSize; @@ -204,28 +249,34 @@ int32_t streamFileStateInit(int64_t memSize, uint32_t keySize, uint32_t rowSize, pFileState->flushMark = INT64_MIN; pFileState->maxTs = INT64_MIN; pFileState->id = taosStrdup(taskId); - QUERY_CHECK_NULL(pFileState->id, code, lino, _error, terrno); + QUERY_CHECK_NULL(pFileState->id, code, lino, _end, terrno); - // todo(liuyao) optimize - if (type == STREAM_STATE_BUFF_HASH) { + pFileState->pGroupIdMap = tSimpleHashInit(1024, hashFn); + QUERY_CHECK_NULL(pFileState->pGroupIdMap, code, lino, _end, terrno); + + pFileState->hasFillCatch = true; + + if (type == STREAM_STATE_BUFF_HASH || type == STREAM_STATE_BUFF_HASH_SEARCH) { code = recoverSnapshot(pFileState, checkpointId); - } else { + } else if (type == STREAM_STATE_BUFF_SORT) { code = recoverSesssion(pFileState, checkpointId); + } else if (type == STREAM_STATE_BUFF_HASH_SORT) { + code = recoverFillSnapshot(pFileState, checkpointId); } - QUERY_CHECK_CODE(code, lino, _error); + QUERY_CHECK_CODE(code, lino, _end); void* valBuf = NULL; int32_t len = 0; int32_t tmpRes = streamDefaultGet_rocksdb(pFileState->pFileStore, STREAM_STATE_INFO_NAME, &valBuf, &len); if (tmpRes == TSDB_CODE_SUCCESS) { - QUERY_CHECK_CONDITION((len == sizeof(TSKEY)), code, lino, _error, TSDB_CODE_QRY_EXECUTOR_INTERNAL_ERROR); + QUERY_CHECK_CONDITION((len == sizeof(TSKEY)), code, lino, _end, TSDB_CODE_QRY_EXECUTOR_INTERNAL_ERROR); streamFileStateDecode(&pFileState->flushMark, valBuf, len); qDebug("===stream===flushMark read:%" PRId64, pFileState->flushMark); } taosMemoryFreeClear(valBuf); (*ppFileState) = pFileState; -_error: +_end: if (code != TSDB_CODE_SUCCESS) { streamFileStateDestroy(pFileState); qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); @@ -274,6 +325,8 @@ void streamFileStateDestroy(SStreamFileState* pFileState) { tdListFreeP(pFileState->usedBuffs, destroyRowBuffAllPosPtr); tdListFreeP(pFileState->freeBuffs, destroyRowBuff); pFileState->stateBuffCleanupFn(pFileState->rowStateBuff); + sessionWinStateCleanup(pFileState->searchBuff); + tSimpleHashCleanup(pFileState->pGroupIdMap); taosMemoryFree(pFileState); } @@ -321,7 +374,7 @@ _end: } } -int32_t clearFlushedRowBuff(SStreamFileState* pFileState, SStreamSnapshot* pFlushList, uint64_t max) { +int32_t clearFlushedRowBuff(SStreamFileState* pFileState, SStreamSnapshot* pFlushList, uint64_t max, bool all) { int32_t code = TSDB_CODE_SUCCESS; int32_t lino = 0; uint64_t i = 0; @@ -331,16 +384,21 @@ int32_t clearFlushedRowBuff(SStreamFileState* pFileState, SStreamSnapshot* pFlus SListNode* pNode = NULL; while ((pNode = tdListNext(&iter)) != NULL && i < max) { SRowBuffPos* pPos = *(SRowBuffPos**)pNode->data; - if (isFlushedState(pFileState, pFileState->getTs(pPos->pKey), 0) && !pPos->beUsed) { - code = tdListAppend(pFlushList, &pPos); - QUERY_CHECK_CODE(code, lino, _end); + if (isFlushedState(pFileState, pFileState->getTs(pPos->pKey), 0)) { + if (all || !pPos->beUsed) { + if (all && !pPos->pRowBuff) { + continue; + } + code = tdListAppend(pFlushList, &pPos); + QUERY_CHECK_CODE(code, lino, _end); - pFileState->flushMark = TMAX(pFileState->flushMark, pFileState->getTs(pPos->pKey)); - pFileState->stateBuffRemoveByPosFn(pFileState, pPos); - SListNode* tmp = tdListPopNode(pFileState->usedBuffs, pNode); - taosMemoryFreeClear(tmp); - if (pPos->pRowBuff) { - i++; + pFileState->flushMark = TMAX(pFileState->flushMark, pFileState->getTs(pPos->pKey)); + pFileState->stateBuffRemoveByPosFn(pFileState, pPos); + SListNode* tmp = tdListPopNode(pFileState->usedBuffs, pNode); + taosMemoryFreeClear(tmp); + if (pPos->pRowBuff) { + i++; + } } } } @@ -411,7 +469,7 @@ int32_t flushRowBuff(SStreamFileState* pFileState) { uint64_t num = (uint64_t)(pFileState->curRowCount * FLUSH_RATIO); num = TMAX(num, FLUSH_NUM); - code = clearFlushedRowBuff(pFileState, pFlushList, num); + code = clearFlushedRowBuff(pFileState, pFlushList, num, false); QUERY_CHECK_CODE(code, lino, _end); if (isListEmpty(pFlushList)) { @@ -424,6 +482,11 @@ int32_t flushRowBuff(SStreamFileState* pFileState) { } } + if (pFileState->searchBuff) { + code = clearFlushedRowBuff(pFileState, pFlushList, pFileState->curRowCount, true); + QUERY_CHECK_CODE(code, lino, _end); + } + flushSnapshot(pFileState, pFlushList, false); SListIter fIter = {0}; @@ -542,18 +605,20 @@ _error: return NULL; } -int32_t getRowBuff(SStreamFileState* pFileState, void* pKey, int32_t keyLen, void** pVal, int32_t* pVLen, - int32_t* pWinCode) { +int32_t addRowBuffIfNotExist(SStreamFileState* pFileState, void* pKey, int32_t keyLen, void** pVal, int32_t* pVLen, + int32_t* pWinCode) { int32_t code = TSDB_CODE_SUCCESS; int32_t lino = 0; (*pWinCode) = TSDB_CODE_SUCCESS; pFileState->maxTs = TMAX(pFileState->maxTs, pFileState->getTs(pKey)); SRowBuffPos** pos = tSimpleHashGet(pFileState->rowStateBuff, pKey, keyLen); if (pos) { - *pVLen = pFileState->rowSize; - *pVal = *pos; - (*pos)->beUsed = true; - (*pos)->beFlushed = false; + if (pVal != NULL) { + *pVLen = pFileState->rowSize; + *pVal = *pos; + (*pos)->beUsed = true; + (*pos)->beFlushed = false; + } goto _end; } SRowBuffPos* pNewPos = getNewRowPosForWrite(pFileState); @@ -569,7 +634,7 @@ int32_t getRowBuff(SStreamFileState* pFileState, void* pKey, int32_t keyLen, voi if (!isDeteled(pFileState, ts) && isFlushedState(pFileState, ts, 0)) { int32_t len = 0; void* p = NULL; - (*pWinCode) = streamStateGet_rocksdb(pFileState->pFileStore, pKey, &p, &len); + (*pWinCode) = pFileState->stateFileGetFn(pFileState, pKey, &p, &len); qDebug("===stream===get %" PRId64 " from disc, res %d", ts, (*pWinCode)); if ((*pWinCode) == TSDB_CODE_SUCCESS) { memcpy(pNewPos->pRowBuff, p, len); @@ -597,11 +662,17 @@ void deleteRowBuff(SStreamFileState* pFileState, const void* pKey, int32_t keyLe qTrace("%s at line %d res:%d", __func__, __LINE__, code_buff); int32_t code_file = pFileState->stateFileRemoveFn(pFileState, pKey); qTrace("%s at line %d res:%d", __func__, __LINE__, code_file); + if (pFileState->searchBuff != NULL) { + deleteHashSortRowBuff(pFileState, pKey); + } } int32_t resetRowBuff(SStreamFileState* pFileState, const void* pKey, int32_t keyLen) { int32_t code_buff = pFileState->stateBuffRemoveFn(pFileState->rowStateBuff, pKey, keyLen); int32_t code_file = pFileState->stateFileRemoveFn(pFileState, pKey); + if (pFileState->searchBuff != NULL) { + deleteHashSortRowBuff(pFileState, pKey); + } if (code_buff == TSDB_CODE_SUCCESS || code_file == TSDB_CODE_SUCCESS) { return TSDB_CODE_SUCCESS; } @@ -625,18 +696,9 @@ _end: return code; } -int32_t getRowBuffByPos(SStreamFileState* pFileState, SRowBuffPos* pPos, void** pVal) { +static int32_t recoverStateRowBuff(SStreamFileState* pFileState, SRowBuffPos* pPos) { int32_t code = TSDB_CODE_SUCCESS; int32_t lino = 0; - if (pPos->pRowBuff) { - if (pPos->needFree) { - code = recoverSessionRowBuff(pFileState, pPos); - QUERY_CHECK_CODE(code, lino, _end); - } - (*pVal) = pPos->pRowBuff; - goto _end; - } - pPos->pRowBuff = getFreeBuff(pFileState); if (!pPos->pRowBuff) { if (pFileState->curRowCount < pFileState->maxRowCount) { @@ -657,9 +719,32 @@ int32_t getRowBuffByPos(SStreamFileState* pFileState, SRowBuffPos* pPos, void** code = recoverSessionRowBuff(pFileState, pPos); QUERY_CHECK_CODE(code, lino, _end); +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +int32_t getRowBuffByPos(SStreamFileState* pFileState, SRowBuffPos* pPos, void** pVal) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + if (pPos->pRowBuff) { + if (pPos->needFree) { + code = recoverSessionRowBuff(pFileState, pPos); + QUERY_CHECK_CODE(code, lino, _end); + } + (*pVal) = pPos->pRowBuff; + goto _end; + } + + code = recoverStateRowBuff(pFileState, pPos); + QUERY_CHECK_CODE(code, lino, _end); + (*pVal) = pPos->pRowBuff; if (!pPos->needFree) { code = tdListPrepend(pFileState->usedBuffs, &pPos); + QUERY_CHECK_CODE(code, lino, _end); } _end: @@ -748,6 +833,8 @@ void flushSnapshot(SStreamFileState* pFileState, SStreamSnapshot* pSnapshot, boo streamStateClearBatch(batch); + clearSearchBuff(pFileState); + int64_t elapsed = taosGetTimestampMs() - st; qDebug("%s flush to disk in batch model completed, rows:%d, batch size:%d, elapsed time:%" PRId64 "ms", pFileState->id, numOfElems, BATCH_LIMIT, elapsed); @@ -914,6 +1001,7 @@ int32_t recoverSnapshot(SStreamFileState* pFileState, int64_t ckId) { if (vlen != pFileState->rowSize) { qError("row size mismatch, expect:%d, actual:%d", pFileState->rowSize, vlen); code = TSDB_CODE_QRY_EXECUTOR_INTERNAL_ERROR; + taosMemoryFreeClear(pVal); QUERY_CHECK_CODE(code, lino, _end); } memcpy(pNewPos->pRowBuff, pVal, vlen); @@ -943,6 +1031,7 @@ void streamFileStateReloadInfo(SStreamFileState* pFileState, TSKEY ts) { } void* getRowStateBuff(SStreamFileState* pFileState) { return pFileState->rowStateBuff; } +void* getSearchBuff(SStreamFileState* pFileState) { return pFileState->searchBuff; } void* getStateFileStore(SStreamFileState* pFileState) { return pFileState->pFileStore; } @@ -953,9 +1042,394 @@ bool isDeteled(SStreamFileState* pFileState, TSKEY ts) { bool isFlushedState(SStreamFileState* pFileState, TSKEY ts, TSKEY gap) { return ts <= (pFileState->flushMark + gap); } +TSKEY getFlushMark(SStreamFileState* pFileState) { return pFileState->flushMark; }; + int32_t getRowStateRowSize(SStreamFileState* pFileState) { return pFileState->rowSize; } int32_t getFunctionRowBuff(SStreamFileState* pFileState, void* pKey, int32_t keyLen, void** pVal, int32_t* pVLen) { int32_t winCode = TSDB_CODE_SUCCESS; return pFileState->stateFunctionGetFn(pFileState, pKey, keyLen, pVal, pVLen, &winCode); } + +int32_t recoverFillSnapshot(SStreamFileState* pFileState, int64_t ckId) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + if (pFileState->maxTs != INT64_MIN) { + int64_t mark = (INT64_MIN + pFileState->deleteMark >= pFileState->maxTs) + ? INT64_MIN + : pFileState->maxTs - pFileState->deleteMark; + code = deleteExpiredCheckPoint(pFileState, mark); + QUERY_CHECK_CODE(code, lino, _end); + } + + SStreamStateCur* pCur = streamStateFillSeekToLast_rocksdb(pFileState->pFileStore); + if (pCur == NULL) { + return code; + } + int32_t recoverNum = TMIN(MIN_NUM_OF_RECOVER_ROW_BUFF, pFileState->maxRowCount); + int32_t winRes = TSDB_CODE_SUCCESS; + while (winRes == TSDB_CODE_SUCCESS) { + if (pFileState->curRowCount >= recoverNum) { + break; + } + + void* pVal = NULL; + int32_t vlen = 0; + SRowBuffPos* pNewPos = getNewRowPosForWrite(pFileState); + winRes = streamStateFillGetKVByCur_rocksdb(pCur, pNewPos->pKey, (const void**)&pVal, &vlen); + if (winRes != TSDB_CODE_SUCCESS || isFlushedState(pFileState, pFileState->getTs(pNewPos->pKey), 0)) { + destroyRowBuffPos(pNewPos); + SListNode* pNode = tdListPopTail(pFileState->usedBuffs); + taosMemoryFreeClear(pNode); + taosMemoryFreeClear(pVal); + break; + } + + memcpy(pNewPos->pRowBuff, pVal, vlen); + taosMemoryFreeClear(pVal); + pNewPos->beFlushed = true; + winRes = tSimpleHashPut(pFileState->rowStateBuff, pNewPos->pKey, pFileState->keyLen, &pNewPos, POINTER_BYTES); + if (winRes != TSDB_CODE_SUCCESS) { + destroyRowBuffPos(pNewPos); + break; + } + streamStateCurPrev_rocksdb(pCur); + } + streamStateFreeCur(pCur); + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +int32_t getRowBuff(SStreamFileState* pFileState, void* pKey, int32_t keyLen, void** pVal, int32_t* pVLen, + int32_t* pWinCode) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + (*pWinCode) = TSDB_CODE_FAILED; + pFileState->maxTs = TMAX(pFileState->maxTs, pFileState->getTs(pKey)); + SRowBuffPos** ppPos = tSimpleHashGet(pFileState->rowStateBuff, pKey, keyLen); + if (ppPos) { + *pVLen = pFileState->rowSize; + *pVal = *ppPos; + (*ppPos)->beUsed = true; + (*ppPos)->beFlushed = false; + (*pWinCode) = TSDB_CODE_SUCCESS; + if ((*ppPos)->pRowBuff == NULL) { + code = recoverStateRowBuff(pFileState, *ppPos); + QUERY_CHECK_CODE(code, lino, _end); + } + goto _end; + } + TSKEY ts = pFileState->getTs(pKey); + if (!isDeteled(pFileState, ts) && isFlushedState(pFileState, ts, 0)) { + int32_t len = 0; + void* p = NULL; + (*pWinCode) = pFileState->stateFileGetFn(pFileState, pKey, &p, &len); + qDebug("===stream===get %" PRId64 " from disc, res %d", ts, (*pWinCode)); + if ((*pWinCode) == TSDB_CODE_SUCCESS) { + SRowBuffPos* pNewPos = getNewRowPosForWrite(pFileState); + if (!pNewPos || !pNewPos->pRowBuff) { + code = TSDB_CODE_OUT_OF_MEMORY; + QUERY_CHECK_CODE(code, lino, _end); + } + + memcpy(pNewPos->pKey, pKey, keyLen); + memcpy(pNewPos->pRowBuff, p, len); + code = tSimpleHashPut(pFileState->rowStateBuff, pKey, keyLen, &pNewPos, POINTER_BYTES); + QUERY_CHECK_CODE(code, lino, _end); + + if (pVal) { + *pVLen = pFileState->rowSize; + *pVal = pNewPos; + } + } + taosMemoryFree(p); + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +int32_t streamFileStateGroupPut(SStreamFileState* pFileState, int64_t groupId, void* value, int32_t vLen) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + if (value != NULL) { + code = TSDB_CODE_INVALID_PARA; + QUERY_CHECK_CODE(code, lino, _end); + } + + if (tSimpleHashGet(pFileState->pGroupIdMap, &groupId, sizeof(int64_t)) == NULL) { + if (tSimpleHashGetSize(pFileState->pGroupIdMap) <= MAX_GROUP_ID_NUM) { + code = tSimpleHashPut(pFileState->pGroupIdMap, &groupId, sizeof(int64_t), NULL, 0); + QUERY_CHECK_CODE(code, lino, _end); + } + code = streamStatePutParTag_rocksdb(pFileState->pFileStore, groupId, value, vLen); + QUERY_CHECK_CODE(code, lino, _end); + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +void streamFileStateGroupCurNext(SStreamStateCur* pCur) { + SStreamFileState* pFileState = (SStreamFileState*)pCur->pStreamFileState; + if (pCur->hashIter == -1) { + streamStateCurNext(pFileState->pFileStore, pCur); + return; + } + + int64_t gpId = *(int64_t*)tSimpleHashGetKey(pCur->pHashData, NULL); + pCur->minGpId = TMAX(pCur->minGpId, gpId); + + SSHashObj* pHash = pFileState->pGroupIdMap; + pCur->pHashData = tSimpleHashIterate(pHash, pCur->pHashData, &pCur->hashIter); + if (!pCur->pHashData) { + pCur->hashIter = -1; + streamStateParTagSeekKeyNext_rocksdb(pFileState->pFileStore, pCur->minGpId, pCur); + return; + } +} + +int32_t streamFileStateGroupGetKVByCur(SStreamStateCur* pCur, int64_t* pKey, void** pVal, int32_t* pVLen) { + int32_t code = TSDB_CODE_SUCCESS; + if (pCur->pHashData) { + *pKey = *(int64_t*)tSimpleHashGetKey(pCur->pHashData, NULL); + return code; + } + return streamStateParTagGetKVByCur_rocksdb(pCur, pKey, NULL, NULL); +} + +SSHashObj* getGroupIdCache(SStreamFileState* pFileState) { + return pFileState->pGroupIdMap; +} + +void setFillInfo(SStreamFileState* pFileState) { + pFileState->hasFillCatch = false; +} + +void clearExpiredState(SStreamFileState* pFileState) { + SSHashObj* pSearchBuff = pFileState->searchBuff; + void* pIte = NULL; + int32_t iter = 0; + while ((pIte = tSimpleHashIterate(pSearchBuff, pIte, &iter)) != NULL) { + SArray* pWinStates = *((void**)pIte); + int32_t size = taosArrayGetSize(pWinStates); + for (int32_t i = 0; i < size - 1; i++) { + SWinKey* pKey = taosArrayGet(pWinStates, i); + int32_t code_buff = pFileState->stateBuffRemoveFn(pFileState->rowStateBuff, pKey, sizeof(SWinKey)); + qTrace("clear expired buff, ts:%" PRId64 ". %s at line %d res:%d", pKey->ts, __func__, __LINE__, code_buff); + + if (isFlushedState(pFileState, pKey->ts, 0)) { + int32_t code_file = pFileState->stateFileRemoveFn(pFileState, pKey); + qTrace("clear expired file, ts:%" PRId64 ". %s at line %d res:%d", pKey->ts, __func__, __LINE__, code_file); + } + + if (pFileState->hasFillCatch == false) { + int32_t code_file = streamStateFillDel_rocksdb(pFileState->pFileStore, pKey); + qTrace("force clear expired file, ts:%" PRId64 ". %s at line %d res %d", pKey->ts, __func__, __LINE__, code_file); + } + } + taosArrayRemoveBatch(pWinStates, 0, size - 1, NULL); + } +} + +int32_t getStateSearchRowBuff(SStreamFileState* pFileState, const SWinKey* pKey, void** pVal, int32_t* pVLen, + int32_t* pWinCode) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + + code = addRowBuffIfNotExist(pFileState, (void*)pKey, sizeof(SWinKey), pVal, pVLen, pWinCode); + QUERY_CHECK_CODE(code, lino, _end); + + SArray* pWinStates = NULL; + SSHashObj* pSearchBuff = getSearchBuff(pFileState); + void** ppBuff = tSimpleHashGet(pSearchBuff, &pKey->groupId, sizeof(uint64_t)); + if (ppBuff) { + pWinStates = (SArray*)(*ppBuff); + } else { + pWinStates = taosArrayInit(16, sizeof(SWinKey)); + QUERY_CHECK_NULL(pWinStates, code, lino, _end, terrno); + + code = tSimpleHashPut(pSearchBuff, &pKey->groupId, sizeof(uint64_t), &pWinStates, POINTER_BYTES); + QUERY_CHECK_CODE(code, lino, _end); + } + + // recover + if (taosArrayGetSize(pWinStates) == 0 && needClearDiskBuff(pFileState)) { + TSKEY ts = getFlushMark(pFileState); + SWinKey start = {.groupId = pKey->groupId, .ts = INT64_MAX}; + void* pState = getStateFileStore(pFileState); + SStreamStateCur* pCur = streamStateSeekKeyPrev_rocksdb(pState, &start); + for (int32_t i = 0; i < NUM_OF_CACHE_WIN; i++) { + SWinKey tmpKey = {.groupId = pKey->groupId}; + int32_t tmpRes = streamStateGetGroupKVByCur_rocksdb(pState, pCur, &tmpKey, NULL, 0); + if (tmpRes != TSDB_CODE_SUCCESS) { + break; + } + void* tmp = taosArrayPush(pWinStates, &tmpKey); + QUERY_CHECK_NULL(tmp, code, lino, _end, terrno); + streamStateCurPrev_rocksdb(pCur); + } + taosArraySort(pWinStates, winKeyCmprImpl); + streamStateFreeCur(pCur); + } + + int32_t size = taosArrayGetSize(pWinStates); + int32_t index = binarySearch(pWinStates, size, pKey, fillStateKeyCompare); + if (!isFlushedState(pFileState, pKey->ts, 0)|| index >= 0) { + // find the first position which is smaller than the pKey + if (index >= 0) { + SWinKey* pTmpKey = taosArrayGet(pWinStates, index); + if (winKeyCmprImpl(pTmpKey, pKey) == 0) { + goto _end; + } + } + index++; + void* tmp = taosArrayInsert(pWinStates, index, pKey); + QUERY_CHECK_NULL(tmp, code, lino, _end, terrno); + } + + if (size >= MAX_NUM_OF_CACHE_WIN) { + int32_t num = size - NUM_OF_CACHE_WIN; + taosArrayRemoveBatch(pWinStates, 0, num, NULL); + } + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +int32_t getRowStatePrevRow(SStreamFileState* pFileState, const SWinKey* pKey, SWinKey* pResKey, void** ppVal, + int32_t* pVLen, int32_t* pWinCode) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + SArray* pWinStates = NULL; + SSHashObj* pSearchBuff = getSearchBuff(pFileState); + void* pState = getStateFileStore(pFileState); + void** ppBuff = (void**) tSimpleHashGet(pSearchBuff, &pKey->groupId, sizeof(uint64_t)); + if (ppBuff) { + pWinStates = (SArray*)(*ppBuff); + } else { + qDebug("===stream=== search buff is empty.group id:%" PRId64, pKey->groupId); + SStreamStateCur* pCur = streamStateSeekKeyPrev_rocksdb(pState, pKey); + void* tmpVal = NULL; + int32_t len = 0; + (*pWinCode) = streamStateGetGroupKVByCur_rocksdb(pState, pCur, pResKey, (const void**)&tmpVal, &len); + if ((*pWinCode) == TSDB_CODE_SUCCESS) { + SRowBuffPos* pNewPos = getNewRowPosForWrite(pFileState); + if (!pNewPos || !pNewPos->pRowBuff) { + code = TSDB_CODE_OUT_OF_MEMORY; + QUERY_CHECK_CODE(code, lino, _end); + } + memcpy(pNewPos->pRowBuff, tmpVal, len); + taosMemoryFreeClear(tmpVal); + *pVLen = getRowStateRowSize(pFileState); + (*ppVal) = pNewPos; + } + streamStateFreeCur(pCur); + return code; + } + int32_t size = taosArrayGetSize(pWinStates); + int32_t index = binarySearch(pWinStates, size, pKey, fillStateKeyCompare); + if (index >= 0) { + SWinKey* pCurKey = taosArrayGet(pWinStates, index); + if (winKeyCmprImpl(pCurKey, pKey) == 0) { + index--; + } else { + qDebug("%s failed at line %d since do not find cur SWinKey. trigger may be force window close", __func__, __LINE__); + } + } + if (index == -1) { + SStreamStateCur* pCur = streamStateSeekKeyPrev_rocksdb(pState, pKey); + void* tmpVal = NULL; + int32_t len = 0; + (*pWinCode) = streamStateGetGroupKVByCur_rocksdb(pState, pCur, pResKey, (const void**)&tmpVal, &len); + if ((*pWinCode) == TSDB_CODE_SUCCESS) { + SRowBuffPos* pNewPos = getNewRowPosForWrite(pFileState); + if (!pNewPos || !pNewPos->pRowBuff) { + code = TSDB_CODE_OUT_OF_MEMORY; + QUERY_CHECK_CODE(code, lino, _end); + } + memcpy(pNewPos->pRowBuff, tmpVal, len); + taosMemoryFreeClear(tmpVal); + *pVLen = getRowStateRowSize(pFileState); + (*ppVal) = pNewPos; + } + streamStateFreeCur(pCur); + return code; + } else { + SWinKey* pPrevKey = taosArrayGet(pWinStates, index); + *pResKey = *pPrevKey; + return addRowBuffIfNotExist(pFileState, (void*)pPrevKey, sizeof(SWinKey), ppVal, pVLen, pWinCode); + } + (*pWinCode) = TSDB_CODE_FAILED; + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +int32_t addSearchItem(SStreamFileState* pFileState, SArray* pWinStates, const SWinKey* pKey) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + int32_t size = taosArrayGetSize(pWinStates); + int32_t index = binarySearch(pWinStates, size, pKey, fillStateKeyCompare); + if (!isFlushedState(pFileState, pKey->ts, 0) || index >= 0 || size == 0) { + if (index >= 0) { + SWinKey* pTmpKey = taosArrayGet(pWinStates, index); + if (winKeyCmprImpl(pTmpKey, pKey) == 0) { + goto _end; + } + } + index++; + void* tmp = taosArrayInsert(pWinStates, index, pKey); + QUERY_CHECK_NULL(tmp, code, lino, _end, terrno); + } + + if (size >= MAX_NUM_OF_CACHE_WIN) { + int32_t num = size - NUM_OF_CACHE_WIN; + taosArrayRemoveBatch(pWinStates, 0, num, NULL); + } +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} + +int32_t addArrayBuffIfNotExist(SSHashObj* pSearchBuff, uint64_t groupId, SArray** ppResStates) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + SArray* pWinStates = NULL; + void** ppBuff = tSimpleHashGet(pSearchBuff, &groupId, sizeof(uint64_t)); + if (ppBuff) { + pWinStates = (SArray*)(*ppBuff); + } else { + pWinStates = taosArrayInit(16, sizeof(SWinKey)); + QUERY_CHECK_NULL(pWinStates, code, lino, _end, terrno); + + code = tSimpleHashPut(pSearchBuff, &groupId, sizeof(uint64_t), &pWinStates, POINTER_BYTES); + QUERY_CHECK_CODE(code, lino, _end); + } + + (*ppResStates) = pWinStates; + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + return code; +} diff --git a/tests/army/query/function/test_resinfo.py b/tests/army/query/function/test_resinfo.py index 51d51f3ce1..5a59ed45cd 100644 --- a/tests/army/query/function/test_resinfo.py +++ b/tests/army/query/function/test_resinfo.py @@ -26,7 +26,7 @@ from frame.sql import * from frame.caseBase import * from frame import * -initial_hash_resinfoInt = "e739cde34b98f13dd9ad696d18f060cc" +initial_hash_resinfoInt = "fbfd69d6f0aa6e015a7b5475b33ee8c8" initial_hash_resinfo = "172d04aa7af0d8cd2e4d9df284079958" class TDTestCase(TBase): @@ -43,6 +43,7 @@ class TDTestCase(TBase): resinfoIntFile = etool.curFile(__file__, "../../../../source/libs/function/inc/functionResInfoInt.h") resinfoFile = etool.curFile(__file__, "../../../../include/libs/function/functionResInfo.h") current_hash = self.get_file_hash(resinfoIntFile) + tdLog.info(current_hash) if current_hash != initial_hash_resinfoInt: tdLog.exit(f"{resinfoIntFile} has been modified.") else: diff --git a/tests/ci/count_assert.py b/tests/ci/count_assert.py index aecc57578d..65a3192aba 100644 --- a/tests/ci/count_assert.py +++ b/tests/ci/count_assert.py @@ -40,6 +40,7 @@ exclude_dirs = [ # List of files to exclude exclude_source_files = [ f"{TD_project_path}/community/source/libs/parser/src/sql.c", + f"{TD_project_path}/community/source/libs/parser/inc/sql.c", f"{TD_project_path}/community/source/util/src/tlog.c", f"{TD_project_path}/community/include/util/tlog.h" ] diff --git a/tests/parallel_test/cases.task b/tests/parallel_test/cases.task index c6596be7fc..151358aec3 100644 --- a/tests/parallel_test/cases.task +++ b/tests/parallel_test/cases.task @@ -74,6 +74,8 @@ #,,n,system-test,python3 ./test.py -f 8-stream/vnode_restart.py -N 4 #,,n,system-test,python3 ./test.py -f 8-stream/snode_restart.py -N 4 ,,n,system-test,python3 ./test.py -f 8-stream/snode_restart_with_checkpoint.py -N 4 +,,y,system-test,./pytest.sh python3 ./test.py -f 8-stream/force_window_close_interp.py +,,y,system-test,./pytest.sh python3 ./test.py -f 8-stream/force_window_close_interval.py ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/pk_error.py ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/pk_func.py @@ -1371,10 +1373,38 @@ ,,y,script,./test.sh -f tsim/stream/sliding.sim ,,y,script,./test.sh -f tsim/stream/state0.sim ,,y,script,./test.sh -f tsim/stream/state1.sim -,,y,script,./test.sh -f tsim/stream/streamPrimaryKey0.sim -,,y,script,./test.sh -f tsim/stream/streamPrimaryKey1.sim -,,y,script,./test.sh -f tsim/stream/streamPrimaryKey2.sim -,,y,script,./test.sh -f tsim/stream/streamPrimaryKey3.sim +#,,y,script,./test.sh -f tsim/stream/streamInterpDelete0.sim +#,,y,script,./test.sh -f tsim/stream/streamInterpDelete1.sim +#,,y,script,./test.sh -f tsim/stream/streamInterpDelete2.sim +#,,y,script,./test.sh -f tsim/stream/streamInterpError.sim +,,y,script,./test.sh -f tsim/stream/streamInterpForceWindowClose.sim +,,y,script,./test.sh -f tsim/stream/streamInterpForceWindowClose1.sim +,,y,script,./test.sh -f tsim/stream/streamInterpFwcError.sim +#,,y,script,./test.sh -f tsim/stream/streamInterpHistory.sim +#,,y,script,./test.sh -f tsim/stream/streamInterpHistory1.sim +#,,y,script,./test.sh -f tsim/stream/streamInterpLarge.sim +#,,y,script,./test.sh -f tsim/stream/streamInterpLinear0.sim +#,,y,script,./test.sh -f tsim/stream/streamInterpNext0.sim +#,,y,script,./test.sh -f tsim/stream/streamInterpOther.sim +#,,y,script,./test.sh -f tsim/stream/streamInterpOther1.sim +#,,y,script,./test.sh -f tsim/stream/streamInterpPartitionBy0.sim +#,,y,script,./test.sh -f tsim/stream/streamInterpPrev0.sim +#,,y,script,./test.sh -f tsim/stream/streamInterpPrev1.sim +#,,y,script,./test.sh -f tsim/stream/streamInterpPrimaryKey0.sim +#,,y,script,./test.sh -f tsim/stream/streamInterpPrimaryKey1.sim +#,,y,script,./test.sh -f tsim/stream/streamInterpPrimaryKey2.sim +#,,y,script,./test.sh -f tsim/stream/streamInterpPrimaryKey3.sim +#,,y,script,./test.sh -f tsim/stream/streamInterpUpdate.sim +#,,y,script,./test.sh -f tsim/stream/streamInterpUpdate1.sim +#,,y,script,./test.sh -f tsim/stream/streamInterpValue0.sim +#,,y,script,./test.sh -f tsim/stream/streamPrimaryKey0.sim +#,,y,script,./test.sh -f tsim/stream/streamPrimaryKey1.sim +#,,y,script,./test.sh -f tsim/stream/streamPrimaryKey2.sim +#,,y,script,./test.sh -f tsim/stream/streamPrimaryKey3.sim +,,y,script,./test.sh -f tsim/stream/streamTwaError.sim +,,y,script,./test.sh -f tsim/stream/streamTwaFwcFill.sim +,,y,script,./test.sh -f tsim/stream/streamTwaFwcFillPrimaryKey.sim +,,y,script,./test.sh -f tsim/stream/streamTwaFwcIntervalPrimaryKey.sim ,,y,script,./test.sh -f tsim/stream/triggerInterval0.sim ,,y,script,./test.sh -f tsim/stream/triggerSession0.sim ,,y,script,./test.sh -f tsim/stream/udTableAndCol0.sim diff --git a/tests/pytest/util/common.py b/tests/pytest/util/common.py index 1dfcf8b5dd..c12f324fd7 100644 --- a/tests/pytest/util/common.py +++ b/tests/pytest/util/common.py @@ -170,6 +170,8 @@ class TDCom: self.fill_tb_source_select_str = ','.join(self.fill_function_list[0:13]) self.ext_tb_source_select_str = ','.join(self.downsampling_function_list[0:13]) self.stream_case_when_tbname = "tbname" + self.tag_value_str = "" + self.tag_value_list = [] self.update = True self.disorder = True @@ -202,7 +204,7 @@ class TDCom: self.cast_tag_stb_filter_des_select_elm = "ts, t1, t2, t3, t4, cast(t1 as TINYINT UNSIGNED), t6, t7, t8, t9, t10, cast(t2 as varchar(256)), t12, cast(t3 as bool)" self.tag_count = len(self.tag_filter_des_select_elm.split(",")) self.state_window_range = list() - + self.custom_col_val = 0 self.part_val_list = [1, 2] # def init(self, conn, logSql): @@ -754,10 +756,10 @@ class TDCom: if len(kwargs) > 0: for param, value in kwargs.items(): ctb_params += f'{param} "{value}" ' - tag_value_list = self.gen_tag_value_list(tag_elm_list) + self.tag_value_list = self.gen_tag_value_list(tag_elm_list) tag_value_str = "" # tag_value_str = ", ".join(str(v) for v in self.tag_value_list) - for tag_value in tag_value_list: + for tag_value in self.tag_value_list: if isinstance(tag_value, str): tag_value_str += f'"{tag_value}", ' else: @@ -913,12 +915,13 @@ class TDCom: else: stream_options += f" ignore update 0" if not use_except: - tdSql.execute(f'create stream if not exists {stream_name} trigger at_once {stream_options} {fill_history} into {des_table} {subtable} as {source_sql} {fill};') + tdSql.execute(f'create stream if not exists {stream_name} trigger at_once {stream_options} {fill_history} into {des_table} {subtable} as {source_sql} {fill};',queryTimes=3) time.sleep(self.create_stream_sleep) return None else: return f'create stream if not exists {stream_name} {stream_options} {fill_history} into {des_table} {subtable} as {source_sql} {fill};' else: + if watermark is None: if trigger_mode == "max_delay": stream_options = f'trigger {trigger_mode} {max_delay}' @@ -938,12 +941,14 @@ class TDCom: stream_options += f" ignore update {ignore_update}" else: stream_options += f" ignore update 0" + if not use_except: - tdSql.execute(f'create stream if not exists {stream_name} {stream_options} {fill_history} into {des_table}{stb_field_name} {tags} {subtable} as {source_sql} {fill};') + tdSql.execute(f'create stream if not exists {stream_name} {stream_options} {fill_history} into {des_table}{stb_field_name} {tags} {subtable} as {source_sql} {fill};',queryTimes=3) time.sleep(self.create_stream_sleep) return None else: return f'create stream if not exists {stream_name} {stream_options} {fill_history} into {des_table}{stb_field_name} {tags} {subtable} as {source_sql} {fill};' + def pause_stream(self, stream_name, if_exist=True, if_not_exist=False): """pause_stream @@ -1007,7 +1012,7 @@ class TDCom: # If no match was found, or the pattern does not match the expected format, return False return False - def check_stream_task_status(self, stream_name, vgroups, stream_timeout=None): + def check_stream_task_status(self, stream_name, vgroups, stream_timeout=0, check_wal_info=True): """check stream status Args: @@ -1043,13 +1048,16 @@ class TDCom: print(f"result_task_status:{result_task_status},result_task_history:{result_task_history},result_task_alll:{result_task_alll}") if result_task_status_rows == 1 and result_task_status ==[('ready',)] : if result_task_history_rows == 1 and result_task_history == [(None,)] : - for vgroup_num in range(vgroups): - if self.check_stream_wal_info(result_task_alll[vgroup_num][4]) : - check_stream_success += 1 - tdLog.info(f"check stream task list[{check_stream_success}] sucessfully :") - else: - check_stream_success = 0 - break + if check_wal_info: + for vgroup_num in range(vgroups): + if self.check_stream_wal_info(result_task_alll[vgroup_num][4]) : + check_stream_success += 1 + tdLog.info(f"check stream task list[{check_stream_success}] sucessfully :") + else: + check_stream_success = 0 + break + else: + check_stream_success = vgroups if check_stream_success == vgroups: break @@ -1761,6 +1769,7 @@ class TDCom: bool: False if failed """ tdLog.info("checking query data ...") + tdLog.info(f"sq1:{sql1}; sql2:{sql2};") if tag_value_list: dvalue = len(self.tag_type_str.split(',')) - defined_tag_count tdSql.query(sql1) @@ -1796,7 +1805,7 @@ class TDCom: res2 = self.round_handle(res2) if not reverse_check: while res1 != res2: - tdLog.info("query retrying ...") + # tdLog.info("query retrying ...") new_list = list() tdSql.query(sql1) res1 = tdSql.queryResult @@ -2160,4 +2169,4 @@ def dict2toml(in_dict: dict, file:str): -tdCom = TDCom() +tdCom = TDCom() \ No newline at end of file diff --git a/tests/script/tsim/stream/checkTaskStatus.sim b/tests/script/tsim/stream/checkTaskStatus.sim index 8b30d27841..0e171b3059 100644 --- a/tests/script/tsim/stream/checkTaskStatus.sim +++ b/tests/script/tsim/stream/checkTaskStatus.sim @@ -9,6 +9,7 @@ sleep 1000 $loop_count = $loop_count + 1 if $loop_count == 60 then + print check task status failed return 1 endi diff --git a/tests/script/tsim/stream/checkpointSession1.sim b/tests/script/tsim/stream/checkpointSession1.sim index 0427585122..5ad7ef0b96 100644 --- a/tests/script/tsim/stream/checkpointSession1.sim +++ b/tests/script/tsim/stream/checkpointSession1.sim @@ -62,6 +62,8 @@ system sh/exec.sh -n dnode1 -s start sleep 2000 +run tsim/stream/checkTaskStatus.sim + sql insert into t1 values(1648791213002,3,2,3,1.1); sql insert into t2 values(1648791233003,4,2,3,1.1); diff --git a/tests/script/tsim/stream/forcewindowclose.sim b/tests/script/tsim/stream/forcewindowclose.sim new file mode 100644 index 0000000000..77def52b3c --- /dev/null +++ b/tests/script/tsim/stream/forcewindowclose.sim @@ -0,0 +1,50 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print =============== create database +sql create database test vgroups 2; +sql select * from information_schema.ins_databases +if $rows != 3 then + return -1 +endi + +print $data00 $data01 $data02 + +sql use test +sql create stable st(ts timestamp, a int) tags(t int); +sql create table tu1 using st tags(1); + +sql create stream stream1 trigger force_window_close into str_dst as select _wstart, count(*) from st partition by tbname interval(5s); + +run tsim/stream/checkTaskStatus.sim + +sql insert into tu1 values(now, 1); +sleep 5500 + +sql pause stream stream1 + +$loop_count = 0 + +loop0: +sleep 500 +$loop_count = $loop_count + 1 +if $loop_count == 20 then + goto end_loop +endi + +sql insert into tu1 values(now, 1); +goto loop0 + +end_loop: + +sql resume stream stream1 +sql select * from str_dst + +if $rows != 3 then + print expect 3, actual: $rows +endi + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamFwcIntervalFill.sim b/tests/script/tsim/stream/streamFwcIntervalFill.sim new file mode 100644 index 0000000000..2785997428 --- /dev/null +++ b/tests/script/tsim/stream/streamFwcIntervalFill.sim @@ -0,0 +1,235 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 4; +sql use test; + +sql create stable st(ts timestamp, a int, b int , c int)tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +sql create stream streams1 trigger force_window_close IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt as select _wstart, count(a) as ca, now, ta, sum(b) as cb, timezone() from st partition by tbname,ta interval(2s) fill(value, 100, 200); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(now + 3000a,1,1,1) (now + 3100a,5,10,10) (now + 3200a,5,10,10) (now + 5100a,20,1,1) (now + 5200a,30,10,10) (now + 5300a,40,10,10); +sql insert into t2 values(now + 3000a,1,1,1) (now + 3100a,2,10,10) (now + 3200a,30,10,10) (now + 5100a,10,1,1) (now + 5200a,40,10,10) (now + 5300a,7,10,10); + + +print sql select _wstart, count(a) as ca, now, ta, sum(b) as cb, timezone() from t1 partition by tbname,ta interval(2s) +sql select _wstart, count(a) as ca, now, ta, sum(b) as cb, timezone() from t1 partition by tbname,ta interval(2s); + +$query1_data01 = $data01 +$query1_data11 = $data11 + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +print sql select _wstart, count(a) as ca, now, ta, sum(b) as cb, timezone() from t2 partition by tbname,ta interval(2s); +sql select _wstart, count(a) as ca, now, ta, sum(b) as cb, timezone() from t2 partition by tbname,ta interval(2s); + +$query2_data01 = $data01 +$query2_data11 = $data11 + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 +loop0: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt where ta == 1 order by 1; +sql select * from streamt where ta == 1 order by 1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows < 2 then + print ======rows=$rows + goto loop0 +endi + +if $data01 != $query1_data01 then + print ======data01========$data01 + print ======query1_data01=$query1_data01 + return -1 +endi + +if $data11 != $query1_data11 then + print ======data11========$data11 + print ======query1_data11=$query1_data11 + goto loop0 +endi + +$loop_count = 0 +loop1: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt where ta == 2 order by 1; +sql select * from streamt where ta == 2 order by 1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows < 2 then + print ======rows=$rows + goto loop1 +endi + +if $data01 != $query2_data01 then + print ======data01======$data01 + print ====query2_data01=$query2_data01 + return -1 +endi + +if $data11 != $query2_data11 then + print ======data11======$data11 + print ====query2_data11=$query2_data11 + goto loop1 +endi + +$loop_count = 0 +loop2: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt; +sql select * from streamt; + +if $rows < 6 then + print ======rows=$rows + goto loop2 +endi + + +print step2 +print =============== create database +sql create database test2 vgroups 4; +sql use test2; + +sql create stable st(ts timestamp, a int, b int , c int)tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +sql create stream streams2 trigger force_window_close IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt as select _wstart, count(*), ta from st partition by tbname,ta interval(2s) fill(NULL); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(now + 3000a,1,1,1) (now + 3100a,3,10,10) (now + 3200a,5,10,10) (now + 5100a,20,1,1) (now + 5200a,30,10,10) (now + 5300a,40,10,10); +sql insert into t2 values(now + 3000a,1,1,1) (now + 3100a,3,10,10) (now + 3200a,5,10,10) (now + 5100a,10,1,1) (now + 5200a,40,10,10) (now + 5300a,7,10,10); + + +print sql select _wstart, count(*) from t1 interval(2s) order by 1; +sql select _wstart, count(*) from t1 interval(2s) order by 1; + +$query1_data01 = $data01 +$query1_data11 = $data11 + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +$loop_count = 0 +loop3: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt where ta == 1 order by 1; +sql select * from streamt where ta == 1 order by 1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +if $data01 != $query1_data01 then + print ======data01======$data01 + print ====query1_data01=$query1_data01 + goto loop3 +endi + + +sql insert into t1 values(now + 3000a,1,1,1) (now + 3100a,3,10,10) (now + 3200a,5,10,10) (now + 5100a,20,1,1) (now + 5200a,30,10,10) (now + 5300a,40,10,10); +sql insert into t2 values(now + 3000a,1,1,1) (now + 3100a,3,10,10) (now + 3200a,5,10,10) (now + 5100a,10,1,1) (now + 5200a,40,10,10) (now + 5300a,7,10,10); + +$loop_count = 0 +loop4: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt; +sql select * from streamt; + +print ======step2=rows=$rows +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +if $rows < 10 then + print ======rows=$rows + goto loop4 +endi + +print end + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpDelete0.sim b/tests/script/tsim/stream/streamInterpDelete0.sim new file mode 100644 index 0000000000..21bac13e4a --- /dev/null +++ b/tests/script/tsim/stream/streamInterpDelete0.sim @@ -0,0 +1,507 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 1; +sql use test; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(prev); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791212001,1,1,1,1.0) (1648791214000,8,1,1,1.0) (1648791215000,10,1,1,1.0) (1648791215009,15,1,1,1.0) (1648791217001,4,1,1,1.0); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(prev); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(prev); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop0: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop0 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop0 +endi + +if $data11 != 8 then + print ======data11=$data11 + goto loop0 +endi + +if $data21 != 10 then + print ======data21=$data21 + goto loop0 +endi + +if $data31 != 15 then + print ======data31=$data31 + goto loop0 +endi + +if $data41 != 15 then + print ======data41=$data41 + goto loop0 +endi + +print 1 sql delete from t1 where ts >= 1648791215000 and ts <= 1648791216000; +sql delete from t1 where ts >= 1648791215000 and ts <= 1648791216000; + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(prev); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(prev); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop1 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop1 +endi + +if $data11 != 8 then + print ======data11=$data11 + goto loop1 +endi + +if $data21 != 8 then + print ======data21=$data21 + goto loop1 +endi + +if $data31 != 8 then + print ======data31=$data31 + goto loop1 +endi + +if $data41 != 8 then + print ======data41=$data41 + goto loop1 +endi + + +print 2 sql delete from t1 where ts >= 1648791212000 and ts <= 1648791213000; +sql delete from t1 where ts >= 1648791212000 and ts <= 1648791213000; + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(prev); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(prev); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop2: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 4 then + print ======rows=$rows + goto loop2 +endi + +# row 0 +if $data01 != 8 then + print ======data01=$data01 + goto loop2 +endi + +if $data11 != 8 then + print ======data11=$data11 + goto loop2 +endi + +if $data21 != 8 then + print ======data21=$data21 + goto loop2 +endi + +if $data31 != 8 then + print ======data31=$data31 + goto loop2 +endi + +print 3 sql delete from t1 where ts >= 1648791217000 and ts <= 1648791218000; +sql delete from t1 where ts >= 1648791217000 and ts <= 1648791218000 + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(prev); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(prev); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop3: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop3 +endi + +# row 0 +if $data01 != 8 then + print ======data01=$data01 + goto loop3 +endi + + +print step2 +print =============== create database +sql create database test2 vgroups 1; +sql use test2; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(next); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791212001,1,1,1,1.0) (1648791214000,8,1,1,1.0) (1648791215000,10,1,1,1.0) (1648791215009,15,1,1,1.0) (1648791217001,4,1,1,1.0); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(next); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(next); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop4: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop4 +endi + +# row 0 +if $data01 != 8 then + print ======data01=$data01 + goto loop4 +endi + +if $data11 != 8 then + print ======data11=$data11 + goto loop4 +endi + +if $data21 != 10 then + print ======data21=$data21 + goto loop4 +endi + +if $data31 != 4 then + print ======data31=$data31 + goto loop4 +endi + +if $data41 != 4 then + print ======data41=$data41 + goto loop4 +endi + +print 1 sql delete from t1 where ts >= 1648791215000 and ts <= 1648791216000; +sql delete from t1 where ts >= 1648791215000 and ts <= 1648791216000; + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(next); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(next); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop5: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop5 +endi + +# row 0 +if $data01 != 8 then + print ======data01=$data01 + goto loop5 +endi + +if $data11 != 8 then + print ======data11=$data11 + goto loop5 +endi + +if $data21 != 4 then + print ======data21=$data21 + goto loop5 +endi + +if $data31 != 4 then + print ======data31=$data31 + goto loop5 +endi + +if $data41 != 4 then + print ======data41=$data41 + goto loop5 +endi + + +print 2 sql delete from t1 where ts >= 1648791212000 and ts <= 1648791213000; +sql delete from t1 where ts >= 1648791212000 and ts <= 1648791213000; + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(next); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(next); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop6: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 4 then + print ======rows=$rows + goto loop6 +endi + +# row 0 +if $data01 != 8 then + print ======data01=$data01 + goto loop6 +endi + +if $data11 != 4 then + print ======data11=$data11 + goto loop6 +endi + +if $data21 != 4 then + print ======data21=$data21 + goto loop6 +endi + +if $data31 != 4 then + print ======data31=$data31 + goto loop6 +endi + +print 3 sql delete from t1 where ts >= 1648791217000 and ts <= 1648791218000; +sql delete from t1 where ts >= 1648791217000 and ts <= 1648791218000 + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(next); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(next); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop7: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop7 +endi + +# row 0 +if $data01 != 8 then + print ======data01=$data01 + goto loop7 +endi + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpDelete1.sim b/tests/script/tsim/stream/streamInterpDelete1.sim new file mode 100644 index 0000000000..162da175e8 --- /dev/null +++ b/tests/script/tsim/stream/streamInterpDelete1.sim @@ -0,0 +1,508 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 1; +sql use test; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(NULL); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791212001,1,1,1,1.0) (1648791214000,8,1,1,1.0) (1648791215000,10,1,1,1.0) (1648791215009,15,1,1,1.0) (1648791217001,4,1,1,1.0); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(NULL); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(NULL); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop0: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop0 +endi + +# row 0 +if $data01 != NULL then + print ======data01=$data01 + goto loop0 +endi + +if $data11 != 8 then + print ======data11=$data11 + goto loop0 +endi + +if $data21 != 10 then + print ======data21=$data21 + goto loop0 +endi + +if $data31 != NULL then + print ======data31=$data31 + goto loop0 +endi + +if $data41 != NULL then + print ======data41=$data41 + goto loop0 +endi + +print 1 sql delete from t1 where ts >= 1648791215000 and ts <= 1648791216000; +sql delete from t1 where ts >= 1648791215000 and ts <= 1648791216000; + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(NULL); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(NULL); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop1 +endi + +# row 0 +if $data01 != NULL then + print ======data01=$data01 + goto loop1 +endi + +if $data11 != 8 then + print ======data11=$data11 + goto loop1 +endi + +if $data21 != NULL then + print ======data21=$data21 + goto loop1 +endi + +if $data31 != NULL then + print ======data31=$data31 + goto loop1 +endi + +if $data41 != NULL then + print ======data41=$data41 + goto loop1 +endi + + +print 2 sql delete from t1 where ts >= 1648791212000 and ts <= 1648791213000; +sql delete from t1 where ts >= 1648791212000 and ts <= 1648791213000; + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(NULL); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(NULL); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop2: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 4 then + print ======rows=$rows + goto loop2 +endi + +# row 0 +if $data01 != 8 then + print ======data01=$data01 + goto loop2 +endi + +if $data11 != NULL then + print ======data11=$data11 + goto loop2 +endi + +if $data21 != NULL then + print ======data21=$data21 + goto loop2 +endi + +if $data31 != NULL then + print ======data31=$data31 + goto loop2 +endi + +print 3 sql delete from t1 where ts >= 1648791217000 and ts <= 1648791218000; +sql delete from t1 where ts >= 1648791217000 and ts <= 1648791218000 + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(NULL); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(NULL); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop3: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop3 +endi + +# row 0 +if $data01 != 8 then + print ======data01=$data01 + goto loop3 +endi + + +print step2 +print =============== create database +sql create database test2 vgroups 1; +sql use test2; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(value,100,200,300,400); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791212001,1,1,1,1.0) (1648791214000,8,1,1,1.0) (1648791215000,10,1,1,1.0) (1648791215009,15,1,1,1.0) (1648791217001,4,1,1,1.0); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(value,100,200,300,400); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(value,100,200,300,400); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop4: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop4 +endi + +# row 0 +if $data01 != 100 then + print ======data01=$data01 + goto loop4 +endi + +if $data11 != 8 then + print ======data11=$data11 + goto loop4 +endi + +if $data21 != 10 then + print ======data21=$data21 + goto loop4 +endi + +if $data31 != 100 then + print ======data31=$data31 + goto loop4 +endi + +if $data41 != 100 then + print ======data41=$data41 + goto loop4 +endi + +print 1 sql delete from t1 where ts >= 1648791215000 and ts <= 1648791216000; +sql delete from t1 where ts >= 1648791215000 and ts <= 1648791216000; + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(value,100,200,300,400); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(value,100,200,300,400); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop5: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop5 +endi + +# row 0 +if $data01 != 100 then + print ======data01=$data01 + goto loop5 +endi + +if $data11 != 8 then + print ======data11=$data11 + goto loop5 +endi + +if $data21 != 100 then + print ======data21=$data21 + goto loop5 +endi + +if $data31 != 100 then + print ======data31=$data31 + goto loop5 +endi + +if $data41 != 100 then + print ======data41=$data41 + goto loop5 +endi + + +print 2 sql delete from t1 where ts >= 1648791212000 and ts <= 1648791213000; +sql delete from t1 where ts >= 1648791212000 and ts <= 1648791213000; + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(value,100,200,300,400); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(value,100,200,300,400); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop6: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 4 then + print ======rows=$rows + goto loop6 +endi + +# row 0 +if $data01 != 8 then + print ======data01=$data01 + goto loop6 +endi + +if $data11 != 100 then + print ======data11=$data11 + goto loop6 +endi + +if $data21 != 100 then + print ======data21=$data21 + goto loop6 +endi + +if $data31 != 100 then + print ======data31=$data31 + goto loop6 +endi + +print 3 sql delete from t1 where ts >= 1648791217000 and ts <= 1648791218000; +sql delete from t1 where ts >= 1648791217000 and ts <= 1648791218000 + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(value,100,200,300,400); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(value,100,200,300,400); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop7: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop7 +endi + +# row 0 +if $data01 != 8 then + print ======data01=$data01 + goto loop7 +endi + + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpDelete2.sim b/tests/script/tsim/stream/streamInterpDelete2.sim new file mode 100644 index 0000000000..be27dcda49 --- /dev/null +++ b/tests/script/tsim/stream/streamInterpDelete2.sim @@ -0,0 +1,258 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 1; +sql use test; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(linear); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791212001,1,1,1,1.0) (1648791214000,8,1,1,1.0) (1648791215000,10,1,1,1.0) (1648791215009,15,1,1,1.0) (1648791217001,4,1,1,1.0); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(linera); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(linear); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop0: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop0 +endi + +# row 0 +if $data01 != 4 then + print ======data01=$data01 + goto loop0 +endi + +if $data11 != 8 then + print ======data11=$data11 + goto loop0 +endi + +if $data21 != 10 then + print ======data21=$data21 + goto loop0 +endi + +if $data31 != 9 then + print ======data31=$data31 + goto loop0 +endi + +if $data41 != 4 then + print ======data41=$data41 + goto loop0 +endi + +print 1 sql delete from t1 where ts >= 1648791215000 and ts <= 1648791216000; +sql delete from t1 where ts >= 1648791215000 and ts <= 1648791216000; + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(linear); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(linear); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop1 +endi + +# row 0 +if $data01 != 4 then + print ======data01=$data01 + goto loop1 +endi + +if $data11 != 8 then + print ======data11=$data11 + goto loop1 +endi + +if $data21 != 6 then + print ======data21=$data21 + goto loop1 +endi + +if $data31 != 5 then + print ======data31=$data31 + goto loop1 +endi + +if $data41 != 4 then + print ======data41=$data41 + goto loop1 +endi + + +print 2 sql delete from t1 where ts >= 1648791212000 and ts <= 1648791213000; +sql delete from t1 where ts >= 1648791212000 and ts <= 1648791213000; + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(linear); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(linear); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop2: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 4 then + print ======rows=$rows + goto loop2 +endi + +# row 0 +if $data01 != 8 then + print ======data01=$data01 + goto loop2 +endi + +if $data11 != 6 then + print ======data11=$data11 + goto loop2 +endi + +if $data21 != 5 then + print ======data21=$data21 + goto loop2 +endi + +if $data31 != 4 then + print ======data31=$data31 + goto loop2 +endi + +print 3 sql delete from t1 where ts >= 1648791217000 and ts <= 1648791218000; +sql delete from t1 where ts >= 1648791217000 and ts <= 1648791218000 + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(linear); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(linear); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop3: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop3 +endi + +# row 0 +if $data01 != 8 then + print ======data01=$data01 + goto loop3 +endi + + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpError.sim b/tests/script/tsim/stream/streamInterpError.sim new file mode 100644 index 0000000000..53a92df772 --- /dev/null +++ b/tests/script/tsim/stream/streamInterpError.sim @@ -0,0 +1,117 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step2 + +sql create database test2 vgroups 1; +sql use test2; + +sql create stable st(ts timestamp,a int,b int,c int, d double) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +print step2_0 + +sql create stream streams2_0_1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_0_1 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(prev); +sql create stream streams2_0_2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_0_2 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(next); +sql create stream streams2_0_3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_0_3 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(linear); +sql create stream streams2_0_4 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_0_4 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(NULL); +sql create stream streams2_0_5 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_0_5 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(value,11,22,33,44); + +print step2_1 + +sql_error create stream streams2_1_1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_1_1 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 range(1648791212000, 1648791215000) every(1s) fill(prev); +sql_error create stream streams2_1_2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_1_2 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 range(1648791212000, 1648791215000) every(1s) fill(next); +sql_error create stream streams2_1_3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_1_3 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 range(1648791212000, 1648791215000) every(1s) fill(linear); +sql_error create stream streams2_1_4 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_1_4 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 range(1648791212000, 1648791215000) every(1s) fill(NULL); +sql_error create stream streams2_1_5 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_1_5 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 range(1648791212000, 1648791215000) every(1s) fill(value,11,22,33,44); + +print step2_2 + +sql_error create stream streams2_2_1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_2_1 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from st every(1s) fill(prev); +sql_error create stream streams2_2_2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_2_2 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from st every(1s) fill(next); +sql_error create stream streams2_2_3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_2_3 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from st every(1s) fill(linear); +sql_error create stream streams2_2_4 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_2_4 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from st every(1s) fill(NULL); +sql_error create stream streams2_2_5 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_2_5 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from st every(1s) fill(value,11,22,33,44); + +print step2_3 + +sql_error create stream streams2_3_1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_3_1 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from st partition by a every(1s) fill(prev); +sql_error create stream streams2_3_2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_3_2 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from st partition by a every(1s) fill(next); +sql_error create stream streams2_3_3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_3_3 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from st partition by a every(1s) fill(linear); +sql_error create stream streams2_3_4 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_3_4 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from st partition by a every(1s) fill(NULL); +sql_error create stream streams2_3_5 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_3_5 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from st partition by a every(1s) fill(value,11,22,33,44); + +print step2_4 + +sql_error create stream streams2_4_1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_4_1 as select INTERP(a) FROM t1 RANGE('2023-01-01 00:00:00') fill(prev); +sql_error create stream streams2_4_2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_4_2 as select INTERP(a) FROM t1 RANGE('2023-01-01 00:00:00') fill(next); +sql_error create stream streams2_4_3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_4_3 as select INTERP(a) FROM t1 RANGE('2023-01-01 00:00:00') fill(linear); +sql_error create stream streams2_4_4 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_4_4 as select INTERP(a) FROM t1 RANGE('2023-01-01 00:00:00') fill(NULL); +sql_error create stream streams2_4_5 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_4_5 as select INTERP(a) FROM t1 RANGE('2023-01-01 00:00:00') fill(value,11,22,33,44); + +print step2_5 + +sql_error create stream streams2_5_1 trigger window_close IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_5_1 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(prev); +sql_error create stream streams2_5_2 trigger window_close IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_5_2 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(next); +sql_error create stream streams2_5_3 trigger window_close IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_5_3 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(linear); +sql_error create stream streams2_5_4 trigger window_close IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_5_4 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(NULL); +sql_error create stream streams2_5_5 trigger window_close IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_5_5 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(value,11,22,33,44); + +run tsim/stream/checkTaskStatus.sim + +print step2_6 + +sql create stream streams2_6_1 trigger at_once FILL_HISTORY 1 IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_6_1 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(prev); +sql create stream streams2_6_2 trigger at_once FILL_HISTORY 1 IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_6_2 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(next); +sql create stream streams2_6_3 trigger at_once FILL_HISTORY 1 IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_6_3 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(linear); +sql create stream streams2_6_4 trigger at_once FILL_HISTORY 1 IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_6_4 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(NULL); +sql create stream streams2_6_5 trigger at_once FILL_HISTORY 1 IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_6_5 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(value,11,22,33,44); + +sql_error create stream streams2_6_6 trigger force_window_close FILL_HISTORY 1 IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt2_6_6 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(prev); +sql_error create stream streams2_6_7 trigger force_window_close FILL_HISTORY 1 IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt2_6_7 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(next); +sql_error create stream streams2_6_8 trigger force_window_close FILL_HISTORY 1 IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt2_6_8 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(linear); +sql_error create stream streams2_6_9 trigger force_window_close FILL_HISTORY 1 IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt2_6_9 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(NULL); +sql_error create stream streams2_6_10 trigger force_window_close FILL_HISTORY 1 IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt2_6_10 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(value,11,22,33,44); + +run tsim/stream/checkTaskStatus.sim + +print step3 + +sql create database test3 vgroups 1; +sql use test3; + +sql create stable st(ts timestamp,a int,b int,c int, d double) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +print step3_0 + +sql create stream streams3_0_1 trigger force_window_close FILL_HISTORY 0 IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt3_0_1 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(prev); + +sleep 5000 + +sql_error create stream streams3_0_2 trigger force_window_close FILL_HISTORY 0 IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt3_0_2 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(next); +sql_error create stream streams3_0_3 trigger force_window_close FILL_HISTORY 0 IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt3_0_3 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(linear); +sql create stream streams3_0_4 trigger force_window_close FILL_HISTORY 0 IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt3_0_4 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(NULL); + +sleep 5000 + +sql create stream streams3_0_5 trigger force_window_close FILL_HISTORY 0 IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt3_0_5 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(value,11,22,33,44); + + +run tsim/stream/checkTaskStatus.sim + +print step4 + +sql_error create stream streams4_1 trigger max_delay 1s FILL_HISTORY 0 IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt4_1 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(prev); +sql_error create stream streams4_2 trigger max_delay 1s FILL_HISTORY 0 IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt4_2 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(next); +sql_error create stream streams4_3 trigger max_delay 1s FILL_HISTORY 0 IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt4_3 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(linear); +sql_error create stream streams4_4 trigger max_delay 1s FILL_HISTORY 0 IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt4_4 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(NULL); +sql_error create stream streams4_5 trigger max_delay 1s FILL_HISTORY 0 IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt4_5 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(value,11,22,33,44); + + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpForceWindowClose.sim b/tests/script/tsim/stream/streamInterpForceWindowClose.sim new file mode 100644 index 0000000000..e96866b3e0 --- /dev/null +++ b/tests/script/tsim/stream/streamInterpForceWindowClose.sim @@ -0,0 +1,235 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 1; +sql use test; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams1 trigger force_window_close IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt as select _irowts, interp(a) as a, interp(b) as b, now from t1 every(2s) fill(prev); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(now,1,1,1,1.1) (now + 10s,2,2,2,2.1) (now + 20s,3,3,3,3.1); + +print sql select * from t1; +sql select * from t1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 +loop0: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt where a == 1; +sql select * from streamt where a == 1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows < 2 then + print ======rows=$rows + goto loop0 +endi + +$loop_count = 0 +loop1: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt where a == 2; +sql select * from streamt where a == 2; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows < 2 then + print ======rows=$rows + goto loop1 +endi + +$loop_count = 0 +loop2: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt where a == 3; +sql select * from streamt where a == 3; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows < 2 then + print ======rows=$rows + goto loop2 +endi + +sleep 4000 + +$loop_count = 0 +loop3: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt where a == 3; +sql select * from streamt where a == 3; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows < 5 then + print ======rows=$rows + goto loop3 +endi + +print step2 +print =============== create database +sql create database test2 vgroups 1; +sql use test2; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams2 trigger force_window_close IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt as select _irowts, interp(a) as a, interp(b) as b, now from t1 every(2s) fill(NULL); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(now,1,1,1,1.1) (now + 10s,2,2,2,2.1) (now + 20s,3,3,3,3.1); + +print sql select * from t1; +sql select * from t1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 +loop4: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt where a is null; +sql select * from streamt where a is null; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +if $rows < 5 then + print ======rows=$rows + goto loop4 +endi + +print step3 +print =============== create database +sql create database test3 vgroups 1; +sql use test3; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams3 trigger force_window_close IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt as select _irowts, interp(a) as a, interp(b) as b, now from t1 every(2s) fill(value,100,200); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(now,1,1,1,1.1) (now + 10s,2,2,2,2.1) (now + 20s,3,3,3,3.1); + +print sql select * from t1; +sql select * from t1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 +loop5: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt where a == 100; +sql select * from streamt where a == 100; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +if $rows < 5 then + print ======rows=$rows + goto loop5 +endi + +print end + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpForceWindowClose1.sim b/tests/script/tsim/stream/streamInterpForceWindowClose1.sim new file mode 100644 index 0000000000..e870e407f9 --- /dev/null +++ b/tests/script/tsim/stream/streamInterpForceWindowClose1.sim @@ -0,0 +1,471 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step prev +print =============== create database +sql create database test vgroups 3; +sql use test; +sql create stable st(ts timestamp,a int,b int,c int, d double) tags(ta int,tb int,tc int); + +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); +sql create table t3 using st tags(2,2,2); + +sql create stream streams1 trigger force_window_close IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt as select _irowts, interp(a) as a, _isfilled, tbname, b, c from st partition by tbname, b,c every(5s) fill(prev); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(now,1,1,1,1.0) (now + 10s,2,1,1,2.0)(now + 20s,3,1,1,3.0) +sql insert into t2 values(now,21,1,1,1.0) (now + 10s,22,1,1,2.0)(now + 20s,23,1,1,3.0) +sql insert into t3 values(now,31,1,1,1.0) (now + 10s,32,1,1,2.0)(now + 20s,33,1,1,3.0) + +print sql select * from t1; +sql select * from t1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +print sql select * from t2; +sql select * from t2; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +print sql select * from t3; +sql select * from t3; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 +loop0: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt where a == 1; +sql select * from streamt where a == 1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows < 2 then + print ======rows=$rows + goto loop0 +endi + +print 2 sql select * from streamt where a == 21; +sql select * from streamt where a == 21; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows < 2 then + print ======rows=$rows + goto loop0 +endi + +$loop_count = 0 +loop1: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt where a == 31; +sql select * from streamt where a == 31; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows < 2 then + print ======rows=$rows + goto loop0 +endi + +$loop_count = 0 +loop1: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print sql select * from streamt where a == 2; +sql select * from streamt where a == 2; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows < 2 then + print ======rows=$rows + goto loop1 +endi + +print 3 sql select * from streamt where a == 22; +sql select * from streamt where a == 22; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows < 2 then + print ======rows=$rows + goto loop1 +endi + +print 3 sql select * from streamt where a == 32; +sql select * from streamt where a == 32; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows < 2 then + print ======rows=$rows + goto loop1 +endi + +$loop_count = 0 +loop2: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 4 sql select * from streamt where a == 3; +sql select * from streamt where a == 3; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows < 2 then + print ======rows=$rows + goto loop2 +endi + +print 4 sql select * from streamt where a == 23; +sql select * from streamt where a == 23; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows < 2 then + print ======rows=$rows + goto loop2 +endi + +print 4 sql select * from streamt where a == 33; +sql select * from streamt where a == 33; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows < 2 then + print ======rows=$rows + goto loop2 +endi + +sleep 4000 + +$loop_count = 0 +loop3: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 5 sql select * from streamt where a == 3; +sql select * from streamt where a == 3; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows < 5 then + print ======rows=$rows + goto loop3 +endi + +print 5 sql select * from streamt where a == 23; +sql select * from streamt where a == 23; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows < 5 then + print ======rows=$rows + goto loop3 +endi + +print 5 sql select * from streamt where a == 33; +sql select * from streamt where a == 33; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows < 5 then + print ======rows=$rows + goto loop3 +endi + +print 2 sql select * from streamt where a == 3; +sql select * from streamt where a == 3; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows < 5 then + print ======rows=$rows + goto loop3 +endi + +print step2 +print =============== create database +sql create database test2 vgroups 1; +sql use test2; + +sql create stable st(ts timestamp,a int,b int,c int, d double) tags(ta int,tb int,tc int); + +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); +sql create table t3 using st tags(2,2,2); + +sql create stream streams2 trigger force_window_close IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt as select _irowts, interp(a) as a, _isfilled, tbname, b, c from st partition by tbname, b,c every(2s) fill(NULL); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(now,1,1,1,1.0) (now + 10s,2,1,1,2.0)(now + 20s,3,1,1,3.0) +sql insert into t2 values(now,21,1,1,1.0) (now + 10s,22,1,1,2.0)(now + 20s,23,1,1,3.0) +sql insert into t3 values(now,31,1,1,1.0) (now + 10s,32,1,1,2.0)(now + 20s,33,1,1,3.0) + +print sql select * from t1; +sql select * from t1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +print sql select * from t2; +sql select * from t2; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +print sql select * from t3; +sql select * from t3; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 +loop4: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt where a is null; +sql select * from streamt where a is null; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows < 5 then + print ======rows=$rows + goto loop4 +endi + +print step3 +print =============== create database +sql create database test3 vgroups 1; +sql use test3; + +sql create stable st(ts timestamp,a int,b int,c int, d double) tags(ta int,tb int,tc int); + +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); +sql create table t3 using st tags(2,2,2); + +sql create stream streams3 trigger force_window_close IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt as select _irowts, interp(a) as a, _isfilled, tbname, b, c from st partition by tbname, b,c every(2s) fill(value,100); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(now,1,1,1,1.0) (now + 10s,2,1,1,2.0)(now + 20s,3,1,1,3.0) +sql insert into t2 values(now,21,1,1,1.0) (now + 10s,22,1,1,2.0)(now + 20s,23,1,1,3.0) +sql insert into t3 values(now,31,1,1,1.0) (now + 10s,32,1,1,2.0)(now + 20s,33,1,1,3.0) + +print sql select * from t1; +sql select * from t1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +print sql select * from t2; +sql select * from t2; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +print sql select * from t3; +sql select * from t3; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 +loop5: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt where a == 100; +sql select * from streamt where a == 100; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows < 10 then + print ======rows=$rows + goto loop5 +endi + +print end + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpFwcError.sim b/tests/script/tsim/stream/streamInterpFwcError.sim new file mode 100644 index 0000000000..a53a6fe189 --- /dev/null +++ b/tests/script/tsim/stream/streamInterpFwcError.sim @@ -0,0 +1,31 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step2 + +sql create database test2 vgroups 1; +sql use test2; + +sql create stable st(ts timestamp,a int,b int,c int, d double) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +sql create stream streams1 trigger force_window_close into streamt1 as select _irowts, _isfilled as a1, interp(a) as a2 from st partition by tbname every(1s) fill(prev); + +run tsim/stream/checkTaskStatus.sim + +sql_error create stream streams2 trigger force_window_close IGNORE EXPIRED 0 into streamt2 as select _irowts, _isfilled as a1, interp(a) as a2 from st partition by tbname every(1s) fill(prev); +sql_error create stream streams3 trigger force_window_close IGNORE UPDATE 0 into streamt3 as select _irowts, _isfilled as a1, interp(a) as a2 from st partition by tbname every(1s) fill(prev); + + +sql create stream streams4 trigger force_window_close IGNORE EXPIRED 1 into streamt4 as select _irowts, _isfilled as a1, interp(a) as a2 from st partition by tbname every(1s) fill(prev); +run tsim/stream/checkTaskStatus.sim + +sql create stream streams5 trigger force_window_close IGNORE UPDATE 1 into streamt5 as select _irowts, _isfilled as a1, interp(a) as a2 from st partition by tbname every(1s) fill(prev); +run tsim/stream/checkTaskStatus.sim + + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpHistory.sim b/tests/script/tsim/stream/streamInterpHistory.sim new file mode 100644 index 0000000000..b9685ebf05 --- /dev/null +++ b/tests/script/tsim/stream/streamInterpHistory.sim @@ -0,0 +1,655 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 1; +sql use test; + +sql create stable st(ts timestamp,a int,b int,c int, d double) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +sql insert into t1 values(1648791212000,1,1,1,1.0); +sql insert into t1 values(1648791215001,2,1,1,1.0); + +sql insert into t2 values(1648791212000,31,1,1,1.0); +sql insert into t2 values(1648791216001,41,1,1,1.0); + +sql create stream streams1 trigger at_once FILL_HISTORY 1 IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, _isfilled as a1, interp(a) as a2 from st partition by tbname every(1s) fill(prev); + + +sql insert into t1 values(1648791217000,5,1,1,1.0); +sql insert into t2 values(1648791217000,61,1,1,1.0); + +print sql select _irowts, _isfilled as a1, interp(a) as a2 from t1 partition by tbname range(1648791212000, 1648791217000) every(1s) fill(prev) order by 3, 1; +sql select _irowts, _isfilled as a1, interp(a) as a2 from t1 partition by tbname range(1648791212000, 1648791217000) every(1s) fill(prev) order by 3, 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + + +$loop_count = 0 + +loop0: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt where a2 <= 10 order by 1; +sql select * from streamt where a2 < 10 order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +# row 0 +if $rows != 6 then + print ======rows=$rows + goto loop0 +endi + +# row 0 +if $data02 != 1 then + print ======data02=$data02 + goto loop0 +endi + +if $data12 != 1 then + print ======data12=$data12 + goto loop0 +endi + +if $data22 != 1 then + print ======data22=$data22 + goto loop0 +endi + +if $data32 != 1 then + print ======data32=$data32 + goto loop0 +endi + +if $data42 != 2 then + print ======data42=$data42 + goto loop0 +endi + +if $data52 != 5 then + print ======data52=$data52 + goto loop0 +endi + + +print sql select _irowts, _isfilled as a1, interp(a) as a2 from t2 partition by tbname range(1648791212000, 1648791217000) every(1s) fill(prev) order by 3, 1; +sql select _irowts, _isfilled as a1, interp(a) as a2 from t2 partition by tbname range(1648791212000, 1648791217000) every(1s) fill(prev) order by 3, 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +$loop_count = 0 + +loop0_1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt where a2 > 10 order by 1; +sql select * from streamt where a2 > 10 order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +# row 0 +if $rows != 6 then + print ======rows=$rows + goto loop0_1 +endi + +if $data02 != 31 then + print ======data02=$data02 + goto loop0_1 +endi + +if $data12 != 31 then + print ======data12=$data12 + goto loop0_1 +endi + +if $data22 != 31 then + print ======data22=$data22 + goto loop0_1 +endi + +if $data32 != 31 then + print ======data32=$data32 + goto loop0_1 +endi + +if $data42 != 31 then + print ======data42=$data42 + goto loop0_1 +endi + +if $data52 != 61 then + print ======data52=$data52 + goto loop0_1 +endi + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791219001,7,1,1,1.0); +sql insert into t2 values(1648791219001,81,1,1,1.0); + +print sql select _irowts, _isfilled as a1, interp(a) as a2 from t1 partition by tbname range(1648791212000, 1648791219000) every(1s) fill(prev) order by 3, 1; +sql select _irowts, _isfilled as a1, interp(a) as a2 from t1 partition by tbname range(1648791212000, 1648791219000) every(1s) fill(prev) order by 3, 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 +print $data60 $data61 $data62 $data63 $data64 $data65 +print $data70 $data71 $data72 $data73 $data74 $data75 + +$loop_count = 0 + +loop1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt where a2 <= 10 order by 1; +sql select * from streamt where a2 < 10 order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 +print $data60 $data61 $data62 $data63 $data64 $data65 +print $data70 $data71 $data72 $data73 $data74 $data75 + +if $rows != 8 then + print ======rows=$rows + goto loop1 +endi + +print sql select _irowts, _isfilled as a1, interp(a) as a2 from t2 partition by tbname range(1648791212000, 1648791219000) every(1s) fill(prev) order by 3, 1; +sql select _irowts, _isfilled as a1, interp(a) as a2 from t2 partition by tbname range(1648791212000, 1648791219000) every(1s) fill(prev) order by 3, 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 +print $data60 $data61 $data62 $data63 $data64 $data65 +print $data70 $data71 $data72 $data73 $data74 $data75 + +$loop_count = 0 + +loop1_1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt where a2 > 10 order by 1; +sql select * from streamt where a2 > 10 order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 +print $data60 $data61 $data62 $data63 $data64 $data65 +print $data70 $data71 $data72 $data73 $data74 $data75 + +if $rows != 8 then + print ======rows=$rows + goto loop1_1 +endi + +print step2 +print =============== create database +sql create database test2 vgroups 1; +sql use test2; + +sql create stable st(ts timestamp,a int,b int,c int, d double) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +sql insert into t1 values(1648791212000,1,1,1,1.0); +sql insert into t1 values(1648791215001,2,1,1,1.0); + +sql insert into t2 values(1648791212000,31,1,1,1.0); +sql insert into t2 values(1648791216001,41,1,1,1.0); + +sql create stream streams2 trigger at_once FILL_HISTORY 1 IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, _isfilled as a1, interp(a) as a2 from st partition by tbname every(1s) fill(next); + + +sql insert into t1 values(1648791217000,5,1,1,1.0); +sql insert into t2 values(1648791217000,61,1,1,1.0); + +print sql select _irowts, _isfilled as a1, interp(a) as a2 from t1 partition by tbname range(1648791212000, 1648791217000) every(1s) fill(next) order by 3, 1; +sql select _irowts, _isfilled as a1, interp(a) as a2 from t1 partition by tbname range(1648791212000, 1648791217000) every(1s) fill(next) order by 3, 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + + +$loop_count = 0 + +loop2: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt where a2 <= 10 order by 1; +sql select * from streamt where a2 < 10 order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +# row 0 +if $rows != 6 then + print ======rows=$rows + goto loop2 +endi + +# row 0 +if $data02 != 1 then + print ======data02=$data02 + goto loop2 +endi + +if $data12 != 2 then + print ======data12=$data12 + goto loop2 +endi + +if $data22 != 2 then + print ======data22=$data22 + goto loop2 +endi + +if $data32 != 2 then + print ======data32=$data32 + goto loop2 +endi + +if $data42 != 5 then + print ======data42=$data42 + goto loop2 +endi + +if $data52 != 5 then + print ======data52=$data52 + goto loop2 +endi + + +print sql select _irowts, _isfilled as a1, interp(a) as a2 from t2 partition by tbname range(1648791212000, 1648791217000) every(1s) fill(next) order by 3, 1; +sql select _irowts, _isfilled as a1, interp(a) as a2 from t2 partition by tbname range(1648791212000, 1648791217000) every(1s) fill(next) order by 3, 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +$loop_count = 0 + +loop2_1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt where a2 > 10 order by 1; +sql select * from streamt where a2 > 10 order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +# row 0 +if $rows != 6 then + print ======rows=$rows + goto loop2_1 +endi + +if $data02 != 31 then + print ======data02=$data02 + goto loop2_1 +endi + +if $data12 != 41 then + print ======data12=$data12 + goto loop2_1 +endi + +if $data22 != 41 then + print ======data22=$data22 + goto loop2_1 +endi + +if $data32 != 41 then + print ======data32=$data32 + goto loop2_1 +endi + +if $data42 != 41 then + print ======data42=$data42 + goto loop2_1 +endi + +if $data52 != 61 then + print ======data52=$data52 + goto loop2_1 +endi + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791219001,7,1,1,1.0); +sql insert into t2 values(1648791219001,81,1,1,1.0); + +print sql select _irowts, _isfilled as a1, interp(a) as a2 from t1 partition by tbname range(1648791212000, 1648791219000) every(1s) fill(next) order by 3, 1; +sql select _irowts, _isfilled as a1, interp(a) as a2 from t1 partition by tbname range(1648791212000, 1648791219000) every(1s) fill(next) order by 3, 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 +print $data60 $data61 $data62 $data63 $data64 $data65 +print $data70 $data71 $data72 $data73 $data74 $data75 + +$loop_count = 0 + +loop3: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt where a2 <= 10 order by 1; +sql select * from streamt where a2 < 10 order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 +print $data60 $data61 $data62 $data63 $data64 $data65 +print $data70 $data71 $data72 $data73 $data74 $data75 + +if $rows != 8 then + print ======rows=$rows + goto loop3 +endi + +print sql select _irowts, _isfilled as a1, interp(a) as a2 from t2 partition by tbname range(1648791212000, 1648791219000) every(1s) fill(next) order by 3, 1; +sql select _irowts, _isfilled as a1, interp(a) as a2 from t2 partition by tbname range(1648791212000, 1648791219000) every(1s) fill(next) order by 3, 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 +print $data60 $data61 $data62 $data63 $data64 $data65 +print $data70 $data71 $data72 $data73 $data74 $data75 + +$loop_count = 0 + +loop3_1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt where a2 > 10 order by 1; +sql select * from streamt where a2 > 10 order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 +print $data60 $data61 $data62 $data63 $data64 $data65 +print $data70 $data71 $data72 $data73 $data74 $data75 + +if $rows != 8 then + print ======rows=$rows + goto loop3_1 +endi + +print step3 + +print =============== create database +sql create database test3 vgroups 1; +sql use test3; + +sql create stable st(ts timestamp,a int,b int,c int, d double) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +sql insert into t1 values(1648791212000,1,1,1,1.0); +sql insert into t1 values(1648791215001,2,1,1,1.0); + +sql insert into t2 values(1648791212000,31,1,1,1.0); +sql insert into t2 values(1648791216001,41,1,1,1.0); + +sql create stream streams3 trigger at_once FILL_HISTORY 1 IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, _isfilled as a1, interp(a) as a2 from st partition by tbname every(1s) fill(prev); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791217000,5,1,1,1.0); +sql insert into t2 values(1648791217000,61,1,1,1.0); + +print sql select _irowts, _isfilled as a1, interp(a) as a2 from t1 partition by tbname range(1648791212000, 1648791217000) every(1s) fill(prev) order by 3, 1; +sql select _irowts, _isfilled as a1, interp(a) as a2 from t1 partition by tbname range(1648791212000, 1648791217000) every(1s) fill(prev) order by 3, 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + + +$loop_count = 0 + +loop4: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt where a2 <= 10 order by 1; +sql select * from streamt where a2 < 10 order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +# row 0 +if $rows != 6 then + print ======rows=$rows + goto loop4 +endi + +# row 0 +if $data02 != 1 then + print ======data02=$data02 + goto loop4 +endi + +if $data12 != 1 then + print ======data12=$data12 + goto loop4 +endi + +if $data22 != 1 then + print ======data22=$data22 + goto loop4 +endi + +if $data32 != 1 then + print ======data32=$data32 + goto loop4 +endi + +if $data42 != 2 then + print ======data42=$data42 + goto loop4 +endi + +if $data52 != 5 then + print ======data52=$data52 + goto loop4 +endi + + +print sql select _irowts, _isfilled as a1, interp(a) as a2 from t2 partition by tbname range(1648791212000, 1648791217000) every(1s) fill(prev) order by 3, 1; +sql select _irowts, _isfilled as a1, interp(a) as a2 from t2 partition by tbname range(1648791212000, 1648791217000) every(1s) fill(prev) order by 3, 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +$loop_count = 0 + +loop5: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt where a2 > 10 order by 1; +sql select * from streamt where a2 > 10 order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +# row 0 +if $rows != 6 then + print ======rows=$rows + goto loop5 +endi + +if $data02 != 31 then + print ======data02=$data02 + goto loop5 +endi + +if $data12 != 31 then + print ======data12=$data12 + goto loop5 +endi + +if $data22 != 31 then + print ======data22=$data22 + goto loop5 +endi + +if $data32 != 31 then + print ======data32=$data32 + goto loop5 +endi + +if $data42 != 31 then + print ======data42=$data42 + goto loop5 +endi + +if $data52 != 61 then + print ======data52=$data52 + goto loop5 +endi + +print end + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpHistory1.sim b/tests/script/tsim/stream/streamInterpHistory1.sim new file mode 100644 index 0000000000..c4d558592c --- /dev/null +++ b/tests/script/tsim/stream/streamInterpHistory1.sim @@ -0,0 +1,737 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 1; +sql use test; + +sql create stable st(ts timestamp,a int,b int,c int, d double) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +sql insert into t1 values(1648791212000,1,1,1,1.0); +sql insert into t1 values(1648791215001,2,1,1,1.0); + +sql insert into t2 values(1648791212000,31,1,1,1.0); +sql insert into t2 values(1648791215001,41,1,1,1.0); + +sql create stream streams1 trigger at_once FILL_HISTORY 1 IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, _isfilled as a1, interp(a) as a2, tbname as tb from st partition by tbname every(1s) fill(NULL); + + +sql insert into t1 values(1648791217000,5,1,1,1.0); +sql insert into t2 values(1648791217000,61,1,1,1.0); + +print sql select _irowts, _isfilled as a1, interp(a) as a2 from t1 partition by tbname range(1648791212000, 1648791217000) every(1s) fill(NULL) order by 3, 1; +sql select _irowts, _isfilled as a1, interp(a) as a2 from t1 partition by tbname range(1648791212000, 1648791217000) every(1s) fill(NULL) order by 3, 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + + +$loop_count = 0 + +loop0: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt where tb = "t1" order by 1; +sql select * from streamt where tb = "t1" order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +# row 0 +if $rows != 6 then + print ======rows=$rows + goto loop0 +endi + +# row 0 +if $data02 != 1 then + print ======data02=$data02 + goto loop0 +endi + +if $data12 != NULL then + print ======data12=$data12 + goto loop0 +endi + +if $data22 != NULL then + print ======data22=$data22 + goto loop0 +endi + +if $data32 != NULL then + print ======data32=$data32 + goto loop0 +endi + +if $data42 != NULL then + print ======data42=$data42 + goto loop0 +endi + +if $data52 != 5 then + print ======data52=$data52 + goto loop0 +endi + + +print sql select _irowts, _isfilled as a1, interp(a) as a2 from t2 partition by tbname range(1648791212000, 1648791217000) every(1s) fill(NULL) order by 3, 1; +sql select _irowts, _isfilled as a1, interp(a) as a2 from t2 partition by tbname range(1648791212000, 1648791217000) every(1s) fill(NULL) order by 3, 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +$loop_count = 0 + +loop0_1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt where tb = "t2" order by 1; +sql select * from streamt where tb = "t2" order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +# row 0 +if $rows != 6 then + print ======rows=$rows + goto loop0_1 +endi + +if $data02 != 31 then + print ======data02=$data02 + goto loop0_1 +endi + +if $data12 != NULL then + print ======data12=$data12 + goto loop0_1 +endi + +if $data22 != NULL then + print ======data22=$data22 + goto loop0_1 +endi + +if $data32 != NULL then + print ======data32=$data32 + goto loop0_1 +endi + +if $data42 != NULL then + print ======data42=$data42 + goto loop0_1 +endi + +if $data52 != 61 then + print ======data52=$data52 + goto loop0_1 +endi + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791219001,7,1,1,1.0); +sql insert into t2 values(1648791219001,81,1,1,1.0); + +print sql select _irowts, _isfilled as a1, interp(a) as a2 from t1 partition by tbname range(1648791212000, 1648791219000) every(1s) fill(NULL) order by 3, 1; +sql select _irowts, _isfilled as a1, interp(a) as a2 from t1 partition by tbname range(1648791212000, 1648791219000) every(1s) fill(NULL) order by 3, 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 +print $data60 $data61 $data62 $data63 $data64 $data65 +print $data70 $data71 $data72 $data73 $data74 $data75 + +$loop_count = 0 + +loop1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt where tb = "t1" order by 1; +sql select * from streamt where tb = "t1" order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 +print $data60 $data61 $data62 $data63 $data64 $data65 +print $data70 $data71 $data72 $data73 $data74 $data75 + +if $rows != 8 then + print ======rows=$rows + goto loop1 +endi + +print sql select _irowts, _isfilled as a1, interp(a) as a2 from t2 partition by tbname range(1648791212000, 1648791219000) every(1s) fill(NULL) order by 3, 1; +sql select _irowts, _isfilled as a1, interp(a) as a2 from t2 partition by tbname range(1648791212000, 1648791219000) every(1s) fill(NULL) order by 3, 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 +print $data60 $data61 $data62 $data63 $data64 $data65 +print $data70 $data71 $data72 $data73 $data74 $data75 + +$loop_count = 0 + +loop1_1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt where tb = "t2" order by 1; +sql select * from streamt where tb = "t2" order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 +print $data60 $data61 $data62 $data63 $data64 $data65 +print $data70 $data71 $data72 $data73 $data74 $data75 + +if $rows != 8 then + print ======rows=$rows + goto loop1_1 +endi + +print step2 +print =============== create database +sql create database test2 vgroups 1; +sql use test2; + +sql create stable st(ts timestamp,a int,b int,c int, d double) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +sql insert into t1 values(1648791212000,1,1,1,1.0); +sql insert into t1 values(1648791215001,2,1,1,1.0); + +sql insert into t2 values(1648791212000,31,1,1,1.0); +sql insert into t2 values(1648791212001,41,1,1,1.0); + +sql create stream streams2 trigger at_once FILL_HISTORY 1 IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, _isfilled as a1, interp(a) as a2, tbname as tb from st partition by tbname every(1s) fill(value, 888); + + +sql insert into t1 values(1648791217000,5,1,1,1.0); +sql insert into t2 values(1648791217000,61,1,1,1.0); + +print sql select _irowts, _isfilled as a1, interp(a) as a2 from t1 partition by tbname range(1648791212000, 1648791217000) every(1s) fill(value, 888) order by 3, 1; +sql select _irowts, _isfilled as a1, interp(a) as a2 from t1 partition by tbname range(1648791212000, 1648791217000) every(1s) fill(value, 888) order by 3, 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + + +$loop_count = 0 + +loop2: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt where tb = "t1" order by 1; +sql select * from streamt where tb = "t1" order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +# row 0 +if $rows != 6 then + print ======rows=$rows + goto loop2 +endi + +# row 0 +if $data02 != 1 then + print ======data02=$data02 + goto loop2 +endi + +if $data12 != 888 then + print ======data12=$data12 + goto loop2 +endi + +if $data22 != 888 then + print ======data22=$data22 + goto loop2 +endi + +if $data32 != 888 then + print ======data32=$data32 + goto loop2 +endi + +if $data42 != 888 then + print ======data42=$data42 + goto loop2 +endi + +if $data52 != 5 then + print ======data52=$data52 + goto loop2 +endi + + +print sql select _irowts, _isfilled as a1, interp(a) as a2 from t2 partition by tbname range(1648791212000, 1648791217000) every(1s) fill(value, 888) order by 3, 1; +sql select _irowts, _isfilled as a1, interp(a) as a2 from t2 partition by tbname range(1648791212000, 1648791217000) every(1s) fill(value, 888) order by 3, 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +$loop_count = 0 + +loop2_1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt where tb = "t2" order by 1; +sql select * from streamt where tb = "t2" order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +# row 0 +if $rows != 6 then + print ======rows=$rows + goto loop2_1 +endi + +if $data02 != 31 then + print ======data02=$data02 + goto loop2_1 +endi + +if $data12 != 888 then + print ======data12=$data12 + goto loop2_1 +endi + +if $data22 != 888 then + print ======data22=$data22 + goto loop2_1 +endi + +if $data32 != 888 then + print ======data32=$data32 + goto loop2_1 +endi + +if $data42 != 888 then + print ======data42=$data42 + goto loop2_1 +endi + +if $data52 != 61 then + print ======data52=$data52 + goto loop2_1 +endi + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791219001,7,1,1,1.0); +sql insert into t2 values(1648791219001,81,1,1,1.0); + +print sql select _irowts, _isfilled as a1, interp(a) as a2 from t1 partition by tbname range(1648791212000, 1648791219000) every(1s) fill(value, 888) order by 3, 1; +sql select _irowts, _isfilled as a1, interp(a) as a2 from t1 partition by tbname range(1648791212000, 1648791219000) every(1s) fill(value, 888) order by 3, 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 +print $data60 $data61 $data62 $data63 $data64 $data65 +print $data70 $data71 $data72 $data73 $data74 $data75 + +$loop_count = 0 + +loop3: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt where tb = "t1" order by 1; +sql select * from streamt where tb = "t1" order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 +print $data60 $data61 $data62 $data63 $data64 $data65 +print $data70 $data71 $data72 $data73 $data74 $data75 + +if $rows != 8 then + print ======rows=$rows + goto loop3 +endi + +print sql select _irowts, _isfilled as a1, interp(a) as a2 from t2 partition by tbname range(1648791212000, 1648791219000) every(1s) fill(value, 888) order by 3, 1; +sql select _irowts, _isfilled as a1, interp(a) as a2 from t2 partition by tbname range(1648791212000, 1648791219000) every(1s) fill(value, 888) order by 3, 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 +print $data60 $data61 $data62 $data63 $data64 $data65 +print $data70 $data71 $data72 $data73 $data74 $data75 + +$loop_count = 0 + +loop3_1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt where tb = "t2" order by 1; +sql select * from streamt where tb = "t2" order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 +print $data60 $data61 $data62 $data63 $data64 $data65 +print $data70 $data71 $data72 $data73 $data74 $data75 + +if $rows != 8 then + print ======rows=$rows + goto loop3_1 +endi + + +print step3 +print =============== create database +sql create database test3 vgroups 1; +sql use test3; + +sql create stable st(ts timestamp,a int,b int,c int, d double) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +sql insert into t1 values(1648791212000,1,1,1,1.0); +sql insert into t1 values(1648791215001,20,1,1,1.0); + +sql insert into t2 values(1648791212000,31,1,1,1.0); +sql insert into t2 values(1648791215001,41,1,1,1.0); + +sql create stream streams3 trigger at_once FILL_HISTORY 1 IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, _isfilled as a1, interp(a) as a2, tbname as tb from st partition by tbname every(1s) fill(linear); + + +sql insert into t1 values(1648791217000,5,1,1,1.0); +sql insert into t2 values(1648791217000,61,1,1,1.0); + +print sql select _irowts, _isfilled as a1, interp(a) as a2 from t1 partition by tbname range(1648791212000, 1648791217000) every(1s) fill(linear) order by 1; +sql select _irowts, _isfilled as a1, interp(a) as a2 from t1 partition by tbname range(1648791212000, 1648791217000) every(1s) fill(linear) order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + + +$loop_count = 0 + +loop4: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt where tb = "t1" order by 1; +sql select * from streamt where tb = "t1" order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +# row 0 +if $rows != 6 then + print ======rows=$rows + goto loop4 +endi + +# row 0 +if $data02 != 1 then + print ======data02=$data02 + goto loop4 +endi + +if $data12 != 7 then + print ======data12=$data12 + goto loop4 +endi + +if $data22 != 13 then + print ======data22=$data22 + goto loop4 +endi + +if $data32 != 19 then + print ======data32=$data32 + goto loop4 +endi + +if $data42 != 12 then + print ======data42=$data42 + goto loop4 +endi + +if $data52 != 5 then + print ======data52=$data52 + goto loop4 +endi + + +print sql select _irowts, _isfilled as a1, interp(a) as a2 from t2 partition by tbname range(1648791212000, 1648791217000) every(1s) fill(linear) order by 1; +sql select _irowts, _isfilled as a1, interp(a) as a2 from t2 partition by tbname range(1648791212000, 1648791217000) every(1s) fill(linear) order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +$loop_count = 0 + +loop4_1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt where tb = "t2" order by 1; +sql select * from streamt where tb = "t2" order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +# row 0 +if $rows != 6 then + print ======rows=$rows + goto loop4_1 +endi + +if $data02 != 31 then + print ======data02=$data02 + goto loop4_1 +endi + +if $data12 != 34 then + print ======data12=$data12 + goto loop4_1 +endi + +if $data22 != 37 then + print ======data22=$data22 + goto loop4_1 +endi + +if $data32 != 40 then + print ======data32=$data32 + goto loop4_1 +endi + +if $data42 != 50 then + print ======data42=$data42 + goto loop4_1 +endi + +if $data52 != 61 then + print ======data52=$data52 + goto loop4_1 +endi + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791219001,7,1,1,1.0); +sql insert into t2 values(1648791219001,81,1,1,1.0); + +print sql select _irowts, _isfilled as a1, interp(a) as a2 from t1 partition by tbname range(1648791212000, 1648791219000) every(1s) fill(linear) order by 1; +sql select _irowts, _isfilled as a1, interp(a) as a2 from t1 partition by tbname range(1648791212000, 1648791219000) every(1s) fill(linear) order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 +print $data60 $data61 $data62 $data63 $data64 $data65 +print $data70 $data71 $data72 $data73 $data74 $data75 + +$loop_count = 0 + +loop5: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt where tb = "t1" order by 1; +sql select * from streamt where tb = "t1" order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 +print $data60 $data61 $data62 $data63 $data64 $data65 +print $data70 $data71 $data72 $data73 $data74 $data75 + +if $rows != 8 then + print ======rows=$rows + goto loop5 +endi + +print sql select _irowts, _isfilled as a1, interp(a) as a2 from t2 partition by tbname range(1648791212000, 1648791219000) every(1s) fill(linear) order by 1; +sql select _irowts, _isfilled as a1, interp(a) as a2 from t2 partition by tbname range(1648791212000, 1648791219000) every(1s) fill(linear) order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 +print $data60 $data61 $data62 $data63 $data64 $data65 +print $data70 $data71 $data72 $data73 $data74 $data75 + +$loop_count = 0 + +loop5_1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt where tb = "t2" order by 1; +sql select * from streamt where tb = "t2" order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 +print $data60 $data61 $data62 $data63 $data64 $data65 +print $data70 $data71 $data72 $data73 $data74 $data75 + +if $rows != 8 then + print ======rows=$rows + goto loop5_1 +endi + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpLarge.sim b/tests/script/tsim/stream/streamInterpLarge.sim new file mode 100644 index 0000000000..85203d2d9e --- /dev/null +++ b/tests/script/tsim/stream/streamInterpLarge.sim @@ -0,0 +1,188 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 1; +sql use test; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(prev); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648700000000,1,1,1,1.0) (1648710000000,100,100,100,100.0) (1648720000000,10,10,10,10.0); + +$loop_count = 0 + +loop0: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 30 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 +print $data10 $data11 $data12 $data13 + +# row 0 +if $rows != 20001 then + print ======rows=$rows + goto loop0 +endi + +print step2 +print =============== create database +sql create database test2 vgroups 1; +sql use test2; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(next); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648700000000,1,1,1,1.0) (1648710000000,100,100,100,100.0) (1648720000000,10,10,10,10.0); + +$loop_count = 0 + +loop2: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 30 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 +print $data10 $data11 $data12 $data13 + +# row 0 +if $rows != 20001 then + print ======rows=$rows + goto loop2 +endi + +print step3 +print =============== create database +sql create database test3 vgroups 1; +sql use test3; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(NULL); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648700000000,1,1,1,1.0) (1648710000000,100,100,100,100.0) (1648720000000,10,10,10,10.0); + +$loop_count = 0 + +loop3: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 30 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 +print $data10 $data11 $data12 $data13 + +# row 0 +if $rows != 20001 then + print ======rows=$rows + goto loop3 +endi + +print step4 +print =============== create database +sql create database test4 vgroups 1; +sql use test4; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams4 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(value, 1,2,3,4); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648700000000,1,1,1,1.0) (1648710000000,100,100,100,100.0) (1648720000000,10,10,10,10.0); + +$loop_count = 0 + +loop4: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 30 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 +print $data10 $data11 $data12 $data13 + +# row 0 +if $rows != 20001 then + print ======rows=$rows + goto loop4 +endi + +print step5 +print =============== create database +sql create database test5 vgroups 1; +sql use test5; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams5 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(linear); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648700000000,1,1,1,1.0) (1648710000000,100,100,100,100.0) (1648720000000,10,10,10,10.0); + +$loop_count = 0 + +loop5: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 30 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 +print $data10 $data11 $data12 $data13 + +# row 0 +if $rows != 20001 then + print ======rows=$rows + goto loop5 +endi + + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpLinear0.sim b/tests/script/tsim/stream/streamInterpLinear0.sim new file mode 100644 index 0000000000..7d4b28d545 --- /dev/null +++ b/tests/script/tsim/stream/streamInterpLinear0.sim @@ -0,0 +1,507 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 1; +sql use test; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(linear); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791213000,1,1,1,1.1); + +$loop_count = 0 + +loop0: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 +print $data10 $data11 $data12 $data13 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop0 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop0 +endi + + +sql insert into t1 values(1648791213001,2,2,2,2.1); +sql insert into t1 values(1648791213009,3,3,3,3.1); + +$loop_count = 0 + +loop1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 1 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 +print $data10 $data11 $data12 $data13 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop1 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop1 +endi + + +sql insert into t1 values(1648791217001,14,14,14,14.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791217001) every(1s) fill(linear); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791217001) every(1s) fill(linear); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +$loop_count = 0 +loop2: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop2 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop2 +endi + +if $data11 != 5 then + print ======data11=$data11 + goto loop2 +endi + +if $data21 != 8 then + print ======data21=$data21 + goto loop2 +endi + +if $data31 != 11 then + print ======data31=$data31 + goto loop2 +endi + +if $data41 != 13 then + print ======data41=$data41 + goto loop2 +endi + + +sql insert into t1 values(1648791215001,7,7,7,7.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791217001) every(1s) fill(linear); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791217001) every(1s) fill(linear); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 +loop3: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 3 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop3 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop3 +endi + +if $data11 != 4 then + print ======data11=$data11 + goto loop3 +endi + +if $data21 != 6 then + print ======data21=$data21 + goto loop3 +endi + +if $data31 != 10 then + print ======data31=$data31 + goto loop3 +endi + +if $data41 != 13 then + print ======data41=$data41 + goto loop3 +endi + + +print step2 + +sql create database test2 vgroups 1; +sql use test2; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2 as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(linear); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791212000,0,0,0,0.0) (1648791213001,11,11,11,11.0) (1648791213009,22,22,22,2.1) (1648791215001,15,15,15,15.1) (1648791217001,34,34,34,34.1); + + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(linear); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(linear); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 +loop4: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 1 sql select * from streamt2; +sql select * from streamt2; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 6 then + print ======rows=$rows + goto loop4 +endi + +# row 0 +if $data01 != 0 then + print ======data01=$data01 + goto loop4 +endi + +if $data11 != 10 then + print ======data11=$data11 + goto loop4 +endi + +if $data21 != 18 then + print ======data21=$data21 + goto loop4 +endi + +if $data31 != 15 then + print ======data31=$data31 + goto loop4 +endi + +if $data41 != 24 then + print ======data41=$data41 + goto loop4 +endi + +if $data51 != 33 then + print ======data51=$data51 + goto loop4 +endi + + +print step2_1 + +sql create database test2_1 vgroups 1; +sql use test2_1; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams2_1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2_1 as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(linear); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791212011,0,0,0,0.0) (1648791212099,20,20,20,20.0) (1648791213011,11,11,11,11.0) (1648791214099,35,35,35,35.1) (1648791215011,10,10,10,10.1) (1648791218099,34,34,34,34.1) (1648791219011,5,5,5,5.1); + + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791219011) every(1s) fill(linear); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791219011) every(1s) fill(linear); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 +print $data60 $data61 $data62 $data63 $data64 +print $data70 $data71 $data72 $data73 $data74 + +$loop_count = 0 +loop4_1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 1 sql select * from streamt2_1; +sql select * from streamt2_1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 +print $data60 $data61 $data62 $data63 $data64 +print $data70 $data71 $data72 $data73 $data74 + + +# row 0 +if $rows != 7 then + print ======rows=$rows + goto loop4_1 +endi + +# row 0 +if $data01 != 11 then + print ======data01=$data01 + goto loop4_1 +endi + +if $data11 != 32 then + print ======data11=$data11 + goto loop4_1 +endi + +if $data21 != 10 then + print ======data21=$data21 + goto loop4_1 +endi + +if $data31 != 17 then + print ======data31=$data31 + goto loop4_1 +endi + +if $data41 != 25 then + print ======data41=$data41 + goto loop4_1 +endi + +if $data51 != 33 then + print ======data51=$data51 + goto loop4_1 +endi + +if $data61 != 5 then + print ======data51=$data51 + goto loop4_1 +endi + + +print step3 +sql create database test3 vgroups 1; +sql use test3; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt3 as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(linear); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791212001,0,0,0,0.0) (1648791217001,8,4,4,4.1); + + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217000) every(1s) fill(linear); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217000) every(1s) fill(linear); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 +loop5: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 1 sql select * from streamt3; +sql select * from streamt3; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop5 +endi + +sql insert into t1 values(1648791213001,11,11,11,11.0) (1648791213009,22,22,22,22.1) (1648791215001,15,15,15,15.1) + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(linear); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(linear); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 +loop6: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt3; +sql select * from streamt3; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop6 +endi + +# row 0 +if $data01 != 10 then + print ======data01=$data01 + goto loop6 +endi + +if $data11 != 18 then + print ======data11=$data11 + goto loop6 +endi + +if $data21 != 15 then + print ======data21=$data21 + goto loop6 +endi + +if $data31 != 11 then + print ======data31=$data31 + goto loop6 +endi + +if $data41 != 8 then + print ======data41=$data41 + goto loop6 +endi + +print end +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpLinear1.sim b/tests/script/tsim/stream/streamInterpLinear1.sim new file mode 100644 index 0000000000..5151c47f62 --- /dev/null +++ b/tests/script/tsim/stream/streamInterpLinear1.sim @@ -0,0 +1,239 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 1; +sql use test; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(linear); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791213000,1,1,1,1.0); + +$loop_count = 0 + +loop0: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 +print $data10 $data11 $data12 $data13 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop0 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop0 +endi + + +sql insert into t1 values(1648791213009,30,3,3,1.0) (1648791217001,4,4,4,4.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791217001) every(1s) fill(linear); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791217001) every(1s) fill(linear); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +$loop_count = 0 +loop2: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop2 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop2 +endi + +if $data11 != 23 then + print ======data11=$data11 + goto loop2 +endi + +if $data21 != 17 then + print ======data21=$data21 + goto loop2 +endi + +if $data31 != 10 then + print ======data31=$data31 + goto loop2 +endi + +if $data41 != 4 then + print ======data41=$data41 + goto loop2 +endi + + +print step2 + +sql create database test2 vgroups 1; +sql use test2; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(linear); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791213000,1,1,1,1.0); + + +$loop_count = 0 + +loop3: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 +print $data10 $data11 $data12 $data13 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop3 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop3 +endi + + +sql insert into t1 values(1648791213009,30,3,3,1.0) (1648791217001,4,4,4,4.1) (1648791219000,50,5,5,5.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791219000) every(1s) fill(linear); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791219000) every(1s) fill(linear); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 +print $data60 $data61 $data62 $data63 $data64 +print $data70 $data71 $data72 $data73 $data74 + + +$loop_count = 0 +loop4: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 +print $data60 $data61 $data62 $data63 $data64 +print $data70 $data71 $data72 $data73 $data74 + +# row 0 +if $rows != 7 then + print ======rows=$rows + goto loop4 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop4 +endi + +if $data11 != 23 then + print ======data11=$data11 + goto loop4 +endi + +if $data21 != 17 then + print ======data21=$data21 + goto loop4 +endi + +if $data31 != 10 then + print ======data31=$data31 + goto loop4 +endi + +if $data41 != 4 then + print ======data41=$data41 + goto loop4 +endi + +if $data51 != 26 then + print ======data51=$data51 + goto loop4 +endi + +if $data61 != 50 then + print ======data61=$data61 + goto loop4 +endi + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpNext0.sim b/tests/script/tsim/stream/streamInterpNext0.sim new file mode 100644 index 0000000000..abdbeda634 --- /dev/null +++ b/tests/script/tsim/stream/streamInterpNext0.sim @@ -0,0 +1,437 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 1; +sql use test; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(next); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791213000,1,1,1,1.0); + +$loop_count = 0 + +loop0: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop0 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop0 +endi + + +sql insert into t1 values(1648791213001,2,2,2,1.1); +sql insert into t1 values(1648791213009,3,3,3,1.0); + +$loop_count = 0 + +loop1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 1 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop1 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop1 +endi + + +sql insert into t1 values(1648791217001,4,4,4,4.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791217001) every(1s) fill(next); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791217001) every(1s) fill(next); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +$loop_count = 0 +loop2: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop2 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop2 +endi + +if $data11 != 4 then + print ======data11=$data11 + goto loop2 +endi + +if $data21 != 4 then + print ======data21=$data21 + goto loop2 +endi + +if $data31 != 4 then + print ======data31=$data31 + goto loop2 +endi + +if $data41 != 4 then + print ======data41=$data41 + goto loop2 +endi + + +sql insert into t1 values(1648791215001,5,5,5,5.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791217001) every(1s) fill(next); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791217001) every(1s) fill(next); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 +loop3: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 3 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop3 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop3 +endi + +if $data11 != 5 then + print ======data11=$data11 + goto loop3 +endi + +if $data21 != 5 then + print ======data21=$data21 + goto loop3 +endi + +if $data31 != 4 then + print ======data31=$data31 + goto loop3 +endi + +if $data41 != 4 then + print ======data41=$data41 + goto loop3 +endi + +print step2 + +sql create database test2 vgroups 1; +sql use test2; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2 as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(next); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791212000,0,0,0,0.0) (1648791213001,1,1,1,1.0) (1648791213009,2,2,2,1.1) (1648791215001,5,5,5,5.1) (1648791217001,4,4,4,4.1); + + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(next); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(next); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 +loop4: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 1 sql select * from streamt2; +sql select * from streamt2; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 6 then + print ======rows=$rows + goto loop4 +endi + +# row 0 +if $data01 != 0 then + print ======data01=$data01 + goto loop4 +endi + +if $data11 != 1 then + print ======data11=$data11 + goto loop4 +endi + +if $data21 != 5 then + print ======data21=$data21 + goto loop4 +endi + +if $data31 != 5 then + print ======data31=$data31 + goto loop4 +endi + +if $data41 != 4 then + print ======data41=$data41 + goto loop4 +endi + +if $data51 != 4 then + print ======data51=$data51 + goto loop4 +endi + + + +print step3 +sql create database test3 vgroups 1; +sql use test3; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt3 as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(next); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791210001,0,0,0,0.0) (1648791217001,4,4,4,4.1); + + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791211000, 1648791217000) every(1s) fill(next); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791211000, 1648791217000) every(1s) fill(next); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 +print $data60 $data61 $data62 $data63 $data64 + +$loop_count = 0 +loop5: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 1 sql select * from streamt3; +sql select * from streamt3; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 +print $data60 $data61 $data62 $data63 $data64 + + +# row 0 +if $rows != 7 then + print ======rows=$rows + goto loop5 +endi + +sql insert into t1 values(1648791213001,1,1,1,1.0) (1648791213009,2,2,2,1.1) (1648791215001,5,5,5,5.1) + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791211000, 1648791217001) every(1s) fill(next); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791211000, 1648791217001) every(1s) fill(next); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 +print $data60 $data61 $data62 $data63 $data64 + +$loop_count = 0 +loop6: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt3; +sql select * from streamt3; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 +print $data60 $data61 $data62 $data63 $data64 + + +# row 0 +if $rows != 7 then + print ======rows=$rows + goto loop6 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop6 +endi + +if $data11 != 1 then + print ======data11=$data11 + goto loop6 +endi + +if $data21 != 1 then + print ======data21=$data21 + goto loop6 +endi + +if $data31 != 5 then + print ======data31=$data31 + goto loop6 +endi + +if $data41 != 5 then + print ======data41=$data41 + goto loop6 +endi + +if $data51 != 4 then + print ======data51=$data51 + goto loop6 +endi + +if $data61 != 4 then + print ======data61=$data61 + goto loop6 +endi + +print end +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpNext1.sim b/tests/script/tsim/stream/streamInterpNext1.sim new file mode 100644 index 0000000000..f74863d7a3 --- /dev/null +++ b/tests/script/tsim/stream/streamInterpNext1.sim @@ -0,0 +1,477 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 1; +sql use test; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(next); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791213000,1,1,1,1.0); + +$loop_count = 0 + +loop0: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop0 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop0 +endi + + +sql insert into t1 values(1648791213009,3,3,3,1.0) (1648791217001,4,4,4,4.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791217001) every(1s) fill(next); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791217001) every(1s) fill(next); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +$loop_count = 0 +loop2: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop2 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop2 +endi + +if $data11 != 4 then + print ======data11=$data11 + goto loop2 +endi + +if $data21 != 4 then + print ======data21=$data21 + goto loop2 +endi + +if $data31 != 4 then + print ======data31=$data31 + goto loop2 +endi + +if $data41 != 4 then + print ======data41=$data41 + goto loop2 +endi + + +print step2 + +sql create database test2 vgroups 1; +sql use test2; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(next); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791213000,1,1,1,1.0); + + +$loop_count = 0 + +loop3: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop3 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop3 +endi + + +sql insert into t1 values(1648791213009,3,3,3,1.0) (1648791217001,4,4,4,4.1) (1648791219000,5,5,5,5.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791219000) every(1s) fill(next); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791219000) every(1s) fill(next); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 +print $data60 $data61 $data62 $data63 $data64 +print $data70 $data71 $data72 $data73 $data74 + + +$loop_count = 0 +loop4: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 +print $data60 $data61 $data62 $data63 $data64 +print $data70 $data71 $data72 $data73 $data74 + +# row 0 +if $rows != 7 then + print ======rows=$rows + goto loop4 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop4 +endi + +if $data11 != 4 then + print ======data11=$data11 + goto loop4 +endi + +if $data21 != 4 then + print ======data21=$data21 + goto loop4 +endi + +if $data31 != 4 then + print ======data31=$data31 + goto loop4 +endi + +if $data41 != 4 then + print ======data41=$data41 + goto loop4 +endi + +if $data51 != 5 then + print ======data51=$data51 + goto loop4 +endi + +if $data61 != 5 then + print ======data61=$data61 + goto loop4 +endi + +print step3 + +sql create database test3 vgroups 1; +sql use test3; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(next); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791213001,1,1,1,1.0) (1648791219001,2,2,2,2.1) (1648791229001,3,3,3,3.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791229001) every(1s) fill(next); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791229001) every(1s) fill(next); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 +print $data60 $data61 $data62 $data63 $data64 +print $data70 $data71 $data72 $data73 $data74 + + +$loop_count = 0 +loop5: + +sleep 300 + +print sql select * from streamt order by 1; +sql select * from streamt order by 1; + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +if $rows != 16 then + print =====rows=$rows + goto loop5 +endi + +sql insert into t1 values(1648791215001,4,4,4,4.0) (1648791217001,5,5,5,5.1) (1648791222000,6,6,6,6.1) (1648791226000,7,7,7,7.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791229001) every(1s) fill(next); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791229001) every(1s) fill(next); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 +print $data60 $data61 $data62 $data63 $data64 +print $data70 $data71 $data72 $data73 $data74 + + +$loop_count = 0 +loop6: + +sleep 300 + +print sql select * from streamt order by 1; +sql select * from streamt order by 1; + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +if $rows != 16 then + goto loop6 +endi + +if $data01 != 4 then + print =====data01=$data01 + goto loop6 +endi + +if $data11 != 4 then + print =====data11=$data11 + goto loop6 +endi + +if $data21 != 5 then + print =====data21=$data21 + goto loop6 +endi + +if $data31 != 5 then + print =====data31=$data31 + goto loop6 +endi + +if $data41 != 2 then + print =====data41=$data41 + goto loop6 +endi + +if $data51 != 2 then + print =====data51=$data51 + goto loop6 +endi + +if $data61 != 6 then + print =====data61=$data61 + goto loop6 +endi + +if $data71 != 6 then + print =====data71=$data71 + goto loop6 +endi + +if $data81 != 6 then + print =====data81=$data81 + goto loop6 +endi + +if $data91 != 7 then + print =====data91=$data91 + goto loop6 +endi + +if $data[10][1] != 7 then + print =====data[10][1]=$data[10][1] + goto loop6 +endi + +if $data[11][1] != 7 then + print =====data[11][1]=$data[11][1] + goto loop6 +endi + +if $data[12][1] != 7 then + print =====data[12][1]=$data[12][1] + goto loop6 +endi + +if $data[13][1] != 3 then + print =====data[13][1]=$data[13][1] + goto loop6 +endi + +if $data[14][1] != 3 then + print =====data[14][1]=$data[14][1] + goto loop6 +endi + +if $data[15][1] != 3 then + print =====data[15][1]=$data[15][1] + goto loop6 +endi + + +print step4 + +sql create database test4 vgroups 1; +sql use test4; + +sql create stable st(ts timestamp,a int,b int,c int) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +sql create stream streams4 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt4 as select _irowts, interp(a) as b, _isfilled as a from st partition by tbname, b as cc every(1s) fill(next); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791217000,20000,2,3); + +sleep 2000 + +sql insert into t1 values(1648791212000,10000,2,3) (1648791215001,20,2,3); + +$loop_count = 0 +loop7: + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +sleep 300 + +print sql select a,b from streamt4; +sql select a,b from streamt4; + +if $rows != 6 then + print ======rows=$rows + goto loop7 +endi + +if $data00 != 0 then + print ======data00=$data00 + goto loop7 +endi + +if $data01 != 10000 then + print ======data01=$data01 + goto loop7 +endi + +if $data10 != 1 then + print ======data10=$data10 + goto loop7 +endi + +if $data20 != 1 then + print ======data20=$data20 + goto loop7 +endi + +if $data41 != 20000 then + print ======data41=$data41 + goto loop7 +endi + +if $data50 != 0 then + print ======data50=$data50 + goto loop7 +endi + +if $data51 != 20000 then + print ======data51=$data51 + goto loop7 +endi + +print end + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpOther.sim b/tests/script/tsim/stream/streamInterpOther.sim new file mode 100644 index 0000000000..8553e67ec8 --- /dev/null +++ b/tests/script/tsim/stream/streamInterpOther.sim @@ -0,0 +1,608 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 4; +sql use test; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams1_1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt1_1 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(prev); +sql create stream streams1_2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt1_2 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(next); +sql create stream streams1_3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt1_3 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(linear); +sql create stream streams1_4 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt1_4 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(NULL); +sql create stream streams1_5 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt1_5 as select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 every(1s) fill(value,11,22,33,44); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791215000,0,0,0,0.0); + +sql insert into t1 values(1648791212000,10,10,10,10.0); + +$loop_count = 0 +loop0: + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +sleep 300 + +print sql desc streamt1_1; +sql desc streamt1_1; + +if $rows != 9 then + print ======rows=$rows + goto loop0 +endi + +sql desc streamt1_2; + +if $rows != 9 then + print ======rows=$rows + goto loop0 +endi + +sql desc streamt1_3; + +if $rows != 9 then + print ======rows=$rows + goto loop0 +endi + +sql desc streamt1_4; + +if $rows != 9 then + print ======rows=$rows + goto loop0 +endi + +sql desc streamt1_5; + +if $rows != 9 then + print ======rows=$rows + goto loop0 +endi + +$loop_count = 0 +loop0_1: + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +sleep 300 + +print sql select * from streamt1_1; +sql select * from streamt1_1; + +if $rows != 4 then + print ======rows=$rows + goto loop0_1 +endi + +print sql select * from streamt1_2; +sql select * from streamt1_2; + +if $rows != 4 then + print ======rows=$rows + goto loop0_1 +endi + +print sql select * from streamt1_3; +sql select * from streamt1_3; + +if $rows != 4 then + print ======rows=$rows + goto loop0_1 +endi + +print sql select * from streamt1_4; +sql select * from streamt1_4; + +if $rows != 4 then + print ======rows=$rows + goto loop0_1 +endi + +print sql select * from streamt1_5; +sql select * from streamt1_5; + +if $rows != 4 then + print ======rows=$rows + goto loop0_1 +endi + +print sql select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 range(1648791212000, 1648791215000) every(1s) fill(value,11,22,33,44); +sql select interp(a), _isfilled as a1, interp(b), _isfilled as a2, interp(c), _isfilled as a3, interp(d) from t1 range(1648791212000, 1648791215000) every(1s) fill(value,11,22,33,44); + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +$loop_count = 0 +loop0_2: + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +sleep 300 + +print sql select * from streamt1_5; +sql select * from streamt1_5; + +if $data01 != 10 then + print ======data01=$data01 + goto loop0_2 +endi + +if $data02 != 0 then + print ======data02=$data02 + goto loop0_2 +endi + +if $data03 != 10 then + print ======data03=$data03 + goto loop0_2 +endi + +if $data04 != 0 then + print ======data04=$data04 + goto loop0_2 +endi + +if $data05 != 10 then + print ======data05=$data05 + goto loop0_2 +endi + +if $data06 != 0 then + print ======data06=$data06 + goto loop0_2 +endi + +if $data07 != 10.000000000 then + print ======data07=$data07 + goto loop0_2 +endi + +if $data11 != 11 then + print ======data11=$data11 + goto loop0_2 +endi + +if $data12 != 1 then + print ======data12=$data12 + goto loop0_2 +endi + +if $data13 != 22 then + print ======data13=$data13 + goto loop0_2 +endi + +if $data14 != 1 then + print ======data14=$data14 + goto loop0_2 +endi + +if $data15 != 33 then + print ======data15=$data15 + goto loop0_2 +endi + +if $data16 != 1 then + print ======data16=$data16 + goto loop0_2 +endi + +if $data17 != 44.000000000 then + print ======data17=$data17 + goto loop0_2 +endi + +print step3 + +sql create database test3 vgroups 4; +sql use test3; + +sql create stable st(ts timestamp,a int,b int,c int) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + + +sql create stream streams3_1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt3_1 TAGS(cc varchar(100)) SUBTABLE(concat(concat("tbn-", tbname), "_1")) as select interp(a), _isfilled as a1 from st partition by tbname, b as cc every(1s) fill(prev); +sql create stream streams3_2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt3_2 TAGS(cc varchar(100)) SUBTABLE(concat(concat("tbn-", tbname), "_2")) as select interp(a), _isfilled as a1 from st partition by tbname, b as cc every(1s) fill(next); +sql create stream streams3_3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt3_3 TAGS(cc varchar(100)) SUBTABLE(concat(concat("tbn-", tbname), "_3")) as select interp(a), _isfilled as a1 from st partition by tbname, b as cc every(1s) fill(linear); +sql create stream streams3_4 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt3_4 TAGS(cc varchar(100)) SUBTABLE(concat(concat("tbn-", tbname), "_4")) as select interp(a), _isfilled as a1 from st partition by tbname, b as cc every(1s) fill(NULL); +sql create stream streams3_5 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt3_5 TAGS(cc varchar(100)) SUBTABLE(concat(concat("tbn-", tbname), "_5")) as select interp(a), _isfilled as a1 from st partition by tbname, b as cc every(1s) fill(value,11); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791217000,1,2,3); + +sleep 500 + +sql insert into t1 values(1648791212000,10,2,3); + +sleep 500 + +sql insert into t1 values(1648791215001,20,2,3); + +sleep 500 + +sql insert into t2 values(1648791215001,20,2,3); + +$loop_count = 0 +loop3: + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +sleep 300 + +print sql select cc, * from `tbn-t1_1_streamt3_1_914568691400502130`; +sql select cc, * from `tbn-t1_1_streamt3_1_914568691400502130`; + +if $rows != 6 then + print ======rows=$rows + goto loop3 +endi + +if $data00 != 2 then + print ======rows=$rows + goto loop3 +endi + +print sql select cc, * from `tbn-t1_2_streamt3_2_914568691400502130`; +sql select cc, * from `tbn-t1_2_streamt3_2_914568691400502130`; + +if $rows != 6 then + print ======rows=$rows + goto loop3 +endi + +if $data00 != 2 then + print ======rows=$rows + goto loop3 +endi + +print sql select cc, * from `tbn-t1_3_streamt3_3_914568691400502130`; +sql select cc, * from `tbn-t1_3_streamt3_3_914568691400502130`; + +if $rows != 6 then + print ======rows=$rows + goto loop3 +endi + +if $data00 != 2 then + print ======rows=$rows + goto loop3 +endi + +print sql select cc, * from `tbn-t1_4_streamt3_4_914568691400502130`; +sql select cc, * from `tbn-t1_4_streamt3_4_914568691400502130`; + +if $rows != 6 then + print ======rows=$rows + goto loop3 +endi + +if $data00 != 2 then + print ======rows=$rows + goto loop3 +endi + +print sql select cc, * from `tbn-t1_5_streamt3_5_914568691400502130`; +sql select cc, * from `tbn-t1_5_streamt3_5_914568691400502130`; + +if $rows != 6 then + print ======rows=$rows + goto loop3 +endi + +if $data00 != 2 then + print ======rows=$rows + goto loop3 +endi + + + +print sql select * from `tbn-t2_1_streamt3_1_8905952758123525205`; +sql select * from `tbn-t2_1_streamt3_1_8905952758123525205`; + +if $rows != 0 then + print ======rows=$rows + goto loop3 +endi + +print sql select * from `tbn-t2_2_streamt3_2_8905952758123525205`; +sql select * from `tbn-t2_2_streamt3_2_8905952758123525205`; + +if $rows != 0 then + print ======rows=$rows + goto loop3 +endi + +print sql select * from `tbn-t2_3_streamt3_3_8905952758123525205`; +sql select * from `tbn-t2_3_streamt3_3_8905952758123525205`; + +if $rows != 0 then + print ======rows=$rows + goto loop3 +endi + +print sql select * from `tbn-t2_4_streamt3_4_8905952758123525205`; +sql select * from `tbn-t2_4_streamt3_4_8905952758123525205`; + +if $rows != 0 then + print ======rows=$rows + goto loop3 +endi + +print sql select * from `tbn-t2_5_streamt3_5_8905952758123525205`; +sql select * from `tbn-t2_5_streamt3_5_8905952758123525205`; + +if $rows != 0 then + print ======rows=$rows + goto loop3 +endi + + +print step4 +print =============== create database +sql drop database if exists test4; +sql create database test4 vgroups 4; +sql use test4; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams4_1 trigger at_once watermark 10s IGNORE EXPIRED 1 IGNORE UPDATE 0 into streamt4_1 as select interp(a, 1), _isfilled as a1 from t1 every(1s) fill(prev); +sql create stream streams4_2 trigger at_once watermark 10s IGNORE EXPIRED 1 IGNORE UPDATE 0 into streamt4_2 as select interp(a, 1), _isfilled as a1 from t1 every(1s) fill(next); +sql create stream streams4_3 trigger at_once watermark 10s IGNORE EXPIRED 1 IGNORE UPDATE 0 into streamt4_3 as select interp(a, 1), _isfilled as a1 from t1 every(1s) fill(linear); +sql create stream streams4_4 trigger at_once watermark 10s IGNORE EXPIRED 1 IGNORE UPDATE 0 into streamt4_4 as select interp(a, 1), _isfilled as a1 from t1 every(1s) fill(NULL); +sql create stream streams4_5 trigger at_once watermark 10s IGNORE EXPIRED 1 IGNORE UPDATE 0 into streamt4_5 as select interp(a, 1), _isfilled as a1 from t1 every(1s) fill(value,11); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791275000,NULL,0,0,0.0); + +sleep 500 + +sql insert into t1 values(1648791276000,NULL,1,0,0.0) (1648791277000,NULL,2,0,0.0) (1648791275000,NULL,3,0,0.0); + +$loop_count = 0 +loop4: + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +sleep 300 + +print sql select * from streamt4_1; +sql select * from streamt4_1; + +if $rows != 0 then + print ======rows=$rows + goto loop4 +endi + +print sql select * from streamt4_2; +sql select * from streamt4_2; + +if $rows != 0 then + print ======rows=$rows + goto loop4 +endi + +print sql select * from streamt4_3; +sql select * from streamt4_3; + +if $rows != 0 then + print ======rows=$rows + goto loop4 +endi + +print sql select * from streamt4_4; +sql select * from streamt4_4; + +if $rows != 0 then + print ======rows=$rows + goto loop4 +endi + +print sql select * from streamt4_5; +sql select * from streamt4_5; + +if $rows != 0 then + print ======rows=$rows + goto loop4 +endi + +print step4_2 + +print sql insert into t1 values(1648791215000,1,0,0,0.0); +sql insert into t1 values(1648791215000,1,0,0,0.0); +sleep 500 + +sql insert into t1 values(1648791216000,2,1,0,0.0) (1648791217000,3,2,0,0.0) (1648791215000,4,3,0,0.0); + +$loop_count = 0 +loop5: + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +sleep 300 + +print sql select * from streamt4_1; +sql select * from streamt4_1; + +if $rows != 0 then + print ======rows=$rows + goto loop5 +endi + +print sql select * from streamt4_2; +sql select * from streamt4_2; + +if $rows != 0 then + print ======rows=$rows + goto loop5 +endi + +print sql select * from streamt4_3; +sql select * from streamt4_3; + +if $rows != 0 then + print ======rows=$rows + goto loop5 +endi + +print sql select * from streamt4_4; +sql select * from streamt4_4; + +if $rows != 0 then + print ======rows=$rows + goto loop5 +endi + +print sql select * from streamt4_5; +sql select * from streamt4_5; + +if $rows != 0 then + print ======rows=$rows + goto loop5 +endi + +print step4_3 + +print sql insert into t1 values(1648791278000,NULL,2,0,0.0) (1648791278001,NULL,2,0,0.0) (1648791279000,1,2,0,0.0) (1648791279001,NULL,2,0,0.0) (1648791280000,NULL,2,0,0.0)(1648791280001,NULL,2,0,0.0)(1648791281000,20,2,0,0.0) (1648791281001,NULL,2,0,0.0)(1648791281002,NULL,2,0,0.0) (1648791282000,NULL,2,0,0.0); +sql insert into t1 values(1648791278000,NULL,2,0,0.0) (1648791278001,NULL,2,0,0.0) (1648791279000,1,2,0,0.0) (1648791279001,NULL,2,0,0.0) (1648791280000,NULL,2,0,0.0)(1648791280001,NULL,2,0,0.0)(1648791281000,20,2,0,0.0) (1648791281001,NULL,2,0,0.0)(1648791281002,NULL,2,0,0.0) (1648791282000,NULL,2,0,0.0); + +$loop_count = 0 +loop6: + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +sleep 300 + +print sql select * from streamt4_1; +sql select * from streamt4_1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +if $rows != 3 then + print ======rows=$rows + goto loop6 +endi + +if $data12 != 1 then + print ======data12=$data12 + goto loop6 +endi + +print sql select * from streamt4_2; +sql select * from streamt4_2; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +if $rows != 3 then + print ======rows=$rows + goto loop6 +endi + +if $data12 != 1 then + print ======data12=$data12 + goto loop6 +endi + +print sql select * from streamt4_3; +sql select * from streamt4_3; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +if $rows != 3 then + print ======rows=$rows + goto loop6 +endi + +if $data12 != 1 then + print ======data12=$data12 + goto loop6 +endi + +print sql select * from streamt4_4; +sql select * from streamt4_4; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +if $rows != 3 then + print ======rows=$rows + goto loop6 +endi + +if $data12 != 1 then + print ======data12=$data12 + goto loop6 +endi + +print sql select * from streamt4_5; +sql select * from streamt4_5; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +if $rows != 3 then + print ======rows=$rows + goto loop6 +endi + +if $data12 != 1 then + print ======data12=$data12 + goto loop6 +endi + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpOther1.sim b/tests/script/tsim/stream/streamInterpOther1.sim new file mode 100644 index 0000000000..941b3e18f0 --- /dev/null +++ b/tests/script/tsim/stream/streamInterpOther1.sim @@ -0,0 +1,510 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step4 + +sql create database test4_1 vgroups 4; +sql use test4_1; + +sql create stable st(ts timestamp,a int,b int,c int) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +sql create stable streamt4_1(ts timestamp,a varchar(10),b tinyint,c tinyint) tags(ta int,cc int,tc int); + +sql create stream streams4_1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt4_1(ts, b, a) TAGS(cc) SUBTABLE(concat(concat("tbn-", tbname), "_1")) as select _irowts, interp(a), _isfilled as a1 from st partition by tbname, b as cc every(1s) fill(prev); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791217000,20000,2,3); + +sleep 2000 + +sql insert into t1 values(1648791212000,10000,2,3); + +sleep 2000 + +sql insert into t1 values(1648791215001,20,2,3); + +$loop_count = 0 +loop4_1: + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +sleep 300 + +print sql select a,b from streamt4_1; +sql select a,b from streamt4_1; + +if $rows != 6 then + print ======rows=$rows + goto loop4_1 +endi + +if $data00 != false then + print ======data00=$data00 + goto loop4_1 +endi + +if $data01 != 16 then + print ======data01=$data01 + goto loop4_1 +endi + +if $data10 != true then + print ======data10=$data10 + goto loop4_1 +endi + +if $data20 != true then + print ======data20=$data20 + goto loop4_1 +endi + +if $data50 != false then + print ======data50=$data50 + goto loop4_1 +endi + +if $data51 != 32 then + print ======data51=$data51 + goto loop4_1 +endi + +print step4_2 + +sql create database test4_2 vgroups 4; +sql use test4_2; + +sql create stable st(ts timestamp,a int,b int,c int) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +sql create stable streamt4_2(ts timestamp,a varchar(10),b tinyint,c tinyint) tags(ta int,cc int,tc int); + +sql create stream streams4_2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt4_2(ts, b, a) TAGS(cc) SUBTABLE(concat(concat("tbn-", tbname), "_2")) as select _irowts, interp(a), _isfilled as a1 from st partition by tbname, b as cc every(1s) fill(next); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791217000,20000,2,3); + +sleep 2000 + +sql insert into t1 values(1648791212000,10000,2,3); + +sleep 2000 + +sql insert into t1 values(1648791215001,20,2,3); + +$loop_count = 0 +loop4_2: + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +sleep 300 + +print sql select a,b from streamt4_2; +sql select a,b from streamt4_2; + +if $rows != 6 then + print ======rows=$rows + goto loop4_2 +endi + +if $data00 != false then + print ======data00=$data00 + goto loop4_2 +endi + +if $data01 != 16 then + print ======data01=$data01 + goto loop4_2 +endi + +if $data10 != true then + print ======data10=$data10 + goto loop4_2 +endi + +if $data20 != true then + print ======data20=$data20 + goto loop4_2 +endi + +if $data50 != false then + print ======data50=$data50 + goto loop4_2 +endi + +if $data51 != 32 then + print ======data51=$data51 + goto loop4_2 +endi + +print step4_3 + +sql create database test4_3 vgroups 4; +sql use test4_3; + +sql create stable st(ts timestamp,a int,b int,c int) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +sql create stable streamt4_3(ts timestamp,a varchar(10),b tinyint,c tinyint) tags(ta int,cc int,tc int); + +sql create stream streams4_3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt4_3(ts, b, a) TAGS(cc) SUBTABLE(concat(concat("tbn-", tbname), "_3")) as select _irowts, interp(a), _isfilled as a1 from st partition by tbname, b as cc every(1s) fill(linear); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791217000,20000,2,3); + +sleep 2000 + +sql insert into t1 values(1648791212000,10000,2,3); + +sleep 2000 + +sql insert into t1 values(1648791215001,20,2,3); + +$loop_count = 0 +loop4_3: + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +sleep 300 + +print sql select a,b from streamt4_3; +sql select a,b from streamt4_3; + +if $rows != 6 then + print ======rows=$rows + goto loop4_3 +endi + +if $data00 != false then + print ======data00=$data00 + goto loop4_3 +endi + +if $data01 != 16 then + print ======data01=$data01 + goto loop4_3 +endi + +if $data10 != true then + print ======data10=$data10 + goto loop4_3 +endi + +if $data20 != true then + print ======data20=$data20 + goto loop4_3 +endi + +if $data50 != false then + print ======data50=$data50 + goto loop4_3 +endi + +if $data51 != 32 then + print ======data51=$data51 + goto loop4_3 +endi + +print step4_4 + +sql create database test4_4 vgroups 4; +sql use test4_4; + +sql create stable st(ts timestamp,a int,b int,c int) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +sql create stable streamt4_4(ts timestamp,a varchar(10),b tinyint,c tinyint) tags(ta int,cc int,tc int); + +sql create stream streams4_4 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt4_4(ts, b, a) TAGS(cc) SUBTABLE(concat(concat("tbn-", tbname), "_4")) as select _irowts, interp(a), _isfilled as a1 from st partition by tbname, b as cc every(1s) fill(NULL); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791217000,20000,2,3); + +sleep 2000 + +sql insert into t1 values(1648791212000,10000,2,3); + +sleep 2000 + +sql insert into t1 values(1648791215001,20,2,3); + +$loop_count = 0 +loop4_4: + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +sleep 300 + +print sql select a,b from streamt4_4; +sql select a,b from streamt4_4; + +if $rows != 6 then + print ======rows=$rows + goto loop4_4 +endi + +if $data00 != false then + print ======data00=$data00 + goto loop4_4 +endi + +if $data01 != 16 then + print ======data01=$data01 + goto loop4_4 +endi + +if $data10 != true then + print ======data10=$data10 + goto loop4_4 +endi + +if $data20 != true then + print ======data20=$data20 + goto loop4_4 +endi + +if $data50 != false then + print ======data50=$data50 + goto loop4_4 +endi + +if $data51 != 32 then + print ======data51=$data51 + goto loop4_4 +endi + +print step4_5 + +sql create database test4_5 vgroups 4; +sql use test4_5; + +sql create stable st(ts timestamp,a int,b int,c int) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +sql create stable streamt4_5(ts timestamp,a varchar(10),b tinyint,c tinyint) tags(ta int,cc int,tc int); + +sql create stream streams4_5 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt4_5(ts, b, a) TAGS(cc) SUBTABLE(concat(concat("tbn-", tbname), "_5")) as select _irowts, interp(a), _isfilled as a1 from st partition by tbname, b as cc every(1s) fill(value,1100); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791217000,20000,2,3); + +sleep 2000 + +sql insert into t1 values(1648791212000,10000,2,3); + +sleep 2000 + +sql insert into t1 values(1648791215001,20,2,3); + +$loop_count = 0 +loop4_5: + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +sleep 300 + +print sql select a,b from streamt4_5; +sql select a,b from streamt4_5; + +if $rows != 6 then + print ======rows=$rows + goto loop4_5 +endi + +if $data00 != false then + print ======data00=$data00 + goto loop4_5 +endi + +if $data01 != 16 then + print ======data01=$data01 + goto loop4_5 +endi + +if $data10 != true then + print ======data10=$data10 + goto loop4_5 +endi + +if $data20 != true then + print ======data20=$data20 + goto loop4_5 +endi + +if $data50 != false then + print ======data50=$data50 + goto loop4_5 +endi + +if $data51 != 32 then + print ======data51=$data51 + goto loop4_5 +endi + +print step5 +print =============== create database +sql drop database if exists test5; +sql create database test5 vgroups 4 precision 'us'; +sql use test5; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams5 trigger at_once watermark 10s IGNORE EXPIRED 1 IGNORE UPDATE 0 into streamt as select interp(a), _isfilled as a1 from t1 every(1s) fill(prev); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791213000001,1,1,1,1.0) (1648791215000001,20,1,1,1.0) (1648791216000000,3,1,1,1.0); + +$loop_count = 0 +loop5: + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +sleep 300 + +print sql select cast(`_irowts` as bigint) from streamt order by 1; +sql select cast(`_irowts` as bigint) from streamt order by 1; + +if $rows != 3 then + print ======rows=$rows + goto loop5 +endi + +if $data00 != 1648791214000000 then + print ======data00=$data00 + goto loop5 +endi + +if $data10 != 1648791215000000 then + print ======data01=$data01 + goto loop5 +endi + +if $data20 != 1648791216000000 then + print ======data01=$data01 + goto loop5 +endi + +print step6 +print =============== create database +sql drop database if exists test6; +sql create database test6 vgroups 4 precision 'us'; +sql use test6; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams6 trigger at_once watermark 10s IGNORE EXPIRED 1 IGNORE UPDATE 0 into streamt as select interp(a), _isfilled as a1 from t1 every(1s) fill(next); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791213000001,1,1,1,1.0) (1648791215000001,20,1,1,1.0) (1648791216000000,3,1,1,1.0); + +$loop_count = 0 +loop6: + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +sleep 300 + +print sql select cast(`_irowts` as bigint) from streamt order by 1; +sql select cast(`_irowts` as bigint) from streamt order by 1; + +if $rows != 3 then + print ======rows=$rows + goto loop6 +endi + +if $data00 != 1648791214000000 then + print ======data00=$data00 + goto loop6 +endi + +if $data10 != 1648791215000000 then + print ======data01=$data01 + goto loop6 +endi + +if $data20 != 1648791216000000 then + print ======data01=$data01 + goto loop6 +endi + +print step7 +print =============== create database +sql drop database if exists test7; +sql create database test7 vgroups 4 precision 'us'; +sql use test7; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams7 trigger at_once watermark 10s IGNORE EXPIRED 1 IGNORE UPDATE 0 into streamt as select interp(a), _isfilled as a1 from t1 every(1s) fill(linear); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791213000001,1,1,1,1.0) (1648791215000001,20,1,1,1.0) (1648791216000000,3,1,1,1.0); + +$loop_count = 0 +loop7: + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +sleep 300 + +print sql select cast(`_irowts` as bigint) from streamt order by 1; +sql select cast(`_irowts` as bigint) from streamt order by 1; + +if $rows != 3 then + print ======rows=$rows + goto loop7 +endi + +if $data00 != 1648791214000000 then + print ======data00=$data00 + goto loop7 +endi + +if $data10 != 1648791215000000 then + print ======data01=$data01 + goto loop7 +endi + +if $data20 != 1648791216000000 then + print ======data01=$data01 + goto loop7 +endi + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpOther2.sim b/tests/script/tsim/stream/streamInterpOther2.sim new file mode 100644 index 0000000000..25d5171a5c --- /dev/null +++ b/tests/script/tsim/stream/streamInterpOther2.sim @@ -0,0 +1,525 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step prev + +sql create database test1 vgroups 4; +sql use test1; + +sql create stable st(ts timestamp,a int,b int,c int) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt1 as select _irowts, _isfilled as a1, interp(b, 1) from st partition by tbname every(1s) fill(prev); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791217000,0,0,3) (1648791212000,10,10,3) (1648791212001,11,NULL,3); + +sleep 500 + +sql insert into t1 values(1648791214001,20,NULL,3) (1648791213000,22,NULL,3); + +print sql select _irowts, _isfilled as a1, interp(b, 1) from st partition by tbname range(1648791212000, 1648791217000) every(1s) fill(prev); +sql select _irowts, _isfilled as a1, interp(b, 1) from st partition by tbname range(1648791212000, 1648791217000) every(1s) fill(prev); + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +$loop_count = 0 +loop0: + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +sleep 300 + +print sql select * from streamt1; +sql select * from streamt1; + +if $rows != 6 then + print ======rows=$rows + goto loop0 +endi + +if $data01 != 0 then + print ======data01=$data01 + goto loop0 +endi + +if $data02 != 10 then + print ======data02=$data02 + goto loop0 +endi + +if $data11 != 1 then + print ======data11=$data11 + goto loop0 +endi + +if $data12 != 10 then + print ======data12=$data12 + goto loop0 +endi + +if $data21 != 1 then + print ======data21=$data21 + goto loop0 +endi + +if $data22 != 10 then + print ======data22=$data22 + goto loop0 +endi + +if $data31 != 1 then + print ======data31=$data31 + goto loop0 +endi + +if $data32 != 10 then + print ======data32=$data32 + goto loop0 +endi + +if $data41 != 1 then + print ======data41=$data41 + goto loop0 +endi + +if $data42 != 10 then + print ======data42=$data42 + goto loop0 +endi + +if $data51 != 0 then + print ======data51=$data51 + goto loop0 +endi + +if $data52 != 0 then + print ======data52=$data52 + goto loop0 +endi + +print step next + +sql create database test2 vgroups 4; +sql use test2; + +sql create stable st(ts timestamp,a int,b int,c int) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2 as select _irowts, _isfilled as a1, interp(b, 1) from st partition by tbname every(1s) fill(next); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791217000,11,11,3) (1648791212000,10,10,3) (1648791212001,11,NULL,3); + +sleep 500 + +sql insert into t1 values(1648791214001,20,NULL,3) (1648791213000,22,NULL,3); + +print sql select _irowts, _isfilled as a1, interp(b, 1) from st partition by tbname range(1648791212000, 1648791217000) every(1s) fill(next); +sql select _irowts, _isfilled as a1, interp(b, 1) from st partition by tbname range(1648791212000, 1648791217000) every(1s) fill(next); + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +$loop_count = 0 +loop1: + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +sleep 300 + +print sql select * from streamt2; +sql select * from streamt2; + +if $rows != 6 then + print ======rows=$rows + goto loop1 +endi + +if $data01 != 0 then + print ======data01=$data01 + goto loop1 +endi + +if $data02 != 10 then + print ======data02=$data02 + goto loop1 +endi + +if $data11 != 1 then + print ======data11=$data11 + goto loop1 +endi + +if $data12 != 11 then + print ======data12=$data12 + goto loop1 +endi + +if $data21 != 1 then + print ======data21=$data21 + goto loop1 +endi + +if $data22 != 11 then + print ======data22=$data22 + goto loop1 +endi + +if $data31 != 1 then + print ======data31=$data31 + goto loop1 +endi + +if $data32 != 11 then + print ======data32=$data32 + goto loop1 +endi + +if $data41 != 1 then + print ======data41=$data41 + goto loop1 +endi + +if $data42 != 11 then + print ======data42=$data42 + goto loop1 +endi + +if $data51 != 0 then + print ======data51=$data51 + goto loop1 +endi + +if $data52 != 11 then + print ======data52=$data52 + goto loop1 +endi + +print step value + +sql create database test3 vgroups 4; +sql use test3; + +sql create stable st(ts timestamp,a int,b int,c int) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +sql create stream streams3_1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt3_1 as select _irowts, _isfilled as a1, interp(b, 1) from st partition by tbname every(1s) fill(NULL); +sql create stream streams3_2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt3_2 as select _irowts, _isfilled as a1, interp(b, 1) from st partition by tbname every(1s) fill(value, 110); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791217000,11,11,3) (1648791212000,10,10,3) (1648791212001,11,NULL,3); + +sleep 500 + +sql insert into t1 values(1648791214001,20,NULL,3) (1648791213000,22,NULL,3); + +print sql select _irowts, _isfilled as a1, interp(b, 1) from st partition by tbname range(1648791212000, 1648791217000) every(1s) fill(NULL); +sql select _irowts, _isfilled as a1, interp(b, 1) from st partition by tbname range(1648791212000, 1648791217000) every(1s) fill(NULL); + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +$loop_count = 0 +loop3: + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +sleep 300 + +print sql select * from streamt3_1; +sql select * from streamt3_1; + +if $rows != 6 then + print ======rows=$rows + goto loop3 +endi + +if $data01 != 0 then + print ======data01=$data01 + goto loop3 +endi + +if $data02 != 10 then + print ======data02=$data02 + goto loop3 +endi + +if $data11 != 1 then + print ======data11=$data11 + goto loop3 +endi + +if $data12 != NULL then + print ======data12=$data12 + goto loop3 +endi + +if $data21 != 1 then + print ======data21=$data21 + goto loop3 +endi + +if $data22 != NULL then + print ======data22=$data22 + goto loop3 +endi + +if $data31 != 1 then + print ======data31=$data31 + goto loop3 +endi + +if $data32 != NULL then + print ======data32=$data32 + goto loop3 +endi + +if $data41 != 1 then + print ======data41=$data41 + goto loop3 +endi + +if $data42 != NULL then + print ======data42=$data42 + goto loop3 +endi + +if $data51 != 0 then + print ======data51=$data51 + goto loop3 +endi + +if $data52 != 11 then + print ======data52=$data52 + goto loop3 +endi + + +print sql select _irowts, _isfilled as a1, interp(b, 1) from st partition by tbname range(1648791212000, 1648791217000) every(1s) fill(value, 110); +sql select _irowts, _isfilled as a1, interp(b, 1) from st partition by tbname range(1648791212000, 1648791217000) every(1s) fill(value, 110); + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +$loop_count = 0 +loop3_2: + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +sleep 300 + +print sql select * from streamt3_2; +sql select * from streamt3_2; + +if $rows != 6 then + print ======rows=$rows + goto loop3_2 +endi + +if $data01 != 0 then + print ======data01=$data01 + goto loop3_2 +endi + +if $data02 != 10 then + print ======data02=$data02 + goto loop3_2 +endi + +if $data11 != 1 then + print ======data11=$data11 + goto loop3_2 +endi + +if $data12 != 110 then + print ======data12=$data12 + goto loop3_2 +endi + +if $data21 != 1 then + print ======data21=$data21 + goto loop3_2 +endi + +if $data22 != 110 then + print ======data22=$data22 + goto loop3_2 +endi + +if $data31 != 1 then + print ======data31=$data31 + goto loop3_2 +endi + +if $data32 != 110 then + print ======data32=$data32 + goto loop3_2 +endi + +if $data41 != 1 then + print ======data41=$data41 + goto loop3_2 +endi + +if $data42 != 110 then + print ======data42=$data42 + goto loop3_2 +endi + +if $data51 != 0 then + print ======data51=$data51 + goto loop3_2 +endi + +if $data52 != 11 then + print ======data52=$data52 + goto loop3_2 +endi + +print step linear + +sql create database test4 vgroups 4; +sql use test4; + +sql create stable st(ts timestamp,a int,b int,c int) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +sql create stream streams4 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt4 as select _irowts, _isfilled as a1, interp(b, 1) from st partition by tbname every(1s) fill(linear); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791217000,11,55,3) (1648791212000,10,10,3) (1648791212001,11,NULL,3); + +sleep 500 + +sql insert into t1 values(1648791214001,20,NULL,3) (1648791213000,22,NULL,3); + +print sql select _irowts, _isfilled as a1, interp(b, 1) from st partition by tbname range(1648791212000, 1648791217000) every(1s) fill(linear); +sql select _irowts, _isfilled as a1, interp(b, 1) from st partition by tbname range(1648791212000, 1648791217000) every(1s) fill(linear); + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +$loop_count = 0 +loop4: + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +sleep 300 + +print sql select * from streamt4; +sql select * from streamt4; + +if $rows != 6 then + print ======rows=$rows + goto loop4 +endi + +if $data01 != 0 then + print ======data01=$data01 + goto loop4 +endi + +if $data02 != 10 then + print ======data02=$data02 + goto loop4 +endi + +if $data11 != 1 then + print ======data11=$data11 + goto loop4 +endi + +if $data12 != 19 then + print ======data12=$data12 + goto loop4 +endi + +if $data21 != 1 then + print ======data21=$data21 + goto loop4 +endi + +if $data22 != 28 then + print ======data22=$data22 + goto loop4 +endi + +if $data31 != 1 then + print ======data31=$data31 + goto loop4 +endi + +if $data32 != 37 then + print ======data32=$data32 + goto loop4 +endi + +if $data41 != 1 then + print ======data41=$data41 + goto loop4 +endi + +if $data42 != 46 then + print ======data42=$data42 + goto loop4 +endi + +if $data51 != 0 then + print ======data51=$data51 + goto loop4 +endi + +if $data52 != 55 then + print ======data52=$data52 + goto loop4 +endi + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpPartitionBy0.sim b/tests/script/tsim/stream/streamInterpPartitionBy0.sim new file mode 100644 index 0000000000..6b222de228 --- /dev/null +++ b/tests/script/tsim/stream/streamInterpPartitionBy0.sim @@ -0,0 +1,592 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step prev +print =============== create database +sql create database test vgroups 1; +sql use test; +sql create stable st(ts timestamp,a int,b int,c int, d double) tags(ta int,tb int,tc int); + +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); +sql create table t3 using st tags(2,2,2); + +sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), _isfilled, tbname, b, c from st partition by tbname, b,c every(1s) fill(prev); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791212000,0,0,0,0.0) (1648791212001,1,0,0,1.0) (1648791217001,2,0,0,2.1) t2 values(1648791212000,0,1,1,0.0) (1648791212001,1,1,1,1.0) (1648791217001,2,1,1,2.1); + +sql insert into t3 values(1648791212000,0,2,2,0.0) (1648791212001,1,2,2,1.0) (1648791217001,2,2,2,2.1); + +print sql select _irowts, interp(a), _isfilled, b from st where b = 0 and c = 0 partition by tbname, b,c range(1648791212000, 1648791217001) every(1s) fill(prev) order by b, 1; +sql select _irowts, interp(a), _isfilled, b from st where b = 0 and c = 0 partition by tbname, b,c range(1648791212000, 1648791217001) every(1s) fill(prev) order by b, 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +print sql select _irowts, interp(a), _isfilled, b from st where b = 1 and c =1 partition by tbname, b,c range(1648791212000, 1648791217001) every(1s) fill(prev) order by b, 1; +sql select _irowts, interp(a), _isfilled, b from st where b = 1 and c = 1 partition by tbname, b,c range(1648791212000, 1648791217001) every(1s) fill(prev) order by b, 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +print sql select _irowts, interp(a), _isfilled, b from st where b = 2 and c = 2 partition by tbname, b,c range(1648791212000, 1648791217001) every(1s) fill(prev) order by b, 1; +sql select _irowts, interp(a), _isfilled, b from st where b = 2 and c = 2 partition by tbname, b,c range(1648791212000, 1648791217001) every(1s) fill(prev) order by b, 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +$loop_count = 0 +loop0: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt where b = 0 and c = 0 order by 1; +sql select * from streamt where b = 0 and c = 0 order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + + +# row 0 +if $rows != 6 then + print ======rows=$rows + goto loop0 +endi + +# row 0 +if $data01 != 0 then + print ======data01=$data01 + goto loop0 +endi + +if $data02 != 0 then + print ======data02=$data02 + goto loop0 +endi + +if $data11 != 1 then + print ======data11=$data11 + goto loop0 +endi + +if $data12 != 1 then + print ======data12=$data12 + goto loop0 +endi + +if $data21 != 1 then + print ======data21=$data21 + goto loop0 +endi + +if $data22 != 1 then + print ======data22=$data22 + goto loop0 +endi + +if $data31 != 1 then + print ======data31=$data31 + goto loop0 +endi + +if $data32 != 1 then + print ======data32=$data32 + goto loop0 +endi + +if $data41 != 1 then + print ======data41=$data41 + goto loop0 +endi + +if $data42 != 1 then + print ======data41=$data41 + goto loop0 +endi + +if $data51 != 1 then + print ======data51=$data51 + goto loop0 +endi + +if $data52 != 1 then + print ======data51=$data51 + goto loop0 +endi + +print 1 sql select * from streamt where b = 1 and c = 1 order by 1; +sql select * from streamt where b = 1 and c = 1 order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + + +# row 0 +if $rows != 6 then + print ======rows=$rows + goto loop0 +endi + +# row 0 +if $data01 != 0 then + print ======data01=$data01 + goto loop0 +endi + +if $data02 != 0 then + print ======data02=$data02 + goto loop0 +endi + +if $data11 != 1 then + print ======data11=$data11 + goto loop0 +endi + +if $data12 != 1 then + print ======data12=$data12 + goto loop0 +endi + +if $data21 != 1 then + print ======data21=$data21 + goto loop0 +endi + +if $data22 != 1 then + print ======data22=$data22 + goto loop0 +endi + +if $data31 != 1 then + print ======data31=$data31 + goto loop0 +endi + +if $data32 != 1 then + print ======data32=$data32 + goto loop0 +endi + +if $data41 != 1 then + print ======data41=$data41 + goto loop0 +endi + +if $data42 != 1 then + print ======data41=$data41 + goto loop0 +endi + +if $data51 != 1 then + print ======data51=$data51 + goto loop0 +endi + +if $data52 != 1 then + print ======data51=$data51 + goto loop0 +endi + +print 2 sql select * from streamt where b = 2 and c = 2 order by 1; +sql select * from streamt where b = 2 and c = 2 order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + + +# row 0 +if $rows != 6 then + print ======rows=$rows + goto loop0 +endi + +# row 0 +if $data01 != 0 then + print ======data01=$data01 + goto loop0 +endi + +if $data02 != 0 then + print ======data02=$data02 + goto loop0 +endi + +if $data11 != 1 then + print ======data11=$data11 + goto loop0 +endi + +if $data12 != 1 then + print ======data12=$data12 + goto loop0 +endi + +if $data21 != 1 then + print ======data21=$data21 + goto loop0 +endi + +if $data22 != 1 then + print ======data22=$data22 + goto loop0 +endi + +if $data31 != 1 then + print ======data31=$data31 + goto loop0 +endi + +if $data32 != 1 then + print ======data32=$data32 + goto loop0 +endi + +if $data41 != 1 then + print ======data41=$data41 + goto loop0 +endi + +if $data42 != 1 then + print ======data41=$data41 + goto loop0 +endi + +if $data51 != 1 then + print ======data51=$data51 + goto loop0 +endi + +if $data52 != 1 then + print ======data51=$data51 + goto loop0 +endi + +print step next +print =============== create database +sql create database test2 vgroups 1; +sql use test2; +sql create stable st(ts timestamp,a int,b int,c int, d double) tags(ta int,tb int,tc int); + +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); +sql create table t3 using st tags(2,2,2); + +sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2 as select _irowts, interp(a), _isfilled, tbname, b, c from st partition by tbname, b,c every(1s) fill(next); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791212000,0,0,0,0.0) (1648791212001,1,0,0,1.0) (1648791217001,2,0,0,2.1) t2 values(1648791212000,0,1,1,0.0) (1648791212001,1,1,1,1.0) (1648791217001,2,1,1,2.1); + +sql insert into t3 values(1648791212000,0,2,2,0.0) (1648791212001,1,2,2,1.0) (1648791217001,2,2,2,2.1); + +print sql select _irowts, interp(a), _isfilled, b from st where b = 0 and c = 0 partition by tbname, b,c range(1648791212000, 1648791217001) every(1s) fill(next) order by b, 1; +sql select _irowts, interp(a), _isfilled, b from st where b = 0 and c = 0 partition by tbname, b,c range(1648791212000, 1648791217001) every(1s) fill(next) order by b, 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +print sql select _irowts, interp(a), _isfilled, b from st where b = 1 and c =1 partition by tbname, b,c range(1648791212000, 1648791217001) every(1s) fill(next) order by b, 1; +sql select _irowts, interp(a), _isfilled, b from st where b = 1 and c = 1 partition by tbname, b,c range(1648791212000, 1648791217001) every(1s) fill(next) order by b, 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +print sql select _irowts, interp(a), _isfilled, b from st where b = 2 and c = 2 partition by tbname, b,c range(1648791212000, 1648791217001) every(1s) fill(next) order by b, 1; +sql select _irowts, interp(a), _isfilled, b from st where b = 2 and c = 2 partition by tbname, b,c range(1648791212000, 1648791217001) every(1s) fill(next) order by b, 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +$loop_count = 0 +loop1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt2 where b = 0 and c = 0 order by 1; +sql select * from streamt2 where b = 0 and c = 0 order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + + +# row 0 +if $rows != 6 then + print ======rows=$rows + goto loop1 +endi + +# row 0 +if $data01 != 0 then + print ======data01=$data01 + goto loop1 +endi + +if $data02 != 0 then + print ======data02=$data02 + goto loop1 +endi + +if $data11 != 2 then + print ======data11=$data11 + goto loop1 +endi + +if $data12 != 1 then + print ======data12=$data12 + goto loop1 +endi + +if $data21 != 2 then + print ======data21=$data21 + goto loop1 +endi + +if $data22 != 1 then + print ======data22=$data22 + goto loop1 +endi + +if $data31 != 2 then + print ======data31=$data31 + goto loop1 +endi + +if $data32 != 1 then + print ======data32=$data32 + goto loop1 +endi + +if $data41 != 2 then + print ======data41=$data41 + goto loop1 +endi + +if $data42 != 1 then + print ======data41=$data41 + goto loop1 +endi + +if $data51 != 2 then + print ======data51=$data51 + goto loop1 +endi + +if $data52 != 1 then + print ======data51=$data51 + goto loop1 +endi + +print 1 sql select * from streamt2 where b = 1 and c = 1 order by 1; +sql select * from streamt2 where b = 1 and c = 1 order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + + +# row 0 +if $rows != 6 then + print ======rows=$rows + goto loop1 +endi + +# row 0 +if $data01 != 0 then + print ======data01=$data01 + goto loop1 +endi + +if $data02 != 0 then + print ======data02=$data02 + goto loop1 +endi + +if $data11 != 2 then + print ======data11=$data11 + goto loop1 +endi + +if $data12 != 1 then + print ======data12=$data12 + goto loop1 +endi + +if $data21 != 2 then + print ======data21=$data21 + goto loop1 +endi + +if $data22 != 1 then + print ======data22=$data22 + goto loop1 +endi + +if $data31 != 2 then + print ======data31=$data31 + goto loop1 +endi + +if $data32 != 1 then + print ======data32=$data32 + goto loop1 +endi + +if $data41 != 2 then + print ======data41=$data41 + goto loop1 +endi + +if $data42 != 1 then + print ======data41=$data41 + goto loop1 +endi + +if $data51 != 2 then + print ======data51=$data51 + goto loop1 +endi + +if $data52 != 1 then + print ======data51=$data51 + goto loop1 +endi + +print 2 sql select * from streamt2 where b = 2 and c = 2 order by 1; +sql select * from streamt2 where b = 2 and c = 2 order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + + +# row 0 +if $rows != 6 then + print ======rows=$rows + goto loop1 +endi + +# row 0 +if $data01 != 0 then + print ======data01=$data01 + goto loop1 +endi + +if $data02 != 0 then + print ======data02=$data02 + goto loop1 +endi + +if $data11 != 2 then + print ======data11=$data11 + goto loop1 +endi + +if $data12 != 1 then + print ======data12=$data12 + goto loop1 +endi + +if $data21 != 2 then + print ======data21=$data21 + goto loop1 +endi + +if $data22 != 1 then + print ======data22=$data22 + goto loop1 +endi + +if $data31 != 2 then + print ======data31=$data31 + goto loop1 +endi + +if $data32 != 1 then + print ======data32=$data32 + goto loop1 +endi + +if $data41 != 2 then + print ======data41=$data41 + goto loop1 +endi + +if $data42 != 1 then + print ======data41=$data41 + goto loop1 +endi + +if $data51 != 2 then + print ======data51=$data51 + goto loop1 +endi + +if $data52 != 1 then + print ======data51=$data51 + goto loop1 +endi + + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpPartitionBy1.sim b/tests/script/tsim/stream/streamInterpPartitionBy1.sim new file mode 100644 index 0000000000..ecb5e0ee62 --- /dev/null +++ b/tests/script/tsim/stream/streamInterpPartitionBy1.sim @@ -0,0 +1,592 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step NULL +print =============== create database +sql create database test vgroups 1; +sql use test; +sql create stable st(ts timestamp,a int,b int,c int, d double) tags(ta int,tb int,tc int); + +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); +sql create table t3 using st tags(2,2,2); + +sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), _isfilled, tbname, b, c from st partition by tbname, b,c every(1s) fill(NULL); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791212000,0,0,0,0.0) (1648791212001,1,0,0,1.0) (1648791217001,2,0,0,2.1) t2 values(1648791212000,0,1,1,0.0) (1648791212001,1,1,1,1.0) (1648791217001,2,1,1,2.1); + +sql insert into t3 values(1648791212000,0,2,2,0.0) (1648791212001,1,2,2,1.0) (1648791217001,2,2,2,2.1); + +print sql select _irowts, interp(a), _isfilled, b from st where b = 0 partition by tbname, b, c range(1648791212000, 1648791217001) every(1s) fill(NULL) order by b, 1; +sql select _irowts, interp(a), _isfilled, b from st where b = 0 partition by tbname, b, c range(1648791212000, 1648791217001) every(1s) fill(NULL) order by b, 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +print sql select _irowts, interp(a), _isfilled, b from st where b = 1 partition by tbname, b, c range(1648791212000, 1648791217001) every(1s) fill(NULL) order by b, 1; +sql select _irowts, interp(a), _isfilled, b from st where b = 1 partition by tbname, b, c range(1648791212000, 1648791217001) every(1s) fill(NULL) order by b, 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +print sql select _irowts, interp(a), _isfilled, b from st where b = 2 partition by tbname, b, c range(1648791212000, 1648791217001) every(1s) fill(NULL) order by b, 1; +sql select _irowts, interp(a), _isfilled, b from st where b = 2 partition by tbname, b, c range(1648791212000, 1648791217001) every(1s) fill(NULL) order by b, 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +$loop_count = 0 +loop0: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt where b = 0 and c = 0 order by 1; +sql select * from streamt where b = 0 and c = 0 order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + + +# row 0 +if $rows != 6 then + print ======rows=$rows + goto loop0 +endi + +# row 0 +if $data01 != 0 then + print ======data01=$data01 + goto loop0 +endi + +if $data02 != 0 then + print ======data02=$data02 + goto loop0 +endi + +if $data11 != NULL then + print ======data11=$data11 + goto loop0 +endi + +if $data12 != 1 then + print ======data12=$data12 + goto loop0 +endi + +if $data21 != NULL then + print ======data21=$data21 + goto loop0 +endi + +if $data22 != 1 then + print ======data22=$data22 + goto loop0 +endi + +if $data31 != NULL then + print ======data31=$data31 + goto loop0 +endi + +if $data32 != 1 then + print ======data32=$data32 + goto loop0 +endi + +if $data41 != NULL then + print ======data41=$data41 + goto loop0 +endi + +if $data42 != 1 then + print ======data41=$data41 + goto loop0 +endi + +if $data51 != NULL then + print ======data51=$data51 + goto loop0 +endi + +if $data52 != 1 then + print ======data51=$data51 + goto loop0 +endi + +print 1 sql select * from streamt where b = 1 and c = 1 order by 1; +sql select * from streamt where b = 1 and c = 1 order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + + +# row 0 +if $rows != 6 then + print ======rows=$rows + goto loop0 +endi + +# row 0 +if $data01 != 0 then + print ======data01=$data01 + goto loop0 +endi + +if $data02 != 0 then + print ======data02=$data02 + goto loop0 +endi + +if $data11 != NULL then + print ======data11=$data11 + goto loop0 +endi + +if $data12 != 1 then + print ======data12=$data12 + goto loop0 +endi + +if $data21 != NULL then + print ======data21=$data21 + goto loop0 +endi + +if $data22 != 1 then + print ======data22=$data22 + goto loop0 +endi + +if $data31 != NULL then + print ======data31=$data31 + goto loop0 +endi + +if $data32 != 1 then + print ======data32=$data32 + goto loop0 +endi + +if $data41 != NULL then + print ======data41=$data41 + goto loop0 +endi + +if $data42 != 1 then + print ======data41=$data41 + goto loop0 +endi + +if $data51 != NULL then + print ======data51=$data51 + goto loop0 +endi + +if $data52 != 1 then + print ======data51=$data51 + goto loop0 +endi + +print 2 sql select * from streamt where b = 2 and c = 2 order by 1; +sql select * from streamt where b = 2 and c = 2 order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + + +# row 0 +if $rows != 6 then + print ======rows=$rows + goto loop0 +endi + +# row 0 +if $data01 != 0 then + print ======data01=$data01 + goto loop0 +endi + +if $data02 != 0 then + print ======data02=$data02 + goto loop0 +endi + +if $data11 != NULL then + print ======data11=$data11 + goto loop0 +endi + +if $data12 != 1 then + print ======data12=$data12 + goto loop0 +endi + +if $data21 != NULL then + print ======data21=$data21 + goto loop0 +endi + +if $data22 != 1 then + print ======data22=$data22 + goto loop0 +endi + +if $data31 != NULL then + print ======data31=$data31 + goto loop0 +endi + +if $data32 != 1 then + print ======data32=$data32 + goto loop0 +endi + +if $data41 != NULL then + print ======data41=$data41 + goto loop0 +endi + +if $data42 != 1 then + print ======data41=$data41 + goto loop0 +endi + +if $data51 != NULL then + print ======data51=$data51 + goto loop0 +endi + +if $data52 != 1 then + print ======data51=$data51 + goto loop0 +endi + +print step linear +print =============== create database +sql create database test2 vgroups 1; +sql use test2; +sql create stable st(ts timestamp,a int,b int,c int, d double) tags(ta int,tb int,tc int); + +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); +sql create table t3 using st tags(2,2,2); + +sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2 as select _irowts, interp(a), _isfilled, tbname, b, c from st partition by tbname, b,c every(1s) fill(linear); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791212000,0,0,0,0.0) (1648791212001,10,0,0,1.0) (1648791217001,20,0,0,2.1) t2 values(1648791212000,0,1,1,0.0) (1648791212001,10,1,1,1.0) (1648791217001,20,1,1,2.1); + +sql insert into t3 values(1648791212000,0,2,2,0.0) (1648791212001,10,2,2,1.0) (1648791217001,20,2,2,2.1); + +print sql select _irowts, interp(a), _isfilled, b from st where b = 0 partition by tbname, b, c range(1648791212000, 1648791217001) every(1s) fill(linear) order by b, 1; +sql select _irowts, interp(a), _isfilled, b from st where b = 0 partition by tbname, b, c range(1648791212000, 1648791217001) every(1s) fill(linear) order by b, 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +$loop_count = 0 +loop1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt2 where b = 0 and c = 0 order by 1; +sql select * from streamt2 where b = 0 and c = 0 order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + + +# row 0 +if $rows != 6 then + print ======rows=$rows + goto loop1 +endi + +# row 0 +if $data01 != 0 then + print ======data01=$data01 + goto loop1 +endi + +if $data02 != 0 then + print ======data02=$data02 + goto loop1 +endi + +if $data11 != 11 then + print ======data11=$data11 + goto loop1 +endi + +if $data12 != 1 then + print ======data12=$data12 + goto loop1 +endi + +if $data21 != 13 then + print ======data21=$data21 + goto loop1 +endi + +if $data22 != 1 then + print ======data22=$data22 + goto loop1 +endi + +if $data31 != 15 then + print ======data31=$data31 + goto loop1 +endi + +if $data32 != 1 then + print ======data32=$data32 + goto loop1 +endi + +if $data41 != 17 then + print ======data41=$data41 + goto loop1 +endi + +if $data42 != 1 then + print ======data41=$data41 + goto loop1 +endi + +if $data51 != 19 then + print ======data51=$data51 + goto loop1 +endi + +if $data52 != 1 then + print ======data51=$data51 + goto loop1 +endi + +print sql select _irowts, interp(a), _isfilled, b from st where b = 1 partition by tbname, b, c range(1648791212000, 1648791217001) every(1s) fill(linear) order by b, 1; +sql select _irowts, interp(a), _isfilled, b from st where b = 1 partition by tbname, b, c range(1648791212000, 1648791217001) every(1s) fill(linear) order by b, 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +print 1 sql select * from streamt2 where b = 1 and c = 1 order by 1; +sql select * from streamt2 where b = 1 and c = 1 order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + + +# row 0 +if $rows != 6 then + print ======rows=$rows + goto loop1 +endi + +# row 0 +if $data01 != 0 then + print ======data01=$data01 + goto loop1 +endi + +if $data02 != 0 then + print ======data02=$data02 + goto loop1 +endi + +if $data11 != 11 then + print ======data11=$data11 + goto loop1 +endi + +if $data12 != 1 then + print ======data12=$data12 + goto loop1 +endi + +if $data21 != 13 then + print ======data21=$data21 + goto loop1 +endi + +if $data22 != 1 then + print ======data22=$data22 + goto loop1 +endi + +if $data31 != 15 then + print ======data31=$data31 + goto loop1 +endi + +if $data32 != 1 then + print ======data32=$data32 + goto loop1 +endi + +if $data41 != 17 then + print ======data41=$data41 + goto loop1 +endi + +if $data42 != 1 then + print ======data41=$data41 + goto loop1 +endi + +if $data51 != 19 then + print ======data51=$data51 + goto loop1 +endi + +if $data52 != 1 then + print ======data51=$data51 + goto loop1 +endi + +print sql select _irowts, interp(a), _isfilled, b from st where b = 2 partition by tbname, b, c range(1648791212000, 1648791217001) every(1s) fill(linear) order by b, 1; +sql select _irowts, interp(a), _isfilled, b from st where b = 2 partition by tbname, b, c range(1648791212000, 1648791217001) every(1s) fill(linear) order by b, 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + +print 2 sql select * from streamt2 where b = 2 and c = 2 order by 1; +sql select * from streamt2 where b = 2 and c = 2 order by 1; + +print $data00 $data01 $data02 $data03 $data04 $data05 +print $data10 $data11 $data12 $data13 $data14 $data15 +print $data20 $data21 $data22 $data23 $data24 $data25 +print $data30 $data31 $data32 $data33 $data34 $data35 +print $data40 $data41 $data42 $data43 $data44 $data45 +print $data50 $data51 $data52 $data53 $data54 $data55 + + +# row 0 +if $rows != 6 then + print ======rows=$rows + goto loop1 +endi + +# row 0 +if $data01 != 0 then + print ======data01=$data01 + goto loop1 +endi + +if $data02 != 0 then + print ======data02=$data02 + goto loop1 +endi + +if $data11 != 11 then + print ======data11=$data11 + goto loop1 +endi + +if $data12 != 1 then + print ======data12=$data12 + goto loop1 +endi + +if $data21 != 13 then + print ======data21=$data21 + goto loop1 +endi + +if $data22 != 1 then + print ======data22=$data22 + goto loop1 +endi + +if $data31 != 15 then + print ======data31=$data31 + goto loop1 +endi + +if $data32 != 1 then + print ======data32=$data32 + goto loop1 +endi + +if $data41 != 17 then + print ======data41=$data41 + goto loop1 +endi + +if $data42 != 1 then + print ======data41=$data41 + goto loop1 +endi + +if $data51 != 19 then + print ======data51=$data51 + goto loop1 +endi + +if $data52 != 1 then + print ======data51=$data51 + goto loop1 +endi + + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpPrev0.sim b/tests/script/tsim/stream/streamInterpPrev0.sim new file mode 100644 index 0000000000..86f7f95a5f --- /dev/null +++ b/tests/script/tsim/stream/streamInterpPrev0.sim @@ -0,0 +1,434 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 1; +sql use test; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(prev); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791213000,1,1,1,1.0); + +$loop_count = 0 + +loop0: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop0 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop0 +endi + + +sql insert into t1 values(1648791213001,2,2,2,1.1); +sql insert into t1 values(1648791213009,3,3,3,1.0); + +$loop_count = 0 + +loop1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 1 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop1 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop1 +endi + + +sql insert into t1 values(1648791217001,4,4,4,4.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791217001) every(1s) fill(prev); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791217001) every(1s) fill(prev); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +$loop_count = 0 +loop2: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop2 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop2 +endi + +if $data11 != 3 then + print ======data11=$data11 + goto loop2 +endi + +if $data21 != 3 then + print ======data21=$data21 + goto loop2 +endi + +if $data31 != 3 then + print ======data31=$data31 + goto loop2 +endi + +if $data41 != 3 then + print ======data41=$data41 + goto loop2 +endi + + +sql insert into t1 values(1648791215001,5,5,5,5.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791217001) every(1s) fill(prev); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791217001) every(1s) fill(prev); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 +loop3: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 3 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop3 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop3 +endi + +if $data11 != 3 then + print ======data11=$data11 + goto loop3 +endi + +if $data21 != 3 then + print ======data21=$data21 + goto loop3 +endi + +if $data31 != 5 then + print ======data31=$data31 + goto loop3 +endi + +if $data41 != 5 then + print ======data41=$data41 + goto loop3 +endi + +print step2 + +sql create database test2 vgroups 1; +sql use test2; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2 as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(prev); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791212000,0,0,0,0.0) (1648791213001,1,1,1,1.0) (1648791213009,2,2,2,1.1) (1648791215001,5,5,5,5.1) (1648791217001,4,4,4,4.1); + + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(prev); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(prev); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 +loop4: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 1 sql select * from streamt2; +sql select * from streamt2; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 6 then + print ======rows=$rows + goto loop4 +endi + +# row 0 +if $data01 != 0 then + print ======data01=$data01 + goto loop4 +endi + +if $data11 != 0 then + print ======data11=$data11 + goto loop4 +endi + +if $data21 != 2 then + print ======data21=$data21 + goto loop4 +endi + +if $data31 != 2 then + print ======data31=$data31 + goto loop4 +endi + +if $data41 != 5 then + print ======data41=$data41 + goto loop4 +endi + +if $data51 != 5 then + print ======data51=$data51 + goto loop4 +endi + +print step3 +sql create database test3 vgroups 1; +sql use test3; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt3 as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(prev); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791210001,0,0,0,0.0) (1648791217001,4,4,4,4.1); + + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791210000, 1648791217000) every(1s) fill(prev); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791210000, 1648791217000) every(1s) fill(prev); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 +print $data60 $data61 $data62 $data63 $data64 + +$loop_count = 0 +loop5: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 1 sql select * from streamt3; +sql select * from streamt3; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 +print $data60 $data61 $data62 $data63 $data64 + + +# row 0 +if $rows != 7 then + print ======rows=$rows + goto loop5 +endi + +sql insert into t1 values(1648791213001,1,1,1,1.0) (1648791213009,2,2,2,1.1) (1648791215001,5,5,5,5.1) + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791210000, 1648791217001) every(1s) fill(prev); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791210000, 1648791217001) every(1s) fill(prev); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 +print $data60 $data61 $data62 $data63 $data64 + +$loop_count = 0 +loop6: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt3; +sql select * from streamt3; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 +print $data60 $data61 $data62 $data63 $data64 + + +# row 0 +if $rows != 7 then + print ======rows=$rows + goto loop6 +endi + +# row 0 +if $data01 != 0 then + print ======data01=$data01 + goto loop6 +endi + +if $data11 != 0 then + print ======data11=$data11 + goto loop6 +endi + +if $data21 != 0 then + print ======data21=$data21 + goto loop6 +endi + +if $data31 != 2 then + print ======data31=$data31 + goto loop6 +endi + +if $data41 != 2 then + print ======data41=$data41 + goto loop6 +endi + +if $data51 != 5 then + print ======data51=$data51 + goto loop6 +endi + +if $data61 != 5 then + print ======data61=$data61 + goto loop6 +endi + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpPrev1.sim b/tests/script/tsim/stream/streamInterpPrev1.sim new file mode 100644 index 0000000000..0beeb3e9a7 --- /dev/null +++ b/tests/script/tsim/stream/streamInterpPrev1.sim @@ -0,0 +1,404 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 1; +sql use test; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(prev); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791213000,1,1,1,1.0); + +$loop_count = 0 + +loop0: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop0 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop0 +endi + + +sql insert into t1 values(1648791213009,3,3,3,1.0) (1648791217001,4,4,4,4.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791217001) every(1s) fill(prev); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791217001) every(1s) fill(prev); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +$loop_count = 0 +loop2: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop2 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop2 +endi + +if $data11 != 3 then + print ======data11=$data11 + goto loop2 +endi + +if $data21 != 3 then + print ======data21=$data21 + goto loop2 +endi + +if $data31 != 3 then + print ======data31=$data31 + goto loop2 +endi + +if $data41 != 3 then + print ======data41=$data41 + goto loop2 +endi + + +print step2 + +sql create database test2 vgroups 1; +sql use test2; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(prev); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791213000,1,1,1,1.0); + + +$loop_count = 0 + +loop3: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop3 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop3 +endi + + +sql insert into t1 values(1648791213009,3,3,3,1.0) (1648791217001,4,4,4,4.1) (1648791219000,5,5,5,5.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791217001) every(1s) fill(prev); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791217001) every(1s) fill(prev); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 +print $data60 $data61 $data62 $data63 $data64 +print $data70 $data71 $data72 $data73 $data74 + + +$loop_count = 0 +loop4: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 +print $data60 $data61 $data62 $data63 $data64 +print $data70 $data71 $data72 $data73 $data74 + +# row 0 +if $rows != 7 then + print ======rows=$rows + goto loop4 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop4 +endi + +if $data11 != 3 then + print ======data11=$data11 + goto loop4 +endi + +if $data21 != 3 then + print ======data21=$data21 + goto loop4 +endi + +if $data31 != 3 then + print ======data31=$data31 + goto loop4 +endi + +if $data41 != 3 then + print ======data41=$data41 + goto loop4 +endi + +if $data51 != 4 then + print ======data51=$data51 + goto loop4 +endi + +if $data61 != 5 then + print ======data61=$data61 + goto loop4 +endi + +print step3 + +sql create database test3 vgroups 1; +sql use test3; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(prev); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791213001,1,1,1,1.0) (1648791219001,2,2,2,2.1) (1648791229001,3,3,3,3.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791229001) every(1s) fill(prev); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791229001) every(1s) fill(prev); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 +print $data60 $data61 $data62 $data63 $data64 +print $data70 $data71 $data72 $data73 $data74 + + +$loop_count = 0 +loop5: + +sleep 300 + +print sql select * from streamt order by 1; +sql select * from streamt order by 1; + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +if $rows != 16 then + print =====rows=$rows + goto loop5 +endi + +sql insert into t1 values(1648791215001,4,4,4,4.0) (1648791217001,5,5,5,5.1) (1648791222000,6,6,6,6.1) (1648791226000,7,7,7,7.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791229001) every(1s) fill(prev); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791229001) every(1s) fill(prev); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 +print $data60 $data61 $data62 $data63 $data64 +print $data70 $data71 $data72 $data73 $data74 + + +$loop_count = 0 +loop6: + +sleep 300 + +print sql select * from streamt order by 1; +sql select * from streamt order by 1; + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +if $rows != 16 then + goto loop6 +endi + +if $data01 != 1 then + print =====data01=$data01 + goto loop6 +endi + +if $data11 != 1 then + print =====data11=$data11 + goto loop6 +endi + +if $data21 != 4 then + print =====data21=$data21 + goto loop6 +endi + +if $data31 != 4 then + print =====data31=$data31 + goto loop6 +endi + +if $data41 != 5 then + print =====data41=$data41 + goto loop6 +endi + +if $data51 != 5 then + print =====data51=$data51 + goto loop6 +endi + +if $data61 != 2 then + print =====data61=$data61 + goto loop6 +endi + +if $data71 != 2 then + print =====data71=$data71 + goto loop6 +endi + +if $data81 != 6 then + print =====data81=$data81 + goto loop6 +endi + +if $data91 != 6 then + print =====data91=$data91 + goto loop6 +endi + +if $data[10][1] != 6 then + print =====data[10][1]=$data[10][1] + goto loop6 +endi + +if $data[11][1] != 6 then + print =====data[11][1]=$data[11][1] + goto loop6 +endi + +if $data[12][1] != 7 then + print =====data[12][1]=$data[12][1] + goto loop6 +endi + +if $data[13][1] != 7 then + print =====data[13][1]=$data[13][1] + goto loop6 +endi + +if $data[14][1] != 7 then + print =====data[14][1]=$data[14][1] + goto loop6 +endi + +if $data[15][1] != 7 then + print =====data[15][1]=$data[15][1] + goto loop6 +endi + +print end + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpPrimaryKey0.sim b/tests/script/tsim/stream/streamInterpPrimaryKey0.sim new file mode 100644 index 0000000000..9edddff6db --- /dev/null +++ b/tests/script/tsim/stream/streamInterpPrimaryKey0.sim @@ -0,0 +1,452 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 1; +sql use test; + +sql create stable st(ts timestamp,a int primary key,b int,c int, d double) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); +sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(b) from st partition by tbname every(1s) fill(prev); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791213000,9,9,9,9.0); + +$loop_count = 0 + +loop0: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop0 +endi + + +sql insert into t1 values(1648791213000,10,10,10,10.0); +sql insert into t1 values(1648791213009,30,30,30,30.0); + +$loop_count = 0 + +loop1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 1 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop1 +endi + +# row 0 +if $data01 != 9 then + print ======data01=$data01 + goto loop1 +endi + + +sql insert into t1 values(1648791213009,20,20,20,20.0) (1648791217001,4,4,4,4.1); + +print sql select _irowts, interp(b) from t1 range(1648791212000, 1648791217001) every(1s) fill(prev); +sql select _irowts, interp(b) from t1 range(1648791212000, 1648791217001) every(1s) fill(prev); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +$loop_count = 0 +loop2: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop2 +endi + +# row 0 +if $data01 != 9 then + print ======data01=$data01 + goto loop2 +endi + +if $data11 != 20 then + print ======data11=$data11 + goto loop2 +endi + +if $data21 != 20 then + print ======data21=$data21 + goto loop2 +endi + +if $data31 != 20 then + print ======data31=$data31 + goto loop2 +endi + +if $data41 != 20 then + print ======data41=$data41 + goto loop2 +endi + +print step2 +print =============== create database +sql create database test2 vgroups 1; +sql use test2; + +sql create stable st(ts timestamp,a int ,b int,c int, d double) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); +sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt(ts, b primary key) as select _irowts, interp(b) from st partition by tbname every(1s) fill(prev); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791213000,9,9,9,9.0); + +$loop_count = 0 + +loop3: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop3 +endi + +sql insert into t1 values(1648791213000,10,10,10,10.0); + +sql insert into t1 values(1648791213009,30,30,30,30.0); + +$loop_count = 0 + +loop4: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 1 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop4 +endi + +# row 0 +if $data01 != 10 then + print ======data01=$data01 + goto loop4 +endi + +sql insert into t1 values(1648791217001,4,4,10,4.1); + +sleep 2000 + +sql insert into t1 values(1648791213009,20,20,10,20.0); + +print sql select _irowts, interp(b) from t1 range(1648791212000, 1648791217001) every(1s) fill(prev); +sql select _irowts, interp(b) from t1 range(1648791212000, 1648791217001) every(1s) fill(prev); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +$loop_count = 0 +loop5: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop5 +endi + +# row 0 +if $data01 != 10 then + print ======data01=$data01 + goto loop5 +endi + +if $data11 != 20 then + print ======data11=$data11 + goto loop5 +endi + +if $data21 != 20 then + print ======data21=$data21 + goto loop5 +endi + +if $data31 != 20 then + print ======data31=$data31 + goto loop5 +endi + +if $data41 != 20 then + print ======data41=$data41 + goto loop5 +endi + +print step3 +print =============== create database +sql create database test3 vgroups 1; +sql use test3; + +sql create stable st(ts timestamp,a int primary key, b int,c int, d double) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); +sql create stream streams3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt(ts, b primary key) as select _irowts, interp(b) from st partition by tbname, c every(1s) fill(prev); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791213000,9,9,10,9.0); + +$loop_count = 0 + +loop6: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop6 +endi + + +sql insert into t1 values(1648791213000,10,10,10,10.0); +sql insert into t1 values(1648791213009,30,30,10,30.0); + +$loop_count = 0 + +loop7: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 1 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop7 +endi + +# row 0 +if $data01 != 9 then + print ======data01=$data01 + goto loop7 +endi + + +sql insert into t1 values(1648791217001,4,4,10,4.1); + +sleep 2000 + +sql insert into t1 values(1648791213009,20,20,10,20.0); + +print sql select _irowts, interp(b) from t1 range(1648791212000, 1648791217001) every(1s) fill(prev); +sql select _irowts, interp(b) from t1 range(1648791212000, 1648791217001) every(1s) fill(prev); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +$loop_count = 0 +loop8: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop8 +endi + +# row 0 +if $data01 != 9 then + print ======data01=$data01 + goto loop8 +endi + +if $data11 != 20 then + print ======data11=$data11 + goto loop8 +endi + +if $data21 != 20 then + print ======data21=$data21 + goto loop8 +endi + +if $data31 != 20 then + print ======data31=$data31 + goto loop8 +endi + +if $data41 != 20 then + print ======data41=$data41 + goto loop8 +endi + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpPrimaryKey1.sim b/tests/script/tsim/stream/streamInterpPrimaryKey1.sim new file mode 100644 index 0000000000..04a1f299be --- /dev/null +++ b/tests/script/tsim/stream/streamInterpPrimaryKey1.sim @@ -0,0 +1,458 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 1; +sql use test; + +sql create stable st(ts timestamp,a int primary key,b int,c int, d double) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); +sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(b) from st partition by tbname every(1s) fill(next); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791213000,10,10,10,10.0); + +$loop_count = 0 + +loop0: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop0 +endi + + +sql insert into t1 values(1648791213000,9,9,9,9.0); +sql insert into t1 values(1648791213009,30,30,30,30.0); + +$loop_count = 0 + +loop1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 1 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop1 +endi + +# row 0 +if $data01 != 9 then + print ======data01=$data01 + goto loop1 +endi + + +sql insert into t1 values(1648791213009,20,20,20,20.0) (1648791217001,40,40,40,40.1); + +sleep 2000 + +sql insert into t1 values(1648791217001,4,4,4,4.1); + +print sql select _irowts, interp(b) from t1 range(1648791212000, 1648791217001) every(1s) fill(next); +sql select _irowts, interp(b) from t1 range(1648791212000, 1648791217001) every(1s) fill(next); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +$loop_count = 0 +loop2: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop2 +endi + +# row 0 +if $data01 != 9 then + print ======data01=$data01 + goto loop2 +endi + +if $data11 != 4 then + print ======data11=$data11 + goto loop2 +endi + +if $data21 != 4 then + print ======data21=$data21 + goto loop2 +endi + +if $data31 != 4 then + print ======data31=$data31 + goto loop2 +endi + +if $data41 != 4 then + print ======data41=$data41 + goto loop2 +endi + +print step2 +print =============== create database +sql create database test2 vgroups 1; +sql use test2; + +sql create stable st(ts timestamp,a int ,b int,c int, d double) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); +sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt(ts, b primary key) as select _irowts, interp(b) from st partition by tbname every(1s) fill(next); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791213000,10,10,10,10.0); + +$loop_count = 0 + +loop3: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop3 +endi + + +sql insert into t1 values(1648791213000,9,9,9,9.0); +sql insert into t1 values(1648791213009,30,30,30,30.0); + +$loop_count = 0 + +loop4: + +sleep 500 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 1 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop4 +endi + +# row 0 +if $data01 != 9 then + print ======data01=$data01 + goto loop4 +endi + + +sql insert into t1 values(1648791213009,20,20,20,20.0) (1648791217001,40,40,40,40.1); + +sleep 2000 + +sql insert into t1 values(1648791217001,4,4,4,4.1); + +print sql select _irowts, interp(b) from t1 range(1648791212000, 1648791217001) every(1s) fill(next); +sql select _irowts, interp(b) from t1 range(1648791212000, 1648791217001) every(1s) fill(next); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +$loop_count = 0 +loop5: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop5 +endi + +# row 0 +if $data01 != 9 then + print ======data01=$data01 + goto loop5 +endi + +if $data11 != 4 then + print ======data11=$data11 + goto loop5 +endi + +if $data21 != 4 then + print ======data21=$data21 + goto loop5 +endi + +if $data31 != 4 then + print ======data31=$data31 + goto loop5 +endi + +if $data41 != 4 then + print ======data41=$data41 + goto loop5 +endi + +print step3 +print =============== create database +sql create database test3 vgroups 1; +sql use test3; + +sql create stable st(ts timestamp,a int primary key, b int,c int, d double) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); +sql create stream streams3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt(ts, b primary key) as select _irowts, interp(b) from st partition by tbname, c every(1s) fill(next); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791213000,10,10,10,10.0); + +$loop_count = 0 + +loop6: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop6 +endi + + +sql insert into t1 values(1648791213000,9,9,10,9.0); +sql insert into t1 values(1648791213009,30,30,10,30.0); + +$loop_count = 0 + +loop7: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 1 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop7 +endi + +# row 0 +if $data01 != 9 then + print ======data01=$data01 + goto loop7 +endi + + +sql insert into t1 values(1648791213009,20,20,10,20.0) (1648791217001,40,40,10,40.1); + +sleep 1000 + +sql insert into t1 values(1648791217001,4,4,10,4.1); + +print sql select _irowts, interp(b) from t1 range(1648791212000, 1648791217001) every(1s) fill(next); +sql select _irowts, interp(b) from t1 range(1648791212000, 1648791217001) every(1s) fill(next); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 +print $data60 $data61 $data62 $data63 $data64 + + +$loop_count = 0 +loop8: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop8 +endi + +# row 0 +if $data01 != 9 then + print ======data01=$data01 + goto loop8 +endi + +if $data11 != 4 then + print ======data11=$data11 + goto loop8 +endi + +if $data21 != 4 then + print ======data21=$data21 + goto loop8 +endi + +if $data31 != 4 then + print ======data31=$data31 + goto loop8 +endi + +if $data41 != 4 then + print ======data41=$data41 + goto loop8 +endi + + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpPrimaryKey2.sim b/tests/script/tsim/stream/streamInterpPrimaryKey2.sim new file mode 100644 index 0000000000..f06e1ecd03 --- /dev/null +++ b/tests/script/tsim/stream/streamInterpPrimaryKey2.sim @@ -0,0 +1,452 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 1; +sql use test; + +sql create stable st(ts timestamp,a int primary key,b int,c int, d double) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); +sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(b) from st partition by tbname every(1s) fill(linear); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791213000,9,9,9,9.0); + +$loop_count = 0 + +loop0: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop0 +endi + + +sql insert into t1 values(1648791213000,10,10,10,10.0); +sql insert into t1 values(1648791213009,30,30,30,30.0); + +$loop_count = 0 + +loop1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 1 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop1 +endi + +# row 0 +if $data01 != 9 then + print ======data01=$data01 + goto loop1 +endi + + +sql insert into t1 values(1648791213009,20,20,20,20.0) (1648791217001,4,4,4,4.1); + +print sql select _irowts, interp(b) from t1 range(1648791212000, 1648791217001) every(1s) fill(linear); +sql select _irowts, interp(b) from t1 range(1648791212000, 1648791217001) every(1s) fill(linear); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +$loop_count = 0 +loop2: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop2 +endi + +# row 0 +if $data01 != 9 then + print ======data01=$data01 + goto loop2 +endi + +if $data11 != 16 then + print ======data11=$data11 + goto loop2 +endi + +if $data21 != 12 then + print ======data21=$data21 + goto loop2 +endi + +if $data31 != 8 then + print ======data31=$data31 + goto loop2 +endi + +if $data41 != 4 then + print ======data41=$data41 + goto loop2 +endi + +print step2 +print =============== create database +sql create database test2 vgroups 1; +sql use test2; + +sql create stable st(ts timestamp,a int ,b int,c int, d double) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); +sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt(ts, b primary key) as select _irowts, interp(b) from st partition by tbname every(1s) fill(linear); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791213000,9,9,9,9.0); + +$loop_count = 0 + +loop3: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop3 +endi + +sql insert into t1 values(1648791213000,10,10,10,10.0); + +sql insert into t1 values(1648791213009,30,30,30,30.0); + +$loop_count = 0 + +loop4: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 1 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop4 +endi + +# row 0 +if $data01 != 10 then + print ======data01=$data01 + goto loop4 +endi + +sql insert into t1 values(1648791217001,4,4,10,4.1); + +sleep 2000 + +sql insert into t1 values(1648791213009,20,20,10,20.0); + +print sql select _irowts, interp(b) from t1 range(1648791212000, 1648791217001) every(1s) fill(linear); +sql select _irowts, interp(b) from t1 range(1648791212000, 1648791217001) every(1s) fill(linear); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +$loop_count = 0 +loop5: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop5 +endi + +# row 0 +if $data01 != 10 then + print ======data01=$data01 + goto loop5 +endi + +if $data11 != 16 then + print ======data11=$data11 + goto loop5 +endi + +if $data21 != 12 then + print ======data21=$data21 + goto loop5 +endi + +if $data31 != 8 then + print ======data31=$data31 + goto loop5 +endi + +if $data41 != 4 then + print ======data41=$data41 + goto loop5 +endi + +print step3 +print =============== create database +sql create database test3 vgroups 1; +sql use test3; + +sql create stable st(ts timestamp,a int primary key, b int,c int, d double) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); +sql create stream streams3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt(ts, b primary key) as select _irowts, interp(b) from st partition by tbname, c every(1s) fill(linear); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791213000,9,9,10,9.0); + +$loop_count = 0 + +loop6: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop6 +endi + + +sql insert into t1 values(1648791213000,10,10,10,10.0); +sql insert into t1 values(1648791213009,30,30,10,30.0); + +$loop_count = 0 + +loop7: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 1 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop7 +endi + +# row 0 +if $data01 != 9 then + print ======data01=$data01 + goto loop7 +endi + + +sql insert into t1 values(1648791217001,4,4,10,4.1); + +sleep 2000 + +sql insert into t1 values(1648791213009,20,20,10,20.0); + +print sql select _irowts, interp(b) from t1 range(1648791212000, 1648791217001) every(1s) fill(linear); +sql select _irowts, interp(b) from t1 range(1648791212000, 1648791217001) every(1s) fill(linear); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +$loop_count = 0 +loop8: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop8 +endi + +# row 0 +if $data01 != 9 then + print ======data01=$data01 + goto loop8 +endi + +if $data11 != 16 then + print ======data11=$data11 + goto loop8 +endi + +if $data21 != 12 then + print ======data21=$data21 + goto loop8 +endi + +if $data31 != 8 then + print ======data31=$data31 + goto loop8 +endi + +if $data41 != 4 then + print ======data41=$data41 + goto loop8 +endi + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpPrimaryKey3.sim b/tests/script/tsim/stream/streamInterpPrimaryKey3.sim new file mode 100644 index 0000000000..725cf8d850 --- /dev/null +++ b/tests/script/tsim/stream/streamInterpPrimaryKey3.sim @@ -0,0 +1,452 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 1; +sql use test; + +sql create stable st(ts timestamp,a int primary key,b int,c int, d double) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); +sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(b) from st partition by tbname every(1s) fill(value,100); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791213000,9,9,9,9.0); + +$loop_count = 0 + +loop0: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop0 +endi + + +sql insert into t1 values(1648791213000,10,10,10,10.0); +sql insert into t1 values(1648791213009,30,30,30,30.0); + +$loop_count = 0 + +loop1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 1 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop1 +endi + +# row 0 +if $data01 != 9 then + print ======data01=$data01 + goto loop1 +endi + + +sql insert into t1 values(1648791213009,20,20,20,20.0) (1648791217001,4,4,4,4.1); + +print sql select _irowts, interp(b) from t1 range(1648791212000, 1648791217001) every(1s) fill(value,100); +sql select _irowts, interp(b) from t1 range(1648791212000, 1648791217001) every(1s) fill(value,100); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +$loop_count = 0 +loop2: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop2 +endi + +# row 0 +if $data01 != 9 then + print ======data01=$data01 + goto loop2 +endi + +if $data11 != 100 then + print ======data11=$data11 + goto loop2 +endi + +if $data21 != 100 then + print ======data21=$data21 + goto loop2 +endi + +if $data31 != 100 then + print ======data31=$data31 + goto loop2 +endi + +if $data41 != 100 then + print ======data41=$data41 + goto loop2 +endi + +print step2 +print =============== create database +sql create database test2 vgroups 1; +sql use test2; + +sql create stable st(ts timestamp,a int ,b int,c int, d double) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); +sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt(ts, b primary key) as select _irowts, interp(b) from st partition by tbname every(1s) fill(value,100); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791213000,9,9,9,9.0); + +$loop_count = 0 + +loop3: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop3 +endi + +sql insert into t1 values(1648791213000,10,10,10,10.0); + +sql insert into t1 values(1648791213009,30,30,30,30.0); + +$loop_count = 0 + +loop4: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 1 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop4 +endi + +# row 0 +if $data01 != 10 then + print ======data01=$data01 + goto loop4 +endi + +sql insert into t1 values(1648791217001,4,4,10,4.1); + +sleep 2000 + +sql insert into t1 values(1648791213009,20,20,10,20.0); + +print sql select _irowts, interp(b) from t1 range(1648791212000, 1648791217001) every(1s) fill(value,100); +sql select _irowts, interp(b) from t1 range(1648791212000, 1648791217001) every(1s) fill(value,100); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +$loop_count = 0 +loop5: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop5 +endi + +# row 0 +if $data01 != 10 then + print ======data01=$data01 + goto loop5 +endi + +if $data11 != 100 then + print ======data11=$data11 + goto loop5 +endi + +if $data21 != 100 then + print ======data21=$data21 + goto loop5 +endi + +if $data31 != 100 then + print ======data31=$data31 + goto loop5 +endi + +if $data41 != 100 then + print ======data41=$data41 + goto loop5 +endi + +print step3 +print =============== create database +sql create database test3 vgroups 1; +sql use test3; + +sql create stable st(ts timestamp,a int primary key, b int,c int, d double) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); +sql create stream streams3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt(ts, b primary key) as select _irowts, interp(b) from st partition by tbname, c every(1s) fill(value,100); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791213000,9,9,10,9.0); + +$loop_count = 0 + +loop6: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop6 +endi + + +sql insert into t1 values(1648791213000,10,10,10,10.0); +sql insert into t1 values(1648791213009,30,30,10,30.0); + +$loop_count = 0 + +loop7: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 1 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop7 +endi + +# row 0 +if $data01 != 9 then + print ======data01=$data01 + goto loop7 +endi + + +sql insert into t1 values(1648791217001,4,4,10,4.1); + +sleep 2000 + +sql insert into t1 values(1648791213009,20,20,10,20.0); + +print sql select _irowts,interp(b) from t1 range(1648791212000, 1648791217001) every(1s) fill(value,100); +sql select _irowts, interp(b) from t1 range(1648791212000, 1648791217001) every(1s) fill(value,100); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +$loop_count = 0 +loop8: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop8 +endi + +# row 0 +if $data01 != 9 then + print ======data01=$data01 + goto loop8 +endi + +if $data11 != 100 then + print ======data11=$data11 + goto loop8 +endi + +if $data21 != 100 then + print ======data21=$data21 + goto loop8 +endi + +if $data31 != 100 then + print ======data31=$data31 + goto loop8 +endi + +if $data41 != 100 then + print ======data41=$data41 + goto loop8 +endi + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpScalar.sim b/tests/script/tsim/stream/streamInterpScalar.sim new file mode 100644 index 0000000000..e4e280138b --- /dev/null +++ b/tests/script/tsim/stream/streamInterpScalar.sim @@ -0,0 +1,417 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 1; +sql use test; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, now, floor(interp(d)), interp(floor(d)), a from t1 partition by tbname, a every(1s) fill(prev); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791217001,1,1,1,1.1); + +sleep 2000 + +sql insert into t1 values(1648791212009,1,3,3,3.3) (1648791214001,1,4,4,4.4) (1648791219001,1,5,5,5.5) (1648791220001,1,6,6,6.6); + +print sql select _irowts, now, floor(interp(d)), interp(floor(d)), a from t1 partition by tbname, a range(1648791213000, 1648791220001) every(1s) fill(prev); +sql select _irowts, now, floor(interp(d)), interp(floor(d)), a from t1 partition by tbname, a range(1648791213000, 1648791220001) every(1s) fill(prev); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +$loop_count = 0 +loop0: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt order by 1; +sql select * from streamt order by 1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 8 then + print ======rows=$rows + goto loop0 +endi + +# row 0 +if $data02 != 3.000000000 then + print ======data02=$data02 + goto loop0 +endi + +if $data03 != 3.000000000 then + print ======data03=$data03 + goto loop0 +endi + +if $data12 != 3.000000000 then + print ======data12=$data12 + goto loop0 +endi + +if $data13 != 3.000000000 then + print ======data13=$data13 + goto loop0 +endi + +if $data22 != 4.000000000 then + print ======data22=$data22 + goto loop0 +endi + +if $data23 != 4.000000000 then + print ======data23=$data23 + goto loop0 +endi + +if $data32 != 4.000000000 then + print ======data32=$data32 + goto loop0 +endi + +if $data33 != 4.000000000 then + print ======data33=$data33 + goto loop0 +endi + +if $data42 != 4.000000000 then + print ======data42=$data42 + goto loop0 +endi + +if $data43 != 4.000000000 then + print ======data43=$data43 + goto loop0 +endi + +if $data52 != 1.000000000 then + print ======data52=$data52 + goto loop0 +endi + +if $data53 != 1.000000000 then + print ======data53=$data53 + goto loop0 +endi + +if $data62 != 1.000000000 then + print ======data62=$data62 + goto loop0 +endi + +if $data63 != 1.000000000 then + print ======data63=$data63 + goto loop0 +endi + +if $data72 != 5.000000000 then + print ======data72=$data72 + goto loop0 +endi + +if $data73 != 5.000000000 then + print ======data73=$data73 + goto loop0 +endi + +print step2 +print =============== create database +sql create database test2 vgroups 1; +sql use test2; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, now, floor(interp(d)), interp(floor(d)), a from t1 partition by tbname, a every(1s) fill(next); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791217001,1,1,1,1.1); + +sleep 2000 + +sql insert into t1 values(1648791212009,1,3,3,3.3) (1648791214001,1,4,4,4.4) (1648791219001,1,5,5,5.5) (1648791220001,1,6,6,6.6); + +print sql select _irowts, now, floor(interp(d)), interp(floor(d)), a from t1 partition by tbname, a range(1648791213000, 1648791220001) every(1s) fill(next); +sql select _irowts, now, floor(interp(d)), interp(floor(d)), a from t1 partition by tbname, a range(1648791213000, 1648791220001) every(1s) fill(next); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +$loop_count = 0 +loop1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt order by 1; +sql select * from streamt order by 1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 8 then + print ======rows=$rows + goto loop1 +endi + +# row 0 +if $data02 != 4.000000000 then + print ======data02=$data02 + goto loop1 +endi + +if $data03 != 4.000000000 then + print ======data03=$data03 + goto loop1 +endi + +if $data12 != 4.000000000 then + print ======data12=$data12 + goto loop1 +endi + +if $data13 != 4.000000000 then + print ======data13=$data13 + goto loop1 +endi + +if $data22 != 1.000000000 then + print ======data22=$data22 + goto loop1 +endi + +if $data23 != 1.000000000 then + print ======data23=$data23 + goto loop1 +endi + +if $data32 != 1.000000000 then + print ======data32=$data32 + goto loop1 +endi + +if $data33 != 1.000000000 then + print ======data33=$data33 + goto loop1 +endi + +if $data42 != 1.000000000 then + print ======data42=$data42 + goto loop1 +endi + +if $data43 != 1.000000000 then + print ======data43=$data43 + goto loop1 +endi + +if $data52 != 5.000000000 then + print ======data52=$data52 + goto loop1 +endi + +if $data53 != 5.000000000 then + print ======data53=$data53 + goto loop1 +endi + +if $data62 != 5.000000000 then + print ======data62=$data62 + goto loop1 +endi + +if $data63 != 5.000000000 then + print ======data63=$data63 + goto loop1 +endi + +if $data72 != 6.000000000 then + print ======data72=$data72 + goto loop1 +endi + +if $data73 != 6.000000000 then + print ======data73=$data73 + goto loop1 +endi + +print step3 +print =============== create database +sql create database test3 vgroups 1; +sql use test3; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, now, floor(interp(d)), interp(floor(d)), a from t1 partition by tbname, a every(1s) fill(value, 100, 200); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791217001,1,1,1,1.1); + +sleep 2000 + +sql insert into t1 values(1648791212009,1,3,3,3.3) (1648791214001,1,4,4,4.4) (1648791219001,1,5,5,5.5) (1648791220001,1,6,6,6.6); + +print sql select _irowts, now, floor(interp(d)), interp(floor(d)), a from t1 partition by tbname, a range(1648791213000, 1648791220001) every(1s) fill(value, 100, 200); +sql select _irowts, now, floor(interp(d)), interp(floor(d)), a from t1 partition by tbname, a range(1648791213000, 1648791220001) every(1s) fill(value, 100, 200); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +$loop_count = 0 +loop2: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt order by 1; +sql select * from streamt order by 1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 8 then + print ======rows=$rows + goto loop2 +endi + +# row 0 +if $data02 != 100.000000000 then + print ======data02=$data02 + goto loop2 +endi + +if $data03 != 200.000000000 then + print ======data03=$data03 + goto loop2 +endi + +if $data12 != 100.000000000 then + print ======data12=$data12 + goto loop2 +endi + +if $data13 != 200.000000000 then + print ======data13=$data13 + goto loop2 +endi + +if $data22 != 100.000000000 then + print ======data22=$data22 + goto loop2 +endi + +if $data23 != 200.000000000 then + print ======data23=$data23 + goto loop2 +endi + +if $data32 != 100.000000000 then + print ======data32=$data32 + goto loop2 +endi + +if $data33 != 200.000000000 then + print ======data33=$data33 + goto loop2 +endi + +if $data42 != 100.000000000 then + print ======data42=$data42 + goto loop2 +endi + +if $data43 != 200.000000000 then + print ======data43=$data43 + goto loop2 +endi + +if $data52 != 100.000000000 then + print ======data52=$data52 + goto loop2 +endi + +if $data53 != 200.000000000 then + print ======data53=$data53 + goto loop2 +endi + +if $data62 != 100.000000000 then + print ======data62=$data62 + goto loop2 +endi + +if $data63 != 200.000000000 then + print ======data63=$data63 + goto loop2 +endi + +if $data72 != 100.000000000 then + print ======data72=$data72 + goto loop2 +endi + +if $data73 != 200.000000000 then + print ======data73=$data73 + goto loop2 +endi + +print end + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpUpdate.sim b/tests/script/tsim/stream/streamInterpUpdate.sim new file mode 100644 index 0000000000..59a188c2a6 --- /dev/null +++ b/tests/script/tsim/stream/streamInterpUpdate.sim @@ -0,0 +1,551 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 1; +sql use test; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(prev); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791212001,1,1,1,1.0) (1648791215000,10,1,1,1.0) (1648791217001,4,1,1,1.0) + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(prev); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(prev); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop0: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop0 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop0 +endi + +if $data11 != 1 then + print ======data11=$data11 + goto loop0 +endi + +if $data21 != 10 then + print ======data21=$data21 + goto loop0 +endi + +if $data31 != 10 then + print ======data31=$data31 + goto loop0 +endi + +if $data41 != 10 then + print ======data41=$data41 + goto loop0 +endi + +sql insert into t1 values(1648791212001,2,2,2,2.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(prev); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(prev); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop1 +endi + +# row 0 +if $data01 != 2 then + print ======data01=$data01 + goto loop1 +endi + +if $data11 != 2 then + print ======data11=$data11 + goto loop1 +endi + +if $data21 != 10 then + print ======data21=$data21 + goto loop1 +endi + +if $data31 != 10 then + print ======data31=$data31 + goto loop1 +endi + +if $data41 != 10 then + print ======data41=$data41 + goto loop1 +endi + + +sql insert into t1 values(1648791215000,20,20,20,20.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(prev); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(prev); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop2: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop2 +endi + +# row 0 +if $data01 != 2 then + print ======data01=$data01 + goto loop2 +endi + +if $data11 != 2 then + print ======data11=$data11 + goto loop2 +endi + +if $data21 != 20 then + print ======data21=$data21 + goto loop2 +endi + +if $data31 != 20 then + print ======data31=$data31 + goto loop2 +endi + +if $data41 != 20 then + print ======data41=$data41 + goto loop2 +endi + +sql insert into t1 values(1648791217001,8,8,8,8.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(prev); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(prev); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop3: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop3 +endi + +# row 0 +if $data01 != 2 then + print ======data01=$data01 + goto loop3 +endi + +if $data11 != 2 then + print ======data11=$data11 + goto loop3 +endi + +if $data21 != 20 then + print ======data21=$data21 + goto loop3 +endi + +if $data31 != 20 then + print ======data31=$data31 + goto loop3 +endi + +if $data41 != 20 then + print ======data41=$data41 + goto loop3 +endi + +print step2 +print =============== create database +sql create database test2 vgroups 1; +sql use test2; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(next); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791212001,1,1,1,1.0) (1648791215000,10,1,1,1.0) (1648791217001,4,1,1,1.0) + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(next); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(next); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop4: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop4 +endi + +# row 0 +if $data01 != 10 then + print ======data01=$data01 + goto loop4 +endi + +if $data11 != 10 then + print ======data11=$data11 + goto loop4 +endi + +if $data21 != 10 then + print ======data21=$data21 + goto loop4 +endi + +if $data31 != 4 then + print ======data31=$data31 + goto loop4 +endi + +if $data41 != 4 then + print ======data41=$data41 + goto loop4 +endi + +sql insert into t1 values(1648791212001,2,2,2,2.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(next); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(next); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop5: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop5 +endi + +# row 0 +if $data01 != 10 then + print ======data01=$data01 + goto loop5 +endi + +if $data11 != 10 then + print ======data11=$data11 + goto loop5 +endi + +if $data21 != 10 then + print ======data21=$data21 + goto loop5 +endi + +if $data31 != 4 then + print ======data31=$data31 + goto loop5 +endi + +if $data41 != 4 then + print ======data41=$data41 + goto loop5 +endi + + +sql insert into t1 values(1648791215000,20,20,20,20.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(next); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(next); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop6: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop6 +endi + +# row 0 +if $data01 != 20 then + print ======data01=$data01 + goto loop6 +endi + +if $data11 != 20 then + print ======data11=$data11 + goto loop6 +endi + +if $data21 != 20 then + print ======data21=$data21 + goto loop6 +endi + +if $data31 != 4 then + print ======data31=$data31 + goto loop6 +endi + +if $data41 != 4 then + print ======data41=$data41 + goto loop6 +endi + +sql insert into t1 values(1648791217001,8,8,8,8.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(next); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(next); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop7: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop7 +endi + +# row 0 +if $data01 != 20 then + print ======data01=$data01 + goto loop7 +endi + +if $data11 != 20 then + print ======data11=$data11 + goto loop7 +endi + +if $data21 != 20 then + print ======data21=$data21 + goto loop7 +endi + +if $data31 != 8 then + print ======data31=$data31 + goto loop7 +endi + +if $data41 != 8 then + print ======data41=$data41 + goto loop7 +endi + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpUpdate1.sim b/tests/script/tsim/stream/streamInterpUpdate1.sim new file mode 100644 index 0000000000..45f16af35d --- /dev/null +++ b/tests/script/tsim/stream/streamInterpUpdate1.sim @@ -0,0 +1,551 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 1; +sql use test; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(NULL); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791212001,1,1,1,1.0) (1648791215000,10,1,1,1.0) (1648791217001,4,1,1,1.0) + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(NULL); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(NULL); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop0: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop0 +endi + +# row 0 +if $data01 != NULL then + print ======data01=$data01 + goto loop0 +endi + +if $data11 != NULL then + print ======data11=$data11 + goto loop0 +endi + +if $data21 != 10 then + print ======data21=$data21 + goto loop0 +endi + +if $data31 != NULL then + print ======data31=$data31 + goto loop0 +endi + +if $data41 != NULL then + print ======data41=$data41 + goto loop0 +endi + +sql insert into t1 values(1648791212001,2,2,2,2.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(NULL); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(NULL); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop1 +endi + +# row 0 +if $data01 != NULL then + print ======data01=$data01 + goto loop1 +endi + +if $data11 != NULL then + print ======data11=$data11 + goto loop1 +endi + +if $data21 != 10 then + print ======data21=$data21 + goto loop1 +endi + +if $data31 != NULL then + print ======data31=$data31 + goto loop1 +endi + +if $data41 != NULL then + print ======data41=$data41 + goto loop1 +endi + + +sql insert into t1 values(1648791215000,20,20,20,20.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(NULL); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(NULL); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop2: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop2 +endi + +# row 0 +if $data01 != NULL then + print ======data01=$data01 + goto loop2 +endi + +if $data11 != NULL then + print ======data11=$data11 + goto loop2 +endi + +if $data21 != 20 then + print ======data21=$data21 + goto loop2 +endi + +if $data31 != NULL then + print ======data31=$data31 + goto loop2 +endi + +if $data41 != NULL then + print ======data41=$data41 + goto loop2 +endi + +sql insert into t1 values(1648791217001,8,8,8,8.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(prev); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(prev); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop3: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop3 +endi + +# row 0 +if $data01 != NULL then + print ======data01=$data01 + goto loop3 +endi + +if $data11 != NULL then + print ======data11=$data11 + goto loop3 +endi + +if $data21 != 20 then + print ======data21=$data21 + goto loop3 +endi + +if $data31 != NULL then + print ======data31=$data31 + goto loop3 +endi + +if $data41 != NULL then + print ======data41=$data41 + goto loop3 +endi + +print step2 +print =============== create database +sql create database test2 vgroups 1; +sql use test2; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(value, 100, 200, 300, 400); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791212001,1,1,1,1.0) (1648791215000,10,1,1,1.0) (1648791217001,4,1,1,1.0) + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(value, 100, 200, 300, 400); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(value, 100, 200, 300, 400); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop4: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop4 +endi + +# row 0 +if $data01 != 100 then + print ======data01=$data01 + goto loop4 +endi + +if $data11 != 100 then + print ======data11=$data11 + goto loop4 +endi + +if $data21 != 10 then + print ======data21=$data21 + goto loop4 +endi + +if $data31 != 100 then + print ======data31=$data31 + goto loop4 +endi + +if $data41 != 100 then + print ======data41=$data41 + goto loop4 +endi + +sql insert into t1 values(1648791212001,2,2,2,2.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(value, 100, 200, 300, 400); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(value, 100, 200, 300, 400); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop5: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop5 +endi + +# row 0 +if $data01 != 100 then + print ======data01=$data01 + goto loop5 +endi + +if $data11 != 100 then + print ======data11=$data11 + goto loop5 +endi + +if $data21 != 10 then + print ======data21=$data21 + goto loop5 +endi + +if $data31 != 100 then + print ======data31=$data31 + goto loop5 +endi + +if $data41 != 100 then + print ======data41=$data41 + goto loop5 +endi + + +sql insert into t1 values(1648791215000,20,20,20,20.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(value, 100, 200, 300, 400); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(value, 100, 200, 300, 400); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop6: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop6 +endi + +# row 0 +if $data01 != 100 then + print ======data01=$data01 + goto loop6 +endi + +if $data11 != 100 then + print ======data11=$data11 + goto loop6 +endi + +if $data21 != 20 then + print ======data21=$data21 + goto loop6 +endi + +if $data31 != 100 then + print ======data31=$data31 + goto loop6 +endi + +if $data41 != 100 then + print ======data41=$data41 + goto loop6 +endi + +sql insert into t1 values(1648791217001,8,8,8,8.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(value, 100, 200, 300, 400); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(value, 100, 200, 300, 400); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop7: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop7 +endi + +# row 0 +if $data01 != 100 then + print ======data01=$data01 + goto loop7 +endi + +if $data11 != 100 then + print ======data11=$data11 + goto loop7 +endi + +if $data21 != 20 then + print ======data21=$data21 + goto loop7 +endi + +if $data31 != 100 then + print ======data31=$data31 + goto loop7 +endi + +if $data41 != 100 then + print ======data41=$data41 + goto loop7 +endi + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpUpdate2.sim b/tests/script/tsim/stream/streamInterpUpdate2.sim new file mode 100644 index 0000000000..2a71474dd7 --- /dev/null +++ b/tests/script/tsim/stream/streamInterpUpdate2.sim @@ -0,0 +1,279 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 1; +sql use test; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(linear); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791212001,1,1,1,1.0) (1648791215000,10,1,1,1.0) (1648791217001,4,1,1,1.0) + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(linear); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(linear); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop0: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop0 +endi + +# row 0 +if $data01 != 3 then + print ======data01=$data01 + goto loop0 +endi + +if $data11 != 6 then + print ======data11=$data11 + goto loop0 +endi + +if $data21 != 10 then + print ======data21=$data21 + goto loop0 +endi + +if $data31 != 7 then + print ======data31=$data31 + goto loop0 +endi + +if $data41 != 4 then + print ======data41=$data41 + goto loop0 +endi + +sql insert into t1 values(1648791212001,2,2,2,2.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(linear); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(linear); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop1 +endi + +# row 0 +if $data01 != 4 then + print ======data01=$data01 + goto loop1 +endi + +if $data11 != 7 then + print ======data11=$data11 + goto loop1 +endi + +if $data21 != 10 then + print ======data21=$data21 + goto loop1 +endi + +if $data31 != 7 then + print ======data31=$data31 + goto loop1 +endi + +if $data41 != 4 then + print ======data41=$data41 + goto loop1 +endi + + +sql insert into t1 values(1648791215000,20,20,20,20.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(linear); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(linear); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop2: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop2 +endi + +# row 0 +if $data01 != 7 then + print ======data01=$data01 + goto loop2 +endi + +if $data11 != 13 then + print ======data11=$data11 + goto loop2 +endi + +if $data21 != 20 then + print ======data21=$data21 + goto loop2 +endi + +if $data31 != 12 then + print ======data31=$data31 + goto loop2 +endi + +if $data41 != 4 then + print ======data41=$data41 + goto loop2 +endi + +sql insert into t1 values(1648791217001,8,8,8,8.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(linear); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(linear); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 + +loop3: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop3 +endi + +# row 0 +if $data01 != 7 then + print ======data01=$data01 + goto loop3 +endi + +if $data11 != 13 then + print ======data11=$data11 + goto loop3 +endi + +if $data21 != 20 then + print ======data21=$data21 + goto loop3 +endi + +if $data31 != 14 then + print ======data31=$data31 + goto loop3 +endi + +if $data41 != 8 then + print ======data41=$data41 + goto loop3 +endi + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpValue0.sim b/tests/script/tsim/stream/streamInterpValue0.sim new file mode 100644 index 0000000000..bce7f0ece6 --- /dev/null +++ b/tests/script/tsim/stream/streamInterpValue0.sim @@ -0,0 +1,754 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 1; +sql use test; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(NULL); +sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2 as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(value, 10,20,30,40); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791213000,1,1,1,1.0); + +$loop_count = 0 + +loop0: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 +print $data10 $data11 $data12 $data13 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop0 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop0 +endi + + +sql insert into t1 values(1648791213001,2,2,2,1.1); +sql insert into t1 values(1648791213009,3,3,3,1.0); + +$loop_count = 0 + +loop1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 1 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 +print $data10 $data11 $data12 $data13 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop1 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop1 +endi + + +sql insert into t1 values(1648791217001,4,4,4,4.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791217001) every(1s) fill(NULL); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791217001) every(1s) fill(NULL); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +$loop_count = 0 +loop2: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop2 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop2 +endi + +if $data11 != NULL then + print ======data11=$data11 + goto loop2 +endi + +if $data21 != NULL then + print ======data21=$data21 + goto loop2 +endi + +if $data31 != NULL then + print ======data31=$data31 + goto loop2 +endi + +if $data41 != NULL then + print ======data41=$data41 + goto loop2 +endi + + + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791217001) every(1s) fill(value, 10,20,30,40); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791217001) every(1s) fill(value, 10,20,30,40); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +$loop_count = 0 +loop2_1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt2; +sql select * from streamt2; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop2_1 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop2_1 +endi + +if $data11 != 10 then + print ======data11=$data11 + goto loop2_1 +endi + +if $data21 != 10 then + print ======data21=$data21 + goto loop2_1 +endi + +if $data31 != 10 then + print ======data31=$data31 + goto loop2_1 +endi + +if $data41 != 10 then + print ======data41=$data41 + goto loop2_1 +endi + +sql insert into t1 values(1648791215001,5,5,5,5.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791217001) every(1s) fill(NULL); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791217001) every(1s) fill(NULL); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 +loop3: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 3 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop3 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop3 +endi + +if $data11 != NULL then + print ======data11=$data11 + goto loop3 +endi + +if $data21 != NULL then + print ======data21=$data21 + goto loop3 +endi + +if $data31 != NULL then + print ======data31=$data31 + goto loop3 +endi + +if $data41 != NULL then + print ======data41=$data41 + goto loop3 +endi + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791217001) every(1s) fill(value, 10,20,30,40); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791217001) every(1s) fill(value, 10,20,30,40); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +$loop_count = 0 +loop3_1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 3 sql select * from streamt2; +sql select * from streamt2; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop3_1 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop3_1 +endi + +if $data11 != 10 then + print ======data11=$data11 + goto loop3_1 +endi + +if $data21 != 10 then + print ======data21=$data21 + goto loop3_1 +endi + +if $data31 != 10 then + print ======data31=$data31 + goto loop3_1 +endi + +if $data41 != 10 then + print ======data41=$data41 + goto loop3_1 +endi + +if $data12 != 20 then + print ======data12=$data12 + goto loop3_1 +endi + +if $data13 != 30 then + print ======data13=$data13 + goto loop3_1 +endi + +if $data14 != 40.000000000 then + print ======data14=$data14 + goto loop3_1 +endi + + + +print step2 + +sql create database test2 vgroups 1; +sql use test2; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams2_1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt3 as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(NULL); +sql create stream streams2_2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt4 as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(value, 10,20,30,40); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791212000,0,0,0,0.0) (1648791213001,1,1,1,1.0) (1648791213009,2,2,2,1.1) (1648791215001,5,5,5,5.1) (1648791217001,4,4,4,4.1); + + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(NULL); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(NULL); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 +loop4: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 1 sql select * from streamt3; +sql select * from streamt3; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 6 then + print ======rows=$rows + goto loop4 +endi + +# row 0 +if $data01 != 0 then + print ======data01=$data01 + goto loop4 +endi + +if $data11 != NULL then + print ======data11=$data11 + goto loop4 +endi + +if $data21 != NULL then + print ======data21=$data21 + goto loop4 +endi + +if $data31 != NULL then + print ======data31=$data31 + goto loop4 +endi + +if $data41 != NULL then + print ======data41=$data41 + goto loop4 +endi + +if $data51 != NULL then + print ======data51=$data51 + goto loop4 +endi + + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(value, 10,20,30,40); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791212000, 1648791217001) every(1s) fill(value, 10,20,30,40); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 +loop4_1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 1 sql select * from streamt4; +sql select * from streamt4; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 6 then + print ======rows=$rows + goto loop4_1 +endi + +# row 0 +if $data01 != 0 then + print ======data01=$data01 + goto loop4_1 +endi + +if $data11 != 10 then + print ======data11=$data11 + goto loop4_1 +endi + +if $data21 != 10 then + print ======data21=$data21 + goto loop4_1 +endi + +if $data31 != 10 then + print ======data31=$data31 + goto loop4_1 +endi + +if $data41 != 10 then + print ======data41=$data41 + goto loop4_1 +endi + +if $data51 != 10 then + print ======data51=$data51 + goto loop4_1 +endi + + + +print step3 +sql create database test3 vgroups 1; +sql use test3; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams3_1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt3_1 as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(NULL); +sql create stream streams3_2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt3_2 as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(value, 10,20,30,40); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791210001,0,0,0,0.0) (1648791217001,4,4,4,4.1); + + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791211000, 1648791217000) every(1s) fill(NULL); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791211000, 1648791217000) every(1s) fill(NULL); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 +loop5: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 1 sql select * from streamt3_1; +sql select * from streamt3_1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 7 then + print ======rows=$rows + goto loop5 +endi + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791211000, 1648791217000) every(1s) fill(value, 10,20,30,40); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791211000, 1648791217000) every(1s) fill(value, 10,20,30,40); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 +loop5_1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 1 sql select * from streamt3_2; +sql select * from streamt3_2; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 7 then + print ======rows=$rows + goto loop5_1 +endi + +sql insert into t1 values(1648791213001,1,1,1,1.0) (1648791213009,2,2,2,1.1) (1648791215001,5,5,5,5.1) + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791211000, 1648791217001) every(1s) fill(NULL); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791211000, 1648791217001) every(1s) fill(NULL); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 +loop6: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt3_1; +sql select * from streamt3_1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 7 then + print ======rows=$rows + goto loop6 +endi + +# row 0 +if $data01 != NULL then + print ======data01=$data01 + goto loop6 +endi + +if $data11 != NULL then + print ======data11=$data11 + goto loop6 +endi + +if $data21 != NULL then + print ======data21=$data21 + goto loop6 +endi + +if $data31 != NULL then + print ======data31=$data31 + goto loop6 +endi + +if $data41 != NULL then + print ======data41=$data41 + goto loop6 +endi + +if $data51 != NULL then + print ======data51=$data51 + goto loop6 +endi + +if $data61 != NULL then + print ======data61=$data61 + goto loop6 +endi + + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791211000, 1648791217001) every(1s) fill(value, 10,20,30,40); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791211000, 1648791217001) every(1s) fill(value, 10,20,30,40); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 +loop6_1: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt3_2; +sql select * from streamt3_2; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 7 then + print ======rows=$rows + goto loop6_1 +endi + +# row 0 +if $data01 != 10 then + print ======data01=$data01 + goto loop6_1 +endi + +if $data11 != 10 then + print ======data11=$data11 + goto loop6_1 +endi + +if $data21 != 10 then + print ======data21=$data21 + goto loop6_1 +endi + +if $data31 != 10 then + print ======data31=$data31 + goto loop6_1 +endi + +if $data41 != 10 then + print ======data41=$data41 + goto loop6_1 +endi + +if $data51 != 10 then + print ======data51=$data51 + goto loop6_1 +endi + +if $data61 != 10 then + print ======data61=$data61 + goto loop6_1 +endi + +print end +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamInterpValue1.sim b/tests/script/tsim/stream/streamInterpValue1.sim new file mode 100644 index 0000000000..84a0e28300 --- /dev/null +++ b/tests/script/tsim/stream/streamInterpValue1.sim @@ -0,0 +1,477 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 1; +sql use test; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(NULL); + +run tsim/stream/checkTaskStatus.sim + + +sql insert into t1 values(1648791213000,1,1,1,1.0); + +$loop_count = 0 + +loop0: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop0 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop0 +endi + + +sql insert into t1 values(1648791213009,3,3,3,1.0) (1648791217001,4,4,4,4.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791217001) every(1s) fill(NULL); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791217001) every(1s) fill(NULL); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +$loop_count = 0 +loop2: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows != 5 then + print ======rows=$rows + goto loop2 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop2 +endi + +if $data11 != NULL then + print ======data11=$data11 + goto loop2 +endi + +if $data21 != NULL then + print ======data21=$data21 + goto loop2 +endi + +if $data31 != NULL then + print ======data31=$data31 + goto loop2 +endi + +if $data41 != NULL then + print ======data41=$data41 + goto loop2 +endi + + +print step2 + +sql create database test2 vgroups 1; +sql use test2; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(NULL); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791213000,1,1,1,1.0); + + +$loop_count = 0 + +loop3: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 0 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows != 1 then + print ======rows=$rows + goto loop3 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop3 +endi + + +sql insert into t1 values(1648791213009,3,3,3,1.0) (1648791217001,4,4,4,4.1) (1648791219000,5,5,5,5.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791219000) every(1s) fill(NULL); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791219000) every(1s) fill(NULL); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 +print $data60 $data61 $data62 $data63 $data64 +print $data70 $data71 $data72 $data73 $data74 + + +$loop_count = 0 +loop4: + +sleep 300 + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +print 2 sql select * from streamt; +sql select * from streamt; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 +print $data60 $data61 $data62 $data63 $data64 +print $data70 $data71 $data72 $data73 $data74 + +# row 0 +if $rows != 7 then + print ======rows=$rows + goto loop4 +endi + +# row 0 +if $data01 != 1 then + print ======data01=$data01 + goto loop4 +endi + +if $data11 != NULL then + print ======data11=$data11 + goto loop4 +endi + +if $data21 != NULL then + print ======data21=$data21 + goto loop4 +endi + +if $data31 != NULL then + print ======data31=$data31 + goto loop4 +endi + +if $data41 != NULL then + print ======data41=$data41 + goto loop4 +endi + +if $data51 != NULL then + print ======data51=$data51 + goto loop4 +endi + +if $data61 != 5 then + print ======data61=$data61 + goto loop4 +endi + +print step3 + +sql create database test3 vgroups 1; +sql use test3; + +sql create table t1(ts timestamp, a int, b int , c int, d double); +sql create stream streams3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 every(1s) fill(NULL); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791213001,1,1,1,1.0) (1648791219001,2,2,2,2.1) (1648791229001,3,3,3,3.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791229001) every(1s) fill(NULL); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791229001) every(1s) fill(NULL); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 +print $data60 $data61 $data62 $data63 $data64 +print $data70 $data71 $data72 $data73 $data74 + + +$loop_count = 0 +loop5: + +sleep 300 + +print sql select * from streamt order by 1; +sql select * from streamt order by 1; + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +if $rows != 16 then + print =====rows=$rows + goto loop5 +endi + +sql insert into t1 values(1648791215001,4,4,4,4.0) (1648791217001,5,5,5,5.1) (1648791222000,6,6,6,6.1) (1648791226000,7,7,7,7.1); + +print sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791229001) every(1s) fill(NULL); +sql select _irowts, interp(a), interp(b), interp(c), interp(d) from t1 range(1648791213000, 1648791229001) every(1s) fill(NULL); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 +print $data60 $data61 $data62 $data63 $data64 +print $data70 $data71 $data72 $data73 $data74 + + +$loop_count = 0 +loop6: + +sleep 300 + +print sql select * from streamt order by 1; +sql select * from streamt order by 1; + +$loop_count = $loop_count + 1 +if $loop_count == 10 then + return -1 +endi + +if $rows != 16 then + goto loop6 +endi + +if $data01 != NULL then + print =====data01=$data01 + goto loop6 +endi + +if $data11 != NULL then + print =====data11=$data11 + goto loop6 +endi + +if $data21 != NULL then + print =====data21=$data21 + goto loop6 +endi + +if $data31 != NULL then + print =====data31=$data31 + goto loop6 +endi + +if $data41 != NULL then + print =====data41=$data41 + goto loop6 +endi + +if $data51 != NULL then + print =====data51=$data51 + goto loop6 +endi + +if $data61 != NULL then + print =====data61=$data61 + goto loop6 +endi + +if $data71 != NULL then + print =====data71=$data71 + goto loop6 +endi + +if $data81 != 6 then + print =====data81=$data81 + goto loop6 +endi + +if $data91 != NULL then + print =====data91=$data91 + goto loop6 +endi + +if $data[10][1] != NULL then + print =====data[10][1]=$data[10][1] + goto loop6 +endi + +if $data[11][1] != NULL then + print =====data[11][1]=$data[11][1] + goto loop6 +endi + +if $data[12][1] != 7 then + print =====data[12][1]=$data[12][1] + goto loop6 +endi + +if $data[13][1] != NULL then + print =====data[13][1]=$data[13][1] + goto loop6 +endi + +if $data[14][1] != NULL then + print =====data[14][1]=$data[14][1] + goto loop6 +endi + +if $data[15][1] != NULL then + print =====data[15][1]=$data[15][1] + goto loop6 +endi + + +print step4 + +sql create database test4 vgroups 1; +sql use test4; + +sql create stable st(ts timestamp,a int,b int,c int) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +sql create stream streams4 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt4 as select _irowts, interp(a) as b, _isfilled as a from st partition by tbname, b as cc every(1s) fill(NULL); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(1648791217000,20000,2,3); + +sleep 2000 + +sql insert into t1 values(1648791212000,10000,2,3) (1648791215001,20,2,3); + +$loop_count = 0 +loop7: + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +sleep 300 + +print sql select a,b from streamt4; +sql select a,b from streamt4; + +if $rows != 6 then + print ======rows=$rows + goto loop7 +endi + +if $data00 != 0 then + print ======data00=$data00 + goto loop7 +endi + +if $data01 != 10000 then + print ======data01=$data01 + goto loop7 +endi + +if $data10 != 1 then + print ======data10=$data10 + goto loop7 +endi + +if $data20 != 1 then + print ======data20=$data20 + goto loop7 +endi + +if $data41 != NULL then + print ======data41=$data41 + goto loop7 +endi + +if $data50 != 0 then + print ======data50=$data50 + goto loop7 +endi + +if $data51 != 20000 then + print ======data51=$data51 + goto loop7 +endi + +print end + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamTwaError.sim b/tests/script/tsim/stream/streamTwaError.sim new file mode 100644 index 0000000000..cda5fa9c4b --- /dev/null +++ b/tests/script/tsim/stream/streamTwaError.sim @@ -0,0 +1,36 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 1; +sql use test; + +sql create stable st(ts timestamp, a int, b int , c int)tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +sql create stream streams1 trigger force_window_close IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt as select _wstart, twa(a) from st partition by tbname,ta interval(2s) fill(prev); + +sql_error create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2 as select _wstart, twa(a) from st partition by tbname,ta interval(2s) fill(prev); +sql_error create stream streams3 trigger window_close IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt3 as select _wstart, twa(a) from st partition by tbname,ta interval(2s) fill(prev); +sql_error create stream streams4 trigger max_delay 5s IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt4 as select _wstart, twa(a) from st partition by tbname,ta interval(2s) fill(prev); + +sql_error create stream streams5 trigger force_window_close IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt5 as select _wstart, twa(a) from st interval(2s) fill(prev); +sql_error create stream streams6 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt6 as select last(ts), twa(a) from st partition by tbname,ta; +sql_error create stream streams7 trigger force_window_close IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt7 as select _wstart, twa(a) from st partition by tbname,ta session(ts, 2s); +sql_error create stream streams8 trigger force_window_close IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt8 as select _wstart, twa(a) from st partition by tbname,ta state_window(a); + +sql_error create stream streams9 trigger at_once IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt9 as select _wstart, elapsed(ts) from st partition by tbname,ta interval(2s) fill(prev); + +sql_error create stream streams10 trigger force_window_close IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt10 as select _wstart, sum(a) from st partition by tbname,ta interval(2s) SLIDING(1s); +sql create stream streams11 trigger force_window_close IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt11 as select _wstart, avg(a) from st partition by tbname,ta interval(2s) SLIDING(2s); + +sql_error create stream streams10 trigger force_window_close IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt10 as select _wstart, sum(a) from st interval(2s); + +print end + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamTwaFwcFill.sim b/tests/script/tsim/stream/streamTwaFwcFill.sim new file mode 100644 index 0000000000..6a742a3178 --- /dev/null +++ b/tests/script/tsim/stream/streamTwaFwcFill.sim @@ -0,0 +1,278 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 1; +sql use test; + +sql create stable st(ts timestamp, a int, b int , c int)tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +sql create stream streams1 trigger force_window_close IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt as select _wstart, twa(a), twa(b), elapsed(ts), now ,timezone(), ta from st partition by tbname,ta interval(2s) fill(prev); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(now + 3s,1,1,1) (now + 4s,10,1,1) (now + 7s,20,2,2) (now + 8s,30,3,3); +sql insert into t2 values(now + 4s,1,1,1) (now + 5s,10,1,1) (now + 8s,20,2,2) (now + 9s,30,3,3); + + +print sql select * from t1; +sql select * from t1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +print sql select * from t2; +sql select * from t2; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 +loop0: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt where ta == 1; +sql select * from streamt where ta == 1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows < 5 then + print ======rows=$rows + goto loop0 +endi + +$loop_count = 0 +loop1: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt where ta == 2; +sql select * from streamt where ta == 2; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows < 5 then + print ======rows=$rows + goto loop1 +endi + + +print step2 +print =============== create database +sql create database test2 vgroups 1; +sql use test2; + +sql create stable st(ts timestamp, a int, b int , c int)tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); +sql create stream streams2 trigger force_window_close IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt as select _wstart, twa(a), twa(b), elapsed(ts), now ,timezone(), ta from st partition by tbname interval(2s) fill(NULL); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(now + 3s,1,1,1) (now + 4s,10,1,1) (now + 7s,20,2,2) (now + 8s,30,3,3); +sql insert into t2 values(now + 4s,1,1,1) (now + 5s,10,1,1) (now + 8s,20,2,2) (now + 9s,30,3,3); + + +print sql select * from t1; +sql select * from t1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +print sql select * from t2; +sql select * from t2; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 +loop2: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt where ta == 1; +sql select * from streamt where ta == 1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows < 5 then + print ======rows=$rows + goto loop2 +endi + +$loop_count = 0 +loop3: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt where ta == 2; +sql select * from streamt where ta == 2; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows < 5 then + print ======rows=$rows + goto loop3 +endi + +print step3 +print =============== create database +sql create database test3 vgroups 1; +sql use test3; + +sql create stable st(ts timestamp, a int, b int , c int)tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); +sql create stream streams3 trigger force_window_close IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt as select _wstart, twa(a), twa(b), elapsed(ts), now ,timezone(), ta from st partition by tbname interval(2s) fill(value,100,200,300); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(now + 3s,1,1,1) (now + 4s,10,1,1) (now + 7s,20,2,2) (now + 8s,30,3,3); +sql insert into t2 values(now + 4s,1,1,1) (now + 5s,10,1,1) (now + 8s,20,2,2) (now + 9s,30,3,3); + + +print sql select * from t1; +sql select * from t1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +print sql select * from t2; +sql select * from t2; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 +loop4: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt where ta == 1; +sql select * from streamt where ta == 1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows < 5 then + print ======rows=$rows + goto loop4 +endi + +$loop_count = 0 +loop5: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt where ta == 2; +sql select * from streamt where ta == 2; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows < 5 then + print ======rows=$rows + goto loop5 +endi + +print end + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamTwaFwcFillPrimaryKey.sim b/tests/script/tsim/stream/streamTwaFwcFillPrimaryKey.sim new file mode 100644 index 0000000000..4282518c9c --- /dev/null +++ b/tests/script/tsim/stream/streamTwaFwcFillPrimaryKey.sim @@ -0,0 +1,222 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 1; +sql use test; + +sql create stable st(ts timestamp, a int primary key, b int , c int)tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +sql create stream streams1 trigger force_window_close IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt as select _wstart, twa(b), count(*),ta from st partition by tbname, ta interval(2s) fill(prev); + +run tsim/stream/checkTaskStatus.sim + +sql select now; + +sql insert into t1 values(now + 3s,1,1,1) (now + 3s,2,10,10) (now + 3s,3,30,30); +sql insert into t2 values(now + 4s,1,1,1) (now + 4s,2,10,10) (now + 4s,3,30,30); + + +print sql select _wstart, twa(b), count(*),ta from t1 partition by tbname, ta interval(2s); +sql select _wstart, twa(b), count(*),ta from t1 partition by tbname, ta interval(2s); + +$query1_data = $data01 + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +print sql select _wstart, twa(b), count(*),ta from t2 partition by tbname, ta interval(2s); +sql select _wstart, twa(b), count(*),ta from t2 partition by tbname, ta interval(2s); + +$query2_data = $data01 + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 +loop0: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt where ta == 1; +sql select * from streamt where ta == 1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows < 6 then + print ======rows=$rows + goto loop0 +endi + +if $data01 != $query1_data then + print ======data01=$data01 + return -1 +endi + + +$loop_count = 0 +loop1: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt where ta == 2; +sql select * from streamt where ta == 2; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows < 6 then + print ======rows=$rows + goto loop1 +endi + + +if $data01 != $query2_data then + print ======data01=$data01 + return -1 +endi + + +print step2 +print =============== create database +sql create database test2 vgroups 1; +sql use test2; + +sql create stable st(ts timestamp, a int primary key, b int , c int)tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); +sql create stream streams2 trigger force_window_close IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt as select _wstart, twa(b), ta from st partition by tbname, ta interval(2s) fill(NULL); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(now + 3s,1,1,1) (now + 3s,2,10,10) (now + 3s,3,30,30); +sql insert into t2 values(now + 4s,1,1,1) (now + 4s,2,10,10) (now + 4s,3,30,30); + + +print sql select _wstart, twa(b), count(*),ta from t1 partition by tbname, ta interval(2s); +sql select _wstart, twa(b), count(*),ta from t1 partition by tbname, ta interval(2s); + +$query1_data = $data01 + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +print sql select _wstart, twa(b), count(*),ta from t2 partition by tbname, ta interval(2s); +sql select _wstart, twa(b), count(*),ta from t2 partition by tbname, ta interval(2s); + +$query2_data = $data01 + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 +loop2: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt where ta == 1; +sql select * from streamt where ta == 1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows < 6 then + print ======rows=$rows + goto loop2 +endi + +if $data01 != $query1_data then + print ======data01=$data01 + return -1 +endi + +$loop_count = 0 +loop3: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt where ta == 2; +sql select * from streamt where ta == 2; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows < 6 then + print ======rows=$rows + goto loop3 +endi + + +if $data01 != $query2_data then + print ======data01=$data01 + return -1 +endi + +print end + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamTwaFwcInterval.sim b/tests/script/tsim/stream/streamTwaFwcInterval.sim new file mode 100644 index 0000000000..8640650310 --- /dev/null +++ b/tests/script/tsim/stream/streamTwaFwcInterval.sim @@ -0,0 +1,294 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 1; +sql use test; + +sql create stable st(ts timestamp, a int, b int , c int)tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +sql create stream streams1 trigger force_window_close IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt as select _wstart, twa(a), ta from st partition by tbname,ta interval(2s); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(now + 3000a,1,1,1) (now + 3100a,5,10,10) (now + 3200a,5,10,10) (now + 5100a,20,1,1) (now + 5200a,30,10,10) (now + 5300a,40,10,10); +sql insert into t2 values(now + 3000a,1,1,1) (now + 3100a,2,10,10) (now + 3200a,30,10,10) (now + 5100a,10,1,1) (now + 5200a,40,10,10) (now + 5300a,7,10,10); + + +print sql select _wstart, twa(a) from t1 interval(2s); +sql select _wstart, twa(a) from t1 interval(2s); + +$query1_data01 = $data01 +$query1_data11 = $data11 + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +print sql select _wstart, twa(a) from t2 interval(2s); +sql select _wstart, twa(a) from t2 interval(2s); + +$query2_data01 = $data01 +$query2_data11 = $data11 + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 +loop0: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt where ta == 1; +sql select * from streamt where ta == 1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows < 2 then + print ======rows=$rows + goto loop0 +endi + +if $data01 != $query1_data01 then + print ======data01========$data01 + print ======query1_data01=$query1_data01 + return -1 +endi + +if $data11 != $query1_data11 then + print ======data11========$data11 + print ======query1_data11=$query1_data11 + goto loop0 +endi + +$loop_count = 0 +loop1: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt where ta == 2; +sql select * from streamt where ta == 2; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows < 2 then + print ======rows=$rows + goto loop1 +endi + +if $data01 != $query2_data01 then + print ======data01======$data01 + print ====query2_data01=$query2_data01 + return -1 +endi + +if $data11 != $query2_data11 then + print ======data11======$data11 + print ====query2_data11=$query2_data11 + goto loop1 +endi + + +print step2 +print =============== create database +sql create database test2 vgroups 4; +sql use test2; + +sql create stable st(ts timestamp, a int, b int , c int)tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +sql create stream streams2 trigger force_window_close IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt as select _wstart, count(*), ta from st partition by tbname,ta interval(2s); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(now + 3000a,1,1,1) (now + 3100a,3,10,10) (now + 3200a,5,10,10) (now + 5100a,20,1,1) (now + 5200a,30,10,10) (now + 5300a,40,10,10); +sql insert into t2 values(now + 3000a,1,1,1) (now + 3100a,3,10,10) (now + 3200a,5,10,10) (now + 5100a,10,1,1) (now + 5200a,40,10,10) (now + 5300a,7,10,10); + + +print sql select _wstart, count(*) from t1 interval(2s) order by 1; +sql select _wstart, count(*) from t1 interval(2s) order by 1; + +$query1_data01 = $data01 +$query1_data11 = $data11 + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +$loop_count = 0 +loop2: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt where ta == 1 order by 1; +sql select * from streamt where ta == 1 order by 1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +if $data01 != $query1_data01 then + print ======data01======$data01 + print ====query1_data01=$query1_data01 + goto loop2 +endi + +if $data11 != $query1_data11 then + print ======data11========$data11 + print ======query1_data11=$query1_data11 + goto loop2 +endi + +sql insert into t1 values(now + 3000a,1,1,1) (now + 3100a,3,10,10) (now + 3200a,5,10,10) (now + 5100a,20,1,1) (now + 5200a,30,10,10) (now + 5300a,40,10,10); +sql insert into t2 values(now + 3000a,1,1,1) (now + 3100a,3,10,10) (now + 3200a,5,10,10) (now + 5100a,10,1,1) (now + 5200a,40,10,10) (now + 5300a,7,10,10); + +print sql select _wstart, count(*) from t1 interval(2s) order by 1; +sql select _wstart, count(*) from t1 interval(2s) order by 1; + +$query1_data21 = $data21 +$query1_data31 = $data31 + + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +$loop_count = 0 +loop3: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt where ta == 1 order by 1; +sql select * from streamt where ta == 1 order by 1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +if $data21 != $query1_data21 then + print ======data21======$data21 + print ====query1_data21=$query1_data21 + goto loop3 +endi + +if $data31 != $query1_data31 then + print ======data31========$data31 + print ======query1_data31=$query1_data31 + goto loop3 +endi + + +sql insert into t1 values(now + 3000a,1,1,1) (now + 3100a,3,10,10) (now + 3200a,5,10,10) (now + 5100a,20,1,1) (now + 5200a,30,10,10) (now + 5300a,40,10,10); +sql insert into t2 values(now + 3000a,1,1,1) (now + 3100a,3,10,10) (now + 3200a,5,10,10) (now + 5100a,10,1,1) (now + 5200a,40,10,10) (now + 5300a,7,10,10); + +print sql select _wstart, count(*) from t1 interval(2s) order by 1; +sql select _wstart, count(*) from t1 interval(2s) order by 1; + +$query1_data41 = $data41 +$query1_data51 = $data51 + + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +$loop_count = 0 +loop3: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt where ta == 1 order by 1; +sql select * from streamt where ta == 1 order by 1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +if $data41 != $query1_data41 then + print ======data41======$data41 + print ====query1_data41=$query1_data41 + goto loop3 +endi + +if $data51 != $query1_data51 then + print ======data51========$data51 + print ======query1_data51=$query1_data51 + goto loop3 +endi + +print end + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamTwaFwcIntervalPrimaryKey.sim b/tests/script/tsim/stream/streamTwaFwcIntervalPrimaryKey.sim new file mode 100644 index 0000000000..b015a5955a --- /dev/null +++ b/tests/script/tsim/stream/streamTwaFwcIntervalPrimaryKey.sim @@ -0,0 +1,109 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 1; +sql use test; + +sql create stable st(ts timestamp, a int primary key, b int , c int)tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +sql create stream streams1 trigger force_window_close IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt as select _wstart, count(*), ta from st partition by tbname,ta interval(2s); + +run tsim/stream/checkTaskStatus.sim + +sql insert into t1 values(now + 3s,1,1,1) (now + 3s,2,10,10) (now + 3s,3,30,30) (now + 11s,1,1,1) (now + 11s,2,10,10); +sql insert into t2 values(now + 4s,1,1,1) (now + 4s,2,10,10) (now + 4s,3,30,30) (now + 12s,1,1,1) (now + 12s,2,10,10); + + +print sql select _wstart, count(*) from st partition by tbname,ta interval(2s); +sql select _wstart, count(*) from st partition by tbname,ta interval(2s); + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +$loop_count = 0 +loop0: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt order by +sql select * from streamt where ta == 1; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + + +# row 0 +if $rows < 2 then + print ======rows=$rows + goto loop0 +endi + +if $data01 != 3 then + print ======data01=$data01 + return -1 +endi + +if $data11 != 2 then + print ======data11=$data11 + goto loop0 +endi + +$loop_count = 0 +loop1: + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +print 2 sql select * from streamt where ta == 2; +sql select * from streamt where ta == 2; + +print $data00 $data01 $data02 $data03 $data04 +print $data10 $data11 $data12 $data13 $data14 +print $data20 $data21 $data22 $data23 $data24 +print $data30 $data31 $data32 $data33 $data34 +print $data40 $data41 $data42 $data43 $data44 +print $data50 $data51 $data52 $data53 $data54 + +# row 0 +if $rows < 2 then + print ======rows=$rows + goto loop1 +endi + +if $data01 != 3 then + print ======data01=$data01 + return -1 +endi + +if $data11 != 2 then + print ======data11=$data11 + goto loop1 +endi + +print end + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/stream/streamTwaInterpFwc.sim b/tests/script/tsim/stream/streamTwaInterpFwc.sim new file mode 100644 index 0000000000..2073378e92 --- /dev/null +++ b/tests/script/tsim/stream/streamTwaInterpFwc.sim @@ -0,0 +1,114 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print step1 +print =============== create database +sql create database test vgroups 4; +sql use test; + +sql create stable st(ts timestamp, a int, b int , c int)tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +sql create stream streams1 trigger force_window_close IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt1 as select _wstart, count(a), sum(b), now, timezone(), ta from st partition by tbname,ta interval(2s) fill(value, 100, 200); +sql create stream streams2 trigger force_window_close IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt2 as select _wstart, count(a), twa(a), sum(b), now, timezone(), ta from st partition by tbname,ta interval(2s) fill(prev); +sql create stream streams3 trigger force_window_close IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt3 as select _irowts, interp(a), interp(b), interp(c), now, timezone(), ta from st partition by tbname,ta every(2s) fill(value, 100, 200, 300); +sql create stream streams4 trigger force_window_close IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt4 as select _irowts, interp(a), interp(b), interp(c), now, timezone(), ta from st partition by tbname,ta every(2s) fill(prev); +sql create stream streams5 trigger force_window_close IGNORE EXPIRED 1 IGNORE UPDATE 1 into streamt5 as select _wstart, count(a), sum(b), now, timezone(), ta from st partition by tbname,ta interval(2s); + +run tsim/stream/checkTaskStatus.sim + +$loop_count = 0 + +_data: + +sql insert into t1 values(now + 3000a,1,1,1) (now + 3100a,5,10,10) (now + 3200a,5,10,10) (now + 5100a,20,1,1) (now + 5200a,30,10,10) (now + 5300a,40,10,10); +sql insert into t2 values(now + 3000a,1,1,1) (now + 3100a,2,10,10) (now + 3200a,30,10,10) (now + 5100a,10,1,1) (now + 5200a,40,10,10) (now + 5300a,7,10,10); + +sleep 2000 + +$loop_count = $loop_count + 1 +if $loop_count < 10 then + goto _data +endi + +print sql select _wstart, count(a), sum(b), now, timezone(), ta from st partition by tbname,ta interval(2s) order by 1, 2; +sql select _wstart, count(a), sum(b), now, timezone(), ta from st partition by tbname,ta interval(2s) order by 1, 2; +$query1_rows = $rows +print ======query1_rows=$query1_rows + +$query1_data01 = $data01 +print ======query1_data01=$query1_data01 + +print select last(*) from (select _wstart, count(a), sum(b), now, timezone(), ta from st partition by tbname,ta interval(2s)) order by 1,2 desc; +sql select _wstart, count(a), sum(b), now, timezone(), ta from st partition by tbname,ta interval(2s) order by 1,2 desc; +print $data00 $data01 $data02 $data03 $data04 + +loop0: + +sleep 2000 + +print sql select * from streamt1 order by 1, 2; +sql select * from streamt1 order by 1, 2; +print ======streamt1=rows=$rows + +if $rows < $query1_rows then + goto loop0 +endi + +if $data01 != $query1_data01 then + print =============data01=$data01 + print ======query1_data01=$query1_data01 + return -1 +endi + +print sql select * from streamt2 order by 1, 2; +sql select * from streamt2 order by 1, 2; +print ======streamt2=rows=$rows + +if $rows < $query1_rows then + goto loop0 +endi + +if $data01 != $query1_data01 then + print =============data01=$data01 + print ======query1_data01=$query1_data01 + return -1 +endi + +print sql select * from streamt3 order by 1, 2; +sql select * from streamt3 order by 1, 2; +print ======streamt3=rows=$rows + +if $rows < $query1_rows then + goto loop0 +endi + +print sql select * from streamt4 order by 1, 2; +sql select * from streamt4 order by 1, 2; +print ======streamt4=rows=$rows + +if $rows < $query1_rows then + goto loop0 +endi + +print sql select * from streamt5 order by 1, 2; +sql select * from streamt5 order by 1, 2; +print ======streamt5=rows=$rows + +if $rows < $query1_rows then + return -1 +endi + +if $data01 != $query1_data01 then + print =============data01=$data01 + print ======query1_data01=$query1_data01 + return -1 +endi + +print end + +system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/system-test/8-stream/force_window_close_interp.py b/tests/system-test/8-stream/force_window_close_interp.py new file mode 100644 index 0000000000..f39ad82ed7 --- /dev/null +++ b/tests/system-test/8-stream/force_window_close_interp.py @@ -0,0 +1,615 @@ +import sys +import threading +from util.log import * +from util.sql import * +from util.cases import * +from util.common import * + + +class TDTestCase: + updatecfgDict = {"debugFlag": 135, "asynclog": 0} + + def init(self, conn, logSql, replicaVar=1): + self.replicaVar = int(replicaVar) + tdLog.debug("start to execute %s" % __file__) + tdSql.init(conn.cursor(), logSql) + self.tdCom = tdCom + + def get_stream_first_ts(self, table_name1, table_name2): + tdSql.query( + f'select * from {table_name1}{table_name2} order by 1 ' + ) + res_ts = tdSql.getData(0, 0) + return res_ts + + def force_window_close( + self, + interval, + partition="tbname", + funciton_name="", + funciton_name_alias="", + delete=False, + fill_value=None, + fill_history_value=None, + case_when=None, + ignore_expired=1, + ignore_update=1, + ): + # partition must be tbname, and not NONE. + tdLog.info( + f"*** testing stream force_window_close+interp+every: every: {interval}, partition: {partition}, fill_history: {fill_history_value}, fill: {fill_value}, delete: {delete}, case_when: {case_when} ***" + ) + self.tdCom.subtable = False + col_value_type = "Incremental" if partition == "c1" else "random" + custom_col_index = 1 if partition == "c1" else None + self.tdCom.custom_col_val = 0 + self.delete = delete + self.tdCom.case_name = sys._getframe().f_code.co_name + self.tdCom.prepare_data( + interval=interval, + fill_history_value=fill_history_value, + custom_col_index=custom_col_index, + col_value_type=col_value_type, + ) + self.stb_name = self.tdCom.stb_name.replace(f"{self.tdCom.dbname}.", "") + self.ctb_name = self.tdCom.ctb_name.replace(f"{self.tdCom.dbname}.", "") + self.tb_name = self.tdCom.tb_name.replace(f"{self.tdCom.dbname}.", "") + self.stb_stream_des_table = f"{self.stb_name}{self.tdCom.des_table_suffix}" + + self.ctb_stream_des_table = f"{self.ctb_name}{self.tdCom.des_table_suffix}" + self.tb_stream_des_table = f"{self.tb_name}{self.tdCom.des_table_suffix}" + if partition == "tbname": + partition_elm_alias = self.tdCom.partition_tbname_alias + + elif partition == "c1": + partition_elm_alias = self.tdCom.partition_col_alias + elif partition == "abs(c1)": + partition_elm_alias = self.tdCom.partition_expression_alias + elif partition is None: + partition_elm_alias = '"no_partition"' + else: + partition_elm_alias = self.tdCom.partition_tag_alias + + if partition: + partition_elm = f"partition by {partition} {partition_elm_alias}" + else: + partition_elm = "" + if fill_value: + if "value" in fill_value.lower(): + fill_value = "VALUE,1" + + # create error stream + tdLog.info("create error stream") + sleep(10) + tdSql.error( + f"create stream itp_force_error_1 trigger force_window_close IGNORE EXPIRED 1 IGNORE UPDATE 0 into itp_force_error_1 as select _irowts,tbname,_isfilled,interp(c1,1) from {self.stb_name} partition by tbname every(5s) fill(prev) ;" + ) + tdSql.error( + f"create stream itp_force_error_1 trigger force_window_close IGNORE EXPIRED 0 IGNORE UPDATE 1 into itp_force_error_1 as select _irowts,tbname,_isfilled,interp(c1,1) from {self.stb_name} partition by tbname every(5s) fill(prev) ;" + ) + tdSql.error( + f"create stream itp_force_error_1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 1 into itp_force_error_1 as select _irowts,tbname,_isfilled,interp(c1,1) from {self.stb_name} partition by tbname every(5s) fill(prev) ;" + ) + tdSql.error( + f"create stream itp_force_error_1 trigger force_window_close IGNORE EXPIRED 1 IGNORE UPDATE 0 into itp_force_error_1 as select _irowts,tbname,_isfilled,interp(c11,1) from {self.stb_name} partition by tbname every(5s) fill(prev) ;" + ) + + # function name : interp + trigger_mode = "force_window_close" + + # # subtable is true + # create stream add :subtable_value=stb_subtable_value or subtable_value=ctb_subtable_value + + # no subtable + # create stream super table and child table + tdLog.info("create stream super table and child table") + self.tdCom.create_stream( + stream_name=f"{self.stb_name}{self.tdCom.stream_suffix}", + des_table=self.stb_stream_des_table, + source_sql=f'select _irowts as irowts,tbname as table_name, _isfilled as isfilled, {funciton_name} as {funciton_name_alias} from {self.stb_name} {partition_elm} every({self.tdCom.dataDict["interval"]}s)', + trigger_mode=trigger_mode, + fill_value=fill_value, + fill_history_value=fill_history_value, + ignore_expired=ignore_expired, + ignore_update=ignore_update, + ) + self.tdCom.create_stream( + stream_name=f"{self.ctb_name}{self.tdCom.stream_suffix}", + des_table=self.ctb_stream_des_table, + source_sql=f'select _irowts as irowts, tbname as table_name, _isfilled as isfilled, {funciton_name} as {funciton_name_alias} from {self.ctb_name} {partition_elm} every({self.tdCom.dataDict["interval"]}s)', + trigger_mode=trigger_mode, + fill_value=fill_value, + fill_history_value=fill_history_value, + ignore_expired=ignore_expired, + ignore_update=ignore_update, + ) + + # creat stream set filter of tag and tbname + tdLog.info("create stream with tag and tbname filter") + tag_t1_value = self.tdCom.tag_value_list[0] + where_tag = f"where t1 = {tag_t1_value}" + where_tbname = f'where tbname="{self.ctb_name}"' + # print(f"tag: {tag_t1_value}") + + self.stb_stream_des_where_tag_table = ( + f"{self.stb_name}_where_tag{self.tdCom.des_table_suffix}" + ) + self.tdCom.create_stream( + stream_name=f"{self.stb_name}_where_tag{self.tdCom.stream_suffix}", + des_table=self.stb_stream_des_where_tag_table, + source_sql=f'select _irowts as irowts,tbname as table_name, _isfilled as isfilled, {funciton_name} as {funciton_name_alias} from {self.stb_name} {where_tag} {partition_elm} every({self.tdCom.dataDict["interval"]}s)', + trigger_mode=trigger_mode, + fill_value=fill_value, + fill_history_value=fill_history_value, + ignore_expired=ignore_expired, + ignore_update=ignore_update, + ) + self.stb_stream_des_where_tbname_table = ( + f"{self.stb_name}_where_tbname{self.tdCom.des_table_suffix}" + ) + self.tdCom.create_stream( + stream_name=f"{self.stb_name}_where_tbname{self.tdCom.stream_suffix}", + des_table=self.stb_stream_des_where_tbname_table, + source_sql=f'select _irowts as irowts,tbname as table_name, _isfilled as isfilled, {funciton_name} as {funciton_name_alias} from {self.stb_name} {where_tbname} {partition_elm} every({self.tdCom.dataDict["interval"]}s)', + trigger_mode=trigger_mode, + fill_value=fill_value, + fill_history_value=fill_history_value, + ignore_expired=ignore_expired, + ignore_update=ignore_update, + ) + + # set partition by tag and column + self.stb_stream_des_partition_tag_table = ( + f"{self.stb_name}_partition_tag{self.tdCom.des_table_suffix}" + ) + self.stb_stream_des_partition_column1_table = ( + f"{self.stb_name}_partition_column1{self.tdCom.des_table_suffix}" + ) + self.stb_stream_des_partition_column2_table = ( + f"{self.stb_name}_partition_column2{self.tdCom.des_table_suffix}" + ) + if partition: + tdLog.info("create stream with partition by tag and tbname ") + partition_elm_new = f"partition by {partition}, t1" + self.tdCom.create_stream( + stream_name=f"{self.stb_name}_partition_tag{self.tdCom.stream_suffix}", + des_table=self.stb_stream_des_partition_tag_table, + source_sql=f'select _irowts as irowts, tbname as table_name, t1 as t_t1, _isfilled as isfilled, {funciton_name} as {funciton_name_alias} from {self.stb_name} {partition_elm_new} every({self.tdCom.dataDict["interval"]}s)', + trigger_mode=trigger_mode, + fill_value=fill_value, + fill_history_value=fill_history_value, + ignore_expired=ignore_expired, + ignore_update=ignore_update, + ) + partition_elm_new = f"partition by {partition}, c1" + self.tdCom.create_stream( + stream_name=f"{self.stb_name}_partition_column1{self.tdCom.stream_suffix}", + des_table=f"{self.stb_name}_partition_column1{self.tdCom.des_table_suffix}", + source_sql=f'select _irowts as irowts, tbname as table_name, c1 as c_c1, _isfilled as isfilled, {funciton_name} as {funciton_name_alias} from {self.stb_name} {partition_elm_new} every({self.tdCom.dataDict["interval"]}s)', + trigger_mode=trigger_mode, + fill_value=fill_value, + fill_history_value=fill_history_value, + ignore_expired=ignore_expired, + ignore_update=ignore_update, + ) + partition_elm_new = f"partition by {partition}, c2" + self.tdCom.create_stream( + stream_name=f"{self.stb_name}_partition_column2{self.tdCom.stream_suffix}", + des_table=f"{self.stb_name}_partition_column2{self.tdCom.des_table_suffix}", + source_sql=f'select _irowts as irowts, tbname as table_name, c2 as c_c2, _isfilled as isfilled, {funciton_name} as {funciton_name_alias} from {self.stb_name} {partition_elm_new} every({self.tdCom.dataDict["interval"]}s)', + trigger_mode=trigger_mode, + fill_value=fill_value, + fill_history_value=fill_history_value, + ignore_expired=ignore_expired, + ignore_update=ignore_update, + ) + + if fill_value: + if "value" in fill_value.lower(): + fill_value = "VALUE,1" + + # create stream general table + tdLog.info("create stream general table") + self.tdCom.create_stream( + stream_name=f"{self.tb_name}{self.tdCom.stream_suffix}", + des_table=self.tb_stream_des_table, + source_sql=f'select _irowts as irowts,tbname as table_name, _isfilled as isfilled, {funciton_name} as {funciton_name_alias} from {self.tb_name} every({self.tdCom.dataDict["interval"]}s)', + trigger_mode=trigger_mode, + fill_value=fill_value, + fill_history_value=fill_history_value, + ignore_expired=ignore_expired, + ignore_update=ignore_update, + ) + + # wait and check stream_task status is ready + time.sleep(self.tdCom.dataDict["interval"]) + tdSql.query("show streams") + tdLog.info(f"tdSql.queryResult:{tdSql.queryResult},tdSql.queryRows:{tdSql.queryRows}") + localQueryResult = tdSql.queryResult + for stream_number in range(tdSql.queryRows): + stream_name = localQueryResult[stream_number][0] + tdCom.check_stream_task_status( + stream_name=stream_name, vgroups=2, stream_timeout=20,check_wal_info=False + ) + time.sleep(self.tdCom.dataDict["interval"]) + time.sleep(30) + + # insert data + self.tdCom.date_time = self.tdCom.genTs(precision=self.tdCom.precision)[0] + start_time = self.tdCom.date_time + time.sleep(1) + tdSql.query("select 1;") + start_force_ts = str(0) + for i in range(self.tdCom.range_count): + cur_time = str(self.tdCom.date_time + self.tdCom.dataDict["interval"]) + ts_value = ( + cur_time + f"+{i*10 + 30}s" + ) + # print(ts_value) + if start_force_ts == "0": + start_force_ts = cur_time + ts_cast_delete_value = self.tdCom.time_cast(ts_value) + self.tdCom.sinsert_rows( + tbname=self.tdCom.ctb_name, + ts_value=ts_value, + custom_col_index=custom_col_index, + col_value_type=col_value_type, + ) + if i % 2 == 0: + self.tdCom.sinsert_rows( + tbname=self.tdCom.ctb_name, + ts_value=ts_value, + custom_col_index=custom_col_index, + col_value_type=col_value_type, + ) + if self.delete and i % 2 != 0: + self.tdCom.sdelete_rows( + tbname=self.tdCom.ctb_name, start_ts=ts_cast_delete_value + ) + self.tdCom.date_time += 1 + self.tdCom.sinsert_rows( + tbname=self.tdCom.tb_name, + ts_value=ts_value, + custom_col_index=custom_col_index, + col_value_type=col_value_type, + ) + if i % 2 == 0: + self.tdCom.sinsert_rows( + tbname=self.tdCom.tb_name, + ts_value=ts_value, + custom_col_index=custom_col_index, + col_value_type=col_value_type, + ) + if self.delete and i % 2 != 0: + self.tdCom.sdelete_rows( + tbname=self.tdCom.tb_name, start_ts=ts_cast_delete_value + ) + self.tdCom.date_time += 1 + + if self.tdCom.subtable: + for tname in [self.stb_name, self.ctb_name]: + group_id = self.tdCom.get_group_id_from_stb(f"{tname}_output") + tdSql.query(f"select * from {self.ctb_name}") + ptn_counter = 0 + for c1_value in tdSql.queryResult: + if partition == "c1": + tbname = self.tdCom.get_subtable_wait( + f"{tname}_{self.tdCom.subtable_prefix}{abs(c1_value[1])}{self.tdCom.subtable_suffix}" + ) + tdSql.query(f"select count(*) from `{tbname}`") + elif partition is None: + tbname = self.tdCom.get_subtable_wait( + f"{tname}_{self.tdCom.subtable_prefix}no_partition{self.tdCom.subtable_suffix}" + ) + tdSql.query(f"select count(*) from `{tbname}`") + elif partition == "abs(c1)": + abs_c1_value = abs(c1_value[1]) + tbname = self.tdCom.get_subtable_wait( + f"{tname}_{self.tdCom.subtable_prefix}{abs_c1_value}{self.tdCom.subtable_suffix}" + ) + tdSql.query(f"select count(*) from `{tbname}`") + elif partition == "tbname" and ptn_counter == 0: + tbname = self.tdCom.get_subtable_wait( + f"{tname}_{self.tdCom.subtable_prefix}{self.ctb_name}{self.tdCom.subtable_suffix}_{tname}_output_{group_id}" + ) + tdSql.query(f"select count(*) from `{tbname}`") + ptn_counter += 1 + tdSql.checkEqual(tdSql.queryResult[0][0] > 0, True) + group_id = self.tdCom.get_group_id_from_stb(f"{self.tb_name}_output") + tdSql.query(f"select * from {self.tb_name}") + ptn_counter = 0 + for c1_value in tdSql.queryResult: + if partition == "c1": + tbname = self.tdCom.get_subtable_wait( + f"{self.tb_name}_{self.tdCom.subtable_prefix}{abs(c1_value[1])}{self.tdCom.subtable_suffix}" + ) + tdSql.query(f"select count(*) from `{tbname}`") + elif partition is None: + tbname = self.tdCom.get_subtable_wait( + f"{self.tb_name}_{self.tdCom.subtable_prefix}no_partition{self.tdCom.subtable_suffix}" + ) + tdSql.query(f"select count(*) from `{tbname}`") + elif partition == "abs(c1)": + abs_c1_value = abs(c1_value[1]) + tbname = self.tdCom.get_subtable_wait( + f"{self.tb_name}_{self.tdCom.subtable_prefix}{abs_c1_value}{self.tdCom.subtable_suffix}" + ) + tdSql.query(f"select count(*) from `{tbname}`") + elif partition == "tbname" and ptn_counter == 0: + tbname = self.tdCom.get_subtable_wait( + f"{self.tb_name}_{self.tdCom.subtable_prefix}{self.tb_name}{self.tdCom.subtable_suffix}_{self.tb_name}_output_{group_id}" + ) + tdSql.query(f"select count(*) from `{tbname}`") + ptn_counter += 1 + + tdSql.checkEqual(tdSql.queryResult[0][0] > 0, True) + if fill_value: + end_date_time = self.tdCom.date_time + final_range_count = self.tdCom.range_count + history_ts = ( + str(start_time) + + f'-{self.tdCom.dataDict["interval"]*(final_range_count+2)}s' + ) + start_ts = self.tdCom.time_cast(history_ts, "-") + future_ts = ( + str(end_date_time) + + f'+{self.tdCom.dataDict["interval"]*(final_range_count+2)}s' + ) + end_ts = self.tdCom.time_cast(future_ts) + tdSql.query("select 2;") + self.tdCom.sinsert_rows(tbname=self.ctb_name, ts_value=history_ts) + self.tdCom.sinsert_rows(tbname=self.tb_name, ts_value=history_ts) + self.tdCom.sinsert_rows(tbname=self.ctb_name, ts_value=future_ts) + self.tdCom.sinsert_rows(tbname=self.tb_name, ts_value=future_ts) + self.tdCom.date_time = start_time + # update + history_ts = ( + str(start_time) + + f'-{self.tdCom.dataDict["interval"]*(final_range_count+2)}s' + ) + start_ts = self.tdCom.time_cast(history_ts, "-") + future_ts = ( + str(end_date_time) + + f'+{self.tdCom.dataDict["interval"]*(final_range_count+2)}s' + ) + end_ts = self.tdCom.time_cast(future_ts) + tdSql.query("select 3;") + self.tdCom.sinsert_rows(tbname=self.ctb_name, ts_value=history_ts) + self.tdCom.sinsert_rows(tbname=self.tb_name, ts_value=history_ts) + self.tdCom.sinsert_rows(tbname=self.ctb_name, ts_value=future_ts) + self.tdCom.sinsert_rows(tbname=self.tb_name, ts_value=future_ts) + + # get query time range using interval count windows + tdSql.query( + f'select _wstart, _wend ,last(ts) from {self.stb_name} where ts >= {start_force_ts} and ts <= {end_ts} partition by tbname interval({self.tdCom.dataDict["interval"]}s)fill ({fill_value}) ' + ) + # getData don't support negative index + end_new_ts = tdSql.getData(tdSql.queryRows - 1, 1) + end_last_but_one_ts = tdSql.getData(tdSql.queryRows - 2, 1) + # source data include that fill valuse is null and "_isfilled" column of the stream output is false + tdSql.execute( + f'insert into {self.ctb_name} (ts,c1) values("{end_new_ts}",-102) ' + ) + tdSql.execute( + f'insert into {self.tb_name} (ts,c1) values("{end_new_ts}",-51) ' + ) + tdSql.execute( + f'insert into {self.ctb_name} (ts,c1) values("{end_last_but_one_ts}",NULL) ' + ) + + tdSql.query("select 4;") + for i in range(self.tdCom.range_count): + ts_value = ( + str(self.tdCom.date_time + self.tdCom.dataDict["interval"]) + + f"+{i*10+30}s" + ) + ts_cast_delete_value = self.tdCom.time_cast(ts_value) + self.tdCom.sinsert_rows(tbname=self.ctb_name, ts_value=ts_value) + self.tdCom.date_time += 1 + self.tdCom.sinsert_rows(tbname=self.tb_name, ts_value=ts_value) + self.tdCom.date_time += 1 + if self.delete: + self.tdCom.sdelete_rows( + tbname=self.ctb_name, + start_ts=self.tdCom.time_cast(start_time), + end_ts=ts_cast_delete_value, + ) + self.tdCom.sdelete_rows( + tbname=self.tb_name, + start_ts=self.tdCom.time_cast(start_time), + end_ts=ts_cast_delete_value, + ) + + # wait for the stream to process the data + # print(self.tdCom.dataDict["interval"]*(final_range_count+2)) + time.sleep(self.tdCom.dataDict["interval"] * (final_range_count + 2)) + + # check the data + for tbname in [self.stb_name, self.ctb_name, self.tb_name]: + tdLog.info(f"tbname:{tbname}") + tdSql.query( + f'select _wstart, _wend ,last(ts) from {tbname} where ts >= {start_force_ts} and ts <= {end_ts} partition by tbname interval({self.tdCom.dataDict["interval"]}s)fill ({fill_value}) ' + ) + start_new_ts = tdSql.getData(0, 1) + ragne_start_ts = start_new_ts + if tbname == self.ctb_name: + if partition == "tbname": + # check data for child table + tdLog.info("check data for child table ") + if fill_value != "PREV": + ragne_start_ts = self.get_stream_first_ts(tbname, self.tdCom.des_table_suffix) + self.tdCom.check_query_data( + f'select irowts, table_name, isfilled, {funciton_name_alias} from {tbname}{self.tdCom.des_table_suffix} where irowts >= {start_force_ts} and irowts <= "{end_new_ts}" order by irowts', + f'select _irowts as irowts ,tb1 as table_name, _isfilled as isfilled , {funciton_name} as {funciton_name_alias} from ( select *, tbname as tb1 from {tbname} where ts >= {start_force_ts} ) partition by tb1 range("{ragne_start_ts}","{end_new_ts}") every({self.tdCom.dataDict["interval"]}s) fill ({fill_value}) order by irowts', + fill_value=fill_value, + ) + elif tbname == self.stb_name: + if partition == "tbname": + # check data for super table + tdLog.info("check data for super table") + if fill_value != "PREV": + ragne_start_ts = self.get_stream_first_ts(tbname, self.tdCom.des_table_suffix) + self.tdCom.check_query_data( + f'select irowts, table_name, isfilled, {funciton_name_alias} from {tbname}{self.tdCom.des_table_suffix} where irowts >= {start_force_ts} and irowts <= "{end_new_ts}" order by irowts', + f'select _irowts as irowts ,tbname as table_name, _isfilled as isfilled , {funciton_name} as {funciton_name_alias} from {tbname} where ts >= {start_force_ts} partition by {partition} range("{ragne_start_ts}","{end_new_ts}") every({self.tdCom.dataDict["interval"]}s) fill ({fill_value}) order by irowts', + fill_value=fill_value, + ) + # tag and tbname filter + tdLog.info("check data for tag and tbname filter") + if fill_value != "PREV": + ragne_start_ts = self.get_stream_first_ts(self.stb_stream_des_where_tag_table, '') + self.tdCom.check_query_data( + f'select irowts, table_name, isfilled, {funciton_name_alias} from {self.stb_stream_des_where_tag_table} where irowts >= {start_force_ts} and irowts <= "{end_new_ts}" order by irowts', + f'select _irowts as irowts ,tbname as table_name, _isfilled as isfilled , {funciton_name} as {funciton_name_alias} from {tbname} {where_tag} and ts >= {start_force_ts} partition by {partition} range("{ragne_start_ts}","{end_new_ts}") every({self.tdCom.dataDict["interval"]}s) fill ({fill_value}) order by irowts', + fill_value=fill_value, + ) + if fill_value != "PREV": + ragne_start_ts = self.get_stream_first_ts(self.stb_stream_des_where_tbname_table, '') + self.tdCom.check_query_data( + f'select irowts, table_name, isfilled, {funciton_name_alias} from {self.stb_stream_des_where_tbname_table} where irowts >= {start_force_ts} and irowts <= "{end_new_ts}" order by irowts', + f'select _irowts as irowts ,tb1 as table_name, _isfilled as isfilled , {funciton_name} as {funciton_name_alias} from ( select *, tbname as tb1 from {tbname} {where_tbname} and ts >= {start_force_ts} ) partition by tb1 range("{ragne_start_ts}","{end_new_ts}") every({self.tdCom.dataDict["interval"]}s) fill ({fill_value}) order by irowts', + fill_value=fill_value, + ) + # check partition by tag and column(c1 or c2) + tdLog.info("check data for partition by tag and column") + if fill_value != "PREV": + ragne_start_ts = self.get_stream_first_ts(self.stb_stream_des_partition_tag_table, '') + self.tdCom.check_query_data( + f'select irowts, table_name, t_t1, isfilled, {funciton_name_alias} from {self.stb_stream_des_partition_tag_table} where irowts >= {start_force_ts} and irowts <= "{end_new_ts}" order by t_t1, irowts', + f'select _irowts as irowts ,tb1 as table_name, t1 as t_t1, _isfilled as isfilled , {funciton_name} as {funciton_name_alias} from ( select *, tbname as tb1 from {tbname} {where_tbname} and ts >= {start_force_ts} ) partition by tb1,t1 range("{ragne_start_ts}","{end_new_ts}") every({self.tdCom.dataDict["interval"]}s) fill ({fill_value}) order by t_t1, irowts', + fill_value=fill_value, + ) + if fill_value == "PREV": + self.tdCom.check_query_data( + f'select irowts, c_c1, isfilled, {funciton_name_alias} from {self.stb_stream_des_partition_column1_table} where irowts >= {start_force_ts} and irowts <= "{end_new_ts}" order by c_c1, irowts', + f'select _irowts as irowts , c1 as c_c1, _isfilled as isfilled , {funciton_name} as {funciton_name_alias} from {tbname} {where_tbname} and ts >= {start_force_ts} partition by {partition},c1 range("{start_new_ts}","{end_new_ts}") every({self.tdCom.dataDict["interval"]}s) fill ({fill_value}) order by c_c1, irowts', + fill_value=fill_value, + ) + self.tdCom.check_query_data( + f'select irowts, c_c2, isfilled, {funciton_name_alias} from {self.stb_stream_des_partition_column2_table} where irowts >= {start_force_ts} and irowts <= "{end_new_ts}" order by c_c2, irowts', + f'select _irowts as irowts , c2 as c_c2, _isfilled as isfilled , {funciton_name} as {funciton_name_alias} from {tbname} {where_tbname} and ts >= {start_force_ts} partition by {partition},c2 range("{start_new_ts}","{end_new_ts}") every({self.tdCom.dataDict["interval"]}s) fill ({fill_value}) order by c_c2, irowts', + fill_value=fill_value, + ) + else: + if partition == "tbname": + # check data for general table + if fill_value != "PREV": + ragne_start_ts = self.get_stream_first_ts(self.stb_stream_des_partition_tag_table, '') + self.tdCom.check_query_data( + f'select irowts, isfilled, {funciton_name_alias} from {tbname}{self.tdCom.des_table_suffix} where irowts >= {start_force_ts} and irowts <= "{end_new_ts}" order by irowts', + f'select _irowts as irowts , _isfilled as isfilled , {funciton_name} as {funciton_name_alias} from ( select *, tbname as tb1 from {tbname} where ts >= {start_force_ts} ) partition by tb1 range("{ragne_start_ts}","{end_new_ts}") every({self.tdCom.dataDict["interval"]}s) fill ({fill_value}) order by irowts', + fill_value=fill_value, + ) + + # Recreate a sub-table that meets the filtering "where_tag" and check if the streaming results are automatically included within it." + where_tag_ctbname = f"{self.ctb_name}_where_tag" + where_tag_ctbname_other_tag = f"{self.ctb_name}_where_tag_1" + tag_t1_value_other = abs(tag_t1_value)-1 + tdSql.execute( + f"create table {where_tag_ctbname} using {self.stb_name} (t1) tags({tag_t1_value}) " + ) + tdSql.execute( + f"create table {where_tag_ctbname_other_tag} using {self.stb_name} (t1) tags({tag_t1_value_other}) " + ) + where_tag_timestamp = self.tdCom.genTs(precision=self.tdCom.precision)[0] + where_tag_ts_start_value = str(where_tag_timestamp) + "+2s" + tdSql.query("select 5;") + self.tdCom.sinsert_rows( + tbname=where_tag_ctbname, ts_value=where_tag_ts_start_value + ) + self.tdCom.sinsert_rows( + tbname=where_tag_ctbname_other_tag, ts_value=where_tag_ts_start_value + ) + time.sleep(self.tdCom.dataDict["interval"]) + for _ in range(self.tdCom.dataDict["interval"]): + tdSql.query( + f"select distinct(table_name) from {self.stb_stream_des_where_tag_table} where table_name=\"{where_tag_ctbname}\"" + ) + if tdSql.queryRows > 0: + if tdSql.checkDataNotExit(0,0, where_tag_ctbname): + break + else: + time.sleep(1) + + if self.delete: + self.tdCom.sdelete_rows( + tbname=self.ctb_name, start_ts=start_ts, end_ts=ts_cast_delete_value + ) + self.tdCom.sdelete_rows( + tbname=self.tb_name, start_ts=start_ts, end_ts=ts_cast_delete_value + ) + for tbname in [self.stb_name, self.ctb_name, self.tb_name]: + if tbname != self.tb_name: + if "value" in fill_value.lower(): + fill_value = ( + "VALUE,1,2,3,6,7,8,9,10,11,1,2,3,4,5,6,7,8,9,10,11" + ) + if partition == "tbname": + self.tdCom.check_query_data( + f"select wstart, {self.tdCom.fill_stb_output_select_str} from {tbname}{self.tdCom.des_table_suffix} order by wstart", + f'select _wstart AS wstart, {self.tdCom.fill_stb_source_select_str} from {tbname} where ts >= {start_ts.replace("-", "+")} and ts <= {end_ts} partition by {partition} interval({self.tdCom.dataDict["interval"]}s) fill ({fill_value}) order by wstart', + fill_value=fill_value, + ) + else: + self.tdCom.check_query_data( + f"select wstart, {self.tdCom.fill_stb_output_select_str} from {tbname}{self.tdCom.des_table_suffix} order by wstart,`min(c1)`", + f'select * from (select _wstart AS wstart, {self.tdCom.fill_stb_source_select_str} from {tbname} where ts >= {start_ts} and ts <= {end_ts} partition by {partition} interval({self.tdCom.dataDict["interval"]}s) fill ({fill_value}) order by wstart) where `min(c1)` is not Null order by wstart,`min(c1)`', + fill_value=fill_value, + ) + + else: + if "value" in fill_value.lower(): + fill_value = "VALUE,1,2,3,6,7,8,9,10,11" + if partition == "tbname": + self.tdCom.check_query_data( + f"select wstart, {self.tdCom.fill_tb_output_select_str} from {tbname}{self.tdCom.des_table_suffix} order by wstart", + f'select _wstart AS wstart, {self.tdCom.fill_tb_source_select_str} from {tbname} where ts >= {start_ts.replace("-", "+")} and ts <= {end_ts} partition by {partition} interval({self.tdCom.dataDict["interval"]}s) fill ({fill_value}) order by wstart', + fill_value=fill_value, + ) + else: + self.tdCom.check_query_data( + f"select wstart, {self.tdCom.fill_tb_output_select_str} from {tbname}{self.tdCom.des_table_suffix} order by wstart,`min(c1)`", + f'select * from (select _wstart AS wstart, {self.tdCom.fill_tb_source_select_str} from {tbname} where ts >= {start_ts} and ts <= {end_ts} partition by {partition} interval({self.tdCom.dataDict["interval"]}s) fill ({fill_value}) order by wstart) where `min(c1)` is not Null order by wstart,`min(c1)`', + fill_value=fill_value, + ) + + def run(self): + for fill_value in ["PREV", "VALUE","NULL"]: + self.force_window_close( + interval=10, + partition="tbname", + funciton_name="interp(c1)", + funciton_name_alias="intp_c1", + delete=False, + ignore_update=1, + fill_value=fill_value, + ) + self.force_window_close( + interval=8, + partition="tbname", + funciton_name="interp(c1)", + funciton_name_alias="intp_c1", + delete=False, + ignore_update=1, + fill_value="PREV", + ) + # self.force_window_close(interval=random.randint(10, 15), partition="c1", ignore_update=1) + # self.force_window_close(interval=random.randint(10, 15), partition="abs(c1)", ignore_update=1) + # self.force_window_close(interval=random.randint(10, 15), partition=None, delete=True) + # self.force_window_close(interval=random.randint(10, 15), partition=self.tdCom.stream_case_when_tbname, case_when=f'case when {self.tdCom.stream_case_when_tbname} = tbname then {self.tdCom.partition_tbname_alias} else tbname end') + # self.force_window_close(interval=random.randint(10, 15), partition="tbname", fill_history_value=1, fill_value="NULL") + # for fill_value in ["NULL", "PREV", "NEXT", "LINEAR", "VALUE,1,2,3,4,5,6,7,8,9,10,11,1,2,3,4,5,6,7,8,9,10,11"]: + # # for fill_value in ["PREV", "NEXT", "LINEAR", "VALUE,1,2,3,4,5,6,7,8,9,10,11,1,2,3,4,5,6,7,8,9,10,11"]: + # self.at_once_interval(interval=random.randint(10, 15), partition="tbname", fill_value=fill_value) + # self.at_once_interval(interval=random.randint(10, 15), partition="tbname", fill_value=fill_value, delete=True) + + def stop(self): + tdSql.close() + tdLog.success(f"{__file__} successfully executed") + + +event = threading.Event() + + +tdCases.addLinux(__file__, TDTestCase()) +tdCases.addWindows(__file__, TDTestCase()) diff --git a/tests/system-test/8-stream/force_window_close_interval.py b/tests/system-test/8-stream/force_window_close_interval.py new file mode 100644 index 0000000000..b75579d220 --- /dev/null +++ b/tests/system-test/8-stream/force_window_close_interval.py @@ -0,0 +1,300 @@ +import sys +import threading +from util.log import * +from util.sql import * +from util.cases import * +from util.common import * + + +class TDTestCase: + updatecfgDict = {"debugFlag": 135, "asynclog": 0} + + def init(self, conn, logSql, replicaVar=1): + self.replicaVar = int(replicaVar) + tdLog.debug("start to execute %s" % __file__) + tdSql.init(conn.cursor(), logSql) + self.tdCom = tdCom + + def get_source_firt_ts(self, table_name1): + tdSql.query( + f'select cast(first(ts) as bigint) from {table_name1} order by 1' + ) + # getData don't support negative index + res_ts = tdSql.getData(0, 0) + return res_ts + + def get_source_last_ts(self, table_name1): + tdSql.query( + f'select cast(last(ts) as bigint) from {table_name1} order by 1' + ) + # getData don't support negative index + res_ts = tdSql.getData(0, 0) + return res_ts + + def get_stream_first_win_ts(self, table_name1): + tdSql.query( + f'select _wstart, count(*) from {table_name1} interval({self.tdCom.dataDict["interval"]}s) order by 1' + ) + res_ts = tdSql.getData(0, 0) + return res_ts + + def insert_data(self, custom_col_index, col_value_type): + self.tdCom.date_time = self.tdCom.genTs(precision=self.tdCom.precision)[0] + time.sleep(1) + min_new_ts = 0 + for i in range(self.tdCom.range_count): + cur_time = str(self.tdCom.date_time + self.tdCom.dataDict["interval"]) + ts_value = ( + cur_time + f"+{i * 5 + 30}s" + ) + if min_new_ts == 0: + min_new_ts = ts_value + + ts_cast_delete_value = self.tdCom.time_cast(ts_value) + self.tdCom.sinsert_rows( + tbname=self.tdCom.ctb_name, + ts_value=ts_value, + custom_col_index=custom_col_index, + col_value_type=col_value_type, + ) + if i % 2 == 0 and min_new_ts != ts_value: + self.tdCom.sinsert_rows( + tbname=self.tdCom.ctb_name, + ts_value=ts_value, + custom_col_index=custom_col_index, + col_value_type=col_value_type, + ) + if self.delete and i % 2 != 0: + self.tdCom.sdelete_rows( + tbname=self.tdCom.ctb_name, start_ts=ts_cast_delete_value + ) + self.tdCom.date_time += 1 + self.tdCom.sinsert_rows( + tbname=self.tdCom.tb_name, + ts_value=ts_value, + custom_col_index=custom_col_index, + col_value_type=col_value_type, + ) + if i % 2 == 0 and min_new_ts != ts_value: + self.tdCom.sinsert_rows( + tbname=self.tdCom.tb_name, + ts_value=ts_value, + custom_col_index=custom_col_index, + col_value_type=col_value_type, + ) + if self.delete and i % 2 != 0: + self.tdCom.sdelete_rows( + tbname=self.tdCom.tb_name, start_ts=ts_cast_delete_value + ) + self.tdCom.date_time += 1 + cur_time = str(self.tdCom.date_time + self.tdCom.dataDict["interval"]) + max_new_ts = (cur_time + f"+{self.tdCom.range_count * 10 + 30}s") + self.tdCom.sinsert_rows( + tbname=self.tdCom.ctb_name, + ts_value=max_new_ts, + custom_col_index=custom_col_index, + col_value_type=col_value_type, + ) + self.tdCom.sinsert_rows( + tbname=self.tdCom.tb_name, + ts_value=max_new_ts, + custom_col_index=custom_col_index, + col_value_type=col_value_type, + ) + return (min_new_ts, max_new_ts) + + def insert_disorder_data(self, custom_col_index, col_value_type): + min_ts = self.get_source_firt_ts(self.tb_name) + max_ts = self.get_source_last_ts(self.tb_name) + min_ts_str = str(min_ts) + f"-10000s" + max_ts_str = str(max_ts) + f"+10000s" + self.tdCom.sinsert_rows( + tbname=self.tdCom.ctb_name, + ts_value=min_ts_str, + custom_col_index=custom_col_index, + col_value_type=col_value_type, + ) + self.tdCom.sinsert_rows( + tbname=self.tdCom.tb_name, + ts_value=min_ts_str, + custom_col_index=custom_col_index, + col_value_type=col_value_type, + ) + self.tdCom.sinsert_rows( + tbname=self.tdCom.ctb_name, + ts_value=max_ts_str, + custom_col_index=custom_col_index, + col_value_type=col_value_type, + ) + self.tdCom.sinsert_rows( + tbname=self.tdCom.tb_name, + ts_value=max_ts_str, + custom_col_index=custom_col_index, + col_value_type=col_value_type, + ) + + def force_window_close( + self, + interval, + partition="tbname", + delete=False, + fill_value=None, + filter=None + ): + # partition must be tbname, and not NONE. + tdLog.info( + f"*** testing stream force_window_close + interval + fill. partition: {partition}, interval: {interval}, fill: {fill_value}, delete: {delete} ***" + ) + fwc_downsampling_function_list = ["min(c1)", "max(c2)", "sum(c3)", "twa(c7)", "count(c8)", "elapsed(ts)", "timediff(1, 0, 1h)", "timezone()","min(t1)", "max(t2)", "sum(t3)", + "twa(t7)", "count(t8)"] + fwc_stb_output_select_str = ','.join(list(map(lambda x:f'`{x}`', fwc_downsampling_function_list))) + fwc_tb_output_select_str = ','.join(list(map(lambda x:f'`{x}`', fwc_downsampling_function_list[0:7]))) + fwc_stb_source_select_str = ','.join(fwc_downsampling_function_list) + fwc_tb_source_select_str = ','.join(fwc_downsampling_function_list[0:7]) + + fill_history_value = 0 + ignore_expired = 1 + ignore_update = 1 + self.tdCom.subtable = False + col_value_type = "Incremental" if partition == "c1" else "random" + custom_col_index = 1 if partition == "c1" else None + self.tdCom.custom_col_val = 0 + self.delete = delete + self.tdCom.case_name = sys._getframe().f_code.co_name + self.tdCom.prepare_data( + interval=interval, + custom_col_index=custom_col_index, + col_value_type=col_value_type, + ) + self.stb_name = self.tdCom.stb_name.replace(f"{self.tdCom.dbname}.", "") + self.ctb_name = self.tdCom.ctb_name.replace(f"{self.tdCom.dbname}.", "") + self.tb_name = self.tdCom.tb_name.replace(f"{self.tdCom.dbname}.", "") + self.stb_stream_des_table = f"{self.stb_name}{self.tdCom.des_table_suffix}" + + self.ctb_stream_des_table = f"{self.ctb_name}{self.tdCom.des_table_suffix}" + self.tb_stream_des_table = f"{self.tb_name}{self.tdCom.des_table_suffix}" + + if partition: + partition_elm = f"partition by {partition}" + else: + partition_elm = "" + + query_partition_elm = partition_elm + + if fill_value: + if "value" in fill_value.lower(): + stb_fill_value='VALUE,1,2,3,4,5,6,1,2,3,4,5' + tb_fill_value='VALUE,1,2,3,4,5,6' + else: + stb_fill_value=fill_value + tb_fill_value=fill_value + query_stb_fill_elm = f"fill({stb_fill_value})" + query_tb_fill_elm = f"fill({tb_fill_value})" + else: + query_stb_fill_elm = "" + query_tb_fill_elm = "" + stb_fill_value = None + tb_fill_value=None + + where_elm = "where 1=1" + if filter: + where_elm = f" and {filter}" + + trigger_mode = "force_window_close" + + # no subtable + # create stream super table and child table + tdLog.info("create stream super table and child table") + self.tdCom.create_stream( + stream_name=f'{self.stb_name}{self.tdCom.stream_suffix}', + des_table=self.stb_stream_des_table, + source_sql=f'select _wstart AS wstart, {fwc_stb_source_select_str} from {self.stb_name} {where_elm} {partition_elm} interval({self.tdCom.dataDict["interval"]}s)', + trigger_mode=trigger_mode, + fill_value=stb_fill_value, + fill_history_value=fill_history_value, + ignore_expired=ignore_expired, + ignore_update=ignore_update, + ) + + self.tdCom.create_stream( + stream_name=f"{self.tb_name}{self.tdCom.stream_suffix}", + des_table=self.tb_stream_des_table, + source_sql=f'select _wstart AS wstart, {fwc_tb_source_select_str} from {self.tb_name} {where_elm} {partition_elm} interval({self.tdCom.dataDict["interval"]}s)', + trigger_mode=trigger_mode, + fill_value=tb_fill_value, + fill_history_value=fill_history_value, + ignore_expired=ignore_expired, + ignore_update=ignore_update, + ) + + # wait and check stream_task status is ready + tdSql.query("show streams") + tdLog.info(f"tdSql.queryResult:{tdSql.queryResult},tdSql.queryRows:{tdSql.queryRows}") + localQueryResult = tdSql.queryResult + for stream_number in range(tdSql.queryRows): + stream_name = localQueryResult[stream_number][0] + tdCom.check_stream_task_status( + stream_name=stream_name, vgroups=2, stream_timeout=20,check_wal_info=False + ) + time.sleep(self.tdCom.dataDict["interval"]) + time.sleep(20) + + # insert data + tdLog.info("insert data") + start_new_ts, temp = self.insert_data(custom_col_index, col_value_type) + time.sleep(self.tdCom.dataDict["interval"] * 2) + tdLog.info("insert data") + temp, end_new_ts = self.insert_data(custom_col_index, col_value_type) + + #history and future + self.insert_disorder_data(custom_col_index, col_value_type) + + time.sleep(self.tdCom.dataDict["interval"] * 6 * 2) + + tdLog.info("check data") + # check the data + where_elm = f'{where_elm} and ts >= {start_new_ts} and ts <= {end_new_ts}' + for tbname in [self.stb_name, self.tb_name]: + if fill_value: + query_first_win_ts = self.get_stream_first_win_ts(tbname) + query_where_elm = f'where wstart >= "{query_first_win_ts}"' + stream_where_elm = f'where wstart <= {end_new_ts}' + else: + query_where_elm = "" + stream_where_elm = "" + + # check data + tdLog.info(f"check data for table {tbname}") + if tbname == self.stb_name: + self.tdCom.check_query_data( + f'select wstart, {fwc_stb_output_select_str} from {tbname}{self.tdCom.des_table_suffix} {stream_where_elm} order by wstart', + f'select * from (select _wstart AS wstart, {fwc_stb_source_select_str} from {tbname} {where_elm} {query_partition_elm} interval({self.tdCom.dataDict["interval"]}s) {query_stb_fill_elm} order by wstart) {query_where_elm}', + sorted=True + ) + else: + self.tdCom.check_query_data( + f'select wstart, {fwc_tb_output_select_str} from {tbname}{self.tdCom.des_table_suffix} {stream_where_elm} order by wstart', + f'select * from (select _wstart AS wstart, {fwc_tb_source_select_str} from {tbname} {where_elm} {query_partition_elm} interval({self.tdCom.dataDict["interval"]}s) {query_tb_fill_elm} order by wstart) {query_where_elm}', + sorted=True + ) + + def run(self): + for fill_value in ["VALUE", "NULL", "PREV", None]: + self.force_window_close( + interval=5, + partition="tbname", + delete=True, + fill_value=fill_value, + ) + + def stop(self): + tdLog.info("stop========================================") + tdSql.close() + tdLog.success(f"{__file__} successfully executed") + + +event = threading.Event() + + +tdCases.addLinux(__file__, TDTestCase()) +tdCases.addWindows(__file__, TDTestCase())