Re: Exposing Kafka on WAN

2018-08-24 Thread Ryanne Dolan
Can you use a Kafka HTTP proxy instead of using the Kafka protocol directly? Ryanne On Thu, Aug 23, 2018, 7:29 PM Jack S wrote: > Hello, > > We have a requirement for opening Kafka on WAN where external producers and > consumers need to be able to talk to Kafka. I was able to get Zookeeper and

Re: Maintain "consumer has all the messages for a key" delivery guarantee with two topics?

2018-08-24 Thread Ryanne Dolan
Instead of using two topics, consider adding a priority field to your records, and then use a priority queue in the consumers. For example, each consumer can have two queues, one for high and one for low priority records, and can process the low priority queue only when the high priority queue is e

Re: Frequent appearance of "Marking the coordinator dead" message in consumer log

2018-08-28 Thread Ryanne Dolan
Shantanu, Sounds like your consumers are processing too many records between poll()s. Notice that max.poll.records is 50. If your consumer is taking up to 200ms to process each record, then you'd see up to 10 seconds between poll()s. If a consumer doesn't call poll() frequently enough, Kafka will

Re: Query related to Kafka Consumer Limit

2018-08-29 Thread Ryanne Dolan
Satarupa, it sounds like you are conflating some concepts here. Some clarifying points: - Only one consumer in a consumer group receives any given record from a topic. So in your scenario of 1 million consumers, they could not be members of the same group. You'd need 1 million consumer "groups" to

Re: Query related to Kafka Consumer Limit

2018-08-29 Thread Ryanne Dolan
e) when there is a Hit fix/ Critical fix is released. So > that all our customers are notified to take the update. > > Thank you, > Satarupa > > On 8/29/18, 11:38 PM, "Ryanne Dolan" wrote: > > Satarupa, it sounds like you are conflating some concepts here. Some &g

Re: Frequent appearance of "Marking the coordinator dead" message in consumer log

2018-08-31 Thread Ryanne Dolan
ic and start consumption. Why might that be happening, any idea? > > > > On Tue, Aug 28, 2018 at 8:44 PM Ryanne Dolan > > wrote: > > > >> Shantanu, > >> > >> Sounds like your consumers are processing too many records between > >> poll()

Re: Official Kafka Disaster Recovery is insufficient - Suggestions needed

2018-09-03 Thread Ryanne Dolan
Hanning, In mission-critical (and indeed GDPR-related) applications, I've ETL'd Kafka to a secondary store e.g. HDFS, and built tooling around recovering state back into Kafka. I've had situations where data is accidentally or incorrectly ingested into Kafka, causing downstream systems to process

Re: Official Kafka Disaster Recovery is insufficient - Suggestions needed

2018-09-03 Thread Ryanne Dolan
Sorry to have misspelled your name Henning. On Mon, Sep 3, 2018, 1:26 PM Ryanne Dolan wrote: > Hanning, > > In mission-critical (and indeed GDPR-related) applications, I've ETL'd > Kafka to a secondary store e.g. HDFS, and built tooling around recovering > state

Re: Official Kafka Disaster Recovery is insufficient - Suggestions needed

2018-09-04 Thread Ryanne Dolan
uted environment, and will > most likely lead to loss of messages with timestamps close to the root bad > message. > > Q3) > Does the statement “Don't rely on unlimited retention in Kafka” imply some > flaw in the implementation, or is it simply a reference to the advice of &

Re: SAM Scala aggregate

2018-09-09 Thread Ryanne Dolan
This means the types of k, v, vr don't match those expected by the aggregate() function. Add explicit types to your code and you'll find the problem. You'll probably find that Scala is inferring an Any somewhere. Ryanne On Sun, Sep 9, 2018, 12:14 AM Michael Eugene wrote: > Hi, > > I am using

Re: Asking about how to consume the message from follower partition

2018-10-01 Thread Ryanne Dolan
I'm not sure whether this is what the OP is asking for, but some partitioned systems let you select a specific follower to consume from _instead_ of from the leader, e.g. couchbase, which can be useful in scenarios where there are a ton more consumers than producers for a partition, and you don't w

