Re: [DISCUSS] KIP-740: Replace the public TaskId class with an interface

2021-05-18 Thread Sophie Blee-Goldman
Another update: After going through all the options currently on the table, I've decided to go back to an earlier version of the proposal, in which we just fix the TaskMetadata API to return a TaskId object rather than its String representation, and just clean up the aspects of the TaskId class

Re: [DISCUSS] KIP-737 Add canTrackSource to ReplicationPolicy

2021-05-18 Thread Ryanne Dolan
Hey Matthew, as you call out in the KIP there are few impls floating around, including my WIP PR here: https://github.com/apache/kafka/pull/10652 The tests are currently passing except for a couple asserts related to failback (commented out). It appears your PR doesn't address failback, so I

Re: [DISCUSS] KIP-739: Block Less on KafkaProducer#send

2021-05-18 Thread Ryanne Dolan
I was thinking the sender would typically wrap send() in a backoff/retry loop, or else ignore any failures and drop sends on the floor (fire-and-forget), and in both cases I think failing immediately is better than blocking for a new spot in the queue or asynchronously failing somehow. I think a

[jira] [Resolved] (KAFKA-10195) Move offset management codes from ConsumerCoordinator to a new class

2021-05-18 Thread dengziming (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-10195?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] dengziming resolved KAFKA-10195. Resolution: Won't Fix > Move offset management codes from ConsumerCoordinator to a new class >

Re: [DISCUSS] KIP-741: Change default serde to be null

2021-05-18 Thread Sophie Blee-Goldman
> > what I think should be a small discussion Dangerous words :P I'm all for the proposal but I do have one question about something in the KIP. You list two methods called defaultKeySerde() and defaultValueSerde() but it's not clear to me where these are coming from. Are they new APIs you

Jenkins build is unstable: Kafka » Kafka Branch Builder » trunk #140

2021-05-18 Thread Apache Jenkins Server
See

Re: KafkaStreams Scala: Replace FunctionsCompatConversions

2021-05-18 Thread John Roesler
Hey Rama, Yes, that's right, the EmbeddedKafkaCluster is not a "public API", which is a perennial source of disappointment, since it is very useful for integration tests. I think that's another area that's "nice to have", but no one has really volunteered to write a KIP and add it officially.

Re: [DISCUSS] KIP-739: Block Less on KafkaProducer#send

2021-05-18 Thread Nakamura
Hi Ryanne, Hmm, that's an interesting idea. Basically it would mean that after calling send, you would also have to check whether the returned future had failed with a specific exception. I would be open to it, although I think it might be slightly more surprising, since right now the paradigm

Build failed in Jenkins: Kafka » Kafka Branch Builder » trunk #139

2021-05-18 Thread Apache Jenkins Server
See Changes: -- [...truncated 3706 lines...] [2021-05-18T20:03:10.987Z] Execution failed for task ':raft:compileJava'. [2021-05-18T20:03:10.987Z] > Compilation failed; see

[GitHub] [kafka-site] mimaison commented on pull request #354: Updated downloads.html to include 2.7.1

2021-05-18 Thread GitBox
mimaison commented on pull request #354: URL: https://github.com/apache/kafka-site/pull/354#issuecomment-843514116 @mjsax Can you take another look? Thanks -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL

Build failed in Jenkins: Kafka » Kafka Branch Builder » trunk #138

2021-05-18 Thread Apache Jenkins Server
See Changes: -- [...truncated 3677 lines...] [2021-05-18T18:19:14.409Z] [2021-05-18T18:19:14.409Z] * Get more help at https://help.gradle.org [2021-05-18T18:19:14.409Z]

[jira] [Created] (KAFKA-12812) Consider refactoring state store registration path

2021-05-18 Thread Guozhang Wang (Jira)
Guozhang Wang created KAFKA-12812: - Summary: Consider refactoring state store registration path Key: KAFKA-12812 URL: https://issues.apache.org/jira/browse/KAFKA-12812 Project: Kafka Issue

Re: [VOTE] KIP-738: Removal of Connect's internal converter properties

2021-05-18 Thread Ryanne Dolan
+1 (non-binding) Thanks! Ryanne On Tue, May 18, 2021, 6:38 AM Chris Egerton wrote: > Hi all, > > I'd like to call for a vote on KIP-738: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-738%3A+Removal+of+Connect%27s+internal+converter+properties > > The discussion thread (which was

Re: [DISCUSS] KIP-739: Block Less on KafkaProducer#send

