]> git.apps.os.sepia.ceph.com Git - ceph-ci.git/commitdiff
mon/OSDMonitor: Use generic priority cache tuner for mon caches
authorSridhar Seshasayee <sseshasa@redhat.com>
Wed, 15 May 2019 09:56:52 +0000 (15:26 +0530)
committerSridhar Seshasayee <sseshasa@redhat.com>
Tue, 6 Aug 2019 14:52:16 +0000 (20:22 +0530)
Use priority cache manager to tune inc, full and rocksdb caches.

Signed-off-by: Sridhar Seshasayee <sseshasa@redhat.com>
doc/rados/configuration/mon-config-ref.rst
src/common/legacy_config_opts.h
src/common/options.cc
src/common/simple_cache.hpp
src/mon/MonitorDBStore.h
src/mon/OSDMonitor.cc
src/mon/OSDMonitor.h
src/os/filestore/DBObjectMap.h

index fd10763f30fe03ba657c6763c5cbf127b405cfcd..7a0c1c34d6c56a2eb11a6a0a00c6a4a3bc7cd4f9 100644 (file)
@@ -1196,6 +1196,26 @@ Miscellaneous
 :Type: Integer
 :Default: 300
 
+``mon osd cache size min``
+
+:Description: The minimum amount of bytes to be kept mapped in memory for osd
+               monitor caches.
+:Type: 64-bit Integer
+:Default: 134217728
+
+``mon memory target``
+
+:Description: The amount of bytes pertaining to osd monitor caches and kv cache
+              to be kept mapped in memory with cache auto-tuning enabled.
+:Type: 64-bit Integer
+:Default: 2147483648
+
+``mon memory autotune``
+
+:Description: Autotune the cache memory being used for osd monitors and kv
+              database.
+:Type: Boolean
+:Default: True
 
 
 .. _Paxos: https://en.wikipedia.org/wiki/Paxos_(computer_science)
index df13b12c80f53ca969d78e3bbfda75d3cecda3eb..55ebc175f2938079fb3c92b45ea3d79c0b3fb05d 100644 (file)
@@ -190,6 +190,9 @@ OPTION(mon_compact_on_bootstrap, OPT_BOOL)  // trigger leveldb compaction on boo
 OPTION(mon_compact_on_trim, OPT_BOOL)       // compact (a prefix) when we trim old states
 OPTION(mon_osd_cache_size, OPT_INT)  // the size of osdmaps cache, not to rely on underlying store's cache
 
+OPTION(mon_osd_cache_size_min, OPT_U64) // minimum amount of memory to cache osdmaps
+OPTION(mon_memory_target, OPT_U64) // amount of mapped memory for osdmaps
+OPTION(mon_memory_autotune, OPT_BOOL) // autotune cache memory for osdmap
 OPTION(mon_cpu_threads, OPT_INT)
 OPTION(mon_osd_mapping_pgs_per_chunk, OPT_INT)
 OPTION(mon_clean_pg_upmaps_per_chunk, OPT_U64)
