Re: MirrorMaker2 ACL Replication

2025-04-11 Thread Ryanne Dolan
Because it (usually) doesn't make sense to write to a "remote topic". Doing so would mean it no longer is a simple copy of the source topic. Only MM itself should be writing to a remote topic. That said, it may make sense to replicate write ACLs for the corresponding *source* topics on the remote

Re: Mirror Maker bidirectional offset sync

2024-01-11 Thread Ryanne Dolan
t; > replicating the consumer offsets back to the primary cluster. > > > > OffsetSync{topicPartition=replicate-me-0, upstreamOffset=28, > > > > downstreamOffset=28} > > > > OffsetSync{topicPartition=replicate-me-0, upstreamOffset=29, > > > > down

Re: Mirror Maker bidirectional offset sync

2024-01-10 Thread Ryanne Dolan
consumer group > may > > not be active on both the primary and secondary cluster as it would block > > MM2 from replicating its offsets from primary to the cluster-prefixed > topic > > on the secondary cluster already. On the first point, I think it would > be a > >

Re: Mirror Maker bidirectional offset sync

2024-01-08 Thread Ryanne Dolan
Jeroen, MirrorClient will correctly translate offsets for both failover and failback, exactly as you describe. It's possible to automate failover and failback using that logic. The integration tests automatically fail over and fail back, for example. I've seen it done two ways: during startup withi

Re: Delay issue when replicating topics creation with Kafka MM2 cluster

2023-02-20 Thread Ryanne Dolan
As far as I know, there is no way for MM2 to be notified when a new topic is created, so it polls at a configurable interval. You can try adjusting the refresh.topics.interval.seconds property. It sounds like it's configured for 10 minutes. Ryanne On Mon, Feb 20, 2023, 11:35 AM Fares Oueslati wr

Re: Help with MM2 active/passive configuration

2022-10-19 Thread Ryanne Dolan
Hey Chris, check out the readme in connect/mirror for examples on how to run mirror maker as a standalone process. It's similar to how Cloudera's mirror maker is configured. If you've got a Connect cluster already, I recommend using that and manually configuring the MirrorSourceConnector. Ryanne

Re: Mirror maker disable auto topic creation

2022-09-14 Thread Ryanne Dolan
The link is correct. Connect and mm2 create those internal topics at startup, whether or not auto topic creation is enabled. On Wed, Sep 14, 2022, 11:15 PM Mcs Vemuri wrote: > Hello, > Is there any way to disable topic creation in MM2? I tried setting the > topic.creation.enable to False but the

Re: MM2 - mapping different replica number

2022-07-12 Thread Ryanne Dolan
Yes, you can configure replication.factor to match the target cluster. Ryanne On Tue, Jul 12, 2022, 12:44 PM An, Hongguo (CORP) wrote: > Hi: > My source has 5 brokers and all topic has 5 replica, but my target cluster > has only 3 brokers and I can鈥檛 allocate 2 more, is it possible for MM2 to >

Re: MirrorMaker 2 not replicating topic ACLs correctly

2022-04-15 Thread Ryanne Dolan
Alex, MM2 is very conservative wrt replicating ACLs. Usually only MM2 is supposed to write to remote topics, so it usually doesn't make sense to replicate WRITE for other identities. Currently we only sync READ to remote topics, and we don't touch non-remote topics. If your use-case requires repli

Re: Mirror Maker 2 - High Throughput Identity Mirroring

2022-03-02 Thread Ryanne Dolan
it internally. > > Thanks, > > Ant贸n > > On Thu, Jul 29, 2021 at 7:02 PM Ryanne Dolan > wrote: > > > Jamie, this would depend on KIP-712 (or similar) aka "shallow mirroring". > > This is a work in progress, but I'm optimistic it'll happen at s

Re: MM2 setup using Kafka 2.4 compatibility with Kafka Broker built using 2.0.0 ver.

2022-02-07 Thread Ryanne Dolan
Ranga, yes that will work. You are correct that mm2's compatibility matrix is essentially the same as the client's. I've tested 2.4 mm2 against 0.10.2 brokers and it works fine. Anything older than that may still generally work, but you may see errors for unsupported apis. Ryanne On Mon, Feb 7, 2

Re: Is it possible to run MirrorMaker in active/active/active?

2022-01-31 Thread Ryanne Dolan
on&utm_content=email-signature-link> > | Facebook< > https://www.facebook.com/perforce/?utm_leadsource=email-signature&utm_source=outlook-direct-email&utm_medium=email&utm_campaign=2019-common&utm_content=email-signature-link> > | YouTube< > https://www.youtube.

Re: Is it possible to run MirrorMaker in active/active/active?

2022-01-31 Thread Ryanne Dolan
Doug, you can have any number of clusters with a fully-connected mesh topology, which I think is what you are looking for. Ryanne On Mon, Jan 31, 2022, 12:44 PM Doug Whitfield wrote: > Hi folks, > > Every example I have seen uses two clusters in active/active and testing > suggests I can only g

Re: MirrorMaker2 fails to replicate - please urgent help needed

2021-10-25 Thread Ryanne Dolan
ache.kafka.connect.mirror.MirrorMaker:212) as my data is on A which > needs to be mirrored to B and that is what I have set in my mm2.properties > (pasting it again fyr) > A->B.enabled = true > > B->A.enabled = false > > Thanks again for your help! > > > ~

