Re: Fwd: Request to be added to kafka contributors list

2024-05-21 Thread Greg Harris
res, or ask a committer for more information. Hope this helps, Greg Harris [1] https://issues.apache.org/jira/browse/KAFKA-16701 [2] https://issues.apache.org/jira/browse/KAFKA-16701?jql=project%20%3D%20KAFKA%20AND%20labels%20%3D%20flaky-test On Tue, May 21, 2024 at 6:43 AM Franck wrote: > &

Re: [ Questions on log4j file & version ]

2024-05-16 Thread Greg Harris
Hi Ashok, Kafka 2.7.1 was built from the 2.7.1 tag [1] and looking at the dependencies in that version [2], it should have shipped with 1.2.17. You can verify this by looking for the log4j jar in your installation. Because of the security vulnerabilities you mention, Kafka switched to reload4j in

Re: Kafka Producer avoid sending more records when met exception while also keeping performance

2024-03-11 Thread Greg Harris
ees (consistency) [2]. If you're not willing to pay the performance cost of max.in.flight.requests.per.connection=1, then you may need to make a compromise on the consistency and find a way to manage the extra data. Thanks, Greg Harris [1] https://kafka.apache.org/37/javadoc/org/apache/kafka/clie

Re: Tiered storage

2024-03-08 Thread Greg Harris
cache [5]. The Apache Kafka implementation serves directly from the plugin without caching [6], and the Aiven plugin has support for in-memory caching and disk caching [7]. Hope this clears things up! Greg Harris [1] https://docs.confluent.io/platform/current/clusters/tiered-storage.html [2

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

2024-03-05 Thread Greg Harris
ly dependent on how user-friendly the offsets published by the > connector are, and does come with the risk of data loss (if the upstream > system is wiped before skipped records can be recovered), but could be > useful in some scenarios. > > Thoughts? > > Chris > > On Tue,

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

2024-03-05 Thread Greg Harris
Hi Yeikel, Thanks for your question. It certainly isn't clear from the original KIP-298, the attached discussion, or the follow-up KIP-610 as to why the situation is asymmetric. The reason as I understand it is: Source connectors are responsible for importing data to Kafka. If an error occurs

Re: Is it possible to include only the connector name in worker logs?

2024-02-14 Thread Greg Harris
Hey Yeikel, Thanks for your question! It appears that we only expose the already-concatenated data to the logs: https://github.com/apache/kafka/blob/e8c70fce26626ed2ab90f2728a45f6e55e907ec1/connect/runtime/src/main/java/org/apache/kafka/connect/util/LoggingContext.java#L186-L204 instead of

Re: KRAFT: Correct way to add a controller only node

2024-02-08 Thread Greg Harris
Hi Denny, I believe the problem you're describing is the motivation for KIP-853 [1] which is currently under discussion for 3.8. If you're performing a rolling upgrade, I believe the current recommendation is to use DNS to "re-assign" the controller from one machine to another. If you're running

Re: Mirror Maker bidirectional offset sync

2024-01-12 Thread Greg Harris
org/apache/kafka/connect/mirror/MirrorSourceConfig.java#L137-L142 [2] https://github.com/apache/kafka/blob/21227bda61e75e3a8f1401ff94b27e9161cd3f1b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConfig.java#L137-L142 On Fri, Jan 12, 2024 at 1:53 PM Greg Harris

Re: Mirror Maker bidirectional offset sync

2024-01-12 Thread Greg Harris
ubmit a draft in the upcoming weeks, though I'd > need a bit of time to get a better grasp on the mirror connector codebase. > > Thank you both for your valuable insights so far! > > Jeroen > > On Thu, Jan 11, 2024 at 8:06 PM Greg Harris > wrote: > > > Hey Je

Re: Mirror Maker bidirectional offset sync

2024-01-11 Thread Greg Harris
d of offset replication into the Mirror connectors. I understand > this is not something that should be enabled by default, but having it > behind configuration toggle could help out users desiring a similar kind of > active/active setup and who understand the restrictions. Do you think

Re: Mirror Maker bidirectional offset sync

2024-01-11 Thread Greg Harris
ated > yet -- a remote topic by definition does not have any records that MM2 > didn't put there. So an offset for a consumer consuming from B's a.topic1 > can be translated back to an offset in A's topic1, where the data came from. > > Ryanne > > On Wed, Jan 10, 2024, 6:07 PM Greg H

Re: Mirror Maker bidirectional offset sync