index 989a2cea68738d48ab7c60046ec62351a0fec144..3fd650183bd58f784aa5f229ceb0443037b359f4 100644 (file)
@@ -1330,6 +1330,23 @@ std::vector<Option> get_global_options() {
     .add_service("mon")
     .set_description("maximum number of OSDMaps to cache in memory"),
 
+    Option("mon_osd_cache_size_min", Option::TYPE_SIZE, Option::LEVEL_ADVANCED)
+    .set_default(128_M)
+    .add_service("mon")
+    .set_description("The minimum amount of bytes to be kept mapped in memory for osd monitor caches."),
+
+    Option("mon_memory_target", Option::TYPE_SIZE, Option::LEVEL_BASIC)
+    .set_default(2_G)
+    .set_flag(Option::FLAG_RUNTIME)
+    .add_service("mon")
+    .set_description("The amount of bytes pertaining to osd monitor caches and kv cache to be kept mapped in memory with cache auto-tuning enabled"),
+
+    Option("mon_memory_autotune", Option::TYPE_BOOL, Option::LEVEL_BASIC)
+    .set_default(true)
+    .set_flag(Option::FLAG_RUNTIME)
+    .add_service("mon")
+    .set_description("Autotune the cache memory being used for osd monitors and kv database"),
+
     Option("mon_cpu_threads", Option::TYPE_INT, Option::LEVEL_ADVANCED)
     .set_default(4)
     .add_service("mon")
@@ -3913,6 +3930,7 @@ std::vector<Option> get_global_options() {
 
     Option("rocksdb_cache_size", Option::TYPE_SIZE, Option::LEVEL_ADVANCED)
     .set_default(512_M)
+    .set_flag(Option::FLAG_RUNTIME)
     .set_description(""),
 
     Option("rocksdb_cache_row_ratio", Option::TYPE_FLOAT, Option::LEVEL_ADVANCED)
index 77905e929ef17d927e0911c8448a7f7804d99758..6e8a452eba91ae53879388a1583be209eb1d024d 100644 (file)
@@ -22,6 +22,8 @@ template <class K, class V, class C = std::less<K>, class H = std::hash<K> >
 class SimpleLRU {
   ceph::mutex lock = ceph::make_mutex("SimpleLRU::lock");
   size_t max_size;
+  size_t max_bytes = 0;
+  size_t total_bytes = 0;
   ceph::unordered_map<K, typename list<pair<K, V> >::iterator, H> contents;
   list<pair<K, V> > lru;
   map<K, V, C> pinned;
@@ -33,12 +35,26 @@ class SimpleLRU {
     }
   }
 
+  void trim_cache_bytes() {
+    while(total_bytes > max_bytes) {
+      total_bytes -= lru.back().second.length();
+      contents.erase(lru.back().first);
+      lru.pop_back();
+    }
+  }
+
   void _add(K key, V&& value) {
     lru.emplace_front(key, std::move(value)); // can't move key because we access it below
     contents[key] = lru.begin();
     trim_cache();
   }
 
+  void _add_bytes(K key, V&& value) {
+    lru.emplace_front(key, std::move(value)); // can't move key because we access it below
+    contents[key] = lru.begin();
+    trim_cache_bytes();
+  }
+
 public:
   SimpleLRU(size_t max_size) : max_size(max_size) {
     contents.rehash(max_size);
@@ -69,6 +85,7 @@ public:
       contents.find(key);
     if (i == contents.end())
       return;
+    total_bytes -= i->second->second.length();
     lru.erase(i->second);
     contents.erase(i);
   }
@@ -79,6 +96,22 @@ public:
     trim_cache();
   }
 
+  size_t get_size() {
+    std::lock_guard l(lock);
+    return contents.size();
+  }
+
+  void set_bytes(size_t num_bytes) {
+    std::lock_guard l(lock);
+    max_bytes = num_bytes;
+    trim_cache_bytes();
+  }
+
+  size_t get_bytes() {
+    std::lock_guard l(lock);
+    return total_bytes;
+  }
+
   bool lookup(K key, V *out) {
     std::lock_guard l(lock);
     typename ceph::unordered_map<K, typename list<pair<K, V> >::iterator, H>::iterator i =
@@ -100,6 +133,12 @@ public:
     std::lock_guard l(lock);
     _add(std::move(key), std::move(value));
   }
+
+  void add_bytes(K key, V value) {
+    std::lock_guard l(lock);
+    total_bytes += value.length();
+    _add_bytes(std::move(key), std::move(value));
+  }
 };
 
 #endif
index 0561dadf8c8d2748c001211ea8f5b12de685a7a7..b285fff48d0b1dc29715bad95f147d3336429292 100644 (file)
@@ -30,6 +30,7 @@
 #include "common/debug.h"
 #include "common/safe_io.h"
 #include "common/blkdev.h"
+#include "common/PriorityCache.h"
 
 #define dout_context g_ceph_context
 
@@ -56,6 +57,10 @@ class MonitorDBStore
     return devname;
   }
 