Re: [ANNOUNCE] New Committer: Manikumar Reddy

2018-10-11 Thread Ryanne Dolan
Bravo! On Thu, Oct 11, 2018 at 1:48 PM Ismael Juma wrote: > Congratulations Manikumar! Thanks for your continued contributions. > > Ismael > > On Thu, Oct 11, 2018 at 10:39 AM Jason Gustafson > wrote: > > > Hi all, > > > > The PMC for Apache Kafka has invited Manikumar Reddy as a committer and

Re: multiple consumers on one partition using assign? (not subscribe)?

2018-10-17 Thread Ryanne Dolan
David, If the consumers never commit() or subscribe(), I don't think the group.id is relevant. Ryanne On Wed, Oct 17, 2018 at 10:04 AM David Baraff wrote: > I’m using Kafka for what I think is a somewhat non-standard purpose. We > have multiple producers which send messages to a topic. We ha

Re: running kafka streams inside kafka connect

2018-10-25 Thread Ryanne Dolan
Dhurandar, definitely! Connect and Streams are both agnostic to how their workers are run. They aren't really platforms per se. You just need to spin up one or more workers and they do their thing. So a Streams app doesn't run "inside" Connect, but you can certainly have Connect and Streams workers

Re: Deciding number of partitions

2018-11-11 Thread Ryanne Dolan
Sachit, there are many factors. Some to consider: - # partitions is the max number of consumers the topic can have per group, since each consumer will need at least one partition assigned to it. - # partitions is usually also the max concurrency factor across a consumer group, since consumers ofte

Re: Using GlobalKTable/KeyValueStore for topic cache

2018-11-13 Thread Ryanne Dolan
Chris, consider using log compaction. Ryanne On Tue, Nov 13, 2018, 3:17 PM Chris Toomey wrote: > We're considering using GlobalKTables / KeyValueStores for locally caching > topic content in services. The topics would be compacted such that only the > latest key/value pair would exist for a giv

Re: Kafka running on AWS - how to retain broker.id on new instance spun-up in-place of instance/broker failed

2018-11-16 Thread Ryanne Dolan
> if we are not using EBS In that case, what's the point of keeping the broker id? The data will be lost anyway right? On Thu, Nov 15, 2018, 11:40 AM Srinivas Rapolu wrote: > Yes, I understand we need to specify the required broker.id in > server.properties/meta.properties file in-order to ret

Re: Kafka in multi-dc deployment

2018-11-19 Thread Ryanne Dolan
Igor, currently the best available disaster recovery strategy looks something like this: - Use MirrorMaker to replicate data from a source cluster to a target cluster. MM should be co-located with the target cluster to minimize producer lag. If you have multiple active data centers, each would nee

Re: Please explain Rest API

2018-11-23 Thread Ryanne Dolan
Satendra, the Connect REST API enables starting, stopping, and reconfiguring Connecters in a distributed Connect cluster. Kafka Connect, more generally, is a framework for connecting Kafka to external data sources, e.g. Change Data Capture from relational databases, etc. Ryanne On Fri, Nov 23, 20

Re: Please explain Rest API

2018-11-28 Thread Ryanne Dolan
tor command and where to run. > > Please elaborate. > > Sent from my iPhone > > > On 23-Nov-2018, at 21:43, Ryanne Dolan wrote: > > > > Satendra, the Connect REST API enables starting, stopping, and > > reconfiguring Connecters in a distributed Connect cluster.

Re: Messgages from different sites

2018-12-07 Thread Ryanne Dolan
Mike, a couple things you might take a look at: Kafka Connect might be useful for writing to your DB. Your consumer->producer->DB flow seems to fit well with Connect. Transactions might be useful for ensuring that your application runs to completion before committing records. It sounds like your

Re: Messgages from different sites

2018-12-10 Thread Ryanne Dolan
. My concern is mainly about: > > - Send data to a central point that can be accessible from all sites. > - Sending time should be minimal. > - Sent data should be grouped ( errors, access, environment) for each > execution. > > Br, > > //mike > > We will use > -

Re: Kafka rebalancing message lost

