Re: [Kafka Connect] Dead letter queues for source connectors?

2024-03-07 Thread Chris Egerton
ErrantRecordReporter, which allows reporting > affected source partition/source offsets when a meaningful key or > value cannot be read. > > Thanks, > Greg > > On Tue, Mar 5, 2024 at 3:03 PM Chris Egerton > wrote: > > > > Hi Greg, > > > > This was my underst

Re: [Kafka Connect] Dead letter queues for source connectors?

2024-03-05 Thread Chris Egerton
Hi Greg, This was my understanding as well--if we can't turn a record into a byte array on the source side, it's difficult to know exactly what to write to a DLQ topic. One idea I've toyed with recently is that we could write the source partition and offset for the failed record (assuming,

Re: [ANNOUNCE] Apache Kafka 3.7.0

2024-02-27 Thread Chris Egerton
a, Alyssa Huang, Aman Singh, Andras Katona, Andrew > Schofield, Anna Sophie Blee-Goldman, Anton Agestam, Apoorv Mittal, > Arnout Engelen, Arpit Goyal, Artem Livshits, Ashwin Pankaj, > ashwinpankaj, atu-sharm, bachmanity1, Bob Barrett, Bruno Cadonna, > Calvin Liu, Cerchie, chern, Chris Egerton, Chris

Re: [PROPOSAL] Add commercial support page on website

2024-01-11 Thread Chris Egerton
Hi François, Is it an official policy of the ASF that projects provide a listing of commercial support options for themselves? I understand that other projects have chosen to provide one, but this doesn't necessarily imply that all projects should do the same, and I can't say I find this point

[DISCUSS] Kafka Connect source task interruption semantics

2023-12-12 Thread Chris Egerton
Hi all, I'd like to solicit input from users and maintainers on a problem we've been dealing with for source task cleanup logic. If you'd like to pore over some Jira history, here's the primary link: https://issues.apache.org/jira/browse/KAFKA-15090 To summarize, we accidentally introduced a

Re: [ANNOUNCE] Apache Kafka 3.6.0

2023-10-11 Thread Chris Egerton
gt; >> > > > > >> ** Building real-time streaming data pipelines that reliably get > data > > > > >> between systems or applications. > > > > >> > > > > >> ** Building real-time streaming applications that transform or

Re: Kafka Connect - Customize REST request headers

2023-10-07 Thread Chris Egerton
Hi Yeikel, Neat question! And thanks for the link to the RestClient code; very helpful. I don't believe there's a way to configure Kafka Connect to add these headers to forwarded requests right now. You may be able to do some kind of out-of-band proxy magic to intercept forwarded requests and

Re: requesting ability to be assigned JIRA ticket

2023-08-15 Thread Chris Egerton
Hi Neil, You should be good to go now. Thanks for your interest in contributing to Apache Kafka! Cheers, Chris On Tue, Aug 15, 2023 at 12:28 PM Neil Buesing wrote: > Looking to make the minor fix to the documentation for a bug I reported , > KAFKA-13945, so I need to get my JIRA ID added. >

Re: [VOTE] 3.5.1 RC1

2023-07-17 Thread Chris Egerton
Hi Divij, Thanks for running this release! To verify, I: - Built from source using Java 11 with both: - - the 3.5.1-rc1 tag on GitHub - - the kafka-3.5.1-src.tgz artifact from https://home.apache.org/~divijv/kafka-3.5.1-rc1/ - Checked signatures and checksums - Ran the quickstart using the

Re: Kafka Connect exactly-once semantic and very large transactions

2023-06-08 Thread Chris Egerton
Hi Vojta, >From my limited understanding of the Debezium snapshot process, I believe that you're correct that producing the entire snapshot in a transaction is the way to provide exactly-once semantics during that phase. If there's a way to recover in-progress snapshots and skip over

Re: [VOTE] 3.4.1 RC3

