You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "Viktor Somogyi-Vass (Jira)" <ji...@apache.org> on 2020/06/02 07:08:00 UTC
[jira] [Commented] (KAFKA-7538) Improve locking model used to
update ISRs and HW
[ https://issues.apache.org/jira/browse/KAFKA-7538?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17123415#comment-17123415 ]
Viktor Somogyi-Vass commented on KAFKA-7538:
--------------------------------------------
[~rsivaram] has subcase 1 and 3 been fixed since resolving this jira or is it still pending?
> Improve locking model used to update ISRs and HW
> ------------------------------------------------
>
> Key: KAFKA-7538
> URL: https://issues.apache.org/jira/browse/KAFKA-7538
> Project: Kafka
> Issue Type: Improvement
> Components: core
> Affects Versions: 2.1.0
> Reporter: Rajini Sivaram
> Assignee: Rajini Sivaram
> Priority: Major
> Fix For: 2.5.0
>
>
> We currently use a ReadWriteLock in Partition to update ISRs and high water mark for the partition. This can result in severe lock contention if there are multiple producers writing a large amount of data into a single partition.
> The current locking model is:
> # read lock while appending to log on every Produce request on the request handler thread
> # write lock on leader change, updating ISRs etc. on request handler or scheduler thread
> # write lock on every replica fetch request to check if ISRs need to be updated and to update HW and ISR on the request handler thread
> 2) is infrequent, but 1) and 3) may be frequent and can result in lock contention. If there are lots of produce requests to a partition from multiple processes, on the leader broker we may see:
> # one slow log append locks up one request thread for that produce while holding onto the read lock
> # (replicationFactor-1) request threads can be blocked waiting for write lock to process replica fetch request
> # potentially several other request threads processing Produce may be queued up to acquire read lock because of the waiting writers.
> In a thread dump with this issue, we noticed several request threads blocked waiting for write, possibly to due to replication fetch retries.
>
> Possible fixes:
> # Process `Partition#maybeExpandIsr` on a single scheduler thread similar to `Partition#maybeShrinkIsr` so that only a single thread is blocked on the write lock. But this will delay updating ISRs and HW.
> # Change locking in `Partition#maybeExpandIsr` so that only read lock is acquired to check if ISR needs updating and write lock is acquired only to update ISRs. Also use a different lock for updating HW (perhaps just the Partition object lock) so that typical replica fetch requests complete without acquiring Partition write lock on the request handler thread.
> I will submit a PR for 2) , but other suggestions to fix this are welcome.
>
--
This message was sent by Atlassian Jira
(v8.3.4#803005)