2018-12-18 Thread Ryanne Dolan
Parth, I am skeptical that you actually need 500+ consumers. A well tuned consumer can process hundreds of thousands of records per second. Some notes to consider: - You'll need at least 500 partitions if you have 500 consumers. - You almost never need more consumers than you have brokers in your

Re: Cannot read messages using a specific group

2018-12-18 Thread Ryanne Dolan
Karim, can you inspect the offsets using kafka-consumer-groups.sh and let us know what you see? Also, is it possible that "group_name" is being used by a consumer group that is still running? For example, say you have a topic with one partition and a consumer with "group_name" already. In that cas

Re: Programmatic method of setting consumer groups offsets

2019-01-03 Thread Ryanne Dolan
Nimi, try using the low-level consumer API via assign() instead of subscribe(), and then commit() each partition. I suspect you are trying to commit partitions that the consumer is not subscribed to. Ryanne On Thu, Jan 3, 2019, 6:13 PM Nimi Wariboko Hi, > > Is there a guide or API that is implem

Re: Declarative Kafka Configuration As Code

2019-01-04 Thread Ryanne Dolan
Brian, you can use AdminClient to configure Kafka topics programmatically. You might consider using a jvm language like Scala that can embed a DSL for configuration. Ryanne On Fri, Jan 4, 2019, 5:38 PM Brian Attwell I'm setting up our Kafka cluster at a new company, and I think it would be > ben

Re: Dynamic Partitioning

2019-01-08 Thread Ryanne Dolan
Can you explain why you think you need dynamic partitioning? Maybe we can help you design without it. Ryanne On Tue, Jan 8, 2019, 4:04 AM marimuthu eee Hi, > > My requirement is to process notification messages in parallel.So i have an > idea to pick kafka as a messaging system.But i need dynami

Re: Dynamic Partitioning

2019-01-08 Thread Ryanne Dolan
Just key your messages by user ID and they'll be processed in sequence. Every message with the same key will go to the same partition, regardless of how many partitions there are. Ryanne On Tue, Jan 8, 2019, 8:57 AM marimuthu eee Hi Ryanne Dolan, > > My requirement is to process n

Re: Configuration guidelines for a specific use-case

2019-01-08 Thread Ryanne Dolan
Latency sounds high to me, maybe your JVMs are GC'ing a lot? Ryanne On Tue, Jan 8, 2019, 11:45 AM Gioacchino Vino Hi expert, > > > I would ask you some guidelines, web-pages or comments regarding my > use-case. > > > *Requirements*: > > - 2000+ producers > > - input rate 600k messages/s > > - co

Re: Configuration guidelines for a specific use-case

2019-01-11 Thread Ryanne Dolan
mprove it just a bit. > > Do you think using 6 partitions in a 3 broker cluster is a good choice? > > > Gioacchino > > > On 08/01/2019 18:52, Ryanne Dolan wrote: > > Latency sounds high to me, maybe your JVMs are GC'ing a lot? > > > > Ryanne > >

Re: Prioritized Topics for Kafka

2019-01-17 Thread Ryanne Dolan
Nick, I think it's worth noting that Kafka is not a real-time system, and most of these use cases (and TBH any that I can imagine) for prioritized topics are real-time use cases. For example, you wouldn't want to pilot a jet (a canonical real-time system) based on Kafka events, as there is no guar

Re: Drawbacks for configuring many partitions for a topic

2019-01-27 Thread Ryanne Dolan
Jazz, the number of partitions isn't necessarily related to message volume. The biggest factors for max message volume would be the number of brokers and their write speed. You should choose the number of partitions based on the number of brokers and the number of consumers you expect to have. Rya

Re: How to balance messages in kafka topics with newly added partitions?

2019-01-27 Thread Ryanne Dolan
You can use MirrorMaker to copy data between topics. Ryanne On Sun, Jan 27, 2019, 7:12 AM jaaz jozz Thanks, Sönke > Is there any available kafka tool to move messages between topics? > > On Sun, Jan 27, 2019 at 2:40 PM Sönke Liebau > wrote: > > > Hi Jazz, > > > > I'm afraid the only way of reba

