|
| 1 | +# This file is automatically generated. See `src/frontend/planner_test/README.md` for more information. |
| 2 | +- name: Left join type for temporal join |
| 3 | + sql: | |
| 4 | + create table stream(id1 int, a1 int, b1 int) APPEND ONLY; |
| 5 | + create table version(id2 int, a2 int, b2 int, primary key (id2)); |
| 6 | + select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF NOW() on id1= id2 |
| 7 | + stream_plan: | |
| 8 | + StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden)], pk_columns: [stream._row_id, id2, id1], pk_conflict: "no check" } |
| 9 | + └─StreamTemporalJoin { type: LeftOuter, predicate: stream.id1 = version.id2, output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } |
| 10 | + ├─StreamExchange { dist: HashShard(stream.id1) } |
| 11 | + | └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } |
| 12 | + └─StreamNoShuffleExchange { dist: UpstreamHashShard(version.id2) } |
| 13 | + └─StreamTableScan { table: version, columns: [version.id2, version.a2], pk: [version.id2], dist: UpstreamHashShard(version.id2) } |
| 14 | + batch_error: |- |
| 15 | + Not supported: do not support temporal join for batch queries |
| 16 | + HINT: please use temporal join in streaming queries |
| 17 | +- name: Inner join type for temporal join |
| 18 | + sql: | |
| 19 | + create table stream(id1 int, a1 int, b1 int) APPEND ONLY; |
| 20 | + create table version(id2 int, a2 int, b2 int, primary key (id2)); |
| 21 | + select id1, a1, id2, a2 from stream join version FOR SYSTEM_TIME AS OF NOW() on id1 = id2 where a2 < 10; |
| 22 | + stream_plan: | |
| 23 | + StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden)], pk_columns: [stream._row_id, id2, id1], pk_conflict: "no check" } |
| 24 | + └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version.id2 AND (version.a2 < 10:Int32), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } |
| 25 | + ├─StreamExchange { dist: HashShard(stream.id1) } |
| 26 | + | └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } |
| 27 | + └─StreamNoShuffleExchange { dist: UpstreamHashShard(version.id2) } |
| 28 | + └─StreamTableScan { table: version, columns: [version.id2, version.a2], pk: [version.id2], dist: UpstreamHashShard(version.id2) } |
| 29 | +- name: implicit join with temporal tables |
| 30 | + sql: | |
| 31 | + create table stream(id1 int, a1 int, b1 int) APPEND ONLY; |
| 32 | + create table version(id2 int, a2 int, b2 int, primary key (id2)); |
| 33 | + select id1, a1, id2, a2 from stream, version FOR SYSTEM_TIME AS OF NOW() where id1 = id2 AND a2 < 10; |
| 34 | + stream_plan: | |
| 35 | + StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden)], pk_columns: [stream._row_id, id2, id1], pk_conflict: "no check" } |
| 36 | + └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version.id2 AND (version.a2 < 10:Int32), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } |
| 37 | + ├─StreamExchange { dist: HashShard(stream.id1) } |
| 38 | + | └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } |
| 39 | + └─StreamNoShuffleExchange { dist: UpstreamHashShard(version.id2) } |
| 40 | + └─StreamTableScan { table: version, columns: [version.id2, version.a2], pk: [version.id2], dist: UpstreamHashShard(version.id2) } |
| 41 | +- name: Multi join key for temporal join |
| 42 | + sql: | |
| 43 | + create table stream(id1 int, a1 int, b1 int) APPEND ONLY; |
| 44 | + create table version(id2 int, a2 int, b2 int, primary key (id2, a2)); |
| 45 | + select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF NOW() on a1 = a2 and id1 = id2 where b2 != a2; |
| 46 | + stream_plan: | |
| 47 | + StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden)], pk_columns: [stream._row_id, id2, a2, id1, a1], pk_conflict: "no check" } |
| 48 | + └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version.id2 AND stream.a1 = version.a2 AND (version.b2 <> version.a2), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } |
| 49 | + ├─StreamExchange { dist: HashShard(stream.id1, stream.a1) } |
| 50 | + | └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } |
| 51 | + └─StreamNoShuffleExchange { dist: UpstreamHashShard(version.id2, version.a2) } |
| 52 | + └─StreamTableScan { table: version, columns: [version.id2, version.a2, version.b2], pk: [version.id2, version.a2], dist: UpstreamHashShard(version.id2, version.a2) } |
| 53 | +- name: Temporal join with Aggregation |
| 54 | + sql: | |
| 55 | + create table stream(id1 int, a1 int, b1 int) APPEND ONLY; |
| 56 | + create table version(id2 int, a2 int, b2 int, primary key (id2)); |
| 57 | + select count(*) from stream left join version FOR SYSTEM_TIME AS OF NOW() on id1 = id2 where a2 < 10; |
| 58 | + stream_plan: | |
| 59 | + StreamMaterialize { columns: [count], pk_columns: [], pk_conflict: "no check" } |
| 60 | + └─StreamProject { exprs: [sum0(count)] } |
| 61 | + └─StreamAppendOnlyGlobalSimpleAgg { aggs: [sum0(count), count] } |
| 62 | + └─StreamExchange { dist: Single } |
| 63 | + └─StreamStatelessLocalSimpleAgg { aggs: [count] } |
| 64 | + └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version.id2 AND (version.a2 < 10:Int32), output: [stream._row_id, stream.id1, version.id2] } |
| 65 | + ├─StreamExchange { dist: HashShard(stream.id1) } |
| 66 | + | └─StreamTableScan { table: stream, columns: [stream.id1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } |
| 67 | + └─StreamNoShuffleExchange { dist: UpstreamHashShard(version.id2) } |
| 68 | + └─StreamTableScan { table: version, columns: [version.id2, version.a2], pk: [version.id2], dist: UpstreamHashShard(version.id2) } |
| 69 | +- name: Temporal join join keys requirement test |
| 70 | + sql: | |
| 71 | + create table stream(id1 int, a1 int, b1 int) APPEND ONLY; |
| 72 | + create table version(id2 int, a2 int, b2 int, primary key (id2, a2)); |
| 73 | + select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF NOW() on id1 = id2 where a2 < 10; |
| 74 | + stream_error: |- |
| 75 | + Not supported: Temporal join requires the lookup table's primary key contained exactly in the equivalence condition |
| 76 | + HINT: Please add the primary key of the lookup table to the join condition and remove any other conditions |
| 77 | +- name: Temporal join append only test |
| 78 | + sql: | |
| 79 | + create table stream(id1 int, a1 int, b1 int); |
| 80 | + create table version(id2 int, a2 int, b2 int, primary key (id2)); |
| 81 | + select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF NOW() on id1 = id2 where a2 < 10; |
| 82 | + stream_error: |- |
| 83 | + Not supported: Temporal join requires a append-only left input |
| 84 | + HINT: Please ensure your left input is append-only |
| 85 | +- name: Temporal join type test |
| 86 | + sql: | |
| 87 | + create table stream(id1 int, a1 int, b1 int); |
| 88 | + create table version(id2 int, a2 int, b2 int, primary key (id2)); |
| 89 | + select id1, a1, id2, a2 from stream right join version FOR SYSTEM_TIME AS OF NOW() on id1 = id2 where a2 < 10; |
| 90 | + stream_error: |- |
| 91 | + Not supported: exist dangling temporal scan |
| 92 | + HINT: please check your temporal join syntax e.g. consider removing the right outer join if it is being used. |
| 93 | +- name: multi-way temporal join with the same key |
| 94 | + sql: | |
| 95 | + create table stream(k int, a1 int, b1 int) APPEND ONLY; |
| 96 | + create table version1(k int, x1 int, y2 int, primary key (k)); |
| 97 | + create table version2(k int, x2 int, y2 int, primary key (k)); |
| 98 | + select stream.k, x1, x2, a1, b1 |
| 99 | + from stream |
| 100 | + join version1 FOR SYSTEM_TIME AS OF NOW() on stream.k = version1.k |
| 101 | + join version2 FOR SYSTEM_TIME AS OF NOW() on stream.k = version2.k where a1 < 10; |
| 102 | + stream_plan: | |
| 103 | + StreamMaterialize { columns: [k, x1, x2, a1, b1, stream._row_id(hidden), version1.k(hidden), version2.k(hidden)], pk_columns: [stream._row_id, version1.k, k, version2.k], pk_conflict: "no check" } |
| 104 | + └─StreamTemporalJoin { type: Inner, predicate: stream.k = version2.k, output: [stream.k, version1.x1, version2.x2, stream.a1, stream.b1, stream._row_id, version1.k, version2.k] } |
| 105 | + ├─StreamTemporalJoin { type: Inner, predicate: stream.k = version1.k, output: [stream.k, stream.a1, stream.b1, version1.x1, stream._row_id, version1.k] } |
| 106 | + | ├─StreamExchange { dist: HashShard(stream.k) } |
| 107 | + | | └─StreamFilter { predicate: (stream.a1 < 10:Int32) } |
| 108 | + | | └─StreamTableScan { table: stream, columns: [stream.k, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } |
| 109 | + | └─StreamNoShuffleExchange { dist: UpstreamHashShard(version1.k) } |
| 110 | + | └─StreamTableScan { table: version1, columns: [version1.k, version1.x1], pk: [version1.k], dist: UpstreamHashShard(version1.k) } |
| 111 | + └─StreamNoShuffleExchange { dist: UpstreamHashShard(version2.k) } |
| 112 | + └─StreamTableScan { table: version2, columns: [version2.k, version2.x2], pk: [version2.k], dist: UpstreamHashShard(version2.k) } |
| 113 | +- name: multi-way temporal join with different keys |
| 114 | + sql: | |
| 115 | + create table stream(id1 int, id2 int, a1 int, b1 int) APPEND ONLY; |
| 116 | + create table version1(id1 int, x1 int, y2 int, primary key (id1)); |
| 117 | + create table version2(id2 int, x2 int, y2 int, primary key (id2)); |
| 118 | + select stream.id1, x1, stream.id2, x2, a1, b1 |
| 119 | + from stream |
| 120 | + join version1 FOR SYSTEM_TIME AS OF NOW() on stream.id1 = version1.id1 |
| 121 | + join version2 FOR SYSTEM_TIME AS OF NOW() on stream.id2 = version2.id2 where a1 < 10; |
| 122 | + stream_plan: | |
| 123 | + StreamMaterialize { columns: [id1, x1, id2, x2, a1, b1, stream._row_id(hidden), version1.id1(hidden), version2.id2(hidden)], pk_columns: [stream._row_id, version1.id1, id1, version2.id2, id2], pk_conflict: "no check" } |
| 124 | + └─StreamTemporalJoin { type: Inner, predicate: stream.id2 = version2.id2, output: [stream.id1, version1.x1, stream.id2, version2.x2, stream.a1, stream.b1, stream._row_id, version1.id1, version2.id2] } |
| 125 | + ├─StreamExchange { dist: HashShard(stream.id2) } |
| 126 | + | └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version1.id1, output: [stream.id1, stream.id2, stream.a1, stream.b1, version1.x1, stream._row_id, version1.id1] } |
| 127 | + | ├─StreamExchange { dist: HashShard(stream.id1) } |
| 128 | + | | └─StreamFilter { predicate: (stream.a1 < 10:Int32) } |
| 129 | + | | └─StreamTableScan { table: stream, columns: [stream.id1, stream.id2, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } |
| 130 | + | └─StreamNoShuffleExchange { dist: UpstreamHashShard(version1.id1) } |
| 131 | + | └─StreamTableScan { table: version1, columns: [version1.id1, version1.x1], pk: [version1.id1], dist: UpstreamHashShard(version1.id1) } |
| 132 | + └─StreamNoShuffleExchange { dist: UpstreamHashShard(version2.id2) } |
| 133 | + └─StreamTableScan { table: version2, columns: [version2.id2, version2.x2], pk: [version2.id2], dist: UpstreamHashShard(version2.id2) } |
| 134 | +- name: multi-way temporal join with different keys |
| 135 | + sql: | |
| 136 | + create table stream(id1 int, id2 int, a1 int, b1 int) APPEND ONLY; |
| 137 | + create table version1(id1 int, x1 int, y2 int, primary key (id1)); |
| 138 | + create table version2(id2 int, x2 int, y2 int, primary key (id2)); |
| 139 | + select stream.id1, x1, stream.id2, x2, a1, b1 |
| 140 | + from stream |
| 141 | + join version1 FOR SYSTEM_TIME AS OF NOW() on stream.id1 = version1.id1 |
| 142 | + join version2 FOR SYSTEM_TIME AS OF NOW() on stream.id2 = version2.id2 where a1 < 10; |
| 143 | + stream_plan: | |
| 144 | + StreamMaterialize { columns: [id1, x1, id2, x2, a1, b1, stream._row_id(hidden), version1.id1(hidden), version2.id2(hidden)], pk_columns: [stream._row_id, version1.id1, id1, version2.id2, id2], pk_conflict: "no check" } |
| 145 | + └─StreamTemporalJoin { type: Inner, predicate: stream.id2 = version2.id2, output: [stream.id1, version1.x1, stream.id2, version2.x2, stream.a1, stream.b1, stream._row_id, version1.id1, version2.id2] } |
| 146 | + ├─StreamExchange { dist: HashShard(stream.id2) } |
| 147 | + | └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version1.id1, output: [stream.id1, stream.id2, stream.a1, stream.b1, version1.x1, stream._row_id, version1.id1] } |
| 148 | + | ├─StreamExchange { dist: HashShard(stream.id1) } |
| 149 | + | | └─StreamFilter { predicate: (stream.a1 < 10:Int32) } |
| 150 | + | | └─StreamTableScan { table: stream, columns: [stream.id1, stream.id2, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } |
| 151 | + | └─StreamNoShuffleExchange { dist: UpstreamHashShard(version1.id1) } |
| 152 | + | └─StreamTableScan { table: version1, columns: [version1.id1, version1.x1], pk: [version1.id1], dist: UpstreamHashShard(version1.id1) } |
| 153 | + └─StreamNoShuffleExchange { dist: UpstreamHashShard(version2.id2) } |
| 154 | + └─StreamTableScan { table: version2, columns: [version2.id2, version2.x2], pk: [version2.id2], dist: UpstreamHashShard(version2.id2) } |
0 commit comments