]> git-server-git.apps.pok.os.sepia.ceph.com Git - ceph.git/commitdiff
os/bluestore: make shared blob fsck much less RAM-greedy.
authorIgor Fedotov <ifed@suse.com>
Tue, 26 Oct 2021 10:35:00 +0000 (13:35 +0300)
committerIgor Fedotov <igor.fedotov@croit.io>
Fri, 7 Jan 2022 18:01:04 +0000 (21:01 +0300)
Fixes: https://tracker.ceph.com/issues/44924
Signed-off-by: Igor Fedotov <igor.fedotov@croit.io>
src/common/options/global.yaml.in
src/os/bluestore/BlueStore.cc
src/os/bluestore/BlueStore.h
src/os/bluestore/bluestore_types.cc
src/os/bluestore/bluestore_types.h
src/test/objectstore/store_test.cc
src/test/objectstore/test_bluestore_types.cc

index 1c81c3a1701cbb68d72cbbfea88f8f6deda32d02..d9129e37ebb6d442534962da7e18946ac7002975 100644 (file)
@@ -4848,6 +4848,14 @@ options:
   desc: Number of additional threads to perform quick-fix (shallow fsck) command
   default: 2
   with_legacy: true
+- name: bluestore_fsck_shared_blob_tracker_size
+  type: size
+  level: dev
+  desc: Size in bytes for a hash table to track shared blobs ref counts. Higher the size, more precise is the tracker -> less overhead during the repair.
+  default: 128_M
+  flags:
+  - runtime
+  with_legacy: true
 - name: bluestore_throttle_bytes
   type: size
   level: advanced
index ff0aade670c5c0256172e1a86cb368a3671fd90c..876005de32b168d12dd48306876186efff51e171 100644 (file)
 #include <sys/types.h>
 #include <sys/stat.h>
 #include <fcntl.h>
+#include <algorithm>
 
 #include <boost/container/flat_set.hpp>
-#include "boost/algorithm/string.hpp"
+#include <boost/algorithm/string.hpp>
 
 #include "include/cpp-btree/btree_set.h"
 
@@ -127,7 +128,7 @@ const string PREFIX_PERPG_OMAP = "p";   // u64(pool) + u32(hash) + u64(id) + key
 const string PREFIX_DEFERRED = "L";    // id -> deferred_transaction_t
 const string PREFIX_ALLOC = "B";       // u64 offset -> u64 length (freelist)
 const string PREFIX_ALLOC_BITMAP = "b";// (see BitmapFreelistManager)
-const string PREFIX_SHARED_BLOB = "X"; // u64 offset -> shared_blob_t
+const string PREFIX_SHARED_BLOB = "X"; // u64 SB id -> shared_blob_t
 
 #ifdef HAVE_LIBZBD
 const string PREFIX_ZONED_FM_META = "Z";  // (see ZonedFreelistManager)
@@ -7661,10 +7662,8 @@ int _fsck_sum_extents(
   return 0;
 }
 