Re: Kafka Streams & Distributed state question

2019-02-13 Thread Ryanne Dolan
Andrian, this looks useful: https://docs.confluent.io/current/streams/developer-guide/interactive-queries.html The gist is you'd need to build your own routing and discovery layers. Also consider materializing your data to an external data store, e.g. Couchbase or Hive, which provides this funct

Re: Odd behaviour of MirrorMaker and ConsumerGroupCommand

2019-03-05 Thread Ryanne Dolan
Tolya, That is the expected behavior. Offsets are not consistent between mirrored clusters. Kafka allows duplicate records ("at least once"), which means the downstream offsets will tend to creep higher than those in the source partitions. For example, if a producer sends a record but doesn't rec

Re: Odd behaviour of MirrorMaker and ConsumerGroupCommand

2019-03-05 Thread Ryanne Dolan
er handles it? > > I have an idea for failover – let consumers read from specific time (not > offset number). Should it work? > > Also, I think MM2 Is a nice idea and waiting for it! > > Tolya > > > 5 марта 2019 г., в 18:08, Ryanne Dolan > написал(а): > > &

Re: Kafka Mirror Maker place of execution

2019-03-12 Thread Ryanne Dolan
Franz, you can run MM on or near either source or target cluster, but it's more efficient near the target because this minimizes producer latency. If latency is high, poducers will block waiting on ACKs for in-flight records, which reduces throughput. I recommend running MM near the target cluster

Re: Operationalizing Zookeeper and common gotchas

2019-03-18 Thread Ryanne Dolan
Eno, I found this useful, thanks. Ryanne On Mon, Mar 18, 2019, 12:16 PM Eno Thereska wrote: > Hi folks, > > The team here has come up with a couple of clarifying tips for > operationalizing Zookeeper for Kafka that we found missing from the > official documentation, and passed them along to sha

Re: Proxying the Kafka protocol

2019-03-19 Thread Ryanne Dolan
Hello James, I'm not aware of anything like that for Kafka, but you can use MirrorMaker for network segmentation. With this approach you have one Kafka cluster in each segment and a MM cluster in the more privileged segment. You don't need to expose the privileged segment at all -- you just need to

Re: Source Connector Task in a distributed env

2019-04-24 Thread Ryanne Dolan
Venkata, the example you have linked creates a single task config s.t. there is no parallelism -- a single task runs on the cluster, regardless of the number of nodes. In order to introduce parallelism, your SourceConnector needs to group all known files among N partitions and return N task configs

Re: Mirror Maker tool is not running

2019-05-05 Thread Ryanne Dolan
This can be caused by compressed messages being decompressed by the MirrorMaker consumer but not recompressed by the producer. In order to prevent this, enable shallow iteration in the consumer, or enable compression in the producer. Ryanne On Sun, May 5, 2019, 8:35 AM ASHOK MACHERLA wrote: > H

Re: Mirror Maker tool is not running

2019-05-07 Thread Ryanne Dolan
Shallow iteration should be enabled for the consumer, not producer. Ryanne On Mon, May 6, 2019, 8:02 AM ASHOK MACHERLA wrote: > Dear Team Members > > > > Please find these below configurations for mirror maker tool scripts > > > > *source-cluster.config* > > > > bootstrap.servers=192.168.175.12

Re: Event Sourcing question

2019-05-08 Thread Ryanne Dolan
Pavel, one thing I'd recommend: don't jam multiple event types into a single topic. You are better served with multiple topics, each with a single schema and event type. In your case, you might have a received topic and a transformed topic, with an app consuming received and producing transformed.

Re: Customers getting duplicate emails

2019-05-10 Thread Ryanne Dolan
Kafka only supports exactly-once and idempotency within the context of streams apps where records are consumed and produced within the same cluster. As soon as you touch the outside world in a non-idempotent way, e.g. by sending an email, these guarantees fall away. It is essentially impossible to

Re: MirrorMaker 2.0 XDCR / KIP-382

2019-05-30 Thread Ryanne Dolan
Hey Jeremy, it looks like you've got a typo or copy-paste artifact in the configuration there -- you've got DC1->DC2 listed twice, but not the reverse. That would result in the behavior you are seeing, as DC1 actually has nothing enabled. Assuming this is just a mistake in the email, your approach

