Hey Sebastien et al,
have you tried rewriting the rolling aggregate as a window-over query? A
window-over aggregation creates an append-only stream which should preserve
the timestamp/watermark of the source. You can then add a deduplication
Hi everyone,
Before digging into what it would it take to implement a general solution, I
narrowed down the scope to write a fix which makes the query mentioned in the
thread work. Here are some findings:
- For the temporal join logic, it's not the watermark that matters but having a
Hi Everyone!
I have discussed this with Sébastien Chevalley, he is going to prepare and
drive the FLIP while I will assist him along the way.
Thanks
Gyula
On Tue, Mar 5, 2024 at 9:57 AM wrote:
> I do agree with Ron Liu.
> This would definitely need a FLIP as it would impact SQL and extend it
I do agree with Ron Liu.
This would definitely need a FLIP as it would impact SQL and extend it with the
equivalent of TimestampAssigners in the Java API.
Is there any existing JIRA here, or is anybody willing to drive a FLIP?
On Feb 26, 2024 at 02:36 +0100, Ron liu , wrote:
> +1,
> But I think
+1,
But I think this should be a more general requirement, that is, support for
declaring watermarks in query, which can be declared for any type of
source, such as table, view. Similar to databricks provided [1], this needs
a FLIP.
[1]
+1 Support this feature. There are many limitations to using time window
aggregation currently, and if we can declare watermark and time attribute
on the view, it will make it easier for us to use time windows. Similarly,
it would be very useful if the primary key could be declared in the view.
+1 for supporting defining time attributes on views.
I once encountered the same problem as yours. I did some regular joins and lost
time attribute, and hence I could no longer do window operations in subsequent
logics. I had to output the joined view to Kafka, read from it again, and
define
Posting this to dev as well as it potentially has some implications on
development effort.
What seems to be the problem here is that we cannot control/override
Timestamps/Watermarks/Primary key on VIEWs. It's understandable that you
cannot create a PRIMARY KEY on the view but I think the temporal
Hi,
I have been trying to write a temporal join in SQL done on a rolling aggregate
view. However it does not work and throws :
org.apache.flink.table.api.ValidationException: Event-Time Temporal Table Join
requires both primary key and row time attribute in versioned table, but no row
time