Hi Jigar, Thanks to the experts in the community (Mickael and Ed), we have a workaround for your issue. Before v3.2.0, you can manually create the topic with problem in the target cluster. In your case, it's "test-topic".
After this bug is fixed in v3.2.0, you can set `config.properties.exclude` in mirror maker config, to exclude the topic config properties that should not be replicated. In your case, it's "remote.storage.enable". Hope this helps. Thank you. Luke On Wed, Dec 15, 2021 at 9:13 PM Luke Chen <[email protected]> wrote: > Hi Jigar, > > This is a known issue (KAFKA-13255 > <https://issues.apache.org/jira/browse/KAFKA-13255>) that we will be > fixed in v3.2.0. > I've asked if there is any workaround for this issue in the JIRA comments. > > Thank you. > Luke > > On Wed, Dec 15, 2021 at 1:24 PM Jigar Shah <[email protected]> > wrote: > >> Hello, >> I am trying to run MirrorMaker 2.0 on kafka version 3.0.0 on my source >> cluster and my target kafka cluster is in kafka version 2.7.2. >> I am facing issues with topic configuration to create topics on my target >> cluster. >> >> >> [2021-12-14 12:28:33,071] WARN [MirrorSourceConnector|worker] Could not >> create topic test-topic. >> >> (org.apache.kafka.connect.mirror.MirrorSourceConnector:371)java.util.concurrent.CompletionException: >> org.apache.kafka.common.errors.InvalidConfigurationException: Unknown >> topic >> config name: remote.storage.enable >> >> at >> >> java.base/java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:331) >> >> at >> >> java.base/java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:346) >> >> at >> >> java.base/java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:632) >> >> at >> >> java.base/java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:506) >> >> at >> >> java.base/java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:2088) >> >> at >> >> org.apache.kafka.common.internals.KafkaCompletableFuture.kafkaCompleteExceptionally(KafkaCompletableFuture.java:49) >> >> at >> >> org.apache.kafka.common.internals.KafkaFutureImpl.completeExceptionally(KafkaFutureImpl.java:130) >> >> at >> >> org.apache.kafka.clients.admin.KafkaAdminClient$1.handleResponse(KafkaAdminClient.java:1601) >> >> at >> >> org.apache.kafka.clients.admin.KafkaAdminClient$AdminClientRunnable.handleResponses(KafkaAdminClient.java:1244) >> >> at >> >> org.apache.kafka.clients.admin.KafkaAdminClient$AdminClientRunnable.processRequests(KafkaAdminClient.java:1397) >> >> at >> >> org.apache.kafka.clients.admin.KafkaAdminClient$AdminClientRunnable.run(KafkaAdminClient.java:1320) >> >> at java.base/java.lang.Thread.run(Thread.java:836) >> >> Caused by: org.apache.kafka.common.errors.InvalidConfigurationException: >> Unknown topic config name: remote.storage.enable >> >> >> >> >> >> I am not familiar with how to configure topic level configuration for >> MirrorMaker 2.0. >> >> I am configuring MirrorMaker 2.0 in a dedicated cluster mode. >> >> >> >> Is there any way to remove this topic configuration for topics when >> mirroring to older versions of kafka ,please? >> >> >> >> Thank you in advance for feedback >> >