Re: MirrorMaker 2.0 XDCR / KIP-382

2019-05-30 Thread Ryanne Dolan
; are conflicting about what should be running since they share the same > group names/ connect name, etc. I tried overriding the groud.id and name > of the connectors which resulted in no replication. Not quite sure what > could be going wrong. > > > > On Thu, May 30, 2019

Re: MirrorMaker 2.0 XDCR / KIP-382

2019-06-04 Thread Ryanne Dolan
issue. Thanks again! Ryanne On Thu, May 30, 2019 at 12:22 PM Ryanne Dolan wrote: > Jeremy, thanks for double checking. I think you are right -- this is a > regression introduced here [1]. For context, we noticed that heartbeats > were only being sent to target clusters, whereas they sho

Re: Consumer offsets with Kafka Mirror Maker

2019-06-24 Thread Ryanne Dolan
Mohit, PR-6295 and KIP-382 introduce MirorrMaker 2.0 which was designed to support this operation. In a nutshell, MM2 maintains a sparse offset sync stream while replicating records between clusters. The offset syncs are used to translate consumer offsets for periodic cross-cluster checkpoints. Le

Re: lag consuming different-sized topics

2019-07-16 Thread Ryanne Dolan
David, I'd look first at ways to speed up the processing downstream of the consumer, i.e. whatever logic you have writing to HDFS, and in particular to reduce blocking there, as that is more likely to be the bottleneck than the consumer itself. Some ideas (that I've had success with): - turn off a

Re: Multiple bootstrap clusters in kafka producer config

2019-08-28 Thread Ryanne Dolan
Will and Garvit, you can use a load balancer with health checks for this purpose. Ryanne On Wed, Aug 28, 2019, 6:09 PM Will Weber wrote: > Apologies for piggybacking on a thread, figured the discussion was pretty > relevant to a thought I had kicking around my brain. > > In the event of complet

Re: Multiple bootstrap clusters in kafka producer config

2019-08-29 Thread Ryanne Dolan
ut I haven’t found much information on that strategy > and the associated tradeoffs. > > Best, > > On Thu, Aug 29, 2019 at 12:36 AM Ryanne Dolan > wrote: > > > Will and Garvit, you can use a load balancer with health checks for this > > purpose. > > > >

Re: Running a stand alone Mirror Maker 2.0 cluster and Issues

2019-10-13 Thread Ryanne Dolan
Vishal, the first issue is easy: you must set tasks.max to something above 1 (the default) in order to achieve any parallelism. This property is passed along to the internal Connect workers. It's unfortunate that Connect is not smart enough to default this property to the number of workers. I suspe

Re: Running a stand alone Mirror Maker 2.0 cluster and Issues

2019-10-14 Thread Ryanne Dolan
What seems strange is that I do not have a single record in the offsets > topic.. Is that normal ? I would imagine that without a record, there is > no way that a restore would happen And that is obvious when I restart > the mm2 instance... Find the screenshot attached. In essence t

Re: Running a stand alone Mirror Maker 2.0 cluster and Issues

2019-10-14 Thread Ryanne Dolan
utor.runWorker(ThreadPoolExecutor.java:1149) > > at > > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > > at java.lang.Thread.run(Thread.java:748) > > Caused by: org.apache.kafka.common.errors.SecurityDisabledException: No > Au

Re: Running a stand alone Mirror Maker 2.0 cluster and Issues

2019-10-14 Thread Ryanne Dolan
> > at > > > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > > > > at > > > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > > > > at java.lang.Thread.r

Re: Running a stand alone Mirror Maker 2.0 cluster and Issues

2019-10-15 Thread Ryanne Dolan
s ( b'coz num.tasks=4 ), and I have 2 topics with > 1 partition each. Do I assume this right, as in there are 4 consumer groups > ( on CG per thread ) ... > > > > > > THANKS A LOT > > > Vishal. > > > > On Mon, Oct 14, 2019 at 3:42 PM Ryanne Dolan > wro

Re: Running a stand alone Mirror Maker 2.0 cluster and Issues

2019-10-15 Thread Ryanne Dolan
r success or a failure, and > nothing in between ) > > Thanks.. > > On Tue, Oct 15, 2019 at 12:34 PM Ryanne Dolan > wrote: > > > Hey Vishal, glad to hear you're making progress. > > > > > 1. It seems though that flushing [...] the producer and set

