]> git.proxmox.com Git - ceph.git/blobdiff - ceph/src/os/bluestore/BlueStore.cc
update sources to 12.2.10
[ceph.git] / ceph / src / os / bluestore / BlueStore.cc
index 47112cc70abad4933c8a5648a411416d808d09d1..b326d39bcd2ecaa9e3bf15ca3667abb5a0b5812c 100644 (file)
 #include "include/stringify.h"
 #include "common/errno.h"
 #include "common/safe_io.h"
+#include "common/PriorityCache.h"
 #include "Allocator.h"
 #include "FreelistManager.h"
 #include "BlueFS.h"
 #include "BlueRocksEnv.h"
 #include "auth/Crypto.h"
 #include "common/EventTrace.h"
+#include "perfglue/heap_profiler.h"
 
 #define dout_context cct
 #define dout_subsys ceph_subsys_bluestore
@@ -555,7 +557,7 @@ struct Int64ArrayMergeOperator : public KeyValueDB::MergeOperator {
   }
   // We use each operator name and each prefix to construct the
   // overall RocksDB operator name for consistency check at open time.
-  string name() const override {
+  const char *name() const override {
     return "int64_array";
   }
 };
@@ -777,78 +779,18 @@ BlueStore::Cache *BlueStore::Cache::create(CephContext* cct, string type,
   return c;
 }
 
-void BlueStore::Cache::trim_all()
+void BlueStore::Cache::trim(uint64_t onode_max, uint64_t buffer_max)
 {
   std::lock_guard<std::recursive_mutex> l(lock);
-  _trim(0, 0);
+  _trim(onode_max, buffer_max);
 }
 
-void BlueStore::Cache::trim(
-  uint64_t target_bytes,
-  float target_meta_ratio,
-  float target_data_ratio,
-  float bytes_per_onode)
+void BlueStore::Cache::trim_all()
 {
   std::lock_guard<std::recursive_mutex> l(lock);
-  uint64_t current_meta = _get_num_onodes() * bytes_per_onode;
-  uint64_t current_buffer = _get_buffer_bytes();
-  uint64_t current = current_meta + current_buffer;
-
-  uint64_t target_meta = target_bytes * target_meta_ratio;
-  uint64_t target_buffer = target_bytes * target_data_ratio;
-
-  // correct for overflow or float imprecision
-  target_meta = min(target_bytes, target_meta);
-  target_buffer = min(target_bytes - target_meta, target_buffer);
-
-  if (current <= target_bytes) {
-    dout(30) << __func__
-            << " shard target " << byte_u_t(target_bytes)
-            << " meta/data ratios " << target_meta_ratio
-            << " + " << target_data_ratio << " ("
-            << byte_u_t(target_meta) << " + "
-            << byte_u_t(target_buffer) << "), "
-            << " current " << byte_u_t(current) << " ("
-            << byte_u_t(current_meta) << " + "
-            << byte_u_t(current_buffer) << ")"
-            << dendl;
-    return;
-  }
-
-  uint64_t need_to_free = current - target_bytes;
-  uint64_t free_buffer = 0;
-  uint64_t free_meta = 0;
-  if (current_buffer > target_buffer) {
-    free_buffer = current_buffer - target_buffer;
-    if (free_buffer > need_to_free) {
-      free_buffer = need_to_free;
-    }
-  }
-  free_meta = need_to_free - free_buffer;
-
-  // start bounds at what we have now
-  uint64_t max_buffer = current_buffer - free_buffer;
-  uint64_t max_meta = current_meta - free_meta;
-  uint64_t max_onodes = max_meta / bytes_per_onode;
-
-  dout(20) << __func__
-          << " shard target " << byte_u_t(target_bytes)
-          << " ratio " << target_meta_ratio << " ("
-          << byte_u_t(target_meta) << " + "
-          << byte_u_t(target_buffer) << "), "
-          << " current " << byte_u_t(current) << " ("
-          << byte_u_t(current_meta) << " + "
-          << byte_u_t(current_buffer) << "),"
-          << " need_to_free " << byte_u_t(need_to_free) << " ("
-          << byte_u_t(free_meta) << " + "
-          << byte_u_t(free_buffer) << ")"
-          << " -> max " << max_onodes << " onodes + "
-          << max_buffer << " buffer"
-          << dendl;
-  _trim(max_onodes, max_buffer);
+  _trim(0, 0);
 }
 
