Jorge Esteban Quilcate Otoya created KAFKA-16229:
----------------------------------------------------

             Summary: Slow expiration of Producer IDs leading to high CPU usage
                 Key: KAFKA-16229
                 URL: https://issues.apache.org/jira/browse/KAFKA-16229
             Project: Kafka
          Issue Type: Bug
            Reporter: Jorge Esteban Quilcate Otoya
            Assignee: Jorge Esteban Quilcate Otoya


Expiration of ProducerIds is implemented with a slow removal of map keys:

```
        producers.keySet().removeAll(keys);
```
 
Unnecessarily going through all producer ids and then throw all expired keys to 
be removed.
This leads to exponential time on worst case when most/all keys need to be 
removed:

```
Benchmark                                        (numProducerIds)  Mode  Cnt    
       Score            Error  Units
ProducerStateManagerBench.testDeleteExpiringIds               100  avgt    3    
    9164.043 ±      10647.877  ns/op
ProducerStateManagerBench.testDeleteExpiringIds              1000  avgt    3    
  341561.093 ±      20283.211  ns/op
ProducerStateManagerBench.testDeleteExpiringIds             10000  avgt    3    
44957983.550 ±    9389011.290  ns/op
ProducerStateManagerBench.testDeleteExpiringIds            100000  avgt    3  
5683374164.167 ± 1446242131.466  ns/op
```

A simple fix is to use map#remove(key) instead, leading to a more linear growth:

```
Benchmark                                        (numProducerIds)  Mode  Cnt    
    Score         Error  Units
ProducerStateManagerBench.testDeleteExpiringIds               100  avgt    3    
 5779.056 ±     651.389  ns/op
ProducerStateManagerBench.testDeleteExpiringIds              1000  avgt    3    
61430.530 ±   21875.644  ns/op
ProducerStateManagerBench.testDeleteExpiringIds             10000  avgt    3   
643887.031 ±  600475.302  ns/op
ProducerStateManagerBench.testDeleteExpiringIds            100000  avgt    3  
7741689.539 ± 3218317.079  ns/op
```



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to