2023-05-30 Thread Chris Egerton
Hi Luke, Many thanks for your continued work on this release! To verify, I: - Built from source using Java 11 with both: - - the 3.4.1-rc3 tag on GitHub - - the kafka-3.4.1-src.tgz artifact from https://home.apache.org/~showuon/kafka-3.4.1-rc3/ - Checked signatures and checksums - Ran the

Re: [VOTE] 3.4.1 RC1

2023-05-22 Thread Chris Egerton
Hi Luke, Thanks for running the release! Steps I took to verify: - Built from source with Java 11 - Checked signatures and checksums - Ran the quickstart with Java 11 in KRaft mode - Ran all unit tests - - The org.apache.kafka.common.utils.UtilsTest.testToLogDateTimeDormat test case failed

Re: MirrorMaker 2.0 question

2023-03-20 Thread Chris Egerton
Mi Miguel, How many nodes are you running MM2 with? Just one? Separately, do you notice anything at ERROR level in the logs? Cheers, Chris On Mon, Mar 20, 2023 at 5:35 PM Miguel Ángel Fernández Fernández < miguelangelprogramac...@gmail.com> wrote: > Hello, > > I'm doing some tests with

Re: Exactly once kafka connect query

2023-03-16 Thread Chris Egerton
mpty list for the current > > > batch > > > > to > > > > > poll() and then when the next file comes in and poll sees new > > records, > > > I > > > > > see InvalidProducerEpochException. > > > > > Please advise me

Re: Ask to join the contribution list.

2023-03-16 Thread Chris Egerton
Hi Gary, You should be good to go now. Cheers, Chris On Thu, Mar 16, 2023 at 10:14 AM Gary Lee wrote: > Hi, > > I just spot an issue related to Kafka Connect ( > https://github.com/apache/kafka/pull/13398) > > I think this issue has been opened at >

Re: Exactly once kafka connect query

2023-03-14 Thread Chris Egerton
t;> producer with the same transactionalId which fences the current > one. > >>> > >> 2023-03-12 11:32:45,222 ERROR [json-sftp-source-connector|task-0] > >>> > >> ExactlyOnceWorkerSourceTask{id=json-sftp-source-connector-0} > Failed >

Re: Exactly once kafka connect query

2023-03-13 Thread Chris Egerton
source-connector-0} Task threw > an > >> uncaught and unrecoverable exception. Task is being killed and will not > >> recover until manually restarted > >> (org.apache.kafka.connect.runtime.WorkerTask) > >> [task-thread-json-sftp-source-connector-0] > >> &g

Re: Exactly once kafka connect query

2023-03-10 Thread Chris Egerton
eturned any > records from SourceTask::poll since the last transaction was > committed/aborted)? --- Yes, that case is possible for us. There is a case > where the first record itself an error record. > > Thanks, > Nitty > > On Thu, Mar 9, 2023 at 3:48 PM Chris Egerton >

Re: Exactly once kafka connect query

2023-03-09 Thread Chris Egerton
. If you see I added the condition >> of transactionContext check to implement exactly once, without >> transaction it was just throwing the exception without calling the >> addLastRecord() method and in the catch block it just logs the message and >> return the list of reco

Re: Exactly once kafka connect query

2023-03-08 Thread Chris Egerton
ng wrong here. > > Please advise. > Thanks, > Nitty > > On Tue 7 Mar 2023 at 2:21 p.m., Chris Egerton > wrote: > > > Hi Nitty, > > > > We've recently added some documentation on implementing exactly-once > source > > connectors here: > > > https:/

Re: Exactly once kafka connect query

2023-03-07 Thread Chris Egerton
Hi Nitty, We've recently added some documentation on implementing exactly-once source connectors here: https://kafka.apache.org/documentation/#connect_exactlyoncesourceconnectors. To quote a relevant passage from those docs: > In order for a source connector to take advantage of this support, it

[ANNOUNCE] Apache Kafka 3.3.2

2023-01-23 Thread Chris Egerton
, Yelp, and Zalando, among others. A big thank you for the following 39 contributors to this release! A. Sophie Blee-Goldman, Alyssa Huang, Artem Livshits, Bill Bejeck, Calvin Liu, Chia-Ping Tsai, Chris Egerton, Christo Lolov, Colin Patrick McCabe, Dan Stelljes, David Arthur, David Jacot, Divij

