[ https://issues.apache.org/jira/browse/KAFKA-4350?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15612592#comment-15612592 ]
Emanuele Cesena commented on KAFKA-4350: ---------------------------------------- I did a bit more investigation. I can reproduce similar errors (not exactly the same, but I think very much related) just using the kafka-console-consumer from kafka 0.10.1 distribution against kafka 0.9 broker(s). If I use the new consumer (default), I get a similar error to the mirror maker above. {code:java} org.apache.kafka.common.protocol.types.SchemaException: Error reading field 'brokers': Error reading field 'host': Error reading string of length 27233, only 515 bytes available at org.apache.kafka.common.protocol.types.Schema.read(Schema.java:73) at org.apache.kafka.clients.NetworkClient.parseResponse(NetworkClient.java:380) at org.apache.kafka.clients.NetworkClient.handleCompletedReceives(NetworkClient.java:449) at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:269) at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:232) at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:180) at org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureCoordinatorReady(AbstractCoordinator.java:193) at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.poll(ConsumerCoordinator.java:248) at org.apache.kafka.clients.consumer.KafkaConsumer.pollOnce(KafkaConsumer.java:1013) at org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:979) at kafka.consumer.NewShinyConsumer.<init>(BaseConsumer.scala:67) at kafka.tools.ConsoleConsumer$.run(ConsoleConsumer.scala:69) at kafka.tools.ConsoleConsumer$.main(ConsoleConsumer.scala:50) at kafka.tools.ConsoleConsumer.main(ConsoleConsumer.scala) {code} Interestingly, even the old consumer fails (and similarly, mirror maker with the old consumer also fails). The setup seems to be fine, with debug logs I can see multiple "Got ping response" messages, but suddenly I receive a SocketTimeoutException: {code:java} [2016-10-27 17:25:32,549] DEBUG Got ping response for sessionid: 0x557fcb463450fb0 after 1ms (org.apache.zookeeper.ClientCnxn) ... [2016-10-27 17:25:34,691] INFO Reconnect due to error: (kafka.consumer.SimpleConsumer) java.net.SocketTimeoutException at sun.nio.ch.SocketAdaptor$SocketInputStream.read(SocketAdaptor.java:211) at sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:103) at java.nio.channels.Channels$ReadableByteChannelImpl.read(Channels.java:385) at org.apache.kafka.common.network.NetworkReceive.readFromReadableChannel(NetworkReceive.java:81) at kafka.network.BlockingChannel.readCompletely(BlockingChannel.scala:129) at kafka.network.BlockingChannel.receive(BlockingChannel.scala:120) at kafka.consumer.SimpleConsumer.liftedTree1$1(SimpleConsumer.scala:86) at kafka.consumer.SimpleConsumer.kafka$consumer$SimpleConsumer$$sendRequest(SimpleConsumer.scala:83) at kafka.consumer.SimpleConsumer$$anonfun$fetch$1$$anonfun$apply$mcV$sp$1.apply$mcV$sp(SimpleConsumer.scala:132) at kafka.consumer.SimpleConsumer$$anonfun$fetch$1$$anonfun$apply$mcV$sp$1.apply(SimpleConsumer.scala:132) at kafka.consumer.SimpleConsumer$$anonfun$fetch$1$$anonfun$apply$mcV$sp$1.apply(SimpleConsumer.scala:132) at kafka.metrics.KafkaTimer.time(KafkaTimer.scala:33) at kafka.consumer.SimpleConsumer$$anonfun$fetch$1.apply$mcV$sp(SimpleConsumer.scala:131) at kafka.consumer.SimpleConsumer$$anonfun$fetch$1.apply(SimpleConsumer.scala:131) at kafka.consumer.SimpleConsumer$$anonfun$fetch$1.apply(SimpleConsumer.scala:131) at kafka.metrics.KafkaTimer.time(KafkaTimer.scala:33) at kafka.consumer.SimpleConsumer.fetch(SimpleConsumer.scala:130) at kafka.consumer.ConsumerFetcherThread.fetch(ConsumerFetcherThread.scala:109) at kafka.consumer.ConsumerFetcherThread.fetch(ConsumerFetcherThread.scala:29) at kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:118) at kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:103) at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63) {code} In the same system (but well isolated) I also have kafka 0.9, and the consumer works fine with the server. I mean, if I run: {code:java} /path/to/kafka0.9/kafka-console-consumer <myparams> {code} works, and with the same params: {code:java} /path/to/kafka0.10.1/kafka-console-consumer <myparams> {code} fails as above. > Can't mirror from Kafka 0.9 to Kafka 0.10.1 > ------------------------------------------- > > Key: KAFKA-4350 > URL: https://issues.apache.org/jira/browse/KAFKA-4350 > Project: Kafka > Issue Type: Bug > Reporter: Emanuele Cesena > > I'm running 2 clusters: K9 with Kafka 0.9 and K10 with Kafka 0.10.1. > In K10, I've set up mirror maker to clone a topic from K9 to K10. > Mirror maker immediately fails while starting, any suggestion? Following > error message and configs. > Error message: > {{ > [2016-10-26 23:54:01,663] FATAL [mirrormaker-thread-0] Mirror maker thread > failure due to (kafka.tools.MirrorMaker$MirrorMakerThread) > org.apache.kafka.common.protocol.types.SchemaException: Error reading field > 'cluster_id': Error reading string of length 418, only 43 bytes available > at org.apache.kafka.common.protocol.types.Schema.read(Schema.java:73) > at > org.apache.kafka.clients.NetworkClient.parseResponse(NetworkClient.java:380) > at > org.apache.kafka.clients.NetworkClient.handleCompletedReceives(NetworkClient.java:449) > at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:269) > at > org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:232) > at > org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:180) > at > org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureCoordinatorReady(AbstractCoordinator.java:193) > at > org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.poll(ConsumerCoordinator.java:248) > at > org.apache.kafka.clients.consumer.KafkaConsumer.pollOnce(KafkaConsumer.java:1013) > at > org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:979) > at > kafka.tools.MirrorMaker$MirrorMakerNewConsumer.receive(MirrorMaker.scala:582) > at > kafka.tools.MirrorMaker$MirrorMakerThread.run(MirrorMaker.scala:431) > [2016-10-26 23:54:01,679] FATAL [mirrormaker-thread-0] Mirror maker thread > exited abnormally, stopping the whole mirror maker. > (kafka.tools.MirrorMaker$MirrorMakerThread) > }} > Consumer: > {{ > group.id=mirrormaker001 > client.id=mirrormaker001 > bootstrap.servers=...K9... > security.protocol=PLAINTEXT > auto.offset.reset=earliest > }} > (note that I first run without client.id, then tried adding a client.id > because -- same error in both cases) > Producer: > {{ > bootstrap.servers=...K10... > security.protocol=PLAINTEXT > }} -- This message was sent by Atlassian JIRA (v6.3.4#6332)