Re: MirrorMaker2 fails to replicate - please urgent help needed

2021-10-25 Thread Ryanne Dolan
Hello, how many mm nodes are you using? Try starting with one and adding more after the first starts working. There is a known race that effects some people trying to start multiple nodes at once. Also check for any potential auth problems. The driver isn't very verbose when it comes to auth probl

Re: Mirror Maker 2: use prefix-less topic names?

2021-10-07 Thread Ryanne Dolan
Jake, in the most recent Kafka 3.0 release you will find IdentityReplicationPolicy, which does what you want. Just be careful that you don't try to replicate the same topics in a loop. Ryanne On Thu, Oct 7, 2021, 1:31 PM Jake Mayward wrote: > Hi, > > I have read a bit on MirrorMaker 2 via > htt

Re: question about mm2 on consumer group offset mirroring

2021-09-30 Thread Ryanne Dolan
Hey Calvin, the property you're looking for is emit.checkpoint.interval.seconds. That's how often MM will write checkpoints, which includes consumer group offsets. Ryanne On Thu, Sep 30, 2021, 9:18 AM Calvin Chen wrote: > Hi all > > I have a question about the mirror make 2, on the consumer gro

Re: Mirror Maker 2 with different Avro Schema Registries

2021-09-14 Thread Ryanne Dolan
Yes it's a consequence of Connect, which has its own serde model. mirror-makers hard-coded ByteArraySerialization for > consumers/producers

Re: Mirror Maker 2 with different Avro Schema Registries

