kafka-users mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Eugen Dueck <eu...@tworks.co.jp>
Subject Re: synchronously flushing messages to disk
Date Sat, 07 Mar 2020 23:49:12 GMT
Thanks Alexandre

So if I understand you correctly, the following configuration will guarantee that when the
producer receives an ack, an fsync() for the message in question is guaranteed to have successfully
finished(). Is that correct?

log.flush.interval.messages=1
log.flush.interval.ms=0
log.flush.scheduler.interval.ms=0

The question whether an fsync() guarantees a durable write is a different one, but as far
as I know, unless you are using a consumer SSD, a completed call to fsync() on Linux (XFS)
will guarantee that the data is on disk if the machine goes down afterwards. After all, this
is what all ACID databases rely on, replicated or not. Without this guarantee, it would have
to be called ACI.

Eugen

________________________________
差出人: Alexandre Dupriez <alexandre.dupriez@gmail.com>
送信日時: 2020年3月8日 0:10
宛先: users@kafka.apache.org <users@kafka.apache.org>
件名: Re: synchronously flushing messages to disk

Hi Eugen,

The first line of config log.flush.interval.messages=1 will make Kafka
force an fsync(2) for every produce requests.
The second line of config is not sufficient for periodic flush, you
also need to update log.flush.scheduler.interval.ms which is Long.Max
by default (in which case period-based flush doesn't happen).

The impact you observe is expected - fsync(2) requires block I/O
operations which can be orders of magnitude longer than page cache
access.

Why would you like to sync on every commit? For durability guarantee
on commits, the standard approach in Kafka is to rely on replication
rather than physical disk writes. Also, note that even an fsync(2)
does not provide you the guarantee data has been physically written,
since data also has to go through other layers such as disk caches.

Le sam. 7 mars 2020 à 10:56, Eugen Dueck <eugen@tworks.co.jp> a écrit :
>
> I was under the impression that these settings
> log.flush.interval.messages=1
> log.flush.interval.ms=0
> guarantee a synchronous fsync for every message, i.e.when the producer receives an ack
for a message, it is guaranteed to have been persisted to as many disks as min.insync.replicas
requires.
>
> As I have heard other opinions on that, I'd like to know if someone in the Kafka community
can clarify.
>
> Best regards
> Eugen
> ________________________________
> 差出人: Eugen Dueck <eugen@tworks.co.jp>
> 送信日時: 2020年2月26日 13:28
> 宛先: users@kafka.apache.org <users@kafka.apache.org>
> 件名: synchronously flushing messages to disk
>
> Hi
>
> I want to benchmark Kafka, configured such that a message that has been acked by the
broker to the producer is guaranteed to have been persisted to disk. I changed the broker
settings:
>
> log.flush.interval.messages=1
> log.flush.interval.ms=0
>
> (Is this the proper way to do it?)
>
> The impact is very noticeable. Whereas without these settings, the msg/sec rate (1 producer,
1 topic, async, enable.idempotence) was well north of 100k, with above settings it drops to
below 5k on this dev box with ssd storage. This huge drop seems to indicate that Kafka is
not doing any batch acking (which would allow it to do batch fsyncing).
>
> Is there a way to increase the msg/sec rate given the fsync constraint? It would seem
that adding topics/partitions would help in case of a cluster, and the fsync load could be
distributed to multiple machines. Is there perhaps also a way to increase the rate per node?
>
> I'm using the latest kafka 2.4.0.
>
> Best regards
> Eugen
Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message