+  std::shared_ptr<PriorityCache::PriCache> get_priority_cache() const {
+    return db->get_priority_cache();
+  }
+
   struct Op {
     uint8_t type;
     string prefix;
index 64430dc55293391517df7b2f3055f9d16833c800..fdcf7dde5d854d4a8f3f40138ac8508388da4b6d 100644 (file)
@@ -63,6 +63,7 @@
 #include "common/Timer.h"
 #include "common/ceph_argparse.h"
 #include "common/perf_counters.h"
+#include "common/PriorityCache.h"
 #include "common/strtol.h"
 #include "common/numa.h"
 
@@ -81,6 +82,7 @@
 #include "include/str_list.h"
 #include "include/str_map.h"
 #include "include/scope_guard.h"
+#include "perfglue/heap_profiler.h"
 
 #include "auth/cephx/CephxKeyServer.h"
 #include "osd/OSDCap.h"
@@ -125,6 +127,105 @@ static const string OSD_SNAP_PREFIX("osd_snap");
 
 namespace {
 
+struct OSDMemCache : public PriorityCache::PriCache {
+  OSDMonitor *osdmon;
+  int64_t cache_bytes[PriorityCache::Priority::LAST+1] = {0};
+  int64_t committed_bytes = 0;
+  double cache_ratio = 0;
+
+  OSDMemCache(OSDMonitor *m) : osdmon(m) {};
+
+  virtual uint64_t _get_used_bytes() const = 0;
+
+  virtual int64_t request_cache_bytes(
+      PriorityCache::Priority pri, uint64_t total_cache) const {
+    int64_t assigned = get_cache_bytes(pri);
+
+    switch (pri) {
+    // All cache items are currently set to have PRI1 priority
+    case PriorityCache::Priority::PRI1:
+      {
+        int64_t request = _get_used_bytes();
+        return (request > assigned) ? request - assigned : 0;
+      }
+    default:
+      break;
+    }
+    return -EOPNOTSUPP;
+  }
+
+  virtual int64_t get_cache_bytes(PriorityCache::Priority pri) const {
+      return cache_bytes[pri];
+  }
+
+  virtual int64_t get_cache_bytes() const {
+    int64_t total = 0;
+
+    for (int i = 0; i < PriorityCache::Priority::LAST + 1; i++) {
+      PriorityCache::Priority pri = static_cast<PriorityCache::Priority>(i);
+      total += get_cache_bytes(pri);
+    }
+    return total;
+  }
+
+  virtual void set_cache_bytes(PriorityCache::Priority pri, int64_t bytes) {
+    cache_bytes[pri] = bytes;
+  }
+  virtual void add_cache_bytes(PriorityCache::Priority pri, int64_t bytes) {
+    cache_bytes[pri] += bytes;
+  }
+  virtual int64_t commit_cache_size(uint64_t total_cache) {
+    committed_bytes = PriorityCache::get_chunk(
+        get_cache_bytes(), total_cache);
+    return committed_bytes;
+  }
+  virtual int64_t get_committed_size() const {
+    return committed_bytes;
+  }
+  virtual double get_cache_ratio() const {
+    return cache_ratio;
+  }
+  virtual void set_cache_ratio(double ratio) {
+    cache_ratio = ratio;
+  }
+  virtual string get_cache_name() const = 0;
+};
+
+struct IncCache : public OSDMemCache {
+  IncCache(OSDMonitor *m) : OSDMemCache(m) {};
+
+  virtual uint64_t _get_used_bytes() const {
+    return osdmon->inc_osd_cache.get_bytes();
+  }
+
+  virtual string get_cache_name() const {
+    return "OSDMap Inc Cache";
+  }
+
+  uint64_t _get_num_osdmaps() const {
+    return osdmon->inc_osd_cache.get_size();
+  }
+};
+
+struct FullCache : public OSDMemCache {
+  FullCache(OSDMonitor *m) : OSDMemCache(m) {};
+
+  virtual uint64_t _get_used_bytes() const {
+    return osdmon->full_osd_cache.get_bytes();
+  }
+
+  virtual string get_cache_name() const {
+    return "OSDMap Full Cache";
+  }
+
+  uint64_t _get_num_osdmaps() const {
+    return osdmon->full_osd_cache.get_size();
+  }
+};
+
+std::shared_ptr<IncCache> inc_cache;
+std::shared_ptr<FullCache> full_cache;
+
 const uint32_t MAX_POOL_APPLICATIONS = 4;
 const uint32_t MAX_POOL_APPLICATION_KEYS = 64;
 const uint32_t MAX_POOL_APPLICATION_LENGTH = 128;
@@ -313,7 +414,42 @@ OSDMonitor::OSDMonitor(
    full_osd_cache(g_conf()->mon_osd_cache_size),
    has_osdmap_manifest(false),
    mapper(mn->cct, &mn->cpu_tp)
-{}
+{
+  inc_cache = std::make_shared<IncCache>(this);
+  full_cache = std::make_shared<FullCache>(this);
+  int r = _set_cache_sizes();
+  if (r < 0) {
+    derr << __func__ << " using default osd cache size - mon_osd_cache_size ("
+         << g_conf()->mon_osd_cache_size
+         << ") without priority cache management"
+         << dendl;
+  }
+}
+
+int OSDMonitor::_set_cache_sizes()
+{
+  if (g_conf()->mon_memory_autotune) {
+    // set the new osdmon cache targets to be managed by pcm
+    mon_osd_cache_size = g_conf()->mon_osd_cache_size;
+    rocksdb_cache_size = g_conf()->rocksdb_cache_size;
+    mon_memory_base = cct->_conf.get_val<Option::size_t>("osd_memory_base");
+    mon_memory_fragmentation = cct->_conf.get_val<double>("osd_memory_expected_fragmentation");
+    mon_memory_target = g_conf()->mon_memory_target;
+    mon_memory_min = g_conf()->mon_osd_cache_size_min;
+    if (mon_memory_target <= 0 || mon_memory_min <= 0) {
+      derr << __func__ << " mon_memory_target:" << mon_memory_target
+           << " mon_memory_min:" << mon_memory_min
+           << ". Invalid size option(s) provided."
+           << dendl;
+      return -EINVAL;
+    }
+    // Set the initial inc and full LRU cache sizes
+    inc_osd_cache.set_bytes(mon_memory_min);
+    full_osd_cache.set_bytes(mon_memory_min);
+    mon_memory_autotune = g_conf()->mon_memory_autotune;
+  }
+  return 0;
+}
 
 bool OSDMonitor::_have_pending_crush()
 {
@@ -505,6 +641,17 @@ void OSDMonitor::update_from_paxos(bool *need_bootstrap)
     int err = get_version(osdmap.epoch+1, inc_bl);
     ceph_assert(err == 0);
     ceph_assert(inc_bl.length());
+    // set priority cache manager levels if the osdmap is
+    // being populated for the first time.
+    if (mon_memory_autotune && pcm == nullptr) {
+      int r = register_cache_with_pcm();
+      if (r < 0) {
+        dout(10) << __func__
+                 << " Error while registering osdmon caches with pcm."
+                 << " Proceeding without cache auto tuning."
+                 << dendl;
+      }
+    }
 
     dout(7) << "update_from_paxos  applying incremental " << osdmap.epoch+1
            << dendl;
@@ -632,6 +779,78 @@ void OSDMonitor::update_from_paxos(bool *need_bootstrap)
   }
 }
 
+int OSDMonitor::register_cache_with_pcm()
+{
+  if (mon_memory_target <= 0 || mon_memory_min <= 0) {
+    derr << __func__ << " Invalid memory size specified for mon caches."
+         << " Caches will not be auto-tuned."
+         << dendl;
+    return -EINVAL;
+  }
+  uint64_t base = mon_memory_base;
+  double fragmentation = mon_memory_fragmentation;
+  // For calculating total target memory, consider rocksdb cache size.
+  uint64_t target = mon_memory_target;
+  uint64_t min = mon_memory_min;
+  uint64_t max = min;
+
+  // Apply the same logic as in bluestore to set the max amount
+  // of memory to use for cache. Assume base memory for OSDMaps
+  // and then add in some overhead for fragmentation.
+  uint64_t ltarget = (1.0 - fragmentation) * target;
+  if (ltarget > base + min) {
+    max = ltarget - base;
+  }
+
+  rocksdb_binned_kv_cache = mon->store->get_priority_cache();
+  ceph_assert(rocksdb_binned_kv_cache);
+
+  int r = _set_cache_ratios();
+  if (r < 0) {
+    derr << __func__ << " Cache ratios for pcm could not be set."
+         << " Review the kv (rocksdb) and mon_memory_target sizes."
+         << dendl;
+    return -EINVAL;
+  }
+
+  pcm = std::make_shared<PriorityCache::Manager>(
+      cct, min, max, target, true);
+  pcm->insert("kv", rocksdb_binned_kv_cache, true);
+  pcm->insert("inc", inc_cache, true);
+  pcm->insert("full", full_cache, true);
+  dout(10) << __func__ << " pcm target: " << target
+           << " pcm max: " << max
+           << " pcm min: " << min
+           << " inc_osd_cache size: " << inc_osd_cache.get_size()
+           << dendl;
+  return 0;
+}
+
+int OSDMonitor::_set_cache_ratios()
+{
+  double old_cache_kv_ratio = cache_kv_ratio;
+
+  // Set the cache ratios for kv(rocksdb), inc and full caches
+  cache_kv_ratio = (double)rocksdb_cache_size / (double)mon_memory_target;
+  if (cache_kv_ratio >= 1.0) {
+    derr << __func__ << " Cache kv ratio (" << cache_kv_ratio
+         << ") must be in range [0,<1.0]."
+         << dendl;
+    cache_kv_ratio = old_cache_kv_ratio;
+    return -EINVAL;
+  }
+  rocksdb_binned_kv_cache->set_cache_ratio(cache_kv_ratio);
+  cache_inc_ratio = cache_full_ratio = (1.0 - cache_kv_ratio) / 2;
+  inc_cache->set_cache_ratio(cache_inc_ratio);
+  full_cache->set_cache_ratio(cache_full_ratio);
+
+  dout(10) << __func__ << " kv ratio " << cache_kv_ratio
+           << " inc ratio " << cache_inc_ratio
+           << " full ratio " << cache_full_ratio
+           << dendl;
+  return 0;
+}
+
 void OSDMonitor::start_mapping()
 {
   // initiate mapping job
@@ -4146,7 +4365,7 @@ int OSDMonitor::get_version(version_t ver, uint64_t features, bufferlist& bl)
       OSDMap::get_significant_features(mon->get_quorum_con_features())) {
     reencode_incremental_map(bl, features);
   }
-  inc_osd_cache.add({ver, significant_features}, bl);
+  inc_osd_cache.add_bytes({ver, significant_features}, bl);
   return 0;
 }
 
@@ -4295,7 +4514,7 @@ int OSDMonitor::get_version_full(version_t ver, uint64_t features,
       OSDMap::get_significant_features(mon->get_quorum_con_features())) {
     reencode_full_map(bl, features);
   }
-  full_osd_cache.add({ver, significant_features}, bl);
+  full_osd_cache.add_bytes({ver, significant_features}, bl);
   return 0;
 }
 
