Ah thanks Matthias, I was looking at the wrong code earlier whoops. I totally agree, the #build static constructor is out of place, as I said originally I believe we should follow the same pattern we used in KIP-1092
As for whether to literally reuse the same CloseOptions object, I'm against that, I think Streams should have its own. Streams has different default behavior and we may want to extend the close options with streams-specific stuff at some point in the future. So I'm definitely in favor of option 2. I also think that if we're going to deprecate the entire class and add a new one, then we may as well also use an enum instead of a boolean for leaveGroup. On Fri, Apr 25, 2025 at 3:39 PM Matthias J. Sax <mj...@apache.org> wrote: > Thanks for the KIP. > > Using `build()` method in not common in the current API. Why do we want > to diverge? > > It seems more aligned to the current API design to replace `build()` > with two static builder methods: > > - withTimeout(Duration) > - withLeaveGropu(boolean) > > (Just for to illustrate. I would not use these two method names though.) > > The problem I see is, misalignment to the usual naming patterns and > KIP-1092 > ( > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=321719077 > ) > > We usually use `withXxx(...)` for the non-static method, and other names > for the static entry point method. However, the existing `CloseOptions` > already uses `timeout(During)` and `leaveGroup(boolean)` so we cannot > just change it. > > > I think there is two options: > > (1) We could just re-use the `CloseOptions` class from KIP-1092, and > deprecate `KafkaStreams.close(KakfaStreams.ClosOptions)` in favor of a > new one, which accepts the KIP-1092 `CloseOptions` object -- I want to > point out, that this idea was discussed on KIP-1092 already, but it was > controversial. > > (2) If we don't want to do it, we can also deprecate the entire existing > (nested) class, `KafkaStreams.CloseOptions`, and create a new (top > level) `o.a.k.streams.CloseOptions` and design it in a way that aligns > to common naming patterns. > > public class CloseOptions { > public static CloseOptions timeout(Duration); > public static CloseOptions leaveGroup(boolean); > > public CloseOptions withTimeout(Duration); > public CloseOptions withLeaveGroup(boolean); > } > > (Or similar names.) > > Btw: Instead of using a `boolean` it could also be beneficial to use an > enum a la KIP-1092 with valued DEFAULT, LEAVE_GROUP, REMAIN_IN_GROUP? > > > Thoughts? > > > -Matthias > > > > On 4/25/25 3:05 PM, Sophie Blee-Goldman wrote: > > Thanks! I personally think this looks good, as we really just wanted to > > remove the public constructor, but I'll ping Matthias to take a look and > > make sure this is in line with his understanding > > > > If yes I think we can move to a vote > > > > On Fri, Apr 25, 2025 at 5:34 AM 黃竣陽 <s7133...@gmail.com> wrote: > > > >> Hello Sophie, > >> > >> Thanks for your comments, > >> > >> I’ve updated the KIP to add a new static `build()` method for > initializing > >> the CloseOptions object. > >> The public constructor has been deprecated, while the existing > >> fluent-style methods remain unchanged. > >> > >> Best Regards, > >> Jiunn-Yang > >> > >>> Sophie Blee-Goldman <sop...@responsive.dev> 於 2025年4月25日 清晨5:15 寫道: > >>> > >>> Thanks for the KIP! > >>> > >>> This looks good but a few comments about the API: I think we actually > >> want > >>> more of a fluent pattern than a literal builder pattern, to be > consistent > >>> with other APIs in Kafka Streams. You can criticize Matthias for saying > >>> "builder pattern" in the ticket, he means a fluent style :P > >>> > >>> In other words instead of introducing a CloseOptions.Builder we should > >> just > >>> have a static constructor and non-static `.withParam()` methods for all > >>> optional parameters. You can actually take a look at the design of the > >>> CloseOptions class we just added for the consumer client, which we > >> designed > >>> specifically to match the style we wanted the Streams CloseOptions to > >> have. > >>> The parameters are a bit different but the API format should be the > same > >>> > >>> Cheers, > >>> Sophie > >>> > >>> On Fri, Apr 11, 2025 at 6:43 PM 黃竣陽 <s7133...@gmail.com> wrote: > >>> > >>>> Hello everyone, > >>>> > >>>> I would like to start a discussion on KIP-1153: Kafka Streams > >>>> `CloseOptions` should not have a public constructor > >>>> <https://cwiki.apache.org/confluence/x/QAq9F> > >>>> > >>>> This proposal aims to improve KafkaStreams.CloseOptions by adopting a > >>>> builder pattern to ensure API consistency. > >>>> > >>>> Best Regards, > >>>> Jiunn-Yang > >> > >> > > > >