2021-09-14 Thread Ryanne Dolan
Hey Anders, take a look at Connect's serdes and SMTs. MirrorMaker can be configured to use them. Ryanne On Tue, Sep 14, 2021, 3:13 AM Anders Engstr枚m wrote: > Hi! > I'm trying to replicate a few topics using Mirror Maker 2 (2.8). > > Both the source and the target cluster use Schema Registry (K

Re: Mirror Maker 2 - High Throughput Identity Mirroring

2021-07-29 Thread Ryanne Dolan
Jamie, this would depend on KIP-712 (or similar) aka "shallow mirroring". This is a work in progress, but I'm optimistic it'll happen at some point. ftr, "IdentityReplicationPolicy" has landed for the upcoming release, tho "identity" in that context just means that topics aren't renamed. Ryanne

Re: MirrorMaker 2.0 compliance question

2021-07-21 Thread Ryanne Dolan
yep! On Wed, Jul 21, 2021, 3:18 AM Tomer Zeltzer wrote: > Hi, > > Can I use MirrorMaker2.0 from Kafka 2.8.0 with Kafka version 2.4.0? > > Thanks, > Tomer Zeltzer > > This email and the information contained herein is proprietary and > confidential and subject to the Amdocs Email Terms of Ser

Re: How to avoid storing password in clear text in server.properties file

2021-06-21 Thread Ryanne Dolan
Take a look at the ConfigProvider interface. On Mon, Jun 21, 2021, 8:26 AM Dhirendra Singh wrote: > Hi All, > I am currently storing various passwords like "ssl.keystore.password", > "ssl.truststore.password", SASL plain user password in cleartext in > server.properties file. > is there any way

Re: Help !! Apache-Kafka On-Premises Setup.

2021-06-18 Thread Ryanne Dolan
Nitin, I know this isn't a particularly helpful response, but you should know using a load balancer like that is really hard to set up with kafka due to the fact that it advertises broker addresses to clients. I've done it a few times but it's something of a dark art :) If you can get away without

Re: kafka mirror with TLS and SASL enabled

2021-06-17 Thread Ryanne Dolan
Calvin, that's an interesting idea. The motivation behind the current behavior is to only grant principals access to data they already have access to. If a principal can access data in one cluster, there's no harm in providing read access to the same data in another cluster. But you are right that

Re: Resend old messages, find partitions for a message

2021-06-14 Thread Ryanne Dolan
Andrew, an offset implies a partition -- an offset is only meaningful within the context of a particular partition -- so if you are able to log offsets you should also be able to log the corresponding partitions. For example, the RecordMetadata object, which provides the offset of a written record

Re: MirrorMaker 2 with SSL

2021-04-05 Thread Ryanne Dolan
Yes it's possible. The most common issue in my experience is the location of the trust store and key store being different or absent on some hosts. You need to make sure that these locations are consistent across all hosts in your Connect cluster, or use a ConfigProvider to provide the location dyn

Re: MirrorMaker 2 and Negative Lag

2021-03-25 Thread Ryanne Dolan
Thanks Alan for the investigation and bug report! On Thu, Mar 25, 2021, 3:25 PM Alan Ning wrote: > Another update on this. I am pretty sure I have found a bug in > MirrorSourceTask. The details are written in > https://issues.apache.org/jira/browse/KAFKA-12558. I hope this helps > others > who h

Re: options for kafka cluster backup?

2021-03-07 Thread Ryanne Dolan
MirrorMaker v1 does not sync offsets, but MM2 does! Ryanne On Sun, Mar 7, 2021, 10:02 PM Pushkar Deole wrote: > Thanks you all! > > Blake, for your comment: > > It'll require having a HA cluster running in another region, of course. > One other caveat is that it doesn't preserve the offsets of

Re: Mirror Maker 2 - Issues

2021-03-07 Thread Ryanne Dolan
kpoints. > source->target.emit.heartbeats.enabled = true > source->target.emit.checkpoints.enabled = true > > # customize as needed > # replication.policy.separator = "" > # source.cluster.alias: "" > # target.cluster.alias: "" > # sync.topic.acls.enab

Re: Mirror Maker 2 - Issues

2021-03-07 Thread Ryanne Dolan
Navneeth, it looks like you are trying to override the cluster aliases to the empty string. If you look closely, these properties are defined using incorrect syntax. I'm not sure how the properties would be parsed, but you should fix that to rule it out as a source of confusion. Then, be aware tha

Re: When using MM2, how should the consumer be switched from source to target?

2021-01-02 Thread Ryanne Dolan
Aki, that's right. Prior to 2.7, you can use the translateOffsets method from within your client code, or you can write a little command-line tool to do it. I've done the latter for Cloudera's srm-control tool, as documented here: https://docs.cloudera.com/csp/2.0.1/srm-using/topics/srm-migrating-c

Re: Announcing 馃巺馃巹#TwelveDaysOfSMT馃巹馃巺

2020-12-14 Thread Ryanne Dolan
Robin, I don't think this is an appropriate use of the user group. Ryanne On Mon, Dec 14, 2020 at 10:20 AM Robin Moffatt wrote: > 馃巺馃巹 #TwelveDaysOfSMT 馃巹 馃巺 > > Day 5锔忊儯: MaskField > https://rmoff.net/2020/12/14/twelve-days-of-smt-day-5-maskfield/ > > > -- > > Robin Moffatt | Senior Developer Advoc

Re: MirrorMaker 2 Reload Configuration

2020-11-11 Thread Ryanne Dolan
Hey guys, this is because the configuration gets loaded into the internal mm2-config topics, and these may get out of sync with the mm2.properties file in some scenarios. I believe this occurs whenever an old/bad configuration gets written to Kafka, which MM2 can read successfully but which causes

Re: Bidirectional XDCR by MirrorMaker2

2020-09-25 Thread Ryanne Dolan
Oleg, if you want bidirectional replication, you should have: dc1->dc2.enabled = true dc2->dc1.enabled = true ... in _both_ DCs. The replication topology should be consistent across all DCs, generally. Otherwise DCs will disagree on what should get replicated and you'll likely encounter confusin

Re: Two MirrorMakers 2 for two DCs

2020-09-21 Thread Ryanne Dolan
Oleg, yes you can run multiple MM2s for multiple DCs, and generally that's what you want to do. Are you using Connect to run MM2, or the connect-mirror-maker.sh driver? Ryanne On Mon, Sep 21, 2020, 3:38 PM Oleg Osipov wrote: > I use the configuration for M2M for both datacentres > clusters: >

Re: MirrorMaker 2.0 - Translating offsets for remote topics and consumer groups

2020-08-21 Thread Ryanne Dolan
gt; Appreciate all your help! > > Josh > > On Wed, Aug 19, 2020 at 12:55 PM Ryanne Dolan > wrote: > > > Josh, if you have two clusters with bidirectional replication, you only > get > > two copies of each record. MM2 won't replicate the data "upst

Re: Mirror Maker 2.0 Queries

2020-08-20 Thread Ryanne Dolan
3. KafkaHeartbeatConnector - focus on checking cluster availability > > *Can we configure tasks.max for each of these connectors separately? That > is, Can I have 3 tasks for KafkaSourceConnector, 5 > for KafkaCheckpointConnector, and 1 for KafkaHeartbeatConnector?* > > > >

Re: Mirror Maker 2.0 Queries

2020-08-20 Thread Ryanne Dolan
Ananya, see responses below. > Can this number of workers be configured? The number of workers is not exactly configurable, but you can control it by spinning up drivers and using the '--clusters' flag. A driver instance without '--clusters' will run one worker for each A->B replication flow. So

Re: MirrorMaker 2.0 - Translating offsets for remote topics and consumer groups

2020-08-19 Thread Ryanne Dolan
t; > for the source topic AND the remote topic as a pair as opposed to having > to > > explicitly replicate the remote topic back to the source cluster just to > > have the checkpoints emitted upstream? > > > > Josh > > > > On Wed, Aug 19, 2020 at 6:16 AM Ryanne

Re: MirrorMaker 2.0 - Translating offsets for remote topics and consumer groups

2020-08-19 Thread Ryanne Dolan
Josh, yes it's possible to migrate the consumer group back to the source topic, but you need to explicitly replicate the remote topic back to the source cluster -- otherwise no checkpoints will flow "upstream": A->B.topics=test1 B->A.topics=A.test1 After the first checkpoint is emitted upstream,

Re: Kafka Mirror Maker 2: RemoteClusterUtils.translateOffsets() returning empty map

2020-08-04 Thread Ryanne Dolan
the RemoteClusterUtils.translateOffsets() here. > 4. The method returns an empty map as well as the checkpoints topic in both > the clusters is empty. > > Let me know if you see anything wrong here. > > Thanks, > Sunny > > > On Tue, Aug 4, 2020 at 8:26 PM Ryanne Dola

Re: Kafka Mirror Maker 2: RemoteClusterUtils.translateOffsets() returning empty map

2020-08-04 Thread Ryanne Dolan
Sunny, is it possible there are no consumer groups? There will be no checkpoints, and thus nothing to use for offset translation, if there are no upstream consumer groups. Ryanne On Tue, Aug 4, 2020, 9:28 AM Sunny Lohani wrote: > Hi, > > I have 2 data centers, each having single node Zookeeper

Re: Mirrormaker 2 logs - WARN Catching up to assignment's config offset

2020-07-29 Thread Ryanne Dolan
Iftach, you can try deleting Connect's internal config and status topics. The status topic records, among other things, the offsets within the config topics iirc, so if you delete the configs without deleting the status, you'll get messages such as those. Just don't delete the mm2-offsets topics, a

Re: mirrormaker 2 monitoring

2020-07-19 Thread Ryanne Dolan
Iftach, MM2 uses the same MetricsReporter interface that brokers use, however seems most people use a JMX exporter. Ryanne On Sun, Jul 19, 2020, 7:12 AM Iftach Ben-Yosef wrote: > Hello, > I want to monitor our mirrormaker 2 cluster using prometheus. We are mostly > interested in lag, incoming/o

Re: Mirror Maker 2.0 Queries

2020-07-13 Thread Ryanne Dolan
Ananya, yes the driver is distributed, but each worker only communicates via kafka. They do not listen on any ports. Ryanne On Sat, Jul 11, 2020, 11:28 AM Ananya Sen wrote: > Hi > > I was exploring the Mirror maker 2.0. I read through this > > https://cwiki.apache.org/confluence/display/KAFKA/K

Re: destination topics in mm2 larger than source topic

2020-07-01 Thread Ryanne Dolan
Iftach, is it possible the source topic is compressed? Ryanne On Wed, Jul 1, 2020, 8:39 AM Iftach Ben-Yosef wrote: > Hello everyone. > > I'm testing mm2 for our cross dc topic replication. We used to do it using > mm1 but faced various issues. > > So far, mm2 is working well, but I have 1 issue

Re: Should there be separate Kafka for producers and consumers

2020-05-23 Thread Ryanne Dolan
Hey Nitin, it's not uncommon to use separate clusters and MirrorMaker for this purpose, e.g. separate ingest and ETL clusters. That way your ETL cluster has only one producer (MM) and only one consumer (Camus). It's also possible you just need more brokers and/or more partitions. Ryanne On Sat,

Re: MM2 Message Handlers

2020-05-18 Thread Ryanne Dolan
Jamie, MM2 uses Connect, so you can use Connect's SMTs for the same effect. Ryanne On Mon, May 18, 2020, 6:17 AM Jamie wrote: > Hi All, > Does MM2 currently support message handlers, the same way MM1 did? If not, > are there any plans to support this in future? > Many Thanks, > Jamie

Re: Identity Mirroring

2020-05-07 Thread Ryanne Dolan
Hey Henry, this was done with MM1 at LinkedIn at one point, but it requires support for shallow iteration in KafkaConsumer, which was removed from Apache Kafka a long time ago. Given recent talk of breaking changes in Kafka 3.0, this might be a good time to revisit this. Ryanne On Thu, May 7, 20

Re: Metrics in Kafka Connect

2020-04-30 Thread Ryanne Dolan
This is what I did inside MM2's Connectors and Tasks. It works pretty well, but I'd certainly prefer if Connect exposed it's own Metrics API. Ryanne On Thu, Apr 30, 2020 at 2:57 PM G茅rald Quintana wrote: > Hello, > > We developed a custom Kafka Connect implementation for a specific need, and >

Re: Kafka Mirror Maker 2

2020-04-29 Thread Ryanne Dolan
> 1. ... Do I need to manually create a topic TOPIC1 in cluster K2 and this topic will be used for producing messages to when failover happens. Correct. When you fail-over producers from K1 to K2, producers will send to TOPIC1 in K2. You may need to manually create that topic and set up write ACLs

Re: MirrorMaker2 ordering guarantees

2020-04-27 Thread Ryanne Dolan
t; Can you please add some more detail about why those "slightly > out-of-order > > records" may happen? > > > > Thanks, > > Peter > > > > On Wed, 22 Apr 2020 at 20:16, Ryanne Dolan > wrote: > > > >> Hey Peter, Connect will need to supp

Re: MM2 with older Kafka version

2020-04-23 Thread Ryanne Dolan
Yes :) On Wed, Apr 22, 2020, 8:03 PM Henry Cai wrote: > Looks like MM2 ships with Kafka 2.4, if our brokers are still running on > older kafka version (2.3), can the MM2 running with 2.4 code work with > brokers running with 2.3 code? > > Thanks. > >