@@ -4733,6 +4952,48 @@ void OSDMonitor::tick()
   if (do_propose ||
       !pending_inc.new_pg_temp.empty())  // also propose if we adjusted pg_temp
     propose_pending();
+
+  if (ceph_using_tcmalloc() && pcm != nullptr) {
+    pcm->tune_memory();
+    pcm->balance();
+    _set_new_cache_sizes();
+    dout(10) << "tick balancer "
+             << " inc cache_bytes: " << inc_cache->get_cache_bytes()
+             << " inc comtd_bytes: " << inc_cache->get_committed_size()
+             << " inc used_bytes: " << inc_cache->_get_used_bytes()
+             << " inc num_osdmaps: " << inc_cache->_get_num_osdmaps()
+             << dendl;
+    dout(10) << "tick balancer "
+             << " full cache_bytes: " << full_cache->get_cache_bytes()
+             << " full comtd_bytes: " << full_cache->get_committed_size()
+             << " full used_bytes: " << full_cache->_get_used_bytes()
+             << " full num_osdmaps: " << full_cache->_get_num_osdmaps()
+             << dendl;
+  }
+}
+
+void OSDMonitor::_set_new_cache_sizes()
+{
+  uint64_t cache_size = 0;
+  int64_t inc_alloc = 0;
+  int64_t full_alloc = 0;
+  int64_t kv_alloc = 0;
+
+  if (pcm != nullptr && rocksdb_binned_kv_cache != nullptr) {
+    cache_size = pcm->get_tuned_mem();
+    inc_alloc = inc_cache->get_committed_size();
+    full_alloc = full_cache->get_committed_size();
+    kv_alloc = rocksdb_binned_kv_cache->get_committed_size();
+  }
+
+  inc_osd_cache.set_bytes(inc_alloc);
+  full_osd_cache.set_bytes(full_alloc);
+
+  dout(10) << __func__ << " cache_size:" << cache_size
+           << " inc_alloc: " << inc_alloc
+           << " full_alloc: " << full_alloc
+           << " kv_alloc: " << kv_alloc
+           << dendl;
 }
 
 bool OSDMonitor::handle_osd_timeouts(const utime_t &now,
index 7e44bce1f712173e851f6a509b71b8c0c481e9a4..cd911465580371e6ee358a5390a77e3cae83ed09 100644 (file)
@@ -27,6 +27,7 @@
 #include "include/types.h"
 #include "include/encoding.h"
 #include "common/simple_cache.hpp"
+#include "common/PriorityCache.h"
 #include "msg/Messenger.h"
 
 #include "osd/OSDMap.h"
@@ -222,6 +223,8 @@ public:
   map<int,utime_t>    down_pending_out;  // osd down -> out
   bool priority_convert = false;
   map<int64_t,set<snapid_t>> pending_pseudo_purged_snaps;
+  std::shared_ptr<PriorityCache::PriCache> rocksdb_binned_kv_cache = nullptr;
+  std::shared_ptr<PriorityCache::Manager> pcm = nullptr;
 
   map<int,double> osd_weight;
 
@@ -305,6 +308,27 @@ private:
   bool is_prune_supported() const;
   bool do_prune(MonitorDBStore::TransactionRef tx);
 
+  // Priority cache control
+  uint32_t mon_osd_cache_size = 0;  ///< Number of cached OSDMaps
+  uint64_t rocksdb_cache_size = 0;  ///< Cache for kv Db
+  double cache_kv_ratio = 0;        ///< Cache ratio dedicated to kv
+  double cache_inc_ratio = 0;       ///< Cache ratio dedicated to inc
+  double cache_full_ratio = 0;      ///< Cache ratio dedicated to full
+  uint64_t mon_memory_base = 0;     ///< Mon base memory for cache autotuning
+  double mon_memory_fragmentation = 0; ///< Expected memory fragmentation
+  uint64_t mon_memory_target = 0;   ///< Mon target memory for cache autotuning
+  uint64_t mon_memory_min = 0;      ///< Min memory to cache osdmaps
+  bool mon_memory_autotune = false; ///< Cache auto tune setting
+  int register_cache_with_pcm();
+  int _set_cache_sizes();
+  int _set_cache_ratios();
+  void _set_new_cache_sizes();
+  void _set_cache_autotuning();
+
+  friend struct OSDMemCache;
+  friend struct IncCache;
+  friend struct FullCache;
+
   /**
    * we haven't delegated full version stashing to paxosservice for some time
    * now, making this function useless in current context.
index a217e8cf908808f618c6491156f27ab631aa575d..313ad81e06c1f3acd10910b5568b587916322424 100644 (file)
@@ -354,6 +354,10 @@ public:
       o.back()->seq = 30;
     }
 
+    size_t length() {
+      return sizeof(_Header);
+    }
+
     _Header() : seq(0), parent(0), num_children(1) {}
   };