[
https://issues.apache.org/jira/browse/KAFKA-5337?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Grant Neale updated KAFKA-5337:
-------------------------------
Description:
Existing partition assignment strategies (Range Assignor and
RoubdRobinAssignor) do not account for the current consumer group lag on each
partition. This can result in sub-optimal assignments when the distribution of
lags for a given topic and consumer group is skewed.
The LagBasedAssignor operates on a per-topic basis, and attempts to assign
partitions such that lag is distributed as evenly across a consumer group.
h4. Algorithm:
For each topic, first obtain the lag on all partitions. Lag on a given
partition is the difference between the end offset and the last offset
committed by the consumer group. If no offsets have been committed for a
partition we determine the lag based on the code auto.offset.reset property. If
auto.offset.reset=latest, we assume a lag of 0. If auto.offset.reset=earliest
(or any other value) we assume lag equal to the total number of message
currently available in that partition.
Next, create a map storing the current total lag of all partitions assigned to
each member of the consumer group. Partitions are assigned in decreasing order
of lag, with each partition assigned to the consumer with least total number of
assigned partitions, breaking ties by assigning to the consumer with the least
total currently assigned lag.
Assigning partitions evenly across consumers (by partition count) ensures that
the assignment is reasonably balanced (by partition count) when all partitions
have a current lag of 0 or if the distribution of lags is heavily skewed. It
also gives the consumer group the best possible chance of remaining balanced if
the assignment is retained for a long period (assuming throughput is consistent
across members of the consumer group).
was:
Existing partition assignment strategies (Range Assignor and
RoubdRobinAssignor) do not account for the current consumer group lag on each
partition. This can result in sub-optimal assignments when the distribution of
lags for a given topic and consumer group is skewed.
The LagBasedAssignor operates on a per-topic basis, and attempts to assign
partitions such that lag is distributed as evenly across a consumer group.
**Algorithm:**
For each topic, first obtain the lag on all partitions. Lag on a given
partition is the difference between the end offset and the last offset
committed by the consumer group. If no offsets have been committed for a
partition we determine the lag based on the code auto.offset.reset property. If
auto.offset.reset=latest, we assume a lag of 0. If auto.offset.reset=earliest
(or any other value) we assume lag equal to the total number of message
currently available in that partition.
Next, create a map storing the current total lag of all partitions assigned to
each member of the consumer group. Partitions are assigned in decreasing order
of lag, with each partition assigned to the consumer with least total number of
assigned partitions, breaking ties by assigning to the consumer with the least
total currently assigned lag.
Assigning partitions evenly across consumers (by partition count) ensures that
the assignment is reasonably balanced (by partition count) when all partitions
have a current lag of 0 or if the distribution of lags is heavily skewed. It
also gives the consumer group the best possible chance of remaining balanced if
the assignment is retained for a long period (assuming throughput is consistent
across members of the consumer group).
> Partition assignment strategy that distributes lag evenly across consumers in
> each group
> ----------------------------------------------------------------------------------------
>
> Key: KAFKA-5337
> URL: https://issues.apache.org/jira/browse/KAFKA-5337
> Project: Kafka
> Issue Type: New Feature
> Components: consumer
> Affects Versions: 0.10.2.1
> Reporter: Grant Neale
> Priority: Minor
>
> Existing partition assignment strategies (Range Assignor and
> RoubdRobinAssignor) do not account for the current consumer group lag on each
> partition. This can result in sub-optimal assignments when the distribution
> of lags for a given topic and consumer group is skewed.
> The LagBasedAssignor operates on a per-topic basis, and attempts to assign
> partitions such that lag is distributed as evenly across a consumer group.
> h4. Algorithm:
> For each topic, first obtain the lag on all partitions. Lag on a given
> partition is the difference between the end offset and the last offset
> committed by the consumer group. If no offsets have been committed for a
> partition we determine the lag based on the code auto.offset.reset property.
> If auto.offset.reset=latest, we assume a lag of 0. If
> auto.offset.reset=earliest (or any other value) we assume lag equal to the
> total number of message currently available in that partition.
> Next, create a map storing the current total lag of all partitions assigned
> to each member of the consumer group. Partitions are assigned in decreasing
> order of lag, with each partition assigned to the consumer with least total
> number of assigned partitions, breaking ties by assigning to the consumer
> with the least total currently assigned lag.
> Assigning partitions evenly across consumers (by partition count) ensures
> that the assignment is reasonably balanced (by partition count) when all
> partitions have a current lag of 0 or if the distribution of lags is heavily
> skewed. It also gives the consumer group the best possible chance of
> remaining balanced if the assignment is retained for a long period (assuming
> throughput is consistent across members of the consumer group).
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)