Re: JIRA access

2023-01-23 Thread Chris Egerton
Hi Titouan, I've added you to our Jira project; you should be good to go now. Cheers, Chris On Mon, Jan 23, 2023 at 10:41 AM Titouan Chary wrote: > Hi, > > It seems that JIRA access are disabled by default. Is it the right email to > reach out in order to get a new JIRA account. I would like

[RESULTS] [VOTE] Release Apache Kafka version 3.3.2

2023-01-11 Thread Chris Egerton
This vote passes with 8 +1 votes (3 bindings) and no 0 or -1 votes. +1 votes PMC Members: * Manikumar Reddy * Mickael Maison * Tom Bentley Committers: * Satish Duggana Community: * Yash Mayya * Divij Vaidya * Jakub Scholz * Federico Valeri 0 votes * No votes -1 votes * No votes Vote thread:

[VOTE] 3.3.2 RC1

2022-12-21 Thread Chris Egerton
Hello Kafka users, developers and client-developers, This is the second candidate for release of Apache Kafka 3.3.2. This is a bugfix release with several fixes since the release of 3.3.1. A few of the major issues include: * KAFKA-14358 Users should not be able to create a regular topic name

[VOTE] 3.3.2 RC0

2022-12-15 Thread Chris Egerton
Hello Kafka users, developers and client-developers, This is the first candidate for release of Apache Kafka 3.3.2. This is a bugfix release with several fixes since the release of 3.3.1. A few of the major issues include: * KAFKA-14358 Users should not be able to create a regular topic name

[ANNOUNCE] Call for papers: Kafka Summit London 2023

2022-11-30 Thread Chris Egerton
Hi everyone, The call for papers (https://sessionize.com/kafka-summit-london-2023/) is now open for Kafka Summit London 2023, and you are all welcome to submit a talk. We are looking for the most interesting, most informative, most advanced, and most generally applicable talks on Apache Kafka®

Re: Granting permission for join jira

2022-11-09 Thread Chris Egerton
Hi, You should be good to go now. Cheers, Chris On Wed, Nov 9, 2022 at 12:52 AM hzhkafka wrote: > Jira id: hzh0425@apache >

Re: [ANNOUNCE] New Kafka PMC Member: Bruno Cadonna

2022-11-01 Thread Chris Egerton
Congrats! On Tue, Nov 1, 2022, 15:44 Bill Bejeck wrote: > Congrats Bruno! Well deserved. > > -Bill > > On Tue, Nov 1, 2022 at 3:36 PM Guozhang Wang wrote: > > > Hi everyone, > > > > I'd like to introduce our new Kafka PMC member, Bruno. > > > > Bruno has been a committer since April. 2021 and

Re: Granting permission for Kafka Contributor

2022-10-28 Thread Chris Egerton
Hi, You should be good to go now. Cheers, Chris On Fri, Oct 28, 2022 at 4:14 PM yuachieve1234 <260738...@qq.com.invalid> wrote: > Jira ID:yuxj109 > > > > > yuachieve1234 > 260738...@qq.com > > > >

Re: Entire Kafka Connect cluster stuck because of a stuck sink connector

2022-10-12 Thread Chris Egerton
Hi, What version of Kafka Connect are you running? This sounds like a bug that was fixed a few releases ago. Cheers, Chris On Wed, Oct 12, 2022, 21:27 Hemanth Savasere wrote: > We have stumbled upon an issue on a running cluster with multiple > source/sink connectors: > >1. One of our

Re: Apply to be a contributor of kafka

2022-10-09 Thread Chris Egerton
Hi Junyang, You should be good to go now, and I've also assigned the ticket to you. Cheers, Chris On Sun, Oct 9, 2022 at 11:36 AM Junyang Liu wrote: > Hi, > I’m a developer of kafka, and want to contribute to the project. I have > made a issue and a PR resolving the issue(Kafka-14285), but I

Re: [ANNOUNCE] New Kafka PMC Member: A. Sophie Blee-Goldman

2022-08-02 Thread Chris Egerton
Congrats, Sophie! On Mon, Aug 1, 2022 at 9:21 PM Luke Chen wrote: > Congrats Sophie! :) > > Luke > > On Tue, Aug 2, 2022 at 7:56 AM Adam Bellemare > wrote: > > > Congratulations Sophie! I’m glad to see you made as a PMC member! Well > > earned. > > > > > On Aug 1, 2022, at 6:42 PM, Guozhang

