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.

Reply via email to