-
Notifications
You must be signed in to change notification settings - Fork 590
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
feat(expr): streaming generate_series
ends with now()
#17371
Changes from 10 commits
da4d048
2dcf92a
60a7371
ae7d3c1
5f94b1a
e355196
26e8634
556c0aa
78bbaac
48214e8
ee26940
e63f93a
092c695
8b77020
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,27 @@ | ||
system ok | ||
psql -h $RISEDEV_RW_FRONTEND_LISTEN_ADDRESS -p $RISEDEV_RW_FRONTEND_PORT -U root -d dev -c " | ||
create materialized view mv as | ||
select * from generate_series( | ||
to_timestamp($(date +%s) - 10), | ||
now(), | ||
interval '1 second' | ||
); | ||
" | ||
|
||
statement ok | ||
flush; | ||
|
||
query I | ||
select count(*) > 10 from mv; | ||
---- | ||
t | ||
|
||
sleep 2s | ||
|
||
query I | ||
select count(*) > 12 from mv; | ||
---- | ||
t | ||
|
||
statement ok | ||
drop materialized view mv; |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,30 @@ | ||
- sql: | | ||
select * from generate_series( | ||
'2024-06-21 17:36:00'::timestamptz, | ||
now(), | ||
interval '1 hour' | ||
); | ||
expected_outputs: | ||
- logical_plan | ||
- optimized_logical_plan_for_stream | ||
- stream_plan | ||
- sql: | | ||
select * from generate_series( | ||
'2024-06-21 17:36:00'::timestamp, -- `timestamp` type is not supported | ||
now(), | ||
interval '1 hour' | ||
); | ||
expected_outputs: | ||
- binder_error | ||
- sql: | | ||
select * from generate_series( | ||
now() - interval '1 hour', | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Interesting. Shouldn't this be able to fold into constant? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Then the There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Maybe we need something like |
||
now(), | ||
interval '1 hour' | ||
); | ||
expected_outputs: | ||
- stream_error | ||
- sql: | | ||
select * from unnest(array[now(), now()]); | ||
expected_outputs: | ||
- stream_error |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,35 @@ | ||
# This file is automatically generated. See `src/frontend/planner_test/README.md` for more information. | ||
- sql: | | ||
select * from generate_series( | ||
'2024-06-21 17:36:00'::timestamptz, | ||
now(), | ||
interval '1 hour' | ||
); | ||
logical_plan: |- | ||
LogicalProject { exprs: [generate_series] } | ||
└─LogicalTableFunction { table_function: GenerateSeries('2024-06-21 17:36:00':Varchar::Timestamptz, Now, '01:00:00':Interval) } | ||
optimized_logical_plan_for_stream: 'LogicalNow { output: [ts] }' | ||
stream_plan: |- | ||
StreamMaterialize { columns: [generate_series], stream_key: [generate_series], pk_columns: [generate_series], pk_conflict: NoCheck, watermark_columns: [generate_series] } | ||
└─StreamNow { output: [ts] } | ||
- sql: | | ||
select * from generate_series( | ||
'2024-06-21 17:36:00'::timestamp, -- `timestamp` type is not supported | ||
now(), | ||
interval '1 hour' | ||
); | ||
binder_error: function generate_series(timestamp without time zone, timestamp with time zone, interval) does not exist | ||
- sql: | | ||
select * from generate_series( | ||
now() - interval '1 hour', | ||
now(), | ||
interval '1 hour' | ||
); | ||
stream_error: |- | ||
Not supported: General `now()` function | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The message does not seem to be good enough. 😕 There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Any suggestions? |
||
HINT: If you are trying to use `generate_series` with `now()`, please kindly check whether you are using it in a correct way. | ||
- sql: | | ||
select * from unnest(array[now(), now()]); | ||
stream_error: |- | ||
Not supported: General `now()` function | ||
HINT: If you are trying to use `generate_series` with `now()`, please kindly check whether you are using it in a correct way. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is clever but it would be better if we can support specifying
NOW() - 10 second
here. 😄There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
They are not exactly same semantic.
generate_series(now() - 10 sec, now(), step)
is equal to the statment with a temporal filterSo we do not have a
current_now()
ornow_at_the_statment()
function 😆There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
PostgreSQL has 4 different "now"s: #2870
start
, if we really want time ofcreate
, it shall betimestamptz 'now'
.Forstop
, if we really want endless stream, it shall betimestamptz 'infinity'
.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
But seems
infinity
cannot clearly convey the idea that the timestamp is emitted along with the current epoch🤔There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It seems that we should've introduced something like
streaming_now()
for the use of dynamically updatednow
in streaming mode. So thatnow()
won't be so confusing when it appears with 2 semantics in one statement.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
When we use
now()
to express the semantics of a temporal filter,now()
represents a constantly changing timestamp within a streaming query. Therefore, it becomes difficult to express a semantics of the current value of now() that does not change later.