Re: Running a stand alone Mirror Maker 2.0 cluster and Issues

2019-10-16 Thread Ryanne Dolan
t 15, 2019 at 3:44 PM Vishal Santoshi > > wrote: > > > Thank you so much for all your help. Will keep you posted on tests I > do.. > > I hope this is helpful to other folks too.. > > > > On Tue, Oct 15, 2019 at 2:44 PM Ryanne Dolan > > wrote: > > > >&g

Re: Running a stand alone Mirror Maker 2.0 cluster and Issues

2019-10-17 Thread Ryanne Dolan
> props.put("enable.auto.commit", "false"); > > -props.put("auto.offset.reset", "earliest"); > > +props.put("auto.offset.reset", "latest"); > > return props; > > } > >

Re: Running a stand alone Mirror Maker 2.0 cluster and Issues

2019-10-18 Thread Ryanne Dolan
sh failures ( as a GAUGE ) given > >> offset.flush.timeout.ms would be highly beneficial. > >> > >> Regards.. > >> > >> > >> On Thu, Oct 17, 2019 at 11:53 PM Ryanne Dolan > >> wrote: > >> > >>> Ah, I s

Re: Running a stand alone Mirror Maker 2.0 cluster and Issues

2019-10-21 Thread Ryanne Dolan
t;> prevented re assignment on a scale up ( as it clearly did ) ? > >> > >> The reason I ask is that I plan to run mm2 cluster on k8s and I want to > >> make sure that I use the version of JVM that is more docker friendly > vis a > >> vis, how

Re: MirrorMaker 2.0 ( working off origin/2.4 )

2019-11-04 Thread Ryanne Dolan
Vishal, b.producer.acks should work, as can be seen in the following unit test with similar producer property "client.id": https://github.com/apache/kafka/blob/6b905ade0cdc7a5f6f746727ecfe4e7a7463a200/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorMakerConfigTest.java#L182 Kee

Re: MirrorMaker 2.0 ( working off origin/2.4 )

2019-11-04 Thread Ryanne Dolan
e in the Sender and > that alleviated the throttle I was seeing on consumption. BTW any ideas > when 2.4 is being released ( I thought it was Oct 30th 2019 )... > > On Mon, Nov 4, 2019 at 1:49 PM Ryanne Dolan wrote: > > > Vishal, b.producer.acks should work, as can be seen in

Re: Detecting cluster down in consumer

2019-11-11 Thread Ryanne Dolan
Sachin, assuming you are using something like MM2, I recommend the following approaches: 1) have an external system monitor the clusters and trigger a failover by terminating the existing consumer group and launching a replacement. This can be done manually or can be automated if your infrastructu

Re: Running Kafka Stream Application in YARN

2019-11-11 Thread Ryanne Dolan
Consider using Flink, Spark, or Samza instead. Ryanne On Fri, Nov 8, 2019, 4:27 AM Debraj Manna wrote: > Hi > > Is there any documentation or link I can refer to for the steps for > deploying the Kafka Streams application in YARN? > > Kafka Client - 0.11.0.3 > Kafka Broker - 2.2.1 > YARN - 2.6.

Re: MirrorMaker 2 Plugin class loader Error

2019-11-11 Thread Ryanne Dolan
Rajeev, the config errors are unavoidable at present and can be ignored or silenced. The Plugin error is concerning, and was previously described by Vishal. I suppose it's possible there is a dependency conflict in these builds. Can you send me the hash that you're building from? I'll try to reprod

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

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: 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: 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: 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: 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: 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
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: 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: 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: 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: 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: 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: 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: 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: 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: 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.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: 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

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: 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: 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: 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: 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: 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: 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: 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: 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

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: 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: 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 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 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: 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: > &

  1   2   >