Re: [VOTE] KIP-884: Add config to configure KafkaClientSupplier in Kafka Streams

2022-11-22 Thread Bruno Cadonna
Hi Hao, Thanks for the KIP! +1 (binding) Best, Bruno On 22.11.22 10:08, Sophie Blee-Goldman wrote: Hey Hao, thanks for the KIP -- I'm +1 (binding) On Mon, Nov 21, 2022 at 12:57 PM Matthias J. Sax wrote: +1 (binding) On 11/21/22 7:39 AM, John Roesler wrote: I'm +1 (binding) Thanks for

Re: [DISCUSS] KIP-890 Server Side Defense

2022-11-22 Thread Jeff Kim
Hi Justine, Thanks for the KIP! I have two questions: 1) For new clients, we can once again return an error UNKNOWN_PRODUCER_ID for sequences that are non-zero when there is no producer state present on the server. This will indicate we missed the 0 sequence and we don't yet want to write to the

[jira] [Created] (KAFKA-14417) Producer doesn't handle REQUEST_TIMED_OUT for InitProducerIdRequest

2022-11-22 Thread Justine Olshan (Jira)
Justine Olshan created KAFKA-14417: -- Summary: Producer doesn't handle REQUEST_TIMED_OUT for InitProducerIdRequest Key: KAFKA-14417 URL: https://issues.apache.org/jira/browse/KAFKA-14417 Project:

Re: [DISCUSS] KIP-890 Server Side Defense

2022-11-22 Thread Artem Livshits
Hi Justine, Thank you for the KIP. I have one question. 5) For new clients, we can once again return an error UNKNOWN_PRODUCER_ID I believe we had problems in the past with returning UNKNOWN_PRODUCER_ID because it was considered fatal and required client restart. It would be good to spell out

Re: [VOTE] KIP-866 ZooKeeper to KRaft Migration

2022-11-22 Thread Jason Gustafson
Thanks, +1 from me. I suspect we might be in for at least one surprise with the re-implemented controller RPCs, but I agree the alternative has risks as well. Best, Jason On Mon, Nov 14, 2022 at 12:00 PM Colin McCabe wrote: > On Fri, Nov 11, 2022, at 08:59, David Arthur wrote: > > Thanks,

Re: [DISCUSS] KIP-881: Rack-aware Partition Assignment for Kafka Consumers

2022-11-22 Thread Artem Livshits
Hi Rajini, Thank you for the KIP, the KIP looks good to match RackAwareReplicaSelector behavior that is available out-of-box. Which should probably be good enough in practice. >From the design perspective, though, RackAwareReplicaSelector is just one possible plugin, in theory the broker could

[jira] [Created] (KAFKA-14416) org.apache.kafka.common.config.internals classes should be relocated

2022-11-22 Thread Greg Harris (Jira)
Greg Harris created KAFKA-14416: --- Summary: org.apache.kafka.common.config.internals classes should be relocated Key: KAFKA-14416 URL: https://issues.apache.org/jira/browse/KAFKA-14416 Project: Kafka

[jira] [Resolved] (KAFKA-14176) Reduce visibility of WorkerConfig.lookupKafkaClusterId

2022-11-22 Thread Greg Harris (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14176?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Harris resolved KAFKA-14176. - Resolution: Fixed > Reduce visibility of WorkerConfig.lookupKafkaClusterId >

Re: [DISCUSS] KIP-891: Running multiple versions of a connector.

2022-11-22 Thread Snehashis
Thanks for the points Sagar. > 1) Should we update the GET /connectors endpoint to include the version of > the plugin that is running? It could be useful to figure out the version of > the plugin or I am assuming it gets returned by the expand=info call? I think this is good to have and

Re: [DISCUSS] KIP-891: Running multiple versions of a connector.

2022-11-22 Thread Snehashis
Thanks for the explanation Ashwin. This is an interesting notion. This is something which many connectors implicitly do anyway. There are several connectors which have different methods of interpreting the configurations provided. Often the user has some control over how provided configuration

Re: [DISCUSS] KIP-889 Versioned State Stores

2022-11-22 Thread Victoria Xia
Thanks, Matthias and Sagar, for your comments! I've responded here for now, and will update the KIP afterwards with the outcome of our discussions as they resolve. --- Matthias's comments --- > (1) Why does the new store not extend KeyValueStore, but StateStore? In the end, it's

[jira] [Resolved] (KAFKA-14307) KRaft controller time based snapshots