-
 // LRUCache
 #undef dout_prefix
 #define dout_prefix *_dout << "bluestore.LRUCache(" << this << ") "
@@ -883,9 +825,10 @@ void BlueStore::LRUCache::_trim(uint64_t onode_max, uint64_t buffer_max)
   }
 
   // onodes
-  int num = onode_lru.size() - onode_max;
-  if (num <= 0)
+  if (onode_max >= onode_lru.size()) {
     return; // don't even try
+  }
+  uint64_t num = onode_lru.size() - onode_max;
 
   auto p = onode_lru.end();
   assert(p != onode_lru.begin());
@@ -1179,9 +1122,10 @@ void BlueStore::TwoQCache::_trim(uint64_t onode_max, uint64_t buffer_max)
   }
 
   // onodes
-  int num = onode_lru.size() - onode_max;
-  if (num <= 0)
+  if (onode_max >= onode_lru.size()) {
     return; // don't even try
+  }
+  uint64_t num = onode_lru.size() - onode_max;
 
   auto p = onode_lru.end();
   assert(p != onode_lru.begin());
@@ -1358,7 +1302,8 @@ void BlueStore::BufferSpace::read(
   uint32_t offset,
   uint32_t length,
   BlueStore::ready_regions_t& res,
-  interval_set<uint32_t>& res_intervals)
+  interval_set<uint32_t>& res_intervals,
+  int flags)
 {
   res.clear();
   res_intervals.clear();
@@ -1372,7 +1317,13 @@ void BlueStore::BufferSpace::read(
          ++i) {
       Buffer *b = i->second.get();
       assert(b->end() > offset);
-      if (b->is_writing() || b->is_clean()) {
+
+      bool val = false;
+      if (flags & BYPASS_CLEAN_CACHE)
+        val = b->is_writing();
+      else
+        val = b->is_writing() || b->is_clean();
+      if (val) {
         if (b->offset < offset) {
          uint32_t skip = offset - b->offset;
          uint32_t l = MIN(length, b->length - skip);
@@ -1668,8 +1619,10 @@ void BlueStore::SharedBlob::put()
       if (coll_snap != coll) {
        goto again;
       }
-      coll_snap->shared_blob_set.remove(this);
-
+      if (!coll_snap->shared_blob_set.remove(this, true)) {
+       // race with lookup
+       return;
+      }
       bc._clear(coll_snap->cache);
       coll_snap->cache->rm_blob();
     }
@@ -3337,35 +3290,58 @@ void BlueStore::Collection::split_cache(
 
 // =======================================================
 
+// MempoolThread
+
+#undef dout_prefix
+#define dout_prefix *_dout << "bluestore.MempoolThread(" << this << ") "
+
 void *BlueStore::MempoolThread::entry()
 {
   Mutex::Locker l(lock);
+
+  std::list<PriorityCache::PriCache *> caches;
+  caches.push_back(store->db);
+  caches.push_back(&meta_cache);
+  caches.push_back(&data_cache);
+  autotune_cache_size = store->osd_memory_cache_min;
+
+  utime_t next_balance = ceph_clock_now();
+  utime_t next_resize = ceph_clock_now();
+
+  bool interval_stats_trim = false;
+  bool interval_stats_resize = false; 
   while (!stop) {
-    uint64_t meta_bytes =
-      mempool::bluestore_cache_other::allocated_bytes() +
-      mempool::bluestore_cache_onode::allocated_bytes();
-    uint64_t onode_num =
-      mempool::bluestore_cache_onode::allocated_items();
+    _adjust_cache_settings();
 
-    if (onode_num < 2) {
-      onode_num = 2;
-    }
+    // Before we trim, check and see if it's time to rebalance/resize.
+    double autotune_interval = store->cache_autotune_interval;
+    double resize_interval = store->osd_memory_cache_resize_interval;
 
-    float bytes_per_onode = (float)meta_bytes / (float)onode_num;
-    size_t num_shards = store->cache_shards.size();
-    float target_ratio = store->cache_meta_ratio + store->cache_data_ratio;
-    // A little sloppy but should be close enough
-    uint64_t shard_target = target_ratio * (store->cache_size / num_shards);
+    if (autotune_interval > 0 && next_balance < ceph_clock_now()) {
+      // Log events at 5 instead of 20 when balance happens.
+      interval_stats_resize = true; 
+      interval_stats_trim = true;
+      if (store->cache_autotune) {
+        _balance_cache(caches);
+      }
 
-    for (auto i : store->cache_shards) {
-      i->trim(shard_target,
-             store->cache_meta_ratio,
-             store->cache_data_ratio,
-             bytes_per_onode);
+      next_balance = ceph_clock_now();
+      next_balance += autotune_interval;
+    }
+    if (resize_interval > 0 && next_resize < ceph_clock_now()) {
+      if (ceph_using_tcmalloc() && store->cache_autotune) {
+        _tune_cache_size(interval_stats_resize);
+        interval_stats_resize = false;
+      }
+      next_resize = ceph_clock_now();
+      next_resize += resize_interval;
     }
 
-    store->_update_cache_logger();
+    // Now Trim
+    _trim_shards(interval_stats_trim);
+    interval_stats_trim = false;
 
+    store->_update_cache_logger();
     utime_t wait;
     wait += store->cct->_conf->bluestore_cache_trim_interval;
     cond.WaitInterval(lock, wait);
@@ -3374,6 +3350,218 @@ void *BlueStore::MempoolThread::entry()
   return NULL;
 }
 
+void BlueStore::MempoolThread::_adjust_cache_settings()
+{
+  store->db->set_cache_ratio(store->cache_kv_ratio);
+  meta_cache.set_cache_ratio(store->cache_meta_ratio);
+  data_cache.set_cache_ratio(store->cache_data_ratio);
+}
+
+void BlueStore::MempoolThread::_trim_shards(bool interval_stats)
+{
+  auto cct = store->cct;
+  size_t num_shards = store->cache_shards.size();
+
+  int64_t kv_used = store->db->get_cache_usage();
+  int64_t meta_used = meta_cache._get_used_bytes();
+  int64_t data_used = data_cache._get_used_bytes();
+
+  uint64_t cache_size = store->cache_size;
+  int64_t kv_alloc =
+     static_cast<int64_t>(store->db->get_cache_ratio() * cache_size); 
+  int64_t meta_alloc =
+     static_cast<int64_t>(meta_cache.get_cache_ratio() * cache_size);
+  int64_t data_alloc =
+     static_cast<int64_t>(data_cache.get_cache_ratio() * cache_size);
+
+  if (store->cache_autotune) {
+    cache_size = autotune_cache_size;
+
+    kv_alloc = store->db->get_cache_bytes();
+    meta_alloc = meta_cache.get_cache_bytes();
+    data_alloc = data_cache.get_cache_bytes();
+  }
+  
+  if (interval_stats) {
+    ldout(cct, 5) << __func__  << " cache_size: " << cache_size
+                  << " kv_alloc: " << kv_alloc
+                  << " kv_used: " << kv_used
+                  << " meta_alloc: " << meta_alloc
+                  << " meta_used: " << meta_used
+                  << " data_alloc: " << data_alloc
+                  << " data_used: " << data_used << dendl;
+  } else {
+    ldout(cct, 20) << __func__  << " cache_size: " << cache_size
+                   << " kv_alloc: " << kv_alloc
+                   << " kv_used: " << kv_used
+                   << " meta_alloc: " << meta_alloc
+                   << " meta_used: " << meta_used
+                   << " data_alloc: " << data_alloc
+                   << " data_used: " << data_used << dendl;
+  }
+
+  uint64_t max_shard_onodes = static_cast<uint64_t>(
+      (meta_alloc / (double) num_shards) / meta_cache.get_bytes_per_onode());
+  uint64_t max_shard_buffer = static_cast<uint64_t>(data_alloc / num_shards);
+
+  ldout(cct, 30) << __func__ << " max_shard_onodes: " << max_shard_onodes
+                 << " max_shard_buffer: " << max_shard_buffer << dendl;
+
+  for (auto i : store->cache_shards) {
+    i->trim(max_shard_onodes, max_shard_buffer);
+  }
+}
+
+void BlueStore::MempoolThread::_tune_cache_size(bool interval_stats)
+{
+  auto cct = store->cct;
+  uint64_t target = store->osd_memory_target;
+  uint64_t base = store->osd_memory_base;
+  double fragmentation = store->osd_memory_expected_fragmentation;
+  uint64_t cache_max = ((1.0 - fragmentation) * target) - base;
+  uint64_t cache_min = store->osd_memory_cache_min;
+
+  size_t heap_size = 0;
+  size_t unmapped = 0;
+  uint64_t mapped = 0;
+
+  ceph_heap_release_free_memory();
+  ceph_heap_get_numeric_property("generic.heap_size", &heap_size);
+  ceph_heap_get_numeric_property("tcmalloc.pageheap_unmapped_bytes", &unmapped);
+  mapped = heap_size - unmapped;
+
+  uint64_t new_size = autotune_cache_size;
+  new_size = (new_size < cache_max) ? new_size : cache_max;
+  new_size = (new_size > cache_min) ? new_size : cache_min;
+
+  // Approach the min/max slowly, but bounce away quickly.
+  if ((uint64_t) mapped < target) {
+    double ratio = 1 - ((double) mapped / target);
+    new_size += ratio * (cache_max - new_size); 
+  } else {
+    double ratio = 1 - ((double) target / mapped);
+    new_size -= ratio * (new_size - cache_min);
+  }
+
+  if (interval_stats) {
+    ldout(cct, 5) << __func__
+                  << " target: " << target
+                  << " heap: " << heap_size
+                  << " unmapped: " << unmapped
+                  << " mapped: " << mapped 
+                  << " old cache_size: " << autotune_cache_size
+                  << " new cache size: " << new_size << dendl;
+  } else {
+    ldout(cct, 20) << __func__
+                   << " target: " << target
+                   << " heap: " << heap_size
+                   << " unmapped: " << unmapped
+                   << " mapped: " << mapped        
+                   << " old cache_size: " << autotune_cache_size
+                   << " new cache size: " << new_size << dendl;
+  }
+  autotune_cache_size = new_size;
+}
+
+void BlueStore::MempoolThread::_balance_cache(
+    const std::list<PriorityCache::PriCache *>& caches)
+{
+  int64_t mem_avail = autotune_cache_size;
+
+  // Assign memory for each priority level
+  for (int i = 0; i < PriorityCache::Priority::LAST + 1; i++) {
+    ldout(store->cct, 10) << __func__ << " assigning cache bytes for PRI: " << i << dendl;
+    PriorityCache::Priority pri = static_cast<PriorityCache::Priority>(i);
+    _balance_cache_pri(&mem_avail, caches, pri);
+  }
+  // Assign any leftover memory based on the default ratios.
+  if (mem_avail > 0) {
+    for (auto it = caches.begin(); it != caches.end(); it++) {
+      int64_t fair_share =
+          static_cast<int64_t>((*it)->get_cache_ratio() * mem_avail);
+      if (fair_share > 0) {
+        (*it)->add_cache_bytes(PriorityCache::Priority::LAST, fair_share);
+      }
+    }
+  }
+  // assert if we assigned more memory than is available.
+  assert(mem_avail >= 0);
+
+  // Finally commit the new cache sizes
+  for (auto it = caches.begin(); it != caches.end(); it++) {
+    (*it)->commit_cache_size();
+  }
+}
+
+void BlueStore::MempoolThread::_balance_cache_pri(int64_t *mem_avail,
+    const std::list<PriorityCache::PriCache *>& caches, PriorityCache::Priority pri) 
+{
+  std::list<PriorityCache::PriCache *> tmp_caches = caches;
+  double cur_ratios = 0;
+  double new_ratios = 0;
+
+  // Zero this priority's bytes, sum the initial ratios.
+  for (auto it = tmp_caches.begin(); it != tmp_caches.end(); it++) {
+    (*it)->set_cache_bytes(pri, 0);
+    cur_ratios += (*it)->get_cache_ratio();
+  }
+
+  // For this priority, loop until caches are satisified or we run out of memory.
+  // Since we can't allocate fractional bytes, stop if we have fewer bytes left
+  // than the number of participating caches.
+  while (!tmp_caches.empty() && *mem_avail > static_cast<int64_t>(tmp_caches.size())) {
+    uint64_t total_assigned = 0;
+
+    for (auto it = tmp_caches.begin(); it != tmp_caches.end(); ) {
+      int64_t cache_wants = (*it)->request_cache_bytes(pri, store->cache_autotune_chunk_size);
+
+      // Usually the ratio should be set to the fraction of the current caches'
+      // assigned ratio compared to the total ratio of all caches that still
+      // want memory.  There is a special case where the only caches left are
+      // all assigned 0% ratios but still want memory.  In that case, give 
+      // them an equal shot at the remaining memory for this priority.
+      double ratio = 1.0 / tmp_caches.size();
+      if (cur_ratios > 0) {
+        ratio = (*it)->get_cache_ratio() / cur_ratios;
+      }
+      int64_t fair_share = static_cast<int64_t>(*mem_avail * ratio);
+
+      if (cache_wants > fair_share) {
+        // If we want too much, take what we can get but stick around for more
+        (*it)->add_cache_bytes(pri, fair_share);
+        total_assigned += fair_share;
+
+        new_ratios += (*it)->get_cache_ratio();
+        ldout(store->cct, 20) << __func__ << " " << (*it)->get_cache_name() 
+                              << " wanted: " << cache_wants << " fair_share: " << fair_share
+                              << " mem_avail: " << *mem_avail
+                              << " staying in list.  Size: " << tmp_caches.size()
+                              << dendl;
+        ++it;
+      } else {
+        // Otherwise assign only what we want
+        if (cache_wants > 0) { 
+          (*it)->add_cache_bytes(pri, cache_wants);
+          total_assigned += cache_wants;
+
+          ldout(store->cct, 20) << __func__ << " " << (*it)->get_cache_name()
+                                << " wanted: " << cache_wants << " fair_share: " << fair_share
+                                << " mem_avail: " << *mem_avail
+                                << " removing from list.  New size: " << tmp_caches.size() - 1
+                                << dendl;
+
+        }
+        // Either the cache didn't want anything or got what it wanted, so remove it from the tmp list. 
+        it = tmp_caches.erase(it);
+      }
+    }
+    // Reset the ratios 
+    *mem_avail -= total_assigned;
+    cur_ratios = new_ratios;
+    new_ratios = 0;
+  }
+}
+
 // =======================================================
 
 // OmapIteratorImpl
@@ -3753,6 +3941,19 @@ void BlueStore::_set_finisher_num()
 int BlueStore::_set_cache_sizes()
 {
   assert(bdev);
+  cache_autotune = cct->_conf->get_val<bool>("bluestore_cache_autotune");
+  cache_autotune_chunk_size = 
+      cct->_conf->get_val<uint64_t>("bluestore_cache_autotune_chunk_size");
+  cache_autotune_interval =
+      cct->_conf->get_val<double>("bluestore_cache_autotune_interval");
+  osd_memory_target = cct->_conf->get_val<uint64_t>("osd_memory_target");
+  osd_memory_base = cct->_conf->get_val<uint64_t>("osd_memory_base");
+  osd_memory_expected_fragmentation =
+      cct->_conf->get_val<double>("osd_memory_expected_fragmentation");
+  osd_memory_cache_min = cct->_conf->get_val<uint64_t>("osd_memory_cache_min");
+  osd_memory_cache_resize_interval = 
+      cct->_conf->get_val<double>("osd_memory_cache_resize_interval");
+
   if (cct->_conf->bluestore_cache_size) {
     cache_size = cct->_conf->bluestore_cache_size;
   } else {
@@ -3763,48 +3964,36 @@ int BlueStore::_set_cache_sizes()
       cache_size = cct->_conf->bluestore_cache_size_ssd;
     }
   }
-  cache_meta_ratio = cct->_conf->bluestore_cache_meta_ratio;
-  cache_kv_ratio = cct->_conf->bluestore_cache_kv_ratio;
-
-  double cache_kv_max = cct->_conf->bluestore_cache_kv_max;
-  double cache_kv_max_ratio = 0;
-
-  // if cache_kv_max is negative, disable it
-  if (cache_size > 0 && cache_kv_max >= 0) {
-    cache_kv_max_ratio = (double) cache_kv_max / (double) cache_size;
-    if (cache_kv_max_ratio < 1.0 && cache_kv_max_ratio < cache_kv_ratio) {
-      dout(1) << __func__ << " max " << cache_kv_max_ratio
-            << " < ratio " << cache_kv_ratio
-            << dendl;
-      cache_meta_ratio = cache_meta_ratio + cache_kv_ratio - cache_kv_max_ratio;
-      cache_kv_ratio = cache_kv_max_ratio;
-    }
-  }  
-
-  cache_data_ratio =
-    (double)1.0 - (double)cache_meta_ratio - (double)cache_kv_ratio;
 
+  cache_meta_ratio = cct->_conf->bluestore_cache_meta_ratio;
   if (cache_meta_ratio < 0 || cache_meta_ratio > 1.0) {
     derr << __func__ << " bluestore_cache_meta_ratio (" << cache_meta_ratio
-        << ") must be in range [0,1.0]" << dendl;
+         << ") must be in range [0,1.0]" << dendl;
     return -EINVAL;
   }
+
+  cache_kv_ratio = cct->_conf->bluestore_cache_kv_ratio;
   if (cache_kv_ratio < 0 || cache_kv_ratio > 1.0) {
     derr << __func__ << " bluestore_cache_kv_ratio (" << cache_kv_ratio
-        << ") must be in range [0,1.0]" << dendl;
+         << ") must be in range [0,1.0]" << dendl;
     return -EINVAL;
   }
+
   if (cache_meta_ratio + cache_kv_ratio > 1.0) {
     derr << __func__ << " bluestore_cache_meta_ratio (" << cache_meta_ratio
-        << ") + bluestore_cache_kv_ratio (" << cache_kv_ratio
-        << ") = " << cache_meta_ratio + cache_kv_ratio << "; must be <= 1.0"
-        << dendl;
+         << ") + bluestore_cache_kv_ratio (" << cache_kv_ratio
+         << ") = " << cache_meta_ratio + cache_kv_ratio << "; must be <= 1.0"
+         << dendl;
     return -EINVAL;
   }
+
+  cache_data_ratio =
+    (double)1.0 - (double)cache_meta_ratio - (double)cache_kv_ratio;
   if (cache_data_ratio < 0) {
     // deal with floating point imprecision
     cache_data_ratio = 0;
   }
+    
   dout(1) << __func__ << " cache_size " << cache_size
           << " meta " << cache_meta_ratio
          << " kv " << cache_kv_ratio
@@ -3942,9 +4131,9 @@ void BlueStore::_init_logger()
            "Number of buffers in cache");
   b.add_u64(l_bluestore_buffer_bytes, "bluestore_buffer_bytes",
            "Number of buffer bytes in cache", NULL, 0, unit_t(BYTES));
-  b.add_u64(l_bluestore_buffer_hit_bytes, "bluestore_buffer_hit_bytes",
+  b.add_u64_counter(l_bluestore_buffer_hit_bytes, "bluestore_buffer_hit_bytes",
            "Sum for bytes of read hit in the cache", NULL, 0, unit_t(BYTES));
-  b.add_u64(l_bluestore_buffer_miss_bytes, "bluestore_buffer_miss_bytes",
+  b.add_u64_counter(l_bluestore_buffer_miss_bytes, "bluestore_buffer_miss_bytes",
            "Sum for bytes of read missed in the cache", NULL, 0, unit_t(BYTES));
 
   b.add_u64_counter(l_bluestore_write_big, "bluestore_write_big",
@@ -4027,7 +4216,7 @@ int BlueStore::_open_path()
     return -EINVAL;
   }
   assert(path_fd < 0);
-  path_fd = TEMP_FAILURE_RETRY(::open(path.c_str(), O_DIRECTORY));
+  path_fd = TEMP_FAILURE_RETRY(::open(path.c_str(), O_DIRECTORY|O_CLOEXEC));
   if (path_fd < 0) {
     int r = -errno;
     derr << __func__ << " unable to open " << path << ": " << cpp_strerror(r)
@@ -4056,7 +4245,7 @@ int BlueStore::_write_bdev_label(CephContext *cct,
   z.zero();
   bl.append(std::move(z));
 
-  int fd = TEMP_FAILURE_RETRY(::open(path.c_str(), O_WRONLY));
+  int fd = TEMP_FAILURE_RETRY(::open(path.c_str(), O_WRONLY|O_CLOEXEC));
   if (fd < 0) {
     fd = -errno;
     derr << __func__ << " failed to open " << path << ": " << cpp_strerror(fd)
@@ -4081,7 +4270,7 @@ int BlueStore::_read_bdev_label(CephContext* cct, string path,
                                bluestore_bdev_label_t *label)
 {
   dout(10) << __func__ << dendl;
-  int fd = TEMP_FAILURE_RETRY(::open(path.c_str(), O_RDONLY));
+  int fd = TEMP_FAILURE_RETRY(::open(path.c_str(), O_RDONLY|O_CLOEXEC));
   if (fd < 0) {
     fd = -errno;
     derr << __func__ << " failed to open " << path << ": " << cpp_strerror(fd)
@@ -4381,7 +4570,7 @@ void BlueStore::_close_alloc()
 int BlueStore::_open_fsid(bool create)
 {
   assert(fsid_fd < 0);
-  int flags = O_RDWR;
+  int flags = O_RDWR|O_CLOEXEC;
   if (create)
     flags |= O_CREAT;
   fsid_fd = ::openat(path_fd, "fsid", flags, 0644);
@@ -4763,6 +4952,7 @@ int BlueStore::_open_db(bool create)
     }
   }
 
+
   db = KeyValueDB::create(cct,
                          kv_backend,
                          fn,
@@ -4783,8 +4973,7 @@ int BlueStore::_open_db(bool create)
 
   FreelistManager::setup_merge_operators(db);
   db->set_merge_operator(PREFIX_STAT, merge_op);
-
-  db->set_cache_size(cache_size * cache_kv_ratio);
+  db->set_cache_size(cache_kv_ratio * cache_size);
 
   if (kv_backend == "rocksdb")
     options = cct->_conf->bluestore_rocksdb_options;
@@ -5085,7 +5274,7 @@ int BlueStore::_setup_block_symlink_or_file(
   dout(20) << __func__ << " name " << name << " path " << epath
           << " size " << size << " create=" << (int)create << dendl;
   int r = 0;
-  int flags = O_RDWR;
+  int flags = O_RDWR|O_CLOEXEC;
   if (create)
     flags |= O_CREAT;
   if (epath.length()) {
@@ -6513,6 +6702,7 @@ int BlueStore::_do_read(
 {
   FUNCTRACE();
   int r = 0;
+  int read_cache_policy = 0; // do not bypass clean or dirty cache
 
   dout(20) << __func__ << " 0x" << std::hex << offset << "~" << length
            << " size 0x" << o->onode.size << " (" << std::dec
@@ -6547,6 +6737,13 @@ int BlueStore::_do_read(
 
   ready_regions_t ready_regions;
 
+  // for deep-scrub, we only read dirty cache and bypass clean cache in
+  // order to read underlying block device in case there are silent disk errors.
+  if (op_flags & CEPH_OSD_OP_FLAG_BYPASS_CLEAN_CACHE) {
+    dout(20) << __func__ << " will bypass cache and do direct read" << dendl;
+    read_cache_policy = BufferSpace::BYPASS_CLEAN_CACHE;
+  }
+
   // build blob-wise list to of stuff read (that isn't cached)
   blobs2read_t blobs2read;
   unsigned left = length;
@@ -6572,7 +6769,8 @@ int BlueStore::_do_read(
     ready_regions_t cache_res;
     interval_set<uint32_t> cache_interval;
     bptr->shared_blob->bc.read(
-      bptr->shared_blob->get_cache(), b_off, b_len, cache_res, cache_interval);
+      bptr->shared_blob->get_cache(), b_off, b_len, cache_res, cache_interval,
+      read_cache_policy);
     dout(20) << __func__ << "  blob " << *bptr << std::hex
             << " need 0x" << b_off << "~" << b_len
             << " cache has 0x" << cache_interval