-
Notifications
You must be signed in to change notification settings - Fork 1k
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
Add LATEST() UDAF to support stream->table conversion just like topic->table. #3985
Comments
cc @rmoff |
As also discussed with @MichaelDrogalis and @purplefox The limitation of the workaround described is that it does not materialise the table, which means that it cannot be used in a pull query. This constrains the narrative that can be told around tables and illustrating their use and difference from streams. |
With regard to KLIP-11, this would be addressed by the Also note, that Kafka Streams will add a |
@mjsax - adding some points here based on our offline discussion. I agree that it shouldn't be used to convert stream->table as is suggested in the original issue description because tombstones would not be properly handled, but I think if |
See #1128
…On Thu, Dec 19, 2019, 3:31 PM Almog Gavra ***@***.***> wrote:
@mjsax <https://github.com/mjsax> - adding some points here based on our
offline discussion.
I agree that it shouldn't be used to convert stream->table as is suggested
in the original issue description because tombstones would not be properly
handled, but I think if LATEST has value independently from stream->table
conversions. GROUP BY makes it clear that tombstones will be ignored and
we could use it for something like SELECT ticker, LATEST(stock_price),
AVG(stock_price) FROM stocks GROUP BY ticker;. Seems powerful.
—
You are receiving this because you are subscribed to this thread.
Reply to this email directly, view it on GitHub
<#3985?email_source=notifications&email_token=ABCXJIHNJFRLAFSKOT5EWY3QZP767A5CNFSM4JSEFPDKYY3PNVWWK3TUL52HS4DFVREXG43VMVBW63LNMVXHJKTDN5WW2ZLOORPWSZGOEHLMMTY#issuecomment-567723599>,
or unsubscribe
<https://github.com/notifications/unsubscribe-auth/ABCXJIGMYXTLGTFQ7CTTV5LQZP767ANCNFSM4JSEFPDA>
.
|
@agavra 👍
What would we need to change/add so that I really believe we must provide a way to convert a STREAM to a TABLE with the same semantics as one is able to create the TABLE from a topic. It's such a basic operation that it always pains me to see us advocating users to do the low-level workaround by fiddling with topics. |
It's not a question what we need to change, but if we should support this -- it seems semantically questionable to me. A aggregation does (by its definition -- at least from my understanding) not support deletes, and thus, it seems "wrong" to change the current behavior.
I agree, and I suggest to introduce a |
IMHO the operation is not always an "aggregation" (though in practice it is most of the times). It is an operation that takes N inputs to produce 1 output, in whichever way it achieves that. Maybe I should have named the title of this issue "Add a LATEST() UDAF", which would have been more clear. For example, both being a UDAF, there's no conceptual difference between |
Just an update. |
Well, I agree that it produces ONE output. However, if you want to implement deletes, it would need to support ZERO output (or a negative retraction output). Example:
This would not be better, because UDAF has aggregation in its name.
I disagree. An aggregation must produce a result record per group, ie, per input key in our case. Hence, resetting the counter to zero, would not remove the whole row for that key form the result. However, a proper "delete" would require this (compare my example from above how I think |
I think that the original proposal here is a good first step towards solving this problem. It would be a relatively small, non-disruptive change that would give users a straightforward way to accomplish what they're ultimately trying to do. I do agree with @mjsax's sentiment that there may be more elegant and native ways to accomplish this, and I think we should continue to explore those options as we better understand what users are doing with I also agree with @agavra that the primary use case here is in the context of aggregations with a I think we should move forward with what @miguno's original proposal on this issue. |
@derekjn FWIW, it's not that small to get timestamp semantics (e.g. |
Using offset ordering seems like trouble since we use event time in many places. I definitely like the idea of In any case, adapting the UDAF interface to accept multiple arguments seems worthwhile to me. |
I'm in agreement with @mjsax one the difference between an aggregation using Thinking out loud though, I think with -- input in form <KEY ->VALUE>: KEY has 1 col: `ROWKEY`; VALUE has col `col0`.
-- input stream: <A->1><A->2><B->10><B->20><A->null>
SELECT latest(col0) FROM s
GROUP BY ROWKEY
HAVING NOT_NULL(latest(col0)); i.e. we use the HAVING clause to remove entries from the table where the latest value for The intermediate tables (for incremental processing) would be
I think this can also work if there are multiple columns, e.g. -- input in form <KEY ->VALUE>: KEY has 1 col: `ROWKEY`; VALUE has cols `col0` & `col1`.
-- input stream: <A->1,x><A->2,y><B->10,x><B->20,x><A->null>
SELECT latest(col0), latest(col1) FROM s
GROUP BY ROWKEY
HAVING NOT_NULL(latest(col0)); The intermediate tables (for incremental processing) would be
And I think it can even deal with source streams where each event contains only partial data, assuming each record always contains at least one non-null field, which is required to differentiate it from a tombstone, e.g. -- input in form <KEY ->VALUE>: KEY has 1 col: `ROWKEY`; VALUE has cols `col0` & `col1`.
-- input stream: <A->1,null><A->null,x><B->null,x><B->20,null><A->null>
SELECT latest(col0), latest(col1) FROM s
GROUP BY ROWKEY
HAVING NOT_NULL(latest(col0)) AND NOT_NULL(latest(col1)); The intermediate tables (for incremental processing) would be
|
Interesting idea -- however, I see some issues. (1) [minor] The result table would not be materialized in the current implementation. (2) From my understanding there is a semantic difference between a tombstone (ie, value = |
I realize this thread is a technical conversation involving the kafka engineering team, but I'd like to offer an end-user perspective. With Event Sourcing I'd want to have a topic of In support of the partial data use case: In terms of the WIP example above:
I don't understand the implications behind the scene in the Streams API method, but at a glance It might make more sense to have a separate but very similar function
Where the backing implementation doesn't only return |
Thank you all for the interesting ideas. In terms of SELECT latest(col0) FROM s GROUP BY ROWKEY HAVING latest(col0) IS NOT NULL; In any case, I think that this Also, I think that an |
I guess I am missing something here but I don't understand why latest needs to be based on timestamps or offsets. Surely for latest you just always overwrite any current value for the key in the aggregate as subsequent values for the same key arrive? |
This would be offset based. ( But if there are out-of-order data in the input topic (or input stream) it seems not to be ideal to follow offset order, because given our temporal semantics, we might want to follow timestamp order but not offset order. Does this make sense? |
Ah I see, I think I would call that processing order. But I see in most cases processing order would be the same as offset order, unless you're consuming from multiple partitions (Not sure if that ever happens).
My inclination would be to go for offset order as I would guess that will work for most people. The use cases we care about seem to be for things like updating a stock ticker price, or an IoT sensor updating its latest temperature reading, or an Uber driver updating his latest position. Imho offset based will work for most people, and it's easier to implement so why not implement 2 functions:
Then if 1 doesn't satisfy users we get significant demand for it then think about implementing:
|
I think adding a |
In Kafka, processing order is always offset order, especially for KSQL because one cannot read from multiple partitions for this case (patter subscriptions are not supported AFAIK). Btw: for Kafka Streams we currently also implement offset-order for If we want to do a "short cut" and oly support offset order for now, I would also recommend to make it explicit in the name. In the end, the UDF approach is a workaround anyway as deletes are not supported and thus a proper solution must follow eventually. |
@mjsax regarding (not) supporting tombstones/deletes for tables: Why wouldn’t we able to support deletes in KSQL for this purpose? (The ConfluentJiraBot message above #3985 (comment) was from me when I replied to the GitHub email notification.) |
Discussed above: #3985 (comment) |
Hi @agavra could you elaborate on this a bit? AIUI, if we do something like:
Then it will repartition by sensor_id, which means all readings for the same sensor_id will be processed by the same node, so offset order is still meaningful. |
Only if the original input data from |
Got it. I think more than likely user will already have sensor data partitioned by sensor id - it seems like the "obvious" thing to do, so latest_by_offset seems a pragmatic first choice. |
I guess it's ok add |
I'm going to close this since it shipped in 0.8.0 under |
For other readers, this feature is now available (as @MichaelDrogalis said above). See the documentation at https://docs.ksqldb.io/en/latest/how-to-guides/convert-changelog-to-table/ for a full example. |
Motivation
It would be very useful to allow users to easily turn a
STREAM
into the same kind ofTABLE
that you get when reading a topic into a table. The respective operation, in both cases, is to only remember the latest value seen per key in the stream/topic.Today, this stream->table conversion (which is an aggregation) is not directly possible, and the workaround is to (1) know the stream's underlying topic, e.g. via
DESCRIBE myStream
, and then (2) create a new table from that topic via a CT statement.Suggestion
This stream->table conversion could be achieved by providing a new aggregation function called
LATEST()
.In Kafka Streams this would be something like:
Mocked ksqlDB example:
The text was updated successfully, but these errors were encountered: