Re: [DISCUSS] KIP-1088: Replace KafkaClientSupplier with KafkaClientInterceptor

2024-10-21 Thread Sophie Blee-Goldman
I think I can answer some of Bruno's questions: BC2: We want access to the configs for the individual client. For example, in our implementation, we need to grab the clientId in order to configure our wrapper. IIRC we also access some other configs so just passing in the clientId wouldn't necessar

Re: [DISCUSS] KIP-1088: Replace KafkaClientSupplier with KafkaClientInterceptor

2024-10-14 Thread Bill Bejeck
HI All, Matthias - thanks for the KIP! I'm in favor of this proposal, but I'd like to summarize my understanding to ensure I'm getting everything. 1. KafkaStreams will remain in control of creating the client unless the user overrides that behavior (I think this would be a rare case). Although w

Re: [DISCUSS] KIP-1088: Replace KafkaClientSupplier with KafkaClientInterceptor

2024-10-14 Thread Bruno Cadonna
Hi, Thanks for the KIP! BC1. I echo point S3 from Sophie. I find it quite confusing having those methods return an interceptor that is used as a client in the Streams code. Naming might already do the trick. Something like Intercepted(Admin|StreamsConsumer|Consumer|Producer) or Wrapped(Admi

Re: [DISCUSS] KIP-1088: Replace KafkaClientSupplier with KafkaClientInterceptor

2024-10-04 Thread Sophie Blee-Goldman
Thanks for the update Matthias! I'm totally in agreement with the new proposal and have mainly only cosmetic points and minor nits remaining. Once we come to agreement on these I would be happy to move this to a vote (unless others chime in with new concerns of course) S1. One thing that jumped ou

Re: [DISCUSS] KIP-1088: Replace KafkaClientSupplier with KafkaClientInterceptor

2024-10-03 Thread Matthias J. Sax
Thanks for your feedback Alieh and Sophie. For everybody: we did sync about open questions in person, and I hope to reply w/o forgetting to add enough context about what we did discuss in person. If anything is missing/unclear, please let us know. Based on the in-person discussion, we agree

Re: [DISCUSS] KIP-1088: Replace KafkaClientSupplier with KafkaClientInterceptor

2024-09-09 Thread Sophie Blee-Goldman
I agree with Alieh, there is definitely some inconsistency in what we do or don't want to support/allow. However imo we should lean into the more flexible approach of allowing an alternative consumer to be constructed and returned, rather than attempting to enforce the interceptor to strictly wrap

Re: [DISCUSS] KIP-1088: Replace KafkaClientSupplier with KafkaClientInterceptor

2024-09-06 Thread Alieh Saeedi
Thanks Matthias for the KIP. A quick question regarding the sentence `It would be invalid to create a new client instance `: What would happen if the implemented class creates a new instance, or, in other words, how do we prevent it? Considering that `Config` is going to be passed in as well (the

Re: [DISCUSS] KIP-1088: Replace KafkaClientSupplier with KafkaClientInterceptor

2024-09-05 Thread Sophie Blee-Goldman
1. Fair enough. In the end it doesn't matter that much to us, so I just figured from basic principles if this is a config then it should go in the StreamsConfig. Also happy to hear what others think 2. We need the client-specific configs (for example to extract the client ids for monitoring, resou