Re: MirrorMaker2 ordering guarantees

2020-04-22 Thread Ryanne Dolan
Hey Peter, Connect will need to support transactions before we can guarantee the order of records in remote topics. We can guarantee that no records are dropped or skipped, even during consumer failover/migration etc, but we can still have duplicates and slightly out-of-order records in the downstr

Re: MirrorMaker2 - uneven loadbalancing

2020-03-23 Thread Ryanne Dolan
Connector tasks, 1 MirrorHeartbeatConnector > task > - instance 3: 1 MirrorSourceConnector task, 1 MirrorCheckpointConnector > task > - instance 4: 1 MirrorSourceConnector task, 1 MirrorHeartbeatConnector task > > So it seems that it is not well balanced, but can be scaled somewhat, not > idea

Re: MirrorMaker2 - uneven loadbalancing

2020-03-20 Thread Ryanne Dolan
s. If you can give some idea how to > check it, I will try. > > Peter > > On Fri, 20 Mar 2020 at 17:47, Ryanne Dolan wrote: > > > Hmm, that's weird. I'd expect the type of tasks to be evenly distributed > as > > well. Is it possible one of the in

Re: MirrorMaker2 not mirroring for 5 minutes when adding a topic

2020-03-20 Thread Ryanne Dolan
k 13 ms > (org.apache.kafka.connect.mirror.Scheduler) > [2020-03-20 10:50:07,457] INFO loading initial consumer groups took 8 ms > (org.apache.kafka.connect.mirror.Scheduler) > [2020-03-20 10:50:07,844] INFO syncing topic configs took 432 ms > (org.apache.kafka.connect.mirror.Schedule

Re: MirrorMaker2 - uneven loadbalancing

2020-03-20 Thread Ryanne Dolan
entation I expected 1-3 or 2-2 tasks of > the MirrorSourceConnectors on the two MM2 instances. > > So is this a bug or an expected behaviour? > > Thanks, > Peter > > On Fri, 20 Mar 2020 at 15:26, Ryanne Dolan wrote: > > > Peter, in Connect the Connectors are only

Re: MirrorMaker2 - uneven loadbalancing

2020-03-20 Thread Ryanne Dolan
Peter, in Connect the Connectors are only run on the leader node. Most of the work is done in the Tasks, which should be divided across nodes. Make sure you have tasks.max set to something higher than the default of 1. Ryanne On Fri, Mar 20, 2020, 8:53 AM P茅ter Sin贸ros-Szab贸 wrote: > Hey, > > I

Re: MirrorMaker2 not mirroring for 5 minutes when adding a topic

2020-03-18 Thread Ryanne Dolan
Peter, can you share any log lines like "x took y ms" or "x took too long (y ms)" or "timed out running task x"? Ryanne On Tue, Mar 17, 2020 at 10:45 AM P茅ter Sin贸ros-Szab贸 wrote: > Hey, > > Running a MM2 cluster to mirror from A->B clusters I noticed that when I > add a new topic to A cluster,

Re: MirrorMaker2 to mirror new topics?

2020-03-09 Thread Ryanne Dolan
Hey Peter, often the problem is the replication factor or min ISRs being misconfigured, which can prevent MM2 (or anything else) from sending. Hope that helps! Ryanne On Mon, Mar 9, 2020, 10:35 AM P茅ter Sin贸ros-Szab贸 wrote: > Hi, > > it is mirroring but the problem was that on of the task faile

Re: How to produce in DR scenario with Mirror maker (automatic failover)

2020-03-07 Thread Ryanne Dolan
Hello, take a look at RemoteClusterUtils.translateOffsets, which will give you the correct offsets _and topics_ to subscribe to. The method automatically renames the topics according to the ReplicationPolicy. You can leverage this method in your consumer itself or in external tooling. Ryanne On S

Re: Mirror Maker 2 MirrorClient

2020-02-27 Thread Ryanne Dolan
Hey Carl, that's what my team has done for our internal tooling, and I designed MirrorClient with that in mind. Given a single mm2.properties file you can create MirrorClients for each cluster and those in turn give you Admin/Consumer/Producer clients if you need them. Our internal tooling essentia

Re: Use a single consumer or create consumer per topic

2020-02-26 Thread Ryanne Dolan
On an older cluster like that, rebalances will stop-the-world and kill your throughput. Much better to have a bunch of consumer groups, one per topic, so they can rebalance independently. On Wed, Feb 26, 2020, 1:05 AM Mark Zang wrote: > Hi, > > I have a 20 brokers kafka cluster and there are abo

Re: [External] Mirror Maker Questions

2020-02-20 Thread Ryanne Dolan
For 1), MM2 will work with older versions of Kafka. I've gotten it to work with clusters as old as 0.10.2 but with some features disabled iirc. Ryanne On Thu, Feb 20, 2020, 2:49 AM Dean Barlan wrote: > Hi everyone, > > I have a few small questions for you regarding MirrorMaker. > > 1. I know t

