You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
At least from the metrics we export the number of requests Redpanda was processing as well as the writes/reads to disk and network were relatively constant leading up to the observed spikes.
One interesting initial observation was that the metrics for;
/proc/pressure/io
were showing that most of the processes on the node were sleeping with io_schedule when the spike had occurred. The spikes roughly correlated with increased disk latency as well.
It is odd though that an increase in disk latency caused such a large increase in produce latency. Firstly write caching was enabled on the cluster which should allow us to absorb these latency increases as long as there is enough memory to store the writes temporarily. Secondly even without write caching since the latency increased only occurred on one node Redpanda should've been able to complete the produce request by having the writes complete on the majority of replicas.
One of the ways the above reasons won't hold is if the reactor thread itself is being blocked and cannot process any requests. To see if this was the case off-cpu-time was measured using;
sudo offcputime-bpfcc -u -m 10000 -f 600
With the following results;
do_io_getevents is an expected reason for the reactor thread not running. It occurs while seastar is polling for completed events. blk_mq_get_tag, however, is unexpected. This trace indicates that the block layer submit queue is full and that the kernel is suspending the reactor thread until a spot frees up.
On EBS disks the size of the queue is;
$ cat queue/nr_requests
63
Which is much smaller than what we see on local NVMe disks which tend to be greater than 1024. Also note that since we don't use an I/O scheduler its not currently possible to increase the queue size.
FIO reproduction
@travisdowns additionally was able to reproduce this issue in FIO using;
# Test the 16 KiB linear write throughput on different sized i3 instances.## This short (10 second) test shows that throughput for instance sizes of# 2xl and smaller has a short period of higher throughput, equaling the# equivalent (in number of disks) larger instance, before cutting down to# the documented IOPS/throughput value, which is very steady. The larger# instances appear to hit the hardware directly without such a rate limiter# and have noisier results.## This test requires only one 1 client node and no redpanda cluster.environment:
client:
provider: awsprovider_config:
client_arch: arm64aws_availability_zone: us-west-2aaws_region: us-west-2client_instance_type: im4gn.2xlargeenable_monitoring: truedeployment:
prometheus_scrape_interval: 1sprometheus_scrape_timeout: 1sbenchmark:
provider: fio.FioBenchlogging: truejobfile: | [file1] name=fio-seq-write time_based rw=write bs=128K direct=1 numjobs=1 unlink=1 runtime=60m # startdelay=1m rate=,400000000 size=1000GB ioengine=libaio iodepth=64 fdatasync=1
From this reproduction he was able to observe a few additional behaviors for EBS disks;
Firstly a similar spike in latency can be observed with the fio repo;
Secondly the variance in latency for writes before the EBS disk has been filled is a lot higher than afterwards during re-writes. We haven't investigated the cause, however, extent allocation/conversion in XFS as well as AWS dynamically allocating regions of the EBS disk are potential causes.
Solutions/mitigations
The "proper" solution seems to be ensuring the the NOWAIT flag is properly passed down into the block layer from iomap. There are reasons why this isn't an easy change though. @travisdowns has opened up an issue in liburing to see if there is a way to implement it.
In terms of things we can do in Redpanda to fix this issue the following are possible;
Submit all aio writes from a separate thread. This doesn't fix the issue per-say, but it prevents the reactor thread from blocking and allows for requests to continue being processed until the EBS latency spike passes.
Use larger write sizes. The testing above used a write size of 128KiB. It is possible to double this to 256KiB and therefore effectively double the block layer queue capacity.
Use libio_uring with a submission kernel worker. This is similar to making writes in a separate thread in that in ensures that the reactor thread isn't the one being blocked on slow I/O.
This issue hasn't seen activity in 3 months. If you want to keep it open, post a comment or remove the stale label – otherwise this will be closed in two weeks.
Issue
While testing Redpanda with write caching enabled on EBS disks we started to notice infrequent latency spikes in the results;
The following bench-runner configuration can reproduce these latency spikes;
Investigation
At least from the metrics we export the number of requests Redpanda was processing as well as the writes/reads to disk and network were relatively constant leading up to the observed spikes.
One interesting initial observation was that the metrics for;
were showing that most of the processes on the node were sleeping with
io_schedule
when the spike had occurred. The spikes roughly correlated with increased disk latency as well.It is odd though that an increase in disk latency caused such a large increase in produce latency. Firstly write caching was enabled on the cluster which should allow us to absorb these latency increases as long as there is enough memory to store the writes temporarily. Secondly even without write caching since the latency increased only occurred on one node Redpanda should've been able to complete the produce request by having the writes complete on the majority of replicas.
One of the ways the above reasons won't hold is if the reactor thread itself is being blocked and cannot process any requests. To see if this was the case off-cpu-time was measured using;
With the following results;
do_io_getevents
is an expected reason for the reactor thread not running. It occurs while seastar is polling for completed events.blk_mq_get_tag
, however, is unexpected. This trace indicates that the block layer submit queue is full and that the kernel is suspending the reactor thread until a spot frees up.On EBS disks the size of the queue is;
Which is much smaller than what we see on local NVMe disks which tend to be greater than 1024. Also note that since we don't use an I/O scheduler its not currently possible to increase the queue size.
FIO reproduction
@travisdowns additionally was able to reproduce this issue in FIO using;
From this reproduction he was able to observe a few additional behaviors for EBS disks;
Firstly a similar spike in latency can be observed with the fio repo;
Secondly the variance in latency for writes before the EBS disk has been filled is a lot higher than afterwards during re-writes. We haven't investigated the cause, however, extent allocation/conversion in XFS as well as AWS dynamically allocating regions of the EBS disk are potential causes.
Solutions/mitigations
The "proper" solution seems to be ensuring the the
NOWAIT
flag is properly passed down into the block layer from iomap. There are reasons why this isn't an easy change though. @travisdowns has opened up an issue in liburing to see if there is a way to implement it.In terms of things we can do in Redpanda to fix this issue the following are possible;
JIRA Link: CORE-6790
The text was updated successfully, but these errors were encountered: