Re: [DISCUSS] KIP-699: Update FindCoordinator to resolve multiple Coordinators at a time

2021-02-18 Thread Mickael Maison
Hi, I have not seen any feedback on this KIP. I'll wait a few more days and if I don't see anything, I'll start a vote early next week. Thanks On Thu, Jan 28, 2021 at 10:43 AM Mickael Maison wrote: > > Hi all, > > I would like to start a discussion on KIP-699: > https://cwiki.apache.org/conflue

Re: [VOTE] KIP - 405: Kafka Tiered Storage.

2021-02-18 Thread Satish Duggana
Thanks everyone for the votes. This KIP is accepted with +3 binding (Harsha, Mani, and Jun) votes, and +4 non-binding (Kowshik, Ivan, Kamal, and Prateek) votes. On Wed, 17 Feb 2021 at 22:38, Prateek Agarwal wrote: > > +1 (non-binding). > > On Wed, Feb 17, 2021 at 5:53 AM Jun Rao wrote: >

Re: [DISCUSS] KIP-708: Rack aware Kafka Streams with pluggable StandbyTask assignor

2021-02-18 Thread Bruno Cadonna
Hi Levani, Thank you for the KIP. Really interesting! Here my comments: 1. To be consistent with the other configs that involve standbys , I would rename standby.task.assignment.awareness -> standby.replicas.awareness 2. I would also rename the prefix instance.tag -> client.tag 3. The foll

[jira] [Created] (KAFKA-12336) custom stream naming does not work while calling stream[K, V](topicPattern: Pattern) API with named Consumed parameter

2021-02-18 Thread Ramil Israfilov (Jira)
Ramil Israfilov created KAFKA-12336: --- Summary: custom stream naming does not work while calling stream[K, V](topicPattern: Pattern) API with named Consumed parameter Key: KAFKA-12336 URL: https://issues.apache.

[jira] [Created] (KAFKA-12337) provide full scala api for operators naming

2021-02-18 Thread Ramil Israfilov (Jira)
Ramil Israfilov created KAFKA-12337: --- Summary: provide full scala api for operators naming Key: KAFKA-12337 URL: https://issues.apache.org/jira/browse/KAFKA-12337 Project: Kafka Issue Type

[jira] [Resolved] (KAFKA-12303) Flatten SMT drops some fields when null values are present