Re: Streams - multiple clusters support

2020-02-13 Thread Ryanne Dolan
Cyrille, I don't see why using MM1/2 would break your isolation requirement. But if you can't mirror topics for some reason consider Flink instead of Kafka Streams. Ryanne On Thu, Feb 13, 2020 at 10:52 AM Cyrille Karmann wrote: > Hello, > > We are trying to create a streaming pipeline of data b

Re: MM2 for DR

2020-02-12 Thread Ryanne Dolan
uggestion in this regard would be greatly appreciated > > > > Thanks in advance again! > > > > > > On Mon, Feb 10, 2020 at 9:40 PM Ryanne Dolan > > wrote: > > > >> Hello, sounds like you have this all figured out actually. A couple > notes: > &

Re: MM2 for DR

2020-02-10 Thread Ryanne Dolan
Hello, sounds like you have this all figured out actually. A couple notes: > For now, we just need to handle DR requirements, i.e., we would not need active-active If your infrastructure is sufficiently advanced, active/active can be a lot easier to manage than active/standby. If you are starting

Re: MirrorMaker 2 does not replicate all messages?

2020-01-23 Thread Ryanne Dolan
Nils, are those record counts or offsets? Ryanne On Thu, Jan 23, 2020, 2:46 AM Nils Hermansson wrote: > Hello, > I have setup MM2 and the config is similar to the example config. > > https://github.com/apache/kafka/blob/trunk/config/connect-mirror-maker.properties > Primary has 3 brokers and se