2021-05-18 Thread Ryanne Dolan
Moses, in the case of a full queue, could we just return a failed future immediately? Ryanne On Tue, May 18, 2021, 10:39 AM Nakamura wrote: > Hi Alexandre, > > Thanks for bringing this up, I think I could use some feedback in this > area. There are two mechanisms here, one for slowing down

Re: Request to be added the contributor list

2021-05-18 Thread Bruno Cadonna
Hi Matthew, I added you to the list of contributors in JIRA. You can now assign tickets to yourself. Thank you for your interest in Apache Kafka. Best, Bruno On 18.05.21 16:57, Matthew de Detrich wrote: Hello, I would like to start contributing to Apache Kafka, could you add my username

Re: Request to be added as a contributor in Kafka Project

2021-05-18 Thread Bruno Cadonna
Hi Abhiruchi, I added you to the contributors group in JIRA. You can now assign tickets to yourself. Thank you for your interest in Apache Kafka. Best, Bruno On 18.05.21 16:58, Abhiruchi Gupta wrote: Hi Bruno, I saw your reply from the archives regarding a mismatch in the username I

Jenkins build is still unstable: Kafka » Kafka Branch Builder » trunk #137

2021-05-18 Thread Apache Jenkins Server
See

Re: KafkaStreams Scala: Replace FunctionsCompatConversions

2021-05-18 Thread Rama Eshel
Hi John In the process of doing what you suggested, I have some scala-kafka-streams tests that fail to compile. They fail because they can't import EmbeddedKafkaCluster/IntegrationTestUtils. I 'jar tf'-ed, and found out that it's part of kafka-streams-n.n.n-*test* .jar, which is an artifact.

Re: [DISCUSS] KIP-739: Block Less on KafkaProducer#send

2021-05-18 Thread Nakamura
Hi Alexandre, Thanks for bringing this up, I think I could use some feedback in this area. There are two mechanisms here, one for slowing down when we don't have the relevant metadata, and the other for slowing down when a queue has filled up. Although the first one applies backpressure

Re: [DISCUSS] KIP-618: Atomic commit of source connector records and offsets

2021-05-18 Thread Chris Egerton
Hi Randall, Thanks for clarifying the issues with large transactions. I think we're starting to converge on an understanding of the problem space, if not quite on an approach to tackle it. I think my biggest issue with defining transaction boundaries on a per-task-poll basis are these points

Re: Request to be added as a contributor in Kafka Project

2021-05-18 Thread Abhiruchi Gupta
Hi Bruno, I saw your reply from the archives regarding a mismatch in the username I shared. And yes, indeed I made a typo while creating the username. Please go ahead and add the username *abhrigup* to the project. Thanks, Abhiruchi On Mon, May 17, 2021 at 10:35 PM Abhiruchi Gupta wrote: >

Request to be added the contributor list

2021-05-18 Thread Matthew de Detrich
Hello, I would like to start contributing to Apache Kafka, could you add my username (mdedetrich-aiven) to the contributors list for the Apache Kafka project? Regards -- Matthew de Detrich *Aiven Deutschland GmbH* Immanuelkirchstraße 26, 10405 Berlin Amtsgericht Charlottenburg, HRB 209739 B

[jira] [Created] (KAFKA-12811) kafka-topics.sh should let the user know they cannot adjust the replication factor for a topic using the --alter flag and not warn about missing the --partition flag

2021-05-18 Thread Richard Joerger (Jira)
Richard Joerger created KAFKA-12811: --- Summary: kafka-topics.sh should let the user know they cannot adjust the replication factor for a topic using the --alter flag and not warn about missing the --partition flag Key:

[jira] [Created] (KAFKA-12810) Remove deprecated TopologyDescription.Source#topics

2021-05-18 Thread Josep Prat (Jira)
Josep Prat created KAFKA-12810: -- Summary: Remove deprecated TopologyDescription.Source#topics Key: KAFKA-12810 URL: https://issues.apache.org/jira/browse/KAFKA-12810 Project: Kafka Issue Type:

[jira] [Created] (KAFKA-12809) Remove De

2021-05-18 Thread Josep Prat (Jira)
Josep Prat created KAFKA-12809: -- Summary: Remove De Key: KAFKA-12809 URL: https://issues.apache.org/jira/browse/KAFKA-12809 Project: Kafka Issue Type: Task Reporter: Josep Prat

[jira] [Created] (KAFKA-12808) Remove Deprecated methods under StreamsMetricsImpl

2021-05-18 Thread Josep Prat (Jira)
Josep Prat created KAFKA-12808: -- Summary: Remove Deprecated methods under StreamsMetricsImpl Key: KAFKA-12808 URL: https://issues.apache.org/jira/browse/KAFKA-12808 Project: Kafka Issue Type:

[jira] [Created] (KAFKA-12807) allow mTLS authorization based on different fields of X509 certificate

2021-05-18 Thread Naresh (Jira)
Naresh created KAFKA-12807: -- Summary: allow mTLS authorization based on different fields of X509 certificate Key: KAFKA-12807 URL: https://issues.apache.org/jira/browse/KAFKA-12807 Project: Kafka

[DISCUSS] KIP-741: Change default serde to be null

2021-05-18 Thread Leah Thomas
Hi all, I'd like to kick-off what I think should be a small discussion for KIP-741: Change default serde to be null. The wiki is here: https://cwiki.apache.org/confluence/display/KAFKA/KIP-741%3A+Change+default+serde+to+be+null Thanks, Leah

[jira] [Created] (KAFKA-12806) KRaft: Confusing leadership status exposed for controller without quorum

2021-05-18 Thread Gunnar Morling (Jira)
Gunnar Morling created KAFKA-12806: -- Summary: KRaft: Confusing leadership status exposed for controller without quorum Key: KAFKA-12806 URL: https://issues.apache.org/jira/browse/KAFKA-12806

[VOTE] KIP-738: Removal of Connect's internal converter properties

2021-05-18 Thread Chris Egerton
Hi all, I'd like to call for a vote on KIP-738: https://cwiki.apache.org/confluence/display/KAFKA/KIP-738%3A+Removal+of+Connect%27s+internal+converter+properties The discussion thread (which was originally titled with "KIP-736") can be found here:

Re: [VOTE] KIP-373: Allow users to create delegation tokens for other users

2021-05-18 Thread Matthew de Detrich
Forgot to mention the code that was never merged in the PR https://github.com/omkreddy/kafka/commit/fc47aa8d06828ef1de1c12b6c33192e10e3afd0c On Tue, May 18, 2021 at 11:42 AM Matthew de Detrich < matthew.dedetr...@aiven.io> wrote: > Apologies for necro/bump on this topic, but I am currently

Re: [VOTE] KIP-373: Allow users to create delegation tokens for other users

2021-05-18 Thread Matthew de Detrich
Apologies for necro/bump on this topic, but I am currently trying to work on tihs topic and I noticed that the PR in question for KAFKA-6945 never up being created/merged (I have checked through git logs plus manually looking at the code). Is there a reason why this PR was never created/merged

[VOTE] KIP-737: Add canTrackSource to ReplicationPolicy

2021-05-18 Thread Matthew de Detrich
Hi everyone I would like to start a vote for the following KIP https://cwiki.apache.org/confluence/display/KAFKA/KIP-737%3A+Add+canTrackSource+to+ReplicationPolicy In summary, the KIP proposes to add a default boolean canTrackSource method to the ReplicationPolicy interface which allows

[jira] [Resolved] (KAFKA-12751) ISRs remain in in-flight state if proposed state is same as actual state

2021-05-18 Thread Rajini Sivaram (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-12751?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-12751. Reviewer: David Arthur Resolution: Fixed > ISRs remain in in-flight state if

[jira] [Created] (KAFKA-12805) Aborted send could have a different exception than DisconnectException

2021-05-18 Thread Nicolas Guyomar (Jira)
Nicolas Guyomar created KAFKA-12805: --- Summary: Aborted send could have a different exception than DisconnectException Key: KAFKA-12805 URL: https://issues.apache.org/jira/browse/KAFKA-12805

Re: Granting permission for Create KIP

2021-05-18 Thread Bruno Cadonna
Hi, I granted you permissions. Thank you for your interest in Apache Kafka! Best, Bruno On 18.05.21 04:32, ping lv wrote: Please grant permission for Create KIP to wiki ID: hiping

Re: Request to be added as a contributor in Kafka Project

2021-05-18 Thread Bruno Cadonna
Hi Abhiruchi, I cannot find the username abhirgup, but I found abhrigup. Could it be that a typo happened somewhere? Is abhrigup your JIRA username? Best, Bruno On 17.05.21 19:05, Abhiruchi Gupta wrote: Hello, My name is Abhiruchi Gupta. I am a software developer with around 5 years of