Re: [DISCUSS] KIP-805: Add range and scan query support in IQ v2

2021-11-29 Thread John Roesler
Thanks for the KIP, Vicky! This KIP will help fill in the parity gap between IQ and IQv2. One thing I noticed, which looks like just a typo is that the value type of the proposed RangeQuery should probably be KeyValueIterator, right? Otherwise, it looks good to me! Thanks, -John On Mon,

Re: [VOTE] KIP-799: Align behaviour for producer callbacks with documented behaviour

2021-11-29 Thread John Roesler
Thanks, Séamus! I'm +1 (binding). On Mon, 2021-11-29 at 16:14 +, Séamus Ó Ceanainn wrote: > Hi everyone, > > I'd like to start a vote for KIP-799: Align behaviour for producer > callbacks with documented behaviour >

Re: KIP-769: Connect API to retrieve connector configuration definitions

2021-11-29 Thread Chris Egerton
Hi Mickael, I think that's a great idea! I especially like how we can establish the expectation that any plugin type that appears in the response from the GET /connector-plugins endpoint will have a corresponding GET /connector-plugins//config endpoint, but (if we decide to add them in the

Re: [DISCUSS] KIP-782: Expandable batch size in producer

2021-11-29 Thread Artem Livshits
Hi Luke, I don't mind increasing the max.request.size to a higher number, e.g. 2MB could be good. I think we should also run some benchmarks to see the effects of different sizes. I agree that changing round robin to random solves an independent existing issue, however the logic in this KIP

Re: [DISCUSS] KIP-778 KRaft Upgrades

2021-11-29 Thread David Arthur
Jun, I updated the KIP with the "disable" CLI. For 16, I think you're asking how we can safely introduce the initial version of other feature flags in the future. This will probably depend on the nature of the feature that the new flag is gating. We can probably take a similar approach and say

Jenkins build is unstable: Kafka » Kafka Branch Builder » 3.0 #157

2021-11-29 Thread Apache Jenkins Server
See

Build failed in Jenkins: Kafka » Kafka Branch Builder » 3.1 #25

2021-11-29 Thread Apache Jenkins Server
See Changes: -- [...truncated 501254 lines...] [2021-11-29T21:48:19.780Z] GetOffsetShellTest > testTopicPartitionsArg() STARTED [2021-11-29T21:48:23.401Z]

Re: Handling retriable exceptions during Connect source task start

2021-11-29 Thread Chris Egerton
Hi Gunnar, I think there's some risk of introducing this retry behavior if we end up invoking Connector::start or Task::start on the same object multiple times. Unexpected behavior may result, such as double-allocation of resources that are initialized in the start method and which are meant to

Re: [DISCUSS] KIP-785 Automatic storage formatting

2021-11-29 Thread Igor Soarez
Hi all, Bumping this thread as it’s been a while. Looking forward to any kind of feedback, pease take a look. I created a short PR with a possible implementation - https://github.com/apache/kafka/pull/11549 -- Igor > On 18 Oct 2021, at 15:11, Igor Soarez wrote: > > Hi all, > > I'd like

Filtering support on Fetch API

2021-11-29 Thread Talat Uyarer
Hi All, I want to get your advice about one subject. I want to create a KIP for message header base filtering on Fetch API. Our current use case We have 1k+ topics and per topic, have 10+ consumers for different use cases. However all consumers are interested in different sets of messages on the

Re: [DISCUSS] KIP-795: Add public APIs for AbstractCoordinator

2021-11-29 Thread Hector Geraldino (BLOOMBERG/ 919 3RD A)
Hello again Tom, kafka devs First, congrats on becoming a PMC member! That's so cool. Since your last reply I've updated the KIP trying to address some of your suggestions. A few more details have been added to the motivation section, and also went ahead and opened a draft pull-request with

Re: [VOTE] KIP-779: Allow Source Tasks to Handle Producer Exceptions

2021-11-29 Thread Knowles Atchison Jr
Thank you all for voting! KIP-779 has been approved: 3 binding votes (John, Mickael, Tom) 4 non-binding votes (Knowles, Chris S., Chris E., Arjun) The vote is now closed. Other than modifying the wiki, is anything additional I need to do vote wise? Knowles On Mon, Nov 29, 2021 at 10:49 AM Tom

[jira] [Created] (KAFKA-13488) Producer fails to recover if topic gets deleted (and gets auto-created)

