[
https://issues.apache.org/jira/browse/KAFKA-9087?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17575835#comment-17575835
]
Markus Werner edited comment on KAFKA-9087 at 8/5/22 1:30 PM:
--------------------------------------------------------------
I was able to reproduce the issue with Kafka 3.2 deployed via strimzi 0.30 in
Kubernetes:
# Create a second volume of type volume
# Run `kafka-reassign-partitions.sh` moving partition from one volume to the
other inside the replicas
# The migration was stuck, so I deleted the broker
Here are the Kafka broker logs after a few rounds of reverting the reassignment
and trying it again. It says "Deleted log" on the target, and indeed I cannot
find any data in file system of the broker itself, but somewhere it seems to
store data that causes this OffsetMisMatch.
{code:java}
2022-08-05 15:13:14
2022-08-05 13:13:14,245 TRACE [Broker id=2] Handling LeaderAndIsr request
correlationId 86 from controller 1 epoch 30 starting the become-follower
transition for partition my-topic-0 with leader 0 (state.change.logger)
[control-plane-kafka-request-handler-0]
2022-08-05 15:13:14
2022-08-05 13:13:14,246 INFO [Broker id=2] Follower my-topic-0 starts at leader
epoch 103 from offset 59173844 with high watermark 59173844. Previous leader
epoch was 102. (state.change.logger) [control-plane-kafka-request-handler-0]
2022-08-05 15:13:14
2022-08-05 13:13:14,246 INFO [ReplicaFetcherManager on broker 2] Removed
fetcher for partitions HashSet(my-topic-0) (kafka.server.ReplicaFetcherManager)
[control-plane-kafka-request-handler-0]
2022-08-05 15:13:14
2022-08-05 13:13:14,247 INFO [ReplicaFetcherManager on broker 2] Added fetcher
to broker 0 for partitions Map(my-topic-0 ->
InitialFetchState(Some(CPOi5CRMSxWoihVANzwwjQ),BrokerEndPoint(id=0,
host=strimzi-kafka-0.strimzi-kafka-brokers.dev-kafka.svc:9091),103,59173844))
(kafka.server.ReplicaFetcherManager) [control-plane-kafka-request-handler-0]
2022-08-05 15:13:14
2022-08-05 13:13:14,247 TRACE [Broker id=2] Completed LeaderAndIsr request
correlationId 86 from controller 1 epoch 30 for the become-follower transition
for partition my-topic-0 with leader 0 (state.change.logger)
[control-plane-kafka-request-handler-0]
2022-08-05 15:13:14
2022-08-05 13:13:14,247 INFO The cleaning for partition my-topic-0 is aborted
and paused (kafka.log.LogManager) [control-plane-kafka-request-handler-0]
2022-08-05 15:13:14
2022-08-05 13:13:14,250 INFO [ReplicaAlterLogDirsManager on broker 2] Added log
dir fetcher for partitions with initial offsets HashMap(my-topic-0 ->
InitialFetchState(Some(CPOi5CRMSxWoihVANzwwjQ),BrokerEndPoint(id=2,
host=localhost:-1),103,59173844)) (kafka.server.ReplicaAlterLogDirsManager)
[control-plane-kafka-request-handler-0]
2022-08-05 15:13:14
2022-08-05 13:13:14,250 INFO [ReplicaAlterLogDirsThread-1]: Truncating
partition my-topic-0 with TruncationState(offset=59173844, completed=true) due
to local high watermark 59173844 (kafka.server.ReplicaAlterLogDirsThread)
[ReplicaAlterLogDirsThread-1]
2022-08-05 15:13:14
2022-08-05 13:13:14,250 INFO [UnifiedLog partition=my-topic-0,
dir=/var/lib/kafka/data-2/kafka-log2] Truncating to 59173844 has no effect as
the largest offset in the log is -1 (kafka.log.UnifiedLog)
[ReplicaAlterLogDirsThread-1]
2022-08-05 15:13:14
2022-08-05 13:13:14,250 INFO [ReplicaAlterLogDirsThread-1]: Beginning/resuming
copy of partition my-topic-0 from offset 59173844. Including this partition,
there are 1 remaining partitions to copy by this thread.
(kafka.server.ReplicaAlterLogDirsThread) [ReplicaAlterLogDirsThread-1]
2022-08-05 15:13:14
2022-08-05 13:13:14,250 ERROR [ReplicaAlterLogDirsThread-1]: Unexpected error
occurred while processing data for partition my-topic-0 at offset 59173844
(kafka.server.ReplicaAlterLogDirsThread) [ReplicaAlterLogDirsThread-1]
2022-08-05 15:13:14
java.lang.IllegalStateException: Offset mismatch for the future replica
my-topic-0: fetched offset = 59173844, log end offset = 0.
2022-08-05 15:13:14
2022-08-05 13:13:14,251 WARN [ReplicaAlterLogDirsThread-1]: Partition
my-topic-0 marked as failed (kafka.server.ReplicaAlterLogDirsThread)
[ReplicaAlterLogDirsThread-1]
2022-08-05 15:13:14
2022-08-05 13:13:14,251 TRACE [Broker id=2] Cached leader info
UpdateMetadataPartitionState(topicName='my-topic', partitionIndex=0,
controllerEpoch=30, leader=0, leaderEpoch=103, isr=[1, 0, 2], zkVersion=190,
replicas=[1, 2, 0], offlineReplicas=[]) for partition my-topic-0 in response to
UpdateMetadata request sent by controller 1 epoch 30 with correlation id 87
(state.change.logger) [control-plane-kafka-request-handler-0]
2022-08-05 15:13:14
2022-08-05 13:13:14,840 TRACE [Broker id=2] Cached leader info
UpdateMetadataPartitionState(topicName='my-topic', partitionIndex=0,
controllerEpoch=30, leader=0, leaderEpoch=103, isr=[1, 0, 2], zkVersion=190,
replicas=[1, 2, 0], offlineReplicas=[]) for partition my-topic-0 in response to
UpdateMetadata request sent by controller 1 epoch 30 with correlation id 88
(state.change.logger) [control-plane-kafka-request-handler-0]
2022-08-05 15:13:15
2022-08-05 13:13:14,906 INFO [ReplicaAlterLogDirsManager on broker 2] Removed
fetcher for partitions Set(my-topic-0)
(kafka.server.ReplicaAlterLogDirsManager) [data-plane-kafka-request-handler-0]
2022-08-05 15:13:15
2022-08-05 13:13:15,031 INFO Log for partition my-topic-0 is renamed to
/var/lib/kafka/data-2/kafka-log2/my-topic-0.fdf28328606944d8b6c37ca113561f79-delete
and is scheduled for deletion (kafka.log.LogManager)
[data-plane-kafka-request-handler-0]
2022-08-05 15:13:15
2022-08-05 13:13:15,031 INFO [Partition my-topic-0 broker=2] Current log
directory /var/lib/kafka/data-1/kafka-log2 is same as requested log dir
/var/lib/kafka/data-1/kafka-log2. Skipping future replica creation.
(kafka.cluster.Partition) [data-plane-kafka-request-handler-0]
2022-08-05 15:14:15
2022-08-05 13:14:15,032 INFO [LocalLog partition=my-topic-0,
dir=/var/lib/kafka/data-2/kafka-log2] Deleting segments as the log has been
deleted: LogSegment(baseOffset=0, size=0, lastModifiedTime=1659704344817,
largestRecordTimestamp=None) (kafka.log.LocalLog) [kafka-scheduler-6]
2022-08-05 15:14:15
2022-08-05 13:14:15,033 INFO [LocalLog partition=my-topic-0,
dir=/var/lib/kafka/data-2/kafka-log2] Deleting segment files
LogSegment(baseOffset=0, size=0, lastModifiedTime=1659704344817,
largestRecordTimestamp=None) (kafka.log.LocalLog$) [kafka-scheduler-6]
2022-08-05 15:14:15
2022-08-05 13:14:15,033 INFO Deleted log
/var/lib/kafka/data-2/kafka-log2/my-topic-0.fdf28328606944d8b6c37ca113561f79-delete/00000000000000000000.log.deleted.
(kafka.log.LogSegment) [kafka-scheduler-6]
2022-08-05 15:14:15
2022-08-05 13:14:15,033 INFO Deleted offset index
/var/lib/kafka/data-2/kafka-log2/my-topic-0.fdf28328606944d8b6c37ca113561f79-delete/00000000000000000000.index.deleted.
(kafka.log.LogSegment) [kafka-scheduler-6]
2022-08-05 15:14:15
2022-08-05 13:14:15,033 INFO Deleted time index
/var/lib/kafka/data-2/kafka-log2/my-topic-0.fdf28328606944d8b6c37ca113561f79-delete/00000000000000000000.timeindex.deleted.
(kafka.log.LogSegment) [kafka-scheduler-6]
2022-08-05 15:14:15
2022-08-05 13:14:15,034 INFO Deleted log for partition my-topic-0 in
/var/lib/kafka/data-2/kafka-log2/my-topic-0.fdf28328606944d8b6c37ca113561f79-delete.
(kafka.log.LogManager) [kafka-scheduler-6]
{code}
was (Author: JIRAUSER294009):
I was able to reproduce the issue with Kafka 3.2 deployed via strimzi 0.30 in
Kubernetes:
# Create a second volume of type volume
# Run `kafka-reassign-partitions.sh` moving partition from one volume to the
other inside the replicas
# The migration was stuck, so I deleted the broker
{code:java}
2022-08-05 12:59:31,404 ERROR [ReplicaAlterLogDirsThread-1]: Unexpected error
occurred while processing data for partition my-topic-0 at offset 59173818
(kafka.server.ReplicaAlterLogDirsThread) [ReplicaAlterLogDirsThread-1]
java.lang.IllegalStateException: Offset mismatch for the future replica
my-topic-0: fetched offset = 59173818, log end offset = 0.{code}
> ReplicaAlterLogDirs stuck and restart fails with
> java.lang.IllegalStateException: Offset mismatch for the future replica
> ------------------------------------------------------------------------------------------------------------------------
>
> Key: KAFKA-9087
> URL: https://issues.apache.org/jira/browse/KAFKA-9087
> Project: Kafka
> Issue Type: Bug
> Components: core
> Affects Versions: 2.2.0
> Reporter: Gregory Koshelev
> Priority: Major
>
> I've started multiple replica movements between log directories and some
> partitions were stuck. After the restart of the broker I've got exception in
> server.log:
> {noformat}
> [2019-06-11 17:58:46,304] ERROR [ReplicaAlterLogDirsThread-1]: Error due to
> (kafka.server.ReplicaAlterLogDirsThread)
> org.apache.kafka.common.KafkaException: Error processing data for partition
> metrics_timers-35 offset 4224887
> at
> kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$7(AbstractFetcherThread.scala:342)
> at scala.Option.foreach(Option.scala:274)
> at
> kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6(AbstractFetcherThread.scala:300)
> at
> kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6$adapted(AbstractFetcherThread.scala:299)
> at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
> at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
> at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
> at
> kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$5(AbstractFetcherThread.scala:299)
> at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
> at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:251)
> at
> kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:299)
> at
> kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3(AbstractFetcherThread.scala:132)
> at
> kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3$adapted(AbstractFetcherThread.scala:131)
> at scala.Option.foreach(Option.scala:274)
> at
> kafka.server.AbstractFetcherThread.maybeFetch(AbstractFetcherThread.scala:131)
> at kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:113)
> at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:82)
> Caused by: java.lang.IllegalStateException: Offset mismatch for the future
> replica metrics_timers-35: fetched offset = 4224887, log end offset = 0.
> at
> kafka.server.ReplicaAlterLogDirsThread.processPartitionData(ReplicaAlterLogDirsThread.scala:107)
> at
> kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$7(AbstractFetcherThread.scala:311)
> ... 16 more
> [2019-06-11 17:58:46,305] INFO [ReplicaAlterLogDirsThread-1]: Stopped
> (kafka.server.ReplicaAlterLogDirsThread)
> {noformat}
> Also, ReplicaAlterLogDirsThread has been stopped. Further restarts do not fix
> the problem. To fix it I've stopped the broker and remove all the stuck
> future partitions.
> Detailed log below
> {noformat}
> [2019-06-11 12:09:52,833] INFO [Log partition=metrics_timers-35,
> dir=/storage2/kafka/data] Truncating to 4224887 has no effect as the largest
> offset in the log is 4224886 (kafka.log.Log)
> [2019-06-11 12:21:34,979] INFO [Log partition=metrics_timers-35,
> dir=/storage2/kafka/data] Loading producer state till offset 4224887 with
> message format version 2 (kafka.log.Log)
> [2019-06-11 12:21:34,980] INFO [ProducerStateManager
> partition=metrics_timers-35] Loading producer state from snapshot file
> '/storage2/kafka/data/metrics_timers-35/00000000000004224887.snapshot'
> (kafka.log.ProducerStateManager)
> [2019-06-11 12:21:34,980] INFO [Log partition=metrics_timers-35,
> dir=/storage2/kafka/data] Completed load of log with 1 segments, log start
> offset 4120720 and log end offset 4224887 in 70 ms (kafka.log.Log)
> [2019-06-11 12:21:45,307] INFO Replica loaded for partition metrics_timers-35
> with initial high watermark 0 (kafka.cluster.Replica)
> [2019-06-11 12:21:45,307] INFO Replica loaded for partition metrics_timers-35
> with initial high watermark 0 (kafka.cluster.Replica)
> [2019-06-11 12:21:45,307] INFO Replica loaded for partition metrics_timers-35
> with initial high watermark 4224887 (kafka.cluster.Replica)
> [2019-06-11 12:21:47,090] INFO [Log partition=metrics_timers-35,
> dir=/storage2/kafka/data] Truncating to 4224887 has no effect as the largest
> offset in the log is 4224886 (kafka.log.Log)
> [2019-06-11 12:30:04,757] INFO [ReplicaFetcher replicaId=1, leaderId=2,
> fetcherId=0] Retrying leaderEpoch request for partition metrics_timers-35 as
> the leader reported an error: UNKNOWN_LEADER_EPOCH
> (kafka.server.ReplicaFetcherThread)
> [2019-06-11 12:30:06,157] INFO [ReplicaFetcher replicaId=1, leaderId=2,
> fetcherId=0] Retrying leaderEpoch request for partition metrics_timers-35 as
> the leader reported an error: UNKNOWN_LEADER_EPOCH
> (kafka.server.ReplicaFetcherThread)
> [2019-06-11 12:30:07,238] INFO [ReplicaFetcher replicaId=1, leaderId=2,
> fetcherId=0] Retrying leaderEpoch request for partition metrics_timers-35 as
> the leader reported an error: UNKNOWN_LEADER_EPOCH
> (kafka.server.ReplicaFetcherThread)
> [2019-06-11 12:30:08,251] INFO [Log partition=metrics_timers-35,
> dir=/storage2/kafka/data] Truncating to 4224887 has no effect as the largest
> offset in the log is 4224886 (kafka.log.Log)
> {noformat}
> I've started replica movement at this moment.
> {noformat}
> [2019-06-11 12:47:32,502] INFO [Log partition=metrics_timers-35,
> dir=/storage5/kafka/data] Loading producer state till offset 0 with message
> format version 2 (kafka.log.Log)
> [2019-06-11 12:47:32,502] INFO [Log partition=metrics_timers-35,
> dir=/storage5/kafka/data] Completed load of log with 1 segments, log start
> offset 0 and log end offset 0 in 1 ms (kafka.log.Log)
> [2019-06-11 12:47:32,502] INFO Created log for partition metrics_timers-35 in
> /storage5/kafka/data with properties {compression.type -> producer,
> message.format.version -> 2.2-IV1, file.delete.delay.ms -> 60000,
> max.message.bytes -> 1000012, min.compaction.lag.ms -> 0,
> message.timestamp.type -> CreateTime, message.downconversion.enable -> true,
> min.insync.replicas -> 2, segment.jitter.ms -> 0, preallocate -> false,
> min.cleanable.dirty.ratio -> 0.5, index.interval.bytes -> 4096,
> unclean.leader.election.enable -> false, retention.bytes -> 137438953472,
> delete.retention.ms -> 86400000, cleanup.policy -> [delete], flush.ms ->
> 9223372036854775807, segment.ms -> 604800000, segment.bytes -> 1073741824,
> retention.ms -> 259200000, message.timestamp.difference.max.ms ->
> 9223372036854775807, segment.index.bytes -> 10485760, flush.messages ->
> 9223372036854775807}. (kafka.log.LogManager)
> [2019-06-11 12:47:32,502] INFO [Partition metrics_timers-35 broker=1] No
> checkpointed highwatermark is found for partition metrics_timers-35
> (kafka.cluster.Partition)
> [2019-06-11 12:47:32,502] INFO Replica loaded for partition metrics_timers-35
> with initial high watermark 0 (kafka.cluster.Replica)
> [2019-06-11 12:47:33,083] INFO [ReplicaAlterLogDirsManager on broker 1] Added
> fetcher to broker BrokerEndPoint(id=1, host=localhost:-1) for partitions
> Map(metrics_timers-35 -> (offset=0, leaderEpoch=27))
> (kafka.server.ReplicaAlterLogDirsManager)
> [2019-06-11 12:47:33,309] INFO [ReplicaAlterLogDirsThread-1]: Truncating
> partition metrics_timers-35 to local high watermark 0
> (kafka.server.ReplicaAlterLogDirsThread)
> [2019-06-11 12:47:33,309] INFO [Log partition=metrics_timers-35,
> dir=/storage5/kafka/data] Truncating to 0 has no effect as the largest offset
> in the log is -1 (kafka.log.Log)
> [2019-06-11 14:02:25,937] INFO [ReplicaAlterLogDirsThread-1]: Partition
> metrics_timers-35 has an older epoch (27) than the current leader. Will await
> the new LeaderAndIsr state before resuming fetching.
> (kafka.server.ReplicaAlterLogDirsThread)
> [2019-06-11 14:02:25,952] INFO [ReplicaFetcherManager on broker 1] Removed
> fetcher for partitions Set(metrics_timer-35, …
> [2019-06-11 14:02:25,980] INFO [ReplicaFetcherManager on broker 1] Added
> fetcher to broker BrokerEndPoint(id=2, host=vostok09:9092) for partitions
> Map(metrics_timers-35 -> (offset=4224887, leaderEpoch=28),…
> [2019-06-11 14:02:25,998] INFO [ReplicaAlterLogDirsThread-1]: Shutting down
> (kafka.server.ReplicaAlterLogDirsThread)
> [2019-06-11 14:02:25,998] INFO [ReplicaAlterLogDirsThread-1]: Stopped
> (kafka.server.ReplicaAlterLogDirsThread)
> [2019-06-11 14:02:25,998] INFO [ReplicaAlterLogDirsThread-1]: Shutdown
> completed (kafka.server.ReplicaAlterLogDirsThread)
> [2019-06-11 14:02:26,803] INFO [ReplicaFetcher replicaId=1, leaderId=2,
> fetcherId=0] Retrying leaderEpoch request for partition metrics_timers-35 as
> the leader reported an error: UNKNOWN_LEADER_EPOCH
> (kafka.server.ReplicaFetcherThread)
> [2019-06-11 14:02:43,406] INFO [Log partition=metrics_timers-35,
> dir=/storage2/kafka/data] Truncating to 4224887 has no effect as the largest
> offset in the log is 4224886 (kafka.log.Log)
> {noformat}
> The broker has been restarted at 17:35
> {noformat}
> [2019-06-11 17:35:32,176] INFO [ReplicaFetcherManager on broker 1] Removed
> fetcher for partitions Set(metrics_timers-35)
> (kafka.server.ReplicaFetcherManager)
> [2019-06-11 17:37:48,265] WARN [Log partition=metrics_timers-35,
> dir=/storage2/kafka/data] Found a corrupted index file corresponding to log
> file /storage2/kafka/data/metrics_timers-35/00000000000004120720.log due to
> Corrupt time index found, time index file
> (/storage2/kafka/data/metrics_timers-35/00000000000004120720.timeindex) has
> non-zero size but the last timestamp is 0 which is less than the first
> timestamp 1560154787249}, recovering segment and rebuilding index files...
> (kafka.log.Log)
> [2019-06-11 17:37:48,265] INFO [Log partition=metrics_timers-35,
> dir=/storage2/kafka/data] Loading producer state till offset 4120720 with
> message format version 2 (kafka.log.Log)
> [2019-06-11 17:37:48,266] INFO [ProducerStateManager
> partition=metrics_timers-35] Writing producer snapshot at offset 4120720
> (kafka.log.ProducerStateManager)
> [2019-06-11 17:37:48,522] INFO [ProducerStateManager
> partition=metrics_timers-35] Writing producer snapshot at offset 4224887
> (kafka.log.ProducerStateManager)
> [2019-06-11 17:37:48,524] INFO [Log partition=metrics_timers-35,
> dir=/storage2/kafka/data] Loading producer state till offset 4224887 with
> message format version 2 (kafka.log.Log)
> [2019-06-11 17:37:48,525] INFO [ProducerStateManager
> partition=metrics_timers-35] Loading producer state from snapshot file
> '/storage2/kafka/data/metrics_timers-35/00000000000004224887.snapshot'
> (kafka.log.ProducerStateManager)
> [2019-06-11 17:37:48,525] INFO [Log partition=metrics_timers-35,
> dir=/storage2/kafka/data] Completed load of log with 1 segments, log start
> offset 4120720 and log end offset 4224887 in 298 ms (kafka.log.Log)
> [2019-06-11 17:38:01,954] INFO Replica loaded for partition metrics_timers-35
> with initial high watermark 0 (kafka.cluster.Replica)
> [2019-06-11 17:38:01,954] INFO Replica loaded for partition metrics_timers-35
> with initial high watermark 0 (kafka.cluster.Replica)
> [2019-06-11 17:38:01,955] INFO Replica loaded for partition metrics_timers-35
> with initial high watermark 4224887 (kafka.cluster.Replica)
> [2019-06-11 17:38:02,582] INFO [Partition metrics_timers-35 broker=1] No
> checkpointed highwatermark is found for partition metrics_timers-35
> (kafka.cluster.Partition)
> [2019-06-11 17:38:02,582] INFO Replica loaded for partition metrics_timers-35
> with initial high watermark 0 (kafka.cluster.Replica)
> [2019-06-11 17:38:02,588] INFO [ReplicaAlterLogDirsManager on broker 1] Added
> fetcher to broker BrokerEndPoint(id=1, host=localhost:-1) for partitions
> Map(metrics_timers-35 -> (offset=4224887, leaderEpoch=29), traces_cloud-4 ->
> (offset=4381208630, leaderEpoch=27), metrics_histograms-11 -> (offset=0,
> leaderEpoch=28), metrics_histograms-25 -> (offset=0, leaderEpoch=34),
> metrics_histograms-39 -> (offset=0, leaderEpoch=29), metrics_counters-15 ->
> (offset=0, leaderEpoch=34), metrics_final-21 -> (offset=1852,
> leaderEpoch=28), metrics_final-7 -> (offset=1926, leaderEpoch=29),
> metrics_any-17 -> (offset=0, leaderEpoch=28), metrics_timers-14 -> (offset=0,
> leaderEpoch=29), metrics_counters-1 -> (offset=0, leaderEpoch=28))
> (kafka.server.ReplicaAlterLogDirsManager)
> [2019-06-11 17:38:02,596] INFO [ReplicaAlterLogDirsThread-1]: Truncating
> partition metrics_timers-35 to local high watermark 4224887
> (kafka.server.ReplicaAlterLogDirsThread)
> [2019-06-11 17:38:02,596] INFO [Log partition=metrics_timers-35,
> dir=/storage5/kafka/data] Truncating to 4224887 has no effect as the largest
> offset in the log is -1 (kafka.log.Log)
> [2019-06-11 17:38:06,005] INFO [Log partition=metrics_timers-35,
> dir=/storage2/kafka/data] Truncating to 4224887 has no effect as the largest
> offset in the log is 4224886 (kafka.log.Log)
> [2019-06-11 17:38:06,080] INFO [Log partition=metrics_timers-35,
> dir=/storage5/kafka/data] Truncating to 4224887 has no effect as the largest
> offset in the log is -1 (kafka.log.Log)
> [2019-06-11 17:38:06,080] INFO [ReplicaAlterLogDirsThread-1]: Truncating
> partition metrics_timers-35 to local high watermark 4224887
> (kafka.server.ReplicaAlterLogDirsThread)
> [2019-06-11 17:38:06,080] INFO [Log partition=metrics_timers-35,
> dir=/storage5/kafka/data] Truncating to 4224887 has no effect as the largest
> offset in the log is -1 (kafka.log.Log)
> [2019-06-11 17:58:46,304] ERROR [ReplicaAlterLogDirsThread-1]: Error due to
> (kafka.server.ReplicaAlterLogDirsThread)
> org.apache.kafka.common.KafkaException: Error processing data for partition
> metrics_timers-35 offset 4224887
> at
> kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$7(AbstractFetcherThread.scala:342)
> at scala.Option.foreach(Option.scala:274)
> at
> kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6(AbstractFetcherThread.scala:300)
> at
> kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6$adapted(AbstractFetcherThread.scala:299)
> at
> scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
> at
> scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
> at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
> at
> kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$5(AbstractFetcherThread.scala:299)
> at
> scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
> at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:251)
> at
> kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:299)
> at
> kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3(AbstractFetcherThread.scala:132)
> at
> kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3$adapted(AbstractFetcherThread.scala:131)
> at scala.Option.foreach(Option.scala:274)
> at
> kafka.server.AbstractFetcherThread.maybeFetch(AbstractFetcherThread.scala:131)
> at
> kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:113)
> at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:82)
> Caused by: java.lang.IllegalStateException: Offset mismatch for the future
> replica metrics_timers-35: fetched offset = 4224887, log end offset = 0.
> at
> kafka.server.ReplicaAlterLogDirsThread.processPartitionData(ReplicaAlterLogDirsThread.scala:107)
> at
> kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$7(AbstractFetcherThread.scala:311)
> ... 16 more
> [2019-06-11 17:58:46,305] INFO [ReplicaAlterLogDirsThread-1]: Stopped
> (kafka.server.ReplicaAlterLogDirsThread)
> {noformat}
> The broker has been restarted at 18:21
> {noformat}
> [2019-06-11 18:21:26,422] INFO [Log partition=metrics_timers-35,
> dir=/storage5/kafka/data] Loading producer state till offset 0 with message
> format version 2 (kafka.log.Log)
> [2019-06-11 18:21:26,423] INFO [Log partition=metrics_timers-35,
> dir=/storage5/kafka/data] Completed load of log with 1 segments, log start
> offset 0 and log end offset 0 in 2 ms (kafka.log.Log)
> [2019-06-11 18:23:21,300] WARN [Log partition=metrics_timers-35,
> dir=/storage2/kafka/data] Found a corrupted index file corresponding to log
> file /storage2/kafka/data/metrics_timers-35/00000000000004120720.log due to
> Corrupt time index found, time index file
> (/storage2/kafka/data/metrics_timers-35/00000000000004120720.timeindex) has
> non-zero size but the last timestamp is 0 which is less than the first
> timestamp 1560154787249}, recovering segment and rebuilding index files...
> (kafka.log.Log)
> [2019-06-11 18:23:21,300] INFO [Log partition=metrics_timers-35,
> dir=/storage2/kafka/data] Loading producer state till offset 4120720 with
> message format version 2 (kafka.log.Log)
> [2019-06-11 18:23:21,301] INFO [ProducerStateManager
> partition=metrics_timers-35] Writing producer snapshot at offset 4120720
> (kafka.log.ProducerStateManager)
> [2019-06-11 18:23:21,559] INFO [ProducerStateManager
> partition=metrics_timers-35] Writing producer snapshot at offset 4224887
> (kafka.log.ProducerStateManager)
> [2019-06-11 18:23:21,561] INFO [Log partition=metrics_timers-35,
> dir=/storage2/kafka/data] Loading producer state till offset 4224887 with
> message format version 2 (kafka.log.Log)
> [2019-06-11 18:23:21,562] INFO [ProducerStateManager
> partition=metrics_timers-35] Loading producer state from snapshot file
> '/storage2/kafka/data/metrics_timers-35/00000000000004224887.snapshot'
> (kafka.log.ProducerStateManager)
> [2019-06-11 18:23:21,563] INFO [Log partition=metrics_timers-35,
> dir=/storage2/kafka/data] Completed load of log with 1 segments, log start
> offset 4120720 and log end offset 4224887 in 353 ms (kafka.log.Log)
> [2019-06-11 18:23:35,928] INFO Replica loaded for partition metrics_timers-35
> with initial high watermark 0 (kafka.cluster.Replica)
> [2019-06-11 18:23:35,928] INFO Replica loaded for partition metrics_timers-35
> with initial high watermark 0 (kafka.cluster.Replica)
> [2019-06-11 18:23:35,929] INFO Replica loaded for partition metrics_timers-35
> with initial high watermark 4224887 (kafka.cluster.Replica)
> [2019-06-11 18:23:36,516] INFO [Partition metrics_timers-35 broker=1] No
> checkpointed highwatermark is found for partition metrics_timers-35
> (kafka.cluster.Partition)
> [2019-06-11 18:23:36,516] INFO Replica loaded for partition metrics_timers-35
> with initial high watermark 0 (kafka.cluster.Replica)
> [2019-06-11 18:23:36,521] INFO [ReplicaAlterLogDirsManager on broker 1] Added
> fetcher to broker BrokerEndPoint(id=1, host=localhost:-1) for partitions
> Map(metrics_timers-35 -> (offset=4224887, leaderEpoch=30),
> metrics_histograms-11 -> (offset=0, leaderEpoch=29), metrics_histograms-25 ->
> (offset=0, leaderEpoch=36), metrics_histograms-39 -> (offset=0,
> leaderEpoch=30), metrics_counters-15 -> (offset=0, leaderEpoch=36),
> metrics_final-21 -> (offset=1861, leaderEpoch=29), metrics_final-7 ->
> (offset=1931, leaderEpoch=30), metrics_any-17 -> (offset=0, leaderEpoch=29),
> metrics_timers-14 -> (offset=0, leaderEpoch=30), metrics_counters-1 ->
> (offset=0, leaderEpoch=29)) (kafka.server.ReplicaAlterLogDirsManager)
> [2019-06-11 18:23:36,522] INFO [ReplicaAlterLogDirsThread-1]: Truncating
> partition metrics_timers-35 to local high watermark 4224887
> (kafka.server.ReplicaAlterLogDirsThread)
> [2019-06-11 18:23:36,523] INFO [Log partition=metrics_timers-35,
> dir=/storage5/kafka/data] Truncating to 4224887 has no effect as the largest
> offset in the log is -1 (kafka.log.Log)
> [2019-06-11 18:23:36,523] INFO [ReplicaAlterLogDirsThread-1]: Truncating
> partition metrics_final-7 to local high watermark 1931
> (kafka.server.ReplicaAlterLogDirsThread)
> [2019-06-11 18:23:36,563] ERROR [ReplicaAlterLogDirsThread-1]: Error due to
> (kafka.server.ReplicaAlterLogDirsThread)
> org.apache.kafka.common.KafkaException: Error processing data for partition
> metrics_timers-35 offset 4224887
> at
> kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$7(AbstractFetcherThread.scala:342)
> at scala.Option.foreach(Option.scala:274)
> at
> kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6(AbstractFetcherThread.scala:300)
> at
> kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$6$adapted(AbstractFetcherThread.scala:299)
> at
> scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
> at
> scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
> at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
> at
> kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$5(AbstractFetcherThread.scala:299)
> at
> scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
> at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:251)
> at
> kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:299)
> at
> kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3(AbstractFetcherThread.scala:132)
> at
> kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3$adapted(AbstractFetcherThread.scala:131)
> at scala.Option.foreach(Option.scala:274)
> at
> kafka.server.AbstractFetcherThread.maybeFetch(AbstractFetcherThread.scala:131)
> at
> kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:113)
> at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:82)
> Caused by: java.lang.IllegalStateException: Offset mismatch for the future
> replica metrics_timers-35: fetched offset = 4224887, log end offset = 0.
> at
> kafka.server.ReplicaAlterLogDirsThread.processPartitionData(ReplicaAlterLogDirsThread.scala:107)
> at
> kafka.server.AbstractFetcherThread.$anonfun$processFetchRequest$7(AbstractFetcherThread.scala:311)
> ... 16 more
> [2019-06-11 18:23:36,564] INFO [GroupMetadataManager brokerId=1] Scheduling
> unloading of offsets and group metadata from __consumer_offsets-19
> (kafka.coordinator.group.GroupMetadataManager)
> [2019-06-11 18:23:36,572] INFO [ReplicaAlterLogDirsThread-1]: Stopped
> (kafka.server.ReplicaAlterLogDirsThread)
> {noformat}
--
This message was sent by Atlassian Jira
(v8.20.10#820010)