Re: MM2 offset sync vs checkpoints

2020-01-22 Thread Ryanne Dolan
Peter, the offset sync records are used internally to create the checkpoint records. You are correct that the checkpoints include everything you need. Ryanne On Wed, Jan 22, 2020, 11:08 AM P茅ter Sin贸ros-Szab贸 wrote: > Hi, > > Why do MM2 need offset sync messages too? It seems to me that the > c

Re: Mirrormaker 2.0

2020-01-21 Thread Ryanne Dolan
Policy? If > there is, please link it here, I could not find it. > > Thanks, > Peter > > > > > On Fri, 17 Jan 2020 at 20:58, Ryanne Dolan wrote: > > > Peter, KIP-382 includes LegacyReplicationPolicy for this purpose, but no, > > it has not been implemented ye

Re: MirrorMaker 2

2020-01-18 Thread Ryanne Dolan
t; > wrote: > > > Thanks > > > > On Tue, Jan 14, 2020 at 8:47 AM Ryanne Dolan > > wrote: > > > >> Take a look at the DefaultConfigPropertyFilter class, which supports > >> customizable blacklists via config.properties.blacklist. > >>

Re: Clustered MirrorMaker 2 configuration update

2020-01-17 Thread Ryanne Dolan
on > Kubernetes. > I only have one mirroring path, but I guess this would work if I have more > paths as well. > > Peter > > On Thu, 16 Jan 2020 at 16:36, Ryanne Dolan wrote: > > > MM2 nodes only communicate via Kafka -- no connection is required between > >

Re: Mirrormaker 2.0

2020-01-17 Thread Ryanne Dolan
originalTopic(String topic) { return topic; } > > Thanks, > Peter > > On Mon, 30 Dec 2019 at 06:57, Ryanne Dolan wrote: > > > Sebastian, you can drop in a custom jar in the "Connect plug-in path" and > > MM2 will be able to load it. That enables you to implement

