Thanks for updating the KIP.
I think you can start a vote.
-Matthias
On 4/13/25 10:54 PM, Herman K. Jakobsen wrote:
Thank you for your inputs!
It looks like my previous answer messed up the mailing thread due to the
use of the hashtag symbol when creating the following list. I've tried to
reformat the list:
(1) I agree and I have changed the `startTime` parameter to be of type
`Instant`.
(2) I have added a section for the unintuitive cases and semantics that you
and Sophie have mentioned. In short, I’m proposing to skip forward to the
next trigger time.
(3) It was a typo - the correct term was wall clock time (i.e. system time).
(4-5) My initial proposal of only supporting wall clock time was based on
my restricted knowledge of the Kafka Streams and Punctuations internals. I
did not know if it would be demanding/complex to work with stream time.
Based on your inputs, it seems like it would be just as much work to
gracefully handle the unsupported stream time than it is to actually just
implement support for it. Hence, I’m proposing to implement support for
both wall clock time and stream time in this KIP.
Best regards,
Herman
tor. 3. apr. 2025 kl. 06:57 skrev Sophie Blee-Goldman <sop...@responsive.dev
:
Thanks for the KIP!
I pretty much echo what Matthias has said so far regarding the API.
Regarding #4-5, assuming we would just be leaving out stream-time in the
initial implementation for time/scope reasons and might want to add this in
the future, I think it's best to just throw an exception if this is used
with STREAM_TIME for now. Especially if it's only a short time before we
support stream-time, which seems likely unless for some reason it is
significantly more difficult to implement this for stream-time vs for
wall-clock time. And I can't imagine that it would be.
As for the actual semantics, perhaps it would be useful to include a brief
example including how it behaves on startup/after a restart.
Let's say you schedule a punctuation with a start time of 100 and an
interval of 10 seconds. If you first start up at t=201, my understanding is
that you'd wait until t=210 to trigger the first punctuation, and after
that they would proceed as usual in 10s increments?
So if your app happened to shut down at t=209 and didn't come back up again
until t=211, you would miss the punctuation at t=210 and would have to wait
until t=220 to run the punctuator?
That's more or less how the punctuator works now, just want to make sure
this is clear in the KIP.
In fact, this KIP seems to actually remedy a long-standing issue with
wall-clock punctuators, which is that they'll never be triggered if the app
or task is restarted within the punctuation interval. With anchored
punctuations you can technically still miss the punctuation if it restarts
exactly when it was scheduled, but that should be fairly rare, so in
general frequent restarts won't be a problem for punctuations anymore.
Neat!
On Tue, Apr 1, 2025 at 6:12 PM Matthias J. Sax <mj...@apache.org> wrote:
Herman,
thanks for the KIP, and sorry for late response. Overall the KIP makes
sense to me, and the propose API change is neat and contained, so I
don't have any concerns about it.
Couple of questions/comments.
(1) I think the propose `startTime` parameter should not be a `long` but
in `Instance` type?
(2) What happens is the passed in `startTime` is in the past? Would an
exception be thrown? Or just "skip forward" to the next "firing point"
in the future?
(3) The KIP says
It is planned that only wall clock (i.e. stream time) is supported in
this first iteration.
This is confusing. Wall-clock time is not the same as stream time. Maybe
just a typo.
(4) Why are you excluding "stream time"? Also, the KIP does not define
what happens if the new overload is used in combination with "stream
time" -- I assume the method would throw an exception, what might not be
ideal API ergonomics (cf (5) below).
(5) If we really want to exclude "stream-time", I am wondering if we
could use
Cancellable schedule(final Duration interval, final long startTime,
final Punctuator callback)
instead, omitting the `PunctuationType` parameter and document clearly
in the JavaDocs that wall-clock time punctuation are used. This way,
users cannot write incorrect code and don't need to worry about
exceptions.
-Matthias
On 3/22/25 1:24 AM, Herman K. Jakobsen wrote:
Hi all,
I would like to start a discussion thread on
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1146%3A+Anchored+wall-clock+punctuation
Best regards,
Herman