[jira] [Comment Edited] (KAFKA-9693) Kafka latency spikes caused by log segment flush on roll

2023-05-26 Thread Ruslan Scherbakov (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-9693?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17726728#comment-17726728
 ] 

Ruslan Scherbakov edited comment on KAFKA-9693 at 5/26/23 8:05 PM:
---

Related pull request: [https://github.com/apache/kafka/pull/13768]

The issue with repeating latency spikes during Kafka log segments rolling still 
reproduced on the latest versions including kafka_2.13-3.4.0.

It was found that flushing Kafka snapshot file during segments rolling blocks 
producer request handling thread for some time:
[https://github.com/apache/kafka/blob/3.4/core/src/main/scala/kafka/log/ProducerStateManager.scala#L452]
{code:java}
  private def writeSnapshot(file: File, entries: mutable.Map[Long, 
ProducerStateEntry]): Unit = {
...
    val fileChannel = FileChannel.open(file.toPath, StandardOpenOption.CREATE, 
StandardOpenOption.WRITE)
    try {
      fileChannel.write(buffer)
      fileChannel.force(true)     <- here
    } finally {
      fileChannel.close()
    }...{code}
More partitions - more cumulative latency effect observed.}}{}}}

Suggested fix offloads flush (fileChannel.force) operation to the background 
thread similar to (but not exactly) how it was done in the UnifiedLog.scala:
{code:java}
  def roll(
   ...
    // Schedule an asynchronous flush of the old segment
    scheduler.schedule("flush-log", () => 
flushUptoOffsetExclusive(newSegment.baseOffset))
  }{code}
The benchmarking using this fix shows significant reduction in repeating 
latency spikes:

*Test config:*