Re: Kafka Broker leader change without effect

2020-01-16 Thread Ryanne Dolan
onfig parameters. > > > -Original Message- > From: Ryanne Dolan > Sent: Thursday, January 16, 2020 10:52 AM > To: Kafka Users > Subject: Re: Kafka Broker leader change without effect > > Marco, the replication factor of 3 is not possible when you only have two > brokers, thu

Re: Kafka Broker leader change without effect

2020-01-16 Thread Ryanne Dolan
Marco, the replication factor of 3 is not possible when you only have two brokers, thus the producer will fail to send records until the third broker is restored. You would need to change the topic replication factor to 2 for your experiment to work as you expect. Ryanne On Thu, Jan 16, 2020, 9:5

Re: Clustered MirrorMaker 2 configuration update

2020-01-16 Thread Ryanne Dolan
MM2 nodes only communicate via Kafka -- no connection is required between them. To reconfigure, a rolling restart probably won't do what you expect, since the configuration is always dictated by a single leader. Once the leader is bounced, it will broadcast the new configuration via Kafka. If you

Re: How to achieve "Effectively one big consumer group" in active/active clusters

2020-01-16 Thread Ryanne Dolan
In this case the consumers just subscribe to "topic1" like normal, and the remote topics (primary.topic1, secondary.topic1) are just for DR. MM2 is not required for things to work under normal circumstances, but if one cluster goes down you can recover its data from the other. Ryanne On Thu, Jan

Re: Mirror Maker 2 problems

2020-01-14 Thread Ryanne Dolan
Nils, a REST API for MM2 was discussed at length, but so far the community hasn't settled on a plan. For now, if you need to monitor MM2's Connectors over HTTP, you can use a bunch of Connect clusters and manually configure them to run MirrorSourceConnector etc. This has some advantages, e.g. you c

Re: MirrorMaker 2

2020-01-14 Thread Ryanne Dolan
option for us. How do I blacklist retention.ms > ? > > On Tue, Jan 14, 2020 at 12:47 AM Ryanne Dolan > wrote: > > > Vishal, there is no support for overriding topic configuration like > > retention. Instead, remote topics will have the same configuration as > their

Re: MirrorMaker 2

2020-01-13 Thread Ryanne Dolan
Vishal, there is no support for overriding topic configuration like retention. Instead, remote topics will have the same configuration as their source topics. You could disable config sync or blacklist retention.ms to prevent that from happening, and then alter retention for remote topics manually

Re: Where to run MM2? Source or destination DC/region?

2020-01-09 Thread Ryanne Dolan
+1, it is preferable to run MM2 at the target/destination cluster. Basically, producer lag is more problematic than consumer lag, so you want the producer as close to the target cluster as possible. Also take a look at the "--clusters" option of the connect-mirror-maker.sh command, which lets you

Re: Mirrormaker 2.0

2020-01-09 Thread Ryanne Dolan
Peter, that's right. So long as ReplicationPolicy is implemented with proper semantics (i.e. the methods do what they say they should do) any naming convention will work. You can also use something like double underscore "__" as a separator with DefaultReplicationPolicy -- it doesn't need to be a s

Re: MirrorMaker 2 throttling

2020-01-08 Thread Ryanne Dolan
Peter, have you tried overriding the client ID used by MM2's consumers? Otherwise, the client IDs are dynamic, which would make it difficult to throttle using quotas. Ryanne On Wed, Jan 8, 2020, 10:12 AM P茅ter Sin贸ros-Szab贸 wrote: > Hi, > > I'd like to throttle the mirroring process when I star

Re: MirrorMaker 2 - Does it write anything to source cluster?

2020-01-08 Thread Ryanne Dolan
Peter, MM2 writes offset syncs upstream to the source cluster, which are then used to emit checkpoints to the target cluster. There is no particular reason why offset syncs are stored on the source cluster instead of the target, and it's been suggested that we swap that around. Ryanne On Wed, Jan

Re: Question On KIP-500

2020-01-07 Thread Ryanne Dolan
Hello. The dev list might be a better place to ask this question. FWIW I believe your interpretation is correct -- the proposal essentially uses two separate clusters, comprising "controllers" and "brokers". N.B. that the brokers cannot become controllers or vice versa. You can find the discussion

Re: Kafka 2.4.0 & Mirror Maker 2.0 Error

2020-01-06 Thread Ryanne Dolan
I just downloaded the 2.4.0 release tarball and didn't run into any issues. Peter, Jamie, can one of you file a jira ticket if you are still seeing this? Thanks! Ryanne On Fri, Dec 27, 2019 at 12:04 PM Ryanne Dolan wrote: > Thanks Peter, I'll take a look. > > Ryanne > >

