Skip to content

JetStream loses acknowledged writes by default due to deferred fsync #7564

@aphyr

Description

@aphyr

Observed behavior

The Jetstream Developer docs say:

If the publisher receives the acknowledgement from the server it can safely discard any state it has for that publication, the message has not only been received correctly by the server, but it has also been successfully persisted.

The top-level JetStream docs say that "the formal consistency model of NATS JetStream is Linearizable", and the system as a whole is Serializable, "because messages are added to a stream in one global order".

There is an important caveat here. Unlike, say, MongoDB or Redpanda, which flush data to disk on at least a majority of nodes before acknowledging a write, NATS Jetstream only syncs to disk once every two minutes by default. However, nodes acknowledge writes immediately. It will happily lose data, or an entire stream, when nodes crash around the same time. This means that JetStream clusters will very likely violate claimed safety guarantees when (e.g.) a rack or datacenter power system fails.

For example, this test run caused the loss of all records written to the single JetStream stream. Every five seconds, nodes would log:

[1886557] 2025/11/19 13:51:45.506132 [WRN] Error applying entries to 'jepsen > jepsen-stream': catchup aborted, no leader for stream 'jepsen > jepsen-stream'
[1886557] 2025/11/19 13:51:45.506183 [WRN] RAFT [fjFyEjc1 - S-R5F-41G3ODY1] Draining and replaying snapshot

Every call to fetch during the five-minute recovery period returned an OK, but empty response.

In this run, the stream files existed on every server, but every file was zero bytes. Every attempt to subscribe to the stream failed for the five minute recovery period, logging [SUB-90007] No matching streams for subject.

If node failures happen a few minutes after a stream is created, JetStream reliably loses a large window of records. For example, a simulated power failure during this run caused JetStream to lose 131,418 out of 930,005 acknowledged writes.

Image

Node failures do not have to happen simultaneously for JetStream to lose data. For example, a rapid series of node crashes, approximately one every two seconds, was sufficient to cause total data loss in this test run. In this case, JetStream denied the entire existence of the stream when the cluster recovered, logging No matching streams for subject.

Image

The only mention of this fact I can find in any of NATS' documentation is buried in the JetStream configuration options list. I suggest that NATS either a.) change the default settings, or b.) prominently document that JetStream cannot tolerate some kinds of node crashes.

Expected behavior

NATS, like other distributed databases which employ a consensus algorithm, should not lose acknowledged operations when nodes crash.

Server and client version

NATS 2.12.1, and jnats 2.24.0

Host environment

These tests are running in LXC containers.

Steps to reproduce

This is in the Jepsen test suite, but give me a minute before I put in repro instructions--I've got one more scenario I want to build out.

Activity

added
defectSuspected defect such as a bug or regression
on Nov 20, 2025
derekcollison

derekcollison commented on Nov 20, 2025

@derekcollison
Member

As always great feedback, much appreciated!

The way the system handles fsync is by design. Most of our production setups, and in fact Synadia Cloud as well is that each replica is in a separate AZ. These have separate power, networking etc. So the possibility of a loss here is extremely low in terms of due to power outages. We do not want to penalize production users looking for higher ingest. However the period between fsync is configurable and we do have async_alwaysoption for the configuration as well which some of our production customers utilize.

aphyr

aphyr commented on Nov 20, 2025

@aphyr
Author

Ah, I have some bad news here--placing nodes in separate AZs does not mean that NATS' strategy of not syncing things to disk is safe. See #7567 for an example of a single node failure causing data loss (and split-brain!).

derekcollison

derekcollison commented on Nov 20, 2025

@derekcollison
Member

Will take a look - but just so we are on the same page, NATS allows the user to choose what works for them in terms of syncing.

MauriceVanVeen

MauriceVanVeen commented on Nov 27, 2025

@MauriceVanVeen
Member

Cross-posting here as the docs update is also relevant here: #7567 (comment)

Not sure if a test command was shared in this issue, but I'm assuming --lazyfs was used here as well just like in #7567 (comment). In that case this issue also requires OS hard kills/power loss.

aphyr

aphyr commented on Nov 28, 2025

@aphyr
Author

Yup! Or a kernel crash, disk failure, or other HW fault.

aphyr

aphyr commented on Dec 7, 2025

@aphyr
Author

It looks like nats-io/nats.docs#896 adds an explanation of this behavior to the JetStream concepts page. I'd be comfortable closing this, if y'all want to, or you can keep it open if you'd like to further discuss moving to fsync by default. Thoughts?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Metadata

Metadata

Assignees

No one assigned

    Labels

    defectSuspected defect such as a bug or regression

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

      Development

      No branches or pull requests

        Participants

        @aphyr@derekcollison@MauriceVanVeen

        Issue actions