hi Siddhartha

> However, I have found that the method increment() is used in
"org.apache.kafka.streams.test", which is the public test utilities package
(https://kafka.apache.org/38/javadoc/index.html). This means external users
writing Kafka Streams tests might already be depending on this method.

I double-checked streams/test-utils but couldn't find any usage of
`increment` there. Could you point me to the specific class that uses it?

> Similarly, BYTES_LEXICO_COMPARATOR (of type ByteArrayComparator) is used
in
a different package like "org.apache.kafka.streams.state.internals". Making
it package-private would require extensive refactoring.

It is totally fine to keep the method public in teams of Java visibility
(so other packages can use it).

My point is that we should not expose it as a public Kafka API contract. By
moving it to an internal package, we indicate to users that this is not for
external use. This saves us from the burden of maintaining strict backward
compatibility in the future
Best,
Chia-Ping

Siddhartha Devineni <[email protected]> 於 2025年12月28日週日
下午3:17寫道:

> Hi Chia-Ping,
>
> I understand your concern about these appearing to be internal utilities -
> you are absolutely right that, ideally, we would have a cleaner, more
> minimal public API.
>
> However, I have found that the method increment() is used in
> "org.apache.kafka.streams.test", which is the public test utilities package
> (https://kafka.apache.org/38/javadoc/index.html). This means external
> users
> writing Kafka Streams tests might already be depending on this method.
>
> Given this constraint, making increment() package-private would potentially
> break existing user code.
>
> Similarly, BYTES_LEXICO_COMPARATOR (of type ByteArrayComparator) is used in
> a different package like "org.apache.kafka.streams.state.internals". Making
> it package-private would require extensive refactoring.
>
> So while I agree these seem like internal utilities, they are already being
> used in ways that make them effectively part of the public contract.
>
> I think the most pragmatic path forward is to accept them as part of the
> public API and ensure they are well-documented.
>
> However, I am open to other approaches if you have suggestions - perhaps
> there's a better way to handle this that I haven't considered?
>
> What are your thoughts?
>
> Best regards,
> Siddhartha
>
> On Sun, Dec 28, 2025 at 10:13 AM Chia-Ping Tsai <[email protected]>
> wrote:
>
> > hi Siddhartha
> >
> > chia_0: Sorry for the confusion. My concern is that both
> > `ByteArrayComparator` and `increment` are being exposed as Kafka public
> > APIs, even though they appear to be intended as Kafka internal APIs
> >
> > Best,
> > Chia-Ping
> >
> >
> > Siddhartha Devineni <[email protected]> 於 2025年12月28日週日
> > 上午2:13寫道:
> >
> > > Hi Chia-Ping,
> > >
> > > Thank you for the feedback.
> > >
> > > chia_0: You raised a valid concern about exposing all helper methods. I
> > > investigated the usage of each:
> > >
> > > 1. increment(Bytes input): Used in "org.apache.kafka.streams.test" (the
> > > public test utilities package for Kafka Streams), so it must remain
> > public.
> > > 2. BYTES_LEXICO_COMPARATOR: Used in
> > > "org.apache.kafka.streams.state.internals package". Since this is a
> > > different package from "org.apache.kafka.common.utils" where Bytes
> > resides,
> > > it must remain public (package-private access wouldn't work across
> > > different packages).
> > > 3. ByteArrayComparator interface: Must remain public since
> > > BYTES_LEXICO_COMPARATOR is public and has this type.
> > >
> > > So while I agree it would be cleaner to minimize the API surface, these
> > > helpers are already being used across different packages and by public
> > test
> > > utilities. Making them package-private would break existing internal
> code
> > > and they are already effectively part of the public contract.
> > >
> > > chia_1: Good suggestion.
> > > BytesSerializer in "org.apache.kafka.common.serialization" is indeed a
> > > strong example since serialization is explicitly listed as a public API
> > > package. I have added this to the motivation section to strengthen the
> > > rationale.
> > >
> > > Best regards,
> > > Siddhartha
> > >
> > > On Sat, Dec 27, 2025 at 10:28 AM Chia-Ping Tsai <[email protected]>
> > > wrote:
> > >
> > > > hi Siddhartha
> > > >
> > > > Thanks for the KIP. The motivation makes sense to me. I have a few
> > > > comments below.
> > > >
> > > > chia_0: do we need to expose all helpers, such as ByteArrayComparator
> > and
> > > > increment?
> > > >
> > > > chia_1: the bytes class is also part of serialization API. Maybe you
> > > could
> > > > mention that in the motivation to strengthen the rationale.
> > > >
> > > > Best,
> > > > Chia-Ping
> > > >
> > > > > Siddhartha Devineni <[email protected]> 於 2025年12月27日
> > > > 上午11:47 寫道:
> > > > >
> > > > > Hi all,
> > > > >
> > > > > I would like to send a gentle reminder about KIP-1247, which
> proposes
> > > to
> > > > > make the Bytes class officially part of the public API.
> > > > >
> > > > > KIP link:
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1247%3A+Make+Bytes+utils+class+part+of+the+public+API
> > > > >
> > > > > Following the earlier discussion, the KIP has been updated to focus
> > > only
> > > > on
> > > > > Bytes (the Time API will be addressed separately, as it needs more
> > > > detailed
> > > > > assessment).
> > > > >
> > > > > If there are no further concerns or feedback, I would like to call
> > for
> > > a
> > > > > vote in the next few days.
> > > > >
> > > > > Please let me know if you have any feedback.
> > > > > Thank you.
> > > > >
> > > > > Best regards,
> > > > > Siddhartha
> > > > >
> > > > >> On Thu, Dec 18, 2025 at 12:15 PM Siddhartha Devineni <
> > > > >> [email protected]> wrote:
> > > > >>
> > > > >> Hi Sean,
> > > > >>
> > > > >> Thank you for the detailed analysis of the Time interface - this
> is
> > an
> > > > >> invaluable context for when we address it in a future KIP.
> > > > >>
> > > > >> Your breakdown of the different responsibilities (wall clock,
> > > monotonic
> > > > >> clock, thread yielding, and Timer instantiation) clearly shows why
> > it
> > > > needs
> > > > >> more careful consideration before making it public.
> > > > >> I agree that a more focused interface would be preferable.
> > > > >>
> > > > >> As discussed, KIP-1247 now focuses only on Bytes, which is
> > > > straightforward
> > > > >> and uncontroversial. We shall address Time in a separate KIP where
> > we
> > > > can
> > > > >> properly evaluate these design concerns you have raised.
> > > > >>
> > > > >> When that discussion happens, your points about:
> > > > >>
> > > > >> 1. Separating the different time-related responsibilities
> > > > >> 2. The fact that many classes only need (1) or (2)
> > > > >> 3. The possibility of splitting out Timer instantiation entirely
> > > > >>
> > > > >> will be valuable input for designing a cleaner public API.
> > > > >>
> > > > >> Thanks again for the feedback!
> > > > >>
> > > > >> Best regards,
> > > > >> Siddhartha
> > > > >>
> > > > >> On Wed, Dec 17, 2025 at 9:49 PM Sean Quah via dev <
> > > [email protected]
> > > > >
> > > > >> wrote:
> > > > >>
> > > > >>> 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
> > > > >>>>>>>>>>
> > > > >>>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>
> > > > >>>>>>
> > > > >>>>>
> > > > >>>>
> > > > >>>
> > > > >>
> > > >
> > >
> >
>

Reply via email to