Hi Siddartha and Kirk, Thank you for your thoughts. For future discussions, my issue with making the `Time` interface public is that it tries to do 3-4 different things related to time depending on how you count them: 1. Provide a wall clock (`milliseconds`) 2. Provide a high resolution monotonic clock (`nanoseconds`, `hiResClockMs`) 3. Provide methods for yielding the current thread (`sleep`, `waitObject`, `waitForFuture`) 4. Provide convenience methods for instantiating `Timer`s (`timer`, `timer`)
Many of the classes which take a `Time` only need (1), especially in the broker side, though it is arguable some of them ought to be using (2) (eg. KAFKA-19888 <https://issues.apache.org/jira/browse/KAFKA-19888>). I would be more supportive if `Time` was more focused and limited to (1) and maybe (2). I appreciate this is easier said than done since we have to mock (1), (2) and (3) together in tests. (4) could be split out entirely since we don't mock `Timer`s at all. `KafkaStreams` in particular seems to mainly use (1) with some occasional usage of (2). Kind regards, Sean On Wed, Dec 17, 2025 at 6:38 AM Siddhartha Devineni < [email protected]> wrote: > Hi all, > > The KIP has been updated to include only the Bytes API to be part of the > public API. > > Here is the KIP's link again: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-1247%3A+Make+Bytes+utils+class+part+of+the+public+API > > Thanks and best regards, > Siddhartha > > On Wed, Dec 17, 2025 at 11:36 AM Siddhartha Devineni < > [email protected]> wrote: > > > Hi Kirk, > > > > Thank you for your suggestion. > > Yes, that seems to be so. > > > > Then, I will update the KIP to include only the Bytes API to be public. > > > > Best regards, > > Siddhartha > > > > On Wed, Dec 17, 2025 at 6:44 AM Kirk True <[email protected]> wrote: > > > >> Hi Siddhartha, > >> > >> It seems prudent to refocus this KIP on promoting the Bytes API to be > >> public and then file a separate KIP for the Time API. It's more > overhead, > >> but it unblock Bytes since Time seems to need a little more work. > >> > >> Thanks, > >> Kirk > >> > >> On Tue, Dec 16, 2025, at 3:07 AM, Siddhartha Devineni wrote: > >> > Hi all, > >> > > >> > Thank you for the feedback. > >> > > >> > @Sean, I understand your concern about "Time" not being suitable for a > >> > public API in its current state. > >> > Could you elaborate on what specific issues make it a "dumping > ground"? > >> > > >> > Regarding your suggestion to exclude the Streams constructors > accepting > >> > "Time" from the public API - I want to clarify the implications: > >> > The constructor KafkaStreams(Topology, Properties, Time) is currently > >> > public and has been available for several releases. > >> > Making it non-public or removing it would be a breaking change that > >> would > >> > affect any users currently using this constructor. > >> > > >> > What do you have in mind? > >> > > >> > 1. Deprecate the constructor now and remove it in a future major > >> version, or > >> > 2. Make it package-private (which would break existing code > >> immediately)? > >> > > >> > @Kirk, Thank you for pointing that out. > >> > You're absolutely right that making "Time" public would require making > >> > "Timer" public as well, since Time.timer() returns Timer objects. > >> > This does expand the scope considerably. > >> > > >> > Given this expanding scope and Sean's concerns about the Time API > >> design, > >> > would it make sense to split this KIP into two parts or create a > >> > separate KIP for the "Time" API and its implications? > >> > > >> > Best regards, > >> > Siddhartha > >> > > >> > > >> > On Tue, Dec 16, 2025 at 6:18 AM Kirk True <[email protected]> wrote: > >> > > >> > > Hi all, > >> > > > >> > > Sean: which parts of the Time API are the most clunky? The > >> waitForFuture() > >> > > and waitObject() methods seem like they could be moved elsewhere, > but > >> the > >> > > others seem OK. > >> > > > >> > > Siddhartha: because the Time API creates Timer objects, we'd need to > >> > > promote Timer to the public API, too. > >> > > > >> > > Thanks, > >> > > Kirk > >> > > > >> > > On Fri, Dec 12, 2025, at 7:12 AM, Sean Quah via dev wrote: > >> > > > Hi Siddhartha, > >> > > > > >> > > > Thanks for the KIP! I'm okay making `Bytes` public. However, the > >> `Time` > >> > > > interface is a bit of a dumping ground for time-related things > and I > >> > > would > >> > > > not be in favor of making it public in its current state. > >> > > > Is it possible to exclude the streams constructors accepting > >> `Time`s from > >> > > > the public API instead? > >> > > > > >> > > > Kind regards, > >> > > > Sean Quah > >> > > > > >> > > > On Sun, Dec 7, 2025 at 1:53 PM Siddhartha Devineni < > >> > > > [email protected]> wrote: > >> > > > > >> > > > > Hello Kafka Community, > >> > > > > > >> > > > > I would like to start a discussion on KIP-1247, which proposes > to > >> > > > > officially make the "Bytes" and "Time" utils classes part of > >> Kafka's > >> > > public > >> > > > > API. > >> > > > > > >> > > > > *KIP Link:* > >> > > > > > >> > > > > > >> > > > >> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-1247%3A+Make+Bytes+and+Time+utils+classes+part+of+the+public+API > >> > > > > > >> > > > > *Background:* > >> > > > > > >> > > > > Currently, "org.apache.kafka.common.utils.Bytes" and > >> > > > > "org.apache.kafka.common.utils.Time" are exposed through > numerous > >> > > public > >> > > > > API interfaces in Kafka Streams and other components, yet they > >> are not > >> > > > > officially designated as public API since the utils package is > not > >> > > included > >> > > > > in Javadoc generation. > >> > > > > > >> > > > > This creates confusion for users who cannot determine if these > >> classes > >> > > are > >> > > > > officially supported, and causes broken Javadoc references. > >> > > > > > >> > > > > *Proposal:* > >> > > > > > >> > > > > This KIP proposes to: > >> > > > > > >> > > > > 1. Include "Bytes" and "Time" in Javadoc generation, > officially > >> > > making > >> > > > > them part of the public API > >> > > > > 2. Move other internal utility classes to an "internals" > >> subpackage > >> > > to > >> > > > > prevent similar issues in the future > >> > > > > > >> > > > > > >> > > > > > >> > > > > *Impact:*This change has no compatibility impact - all classes > >> remain > >> > > in > >> > > > > their current locations and no user code changes are required. > >> > > > > > >> > > > > You can find more details in the attached KIP link. > >> > > > > Looking forward to your thoughts. > >> > > > > > >> > > > > Thank you. > >> > > > > > >> > > > > Best regards. > >> > > > > Siddhartha > >> > > > > > >> > > > > >> > > > >> > > >> > > >
