support closing the Overlay class even if it is still in use

Summary:
Throw errors if a read or write is called on the `Overlay` class after `close()` has been called. Adds IO reference counting to know when it is safe to release the overlay lock after calling `close()`

This takes care to catch errors that could be thrown around overlay IO and makes sure to correctly decrement the IO count in the case of errors.

Reviewed By: chadaustin

Differential Revision: D17957360

fbshipit-source-id: af5e3c3f98c77800914d03cd748fb2f1c59ae264
This commit is contained in:
Genevieve Helsel 2019-11-11 10:53:50 -08:00 committed by Facebook Github Bot
parent 60d252f6c3
commit f23c60212d
5 changed files with 286 additions and 3 deletions

View File

@ -561,6 +561,7 @@ folly::SemiFuture<SerializedInodeMap> EdenMount::shutdownImpl(bool doTakeover) {
// released the lock before the new edenfs process begins to take over
// the mount point.
overlay_->close();
XLOG(DBG1) << "successfully closed overlay at " << getPath();
auto oldState =
state_.exchange(State::SHUT_DOWN, std::memory_order_acq_rel);
if (oldState == State::DESTROYING) {

View File

@ -70,11 +70,13 @@ void Overlay::close() {
optNextInodeNumber = InodeNumber{nextInodeNumber};
}
closeAndWaitForOutstandingIO();
inodeMetadataTable_.reset();
fsOverlay_.close(optNextInodeNumber);
}
struct statfs Overlay::statFs() const {
struct statfs Overlay::statFs() {
IORequest req{this};
return fsOverlay_.statFs();
}
@ -103,6 +105,7 @@ folly::SemiFuture<Unit> Overlay::initialize() {
}
void Overlay::initOverlay() {
IORequest req{this};
auto optNextInodeNumber = fsOverlay_.initOverlay(true);
if (!optNextInodeNumber.has_value()) {
// If the next-inode-number data is missing it means that this overlay was
@ -149,6 +152,7 @@ InodeNumber Overlay::allocateInodeNumber() {
}
optional<DirContents> Overlay::loadOverlayDir(InodeNumber inodeNumber) {
IORequest req{this};
auto dirData = fsOverlay_.loadOverlayDir(inodeNumber);
if (!dirData.has_value()) {
return std::nullopt;
@ -189,6 +193,7 @@ optional<DirContents> Overlay::loadOverlayDir(InodeNumber inodeNumber) {
}
void Overlay::saveOverlayDir(InodeNumber inodeNumber, const DirContents& dir) {
IORequest req{this};
auto nextInodeNumber = nextInodeNumber_.load(std::memory_order_relaxed);
CHECK_LT(inodeNumber.get(), nextInodeNumber)
<< "saveOverlayDir called with unallocated inode number";
@ -229,12 +234,14 @@ void Overlay::saveOverlayDir(InodeNumber inodeNumber, const DirContents& dir) {
}
void Overlay::removeOverlayData(InodeNumber inodeNumber) {
IORequest req{this};
// TODO: batch request during GC
getInodeMetadataTable()->freeInode(inodeNumber);
fsOverlay_.removeOverlayFile(inodeNumber);
}
void Overlay::recursivelyRemoveOverlayData(InodeNumber inodeNumber) {
IORequest req{this};
auto dirData = fsOverlay_.loadOverlayDir(inodeNumber);
// This inode's data must be removed from the overlay before
@ -260,6 +267,7 @@ folly::Future<folly::Unit> Overlay::flushPendingAsync() {
}
bool Overlay::hasOverlayData(InodeNumber inodeNumber) {
IORequest req{this};
return fsOverlay_.hasOverlayData(inodeNumber);
}
@ -268,11 +276,13 @@ bool Overlay::hasOverlayData(InodeNumber inodeNumber) {
OverlayFile Overlay::openFile(
InodeNumber inodeNumber,
folly::StringPiece headerId) {
IORequest req{this};
return OverlayFile(
fsOverlay_.openFile(inodeNumber, headerId), weak_from_this());
}
OverlayFile Overlay::openFileNoVerify(InodeNumber inodeNumber) {
IORequest req{this};
return OverlayFile(
fsOverlay_.openFileNoVerify(inodeNumber), weak_from_this());
}
@ -280,6 +290,7 @@ OverlayFile Overlay::openFileNoVerify(InodeNumber inodeNumber) {
OverlayFile Overlay::createOverlayFile(
InodeNumber inodeNumber,
folly::ByteRange contents) {
IORequest req{this};
CHECK_LT(inodeNumber.get(), nextInodeNumber_.load(std::memory_order_relaxed))
<< "createOverlayFile called with unallocated inode number";
return OverlayFile(
@ -289,6 +300,7 @@ OverlayFile Overlay::createOverlayFile(
OverlayFile Overlay::createOverlayFile(
InodeNumber inodeNumber,
const folly::IOBuf& contents) {
IORequest req{this};
CHECK_LT(inodeNumber.get(), nextInodeNumber_.load(std::memory_order_relaxed))
<< "createOverlayFile called with unallocated inode number";
return OverlayFile(
@ -301,6 +313,55 @@ InodeNumber Overlay::getMaxInodeNumber() {
return InodeNumber{ino - 1};
}
static constexpr uint64_t ioCountMask = 0x7FFFFFFFFFFFFFFFull;
static constexpr uint64_t ioClosedMask = 1ull << 63;
bool Overlay::tryIncOutstandingIORequests() {
uint64_t currentOutstandingIO =
outstandingIORequests_.load(std::memory_order_seq_cst);
// Retry incrementing the IO count while we have not either successfully
// updated outstandingIORequests_ or closed the overlay
while (!(currentOutstandingIO & ioClosedMask)) {
// If not closed, currentOutstandingIO now holds what
// outstandingIORequests_ actually contained
if (outstandingIORequests_.compare_exchange_weak(
currentOutstandingIO,
currentOutstandingIO + 1,
std::memory_order_seq_cst)) {
return true;
}
}
// If we have broken out of the above loop, the overlay is closed and we
// been unable to increment outstandingIORequests_.
return false;
}
void Overlay::decOutstandingIORequests() {
uint64_t outstanding =
outstandingIORequests_.fetch_sub(1, std::memory_order_seq_cst);
XCHECK_NE(0ull, outstanding) << "Decremented too far!";
// If the overlay is closed and we just finished our last IO request (meaning
// the previous value of outstandingIORequests_ was 1), then wake the waiting
// thread.
if ((outstanding & ioClosedMask) && (outstanding & ioCountMask) == 1) {
lastOutstandingRequestIsComplete_.post();
}
}
void Overlay::closeAndWaitForOutstandingIO() {
uint64_t outstanding =
outstandingIORequests_.fetch_or(ioClosedMask, std::memory_order_seq_cst);
// If we have outstanding IO requests, wait for them. This should not block if
// this baton has already been posted between the load in the fetch_or and
// this if statement.
if (outstanding & ioCountMask) {
lastOutstandingRequestIsComplete_.wait();
}
}
void Overlay::gcThread() noexcept {
for (;;) {
std::vector<GCRequest> requests;
@ -329,6 +390,7 @@ void Overlay::gcThread() noexcept {
}
void Overlay::handleGCRequest(GCRequest& request) {
IORequest req{this};
if (request.flush) {
request.flush->setValue();
return;

View File

@ -11,6 +11,7 @@
#include <folly/futures/Future.h>
#include <folly/futures/Promise.h>
#include <array>
#include <atomic>
#include <condition_variable>
#include <optional>
#include <thread>
@ -183,7 +184,7 @@ class Overlay : public std::enable_shared_from_this<Overlay> {
/**
* call statfs(2) on the filesystem in which the overlay is located
*/
struct statfs statFs() const;
struct statfs statFs();
private:
explicit Overlay(AbsolutePathPiece localDir);
@ -218,6 +219,10 @@ class Overlay : public std::enable_shared_from_this<Overlay> {
void gcThread() noexcept;
void handleGCRequest(GCRequest& request);
bool tryIncOutstandingIORequests();
void decOutstandingIORequests();
void closeAndWaitForOutstandingIO();
/**
* The next inode number to allocate. Zero indicates that neither
* initializeFromTakeover nor getMaxRecordedInode have been called.
@ -242,6 +247,50 @@ class Overlay : public std::enable_shared_from_this<Overlay> {
std::thread gcThread_;
folly::Synchronized<GCQueue, std::mutex> gcQueue_;
std::condition_variable gcCondVar_;
/**
* This uint64_t holds two values, a single bit on the MSB that
* acts a boolean closed: True if the the Overlay has been closed with
* calling setClosed(). When this is true, reads and writes will throw an
* error instead of applying an overlay change or read. On the rest of the
* bits, the actual number of outstanding IO requests is held. This has been
* done in order to synchronize these two variables and treat checking if the
* overlay is closed and incrementing the IO reference count as a single
* atomic action.
*/
mutable std::atomic<uint64_t> outstandingIORequests_{0};
folly::Baton<> lastOutstandingRequestIsComplete_;
friend class IORequest;
};
/**
* Used to reference count IO requests. In any place that there
* is an overlay read or write, this struct should be constructed in order to
* properly reference count and to properly deny overlay reads and
* modifications in the case that the overlay is closed.
*/
class IORequest {
public:
explicit IORequest(Overlay* o) : overlay_{o} {
if (!overlay_->tryIncOutstandingIORequests()) {
throw std::system_error(
EIO,
std::generic_category(),
folly::to<std::string>("cannot access overlay after it is closed"));
}
}
~IORequest() {
overlay_->decOutstandingIORequests();
}
private:
IORequest(IORequest&&) = delete;
IORequest& operator=(IORequest&&) = delete;
Overlay* const overlay_;
};
} // namespace eden

View File

@ -18,6 +18,12 @@ OverlayFile::OverlayFile(folly::File file, std::weak_ptr<Overlay> overlay)
: file_{std::move(file)}, overlay_{overlay} {}
folly::Expected<struct stat, int> OverlayFile::fstat() const {
std::shared_ptr<Overlay> overlay = overlay_.lock();
if (!overlay) {
return folly::makeUnexpected(EIO);
}
IORequest req{overlay.get()};
struct stat st {};
if (::fstat(file_.fd(), &st)) {
return folly::makeUnexpected(errno);
@ -27,6 +33,12 @@ folly::Expected<struct stat, int> OverlayFile::fstat() const {
folly::Expected<ssize_t, int>
OverlayFile::preadNoInt(void* buf, size_t n, off_t offset) const {
std::shared_ptr<Overlay> overlay = overlay_.lock();
if (!overlay) {
return folly::makeUnexpected(EIO);
}
IORequest req{overlay.get()};
auto ret = folly::preadNoInt(file_.fd(), buf, n, offset);
if (ret == -1) {
return folly::makeUnexpected(errno);
@ -35,6 +47,12 @@ OverlayFile::preadNoInt(void* buf, size_t n, off_t offset) const {
}
folly::Expected<off_t, int> OverlayFile::lseek(off_t offset, int whence) const {
std::shared_ptr<Overlay> overlay = overlay_.lock();
if (!overlay) {
return folly::makeUnexpected(EIO);
}
IORequest req{overlay.get()};
auto ret = ::lseek(file_.fd(), offset, whence);
if (ret == -1) {
return folly::makeUnexpected(errno);
@ -44,6 +62,12 @@ folly::Expected<off_t, int> OverlayFile::lseek(off_t offset, int whence) const {
folly::Expected<ssize_t, int>
OverlayFile::pwritev(const iovec* iov, int iovcnt, off_t offset) const {
std::shared_ptr<Overlay> overlay = overlay_.lock();
if (!overlay) {
return folly::makeUnexpected(EIO);
}
IORequest req{overlay.get()};
auto ret = folly::pwritevNoInt(file_.fd(), iov, iovcnt, offset);
if (ret == -1) {
return folly::makeUnexpected(errno);
@ -52,6 +76,12 @@ OverlayFile::pwritev(const iovec* iov, int iovcnt, off_t offset) const {
}
folly::Expected<int, int> OverlayFile::ftruncate(off_t length) const {
std::shared_ptr<Overlay> overlay = overlay_.lock();
if (!overlay) {
return folly::makeUnexpected(EIO);
}
IORequest req{overlay.get()};
auto ret = ::ftruncate(file_.fd(), length);
if (ret == -1) {
return folly::makeUnexpected(errno);
@ -60,6 +90,12 @@ folly::Expected<int, int> OverlayFile::ftruncate(off_t length) const {
}
folly::Expected<int, int> OverlayFile::fsync() const {
std::shared_ptr<Overlay> overlay = overlay_.lock();
if (!overlay) {
return folly::makeUnexpected(EIO);
}
IORequest req{overlay.get()};
auto ret = ::fsync(file_.fd());
if (ret == -1) {
return folly::makeUnexpected(errno);
@ -69,6 +105,12 @@ folly::Expected<int, int> OverlayFile::fsync() const {
folly::Expected<int, int> OverlayFile::fdatasync() const {
#ifndef __APPLE__
std::shared_ptr<Overlay> overlay = overlay_.lock();
if (!overlay) {
return folly::makeUnexpected(EIO);
}
IORequest req{overlay.get()};
auto ret = ::fdatasync(file_.fd());
if (ret == -1) {
return folly::makeUnexpected(errno);
@ -80,6 +122,12 @@ folly::Expected<int, int> OverlayFile::fdatasync() const {
}
folly::Expected<std::string, int> OverlayFile::readFile() const {
std::shared_ptr<Overlay> overlay = overlay_.lock();
if (!overlay) {
return folly::makeUnexpected(EIO);
}
IORequest req{overlay.get()};
std::string out;
if (!folly::readFile(file_.fd(), out)) {
return folly::makeUnexpected(errno);

View File

@ -13,14 +13,15 @@
#include <folly/FileUtil.h>
#include <folly/Range.h>
#include <folly/Subprocess.h>
#include <folly/executors/CPUThreadPoolExecutor.h>
#include <folly/experimental/TestUtil.h>
#include <folly/logging/test/TestLogHandler.h>
#include <folly/synchronization/test/Barrier.h>
#include <folly/test/TestUtils.h>
#include <gtest/gtest.h>
#include <algorithm>
#include <iomanip>
#include <sstream>
#include "eden/fs/utils/PathFuncs.h"
#include "eden/fs/inodes/EdenMount.h"
#include "eden/fs/inodes/FileInode.h"
@ -33,6 +34,7 @@
#include "eden/fs/testharness/TestChecks.h"
#include "eden/fs/testharness/TestMount.h"
#include "eden/fs/testharness/TestUtil.h"
#include "eden/fs/utils/PathFuncs.h"
using namespace folly::string_piece_literals;
using folly::Subprocess;
@ -316,6 +318,127 @@ class RawOverlayTest : public ::testing::TestWithParam<OverlayRestartMode> {
std::shared_ptr<Overlay> overlay;
};
TEST_P(RawOverlayTest, closed_overlay_stress_test) {
constexpr unsigned kThreadCount = 10;
auto executor = folly::CPUThreadPoolExecutor(kThreadCount + 1);
std::vector<folly::Future<folly::Unit>> futures;
futures.reserve(kThreadCount);
folly::test::Barrier gate{kThreadCount + 1};
for (unsigned i = 0; i < kThreadCount; ++i) {
futures.emplace_back(folly::via(&executor, [&] {
auto ino = overlay->allocateInodeNumber();
OverlayFile result;
try {
result =
overlay->createOverlayFile(ino, folly::ByteRange{"contents"_sp});
} catch (std::system_error& e) {
if ("cannot access overlay after it is closed: Input/output error"_sp !=
e.what()) {
printf("createOverlayFile failed: %s\n", e.what());
throw e;
}
}
// Block until after overlay has closed
gate.wait();
try {
char data[] = "new contents";
struct iovec iov;
iov.iov_base = data;
iov.iov_len = sizeof(data);
result.pwritev(&iov, 1, FsOverlay::kHeaderLength);
throw std::system_error(
EIO,
std::generic_category(),
"should not be able to successfully write to overlay");
} catch (std::system_error& e) {
if (strcmp(
e.what(),
"cannot access overlay after it is closed: Input/output error")) {
printf("pwritev failed: %s\n", e.what());
throw e;
}
}
}));
}
overlay->close();
// Wake the waiting threads
gate.wait();
auto finished = folly::collectAll(futures).get();
for (auto& f : finished) {
EXPECT_FALSE(f.hasException());
}
}
TEST_P(RawOverlayTest, cannot_create_overlay_file_in_corrupt_overlay) {
auto ino2 = overlay->allocateInodeNumber();
EXPECT_EQ(2_ino, ino2);
// Remove the overlay directory in order to make file creation fail.
auto path = testDir_.path();
boost::filesystem::remove_all(path);
EXPECT_THROW(
overlay->createOverlayFile(ino2, folly::ByteRange{"contents"_sp}),
std::system_error);
// Restore the overlay directory and make sure we can create an overlay file
// and close the overlay.
boost::filesystem::create_directory(path);
loadOverlay();
ino2 = overlay->allocateInodeNumber();
EXPECT_EQ(2_ino, ino2);
EXPECT_NO_THROW(
overlay->createOverlayFile(ino2, folly::ByteRange{"contents"_sp}));
overlay->close();
}
TEST_P(RawOverlayTest, cannot_save_overlay_dir_when_closed) {
overlay->close();
auto ino2 = overlay->allocateInodeNumber();
EXPECT_EQ(2_ino, ino2);
DirContents dir;
EXPECT_THROW_RE(
overlay->saveOverlayDir(ino2, dir),
std::system_error,
"cannot access overlay after it is closed");
}
TEST_P(RawOverlayTest, cannot_create_overlay_file_when_closed) {
overlay->close();
auto ino2 = overlay->allocateInodeNumber();
EXPECT_EQ(2_ino, ino2);
EXPECT_THROW_RE(
overlay->createOverlayFile(ino2, folly::ByteRange{"contents"_sp}),
std::system_error,
"cannot access overlay after it is closed");
}
TEST_P(RawOverlayTest, cannot_remove_overlay_file_when_closed) {
auto ino2 = overlay->allocateInodeNumber();
EXPECT_EQ(2_ino, ino2);
EXPECT_NO_THROW(
overlay->createOverlayFile(ino2, folly::ByteRange{"contents"_sp}));
overlay->close();
EXPECT_THROW_RE(
overlay->removeOverlayData(ino2),
std::system_error,
"cannot access overlay after it is closed");
}
TEST_P(RawOverlayTest, max_inode_number_is_1_if_overlay_is_empty) {
EXPECT_EQ(kRootNodeId, overlay->getMaxInodeNumber());
EXPECT_EQ(2_ino, overlay->allocateInodeNumber());