ations.
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 th
'd 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
>
s://docs.aerospike.com/connect/kafka/to-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.
>
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:
confluence/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, A
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 rel
k solution 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
,
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.
>
more detailed logs 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,
> >
> &g
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 reconf
/workaround?The 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
ign, propose and implement it.
It would be valuable 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, 202
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 these l
not solve 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,
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
.sh are 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
/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
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 an
at is upgrade 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
he operation 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 t
ovide your 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:
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() befor
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 kafka-connec
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 informat
sync
> > feature.
> >
> > 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 negativ
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
4
apache.org/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 featur
e logs to debug it.
Best,
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
--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 attach
n "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@kafk
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,
> G
ocumentation 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 tar
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 o
once during the 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 thi
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
ht
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 consu
a.topic1 that hasn't been replicated
> 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.
>
>
sage, we'd love to help out building
> this kind 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 s
I'm hoping to submit 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
> wro
9161cd3f1b/connect/mirror/src/main/java/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 Fr
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 e
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 letti
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 dur
es, is
> highly 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?
>
> Chri
es an in-memory 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-
arantees (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/
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
failures, 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 wro
.
Hope this helps,
Greg Harris
On Tue, May 28, 2024 at 12:35 PM Mehrtens, M
wrote:
> Hi there,
> I’m running into an issue with MirrorMaker2 (MM2), specifically the
> MirrorCheckpointConnector (CPC). In short, CPC doesn’t replicate the final
> batch of offsets to the target Kafka clus
Hi Sebastian,
Thanks for your question! A consumer joining a group should always notify
the group coordinator and the leader consumer, and the
partition.assignment.strategy [1] will determine the rebalance protocol,
and which partitions are affected by the rebalance.
For example, I expect that th
Hi Akash,
Thanks for your question. When you say:
> kafkaconsumer misses some of the records written into the kafka source
topic.
Do you mean that there are records in the source topic that are never
returned by Consumer#poll(), SourceTask#poll(), or written to the target
topic by KafkaConnect?
l paused once the source cluster
> consumer group was turned off.
>
> There is a gap in offsets between source & target cluster, where the
> source cluster has a last committed offset for the consumer group that is
> greater than the last committed offset replicated in the
Hey Burton,
Thanks for your question and bug report.
The exception you included does not indicate that your connectors are
overloaded. The primary way of diagnosing an overloaded connector is the
consumer lag metric, and if you're seeing acceptable lag, that should
indicate that your connectors a
Hi Josep,
I found this blocker regression:
https://issues.apache.org/jira/browse/KAFKA-17150
Summary: Connector configurations that specified converters with aliases
(e.g. JsonConverter instead of org.apache.kafka.connect.json.JsonConverter)
previously worked, and in this RC they throw validation
Hi Josep,
+1 (binding)
I performed the following validations:
1. I formatted a new log directory and verified that the stray log message
is no longer present (KAFKA-17148)
2. I configured a connect worker with a custom plugin path and
plugin.discovery=service_load and verified that plugin scannin
Hi Andreas,
Thank you for the reminder, it is unfortunate that this slipped through the
cracks.
This is certainly something that we should fix for everyone, so you don't
have to maintain patches yourself. I'll investigate some more and get back
to you.
Thanks!
Greg
On Fri, Aug 2, 2024 at 12:33 A
Hi Ran,
The config.action.reload, subscribe, unsubscribe, and ConfigChangeCallback
features are partially-implemented. As far as I can tell if you're using
only AK connect-runtime and config providers, none of it is functional.
If you have a config provider that implements TTLs, the runtime will
Hi Tom,
Thanks for your question!
In Kafka 3.5 there was an overhaul of the offset translation logic to avoid
a data loss bug. This [1] is the ticket that I've been using to represent
that overhaul.
In the implementation for that fix, we traded a lot of availability (read:
consumer lag) for corre
Hi Harry,
Thanks for your question!
There were negative lag/data loss bugs fixed some time back [1, 2] so check
what version of the MirrorMaker connectors you're using. I would recommend
using the latest version of MM2 (3.8.0) if possible.
After upgrading you will need to clear the checkpoints t
Hi Douglas,
Thanks for your interest in Apache Kafka!
At this time, we don't have a timeline for 3.7.2 yet, and no committer has
volunteered. Usually that happens once there are a sufficient number of
backported patches accumulated or a security patch is necessary. At this
time, the next planned
Hey everyone,
Bumping this topic to see if anyone has experience migrating Streams jobs
between clusters with Mirror Maker, and how you handled repartition topics.
Thanks!
Greg
On Mon, Aug 12, 2024 at 8:55 AM Greg Harris wrote:
> Hi all,
>
> We were recently working to set up a Mir
Hi Massimiliano,
A small amount of re-delivery is expected when using MirrorMaker2 3.5+
offsets replication. This is because MM2 does not offer exactly once offset
replication.
I gave a conference talk recently about this which you may find
informative:
https://current.confluent.io/2024-sessions/
t ? Do they get updated ?
>
> Sorry for all the questions, I have been using Kafka for several years and
> want to dive deep into it a little bit. I have become more interested and
> ready to find out on my own. But still look forward to your thoughts on
> this if the questions above do
Hi Artur,
Thanks for the question!
In transactions there's no distinction between partitions within the same
topic and partitions in different topics. You can have transactions across
multiple topics, and multiple partitions in those topics.
Hope this is more clear,
Greg
On Fri, Oct 25, 2024 at
affects Apache Kafka Clients: from 2.3.0 through 3.5.2, 3.6.2,
3.7.0.
Thanks,
Greg Harris
On Mon, Nov 18, 2024 at 10:42 AM Greg Harris wrote:
> Severity: moderate
>
> Affected versions:
>
> - Apache Kafka Clients 2.3.0 through 3.5.2
> - Apache Kafka Clients 3.6.0 through 3.6.
affects Apache Kafka Clients: from 2.3.0 through 3.5.2, 3.6.2, 3.7.0.
Thanks,
Greg Harris
On Mon, Nov 18, 2024 at 10:42 AM Greg Harris wrote:
> Severity: moderate
>
> Affected versions:
>
> - Apache Kafka Clients 2.3.0 through 3.5.2
> - Apache Kafka Clients 3.6.0 through 3.6.
fka Broker, Kafka MirrorMaker 2.0, Kafka Streams, and Kafka
command-line tools, it is not recommended to set the system property.
Credit:
Greg Harris (finder)
Mickael Maison (remediation reviewer)
Chris Egerton (remediation reviewer)
References:
https://kafka.apache.org/
https://www.cve.org/CVERecord?id=CVE-2024-31141
we backport things into 3.9.x, it will slow adoption of 4.x and
> increase our maintenance burden over time.
>
> Just my $0.02
>
> Thanks!
> David A
>
> On Wed, Nov 20, 2024 at 12:22 PM Greg Harris >
> wrote:
>
> > Hi all,
> >
> > Now that 3.9.0 is
Hi all,
Now that 3.9.0 is released and 4.0.x is progressing, I'd like to understand
everyone's expectations about the 3.9.x branch, and ask for a specific
consensus on Java 23 support.
Some context that I think is relevant to the discussion:
* KIP-1006 [1] proposes a backwards-compatible strategy
tatic quorum cannot use KIP-853 even after upgrading [2].
I think to have a dynamic quorum, you will need to either create a new 3.9+
cluster, or perform the ZK->Kraft migration while running 3.9+.
I hope this helps,
Greg Harris
[1] https://kafka.apache.org/documentation/#kraft_reconfig
[
Hi,
Thanks for your question.
It appears you're using the legacy consumer API, which was removed in 2.0.0
and is no longer supported.
I would strongly suggest building on top of the modern Java Consumer API at
this time.
The modern API exposes the deserialized headers via the
ConsumerRecord#head
28 Dec, 2024, 04:47 Greg Harris,
> wrote:
>
> > Hi,
> >
> > Thanks for your question.
> >
> > It appears you're using the legacy consumer API, which was removed in
> 2.0.0
> > and is no longer supported.
> > I would strongly suggest building o
Hi,
Yes you are correct. The "classic" Kafka Group Protocol is a synchronizing
barrier for all members.
All JoinGroup member responses are returned after all JoinGroup member
requests are received.
Thanks,
Greg
On Tue, Jan 21, 2025 at 7:10 AM Chain Head wrote:
> Assume that three consumers of
Hello Jan,
I also have not heard of a use case like this for Kafka. One statistic that
I think you might need to manage is batch size, and its effect on
compression and read amplification.
Larger batches on the producer side can make your producers more performant
and compression more effective.
ad returning data after 10 bytes read and second one waiting for 10MB).
> By this understanding (and, as said, we may be wrong here) batch sizes can
> be an issue, but they are configurable on the individual producer and
> consumer level.
>
> Best regards,
> Jan
>
> -Orig
.org/confluence/display/KAFKA/KIP-932%3A+Queues+for+Kafka
On Tue, Jan 21, 2025 at 4:41 PM Chain Head wrote:
> Thanks.
>
> By "classic" you mean pre-KRaft consensus? What is the "current" Kafka
> Group protocol?
>
> Best regards.
>
> On Tue, Jan 21, 2025
Hi Sisindri,
Thanks for your question.
This is not supported with the current KRaft implementation, and I don't
believe there's any plans to support it in the future.
You will need to migrate from a single ZK cluster to multiple KRaft
quorums, as the final bridge release 3.9 does not support a
ch
Hi Adrien,
Thanks for the report!
I had some questions about your observations:
> Upon
> waking, the thread simply continues its sleep operation without any
> awareness that a significant amount of time may have passed.
Were you able to take a stack trace, attach a debugger to see the state of
Hi Mehrtens,
The MirrorSourceConnector/Task relies on the Connect framework to
instantiate the producer used for mirroring.
The "target.cluster.bootstrap.servers" is indeed ineffective for changing
the framework client configuration, that should only affect clients
instantiated within the task (ad
79 matches
Mail list logo