AWS
3 node cluster (i3en.2xlarge)
zulu11.62.17-ca-jdk11.0.18-linux_x64, heap 6G per broker
1 loadgen (m5n.8xlarge) - OpenMessaging benchmark 
([OMB|https://github.com/openmessaging/benchmark])
1 zookeeper (t2.small)
acks=all batchSize=1048510 consumers=4 insyncReplicas=2 lingerMs=1 mlen=1024 
producers=4 rf=3 subscriptions=1 targetRate=200k time=12m topics=1 warmup=1m

*variation 1:*

partitions=10
||metric||kafka_2.13-3.4.0||kafka_2.13-3.4.0 patched||
|endToEnd service_time (ms) p50 max|2.00|2.00|
|endToEnd service_time (ms) p75 max|3.00|2.00|
|endToEnd service_time (ms) p95 max|94.0|3.00|
|endToEnd service_time (ms) p99 max|290|6.00|
|endToEnd service_time (ms) p99.9 max|355|21.0|
|endToEnd service_time (ms) p99.99 max|372|34.0|
|endToEnd service_time (ms) p100 max|374|36.0|
|publish service_time (ms) p50 max|1.70|1.67|
|publish service_time (ms) p75 max|2.23|2.09|
|publish service_time (ms) p95 max|90.7|2.82|
|publish service_time (ms) p99 max|287|4.69|
|publish service_time (ms) p99.9 max|353|19.6|
|publish service_time (ms) p99.99 max|369|31.3|
|publish service_time (ms) p100 max|371|33.5|

 
||kafka||endToEnd chart||
|kafka_2.13-3.4.0|[!https://user-images.githubusercontent.com/6793713/241306935-ec329711-47d4-459f-92d7-06310b770023.png!|https://user-images.githubusercontent.com/6793713/241306935-ec329711-47d4-459f-92d7-06310b770023.png]|
|kafka_2.13-3.4.0 
patched|[!https://user-images.githubusercontent.com/6793713/241307047-e5aeb6a6-d33a-4d57-be80-c916fa1f05be.png!|https://user-images.githubusercontent.com/6793713/241307047-e5aeb6a6-d33a-4d57-be80-c916fa1f05be.png]|

latency score improved up to 10x times in high percentiles ^^^, spikes almost 
invisible

*variation 2:*

partitions=100
||metric||kafka_2.13-3.4.0||kafka_2.13-3.4.0 patched||
|endToEnd service_time (ms) p50 max|91.0|2.00|
|endToEnd service_time (ms) p75 max|358|3.00|
|endToEnd service_time (ms) p95 max|1814|4.00|
|endToEnd service_time (ms) p99 max|2777|21.0|
|endToEnd service_time (ms) p99.9 max|3643|119|
|endToEnd service_time (ms) p99.99 max|3724|141|
|endToEnd service_time (ms) p100 max|3726|143|
|publish service_time (ms) p50 max|77.4|1.92|
|publish service_time (ms) p75 max|352|2.35|
|publish service_time (ms) p95 max|1748|3.80|
|publish service_time (ms) p99 max|2740|18.9|
|publish service_time (ms) p99.9 max|3619|116|
|publish service_time (ms) p99.99 max|3720|139|
|publish service_time (ms) p100 max|3722|141|
|endToEnd service_time| | |

 
||kafka||endToEnd chart||
|kafka_2.13-3.4.0|[!https://user-images.githubusercontent.com/6793713/241307517-dc6e9820-c3b7-4bd0-8dac-bce9f3886d91.png!|https://user-images.githubusercontent.com/6793713/241307517-dc6e9820-c3b7-4bd0-8dac-bce9f3886d91.png]|
|kafka_2.13-3.4.0 
patched|[!https://user-images.githubusercontent.com/6793713/241307546-113b4480-97a4-4dd5-8d5c-f7dc87a3d7a5.png!|https://user-images.githubusercontent.com/6793713/241307546-113b4480-97a4-4dd5-8d5c-f7dc87a3d7a5.png]|

latency score improved up to 25x times in high percentiles ^^^

The fix was done for 3.4 branch - scala version of ProducerStateManager. Trunk 
needs corresponding fix for ProducerStateManager.java.


was (Author: novosibman):
Related pull request: [https://github.com/apache/kafka/pull/13768]

The issue with repeating latency spikes during Kafka log segments rolling still 
reproduced on the latest versions including 

[jira] [Comment Edited] (KAFKA-9693) Kafka latency spikes caused by log segment flush on roll

2023-05-26 Thread Ruslan Scherbakov (Jira)


[ 
https://issues.apache.org/jira/browse/KAFKA-9693?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17726728#comment-17726728
 ] 

Ruslan Scherbakov edited comment on KAFKA-9693 at 5/26/23 8:00 PM:
---

Related pull request: [https://github.com/apache/kafka/pull/13768]

The issue with repeating latency spikes during Kafka log segments rolling still 
reproduced on the latest versions including kafka_2.13-3.4.0.

It was found that flushing Kafka snapshot file during segments rolling blocks 
producer request handling thread for some time:
[https://github.com/apache/kafka/blob/3.4/core/src/main/scala/kafka/log/ProducerStateManager.scala#L452]
{code:java}
  private def writeSnapshot(file: File, entries: mutable.Map[Long, 
ProducerStateEntry]): Unit = {
...
    val fileChannel = FileChannel.open(file.toPath, StandardOpenOption.CREATE, 
StandardOpenOption.WRITE)
    try {
      fileChannel.write(buffer)
      fileChannel.force(true)     <- here
    } finally {
      fileChannel.close()
    }...{code}
More partitions - more cumulative latency effect observed.{{{}{}}}

Suggested fix offloads flush (fileChannel.force) operation to the background 
thread similar to (but not exactly) how it was done in the UnifiedLog.scala:
{code:java}
  def roll(
   ...
    // Schedule an asynchronous flush of the old segment
    scheduler.schedule("flush-log", () => 
flushUptoOffsetExclusive(newSegment.baseOffset))
  }{code}
The benchmarking using this fix shows significant reduction in repeating 
latency spikes:

*Test config:*

AWS
3 node cluster (i3en.2xlarge)
zulu11.62.17-ca-jdk11.0.18-linux_x64, heap 6G per broker
1 loadgen (m5n.8xlarge) - OpenMessaging benchmark 
([OMB|https://github.com/openmessaging/benchmark])
1 zookeeper (t2.small)
acks=all batchSize=1048510 consumers=4 insyncReplicas=2 lingerMs=1 mlen=1024 
producers=4 rf=3 subscriptions=1 targetRate=200k time=12m topics=1 warmup=1m

*variation 1:*

partitions=10
||metric||kafka_2.13-3.4.0||kafka_2.13-3.4.0 patched||
|endToEnd service_time (ms) p50 max|2.00|2.00|
|endToEnd service_time (ms) p75 max|3.00|2.00|
|endToEnd service_time (ms) p95 max|94.0|3.00|
|endToEnd service_time (ms) p99 max|290|6.00|
|endToEnd service_time (ms) p99.9 max|355|21.0|
|endToEnd service_time (ms) p99.99 max|372|34.0|
|endToEnd service_time (ms) p100 max|374|36.0|
|publish service_time (ms) p50 max|1.70|1.67|
|publish service_time (ms) p75 max|2.23|2.09|
|publish service_time (ms) p95 max|90.7|2.82|
|publish service_time (ms) p99 max|287|4.69|
|publish service_time (ms) p99.9 max|353|19.6|
|publish service_time (ms) p99.99 max|369|31.3|
|publish service_time (ms) p100 max|371|33.5|
||kafka||endToEnd chart||
|kafka_2.13-3.4.0|[!https://user-images.githubusercontent.com/6793713/241306935-ec329711-47d4-459f-92d7-06310b770023.png|width=703,height=281!|https://user-images.githubusercontent.com/6793713/241306935-ec329711-47d4-459f-92d7-06310b770023.png]|
|kafka_2.13-3.4.0 
patched|[!https://user-images.githubusercontent.com/6793713/241307047-e5aeb6a6-d33a-4d57-be80-c916fa1f05be.png|width=700,height=280!|https://user-images.githubusercontent.com/6793713/241307047-e5aeb6a6-d33a-4d57-be80-c916fa1f05be.png]|

latency score improved up to 10x times in high percentiles ^^^, spikes almost 
invisible

*variation 2:*

partitions=100
||metric||kafka_2.13-3.4.0||kafka_2.13-3.4.0 patched||
|endToEnd service_time (ms) p50 max|91.0|2.00|
|endToEnd service_time (ms) p75 max|358|3.00|
|endToEnd service_time (ms) p95 max|1814|4.00|
|endToEnd service_time (ms) p99 max|2777|21.0|
|endToEnd service_time (ms) p99.9 max|3643|119|
|endToEnd service_time (ms) p99.99 max|3724|141|
|endToEnd service_time (ms) p100 max|3726|143|
|publish service_time (ms) p50 max|77.4|1.92|
|publish service_time (ms) p75 max|352|2.35|
|publish service_time (ms) p95 max|1748|3.80|
|publish service_time (ms) p99 max|2740|18.9|
|publish service_time (ms) p99.9 max|3619|116|
|publish service_time (ms) p99.99 max|3720|139|
|publish service_time (ms) p100 max|3722|141|
|endToEnd service_time| | |
||kafka||endToEnd chart||
|kafka_2.13-3.4.0|[!https://user-images.githubusercontent.com/6793713/241307517-dc6e9820-c3b7-4bd0-8dac-bce9f3886d91.png|width=645,height=258!|https://user-images.githubusercontent.com/6793713/241307517-dc6e9820-c3b7-4bd0-8dac-bce9f3886d91.png]|
|kafka_2.13-3.4.0 
patched|[!https://user-images.githubusercontent.com/6793713/241307546-113b4480-97a4-4dd5-8d5c-f7dc87a3d7a5.png|width=643,height=257!|https://user-images.githubusercontent.com/6793713/241307546-113b4480-97a4-4dd5-8d5c-f7dc87a3d7a5.png]|

latency score improved up to 25x times in high percentiles ^^^

The fix was done for 3.4 branch - scala version of ProducerStateManager. Trunk 
needs corresponding fix for ProducerStateManager.java.


was (Author: novosibman):
Related pull request: https://github.com/apache/kafka/pull/13768

The issue with repeating latency spikes during Kafka log 

[jira] [Comment Edited] (KAFKA-9693) Kafka latency spikes caused by log segment flush on roll

2020-06-23 Thread Jira


[ 
https://issues.apache.org/jira/browse/KAFKA-9693?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17142857#comment-17142857
 ] 

João Oliveirinha edited comment on KAFKA-9693 at 6/23/20, 11:38 AM:


Has anyone been able to reproduce this and identify the root cause directly? I 
believe that this happens independetly if the log is being rolled or not.

I am having simular problems. The only difference is that this happen on the 
high percentiles (99.999 and 99.99 sometimes) without any log.roll or 
application flush happening.

I still see spikes that normally go to the 200ms independently of the 
throughput pushed by the producer. For these tests I am using a single producer 
and single consumer on the same process. In attachment you could see a plot on 
the end-2-end lantency (producer to consumer).

I am using default configurations and I have been able to push those spikes to 
the 99.999 most of the times by tuning the kernel parameters for page flush 
cache to be done every 1 second instead of the default 30s.

What is strange to me is the 200ms being pretty much consistent and unrelated 
to throuthput. I also tried to increment the max inflight requests as a way to 
push these latencies even more to the higher percentiles but it had an 
unexpected behaviour to me. In the higher throughput  runs the average latency 
increases from 2ms to ~40ms.

 

My configs:

*Producer*:
 linger=0
 batch=16k
 recordSize=2k
 compression=none
 max.in.flight.conn=5

*Setup*:

6 i3.2xlarge with 1.9TB SSD on 3 AZs1 
c5.2xlarge for the injector/consumer

 

i3s are configured with:
echo tsc > /sys/devices/system/clocksource/clocksource0/current_clocksource
sysctl -w vm.dirty_expire_centisecs=100
sysctl -w vm.dirty_writeback_centisecs=100

 

 And SSDs are formatted with XFS and mount with 
defaults,noatime,discard,nobarrier options.

!image-2020-06-23-12-24-58-788.png|width=975,height=520!

 

 


was (Author: joliveirinha):
Has anyone been able to reproduce this and identify the root cause directly? I 
believe that this happens independetly if the log is being rolled or not.

I am having simular problems. The only difference is that this happen on the 
high percentiles (99.999 and 99.99 sometimes) without any log.roll or 
application flush happening.

I still see spikes that normally go to the 200ms independently of the 
throughput pushed by the producer. For these tests I am using a single producer 
and single consumer on the same process. In attachment you could see a plot on 
the end-2-end lantency (producer to consumer).

I am using default configurations and I have been able to push those spikes to 
the 99.999 most of the times by tuning the kernel parameters for page flush 
cache to be done every 1 second instead of the default 30s.

What is strange to me is the 200ms being pretty much consistent and unrelated 
to throuthput. I also tried to increment the max inflight requests as a way to 
push these latencies even more to the higher percentiles but it had an 
unexpected behaviour to me. In the higher throughput  runs the average latency 
increases from 2ms to ~40ms.

 

My configs:

*Producer*:
 linger=0
 batch=16k
 recordSize=2k
 compression=none
 max.in.flight.conn=5

*Setup*:

6 i3.2xlarge with 1.9TB SSD on 3 AZs1 c5.2xlarge for the injector/consumer

 

 

!image-2020-06-23-12-24-58-788.png|width=975,height=520!

 

 

> Kafka latency spikes caused by log segment flush on roll
> 
>
> Key: KAFKA-9693
> URL: https://issues.apache.org/jira/browse/KAFKA-9693
> Project: Kafka
>  Issue Type: Improvement
>  Components: core
> Environment: OS: Amazon Linux 2
> Kafka version: 2.2.1
>Reporter: Paolo Moriello
>Assignee: Paolo Moriello
>Priority: Major
>  Labels: Performance, latency, performance
> Attachments: image-2020-03-10-13-17-34-618.png, 
> image-2020-03-10-14-36-21-807.png, image-2020-03-10-15-00-23-020.png, 
> image-2020-03-10-15-00-54-204.png, image-2020-06-23-12-24-46-548.png, 
> image-2020-06-23-12-24-58-788.png, latency_plot2.png
>
>
> h1. Summary
> When a log segment fills up, Kafka rolls over onto a new active segment and 
> force the flush of the old segment to disk. When this happens, log segment 
> _append_ duration increase causing important latency spikes on producer(s) 
> and replica(s). This ticket aims to highlight the problem and propose a 
> simple mitigation: add a new configuration to enable/disable rolled segment 
> flush.
> h1. 1. Phenomenon
> Response time of produce request (99th ~ 99.9th %ile) repeatedly spikes to 
> ~50x-200x more than usual. For instance, normally 99th %ile is lower than 
> 5ms, but when this issue occurs, it marks 100ms to 200ms. 99.9th and 99.99th 
> %iles even jump to 500-700ms.
> Latency spikes 

[jira] [Comment Edited] (KAFKA-9693) Kafka latency spikes caused by log segment flush on roll

2020-06-23 Thread Jira


[ 
https://issues.apache.org/jira/browse/KAFKA-9693?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17142857#comment-17142857
 ] 

João Oliveirinha edited comment on KAFKA-9693 at 6/23/20, 11:36 AM:


Has anyone been able to reproduce this and identify the root cause directly? I 
believe that this happens independetly if the log is being rolled or not.

I am having simular problems. The only difference is that this happen on the 
high percentiles (99.999 and 99.99 sometimes) without any log.roll or 
application flush happening.

I still see spikes that normally go to the 200ms independently of the 
throughput pushed by the producer. For these tests I am using a single producer 
and single consumer on the same process. In attachment you could see a plot on 
the end-2-end lantency (producer to consumer).

I am using default configurations and I have been able to push those spikes to 
the 99.999 most of the times by tuning the kernel parameters for page flush 
cache to be done every 1 second instead of the default 30s.

What is strange to me is the 200ms being pretty much consistent and unrelated 
to throuthput. I also tried to increment the max inflight requests as a way to 
push these latencies even more to the higher percentiles but it had an 
unexpected behaviour to me. In the higher throughput  runs the average latency 
increases from 2ms to ~40ms.

 

My configs:

*Producer*:
 linger=0
 batch=16k
 recordSize=2k
 compression=none
 max.in.flight.conn=5

*Setup*:

6 i3.2xlarge with 1.9TB SSD on 3 AZs1 c5.2xlarge for the injector/consumer

 

 

!image-2020-06-23-12-24-58-788.png|width=975,height=520!

 

 


was (Author: joliveirinha):
Has anyone been able to reproduce this and identify the root cause directly? I 
believe that this happens independetly if the log is being rolled or not.

I am having simular problems. The only difference is that this happen on the 
high percentiles (99.999 and 99.99 sometimes) without any log.roll or 
application flush happening.

I still see spikes that normally go to the 200ms independently of the 
throughput pushed by the producer. For these tests I am using a single producer 
and single consumer on the same process. In attachment you could see a plot on 
the end-2-end lantency (producer to consumer).

I am using default configurations and I have been able to push those spikes to 
the 99.999 most of the times by tuning the kernel parameters for page flush 
cache to be done every 1 second instead of the default 30s.

What is strange to me is the 200ms being pretty much consistent and unrelated 
to throuthput. I also tried to increment the max inflight requests as a way to 
push these latencies even more to the higher percentiles but it had an 
unexpected behaviour to me. In the higher throughput  runs the average latency 
increases from 2ms to ~40ms.

 

My configs:

*Producer*:
linger=0
batch=16k
recordSize=2k
compression=none
max.in.flight.conn=5

*Setup*:

6 i3.2xlarge with 1.9TB SSD on 3 AZs1 c5.2xlarge for the injector/consumer

 

 

!image-2020-06-23-12-24-58-788.png|width=975,height=520!

 

 

> Kafka latency spikes caused by log segment flush on roll
> 
>
> Key: KAFKA-9693
> URL: https://issues.apache.org/jira/browse/KAFKA-9693
> Project: Kafka
>  Issue Type: Improvement
>  Components: core
> Environment: OS: Amazon Linux 2
> Kafka version: 2.2.1
>Reporter: Paolo Moriello
>Assignee: Paolo Moriello
>Priority: Major
>  Labels: Performance, latency, performance
> Attachments: image-2020-03-10-13-17-34-618.png, 
> image-2020-03-10-14-36-21-807.png, image-2020-03-10-15-00-23-020.png, 
> image-2020-03-10-15-00-54-204.png, image-2020-06-23-12-24-46-548.png, 
> image-2020-06-23-12-24-58-788.png, latency_plot2.png
>
>
> h1. Summary
> When a log segment fills up, Kafka rolls over onto a new active segment and 
> force the flush of the old segment to disk. When this happens, log segment 
> _append_ duration increase causing important latency spikes on producer(s) 
> and replica(s). This ticket aims to highlight the problem and propose a 
> simple mitigation: add a new configuration to enable/disable rolled segment 
> flush.
> h1. 1. Phenomenon
> Response time of produce request (99th ~ 99.9th %ile) repeatedly spikes to 
> ~50x-200x more than usual. For instance, normally 99th %ile is lower than 
> 5ms, but when this issue occurs, it marks 100ms to 200ms. 99.9th and 99.99th 
> %iles even jump to 500-700ms.
> Latency spikes happen at constant frequency (depending on the input 
> throughput), for small amounts of time. All the producers experience a 
> latency increase at the same time.
> h1. !image-2020-03-10-13-17-34-618.png|width=942,height=314!
> {{Example of response time plot observed during on a