Wenbing Shen created KAFKA-12157:
------------------------------------
Summary: test Upgrade 2.7.0 from 2.0.0 occur a question
Key: KAFKA-12157
URL: https://issues.apache.org/jira/browse/KAFKA-12157
Project: Kafka
Issue Type: Bug
Components: log
Affects Versions: 2.7.0
Reporter: Wenbing Shen
Attachments: 1001server.log, 1001serverlog.png, 1003server.log,
1003serverlog.png, 1003statechange.log
I was in a test environment, rolling upgrade from version 2.0.0 to version
2.7.0, and encountered the following problems. When the rolling upgrade
progressed to the second round, I stopped the first broker(1001) in the second
round and the following error occurred. When an agent processes the client
producer request, the starting offset of the leader epoch of the partition
leader suddenly becomes 0, and then continues to process write requests for the
same partition, and an error log will appear.All partition leaders with 1001
replicas are transferred to the 1003 node, and these partitions on the 1003
node will generate this error if they receive production requests.When I
restart 1001, the 1001 broker will report the following error:
[2021-01-06 16:46:55,955] ERROR (ReplicaFetcherThread-8-1003
kafka.server.ReplicaFetcherThread 76) [ReplicaFetcher replicaId=1001,
leaderId=1003, fetcherId=8] Unexpected error occurred while processing data for
partition test-perf1-9 at offset 9666953
I use the following command to make a production request:
nohup /home/kafka/software/kafka/bin/kafka-producer-perf-test.sh --num-records
1000000000000 --record-size 1000 --throughput 30000 --producer-props
bootstrap.servers=hdp1:9092,hdp2:9092,hdp3:9092 acks=1 --topic test-perf1 >
1pro.log 2>&1 &
I tried to reproduce the problem again, but after three attempts, it did not
reappear. I am curious how this problem occurred and why the 1003 broker resets
startOffset to 0 of leaderEpoch 4 when the offset is assigned by broker in
Log.append function.
broker 1003: server.log
[2021-01-06 16:37:59,492] WARN (data-plane-kafka-request-handler-131
kafka.server.epoch.LeaderEpochFileCache 70) [LeaderEpochCache test-perf1-9] New
epoch en
try EpochEntry(epoch=4, startOffset=0) caused truncation of conflicting entries
ListBuffer(EpochEntry(epoch=4, startOffset=9667122), EpochEntry(epoch=3, star
tOffset=9195729), EpochEntry(epoch=2, startOffset=8348201)). Cache now contains
0 entries.
[2021-01-06 16:37:59,493] WARN (data-plane-kafka-request-handler-131
kafka.server.epoch.LeaderEpochFileCache 70) [LeaderEpochCache test-perf1-8] New
epoch en
try EpochEntry(epoch=3, startOffset=0) caused truncation of conflicting entries
ListBuffer(EpochEntry(epoch=3, startOffset=9667478), EpochEntry(epoch=2, star
tOffset=9196127), EpochEntry(epoch=1, startOffset=8342787)). Cache now contains
0 entries.
[2021-01-06 16:37:59,495] WARN (data-plane-kafka-request-handler-131
kafka.server.epoch.LeaderEpochFileCache 70) [LeaderEpochCache test-perf1-2] New
epoch en
try EpochEntry(epoch=3, startOffset=0) caused truncation of conflicting entries
ListBuffer(EpochEntry(epoch=3, startOffset=9667478), EpochEntry(epoch=2, star
tOffset=9196127), EpochEntry(epoch=1, startOffset=8336727)). Cache now contains
0 entries.
[2021-01-06 16:37:59,498] ERROR (data-plane-kafka-request-handler-142
kafka.server.ReplicaManager 76) [ReplicaManager broker=1003] Error processing
append op
eration on partition test-perf1-9
java.lang.IllegalArgumentException: Received invalid partition leader epoch
entry EpochEntry(epoch=4, startOffset=-3)
at
kafka.server.epoch.LeaderEpochFileCache.assign(LeaderEpochFileCache.scala:67)
at
kafka.server.epoch.LeaderEpochFileCache.assign(LeaderEpochFileCache.scala:59)
at kafka.log.Log.maybeAssignEpochStartOffset(Log.scala:1268)
at kafka.log.Log.$anonfun$append$6(Log.scala:1181)
at kafka.log.Log$$Lambda$935/184936331.accept(Unknown Source)
at java.lang.Iterable.forEach(Iterable.java:75)
at kafka.log.Log.$anonfun$append$2(Log.scala:1179)
at kafka.log.Log.append(Log.scala:2387)
at kafka.log.Log.appendAsLeader(Log.scala:1050)
at
kafka.cluster.Partition.$anonfun$appendRecordsToLeader$1(Partition.scala:1079)
at kafka.cluster.Partition.appendRecordsToLeader(Partition.scala:1067)
at
kafka.server.ReplicaManager.$anonfun$appendToLocalLog$4(ReplicaManager.scala:953)
at kafka.server.ReplicaManager$$Lambda$1025/1369541490.apply(Unknown Source)
at scala.collection.StrictOptimizedMapOps.map(StrictOptimizedMapOps.scala:28)
at scala.collection.StrictOptimizedMapOps.map$(StrictOptimizedMapOps.scala:27)
at scala.collection.mutable.HashMap.map(HashMap.scala:35)
at kafka.server.ReplicaManager.appendToLocalLog(ReplicaManager.scala:941)
at kafka.server.ReplicaManager.appendRecords(ReplicaManager.scala:621)
at kafka.server.KafkaApis.handleProduceRequest(KafkaApis.scala:625)
broker 1001:server.log
[2021-01-06 16:46:55,955] ERROR (ReplicaFetcherThread-8-1003
kafka.server.ReplicaFetcherThread 76) [ReplicaFetcher replicaId=1001,
leaderId=1003, fetcherId=8] Unexpected error occurred while processing data for
partition test-perf1-9 at offset 9666953
kafka.common.OffsetsOutOfOrderException: Out of order offsets found in append
to test-perf1-9: ArrayBuffer(9666953, 9666954, 9666955, 9666956, 9666957,
9666958, 9666959, 9666960, 9666961, 9666962, 9666963, 9666964, 9666965,
9666966, 9666967, 9666968, 9666969, 9666970, 9666971, 9666972, 9666973,
9666974, 9666975, 9666976, 9666977, 9666978, 9666979, 9666980, 9666981,
9666982, 9666983, 9666984, 9666985, 9666986, 9666987, 9666988, 9666989,
9666990, 9666991, 9666992, 9666993, 9666994, 9666995, 9666996, 9666997,
9666998, 9666999, 9667000, 9667001, 9667002, 9667003, 9667004, 9667005,
9667006, 9667007, 9667008, 9667009, 9667010, 9667011, 9667012, 9667013,
9667014, 9667015, 9667016, 9667017, 9667018, 9667019, 9667020, 9667021,
9667022, 9667023, 9667024, 9667025, 9667026, 9667027, 9667028, 9667029,
9667030, 9667031, 9667032, 9667033, 9667034, 9667035, 9667036, 9667037,
9667038, 9667039, 9667040, 9667041, 9667042, 9667043, 9667044, 9667045,
9667046, 9667047, 9667048, 9667049, 9667050, 9667051, 9667052, 9667053,
9667054, 9667055, 9667056, 9667057, 9667058, 9667059, 9667060, 9667061,
9667062, 9667063, 9667064, 9667065, 9667066, 9667067, 9667068, 9667069,
9667070, 9667071, 9667072, 9667073, 9667074, 9667075, 9667076, 9667077,
9667078...
--
This message was sent by Atlassian Jira
(v8.3.4#803005)