[
https://issues.apache.org/jira/browse/KAFKA-6292?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16274136#comment-16274136
]
Terence Yi edited comment on KAFKA-6292 at 12/1/17 9:09 AM:
------------------------------------------------------------
The time EOFException occurs, no log printed with "Truncating log...), but i do
got this log in other log files at differ time with different partition
[2017-11-27 09:44:48,099] INFO Truncating
DDI.DISPATCHER.MESSAGE_FORWARD_d694b9fa-d99a-4f4d-9062-b75e73b466a0-1 to 0 has
no effect as the largest offset in the log is -1. (kafka.log.Log)
was (Author: terence.yi):
The time EOFException occurs, no log printed with "Truncating log...), but i do
got this log in other log files at differ time.
[2017-11-27 09:44:48,099] INFO Truncating
DDI.DISPATCHER.MESSAGE_FORWARD_d694b9fa-d99a-4f4d-9062-b75e73b466a0-1 to 0 has
no effect as the largest offset in the log is -1. (kafka.log.Log)
> ReplicaManager not respect the rolled log
> -----------------------------------------
>
> Key: KAFKA-6292
> URL: https://issues.apache.org/jira/browse/KAFKA-6292
> Project: Kafka
> Issue Type: Bug
> Components: log
> Affects Versions: 0.11.0.0
> Environment: OS:Red Hat Enterprise Linux Server release 7.3 (Maipo)
> Kafka: kafka_2.12-0.11.0.0
> JDK: jdk1.8.0_121
> Reporter: Terence Yi
> Labels: reliability
> Fix For: 1.1.0
>
>
> After the log segment rolled successfully will followed with a failed fetch
> with the rolled partition.
> Consumer constantly poll from the Kafka cluster, and consume the message in
> order and end with a manually commit for each record.
> Here is the log in consumer side:
> 2017-11-28 20:44:26.560 WARN XXXXX
> [HOSTNAME][device-message-subscriber-pool-3-thread-1]
> org.apache.kafka.clients.consumer.internals.Fetcher {} Unknown error fetching
> data for topic-partition
> DDI.DISPATCHER.MESSAGE_FORWARD_d694b9fa-d99a-4f4d-9062-b75e73b466a0-3
> Observe below ERROR log in server.log
> [2017-11-27 12:16:24,182] INFO Rolled new log segment for
> 'DDI.DISPATCHER.P_TVIN.W_SL.P_appx.P_ul.P_pos-3' in 1 ms. (kafka.log.Log)
> [2017-11-27 12:16:35,555] INFO Rolled new log segment for
> 'DDI.DISPATCHER.MESSAGE_FORWARD_d694b9fa-d99a-4f4d-9062-b75e73b466a0-3' in 1
> ms. (kafka.log.Log)
> [2017-11-27 12:16:35,569] ERROR [Replica Manager on Broker 4]: Error
> processing fetch operation on partition
> DDI.DISPATCHER.MESSAGE_FORWARD_d694b9fa-d99a-4f4d-9062-b75e73b466a0-3, offset
> 12813782 (kafka.server.ReplicaManager)
> org.apache.kafka.common.KafkaException: java.io.EOFException: Failed to read
> `log header` from file channel `sun.nio.ch.FileChannelImpl@1a493fba`.
> Expected to read 17 bytes, but reached end of file after reading 0 bytes.
> Started read from position 2147483635.
> at
> org.apache.kafka.common.record.RecordBatchIterator.makeNext(RecordBatchIterator.java:40)
> at
> org.apache.kafka.common.record.RecordBatchIterator.makeNext(RecordBatchIterator.java:24)
> at
> org.apache.kafka.common.utils.AbstractIterator.maybeComputeNext(AbstractIterator.java:79)
> at
> org.apache.kafka.common.utils.AbstractIterator.hasNext(AbstractIterator.java:45)
> at
> org.apache.kafka.common.record.FileRecords.searchForOffsetWithSize(FileRecords.java:279)
> at kafka.log.LogSegment.translateOffset(LogSegment.scala:176)
> at kafka.log.LogSegment.read(LogSegment.scala:228)
> at kafka.log.Log.read(Log.scala:938)
> at kafka.server.ReplicaManager.read$1(ReplicaManager.scala:719)
> at
> kafka.server.ReplicaManager.$anonfun$readFromLocalLog$6(ReplicaManager.scala:780)
> at
> scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:59)
> at
> scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:52)
> at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
> at
> kafka.server.ReplicaManager.readFromLocalLog(ReplicaManager.scala:779)
> at kafka.server.ReplicaManager.fetchMessages(ReplicaManager.scala:617)
> at kafka.server.KafkaApis.handleFetchRequest(KafkaApis.scala:615)
> at kafka.server.KafkaApis.handle(KafkaApis.scala:98)
> at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:66)
> at java.lang.Thread.run(Thread.java:745)
> Caused by: java.io.EOFException: Failed to read `log header` from file
> channel `sun.nio.ch.FileChannelImpl@1a493fba`. Expected to read 17 bytes, but
> reached end of file after reading 0 bytes. Started read from position
> 2147483635.
> at org.apache.kafka.common.utils.Utils.readFullyOrFail(Utils.java:751)
> at
> org.apache.kafka.common.record.FileLogInputStream.nextBatch(FileLogInputStream.java:66)
> at
> org.apache.kafka.common.record.FileLogInputStream.nextBatch(FileLogInputStream.java:40)
> at
> org.apache.kafka.common.record.RecordBatchIterator.makeNext(RecordBatchIterator.java:35)
> ... 18 more
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)