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

2021-10-27 Thread Arjun Satish
Hi Knowles, Thanks for the KIP! Could you please call out some use-cases on what the source connectors would do when they hit such exceptions? I'm wondering if we would need to do anything other than skipping such records, writing some log messages, and/or writing some error context to a DLQ? On

Re: Request for permission to assign JIRA ticket (KAFKA-13403) to myself

2021-10-27 Thread Matthias J. Sax
What is you user name? On 10/27/21 6:08 PM, Arun Mathew wrote: Hi, Please give me the relevant permissions to take up tickets. -- With Regards, Arun Mathew

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

2021-10-27 Thread Apache Jenkins Server
See Changes: -- [...truncated 493797 lines...] [2021-10-28T02:14:10.118Z] [2021-10-28T02:14:10.118Z] AclAuthorizerTest > testEmptyAclThrowsException() STARTED [2021-10-28T

Request for permission to assign JIRA ticket (KAFKA-13403) to myself

2021-10-27 Thread Arun Mathew
Hi, Please give me the relevant permissions to take up tickets. -- With Regards, Arun Mathew

[jira] [Created] (KAFKA-13412) Retry of initTransactions after timeout may cause invalid transition

2021-10-27 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-13412: --- Summary: Retry of initTransactions after timeout may cause invalid transition Key: KAFKA-13412 URL: https://issues.apache.org/jira/browse/KAFKA-13412 Project: K

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

2021-10-27 Thread Chris Egerton
Hi Knowles, Thanks for the KIP. I may have more to say later but there's one thing I'd like to make sure to share now. In the Javadocs for the proposed SourceTask::ignoreNonRetriableProducerException method, the InvalidProducerEpochException exception class is included as an example of a non-retri

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

2021-10-27 Thread Apache Jenkins Server
See Changes: -- [...truncated 4155 lines...] [2021-10-27T22:26:32.483Z] [2021-10-27T22:26:32.483Z] See https://docs.gradle.org/7.2/userguide/command_line_interface.html#se

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

2021-10-27 Thread John Roesler
Hi Knowles, Thanks for the reply! That all sounds reasonable to me, and that's a good catch regarding the SourceRecord. Thanks, -John On Wed, 2021-10-27 at 15:32 -0400, Knowles Atchison Jr wrote: > John, > > Thank you for the response and feedback! > > I originally started my first pass with t

Re: [DISCUSS] KIP-786: Emit Metric Client Quota Values

2021-10-27 Thread Mason Legere
Hi All, Haven't received any feedback on this yet but as it was a small change have made a PR showing the functional components: pull request Will update the related documentation outlining the new metric attributes in a bit. Best, Mason Legere On Sat

Re: Apache Kafka : start up scripts

2021-10-27 Thread Israel Ekpo
Start here https://github.com/apache/kafka/blob/trunk/bin/kafka-server-start.sh https://github.com/apache/kafka/tree/trunk/core/src/main/scala/kafka/server Also take a look at the logs when the server starts up. That should give you some insights. On Wed, Oct 27, 2021 at 5:03 PM Kafka Life wro

[jira] [Created] (KAFKA-13411) Exception while connecting from kafka client consumer producers deployed in a wildfly context to a kafka broker implementing OAUTHBEARER sasl mechanism

2021-10-27 Thread Shankar Bhaskaran (Jira)
Shankar Bhaskaran created KAFKA-13411: - Summary: Exception while connecting from kafka client consumer producers deployed in a wildfly context to a kafka broker implementing OAUTHBEARER sasl mechanism Key: KAFKA-13411

Apache Kafka : start up scripts

2021-10-27 Thread Kafka Life
Dear Kafka experts when an broker is started using start script , could any of you please let me know the sequence of steps that happens in the back ground when the node UP like : when the script is initiated to start , 1/ is it checking indexes .. ? 2/ is it checking isr ? 3/ is URP being made t

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

2021-10-27 Thread Apache Jenkins Server
See Changes: -- [...truncated 494959 lines...] [2021-10-27T19:54:48.778Z] [2021-10-27T19:54:48.778Z] AclAuthorizerTest > testAuthorizerZkConfigFromPrefixOverrides() STARTE

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

2021-10-27 Thread Knowles Atchison Jr
John, Thank you for the response and feedback! I originally started my first pass with the ProducerRecord. For our connector, we need some of the information out of the SourceRecord to ack our source system. If I had the actual ProducerRecord, I would have to convert it back before I would be abl

[jira] [Created] (KAFKA-13410) KRaft Upgrades

2021-10-27 Thread David Arthur (Jira)
David Arthur created KAFKA-13410: Summary: KRaft Upgrades Key: KAFKA-13410 URL: https://issues.apache.org/jira/browse/KAFKA-13410 Project: Kafka Issue Type: New Feature Reporter:

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

2021-10-27 Thread John Roesler
Good morning, Knowles, Thanks for the KIP! To address your latest questions, it is fine to call for a vote if a KIP doesn't generate much discussion. Either the KIP was just not controversial enough for anyone to comment, in which case a vote is appropriate; or no one had time to review it, in wh

Re: Issue with Kafka consumers and producers on wildfly and SASL

2021-10-27 Thread Shankar Bhaskaran
Hi , I have a fix for this issue , how should i submit a patch ? Regards, Shankar On Mon, Aug 30, 2021 at 3:40 AM Shankar Bhaskaran wrote: > Hi , > > > > I have set up a Kafka cluster on my linux machine secured using keycloak > (OAUTHBEARER) Mechanism. I can use the Kafka Console Consumers a

Re: [DISCUSS] KIP-787 - MM2 Interface to manage Kafka resources

2021-10-27 Thread Ryanne Dolan
Well I'm convinced! Thanks for looking into it. Ryanne On Wed, Oct 27, 2021, 8:49 AM Omnia Ibrahim wrote: > I checked the difference between the number of methods in the Admin > interface and the number of methods MM2 invokes from Admin, and this diff > is enormous (it's more than 70 methods).