2024-01-10 Thread Greg Harris
Hi Jeroen, I'm glad you're experimenting with MM2, and I hope we can give you some more context to explain what you're seeing. > I wrote a small program to produce these offset syncs for the prefixed > topic, and this successfully triggers the Checkpoint connector to start > replicating the

Re: MM2 Question

2024-01-08 Thread Greg Harris
Hi Vinay, I am sorry to hear about your difficulties with MirrorMaker2. Are you using the MirrorMaker2 dedicated mode, or do you have the MirrorMaker2 connectors running in a separate Connect cluster? If you're using the dedicated mode, that is a known problem on <3.5 and is the motivation for

Re: Mirror Maker bidirectional offset sync

2024-01-08 Thread Greg Harris
fail-back with a script, when you know the original consumer group will be offline. Neither of those seem very good to me, so you may need to design your application around this design constraint or handle the "fail-back" state management with some other mechanism. Hope this helps, Greg Harris

Re: where to capture a failed task's exception

2023-12-26 Thread Greg Harris
Hey Akash, Thanks for the question! For a direct answer, no: throwing exceptions from poll() is only one of many ways that a task can fail. If you look at the AK source, every failure ultimately uses the AbstractStatus.State.FAILED enum [1]. You can trace the usages of this enum back to see all

Re: Mirror Maker 2 - offset sync from source to target

2023-10-24 Thread Greg Harris
cumentation explaining the same ? > I tried to find a lot in the past > > ________ > From: Greg Harris > Sent: Monday, October 23, 2023 11:23:26 PM > To: users@kafka.apache.org > Subject: Re: Mirror Maker 2 - offset sync from source to target &

Re: Mirror Maker 2 - offset sync from source to target

2023-10-23 Thread Greg Harris
Alexander, My apologies for calling you Andrew. Greg On Mon, Oct 23, 2023 at 1:22 PM Greg Harris wrote: > > Andrew, > > Yes, there isn't an explicit "create consumer group" operation, it > should be created when MM2 emits a sync for it. > > Best, > Greg &

Re: Mirror Maker 2 - offset sync from source to target

2023-10-23 Thread Greg Harris
ot;4. The target group does not exist, or has no > active consumers" > If group on target does not exist, will it be created without active > consumers ? > > -Original Message- > From: Greg Harris > Sent: Monday, October 23, 2023 8:56 PM > To: users@kafka.apa

Re: Mirror Maker 2 - offset sync from source to target

2023-10-23 Thread Greg Harris
--Original Message- > From: Greg Harris > Sent: Monday, October 23, 2023 8:42 PM > To: users@kafka.apache.org > Subject: Re: Mirror Maker 2 - offset sync from source to target > > CAUTION: This email is from an external source. Please don’t open any unknown > links or at

Re: Mirror Maker 2 - offset sync from source to target

2023-10-23 Thread Greg Harris
, Greg Harris On Sun, Oct 22, 2023 at 5:36 AM Alexander Shapiro (ashapiro) wrote: > > Hi > Can someone advise please > if sync.group.offsets.enabled : true to sync offset from source to target for > particular consumer group > That group must be created on target, even if no

Re: The Plan To Introduce Virtual Threads To Kafka Connect

2023-10-16 Thread Greg Harris
/jira/browse/KAFKA-14606 . Thanks! Greg Harris On Mon, Oct 16, 2023 at 6:20 AM Boyee wrote: > > Kafka Connect as a kind of thread-intense program, can benifit a lot from the > usage of virtual threads. > From JDK 21, released in last month, virtual threads is a formal feature of >

Re: [kafka-clients] [VOTE] 3.6.0 RC1

2023-09-20 Thread Greg Harris
Hi all, I verified the functionality of KIP-898 and the recent fix for KAFKA-15473 with the following steps: 1. I started a 3.5.1 broker, and a 3.5.1 worker with most (>400) publicly available plugins installed 2. I captured the output of /connector-plugins 3. I upgraded the worker to 3.6.0-rc1

Re: Consumer group offset translation in Mirror Maker 2

2023-09-03 Thread Greg Harris
t; > > I read through the KIP 545 regarding the Mirror Maker 2 but could not get > > the context more on why this was being done, and can we still use the > > RemoteClusterUtils.translateOffsets() as we are using the Kafka 2.8.2 > > version which has some bugs related to negative consumer offsets > > https:

Re: Consumer group offset translation in Mirror Maker 2

2023-08-29 Thread Greg Harris
Hey Hemanth! Thank you for asking about Mirror Maker 2! Offset translation is not so simple, so I'll summarize the main functionality and leave some pointers into the code for you to examine yourself. 1. After MirrorSourceTask writes a record, it receives a commitRecord callback [1] with

