[jira] [Commented] (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=16401313#comment-16401313 ] ASF GitHub Bot commented on KAFKA-3978: --- lindong28 opened a new pull request #4722: MINOR KAFKA-3978 followup URL: https://github.com/apache/kafka/pull/4722 We should use logStartOffset as HW offset if the current HW offset is out of range. ### Committer Checklist (excluded from commit message) - [ ] Verify design and implementation - [ ] Verify test coverage and CI build status - [ ] Verify documentation (including upgrade notes) This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > 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 >Assignee: Dong Lin >Priority: Critical > Labels: reliability, startup > Fix For: 1.1.0 > > > 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 >
[jira] [Commented] (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=16398124#comment-16398124 ] ASF GitHub Bot commented on KAFKA-3978: --- hachikuji closed pull request #4695: KAFKA-3978; highwatermark should always be positive URL: https://github.com/apache/kafka/pull/4695 This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 3b97671524d..68faf00c079 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -460,7 +460,11 @@ class Partition(val topic: String, }.map(_.logEndOffset) val newHighWatermark = allLogEndOffsets.min(new LogOffsetMetadata.OffsetOrdering) val oldHighWatermark = leaderReplica.highWatermark -if (oldHighWatermark.messageOffset < newHighWatermark.messageOffset || oldHighWatermark.onOlderSegment(newHighWatermark)) { + +// Ensure that the high watermark increases monotonically. We also update the high watermark when the new +// offset metadata is on a newer segment, which occurs whenever the log is rolled to a new segment. +if (oldHighWatermark.messageOffset < newHighWatermark.messageOffset || + (oldHighWatermark.messageOffset == newHighWatermark.messageOffset && oldHighWatermark.onOlderSegment(newHighWatermark))) { leaderReplica.highWatermark = newHighWatermark debug(s"High watermark updated to $newHighWatermark") true diff --git a/core/src/main/scala/kafka/cluster/Replica.scala b/core/src/main/scala/kafka/cluster/Replica.scala index e41e389e22d..030e5b7eb58 100644 --- a/core/src/main/scala/kafka/cluster/Replica.scala +++ b/core/src/main/scala/kafka/cluster/Replica.scala @@ -138,6 +138,9 @@ class Replica(val brokerId: Int, def highWatermark_=(newHighWatermark: LogOffsetMetadata) { if (isLocal) { + if (newHighWatermark.messageOffset < 0) +throw new IllegalArgumentException("High watermark offset should be non-negative") + highWatermarkMetadata = newHighWatermark log.foreach(_.onHighWatermarkIncremented(newHighWatermark.messageOffset)) trace(s"Setting high watermark for replica $brokerId partition $topicPartition to [$newHighWatermark]") @@ -165,9 +168,16 @@ class Replica(val brokerId: Int, s"non-local replica $brokerId")) } - def convertHWToLocalOffsetMetadata() = { + /* + * Convert hw to local offset metadata by reading the log at the hw offset. + * If the hw offset is out of range, return the first offset of the first log segment as the offset metadata. + */ + def convertHWToLocalOffsetMetadata() { if (isLocal) { - highWatermarkMetadata = log.get.convertToOffsetMetadata(highWatermarkMetadata.messageOffset) + highWatermarkMetadata = log.get.convertToOffsetMetadata(highWatermarkMetadata.messageOffset).getOrElse { +val firstOffset = log.get.logSegments.head.baseOffset +new LogOffsetMetadata(firstOffset, firstOffset, 0) + } } else { throw new KafkaException(s"Should not construct complete high watermark on partition $topicPartition's non-local replica $brokerId") } diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 257dd8f9ba4..f0050f54aef 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -1126,14 +1126,14 @@ class Log(@volatile var dir: File, /** * Given a message offset, find its corresponding offset metadata in the log. - * If the message offset is out of range, return unknown offset metadata + * If the message offset is out of range, return None to the caller. */ - def convertToOffsetMetadata(offset: Long): LogOffsetMetadata = { + def convertToOffsetMetadata(offset: Long): Option[LogOffsetMetadata] = { try { val fetchDataInfo = readUncommitted(offset, 1) - fetchDataInfo.fetchOffsetMetadata + Some(fetchDataInfo.fetchOffsetMetadata) } catch { - case _: OffsetOutOfRangeException => LogOffsetMetadata.UnknownOffsetMetadata + case _: OffsetOutOfRangeException => None } } diff --git a/core/src/test/scala/unit/kafka/admin/ReassignPartitionsClusterTest.scala b/core/src/test/scala/unit/kafka/admin/ReassignPartitionsClusterTest.scala index 2a24a37f151..0c41519d211 100644 --- a/core/src/test/scala/unit/kafka/admin/ReassignPartitionsClusterTest.scala +++ b/core/src/test/scala/unit/kafka/admin/ReassignPartitionsClusterTest.scala @@ -76,6 +76,32 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging {
[jira] [Commented] (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=16394107#comment-16394107 ] Dong Lin commented on KAFKA-3978: - Just realized that the problem I have been investigating is different from this Jira ticket. But they may be related in the sensor that both issues are due to the use of hw -1. As shown in the previous comment, a new leader may instantiate leader replica with hw=-1. And this negative hw can also be passed to follower. This causes problem either when we truncate log to the hw, or when we try to increase log start offset to a positive offset. > 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 >Assignee: Dong Lin >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
[jira] [Commented] (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=16394103#comment-16394103 ] Dong Lin commented on KAFKA-3978: - Finally confirmed the root cause and is able to reproduce this using a test: 1) Partition P1 has replica set size 1. Broker A is the leader. The segment is empty and log start offset is 100 2) User executes partition reassignment to change replica set from \{A} to \{B, C} 3) Broker B starts ReplicaFetcherThread, which triggers handleOffsetOutOfRange(), truncates the log fully and start at offset 100. At this moment its high watermark is still 0 (or -1). Sam for broker C. 4) Broker B sends FetchRequest to A at offset 100, broker A immediately adds broker B to ISR set, and controller moves leadership to broker B. 5) Broker B handles LeaderAndIsrRequest to become leader. It calls `leaderReplica.convertHWToLocalOffsetMetadata()` to initialize its HW. Since its HW was smaller than logStartOffset=100, now its HW will be overridden to LogOffsetMetadata.UnknownOffsetMetadata, i.e. -1. 6) Broker C handles LeaderAndIsrRequest to fetch from broker B. Broker C updates its HW to the FetchRequest's HW, i.e. -1. Then broker C calls replica.maybeIncrementLogStartOffset(leaderLogStartOffset) where leaderLogStartOffset=100. This cause exception because leaderLogStartOffset > HW. > 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 >Assignee: Dong Lin >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
[jira] [Commented] (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=16391863#comment-16391863 ] Dong Lin commented on KAFKA-3978: - [~junrao] [~hachikuji] [~ijuma] I have a good theory now. I will submit a patch to fix it. - Broker A is leader for partition P1 and broker B is follower for partition P1. - Broker C receives LeaderAndIsrRequest to become follower for P1. In broker C's memory, P1's highWatermarkMetadata = LogOffsetMetadata(hw=0, segmentBaseOffset=-1). Local replica's LEO for P1 is 0. - Broker C's ReplicaFetchRequest sends leader epoch request to broker A and then truncates its local replica's LEO to 100. - Broker C receives LeaderAndIsrRequest to become leader for P1 with ISR=(A,B,C). In broker C's memory, P1's highWatermarkMetadata = LogOffsetMetadata(hw=0, segmentBaseOffset=-1). And in broker C's memory, according to partition.makeLeader(), replica A's logEndOffsetMetadata is initialized to be LogOffsetMetadata.UnknownOffsetMetadata, which has HW= - 1 and segmentBaseOffset = 0. - Broker C receives FetchRequest from broker B. In Partition.maybeIncrementLeaderHW(), new highWaterMark will be derived with min(logEndOffset metadata of all replicas), which will be LogOffsetMetadata(HW=-1, segmentBaseOffset=0) because A's logEndOffset metadata is smallest. Because new hw's segmentBaseOffset > old hw's segmentBaseOffset, the high watermark is updated to be LogOffsetMetadata(hw=-1, segmentBaseOffset=100L). - Now we have a partition whose HW is negative, which cause problem for broker B when it fetches from broker C. > 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) >
[jira] [Commented] (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=16391698#comment-16391698 ] Dong Lin commented on KAFKA-3978: - I also encountered this issue and I have been looking at this all day yesterday. Still no clue how leader can sender FetchResponse with error=None and hw=-1 > 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 >
[jira] [Commented] (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=16387125#comment-16387125 ] Jun Rao commented on KAFKA-3978: Agreed. For 1) we could add an assertion. For 2), if we detect that we are truncating to -1, we could log a warn and just truncate the whole local log. > 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 >
[jira] [Commented] (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=16387076#comment-16387076 ] Jason Gustafson commented on KAFKA-3978: [~ijuma] [~junrao] We're still not sure the cause of this, but it should be easy to either 1) add an assertion when writing the checkpoint that offsets are not negative, and 2) detect the problem at startup and just use the log start offset. Since this problem has persisted for some time, shall we go ahead and do either of these? > 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 >
[jira] [Commented] (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=16096887#comment-16096887 ] Dominic Evans commented on KAFKA-3978: -- [~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 >