Ryanne, I didn't do much work yet, just checked the Interface to see if it is easy to implement or not.
> The PR for LegacyReplicationPolicy should include any relevant fixes to get it to run without crashing Do you mean that there is already a PR for LegacyReplicationPolicy? If there is, please link it here, I could not find it. Thanks, Peter On Fri, 17 Jan 2020 at 20:58, Ryanne Dolan <ryannedo...@gmail.com> wrote: > Peter, KIP-382 includes LegacyReplicationPolicy for this purpose, but no, > it has not been implemented yet. If you are interested in writing the PR, > it would not require a separate KIP before merging. Looks like you are > already doing the work :) > > It is possible, as you point out, that returning nulls like that will break > parts of MM2. The PR for LegacyReplicationPolicy should include any > relevant fixes to get it to run without crashing. > > Certainly some parts of MM2 will not work as intended when used with > LegacyReplicationPolicy, e.g. MM2 would not be able to prevent cycles. But > of course it should still _run_ and should work roughly the same as MM1. > > I'm happy to work with you on the PR if you are interested. > > Ryanne > > On Fri, Jan 17, 2020 at 9:34 AM Péter Sinóros-Szabó > <peter.sinoros-sz...@transferwise.com.invalid> wrote: > > > Hi Sebastian & Ryanne, > > > > do you have maybe an implementation of this is just some ideas about how > to > > implement the policy that does not rename topics? > > I am checking the ReplicationPolicy interface and don't really know what > > the impact will be if I implement this: > > > > public String formatRemoteTopic(String sourceClusterAlias, String topic) > { > > return topic; } > > public String topicSource(String topic) { return null; // well, I do not > > really know if this were a mirrored topic or not } > > public String upstreamTopic(String topic) { return topic; // well, I do > > not really know if this were a mirrored topic or not} > > public String originalTopic(String topic) { return topic; } > > > > Thanks, > > Peter > > > > On Mon, 30 Dec 2019 at 06:57, Ryanne Dolan <ryannedo...@gmail.com> > 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 your own > > > ReplicationPolicy (and other pluggable interfaces) without compiling > > > everything. > > > > > > In an upcoming release we'll have a "LegacyReplicationPolicy" that does > > not > > > rename topics. It's possible "SimpleReplicationPolicy" is a better > name. > > > > > > Be advised that some features depend on correct ReplicationPolicy > > > semantics, which LegacyReplicationPolicy will explicitly break. For > > > example, MM2 cannot prevent cycles if topics are not renamed (or some > > other > > > similar mechanism is used). > > > > > > Ryanne > > > > > > On Sun, Dec 29, 2019, 7:41 PM Sebastian Schmitz < > > > sebastian.schm...@propellerhead.co.nz> wrote: > > > > > > > Hello, > > > > > > > > I found that it's using the DefaultReplicationPolicy that always > > returns > > > > "sourceClusterAlias + separator + topic" with only the separator > being > > > > configurable in the configuration-file with > > REPLICATION_POLICY_SEPARATOR. > > > > > > > > It seems like I need a different ReplicationPolicy, like a > > > > SimpleReplicationPolicy which always returns "topic" for the > > > > formatRemoteTopic, then. But that would mean that I can't download > the > > > > Binaries and have to build the whole thing myself after adding the > new > > > > Policy-file!? > > > > Or I could create a PR for a SimpleReplicationPolicy to be in some > > > > future build... > > > > > > > > Any suggestions for this? > > > > > > > > Thanks > > > > > > > > Sebastian > > > > > > > > > > > > On 30-Dec-19 1:39 PM, Sebastian Schmitz wrote: > > > > > Hello, > > > > > > > > > > another thing I found and didn't find any configuration in the KIP > > yet > > > > > was that if I have two clusters (source and target) and a topic > > > > > "replicateme" on the source-cluster it will get replicated to the > > > > > target-cluster as "source.replicateme". > > > > > > > > > > How can I stop it from adding the cluster-name in front of the > > > > > topic-name on target-cluster? > > > > > > > > > > Thanks > > > > > > > > > > Sebastian > > > > > > > > > > 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 replicate everything again that reporting would > > > > >> probably have some problems. > > > > >> > > > > >> Yes, 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 probably started mirroring the last seven days as there was > no > > > > >>>> offset > > > > >>> for the new consumer-group. > > > > >>> > > > > >>> That's correct -- MM2 will replicate the entire topic, as far > back > > > > >>> as the > > > > >>> retention period. However, technically there are no consumer > groups > > > > >>> in MM2! > > > > >>> > > > > >>> 550MB/s in a test cluster sounds pretty good to me. Try > increasing > > > > >>> "tasks.max" and adding additional nodes. > > > > >>> > > > > >>> Ryanne > > > > >>> > > > > >>> > > > > >>> On Mon, Dec 23, 2019 at 5:40 PM Sebastian Schmitz < > > > > >>> sebastian.schm...@propellerhead.co.nz> wrote: > > > > >>> > > > > >>>> Hello again! > > > > >>>> > > > > >>>> Some probably important configs I found out: > > > > >>>> > > > > >>>> We need this to enable mirroring as it seems to disabled by > > default? > > > > >>>> > > > > >>>> source->target.enabled = true > > > > >>>> target->source.enabled = true > > > > >>>> > > > > >>>> Also, the Client-IDs can be configured using: > > > > >>>> > > > > >>>> source.client.id = my_cool_id > > > > >>>> target.client.id = my_cooler_id > > > > >>>> > > > > >>>> I configured them to include the ID of the server and the name > of > > > the > > > > >>>> environment to have separate IDs per mirror-node. > > > > >>>> > > > > >>>> After adding these two, it looks a bit better than before, but > > > > >>>> still not > > > > >>>> satisfied as it started to mirror from my prod to test with > > 550MB/s > > > as > > > > >>>> it probably started mirroring the last seven days as there was > no > > > > >>>> offset > > > > >>>> for the new consumer-group. That's next on my list to solve. > > > > >>>> > > > > >>>> Best regards > > > > >>>> > > > > >>>> Sebastian > > > > >>>> > > > > >>>> On 24-Dec-19 8:34 AM, Sebastian Schmitz wrote: > > > > >>>>> Hello, > > > > >>>>> > > > > >>>>> I tried running this connect-mirror-config: > > > > >>>>> > > > > >>>>> <snip> > > > > >>>>> name = $MIRROR_NAME > > > > >>>>> clusters = source, target > > > > >>>>> source.bootstrap.servers = $SOURCE_SERVERS > > > > >>>>> target.bootstrap.servers = $TARGET_SERVERS > > > > >>>>> source->target.topics = $SOURCE_TARGET_TOPICS > > > > >>>>> target->source.topics = $TARGET_SOURCE_TOPICS > > > > >>>>> source->target.emit.heartbeats.enabled = true > > > > >>>>> target->source.emit.heartbeats.enabled = true > > > > >>>>> connector.class = > > > > >>>>> org.apache.kafka.connect.mirror.MirrorSourceConnector > > > > >>>>> > > > > >>>>> # disable some new features > > > > >>>>> refresh.topics.enabled = false > > > > >>>>> refresh.groups.enabled = false > > > > >>>>> emit.checkpoints.enables = true > > > > >>>>> emit.heartbeats.enabled = true > > > > >>>>> sync.topic.configs.enabled = false > > > > >>>>> sync.topic.acls.enabled = false > > > > >>>>> </snip> > > > > >>>>> > > > > >>>>> SOURCE_SERVERS and TARGET_SERVERS are a comma-separated list of > > > three > > > > >>>>> brokers with ports. > > > > >>>>> The TOPICS are |-separated lists of topics. > > > > >>>>> > > > > >>>>> I get these warning during startup which is a bit weird as I > > never > > > > >>>>> supplied any of those settings, but maybe I should? > > > > >>>>> > > > > >>>>> [2019-12-23 00:36:25,918] WARN The configuration > > > > >>>>> 'config.storage.topic' was supplied but isn't a known config. > > > > >>>>> (org.apache.kafka.clients.producer.ProducerConfig:355) > > > > >>>>> [2019-12-23 00:36:25,918] WARN The configuration > > > > >>>>> 'producer.bootstrap.servers' was supplied but isn't a known > > config. > > > > >>>>> (org.apache.kafka.clients.producer.ProducerConfig:355) > > > > >>>>> [2019-12-23 00:36:25,918] WARN The configuration 'group.id' > was > > > > >>>>> supplied but isn't a known config. > > > > >>>>> (org.apache.kafka.clients.producer.ProducerConfig:355) > > > > >>>>> [2019-12-23 00:36:25,919] WARN The configuration > > > > >>>>> 'status.storage.topic' was supplied but isn't a known config. > > > > >>>>> (org.apache.kafka.clients.producer.ProducerConfig:355) > > > > >>>>> [2019-12-23 00:36:25,919] WARN The configuration > > 'header.converter' > > > > >>>>> was supplied but isn't a known config. > > > > >>>>> (org.apache.kafka.clients.producer.ProducerConfig:355) > > > > >>>>> [2019-12-23 00:36:25,919] WARN The configuration > > > > >>>>> 'consumer.bootstrap.servers' was supplied but isn't a known > > config. > > > > >>>>> (org.apache.kafka.clients.producer.ProducerConfig:355) > > > > >>>>> [2019-12-23 00:36:25,919] WARN The configuration > > > > >>>>> 'offset.storage.topic' was supplied but isn't a known config. > > > > >>>>> (org.apache.kafka.clients.producer.ProducerConfig:355) > > > > >>>>> [2019-12-23 00:36:25,919] WARN The configuration > > 'value.converter' > > > > >>>>> was > > > > >>>>> supplied but isn't a known config. > > > > >>>>> (org.apache.kafka.clients.producer.ProducerConfig:355) > > > > >>>>> [2019-12-23 00:36:25,919] WARN The configuration > 'key.converter' > > > was > > > > >>>>> supplied but isn't a known config. > > > > >>>>> (org.apache.kafka.clients.producer.ProducerConfig:355) > > > > >>>>> [2019-12-23 00:36:25,919] WARN The configuration > > > > >>>>> 'admin.bootstrap.servers' was supplied but isn't a known > config. > > > > >>>>> (org.apache.kafka.clients.producer.ProducerConfig:355) > > > > >>>>> > > > > >>>>> And this error: > > > > >>>>> > > > > >>>>> [2019-12-23 00:36:29,320] ERROR Plugin class loader for > > connector: > > > > >>>>> 'org.apache.kafka.connect.mirror.MirrorSourceConnector' was not > > > > >>>>> found. > > > > >>>>> Returning: > > > > >>>>> > > > > > > org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader@5c316230 > > > > >>>>> > > > > >>>>> > > > > > (org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader:165) > > > > >>>>> > > > > >>>>> > > > > >>>>> First I tried the config mentioned in the KIP for "MirrorMaker > > > > >>>>> Clusters" which didn't work and I found removing the "cluster." > > > from > > > > >>>>> the bootstrap-servers 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." 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 working fine > though. > > > > >>>>> I'll > > > > >>>>> try to do some more digging today, so if you need some of those > > > very > > > > >>>>> verbose logs or something else just let me know. I am sure > that I > > > can > > > > >>>>> figure this out and just wanted to know if the documentation > will > > > get > > > > >>>>> extended as the new MM2 has a lot of features and is a bit more > > > > >>>>> complicated than the old one... > > > > >>>>> > > > > >>>>> Thanks > > > > >>>>> > > > > >>>>> Sebastian > > > > >>>>> > > > > >>>>> On 24-Dec-19 8:06 AM, Ryanne Dolan wrote: > > > > >>>>>> 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.nz> wrote: > > > > >>>>>> > > > > >>>>>>> Hello, > > > > >>>>>>> > > > > >>>>>>> I'm currently trying to implement the new Kafka 2.4.0 and the > > > > >>>>>>> new MM2. > > > > >>>>>>> > > > > >>>>>>> However, it looks like the only documentation available is > the > > > > >>>>>>> KIP-382, > > > > >>>>>>> and the documentation > > > > >>>>>>> ( > > https://kafka.apache.org/documentation/#basic_ops_mirror_maker) > > > > >>>>>>> for > > > > >>>>>>> the > > > > >>>>>>> MM isn't yet updated, and the documentation in the KIP seems > to > > > be > > > > >>>>>>> missing some stuff as I get a lot of errors and warning when > > > > >>>>>>> starting > > > > >>>>>>> the MM2 as connect-mirror, and it doesn't mirror, so I > probably > > > > >>>>>>> have > > > > >>>>>>> some mistakes in my configuration, but can't confirm this as > > > > >>>>>>> it's the > > > > >>>>>>> same as in the KIP. > > > > >>>>>>> > > > > >>>>>>> Any plans when the documentation will be updated? > > > > >>>>>>> > > > > >>>>>>> Thanks > > > > >>>>>>> > > > > >>>>>>> Sebastian > > > > >>>>>>> > > > > >>>>>>> > > > > >>>>>>> -- > > > > >>>>>>> DISCLAIMER > > > > >>>>>>> This email contains information that is confidential and > which > > > > >>>>>>> may be > > > > >>>>>>> legally privileged. If you have received this email in error > > > please > > > > >>>>>>> > > > > >>>>>>> notify the sender immediately and delete the email. > > > > >>>>>>> This email is intended > > > > >>>>>>> solely for the use of the intended recipient and you may not > > use > > > or > > > > >>>>>>> disclose this email in any way. > > > > >>>>>>> > > > > >>>> -- > > > > >>>> DISCLAIMER > > > > >>>> This email contains information that is confidential and which > > > > >>>> may be > > > > >>>> legally privileged. If you have received this email in error > > please > > > > >>>> > > > > >>>> notify the sender immediately and delete the email. > > > > >>>> This email is intended > > > > >>>> solely for the use of the intended recipient and you may not use > > or > > > > >>>> disclose this email in any way. > > > > >>>> > > > > > > > > -- > > > > DISCLAIMER > > > > This email contains information that is confidential and which > > > > may be > > > > legally privileged. If you have received this email in error please > > > > > > > > notify the sender immediately and delete the email. > > > > This email is intended > > > > solely for the use of the intended recipient and you may not use or > > > > disclose this email in any way. > > > > > > > > > >