Re: Kafka connect Graceful stop of task failed

2023-08-21 Thread Greg Harris
Hey Robson, Thanks for opening an issue on the JDBC repo, I think this is certainly relevant feedback for the connector developers. I commented on the issue with a potential regression that I saw, you can try downgrading your connector to see if the behavior improves. I also know that

Re: Kafka connect Graceful stop of task failed

2023-08-16 Thread Greg Harris
Hi Robson, Thank you for the detailed bug report. I believe the behavior that you're describing is caused by this flaw: https://issues.apache.org/jira/browse/KAFKA-15090 which is still under discussion. Since the above flaw was introduced in 3.0, source connectors need to return from poll()

Re: Kafka Connect Rest Extension Question

2023-07-31 Thread Greg Harris
redacted worker startup logs after https://github.com/apache/kafka/blob/3.5.0/connect/runtime/src/main/java/org/apache/kafka/connect/cli/AbstractConnectCli.java#L120 and before the worker starts printing configurations? Thanks, Greg Harris On Mon, Jul 31, 2023 at 5:38 AM 양형욱 wrote: > >

Re: [VOTE] 3.4.1 RC1

2023-05-22 Thread Greg Harris
or correctness of the connectors. If something else forces a new RC, perhaps consider rolling this into the next RC. Thanks, Greg On Mon, May 22, 2023 at 1:47 PM Greg Harris wrote: > > Hi Luke, > > I performed a test upgrade of MM2 from 3.4.0 to 3.4.1-RC1, and > verified that the new off

Re: Re: Kafka 3.2.1 performance issue with JDK 11

2023-05-22 Thread Greg Harris
ade JDK from 11 to 17. Do you > recommend this solution? > > On 2023/05/21 17:58:42 Greg Harris wrote: > > Vic, > > > > I found an open JIRA issue that previously reported this problem: > > https://issues.apache.org/jira/browse/KAFKA-10877 . > > I believe o

Re: [VOTE] 3.4.1 RC1

2023-05-22 Thread Greg Harris
Hi Luke, I performed a test upgrade of MM2 from 3.4.0 to 3.4.1-RC1, and verified that the new offset translation logic worked as intended. Steps I took to verify: - Downloaded 3.4.0_2.13 from the Kafka website - Formatted and started two 3.4.0 1-node clusters, and configured MM2 to mirror data

Re: Kafka 3.2.1 performance issue with JDK 11

2023-05-21 Thread Greg Harris
/jira/browse/KAFKA-9366 . I will look into reviving or replacing the performance patch for 3.x. Hope this helps, Greg Harris On Sun, May 21, 2023 at 6:31 AM Vic Xu wrote: > > Hello all, I have a Kafka cluster deployed with version 3.2.1 , JDK 11 and > log4j 2.18.0. I built my own Ka

Re: Kafka connect process listens to an unknown port

2023-05-19 Thread Greg Harris
re relevant. Thanks for the question, I learned something new! Greg Harris On Fri, May 19, 2023 at 4:45 AM Jorge Martin Cristobal wrote: > > Hi all, > > I'm testing apache kafka connect for a project and I found that the main > process listens to two different ports, the one

Re: MM2 -- failed to copy old data

2023-05-02 Thread Greg Harris
hope this helps! Greg Harris On Tue, May 2, 2023 at 12:49 PM An, Hongguo (CORP) wrote: > Hi: > When I am running MM2 (3.4.0), any message older than 1 days are not > copying, the topic has retention period as 5 days. > Got error: > > org.apache.kafka.common.errors.Invalid

Re: Sudden imbalance between partitions

2023-03-24 Thread Greg Harris
the problem on its own, and may make it temporarily worse while partitions are being replicated to the added nodes. If you're already running the patched version of the partitioner, then a more detailed investigation will be necessary. I hope some of this helps! Greg Harris On Fri, Mar 24, 2023

Re: where is KafkaYammerMetrics.java moved to?

2023-03-22 Thread Greg Harris
to breakages like this when updating, even in patch releases. This is not recommended, and you should look into a long-term replacement for the KafkaYammerMetrics class. I hope this helps, Greg Harris On Wed, Mar 22, 2023 at 6:27 AM Rui wrote: > Hi Kafka group: > recently I upgrade thes

Re: Kafka Connect ClusterConfigState.inconsistentConnectors() not handled by distributed Worker?

2023-02-15 Thread Greg Harris
even without any bugs present, as the connectors have to transit through state (A) on each reconfiguration. We can look into this after getting some tactical fixes in place to avoid the long-term state (A). Thanks, Greg Harris On Wed, Feb 15, 2023 at 9:34 AM Frank Grimes wrote: > So we've just hit th

