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