Re: [ANNOUNCE] New Committer: Chris Egerton

2022-07-25 Thread Chris Egerton
gt; > On Mon, Jul 25, 2022 at 4:23 PM Martin Gainty wrote: > > > Congratulations Chris! > > > > martin~ > > > > From: Mickael Maison > > Sent: Monday, July 25, 2022 12:25 PM > > To: dev ; Users > > Subject: [ANNO

Re: a little problem in quickstart

2022-06-26 Thread Chris Egerton
Hi Mason, You're correct that the quickstart should use 'libs' instead of 'lib'. This has already been fixed in the docs for the upcoming 3.3.0 release with https://github.com/apache/kafka/pull/12252. We might consider backporting that change; I've CC'd Luke Chen, who merged that fix and might be

Re: Kafka Connect - offset.storage.topic reuse across clusters

2022-03-30 Thread Chris Egerton
Connectors overwriting each other's offsets is the primary concern. If you have a guarantee that there will only ever be one connector with a given name running at once on any of the Connect clusters that use the same offsets topic, and you want offsets to be shared for all source connectors on

Re: Running multiple MM2 instances

2022-03-23 Thread Chris Egerton
Hi Julia, Sounds like KAFKA-9981 [1]. This is a known issue with MirrorMaker 2 that impacts horizontal scalability and has not yet been addressed. There is some work in progress to fix this issue [2], but the effort hasn't received much attention to date. There may be other issues as well, but

Re: securing sasl/scram username and password in kafka connect

2022-03-07 Thread Chris Egerton
tials.prop'@'172.x.x.x' (using > password: YES) > > On Mon, Mar 7, 2022 at 1:55 PM Chris Egerton > wrote: > > > Hi Men, > > > > That config snippet has a small syntax error: all double quotes should be > > escaped. Assumi

Re: securing sasl/scram username and password in kafka connect

2022-03-07 Thread Chris Egerton
> username=\"000\" password=\"00\";", > ... > } > } > > I changed the database.history.producer.sasl.jaas.config to: > > "database.history.producer.sasl.jaas.config": > "org.apache.kafka.common.security.scram.ScramLoginModule required > username="${file:/pat

Re: securing sasl/scram username and password in kafka connect

2022-03-07 Thread Chris Egerton
Hi Men, The config provider mechanism should work for every property in a connector config, and every property in a worker config except for the plugin.path property (see KAFKA-9845 [1]). You can also use it for only part of a single property, or even multiple parts, like in this example

Re: Is MirrorMaker 2 horizontally scalable?

2022-03-03 Thread Chris Egerton
Hi Dmitri, There's at least one issue with MirrorMaker 2 that impacts horizontal scalability and has not yet been addressed: https://issues.apache.org/jira/browse/KAFKA-9981. There is some work in progress to fix it (

Re: [VOTE] 2.8.1 RC1

2021-09-14 Thread Chris Egerton
Hi David, I took a look at the Javadocs and noticed a small issue. Using the search bar at the top of the landing page ( https://home.apache.org/~dajac/kafka-2.8.1-rc1/javadoc/), I entered "KafkaProducer" and clicked on the search item that came up. This brought me to

New Kafka Connector

2016-08-22 Thread Chris Egerton
, and the connector itself has been deployed to Maven Central (latest version is 0.2.1, but it may still be in the process of synching at the time of writing). Is there anything else you'd like to know about it before posting it to your page? Cheers! Chris Egerton Software Engineering Intern, WePay