Skip to content

Commit

Permalink
utils/file_io: fix double closing of ss::file in write_fully
Browse files Browse the repository at this point in the history
ss::output_stream internally keeps track of in-flight exceptions. In the
close() method, it will close the underlying ss::file and rethrow the
exception.

ss::with_file_close_on_failure will too close the underlying ss::file if
the future fails, this can result in a double close triggering an
assertion like

```
seastar-prefix/src/seastar/include/seastar/core/future.hh:1917: future<T> seastar::promise<>::get_future() [T = void]: Assertion `!this->_future && this->_state && !this->_task' failed
```

This unit test shows how this assert could be triggered, if
ss::output_stream as an active exception:

```
SEASTAR_THREAD_TEST_CASE(test_with_file_close_on_failure) {
    auto flags = ss::open_flags::rw | ss::open_flags::create
                 | ss::open_flags::truncate;
    ss::with_file_close_on_failure(
      ss::open_file_dma("/tmp/tmp.YuupbuphlR", flags),
      [](ss::file f) mutable {
          return f.close().then([] { throw "any value"; });
       })
      .get();
}
```

This commit moves out ss::output_stream::close() from
ss::with_file_close_on_failure.

The method is coroutinized for clarity.

(cherry picked from commit d83a6cf)
  • Loading branch information
andijcr authored and vbotbuildovich committed May 13, 2024
1 parent 378f12e commit 627613c
Showing 1 changed file with 7 additions and 12 deletions.
19 changes: 7 additions & 12 deletions src/v/utils/file_io.cc
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@
#include "bytes/iobuf.h"
#include "bytes/iostream.h"

#include <seastar/core/coroutine.hh>
#include <seastar/core/file.hh>
#include <seastar/core/fstream.hh>
#include <seastar/core/temporary_buffer.hh>
Expand Down Expand Up @@ -57,17 +58,11 @@ ss::future<> write_fully(const std::filesystem::path& p, iobuf buf) {
| ss::open_flags::truncate;
/// Closes file on failure, otherwise file is expected to be closed in the
/// success case where the ss::output_stream calls close()
return ss::with_file_close_on_failure(
ss::open_file_dma(p.string(), flags),
[buf = std::move(buf)](ss::file f) mutable {
return ss::make_file_output_stream(std::move(f), buf_size)
.then([buf = std::move(buf)](ss::output_stream<char> out) mutable {
return ss::do_with(
std::move(out),
[buf = std::move(buf)](ss::output_stream<char>& out) mutable {
return write_iobuf_to_output_stream(std::move(buf), out)
.then([&out]() mutable { return out.close(); });
});
});
auto out = co_await ss::with_file_close_on_failure(
ss::open_file_dma(p.string(), flags), [](ss::file f) {
return ss::make_file_output_stream(std::move(f), buf_size);
});
co_await write_iobuf_to_output_stream(std::move(buf), out).finally([&out] {
return out.close();
});
}

0 comments on commit 627613c

Please sign in to comment.