[jira] [Created] (KAFKA-13409) JUnit test runs often end with "non-zero exit value 1"

2021-10-27 Thread Colin McCabe (Jira)
Colin McCabe created KAFKA-13409: Summary: JUnit test runs often end with "non-zero exit value 1" Key: KAFKA-13409 URL: https://issues.apache.org/jira/browse/KAFKA-13409 Project: Kafka Issue

Re: [DISCUSS] KIP-787 - MM2 Interface to manage Kafka resources

2021-10-27 Thread Omnia Ibrahim
I checked the difference between the number of methods in the Admin interface and the number of methods MM2 invokes from Admin, and this diff is enormous (it's more than 70 methods). As far as I can see, the following methods MM2 depends on (in MirrorSourceConnector, MirrorMaker, MirrorCheckpointTa

Re: [VOTE] KIP-784: Add top-level error code field to DescribeLogDirsResponse

2021-10-27 Thread Luke Chen
Hi Mickael, Thanks for the KIP. It's good to keep it consistent with others, to have top-level error field. + 1 (non-binding) Thank you. Luke On Wed, Oct 27, 2021 at 9:01 PM Mickael Maison wrote: > Hi all, > > I'd like to start the vote on this minor KIP. > > https://cwiki.apache.org/confluenc

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

2021-10-27 Thread Knowles Atchison Jr
Good morning, Bumping this thread. Is there someone specific on the Connect framework team that I should ping? Is it appropriate to just call a vote? All source connectors are dead in the water without a way to handle producer write exceptions. Thank you. Knowles On Mon, Oct 18, 2021 at 8:33 AM

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

2021-10-27 Thread Apache Jenkins Server
See

[VOTE] KIP-784: Add top-level error code field to DescribeLogDirsResponse

2021-10-27 Thread Mickael Maison
Hi all, I'd like to start the vote on this minor KIP. https://cwiki.apache.org/confluence/display/KAFKA/KIP-784%3A+Add+top-level+error+code+field+to+DescribeLogDirsResponse Please take a look, vote or let me know if you have any feedback. Thanks

Re: [VOTE] KIP-780: Support fine-grained compression options

2021-10-27 Thread Luke Chen
Hi Dongjin, Thanks for the KIP. +1 (non-binding) Luke On Wed, Oct 27, 2021 at 8:44 PM Dongjin Lee wrote: > Bumping up the voting thread. > > If you have any questions or opinions, don't hesitate to leave them in the > discussion thread. > > Best, > Dongjin > > On Thu, Oct 14, 2021 at 3:02 AM Do

Re: [VOTE] KIP-780: Support fine-grained compression options

2021-10-27 Thread Dongjin Lee
Bumping up the voting thread. If you have any questions or opinions, don't hesitate to leave them in the discussion thread. Best, Dongjin On Thu, Oct 14, 2021 at 3:02 AM Dongjin Lee wrote: > Hi, Kafka dev, > > I'd like to open a vote for KIP-780: Support fine-grained compression > options: > >

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

2021-10-27 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-13408) Add a new metric to track invalid task provided offset

2021-10-27 Thread Nicolas Guyomar (Jira)
Nicolas Guyomar created KAFKA-13408: --- Summary: Add a new metric to track invalid task provided offset Key: KAFKA-13408 URL: https://issues.apache.org/jira/browse/KAFKA-13408 Project: Kafka

[jira] [Created] (KAFKA-13407) Kafka controller out of service after ZK leader restart

2021-10-27 Thread Daniel (Jira)
Daniel created KAFKA-13407: -- Summary: Kafka controller out of service after ZK leader restart Key: KAFKA-13407 URL: https://issues.apache.org/jira/browse/KAFKA-13407 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-13406) Cooperative sticky assignor got stuck due to assignment validation failed

2021-10-27 Thread Luke Chen (Jira)
Luke Chen created KAFKA-13406: - Summary: Cooperative sticky assignor got stuck due to assignment validation failed Key: KAFKA-13406 URL: https://issues.apache.org/jira/browse/KAFKA-13406 Project: Kafka