You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "Dominic Evans (JIRA)" <ji...@apache.org> on 2017/07/21 21:22:00 UTC

[jira] [Comment Edited] (KAFKA-3978) Cannot truncate to a negative offset (-1) exception at broker startup

    [ https://issues.apache.org/jira/browse/KAFKA-3978?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16096887#comment-16096887 ] 

Dominic Evans edited comment on KAFKA-3978 at 7/21/17 9:21 PM:
---------------------------------------------------------------

[~ijuma] we actually hit this issue the other day on 0.10.2 and it seemed to have more serious repercussions than initially suggested here.

Because the truncateTo exception causes kafka.server.ReplicaManager.makeFollowers to exit, having removed any fetchers for the set of partitions passed in, we seem to end up with the broker never following/replicating those partitions again unless the "bad" partition (seemingly with an unknown checkpoint offset) is manually removed from ZK and the broker is restarted. 

I know that KIP-101 reworked this area, so it may no longer be possible under 0.11 onwards. I also wasn't entirely sure how we got into the state and was unable to reproduce it by manually fiddling with replication-offset-checkpoint or otherwise.


was (Author: dnwe):
[~ijuma] we actually hit this issue the other day on 0.10.2 and it seemed to have more serious repercussions than initially suggested here.

Because the truncateTo exception causes kafka.server.ReplicaManager.makeFollowers to exit, having removed any fetchers for the set of partitions passed in, we seem to end up with the broker never following/replicating those partitions again unless the "bad" partition (seemingly with an unknown checkpoint offset) is manually removed from ZK and the broker is restarted. 

> Cannot truncate to a negative offset (-1) exception at broker startup
> ---------------------------------------------------------------------
>
>                 Key: KAFKA-3978
>                 URL: https://issues.apache.org/jira/browse/KAFKA-3978
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 0.10.0.0
>         Environment: 3.13.0-87-generic 
>            Reporter: Juho Mäkinen
>            Priority: Critical
>              Labels: reliability, startup
>
> During broker startup sequence the broker server.log has this exception. Problem persists after multiple restarts and also on another broker in the cluster.
> {code}
> INFO [Socket Server on Broker 1002], Started 1 acceptor threads (kafka.network.SocketServer)
> INFO [Socket Server on Broker 1002], Started 1 acceptor threads (kafka.network.SocketServer)
> INFO [ExpirationReaper-1002], Starting  (kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
> INFO [ExpirationReaper-1002], Starting  (kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
> INFO [ExpirationReaper-1002], Starting  (kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
> INFO [ExpirationReaper-1002], Starting  (kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
> INFO [ExpirationReaper-1002], Starting  (kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
> INFO [ExpirationReaper-1002], Starting  (kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
> INFO [ExpirationReaper-1002], Starting  (kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
> INFO [ExpirationReaper-1002], Starting  (kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
> INFO [GroupCoordinator 1002]: Starting up. (kafka.coordinator.GroupCoordinator)
> INFO [GroupCoordinator 1002]: Starting up. (kafka.coordinator.GroupCoordinator)
> INFO [GroupCoordinator 1002]: Startup complete. (kafka.coordinator.GroupCoordinator)
> INFO [GroupCoordinator 1002]: Startup complete. (kafka.coordinator.GroupCoordinator)
> INFO [Group Metadata Manager on Broker 1002]: Removed 0 expired offsets in 9 milliseconds. (kafka.coordinator.GroupMetadataManager)
> INFO [Group Metadata Manager on Broker 1002]: Removed 0 expired offsets in 9 milliseconds. (kafka.coordinator.GroupMetadataManager)
> INFO [ThrottledRequestReaper-Produce], Starting  (kafka.server.ClientQuotaManager$ThrottledRequestReaper)
> INFO [ThrottledRequestReaper-Produce], Starting  (kafka.server.ClientQuotaManager$ThrottledRequestReaper)
> INFO [ThrottledRequestReaper-Fetch], Starting  (kafka.server.ClientQuotaManager$ThrottledRequestReaper)
> INFO [ThrottledRequestReaper-Fetch], Starting  (kafka.server.ClientQuotaManager$ThrottledRequestReaper)
> INFO Will not load MX4J, mx4j-tools.jar is not in the classpath (kafka.utils.Mx4jLoader$)
> INFO Will not load MX4J, mx4j-tools.jar is not in the classpath (kafka.utils.Mx4jLoader$)
> INFO Creating /brokers/ids/1002 (is it secure? false) (kafka.utils.ZKCheckedEphemeral)
> INFO Creating /brokers/ids/1002 (is it secure? false) (kafka.utils.ZKCheckedEphemeral)
> INFO Result of znode creation is: OK (kafka.utils.ZKCheckedEphemeral)
> INFO Result of znode creation is: OK (kafka.utils.ZKCheckedEphemeral)
> INFO Registered broker 1002 at path /brokers/ids/1002 with addresses: PLAINTEXT -> EndPoint(172.16.2.22,9092,PLAINTEXT) (kafka.utils.ZkUtils)
> INFO Registered broker 1002 at path /brokers/ids/1002 with addresses: PLAINTEXT -> EndPoint(172.16.2.22,9092,PLAINTEXT) (kafka.utils.ZkUtils)
> INFO Kafka version : 0.10.0.0 (org.apache.kafka.common.utils.AppInfoParser)
> INFO Kafka commitId : b8642491e78c5a13 (org.apache.kafka.common.utils.AppInfoParser)
> INFO [Kafka Server 1002], started (kafka.server.KafkaServer)
> INFO [Kafka Server 1002], started (kafka.server.KafkaServer)
> Error when handling request {controller_id=1004,controller_epoch=1,partition_states=[..REALLY LONG OUTPUT SNIPPED AWAY..], live_leaders=[{id=1004,host=172.16.6.187,port=9092},{id=1003,host=172.16.2.21,port=9092}]} (kafka.server.KafkaApis)
> ERROR java.lang.IllegalArgumentException: Cannot truncate to a negative offset (-1).
>         at kafka.log.Log.truncateTo(Log.scala:731)
>         at kafka.log.LogManager$$anonfun$truncateTo$2.apply(LogManager.scala:288)
>         at kafka.log.LogManager$$anonfun$truncateTo$2.apply(LogManager.scala:280)
>         at scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:733)
>         at scala.collection.immutable.HashMap$HashMap1.foreach(HashMap.scala:221)
>         at scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:428)
>         at scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:428)
>         at scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:428)
>         at scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:732)
>         at kafka.log.LogManager.truncateTo(LogManager.scala:280)
>         at kafka.server.ReplicaManager.makeFollowers(ReplicaManager.scala:802)
>         at kafka.server.ReplicaManager.becomeLeaderOrFollower(ReplicaManager.scala:648)
>         at kafka.server.KafkaApis.handleLeaderAndIsrRequest(KafkaApis.scala:144)
>         at kafka.server.KafkaApis.handle(KafkaApis.scala:80)
>         at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:60)
>         at java.lang.Thread.run(Thread.java:745)
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)