2021-11-29 Thread Prateek Agarwal (Jira)
Prateek Agarwal created KAFKA-13488: --- Summary: Producer fails to recover if topic gets deleted (and gets auto-created) Key: KAFKA-13488 URL: https://issues.apache.org/jira/browse/KAFKA-13488

[VOTE] KIP-799: Align behaviour for producer callbacks with documented behaviour

2021-11-29 Thread Séamus Ó Ceanainn
Hi everyone, I'd like to start a vote for KIP-799: Align behaviour for producer callbacks with documented behaviour . The KIP proposes a breaking change in the

Re: [VOTE] KIP-779: Allow Source Tasks to Handle Producer Exceptions

2021-11-29 Thread Tom Bentley
Hi Knowles, Thanks for the KIP, +1 (binding) Kind regards, Tom On 11/29/21, Mickael Maison wrote: > Hi Knowles, > > +1 (binding) > > Thanks for the KIP! > > On Mon, Nov 29, 2021 at 12:56 PM Knowles Atchison Jr > wrote: >> >> Good morning, >> >> Bringing this back to the top. >> >> We

[jira] [Resolved] (KAFKA-13200) Fix version of MirrorMaker2 connectors

2021-11-29 Thread Mickael Maison (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13200?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mickael Maison resolved KAFKA-13200. Fix Version/s: 3.2.0 Resolution: Fixed > Fix version of MirrorMaker2 connectors >

Re: [DISCUSS] KIP-714: Client metrics and observability

2021-11-29 Thread Magnus Edenhill
Hey Bob, That's a good point. Request type labels were considered but since they're already tracked by broker-side metrics they were left out as to avoid metric duplication, however those metrics are not per connection, so they won't be that useful in practice for troubleshooting specific client

Re: [DISCUSS] KIP-714: Client metrics and observability

2021-11-29 Thread Magnus Edenhill
Hi Viktor, that's a good idea, I've added a bunch of broker-side metrics for the client metrics handling. There might be more added during development as the need arise.

Re: [VOTE] KIP-779: Allow Source Tasks to Handle Producer Exceptions

2021-11-29 Thread Mickael Maison
Hi Knowles, +1 (binding) Thanks for the KIP! On Mon, Nov 29, 2021 at 12:56 PM Knowles Atchison Jr wrote: > > Good morning, > > Bringing this back to the top. > > We currently have > > 1 binding > 4 non-binding > > Knowles > > On Fri, Nov 19, 2021 at 10:02 AM Knowles Atchison Jr > wrote: > > >

[DISCUSS] KIP-805: Add range and scan query support in IQ v2

2021-11-29 Thread Vasiliki Papavasileiou
Hello everyone, I would like to start the discussion for KIP-805: Add range and scan query support in IQ v2 The KIP can be found here: https://cwiki.apache.org/confluence/display/KAFKA/KIP-805%3A+Add+range+and+scan+query+support+in+IQ+v2 Any suggestions are more than welcome. Many thanks,

Re: [VOTE] KIP-779: Allow Source Tasks to Handle Producer Exceptions

2021-11-29 Thread Knowles Atchison Jr
Good morning, Bringing this back to the top. We currently have 1 binding 4 non-binding Knowles On Fri, Nov 19, 2021 at 10:02 AM Knowles Atchison Jr wrote: > Thank you all for voting. We still need two more binding votes. > > I have rebased and updated the PR to be ready to go once this vote

Jenkins build is unstable: Kafka » Kafka Branch Builder » 3.1 #24

2021-11-29 Thread Apache Jenkins Server
See

[jira] [Resolved] (KAFKA-13482) JRE: Duplicate Key: Multiple bootstrap server URLs

2021-11-29 Thread Michael Anstis (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-13482?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Anstis resolved KAFKA-13482. Resolution: Not A Problem > JRE: Duplicate Key: Multiple bootstrap server URLs >

Re: KIP-769: Connect API to retrieve connector configuration definitions

2021-11-29 Thread Mickael Maison
Hi Chris, Yes to keep compatibility we want a default implementation for Converter.configs(), I've updated the KIP. Regarding worker plugins, the use case you described seems valuable. I'd prefer not mixing worker and connector plugins on the same endpoint but I agree using /plugins and

[jira] [Created] (KAFKA-13487) Create a topic partition directory based on the size of the directory

2021-11-29 Thread Gongwenzhou (Jira)
Gongwenzhou created KAFKA-13487: --- Summary: Create a topic partition directory based on the size of the directory Key: KAFKA-13487 URL: https://issues.apache.org/jira/browse/KAFKA-13487 Project: Kafka