Skip to content

Commit

Permalink
add planner tests
Browse files Browse the repository at this point in the history
Signed-off-by: Richard Chien <[email protected]>
  • Loading branch information
stdrc committed Jul 18, 2024
1 parent 88ff57b commit 2a52433
Show file tree
Hide file tree
Showing 2 changed files with 57 additions and 19 deletions.
50 changes: 31 additions & 19 deletions src/frontend/planner_test/tests/testdata/input/temporal_filter.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -3,96 +3,96 @@
create table t1 (ts timestamp with time zone);
select * from t1 where ts + interval '1 hour' > now();
expected_outputs:
- stream_plan
- stream_plan
- name: Temporal filter works on complex columns on LHS (part 2)
sql: |
create table t1 (ts timestamp with time zone, time_to_live interval);
select * from t1 where ts + time_to_live * 1.5 > now();
expected_outputs:
- stream_plan
- stream_plan
- name: Temporal filter works on complex columns on LHS (part 2, flipped)
sql: |
create table t1 (ts timestamp with time zone, additional_time_to_live interval);
select * from t1 where now() - interval '15 minutes' < ts + additional_time_to_live * 1.5;
expected_outputs:
- stream_plan
- stream_plan
- name: Temporal filter with `now()` in upper bound
sql: |-
create table t1 (ts timestamp with time zone);
select * from t1 where now() - interval '15 minutes' > ts;
expected_outputs:
- stream_plan
- stream_dist_plan
- stream_plan
- stream_dist_plan
- name: Temporal filter with equal condition
sql: |-
create table t1 (ts timestamp with time zone);
select * from t1 where date_trunc('week', now()) = date_trunc('week',ts);
expected_outputs:
- stream_plan
- stream_dist_plan
- stream_plan
- stream_dist_plan
- name: Temporal filter with `now()` in upper bound on append only table
sql: |-
create table t1 (ts timestamp with time zone) APPEND ONLY;
select * from t1 where now() - interval '15 minutes' > ts;
expected_outputs:
- stream_plan
- stream_dist_plan
- stream_plan
- stream_dist_plan
- name: Temporal filter reorders now expressions correctly
sql: |
create table t1 (ts timestamp with time zone);
select * from t1 where ts < now() - interval '1 hour' and ts >= now() - interval '2 hour';
expected_outputs:
- stream_plan
- stream_dist_plan
- stream_plan
- stream_dist_plan
- name: Temporal filter in on clause for inner join's left side
sql: |
create table t1 (a int, ta timestamp with time zone);
create table t2 (b int, tb timestamp with time zone);
select * from t1 join t2 on a = b AND ta < now() - interval '1 hour' and ta >= now() - interval '2 hour';
expected_outputs:
- stream_plan
- stream_plan
- name: Temporal filter in on clause for left join's left side
sql: |
create table t1 (a int, ta timestamp with time zone);
create table t2 (b int, tb timestamp with time zone);
select * from t1 left join t2 on a = b AND ta < now() - interval '1 hour' and ta >= now() - interval '2 hour';
expected_outputs:
- stream_error
- stream_error
- name: Temporal filter in on clause for right join's left side
sql: |
create table t1 (a int, ta timestamp with time zone);
create table t2 (b int, tb timestamp with time zone);
select * from t1 right join t2 on a = b AND ta < now() - interval '1 hour' and ta >= now() - interval '2 hour';
expected_outputs:
- stream_plan
- stream_plan
- name: Temporal filter in on clause for full join's left side
sql: |
create table t1 (a int, ta timestamp with time zone);
create table t2 (b int, tb timestamp with time zone);
select * from t1 full join t2 on a = b AND ta < now() - interval '1 hour' and ta >= now() - interval '2 hour';
expected_outputs:
- stream_error
- stream_error
- name: Temporal filter in on clause for left join's right side
sql: |
create table t1 (a int, ta timestamp with time zone);
create table t2 (b int, tb timestamp with time zone);
select * from t1 left join t2 on a = b AND tb < now() - interval '1 hour' and tb >= now() - interval '2 hour';
expected_outputs:
- stream_plan
- stream_plan
- name: Temporal filter in on clause for right join's right side
sql: |
create table t1 (a int, ta timestamp with time zone);
create table t2 (b int, tb timestamp with time zone);
select * from t1 right join t2 on a = b AND tb < now() - interval '1 hour' and tb >= now() - interval '2 hour';
expected_outputs:
- stream_error
- stream_error
- name: Temporal filter after temporal join
sql: |
create table stream(id1 int, a1 int, b1 int, v1 timestamp with time zone) APPEND ONLY;
create table version(id2 int, a2 int, b2 int, primary key (id2));
select id1, a1, id2, v1 from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on id1 = id2 where v1 > now();
expected_outputs:
- stream_plan
- stream_plan
- name: Temporal filter with or predicate
sql: |
create table t1 (ts timestamp with time zone);
Expand All @@ -116,4 +116,16 @@
create table t (t timestamp with time zone, a int);
select * from t where (t > NOW() - INTERVAL '1 hour' OR t is NULL OR a < 1) AND (t < NOW() - INTERVAL '1 hour' OR a > 1);
expected_outputs:
- stream_plan
- stream_plan
- name: Non-trivial now expression
sql: |
create table t (ts timestamp with time zone, a int);
select * from t where ts + interval '1 hour' > date_trunc('day', now());
expected_outputs:
- stream_plan
- name: Non-trivial now expression 2
sql: |
create table t (ts timestamp with time zone, a int);
select * from t where ts + interval '1 hour' > date_trunc('day', ('2024-07-18 00:00:00+00:00'::timestamptz - ('2024-07-18 00:00:00+00:00'::timestamptz - now())));
expected_outputs:
- stream_plan
Original file line number Diff line number Diff line change
Expand Up @@ -460,3 +460,29 @@
└─StreamShare { id: 2 }
└─StreamFilter { predicate: IsNotNull(t.a) AND (((Not(IsNull(t.t)) AND Not((t.a < 1:Int32))) OR IsNull(t.t)) OR (t.a < 1:Int32)) }
└─StreamTableScan { table: t, columns: [t.t, t.a, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) }
- name: Non-trivial now expression
sql: |
create table t (ts timestamp with time zone, a int);
select * from t where ts + interval '1 hour' > date_trunc('day', now());
stream_plan: |-
StreamMaterialize { columns: [ts, a, t._row_id(hidden)], stream_key: [t._row_id], pk_columns: [t._row_id], pk_conflict: NoCheck }
└─StreamProject { exprs: [t.ts, t.a, t._row_id] }
└─StreamDynamicFilter { predicate: ($expr1 > $expr2), output_watermarks: [$expr1], output: [t.ts, t.a, $expr1, t._row_id], cleaned_by_watermark: true }
├─StreamProject { exprs: [t.ts, t.a, AddWithTimeZone(t.ts, '01:00:00':Interval, 'UTC':Varchar) as $expr1, t._row_id] }
│ └─StreamTableScan { table: t, columns: [t.ts, t.a, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) }
└─StreamExchange { dist: Broadcast }
└─StreamProject { exprs: [DateTrunc('day':Varchar, now, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] }
└─StreamNow { output: [now] }
- name: Non-trivial now expression 2
sql: |
create table t (ts timestamp with time zone, a int);
select * from t where ts + interval '1 hour' > date_trunc('day', ('2024-07-18 00:00:00+00:00'::timestamptz - ('2024-07-18 00:00:00+00:00'::timestamptz - now())));
stream_plan: |-
StreamMaterialize { columns: [ts, a, t._row_id(hidden)], stream_key: [t._row_id], pk_columns: [t._row_id], pk_conflict: NoCheck }
└─StreamProject { exprs: [t.ts, t.a, t._row_id] }
└─StreamDynamicFilter { predicate: ($expr1 > $expr2), output: [t.ts, t.a, $expr1, t._row_id] }
├─StreamProject { exprs: [t.ts, t.a, AddWithTimeZone(t.ts, '01:00:00':Interval, 'UTC':Varchar) as $expr1, t._row_id] }
│ └─StreamTableScan { table: t, columns: [t.ts, t.a, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) }
└─StreamExchange { dist: Broadcast }
└─StreamProject { exprs: [DateTrunc('day':Varchar, SubtractWithTimeZone('2024-07-18 00:00:00+00:00':Timestamptz, ('2024-07-18 00:00:00+00:00':Timestamptz - now), 'UTC':Varchar), 'UTC':Varchar) as $expr2] }
└─StreamNow { output: [now] }

0 comments on commit 2a52433

Please sign in to comment.