2021-02-18 Thread Mickael Maison (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12303?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mickael Maison resolved KAFKA-12303. Fix Version/s: 3.0.0 Resolution: Fixed > Flatten SMT drops some fields when null va

[jira] [Resolved] (KAFKA-10833) KIP-661: Expose task configurations in Connect REST API

2021-02-18 Thread Mickael Maison (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10833?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mickael Maison resolved KAFKA-10833. Fix Version/s: 2.8.0 Resolution: Fixed > KIP-661: Expose task configurations in Con

[jira] [Created] (KAFKA-12338) The code of MetadataRecordSerde duplicate with MetadataParser

2021-02-18 Thread dengziming (Jira)
dengziming created KAFKA-12338: -- Summary: The code of MetadataRecordSerde duplicate with MetadataParser Key: KAFKA-12338 URL: https://issues.apache.org/jira/browse/KAFKA-12338 Project: Kafka Is

Re: [DISCUSS] KIP-710: Full support for distributed mode in dedicated MirrorMaker 2.0 clusters

2021-02-18 Thread Dániel Urbán
Hello everyone, * Sorry, I meant KIP-710. Right now the MirrorMaker cluster is somewhat unreliable, and not supporting running in a cluster properly. I'd say that fixing this would be a nice addition. Does anyone have some input on this? Thanks in advance Daniel Dániel Urbán ezt írta (időpont:

[DISCUSS] Apache Kafka 2.7.1 release

2021-02-18 Thread Mickael Maison
Hi, I'd like to volunteer to be the release manager for the next bugfix release, 2.7.1. I created the release plan on the wiki: https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+2.7.1 Thanks

About Kafka 2.7.0 source code compilation error

2021-02-18 Thread 韩可
Hello! Recently, we need to build a compilation and development environment for Kafka 2.7.0 source code. Now the source code can run successfully. However, when we execute "gradle install" or "gradle build", we will report an error: @ nowarn annotation does not suppress any warnings. The detai

Re: [DISCUSS] Apache Kafka 2.8.0 release

2021-02-18 Thread John Roesler
Hello again, all. This is a notice that we are now in Code Freeze for the 2.8 branch. >From now until the release, only fixes for blockers should be merged to the >release branch. Fixes for failing tests are allowed and encouraged. >Documentation-only commits are also ok, in case you have forg

Re: [DISCUSS] Apache Kafka 2.7.1 release

2021-02-18 Thread Ismael Juma
Thanks for volunteering Mickael. Are there any critical bugs in 2.7.0 so far that motivated the release? There doesn't have to be, but I'd be interested to know if there are. Ismael On Thu, Feb 18, 2021 at 7:58 AM Mickael Maison wrote: > Hi, > > I'd like to volunteer to be the release manager f

Re: [VOTE] KIP-516: Topic Identifiers

2021-02-18 Thread Justine Olshan
Hi all, I realized that the DISCUSS thread got very long, so I'll be posting updates to this thread from now on. Just a quick update to the KIP. As a part of https://issues.apache.org/jira/browse/KAFKA-12332 and https://github.com/apache/kafka/pull/10143, I'm proposing adding a new error. INCONSIST

[jira] [Created] (KAFKA-12339) Starting new connector cluster with new internal topics is unstable due to UnknownTopicOrPartitionException

2021-02-18 Thread Chia-Ping Tsai (Jira)
Chia-Ping Tsai created KAFKA-12339: -- Summary: Starting new connector cluster with new internal topics is unstable due to UnknownTopicOrPartitionException Key: KAFKA-12339 URL: https://issues.apache.org/jira/brows

[jira] [Created] (KAFKA-12340) Recent change to use SharedTopicAdmin results in potential resource leak in deprecated backing store constructors

2021-02-18 Thread Randall Hauch (Jira)
Randall Hauch created KAFKA-12340: - Summary: Recent change to use SharedTopicAdmin results in potential resource leak in deprecated backing store constructors Key: KAFKA-12340 URL: https://issues.apache.org/jira/b

[jira] [Created] (KAFKA-12341) Duplicate javassist library found in broker classpath

2021-02-18 Thread Cosmin Giurgiu (Jira)
Cosmin Giurgiu created KAFKA-12341: -- Summary: Duplicate javassist library found in broker classpath Key: KAFKA-12341 URL: https://issues.apache.org/jira/browse/KAFKA-12341 Project: Kafka Iss

Client/Broker Compatibility Matrix

2021-02-18 Thread Gary Russell
You guys have done a great job enabling mixed client/broker compatibility since the early days, before 0.10.x.x Are there any plans to bring the matrix [1] up-to-date? Even just adding "any broker > 1.0.0 - Java: any version" (if that's true) would be helpful. TIA [1]: https://cwiki.apache.or

[jira] [Created] (KAFKA-12342) Get rid of raft/meta log shim layer

2021-02-18 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-12342: --- Summary: Get rid of raft/meta log shim layer Key: KAFKA-12342 URL: https://issues.apache.org/jira/browse/KAFKA-12342 Project: Kafka Issue Type: Improve

[jira] [Created] (KAFKA-12343) Recent change to use SharedTopicAdmin in KakfkaBasedLog fails with AK 0.10.x brokers

2021-02-18 Thread Randall Hauch (Jira)
Randall Hauch created KAFKA-12343: - Summary: Recent change to use SharedTopicAdmin in KakfkaBasedLog fails with AK 0.10.x brokers Key: KAFKA-12343 URL: https://issues.apache.org/jira/browse/KAFKA-12343

[jira] [Created] (KAFKA-12344) Support SlidingWindows in the Scala API

2021-02-18 Thread Leah Thomas (Jira)
Leah Thomas created KAFKA-12344: --- Summary: Support SlidingWindows in the Scala API Key: KAFKA-12344 URL: https://issues.apache.org/jira/browse/KAFKA-12344 Project: Kafka Issue Type: Improvement

Re: Client/Broker Compatibility Matrix

2021-02-18 Thread Matthias J. Sax
Feel free to edit the wiki page directly. On 2/18/21 11:19 AM, Gary Russell wrote: > You guys have done a great job enabling mixed client/broker compatibility > since the early days, before 0.10.x.x > > Are there any plans to bring the matrix [1] up-to-date? > > Even just adding "any broker > 1

[jira] [Resolved] (KAFKA-12331) KafkaRaftClient should use the LEO when appending LeaderChangeMessage

2021-02-18 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12331?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-12331. - Resolution: Fixed > KafkaRaftClient should use the LEO when appending LeaderChangeMessag

[jira] [Resolved] (KAFKA-12278) Keep api versions consistent with api scope

2021-02-18 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12278?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-12278. - Resolution: Fixed > Keep api versions consistent with api scope > --

[jira] [Resolved] (KAFKA-12232) Distinguish API scope by broker/controller

2021-02-18 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12232?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-12232. - Resolution: Duplicate > Distinguish API scope by broker/controller > ---

[jira] [Created] (KAFKA-12345) KIP-500: AlterIsrManager crashes on broker idle-state

2021-02-18 Thread Alok Nikhil (Jira)
Alok Nikhil created KAFKA-12345: --- Summary: KIP-500: AlterIsrManager crashes on broker idle-state Key: KAFKA-12345 URL: https://issues.apache.org/jira/browse/KAFKA-12345 Project: Kafka Issue Typ

Re: About Kafka 2.7.0 source code compilation error

2021-02-18 Thread deng ziming
Hello, please use gradlew, for example `./gradlew jar` `./gradlew idea`, or you can use the gradle plugin of IDEA. The `@ nowarn` warn seems to be related to different version of scala and jdk which you can just ignore. > On Feb 18, 2021, at 16:38, 韩可 mailto:han...@cvicse.com>> > wrote: > > H

Re: About Kafka 2.7.0 source code compilation error

2021-02-18 Thread Luke Chen
Yes, and also, you can check the readme in kafka github repo for more details. ref: https://github.com/apache/kafka/blob/trunk/README.md Thanks. Luke On Fri, Feb 19, 2021 at 9:51 AM deng ziming wrote: > Hello, please use gradlew, for example `./gradlew jar` `./gradlew idea`, > or you can use th

[jira] [Resolved] (KAFKA-9524) Default window retention does not consider grace period

2021-02-18 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9524?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-9524. Fix Version/s: 2.8.0 Resolution: Fixed > Default window retention does not consider

[jira] [Resolved] (KAFKA-12258) Change the BatchAccumulator to split records into batches

2021-02-18 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12258?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-12258. - Resolution: Fixed > Change the BatchAccumulator to split records into batches >

Splitting partition may cause message loss for consumers

2021-02-18 Thread Haruki Okada
Hi, Kafka. Recently I noticed that splitting partition may cause message delivery loss for consumers with auto.offset.reset=latest. I described details in https://issues.apache.org/jira/browse/KAFKA-12261 . Since delivery loss is undesired in most cases, I think this should be described in Consu

[jira] [Created] (KAFKA-12346) punctuate is called at twice the duration passed as the first argument to Processor.Schedule (with PunctuationType.WALL_CLOCK_TIME)

2021-02-18 Thread Arindam Ray (Jira)
Arindam Ray created KAFKA-12346: --- Summary: punctuate is called at twice the duration passed as the first argument to Processor.Schedule (with PunctuationType.WALL_CLOCK_TIME) Key: KAFKA-12346 URL: https://issues.apa

Re: Splitting partition may cause message loss for consumers

2021-02-18 Thread Luke Chen
Hi Okada san, Yes, I agree the "latest" setting in this situation is not good, and we should document it. But I don't think we should change the default auto.offset.reset setting to the earliest. The auto.offset.reset setting starts before kafka V1.0, which means, there are already a lot of users u

Re: Splitting partition may cause message loss for consumers

2021-02-18 Thread Haruki Okada
Thanks for your quick response. Yeah, agree with that. (also replied on the issue) 2021年2月19日(金) 16:25 Luke Chen : > Hi Okada san, > Yes, I agree the "latest" setting in this situation is not good, and we > should document it. > But I don't think we should change the default auto.offset.reset set