Exposing additional metadata in Kafka Connect schema parameters

2018-03-06 Thread Gunnar Morling
Hi, A user of the Debezium CDC Kafka Connect connectors has asked whether we could provide information about the original source type of captured table columns. Usually the type info we provide by using the Kafka Connect types and some custom semantic types is good enough. But there are some case

KIP-199 - Could offset management be part of the Connect REST API?

2017-11-17 Thread Gunnar Morling
Hi, I was reading KIP-199 [1] for adding a tool for Kafka Connect offset management. This would be a very useful functionality for users of the Debezium CDC connectors, too. What I was wondering, instead of having a separate tool for this, has it been considered to expose offset management via th

Re: [DISCUSS] KIP-585: Conditional SMT

2020-04-03 Thread Gunnar Morling
Hi all, Thanks a lot for this initiative, Tom! To shed some light, the use case where this first came up, were issues we saw with SMTs being applied to the different topics produced by the Debezium change data capture connectors. There are different kinds of topics (for change data, schema histor

Re: [VOTE] KIP 585: Filter and conditional SMTs

2020-05-19 Thread Gunnar Morling
+1 (non-binding) Thanks for working on this, Tom! This KIP will be very useful for connectors like Debezium. --Gunnar Am Fr., 15. Mai 2020 um 20:02 Uhr schrieb Konstantine Karantasis : > > +1 (binding) > > Thanks Tom. > > Konstantine > > On Fri, May 15, 2020 at 5:03 AM Andrew Schofield > wrote:

[Connect] Different validation requirements for connector creation and update

2021-01-21 Thread Gunnar Morling
Hi, In the Debezium community, we ran into an interesting corner case of connector config validation [1]. The Debezium Postgres connector requires a database resource called a "replication slot", which identifies this connector to the database and tracks progress it has made reading the TX log. T

Testing KRaft mode

2021-05-16 Thread Gunnar Morling
Hi, I was testing the early access preview of the new ZooKeeper-less mode and noticed two things I wanted to bring up here. My testing scenario was a cluster of three Kafka nodes in combined mode and a single Kafka Connect node, all running via Docker Compose. * I stopped two of the Kafka nodes;

Re: Testing KRaft mode

2021-05-17 Thread Gunnar Morling
compatible' (easy mode) or cherry-pick the recently-merged fix for it ( > https://github.com/apache/kafka/pull/10014) onto the version of Connect > you're running, rebuild, and try again. Either step should prevent the > symptoms you describe, and if not, a new Jira ticket is likel

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

2021-05-27 Thread Gunnar Morling
Chris, all, I've just read KIP-618, and let me congratulate you first of all for this impressive piece of work! Here's a few small suggestions and questions I had while reading: * TransactionContext: What's the use case for the methods accepting a source record (commitTransaction(SourceRecord rec

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

2021-05-27 Thread Gunnar Morling
, e.g. hours or days (at least in theory). Or would this sort of usage not be considered a reasonable one? Thanks, --Gunnar Am Do., 27. Mai 2021 um 23:15 Uhr schrieb Gunnar Morling < gunnar.morl...@googlemail.com>: > Chris, all, > > I've just read KIP-618, and let me congratul

Re: Handling retriable exceptions during Connect source task start

2021-08-09 Thread Gunnar Morling
Hi, To ask slightly differently: would there be interest in a pull request for implementing retries, in case RetriableException is thrown from the Task::start() method? Thanks, --Gunnar Am Do., 5. Aug. 2021 um 22:27 Uhr schrieb Sergei Morozov : > Hi, > > I'm trying to address an issue in Debe

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

2021-11-16 Thread Gunnar Morling
Hi, I'm +1 for adding a GET endpoint for obtaining config definitions. It always felt odd to me that one has to issue a PUT for that purpose. If nothing else, it'd be better in terms of discoverability of the KC REST API. One additional feature request I'd have is to expose the valid enum constan

Requesting permissions to contribute to Apache Kafka

2021-11-17 Thread Gunnar Morling
Hi, As per the instructions given in [1], I would like to request the permissions for creating a KIP. My ids are: * Wiki: gunnarmorling * Jira: gunnar.morling Thanks a lot, --Gunnar [1] https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals

Re: Requesting permissions to contribute to Apache Kafka

2021-11-17 Thread Gunnar Morling
Excellent, thank you so much for the quick help, David! --Gunnar Am Mi., 17. Nov. 2021 um 10:42 Uhr schrieb David Jacot : > Hi Gunnar, > > I have granted you the requested permissions. > > I am looking forward to your contributions. > > Best, > David > > On Wed, N

Re: Do we want to add more SMTs to Apache Kafka?

2021-11-19 Thread Gunnar Morling
Hi all, Just came across this thread, I hope the late reply is ok. FWIW, we're in a similar situation in Debezium, where users often request new (Debezium-specific) SMTs, and we generally tend to recommend them to be maintained by users themselves, unless they are truly generic. This excludes a s

[DISCUSS] KIP-802: Validation Support for Kafka Connect SMT Options

2021-11-24 Thread Gunnar Morling
Hey all, I would like to propose a KIP for Apache Kafka Connect which adds validation support for SMT-related configuration options: https://cwiki.apache.org/confluence/display/KAFKA/KIP-802%3A+Validation+Support+for+Kafka+Connect+SMT+Options This feature allows users to make sure an SMT is con

Re: Handling retriable exceptions during Connect source task start

2021-11-26 Thread Gunnar Morling
. 2021 um 10:47 Uhr schrieb Gunnar Morling < gunnar.morl...@googlemail.com>: > Hi, > > To ask slightly differently: would there be interest in a pull request for > implementing retries, in case RetriableException is thrown from the > Task::start() method? > > Thanks, > &

Re: [DISCUSS] KIP-802: Validation Support for Kafka Connect SMT Options

2021-12-21 Thread Gunnar Morling
tps://cwiki.apache.org/confluence/display/KAFKA/KIP-769%3A+Connect+APIs+to+list+all+plugins+and+retrieve+their+configuration+definitions#KIP769:ConnectAPIstolistallpluginsandretrievetheirconfigurationdefinitions-PublicInterfaces > (section labeled "Converter interface" > > Cheers,

Re: [VOTE] KIP-665 Kafka Connect Hash SMT

2020-10-22 Thread Gunnar Morling
Hey Brandon, I think that's an interesting idea, we got something as a built-in connector feature in Debezium, too [1]. Two questions: * Can "field" select nested fields, e.g. "after.email"? * Did you consider an option for specifying salt for the hash functions? --Gunnar [1] https://debezium.i

[jira] [Created] (KAFKA-7058) ConnectSchema#equals() broken for array-typed default values

2018-06-14 Thread Gunnar Morling (JIRA)
Gunnar Morling created KAFKA-7058: - Summary: ConnectSchema#equals() broken for array-typed default values Key: KAFKA-7058 URL: https://issues.apache.org/jira/browse/KAFKA-7058 Project: Kafka

[jira] [Created] (KAFKA-6456) Improve JavaDoc of SourceTask

2018-01-17 Thread Gunnar Morling (JIRA)
Gunnar Morling created KAFKA-6456: - Summary: Improve JavaDoc of SourceTask Key: KAFKA-6456 URL: https://issues.apache.org/jira/browse/KAFKA-6456 Project: Kafka Issue Type: Improvement

[jira] [Created] (KAFKA-6551) Unbounded queues in WorkerSourceTask cause OutOfMemoryError

2018-02-09 Thread Gunnar Morling (JIRA)
Gunnar Morling created KAFKA-6551: - Summary: Unbounded queues in WorkerSourceTask cause OutOfMemoryError Key: KAFKA-6551 URL: https://issues.apache.org/jira/browse/KAFKA-6551 Project: Kafka

[jira] [Created] (KAFKA-6566) SourceTask#stop() not called after exception raised in poll()

2018-02-15 Thread Gunnar Morling (JIRA)
Gunnar Morling created KAFKA-6566: - Summary: SourceTask#stop() not called after exception raised in poll() Key: KAFKA-6566 URL: https://issues.apache.org/jira/browse/KAFKA-6566 Project: Kafka

[jira] [Created] (KAFKA-7336) Kafka Connect source task when producing record with invalid topic name

2018-08-24 Thread Gunnar Morling (JIRA)
Gunnar Morling created KAFKA-7336: - Summary: Kafka Connect source task when producing record with invalid topic name Key: KAFKA-7336 URL: https://issues.apache.org/jira/browse/KAFKA-7336 Project

[jira] [Created] (KAFKA-8476) Kafka 2.2.1 distribution contains JAX-RS API twice

2019-06-03 Thread Gunnar Morling (JIRA)
Gunnar Morling created KAFKA-8476: - Summary: Kafka 2.2.1 distribution contains JAX-RS API twice Key: KAFKA-8476 URL: https://issues.apache.org/jira/browse/KAFKA-8476 Project: Kafka Issue

[jira] [Created] (KAFKA-8523) InsertField transformation fails when encountering tombstone event

2019-06-11 Thread Gunnar Morling (JIRA)
Gunnar Morling created KAFKA-8523: - Summary: InsertField transformation fails when encountering tombstone event Key: KAFKA-8523 URL: https://issues.apache.org/jira/browse/KAFKA-8523 Project: Kafka

[jira] [Created] (KAFKA-12801) High CPU load after restarting brokers subsequent to quorum loss

2021-05-17 Thread Gunnar Morling (Jira)
Gunnar Morling created KAFKA-12801: -- Summary: High CPU load after restarting brokers subsequent to quorum loss Key: KAFKA-12801 URL: https://issues.apache.org/jira/browse/KAFKA-12801 Project: Kafka

[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 Project

[jira] [Created] (KAFKA-13478) KIP-802: Validation Support for Kafka Connect SMT Options

2021-11-24 Thread Gunnar Morling (Jira)
Gunnar Morling created KAFKA-13478: -- Summary: KIP-802: Validation Support for Kafka Connect SMT Options Key: KAFKA-13478 URL: https://issues.apache.org/jira/browse/KAFKA-13478 Project: Kafka

[jira] [Created] (KAFKA-13485) Restart connectors after RetriableException raised from Task::start()

2021-11-26 Thread Gunnar Morling (Jira)
Gunnar Morling created KAFKA-13485: -- Summary: Restart connectors after RetriableException raised from Task::start() Key: KAFKA-13485 URL: https://issues.apache.org/jira/browse/KAFKA-13485 Project