Re: Kafka Connect ClusterConfigState.inconsistentConnectors() not handled by distributed Worker?

2023-02-09 Thread Greg Harris
possible slightly > negative consequence of that change would be that re-submitting the same > config which would effectively be a no-op in the current implementation > would now force task reconfigures/restarts? > On Wednesday, February 8, 2023, 12:47:19 PM EST, Greg Harris > wrote

Re: Mirror maker worker can't issue with REST uri

2023-02-08 Thread Greg Harris
. The MirrorMaker2 nodes started by the entry script connect-mirror-maker.sh will not have the REST API enabled, and cannot perform these operations. Greg On Wed, Feb 8, 2023 at 10:02 AM Greg Harris wrote: > Anup, > > Here's the best workaround I can think of: > > I think you can reconfigure

Re: Kafka Mirror maker stops replicating

2023-02-08 Thread Greg Harris
ogs and watched the progress of > the replication flow around the time it stops replicating? > Could you tell me how can I enable more logging ? > > Thanks, > Arpit > > On Wed, Feb 8, 2023, 18:16 Greg Harris > wrote: > > > Arpit, > > > > Unfortu

Re: Kafka Mirror maker stops replicating

2023-02-08 Thread Greg Harris
, Greg Harris On Tue, Feb 7, 2023 at 6:27 AM Arpit Jain wrote: > Hi, > > Hope this is the right forum to ask for Kafka mirror maker issues. > We are facing an issue where the mirror maker replicates the trades and > then doesn't work for long time and again replicates. > Also

Re: Mirror maker worker can't issue with REST uri

2023-02-08 Thread Greg Harris
possible to slow down the logs. > > Cheers. > > From: Greg Harris > Date: Wednesday, 8 February 2023 at 1:08 pm > To: users@kafka.apache.org > Subject: Re: Mirror maker worker can't issue with REST uri > NetApp Security WARNING: This is an external email. Do not click links

Re: Kafka Connect ClusterConfigState.inconsistentConnectors() not handled by distributed Worker?

2023-02-08 Thread Greg Harris
Frank, > I'm operating on the assumption that the connectors in question get stuck in an inconsistent state > Another thought... if an API exists to list all connectors in such a state, then at least some monitoring/alerting could be put in place, right? There is two different inconsistencies

Re: Mirror maker worker can't issue with REST uri

2023-02-07 Thread Greg Harris
/display/KAFKA/KIP-710%3A+Full+support+for+distributed+mode+in+dedicated+MirrorMaker+2.0+clusters for more details. As a workaround, I believe you can restart the MirrorMaker2 connectors to force a reconfiguration. I hope this helps, Greg Harris On Tue, Feb 7, 2023, 10:47 PM Shirolkar, Anup wrote

Re: Kafka Connect ClusterConfigState.inconsistentConnectors() not handled by distributed Worker?

2023-02-06 Thread Greg Harris
Frank, I don't think that the fix needs to necessarily follow the #12450 PR, we can choose to start from scratch now that we know more about the issue. If that PR is useful as a starting point, we can also include it, that is up to you. Greg On Mon, Feb 6, 2023 at 10:21 AM Frank Grimes wrote:

Re: Kafka Connect ClusterConfigState.inconsistentConnectors() not handled by distributed Worker?

2023-02-06 Thread Greg Harris
-asdb/from-kafka-to-asdb-overview > ) > We're wondering if it would be possible to re-evaluate the impact of this > bug and look at addressing it either with the pre-existing PR ( > https://github.com/apache/kafka/pull/7823) or a new one. > Thanks!On Friday, February 3, 2023, 04:2

Re: Kafka Connect ClusterConfigState.inconsistentConnectors() not handled by distributed Worker?

2023-02-03 Thread Greg Harris
be very interested in a reproduction case for that. Thanks! Greg Harris On Fri, Feb 3, 2023 at 1:05 PM Greg Harris wrote: > Frank, > > The inconsistentConnectors method is related to an extremely specific > inconsistency that can happen when a worker writes some task >

Re: Kafka Connect ClusterConfigState.inconsistentConnectors() not handled by distributed Worker?

2023-02-03 Thread Greg Harris
s. I hope this gives some more insight to the behavior you're seeing. Thanks, Greg Harris On Fri, Feb 3, 2023 at 7:36 AM Frank Grimes wrote: > Hi, we're investigating an issue where occasionally config changes don't > propagate to connectors/tasks. > > When this occurs,