[
https://issues.apache.org/jira/browse/KAFKA-13959?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17552106#comment-17552106
]
Luke Chen commented on KAFKA-13959:
-----------------------------------
[~dengziming] [~jagsancio] , I did some investigation today, and here's my
finding:
# broker heartbeat to active controller won't fetch any data or increase the
offset. broker just sends the current offset and some broker info to the
controller. So, even if we have small interval of heartbeat, it still won't
help.
# So, when will the broker offset increased? It only happened in broker
metadataListener. the metadataListener is listening to raftClient. And
raftClient is polling metadata from active controller.
# About when the highwatermark will be updated in active controller: Whenever
there's record append to the active controller log, it won't update the
highwatermark, until there are voters fetch records from active controller and
also update the highwatermark. ex: current active controller is in
highwatermark 9, and a record append to active controller log to offset 10,
it'll wait, until voters send fetch request to active controller to update
highwatermark, and then, commit the offset 10 record, update the new
highwatermark to 10, to make sure the record is replicated to a majority of the
voters.
# So, that explains what we saw in the issue:
## active controller send no-op message to metadata topic, active controller
append into log, but don't update highwatermark (still 9)
## broker raftClient fetch records from active controller,
## active controller return the records to offset 9, and then update the
highwatermark to 10
## broker metaListener will operate the records
## broker send heartbeat to active controller with offest 9
## since offset 9 is < active controller highwatermark 10
## keep trying, and in the meantime, no-op message sent again, and back to
step 1
> Controller should unfence Broker with busy metadata log
> -------------------------------------------------------
>
> Key: KAFKA-13959
> URL: https://issues.apache.org/jira/browse/KAFKA-13959
> Project: Kafka
> Issue Type: Bug
> Components: kraft
> Affects Versions: 3.3.0
> Reporter: Jose Armando Garcia Sancio
> Priority: Blocker
>
> https://issues.apache.org/jira/browse/KAFKA-13955 showed that it is possible
> for the controller to not unfence a broker if the committed offset keeps
> increasing.
>
> One solution to this problem is to require the broker to only catch up to the
> last committed offset when they last sent the heartbeat. For example:
> # Broker sends a heartbeat with current offset of {{{}Y{}}}. The last commit
> offset is {{{}X{}}}. The controller remember this last commit offset, call it
> {{X'}}
> # Broker sends another heartbeat with current offset of {{{}Z{}}}. Unfence
> the broker if {{Z >= X}} or {{{}Z >= X'{}}}.
>
> This change should also set the default for MetadataMaxIdleIntervalMs back to
> 500.
--
This message was sent by Atlassian Jira
(v8.20.7#820007)