]> git.proxmox.com Git - ceph.git/blobdiff - ceph/src/os/bluestore/BlueFS.h
import ceph quincy 17.2.6
[ceph.git] / ceph / src / os / bluestore / BlueFS.h
index 14fca0fb5b3c8d73b38abc50b5445af3604871ad..736450c5e99391d16d195a0ea3a967f9794bba0e 100644 (file)
@@ -5,9 +5,10 @@
 
 #include <atomic>
 #include <mutex>
+#include <limits>
 
 #include "bluefs_types.h"
-#include "BlockDevice.h"
+#include "blk/BlockDevice.h"
 
 #include "common/RefCountedObj.h"
 #include "common/ceph_context.h"
@@ -21,8 +22,6 @@ class Allocator;
 
 enum {
   l_bluefs_first = 732600,
-  l_bluefs_gift_bytes,
-  l_bluefs_reclaim_bytes,
   l_bluefs_db_total_bytes,
   l_bluefs_db_used_bytes,
   l_bluefs_wal_total_bytes,
@@ -41,39 +40,36 @@ enum {
   l_bluefs_max_bytes_wal,
   l_bluefs_max_bytes_db,
   l_bluefs_max_bytes_slow,
+  l_bluefs_main_alloc_unit,
+  l_bluefs_db_alloc_unit,
+  l_bluefs_wal_alloc_unit,
   l_bluefs_read_random_count,
   l_bluefs_read_random_bytes,
   l_bluefs_read_random_disk_count,
   l_bluefs_read_random_disk_bytes,
+  l_bluefs_read_random_disk_bytes_wal,
+  l_bluefs_read_random_disk_bytes_db,
+  l_bluefs_read_random_disk_bytes_slow,
   l_bluefs_read_random_buffer_count,
   l_bluefs_read_random_buffer_bytes,
   l_bluefs_read_count,
   l_bluefs_read_bytes,
+  l_bluefs_read_disk_count,
+  l_bluefs_read_disk_bytes,
+  l_bluefs_read_disk_bytes_wal,
+  l_bluefs_read_disk_bytes_db,
+  l_bluefs_read_disk_bytes_slow,
   l_bluefs_read_prefetch_count,
   l_bluefs_read_prefetch_bytes,
-
+  l_bluefs_compaction_lat,
+  l_bluefs_compaction_lock_lat,
+  l_bluefs_alloc_shared_dev_fallbacks,
+  l_bluefs_alloc_shared_size_fallbacks,
+  l_bluefs_read_zeros_candidate,
+  l_bluefs_read_zeros_errors,
   l_bluefs_last,
 };
 
-class BlueFSDeviceExpander {
-protected:
-  ~BlueFSDeviceExpander() {}
-public:
-  virtual uint64_t get_recommended_expansion_delta(uint64_t bluefs_free,
-    uint64_t bluefs_total) = 0;
-  virtual int allocate_freespace(
-    uint64_t min_size,
-    uint64_t size,
-    PExtentVector& extents) = 0;
-  /** Reports amount of space that can be transferred to BlueFS.
-   * This gives either current state, when alloc_size is currently used
-   * BlueFS's size, or simulation when alloc_size is different.
-   * @params
-   * alloc_size - allocation unit size to check
-   */
-  virtual uint64_t available_freespace(uint64_t alloc_size) = 0;
-};
-
 class BlueFSVolumeSelector {
 public:
   typedef std::vector<std::pair<std::string, uint64_t>> paths;
@@ -81,7 +77,7 @@ public:
   virtual ~BlueFSVolumeSelector() {
   }
   virtual void* get_hint_for_log() const = 0;
-  virtual void* get_hint_by_dir(const std::string& dirname) const = 0;
+  virtual void* get_hint_by_dir(std::string_view dirname) const = 0;
 
   virtual void add_usage(void* file_hint, const bluefs_fnode_t& fnode) = 0;
   virtual void sub_usage(void* file_hint, const bluefs_fnode_t& fnode) = 0;
@@ -89,9 +85,31 @@ public:
   virtual void sub_usage(void* file_hint, uint64_t fsize) = 0;
   virtual uint8_t select_prefer_bdev(void* hint) = 0;
   virtual void get_paths(const std::string& base, paths& res) const = 0;
-  virtual void dump(ostream& sout) = 0;
+  virtual void dump(std::ostream& sout) = 0;
+
+  /* used for sanity checking of vselector */
+  virtual BlueFSVolumeSelector* clone_empty() const { return nullptr; }
+  virtual bool compare(BlueFSVolumeSelector* other) { return true; };
+};
+
+struct bluefs_shared_alloc_context_t {
+  bool need_init = false;
+  Allocator* a = nullptr;
+  uint64_t alloc_unit = 0;
+
+  std::atomic<uint64_t> bluefs_used = 0;
+
+  void set(Allocator* _a, uint64_t _au) {
+    a = _a;
+    alloc_unit = _au;
+    need_init = true;
+    bluefs_used = 0;
+  }
+  void reset() {
+    a = nullptr;
+    alloc_unit = 0;
+  }
 };
-class BlueFS;
 
 class BlueFS {
 public:
@@ -117,12 +135,18 @@ public:
     uint64_t dirty_seq;
     bool locked;
     bool deleted;
+    bool is_dirty;
     boost::intrusive::list_member_hook<> dirty_item;
 
     std::atomic_int num_readers, num_writers;
     std::atomic_int num_reading;
 
     void* vselector_hint = nullptr;
+    /* lock protects fnode and other the parts that can be modified during read & write operations.
+       Does not protect values that are fixed
+       Does not need to be taken when doing one-time operations:
+       _replay, device_migrate_to_existing, device_migrate_to_new */
+    ceph::mutex lock = ceph::make_mutex("BlueFS::File::lock");
 
   private:
     FRIEND_MAKE_REF(File);
@@ -132,6 +156,7 @@ public:
        dirty_seq(0),
        locked(false),
        deleted(false),
+       is_dirty(false),
        num_readers(0),
        num_writers(0),
        num_reading(0),
@@ -156,7 +181,7 @@ public:
   struct Dir : public RefCountedObject {
     MEMPOOL_CLASS_HELPERS();
 
-    mempool::bluefs::map<string,FileRef> file_map;
+    mempool::bluefs::map<std::string, FileRef, std::less<>> file_map;
 
   private:
     FRIEND_MAKE_REF(Dir);
@@ -169,9 +194,20 @@ public:
 
     FileRef file;
     uint64_t pos = 0;       ///< start offset for buffer
-    bufferlist buffer;      ///< new data to write (at end of file)
-    bufferlist tail_block;  ///< existing partial block at end of file, if any
-    bufferlist::page_aligned_appender buffer_appender;  //< for const char* only
+  private:
+    ceph::buffer::list buffer;      ///< new data to write (at end of file)
+    ceph::buffer::list tail_block;  ///< existing partial block at end of file, if any
+  public:
+    unsigned get_buffer_length() const {
+      return buffer.length();
+    }
+    ceph::bufferlist flush_buffer(
+      CephContext* cct,
+      const bool partial,
+      const unsigned length,
+      const bluefs_super_t& super);
+    ceph::buffer::list::page_aligned_appender buffer_appender;  //< for const char* only
+  public:
     int writer_type = 0;    ///< WRITER_*
     int write_hint = WRITE_LIFE_NOT_SET;
 
@@ -181,8 +217,8 @@ public:
 
     FileWriter(FileRef f)
       : file(std::move(f)),
-       buffer_appender(buffer.get_page_aligned_appender(
-                         g_conf()->bluefs_alloc_size / CEPH_PAGE_SIZE)) {
+       buffer_appender(buffer.get_page_aligned_appender(
+                         g_conf()->bluefs_alloc_size / CEPH_PAGE_SIZE)) {
       ++file->num_writers;
       iocv.fill(nullptr);
       dirty_devs.fill(false);
@@ -196,19 +232,33 @@ public:
     }
 
     // note: BlueRocksEnv uses this append exclusively, so it's safe
-    // to use buffer_appender exclusively here (e.g., it's notion of
+    // to use buffer_appender exclusively here (e.g., its notion of
     // offset will remain accurate).
     void append(const char *buf, size_t len) {
+      uint64_t l0 = get_buffer_length();
+      ceph_assert(l0 + len <= std::numeric_limits<unsigned>::max());
       buffer_appender.append(buf, len);
     }
 
+    void append(const std::byte *buf, size_t len) {
+      // allow callers to use byte type instead of char* as we simply pass byte array
+      append((const char*)buf, len);
+    }
+
     // note: used internally only, for ino 1 or 0.
-    void append(bufferlist& bl) {
+    void append(ceph::buffer::list& bl) {
+      uint64_t l0 = get_buffer_length();
+      ceph_assert(l0 + bl.length() <= std::numeric_limits<unsigned>::max());
       buffer.claim_append(bl);
     }
 
+    void append_zero(size_t len) {
+      uint64_t l0 = get_buffer_length();
+      ceph_assert(l0 + len <= std::numeric_limits<unsigned>::max());
+      buffer_appender.append_zero(len);
+    }
+
     uint64_t get_effective_write_pos() {
-      buffer_appender.flush();
       return pos + buffer.length();
     }
   };
@@ -217,7 +267,7 @@ public:
     MEMPOOL_CLASS_HELPERS();
 
     uint64_t bl_off = 0;    ///< prefetch buffer logical offset
-    bufferlist bl;          ///< prefetch buffer
+    ceph::buffer::list bl;          ///< prefetch buffer
     uint64_t pos = 0;       ///< current logical offset
     uint64_t max_prefetch;  ///< max allowed prefetch
 
@@ -236,8 +286,14 @@ public:
     void skip(size_t n) {
       pos += n;
     }
-    void seek(uint64_t offset) {
-      pos = offset;
+
+    // For the sake of simplicity, we invalidate completed rather than
+    // for the provided extent
+    void invalidate_cache(uint64_t offset, uint64_t length) {
+      if (offset >= bl_off && offset < get_buf_end()) {
+       bl.clear();
+       bl_off = 0;
+      }
     }
   };
 
@@ -274,8 +330,6 @@ public:
   };
 
 private:
-  ceph::mutex lock = ceph::make_mutex("BlueFS::lock");
-
   PerfCounters *logger = nullptr;
 
   uint64_t max_bytes[MAX_BDEV] = {0};
@@ -283,29 +337,44 @@ private:
     l_bluefs_max_bytes_wal,
     l_bluefs_max_bytes_db,
     l_bluefs_max_bytes_slow,
+    l_bluefs_max_bytes_wal,
+    l_bluefs_max_bytes_db,
   };
 
   // cache
-  mempool::bluefs::map<string, DirRef> dir_map;              ///< dirname -> Dir
-  mempool::bluefs::unordered_map<uint64_t,FileRef> file_map; ///< ino -> File
-
-  // map of dirty files, files of same dirty_seq are grouped into list.
-  map<uint64_t, dirty_file_list_t> dirty_files;
+  struct {
+    ceph::mutex lock = ceph::make_mutex("BlueFS::nodes.lock");
+    mempool::bluefs::map<std::string, DirRef, std::less<>> dir_map; ///< dirname -> Dir
+    mempool::bluefs::unordered_map<uint64_t, FileRef> file_map;     ///< ino -> File
+  } nodes;
 
   bluefs_super_t super;        ///< latest superblock (as last written)
   uint64_t ino_last = 0;       ///< last assigned ino (this one is in use)
-  uint64_t log_seq = 0;        ///< last used log seq (by current pending log_t)
-  uint64_t log_seq_stable = 0; ///< last stable/synced log seq
-  FileWriter *log_writer = 0;  ///< writer for the log
-  bluefs_transaction_t log_t;  ///< pending, unwritten log transaction
-  bool log_flushing = false;   ///< true while flushing the log
-  ceph::condition_variable log_cond;
-
-  uint64_t new_log_jump_to = 0;
-  uint64_t old_log_jump_to = 0;
-  FileRef new_log = nullptr;
-  FileWriter *new_log_writer = nullptr;
 
+  struct {
+    ceph::mutex lock = ceph::make_mutex("BlueFS::log.lock");
+    uint64_t seq_live = 1;   //seq that log is currently writing to; mirrors dirty.seq_live
+    FileWriter *writer = 0;
+    bluefs_transaction_t t;
+  } log;
+
+  struct {
+    ceph::mutex lock = ceph::make_mutex("BlueFS::dirty.lock");
+    uint64_t seq_stable = 0; //seq that is now stable on disk
+    uint64_t seq_live = 1;   //seq that is ongoing and dirty files will be written to
+    // map of dirty files, files of same dirty_seq are grouped into list.
+    std::map<uint64_t, dirty_file_list_t> files;
+    std::vector<interval_set<uint64_t>> pending_release; ///< extents to release
+    // TODO: it should be examined what makes pending_release immune to
+    // eras in a way similar to dirty_files. Hints:
+    // 1) we have actually only 2 eras: log_seq and log_seq+1
+    // 2) we usually not remove extents from files. And when we do, we force log-syncing.
+  } dirty;
+
+  ceph::condition_variable log_cond;                             ///< used for state control between log flush / log compaction
+  std::atomic<bool> log_is_compacting{false};                    ///< signals that bluefs log is already ongoing compaction
+  std::atomic<bool> log_forbidden_to_expand{false};              ///< used to signal that async compaction is in state
+                                                                 ///  that prohibits expansion of bluefs log
   /*
    * There are up to 3 block devices:
    *
@@ -313,21 +382,29 @@ private:
    *  BDEV_WAL  db.wal/  - a small, fast device, specifically for the WAL
    *  BDEV_SLOW db.slow/ - a big, slow device, to spill over to as BDEV_DB fills
    */
-  vector<BlockDevice*> bdev;                  ///< block devices we can use
-  vector<IOContext*> ioc;                     ///< IOContexts for bdevs
-  vector<interval_set<uint64_t> > block_all;  ///< extents in bdev we own
-  vector<Allocator*> alloc;                   ///< allocators for bdevs
-  vector<uint64_t> alloc_size;                ///< alloc size for each device
-  vector<interval_set<uint64_t>> pending_release; ///< extents to release
-  vector<interval_set<uint64_t>> block_unused_too_granular;
+  std::vector<BlockDevice*> bdev;                  ///< block devices we can use
+  std::vector<IOContext*> ioc;                     ///< IOContexts for bdevs
+  std::vector<uint64_t> block_reserved;            ///< starting reserve extent per device
+  std::vector<Allocator*> alloc;                   ///< allocators for bdevs
+  std::vector<uint64_t> alloc_size;                ///< alloc size for each device
+
+  //std::vector<interval_set<uint64_t>> block_unused_too_granular;
 
   BlockDevice::aio_callback_t discard_cb[3]; //discard callbacks for each dev
 
-  BlueFSDeviceExpander* slow_dev_expander = nullptr;
   std::unique_ptr<BlueFSVolumeSelector> vselector;
 
+  bluefs_shared_alloc_context_t* shared_alloc = nullptr;
+  unsigned shared_alloc_id = unsigned(-1);
+  inline bool is_shared_alloc(unsigned id) const {
+    return id == shared_alloc_id;
+  }
+  std::atomic<int64_t> cooldown_deadline = 0;
+
   class SocketHook;
   SocketHook* asok_hook = nullptr;
+  // used to trigger zeros into read (debug / verify)
+  std::atomic<uint64_t> inject_read_zeros{0};
 
   void _init_logger();
   void _shutdown_logger();
@@ -336,36 +413,58 @@ private:
   void _init_alloc();
   void _stop_alloc();
 
-  void _pad_bl(bufferlist& bl);  ///< pad bufferlist to block size w/ zeros
+  ///< pad ceph::buffer::list to max(block size, pad_size) w/ zeros
+  void _pad_bl(ceph::buffer::list& bl, uint64_t pad_size = 0);
+
+  uint64_t _get_used(unsigned id) const;
+  uint64_t _get_total(unsigned id) const;
+
 
   FileRef _get_file(uint64_t ino);
-  void _drop_link(FileRef f);
+  void _drop_link_D(FileRef f);
 
   unsigned _get_slow_device_id() {
     return bdev[BDEV_SLOW] ? BDEV_SLOW : BDEV_DB;
   }
   const char* get_device_name(unsigned id);
-  int _expand_slow_device(uint64_t min_size, PExtentVector& extents);
   int _allocate(uint8_t bdev, uint64_t len,
-               bluefs_fnode_t* node);
-  int _allocate_without_fallback(uint8_t id, uint64_t len,
-                                PExtentVector* extents);
-
-  int _flush_range(FileWriter *h, uint64_t offset, uint64_t length);
-  int _flush(FileWriter *h, bool focce, std::unique_lock<ceph::mutex>& l);
-  int _flush(FileWriter *h, bool force, bool *flushed = nullptr);
-  int _fsync(FileWriter *h, std::unique_lock<ceph::mutex>& l);
+                uint64_t alloc_unit,
+               bluefs_fnode_t* node,
+                size_t alloc_attempts = 0,
+                bool permit_dev_fallback = true);
+
+  /* signal replay log to include h->file in nearest log flush */
+  int _signal_dirty_to_log_D(FileWriter *h);
+  int _flush_range_F(FileWriter *h, uint64_t offset, uint64_t length);
+  int _flush_data(FileWriter *h, uint64_t offset, uint64_t length, bool buffered);
+  int _flush_F(FileWriter *h, bool force, bool *flushed = nullptr);
+  uint64_t _flush_special(FileWriter *h);
+  int _fsync(FileWriter *h);
 
 #ifdef HAVE_LIBAIO
-  void _claim_completed_aios(FileWriter *h, list<aio_t> *ls);
-  void wait_for_aio(FileWriter *h);  // safe to call without a lock
+  void _claim_completed_aios(FileWriter *h, std::list<aio_t> *ls);
+  void _wait_for_aio(FileWriter *h);  // safe to call without a lock
 #endif
 
-  int _flush_and_sync_log(std::unique_lock<ceph::mutex>& l,
-                         uint64_t want_seq = 0,
-                         uint64_t jump_to = 0);
-  uint64_t _estimate_log_size();
-  bool _should_compact_log();
+  int64_t _maybe_extend_log();
+  void _extend_log();
+  uint64_t _log_advance_seq();
+  void _consume_dirty(uint64_t seq);
+  void _clear_dirty_set_stable_D(uint64_t seq_stable);
+  void _release_pending_allocations(std::vector<interval_set<uint64_t>>& to_release);
+
+  void _flush_and_sync_log_core(int64_t available_runway);
+  int _flush_and_sync_log_jump_D(uint64_t jump_to,
+                              int64_t available_runway);
+  int _flush_and_sync_log_LD(uint64_t want_seq = 0);
+
+  uint64_t _estimate_transaction_size(bluefs_transaction_t* t);
+  uint64_t _make_initial_transaction(uint64_t start_seq,
+                                     bluefs_fnode_t& fnode,
+                                     uint64_t expected_final_size,
+                                     bufferlist* out);
+  uint64_t _estimate_log_size_N();
+  bool _should_start_compact_log_L_N();
 
   enum {
     REMOVE_DB = 1,
@@ -373,12 +472,15 @@ private:
     RENAME_SLOW2DB = 4,
     RENAME_DB2SLOW = 8,
   };
-  void _compact_log_dump_metadata(bluefs_transaction_t *t,
-                                 int flags);
-  void _compact_log_sync();
-  void _compact_log_async(std::unique_lock<ceph::mutex>& l);
+  void _compact_log_dump_metadata_NF(uint64_t start_seq,
+                                     bluefs_transaction_t *t,
+                                    int flags,
+                                    uint64_t capture_before_seq);
+
+  void _compact_log_sync_LNF_LD();
+  void _compact_log_async_LD_LNF_D();
 
-  void _rewrite_log_and_layout_sync(bool allocate_with_fallback,
+  void _rewrite_log_and_layout_sync_LNF_LD(bool permit_dev_fallback,
                                    int super_dev,
                                    int log_dev,
                                    int new_log_dev,
@@ -387,19 +489,18 @@ private:
 
   //void _aio_finish(void *priv);
 
-  void _flush_bdev_safely(FileWriter *h);
-  void flush_bdev();  // this is safe to call without a lock
-  void flush_bdev(std::array<bool, MAX_BDEV>& dirty_bdevs);  // this is safe to call without a lock
+  void _flush_bdev(FileWriter *h, bool check_mutex_locked = true);
+  void _flush_bdev();  // this is safe to call without a lock
+  void _flush_bdev(std::array<bool, MAX_BDEV>& dirty_bdevs);  // this is safe to call without a lock
 
   int _preallocate(FileRef f, uint64_t off, uint64_t len);
   int _truncate(FileWriter *h, uint64_t off);
 
   int64_t _read(
     FileReader *h,   ///< [in] read from here
-    FileReaderBuffer *buf, ///< [in] reader state
     uint64_t offset, ///< [in] offset
     size_t len,      ///< [in] this many bytes
-    bufferlist *outbl,   ///< [out] optional: reference the result here
+    ceph::buffer::list *outbl,   ///< [out] optional: reference the result here
     char *out);      ///< [out] optional: or copy it here
   int64_t _read_random(
     FileReader *h,   ///< [in] read from here
@@ -407,22 +508,20 @@ private:
     uint64_t len,    ///< [in] this many bytes
     char *out);      ///< [out] optional: or copy it here
 
-  void _invalidate_cache(FileRef f, uint64_t offset, uint64_t length);
-
   int _open_super();
   int _write_super(int dev);
-  int _check_new_allocations(const bluefs_fnode_t& fnode,
-    size_t dev_count,
-    boost::dynamic_bitset<uint64_t>* owned_blocks,
-    boost::dynamic_bitset<uint64_t>* used_blocks);
+  int _check_allocations(const bluefs_fnode_t& fnode,
+    boost::dynamic_bitset<uint64_t>* used_blocks,
+    bool is_alloc, //true when allocating, false when deallocating
+    const char* op_name);
   int _verify_alloc_granularity(
     __u8 id, uint64_t offset, uint64_t length,
+    uint64_t alloc_unit,
     const char *op);
-  int _adjust_granularity(
-    __u8 id, uint64_t *offset, uint64_t *length, bool alloc);
   int _replay(bool noop, bool to_stdout = false); ///< replay journal
 
   FileWriter *_create_writer(FileRef f);
+  void _drain_writer(FileWriter *h);
   void _close_writer(FileWriter *h);
 
   // always put the super in the second 4k block.  FIXME should this be
@@ -433,10 +532,11 @@ private:
   unsigned get_super_length() {
     return 4096;
   }
-
-  void _add_block_extent(unsigned bdev, uint64_t offset, uint64_t len,
-                         bool skip=false);
-
+  void _maybe_check_vselector_LNF() {
+    if (cct->_conf->bluefs_check_volume_selector_often) {
+      _check_vselector_LNF();
+    }
+  }
 public:
   BlueFS(CephContext* cct);
   ~BlueFS();
@@ -450,8 +550,8 @@ public:
   
   int log_dump();
 
-  void collect_metadata(map<string,string> *pm, unsigned skip_bdev_id);
-  void get_devices(set<string> *ls);
+  void collect_metadata(std::map<std::string,std::string> *pm, unsigned skip_bdev_id);
+  void get_devices(std::set<std::string> *ls);
   uint64_t get_alloc_size(int id) {
     return alloc_size[id];
   }
@@ -459,74 +559,68 @@ public:
 
   int device_migrate_to_new(
     CephContext *cct,
-    const set<int>& devs_source,
+    const std::set<int>& devs_source,
     int dev_target,
     const bluefs_layout_t& layout);
   int device_migrate_to_existing(
     CephContext *cct,
-    const set<int>& devs_source,
+    const std::set<int>& devs_source,
     int dev_target,
     const bluefs_layout_t& layout);
 
   uint64_t get_used();
   uint64_t get_total(unsigned id);
   uint64_t get_free(unsigned id);
-  void get_usage(vector<pair<uint64_t,uint64_t>> *usage); // [<free,total> ...]
-  void dump_perf_counters(Formatter *f);
+  uint64_t get_used(unsigned id);
+  void dump_perf_counters(ceph::Formatter *f);
 
-  void dump_block_extents(ostream& out);
+  void dump_block_extents(std::ostream& out);
 
   /// get current extents that we own for given block device
   int get_block_extents(unsigned id, interval_set<uint64_t> *extents);
 
   int open_for_write(
-    const string& dir,
-    const string& file,
+    std::string_view dir,
+    std::string_view file,
     FileWriter **h,
     bool overwrite);
 
   int open_for_read(
-    const string& dir,
-    const string& file,
+    std::string_view dir,
+    std::string_view file,
     FileReader **h,
     bool random = false);
 
-  void close_writer(FileWriter *h) {
-    std::lock_guard l(lock);
-    _close_writer(h);
-  }
+  // data added after last fsync() is lost
+  void close_writer(FileWriter *h);
 
-  int rename(const string& old_dir, const string& old_file,
-            const string& new_dir, const string& new_file);
+  int rename(std::string_view old_dir, std::string_view old_file,
+            std::string_view new_dir, std::string_view new_file);
 
-  int readdir(const string& dirname, vector<string> *ls);
+  int readdir(std::string_view dirname, std::vector<std::string> *ls);
 
-  int unlink(const string& dirname, const string& filename);
-  int mkdir(const string& dirname);
-  int rmdir(const string& dirname);
+  int unlink(std::string_view dirname, std::string_view filename);
+  int mkdir(std::string_view dirname);
+  int rmdir(std::string_view dirname);
   bool wal_is_rotational();
+  bool db_is_rotational();
 
-  bool dir_exists(const string& dirname);
-  int stat(const string& dirname, const string& filename,
+  bool dir_exists(std::string_view dirname);
+  int stat(std::string_view dirname, std::string_view filename,
           uint64_t *size, utime_t *mtime);
 
-  int lock_file(const string& dirname, const string& filename, FileLock **p);
+  int lock_file(std::string_view dirname, std::string_view filename, FileLock **p);
   int unlock_file(FileLock *l);
 
   void compact_log();
 
   /// sync any uncommitted state to disk
   void sync_metadata(bool avoid_compact);
-  /// test and compact log, if necessary
-  void _maybe_compact_log(std::unique_lock<ceph::mutex>& l);
 
-  void set_slow_device_expander(BlueFSDeviceExpander* a) {
-    slow_dev_expander = a;
-  }
   void set_volume_selector(BlueFSVolumeSelector* s) {
     vselector.reset(s);
   }
-  void dump_volume_selector(ostream& sout) {
+  void dump_volume_selector(std::ostream& sout) {
     vselector->dump(sout);
   }
   void get_vselector_paths(const std::string& base,
@@ -534,48 +628,26 @@ public:
     return vselector->get_paths(base, res);
   }
 
-  int add_block_device(unsigned bdev, const string& path, bool trim,
-                      bool shared_with_bluestore=false);
+  int add_block_device(unsigned bdev, const std::string& path, bool trim,
+                       uint64_t reserved,
+                      bluefs_shared_alloc_context_t* _shared_alloc = nullptr);
   bool bdev_support_label(unsigned id);
-  uint64_t get_block_device_size(unsigned bdev);
-
-  /// gift more block space
-  void add_block_extent(unsigned bdev, uint64_t offset, uint64_t len,
-                        bool skip=false) {
-    std::unique_lock l(lock);
-    _add_block_extent(bdev, offset, len, skip);
-    int r = _flush_and_sync_log(l);
-    ceph_assert(r == 0);
-  }
-
-  /// reclaim block space
-  int reclaim_blocks(unsigned bdev, uint64_t want,
-                    PExtentVector *extents);
+  uint64_t get_block_device_size(unsigned bdev) const;
 
   // handler for discard event
   void handle_discard(unsigned dev, interval_set<uint64_t>& to_release);
 
-  void flush(FileWriter *h, bool force = false) {
-    std::unique_lock l(lock);
-    int r = _flush(h, force, l);
-    ceph_assert(r == 0);
-  }
-  void flush_range(FileWriter *h, uint64_t offset, uint64_t length) {
-    std::lock_guard l(lock);
-    _flush_range(h, offset, length);
-  }
-  int fsync(FileWriter *h) {
-    std::unique_lock l(lock);
-    int r = _fsync(h, l);
-    _maybe_compact_log(l);
-    return r;
-  }
-  int64_t read(FileReader *h, FileReaderBuffer *buf, uint64_t offset, size_t len,
-          bufferlist *outbl, char *out) {
+  void flush(FileWriter *h, bool force = false);
+
+  void append_try_flush(FileWriter *h, const char* buf, size_t len);
+  void flush_range(FileWriter *h, uint64_t offset, uint64_t length);
+  int fsync(FileWriter *h);
+  int64_t read(FileReader *h, uint64_t offset, size_t len,
+          ceph::buffer::list *outbl, char *out) {
     // no need to hold the global lock here; we only touch h and
     // h->file, and read vs write or delete is already protected (via
     // atomics and asserts).
-    return _read(h, buf, offset, len, outbl, out);
+    return _read(h, offset, len, outbl, out);
   }
   int64_t read_random(FileReader *h, uint64_t offset, size_t len,
                  char *out) {
@@ -584,29 +656,38 @@ public:
     // atomics and asserts).
     return _read_random(h, offset, len, out);
   }
-  void invalidate_cache(FileRef f, uint64_t offset, uint64_t len) {
-    std::lock_guard l(lock);
-    _invalidate_cache(f, offset, len);
-  }
-  int preallocate(FileRef f, uint64_t offset, uint64_t len) {
-    std::lock_guard l(lock);
-    return _preallocate(f, offset, len);
-  }
-  int truncate(FileWriter *h, uint64_t offset) {
-    std::lock_guard l(lock);
-    return _truncate(h, offset);
-  }
-  int do_replay_recovery_read(FileReader *log,
-                             size_t log_pos,
-                             size_t read_offset,
-                             size_t read_len,
-                             bufferlist* bl);
+  void invalidate_cache(FileRef f, uint64_t offset, uint64_t len);
+  int preallocate(FileRef f, uint64_t offset, uint64_t len);
+  int truncate(FileWriter *h, uint64_t offset);
+
+  size_t probe_alloc_avail(int dev, uint64_t alloc_size);
 
   /// test purpose methods
-  void debug_inject_duplicate_gift(unsigned bdev, uint64_t offset, uint64_t len);
   const PerfCounters* get_perf_counters() const {
     return logger;
   }
+  uint64_t debug_get_dirty_seq(FileWriter *h);
+  bool debug_get_is_dev_dirty(FileWriter *h, uint8_t dev);
+
+private:
+  // Wrappers for BlockDevice::read(...) and BlockDevice::read_random(...)
+  // They are used for checking if read values are all 0, and reread if so.
+  int _read_and_check(uint8_t ndev, uint64_t off, uint64_t len,
+          ceph::buffer::list *pbl, IOContext *ioc, bool buffered);
+  int _read_random_and_check(uint8_t ndev, uint64_t off, uint64_t len, char *buf, bool buffered);
+
+  int _bdev_read(uint8_t ndev, uint64_t off, uint64_t len,
+    ceph::buffer::list* pbl, IOContext* ioc, bool buffered);
+  int _bdev_read_random(uint8_t ndev, uint64_t off, uint64_t len, char* buf, bool buffered);
+
+  /// test and compact log, if necessary
+  void _maybe_compact_log_LNF_NF_LD_D();
+  int _do_replay_recovery_read(FileReader *log,
+                              size_t log_pos,
+                              size_t read_offset,
+                              size_t read_len,
+                              bufferlist* bl);
+  void _check_vselector_LNF();
 };
 
 class OriginalVolumeSelector : public BlueFSVolumeSelector {
@@ -622,7 +703,7 @@ public:
     : wal_total(_wal_total), db_total(_db_total), slow_total(_slow_total) {}
 
   void* get_hint_for_log() const override;
-  void* get_hint_by_dir(const std::string& dirname) const override;
+  void* get_hint_by_dir(std::string_view dirname) const override;
 
   void add_usage(void* hint, const bluefs_fnode_t& fnode) override {
     // do nothing
@@ -643,7 +724,35 @@ public:
 
   uint8_t select_prefer_bdev(void* hint) override;
   void get_paths(const std::string& base, paths& res) const override;
-  void dump(ostream& sout) override;
+  void dump(std::ostream& sout) override;
 };
 
+class FitToFastVolumeSelector : public OriginalVolumeSelector {
+public:
+  FitToFastVolumeSelector(
+    uint64_t _wal_total,
+    uint64_t _db_total,
+    uint64_t _slow_total)
+    : OriginalVolumeSelector(_wal_total, _db_total, _slow_total) {}
+
+  void get_paths(const std::string& base, paths& res) const override;
+};
+/**
+ * Directional graph of locks.
+ * Vertices - Locks. Edges (directed) - locking progression.
+ * Edge A->B exist if last taken lock was A and next taken lock is B.
+ * 
+ * Row represents last lock taken.
+ * Column represents next lock taken.
+ *
+ *     >        | W | L | N | D | F
+ * -------------|---|---|---|---|---
+ * FileWriter W |   | > | > | > | >
+ * log        L |       | > | > | >
+ * nodes      N |           | > | >
+ * dirty      D |           |   | >
+ * File       F |
+ * 
+ * Claim: Deadlock is possible IFF graph contains cycles.
+ */
 #endif