sapling/eden/fs/inodes/EdenMount.h
Adam Simpkins 4949aada7f fix EdenServer::unmount() to fully wait for mount point cleanup
Summary:
This fixes EdenServer::unmount() to actually wait for all EdenMount cleanup
to complete, and fixes unmountAll() to return a Future that correctly waits for
all mount points to be cleaned up.

Previously `unmount()` waited for the mount point to be unmounted from the
kernel, but did not wait for EdenMount shutdown to complete.  Previously
EdenMount shutdown was not triggered until the last reference to the
shared_ptr<EdenMount> was released.  This often happened in the FUSE channel
thread that triggered the mountFinished() call--it would still hold a
reference to this pointer, and would not release it until after
mountFinished() returns.  As a result, when the main thread was shutting down,
`main()` would call `unmountAll()`, and then return soon after it completed.
Some FUSE channel threads may still be running at this point, still performing
`EdenMount` shutdown while the main thread was exiting.  This could result in
crashes and deadlocks as shutdown tried to access objects already destroyed by
the main thread.

With this change `EdenMount::shutdown()` is triggered explicitly during
`mountFinished()`, and `unmount()` will not complete until this finishes.
The `EdenMount` object may still exist at this point, and could still be
deleted by the FUSE channel thread, but the deletion now only requires freeing
the memory and does not require accessing other data that may have been cleaned
up by the main thread.

We should still clean up the FUSE channel thread handling in the future, to
make sure these threads are joined before the main thread exits.  However, that
cleanup can wait until a separate diff.  Ideally I would like to move more of
the mount and unmount logic from EdenServer and EdenServiceHandler and put that
code in EdenMount instead.

Reviewed By: bolinfest

Differential Revision: D5541318

fbshipit-source-id: 470332478357a85c314bc40458373cb0f827f62b
2017-08-02 17:07:19 -07:00

511 lines
15 KiB
C++