2022-11-22 Thread Jira
[ https://issues.apache.org/jira/browse/KAFKA-14307?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] José Armando García Sancio resolved KAFKA-14307. Resolution: Fixed > KRaft controller time based snapshots >

[jira] [Resolved] (KAFKA-14394) BrokerToControllerChannelManager has 2 separate timeouts

2022-11-22 Thread Ron Dagostino (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-14394?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ron Dagostino resolved KAFKA-14394. --- Resolution: Not A Problem > BrokerToControllerChannelManager has 2 separate timeouts >

Re: [DISCUSS] KIP-890 Server Side Defense

2022-11-22 Thread Justine Olshan
Thanks for taking a look Matthias. I've tried to answer your questions below: 10) Right — so the hanging transaction only occurs when we have a late message come in and the partition is never added to a transaction again. If we never add the partition to a transaction, we will never write a

Build failed in Jenkins: Kafka » Kafka Branch Builder » 3.0 #213

2022-11-22 Thread Apache Jenkins Server
See Changes: -- [...truncated 284177 lines...] [2022-11-22T11:53:36.990Z] [2022-11-22T11:53:36.990Z] > Task :clients:srcJar [2022-11-22T11:53:36.990Z] Execution

Re: [DISCUSS] KIP-892: Transactional Semantics for StateStores

2022-11-22 Thread Nick Telford
Hi John, Thanks for the review and feedback! 1. Custom Stores: I've been mulling over this problem myself. As it stands, custom stores would essentially lose checkpointing with no indication that they're expected to make changes, besides a line in the release notes. I agree that the best

Re: [DISCUSS] KIP-892: Transactional Semantics for StateStores

2022-11-22 Thread John Roesler
Thanks for publishing this alternative, Nick! The benchmark you mentioned in the KIP-844 discussion seems like a compelling reason to revisit the built-in transactionality mechanism. I also appreciate you analysis, showing that for most use cases, the write batch approach should be just fine.

[jira] [Created] (KAFKA-14415) `ThreadCache` is getting slower with every additional state store

2022-11-22 Thread Lucas Brutschy (Jira)
Lucas Brutschy created KAFKA-14415: -- Summary: `ThreadCache` is getting slower with every additional state store Key: KAFKA-14415 URL: https://issues.apache.org/jira/browse/KAFKA-14415 Project: Kafka

Jenkins build is unstable: Kafka » Kafka Branch Builder » 3.3 #124

2022-11-22 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-889 Versioned State Stores

2022-11-22 Thread Sagar
Hi Victoria, Thanks for the KIP. Seems like a very interesting idea! I have a couple of questions: 1) Did you consider adding a method similar to : List> get(K key, long from, long to)? I think this could be useful considering that this versioning scheme unlocks time travel at a key basis.

Re: [DISCUSS] KIP-891: Running multiple versions of a connector.

2022-11-22 Thread Sagar
Hey Snehashsih, Thanks for the KIP. It looks like a very useful feature. Couple of small-ish points, let me know what you think: 1) Should we update the GET /connectors endpoint to include the version of the plugin that is running? It could be useful to figure out the version of the plugin or I

Re: Request for Kafka Jira account

2022-11-22 Thread Gantigmaa Selenge
Thank you Mickael! Regards, Gantigmaa On Mon, Nov 21, 2022 at 2:40 PM Mickael Maison wrote: > Hi, > > I've created your account, you should receive an email with all the > details. > > Thanks, > Mickael > > On Mon, Nov 21, 2022 at 3:23 PM Gantigmaa Selenge > wrote: > > > > Hi team, > > > >

Re: [VOTE] KIP-884: Add config to configure KafkaClientSupplier in Kafka Streams

2022-11-22 Thread Sophie Blee-Goldman
Hey Hao, thanks for the KIP -- I'm +1 (binding) On Mon, Nov 21, 2022 at 12:57 PM Matthias J. Sax wrote: > +1 (binding) > > On 11/21/22 7:39 AM, John Roesler wrote: > > I'm +1 (binding) > > > > Thanks for the KIP! > > -John > > > > On 2022/11/17 21:06:29 Hao Li wrote: > >> Hi all, > >> > >> I

Re: [DISCUSS] KIP-891: Running multiple versions of a connector.

2022-11-22 Thread Ashwin
Hi Snehasis, > IIUC (please correct me if I am wrong here), what you highlighted above, is a versioning scheme for a connector config for the same connector (and not different versions of a connector plugin). Sorry for not being more precise in my wording - I meant registering versions of