-
 int BlueStore::_fsck_check_extents(
-  const coll_t& cid,
-  const ghobject_t& oid,
+  std::string_view ctx_descr,
   const PExtentVector& extents,
   bool compressed,
   mempool_dynamic_bitset &used_blocks,
@@ -7673,7 +7672,7 @@ int BlueStore::_fsck_check_extents(
   store_statfs_t& expected_statfs,
   FSCKDepth depth)
 {
-  dout(30) << __func__ << " oid " << oid << " extents " << extents << dendl;
+  dout(30) << __func__ << " " << ctx_descr << ", extents " << extents << dendl;
   int errors = 0;
   for (auto e : extents) {
     if (!e.is_valid())
@@ -7684,7 +7683,6 @@ int BlueStore::_fsck_check_extents(
     }
     if (depth != FSCK_SHALLOW) {
       bool already = false;
-      //dout(1) << __func__ << "::NCB::FSCK<" << e.offset << "," << e.length << ">" << dendl;
       apply_for_bitset_range(
         e.offset, e.length, granularity, used_blocks,
         [&](uint64_t pos, mempool_dynamic_bitset &bs) {
@@ -7694,7 +7692,7 @@ int BlueStore::_fsck_check_extents(
                pos * min_alloc_size, min_alloc_size, !already);
            }
             if (!already) {
-              derr << __func__ << "::fsck error: " << oid << " extent " << e
+              derr << __func__ << "::fsck error: " << ctx_descr << ", extent " << e
                   << " or a subset is already allocated (misreferenced)" << dendl;
              ++errors;
              already = true;
@@ -7703,12 +7701,9 @@ int BlueStore::_fsck_check_extents(
          else
            bs.set(pos);
         });
-        if (repairer) {
-         repairer->set_space_used(e.offset, e.length, cid, oid);
-        }
 
       if (e.end() > bdev->get_size()) {
-        derr << "fsck error:  " << oid << " extent " << e
+        derr << "fsck error:  " << ctx_descr << ", extent " << e
             << " past end of block device" << dendl;
         ++errors;
       }
@@ -7824,6 +7819,143 @@ void BlueStore::_fsck_check_pool_statfs(
   }
 }
 
+void BlueStore::_fsck_repair_shared_blobs(
+  BlueStoreRepairer& repairer,
+  shared_blob_2hash_tracker_t& sb_ref_counts,
+  sb_info_space_efficient_map_t& sb_info)
+{
+  auto sb_ref_mismatches = sb_ref_counts.count_non_zero();
+  dout(1) << __func__ << " repairing shared_blobs, ref mismatch estimate: "
+         << sb_ref_mismatches << dendl;
+  if (!sb_ref_mismatches) // not expected to succeed, just in case
+    return;
+
+  mempool::bluestore_fsck::map<uint64_t, bluestore_extent_ref_map_t> ref_maps;
+  auto it = db->get_iterator(PREFIX_OBJ, KeyValueDB::ITERATOR_NOCACHE);
+  if (it) {
+    CollectionRef c;
+    spg_t pgid;
+    for (it->lower_bound(string()); it->valid(); it->next()) {
+      dout(30) << __func__ << " key "
+       << pretty_binary_string(it->key()) << dendl;
+      if (is_extent_shard_key(it->key())) {
+       continue;
+      }
+
+      ghobject_t oid;
+      int r = get_key_object(it->key(), &oid);
+      if (r < 0) {
+       continue;
+      }
+
+      if (!c ||
+       oid.shard_id != pgid.shard ||
+       oid.hobj.get_logical_pool() != (int64_t)pgid.pool() ||
+       !c->contains(oid)) {
+       c = nullptr;
+       for (auto& p : coll_map) {
+         if (p.second->contains(oid)) {
+           c = p.second;
+           break;
+         }
+       }
+       if (!c) {
+         continue;
+       }
+      }
+      dout(20) << __func__ << " inspecting shared blob refs for col:" << c->cid
+       << " obj:" << oid << dendl;
+
+      OnodeRef o;
+      o.reset(Onode::decode(c, oid, it->key(), it->value()));
+      o->extent_map.fault_range(db, 0, OBJECT_MAX_SIZE);
+
+      _dump_onode<30>(cct, *o);
+
+      mempool::bluestore_fsck::set<BlobRef> passed_sbs;
+      for (auto& e : o->extent_map.extent_map) {
+       auto& b = e.blob->get_blob();
+       if (b.is_shared() && passed_sbs.count(e.blob) == 0) {
+         auto sbid = e.blob->shared_blob->get_sbid();
+         passed_sbs.emplace(e.blob);
+         bluestore_extent_ref_map_t ref_map_candidate;
+         bool to_be_updated = false;
+         auto it = ref_maps.lower_bound(sbid);
+         bool add_existing = it != ref_maps.end() && it->first == sbid;
+         for (auto& p : b.get_extents()) {
+           if (p.is_valid()) {
+             if (add_existing) {
+               it->second.get(p.offset, p.length);
+             } else {
+               ref_map_candidate.get(p.offset, p.length);
+               if (!to_be_updated) {
+                 to_be_updated =
+                   !sb_ref_counts.test_all_zero_range(sbid, p.offset, p.length);
+               }
+             }
+           }
+         }
+         if (to_be_updated) {
+           ceph_assert(!ref_map_candidate.empty());
+           ceph_assert(!add_existing);
+           ref_maps.emplace_hint(it, sbid, std::move(ref_map_candidate));
+         }
+         dout(20) << __func__ << " inspected shared blob refs for col:" << c->cid
+           << " obj:" << oid << " sbid 0x " << std::hex << sbid << std::dec
+           << " existed " << add_existing
+           << " to be updated " << to_be_updated
+           << dendl;
+       } // if b.shared ....
+      } // for ... extent_map
+    } // for ... it->valid
+  } //if (it(PREFIX_OBJ))
+
+  // update shared blob records
+  auto ref_it = ref_maps.begin();
+  while (ref_it != ref_maps.end()) {
+    size_t cnt = 0;
+    const size_t max_transactions = 4096;
+    KeyValueDB::Transaction txn = db->get_transaction();
+    for (cnt = 0;
+      cnt < max_transactions && ref_it != ref_maps.end();
+      ref_it++) {
+      auto sbid = ref_it->first;
+      dout(20) << __func__ << " repaired shared_blob 0x"
+       << std::hex << sbid << std::dec
+       << ref_it->second << dendl;
+      repairer.fix_shared_blob(txn, sbid, &ref_it->second, 0);
+      cnt++;
+    }
+    if (cnt) {
+      db->submit_transaction_sync(txn);
+      cnt = 0;
+    }
+  }
+  // remove stray shared blob records
+  size_t cnt = 0;
+  const size_t max_transactions = 4096;
+  KeyValueDB::Transaction txn = db->get_transaction();
+  sb_info.foreach_stray([&](const sb_info_t& sbi) {
+    auto sbid = sbi.get_sbid();
+    dout(20) << __func__ << " removing stray shared_blob 0x"
+      << std::hex << sbid << std::dec
+      << dendl;
+    repairer.fix_shared_blob(txn, sbid, nullptr, 0);
+    cnt++;
+    if (cnt >= max_transactions) {}
+      db->submit_transaction_sync(txn);
+      txn = db->get_transaction();
+      cnt = 0;
+    });
+  if (cnt > 0) {
+    db->submit_transaction_sync(txn);
+  }
+
+  // amount of repairs to report to be equal to previously
+  // determined error estimation, not the actual number of updated shared blobs
+  repairer.inc_repaired(sb_ref_mismatches);
+}
+
 BlueStore::OnodeRef BlueStore::fsck_check_objects_shallow(
   BlueStore::FSCKDepth depth,
   int64_t pool_id,
@@ -7844,6 +7976,7 @@ BlueStore::OnodeRef BlueStore::fsck_check_objects_shallow(
   auto used_blocks = ctx.used_blocks;
   auto sb_info_lock = ctx.sb_info_lock;
   auto& sb_info = ctx.sb_info;
+  auto& sb_ref_counts = ctx.sb_ref_counts;
   auto repairer = ctx.repairer;
 
   store_statfs_t* res_statfs = (per_pool_stat_collection || repairer) ?
@@ -7973,14 +8106,20 @@ BlueStore::OnodeRef BlueStore::fsck_check_objects_shallow(
       res_statfs->data_compressed_original +=
         i.first->get_referenced_bytes();
     }
+    if (depth != FSCK_SHALLOW && repairer) {
+      for (auto e : blob.get_extents()) {
+       if (!e.is_valid())
+         continue;
+       repairer->set_space_used(e.offset, e.length, c->cid, oid);
+      }
+    }
     if (blob.is_shared()) {
       if (i.first->shared_blob->get_sbid() > blobid_max) {
         derr << "fsck error: " << oid << " blob " << blob
           << " sbid " << i.first->shared_blob->get_sbid() << " > blobid_max "
           << blobid_max << dendl;
         ++errors;
-      }
-      else if (i.first->shared_blob->get_sbid() == 0) {
+      } else if (i.first->shared_blob->get_sbid() == 0) {
         derr << "fsck error: " << oid << " blob " << blob
           << " marked as shared but has uninitialized sbid"
           << dendl;
@@ -7990,18 +8129,22 @@ BlueStore::OnodeRef BlueStore::fsck_check_objects_shallow(
       if (sb_info_lock) {
         sb_info_lock->lock();
       }
-      sb_info_t& sbi = sb_info[i.first->shared_blob->get_sbid()];
-      ceph_assert(sbi.cid == coll_t() || sbi.cid == c->cid);
-      ceph_assert(sbi.pool_id == INT64_MIN ||
+      auto sbid = i.first->shared_blob->get_sbid();
+      sb_info_t& sbi = sb_info.add_or_adopt(i.first->shared_blob->get_sbid());
+      ceph_assert(sbi.pool_id == sb_info_t::INVALID_POOL_ID ||
         sbi.pool_id == oid.hobj.get_logical_pool());
-      sbi.cid = c->cid;
       sbi.pool_id = oid.hobj.get_logical_pool();
-      sbi.sb = i.first->shared_blob;
-      sbi.oids.push_back(oid);
-      sbi.compressed = blob.is_compressed();
+      bool compressed = blob.is_compressed();
       for (auto e : blob.get_extents()) {
         if (e.is_valid()) {
-          sbi.ref_map.get(e.offset, e.length);
+         if (compressed) {
+           ceph_assert(sbi.allocated_chunks <= 0);
+           sbi.allocated_chunks -= (e.length >> min_alloc_size_order);
+         } else {
+           ceph_assert(sbi.allocated_chunks >= 0);
+           sbi.allocated_chunks += (e.length >> min_alloc_size_order);
+         }
+         sb_ref_counts.inc_range(sbid, e.offset, e.length, 1);
         }
       }
       if (sb_info_lock) {
@@ -8009,11 +8152,13 @@ BlueStore::OnodeRef BlueStore::fsck_check_objects_shallow(
       }
     } else if (depth != FSCK_SHALLOW) {
       ceph_assert(used_blocks);
-      errors += _fsck_check_extents(c->cid, oid, blob.get_extents(),
+      string ctx_descr = " oid " + stringify(oid);
+      errors += _fsck_check_extents(ctx_descr,
+       blob.get_extents(),
         blob.is_compressed(),
         *used_blocks,
         fm->get_alloc_size(),
-        repairer,
+       repairer,
         *res_statfs,
         depth);
     } else {
@@ -8143,7 +8288,8 @@ public:
     BlueStore* store = nullptr;
 
     ceph::mutex* sb_info_lock = nullptr;
-    BlueStore::sb_info_map_t* sb_info = nullptr;
+    sb_info_space_efficient_map_t* sb_info = nullptr;
+    shared_blob_2hash_tracker_t* sb_ref_counts = nullptr;
     BlueStoreRepairer* repairer = nullptr;
 
     Batch* batches = nullptr;
@@ -8154,13 +8300,15 @@ public:
                   size_t _batchCount,
                   BlueStore* _store,
                   ceph::mutex* _sb_info_lock,
-                  BlueStore::sb_info_map_t& _sb_info,
+                  sb_info_space_efficient_map_t& _sb_info,
+                 shared_blob_2hash_tracker_t& _sb_ref_counts,
                   BlueStoreRepairer* _repairer) :
       WorkQueue_(n, ceph::timespan::zero(), ceph::timespan::zero()),
       batchCount(_batchCount),
       store(_store),
       sb_info_lock(_sb_info_lock),
       sb_info(&_sb_info),
+      sb_ref_counts(&_sb_ref_counts),
       repairer(_repairer)
     {
       batches = new Batch[batchCount];
@@ -8214,6 +8362,7 @@ public:
        nullptr,
         sb_info_lock,
         *sb_info,
+       *sb_ref_counts,
         batch->expected_store_statfs,
         batch->expected_pool_statfs,
         repairer);
@@ -8487,6 +8636,7 @@ void BlueStore::_fsck_check_objects(
   auto& errors = ctx.errors;
   auto sb_info_lock = ctx.sb_info_lock;
   auto& sb_info = ctx.sb_info;
+  auto& sb_ref_counts = ctx.sb_ref_counts;
   auto repairer = ctx.repairer;
 
   uint64_t_btree_t used_nids;
@@ -8505,6 +8655,7 @@ void BlueStore::_fsck_check_objects(
         this,
         sb_info_lock,
         sb_info,
+       sb_ref_counts,
         repairer));
 
     ShallowFSCKThreadPool thread_pool(cct, "ShallowFSCKThreadPool", "ShallowFSCK", thread_count);
@@ -8844,7 +8995,11 @@ int BlueStore::_fsck_on_open(BlueStore::FSCKDepth depth, bool repair)
   store_statfs_t expected_store_statfs, actual_statfs;
   per_pool_statfs expected_pool_statfs;
 
-  sb_info_map_t sb_info;
+  sb_info_space_efficient_map_t sb_info;
+  shared_blob_2hash_tracker_t sb_ref_counts(
+    cct->_conf->bluestore_fsck_shared_blob_tracker_size,
+    min_alloc_size);
+  size_t sb_ref_mismatches = 0;
 
   /// map of oid -> (first_)offset for each zone
   std::vector<std::unordered_map<ghobject_t, uint64_t>> zone_refs;   // FIXME: this may be a lot of RAM!
@@ -9007,6 +9162,42 @@ int BlueStore::_fsck_on_open(BlueStore::FSCKDepth depth, bool repair)
   }
 #endif
 
+  dout(1) << __func__ << " checking shared_blobs (phase 1)" << dendl;
+  it = db->get_iterator(PREFIX_SHARED_BLOB, KeyValueDB::ITERATOR_NOCACHE);
+  if (it) {
+    for (it->lower_bound(string()); it->valid(); it->next()) {
+      string key = it->key();
+      uint64_t sbid;
+      if (get_key_shared_blob(key, &sbid)) {
+       // this gonna to be handled at the second stage
+       continue;
+      }
+      bluestore_shared_blob_t shared_blob(sbid);
+      bufferlist bl = it->value();
+      auto blp = bl.cbegin();
+      try {
+       decode(shared_blob, blp);
+      }
+      catch (ceph::buffer::error& e) {
+       // this gonna to be handled at the second stage
+       continue;
+      }
+      dout(20) << __func__ << "  " << shared_blob << dendl;
+      auto& sbi = sb_info.add_maybe_stray(sbid);
+
+      // primarily to silent the 'unused' warning
+      ceph_assert(sbi.pool_id == sb_info_t::INVALID_POOL_ID);
+
+      for (auto& r : shared_blob.ref_map.ref_map) {
+       sb_ref_counts.inc_range(
+         sbid,
+         r.first,
+         r.second.length,
+         -r.second.refs);
+      }
+    }
+  } // if (it) //checking shared_blobs (phase1)
+
   // walk PREFIX_OBJ
   {
     dout(1) << __func__ << " walking object keyspace" << dendl;
@@ -9026,6 +9217,7 @@ int BlueStore::_fsck_on_open(BlueStore::FSCKDepth depth, bool repair)
       // there is no multithreading in this case
       depth == FSCK_SHALLOW ? &sb_info_lock : nullptr,
       sb_info,
+      sb_ref_counts,
       expected_store_statfs,
       expected_pool_statfs,
       repair ? &repairer : nullptr);
@@ -9047,19 +9239,28 @@ int BlueStore::_fsck_on_open(BlueStore::FSCKDepth depth, bool repair)
   }
 #endif
 
-  dout(1) << __func__ << " checking shared_blobs" << dendl;
+  sb_ref_mismatches = sb_ref_counts.count_non_zero();
+  if (sb_ref_mismatches != 0) {
+    derr << "fsck error: shared blob references aren't matching, at least "
+      << sb_ref_mismatches << " found" << dendl;
+    errors += sb_ref_mismatches;
+  }
+
+  if (depth != FSCK_SHALLOW && repair && sb_ref_mismatches) {
+    _fsck_repair_shared_blobs(repairer, sb_ref_counts, sb_info);
+  }
+  dout(1) << __func__ << " checking shared_blobs (phase 2)" << dendl;
   it = db->get_iterator(PREFIX_SHARED_BLOB, KeyValueDB::ITERATOR_NOCACHE);
   if (it) {
     // FIXME minor: perhaps simplify for shallow mode?
     // fill global if not overriden below
     auto expected_statfs = &expected_store_statfs;
-
     for (it->lower_bound(string()); it->valid(); it->next()) {
       string key = it->key();
       uint64_t sbid;
       if (get_key_shared_blob(key, &sbid)) {
        derr << "fsck error: bad key '" << key
-            << "' in shared blob namespace" << dendl;
+         << "' in shared blob namespace" << dendl;
        if (repair) {
          repairer.remove_key(db, PREFIX_SHARED_BLOB, key);
        }
@@ -9068,63 +9269,62 @@ int BlueStore::_fsck_on_open(BlueStore::FSCKDepth depth, bool repair)
       }
       auto p = sb_info.find(sbid);
       if (p == sb_info.end()) {
-       derr << "fsck error: found stray shared blob data for sbid 0x"
-            << std::hex << sbid << std::dec << dendl;
-       if (repair) {
-         repairer.remove_key(db, PREFIX_SHARED_BLOB, key);
+        if (sb_ref_mismatches > 0) {
+         // highly likely this has been already reported before, ignoring...
+         dout(5) << __func__ << " found duplicate(?) stray shared blob data for sbid 0x"
+           << std::hex << sbid << std::dec << dendl;
+       } else {
+         derr<< "fsck error: found stray shared blob data for sbid 0x"
+           << std::hex << sbid << std::dec << dendl;
+         ++errors;
+         if (repair) {
+           repairer.remove_key(db, PREFIX_SHARED_BLOB, key);
+         }
        }
-       ++errors;
       } else {
        ++num_shared_blobs;
-       sb_info_t& sbi = p->second;
+       sb_info_t& sbi = *p;
        bluestore_shared_blob_t shared_blob(sbid);
        bufferlist bl = it->value();
        auto blp = bl.cbegin();
        try {
-         decode(shared_blob, blp);
-       } catch (ceph::buffer::error& e) {
-          ++errors;
-          // Force update and don't report as missing
-          sbi.updated = sbi.passed = true;
-
-          derr << "fsck error: failed to decode Shared Blob"
-              << pretty_binary_string(it->key()) << dendl;
-          if (repair) {
-           dout(20) << __func__ << " undecodable Shared Blob, key:'"
-                    << pretty_binary_string(it->key())
-                    << "', removing" << dendl;
-            repairer.remove_key(db, PREFIX_SHARED_BLOB, it->key());
-          }
-          continue;
-        }      
-       dout(20) << __func__ << "  " << *sbi.sb << " " << shared_blob << dendl;
-       if (shared_blob.ref_map != sbi.ref_map) {
-         derr << "fsck error: shared blob 0x" << std::hex << sbid
-               << std::dec << " ref_map " << shared_blob.ref_map
-               << " != expected " << sbi.ref_map << dendl;
-         sbi.updated = true; // will update later in repair mode only!
+         decode(shared_blob, blp);
+       }
+       catch (ceph::buffer::error& e) {
          ++errors;
+
+         derr << "fsck error: failed to decode Shared Blob"
+           << pretty_binary_string(key) << dendl;
+         if (repair) {
+           dout(20) << __func__ << " undecodable Shared Blob, key:'"
+             << pretty_binary_string(key)
+             << "', removing" << dendl;
+           repairer.remove_key(db, PREFIX_SHARED_BLOB, key);
+         }
+         continue;
        }
+       dout(20) << __func__ << "  " << shared_blob << dendl;
        PExtentVector extents;
-       for (auto &r : shared_blob.ref_map.ref_map) {
+       for (autor : shared_blob.ref_map.ref_map) {
          extents.emplace_back(bluestore_pextent_t(r.first, r.second.length));
        }
-       if (per_pool_stat_collection || repair) {
+       if (sbi.pool_id != sb_info_t::INVALID_POOL_ID &&
+           (per_pool_stat_collection || repair)) {
          expected_statfs = &expected_pool_statfs[sbi.pool_id];
        }
-       errors += _fsck_check_extents(sbi.cid,
-                                     sbi.oids.front(),
-                                     extents,
-                                     sbi.compressed,
-                                     used_blocks,
-                                     fm->get_alloc_size(),
-                                     repair ? &repairer : nullptr,
-                                     *expected_statfs,
-                                      depth);
-       sbi.passed = true;
+       std::stringstream ss;
+       ss << "sbid 0x" << std::hex << sbid << std::dec;
+       errors += _fsck_check_extents(ss.str(),
+         extents,
+         sbi.allocated_chunks < 0,
+         used_blocks,
+         fm->get_alloc_size(),
+         repair ? &repairer : nullptr,
+         *expected_statfs,
+         depth);
       }
     }
-  } // if (it)
+  } // if (it) /* checking shared_blobs (phase 2)*/
 
   if (repair && repairer.preprocess_misreference(db)) {
 
@@ -9276,22 +9476,24 @@ int BlueStore::_fsck_on_open(BlueStore::FSCKDepth depth, bool repair)
          if (b->get_blob().is_shared()) {
             b->dirty_blob().clear_flag(bluestore_blob_t::FLAG_SHARED);
 
-           auto sb_it = sb_info.find(b->shared_blob->get_sbid());
+           auto sbid = b->shared_blob->get_sbid();
+           auto sb_it = sb_info.find(sbid);
            ceph_assert(sb_it != sb_info.end());
-           sb_info_t& sbi = sb_it->second;
-
-           for (auto& r : sbi.ref_map.ref_map) {
-             expected_statfs->allocated -= r.second.length;
-             if (sbi.compressed) {
-               // NB: it's crucial to use compressed flag from sb_info_t
-               // as we originally used that value while accumulating 
-               // expected_statfs
-               expected_statfs->data_compressed_allocated -= r.second.length;
-             }
+           sb_info_t& sbi = *sb_it;
+
+           if (sbi.allocated_chunks < 0) {
+             // NB: it's crucial to use compressed_allocated_chunks from sb_info_t
+             // as we originally used that value while accumulating
+             // expected_statfs
+             expected_statfs->allocated -= uint64_t(-sbi.allocated_chunks) << min_alloc_size_order;
+             expected_statfs->data_compressed_allocated -=
+               uint64_t(-sbi.allocated_chunks) << min_alloc_size_order;
+           } else {
+             expected_statfs->allocated -= uint64_t(sbi.allocated_chunks) << min_alloc_size_order;
            }
-           sbi.updated = sbi.passed = true;
-           sbi.ref_map.clear();
-           
+           sbi.allocated_chunks = 0;
+           repairer.fix_shared_blob(txn, sbid, nullptr, 0);
+
            // relying on blob's pextents to decide what to release.
            for (auto& p : pext_to_release) {
              to_release.union_insert(p.offset, p.length);
@@ -9324,57 +9526,8 @@ int BlueStore::_fsck_on_open(BlueStore::FSCKDepth depth, bool repair)
       to_release.clear();
     } // if (it) {
   } //if (repair && repairer.preprocess_misreference()) {
-
-  if (depth != FSCK_SHALLOW) {
-    for (auto &p : sb_info) {
-      sb_info_t& sbi = p.second;
-      if (!sbi.passed) {
-        derr << "fsck error: missing " << *sbi.sb << dendl;
-        ++errors;
-      }
-      if (repair && (!sbi.passed || sbi.updated)) {
-        auto sbid = p.first;
-        if (sbi.ref_map.empty()) {
-         ceph_assert(sbi.passed);
-         dout(20) << __func__ << " " << *sbi.sb
-                  << " is empty, removing" << dendl;
-         repairer.fix_shared_blob(db, sbid, nullptr);
-        } else {
-         bufferlist bl;
-         bluestore_shared_blob_t persistent(sbid, std::move(sbi.ref_map));
-         encode(persistent, bl);
-         dout(20) << __func__ << " " << *sbi.sb
-                  << " is " << bl.length() << " bytes, updating"
-                  << dendl;
-
-         repairer.fix_shared_blob(db, sbid, &bl);
-         // we need to account for shared blob pextents at both
-         // stats and used blocks to avoid related errors.
-         PExtentVector extents;
-         for (auto& r : persistent.ref_map.ref_map) {
-           extents.emplace_back(bluestore_pextent_t(r.first, r.second.length));
-         }
-         auto* expected_statfs = &expected_pool_statfs[sbi.pool_id];
-         int errors = _fsck_check_extents(sbi.cid,
-           ghobject_t(), // doesn't matter
-           extents,
-           sbi.compressed,
-           used_blocks,
-           fm->get_alloc_size(),
-           nullptr,
-           *expected_statfs,
-           depth);
-         if (errors) {
-           derr << __func__ << " " << errors
-                << "  unexpected error(s) after missed shared blob repair,"
-                << " perhaps worth one more repair attempt"
-                << dendl;
-         }
-        }
-      }
-    }
-  }
   sb_info.clear();
+  sb_ref_counts.reset();
 
   // check global stats only if fscking (not repairing) w/o per-pool stats
   if (!per_pool_stat_collection &&
@@ -9729,6 +9882,27 @@ void BlueStore::inject_no_shared_blob_key()
   db->submit_transaction_sync(txn);
 };
 
+void BlueStore::inject_stray_shared_blob_key(uint64_t sbid)
+{
+  KeyValueDB::Transaction txn;
+  txn = db->get_transaction();
+
+  dout(5) << __func__ << " " << sbid << dendl;
+
+  string key;
+  get_shared_blob_key(sbid, &key);
+  bluestore_shared_blob_t persistent(sbid);
+  persistent.ref_map.get(0xdead0000, 0x1000);
+  bufferlist bl;
+  encode(persistent, bl);
+  dout(20) << __func__ << " sbid " << sbid
+    << " takes " << bl.length() << " bytes, updating"
+    << dendl;
+
+  txn->set(PREFIX_SHARED_BLOB, key, bl);
+  db->submit_transaction_sync(txn);
+};
+
 
 void BlueStore::inject_leaked(uint64_t len)
 {
@@ -17352,29 +17526,22 @@ void BlueStoreRepairer::fix_per_pool_omap(KeyValueDB *db, int val)
 }
 
 bool BlueStoreRepairer::fix_shared_blob(
-  KeyValueDB *db,
+  KeyValueDB::Transaction txn,
   uint64_t sbid,
-  const bufferlist* bl)
+  bluestore_extent_ref_map_t* ref_map,
+  size_t repaired)
 {
-  std::lock_guard l(lock); // possibly redundant
-  KeyValueDB::Transaction txn;
-  if (fix_misreferences_txn) { // reuse this txn
-    txn = fix_misreferences_txn;
-  } else {
-    if (!fix_shared_blob_txn) {
-      fix_shared_blob_txn = db->get_transaction();
-    }
-    txn = fix_shared_blob_txn;
-  }
   string key;
   get_shared_blob_key(sbid, &key);
-
-  ++to_repair_cnt;
-  if (bl) {
-    txn->set(PREFIX_SHARED_BLOB, key, *bl);
+  if (ref_map) {
+    bluestore_shared_blob_t persistent(sbid, std::move(*ref_map));
+    bufferlist bl;
+    encode(persistent, bl);
+    txn->set(PREFIX_SHARED_BLOB, key, bl);
   } else {
     txn->rmkey(PREFIX_SHARED_BLOB, key);
   }
+  to_repair_cnt += repaired;
   return true;
 }
 
index f96b82b34c461de76500f1b17674172f309cb14b..98d9a91fafef6ba0b7e51eba66fbe269918ce559 100644 (file)
@@ -2576,8 +2576,7 @@ public:
 
 private:
   int _fsck_check_extents(
-    const coll_t& cid,
-    const ghobject_t& oid,
+    std::string_view ctx_descr,
     const PExtentVector& extents,
     bool compressed,
     mempool_dynamic_bitset &used_blocks,
@@ -2591,6 +2590,10 @@ private:
     int64_t& errors,
     int64_t &warnings,
     BlueStoreRepairer* repairer);
+  void _fsck_repair_shared_blobs(
+    BlueStoreRepairer& repairer,
+    shared_blob_2hash_tracker_t& sb_ref_counts,
+    sb_info_space_efficient_map_t& sb_info);
 
   int _fsck(FSCKDepth depth, bool repair);
   int _fsck_on_open(BlueStore::FSCKDepth depth, bool repair);
@@ -3056,6 +3059,7 @@ public:
   void inject_broken_shared_blob_key(const std::string& key,
                         const ceph::buffer::list& bl);
   void inject_no_shared_blob_key();
+  void inject_stray_shared_blob_key(uint64_t sbid);
 
   void inject_leaked(uint64_t len);
   void inject_false_free(coll_t cid, ghobject_t oid);
@@ -3447,21 +3451,10 @@ private:
   inline bool _use_rotational_settings();
 
 public:
-  struct sb_info_t {
-    coll_t cid;
-    int64_t pool_id = INT64_MIN;
-    std::list<ghobject_t> oids;
-    BlueStore::SharedBlobRef sb;
-    bluestore_extent_ref_map_t ref_map;
-    bool compressed = false;
-    bool passed = false;
-    bool updated = false;
-  };
   typedef btree::btree_set<
     uint64_t, std::less<uint64_t>,
     mempool::bluestore_fsck::pool_allocator<uint64_t>> uint64_t_btree_t;
 
-  typedef mempool::bluestore_fsck::map<uint64_t, sb_info_t> sb_info_map_t;
   struct FSCK_ObjectCtx {
     int64_t& errors;
     int64_t& warnings;
@@ -3476,7 +3469,9 @@ public:
     std::vector<std::unordered_map<ghobject_t, uint64_t>> *zone_refs;
 
     ceph::mutex* sb_info_lock;
-    sb_info_map_t& sb_info;
+    sb_info_space_efficient_map_t& sb_info;
+    // approximate amount of references per <shared blob, chunk>
+    shared_blob_2hash_tracker_t& sb_ref_counts;
 
     store_statfs_t& expected_store_statfs;
     per_pool_statfs& expected_pool_statfs;
@@ -3492,8 +3487,10 @@ public:
                    mempool_dynamic_bitset* _ub,
                    uint64_t_btree_t* _used_omap_head,
                   std::vector<std::unordered_map<ghobject_t, uint64_t>> *_zone_refs,
-                  ceph::mutex* _sb_info_lock,
-                   sb_info_map_t& _sb_info,
+
+                   ceph::mutex* _sb_info_lock,
+                   sb_info_space_efficient_map_t& _sb_info,
+                  shared_blob_2hash_tracker_t& _sb_ref_counts,
                    store_statfs_t& _store_statfs,
                    per_pool_statfs& _pool_statfs,
                    BlueStoreRepairer* _repairer) :
@@ -3509,6 +3506,7 @@ public:
       zone_refs(_zone_refs),
       sb_info_lock(_sb_info_lock),
       sb_info(_sb_info),
+      sb_ref_counts(_sb_ref_counts),
       expected_store_statfs(_store_statfs),
       expected_pool_statfs(_pool_statfs),
       repairer(_repairer) {
@@ -3788,9 +3786,13 @@ public:
 public:
   void fix_per_pool_omap(KeyValueDB *db, int);
   bool remove_key(KeyValueDB *db, const std::string& prefix, const std::string& key);
-  bool fix_shared_blob(KeyValueDB *db,
-                        uint64_t sbid,
-                      const ceph::buffer::list* bl);
+  //bool fix_shared_blob(KeyValueDB *db,
+  //                  uint64_t sbid,
+  //                  const ceph::buffer::list* bl);
+  bool fix_shared_blob(KeyValueDB::Transaction txn,
+                       uint64_t sbid,
+                       bluestore_extent_ref_map_t* ref_map,
+                       size_t repaired = 1);
   bool fix_statfs(KeyValueDB *db, const std::string& key,
     const store_statfs_t& new_statfs);
 
@@ -3817,8 +3819,8 @@ public:
   }
   //////////////////////
   //In fact two methods below are the only ones in this class which are thread-safe!!
-  void inc_repaired() {
-    ++to_repair_cnt;
+  void inc_repaired(size_t n = 1) {
+    to_repair_cnt += n;
   }
   void request_compaction() {
     need_compact = true;
index b45e166541127c85c894e3f4c4704892610f7230..121f7ccd3049f6aba8fd2478597212a49f3ab611 100644 (file)
@@ -1181,3 +1181,88 @@ void bluestore_compression_header_t::generate_test_instances(
   o.push_back(new bluestore_compression_header_t(1));
   o.back()->length = 1234;
 }
+
+// adds more salt to build a hash func input
+shared_blob_2hash_tracker_t::hash_input_t
+  shared_blob_2hash_tracker_t::build_hash_input(
+    uint64_t sbid,
+    uint64_t offset) const
+{
+  hash_input_t res = {
+    sbid,
+    offset >> au_void_bits,
+    ((sbid & 0xffffffff) << 32) + ~(uint32_t((offset >> au_void_bits) & 0xffffffff))
+  };
+  return res;
+}
+
+void shared_blob_2hash_tracker_t::inc(
+  uint64_t sbid,
+  uint64_t offset,
+  int n)
+{
+  auto hash_input = build_hash_input(sbid, offset);
+  ref_counter_2hash_tracker_t::inc(
+    (char*)hash_input.data(),
+    get_hash_input_size(),
+    n);
+}
+
+void shared_blob_2hash_tracker_t::inc_range(
+  uint64_t sbid,
+  uint64_t offset,
+  uint32_t len,
+  int n)
+{
+  uint32_t alloc_unit = 1 << au_void_bits;
+  int64_t l = len;
+  while (l > 0) {
+    // don't care about ofset alignment as inc() trims it anyway
+    inc(sbid, offset, n);
+    offset += alloc_unit;
+    l -= alloc_unit;
+  }
+}
+
+bool shared_blob_2hash_tracker_t::test_hash_conflict(
+  uint64_t sbid1,
+  uint64_t offset1,
+  uint64_t sbid2,
+  uint64_t offset2) const
+{
+  auto hash_input1 = build_hash_input(sbid1, offset1);
+  auto hash_input2 = build_hash_input(sbid2, offset2);
+  return ref_counter_2hash_tracker_t::test_hash_conflict(
+    (char*)hash_input1.data(),
+    (char*)hash_input2.data(),
+    get_hash_input_size());
+}
+
+bool shared_blob_2hash_tracker_t::test_all_zero(
+  uint64_t sbid,
+  uint64_t offset) const
+{
+  auto hash_input = build_hash_input(sbid, offset);
+  return
+    ref_counter_2hash_tracker_t::test_all_zero(
+      (char*)hash_input.data(),
+      get_hash_input_size());
+}
+
+bool shared_blob_2hash_tracker_t::test_all_zero_range(
+  uint64_t sbid,
+  uint64_t offset,
+  uint32_t len) const
+{
+  uint32_t alloc_unit = 1 << au_void_bits;
+  int64_t l = len;
+  while (l > 0) {
+    // don't care about ofset alignment as inc() trims it anyway
+    if (!test_all_zero(sbid, offset)) {
+      return false;
+    }
+    offset += alloc_unit;
+    l -= alloc_unit;
+  }
+  return true;
+}
index 851278e7e062c9d415dbe59a0cb8cac203b07394..78f9d634757e6f40768f3b9ff237426970c75b0c 100644 (file)
@@ -16,8 +16,9 @@
 #define CEPH_OSD_BLUESTORE_BLUESTORE_TYPES_H
 
 #include <ostream>
-#include <bitset>
 #include <type_traits>
+#include <vector>
+#include <array>
 #include "include/mempool.h"
 #include "include/types.h"
 #include "include/interval_set.h"
@@ -26,6 +27,7 @@
 #include "compressor/Compressor.h"
 #include "common/Checksummer.h"
 #include "include/mempool.h"
+#include "include/ceph_hash.h"
 
 namespace ceph {
   class Formatter;
@@ -238,7 +240,7 @@ static inline bool operator!=(const bluestore_extent_ref_map_t& l,
   return !(l == r);
 }
 
-/// blob_use_tracker: a set of per-alloc unit ref counters to track blob usage
+/// blob_use_tracker: a set of per-alloc unit ref buckets to track blob usage
 struct bluestore_blob_use_tracker_t {
   // N.B.: There is no need to minimize au_size/num_au
   //   as much as possible (e.g. have just a single byte for au_size) since:
@@ -1121,5 +1123,245 @@ struct bluestore_compression_header_t {
 };
 WRITE_CLASS_DENC(bluestore_compression_header_t)
 
+template <template <typename> typename V, class COUNTER_TYPE = int32_t>
+class ref_counter_2hash_tracker_t {
+  size_t num_non_zero = 0;
+  size_t num_buckets = 0;
+  V<COUNTER_TYPE> buckets1;
+  V<COUNTER_TYPE> buckets2;
+
+public:
+  ref_counter_2hash_tracker_t(uint64_t mem_cap) {
+    num_buckets = mem_cap / sizeof(COUNTER_TYPE) / 2;
+    ceph_assert(num_buckets);
+    buckets1.resize(num_buckets);
+    buckets2.resize(num_buckets);
+    reset();
+  }
+
+  size_t get_num_buckets() const {
+    return num_buckets;
+  }
+
+  void inc(const char* hash_val, size_t hash_val_len, int n) {
+    auto h = ceph_str_hash_rjenkins((const char*)hash_val, hash_val_len) %
+      num_buckets;
+    if (buckets1[h] == 0 && n) {
+      ++num_non_zero;
+    } else if (buckets1[h] == -n) {
+      --num_non_zero;
+    }
+    buckets1[h] += n;
+    h = ceph_str_hash_linux((const char*)hash_val, hash_val_len) % num_buckets;
+    if (buckets2[h] == 0 && n) {
+      ++num_non_zero;
+    } else if (buckets2[h] == -n) {
+      --num_non_zero;
+    }
+    buckets2[h] += n;
+  }
+
+  bool test_hash_conflict(
+    const char* hash_val1,
+    const char* hash_val2,
+    size_t hash_val_len) const {
+
+    auto h1 = ceph_str_hash_rjenkins((const char*)hash_val1, hash_val_len);
+    auto h2 = ceph_str_hash_rjenkins((const char*)hash_val2, hash_val_len);
+    auto h3 = ceph_str_hash_linux((const char*)hash_val1, hash_val_len);
+    auto h4 = ceph_str_hash_linux((const char*)hash_val2, hash_val_len);
+    return ((h1 % num_buckets) == (h2 % num_buckets)) &&
+      ((h3 % num_buckets) == (h4 % num_buckets));
+  }
+
+  bool test_all_zero(const char* hash_val, size_t hash_val_len) const {
+    auto h = ceph_str_hash_rjenkins((const char*)hash_val, hash_val_len);
+    if (buckets1[h % num_buckets] != 0) {
+      return false;
+    }
+    h = ceph_str_hash_linux((const char*)hash_val, hash_val_len);
+    return buckets2[h % num_buckets] == 0;
+  }
+
+  // returns number of mismatching buckets
+  size_t count_non_zero() const {
+    return num_non_zero;
+  }
+  void reset() {
+    for (size_t i = 0; i < num_buckets; i++) {
+      buckets1[i] = 0;
+      buckets2[i] = 0;
+    }
+    num_non_zero = 0;
+  }
+};
+
+class shared_blob_2hash_tracker_t
+  : public ref_counter_2hash_tracker_t<mempool::bluestore_fsck::vector> {
+
+  static const size_t hash_input_len = 3;
+  typedef std::array<uint64_t, hash_input_len> hash_input_t;
+  inline hash_input_t build_hash_input(uint64_t sbid, uint64_t offset) const;
+  static size_t get_hash_input_size() {
+    return hash_input_len * sizeof(hash_input_t::value_type);
+  }
+
+  size_t au_void_bits = 0;
+
+public:
+  shared_blob_2hash_tracker_t(uint64_t mem_cap, size_t alloc_unit)
+    : ref_counter_2hash_tracker_t(mem_cap) {
+    ceph_assert(!!alloc_unit);
+    ceph_assert(isp2(alloc_unit));
+    au_void_bits = ctz(alloc_unit);
+  }
+  void inc(uint64_t sbid, uint64_t offset, int n);
+  void inc_range(uint64_t sbid, uint64_t offset, uint32_t len, int n);
+
+  bool test_hash_conflict(
+    uint64_t sbid,
+    uint64_t offset,
+    uint64_t sbid2,
+    uint64_t offset2) const;
+  bool test_all_zero(
+    uint64_t sbid,
+    uint64_t offset) const;
+  bool test_all_zero_range(
+    uint64_t sbid,
+    uint64_t offset,
+    uint32_t len) const;
+};
+
+class sb_info_t {
+  // subzero value indicates (potentially) stray blob,
+  // i.e. blob that has got no real references from onodes
+  int64_t sbid = 0;
+
+public:
+  enum {
+    INVALID_POOL_ID = INT64_MIN
+  };
+
+  int64_t pool_id = INVALID_POOL_ID;
+  // subzero value indicates compressed_allocated as well
+  int32_t allocated_chunks = 0;
+
+  sb_info_t(int64_t _sbid = 0) : sbid(_sbid)
+  {
+  }
+  bool operator< (const sb_info_t& other) const {
+    return std::abs(sbid) < std::abs(other.sbid);
+  }
+  bool operator< (const uint64_t& other_sbid) const {
+    return uint64_t(std::abs(sbid)) < other_sbid;
+  }
+  bool is_stray() const {
+    return sbid < 0;
+  }
+  uint64_t get_sbid() const {
+    return uint64_t(std::abs(sbid));
+  }
+  void adopt() {
+    sbid = std::abs(sbid);
+  }
+} __attribute__((packed));
+
+// Space-efficient container to keep a set of sb_info structures
+// given that the majority of entries are appended in a proper id-sorted
+// order. Hence one can keep them in a regular vector and apply binary search
+// whenever specific entry to be found.
+// For the rare occasions when out-of-order append takes place - an auxilliary
+// regular map is used.
+struct sb_info_space_efficient_map_t {
+  // large array sorted by the user
+  mempool::bluestore_fsck::vector<sb_info_t> items;
+  // small additional set of items we maintain sorting ourselves
+  // this would never keep an entry with id > items.back().id
+  mempool::bluestore_fsck::vector<sb_info_t> aux_items;
+
+  sb_info_t& add_maybe_stray(uint64_t sbid) {
+    return _add(-int64_t(sbid));
+  }
+  sb_info_t& add_or_adopt(uint64_t sbid) {
+    auto& r = _add(sbid);
+    r.adopt();
+    return r;
+  }
+  auto find(uint64_t id) {
+    if (items.size() != 0) {
+      auto it = std::lower_bound(
+       items.begin(),
+       items.end() - 1,
+       id,
+       [](const sb_info_t& a, const uint64_t& b) {
+         return a < b;
+       });
+      if (it->get_sbid() == id) {
+       return it;
+      }
+      if (aux_items.size() != 0) {
+       auto it = std::lower_bound(
+         aux_items.begin(),
+         aux_items.end() - 1,
+         id,
+         [](const sb_info_t& a, const uint64_t& b) {
+           return a < b;
+         });
+       if (it->get_sbid() == id) {
+         return it;
+       }
+      }
+    }
+    return items.end();
+  }
+  // enumerates strays, order isn't guaranteed.
+  void foreach_stray(std::function<void(const sb_info_t&)> cb) {
+    for (auto& sbi : items) {
+      if (sbi.is_stray()) {
+       cb(sbi);
+      }
+    }
+    for (auto& sbi : aux_items) {
+      if (sbi.is_stray()) {
+       cb(sbi);
+      }
+    }
+  }
+  auto end() {
+    return items.end();
+  }
+
+  void shrink() {
+    items.shrink_to_fit();
+    aux_items.shrink_to_fit();
+  }
+  void clear() {
+    items.clear();
+    aux_items.clear();
+    shrink();
+  }
+private:
+  sb_info_t& _add(int64_t id) {
+    if (items.size() == 0 || uint64_t(std::abs(id)) > items.back().get_sbid()) {
+      return items.emplace_back(id);
+    }
+    auto it = find(uint64_t(std::abs(id)));
+    if (it != items.end()) {
+      return *it;
+    }
+    if (aux_items.size() == 0 || uint64_t(std::abs(id)) > aux_items.back().get_sbid()) {
+      return aux_items.emplace_back(id);
+    }
+    // do sorted insertion, may be expensive!
+    it = std::upper_bound(
+      aux_items.begin(),
+      aux_items.end() - 1,
+      uint64_t(std::abs(id)),
+      [](const uint64_t& a, const sb_info_t& b) {
+       return a < b.get_sbid();
+      });
+    return *aux_items.emplace(it, id);
+  }
+};
 
 #endif
index 050fbe526ba4a8cdab279610e66b020e6e8dc574..029bdc1491737802d3a98e654649005b673a9523 100644 (file)
@@ -9100,7 +9100,7 @@ TEST_P(StoreTestSpecificAUSize, BluestoreRepairTest) {
     }
 
     bstore->umount();
-    ASSERT_EQ(bstore->fsck(false), 3);
+    ASSERT_EQ(bstore->fsck(false), 4);
     ASSERT_LE(bstore->repair(false), 0);
     ASSERT_EQ(bstore->fsck(false), 0);
   }
@@ -9266,6 +9266,7 @@ TEST_P(StoreTestSpecificAUSize, BluestoreBrokenNoSharedBlobRepairTest) {
   cerr << "injecting" << std::endl;
   sleep(3); // need some time for the previous write to land
   bstore->inject_no_shared_blob_key();
+  bstore->inject_stray_shared_blob_key(12345678);
 
   {
     cerr << "fscking/fixing" << std::endl;
@@ -9276,9 +9277,12 @@ TEST_P(StoreTestSpecificAUSize, BluestoreBrokenNoSharedBlobRepairTest) {
     // value
     size_t expected_error_count =
       g_ceph_context->_conf->bluestore_allocation_from_file ?
-      2 :
-      3;
+      5: // 4 sb ref mismatch errors + 1 statfs mismatch
+      7; // 4 sb ref mismatch errors + 1 statfs + 1 block leak + 1 non-free
     ASSERT_EQ(bstore->fsck(false), expected_error_count);
+    // repair might report less errors than fsck above showed
+    // as some errors, e.g. statfs mismatch, are implicitly fixed
+    // before the detection during the previous repair steps...
     ASSERT_LE(bstore->repair(false), expected_error_count);
     ASSERT_EQ(bstore->fsck(false), 0);
   }
index 8d5efbf4f8fcd862c42bded8a4531e77e753eef8..7cb0b4f3cb4e2a6c187813d963becea7ad4e52f5 100644 (file)
@@ -11,6 +11,7 @@
 #include "common/ceph_argparse.h"
 #include "global/global_init.h"
 #include "global/global_context.h"
+#include "perfglue/heap_profiler.h"
 
 #include <sstream>
 
@@ -33,6 +34,7 @@ TEST(bluestore, sizeof) {
   P(bluestore_onode_t);
   P(bluestore_blob_t);
   P(PExtentVector);
+  P(ghobject_t);
   P(bluestore_shared_blob_t);
   P(bluestore_extent_ref_map_t);
   P(bluestore_extent_ref_map_t::record_t);
@@ -44,10 +46,78 @@ TEST(bluestore, sizeof) {
   P(bufferlist);
   P(bufferptr);
   P(range_seg_t);
+  P(sb_info_t);
   cout << "map<uint64_t,uint64_t>\t" << sizeof(map<uint64_t,uint64_t>) << std::endl;
   cout << "map<char,char>\t" << sizeof(map<char,char>) << std::endl;
 }
 
+void dump_mempools()
+{
+  ostringstream ostr;
+  Formatter* f = Formatter::create("json-pretty", "json-pretty", "json-pretty");
+  ostr << "Mempools: ";
+  f->open_object_section("mempools");
+  mempool::dump(f);
+  f->close_section();
+  f->flush(ostr);
+  delete f;
+  cout << ostr.str() << std::endl;
+}
+
+TEST(sb_info_space_efficient_map_t, basic) {
+  sb_info_space_efficient_map_t sb_info;
+  const size_t num_shared = 1000;
+  for (size_t i = 0; i < num_shared; i += 2) {
+    auto& sbi = sb_info.add_maybe_stray(i);
+    sbi.pool_id = i;
+  }
+  ASSERT_TRUE(sb_info.find(0) != sb_info.end());
+  ASSERT_TRUE(sb_info.find(1) == sb_info.end());
+  ASSERT_TRUE(sb_info.find(2) != sb_info.end());
+  ASSERT_TRUE(sb_info.find(4)->pool_id == 4);
+  ASSERT_TRUE(sb_info.find(num_shared) == sb_info.end());
+
+  // ordered insertion
+  sb_info.add_or_adopt(num_shared).pool_id = num_shared;
+  ASSERT_TRUE(sb_info.find(num_shared) != sb_info.end());
+  ASSERT_TRUE(sb_info.find(num_shared)->pool_id == num_shared);
+
+  // out of order insertion
+  sb_info.add_or_adopt(1).pool_id = 1;
+  ASSERT_TRUE(sb_info.find(1) != sb_info.end());
+  ASSERT_TRUE(sb_info.find(1)->pool_id == 1);
+
+  // ordered insertion
+  sb_info.add_maybe_stray(num_shared + 1).pool_id = num_shared + 1;
+  ASSERT_TRUE(sb_info.find(num_shared + 1) != sb_info.end());
+  ASSERT_TRUE(sb_info.find(num_shared + 1)->pool_id == num_shared + 1);
+
+  // out of order insertion
+  sb_info.add_maybe_stray(105).pool_id = 105;
+  ASSERT_TRUE(sb_info.find(105) != sb_info.end());
+  ASSERT_TRUE(sb_info.find(105)->pool_id == 105);
+}
+
+TEST(sb_info_space_efficient_map_t, size) {
+  const size_t num_shared = 10000000;
+  sb_info_space_efficient_map_t sb_info;
+
+  BlueStore store(g_ceph_context, "", 4096);
+  BlueStore::OnodeCacheShard* oc = BlueStore::OnodeCacheShard::create(
+    g_ceph_context, "lru", NULL);
+  BlueStore::BufferCacheShard* bc = BlueStore::BufferCacheShard::create(
+    g_ceph_context, "lru", NULL);
+
+  auto coll = ceph::make_ref<BlueStore::Collection>(&store, oc, bc, coll_t());
+
+  for (size_t i = 0; i < num_shared; i++) {
+    auto& sbi = sb_info.add_or_adopt(i);
+    // primarily to silent the 'unused' warning
+    ceph_assert(sbi.pool_id == sb_info_t::INVALID_POOL_ID);
+  }
+  dump_mempools();
+}
+
 TEST(bluestore_extent_ref_map_t, add)
 {
   bluestore_extent_ref_map_t m;
@@ -1739,6 +1809,71 @@ TEST(bluestore_blob_t, wrong_map_bl_in_51682)
   }
 }
 
+TEST(shared_blob_2hash_tracker_t, basic_test)
+{
+  shared_blob_2hash_tracker_t t1(1024 * 1024, 4096);
+
+  ASSERT_TRUE(t1.count_non_zero() == 0);
+
+  t1.inc(0, 0, 1);
+  ASSERT_TRUE(t1.count_non_zero() != 0);
+  t1.inc(0, 0, -1);
+  ASSERT_TRUE(t1.count_non_zero() == 0);
+
+  t1.inc(3, 0x1000, 2);
+  ASSERT_TRUE(t1.count_non_zero() != 0);
+  t1.inc(3, 0x1000, -1);
+  ASSERT_TRUE(t1.count_non_zero() != 0);
+  t1.inc(3, 0x1000, -1);
+  ASSERT_TRUE(t1.count_non_zero() == 0);
+
+  t1.inc(2, 0x2000, 5);
+  ASSERT_TRUE(t1.count_non_zero() != 0);
+  t1.inc(18, 0x2000, -5);
+  ASSERT_TRUE(t1.count_non_zero() != 0);
+  t1.inc(18, 0x2000, 1);
+  ASSERT_TRUE(t1.count_non_zero() != 0);
+  t1.inc(2, 0x2000, -1);
+  ASSERT_TRUE(t1.count_non_zero() != 0);
+  t1.inc(18, 0x2000, 4);
+  ASSERT_TRUE(t1.count_non_zero() != 0);
+  t1.inc(2, 0x2000, -4);
+  ASSERT_TRUE(t1.count_non_zero() == 0);
+
+  t1.inc(3, 0x3000, 2);
+  ASSERT_TRUE(t1.count_non_zero() != 0);
+  t1.inc(4, 0x3000, -1);
+  ASSERT_TRUE(t1.count_non_zero() != 0);
+  t1.inc(4, 0x3000, -1);
+  ASSERT_TRUE(t1.count_non_zero() != 0);
+  t1.inc(3, 0x3000, -2);
+  ASSERT_TRUE(t1.count_non_zero() != 0);
+  t1.inc(4, 0x3000, 1);
+  ASSERT_TRUE(t1.count_non_zero() != 0);
+  t1.inc(4, 0x3000, 1);
+  ASSERT_TRUE(t1.count_non_zero() == 0);
+
+  t1.inc(5, 0x1000, 1);
+  t1.inc(5, 0x2000, 3);
+  t1.inc(5, 0x3000, 2);
+  t1.inc(5, 0x8000, 1);
+
+  ASSERT_TRUE(t1.count_non_zero() != 0);
+
+  ASSERT_TRUE(!t1.test_all_zero(5,0x1000));
+  ASSERT_TRUE(!t1.test_all_zero(5, 0x2000));
+  ASSERT_TRUE(!t1.test_all_zero(5, 0x3000));
+  ASSERT_TRUE(t1.test_all_zero(5, 0x4000));
+  ASSERT_TRUE(!t1.test_all_zero(5, 0x8000));
+
+  ASSERT_TRUE(t1.test_all_zero_range(5, 0, 0x1000));
+  ASSERT_TRUE(t1.test_all_zero_range(5, 0x500, 0x500));
+  ASSERT_TRUE(!t1.test_all_zero_range(5, 0x500, 0x1500));
+  ASSERT_TRUE(!t1.test_all_zero_range(5, 0x1500, 0x3200));
+  ASSERT_TRUE(t1.test_all_zero_range(5, 0x4500, 0x1500));
+  ASSERT_TRUE(t1.test_all_zero_range(5, 0x4500, 0x3b00));
+  ASSERT_TRUE(!t1.test_all_zero_range(5, 0, 0x9000));
+}
 int main(int argc, char **argv) {
   auto args = argv_to_vec(argc, argv);
   auto cct = global_init(NULL, args, CEPH_ENTITY_TYPE_CLIENT,