/*
* Copyright (c) 2016-present, Facebook, Inc.
* All rights reserved.
*
* This source code is licensed under the BSD-style license found in the
* LICENSE file in the root directory of this source tree. An additional grant
* of patent rights can be found in the PATENTS file in the same directory.
*
*/
#pragma once
#include <folly/Portability.h>
#include <folly/SharedMutex.h>
#include <folly/Synchronized.h>
#include <folly/ThreadLocal.h>
#include <folly/experimental/logging/Logger.h>
#include <memory>
#include <mutex>
#include <shared_mutex>
#include "eden/fs/fuse/EdenStats.h"
#include "eden/fs/fuse/fuse_headers.h"
#include "eden/fs/inodes/InodePtrFwd.h"
#include "eden/fs/journal/JournalDelta.h"
#include "eden/fs/model/ParentCommits.h"
#include "eden/fs/utils/PathFuncs.h"
namespace folly {
template <typename T>
class Future;
}
namespace facebook {
namespace eden {
namespace fusell {
class Channel;
class MountPoint;
}
class BindMount;
class CheckoutConflict;
class ClientConfig;
class Dirstate;
class EdenDispatcher;
class InodeDiffCallback;
class InodeMap;
class ObjectStore;
class Overlay;
class Journal;
class Tree;
class RenameLock;
class SharedRenameLock;
/**
* EdenMount contains all of the data about a specific eden mount point.
*
* This contains:
* - The fusell::MountPoint object which manages our FUSE interactions with the
* kernel.
* - The ObjectStore object used for retreiving/storing object data.
* - The Overlay object used for storing local changes (that have not been
* committed/snapshotted yet).
*/
class EdenMount {
public:
/**
* Create a shared_ptr to an EdenMount.
*
* Create an EdenMount instance and asynchronously initialize it. We use
* an EdenMountDeleter.
*/
static folly::Future<std::shared_ptr<EdenMount>> create(
std::unique_ptr<ClientConfig> config,
std::unique_ptr<ObjectStore> objectStore,
AbsolutePathPiece socketPath,
folly::ThreadLocal<fusell::EdenStats>* globalStats,
std::chrono::system_clock::time_point lastCheckoutTime =
std::chrono::system_clock::now());
/**
* Destroy the EdenMount.
*
* This method generally does not need to be invoked directly, and will
* instead be invoked automatically by the shared_ptr<EdenMount> returned by
* create(), once it becomes unreferenced.
*
* If the EdenMount has not already been explicitly shutdown(), destroy()
* will trigger the shutdown(). destroy() blocks until the shutdown is
* complete, so it is advisable for callers to callers to explicitly trigger
* shutdown() themselves if they want to ensure that the shared_ptr
* destruction will not block on this operation.
*/
void destroy();
/**
* Shutdown the EdenMount.
*
* This should be called *after* the FUSE mount point has been unmounted from
* the kernel.
*
* This cleans up the in-memory data associated with the EdenMount, and waits
* for all outstanding InodeBase objects to become unreferenced and be
* destroyed.
*/
folly::Future<folly::Unit> shutdown();
/**
* Get the MountPoint object.
*
* This returns a raw pointer since the EdenMount owns the mount point.
* The caller should generally maintain a reference to the EdenMount object,
* and not directly to the MountPoint object itself.
*/
fusell::MountPoint* getMountPoint() const {
return mountPoint_.get();
}
/**
* Get the FUSE channel for this mount point.
*
* This should only be called after the mount point has been successfully
* started. (It is the caller's responsibility to perform proper
* synchronization here with the mount start operation. This method provides
* no internal synchronization of its own.)
*/
fusell::Channel* getFuseChannel() const;
/**
* Return the path to the mount point.
*/
const AbsolutePath& getPath() const;
/**
* Get the commit IDs of the working directory's parent commit(s).
*/
ParentCommits getParentCommits() const {
return parentInfo_.rlock()->parents;
}
/*
* Return bind mounts that are applied for this mount. These are based on the
* state of the ClientConfig when this EdenMount was created.
*/
const std::vector<BindMount>& getBindMounts() const;
/**
* Return the ObjectStore used by this mount point.
*
* The ObjectStore is guaranteed to be valid for the lifetime of the
* EdenMount.
*/
ObjectStore* getObjectStore() const {
return objectStore_.get();
}
/**
* Return the EdenDispatcher used for this mount.
*/
EdenDispatcher* getDispatcher() const {
return dispatcher_.get();
}
/**
* Return the InodeMap for this mount.
*/
InodeMap* getInodeMap() const {
return inodeMap_.get();
}
const std::shared_ptr<Overlay>& getOverlay() const {
return overlay_;
}
Dirstate* getDirstate() {
return dirstate_.get();
}
folly::Synchronized<Journal>& getJournal() {
return journal_;
}
uint64_t getMountGeneration() const {
return mountGeneration_;
}
const ClientConfig* getConfig() const {
return config_.get();
}
/** Get the TreeInode for the root of the mount. */
TreeInodePtr getRootInode() const;
/** Get the inode number for the .eden dir */
fuse_ino_t getDotEdenInodeNumber() const;
/** Convenience method for getting the Tree for the root of the mount. */
std::unique_ptr<Tree> getRootTree() const;
folly::Future<std::unique_ptr<Tree>> getRootTreeFuture() const;
/**
* Look up the Inode object for the specified path.
*
* This may fail with an InodeError containing ENOENT if the path does not
* exist, or ENOTDIR if one of the intermediate components along the path is
* not a directory.
*
* This may also fail with other exceptions if something else goes wrong
* besides the path being invalid (for instance, an error loading data from
* the ObjectStore).
*/
folly::Future<InodePtr> getInode(RelativePathPiece path) const;
/**
* A blocking version of getInode().
*
* @return the InodeBase for the specified path or throws a std::system_error
* with ENOENT.
*
* TODO: We should switch all callers to use the Future-base API, and remove
* the blocking API.
*/
InodePtr getInodeBlocking(RelativePathPiece path) const;
/**
* Syntactic sugar for getInode().get().asTreePtr()
*
* TODO: We should switch all callers to use the Future-base API, and remove
* the blocking API.
*/
TreeInodePtr getTreeInodeBlocking(RelativePathPiece path) const;
/**
* Syntactic sugar for getInode().get().asFilePtr()
*
* TODO: We should switch all callers to use the Future-base API, and remove
* the blocking API.
*/
FileInodePtr getFileInodeBlocking(RelativePathPiece path) const;
/**
* Check out the specified commit.
*/
folly::Future<std::vector<CheckoutConflict>> checkout(
Hash snapshotHash,
bool force = false);
/**
* Compute differences between the current commit and the working directory
* state.
*
* @param callback This callback will be invoked as differences are found.
* Note that the callback methods may be invoked simultaneously from
* multiple different threads, and the callback is responsible for
* performing synchronization (if it is needed).
* @param listIgnored Whether or not to inform the callback of ignored files.
* When listIgnored to false can speed up the diff computation, as the
* code does not need to descend into ignord directories at all.
*
* @return Returns a folly::Future that will be fulfilled when the diff
* operation is complete. This is marked FOLLY_NODISCARD to
* make sure callers do not forget to wait for the operatio to complete.
*/
FOLLY_NODISCARD folly::Future<folly::Unit> diff(
InodeDiffCallback* callback,
bool listIgnored = false);
/**
* Reset the state to point to the specified parent commit(s), without
* modifying the working directory contents at all.
*
* @return Returns a folly::Future that will be fulfilled when the operation
* is complete. This is marked FOLLY_NODISCARD to make sure
* callers do not forget to wait for the operation to complete.
*/
FOLLY_NODISCARD folly::Future<folly::Unit> resetParents(
const ParentCommits& parents);
/**
* Reset the state to point to the specified parent commit, without
* modifying the working directory contents at all.
*
* This is a small wrapper around resetParents() for when the code knows at
* compile time that it will only ever have a single parent commit on this
* code path.
*/
FOLLY_NODISCARD folly::Future<folly::Unit> resetParent(const Hash& parent);
/**
* Acquire the rename lock in exclusive mode.
*/
RenameLock acquireRenameLock();
/**
* Acquire the rename lock in shared mode.
*/
SharedRenameLock acquireSharedRenameLock();
const AbsolutePath& getSocketPath() const;
/**
* Returns a pointer to a stats instance associated with this mountpoint.
* Today this is the global stats instance, but in the future it will be
* a mount point specific instance. */
folly::ThreadLocal<fusell::EdenStats>* getStats() const;
folly::Logger& getStraceLogger() {
return straceLogger_;
}
/**
* Returns the last checkout time in the Eden mount.
*/
struct timespec getLastCheckoutTime();
struct ParentInfo {
ParentCommits parents;
std::chrono::system_clock::time_point lastCheckoutTime;
};
private:
friend class RenameLock;
friend class SharedRenameLock;
/**
* The current running state of the EdenMount.
*
* For now this primarily tracks the status of the shutdown process.
* In the future we may want to add other states to also track the status of
* the actual mount point in the kernel. (e.g., a "STARTING" state before
* RUNNING for when the kernel mount point has not been fully set up yet, and
* an "UNMOUNTING" state if we have requested the kernel to unmount the mount
* point and that has not completed yet. UNMOUNTING would occur between
* RUNNING and SHUT_DOWN.) One possible downside of tracking
* STARTING/UNMOUNTING is that not every EdenMount object actually has a FUSE
* mount. During unit tests we create EdenMount objects without ever
* actually mounting them in the kernel.
*/
enum class State : uint32_t {
/**
* The EdenMount is running normally.
*/
RUNNING,
/**
* EdenMount::shutdown() has been called, but it is not complete yet.
*/
SHUTTING_DOWN,
/**
* EdenMount::shutdown() has completed, but there are still outstanding
* references so EdenMount::destroy() has not been called yet.
*
* When EdenMount::destroy() is called the object can be destroyed
* immediately.
*/
SHUT_DOWN,
/**
* EdenMount::destroy() has been called, but the shutdown is not complete
* yet. There are no remaining references to the EdenMount at this point,
* so when the shutdown completes it will be automatically destroyed.
*/
DESTROYING
};
EdenMount(
std::unique_ptr<ClientConfig> config,
std::unique_ptr<ObjectStore> objectStore,
AbsolutePathPiece socketPath,
folly::ThreadLocal<fusell::EdenStats>* globalStats,
std::chrono::system_clock::time_point lastCheckoutTime =
std::chrono::system_clock::now());
// Forbidden copy constructor and assignment operator
EdenMount(EdenMount const&) = delete;
EdenMount& operator=(EdenMount const&) = delete;
/**
* Asynchronous EdenMount initialization - post instantiation.
*/
folly::Future<folly::Unit> initialize();
folly::Future<TreeInodePtr> createRootInode(
const ParentCommits& parentCommits);
folly::Future<folly::Unit> setupDotEden(TreeInodePtr root);
folly::Future<folly::Unit> shutdownImpl();
/**
* Private destructor.
*
* This should not be invoked by callers directly. Use the destroy() method
* above (or the EdenMountDeleter if you plan to store the EdenMount in a
* std::unique_ptr or std::shared_ptr).
*/
~EdenMount();
/**
* The stats instance associated with this mount point.
* This is just a reference to a global stats instance today, but we'd
* like to make this its own child instance that aggregates up into
* the global instance in the future.
*/
folly::ThreadLocal<fusell::EdenStats>* globalEdenStats_{nullptr};
std::unique_ptr<ClientConfig> config_;
std::unique_ptr<InodeMap> inodeMap_;
std::unique_ptr<EdenDispatcher> dispatcher_;
std::unique_ptr<fusell::MountPoint> mountPoint_;
std::unique_ptr<ObjectStore> objectStore_;
std::shared_ptr<Overlay> overlay_;
std::unique_ptr<Dirstate> dirstate_;
fuse_ino_t dotEdenInodeNumber_{0};
/**
* A mutex around all name-changing operations in this mount point.
*
* This includes rename() operations as well as unlink() and rmdir().
* Any operation that modifies an existing InodeBase's location_ data must
* hold the rename lock.
*/
folly::SharedMutex renameMutex_;
/**
* The IDs of the parent commit(s) of the working directory.
*
* In most circumstances there will only be a single parent, but there
* will be two parents when in the middle of resolving a merge conflict.
*/
folly::Synchronized<ParentInfo> parentInfo_;
/*
* Note that this config will not be updated if the user modifies the
* underlying config files after the ClientConfig was created.
*/
const std::vector<BindMount> bindMounts_;
folly::Synchronized<Journal> journal_;
/**
* A number to uniquely identify this particular incarnation of this mount.
* We use bits from the process id and the time at which we were mounted.
*/
const uint64_t mountGeneration_;
/**
* The path to the unix socket that can be used to address us via thrift
*/
AbsolutePath socketPath_;
/**
* A log category for logging strace-events for this mount point.
*
* All FUSE operations to this mount point will get logged to this category.
* The category name is of the following form: "eden.strace.<mount_path>"
*/
folly::Logger straceLogger_;
/**
* This needs to use system_clock rather than steady_clock
* since this is used for filesystem timestamps that
* get displayed to users.
*/
std::chrono::system_clock::time_point lastCheckoutTime_;
/**
* The current state of the mount point.
*/
std::atomic<State> state_{State::RUNNING};
};
/**
* RenameLock is a holder for an EdenMount's rename mutex.
*
* This is primarily useful so it can be forward declared easily,
* but it also provides a helper method to ensure that it is currently holding
* a lock on the desired mount.
*/
class RenameLock : public std::unique_lock<folly::SharedMutex> {
public:
RenameLock() {}
explicit RenameLock(EdenMount* mount)
: std::unique_lock<folly::SharedMutex>{mount->renameMutex_} {}
bool isHeld(EdenMount* mount) const {
return owns_lock() && (mutex() == &mount->renameMutex_);
}
};
/**
* SharedRenameLock is a holder for an EdenMount's rename mutex in shared mode.
*/
class SharedRenameLock : public std::shared_lock<folly::SharedMutex> {
public:
explicit SharedRenameLock(EdenMount* mount)
: std::shared_lock<folly::SharedMutex>{mount->renameMutex_} {}
bool isHeld(EdenMount* mount) const {
return owns_lock() && (mutex() == &mount->renameMutex_);
}
};
/**
* EdenMountDeleter acts as a deleter argument for std::shared_ptr or
* std::unique_ptr.
*/
class EdenMountDeleter {
public:
void operator()(EdenMount* mount) {
mount->destroy();
}
};
}
}