Re: Mirrormaker 2.0

2019-12-29 Thread Ryanne Dolan
s, I wondered when the Networking-guys would come and complain about > me using too much bandwidth on the VPN-Link ;) > > Thanks > > Sebastian > > On 24-Dec-19 1:11 PM, Ryanne Dolan wrote: > > Glad to hear you are replicating now :) > > > >> it proba

Re: Mirrormaker 2.0

2019-12-29 Thread Ryanne Dolan
n > > > > On 27-Dec-19 7:24 AM, Sebastian Schmitz wrote: > >> Hello Ryanne, > >> > >> Is there a way to prevent that from happening? We have two separate > >> clusters with some topics being replicated to the second one for > >> reporting. If we

Re: MM2 startup delay

2019-12-27 Thread Ryanne Dolan
d task: syncing topic ACLs > (org.apache.kafka.connect.mirror.Scheduler:102) > java.util.concurrent.ExecutionException: > org.apache.kafka.common.errors.SecurityDisabledException: No Authorizer is > configured on the broker > at > > org.apache.kafka.common.internals.Kaf

Re: Kafka 2.4.0 & Mirror Maker 2.0 Error

2019-12-27 Thread Ryanne Dolan
t; Peter > > On Mon, 16 Dec 2019 at 17:14, Ryanne Dolan wrote: > > > Hey Jamie, are you running the MM2 connectors on an existing Connect > > cluster, or with the connet-mirror-maker.sh driver? Given your question > > about plugin.path I'm guessing the former. Is the

Re: Simplifying standalone mm2-connect config

2019-12-24 Thread Ryanne Dolan
Hello Karan. I agree the initial experience could be a lot friendlier. Most of the complexity there is inherited from Connect, but it's compounded when multiple clusters are involved. I don't think we want to change Connect's (or MM2's) defaults to assume a single broker cluster -- it'd be too eas

Re: Mirrormaker 2.0

2019-12-23 Thread Ryanne Dolan
rs made it work a bit more, at least it didn't > > complain about not having any servers in the config. > > So, I checked the "Running a dedicated MirrorMaker cluster"from the > > KIP, which is basically more or less the same, but without the > > "cluster

Re: Mirrormaker 2.0

2019-12-23 Thread Ryanne Dolan
er"from the KIP, > which is basically more or less the same, but without the "cluster." for > the servers and it does at least start and it looks like all the three > MMs find each other, but no mirroring taking place. > > Running the legacy-config from the old MM is wor

Re: Mirrormaker 2.0

2019-12-23 Thread Ryanne Dolan
Hello Sebastian, please let us know what issues you are facing and we can probably help. Which config from the KIP are you referencing? Also check out the readme under ./connect/mirror for more examples. Ryanne On Mon, Dec 23, 2019, 12:58 PM Sebastian Schmitz < sebastian.schm...@propellerhead.co.

Re: Kafka 2.4.0 & Mirror Maker 2.0 Error

2019-12-16 Thread Ryanne Dolan
Hey Jamie, are you running the MM2 connectors on an existing Connect cluster, or with the connet-mirror-maker.sh driver? Given your question about plugin.path I'm guessing the former. Is the Connect cluster running 2.4.0 as well? The jars should land in the Connect runtime without any need to modif

Re: rename.topics setting missing from MirrorMaker 2 ?

2019-12-14 Thread Ryanne Dolan
Hey Alan, good catch. I've removed that property from the KIP. We'll have a LegacyReplicationPolicy in a future release that will not rename topics. You could implement your own in the meantime (see replication.policy.class). Ryanne On Mon, Nov 25, 2019, 11:09 AM Alan wrote: > Hi, > > > > I've

Re: MM2 startup delay

2019-12-11 Thread Ryanne Dolan
Hey Peter. Do you see any timeouts in the logs? The internal scheduler will timeout each task after 60 seconds by default, which might not be long enough to finish some of the bootstrap tasks in your case. My team has observed that behavior in super-flaky environments, e.g. when connectivity drops

Re: Auto Scaling in Kafka

2019-12-04 Thread Ryanne Dolan
Akash, take a look at LinkedIn's Cruise Control project. It can automatically rebalance partitions across brokers, etc. Ryanne On Wed, Dec 4, 2019, 12:10 AM Goel, Akash Deep wrote: > Hi , > > Is it possible to auto scale Kafka? If it is not directly supported, then > is there automated way of a

Re: Running Kafka Stream Application in YARN

2019-11-15 Thread Ryanne Dolan
n. > Hence, and tutorial on how to run a Java application on YARN should help. > > > -Matthias > > On 11/11/19 10:33 AM, Ryanne Dolan wrote: > > Consider using Flink, Spark, or Samza instead. > > > > Ryanne > > > > On Fri, Nov 8, 2019, 4:27 AM Debraj M

  1   2   >