Uploaded image for project: 'Kafka'
  1. Kafka
  2. KAFKA-3147

Memory records is not writable in MirrorMaker

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 0.9.0.0
    • 0.10.0.0
    • None
    • None

    Description

      Hi,
      We are running a 3 node cluster (kafka version 0.9) and Node 0 also has a few mirror makers running.
      When we do a rolling restart of the cluster, the mirror maker shuts down with the following errors.

      [2016-01-11 20:16:00,348] WARN Got error produce response with correlation id 12491674 on topic-partition test-99, retrying (2147483646 attempts left). Error: NOT_LEADER_FOR_PARTITION (org.apache.kafka.clients.producer.internals.Sender)
      [2016-01-11 20:16:00,853] FATAL [mirrormaker-thread-0] Mirror maker thread failure due to (kafka.tools.MirrorMaker$MirrorMakerThread)
      java.lang.IllegalStateException: Memory records is not writable
      at org.apache.kafka.common.record.MemoryRecords.append(MemoryRecords.java:93)
      at org.apache.kafka.clients.producer.internals.RecordBatch.tryAppend(RecordBatch.java:69)
      at org.apache.kafka.clients.producer.internals.RecordAccumulator.append(RecordAccumulator.java:168)
      at org.apache.kafka.clients.producer.KafkaProducer.send(KafkaProducer.java:435)
      at kafka.tools.MirrorMaker$MirrorMakerProducer.send(MirrorMaker.scala:593)
      at kafka.tools.MirrorMaker$MirrorMakerThread$$anonfun$run$3.apply(MirrorMaker.scala:398)
      at kafka.tools.MirrorMaker$MirrorMakerThread$$anonfun$run$3.apply(MirrorMaker.scala:398)
      at scala.collection.Iterator$class.foreach(Iterator.scala:742)
      at scala.collection.AbstractIterator.foreach(Iterator.scala:1194)
      at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
      at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
      at kafka.tools.MirrorMaker$MirrorMakerThread.run(MirrorMaker.scala:398)
      [2016-01-11 20:16:01,072] WARN Got error produce response with correlation id 12491679 on topic-partition test-75, retrying (2147483646 attempts left). Error: NETWORK_EXCEPTION (org.apache.kafka.clients.producer.internals.Sender)
      [2016-01-11 20:16:01,073] WARN Got error produce response with correlation id 12491679 on topic-partition test-93, retrying (2147483646 attempts left). Error: NETWORK_EXCEPTION (org.apache.kafka.clients.producer.internals.Sender)
      [2016-01-11 20:16:01,073] WARN Got error produce response with correlation id 12491679 on topic-partition test-24, retrying (2147483646 attempts left). Error: NETWORK_EXCEPTION (org.apache.kafka.clients.producer.internals.Sender)

      [2016-01-11 20:16:20,479] FATAL [mirrormaker-thread-0] Mirror maker thread exited abnormally, stopping the whole mirror maker. (kafka.tools.MirrorMaker$MirrorMakerThread)

      Curious if the NOT_LEADER_FOR_PARTITION is because of a potential bug hinted at in the thread , http://mail-archives.apache.org/mod_mbox/kafka-users/201505.mbox/%3CCAJS3ho_u8s1Xou_kudNfjAMyPJtMrjLW10QVkNGn2YQkdan0+A@mail.gmail.com%3E

      And I think the mirror maker shuts down because of the "abort.on.send.failure" which is set to true in our case.

      Attachments

        Activity

          People

            mgharat Mayuresh Gharat
            megnarasimhan Meghana Narasimhan
            Votes:
            0 Vote for this issue
            Watchers:
            12 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: