From 9a2d122e6d3be267bc8f9d1b71de8ebdf2800d36 Mon Sep 17 00:00:00 2001 From: Jim Hughes Date: Mon, 27 Nov 2023 17:09:04 -0500 Subject: [PATCH 1/4] [FLINK-33667] Implement restore tests for MatchRecognize node --- .../stream/MatchRecognizeRestoreTest.java | 44 ++ .../stream/MatchRecognizeTestPrograms.java | 221 ++++++++ .../complex-match/plan/complex-match.json | 510 ++++++++++++++++++ .../complex-match/savepoint/_metadata | Bin 0 -> 21081 bytes .../simple-match/plan/simple-match.json | 369 +++++++++++++ .../simple-match/savepoint/_metadata | Bin 0 -> 16607 bytes .../plan/skip-past-last-row.json | 405 ++++++++++++++ .../skip-past-last-row/savepoint/_metadata | Bin 0 -> 17991 bytes .../skip-to-first/plan/skip-to-first.json | 407 ++++++++++++++ .../skip-to-first/savepoint/_metadata | Bin 0 -> 18079 bytes .../skip-to-last/plan/skip-to-last.json | 407 ++++++++++++++ .../skip-to-last/savepoint/_metadata | Bin 0 -> 18079 bytes .../plan/skip-to-next-row.json | 405 ++++++++++++++ .../skip-to-next-row/savepoint/_metadata | Bin 0 -> 18431 bytes 14 files changed, 2768 insertions(+) create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeRestoreTest.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeTestPrograms.java create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/complex-match/plan/complex-match.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/complex-match/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/simple-match/plan/simple-match.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/simple-match/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-past-last-row/plan/skip-past-last-row.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-past-last-row/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-first/plan/skip-to-first.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-first/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-last/plan/skip-to-last.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-last/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-next-row/plan/skip-to-next-row.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-next-row/savepoint/_metadata diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeRestoreTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeRestoreTest.java new file mode 100644 index 0000000000000..5322182ce1e51 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeRestoreTest.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.nodes.exec.stream; + +import org.apache.flink.table.planner.plan.nodes.exec.testutils.RestoreTestBase; +import org.apache.flink.table.test.program.TableTestProgram; + +import java.util.Arrays; +import java.util.List; + +/** Restore tests for {@link StreamExecMatch}. */ +public class MatchRecognizeRestoreTest extends RestoreTestBase { + + public MatchRecognizeRestoreTest() { + super(StreamExecMatch.class); + } + + @Override + public List programs() { + return Arrays.asList( + MatchRecognizeTestPrograms.SIMPLE_MATCH, + MatchRecognizeTestPrograms.COMPLEX_MATCH, + MatchRecognizeTestPrograms.SKIP_TO_FIRST, + MatchRecognizeTestPrograms.SKIP_TO_LAST, + MatchRecognizeTestPrograms.SKIP_TO_NEXT_ROW, + MatchRecognizeTestPrograms.SKIP_PAST_LAST_ROW); + } +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeTestPrograms.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeTestPrograms.java new file mode 100644 index 0000000000000..ee75742a1e753 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeTestPrograms.java @@ -0,0 +1,221 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.nodes.exec.stream; + +import org.apache.flink.table.test.program.SinkTestStep; +import org.apache.flink.table.test.program.SourceTestStep; +import org.apache.flink.table.test.program.TableTestProgram; +import org.apache.flink.types.Row; + +/** {@link TableTestProgram} definitions for testing {@link StreamExecMatch}. */ +public class MatchRecognizeTestPrograms { + static final Row[] SIMPLE_DATA = { + Row.of(1L, "a"), + Row.of(2L, "z"), + Row.of(3L, "b"), + Row.of(4L, "c"), + Row.of(5L, "d"), + Row.of(6L, "a"), + Row.of(7L, "b"), + Row.of(8L, "c"), + Row.of(9L, "a"), + Row.of(10L, "b") + }; + + static final Row[] SIMPLE_DATA2 = {Row.of(11L, "c")}; + + static final Row[] COMPLEX_DATA = { + Row.of("ACME", 1L, 19, 1), + Row.of("BETA", 2L, 18, 1), + Row.of("ACME", 3L, 17, 2), + Row.of("ACME", 4L, 13, 3), + Row.of("BETA", 5L, 16, 2), + Row.of("ACME", 6L, 20, 4) + }; + + static final Row[] COMPLEX_DATA2 = {Row.of("BETA", 7L, 22, 4)}; + + static final TableTestProgram SIMPLE_MATCH = + TableTestProgram.of("simple-match", "simple match recognize test") + .setupTableSource( + SourceTestStep.newBuilder("MyTable") + .addSchema( + "id bigint", "name varchar", "proctime as PROCTIME()") + .producedBeforeRestore(SIMPLE_DATA) + .producedAfterRestore(SIMPLE_DATA2) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("MySink") + .addSchema("a bigint", "b bigint", "c bigint") + .consumedBeforeRestore(Row.of(6L, 7L, 8L)) + .consumedAfterRestore(Row.of(9L, 10L, 11L)) + .build()) + .runSql( + "insert into MySink" + + " SELECT T.aid, T.bid, T.cid\n" + + " FROM MyTable MATCH_RECOGNIZE (\n" + + " ORDER BY proctime\n" + + " MEASURES\n" + + " `A\"`.id AS aid,\n" + + " \u006C.id AS bid,\n" + + " C.id AS cid\n" + + " PATTERN (`A\"` \u006C C)\n" + + " DEFINE\n" + + " `A\"` AS name = 'a',\n" + + " \u006C AS name = 'b',\n" + + " C AS name = 'c'\n" + + " ) AS T") + .build(); + + static final TableTestProgram COMPLEX_MATCH = + TableTestProgram.of("complex-match", "complex match recognize test") + .setupTableSource( + SourceTestStep.newBuilder("MyTable") + .addSchema( + "symbol string", + "tstamp bigint", + "price int", + "tax int", + "proctime as PROCTIME()") + .producedBeforeRestore(COMPLEX_DATA) + .producedAfterRestore(COMPLEX_DATA2) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("MySink") + .addSchema( + "somestring string", "a bigint", "b bigint", "c bigint") + .consumedBeforeRestore(Row.of("ACME", 19L, 13L, null)) + .consumedAfterRestore(Row.of("BETA", 18L, 16L, null)) + .build()) + .runSql( + "insert into MySink SELECT * FROM MyTable MATCH_RECOGNIZE (\n" + + " PARTITION BY symbol\n" + + " ORDER BY proctime\n" + + " MEASURES\n" + + " FIRST(DOWN.price) as first,\n" + + " LAST(DOWN.price) as last,\n" + + " FIRST(DOWN.price, 5) as nullPrice\n" + + " ONE ROW PER MATCH\n" + + " AFTER MATCH SKIP PAST LAST ROW\n" + + " PATTERN (DOWN{2,} UP)\n" + + " DEFINE\n" + + " DOWN AS price < LAST(DOWN.price, 1) OR LAST(DOWN.price, 1) IS NULL,\n" + + " UP AS price > LAST(DOWN.price)\n" + + ") AS T") + .build(); + + static final TableTestProgram SKIP_TO_FIRST = + getSkipTestProgram( + "skip-to-first", + "skip to first match recognize test", + "AFTER MATCH SKIP TO FIRST B", + new Row[] {Row.of(1L, 100, 106), Row.of(1L, 105, 107), Row.of(1L, 101, 101)}, + new Row[] {Row.of(1L, 100, 111)}); + + static final TableTestProgram SKIP_TO_LAST = + getSkipTestProgram( + "skip-to-last", + "skip to last match recognize test", + "AFTER MATCH SKIP TO LAST B", + new Row[] {Row.of(1L, 100, 106), Row.of(1L, 105, 107), Row.of(1L, 101, 101)}, + new Row[] {Row.of(1L, 100, 111)}); + + static final TableTestProgram SKIP_TO_NEXT_ROW = + getSkipTestProgram( + "skip-to-next-row", + "skip to next row match recognize test", + "AFTER MATCH SKIP TO NEXT ROW", + new Row[] { + Row.of(1L, 100, 106), + Row.of(1L, 102, 106), + Row.of(1L, 104, 106), + Row.of(1L, 106, 106), + Row.of(1L, 105, 107), + Row.of(1L, 107, 107), + Row.of(1L, 101, 101) + }, + new Row[] {Row.of(1L, 100, 111), Row.of(1L, 110, 111), Row.of(1L, 111, 111)}); + + static final TableTestProgram SKIP_PAST_LAST_ROW = + getSkipTestProgram( + "skip-past-last-row", + "skip past last row match recognize test", + "AFTER MATCH SKIP PAST LAST ROW", + new Row[] {Row.of(1L, 100, 106), Row.of(1L, 107, 107)}, + new Row[] {Row.of(1L, 100, 111)}); + + private static TableTestProgram getSkipTestProgram( + final String name, + final String description, + final String skipClause, + final Row[] beforeRows, + final Row[] afterRows) { + return TableTestProgram.of(name, description) + .setupTableSource( + SourceTestStep.newBuilder("MyTable") + .addSchema( + " vehicle_id bigint,\n" + + " engine_temperature int,\n" + + " proctime as PROCTIME()") + .producedBeforeRestore( + Row.of(1L, 100), + Row.of(1L, 102), + Row.of(1L, 104), + Row.of(1L, 106), + Row.of(1L, 105), + Row.of(1L, 107), + Row.of(1L, 101), + Row.of(1L, 100)) + .producedAfterRestore( + Row.of(1L, 110), Row.of(1L, 111), Row.of(1L, 99)) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("MySink") + .addSchema( + " vehicle_id bigint,\n" + + " Initial_Temp int,\n" + + " Final_Temp int\n") + .consumedBeforeRestore(beforeRows) + .consumedAfterRestore(afterRows) + .build()) + .runSql(getSql(skipClause)) + .build(); + } + + private static String getSql(final String afterClause) { + final String sql = + "insert into MySink" + + " SELECT * FROM\n" + + " MyTable\n" + + " MATCH_RECOGNIZE(\n" + + " PARTITION BY vehicle_id\n" + + " ORDER BY `proctime`\n" + + " MEASURES \n" + + " FIRST(A.engine_temperature) as Initial_Temp,\n" + + " LAST(A.engine_temperature) as Final_Temp\n" + + " ONE ROW PER MATCH\n" + + " %s\n" + + " PATTERN (A+ B)\n" + + " DEFINE\n" + + " A as LAST(A.engine_temperature,1) is NULL OR A.engine_temperature > LAST(A.engine_temperature,1),\n" + + " B as B.engine_temperature < LAST(A.engine_temperature)\n" + + " )MR;"; + return String.format(sql, afterClause); + } +} diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/complex-match/plan/complex-match.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/complex-match/plan/complex-match.json new file mode 100644 index 0000000000000..d266ee2c36769 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/complex-match/plan/complex-match.json @@ -0,0 +1,510 @@ +{ + "flinkVersion" : "1.19", + "nodes" : [ { + "id" : 6, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`MyTable`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "symbol", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "tstamp", + "dataType" : "BIGINT" + }, { + "name" : "price", + "dataType" : "INT" + }, { + "name" : "tax", + "dataType" : "INT" + }, { + "name" : "proctime", + "kind" : "COMPUTED", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "internalName" : "$PROCTIME$1", + "operands" : [ ], + "type" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + }, + "serializableString" : "PROCTIME()" + } + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "outputType" : "ROW<`symbol` VARCHAR(2147483647), `tstamp` BIGINT, `price` INT, `tax` INT>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[symbol, tstamp, price, tax])", + "inputProperties" : [ ] + }, { + "id" : 7, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "BIGINT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "INT" + }, { + "kind" : "CALL", + "internalName" : "$PROCTIME$1", + "operands" : [ ], + "type" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "symbol", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "tstamp", + "fieldType" : "BIGINT" + }, { + "name" : "price", + "fieldType" : "INT" + }, { + "name" : "tax", + "fieldType" : "INT" + }, { + "name" : "proctime", + "fieldType" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + } ] + }, + "description" : "Calc(select=[symbol, tstamp, price, tax, PROCTIME() AS proctime])" + }, { + "id" : 8, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "symbol", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "tstamp", + "fieldType" : "BIGINT" + }, { + "name" : "price", + "fieldType" : "INT" + }, { + "name" : "tax", + "fieldType" : "INT" + }, { + "name" : "proctime", + "fieldType" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + } ] + }, + "description" : "Exchange(distribution=[hash[symbol]])" + }, { + "id" : 9, + "type" : "stream-exec-match_1", + "matchSpec" : { + "pattern" : { + "kind" : "CALL", + "syntax" : "BINARY", + "sqlKind" : "PATTERN_CONCAT", + "operands" : [ { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$PATTERN_QUANTIFIER$1", + "operands" : [ { + "kind" : "LITERAL", + "value" : "DOWN", + "type" : "CHAR(4) NOT NULL" + }, { + "kind" : "LITERAL", + "value" : 2, + "type" : "INT NOT NULL" + }, { + "kind" : "LITERAL", + "value" : -1, + "type" : "INT NOT NULL" + }, { + "kind" : "LITERAL", + "value" : false, + "type" : "BOOLEAN NOT NULL" + } ], + "type" : "NULL" + }, { + "kind" : "LITERAL", + "value" : "UP", + "type" : "CHAR(2) NOT NULL" + } ], + "type" : "NULL" + }, + "patternDefinitions" : { + "DOWN" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$OR$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "*", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "DOWN", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "BOOLEAN" + }, { + "kind" : "CALL", + "syntax" : "POSTFIX", + "internalName" : "$IS NULL$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "DOWN", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "BOOLEAN NOT NULL" + } ], + "type" : "BOOLEAN" + }, + "UP" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$>$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "*", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "DOWN", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "BOOLEAN" + } + }, + "measures" : { + "first" : { + "kind" : "CALL", + "syntax" : "PREFIX", + "internalName" : "$FINAL$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$FIRST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "DOWN", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "INT" + }, + "last" : { + "kind" : "CALL", + "syntax" : "PREFIX", + "internalName" : "$FINAL$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "DOWN", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "INT" + }, + "nullPrice" : { + "kind" : "CALL", + "syntax" : "PREFIX", + "internalName" : "$FINAL$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$FIRST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "DOWN", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 5, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "INT" + } + }, + "after" : { + "kind" : "LITERAL", + "symbol" : "MATCH_RECOGNIZE_AFTER_OPTION", + "value" : "SKIP_PAST_LAST_ROW", + "type" : { + "type" : "SYMBOL", + "nullable" : false + } + }, + "subsets" : { }, + "allRows" : false, + "partition" : { + "fields" : [ 0 ] + }, + "orderBy" : { + "fields" : [ { + "index" : 4, + "isAscending" : true, + "nullIsLast" : false + } ] + }, + "interval" : null + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`symbol` VARCHAR(2147483647), `first` INT, `last` INT, `nullPrice` INT>", + "description" : "Match(partitionBy=[symbol], orderBy=[proctime ASC], measures=[FINAL(FIRST(DOWN.price, 0)) AS first, FINAL(LAST(DOWN.price, 0)) AS last, FINAL(FIRST(DOWN.price, 5)) AS nullPrice], rowsPerMatch=[ONE ROW PER MATCH], after=[SKIP PAST LAST ROW], pattern=[(PATTERN_QUANTIFIER(_UTF-16LE'DOWN', 2, -1, false), _UTF-16LE'UP')], define=[{DOWN=OR(<(LAST(*.$2, 0), LAST(DOWN.$2, 1)), IS NULL(LAST(DOWN.$2, 1))), UP=>(LAST(*.$2, 0), LAST(DOWN.$2, 0))}])" + }, { + "id" : 10, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$CAST$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT" + } ], + "type" : "BIGINT" + }, { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$CAST$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + } ], + "type" : "BIGINT" + }, { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$CAST$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "INT" + } ], + "type" : "BIGINT" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`somestring` VARCHAR(2147483647), `a` BIGINT, `b` BIGINT, `c` BIGINT>", + "description" : "Calc(select=[symbol AS somestring, CAST(first AS BIGINT) AS a, CAST(last AS BIGINT) AS b, CAST(nullPrice AS BIGINT) AS c])" + }, { + "id" : 11, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`MySink`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "somestring", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "a", + "dataType" : "BIGINT" + }, { + "name" : "b", + "dataType" : "BIGINT" + }, { + "name" : "c", + "dataType" : "BIGINT" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`somestring` VARCHAR(2147483647), `a` BIGINT, `b` BIGINT, `c` BIGINT>", + "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[somestring, a, b, c])" + } ], + "edges" : [ { + "source" : 6, + "target" : 7, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 7, + "target" : 8, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 8, + "target" : 9, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 9, + "target" : 10, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 10, + "target" : 11, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/complex-match/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/complex-match/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..0ee410d21b5d8aa7b87a3f140159847d38004105 GIT binary patch literal 21081 zcmeGkZHya7b?o~{u5FrY6NRQtqo}#039Y^U_PId_-<^}Vb?n1-lJ*L5#=GOY?b*A| z?)q{Up(YgwkcyfLs02UyfhginQIQY`{_qoosE{Zi{t=K06$m86hma6ed2e?2ynAb> z4vF(Yb|rgf=i|+r_ukIDH}B2nPcMcDA*1laD`4Kp*p4R;{^aMUFYP;h>Ao-haqN}f zTV)rp+^&RavWfo_a>2U$x7&&W54Uft;a>G%5B}k*LENMO0_2Mgxyc?AR;VU4)TYEV zU1&vxS}L}j$kd{db7zvN*1!lD2uciK0&qFC8|{+ z6H$XJ5F~*-8E-OW(O6m?6i&f}=Y|tpAa}@mvD0kg=n7;r_FYz0!yu^wpeL9l5I6}0 zP6C0WCs^nS4zSqn&}uvItVR+84Q)ePb9987|to3mb0*JkWDAD z<=Laz^7;4Qf1hkW`5uym{}%f6>}6cV&}3zC1Mhley4bgg+@&;V#h?Z&($+dl$mA2b zr*d<(N;O-})v8a;=g8QJY~gqgkP4DXo~%`J<$Sh~U&xhtZrp>a$l76m9xfiv@*cWE z<-%dhySNAk1CD`@kSF^>n`39$TCvlrgKRp|UEsZQU0t5&3tb1{WKzA;XfQ2YR3(;| zrp%k8gN+$3*yeDYqY03w2J0`k!W08KwC;``Jvpu}hYWsuF?%dmna|GVHuBs{M_Vv; z6shnDRhCe$beMXWV>o^CU4z-rLfgbWXax6O!Mg64nV;&CBwj37K zR!dbJ(_fwHEPRZ%ySTb=^71w~Y~Id!5Vv0|m#7w0aTzB)oDc^_J}}gUVa#6`%r);C zEesYp1&=_OYdK%Y*4Nn?c`>gTC=L9L;ma(pLxZi-4vf3aZ`sv#d?tS+U#wd6dvqZB z`NTuL1yvKET<%D&>{Ab4CR7ToH)1fw&P~0K3?zFnm(%8gsxYYv$~mm0t+x9`4(7>3 zTVtZE%c@c!qsy`)86+{7in|W;321fb=9?18^-#EXSM#9KRI|tCYbW#7qqRb5He0A+ zW^JKVL^XE=(1;DhE*C+a!f}Q?OU4bddA?klMbHifcZ>{Mv!D0i-e2JG<2A!J$Uvrc-OhN!F1}A@4y@_v=MS+l;iDOAO3iUb(lVg^>)Lw z|hG_mevPb z*~2iBeT~>~J1Kf?YftscbglNoQAK=>xa$tOks9#<_ja)Y?hvMj&dSD8sA=tuhM2U- zZpuAm?>>6ZP2O9@A2HlL>&HFHe@|jnxF^SSA0Xknwx~TToQQD?u9p>HTzMZk5Uwd_ zr3mOKI(?XXbT`hRE8W@ z+f1V{<>+DDb<(<%Mvs{x=-5T{13PY|-3kZe0z=>a^)4CL&NIK?^>*-`NdLufL{%B! zDDoqoZqkUSVZIo+2JHAD{7qsCubEsm3%Ac;`c3$ajLnr!7ST?EEAWL6U5wtzbKr@` z((!!$#|tOyO3oG&n-Ioh5og{ylpr+5A6zDQ`|hf3LWuHO@HYZsXRZqi%-HDN{f1*h zCu3MOz|lYOQAm2n?&{he?ld;Du?;w@OxRVh)G9FV_<)%QdLz{n$PTE_TVpA&VJ7hZ zq2Bmn3#g0S0S1r^e}cb=JvN)G7NOA-%r6e!N?W&wdZTL9=K=uf{#|hDLJga$c9#}- z0`Y=)?RW?N>^dYn3(iJM>JCC8uz}xWE@s06)PW1b@OJkDw1MAt8))*k@c4`GKJmcA z2fp(1C-iu%1X)Ok3-B`ujtTem^;!C}rBk2$)k{-vuU-7-mfB?CtqINWql^KOdT<#p>6|Z}kZd`06 zB$kMTl1Um5C8TLFlu0)lp(K?u(NroKtEVYo_9ys3m~J1uacn@9(tNI*t(MAGMCM$+ zQZ46aj#u-gV(nOIE=MLS^M!o1wqAhz@28Gx9^yl)huoV`uB6o8ft`ucK6V}?dW1MYgKX~qv)pm*iTT~-xOQRlkLN+cRh(*%u% z;_2y3D3NH?Lo}Akgr*}Y%2-N_rA11}uCKv>L00;Y4hMIw#l^c~_VcCZR>3+_XwxAa z-Yl0$SbHy)BkvK9U^+wF4$G`5>FB0Mm)AeO{l>0uJ@;x2qQ?v5?gp0GMY=&I3m9)L zU`)GE00%I}x$L_vbi477F8+P{E5T2$E)@T}Tw0?UAv`qXiy zrAl%`=I0wguK%-jU-Y|weqoZo1*1*Kv1#|{2LtE=-Z?`q{Pfhnc5^TR(I7h<9PU;# z*}JU^nJ)$Seyu+D_Jd#jF@1Ri&vK0FaByU`4H7$g7MJ8v7*EnG-TVZK+g7q<0?(Wr z#y1+J9xm{!jey`wOcyn|Z3^|h4w*JxKPb}*yc9@*2Oy20F+IB4hB(GMbQNRi@Q}nc zJFJ4xw*;QW+obyt2N&V{q1yn3-mWHO*WHKZxbK3m^W;^d560> zW_!+qlF6S1Xy&jCG1TnP^Y6U)*2TS-K=>XfV=J`TVesxdV4H_MKKsT?58nCZ-#*`k z?Hsj$V@x|oZFc9$UL~%5m|>ob8nu?S$Uw61pN#sTlTm)S&ov46xtiI?KR|GGv-xdJ zKgYy7I`j?j;kdYNC8!(CTi-TByT-Q-rI|I~HoVry#IH+?o^KnrnQt4~Cb4OUb?r~z zHuP;n&lf)2`zwx3M=nKtYh@TA%h&y@8NO}k+lKtKJ?<%A&ri3wkJ9+I^_8BcvFPX9 zhFzzb%R@FCixX%617x$g2-x8Q+pINVi*opDxJFC zXdOp!2ayrDwC03;?lirAzOgDHX6tBV`XDLox~)-Z%_Rl#N+wgG<$A3=bC z6CgSW5i0hFzp(jhpZmq;*Pq<}eeV6Z18{6n{zVUiB)v{MQPr5hR+$JsBUKUB0Ucg% zX8T7B($jac9lU%-mkC+!aWcqJZ&$rW?ZkTu%u5d>paq<$@9~i+z&pH-cen@Mwmdo5TUAjgI;q4f~O^#;g=o}+Q%gf4~&aQ{?} KSLSoYx&Hz;m})}+ literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/simple-match/plan/simple-match.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/simple-match/plan/simple-match.json new file mode 100644 index 0000000000000..00d4cc7552d5c --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/simple-match/plan/simple-match.json @@ -0,0 +1,369 @@ +{ + "flinkVersion" : "1.19", + "nodes" : [ { + "id" : 1, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`MyTable`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "id", + "dataType" : "BIGINT" + }, { + "name" : "name", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "proctime", + "kind" : "COMPUTED", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "internalName" : "$PROCTIME$1", + "operands" : [ ], + "type" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + }, + "serializableString" : "PROCTIME()" + } + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "outputType" : "ROW<`id` BIGINT, `name` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[id, name])", + "inputProperties" : [ ] + }, { + "id" : 2, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "BIGINT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "internalName" : "$PROCTIME$1", + "operands" : [ ], + "type" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "id", + "fieldType" : "BIGINT" + }, { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "proctime", + "fieldType" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + } ] + }, + "description" : "Calc(select=[id, name, PROCTIME() AS proctime])" + }, { + "id" : 3, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "SINGLETON" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "id", + "fieldType" : "BIGINT" + }, { + "name" : "name", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "proctime", + "fieldType" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + } ] + }, + "description" : "Exchange(distribution=[single])" + }, { + "id" : 4, + "type" : "stream-exec-match_1", + "matchSpec" : { + "pattern" : { + "kind" : "CALL", + "syntax" : "BINARY", + "sqlKind" : "PATTERN_CONCAT", + "operands" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "sqlKind" : "PATTERN_CONCAT", + "operands" : [ { + "kind" : "LITERAL", + "value" : "A\"", + "type" : "CHAR(2) NOT NULL" + }, { + "kind" : "LITERAL", + "value" : "l", + "type" : "CHAR(1) NOT NULL" + } ], + "type" : "NULL" + }, { + "kind" : "LITERAL", + "value" : "C", + "type" : "CHAR(1) NOT NULL" + } ], + "type" : "NULL" + }, + "patternDefinitions" : { + "A\"" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "*", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : "a", + "type" : "VARCHAR(2147483647) NOT NULL" + } ], + "type" : "BOOLEAN" + }, + "C" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "*", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : "c", + "type" : "VARCHAR(2147483647) NOT NULL" + } ], + "type" : "BOOLEAN" + }, + "l" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "*", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : "b", + "type" : "VARCHAR(2147483647) NOT NULL" + } ], + "type" : "BOOLEAN" + } + }, + "measures" : { + "aid" : { + "kind" : "CALL", + "syntax" : "PREFIX", + "internalName" : "$FINAL$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A\"", + "inputIndex" : 0, + "type" : "BIGINT" + } ], + "type" : "BIGINT" + }, + "bid" : { + "kind" : "CALL", + "syntax" : "PREFIX", + "internalName" : "$FINAL$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "l", + "inputIndex" : 0, + "type" : "BIGINT" + } ], + "type" : "BIGINT" + }, + "cid" : { + "kind" : "CALL", + "syntax" : "PREFIX", + "internalName" : "$FINAL$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "C", + "inputIndex" : 0, + "type" : "BIGINT" + } ], + "type" : "BIGINT" + } + }, + "after" : { + "kind" : "LITERAL", + "symbol" : "MATCH_RECOGNIZE_AFTER_OPTION", + "value" : "SKIP_TO_NEXT_ROW", + "type" : { + "type" : "SYMBOL", + "nullable" : false + } + }, + "subsets" : { }, + "allRows" : false, + "partition" : { + "fields" : [ ] + }, + "orderBy" : { + "fields" : [ { + "index" : 2, + "isAscending" : true, + "nullIsLast" : false + } ] + }, + "interval" : null + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`aid` BIGINT, `bid` BIGINT, `cid` BIGINT>", + "description" : "Match(orderBy=[proctime ASC], measures=[FINAL(A\".id) AS aid, FINAL(l.id) AS bid, FINAL(C.id) AS cid], rowsPerMatch=[ONE ROW PER MATCH], after=[SKIP TO NEXT ROW], pattern=[((_UTF-16LE'A\"', _UTF-16LE'l'), _UTF-16LE'C')], define=[{A\"==(LAST(*.$1, 0), _UTF-16LE'a'), l==(LAST(*.$1, 0), _UTF-16LE'b'), C==(LAST(*.$1, 0), _UTF-16LE'c')}])" + }, { + "id" : 5, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`MySink`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a", + "dataType" : "BIGINT" + }, { + "name" : "b", + "dataType" : "BIGINT" + }, { + "name" : "c", + "dataType" : "BIGINT" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`aid` BIGINT, `bid` BIGINT, `cid` BIGINT>", + "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[aid, bid, cid])" + } ], + "edges" : [ { + "source" : 1, + "target" : 2, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 2, + "target" : 3, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 3, + "target" : 4, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 4, + "target" : 5, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/simple-match/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/simple-match/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..0e252bf28ee0f0e95d5e02d1ac9166d787340662 GIT binary patch literal 16607 zcmeGjZEPE7`L&&<>1VPvYH8P&Xw4?d>bv+Wu_-@r5|?^&9nW^tHkI&R-@SIOIp4W= z=QOEANrkjc8lnTikNyCrF$p0$2?>FaHuh^Olg1yIrv2c@Kte*A;18PwLWAvV zIXgJ(oOYE@O5XGP@qFL=@jTC)ex@2k2o1m=eSBtt2G-vF$@3q7bY%a~yC2AJ4!r!V z)wV|4?N0RF@%;W@H_m_l>%aQVd(WNt;_aV)7=k9+bB%VfhxjNWKN{5~Whtl;L&Jgr zrQVPY-O=yB4Yb?p;%WMY4}G^H$>dA?GS>KtDvLzZ`C5oCO@^1EiBf3d>gCwvQYCyj zQdg?gsR{a2FY`tIQbUo9P-G$z3B}^EL@1ORD zAyJ;1BC!}xBH>UXR89nOEEEq$%LySEuf!_Ba!3f31tB^So~R&n;5}eqa3k)h5PWEC zE|p_nYDO0(%gF`u43TTIVoOK0Y0Q)r}+%cP5~h7=n&GM~(xO_d6%d^(v) zFQoEN>qqmdR^@OV3yXxS$RJhDz-5_mT0=3U8sUu9I%yb^taGUqgJ=qt3q+H!EL|nq zxVNgH;JUu38UQ+q}Be^l}# zUk2^1&^EhCH_%At5?;nUC5z8Ux^Z$vLyv6;-5z}V#)qRn^nbJFNB#^N7F1;!bT6sO zjEr>v{E!J)#!8iERVOoOm#kJL0n3yu1BEkyhd1TLGkNh$UOeN1xBBv#)|itk^<@nm z@{-5c&Z}2IMDz*x(Ewm$yiTuXaSgByI~YBK>nAUwVXHl->XHG38))A}2hnroOGGe$ z>VYLm5e+ofhFpr52fR-~59qnKZh^9q9&{khfI$%PG}ad>`jtAU@r0KXW-z82Edg)< zp9;)m4f&WJ2N~`9WpoYsKw}eK0dh#P1vn0{67Fp1480*k#uKkoqW6YENDmQ9R9d5V6~ zmTbT{0$}(e(FIMan@0Vx!=?*{CMi{F(+YePR05B{JJmRSA6Th_Tzwd*;Jcz?TB@cM z!)=^sg@W4SyGDx<0}W26znab#nW-M4;LGgwsE@XfwWs?uEhc@ z$yzAxMvwR1wBp-}X~nYFzWl6pdb87VUw&BT94qZ7l%Hd!2%wy2roeTH6VzHwRUEs@ zGS&&8R;rGP!u``VOMI4~u0MhFx9@c8+26j4)2sdMdw=`RZMgq$?aTYycj_Oo50?Gy zJKeeWx9?4#OzRZc3$H+b`+j#`mA?EwJM!yq-(5!yIWA^I? zee5n!zVna6KfC?k;rEx{dgD76|M=+3P(=?0Yv9-yBb9JSz-3{wLMAF=WolB0mq{WN zD~siD7>A+TZmT;K?4d*H!{Q=7^iV-8feq6GyX6s&XlQaufIvtv5>G^e zQK1~85szpP10P`$PlQQm5~%(F{y=IS=%&1V+Tjju6Q@#4{h!?QszGv^R0DVWGGQ)J z$4PagMvpN8JN9$)$9vv-<@!r|U%h_wji=vy`xw+SBcScJN?Bjq-qzT!p8Y%gNY6m` z0rw@)m%v?1V9Td)5|2!XaWNXKR3<{fD2|X|LLfpgA_yS}fa8g1G>XuXKLclM;q#+j z#s-{G2!yty-AV-*?RbRv!5zCEAg=kz> z6-ingHwy=X^%XAIR;=E^jSpSu$c+nMYMCeoAhb#cMo*0kWskv+W|L=9g^$Y?n_M1+t=$-KBTTV{HcJ zE_XN{vJ3`gqEwB=`!7voOB@>8=ac!F)5-id?%cU!iSx^z56aBbqInQ4?uB`G{di)mO^ipev%(z$f; zbSX18lgyN8Wocn9OH=Iv5b_Uqr7o91VI+HK9uf8N4jO1Em!F%VpdHI{uOh1p9VoV! zMrvqgBA}U&vo5)#U8QRgdZugmO{uIy(V^afg&v&)=-w>+qin5zx*H=(44aVY7e#wsc#QR(4xQEw=#Y=yKm~R_M6M7rt;mYgx&sX-)>;RPzJ@0~8=JPCi~{q(nQkBI zX!GuG=4t${c`3%RqxT_W^EgsqT;oopw4#M{gC^H^+RioffSE^ceX{rFuyN2m{V*?M zP*CB{7CS=TXTHYXoH(+ckEZj918#gcz75we*XM9=wE0BI0B?;}V%}%IuBigi;r3#+ zbOpT8S`BMUL2&EHWh^Yw^(4^Jaoa0>UZ=5cXav^)BHZ+oIJHhR3{#HI(G#`S5{yd} zR(!)TPi)OI_?_kY`0i|+gTRNwDFLe7CP8wA6}Zk#_a3X zJ6Q*W%4T`p`bmpJZqm#aTcT}1?XbpTvuuL8bH{2x$nr$TJ@c+@FmQ)hSYp?pxtZo` zPNVaU0o!8>kl&An7SM<&=>pco8LS8du0Vq*a3z6;hz!Wk*91WDHZB2)A5Yv{8q6+l#yw8+@xPZ_kKy3_Wtv$A{pq zVW=A+6zp5!bofipy)*R6`O#mq+m1Vd#u{d~5DgUbdfx?ABRp9l0(e|ig>M2n)f{Hu zhcr;zyON`DtJi2LA<1r-8tAkaRXeVobdSSb1rH>U1vt6e#Lj_ku1Po7hHlFUy1Fg9 zV", + "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[vehicle_id, engine_temperature])", + "inputProperties" : [ ] + }, { + "id" : 28, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "BIGINT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "CALL", + "internalName" : "$PROCTIME$1", + "operands" : [ ], + "type" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "vehicle_id", + "fieldType" : "BIGINT" + }, { + "name" : "engine_temperature", + "fieldType" : "INT" + }, { + "name" : "proctime", + "fieldType" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + } ] + }, + "description" : "Calc(select=[vehicle_id, engine_temperature, PROCTIME() AS proctime])" + }, { + "id" : 29, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "vehicle_id", + "fieldType" : "BIGINT" + }, { + "name" : "engine_temperature", + "fieldType" : "INT" + }, { + "name" : "proctime", + "fieldType" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + } ] + }, + "description" : "Exchange(distribution=[hash[vehicle_id]])" + }, { + "id" : 30, + "type" : "stream-exec-match_1", + "matchSpec" : { + "pattern" : { + "kind" : "CALL", + "syntax" : "BINARY", + "sqlKind" : "PATTERN_CONCAT", + "operands" : [ { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$PATTERN_QUANTIFIER$1", + "operands" : [ { + "kind" : "LITERAL", + "value" : "A", + "type" : "CHAR(1) NOT NULL" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + }, { + "kind" : "LITERAL", + "value" : -1, + "type" : "INT NOT NULL" + }, { + "kind" : "LITERAL", + "value" : false, + "type" : "BOOLEAN NOT NULL" + } ], + "type" : "NULL" + }, { + "kind" : "LITERAL", + "value" : "B", + "type" : "CHAR(1) NOT NULL" + } ], + "type" : "NULL" + }, + "patternDefinitions" : { + "A" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$OR$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "POSTFIX", + "internalName" : "$IS NULL$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "BOOLEAN NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$>$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$PREV$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "BOOLEAN" + } ], + "type" : "BOOLEAN" + }, + "B" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$PREV$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "B", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "BOOLEAN" + } + }, + "measures" : { + "Final_Temp" : { + "kind" : "CALL", + "syntax" : "PREFIX", + "internalName" : "$FINAL$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "INT" + }, + "Initial_Temp" : { + "kind" : "CALL", + "syntax" : "PREFIX", + "internalName" : "$FINAL$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$FIRST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "INT" + } + }, + "after" : { + "kind" : "LITERAL", + "symbol" : "MATCH_RECOGNIZE_AFTER_OPTION", + "value" : "SKIP_PAST_LAST_ROW", + "type" : { + "type" : "SYMBOL", + "nullable" : false + } + }, + "subsets" : { }, + "allRows" : false, + "partition" : { + "fields" : [ 0 ] + }, + "orderBy" : { + "fields" : [ { + "index" : 2, + "isAscending" : true, + "nullIsLast" : false + } ] + }, + "interval" : null + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`vehicle_id` BIGINT, `Initial_Temp` INT, `Final_Temp` INT>", + "description" : "Match(partitionBy=[vehicle_id], orderBy=[proctime ASC], measures=[FINAL(FIRST(A.engine_temperature, 0)) AS Initial_Temp, FINAL(LAST(A.engine_temperature, 0)) AS Final_Temp], rowsPerMatch=[ONE ROW PER MATCH], after=[SKIP PAST LAST ROW], pattern=[(PATTERN_QUANTIFIER(_UTF-16LE'A', 1, -1, false), _UTF-16LE'B')], define=[{A=OR(IS NULL(LAST(A.$1, 1)), >(PREV(A.$1, 0), LAST(A.$1, 1))), B=<(PREV(B.$1, 0), LAST(A.$1, 0))}])" + }, { + "id" : 31, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`MySink`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "vehicle_id", + "dataType" : "BIGINT" + }, { + "name" : "Initial_Temp", + "dataType" : "INT" + }, { + "name" : "Final_Temp", + "dataType" : "INT" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`vehicle_id` BIGINT, `Initial_Temp` INT, `Final_Temp` INT>", + "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[vehicle_id, Initial_Temp, Final_Temp])" + } ], + "edges" : [ { + "source" : 27, + "target" : 28, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 28, + "target" : 29, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 29, + "target" : 30, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 30, + "target" : 31, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-past-last-row/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-past-last-row/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..000fbcf4013d95b614c35856babb0818b08beb65 GIT binary patch literal 17991 zcmeGkTZ|i5^^UzsHfbK4WEHxBM%#9?5G9^jKX;vggm>55xOKe7c9Lc*;f!aly^|f! zI5XqTZa$&}BvPpnDTqh=0YsumNTd=HAP^6UukukO1bpy=2&#~fQ2C-#D+CJX-kGsy zcE%fTgEwj7kvucLk8{pF_sqTbJnqe$uZ0jo1MnjkV?NNpPsan_`^;;{zwp)3%AF&( ze*K=6ca`MZg&0yRfQEHRSqf>`(0IXsr{0teosZ@aaC0ov8HqNFjr1QmSV|rc;fQKcw(stc!Dy7S_nPRDs znLb_0%;(Cd=4aDrsF=@YO6`m^tv7Tgl|7v<7t@7IDw|nI7a%o&&Zt_A{%g9WW6LIy)y}S=48FgD=7V_kMW$6K`JoPC0;>EZSWqd3Kg=prPyqewpV8 zSzK1qjT0*xI=n7)yYgE%|26#Wz!&QQ6v(1If~qV-*GsB0Bl9`{-fIGud8NkDq!U^6 zgsj#iftLwc28v_>4`=d=qx|A1zc|_o&MM1g+jUN?G?q1V&`Ta|c0s)aEFxDhfCc~? zZR_-Ej;{l@J&uN+;u|MkLVK+IysAqE5N@FTFFA;wuUx=_0aOnxNs4HoQ6F*%UJmfS z041R0?uG@*Mmpj^ngxNt;z?d#BT@cKsr{ zf*9zriLL-SIN1gkSU@`@J=c_Ff;x)^3G#xf8U{-Ew3rm?+(D!sG!0$RP;8?lk-8RH zXdgn*s@+61St3DfT4~munGFL43-f1zhvT#mkzaO3m1XEM=t_O;v{F|^sVdP%fn5J* z&*AV--u?0r<=%h{DK_tp-5Wp~@W?c}`u6AFKSaSWf`JY=dbnN8?B4xt$SFU|EGE*hGo)QC(g@KF#Bd>r0MkCXR-l?L#2$1+}&1R0kl5y;`XB4=8vWfa3LrD%n~ z+{IiWk0AqXoz6U$$(5+g9t1o)GcWuMg%+M2a}tcSCKf`< z7(?#kl)4dG>Pc4ztUtD<47Dx^_855Z<>HZlUw$?MGbe=?fZ@p@wOJLI{mS27{q4^{q+?BsX?haMClx*@5cb{-gXsW0mh<)(B~c*mp3}Q+T3%D5)t+FEX3h3Kmg?Wx^0nfFzQwh3&b>$tuO;jnsblzH|dxUWL= zRS2-=cUE>>ipZGj-1fA>o*n>3wzR zgFlT(FY70MeEs`B`1#w!t3iNYC7CuB@{YFAyOm?T_aA)@JSH3%{qAGZ(0ev>v)#XZ zHHn3IA`+Sq1ThrD6+Q%x;b=(YaU~L~PE~{`kI>Wa0^#8PB~IC1TDEO>Y~QlI(Uq08 zEOl5HnI2gex1pU%l`k5+fr%s7l4u4Ey^ubap0)h`rE~c-+CrWBRA~JD`%#afQos!s zptHHT6vfa%fia!;0mjkXAbPPQv{`%!ubOV5F~@rwxPPi^rQwZ5uxZs7c@2w|X0?hn zHm8a>BaWF52ZFT;F4$Hqf}=GII@hr^E_~@_tQdgMdO9$AQd}r|41PG5I+ZTwQ!`z} zbl5I2orn~BMwLV&S2~1!U`KO0q`Mlm-(<}+?TQd4tf6r@AcB7K89bHBuOpqG_4GjJ z^(sUlv}twGsN{c=L%W(lzl+PsjhhXVZ@6`H644(I_U3~2kMQTSgjW@(x z=3T;^iO>b8*QCNAW8*l|AY5Zkq$Ht@be$~6lDM^z$e}j)ZCvtg@khm2*jvl$gV85O z+`|vG6@~;Ac9vPn;X`FH?2S^ObkKw;b0rsxZs=RNZ-9Bc#4{kuQLT^|9pm-OY|v2ECGzx9ugeLT1JJR0&( z2K57ff!mNZyt`to&O>4#kg~$Am>VCvnb;Xfkwo|n!VfMp)qc8)4y2Ax&?0c#es|49 zGR!js4t2tBgGO6OQHD-Ihy4q?@KO>Fm~}`!^G9C>>HQghAf$%f5O_I4|G{n4KI*{2 z(u#5;1;};xuK9%fpO5^0^68`DeLuQB{rcfwz4bdtBDJoM8o7Z zH5rQ}laY8u6cR!eitV(D!-HNnLV9WfgZ38HrY7J5=HZ9~YSFbyZSMI8n8y=>n4AhF zs^NGjHZeIFN+y$4xK(4Z#8fgV@B$F_5BNaN}A*0pbYrG z$Nn!VlD_CnDK;y5vrbZMFvX_cz=}wcwpLY;KX}>RR0Nrag`5Z{4zy__-aOw!6|81( z?V_B`K~jh8*C!W)V7qs?8zvO(6FNxsTi5?M_{NLFKchR(cL0rbo^Av;P~7V}6jTj! zcm)gK+EEp*1!%5S&EA`CAm4rK!*Ifa(e8v4-R*CnlU`JQt#;Bq279(VkboE9q4SCJ z0lL{1-K-DYmJoDxa>4eQRBk?(0p}nblVSDnXbUC~0PI+Ekcf{cwknWb@NJEgCgL{S OZlG<&)5Ux`H~T+^o~j4{ literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-first/plan/skip-to-first.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-first/plan/skip-to-first.json new file mode 100644 index 0000000000000..8bff998b7445e --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-first/plan/skip-to-first.json @@ -0,0 +1,407 @@ +{ + "flinkVersion" : "1.19", + "nodes" : [ { + "id" : 12, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`MyTable`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "vehicle_id", + "dataType" : "BIGINT" + }, { + "name" : "engine_temperature", + "dataType" : "INT" + }, { + "name" : "proctime", + "kind" : "COMPUTED", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "internalName" : "$PROCTIME$1", + "operands" : [ ], + "type" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + }, + "serializableString" : "PROCTIME()" + } + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "outputType" : "ROW<`vehicle_id` BIGINT, `engine_temperature` INT>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[vehicle_id, engine_temperature])", + "inputProperties" : [ ] + }, { + "id" : 13, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "BIGINT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "CALL", + "internalName" : "$PROCTIME$1", + "operands" : [ ], + "type" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "vehicle_id", + "fieldType" : "BIGINT" + }, { + "name" : "engine_temperature", + "fieldType" : "INT" + }, { + "name" : "proctime", + "fieldType" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + } ] + }, + "description" : "Calc(select=[vehicle_id, engine_temperature, PROCTIME() AS proctime])" + }, { + "id" : 14, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "vehicle_id", + "fieldType" : "BIGINT" + }, { + "name" : "engine_temperature", + "fieldType" : "INT" + }, { + "name" : "proctime", + "fieldType" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + } ] + }, + "description" : "Exchange(distribution=[hash[vehicle_id]])" + }, { + "id" : 15, + "type" : "stream-exec-match_1", + "matchSpec" : { + "pattern" : { + "kind" : "CALL", + "syntax" : "BINARY", + "sqlKind" : "PATTERN_CONCAT", + "operands" : [ { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$PATTERN_QUANTIFIER$1", + "operands" : [ { + "kind" : "LITERAL", + "value" : "A", + "type" : "CHAR(1) NOT NULL" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + }, { + "kind" : "LITERAL", + "value" : -1, + "type" : "INT NOT NULL" + }, { + "kind" : "LITERAL", + "value" : false, + "type" : "BOOLEAN NOT NULL" + } ], + "type" : "NULL" + }, { + "kind" : "LITERAL", + "value" : "B", + "type" : "CHAR(1) NOT NULL" + } ], + "type" : "NULL" + }, + "patternDefinitions" : { + "A" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$OR$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "POSTFIX", + "internalName" : "$IS NULL$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "BOOLEAN NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$>$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$PREV$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "BOOLEAN" + } ], + "type" : "BOOLEAN" + }, + "B" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$PREV$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "B", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "BOOLEAN" + } + }, + "measures" : { + "Final_Temp" : { + "kind" : "CALL", + "syntax" : "PREFIX", + "internalName" : "$FINAL$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "INT" + }, + "Initial_Temp" : { + "kind" : "CALL", + "syntax" : "PREFIX", + "internalName" : "$FINAL$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$FIRST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "INT" + } + }, + "after" : { + "kind" : "CALL", + "syntax" : "PREFIX", + "internalName" : "$SKIP TO FIRST$1", + "operands" : [ { + "kind" : "LITERAL", + "value" : "B", + "type" : "CHAR(1) NOT NULL" + } ], + "type" : "NULL" + }, + "subsets" : { }, + "allRows" : false, + "partition" : { + "fields" : [ 0 ] + }, + "orderBy" : { + "fields" : [ { + "index" : 2, + "isAscending" : true, + "nullIsLast" : false + } ] + }, + "interval" : null + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`vehicle_id` BIGINT, `Initial_Temp` INT, `Final_Temp` INT>", + "description" : "Match(partitionBy=[vehicle_id], orderBy=[proctime ASC], measures=[FINAL(FIRST(A.engine_temperature, 0)) AS Initial_Temp, FINAL(LAST(A.engine_temperature, 0)) AS Final_Temp], rowsPerMatch=[ONE ROW PER MATCH], after=[SKIP TO FIRST _UTF-16LE'B'], pattern=[(PATTERN_QUANTIFIER(_UTF-16LE'A', 1, -1, false), _UTF-16LE'B')], define=[{A=OR(IS NULL(LAST(A.$1, 1)), >(PREV(A.$1, 0), LAST(A.$1, 1))), B=<(PREV(B.$1, 0), LAST(A.$1, 0))}])" + }, { + "id" : 16, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`MySink`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "vehicle_id", + "dataType" : "BIGINT" + }, { + "name" : "Initial_Temp", + "dataType" : "INT" + }, { + "name" : "Final_Temp", + "dataType" : "INT" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`vehicle_id` BIGINT, `Initial_Temp` INT, `Final_Temp` INT>", + "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[vehicle_id, Initial_Temp, Final_Temp])" + } ], + "edges" : [ { + "source" : 12, + "target" : 13, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 13, + "target" : 14, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 14, + "target" : 15, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 15, + "target" : 16, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-first/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-first/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..687d7898d87ff93cc5b607a9d776b1d014e47177 GIT binary patch literal 18079 zcmeGkTWlLy_1aF;wA(gm8%k5yD!Wa~E?dv|9lPBRoW$9>bsZDC?KUgnj%Ti&X~r|| z%s7uCRV|2JL}3f!C4_*L*dIiBNC*iD@d2Z&t%4_ zL*=yFdZc;X$9dgz@11k*xs$zA3nGLD;3S97xS@g1{qtXg(c;Y`@B8}b4gC-MPn&ty zeP}!>V2rDwL^K+YCM%I>IG(D6W6AJjOh|^KF%F9EvVkHHEwDa<$A|V7m8Qz$dCUn| zE$W<(Ng>GiLRFOT33i!N*{UKzrpDI8Y&j8Gilxfo(A6vP#8NeKCEAc{waE}UYFF42 zd$}o#dN>k_#!}%}EESK(CS!2KS&du94MmjoU_8#p;zBT$;&C`sNhV_92wTO`M1rq` zgRy8t2*&sdA52wnDpPM=g))=nA@r#(L zN}{|3FCxP$^|~T6`dR}wby3oo3a4R)Q&nz_nOf6vK^5zwE-vFVNh)Gh(0H?Z~X^Sd}@+rU>pN zNAvIgEA5`Q|ADq?Dte&_A&}KNDE5ZBQKfL|H@7?%T*^hiVw5Lk)tSn7O zBe~1mGRG37WpkpYpIlYZ(G8(nm49*T-=p93f3ogJ{u~XbTph3t+Za8|HBP>QhRytfqCwYGWF75)#YXg86i^^4tgOLPI1~ILBHC7W<2iZger{DjHUn>fKLQw^5&!M zI7n&NuApnk2Q3!r3Xn~bZD9Tdv{TgbO-UlCb7+VlpH~!JM+py(NukaSMCt)+=$eXR znfHvT@s&bimfa?P5mr?r%e; z!#?R>t2ci8^w)mLeItN;Su~)*901~ip?HOK?*Q-7uTRa*gP;44r~V-~;eXTAVx&tTsTdJD2gI$V<2^3P29q6**I4Ei3Cz zEk&yY=5F6LavRan;8gadY`#Qw_7LFNm4y!jm#?=GvC6qb5?|r z*1$qYnedV8F-qMSEp^b*02`03XGN@bySH5h zgn(;U-S_7=-hAiAYe@i|wLlXpSX#&A)2L69WA@}#ghnsC3yd2RVW~qiyQePDIKsVc z=5nPtCN9uwbJBVcReI!xINw$bab}k><=5S8;n6{!OyOArL0@`gT;A;LYDv#FN(4~B zHA-~0eaP5n8zh{sju9fi38A zZ$dDJTZEV$8e>_xeG{T@LbRuL$7J3&A=)mm&3?!AO$eLUYqP?$K7#uuMBjwiaG}Rx zh)g)%){SHcjhnu@)-sLdyt`~=c$5*%yrm7wSRP{Q4n@bU;>1=VM-Oos2yStYAomv6LRL^L$M2UO5YKLcmeg$=axBd!+c z?Bs_?hBmDZn%r&c?{(YGxCr-D)ttgFk*EM-(sWI{uczaGe}&YF3i?0C|Gt6#AO^ZA zy%$~m$IDL(-G`GF6d4O8h-rhttKJ~Et@#k5!Ln#aL&dyEg27tRQ?c%1iCQpeMAZ`- z*>t#+g;}JOK3gbX$d*o(b92+_T$v=67v}OA5On~e4*NWq$2bS-=(rvRD@m&Pa!2&_lCT?P_n~%?j zHo5mEHX!Y4-p(H*B)qOap@XhM6kso+I{P6{4-gW&CBj>`UeTY>ACXYy{)CPh$D+sv zOC*o`g#MhjJN92roq2|H4f&_=pJmhF?+iyA+A%+#Z|p4EqAPmDQ- zAL=U%@(S!wGmXPX%466YqoNl9#}B79UnuL~gi*`Xhs-uqg~u9f-K>>Yz^AO%Idv%r z-WsXG@k?Yrv38L{8+-0;b-R8wUBz4-Aktl~N&~AL3^^J@jApexXn2eau=TRFzofr< zMz#xgx5trqKwnhw|eHD87v3QgD-y^b94ggFX`()-I?y?zx9r@eKNNQJs$c` zChh}&gv$tdB8L_2MV>Fg>G!A2B;RM>e=Bm%pC*ZL4#5eBk!sx?MF-L+LbM26cHCQc zkPPGh6o)(Ew?Lz9q^OThK}Wp{JMdBx4;Xa}hG;gq3{n1`^#(#}Sd75K^YjfaCo!`8bf zi`t?+rP!=!%{oc3z!aNW1Iq$Q8mubtZx49b(Uf_KgGpR~UF&Vy$V!gq#(a|vwq2C9 z=6q6z?D-~#55bnFe!ANYfqTIy&V><;As~?g`lU;6;Xp^B{vq!m25vC~ABjCSZ~ Md+}Vckjc;d4-FoyU;qFB literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-last/plan/skip-to-last.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-last/plan/skip-to-last.json new file mode 100644 index 0000000000000..0c57df1e65783 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-last/plan/skip-to-last.json @@ -0,0 +1,407 @@ +{ + "flinkVersion" : "1.19", + "nodes" : [ { + "id" : 17, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`MyTable`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "vehicle_id", + "dataType" : "BIGINT" + }, { + "name" : "engine_temperature", + "dataType" : "INT" + }, { + "name" : "proctime", + "kind" : "COMPUTED", + "expression" : { + "rexNode" : { + "kind" : "CALL", + "internalName" : "$PROCTIME$1", + "operands" : [ ], + "type" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + }, + "serializableString" : "PROCTIME()" + } + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "outputType" : "ROW<`vehicle_id` BIGINT, `engine_temperature` INT>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[vehicle_id, engine_temperature])", + "inputProperties" : [ ] + }, { + "id" : 18, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "BIGINT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "CALL", + "internalName" : "$PROCTIME$1", + "operands" : [ ], + "type" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "vehicle_id", + "fieldType" : "BIGINT" + }, { + "name" : "engine_temperature", + "fieldType" : "INT" + }, { + "name" : "proctime", + "fieldType" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + } ] + }, + "description" : "Calc(select=[vehicle_id, engine_temperature, PROCTIME() AS proctime])" + }, { + "id" : 19, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "vehicle_id", + "fieldType" : "BIGINT" + }, { + "name" : "engine_temperature", + "fieldType" : "INT" + }, { + "name" : "proctime", + "fieldType" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + } ] + }, + "description" : "Exchange(distribution=[hash[vehicle_id]])" + }, { + "id" : 20, + "type" : "stream-exec-match_1", + "matchSpec" : { + "pattern" : { + "kind" : "CALL", + "syntax" : "BINARY", + "sqlKind" : "PATTERN_CONCAT", + "operands" : [ { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$PATTERN_QUANTIFIER$1", + "operands" : [ { + "kind" : "LITERAL", + "value" : "A", + "type" : "CHAR(1) NOT NULL" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + }, { + "kind" : "LITERAL", + "value" : -1, + "type" : "INT NOT NULL" + }, { + "kind" : "LITERAL", + "value" : false, + "type" : "BOOLEAN NOT NULL" + } ], + "type" : "NULL" + }, { + "kind" : "LITERAL", + "value" : "B", + "type" : "CHAR(1) NOT NULL" + } ], + "type" : "NULL" + }, + "patternDefinitions" : { + "A" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$OR$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "POSTFIX", + "internalName" : "$IS NULL$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "BOOLEAN NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$>$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$PREV$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "BOOLEAN" + } ], + "type" : "BOOLEAN" + }, + "B" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$PREV$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "B", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "BOOLEAN" + } + }, + "measures" : { + "Final_Temp" : { + "kind" : "CALL", + "syntax" : "PREFIX", + "internalName" : "$FINAL$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "INT" + }, + "Initial_Temp" : { + "kind" : "CALL", + "syntax" : "PREFIX", + "internalName" : "$FINAL$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$FIRST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "INT" + } + }, + "after" : { + "kind" : "CALL", + "syntax" : "PREFIX", + "internalName" : "$SKIP TO LAST$1", + "operands" : [ { + "kind" : "LITERAL", + "value" : "B", + "type" : "CHAR(1) NOT NULL" + } ], + "type" : "NULL" + }, + "subsets" : { }, + "allRows" : false, + "partition" : { + "fields" : [ 0 ] + }, + "orderBy" : { + "fields" : [ { + "index" : 2, + "isAscending" : true, + "nullIsLast" : false + } ] + }, + "interval" : null + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`vehicle_id` BIGINT, `Initial_Temp` INT, `Final_Temp` INT>", + "description" : "Match(partitionBy=[vehicle_id], orderBy=[proctime ASC], measures=[FINAL(FIRST(A.engine_temperature, 0)) AS Initial_Temp, FINAL(LAST(A.engine_temperature, 0)) AS Final_Temp], rowsPerMatch=[ONE ROW PER MATCH], after=[SKIP TO LAST _UTF-16LE'B'], pattern=[(PATTERN_QUANTIFIER(_UTF-16LE'A', 1, -1, false), _UTF-16LE'B')], define=[{A=OR(IS NULL(LAST(A.$1, 1)), >(PREV(A.$1, 0), LAST(A.$1, 1))), B=<(PREV(B.$1, 0), LAST(A.$1, 0))}])" + }, { + "id" : 21, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`MySink`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "vehicle_id", + "dataType" : "BIGINT" + }, { + "name" : "Initial_Temp", + "dataType" : "INT" + }, { + "name" : "Final_Temp", + "dataType" : "INT" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`vehicle_id` BIGINT, `Initial_Temp` INT, `Final_Temp` INT>", + "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[vehicle_id, Initial_Temp, Final_Temp])" + } ], + "edges" : [ { + "source" : 17, + "target" : 18, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 18, + "target" : 19, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 19, + "target" : 20, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 20, + "target" : 21, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-last/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-last/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..d592b2403b8fe907ee1ee117d9a1c1ceac9a8cc0 GIT binary patch literal 18079 zcmeGkTWlLy_1aF`wA*geHY%IKR-u~~R$I?Fj_uejNH~eJ_11Ml?6%t!;oi(#C)12) z+?jEjR#3Ho*hLEzB+v?^g-YAcB9IUdi66dpKi~tw4Z(%b=14 zSF@&CCbkRViIAsWb*{f~u5Xd;PM5V!hny}$AG4bd9S@LimKBv95m%@omUUHThACD? z#Zn@=97~l(BbP746U*i3g|Vu(xHulcWATDm6wlW*#TrYFMMh)sWF#>fO^l~fsYqNj z=?bgrie`nQWm!rtglRmLOeUi-Ns`A!7NfDUB*(+4cxp5plNk%s2up>TT$aj`B*n`t zLCBB}^le^m@W`V88J?NV%%zJnbEQHtUCflGvW4PYcH(3)JCiS+n3>9ufx>JqTWnNh zc)x*D>D(-kTrSz=Bxt)r_3Ik+x#xAPm<|1tQz z;1?@F63mgElCG`5)GN9+sZtXF@3H}_R9h5z(a{`vLe&=)iKl#S(|EXfBwS$ZX>=GM0DNSaJv0*F^SGJQZeAc)NC0Lm(h(pJCmXXyl4PVjCq1`#NNi-GsoXo2( z&71}v9^#EqeuYV0Rbk3tD$S9TT1A(Yvcd-ia{Zs32SD|+SoHh)437@akYrlQ^J8#n0LL`tSeI`8mz%G#3 zi}-klU_9}T_~yvO2wbP}iFSAc!Vv<&mzgOUO4SzX-5!}PScal4qD*V>P#_0B4EH$W zc;B~L1-@=uVM~gnvXUYLIbuiTOel+4&GJhrSw)yT0$1=dV3Exe+2^zQBA3~HfM2lmAAj~>+8?F1rqZD*|b8{8UxWDXoz;cR#{-i zmv4Rh=_kJR=T{I?l~c8!(>cuV4+Gp?jV{0hw#ba#f4y<@_O+Li06Ob}Mk++!#~G*H zAB$t&;#EQhPrnZXw-sTr%``WwE|54Jy+?C3mG~4FNVTK1?nRUyx+N}fKbAP>k#YFf znYL*0fIObea~Fah_i$a_sCIR^=UF8}WVUIQpjAbX^h!n7Ja4xP)MNs#eV#?a|JS!d z zkc~t8e@CH=4Vcpu-O|FuC5z1(N(J1pD~x-p8;W>jy~-xp=JRErV!xyZPXG99Y~l+A z?ZB5lhfZ!T16b?MC~vuipPkU9bKkr4IPjQoVEFaNWT1B)=KWs583Qd8OQphN<1r~5 zOD5ytg;*I<2~yH{IgyYc?I1bBfZfL(3CE4g(x?|#aJk^0Nd8)UN z?OK`gcMeP{a3nfF299UWWTu=PlH!@!4B5mJM7Yqj&J`JJ($W&E3IJWmPp3JCh7R_r zSvwIV81M`?L|$wQZHirFm+TPZi08fmymhG$WpiGOORKp=4JI$t%4KEW!W?>TAi#Yy9E?-NbWkDbEk z_tnGWL)Cd*V{#Gl8e#FOR1vqU4_9b~If+hB}kPQ|vG+XtPE7XXj=n5wu4O ze~^Ox=h|?=)vhSKcHV8B6KhhK+LU{=8M-V&GC{kM+1pEdyN9&s`@+06F{_?Je@N2RdkUQizQEWzVX{YEp?}e#aarhj%5lPniknZX~92Y-k|P!3|?q@cWLR5C~!ZT?Yu#ES@SE*sVIK| zZFZ>yh_!JX84#`!FH$UMAYG@*^GaObl;$xTT5W=BIpPnBv#`6K)dyow4EdKI?kfyS zI_y$(jKhb@W7r*|q8mx)4`w!>D_P)#F-qKrELII&VkT_gT`XM$pR!S*#&Q_EHR=MD zmQkNLJ9yD6+;S6Zv+LKi3`Q#ekte|DRc26Fa!dh@W}~rav{)Hn?{R&PQ7?Li`+_^$ z=ZYRs*g|+;?1lRmJ!^h(a2+)F9{xDwdkDC{q$mHLE$Dv#ZR=d?$8($5qapvOxcB}E zP6K#{hd0OS9Q+Rk(@s&~s~@^P_T69_3*pxfKRE1C_v&lfmp+22Y#ij)J8KTCus>+<7$p7kWI$F-i5l`G)g%Uc z)a7}y!zr4k@bwJ%ztjD%Xo|VyEh*F%%vuFYTrh>2S!J4xMVreyPj7?7kJOk}G}0w2AuiFH8J^=|Hlb&Isz%svR8U9cLHkmjENu*k7yRINts zrF#T6ay3B$UO=d&MZyNqE!622TF~tXK}Vl6*gBcc&*ZZZ<$-T!I5Rw4hY17#JJx(% dq=hKfJK!v|9F3PIjQ?!4$d", + "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[vehicle_id, engine_temperature])", + "inputProperties" : [ ] + }, { + "id" : 23, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "BIGINT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "CALL", + "internalName" : "$PROCTIME$1", + "operands" : [ ], + "type" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "vehicle_id", + "fieldType" : "BIGINT" + }, { + "name" : "engine_temperature", + "fieldType" : "INT" + }, { + "name" : "proctime", + "fieldType" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + } ] + }, + "description" : "Calc(select=[vehicle_id, engine_temperature, PROCTIME() AS proctime])" + }, { + "id" : 24, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "vehicle_id", + "fieldType" : "BIGINT" + }, { + "name" : "engine_temperature", + "fieldType" : "INT" + }, { + "name" : "proctime", + "fieldType" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + } ] + }, + "description" : "Exchange(distribution=[hash[vehicle_id]])" + }, { + "id" : 25, + "type" : "stream-exec-match_1", + "matchSpec" : { + "pattern" : { + "kind" : "CALL", + "syntax" : "BINARY", + "sqlKind" : "PATTERN_CONCAT", + "operands" : [ { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$PATTERN_QUANTIFIER$1", + "operands" : [ { + "kind" : "LITERAL", + "value" : "A", + "type" : "CHAR(1) NOT NULL" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + }, { + "kind" : "LITERAL", + "value" : -1, + "type" : "INT NOT NULL" + }, { + "kind" : "LITERAL", + "value" : false, + "type" : "BOOLEAN NOT NULL" + } ], + "type" : "NULL" + }, { + "kind" : "LITERAL", + "value" : "B", + "type" : "CHAR(1) NOT NULL" + } ], + "type" : "NULL" + }, + "patternDefinitions" : { + "A" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$OR$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "POSTFIX", + "internalName" : "$IS NULL$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "BOOLEAN NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$>$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$PREV$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "BOOLEAN" + } ], + "type" : "BOOLEAN" + }, + "B" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$PREV$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "B", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "BOOLEAN" + } + }, + "measures" : { + "Final_Temp" : { + "kind" : "CALL", + "syntax" : "PREFIX", + "internalName" : "$FINAL$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "INT" + }, + "Initial_Temp" : { + "kind" : "CALL", + "syntax" : "PREFIX", + "internalName" : "$FINAL$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$FIRST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "A", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "INT" + } + }, + "after" : { + "kind" : "LITERAL", + "symbol" : "MATCH_RECOGNIZE_AFTER_OPTION", + "value" : "SKIP_TO_NEXT_ROW", + "type" : { + "type" : "SYMBOL", + "nullable" : false + } + }, + "subsets" : { }, + "allRows" : false, + "partition" : { + "fields" : [ 0 ] + }, + "orderBy" : { + "fields" : [ { + "index" : 2, + "isAscending" : true, + "nullIsLast" : false + } ] + }, + "interval" : null + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`vehicle_id` BIGINT, `Initial_Temp` INT, `Final_Temp` INT>", + "description" : "Match(partitionBy=[vehicle_id], orderBy=[proctime ASC], measures=[FINAL(FIRST(A.engine_temperature, 0)) AS Initial_Temp, FINAL(LAST(A.engine_temperature, 0)) AS Final_Temp], rowsPerMatch=[ONE ROW PER MATCH], after=[SKIP TO NEXT ROW], pattern=[(PATTERN_QUANTIFIER(_UTF-16LE'A', 1, -1, false), _UTF-16LE'B')], define=[{A=OR(IS NULL(LAST(A.$1, 1)), >(PREV(A.$1, 0), LAST(A.$1, 1))), B=<(PREV(B.$1, 0), LAST(A.$1, 0))}])" + }, { + "id" : 26, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`MySink`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "vehicle_id", + "dataType" : "BIGINT" + }, { + "name" : "Initial_Temp", + "dataType" : "INT" + }, { + "name" : "Final_Temp", + "dataType" : "INT" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`vehicle_id` BIGINT, `Initial_Temp` INT, `Final_Temp` INT>", + "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[vehicle_id, Initial_Temp, Final_Temp])" + } ], + "edges" : [ { + "source" : 22, + "target" : 23, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 23, + "target" : 24, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 24, + "target" : 25, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 25, + "target" : 26, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-next-row/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-next-row/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..901bfa05dfbdfe1d86b82c03fa9f4926d2c6b9a3 GIT binary patch literal 18431 zcmeGkZHyaN@!7s4cWKgGl2gcC+UOvc(5kW5UuXLoe&DKtKhRDo_xWP(&rfPl^PUDhj`-L=Zwks)9fQsTBf+d2e^^ z-P`r`@^P1>SxffYeIGOP-kY77H#6(}nR*x@)B|7o2^aCb~%8CaXeiWlIdhTlfi^fjmAhShR2dA9`mVGh98BlTWp{xL%*^2o5I+u7#_K(L|Wkgp^Q?j*+pk43PC6d^dI_n@$ZDWqKwzn=MbzR!Ze; zIais?m&&vG@e}3zbg^=LdNPOlOEZOhxzmtibNSAh}x2Mu|aIobyhh)iFIs~qpq4|!ZUE~?`VUjBCZP#D!H|Y zq|`+oOObK0o|pA9)S1f~2qLtkqUfqd7Yr!Q;lVkyS=5RxNuqNqpk9h`R#9{vr3wI+ zGjs~abPC5N&M~4mlPJyxmX15L)C^owQSx?4qGQcjp&e{ikDtYhI3i)W9?8kAhP|<= zqmASF!}($v4V?NPv=}ON#)Y)I5-cck-L_HATgNVKLR;h-W;RC=H`+^&`j6#K=O#_< zT|PaNLmQZC26BSPTT3Tg1oEUA0qBw9RF+}rkmR6Y>))su6b#s=8$!=@g*HjflMBUG zqYA3rcHIH)UF$%3vMY3L9;-yCwrVw^Mv96+^1_Hw*bpqQaKJXEW1md`on9Tj+#->6 zKxmdWjIJ67$~6W*P|P0Bm1eRNF3Zflj+Wris2LkMsfYsQD-Db{Vw+BvddH-88e1{W zwjxaE(rD&$67b;9(DCfd3f%c=*A8@le2(L0hgJto?!67J_1;dp2)7<92dfqoet}9k zDgjqbd|*xGf5?~rpxEe!+@bt8*YBW<0}LI_R8?Dl`pT1PR#2XsKy&5}=Vr~-`_!sd z4;~*Z%_%Yw%3$_@@M<*a##!9fWWyX1zdK(sMR)*01(d`3F zoqwi9T0~oo@p<2r``WNf*0sIeS*9RrHlDHVUkJIa5nrKGC*cEopz{^_omfM@LceXM z_7%D-FyM58j5u!{V%1mZw|`iF%6V9qbyXx<_`Imkhb8l58YV)WJdh`Nw->|ni2bm5 zgVxq#aMWeHcrI3n`OysL4PCg7*=s#(C*#Zj^)h1g+sBau<{Gghr4=2dD>Pa3bfG4Q zK4R>#ww~4dbDkJlZyUO8EA|7d}R(~E5=TApnrX8%|^xF2h@|`_zFO>f{{>vX1 zuRQVXsdu2sHH76bf?5Oxy%1K3u3}z?Qfo;@xYkUD+x<-g?(Sch)cABdRz+y&D$oacINcYIF7~}nrgT5E-9ihwC9t++EOiW`5qFh`jb!EhJmWgW z*(LXsNF+g{Q48u5{P*kEp7_ZN->(Ewpnx8)(Kf3|(@}qcrps_N&6A`4a2i6}zan&N z@S9ivHSoRQ=Nk~fE1+$>A}>M`Eky7pB&-48?FL|)hr_X|Lk0Ahnfyb^(ouYM5k6a@ z5F`mLs%Xef9>CKgkThj*B?<*m4`5?+9lt=cBy_aRHc`kzIP^T)X132L8YHtSaG=@w zyp8DT>RH0;K(!f=9QGiW;^hD@3qHc?Hce0#(m@;20zAb;ID)l#ihijH>07G{KLdPc zz!dgzBcolTF}c7a8uSd!PM?A`JiyYuDE~$#6iI?DgRRs?PRI>K5Nje!PXoUG@3#H1 z>o>3Tv!|d(f*Nb>4t_X*4&cFYbosZReSa?l6X12U%eKRvZpQZR>_BFT()9USdf}ZX zzx8YU-4F`oQI7^$(@Ra@+R%BLs1<_wjJmm-r48zg$IS-5h$nV6Kyg+B=&`X z@dcvss@OD`dWVgtOW-_|>y)Qucqr%!?15_vnqK!TH6h5qd6CSEyhJLZaF-c|6z0~z zC0g{iGw&GB4gsDmc}S&)#PLd*G*B1l>T3+*}*puY!}=^H;-2jd`iZ(1_ zsYT#{?PsjPk`#@6yDwh*#*>eI{m(B^NKHo7UPk9IyFLhTw|9oXB9J;!cl_m*SKoi( z#SDPXTc9Zw92;ZK(-=>yllE#zvA3r{u4S^SpR7c)aOvafCo8RErmrOk{K)W~u%5Mg zHYpK8507_sEj+f%4^93Nf|?}$5keTh;2$CG^~)gZl4ju_A^al*dl!nD$DNakrTdt@ zOphx55#ka$=pP{-;`iNrBlvENV1M=QX7&6dgyU}yhD~2xJ2GTB@6K8c2)X*y%(j*K z%&(%4HQp*=hzpl!!dWm&uRd0rReJTYUh%O?uRf2^+&im@OnH)Vxq7V}2gtPN7ues4 zL?+soGb&A|(INZO1QiUT-Z|7Sh#HSoVFJrM0SCA+hqjtkQyBi_1n#$4_o6In^Y)rj ztE#mcw8jEcYH3X(3$$uuO@aSTfs4MD%u5(Dcm#-XbZDbO!843wPJ`=o<*Z;99RuER zHiG~H2XkySi%_!uU4;&R<;CCkzWUt28_d_e8E9-^_Hw(9M%^JAUQr20mIx0K0!8N9 zfTr5REN{4uJU!(D5Yy2+n~-Jk1sxr6qw<=yo$e7hCAuKNTEMGW9xr$U-H|rk5f8d8 zexj)f4zIJvTkm}n+2V9D4{wD*V9wm(;WkVl0G4Cf|G;>#VtWAH1<%>oX`=RGAG|YI LI#HU*6(|1>>=VF8 literal 0 HcmV?d00001 From 76d0fd68b9dd0891f54737580e72b46300169350 Mon Sep 17 00:00:00 2001 From: Jim Hughes Date: Mon, 27 Nov 2023 17:36:16 -0500 Subject: [PATCH 2/4] [FLINK-33667] Deleting MatchRecognizeJsonPlanTest.java and MatchRecognizeJsonPlanITCase.java --- .../stream/MatchRecognizeJsonPlanTest.java | 145 ------ .../MatchRecognizeJsonPlanITCase.java | 109 ---- .../testMatch.out | 378 -------------- .../testSkipPastLastRow.out | 468 ----------------- .../testSkipToFirst.out | 470 ------------------ .../testSkipToLast.out | 470 ------------------ .../testSkipToNextRow.out | 468 ----------------- 7 files changed, 2508 deletions(-) delete mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest.java delete mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/MatchRecognizeJsonPlanITCase.java delete mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testMatch.out delete mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipPastLastRow.out delete mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipToFirst.out delete mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipToLast.out delete mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipToNextRow.out diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest.java deleted file mode 100644 index c31f300ca4e5d..0000000000000 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest.java +++ /dev/null @@ -1,145 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.table.planner.plan.nodes.exec.stream; - -import org.apache.flink.table.api.TableConfig; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.planner.utils.StreamTableTestUtil; -import org.apache.flink.table.planner.utils.TableTestBase; - -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -/** Test json serialization for match recognize. */ -class MatchRecognizeJsonPlanTest extends TableTestBase { - private StreamTableTestUtil util; - private TableEnvironment tEnv; - - @BeforeEach - void setup() { - util = streamTestUtil(TableConfig.getDefault()); - tEnv = util.getTableEnv(); - } - - @Test - void testMatch() { - String srcTableDdl = - "CREATE TABLE MyTable (\n" - + " id bigint,\n" - + " name varchar,\n" - + " proctime as PROCTIME()\n" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'bounded' = 'false')"; - tEnv.executeSql(srcTableDdl); - String sinkTableDdl = - "CREATE TABLE MySink (\n" - + " a bigint,\n" - + " b bigint,\n" - + " c bigint\n" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'sink-insert-only' = 'false',\n" - + " 'table-sink-class' = 'DEFAULT')"; - tEnv.executeSql(sinkTableDdl); - - String sql = - "insert into MySink" - + " SELECT T.aid, T.bid, T.cid\n" - + " FROM MyTable MATCH_RECOGNIZE (\n" - + " ORDER BY proctime\n" - + " MEASURES\n" - + " `A\"`.id AS aid,\n" - + " \u006C.id AS bid,\n" - + " C.id AS cid\n" - + " PATTERN (`A\"` \u006C C)\n" - + " DEFINE\n" - + " `A\"` AS name = 'a',\n" - + " \u006C AS name = 'b',\n" - + " C AS name = 'c'\n" - + " ) AS T"; - util.verifyJsonPlan(sql); - } - - @Test - void testSkipToLast() { - doTestAfterMatch("AFTER MATCH SKIP TO LAST B"); - } - - @Test - void testSkipToFirst() { - doTestAfterMatch("AFTER MATCH SKIP TO FIRST B"); - } - - @Test - void testSkipPastLastRow() { - doTestAfterMatch("AFTER MATCH SKIP PAST LAST ROW"); - } - - @Test - void testSkipToNextRow() { - doTestAfterMatch("AFTER MATCH SKIP TO NEXT ROW"); - } - - private void doTestAfterMatch(Object afterClause) { - String srcTableDdl = - "CREATE TABLE MyTable (\n" - + " vehicle_id bigint,\n" - + " engine_temperature int,\n" - + " rowtime timestamp_ltz(3)," - + " WATERMARK FOR rowtime AS SOURCE_WATERMARK()\n" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'bounded' = 'false')"; - tEnv.executeSql(srcTableDdl); - String sinkTableDdl = - "CREATE TABLE MySink (\n" - + " vehicle_id bigint,\n" - + " startTime timestamp_ltz(3),\n" - + " endTime timestamp_ltz(3),\n" - + " Initial_Temp int,\n" - + " Final_Temp int\n" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'sink-insert-only' = 'false',\n" - + " 'table-sink-class' = 'DEFAULT')"; - tEnv.executeSql(sinkTableDdl); - - String sql = - "insert into MySink" - + " SELECT * FROM\n" - + " MyTable\n" - + " MATCH_RECOGNIZE(\n" - + " PARTITION BY vehicle_id\n" - + " ORDER BY `rowtime`\n" - + " MEASURES \n" - + " FIRST(A.`rowtime`) as startTime,\n" - + " LAST(A.`rowtime`) as endTime,\n" - + " FIRST(A.engine_temperature) as Initial_Temp,\n" - + " LAST(A.engine_temperature) as Final_Temp\n" - + " ONE ROW PER MATCH\n" - + " %s\n" - + " PATTERN (A+ B)\n" - + " DEFINE\n" - + " A as LAST(A.engine_temperature,1) is NULL OR A.engine_temperature > LAST(A.engine_temperature,1),\n" - + " B as B.engine_temperature < LAST(A.engine_temperature)\n" - + " )MR;"; - util.verifyJsonPlan(String.format(sql, afterClause)); - } -} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/MatchRecognizeJsonPlanITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/MatchRecognizeJsonPlanITCase.java deleted file mode 100644 index a5cb80e3709d0..0000000000000 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/MatchRecognizeJsonPlanITCase.java +++ /dev/null @@ -1,109 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.table.planner.runtime.stream.jsonplan; - -import org.apache.flink.table.planner.factories.TestValuesTableFactory; -import org.apache.flink.table.planner.utils.JsonPlanTestBase; -import org.apache.flink.types.Row; - -import org.junit.jupiter.api.Test; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - -/** Test json deserialization for match recognize. */ -class MatchRecognizeJsonPlanITCase extends JsonPlanTestBase { - @Test - void testSimpleMatch() throws Exception { - List data = - Arrays.asList( - Row.of(1L, "a"), - Row.of(2L, "z"), - Row.of(3L, "b"), - Row.of(4L, "c"), - Row.of(5L, "d"), - Row.of(6L, "a"), - Row.of(7L, "b"), - Row.of(8L, "c"), - Row.of(9L, "h")); - - createTestValuesSourceTable( - "MyTable", data, "id bigint", "name varchar", "proctime as PROCTIME()"); - createTestValuesSinkTable("MySink", "a bigint", "b bigint", "c bigint"); - - String sql = - "insert into MySink" - + " SELECT T.aid, T.bid, T.cid\n" - + " FROM MyTable MATCH_RECOGNIZE (\n" - + " ORDER BY proctime\n" - + " MEASURES\n" - + " `A\"`.id AS aid,\n" - + " \u006C.id AS bid,\n" - + " C.id AS cid\n" - + " PATTERN (`A\"` \u006C C)\n" - + " DEFINE\n" - + " `A\"` AS name = 'a',\n" - + " \u006C AS name = 'b',\n" - + " C AS name = 'c'\n" - + " ) AS T"; - compileSqlAndExecutePlan(sql).await(); - - List expected = Collections.singletonList("+I[6, 7, 8]"); - assertResult(expected, TestValuesTableFactory.getResultsAsStrings("MySink")); - } - - @Test - void testComplexMatch() throws Exception { - List data = - Arrays.asList( - Row.of("ACME", 1L, 19, 1), - Row.of("ACME", 2L, 17, 2), - Row.of("ACME", 3L, 13, 3), - Row.of("ACME", 4L, 20, 4)); - createTestValuesSourceTable( - "MyTable", - data, - "symbol string", - "tstamp bigint", - "price int", - "tax int", - "proctime as PROCTIME()"); - createTestValuesSinkTable("MySink", "a bigint", "b bigint", "c bigint"); - - String sql = - "insert into MySink SELECT * FROM MyTable MATCH_RECOGNIZE (\n" - + " ORDER BY proctime\n" - + " MEASURES\n" - + " FIRST(DOWN.price) as first,\n" - + " LAST(DOWN.price) as last,\n" - + " FIRST(DOWN.price, 5) as nullPrice\n" - + " ONE ROW PER MATCH\n" - + " AFTER MATCH SKIP PAST LAST ROW\n" - + " PATTERN (DOWN{2,} UP)\n" - + " DEFINE\n" - + " DOWN AS price < LAST(DOWN.price, 1) OR LAST(DOWN.price, 1) IS NULL,\n" - + " UP AS price > LAST(DOWN.price)\n" - + ") AS T"; - compileSqlAndExecutePlan(sql).await(); - - List expected = Collections.singletonList("+I[19, 13, null]"); - assertResult(expected, TestValuesTableFactory.getResultsAsStrings("MySink")); - } -} diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testMatch.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testMatch.out deleted file mode 100644 index 562fc04e8801d..0000000000000 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testMatch.out +++ /dev/null @@ -1,378 +0,0 @@ -{ - "flinkVersion" : "", - "nodes" : [ { - "id" : 1, - "type" : "stream-exec-table-source-scan_1", - "scanTableSource" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`MyTable`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "id", - "dataType" : "BIGINT" - }, { - "name" : "name", - "dataType" : "VARCHAR(2147483647)" - }, { - "name" : "proctime", - "kind" : "COMPUTED", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "internalName" : "$PROCTIME$1", - "operands" : [ ], - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, - "serializableString" : "PROCTIME()" - } - } ], - "watermarkSpecs" : [ ] - }, - "partitionKeys" : [ ], - "options" : { - "bounded" : "false", - "connector" : "values" - } - } - } - }, - "outputType" : "ROW<`id` BIGINT, `name` VARCHAR(2147483647)>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[id, name])", - "inputProperties" : [ ] - }, { - "id" : 2, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "BIGINT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "VARCHAR(2147483647)" - }, { - "kind" : "CALL", - "internalName" : "$PROCTIME$1", - "operands" : [ ], - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "id", - "fieldType" : "BIGINT" - }, { - "name" : "name", - "fieldType" : "VARCHAR(2147483647)" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ] - }, - "description" : "Calc(select=[id, name, PROCTIME() AS proctime])" - }, { - "id" : 3, - "type" : "stream-exec-exchange_1", - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "SINGLETON" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "id", - "fieldType" : "BIGINT" - }, { - "name" : "name", - "fieldType" : "VARCHAR(2147483647)" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ] - }, - "description" : "Exchange(distribution=[single])" - }, { - "id" : 4, - "type" : "stream-exec-match_1", - "matchSpec" : { - "pattern" : { - "kind" : "CALL", - "syntax" : "BINARY", - "sqlKind" : "PATTERN_CONCAT", - "operands" : [ { - "kind" : "CALL", - "syntax" : "BINARY", - "sqlKind" : "PATTERN_CONCAT", - "operands" : [ { - "kind" : "LITERAL", - "value" : "A\"", - "type" : "CHAR(2) NOT NULL" - }, { - "kind" : "LITERAL", - "value" : "l", - "type" : "CHAR(1) NOT NULL" - } ], - "type" : "NULL" - }, { - "kind" : "LITERAL", - "value" : "C", - "type" : "CHAR(1) NOT NULL" - } ], - "type" : "NULL" - }, - "patternDefinitions" : { - "A\"" : { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$=$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$LAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "*", - "inputIndex" : 1, - "type" : "VARCHAR(2147483647)" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "VARCHAR(2147483647)" - }, { - "kind" : "LITERAL", - "value" : "a", - "type" : "VARCHAR(2147483647) NOT NULL" - } ], - "type" : "BOOLEAN" - }, - "C" : { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$=$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$LAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "*", - "inputIndex" : 1, - "type" : "VARCHAR(2147483647)" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "VARCHAR(2147483647)" - }, { - "kind" : "LITERAL", - "value" : "c", - "type" : "VARCHAR(2147483647) NOT NULL" - } ], - "type" : "BOOLEAN" - }, - "l" : { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$=$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$LAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "*", - "inputIndex" : 1, - "type" : "VARCHAR(2147483647)" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "VARCHAR(2147483647)" - }, { - "kind" : "LITERAL", - "value" : "b", - "type" : "VARCHAR(2147483647) NOT NULL" - } ], - "type" : "BOOLEAN" - } - }, - "measures" : { - "aid" : { - "kind" : "CALL", - "syntax" : "PREFIX", - "internalName" : "$FINAL$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A\"", - "inputIndex" : 0, - "type" : "BIGINT" - } ], - "type" : "BIGINT" - }, - "bid" : { - "kind" : "CALL", - "syntax" : "PREFIX", - "internalName" : "$FINAL$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "l", - "inputIndex" : 0, - "type" : "BIGINT" - } ], - "type" : "BIGINT" - }, - "cid" : { - "kind" : "CALL", - "syntax" : "PREFIX", - "internalName" : "$FINAL$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "C", - "inputIndex" : 0, - "type" : "BIGINT" - } ], - "type" : "BIGINT" - } - }, - "after" : { - "kind" : "LITERAL", - "symbol" : "MATCH_RECOGNIZE_AFTER_OPTION", - "value" : "SKIP_TO_NEXT_ROW", - "type" : { - "type" : "SYMBOL", - "nullable" : false - } - }, - "subsets" : { }, - "allRows" : false, - "partition" : { - "fields" : [ ] - }, - "orderBy" : { - "fields" : [ { - "index" : 2, - "isAscending" : true, - "nullIsLast" : false - } ] - }, - "interval" : null - }, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`aid` BIGINT, `bid` BIGINT, `cid` BIGINT>", - "description" : "Match(orderBy=[proctime ASC], measures=[FINAL(A\".id) AS aid, FINAL(l.id) AS bid, FINAL(C.id) AS cid], rowsPerMatch=[ONE ROW PER MATCH], after=[SKIP TO NEXT ROW], pattern=[((_UTF-16LE'A\"', _UTF-16LE'l'), _UTF-16LE'C')], define=[{A\"==(LAST(*.$1, 0), _UTF-16LE'a'), l==(LAST(*.$1, 0), _UTF-16LE'b'), C==(LAST(*.$1, 0), _UTF-16LE'c')}])" - }, { - "id" : 5, - "type" : "stream-exec-sink_1", - "configuration" : { - "table.exec.sink.keyed-shuffle" : "AUTO", - "table.exec.sink.not-null-enforcer" : "ERROR", - "table.exec.sink.rowtime-inserter" : "ENABLED", - "table.exec.sink.type-length-enforcer" : "IGNORE", - "table.exec.sink.upsert-materialize" : "AUTO" - }, - "dynamicTableSink" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`MySink`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "a", - "dataType" : "BIGINT" - }, { - "name" : "b", - "dataType" : "BIGINT" - }, { - "name" : "c", - "dataType" : "BIGINT" - } ], - "watermarkSpecs" : [ ] - }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values", - "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT" - } - } - } - }, - "inputChangelogMode" : [ "INSERT" ], - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`aid` BIGINT, `bid` BIGINT, `cid` BIGINT>", - "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[aid, bid, cid])" - } ], - "edges" : [ { - "source" : 1, - "target" : 2, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 2, - "target" : 3, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 3, - "target" : 4, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 4, - "target" : 5, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - } ] -} diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipPastLastRow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipPastLastRow.out deleted file mode 100644 index f6327ec0b2630..0000000000000 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipPastLastRow.out +++ /dev/null @@ -1,468 +0,0 @@ -{ - "flinkVersion" : "", - "nodes" : [ { - "id" : 1, - "type" : "stream-exec-table-source-scan_1", - "scanTableSource" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`MyTable`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "vehicle_id", - "dataType" : "BIGINT" - }, { - "name" : "engine_temperature", - "dataType" : "INT" - }, { - "name" : "rowtime", - "dataType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ], - "watermarkSpecs" : [ { - "rowtimeAttribute" : "rowtime", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "internalName" : "$SOURCE_WATERMARK$1", - "operands" : [ ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - }, - "serializableString" : "`SOURCE_WATERMARK`()" - } - } ] - }, - "partitionKeys" : [ ], - "options" : { - "bounded" : "false", - "connector" : "values" - } - } - } - }, - "outputType" : "ROW<`vehicle_id` BIGINT, `engine_temperature` INT, `rowtime` TIMESTAMP(3) WITH LOCAL TIME ZONE>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[vehicle_id, engine_temperature, rowtime])", - "inputProperties" : [ ] - }, { - "id" : 2, - "type" : "stream-exec-watermark-assigner_1", - "watermarkExpr" : { - "kind" : "CALL", - "internalName" : "$SOURCE_WATERMARK$1", - "operands" : [ ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - }, - "rowtimeFieldIndex" : 2, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "vehicle_id", - "fieldType" : "BIGINT" - }, { - "name" : "engine_temperature", - "fieldType" : "INT" - }, { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ] - }, - "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[SOURCE_WATERMARK()])" - }, { - "id" : 3, - "type" : "stream-exec-exchange_1", - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "HASH", - "keys" : [ 0 ] - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "vehicle_id", - "fieldType" : "BIGINT" - }, { - "name" : "engine_temperature", - "fieldType" : "INT" - }, { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ] - }, - "description" : "Exchange(distribution=[hash[vehicle_id]])" - }, { - "id" : 4, - "type" : "stream-exec-match_1", - "matchSpec" : { - "pattern" : { - "kind" : "CALL", - "syntax" : "BINARY", - "sqlKind" : "PATTERN_CONCAT", - "operands" : [ { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$PATTERN_QUANTIFIER$1", - "operands" : [ { - "kind" : "LITERAL", - "value" : "A", - "type" : "CHAR(1) NOT NULL" - }, { - "kind" : "LITERAL", - "value" : 1, - "type" : "INT NOT NULL" - }, { - "kind" : "LITERAL", - "value" : -1, - "type" : "INT NOT NULL" - }, { - "kind" : "LITERAL", - "value" : false, - "type" : "BOOLEAN NOT NULL" - } ], - "type" : "NULL" - }, { - "kind" : "LITERAL", - "value" : "B", - "type" : "CHAR(1) NOT NULL" - } ], - "type" : "NULL" - }, - "patternDefinitions" : { - "A" : { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$OR$1", - "operands" : [ { - "kind" : "CALL", - "syntax" : "POSTFIX", - "internalName" : "$IS NULL$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$LAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 1, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - } ], - "type" : "BOOLEAN NOT NULL" - }, { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$>$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$PREV$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - }, { - "kind" : "CALL", - "internalName" : "$LAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 1, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - } ], - "type" : "BOOLEAN" - } ], - "type" : "BOOLEAN" - }, - "B" : { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$<$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$PREV$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "B", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - }, { - "kind" : "CALL", - "internalName" : "$LAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - } ], - "type" : "BOOLEAN" - } - }, - "measures" : { - "Final_Temp" : { - "kind" : "CALL", - "syntax" : "PREFIX", - "internalName" : "$FINAL$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$LAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - } ], - "type" : "INT" - }, - "Initial_Temp" : { - "kind" : "CALL", - "syntax" : "PREFIX", - "internalName" : "$FINAL$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$FIRST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - } ], - "type" : "INT" - }, - "endTime" : { - "kind" : "CALL", - "syntax" : "PREFIX", - "internalName" : "$FINAL$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$LAST$1", - "operands" : [ { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$CAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 2, - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - }, - "startTime" : { - "kind" : "CALL", - "syntax" : "PREFIX", - "internalName" : "$FINAL$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$FIRST$1", - "operands" : [ { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$CAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 2, - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - } - }, - "after" : { - "kind" : "LITERAL", - "symbol" : "MATCH_RECOGNIZE_AFTER_OPTION", - "value" : "SKIP_PAST_LAST_ROW", - "type" : { - "type" : "SYMBOL", - "nullable" : false - } - }, - "subsets" : { }, - "allRows" : false, - "partition" : { - "fields" : [ 0 ] - }, - "orderBy" : { - "fields" : [ { - "index" : 2, - "isAscending" : true, - "nullIsLast" : false - } ] - }, - "interval" : null - }, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`vehicle_id` BIGINT, `startTime` TIMESTAMP(3) WITH LOCAL TIME ZONE, `endTime` TIMESTAMP(3) WITH LOCAL TIME ZONE, `Initial_Temp` INT, `Final_Temp` INT>", - "description" : "Match(partitionBy=[vehicle_id], orderBy=[rowtime ASC], measures=[FINAL(FIRST(CAST(A.rowtime AS TIMESTAMP_WITH_LOCAL_TIME_ZONE(3)), 0)) AS startTime, FINAL(LAST(CAST(A.rowtime AS TIMESTAMP_WITH_LOCAL_TIME_ZONE(3)), 0)) AS endTime, FINAL(FIRST(A.engine_temperature, 0)) AS Initial_Temp, FINAL(LAST(A.engine_temperature, 0)) AS Final_Temp], rowsPerMatch=[ONE ROW PER MATCH], after=[SKIP PAST LAST ROW], pattern=[(PATTERN_QUANTIFIER(_UTF-16LE'A', 1, -1, false), _UTF-16LE'B')], define=[{A=OR(IS NULL(LAST(A.$1, 1)), >(PREV(A.$1, 0), LAST(A.$1, 1))), B=<(PREV(B.$1, 0), LAST(A.$1, 0))}])" - }, { - "id" : 5, - "type" : "stream-exec-sink_1", - "configuration" : { - "table.exec.sink.keyed-shuffle" : "AUTO", - "table.exec.sink.not-null-enforcer" : "ERROR", - "table.exec.sink.rowtime-inserter" : "ENABLED", - "table.exec.sink.type-length-enforcer" : "IGNORE", - "table.exec.sink.upsert-materialize" : "AUTO" - }, - "dynamicTableSink" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`MySink`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "vehicle_id", - "dataType" : "BIGINT" - }, { - "name" : "startTime", - "dataType" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - }, { - "name" : "endTime", - "dataType" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - }, { - "name" : "Initial_Temp", - "dataType" : "INT" - }, { - "name" : "Final_Temp", - "dataType" : "INT" - } ], - "watermarkSpecs" : [ ] - }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values", - "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT" - } - } - } - }, - "inputChangelogMode" : [ "INSERT" ], - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`vehicle_id` BIGINT, `startTime` TIMESTAMP(3) WITH LOCAL TIME ZONE, `endTime` TIMESTAMP(3) WITH LOCAL TIME ZONE, `Initial_Temp` INT, `Final_Temp` INT>", - "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[vehicle_id, startTime, endTime, Initial_Temp, Final_Temp])" - } ], - "edges" : [ { - "source" : 1, - "target" : 2, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 2, - "target" : 3, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 3, - "target" : 4, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 4, - "target" : 5, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - } ] -} diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipToFirst.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipToFirst.out deleted file mode 100644 index ed49d18cacc16..0000000000000 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipToFirst.out +++ /dev/null @@ -1,470 +0,0 @@ -{ - "flinkVersion" : "", - "nodes" : [ { - "id" : 1, - "type" : "stream-exec-table-source-scan_1", - "scanTableSource" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`MyTable`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "vehicle_id", - "dataType" : "BIGINT" - }, { - "name" : "engine_temperature", - "dataType" : "INT" - }, { - "name" : "rowtime", - "dataType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ], - "watermarkSpecs" : [ { - "rowtimeAttribute" : "rowtime", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "internalName" : "$SOURCE_WATERMARK$1", - "operands" : [ ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - }, - "serializableString" : "`SOURCE_WATERMARK`()" - } - } ] - }, - "partitionKeys" : [ ], - "options" : { - "bounded" : "false", - "connector" : "values" - } - } - } - }, - "outputType" : "ROW<`vehicle_id` BIGINT, `engine_temperature` INT, `rowtime` TIMESTAMP(3) WITH LOCAL TIME ZONE>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[vehicle_id, engine_temperature, rowtime])", - "inputProperties" : [ ] - }, { - "id" : 2, - "type" : "stream-exec-watermark-assigner_1", - "watermarkExpr" : { - "kind" : "CALL", - "internalName" : "$SOURCE_WATERMARK$1", - "operands" : [ ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - }, - "rowtimeFieldIndex" : 2, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "vehicle_id", - "fieldType" : "BIGINT" - }, { - "name" : "engine_temperature", - "fieldType" : "INT" - }, { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ] - }, - "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[SOURCE_WATERMARK()])" - }, { - "id" : 3, - "type" : "stream-exec-exchange_1", - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "HASH", - "keys" : [ 0 ] - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "vehicle_id", - "fieldType" : "BIGINT" - }, { - "name" : "engine_temperature", - "fieldType" : "INT" - }, { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ] - }, - "description" : "Exchange(distribution=[hash[vehicle_id]])" - }, { - "id" : 4, - "type" : "stream-exec-match_1", - "matchSpec" : { - "pattern" : { - "kind" : "CALL", - "syntax" : "BINARY", - "sqlKind" : "PATTERN_CONCAT", - "operands" : [ { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$PATTERN_QUANTIFIER$1", - "operands" : [ { - "kind" : "LITERAL", - "value" : "A", - "type" : "CHAR(1) NOT NULL" - }, { - "kind" : "LITERAL", - "value" : 1, - "type" : "INT NOT NULL" - }, { - "kind" : "LITERAL", - "value" : -1, - "type" : "INT NOT NULL" - }, { - "kind" : "LITERAL", - "value" : false, - "type" : "BOOLEAN NOT NULL" - } ], - "type" : "NULL" - }, { - "kind" : "LITERAL", - "value" : "B", - "type" : "CHAR(1) NOT NULL" - } ], - "type" : "NULL" - }, - "patternDefinitions" : { - "A" : { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$OR$1", - "operands" : [ { - "kind" : "CALL", - "syntax" : "POSTFIX", - "internalName" : "$IS NULL$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$LAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 1, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - } ], - "type" : "BOOLEAN NOT NULL" - }, { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$>$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$PREV$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - }, { - "kind" : "CALL", - "internalName" : "$LAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 1, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - } ], - "type" : "BOOLEAN" - } ], - "type" : "BOOLEAN" - }, - "B" : { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$<$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$PREV$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "B", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - }, { - "kind" : "CALL", - "internalName" : "$LAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - } ], - "type" : "BOOLEAN" - } - }, - "measures" : { - "Final_Temp" : { - "kind" : "CALL", - "syntax" : "PREFIX", - "internalName" : "$FINAL$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$LAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - } ], - "type" : "INT" - }, - "Initial_Temp" : { - "kind" : "CALL", - "syntax" : "PREFIX", - "internalName" : "$FINAL$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$FIRST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - } ], - "type" : "INT" - }, - "endTime" : { - "kind" : "CALL", - "syntax" : "PREFIX", - "internalName" : "$FINAL$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$LAST$1", - "operands" : [ { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$CAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 2, - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - }, - "startTime" : { - "kind" : "CALL", - "syntax" : "PREFIX", - "internalName" : "$FINAL$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$FIRST$1", - "operands" : [ { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$CAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 2, - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - } - }, - "after" : { - "kind" : "CALL", - "syntax" : "PREFIX", - "internalName" : "$SKIP TO FIRST$1", - "operands" : [ { - "kind" : "LITERAL", - "value" : "B", - "type" : "CHAR(1) NOT NULL" - } ], - "type" : "NULL" - }, - "subsets" : { }, - "allRows" : false, - "partition" : { - "fields" : [ 0 ] - }, - "orderBy" : { - "fields" : [ { - "index" : 2, - "isAscending" : true, - "nullIsLast" : false - } ] - }, - "interval" : null - }, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`vehicle_id` BIGINT, `startTime` TIMESTAMP(3) WITH LOCAL TIME ZONE, `endTime` TIMESTAMP(3) WITH LOCAL TIME ZONE, `Initial_Temp` INT, `Final_Temp` INT>", - "description" : "Match(partitionBy=[vehicle_id], orderBy=[rowtime ASC], measures=[FINAL(FIRST(CAST(A.rowtime AS TIMESTAMP_WITH_LOCAL_TIME_ZONE(3)), 0)) AS startTime, FINAL(LAST(CAST(A.rowtime AS TIMESTAMP_WITH_LOCAL_TIME_ZONE(3)), 0)) AS endTime, FINAL(FIRST(A.engine_temperature, 0)) AS Initial_Temp, FINAL(LAST(A.engine_temperature, 0)) AS Final_Temp], rowsPerMatch=[ONE ROW PER MATCH], after=[SKIP TO FIRST _UTF-16LE'B'], pattern=[(PATTERN_QUANTIFIER(_UTF-16LE'A', 1, -1, false), _UTF-16LE'B')], define=[{A=OR(IS NULL(LAST(A.$1, 1)), >(PREV(A.$1, 0), LAST(A.$1, 1))), B=<(PREV(B.$1, 0), LAST(A.$1, 0))}])" - }, { - "id" : 5, - "type" : "stream-exec-sink_1", - "configuration" : { - "table.exec.sink.keyed-shuffle" : "AUTO", - "table.exec.sink.not-null-enforcer" : "ERROR", - "table.exec.sink.rowtime-inserter" : "ENABLED", - "table.exec.sink.type-length-enforcer" : "IGNORE", - "table.exec.sink.upsert-materialize" : "AUTO" - }, - "dynamicTableSink" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`MySink`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "vehicle_id", - "dataType" : "BIGINT" - }, { - "name" : "startTime", - "dataType" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - }, { - "name" : "endTime", - "dataType" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - }, { - "name" : "Initial_Temp", - "dataType" : "INT" - }, { - "name" : "Final_Temp", - "dataType" : "INT" - } ], - "watermarkSpecs" : [ ] - }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values", - "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT" - } - } - } - }, - "inputChangelogMode" : [ "INSERT" ], - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`vehicle_id` BIGINT, `startTime` TIMESTAMP(3) WITH LOCAL TIME ZONE, `endTime` TIMESTAMP(3) WITH LOCAL TIME ZONE, `Initial_Temp` INT, `Final_Temp` INT>", - "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[vehicle_id, startTime, endTime, Initial_Temp, Final_Temp])" - } ], - "edges" : [ { - "source" : 1, - "target" : 2, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 2, - "target" : 3, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 3, - "target" : 4, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 4, - "target" : 5, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - } ] -} diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipToLast.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipToLast.out deleted file mode 100644 index d5c13f04e042b..0000000000000 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipToLast.out +++ /dev/null @@ -1,470 +0,0 @@ -{ - "flinkVersion" : "", - "nodes" : [ { - "id" : 1, - "type" : "stream-exec-table-source-scan_1", - "scanTableSource" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`MyTable`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "vehicle_id", - "dataType" : "BIGINT" - }, { - "name" : "engine_temperature", - "dataType" : "INT" - }, { - "name" : "rowtime", - "dataType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ], - "watermarkSpecs" : [ { - "rowtimeAttribute" : "rowtime", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "internalName" : "$SOURCE_WATERMARK$1", - "operands" : [ ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - }, - "serializableString" : "`SOURCE_WATERMARK`()" - } - } ] - }, - "partitionKeys" : [ ], - "options" : { - "bounded" : "false", - "connector" : "values" - } - } - } - }, - "outputType" : "ROW<`vehicle_id` BIGINT, `engine_temperature` INT, `rowtime` TIMESTAMP(3) WITH LOCAL TIME ZONE>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[vehicle_id, engine_temperature, rowtime])", - "inputProperties" : [ ] - }, { - "id" : 2, - "type" : "stream-exec-watermark-assigner_1", - "watermarkExpr" : { - "kind" : "CALL", - "internalName" : "$SOURCE_WATERMARK$1", - "operands" : [ ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - }, - "rowtimeFieldIndex" : 2, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "vehicle_id", - "fieldType" : "BIGINT" - }, { - "name" : "engine_temperature", - "fieldType" : "INT" - }, { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ] - }, - "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[SOURCE_WATERMARK()])" - }, { - "id" : 3, - "type" : "stream-exec-exchange_1", - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "HASH", - "keys" : [ 0 ] - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "vehicle_id", - "fieldType" : "BIGINT" - }, { - "name" : "engine_temperature", - "fieldType" : "INT" - }, { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ] - }, - "description" : "Exchange(distribution=[hash[vehicle_id]])" - }, { - "id" : 4, - "type" : "stream-exec-match_1", - "matchSpec" : { - "pattern" : { - "kind" : "CALL", - "syntax" : "BINARY", - "sqlKind" : "PATTERN_CONCAT", - "operands" : [ { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$PATTERN_QUANTIFIER$1", - "operands" : [ { - "kind" : "LITERAL", - "value" : "A", - "type" : "CHAR(1) NOT NULL" - }, { - "kind" : "LITERAL", - "value" : 1, - "type" : "INT NOT NULL" - }, { - "kind" : "LITERAL", - "value" : -1, - "type" : "INT NOT NULL" - }, { - "kind" : "LITERAL", - "value" : false, - "type" : "BOOLEAN NOT NULL" - } ], - "type" : "NULL" - }, { - "kind" : "LITERAL", - "value" : "B", - "type" : "CHAR(1) NOT NULL" - } ], - "type" : "NULL" - }, - "patternDefinitions" : { - "A" : { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$OR$1", - "operands" : [ { - "kind" : "CALL", - "syntax" : "POSTFIX", - "internalName" : "$IS NULL$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$LAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 1, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - } ], - "type" : "BOOLEAN NOT NULL" - }, { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$>$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$PREV$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - }, { - "kind" : "CALL", - "internalName" : "$LAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 1, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - } ], - "type" : "BOOLEAN" - } ], - "type" : "BOOLEAN" - }, - "B" : { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$<$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$PREV$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "B", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - }, { - "kind" : "CALL", - "internalName" : "$LAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - } ], - "type" : "BOOLEAN" - } - }, - "measures" : { - "Final_Temp" : { - "kind" : "CALL", - "syntax" : "PREFIX", - "internalName" : "$FINAL$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$LAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - } ], - "type" : "INT" - }, - "Initial_Temp" : { - "kind" : "CALL", - "syntax" : "PREFIX", - "internalName" : "$FINAL$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$FIRST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - } ], - "type" : "INT" - }, - "endTime" : { - "kind" : "CALL", - "syntax" : "PREFIX", - "internalName" : "$FINAL$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$LAST$1", - "operands" : [ { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$CAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 2, - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - }, - "startTime" : { - "kind" : "CALL", - "syntax" : "PREFIX", - "internalName" : "$FINAL$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$FIRST$1", - "operands" : [ { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$CAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 2, - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - } - }, - "after" : { - "kind" : "CALL", - "syntax" : "PREFIX", - "internalName" : "$SKIP TO LAST$1", - "operands" : [ { - "kind" : "LITERAL", - "value" : "B", - "type" : "CHAR(1) NOT NULL" - } ], - "type" : "NULL" - }, - "subsets" : { }, - "allRows" : false, - "partition" : { - "fields" : [ 0 ] - }, - "orderBy" : { - "fields" : [ { - "index" : 2, - "isAscending" : true, - "nullIsLast" : false - } ] - }, - "interval" : null - }, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`vehicle_id` BIGINT, `startTime` TIMESTAMP(3) WITH LOCAL TIME ZONE, `endTime` TIMESTAMP(3) WITH LOCAL TIME ZONE, `Initial_Temp` INT, `Final_Temp` INT>", - "description" : "Match(partitionBy=[vehicle_id], orderBy=[rowtime ASC], measures=[FINAL(FIRST(CAST(A.rowtime AS TIMESTAMP_WITH_LOCAL_TIME_ZONE(3)), 0)) AS startTime, FINAL(LAST(CAST(A.rowtime AS TIMESTAMP_WITH_LOCAL_TIME_ZONE(3)), 0)) AS endTime, FINAL(FIRST(A.engine_temperature, 0)) AS Initial_Temp, FINAL(LAST(A.engine_temperature, 0)) AS Final_Temp], rowsPerMatch=[ONE ROW PER MATCH], after=[SKIP TO LAST _UTF-16LE'B'], pattern=[(PATTERN_QUANTIFIER(_UTF-16LE'A', 1, -1, false), _UTF-16LE'B')], define=[{A=OR(IS NULL(LAST(A.$1, 1)), >(PREV(A.$1, 0), LAST(A.$1, 1))), B=<(PREV(B.$1, 0), LAST(A.$1, 0))}])" - }, { - "id" : 5, - "type" : "stream-exec-sink_1", - "configuration" : { - "table.exec.sink.keyed-shuffle" : "AUTO", - "table.exec.sink.not-null-enforcer" : "ERROR", - "table.exec.sink.rowtime-inserter" : "ENABLED", - "table.exec.sink.type-length-enforcer" : "IGNORE", - "table.exec.sink.upsert-materialize" : "AUTO" - }, - "dynamicTableSink" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`MySink`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "vehicle_id", - "dataType" : "BIGINT" - }, { - "name" : "startTime", - "dataType" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - }, { - "name" : "endTime", - "dataType" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - }, { - "name" : "Initial_Temp", - "dataType" : "INT" - }, { - "name" : "Final_Temp", - "dataType" : "INT" - } ], - "watermarkSpecs" : [ ] - }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values", - "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT" - } - } - } - }, - "inputChangelogMode" : [ "INSERT" ], - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`vehicle_id` BIGINT, `startTime` TIMESTAMP(3) WITH LOCAL TIME ZONE, `endTime` TIMESTAMP(3) WITH LOCAL TIME ZONE, `Initial_Temp` INT, `Final_Temp` INT>", - "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[vehicle_id, startTime, endTime, Initial_Temp, Final_Temp])" - } ], - "edges" : [ { - "source" : 1, - "target" : 2, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 2, - "target" : 3, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 3, - "target" : 4, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 4, - "target" : 5, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - } ] -} diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipToNextRow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipToNextRow.out deleted file mode 100644 index 8e6d41a62367c..0000000000000 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testSkipToNextRow.out +++ /dev/null @@ -1,468 +0,0 @@ -{ - "flinkVersion" : "", - "nodes" : [ { - "id" : 1, - "type" : "stream-exec-table-source-scan_1", - "scanTableSource" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`MyTable`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "vehicle_id", - "dataType" : "BIGINT" - }, { - "name" : "engine_temperature", - "dataType" : "INT" - }, { - "name" : "rowtime", - "dataType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ], - "watermarkSpecs" : [ { - "rowtimeAttribute" : "rowtime", - "expression" : { - "rexNode" : { - "kind" : "CALL", - "internalName" : "$SOURCE_WATERMARK$1", - "operands" : [ ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - }, - "serializableString" : "`SOURCE_WATERMARK`()" - } - } ] - }, - "partitionKeys" : [ ], - "options" : { - "bounded" : "false", - "connector" : "values" - } - } - } - }, - "outputType" : "ROW<`vehicle_id` BIGINT, `engine_temperature` INT, `rowtime` TIMESTAMP(3) WITH LOCAL TIME ZONE>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[vehicle_id, engine_temperature, rowtime])", - "inputProperties" : [ ] - }, { - "id" : 2, - "type" : "stream-exec-watermark-assigner_1", - "watermarkExpr" : { - "kind" : "CALL", - "internalName" : "$SOURCE_WATERMARK$1", - "operands" : [ ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - }, - "rowtimeFieldIndex" : 2, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "vehicle_id", - "fieldType" : "BIGINT" - }, { - "name" : "engine_temperature", - "fieldType" : "INT" - }, { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ] - }, - "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[SOURCE_WATERMARK()])" - }, { - "id" : 3, - "type" : "stream-exec-exchange_1", - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "HASH", - "keys" : [ 0 ] - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "vehicle_id", - "fieldType" : "BIGINT" - }, { - "name" : "engine_temperature", - "fieldType" : "INT" - }, { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ] - }, - "description" : "Exchange(distribution=[hash[vehicle_id]])" - }, { - "id" : 4, - "type" : "stream-exec-match_1", - "matchSpec" : { - "pattern" : { - "kind" : "CALL", - "syntax" : "BINARY", - "sqlKind" : "PATTERN_CONCAT", - "operands" : [ { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$PATTERN_QUANTIFIER$1", - "operands" : [ { - "kind" : "LITERAL", - "value" : "A", - "type" : "CHAR(1) NOT NULL" - }, { - "kind" : "LITERAL", - "value" : 1, - "type" : "INT NOT NULL" - }, { - "kind" : "LITERAL", - "value" : -1, - "type" : "INT NOT NULL" - }, { - "kind" : "LITERAL", - "value" : false, - "type" : "BOOLEAN NOT NULL" - } ], - "type" : "NULL" - }, { - "kind" : "LITERAL", - "value" : "B", - "type" : "CHAR(1) NOT NULL" - } ], - "type" : "NULL" - }, - "patternDefinitions" : { - "A" : { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$OR$1", - "operands" : [ { - "kind" : "CALL", - "syntax" : "POSTFIX", - "internalName" : "$IS NULL$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$LAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 1, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - } ], - "type" : "BOOLEAN NOT NULL" - }, { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$>$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$PREV$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - }, { - "kind" : "CALL", - "internalName" : "$LAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 1, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - } ], - "type" : "BOOLEAN" - } ], - "type" : "BOOLEAN" - }, - "B" : { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$<$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$PREV$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "B", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - }, { - "kind" : "CALL", - "internalName" : "$LAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - } ], - "type" : "BOOLEAN" - } - }, - "measures" : { - "Final_Temp" : { - "kind" : "CALL", - "syntax" : "PREFIX", - "internalName" : "$FINAL$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$LAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - } ], - "type" : "INT" - }, - "Initial_Temp" : { - "kind" : "CALL", - "syntax" : "PREFIX", - "internalName" : "$FINAL$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$FIRST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "INT" - } ], - "type" : "INT" - }, - "endTime" : { - "kind" : "CALL", - "syntax" : "PREFIX", - "internalName" : "$FINAL$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$LAST$1", - "operands" : [ { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$CAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 2, - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - }, - "startTime" : { - "kind" : "CALL", - "syntax" : "PREFIX", - "internalName" : "$FINAL$1", - "operands" : [ { - "kind" : "CALL", - "internalName" : "$FIRST$1", - "operands" : [ { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$CAST$1", - "operands" : [ { - "kind" : "PATTERN_INPUT_REF", - "alpha" : "A", - "inputIndex" : 2, - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - }, { - "kind" : "LITERAL", - "value" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - } ], - "type" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - } - }, - "after" : { - "kind" : "LITERAL", - "symbol" : "MATCH_RECOGNIZE_AFTER_OPTION", - "value" : "SKIP_TO_NEXT_ROW", - "type" : { - "type" : "SYMBOL", - "nullable" : false - } - }, - "subsets" : { }, - "allRows" : false, - "partition" : { - "fields" : [ 0 ] - }, - "orderBy" : { - "fields" : [ { - "index" : 2, - "isAscending" : true, - "nullIsLast" : false - } ] - }, - "interval" : null - }, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`vehicle_id` BIGINT, `startTime` TIMESTAMP(3) WITH LOCAL TIME ZONE, `endTime` TIMESTAMP(3) WITH LOCAL TIME ZONE, `Initial_Temp` INT, `Final_Temp` INT>", - "description" : "Match(partitionBy=[vehicle_id], orderBy=[rowtime ASC], measures=[FINAL(FIRST(CAST(A.rowtime AS TIMESTAMP_WITH_LOCAL_TIME_ZONE(3)), 0)) AS startTime, FINAL(LAST(CAST(A.rowtime AS TIMESTAMP_WITH_LOCAL_TIME_ZONE(3)), 0)) AS endTime, FINAL(FIRST(A.engine_temperature, 0)) AS Initial_Temp, FINAL(LAST(A.engine_temperature, 0)) AS Final_Temp], rowsPerMatch=[ONE ROW PER MATCH], after=[SKIP TO NEXT ROW], pattern=[(PATTERN_QUANTIFIER(_UTF-16LE'A', 1, -1, false), _UTF-16LE'B')], define=[{A=OR(IS NULL(LAST(A.$1, 1)), >(PREV(A.$1, 0), LAST(A.$1, 1))), B=<(PREV(B.$1, 0), LAST(A.$1, 0))}])" - }, { - "id" : 5, - "type" : "stream-exec-sink_1", - "configuration" : { - "table.exec.sink.keyed-shuffle" : "AUTO", - "table.exec.sink.not-null-enforcer" : "ERROR", - "table.exec.sink.rowtime-inserter" : "ENABLED", - "table.exec.sink.type-length-enforcer" : "IGNORE", - "table.exec.sink.upsert-materialize" : "AUTO" - }, - "dynamicTableSink" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`MySink`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "vehicle_id", - "dataType" : "BIGINT" - }, { - "name" : "startTime", - "dataType" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - }, { - "name" : "endTime", - "dataType" : "TIMESTAMP(3) WITH LOCAL TIME ZONE" - }, { - "name" : "Initial_Temp", - "dataType" : "INT" - }, { - "name" : "Final_Temp", - "dataType" : "INT" - } ], - "watermarkSpecs" : [ ] - }, - "partitionKeys" : [ ], - "options" : { - "connector" : "values", - "sink-insert-only" : "false", - "table-sink-class" : "DEFAULT" - } - } - } - }, - "inputChangelogMode" : [ "INSERT" ], - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : "ROW<`vehicle_id` BIGINT, `startTime` TIMESTAMP(3) WITH LOCAL TIME ZONE, `endTime` TIMESTAMP(3) WITH LOCAL TIME ZONE, `Initial_Temp` INT, `Final_Temp` INT>", - "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[vehicle_id, startTime, endTime, Initial_Temp, Final_Temp])" - } ], - "edges" : [ { - "source" : 1, - "target" : 2, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 2, - "target" : 3, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 3, - "target" : 4, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 4, - "target" : 5, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - } ] -} From 74f3b69430d7f55fb385d1463e71dbbb8142a29f Mon Sep 17 00:00:00 2001 From: Jim Hughes Date: Fri, 1 Dec 2023 13:21:35 -0500 Subject: [PATCH 3/4] Adding MR tests with rowtime and secondary ordering. --- .../stream/MatchRecognizeRestoreTest.java | 2 + .../stream/MatchRecognizeTestPrograms.java | 79 +++ .../complex-match/savepoint/_metadata | Bin 21081 -> 21078 bytes .../plan/order-by-event-time-match.json | 597 +++++++++++++++++ .../savepoint/_metadata | Bin 0 -> 18898 bytes .../plan/order-by-int-column-match.json | 601 ++++++++++++++++++ .../savepoint/_metadata | Bin 0 -> 18895 bytes .../simple-match/savepoint/_metadata | Bin 16607 -> 16604 bytes .../plan/skip-past-last-row.json | 26 +- .../skip-past-last-row/savepoint/_metadata | Bin 17991 -> 17994 bytes .../skip-to-first/plan/skip-to-first.json | 26 +- .../skip-to-first/savepoint/_metadata | Bin 18079 -> 18082 bytes .../skip-to-last/plan/skip-to-last.json | 26 +- .../skip-to-last/savepoint/_metadata | Bin 18079 -> 18082 bytes .../plan/skip-to-next-row.json | 26 +- .../skip-to-next-row/savepoint/_metadata | Bin 18431 -> 18431 bytes 16 files changed, 1331 insertions(+), 52 deletions(-) create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/order-by-event-time-match/plan/order-by-event-time-match.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/order-by-event-time-match/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/order-by-int-column-match/plan/order-by-int-column-match.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/order-by-int-column-match/savepoint/_metadata diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeRestoreTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeRestoreTest.java index 5322182ce1e51..c9ad1c85f34e9 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeRestoreTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeRestoreTest.java @@ -36,6 +36,8 @@ public List programs() { return Arrays.asList( MatchRecognizeTestPrograms.SIMPLE_MATCH, MatchRecognizeTestPrograms.COMPLEX_MATCH, + MatchRecognizeTestPrograms.ORDER_BY_EVENT_TIME_MATCH, + MatchRecognizeTestPrograms.ORDER_BY_INT_COLUMN_MATCH, MatchRecognizeTestPrograms.SKIP_TO_FIRST, MatchRecognizeTestPrograms.SKIP_TO_LAST, MatchRecognizeTestPrograms.SKIP_TO_NEXT_ROW, diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeTestPrograms.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeTestPrograms.java index ee75742a1e753..cd9b81844d178 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeTestPrograms.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeTestPrograms.java @@ -120,6 +120,85 @@ public class MatchRecognizeTestPrograms { + ") AS T") .build(); + static final Row[] BEFORE_DATA = { + Row.of("2020-10-10 00:00:01", 10, 3), + Row.of("2020-10-10 00:00:01", 8, 2), + Row.of("2020-10-10 00:00:01", 9, 1), + Row.of("2020-10-10 00:00:04", 7, 4), + Row.of("2020-10-10 00:00:07", 8, 5), + // out of order - should be processed with a 2-second watermark in use. + Row.of("2020-10-10 00:00:06", 5, 6), + Row.of("2020-10-10 00:00:12", 3, 7), + // late event - should be ignored with a 2-second watermark in use. + Row.of("2020-10-10 00:00:08", 4, 8), + Row.of("2020-10-10 00:00:16", 4, 9), + Row.of("2020-10-10 00:00:32", 7, 10), + Row.of("2020-10-10 00:00:34", 5, 11) + }; + + static final Row[] AFTER_DATA = { + Row.of("2020-10-10 00:00:33", 9, 12), + Row.of("2020-10-10 00:00:41", 3, 13), + Row.of("2020-10-10 00:00:42", 11, 16), + Row.of("2020-10-10 00:00:43", 12, 15), + Row.of("2020-10-10 00:00:44", 13, 14) + }; + + static final SourceTestStep SOURCE = + SourceTestStep.newBuilder("MyEventTimeTable") + .addSchema( + "ts STRING", + "price INT", + "sequence_num INT", + "`rowtime` AS TO_TIMESTAMP(`ts`)", + "`proctime` AS PROCTIME()", + "WATERMARK for `rowtime` AS `rowtime` - INTERVAL '2' SECOND") + .producedBeforeRestore(BEFORE_DATA) + .producedAfterRestore(AFTER_DATA) + .build(); + + static final TableTestProgram ORDER_BY_EVENT_TIME_MATCH = + TableTestProgram.of("order-by-event-time-match", "complex match recognize test") + .setupTableSource(SOURCE) + .setupTableSink( + SinkTestStep.newBuilder("MySink") + .addSchema("first bigint", "last bigint", "up bigint") + .consumedBeforeRestore(Row.of(10L, 8L, 9L), Row.of(7L, 5L, 8L)) + .consumedAfterRestore(Row.of(9L, 3L, 11L)) + .build()) + .runSql(getEventTimeSql("ORDER BY rowtime")) + .build(); + + static final TableTestProgram ORDER_BY_INT_COLUMN_MATCH = + TableTestProgram.of("order-by-int-column-match", "complex match recognize test") + .setupTableSource(SOURCE) + .setupTableSink( + SinkTestStep.newBuilder("MySink") + .addSchema("first bigint", "last bigint", "up bigint") + .consumedBeforeRestore(Row.of(9L, 8L, 10L), Row.of(7L, 5L, 8L)) + .consumedAfterRestore(Row.of(9L, 3L, 11L)) + .build()) + .runSql(getEventTimeSql("ORDER BY rowtime, sequence_num")) + .build(); + + private static String getEventTimeSql(final String orderByClause) { + final String sql = + "insert into MySink SELECT * FROM MyEventTimeTable MATCH_RECOGNIZE (\n" + + " %s\n" + + " MEASURES\n" + + " FIRST(DOWN.price) as first,\n" + + " LAST(DOWN.price) as last,\n" + + " UP.price as up\n" + + " ONE ROW PER MATCH\n" + + " AFTER MATCH SKIP PAST LAST ROW\n" + + " PATTERN (DOWN{2,} UP)\n" + + " DEFINE\n" + + " DOWN AS price < LAST(DOWN.price, 1) OR LAST(DOWN.price, 1) IS NULL,\n" + + " UP AS price > LAST(DOWN.price)\n" + + ") AS T"; + return String.format(sql, orderByClause); + } + static final TableTestProgram SKIP_TO_FIRST = getSkipTestProgram( "skip-to-first", diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/complex-match/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/complex-match/savepoint/_metadata index 0ee410d21b5d8aa7b87a3f140159847d38004105..c9a137f19f81a760aad20008091da0a9b427fa1f 100644 GIT binary patch delta 511 zcmbVJJxc>Y5WT}hD;vQ=h$JAACUe=3%igSFp~ObSfSlS1dw07af(mM|7c9g^w-PHM zKR`%rZD|(}1pOm|yIcwj8(n7Dd2inA?C!h`!qFgn+N^}Pw!xNnCDMdBBT^)Ylo3JO zr>d_{jt`E`pvZF5k)XLW1!_ZdEZaJBh)4ptQUYR%3KVBKaGDWg5+|ZK{HUBa@1liD zbH;4j>uBC~qcsmw2Uctk7E5pe^-}Txt7Y&B4V$*;hSyfRy}hj<2qv6|&c`df@uxb< z@98*El2g87iTvbJ``VhF^UB{UQv20fpDwdO)i((3OLXY(vSUU2`QN`p3c(o`5-1uC zF-if*c>!8Ul_Mr9N%ZiF+=pd{{lCvaC=-(o)4+3MAdd3_5Q%_Jf}EgWG0kKU)E?cN NCha?|4M*&<{tZp(p&Uaf71515bJ4u_`UyRj7R8@q$IM6wm$nx{h|TCbCT zB(t7(9LFGpVZb|?1+BaXVLF;fDQK=taf2Vw", + "description" : "TableSourceScan(table=[[default_catalog, default_database, MyEventTimeTable]], fields=[ts, price, sequence_num])", + "inputProperties" : [ ] + }, { + "id" : 13, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "CALL", + "internalName" : "$TO_TIMESTAMP$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "TIMESTAMP(3)" + }, { + "kind" : "CALL", + "internalName" : "$PROCTIME$1", + "operands" : [ ], + "type" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "ts", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "price", + "fieldType" : "INT" + }, { + "name" : "sequence_num", + "fieldType" : "INT" + }, { + "name" : "rowtime", + "fieldType" : "TIMESTAMP(3)" + }, { + "name" : "proctime", + "fieldType" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + } ] + }, + "description" : "Calc(select=[ts, price, sequence_num, TO_TIMESTAMP(ts) AS rowtime, PROCTIME() AS proctime])" + }, { + "id" : 14, + "type" : "stream-exec-watermark-assigner_1", + "watermarkExpr" : { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$-$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "TIMESTAMP(3)" + }, { + "kind" : "LITERAL", + "value" : "2000", + "type" : "INTERVAL SECOND(6) NOT NULL" + } ], + "type" : "TIMESTAMP(3)" + }, + "rowtimeFieldIndex" : 3, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "ts", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "price", + "fieldType" : "INT" + }, { + "name" : "sequence_num", + "fieldType" : "INT" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + }, { + "name" : "proctime", + "fieldType" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + } ] + }, + "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 2000:INTERVAL SECOND)])" + }, { + "id" : 15, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "SINGLETON" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "ts", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "price", + "fieldType" : "INT" + }, { + "name" : "sequence_num", + "fieldType" : "INT" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + }, { + "name" : "proctime", + "fieldType" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + } ] + }, + "description" : "Exchange(distribution=[single])" + }, { + "id" : 16, + "type" : "stream-exec-match_1", + "matchSpec" : { + "pattern" : { + "kind" : "CALL", + "syntax" : "BINARY", + "sqlKind" : "PATTERN_CONCAT", + "operands" : [ { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$PATTERN_QUANTIFIER$1", + "operands" : [ { + "kind" : "LITERAL", + "value" : "DOWN", + "type" : "CHAR(4) NOT NULL" + }, { + "kind" : "LITERAL", + "value" : 2, + "type" : "INT NOT NULL" + }, { + "kind" : "LITERAL", + "value" : -1, + "type" : "INT NOT NULL" + }, { + "kind" : "LITERAL", + "value" : false, + "type" : "BOOLEAN NOT NULL" + } ], + "type" : "NULL" + }, { + "kind" : "LITERAL", + "value" : "UP", + "type" : "CHAR(2) NOT NULL" + } ], + "type" : "NULL" + }, + "patternDefinitions" : { + "DOWN" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$OR$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "*", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "DOWN", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "BOOLEAN" + }, { + "kind" : "CALL", + "syntax" : "POSTFIX", + "internalName" : "$IS NULL$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "DOWN", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "BOOLEAN NOT NULL" + } ], + "type" : "BOOLEAN" + }, + "UP" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$>$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "*", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "DOWN", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "BOOLEAN" + } + }, + "measures" : { + "first" : { + "kind" : "CALL", + "syntax" : "PREFIX", + "internalName" : "$FINAL$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$FIRST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "DOWN", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "INT" + }, + "last" : { + "kind" : "CALL", + "syntax" : "PREFIX", + "internalName" : "$FINAL$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "DOWN", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "INT" + }, + "up" : { + "kind" : "CALL", + "syntax" : "PREFIX", + "internalName" : "$FINAL$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "UP", + "inputIndex" : 1, + "type" : "INT" + } ], + "type" : "INT" + } + }, + "after" : { + "kind" : "LITERAL", + "symbol" : "MATCH_RECOGNIZE_AFTER_OPTION", + "value" : "SKIP_PAST_LAST_ROW", + "type" : { + "type" : "SYMBOL", + "nullable" : false + } + }, + "subsets" : { }, + "allRows" : false, + "partition" : { + "fields" : [ ] + }, + "orderBy" : { + "fields" : [ { + "index" : 3, + "isAscending" : true, + "nullIsLast" : false + } ] + }, + "interval" : null + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`first` INT, `last` INT, `up` INT>", + "description" : "Match(orderBy=[rowtime ASC], measures=[FINAL(FIRST(DOWN.price, 0)) AS first, FINAL(LAST(DOWN.price, 0)) AS last, FINAL(UP.price) AS up], rowsPerMatch=[ONE ROW PER MATCH], after=[SKIP PAST LAST ROW], pattern=[(PATTERN_QUANTIFIER(_UTF-16LE'DOWN', 2, -1, false), _UTF-16LE'UP')], define=[{DOWN=OR(<(LAST(*.$1, 0), LAST(DOWN.$1, 1)), IS NULL(LAST(DOWN.$1, 1))), UP=>(LAST(*.$1, 0), LAST(DOWN.$1, 0))}])" + }, { + "id" : 17, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$CAST$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + } ], + "type" : "BIGINT" + }, { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$CAST$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT" + } ], + "type" : "BIGINT" + }, { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$CAST$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + } ], + "type" : "BIGINT" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`first` BIGINT, `last` BIGINT, `up` BIGINT>", + "description" : "Calc(select=[CAST(first AS BIGINT) AS first, CAST(last AS BIGINT) AS last, CAST(up AS BIGINT) AS up])" + }, { + "id" : 18, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`MySink`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "first", + "dataType" : "BIGINT" + }, { + "name" : "last", + "dataType" : "BIGINT" + }, { + "name" : "up", + "dataType" : "BIGINT" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`first` BIGINT, `last` BIGINT, `up` BIGINT>", + "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[first, last, up])" + } ], + "edges" : [ { + "source" : 12, + "target" : 13, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 13, + "target" : 14, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 14, + "target" : 15, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 15, + "target" : 16, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 16, + "target" : 17, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 17, + "target" : 18, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/order-by-event-time-match/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/order-by-event-time-match/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..4251befd8513b258ff2d0828f77729f4566efd3c GIT binary patch literal 18898 zcmeGkTZ|i5_1fOuJe%D%OWGu~T9i$pYV8@@`%EfY@2=B0b-a%4B+XXfj>p&D>5gZd znek?mk2Ea;3K1=;K%%OkRO$y(TLD5KB!u|j^+BapJS6@o0;y`nOFYzoT7`4&%&cd2 z#)(7XJjh&m9=?xr&pr3dx#ym9W-^b}qlAz?I536G3t9Jtzx;I|SvT2CDWx@jBAFadPE``g`1o`sJ~kDf9IH;nlVbu@-RuB`5N)s_>?cHSEvhX| zqy;Kesa7-ugJLBpc&;YNbPu;IXk1N|p-|@3qS|PvO3XOfq%A{|^;kvFX-v>G;bd&)q(SqV)Q}8mnWiwS zNHs~2rBhTJy-;CM5t{m%IqjK;R#3IL^_8$g%GFOD|K!fDWyzqeoAUtIHmPAI_x?ri=O1tVap(($Qv79aW~- z5ml-pU+JJHP{(jOMqo-aR1)==tkxw_ zkYh&#Z5EzA#GlPE%yO87GGsueO5Ipo=@w4bgy)76ULYGKJ=cuP9g+FHXSw-ExP+VZ%tEW~6%smIbv ztKlqc8f4v(RAF|1s_?}5^XDyc-q%&18O3mu0Q>50CLHLtXX3`CU?x{eKa?&6)I)wi z-gUCrMfuN7y+l{MR{|Xc+bnmh50bF+Iy4o z^3GBQ7UELsV7`1bQ`$d&xKzfR^1^%$x9Sm)NS8Q6NKGd{94DvA0EnH!{85B#Jv8@s zHSMqj3G#xf(CX@B-9!vPuEuh3mzx2~+4DmNub;Z z$~_v5P$IXtS%GpN3${kM^Tf?%pxj?aTm3+}haKZ}y1NUMd&BK_@YCZ*{W4ujN}$}g zJ=r&ha{n>65nq8lA=RTNBx5luTNdjmt=8#vY1#LBH+)U+H;R|2x1NLpo_&ljv5L&U zW;%6{aP3W!Eh&y9*tXYiLt|WeA8BB&QD>)^(eCsDMK(SiunCj@Gxq4M$MxFS7Ph;^ z5Az{LMHSrWmeKRN@+9`gz|f0D_u9Kv3vr2&00gymZD&R zmMem|gzJeND+Y7y8^Pvv3G0SNg$6)mv21FSY66HHJ%*+_tt~WsBtzI?B)yvP8(tY? z8)7ft0e`oP+{{WBP}&jic;zUKy5Nqpa9VGPCHoR;4Zg7*{~W>-R@+a8@RpFRaPV;0 zag7@>6VvuL23uIuN(nx}N8|V#;rTuAOXB(p2_KIUcc)e}v)BH)`J*|gvkfa_iG8q} z4VX)|xZ6X{5**=m*mMu?3oZ;5%*MnXoq2WDHX+2JF6(V0xW8Qq3(SzA z5`u^j7=vG>kvE~FGhtbCsIdtSOmB`{jIbDp3kid`ww*o0hR6DF&+7T}tQMZ`Q@?Wl z-FJSp<-MOgxbr~$Z>tf^SZ0v}Z=LCbx1JdYHx51U(I-Cln;-BWnS(N{c?Rp)O;`vs zNF5bo-0e^~YqhRn=4FZL`W9o347)T!QE0@k(+kIw{bJ z=-7Cr60JL|v7Kw7DW1qcJx0CZHy#gzfBFOEXLANTNFzzr!>Q;Wf(|+2KWx*|H(T(PT>!36nnPjg58YBu1JUWY95Ch~(P)C>gZM z^QsQ9=PD?FTOV~GJzsg8iUwe9MI?;)po{%-&@Z!30d<=eD7(|`j!v`i*wN~KL0`n~ zSDFw4)m_XEz{h#C#)jcfSu((I*U>K@y8fabu8LhO0>KDC!@F6a6!LFuR+VL#GMGwz z^sv%Ut5Qv3fnC7Y?+@;dfA>$%3^D5U$&h2y?!JoyXan9pL(cr-lkeTdz=TADY;(+T zyP7$@TicM?dky$}tvvb5JD>l#@YM(gvGwWjlEX?9W;SvHgIyyqo^daCvmjr$zsBN; zQMm5IC%VlX5n%i()kRHenoND0gQrbX9P+dR4+Tm~dTT zG9%S}JSPb5me6Uu3>jo#Ci4(D6&SnjfG+w#L>TPfT_LP8vRV>uHJ$k@Sh}6%+~K0q zg#}>BP#cnH3#uJY7w`Gcsa*+J-x$0w$Ph<(9Xet6p8ew&Uw?Mj?_gRFkp5*sZc%uu zhS>bE7(6qu_%pA5`OZyWdh0Aga^cA^ya$nuI??Zty+Rz@g5LULBi4d(Idge)1_)X_ z8_*B-&ba(sYKUM!KZss-=q~>w{Tz}aLV9Bfx{;u+-H`-pJgT698eg3`8L07B`c;NN zjc2>GK#dR7_|wFz-3My?4e+K^poU%%H8j`;UEVe*P~%u12^Td!P)Lb z_`r&|#6A!cn_a!lYv`ReAehR+A{-{`7syam(nUe5&I*c1!A4qLAh%dq`vmxI4g4Rp z|4WLbFFJFIt%}}iV2%x@*wUL+sbbbZO@+_lz|Dr1BFX|d#H(NiZ{voR17AzOH4Ls@ zm9sen+kgR;jUYt85E<`=2u1sQIoSN`pZ(SPmmeAaewctcbR*!{5ZHUK1{wES@kLdm z99^Lz7?D+lTLrXlwV7=&HHfb", + "description" : "TableSourceScan(table=[[default_catalog, default_database, MyEventTimeTable]], fields=[ts, price, sequence_num])", + "inputProperties" : [ ] + }, { + "id" : 20, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "CALL", + "internalName" : "$TO_TIMESTAMP$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "TIMESTAMP(3)" + }, { + "kind" : "CALL", + "internalName" : "$PROCTIME$1", + "operands" : [ ], + "type" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "ts", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "price", + "fieldType" : "INT" + }, { + "name" : "sequence_num", + "fieldType" : "INT" + }, { + "name" : "rowtime", + "fieldType" : "TIMESTAMP(3)" + }, { + "name" : "proctime", + "fieldType" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + } ] + }, + "description" : "Calc(select=[ts, price, sequence_num, TO_TIMESTAMP(ts) AS rowtime, PROCTIME() AS proctime])" + }, { + "id" : 21, + "type" : "stream-exec-watermark-assigner_1", + "watermarkExpr" : { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$-$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "TIMESTAMP(3)" + }, { + "kind" : "LITERAL", + "value" : "2000", + "type" : "INTERVAL SECOND(6) NOT NULL" + } ], + "type" : "TIMESTAMP(3)" + }, + "rowtimeFieldIndex" : 3, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "ts", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "price", + "fieldType" : "INT" + }, { + "name" : "sequence_num", + "fieldType" : "INT" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + }, { + "name" : "proctime", + "fieldType" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + } ] + }, + "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 2000:INTERVAL SECOND)])" + }, { + "id" : 22, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "SINGLETON" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : { + "type" : "ROW", + "fields" : [ { + "name" : "ts", + "fieldType" : "VARCHAR(2147483647)" + }, { + "name" : "price", + "fieldType" : "INT" + }, { + "name" : "sequence_num", + "fieldType" : "INT" + }, { + "name" : "rowtime", + "fieldType" : { + "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", + "precision" : 3, + "kind" : "ROWTIME" + } + }, { + "name" : "proctime", + "fieldType" : { + "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", + "nullable" : false, + "precision" : 3, + "kind" : "PROCTIME" + } + } ] + }, + "description" : "Exchange(distribution=[single])" + }, { + "id" : 23, + "type" : "stream-exec-match_1", + "matchSpec" : { + "pattern" : { + "kind" : "CALL", + "syntax" : "BINARY", + "sqlKind" : "PATTERN_CONCAT", + "operands" : [ { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$PATTERN_QUANTIFIER$1", + "operands" : [ { + "kind" : "LITERAL", + "value" : "DOWN", + "type" : "CHAR(4) NOT NULL" + }, { + "kind" : "LITERAL", + "value" : 2, + "type" : "INT NOT NULL" + }, { + "kind" : "LITERAL", + "value" : -1, + "type" : "INT NOT NULL" + }, { + "kind" : "LITERAL", + "value" : false, + "type" : "BOOLEAN NOT NULL" + } ], + "type" : "NULL" + }, { + "kind" : "LITERAL", + "value" : "UP", + "type" : "CHAR(2) NOT NULL" + } ], + "type" : "NULL" + }, + "patternDefinitions" : { + "DOWN" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$OR$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "*", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "DOWN", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "BOOLEAN" + }, { + "kind" : "CALL", + "syntax" : "POSTFIX", + "internalName" : "$IS NULL$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "DOWN", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "BOOLEAN NOT NULL" + } ], + "type" : "BOOLEAN" + }, + "UP" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$>$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "*", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "DOWN", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "BOOLEAN" + } + }, + "measures" : { + "first" : { + "kind" : "CALL", + "syntax" : "PREFIX", + "internalName" : "$FINAL$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$FIRST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "DOWN", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "INT" + }, + "last" : { + "kind" : "CALL", + "syntax" : "PREFIX", + "internalName" : "$FINAL$1", + "operands" : [ { + "kind" : "CALL", + "internalName" : "$LAST$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "DOWN", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "type" : "INT" + }, + "up" : { + "kind" : "CALL", + "syntax" : "PREFIX", + "internalName" : "$FINAL$1", + "operands" : [ { + "kind" : "PATTERN_INPUT_REF", + "alpha" : "UP", + "inputIndex" : 1, + "type" : "INT" + } ], + "type" : "INT" + } + }, + "after" : { + "kind" : "LITERAL", + "symbol" : "MATCH_RECOGNIZE_AFTER_OPTION", + "value" : "SKIP_PAST_LAST_ROW", + "type" : { + "type" : "SYMBOL", + "nullable" : false + } + }, + "subsets" : { }, + "allRows" : false, + "partition" : { + "fields" : [ ] + }, + "orderBy" : { + "fields" : [ { + "index" : 3, + "isAscending" : true, + "nullIsLast" : false + }, { + "index" : 2, + "isAscending" : true, + "nullIsLast" : false + } ] + }, + "interval" : null + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`first` INT, `last` INT, `up` INT>", + "description" : "Match(orderBy=[rowtime ASC, sequence_num ASC], measures=[FINAL(FIRST(DOWN.price, 0)) AS first, FINAL(LAST(DOWN.price, 0)) AS last, FINAL(UP.price) AS up], rowsPerMatch=[ONE ROW PER MATCH], after=[SKIP PAST LAST ROW], pattern=[(PATTERN_QUANTIFIER(_UTF-16LE'DOWN', 2, -1, false), _UTF-16LE'UP')], define=[{DOWN=OR(<(LAST(*.$1, 0), LAST(DOWN.$1, 1)), IS NULL(LAST(DOWN.$1, 1))), UP=>(LAST(*.$1, 0), LAST(DOWN.$1, 0))}])" + }, { + "id" : 24, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$CAST$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + } ], + "type" : "BIGINT" + }, { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$CAST$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT" + } ], + "type" : "BIGINT" + }, { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$CAST$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + } ], + "type" : "BIGINT" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`first` BIGINT, `last` BIGINT, `up` BIGINT>", + "description" : "Calc(select=[CAST(first AS BIGINT) AS first, CAST(last AS BIGINT) AS last, CAST(up AS BIGINT) AS up])" + }, { + "id" : 25, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`MySink`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "first", + "dataType" : "BIGINT" + }, { + "name" : "last", + "dataType" : "BIGINT" + }, { + "name" : "up", + "dataType" : "BIGINT" + } ], + "watermarkSpecs" : [ ] + }, + "partitionKeys" : [ ] + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`first` BIGINT, `last` BIGINT, `up` BIGINT>", + "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[first, last, up])" + } ], + "edges" : [ { + "source" : 19, + "target" : 20, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 20, + "target" : 21, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 21, + "target" : 22, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 22, + "target" : 23, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 23, + "target" : 24, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 24, + "target" : 25, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/order-by-int-column-match/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/order-by-int-column-match/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..772bf3fad7c9011d83b32865cca1957f6e4b624e GIT binary patch literal 18895 zcmeGkZHybmb?m##+KARw@@6JgKj_=DnTFqqP7bWDtIseCCA=?^}Fnzxu}EUtP?6Z;br@ z&4;YAvsi8e;k_gM^wrz9KD1c=%+1CB-gMyy_x%Hk5b`X7bg6P-s(jPJ)88?k-}dEi z+&**h=N6=^iVu17lQCUVmI4|xG%6Z!)tj=RLkSe|bo2Ap^=ITd3)BPhmI<O6e5o2a6>TWB+EfTr?UYay9&0L+5l)6D z!?AdLDwYVvCX>m;GlI2mCgC1j5P{B2pQu_#6#8K0X^=TpVGe5p`O71O2JOre<1Odl&| z=5nQ@bF*nOTA0sfitU0luQz%;l|7a&71H@kDw|nI=ONcmj;mTNNE=jKWWlNoLJk_V zEVH22R1B%kg2tH!YZ{WQ2h%GC(-bNfm?lwKdYow!-mHQ`8~UPZ0PX>D{ae_1zVmOB zAzdSPb)ke5^xz?h$rXK;8ng({@4S5eV=q4Se92FIS+cW=Wp9p@n^}1s4C0Q^^&U0$W#Z!H<*ZJs?-FYbudeI$ZAaz zsf=tHB$5R>f+;V7%S+($5_l^F3s%Us>l|EZENf(+mp$I>ym}f$glWJ}27wxH>-3o% ztpl|YM?;U&#=%F(h*dtX>XHGB8)Vlb4yNbIk1^2zt_PPSrDBlr4(uXb0q8ym5CFNe zVWF~#?sX8&fq6v^WE+`(fozxbTvL{jbe3#I;(1jy3=;3qVg%~mL0Au(MxM|}>~dMcx)xjHMna%f zJ5e=*s6oNB(yTiR8wMH3&z%4n-ob%Te}glsEJK$;SLzeTl)74xsuFJ$==FOe2g2X~ z%NItu^af?fv1NDn`y*&0-aAdszV@kq-p0}RM1$;h^l%%@?A~2%%qdNyXYl=6dFt1< zKKCm9<^b_!$e<3dK8Oo6_B6iU1JItyb;-@ri3zwK!Z+HI4JgL|D1L?Mq9!#=rM}Uj z(*;A5lp5-^0xtzh;1;;Y9>@E^l?KRls48i?0h|cy;$&K?Wfa4$pkxKnUGIAWFQW$8 zGMzb`$rZV}?gJ8oFhm&q={6sNjJHbs&4x3dAd~y!&#HIwC?03mtA!O{tf1o|)BvkAoXYaHDy^W1>%o?5?*Q5a&Wlb0x>C3Bb zc=2t<@M7s~Uw+nneYwMPUw%{$I%eGeQ+|$-Vt~whMhe=Hf}&cltBS+AEK{8UX{F{E zDBO>(QR2gdb@hUzZ^oku+Bf4X(`Wi-eBX=@uDK3q+vj~V-rSAQgZ3h`Z^qlpkiHq; z+M{W&Dtoa@&^O~Zw#(9&--kzjeKX#*dc7H8x2r@k&_?C&J}wxLTeH zC6nmIh*kmYItv^wJYiK#-2LJsHx$&SCbB%E;43K@)G&8;H`|fI!m(I1Mxz0-%qoFc ztjq$*@}wAuMC0*jGD(wDi8wI*XZV35r3>5Gv=7>gqFu%!m*Tc+aQckF<~6AfyZFnD z@3cA=zH1eFbP}3lrC#}NG1X_HkAxSX(Sp6M-tI^1*W$He}dM*jDQaB zUV1K^J`zHP$#qJV^4SvIxt4%^Wc0!G$@Hw{y)T}ePm>|;n&<3w_R>c^^NYwDLBJl& z9ZGQu?b*XML6|Ohzt3gMJ=_)BEIZB4n0})P$9)@l=Uf*_Z$yF3tG-AzRw*~DRi*`V zYK3Jg6XwN1V0DHIw-r&~XpN7Y?AjU^z4S6u3?OJ-9TYt|E|NV)KbA`!O&8`%N`!Fy2vBv#Pj+K6!K)ubX*aPe+;|%g49AAd2Hun}K)F?eGR) z{uY@jHDmGW3&z-#ZGz$Bsr<~5RQ}0Jmo8c2TwicYhuDE@)89M{0TSuM>3m;$_-b9Q zw{(^CBA&s>jKnk0Th`k1E<#WqjQ{n9`(^fL$hL;YL+P?}JaCkzh5P@63V8HN}z#B9AZ z_pYnkVG5%1f~v4etFx{l1=y6>yxZIgP|D8Dq_REIfvxlTxf!JAuw;kGLKjuA?LHg{ z%vfAACT>++Ri{jsg_ZN*?{25T9M?8=CR|TF$N8v-&u%fvt(ReyD7xf3ILGa)0kV0N zu8f^O#!dT&nykt6^(;iMn4#aBc%GC;k-9592D-L2-k~Y2ICjUo+Bfbu5NX{v?l&5% zzBv-?8}}Osh3p&m@AqL0E|=h&gH6%pzHtvL#;bI7*EjAB_lI_!?D&aJnJy!xZ``*( zJ-jxI`%kzx@fG+^is^yVlCc<&tsUzCtJK(4xwHQf^Ah#elkk9N9n;ZIQQ-bH)2V}Q zH1BfeIL3E$UeFkqPa_S=HQ^+R8Ev9#6uCyOPL7|Cn0oZq<9cQG7WTS_AMQg8h$?(H zZf<#AU7p0=Y%ug<(!Dahc~=Ro5Us?0%R)m_MW(|+3boQ{@HK07sx1Xz2U;#uaS7)W z3Vh^ggaMXQ+Yat;SHc4GOqkuPS+KDZ z9@FgOr>}Gb8h-;&bQdhI4iGz#5~MfA-%qeH6A**AcAq=TJCDu==+@_nWORJUsRzem28);IU5mc?kxY^!k)V zRbv8MVInxER7GeZI@GFWpWa}Qjwd^e!5Jh*yAx9UJPd;z@sjG)YA4?l@I|Eu6UYL* zw;f{WfbU?7?_dYMEhSjj*&f?wQn|TY2Aqa)kc`#C<1L)P0ElDNQ9B(}u~h+kp<`>D TJfTZ@8@QAU#|rc5-0c4VP=24R literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/simple-match/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/simple-match/savepoint/_metadata index 0e252bf28ee0f0e95d5e02d1ac9166d787340662..6dfe41e20d98701c9e9c94cb29be477f87444c19 100644 GIT binary patch delta 449 zcmbV{zfQw25XR$_jx4Z1iB!bW$<+4wZ;1yscmY-vC$>N!Kmv8|&H%CyeUFHdHvz$m zAaL3a%suJuXn=@7s8gc-8SWSH)b4GJw(8f|S{z^O zpnJ^Yg*}Y#WwD%mmU>BdZL!_$_Kpv-EXz~&Qf9FzN6^Xg>lIqyZj@!{u~V%29Q}uJ RTAIyHmtcRf+kV^M{0|sGa54Y@ delta 452 zcmbV|ze)o^5XSEkQ)*+Okx0bCZjzn-bGL(cf|ZY8DYtuj5)c&>`+%jOxP{LV4lDB% z){5_8qq`=Bbq0nRzWF|WFdx(OYnnd2rq6ydJUm6gNTAY+5Q#`-IoIQx>dswXO)hVt z(IV41P+coaCeNKxV<$Kg8iGop25HEGZd3yXEgEYj8hgqw4SZ{TN(}LByme4c|K~};6S!Y35i{T>`lTx@Z@b58?8*ob&X*=wC`&kGUZB{=9xx?5;(>e=l@g=6<6bY(}MfydNx`&=HXM Welb`W?u(H>FZzoZ70`Z`Kb>C@Ty^XK diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-past-last-row/plan/skip-past-last-row.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-past-last-row/plan/skip-past-last-row.json index e9c9d3b90a12d..f42a2779c3ece 100644 --- a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-past-last-row/plan/skip-past-last-row.json +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-past-last-row/plan/skip-past-last-row.json @@ -1,7 +1,7 @@ { "flinkVersion" : "1.19", "nodes" : [ { - "id" : 27, + "id" : 41, "type" : "stream-exec-table-source-scan_1", "scanTableSource" : { "table" : { @@ -42,7 +42,7 @@ "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[vehicle_id, engine_temperature])", "inputProperties" : [ ] }, { - "id" : 28, + "id" : 42, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", @@ -91,7 +91,7 @@ }, "description" : "Calc(select=[vehicle_id, engine_temperature, PROCTIME() AS proctime])" }, { - "id" : 29, + "id" : 43, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -121,7 +121,7 @@ }, "description" : "Exchange(distribution=[hash[vehicle_id]])" }, { - "id" : 30, + "id" : 44, "type" : "stream-exec-match_1", "matchSpec" : { "pattern" : { @@ -332,7 +332,7 @@ "outputType" : "ROW<`vehicle_id` BIGINT, `Initial_Temp` INT, `Final_Temp` INT>", "description" : "Match(partitionBy=[vehicle_id], orderBy=[proctime ASC], measures=[FINAL(FIRST(A.engine_temperature, 0)) AS Initial_Temp, FINAL(LAST(A.engine_temperature, 0)) AS Final_Temp], rowsPerMatch=[ONE ROW PER MATCH], after=[SKIP PAST LAST ROW], pattern=[(PATTERN_QUANTIFIER(_UTF-16LE'A', 1, -1, false), _UTF-16LE'B')], define=[{A=OR(IS NULL(LAST(A.$1, 1)), >(PREV(A.$1, 0), LAST(A.$1, 1))), B=<(PREV(B.$1, 0), LAST(A.$1, 0))}])" }, { - "id" : 31, + "id" : 45, "type" : "stream-exec-sink_1", "configuration" : { "table.exec.sink.keyed-shuffle" : "AUTO", @@ -374,29 +374,29 @@ "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[vehicle_id, Initial_Temp, Final_Temp])" } ], "edges" : [ { - "source" : 27, - "target" : 28, + "source" : 41, + "target" : 42, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 28, - "target" : 29, + "source" : 42, + "target" : 43, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 29, - "target" : 30, + "source" : 43, + "target" : 44, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 30, - "target" : 31, + "source" : 44, + "target" : 45, "shuffle" : { "type" : "FORWARD" }, diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-past-last-row/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-past-last-row/savepoint/_metadata index 000fbcf4013d95b614c35856babb0818b08beb65..2ba80159c539619a525b1bd9df7188262ad92387 100644 GIT binary patch delta 587 zcmbV}&1+OK6vgw#+KP5?C@QunRtBw`IQh8mC65tj+R=?5f`#g)X!4SP2(@6*MM1i1 zE1iO4a3LsE6c^pTbrIYMet=r=4-j1lc3VU!HE*B`w+$RZ?mdV6xc6Nl2g{ z282;yAPMJjNYdI}bg>n$Pnh@G@n+cpbFc@N*;nae{iPQ-N z#fXc2b!@8=o!+$h{wuAo_uqUR|D-oOX;mx!wR(8@*D5%Z2e|F6G#U0`Jn?nT^6$o77cxcNK6GFbn5`;=V z!jt*(iSxsLty%3E8JK$Ie>8C~+7p_m{O87zaZj1De#PoB#j- delta 578 zcmX@r!+5-hae{=vb~UD@wll1nCMhJ{*IaPm%j5*6EXIiw`>n%FjV;ZM&5g`W%#2M< zjm<60OfB_`6U$Ny@-y>FbPX-il2eRzP0h^Bk_;^kER&23^o>m{jEzi9Ep!vj%+hpC zlFcl1Ee(y6b<<1|Qw@!h43bk*%s0m}TS^L43jDY(nb}cny`_D{>RqRUfVPTk-X$Zz zLax<`X_l$Ure;RE2Fb}Ox+bYfiMl{X8SADbrY0Ggq*){-8z*j#Ra~JkdA^gLzIeYG z50fJU0|OHfa{w_AF!q2%VKgJM;N(9}PP`CV23D}z$&pT~YH(A;z+xaXpcu^H0Fz3S z!kmfRsqP}1WmH8Is7ImOaK*DTG@RM*77++5ev z(lQN5q?wqQSy);oCr(yi{3{Tz*3%-&R>`SA<3Cm diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-first/plan/skip-to-first.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-first/plan/skip-to-first.json index 8bff998b7445e..43e83ec5ef7e3 100644 --- a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-first/plan/skip-to-first.json +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-first/plan/skip-to-first.json @@ -1,7 +1,7 @@ { "flinkVersion" : "1.19", "nodes" : [ { - "id" : 12, + "id" : 26, "type" : "stream-exec-table-source-scan_1", "scanTableSource" : { "table" : { @@ -42,7 +42,7 @@ "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[vehicle_id, engine_temperature])", "inputProperties" : [ ] }, { - "id" : 13, + "id" : 27, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", @@ -91,7 +91,7 @@ }, "description" : "Calc(select=[vehicle_id, engine_temperature, PROCTIME() AS proctime])" }, { - "id" : 14, + "id" : 28, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -121,7 +121,7 @@ }, "description" : "Exchange(distribution=[hash[vehicle_id]])" }, { - "id" : 15, + "id" : 29, "type" : "stream-exec-match_1", "matchSpec" : { "pattern" : { @@ -334,7 +334,7 @@ "outputType" : "ROW<`vehicle_id` BIGINT, `Initial_Temp` INT, `Final_Temp` INT>", "description" : "Match(partitionBy=[vehicle_id], orderBy=[proctime ASC], measures=[FINAL(FIRST(A.engine_temperature, 0)) AS Initial_Temp, FINAL(LAST(A.engine_temperature, 0)) AS Final_Temp], rowsPerMatch=[ONE ROW PER MATCH], after=[SKIP TO FIRST _UTF-16LE'B'], pattern=[(PATTERN_QUANTIFIER(_UTF-16LE'A', 1, -1, false), _UTF-16LE'B')], define=[{A=OR(IS NULL(LAST(A.$1, 1)), >(PREV(A.$1, 0), LAST(A.$1, 1))), B=<(PREV(B.$1, 0), LAST(A.$1, 0))}])" }, { - "id" : 16, + "id" : 30, "type" : "stream-exec-sink_1", "configuration" : { "table.exec.sink.keyed-shuffle" : "AUTO", @@ -376,29 +376,29 @@ "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[vehicle_id, Initial_Temp, Final_Temp])" } ], "edges" : [ { - "source" : 12, - "target" : 13, + "source" : 26, + "target" : 27, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 13, - "target" : 14, + "source" : 27, + "target" : 28, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 14, - "target" : 15, + "source" : 28, + "target" : 29, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 15, - "target" : 16, + "source" : 29, + "target" : 30, "shuffle" : { "type" : "FORWARD" }, diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-first/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-first/savepoint/_metadata index 687d7898d87ff93cc5b607a9d776b1d014e47177..4ccb631c9ffecf160c3b8786be938da28161bfb2 100644 GIT binary patch delta 548 zcmbQ=%ebhQae{=vK|^JQgYgNwBt`!|(3+Tkb#jNZvH(lE){XQ_^V#1=@bG&TEx$fl zhpCEj(!_q7FcV`-BU2L-GfN{&BO?nla|=^r{o=&3)PnrXyb|3sGh@RvBVD89v{XZL z!_;I`i$r~MAlt;k+(OqhIoVRz#55^MH!&sIOxGgK)GWo)BGJOw(s;84bEBlS;e&m; zoB8DAZ~nH-hZk zlguUtTa)WPOQ5-i=EerPCW!_~x+W<`M!FWpsj0e_mZpX&=BX(rMoFfV6&U{r^h~bi zcq#EyWdFbDz#Nvv!JALYsIY)-4JAEP3=B-oj7%*pfnk*bwAI+uLf0ZW(MUJh+|(c` zCDqa_(R_2A;!1_dlFrI3>O$F?lO3FO*%+9C;h{QN&RJ~odnZna_#Y=(R-h<@7*JSp yva6GTK1ex}BT$G5h&e!Vz<}%liNa`5u)+l=w>UZRLu47iCNcj1zj=}KUnT(QV3^tf delta 540 zcmbVJO-NKx6n<|ULj+-9zvsYFFVH!9Xboocwy zrbWB!*})Yb!WBy{l^tDx;b$D z?X}hUuU8)gK@etK34=aiopTA~gfJpBCWuLms6Y|50uieGV7VGL{NwWQIEI8uM5$zq zMl=Jf%&7%8^K9zrJYdXFW`UH(A*cnX2v-tEI5!#rO0WggXal5n5+p8iN^=MkY5dRf z?^3!mT{`~L|8eBN-+gIMzBMM6F6+r9w}xs4oQ48WOC%72;k-%&=R_!O)5&nPtMumc z;qt8)WP11xl-mzmR?^d+!D7>P?7vS%iVzh=2*^lfW+;L{2_b;QSYczZMJnCv-|q9j z;-SjTS9rW|KYKtSV_U`y;a09*", "description" : "Match(partitionBy=[vehicle_id], orderBy=[proctime ASC], measures=[FINAL(FIRST(A.engine_temperature, 0)) AS Initial_Temp, FINAL(LAST(A.engine_temperature, 0)) AS Final_Temp], rowsPerMatch=[ONE ROW PER MATCH], after=[SKIP TO LAST _UTF-16LE'B'], pattern=[(PATTERN_QUANTIFIER(_UTF-16LE'A', 1, -1, false), _UTF-16LE'B')], define=[{A=OR(IS NULL(LAST(A.$1, 1)), >(PREV(A.$1, 0), LAST(A.$1, 1))), B=<(PREV(B.$1, 0), LAST(A.$1, 0))}])" }, { - "id" : 21, + "id" : 35, "type" : "stream-exec-sink_1", "configuration" : { "table.exec.sink.keyed-shuffle" : "AUTO", @@ -376,29 +376,29 @@ "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[vehicle_id, Initial_Temp, Final_Temp])" } ], "edges" : [ { - "source" : 17, - "target" : 18, + "source" : 31, + "target" : 32, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 18, - "target" : 19, + "source" : 32, + "target" : 33, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 19, - "target" : 20, + "source" : 33, + "target" : 34, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 20, - "target" : 21, + "source" : 34, + "target" : 35, "shuffle" : { "type" : "FORWARD" }, diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-last/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-last/savepoint/_metadata index d592b2403b8fe907ee1ee117d9a1c1ceac9a8cc0..ff7d8a1a6660be91b2ade4476966ad9f03688931 100644 GIT binary patch delta 589 zcmbQ=%ebhQae{=vlbjpE9zx&da0vapa697diiy!xjFToSDq4pd8XH*{SQ;5xn3@@z zSsI%e8k?Ex7bljb7UXBRKiz834H{X%=b8Cdp)s&$H2hY1D4cxmSqKT#ei~>lPjDA z^g+T*jzA$MAm#wc!4$)2R1>y4Icb3O0F{AFV*LLftOrEwRbQ?)v%`4#O8YNQ{0=eN zPW~e!Iaybh%i4wU?~!NkZnifU9;lftrECUN3o^BVkwL-4#K15uCD|m^EHyR7(!$)r z*xWGLz&zQ~*f7z^*w}2cva~h%F=?2Tm~3vCl&ou(lxPABRbx|NfTkMi8dw@y7?~s) RB$)th_#pjf^J?cWOaLQ*pgsTq delta 582 zcmbVI&r4KM7=3Rv8KZ$&S&7gvWkk$v?tAyQL-^in+b+n35!|^yKnN9#AW0MvL2$_7 zE&?%>QfL?Spq90=ZOazA2!#HE2r{udW4f;o&i7r;J$(1v^QskYw!*bb<+)!ke;obX z+HQULn%>^qdh3?ctK}CxedF2iw=V~~v)}ti9@W}6X2Z6tWQ~W2BN$=AA&p|ngb<&-<0yO32@j9loCF@rux6Aq*-+K^ZH>w2on{xPTGjO-h(; zO2iOE?jUWPIKKbyjwxf2rceWEBLlD`+GwD ziXOhg)ed_)J6WBPZYa%4C%3`5F72Ks!`IKY40O4U{4ctPVz*b_d@|sYq}bb$ySoV} M93SU9ca92w07ey|umAu6 diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-next-row/plan/skip-to-next-row.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-next-row/plan/skip-to-next-row.json index fb1cef7d713e7..3635610a8a053 100644 --- a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-next-row/plan/skip-to-next-row.json +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-next-row/plan/skip-to-next-row.json @@ -1,7 +1,7 @@ { "flinkVersion" : "1.19", "nodes" : [ { - "id" : 22, + "id" : 36, "type" : "stream-exec-table-source-scan_1", "scanTableSource" : { "table" : { @@ -42,7 +42,7 @@ "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[vehicle_id, engine_temperature])", "inputProperties" : [ ] }, { - "id" : 23, + "id" : 37, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", @@ -91,7 +91,7 @@ }, "description" : "Calc(select=[vehicle_id, engine_temperature, PROCTIME() AS proctime])" }, { - "id" : 24, + "id" : 38, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -121,7 +121,7 @@ }, "description" : "Exchange(distribution=[hash[vehicle_id]])" }, { - "id" : 25, + "id" : 39, "type" : "stream-exec-match_1", "matchSpec" : { "pattern" : { @@ -332,7 +332,7 @@ "outputType" : "ROW<`vehicle_id` BIGINT, `Initial_Temp` INT, `Final_Temp` INT>", "description" : "Match(partitionBy=[vehicle_id], orderBy=[proctime ASC], measures=[FINAL(FIRST(A.engine_temperature, 0)) AS Initial_Temp, FINAL(LAST(A.engine_temperature, 0)) AS Final_Temp], rowsPerMatch=[ONE ROW PER MATCH], after=[SKIP TO NEXT ROW], pattern=[(PATTERN_QUANTIFIER(_UTF-16LE'A', 1, -1, false), _UTF-16LE'B')], define=[{A=OR(IS NULL(LAST(A.$1, 1)), >(PREV(A.$1, 0), LAST(A.$1, 1))), B=<(PREV(B.$1, 0), LAST(A.$1, 0))}])" }, { - "id" : 26, + "id" : 40, "type" : "stream-exec-sink_1", "configuration" : { "table.exec.sink.keyed-shuffle" : "AUTO", @@ -374,29 +374,29 @@ "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[vehicle_id, Initial_Temp, Final_Temp])" } ], "edges" : [ { - "source" : 22, - "target" : 23, + "source" : 36, + "target" : 37, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 23, - "target" : 24, + "source" : 37, + "target" : 38, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 24, - "target" : 25, + "source" : 38, + "target" : 39, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 25, - "target" : 26, + "source" : 39, + "target" : 40, "shuffle" : { "type" : "FORWARD" }, diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-next-row/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-next-row/savepoint/_metadata index 901bfa05dfbdfe1d86b82c03fa9f4926d2c6b9a3..ec52102cfecb85f7aa1351ed7885e81f60d4c2b5 100644 GIT binary patch delta 580 zcmbVJziSjh6y9Cq`6CEu^jW+$!rvH5A%Cma) z>m(nn)k>9%d)LCM=Xt)v6W^;y00Wq{DPufUA_74XGDS`(QW}{olhEd#cj1e(qu}5L_U+V4_r|HEid_%q&N!R=joP{U9s4RI z1@DAoT(VEYQZe`MU-Nc2dorIcxn6m>A#{%-?jh zL)Bn3I-JSV<^IPqUtLp8eMNvYAh`rs*abn;Pq;rlpvrBpFzy zT3T96R$%-iuz^?F>*9Hq9gM8^Z|5^{R&0)EN?;+wLQ_L?Q{&`B6I}zNL}Oi(B;zDq z3v-iXUBkpQLsN5O!!&~=gUzDsD-qpa?5ic(R~|0w+S=N<((?UUfMnktgc9(*OU1)PjLQ%jWeTlm(W*TK?n5 z)~)XiCdV^LPVQym66l*PeLg$nmebx Date: Tue, 5 Dec 2023 08:59:36 -0500 Subject: [PATCH 4/4] Cleaning up test names. --- .../stream/MatchRecognizeRestoreTest.java | 16 ++++----- .../stream/MatchRecognizeTestPrograms.java | 32 +++++++++--------- .../plan/match-complex.json} | 0 .../savepoint/_metadata | Bin 21078 -> 21075 bytes .../plan/match-order-by-event-time.json} | 0 .../savepoint/_metadata | Bin 18898 -> 18895 bytes .../plan/match-order-by-int-column.json} | 0 .../savepoint/_metadata | Bin 18895 -> 18898 bytes .../plan/match-simple.json} | 0 .../savepoint/_metadata | Bin 16604 -> 16607 bytes .../plan/match-skip-past-last-row.json} | 0 .../savepoint/_metadata | Bin 17994 -> 17991 bytes .../plan/match-skip-to-first.json} | 0 .../savepoint/_metadata | Bin 18082 -> 18082 bytes .../plan/match-skip-to-last.json} | 0 .../savepoint/_metadata | Bin 18082 -> 18079 bytes .../plan/match-skip-to-next-row.json} | 0 .../savepoint/_metadata | Bin 18431 -> 18434 bytes 18 files changed, 24 insertions(+), 24 deletions(-) rename flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/{complex-match/plan/complex-match.json => match-complex/plan/match-complex.json} (100%) rename flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/{complex-match => match-complex}/savepoint/_metadata (86%) rename flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/{order-by-event-time-match/plan/order-by-event-time-match.json => match-order-by-event-time/plan/match-order-by-event-time.json} (100%) rename flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/{order-by-event-time-match => match-order-by-event-time}/savepoint/_metadata (79%) rename flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/{order-by-int-column-match/plan/order-by-int-column-match.json => match-order-by-int-column/plan/match-order-by-int-column.json} (100%) rename flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/{order-by-int-column-match => match-order-by-int-column}/savepoint/_metadata (79%) rename flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/{simple-match/plan/simple-match.json => match-simple/plan/match-simple.json} (100%) rename flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/{simple-match => match-simple}/savepoint/_metadata (76%) rename flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/{skip-past-last-row/plan/skip-past-last-row.json => match-skip-past-last-row/plan/match-skip-past-last-row.json} (100%) rename flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/{skip-past-last-row => match-skip-past-last-row}/savepoint/_metadata (87%) rename flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/{skip-to-first/plan/skip-to-first.json => match-skip-to-first/plan/match-skip-to-first.json} (100%) rename flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/{skip-to-first => match-skip-to-first}/savepoint/_metadata (95%) rename flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/{skip-to-last/plan/skip-to-last.json => match-skip-to-last/plan/match-skip-to-last.json} (100%) rename flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/{skip-to-last => match-skip-to-last}/savepoint/_metadata (85%) rename flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/{skip-to-next-row/plan/skip-to-next-row.json => match-skip-to-next-row/plan/match-skip-to-next-row.json} (100%) rename flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/{skip-to-next-row => match-skip-to-next-row}/savepoint/_metadata (86%) diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeRestoreTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeRestoreTest.java index c9ad1c85f34e9..5bfd8d57c460f 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeRestoreTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeRestoreTest.java @@ -34,13 +34,13 @@ public MatchRecognizeRestoreTest() { @Override public List programs() { return Arrays.asList( - MatchRecognizeTestPrograms.SIMPLE_MATCH, - MatchRecognizeTestPrograms.COMPLEX_MATCH, - MatchRecognizeTestPrograms.ORDER_BY_EVENT_TIME_MATCH, - MatchRecognizeTestPrograms.ORDER_BY_INT_COLUMN_MATCH, - MatchRecognizeTestPrograms.SKIP_TO_FIRST, - MatchRecognizeTestPrograms.SKIP_TO_LAST, - MatchRecognizeTestPrograms.SKIP_TO_NEXT_ROW, - MatchRecognizeTestPrograms.SKIP_PAST_LAST_ROW); + MatchRecognizeTestPrograms.MATCH_SIMPLE, + MatchRecognizeTestPrograms.MATCH_COMPLEX, + MatchRecognizeTestPrograms.MATCH_ORDER_BY_EVENT_TIME, + MatchRecognizeTestPrograms.MATCH_ORDER_BY_INT_COLUMN, + MatchRecognizeTestPrograms.MATCH_SKIP_TO_FIRST, + MatchRecognizeTestPrograms.MATCH_SKIP_TO_LAST, + MatchRecognizeTestPrograms.MATCH_SKIP_TO_NEXT_ROW, + MatchRecognizeTestPrograms.MATCH_SKIP_PAST_LAST_ROW); } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeTestPrograms.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeTestPrograms.java index cd9b81844d178..d805b720d53bb 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeTestPrograms.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeTestPrograms.java @@ -51,8 +51,8 @@ public class MatchRecognizeTestPrograms { static final Row[] COMPLEX_DATA2 = {Row.of("BETA", 7L, 22, 4)}; - static final TableTestProgram SIMPLE_MATCH = - TableTestProgram.of("simple-match", "simple match recognize test") + static final TableTestProgram MATCH_SIMPLE = + TableTestProgram.of("match-simple", "simple match recognize test") .setupTableSource( SourceTestStep.newBuilder("MyTable") .addSchema( @@ -83,8 +83,8 @@ public class MatchRecognizeTestPrograms { + " ) AS T") .build(); - static final TableTestProgram COMPLEX_MATCH = - TableTestProgram.of("complex-match", "complex match recognize test") + static final TableTestProgram MATCH_COMPLEX = + TableTestProgram.of("match-complex", "complex match recognize test") .setupTableSource( SourceTestStep.newBuilder("MyTable") .addSchema( @@ -157,8 +157,8 @@ public class MatchRecognizeTestPrograms { .producedAfterRestore(AFTER_DATA) .build(); - static final TableTestProgram ORDER_BY_EVENT_TIME_MATCH = - TableTestProgram.of("order-by-event-time-match", "complex match recognize test") + static final TableTestProgram MATCH_ORDER_BY_EVENT_TIME = + TableTestProgram.of("match-order-by-event-time", "complex match recognize test") .setupTableSource(SOURCE) .setupTableSink( SinkTestStep.newBuilder("MySink") @@ -169,8 +169,8 @@ public class MatchRecognizeTestPrograms { .runSql(getEventTimeSql("ORDER BY rowtime")) .build(); - static final TableTestProgram ORDER_BY_INT_COLUMN_MATCH = - TableTestProgram.of("order-by-int-column-match", "complex match recognize test") + static final TableTestProgram MATCH_ORDER_BY_INT_COLUMN = + TableTestProgram.of("match-order-by-int-column", "complex match recognize test") .setupTableSource(SOURCE) .setupTableSink( SinkTestStep.newBuilder("MySink") @@ -199,25 +199,25 @@ private static String getEventTimeSql(final String orderByClause) { return String.format(sql, orderByClause); } - static final TableTestProgram SKIP_TO_FIRST = + static final TableTestProgram MATCH_SKIP_TO_FIRST = getSkipTestProgram( - "skip-to-first", + "match-skip-to-first", "skip to first match recognize test", "AFTER MATCH SKIP TO FIRST B", new Row[] {Row.of(1L, 100, 106), Row.of(1L, 105, 107), Row.of(1L, 101, 101)}, new Row[] {Row.of(1L, 100, 111)}); - static final TableTestProgram SKIP_TO_LAST = + static final TableTestProgram MATCH_SKIP_TO_LAST = getSkipTestProgram( - "skip-to-last", + "match-skip-to-last", "skip to last match recognize test", "AFTER MATCH SKIP TO LAST B", new Row[] {Row.of(1L, 100, 106), Row.of(1L, 105, 107), Row.of(1L, 101, 101)}, new Row[] {Row.of(1L, 100, 111)}); - static final TableTestProgram SKIP_TO_NEXT_ROW = + static final TableTestProgram MATCH_SKIP_TO_NEXT_ROW = getSkipTestProgram( - "skip-to-next-row", + "match-skip-to-next-row", "skip to next row match recognize test", "AFTER MATCH SKIP TO NEXT ROW", new Row[] { @@ -231,9 +231,9 @@ private static String getEventTimeSql(final String orderByClause) { }, new Row[] {Row.of(1L, 100, 111), Row.of(1L, 110, 111), Row.of(1L, 111, 111)}); - static final TableTestProgram SKIP_PAST_LAST_ROW = + static final TableTestProgram MATCH_SKIP_PAST_LAST_ROW = getSkipTestProgram( - "skip-past-last-row", + "match-skip-past-last-row", "skip past last row match recognize test", "AFTER MATCH SKIP PAST LAST ROW", new Row[] {Row.of(1L, 100, 106), Row.of(1L, 107, 107)}, diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/complex-match/plan/complex-match.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/match-complex/plan/match-complex.json similarity index 100% rename from flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/complex-match/plan/complex-match.json rename to flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/match-complex/plan/match-complex.json diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/complex-match/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/match-complex/savepoint/_metadata similarity index 86% rename from flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/complex-match/savepoint/_metadata rename to flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/match-complex/savepoint/_metadata index c9a137f19f81a760aad20008091da0a9b427fa1f..981e6cc93500d0337c37dd18f093bba955efc2aa 100644 GIT binary patch delta 544 zcmbVJy-EW?5Z(>95q*M)7`Y(E;wHDdw|lo6A(dE(R$^hbx$J?B3Pzv6(n7L@*as{u z6w=tlHxLxTEy#MXDLLQ%Z`vT(@YG$yoV<=&yPIGc8(pyi$e|7VF&uQ zu+bgcbTS;yd(=b;rD)rLQgI3*&X2A;T!2hsqlK{|f6U;D-r;!#sFp?wt{^tdfJj*W VwUklFm~BUqiL`9TgGYY9_6_}QeQE#z delta 532 zcmbVIJxc^J5WUqEu@$tF<5W0{o!w-U%T5%mEd7Av)WRyen}CHM;AvrVU~MC*uCY3> zu@J9S@Gpp9XXPI;aZl0qh8c$U^4`plw?Xhe2p%@W;Kpu+YkLxDmbxTTWQa5+f+lC` zLZ6--9(SS43)7OInKUJ8Lo}C(&K)9>K%tZXPEmnkX#tGp#F)&9C=Nfu^QL`;rLZ|` z+V&t`w<2zM&~aeF&czMej+QIpC8}4l&!`cs*xPvC9z`=1br~(%5R#sEPHpV$Z`W$I zNmqfZQ3Th1hogM&j2PCv@&D_ NMapR|hxh4S{R>mWh9Cd{ diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/order-by-event-time-match/plan/order-by-event-time-match.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/match-order-by-event-time/plan/match-order-by-event-time.json similarity index 100% rename from flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/order-by-event-time-match/plan/order-by-event-time-match.json rename to flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/match-order-by-event-time/plan/match-order-by-event-time.json diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/order-by-event-time-match/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/match-order-by-event-time/savepoint/_metadata similarity index 79% rename from flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/order-by-event-time-match/savepoint/_metadata rename to flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/match-order-by-event-time/savepoint/_metadata index 4251befd8513b258ff2d0828f77729f4566efd3c..e0731512398134d02025cacffca126bf62125ce1 100644 GIT binary patch delta 332 zcmbV{IZgyI6hyH^&lwOCA+n)A(AR_@P9f0%@6#!Np2qB+Y1Iu_0^k&9r~~w)_WN Cs$F9M delta 339 zcmbV`J4!@B5C9!;H!&4dOa!xa+wGp7FP>r}PXX#t!s1xlWj0v9H67=s)4PG1lI E0Bf;c5dZ)H diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/order-by-int-column-match/plan/order-by-int-column-match.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/match-order-by-int-column/plan/match-order-by-int-column.json similarity index 100% rename from flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/order-by-int-column-match/plan/order-by-int-column-match.json rename to flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/match-order-by-int-column/plan/match-order-by-int-column.json diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/order-by-int-column-match/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/match-order-by-int-column/savepoint/_metadata similarity index 79% rename from flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/order-by-int-column-match/savepoint/_metadata rename to flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/match-order-by-int-column/savepoint/_metadata index 772bf3fad7c9011d83b32865cca1957f6e4b624e..5f449fc93bc4a070c56e996385221ba8f09c614f 100644 GIT binary patch delta 332 zcmbWwJxW9|5CC9RN;ek~Uf}^efS^YZ z@1Tw78TPxt_twmNGmqb9{q%Eq8ay$J29n?`J5qIv>vUHyFV0stkZn?OV5uUN5H&ek z(5Nx`EKmka;58{^jR+3W6^lY>!}MTYW}5~=A#f0aGExRVP=QFPfJV>0O0CY0)5qR@ zw`nj$k$}vQ6a~B;Mo7gn44g~B%t)b3U;DoY?Wn(MXUo_2d2(xZ+HpTyFT34t>z|pP H`}y`Czms2j delta 323 zcmbWvJxYT?5CBkC$sIxp5hUf1?9S}&45ZZW2BPikAK0i6O>J+Hggwe9cmNL|2;RXX zXrtaC?*Q+koA+*Z-)6u6SzU_tmO_ZcOY$P@<6^CwdVhCw`vBESV-F~;kcPAjkr%4j z*P0uUO#)PE17^)27&5XISF?`K=5>1DUo8=mBIHyH5Ty!|dd;xo7)l$zPJhq(Y5A^EJ?gk2C1B&;8H$8|s_#Xy;L_7-q zhbNOJ2hSat_lC#3_r5pmXTx4T*lV?#onIo?h+1)h7;~9usf<|M*@sZBSGV`zcO+4>l6uJr(?dqdj)A57bX-yQV5Vr$3TS*V2lYWNK^^>`ge9Rt=_Zg@N9Xou0FH0 zJ<94Z@0EF;h*sIViK1xGZ$s^{=hoD7$^MJgTYg%Nvq=|gvWqVEF`G|@&EL^e1EbEc M#cfnsf90dWA9;0fCjbBd delta 466 zcmbV{&q~8U5XO_XLcyB{ky?p(@w9e#Xa8z)^3W&Hv)N4+L{UKzJoc=Jl0)0)2tJD! z!6y-%mIs`Kw< Ud)otSHy<&r{-Is2-?L%=8(Gb4+W-In diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-past-last-row/plan/skip-past-last-row.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/match-skip-past-last-row/plan/match-skip-past-last-row.json similarity index 100% rename from flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-past-last-row/plan/skip-past-last-row.json rename to flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/match-skip-past-last-row/plan/match-skip-past-last-row.json diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-past-last-row/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/match-skip-past-last-row/savepoint/_metadata similarity index 87% rename from flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-past-last-row/savepoint/_metadata rename to flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/match-skip-past-last-row/savepoint/_metadata index 2ba80159c539619a525b1bd9df7188262ad92387..9aa932e98ddb4b00fdc190029974c8731195e07a 100644 GIT binary patch delta 397 zcmbV`J4ysW5Qe9B6(%+?ligKA%jx$+HnA5>Of?Zjx_i38WEB(-U_7WxUqa(Kgy1PW zh=DVs9zY9zs_F|q{`UsGHR$;ftyWL-Q-Z1CR1tA1aRjAIE6i6 zQ&S~K!n$tH$IGpM*4M=7M3@4lJn8E*FcL>_Qc&uJ)J)3uV)io4XLfh(;ArC`KiK1{ xKSqZ|QH;8jUAjR{Pk%QW*bmH~c4xgO+xgON=BX{eReN?nPZpEmY3-sMe*tkWX{`VN delta 402 zcmbV{y-LJT5QTGh6&6b?3&mX(1YykmnfntE#H6&=V!L~jxnL6o#it1VJxffSV%%7_)97A0ha9*he{pqD)AvYWFHHWpm{y?`naGhryG(J$boYs5)eITHf(FX~NBm&w@LvYf9wa$^NxE2<><>Gz5$&w^FXcxuKRxc9P y9Bnq8^|@g-b2psv0ded?QXF@26X4eZ2AL<`e{x8 diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-first/plan/skip-to-first.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/match-skip-to-first/plan/match-skip-to-first.json similarity index 100% rename from flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-first/plan/skip-to-first.json rename to flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/match-skip-to-first/plan/match-skip-to-first.json diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-first/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/match-skip-to-first/savepoint/_metadata similarity index 95% rename from flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/skip-to-first/savepoint/_metadata rename to flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-match_1/match-skip-to-first/savepoint/_metadata index 4ccb631c9ffecf160c3b8786be938da28161bfb2..b555d95de7fdc907313a000651fda20d3e7caebc 100644 GIT binary patch delta 377 zcmbV|J4ysW5Qe8$h0O#*4OG^&oUX2ZRGT>iCZ=L$(9=DG20lR13mCdlXNDUYio;&U zV|W5*6v224e*Q=B1+|!q#az5^iF0J_5xrza>zE{yD(}N%dv|+v^8i|-3<_*V34KzU zh)QzUBS75VmoI|C+JBpyaLhwyw@D$MoOlVr} zzjm%SR$$v0Trw_zGR}gg6u?`_;GAs6FR)3WU+%x{<&WxM{pe(~%AeI~Iflt`Q52Iw j(?P?DmvQ}b@qf%`^(5Jjg~g^f4Vz{d)UK{+$kUEN)wGm&g=X3(!e5g(xF7nr%xWQH3V3WGo9 z4`{C<7;nL)Zk^yjEf-?B5bqOlrX(XNr5PE~YGX;3_wF&>-Co{2K&NHhAz?`~Rh?3QG~28R#~RX!%x&fxH||F+j3wA0PmY6x=(BDC9sJK?o#h{mkv#Z!L0{B9*meU@vh_) ziIh=S8xO)&8n~3;UYbJ5*T2uWt=ql25UEJl2DrDZIXM(jnz zS$qC=w2_T9bjs7#ia80d#Zmj8bXlB*LD|h#0a0j0kTl$u-&=p@YHc0o1}0b`aOyl* T^d1=26Qz{2mE+CV@Mq^2OU7&l delta 469 zcmbVHOG*Pl5S>YgC^-1p1;mIVx@ps&>YiytoXJ8E58zJePX+>N1jUU<(11Ei7eZF< zT#Q%n8lJ%;Sd)z4It^94SN*CU?`;&0N8!^rd@0s@`{zvXglf(bsf5x(F`?q_jk)Vz z4+mGb1RO{U#PVENPfeUF*#&K!W!@4CjuY@)6YVTT>hmOb;FPC9&>jXsF!y}8R=!4J zv3!WmR;HGxvsgONkO)ISa%|xGBwW1UR0uMuxC~H9t|-)4$*PLcREvZWRYvC#oz_-KsXi lVvv2#9-CR?VrCst)&OP~5@Yunp_(NeEVUq1`O*7n{sK?9M0ii(hG|gB`PCEH#BXouP!ccAy9}$ z0-@mn11c`m$k{qHC>?08IABx~Jk~83uThh)l_cHovskRUAd=V4kp+Vx0{Y0o)o4JV z5Ii=Gldb8u*PiwYmSx#N_g?J`vXrv>a%(c}o~x7Yr5tYjbw0{7{H^lSM9Pz|Cr|dx z`iTklPl~F;?yVYbE_KtOSmN`pIC@?6G?F@4;{=ex0O^SYL9_~NFpbrcI0ot`$B(;T Dfmv!m