]> git.apps.os.sepia.ceph.com Git - ceph-ci.git/commitdiff
os/bluestore: shard extent map
authorSage Weil <sage@redhat.com>
Fri, 2 Sep 2016 16:50:39 +0000 (12:50 -0400)
committerSage Weil <sage@redhat.com>
Wed, 7 Sep 2016 15:26:05 +0000 (11:26 -0400)
Rewrote much of the persistence of onode metadata.  The
highlights:

 - extents and blobs stored together (the blob with the
   first referencing extent).
 - extents sharded across multiple k/v keys
 - if a blob if referenced from multiple blobs, it's
   stored in the onode key (called a "spanning blob").
 - when we clone a blob we copy the metadata, but mark
   it shared and put (just) the ref_map on the underlying
   blocks in a shared_blob key.  at this point we also
   assign a globally unique id (sbid = shared blob id)
   so the key has a unique name.
 - we instantiate a SharedBlob in memory regardless of
   whether we need to load the ref_map (which is only
   needed for deallocations!).  the BufferSpace is
   attached to this SharedBlob so we get unified caching
   across clones.

Signed-off-by: Sage Weil <sage@redhat.com>
src/common/config_opts.h
src/os/bluestore/BlueStore.cc
src/os/bluestore/BlueStore.h
src/os/bluestore/bluestore_types.cc
src/os/bluestore/bluestore_types.h
src/osd/osd_types.cc
src/test/objectstore/test_bluestore_types.cc

index 4a25c6f37a550f92a7140bbed573afa632dfcd61..fb42901547ef6fbb26cd7d50e6b462e3ed275dda 100644 (file)
@@ -976,9 +976,13 @@ OPTION(bluestore_compression_max_blob_size, OPT_U32, 4*1024*1024)
  * And ask for compressing at least 12.5%(1/8) off, by default.
  */
 OPTION(bluestore_compression_required_ratio, OPT_DOUBLE, .875)
+OPTION(bluestore_extent_map_shard_max_size, OPT_U32, 1200)
+OPTION(bluestore_extent_map_shard_target_size, OPT_U32, 500)
+OPTION(bluestore_extent_map_shard_min_size, OPT_U32, 150)
 OPTION(bluestore_cache_type, OPT_STR, "2q")   // lru, 2q
 OPTION(bluestore_onode_cache_size, OPT_U32, 16*1024)
 OPTION(bluestore_buffer_cache_size, OPT_U32, 512*1024*1024)
+OPTION(bluestore_shared_blob_hash_table_size_ratio, OPT_FLOAT, 2)  // multiple of onode_cache_size
 OPTION(bluestore_kvbackend, OPT_STR, "rocksdb")
 OPTION(bluestore_allocator, OPT_STR, "bitmap")     // stupid | bitmap
 OPTION(bluestore_freelist_type, OPT_STR, "bitmap") // extent | bitmap
@@ -1000,6 +1004,7 @@ OPTION(bluestore_max_bytes, OPT_U64, 64*1024*1024)
 OPTION(bluestore_wal_max_ops, OPT_U64, 512)
 OPTION(bluestore_wal_max_bytes, OPT_U64, 128*1024*1024)
 OPTION(bluestore_nid_prealloc, OPT_INT, 1024)
+OPTION(bluestore_blobid_prealloc, OPT_U64, 10240)
 OPTION(bluestore_overlay_max_length, OPT_INT, 65536)
 OPTION(bluestore_overlay_max, OPT_INT, 0)
 OPTION(bluestore_clone_cow, OPT_BOOL, false)  // do copy-on-write for clones
index 8f98650c804179f3ca5b623776cde91d0ff39f59..4bd7142a1d84238e9e62579a80253ba757994674 100644 (file)
@@ -40,6 +40,7 @@ const string PREFIX_OBJ = "O";     // object name -> onode_t
 const string PREFIX_OMAP = "M";    // u64 + keyname -> value
 const string PREFIX_WAL = "L";     // id -> wal_transaction_t
 const string PREFIX_ALLOC = "B";   // u64 offset -> u64 length (freelist)
+const string PREFIX_SHARED_BLOB = "X"; // u64 offset -> shared_blob_t
 
 // write a label in the first block.  always use this size.  note that
 // bluefs makes a matching assumption about the location of its
@@ -49,6 +50,21 @@ const string PREFIX_ALLOC = "B";   // u64 offset -> u64 length (freelist)
 // for bluefs, label (4k) + bluefs super (4k), means we start at 8k.
 #define BLUEFS_START  8192
 
+#define OBJECT_MAX_SIZE 0xffffffff // 32 bits
+
+
+/*
+ * extent map blob encoding
+ *
+ * we use the low bits of the blobid field to indicate some common scenarios
+ * and spanning vs local ids.  See ExtentMap::{encode,decode}_some().
+ */
+#define BLOBID_FLAG_CONTIGUOUS 0x1  // this extent starts at end of previous
+#define BLOBID_FLAG_ZEROOFFSET 0x2  // blob_offset is 0
+#define BLOBID_FLAG_SAMELENGTH 0x4  // length matches previous extent
+#define BLOBID_FLAG_SPANNING   0x8  // has spanning blob id
+#define BLOBID_SHIFT_BITS        4
+
 /*
  * object name key structure
  *
@@ -83,6 +99,16 @@ const string PREFIX_ALLOC = "B";   // u64 offset -> u64 length (freelist)
  *
  */
 
+/*
+ * extent shard key
+ *
+ * object prefix key
+ * u32
+ * 'x'  (this char can be anything that is not part of a u64 (hex)
+ *       encoding so we can distinguish this key from an object key)
+ */
+#define EXTENT_SHARD_KEY_SUFFIX 'x'
+
 static void append_escaped(const string &in, string *out)
 {
   char hexbyte[8];
@@ -251,34 +277,18 @@ static void get_coll_key_range(const coll_t& cid, int bits,
   }
 }
 
-static bool is_bnode_key(const string& key)
-{
-  if (key.size() == 2 + 8 + 4)
-    return true;
-  return false;
-}
-
-static void get_bnode_key(shard_id_t shard, int64_t pool, uint32_t hash,
-                         string *key)
+static void get_shared_blob_key(uint64_t sbid, string *key)
 {
   key->clear();
-  _key_encode_shard(shard, key);
-  _key_encode_u64(pool + 0x8000000000000000ull, key);
-  _key_encode_u32(hobject_t::_reverse_bits(hash), key);
+  _key_encode_u64(sbid, key);
 }
 
-static int get_key_bnode(const string& key, shard_id_t *shard,
-                        int64_t *pool, uint32_t *hash)
+static int get_key_shared_blob(const string& key, uint64_t *sbid)
 {
   const char *p = key.c_str();
-  if (key.length() < 2 + 8 + 4)
+  if (key.length() < sizeof(uint64_t))
     return -1;
-  p = _key_decode_shard(p, shard);
-  p = _key_decode_u64(p, (uint64_t*)pool);
-  *pool -= 0x8000000000000000ull;
-  uint32_t hash_reverse_bits;
-  p = _key_decode_u32(p, &hash_reverse_bits);
-  *hash = hobject_t::_reverse_bits(hash_reverse_bits);
+  p = _key_decode_u64(p, sbid);
   return 0;
 }
 
@@ -391,11 +401,36 @@ static int get_key_object(const string& key, ghobject_t *oid)
     // if we get something other than a null terminator here, 
     // something goes wrong.
     return -8;
-  }  
+  }
 
   return 0;
 }
 
+// extent shard keys are the onode key, plus a u32, plus 'x'.  the trailing
+// char lets us quickly test whether it is a shard key without decoding any
+// of the prefix bytes.
+static void get_extent_shard_key(const string& onode_key, uint32_t offset,
+                                string *key)
+{
+  key->clear();
+  key->append(onode_key);
+  _key_encode_u32(offset, key);
+  key->push_back(EXTENT_SHARD_KEY_SUFFIX);
+}
+
+int get_key_extent_shard(const string& key, string *onode_key, uint32_t *offset)
+{
+  assert(key.size() > sizeof(uint32_t) + 1);
+  assert(key[key.size() - 1] == EXTENT_SHARD_KEY_SUFFIX);
+  const char *p = key.data() + key.size() - sizeof(uint32_t) - 1;
+  p = _key_decode_u32(p, offset);
+  return 0;
+}
+
+static bool is_extent_shard_key(const string& key)
+{
+  return key[key.size()-1] == EXTENT_SHARD_KEY_SUFFIX;
+}
 
 // '-' < '.' < '~'
 static void get_omap_header(uint64_t id, string *out)
@@ -555,7 +590,6 @@ void BlueStore::LRUCache::trim(uint64_t onode_max, uint64_t buffer_max)
     }
     o->get();  // paranoia
     o->space->onode_map.erase(o->oid);
-    o->blob_map._clear();    // clear blobs and their buffers, too
     o->put();
     --num;
   }
@@ -826,7 +860,6 @@ void BlueStore::TwoQCache::trim(uint64_t onode_max, uint64_t buffer_max)
     }
     o->get();  // paranoia
     o->space->onode_map.erase(o->oid);
-    o->blob_map._clear();    // clear blobs and their buffers, too
     o->put();
     --num;
   }
@@ -1104,9 +1137,6 @@ void BlueStore::OnodeSpace::clear()
   dout(10) << __func__ << dendl;
   for (auto &p : onode_map) {
     cache->_rm_onode(p.second);
-
-    // clear blobs and their buffers too, while we have cache->lock
-    p.second->blob_map._clear();
   }
   onode_map.clear();
 }
@@ -1131,7 +1161,7 @@ void BlueStore::OnodeSpace::rename(OnodeRef& oldo,
   OnodeRef o = po->second;
 
   // install a non-existent onode at old location
-  oldo.reset(new Onode(this, old_oid, o->key));
+  oldo.reset(new Onode(this, o->c, old_oid, o->key));
   po->second = oldo;
   cache->_add_onode(po->second, 1);
 
@@ -1155,11 +1185,76 @@ bool BlueStore::OnodeSpace::map_any(std::function<bool(OnodeRef)> f)
 }
 
 
+// SharedBlob
+
+#undef dout_prefix
+#define dout_prefix *_dout << "bluestore.sharedblob(" << this << ") "
+
+ostream& operator<<(ostream& out, const BlueStore::SharedBlob& sb)
+{
+  out << "SharedBlob(" << &sb;
+  if (sb.parent_set) {
+    out << " shared sbid 0x" << std::hex << sb.sbid << std::dec
+       << " parent_set " << sb.parent_set;
+  }
+  if (sb.loaded) {
+    out << " loaded " << sb.shared_blob;
+  }
+  return out << ")";
+}
+
+void BlueStore::SharedBlob::put()
+{
+  if (--nref == 0) {
+    dout(20) << __func__ << " " << this
+            << " removing self from set " << parent_set << dendl;
+    if (parent_set) {
+      if (parent_set->remove(this)) {
+       delete this;
+      } else {
+       dout(20) << __func__ << " " << this
+                << " lost race to remove myself from set" << dendl;
+      }
+    }
+  }
+}
+
+
+// SharedBlobSet
+
+#undef dout_prefix
+#define dout_prefix *_dout << "bluestore.sharedblobset(" << this << ") "
+
+BlueStore::SharedBlobRef BlueStore::SharedBlobSet::lookup(uint64_t sbid)
+{
+  std::lock_guard<std::mutex> l(lock);
+  dummy.sbid = sbid;
+  auto p = uset.find(dummy);
+  if (p == uset.end()) {
+    return nullptr;
+  }
+  return &*p;
+}
+
+
 // Blob
 
 #undef dout_prefix
 #define dout_prefix *_dout << "bluestore.blob(" << this << ") "
 
+ostream& operator<<(ostream& out, const BlueStore::Blob& b)
+{
+  out << "Blob(" << &b;
+  if (b.id >= 0) {
+    out << " spanning " << b.id;
+  }
+  out << " " << b.get_blob() << " " << b.ref_map
+      << (b.is_dirty() ? " (dirty)" : " (clean)")
+      << " " << *b.shared_blob
+      << ")";
+  return out;
+}
+
 void BlueStore::Blob::discard_unallocated()
 {
   get_blob();
@@ -1177,81 +1272,726 @@ void BlueStore::Blob::discard_unallocated()
     assert(discard == all_invalid); // in case of compressed blob all
                                    // or none pextents are invalid.
     if (discard) {
-      bc.discard(0, blob.get_compressed_payload_original_length());
+      shared_blob->bc.discard(0, blob.get_compressed_payload_original_length());
     }
   } else {
     for (auto e : blob.extents) {
       if (!e.is_valid()) {
-        bc.discard(pos, e.length);
+        shared_blob->bc.discard(pos, e.length);
+      }
+      pos += e.length;
+    }
+  }
+}
+
+void BlueStore::Blob::get_ref(
+  uint64_t offset,
+  uint64_t length)
+{
+  ref_map.get(offset, length);
+}
+
+bool BlueStore::Blob::put_ref(
+  uint64_t offset,
+  uint64_t length,
+  uint64_t min_release_size,
+  vector<bluestore_pextent_t> *r)
+{
+  vector<bluestore_pextent_t> logical;
+  ref_map.put(offset, length, &logical);
+  r->clear();
+
+  bluestore_blob_t& b = dirty_blob();
+
+  // common case: all of it?
+  if (ref_map.empty()) {
+    uint64_t pos = 0;
+    for (auto& e : b.extents) {
+      if (e.is_valid()) {
+       r->push_back(e);
       }
       pos += e.length;
     }
+    b.extents.resize(1);
+    b.extents[0].offset = bluestore_pextent_t::INVALID_OFFSET;
+    b.extents[0].length = pos;
+    return true;
+  }
+
+  // we cannot do partial deallocation on compressed blobs
+  if (b.has_flag(bluestore_blob_t::FLAG_COMPRESSED)) {
+    return false;
+  }
+
+  // we cannot release something smaller than our csum chunk size
+  if (b.has_csum() && b.get_csum_chunk_size() > min_release_size) {
+    min_release_size = b.get_csum_chunk_size();
   }
+
+  // search from logical releases
+  for (auto le : logical) {
+    uint64_t r_off = le.offset;
+    auto p = ref_map.ref_map.lower_bound(le.offset);
+    if (p != ref_map.ref_map.begin()) {
+      --p;
+      r_off = p->first + p->second.length;
+      ++p;
+    } else {
+      r_off = 0;
+    }
+    uint64_t end;
+    if (p == ref_map.ref_map.end()) {
+      end = b.get_ondisk_length();
+    } else {
+      end = p->first;
+    }
+    r_off = ROUND_UP_TO(r_off, min_release_size);
+    end -= end % min_release_size;
+    if (r_off >= end) {
+      continue;
+    }
+    uint64_t r_len = end - r_off;
+
+    // cut it out of extents
+    struct vecbuilder {
+      vector<bluestore_pextent_t> v;
+      uint64_t invalid = 0;
+
+      void add_invalid(uint64_t length) {
+       invalid += length;
+      }
+      void flush() {
+       if (invalid) {
+         v.emplace_back(bluestore_pextent_t(bluestore_pextent_t::INVALID_OFFSET,
+                                            invalid));
+         invalid = 0;
+       }
+      }
+      void add(uint64_t offset, uint64_t length) {
+       if (offset == bluestore_pextent_t::INVALID_OFFSET) {
+         add_invalid(length);
+       } else {
+         flush();
+         v.emplace_back(bluestore_pextent_t(offset, length));
+       }
+      }
+    } vb;
+
+    assert(r_len > 0);
+    auto q = b.extents.begin();
+    assert(q != b.extents.end());
+    while (r_off >= q->length) {
+      vb.add(q->offset, q->length);
+      r_off -= q->length;
+      ++q;
+      assert(q != b.extents.end());
+    }
+    while (r_len > 0) {
+      uint64_t l = MIN(r_len, q->length - r_off);
+      if (q->is_valid()) {
+       r->emplace_back(bluestore_pextent_t(q->offset + r_off, l));
+      }
+      if (r_off) {
+       vb.add(q->offset, r_off);
+      }
+      vb.add_invalid(l);
+      if (r_off + l < q->length) {
+       vb.add(q->offset + r_off + l, q->length - (r_off + l));
+      }
+      r_len -= l;
+      r_off = 0;
+      ++q;
+      assert(q != b.extents.end() || r_len == 0);
+    }
+    while (q != b.extents.end()) {
+      vb.add(q->offset, q->length);
+      ++q;
+    }
+    vb.flush();
+    b.extents.swap(vb.v);
+  }
+  return false;
 }
 
-// BlobMap
+
+// Extent
+
+ostream& operator<<(ostream& out, const BlueStore::Extent& e)
+{
+  return out << std::hex << "0x" << e.logical_offset << "~" << e.length
+            << ": 0x" << e.blob_offset << "~" << e.length << std::dec
+            << " " << *e.blob;
+}
+
+// ExtentMap
 
 #undef dout_prefix
-#define dout_prefix *_dout << "bluestore.blobmap(" << this << ") "
+#define dout_prefix *_dout << "bluestore.extentmap(" << this << ") "
+
+bool BlueStore::ExtentMap::update(Onode *o, KeyValueDB::Transaction t,
+                                 bool force)
+{
+  if (o->onode.extent_map_shards.empty()) {
+    if (inline_bl.length() == 0) {
+      unsigned n;
+      if (encode_some(0, OBJECT_MAX_SIZE, inline_bl, &n)) {
+       return true;
+      }
+      size_t len = inline_bl.length();
+      dout(20) << __func__ << " inline shard "
+              << len << " bytes from " << n << " extents" << dendl;
+      if (!force && len > g_conf->bluestore_extent_map_shard_max_size) {
+       return true;
+      }
+    }
+    // will persist in the onode key, see below
+  } else {
+    auto p = shards.begin();
+    while (p != shards.end()) {
+      auto n = p;
+      ++n;
+      if (p->dirty) {
+       uint32_t endoff;
+       if (n == shards.end()) {
+         endoff = OBJECT_MAX_SIZE;
+       } else {
+         endoff = n->offset;
+       }
+       bufferlist bl;
+       unsigned n;
+       if (encode_some(p->offset, endoff - p->offset, bl, &n)) {
+         return true;
+       }
+       dout(20) << __func__ << " shard 0x" << std::hex
+                << p->offset << std::dec << " is " << bl.length()
+                << " bytes (was " << p->shard_info->bytes << ") from " << n
+                << " extents" << dendl;
+       p->shard_info->bytes = bl.length();
+       p->shard_info->extents = n;
+       if (!force &&
+           (bl.length() > g_conf->bluestore_extent_map_shard_max_size ||
+            bl.length() < g_conf->bluestore_extent_map_shard_min_size)) {
+         return true;
+       }
+       t->set(PREFIX_OBJ, p->key, bl);
+       p->dirty = false;
+      }
+      p = n;
+    }
+  }
+  return false;
+}
+
+void BlueStore::ExtentMap::reshard(Onode *o)
+{
+  // un-span all blobs
+  auto p = spanning_blob_map.begin();
+  while (p != spanning_blob_map.end()) {
+    auto n = spanning_blob_map.erase(p);
+    p->id = -1;
+    p->put();
+    p = n;
+  }
 
-void BlueStore::BlobMap::encode(bufferlist& bl) const
+  if (extent_map.size() <= 1) {
+    dout(20) << __func__ << " <= 1 extent, going inline" << dendl;
+    shards.clear();
+    o->onode.extent_map_shards.clear();
+    return;
+  }
+
+  unsigned bytes = 0;
+  if (o->onode.extent_map_shards.empty()) {
+    bytes = inline_bl.length();
+  } else {
+    for (auto &s : o->onode.extent_map_shards) {
+      bytes += s.bytes;
+    }
+  }
+  unsigned target = g_conf->bluestore_extent_map_shard_target_size;
+  unsigned extent_avg = bytes / extent_map.size();
+  dout(20) << __func__ << " extent_avg " << extent_avg
+          << " target " << target << dendl;
+
+  // reshard
+  auto ep = extent_map.begin();
+  auto sp = o->onode.extent_map_shards.begin();
+  auto esp = o->onode.extent_map_shards.end();
+  unsigned shard_end = 0;
+  unsigned estimate = 0;
+  unsigned offset = 0;
+  vector<bluestore_onode_t::shard_info> new_shard_info;
+  while (ep != extent_map.end()) {
+    dout(30) << " ep " << *ep << dendl;
+    if (shard_end == 0 ||
+       ep->logical_offset >= shard_end) {
+      if (sp == esp) {
+       // inline case
+       shard_end = o->onode.size;
+      } else {
+       auto next = sp;
+       ++next;
+       if (next == esp) {
+         shard_end = o->onode.size;
+       } else {
+         shard_end = next->offset;
+       }
+       sp = next;
+      }
+      dout(20) << __func__ << " old shard end 0x" << std::hex << shard_end
+              << std::dec << dendl;
+    }
+    if (estimate && estimate + extent_avg > target) {
+      // new shard
+      if (offset == 0) {
+       new_shard_info.emplace_back(bluestore_onode_t::shard_info());
+       new_shard_info.back().offset = offset;
+       dout(20) << __func__ << "  new shard 0x" << std::hex << offset
+                << std::dec << dendl;
+      }
+      offset = ep->logical_offset;
+      new_shard_info.emplace_back(bluestore_onode_t::shard_info());
+      new_shard_info.back().offset = offset;
+      dout(20) << __func__ << "  new shard 0x" << std::hex << offset << std::dec
+              << dendl;
+      estimate = 0;
+    }
+    estimate += extent_avg;
+    ++ep;
+  }
+  o->onode.extent_map_shards.swap(new_shard_info);
+
+  // set up new shards vector; ensure shards/inline both dirty/invalidated.
+  init_shards(o, true, true);
+  inline_bl.clear();
+
+  // identify spanning blobs
+  if (!o->onode.extent_map_shards.empty()) {
+    dout(20) << __func__ << " checking for spanning blobs" << dendl;
+    auto ep = extent_map.begin();
+    auto sp = o->onode.extent_map_shards.begin();
+    auto esp = o->onode.extent_map_shards.end();
+    unsigned shard_start = 0;
+    ++sp;
+    unsigned shard_end;
+    if (sp == esp) {
+      shard_end = OBJECT_MAX_SIZE;
+    } else {
+      shard_end = sp->offset;
+    }
+    int bid = 0;
+    while (ep != extent_map.end()) {
+      dout(30) << " ep " << *ep << dendl;
+      while (ep->logical_offset >= shard_end) {
+       shard_start = shard_end;
+       ++sp;
+       if (sp == esp) {
+         shard_end = OBJECT_MAX_SIZE;
+       } else {
+         shard_end = sp->offset;
+       }
+       dout(30) << __func__ << "  shard 0x" << std::hex << shard_start
+                << " to 0x" << shard_end << std::dec << dendl;
+      }
+      if (ep->blob->id < 0 &&
+         ep->blob_escapes_range(shard_start, shard_end - shard_start)) {
+       ep->blob->id = bid++;
+       spanning_blob_map.insert(*ep->blob);
+       ep->blob->get();
+       dout(20) << __func__ << "   adding spanning " << *ep << dendl;
+      }
+      ++ep;
+    }
+  }
+}
+
+bool BlueStore::ExtentMap::encode_some(uint32_t offset, uint32_t length,
+                                      bufferlist& bl, unsigned *pn)
 {
-  uint32_t n = blob_map.size();
-  ::encode(n, bl);
-  for (auto p = blob_map.begin(); n--; ++p) {
-    ::encode(p->id, bl);
-    p->encode(bl);
+  Extent dummy(offset);
+  auto start = extent_map.lower_bound(dummy);
+  uint32_t end = offset + length;
+
+  unsigned n = 0;
+  for (auto p = start;
+       p != extent_map.end() && p->logical_offset < end;
+       ++p, ++n) {
+    assert(p->logical_offset >= offset);
+    p->blob->last_encoded_id = -1;
+    if (p->blob->id < 0 && p->blob_escapes_range(offset, length)) {
+      dout(30) << __func__ << " 0x" << std::hex << offset << "~" << length
+              << std::dec << " hit new spanning blob " << *p << dendl;
+      return true;
+    }
+  }
+  small_encode_varint(n, bl);
+  if (pn) {
+    *pn = n;
   }
+
+  n = 0;
+  uint64_t pos = 0;
+  uint64_t prev_len = 0;
+  for (auto p = start;
+       p != extent_map.end() && p->logical_offset < end;
+       ++p, ++n) {
+    unsigned blobid;
+    bool include_blob = false;
+    if (p->blob->id >= 0) {
+      blobid = p->blob->id << BLOBID_SHIFT_BITS;
+      blobid |= BLOBID_FLAG_SPANNING;
+    } else if (p->blob->last_encoded_id < 0) {
+      p->blob->last_encoded_id = n + 1;  // so it is always non-zero
+      include_blob = true;
+      blobid = 0;  // the decoder will infer the id from n
+    } else {
+      blobid = p->blob->last_encoded_id << BLOBID_SHIFT_BITS;
+    }
+    if (p->logical_offset == pos) {
+      blobid |= BLOBID_FLAG_CONTIGUOUS;
+    }
+    if (p->blob_offset == 0) {
+      blobid |= BLOBID_FLAG_ZEROOFFSET;
+    }
+    if (p->length == prev_len) {
+      blobid |= BLOBID_FLAG_SAMELENGTH;
+    } else {
+      prev_len = p->length;
+    }
+    small_encode_varint(blobid, bl);
+    if ((blobid & BLOBID_FLAG_CONTIGUOUS) == 0) {
+      small_encode_varint_lowz(p->logical_offset - pos, bl);
+    }
+    if ((blobid & BLOBID_FLAG_ZEROOFFSET) == 0) {
+      small_encode_varint_lowz(p->blob_offset, bl);
+    }
+    if ((blobid & BLOBID_FLAG_SAMELENGTH) == 0) {
+      small_encode_varint_lowz(p->length, bl);
+    }
+    pos = p->logical_offset + p->length;
+    if (include_blob) {
+      p->blob->encode(bl);
+    }
+  }
+  /*
+  derr << __func__ << ":";
+  bl.hexdump(*_dout);
+  *_dout << dendl;
+  */
+  return false;
 }
 
-void BlueStore::BlobMap::decode(bufferlist::iterator& p, Cache *c)
+void BlueStore::ExtentMap::decode_some(bufferlist& bl)
 {
-  assert(blob_map.empty());
-  uint32_t n;
-  ::decode(n, p);
+/*  derr << __func__ << ":";
+  bl.hexdump(*_dout);
+  *_dout << dendl;
+  */
+  bufferlist::iterator p = bl.begin();
+  uint32_t num;
+  small_decode_varint(num, p);
+  vector<BlobRef> blobs(num);
+  uint64_t pos = 0;
+  uint64_t prev_len = 0;
+  unsigned n = 0;
+  while (!p.end()) {
+    Extent *le = new Extent();
+    uint64_t blobid;
+    small_decode_varint(blobid, p);
+    if ((blobid & BLOBID_FLAG_CONTIGUOUS) == 0) {
+      uint64_t gap;
+      small_decode_varint_lowz(gap, p);
+      pos += gap;
+    }
+    le->logical_offset = pos;
+    if ((blobid & BLOBID_FLAG_ZEROOFFSET) == 0) {
+      small_decode_varint_lowz(le->blob_offset, p);
+    } else {
+      le->blob_offset = 0;
+    }
+    if ((blobid & BLOBID_FLAG_SAMELENGTH) == 0) {
+      small_decode_varint_lowz(prev_len, p);
+    }
+    le->length = prev_len;
+    if (blobid & BLOBID_FLAG_SPANNING) {
+      le->blob = get_spanning_blob(blobid >> BLOBID_SHIFT_BITS);
+    } else {
+      blobid >>= BLOBID_SHIFT_BITS;
+      if (blobid) {
+       le->blob = blobs[blobid - 1];
+       assert(le->blob);
+      } else {
+       le->blob = new Blob();
+       le->blob->decode(p);
+       blobs[n] = le->blob;
+       onode->c->open_shared_blob(le->blob);
+      }
+      // we build ref_map dynamically for non-spanning blobs
+      le->blob->ref_map.get(le->blob_offset, le->length);
+    }
+    pos += prev_len;
+    ++n;
+    extent_map.insert(*le);
+  }
+}
+
+void BlueStore::ExtentMap::encode_spanning_blobs(bufferlist& bl)
+{
+  unsigned n = spanning_blob_map.size();
+  small_encode_varint(n, bl);
+  for (auto& b : spanning_blob_map) {
+    small_encode_varint(b.id, bl);
+    b.encode(bl);
+    b.ref_map.encode(bl);
+  }
+}
+
+void BlueStore::ExtentMap::decode_spanning_blobs(
+  Collection *c,
+  bufferlist::iterator& p)
+{
+  unsigned n;
+  small_decode_varint(n, p);
   while (n--) {
-    int64_t id;
-    ::decode(id, p);
-    Blob *b = new Blob(id, c);
-    b->decode(p);
+    BlobRef b(new Blob());
+    small_decode_varint(b->id, p);
+    spanning_blob_map.insert(*b);
     b->get();
-    blob_map.insert(*b);
+    b->decode(p);
+    b->ref_map.decode(p);
+    c->open_shared_blob(b);
   }
 }
 
-// Bnode
+BlueStore::BlobRef BlueStore::ExtentMap::get_spanning_blob(
+  int id)
+{
+  Blob dummy;
+  dummy.id = id;
+  auto p = spanning_blob_map.find(dummy);
+  assert(p != spanning_blob_map.end());
+  return &*p;
+}
 
-#undef dout_prefix
-#define dout_prefix *_dout << "bluestore.bnode(" << this << ") "
+void BlueStore::ExtentMap::init_shards(Onode *on, bool loaded, bool dirty)
+{
+  shards.resize(on->onode.extent_map_shards.size());
+  unsigned i = 0;
+  for (auto &s : on->onode.extent_map_shards) {
+    get_extent_shard_key(on->key, s.offset, &shards[i].key);
+    shards[i].offset = s.offset;
+    shards[i].shard_info = &s;
+    shards[i].loaded = loaded;
+    shards[i].dirty = dirty;
+    ++i;
+  }
+}
 
-void BlueStore::Bnode::put()
+void BlueStore::ExtentMap::fault_range(
+  KeyValueDB *db,
+  uint32_t offset,
+  uint32_t length)
 {
-  if (--nref == 0) {
-    dout(20) << __func__ << " removing self from set " << bnode_set << dendl;
-    if (bnode_set->remove(this)) {
-      delete this;
-    } else {
-      dout(20) << __func__ << " lost race to remove myself from set" << dendl;
+  dout(30) << __func__ << " 0x" << std::hex << offset << "~" << length
+          << std::dec << dendl;
+  faulted = true;
+  auto p = seek_shard(offset);
+  auto last = seek_shard(offset + length);
+  while (p != shards.end()) {
+    if (!p->loaded) {
+      string key;
+      get_extent_shard_key(onode->key, p->offset, &key);
+      bufferlist v;
+      int r = db->get(PREFIX_OBJ, key, &v);
+      if (r < 0) {
+       derr << __func__ << " missing shard 0x" << std::hex << p->offset
+            << std::dec << " for " << onode->oid << dendl;
+       assert(r >= 0);
+      }
+      decode_some(v);
+      p->loaded = true;
+      dout(20) << __func__ << " open shard 0x" << std::hex << p->offset
+              << std::dec << " (" << v.length() << " bytes)" << dendl;
+      assert(p->dirty == false);
+      assert(v.length() == p->shard_info->bytes);
+    }
+    if (p == last) {
+      break;
+    }
+    ++p;
+  }
+}
+
+void BlueStore::ExtentMap::dirty_range(
+  KeyValueDB::Transaction t,
+  uint32_t offset,
+  uint32_t length)
+{
+  dout(30) << __func__ << " 0x" << std::hex << offset << "~" << length
+          << std::dec << dendl;
+  assert(faulted);
+  if (shards.empty()) {
+    dout(20) << __func__ << " mark inline shard dirty" << dendl;
+    inline_bl.clear();
+    return;
+  }
+  auto p = seek_shard(offset);
+  auto last = seek_shard(offset + length);
+  while (p != shards.end()) {
+    if (!p->loaded) {
+      dout(20) << __func__ << " shard 0x" << std::hex << p->offset << std::dec
+              << " is not loaded, can't mark dirty" << dendl;
+      assert(0 == "can't mark unloaded shard dirty");
+    }
+    if (!p->dirty) {
+      dout(20) << __func__ << " mark shard 0x" << std::hex << p->offset
+              << std::dec << " dirty" << dendl;
+      p->dirty = true;
+    }
+    if (p == last) {
+      break;
+    }
+    ++p;
+  }
+}
+
+BlueStore::extent_map_t::iterator BlueStore::ExtentMap::find_lextent(
+  uint64_t offset)
+{
+  assert(faulted);
+  Extent dummy(offset);
+  auto fp = extent_map.lower_bound(dummy);
+  if (fp != extent_map.begin()) {
+    --fp;
+    if (fp->logical_offset + fp->length <= offset) {
+      ++fp;
+    }
+  }
+  if (fp != extent_map.end() && fp->logical_offset > offset)
+    return extent_map.end();  // extent is past offset
+  return fp;
+}
+
+BlueStore::extent_map_t::iterator BlueStore::ExtentMap::seek_lextent(
+  uint64_t offset)
+{
+  assert(faulted);
+  Extent dummy(offset);
+  auto fp = extent_map.lower_bound(dummy);
+  if (fp != extent_map.begin()) {
+    --fp;
+    if (fp->logical_offset + fp->length <= offset) {
+      ++fp;
+    }
+  }
+  return fp;
+}
+
+bool BlueStore::ExtentMap::has_any_lextents(uint64_t offset, uint64_t length)
+{
+  assert(faulted);
+  Extent dummy(offset);
+  auto fp = extent_map.lower_bound(dummy);
+  if (fp != extent_map.begin()) {
+    --fp;
+    if (fp->logical_offset + fp->length <= offset) {
+      ++fp;
+    }
+  }
+  if (fp == extent_map.end() || fp->logical_offset >= offset + length) {
+    return false;
+  }
+  return true;
+}
+
+int BlueStore::ExtentMap::compress_extent_map()
+{
+  if (extent_map.empty())
+    return 0;
+  int removed = 0;
+  auto p = extent_map.begin();
+  auto n = p;
+  for (++n; n != extent_map.end(); p = n++) {
+    while (n != extent_map.end() &&
+          p->logical_offset + p->length == n->logical_offset &&
+          p->blob == n->blob &&
+          p->blob_offset + p->length == n->blob_offset) {
+      p->length += n->length;
+      extent_map.erase(n++);
+      ++removed;
+    }
+    if (n == extent_map.end()) {
+      break;
+    }
+  }
+  return removed;
+}
+
+void BlueStore::ExtentMap::punch_hole(
+  uint64_t offset,
+  uint64_t length,
+  extent_map_t *old_extents)
+{
+  auto p = seek_lextent(offset);
+  uint64_t end = offset + length;
+  while (p != extent_map.end()) {
+    if (p->logical_offset >= end) {
+      break;
+    }
+    if (p->logical_offset < offset) {
+      if (p->logical_offset + p->length > end) {
+       // split and deref middle
+       uint64_t front = offset - p->logical_offset;
+       old_extents->insert(
+         *new Extent(offset, p->blob_offset + front, length, p->blob));
+       extent_map.insert(*new Extent(end,
+                                     p->blob_offset + front + length,
+                                     p->length - front - length,
+                                     p->blob));
+       p->length = front;
+       break;
+      } else {
+       // deref tail
+       assert(p->logical_offset + p->length > offset); // else seek_lextent bug
+       uint64_t keep = offset - p->logical_offset;
+       old_extents->insert(*new Extent(offset, p->blob_offset + keep,
+                                       p->length - keep, p->blob));
+       p->length = keep;
+       ++p;
+       continue;
+      }
+    }
+    if (p->logical_offset + p->length <= end) {
+      // deref whole lextent
+      old_extents->insert(*new Extent(p->logical_offset, p->blob_offset,
+                                     p->length, p->blob));
+      extent_map.erase(p++);
+      continue;
     }
+    // deref head
+    uint64_t keep = (p->logical_offset + p->length) - end;
+    old_extents->insert(*new Extent(p->logical_offset, p->blob_offset,
+                                   p->length - keep, p->blob));
+    extent_map.insert(*new Extent(end, p->blob_offset + p->length - keep, keep,
+                                 p->blob));
+    extent_map.erase(p);
+    break;
   }
 }
 
-// BnodeSet
-
-#undef dout_prefix
-#define dout_prefix *_dout << "bluestore.bnodeset(" << this << ") "
-
-BlueStore::BnodeRef BlueStore::BnodeSet::get(uint32_t hash)
+BlueStore::Extent *BlueStore::ExtentMap::set_lextent(
+  uint64_t logical_offset,
+  uint64_t offset, uint64_t length, BlobRef b,
+  extent_map_t *old_extents)
 {
-  std::lock_guard<std::mutex> l(lock);
-  dummy.hash = hash;
-  auto p = uset.find(dummy);
-  if (p == uset.end()) {
-    return nullptr;
-  }
-  return &*p;
+  punch_hole(logical_offset, length, old_extents);
+  b->ref_map.get(offset, length);
+  Extent *le = new Extent(logical_offset, offset, length, b);
+  extent_map.insert(*le);
+  return le;
 }
 
+
 // Onode
 
 #undef dout_prefix
@@ -1275,51 +2015,80 @@ void BlueStore::Onode::flush()
 #undef dout_prefix
 #define dout_prefix *_dout << "bluestore(" << store->path << ").collection(" << cid << ") "
 
-BlueStore::Collection::Collection(BlueStore *ns, Cache *cs, coll_t c)
+BlueStore::Collection::Collection(BlueStore *ns, Cache *c, coll_t cid)
   : store(ns),
-    cache(cs),
-    cid(c),
+    cache(c),
+    cid(cid),
     lock("BlueStore::Collection::lock", true, false),
     exists(true),
-    bnode_set(MAX(16, g_conf->bluestore_onode_cache_size / 128)),
-    onode_map(cs)
+    // size the shared blob hash table as a ratio of the onode cache size.
+    shared_blob_set(MAX(16,
+                       g_conf->bluestore_onode_cache_size *
+                       g_conf->bluestore_shared_blob_hash_table_size_ratio)),
+    onode_map(c)
 {
 }
 
-BlueStore::BnodeRef BlueStore::Collection::get_bnode(
-  uint32_t hash
-  )
+void BlueStore::Collection::open_shared_blob(BlobRef b)
 {
-  BnodeRef b = bnode_set.get(hash);
-  if (b) {
-    dout(10) << __func__ << " hash " << std::hex << hash << std::dec
-            << " had " << b << dendl;
-    return b;
+  assert(!b->shared_blob);
+  const bluestore_blob_t& blob = b->get_blob();
+  if (!blob.is_shared()) {
+    b->shared_blob = new SharedBlob(0, string(), cache);
+    return;
   }
 
-  spg_t pgid;
-  if (!cid.is_pg(&pgid))
-    pgid = spg_t();  // meta
-  string key;
-  get_bnode_key(pgid.shard, pgid.pool(), hash, &key);
-  b = new Bnode(hash, key, &bnode_set);
-  dout(10) << __func__ << " hash " << std::hex << hash << std::dec
-          << " created " << b << dendl;
+  b->shared_blob = shared_blob_set.lookup(blob.sbid);
+  if (b->shared_blob) {
+    dout(10) << __func__ << " sbid 0x" << std::hex << blob.sbid << std::dec
+            << " had " << *b->shared_blob << dendl;
+  } else {
+    b->shared_blob = new SharedBlob(blob.sbid, string(), cache);
+    get_shared_blob_key(blob.sbid, &b->shared_blob->key);
+    shared_blob_set.add(b->shared_blob.get());
+    dout(10) << __func__ << " sbid 0x" << std::hex << blob.sbid << std::dec
+            << " opened " << *b->shared_blob << dendl;
+  }
+}
 
+void BlueStore::Collection::load_shared_blob(SharedBlobRef sb)
+{
+  assert(!sb->loaded);
   bufferlist v;
-  int r = store->db->get(PREFIX_OBJ, key, &v);
-  if (r >= 0) {
-    assert(v.length() > 0);
-    bufferlist::iterator p = v.begin();
-    b->blob_map.decode(p, cache);
-    dout(10) << __func__ << " hash " << std::hex << hash << std::dec
-            << " loaded blob_map " << b->blob_map << dendl;
-  } else {
-    dout(10) << __func__ << " hash " <<std::hex << hash << std::dec
-            << " missed, new blob_map" << dendl;
+  int r = store->db->get(PREFIX_SHARED_BLOB, sb->key, &v);
+  if (r < 0) {
+    derr << __func__ << " sbid 0x" << std::hex << sb->sbid << std::dec
+        << " not found at key " << pretty_binary_string(sb->key) << dendl;
+    assert(0 == "uh oh, missing shared_blob");
+  }
+
+  bufferlist::iterator p = v.begin();
+  ::decode(sb->shared_blob, p);
+  dout(10) << __func__ << " sbid 0x" << std::hex << sb->sbid << std::dec
+          << " loaded shared_blob " << *sb << dendl;
+  sb->loaded = true;
+}
+
+void BlueStore::Collection::make_blob_shared(BlobRef b)
+{
+  dout(10) << __func__ << " " << *b << dendl;
+  bluestore_blob_t& blob = b->dirty_blob();
+
+  // update blob
+  blob.set_flag(bluestore_blob_t::FLAG_SHARED);
+  blob.clear_flag(bluestore_blob_t::FLAG_MUTABLE);
+
+  // update shared blob
+  b->shared_blob->loaded = true;  // we are new and therefore up to date
+  b->shared_blob->sbid = blob.sbid;
+  get_shared_blob_key(blob.sbid, &b->shared_blob->key);
+  shared_blob_set.add(b->shared_blob.get());
+  for (auto p : blob.extents) {
+    if (p.is_valid()) {
+      b->shared_blob->shared_blob.ref_map.get(p.offset, p.length);
+    }
   }
-  bnode_set.add(b.get());
-  return b;
+  dout(20) << __func__ << " now " << *b << dendl;
 }
 
 BlueStore::OnodeRef BlueStore::Collection::get_onode(
@@ -1357,16 +2126,24 @@ BlueStore::OnodeRef BlueStore::Collection::get_onode(
        !create)
       return OnodeRef();
 
-    // new
-    on = new Onode(&onode_map, oid, key);
+    // new object, new onode
+    on = new Onode(&onode_map, this, oid, key);
   } else {
     // loaded
-    assert(r >= 0);
-    on = new Onode(&onode_map, oid, key);
+    assert(r >=);
+    on = new Onode(&onode_map, this, oid, key);
     on->exists = true;
     bufferlist::iterator p = v.begin();
     ::decode(on->onode, p);
-    on->blob_map.decode(p, cache);
+
+    // initialize extent_map
+    on->extent_map.decode_spanning_blobs(this, p);
+    if (on->onode.extent_map_shards.empty()) {
+      ::decode(on->extent_map.inline_bl, p);
+      on->extent_map.decode_some(on->extent_map.inline_bl);
+    } else {
+      on->extent_map.init_shards(on, false, false);
+    }
   }
   o.reset(on);
   onode_map.add(oid, o);
@@ -3001,94 +3778,64 @@ void apply(uint64_t off,
   }
 }
 
-int BlueStore::_fsck_verify_blob_map(
-  string what,
-  const BlobMap& blob_map,
-  map<int64_t,bluestore_extent_ref_map_t>& v,
+int BlueStore::_fsck_check_extents(
+  const ghobject_t& oid,
+  const vector<bluestore_pextent_t>& extents,
+  bool compressed,
   boost::dynamic_bitset<> &used_blocks,
   store_statfs_t& expected_statfs)
 {
+  dout(30) << __func__ << " oid " << oid << " extents " << extents << dendl;
   int errors = 0;
-  dout(20) << __func__ << " " << what << " " << v << dendl;
-  for (auto& b : blob_map.blob_map) {
-    auto pv = v.find(b.id);
-    if (pv == v.end()) {
-      derr << " " << what << " blob " << b.id
-          << " has no lextent refs" << dendl;
-      ++errors;
+  for (auto e : extents) {
+    if (!e.is_valid())
       continue;
-    }
-    if (b.get_blob().has_refmap() && pv->second != b.get_blob().ref_map) {
-      derr << " " << what << " blob " << b.id
-          << " ref_map " << b.get_blob().ref_map
-          << " != expected " << pv->second << dendl;
-      ++errors;
-    }
-    bool compressed = b.get_blob().is_compressed();
+    expected_statfs.allocated += e.length;
     if (compressed) {
-      expected_statfs.compressed += b.get_blob().compressed_length;
-      for (auto& r : b.get_blob().has_refmap() ? b.get_blob().ref_map.ref_map :
-            v[b.id].ref_map) {
-       expected_statfs.compressed_original += r.second.refs * r.second.length;
-      }
+      expected_statfs.compressed_allocated += e.length;
+    }
+    bool already = false;
+    apply(
+      e.offset, e.length, min_alloc_size, used_blocks,
+      [&](uint64_t pos, boost::dynamic_bitset<> &bs) {
+       if (bs.test(pos))
+         already = true;
+       else
+         bs.set(pos);
+      });
+    if (already) {
+      derr << " " << oid << " extent " << e
+          << " or a subset is already allocated" << dendl;
+      ++errors;
     }
-    v.erase(pv);
-    for (auto& p : b.get_blob().extents) {
-      if (!p.is_valid()) {
-        continue;
-      }
-      expected_statfs.allocated += p.length;
-      if (compressed) {
-        expected_statfs.compressed_allocated += p.length;
-      }
-      bool already_allocated = false;
-      apply( 
-        p.offset, p.length, min_alloc_size, used_blocks,
-        [&](uint64_t pos, boost::dynamic_bitset<> &bs) {
-          if (bs.test(pos)) {
-            already_allocated = true;
-          } else {
-            bs.set(pos);
-          }
-        }
-      );
-
-      if (already_allocated) {
-       derr << " " << what << " extent 0x" << std::hex
-            << p.offset << "~" << p.length
-            << " or its' subset is already allocated" << std::dec << dendl;
-       ++errors;
-      } else {
-       if (p.end() > bdev->get_size()) {
-         interval_set<uint64_t> e;
-         e.insert(p.offset, p.length);
-         derr << " " << what << " blob " << b.id << " extent " << e
-              << " past end of block device" << dendl;
-         ++errors;
-       }
-      }
+    if (e.end() > bdev->get_size()) {
+      derr << " " << oid << " extent " << e
+          << " past end of block device" << dendl;
+      ++errors;
     }
   }
-  for (auto& p : v) {
-    derr << " " << what << " blob " << p.first
-        << " dne, has extent refs " << p.second << dendl;
-    ++errors;
-  }
   return errors;
 }
 
 int BlueStore::fsck()
 {
-  dout(1) << __func__ << dendl;
+  dout(1) << __func__ << " start" << dendl;
   int errors = 0;
   set<uint64_t> used_nids;
   set<uint64_t> used_omap_head;
   boost::dynamic_bitset<> used_blocks;
+  set<uint64_t> used_sbids;
   KeyValueDB::Iterator it;
-  BnodeRef bnode;
-  map<int64_t,bluestore_extent_ref_map_t> hash_shared;
   store_statfs_t expected_statfs, actual_statfs;
+  struct sb_info_t {
+    list<ghobject_t> oids;
+    SharedBlobRef sb;
+    bluestore_extent_ref_map_t ref_map;
+    bool compressed;
+  };
+  map<uint64_t,sb_info_t> sb_info;
 
+  utime_t start = ceph_clock_now(NULL);
 
   int r = _open_path();
   if (r < 0)
@@ -3162,157 +3909,51 @@ int BlueStore::fsck()
   expected_statfs.total = actual_statfs.total;
   expected_statfs.available = actual_statfs.available;
 
-  // walk collections, objects
-  for (ceph::unordered_map<coll_t, CollectionRef>::iterator p = coll_map.begin();
-       p != coll_map.end();
-       ++p) {
-    dout(1) << __func__ << " collection " << p->first << dendl;
-    CollectionRef c = _get_collection(p->first);
-    RWLock::RLocker l(c->lock);
-    ghobject_t pos;
-    while (true) {
-      vector<ghobject_t> ols;
-      int r = collection_list(p->first, pos, ghobject_t::get_max(), true,
-                             100, &ols, &pos);
-      if (r < 0) {
-       ++errors;
-       break;
-      }
-      if (ols.empty()) {
-       break;
-      }
-      for (auto& oid : ols) {
-       OnodeRef o = c->get_onode(oid, false);
-       if (!o || !o->exists) {
-         derr << __func__ << "  " << oid << " missing" << dendl;
+  // walk PREFIX_OBJ
+  dout(1) << __func__ << " walking object keyspace" << dendl;
+  it = db->get_iterator(PREFIX_OBJ);
+  if (it) {
+    CollectionRef c;
+    spg_t pgid;
+    list<string> expecting_shards;
+    for (it->lower_bound(string()); it->valid(); it->next()) {
+      dout(30) << " key " << pretty_binary_string(it->key()) << dendl;
+      ghobject_t oid;
+      if (is_extent_shard_key(it->key())) {
+       while (!expecting_shards.empty() &&
+              expecting_shards.front() < it->key()) {
+         derr << __func__ << " missing shard key "
+              << pretty_binary_string(expecting_shards.front())
+              << dendl;
          ++errors;
-         continue; // go for next object
-       }
-       if (!bnode || bnode->hash != o->oid.hobj.get_hash()) {
-         if (bnode)
-           errors += _fsck_verify_blob_map(
-             "hash " + stringify(bnode->hash),
-             bnode->blob_map,
-             hash_shared,
-             used_blocks,
-             expected_statfs);
-         bnode = c->get_bnode(o->oid.hobj.get_hash());
-         hash_shared.clear();
-       }
-       dout(10) << __func__ << "  " << oid << dendl;
-       _dump_onode(o, 30);
-       if (o->onode.nid) {
-         if (used_nids.count(o->onode.nid)) {
-           derr << " " << oid << " nid " << o->onode.nid << " already in use"
-                << dendl;
-           ++errors;
-           continue; // go for next object
-         }
-         used_nids.insert(o->onode.nid);
+         expecting_shards.pop_front();
        }
-       // lextents
-       map<int64_t,bluestore_extent_ref_map_t> local_blobs;
-       uint64_t lext_next_offset = 0, lext_prev_offset = 0;
-       for (auto& l : o->onode.extent_map) {
-         if (l.first < lext_next_offset) {
-           derr << " " << oid << " lextent at 0x" 
-                << std::hex << l.first
-                << "overlaps with the previous one 0x" 
-                << lext_prev_offset << "~"
-                << (lext_next_offset - lext_prev_offset)
-                << std::dec << dendl;
-           ++errors;
-         }
-         lext_next_offset = l.first + l.second.length;
-         lext_prev_offset = l.first;
-         if (l.second.blob >= 0) {
-           local_blobs[l.second.blob].get(l.second.offset, l.second.length);
-         } else {
-           hash_shared[-l.second.blob].get(l.second.offset, l.second.length);
-         }
-         expected_statfs.stored += l.second.length;
+       if (!expecting_shards.empty() &&
+           expecting_shards.front() == it->key()) {
+         // all good
+         expecting_shards.pop_front();
+         continue;
        }
-       // blobs
-       errors += _fsck_verify_blob_map(
-         "object " + stringify(oid),
-         o->blob_map,
-         local_blobs,
-         used_blocks,
-         expected_statfs);
-       // omap
-       while (o->onode.omap_head) {
-         if (used_omap_head.count(o->onode.omap_head)) {
-           derr << " " << oid << " omap_head " << o->onode.omap_head
-                << " already in use" << dendl;
-           ++errors;
-           break;
-         }
-         used_omap_head.insert(o->onode.omap_head);
-         // hrm, scan actual key/value pairs?
-         KeyValueDB::Iterator it = db->get_iterator(PREFIX_OMAP);
-         if (!it)
+       while (expecting_shards.empty() ||
+              expecting_shards.front() > it->key()) {
+         uint32_t offset;
+         string okey;
+         get_key_extent_shard(it->key(), &okey, &offset);
+         string ekey;
+         derr << __func__ << " stray shard 0x" << std::hex << offset << std::dec
+              << dendl;
+         derr << __func__ << "   saw " << pretty_binary_string(it->key())
+              << dendl;
+         derr << __func__ << "   exp "
+              << pretty_binary_string(expecting_shards.front()) << dendl;
+         ++errors;
+         expecting_shards.pop_front();
+         if (expecting_shards.empty()) {
            break;
-         string head, tail;
-         get_omap_header(o->onode.omap_head, &head);
-         get_omap_tail(o->onode.omap_head, &tail);
-         it->lower_bound(head);
-         while (it->valid()) {
-           if (it->key() == head) {
-             dout(30) << __func__ << "  got header" << dendl;
-           } else if (it->key() >= tail) {
-             dout(30) << __func__ << "  reached tail" << dendl;
-             break;
-           } else {
-             string user_key;
-             decode_omap_key(it->key(), &user_key);
-             dout(30) << __func__
-                      << "  got " << pretty_binary_string(it->key())
-                      << " -> " << user_key << dendl;
-           }
-           it->next();
          }
-         break;
        }
-      }
-    }
-  }
-  if (bnode) {
-    errors += _fsck_verify_blob_map(
-      "hash " + stringify(bnode->hash),
-      bnode->blob_map,
-      hash_shared,
-      used_blocks,
-      expected_statfs);
-    hash_shared.clear();
-    bnode.reset();
-  }
-  if (!(actual_statfs == expected_statfs)) {
-    dout(30) << __func__ << "  actual statfs differs from the expected one:"
-             << actual_statfs << " vs. "
-             << expected_statfs << dendl;
-    ++errors;
-  }
-
-  dout(1) << __func__ << " checking for stray bnodes and onodes" << dendl;
-  it = db->get_iterator(PREFIX_OBJ);
-  if (it) {
-    CollectionRef c;
-    shard_id_t expecting_shard;
-    int64_t expecting_pool;
-    uint32_t expecting_hash;
-    for (it->lower_bound(string()); it->valid(); it->next()) {
-      ghobject_t oid;
-      if (is_bnode_key(it->key())) {
-       int r = get_key_bnode(it->key(), &expecting_shard, &expecting_pool,
-                     &expecting_hash);
-        if (r < 0) {
-          dout(30) << __func__ << "  bad bnode key "
-                   << pretty_binary_string(it->key()) << dendl;
-          ++errors;
-        }
        continue;
       }
-
       int r = get_key_object(it->key(), &oid);
       if (r < 0) {
        dout(30) << __func__ << "  bad object key "
@@ -3320,9 +3961,11 @@ int BlueStore::fsck()
        ++errors;
        continue;
       }
-
-      if (!c || !c->contains(oid)) {
-       c = NULL;
+      if (!c ||
+         oid.shard_id != pgid.shard ||
+         oid.hobj.pool != (int64_t)pgid.pool() ||
+         !c->contains(oid)) {
+       c = nullptr;
        for (ceph::unordered_map<coll_t, CollectionRef>::iterator p =
               coll_map.begin();
             p != coll_map.end();
@@ -3338,9 +3981,186 @@ int BlueStore::fsck()
          ++errors;
          continue;
        }
+       c->cid.is_pg(&pgid);
+       dout(20) << __func__ << "  collection " << c->cid << dendl;
+      }
+
+      if (!expecting_shards.empty()) {
+       for (auto &k : expecting_shards) {
+         derr << __func__ << " missing shard key "
+              << pretty_binary_string(k) << dendl;
+       }
+       ++errors;
+       expecting_shards.clear();
+      }
+
+      dout(10) << __func__ << "  " << oid << dendl;
+      RWLock::RLocker l(c->lock);
+      OnodeRef o = c->get_onode(oid, false);
+      _dump_onode(o, 30);
+      if (o->onode.nid) {
+       if (o->onode.nid > nid_max) {
+         derr << __func__ << " " << oid << " nid " << o->onode.nid
+              << " > nid_max " << nid_max << dendl;
+         ++errors;
+       }
+       if (used_nids.count(o->onode.nid)) {
+         derr << __func__ << " " << oid << " nid " << o->onode.nid
+              << " already in use" << dendl;
+         ++errors;
+         continue; // go for next object
+       }
+       used_nids.insert(o->onode.nid);
+      }
+      o->extent_map.fault_range(db, 0, OBJECT_MAX_SIZE);
+      // shards
+      for (auto& s : o->extent_map.shards) {
+       dout(20) << __func__ << "    shard " << *s.shard_info << dendl;
+       expecting_shards.push_back(s.key);
+      }
+      // lextents
+      uint64_t pos = 0;
+      map<BlobRef,bluestore_extent_ref_map_t> ref_map;
+      for (auto& l : o->extent_map.extent_map) {
+       dout(20) << __func__ << "    " << l << dendl;
+       if (l.logical_offset < pos) {
+         derr << __func__ << " " << oid << " lextent at 0x"
+              << std::hex << l.logical_offset
+              << "overlaps with the previous, which ends at 0x" << pos
+              << std::dec << dendl;
+         ++errors;
+       }
+       pos = l.logical_offset + l.length;
+       expected_statfs.stored += l.length;
+       assert(l.blob);
+       ref_map[l.blob].get(l.blob_offset, l.length);
+      }
+      for (auto &i : ref_map) {
+       if (i.first->ref_map != i.second) {
+         derr << __func__ << " " << oid << " blob " << *i.first
+              << " doesn't match expected ref_map " << i.second << dendl;
+         ++errors;
+       }
+       const bluestore_blob_t& blob = i.first->get_blob();
+       if (blob.is_compressed()) {
+         expected_statfs.compressed += blob.compressed_length;
+         for (auto& r : i.first->ref_map.ref_map) {
+           expected_statfs.compressed_original +=
+             r.second.refs * r.second.length;
+         }
+       }
+       if (blob.is_shared()) {
+         if (blob.sbid > blobid_max) {
+           derr << __func__ << " " << oid << " blob " << blob
+                << " sbid " << blob.sbid << " > blobid_max "
+                << blobid_max << dendl;
+           ++errors;
+         }
+         sb_info_t& sbi = sb_info[blob.sbid];
+         sbi.sb = i.first->shared_blob;
+         sbi.oids.push_back(oid);
+         sbi.compressed = blob.is_compressed();
+         for (auto e : blob.extents) {
+           if (e.is_valid()) {
+             sbi.ref_map.get(e.offset, e.length);
+           }
+         }
+       } else {
+         errors += _fsck_check_extents(oid, blob.extents,
+                                       blob.is_compressed(),
+                                       used_blocks,
+                                       expected_statfs);
+        }
+      }
+      // omap
+      while (o->onode.omap_head) {
+       if (used_omap_head.count(o->onode.omap_head)) {
+         derr << __func__ << " " << oid << " omap_head " << o->onode.omap_head
+              << " already in use" << dendl;
+         ++errors;
+         break;
+       }
+       used_omap_head.insert(o->onode.omap_head);
+       // hrm, scan actual key/value pairs?
+       KeyValueDB::Iterator it = db->get_iterator(PREFIX_OMAP);
+       if (!it)
+         break;
+       string head, tail;
+       get_omap_header(o->onode.omap_head, &head);
+       get_omap_tail(o->onode.omap_head, &tail);
+       it->lower_bound(head);
+       while (it->valid()) {
+         if (it->key() == head) {
+           dout(30) << __func__ << "  got header" << dendl;
+         } else if (it->key() >= tail) {
+           dout(30) << __func__ << "  reached tail" << dendl;
+           break;
+         } else {
+           string user_key;
+           decode_omap_key(it->key(), &user_key);
+           dout(30) << __func__
+                    << "  got " << pretty_binary_string(it->key())
+                    << " -> " << user_key << dendl;
+           assert(it->key() < tail);
+         }
+         it->next();
+       }
+       break;
+      }
+    }
+  }
+  dout(1) << __func__ << " checking shared_blobs" << dendl;
+  it = db->get_iterator(PREFIX_SHARED_BLOB);
+  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)) {
+       derr << __func__ << " bad key '" << key << "' in shared blob namespace"
+            << dendl;
+       ++errors;
+       continue;
+      }
+      auto p = sb_info.find(sbid);
+      if (p == sb_info.end()) {
+       derr << __func__ << " found stray shared blob data for sbid 0x"
+            << std::hex << sbid << std::dec << dendl;
+       ++errors;
+      } else {
+       sb_info_t& sbi = p->second;
+       bluestore_shared_blob_t shared_blob;
+       bufferlist bl = it->value();
+       bufferlist::iterator blp = bl.begin();
+       shared_blob.decode(blp);
+       dout(20) << __func__ << "  " << *sbi.sb << " " << shared_blob << dendl;
+       if (shared_blob.ref_map != sbi.ref_map) {
+         derr << __func__ << " shared blob 0x" << std::hex << sbid << std::dec
+              << " ref_map " << shared_blob.ref_map << " != expected "
+              << sbi.ref_map << dendl;
+         ++errors;
+       }
+       vector<bluestore_pextent_t> extents;
+       for (auto &r : shared_blob.ref_map.ref_map) {
+         extents.emplace_back(bluestore_pextent_t(r.first, r.second.length));
+       }
+       _fsck_check_extents(p->second.oids.front(),
+                           extents,
+                           p->second.compressed,
+                           used_blocks, expected_statfs);
+       sb_info.erase(p);
       }
     }
   }
+  for (auto &p : sb_info) {
+    derr << __func__ << " shared_blob 0x" << p.first << " key is missing ("
+        << *p.second.sb << ")" << dendl;
+    ++errors;
+  }
+  if (!(actual_statfs == expected_statfs)) {
+    derr << __func__ << " actual " << actual_statfs
+        << " != expected " << expected_statfs << dendl;
+    ++errors;
+  }
 
   dout(1) << __func__ << " checking for stray omap data" << dendl;
   it = db->get_iterator(PREFIX_OMAP);
@@ -3450,7 +4270,9 @@ int BlueStore::fsck()
   if (r < 0)
     return r;
 
-  dout(1) << __func__ << " finish with " << errors << " errors" << dendl;
+  utime_t duration = ceph_clock_now(NULL) - start;
+  dout(1) << __func__ << " finish with " << errors << " errors in "
+         << duration << " seconds" << dendl;
   return errors;
 }
 
@@ -3729,10 +4551,6 @@ struct region_t {
   }
 };
 
-bool less(const BlueStore::BlobRef& a, const BlueStore::BlobRef& b)
-{
-  return *a < *b;
-}
 typedef list<region_t> regions2read_t;
 typedef map<BlueStore::BlobRef, regions2read_t> blobs2read_t;
 
@@ -3744,7 +4562,7 @@ int BlueStore::_do_read(
   bufferlist& bl,
   uint32_t op_flags)
 {
-  map<uint64_t,bluestore_lextent_t>::iterator ep, eend;
+  boost::intrusive::set<Extent>::iterator ep, eend;
   int r = 0;
 
   dout(20) << __func__ << " 0x" << std::hex << offset << "~" << length
@@ -3774,6 +4592,8 @@ int BlueStore::_do_read(
   }
 
   o->flush();
+
+  o->extent_map.fault_range(db, offset, length);
   _dump_onode(o);
 
   ready_regions_t ready_regions;
@@ -3782,10 +4602,10 @@ int BlueStore::_do_read(
   blobs2read_t blobs2read;
   unsigned left = length;
   uint64_t pos = offset;
-  auto lp = o->onode.seek_lextent(offset);
-  while (left > 0 && lp != o->onode.extent_map.end()) {
-    if (pos < lp->first) {
-      unsigned hole = lp->first - pos;
+  auto lp = o->extent_map.seek_lextent(offset);
+  while (left > 0 && lp != o->extent_map.extent_map.end()) {
+    if (pos < lp->logical_offset) {
+      unsigned hole = lp->logical_offset - pos;
       if (hole >= left) {
        break;
       }
@@ -3794,18 +4614,14 @@ int BlueStore::_do_read(
       pos += hole;
       left -= hole;
     }
-    BlobRef bptr = c->get_blob(o, lp->second.blob);
-    if (bptr == nullptr) {
-      dout(20) << __func__ << "  missed blob " << lp->second.blob << dendl;
-      assert(bptr != nullptr);
-    }
-    unsigned l_off = pos - lp->first;
-    unsigned b_off = l_off + lp->second.offset;
-    unsigned b_len = std::min(left, lp->second.length - l_off);
+    BlobRef bptr = lp->blob;
+    unsigned l_off = pos - lp->logical_offset;
+    unsigned b_off = l_off + lp->blob_offset;
+    unsigned b_len = std::min(left, lp->length - l_off);
 
     ready_regions_t cache_res;
     interval_set<uint64_t> cache_interval;
-    bptr->bc.read(b_off, b_len, cache_res, cache_interval);
+    bptr->shared_blob->bc.read(b_off, b_len, cache_res, cache_interval);
     dout(20) << __func__ << "  blob " << *bptr << std::hex
             << " need 0x" << b_off << "~" << b_len
             << " cache has 0x" << cache_interval
@@ -3839,7 +4655,7 @@ int BlueStore::_do_read(
     ++lp;
   }
 
-  //enumerate and read/decompress desired blobs
+  // enumerate and read/decompress desired blobs
   blobs2read_t::iterator b2r_it = blobs2read.begin();
   while (b2r_it != blobs2read.end()) {
     BlobRef bptr = b2r_it->first;
@@ -3868,7 +4684,7 @@ int BlueStore::_do_read(
       if (r < 0)
        return r;
       if (buffered) {
-       bptr->bc.did_read(0, raw_bl);
+       bptr->shared_blob->bc.did_read(0, raw_bl);
       }
       for (auto& i : b2r_it->second) {
        ready_regions[i.logical_offset].substr_of(
@@ -3916,7 +4732,7 @@ int BlueStore::_do_read(
          return -EIO;
        }
        if (buffered) {
-         bptr->bc.did_read(r_off, bl);
+         bptr->shared_blob->bc.did_read(r_off, bl);
        }
 
        // prune and keep result
@@ -4008,7 +4824,7 @@ int BlueStore::_decompress(bufferlist& source, bufferlist* result)
   if (!cp.get()) {
     // if compressor isn't available - error, because cannot return
     // decompressed data?
-    derr << __func__ << " can't load decompressor " << chdr.type << dendl;
+    derr << __func__ << " can't load decompressor " << (int)chdr.type << dendl;
     r = -EIO;
   } else {
     r = cp->decompress(i, chdr.length, *result);
@@ -4057,7 +4873,7 @@ int BlueStore::fiemap(
     dout(20) << __func__ << " 0x" << std::hex << offset << "~" << length
             << " size 0x" << o->onode.size << std::dec << dendl;
 
-    map<uint64_t,bluestore_lextent_t>::iterator ep, eend;
+    boost::intrusive::set<Extent>::iterator ep, eend;
     if (offset > o->onode.size)
       goto out;
 
@@ -4065,35 +4881,33 @@ int BlueStore::fiemap(
       length = o->onode.size - offset;
     }
 
-    eend = o->onode.extent_map.end();
-    ep = o->onode.extent_map.lower_bound(offset);
-    if (ep != o->onode.extent_map.begin()) {
-      --ep;
-    }
+    o->extent_map.fault_range(db, offset, length);
+    eend = o->extent_map.extent_map.end();
+    ep = o->extent_map.seek_lextent(offset);
     while (length > 0) {
       dout(20) << __func__ << " offset " << offset << dendl;
-      if (ep != eend && ep->first + ep->second.length <= offset) {
+      if (ep != eend && ep->logical_offset + ep->length <= offset) {
         ++ep;
         continue;
       }
 
       uint64_t x_len = length;
-      if (ep != eend && ep->first <= offset) {
-        uint64_t x_off = offset - ep->first;
-        x_len = MIN(x_len, ep->second.length - x_off);
+      if (ep != eend && ep->logical_offset <= offset) {
+        uint64_t x_off = offset - ep->logical_offset;
+        x_len = MIN(x_len, ep->length - x_off);
         dout(30) << __func__ << " lextent 0x" << std::hex << offset << "~"
-                << x_len << std::dec << " blob " << ep->second.blob << dendl;
+                << x_len << std::dec << " blob " << ep->blob << dendl;
         m.insert(offset, x_len);
         length -= x_len;
         offset += x_len;
-        if (x_off + x_len == ep->second.length)
+        if (x_off + x_len == ep->length)
          ++ep;
         continue;
       }
       if (ep != eend &&
-         ep->first > offset &&
-         ep->first - offset < x_len) {
-        x_len = ep->first - offset;
+         ep->logical_offset > offset &&
+         ep->logical_offset - offset < x_len) {
+        x_len = ep->logical_offset - offset;
       }
       offset += x_len;
       length -= x_len;
@@ -4349,14 +5163,11 @@ int BlueStore::collection_list(
         }
         break;
       }
-      if (is_bnode_key(it->key())) {
-        dout(20) << __func__ << " key "
-                << pretty_binary_string(it->key())
-                << " (bnode, skipping)" << dendl;
-        it->next();
-        continue;
-      }
       dout(20) << __func__ << " key " << pretty_binary_string(it->key()) << dendl;
+      if (is_extent_shard_key(it->key())) {
+       it->next();
+       continue;
+      }
       ghobject_t oid;
       int r = get_key_object(it->key(), &oid);
       assert(r == 0);
@@ -4803,6 +5614,20 @@ int BlueStore::_open_super_meta()
     nid_last = nid_max;
   }
 
+  // blobid
+  {
+    blobid_max = 0;
+    bufferlist bl;
+    db->get(PREFIX_SUPER, "blobid_max", &bl);
+    bufferlist::iterator p = bl.begin();
+    try {
+      ::decode(blobid_max, p);
+    } catch (buffer::error& e) {
+    }
+    dout(10) << __func__ << " old blobid_max " << blobid_max << dendl;
+    blobid_last = blobid_max;
+  }
+
   // freelist
   {
     bufferlist bl;
@@ -4836,6 +5661,7 @@ int BlueStore::_open_super_meta()
 
 void BlueStore::_assign_nid(TransContext *txc, OnodeRef o)
 {
+#warning racy because txn commit may be delayed by io
   if (o->onode.nid)
     return;
   std::lock_guard<std::mutex> l(nid_lock);
@@ -4850,6 +5676,23 @@ void BlueStore::_assign_nid(TransContext *txc, OnodeRef o)
   }
 }
 
+uint64_t BlueStore::_assign_blobid(TransContext *txc)
+{
+#warning racy because txn commit may be delayed by io
+  std::lock_guard<std::mutex> l(blobid_lock);
+  uint64_t bid = ++blobid_last;
+  dout(20) << __func__ << " " << bid << dendl;
+  if (blobid_last > blobid_max) {
+    blobid_max += g_conf->bluestore_blobid_prealloc;
+    bufferlist bl;
+    ::encode(blobid_max, bl);
+    txc->t->set(PREFIX_SUPER, "blobid_max", bl);
+    dout(10) << __func__ << " blobid_max now " << blobid_max << dendl;
+  }
+  return bid;
+}
+
+
 BlueStore::TransContext *BlueStore::_txc_create(OpSequencer *osr)
 {
   TransContext *txc = new TransContext(osr);
@@ -4902,10 +5745,10 @@ void BlueStore::_txc_state_proc(TransContext *txc)
       //assert(txc->osr->qlock.is_locked());  // see _txc_finish_io
       txc->log_state_latency(logger, l_bluestore_state_io_done_lat);
       txc->state = TransContext::STATE_KV_QUEUED;
-      for (auto& b : txc->blobs) {
-        b->bc.finish_write(txc->seq);
+      for (auto& sb : txc->shared_blobs_written) {
+        sb->bc.finish_write(txc->seq);
       }
-      txc->blobs.clear();
+      txc->shared_blobs_written.clear();
       if (!g_conf->bluestore_sync_transaction) {
        if (g_conf->bluestore_sync_submit_transaction) {
          _txc_finalize_kv(txc, txc->t);
@@ -5012,40 +5855,61 @@ void BlueStore::_txc_write_nodes(TransContext *txc, KeyValueDB::Transaction t)
 {
   dout(20) << __func__ << " txc " << txc
           << " onodes " << txc->onodes
-          << " bnodes " << txc->bnodes
+          << " shared_blobs " << txc->shared_blobs
           << dendl;
 
   // finalize onodes
-  for (set<OnodeRef>::iterator p = txc->onodes.begin();
-       p != txc->onodes.end();
-       ++p) {
+  for (auto o : txc->onodes) {
+    // finalize extent_map shards
+    bool reshard = o->extent_map.update(o.get(), t, false);
+    if (reshard) {
+      dout(20) << __func__ << "  resharding extents for " << o->oid << dendl;
+      for (auto &s : o->extent_map.shards) {
+       t->rmkey(PREFIX_OBJ, s.key);
+      }
+      o->extent_map.fault_range(db, 0, o->onode.size);
+      o->extent_map.reshard(o.get());
+      reshard = o->extent_map.update(o.get(), t, true);
+      if (reshard) {
+       derr << __func__ << " warning: still wants reshard, check options?"
+            << dendl;
+       assert(0 == "reshard problem");
+      }
+    }
+
     bufferlist bl;
-    ::encode((*p)->onode, bl);
-    unsigned first_part = bl.length();
-    (*p)->blob_map.encode(bl);
-    dout(20) << "  onode " << (*p)->oid << " is " << bl.length()
-            << " (" << first_part << " onode + "
-            << (bl.length() - first_part) << " blob_map)" << dendl;
-    t->set(PREFIX_OBJ, (*p)->key, bl);
+    ::encode(o->onode, bl);
+    unsigned onode_part = bl.length();
+    o->extent_map.encode_spanning_blobs(bl);
+    unsigned blob_part = bl.length() - onode_part;
+    if (o->onode.extent_map_shards.empty()) {
+      ::encode(o->extent_map.inline_bl, bl);
+    }
+    unsigned extent_part = bl.length() - onode_part - blob_part;
+
+    dout(20) << "  onode " << o->oid << " is " << bl.length()
+            << " (" << onode_part << " bytes onode + "
+            << blob_part << " bytes spanning blobs + "
+            << extent_part << " bytes inline extents)"
+            << dendl;
+    t->set(PREFIX_OBJ, o->key, bl);
 
-    std::lock_guard<std::mutex> l((*p)->flush_lock);
-    (*p)->flush_txns.insert(txc);
+    std::lock_guard<std::mutex> l(o->flush_lock);
+    o->flush_txns.insert(txc);
   }
 
-  // finalize bnodes
-  for (set<BnodeRef>::iterator p = txc->bnodes.begin();
-       p != txc->bnodes.end();
-       ++p) {
-    if ((*p)->blob_map.empty()) {
-      dout(20) << "  bnode " << std::hex << (*p)->hash << std::dec
-              << " blob_map is empty" << dendl;
-      t->rmkey(PREFIX_OBJ, (*p)->key);
+  // finalize shared_blobs
+  for (auto sb : txc->shared_blobs) {
+    if (sb->shared_blob.empty()) {
+      dout(20) << "  shared_blob 0x" << std::hex << sb->sbid << std::dec
+              << " is empty" << dendl;
+      t->rmkey(PREFIX_SHARED_BLOB, sb->key);
     } else {
       bufferlist bl;
-      (*p)->blob_map.encode(bl);
-      dout(20) << "  bnode " << std::hex << (*p)->hash << std::dec
-              << " blob_map is " << bl.length() << dendl;
-      t->set(PREFIX_OBJ, (*p)->key, bl);
+      sb->shared_blob.encode(bl);
+      dout(20) << "  shared_blob 0x" << std::hex << sb->sbid << std::dec
+              << " is " << bl.length() << dendl;
+      t->set(PREFIX_SHARED_BLOB, sb->key, bl);
     }
   }
 }
@@ -5472,11 +6336,10 @@ int BlueStore::queue_transactions(
 
   // delayed csum calculation?
   for (auto& d : txc->deferred_csum) {
-    BlobRef b = d.onode->get_blob(d.blob);
     dout(20) << __func__ << "  deferred csum calc blob " << d.blob
             << " b_off 0x" << std::hex << d.b_off << std::dec
-            << " on " << d.onode->oid << dendl;
-    b->dirty_blob().calc_csum(d.b_off, d.data);
+            << dendl;
+    d.blob->dirty_blob().calc_csum(d.b_off, d.data);
   }
 
   _txc_write_nodes(txc, txc->t);
@@ -5900,6 +6763,7 @@ void BlueStore::_dump_onode(OnodeRef o, int log_level)
                  << " (" << std::dec << o->onode.size << ")"
                  << " expected_object_size " << o->onode.expected_object_size
                  << " expected_write_size " << o->onode.expected_write_size
+                 << " in " << o->onode.extent_map_shards.size() << " shards"
                  << dendl;
   for (map<string,bufferptr>::iterator p = o->onode.attrs.begin();
        p != o->onode.attrs.end();
@@ -5907,48 +6771,34 @@ void BlueStore::_dump_onode(OnodeRef o, int log_level)
     dout(log_level) << __func__ << "  attr " << p->first
                    << " len " << p->second.length() << dendl;
   }
-  uint64_t pos = 0;
-  for (auto& p : o->onode.extent_map) {
-    dout(log_level) << __func__ << "  lextent 0x" << std::hex << p.first
-                   << std::dec << ": " << p.second
-                   << dendl;
-    assert(p.first >= pos);
-    pos = p.first + p.second.length;
-  }
-  _dump_blob_map(o->blob_map, log_level);
-  if (o->bnode) {
-    _dump_bnode(o->bnode, log_level);
-  }
-}
-
-void BlueStore::_dump_bnode(BnodeRef b, int log_level)
-{
-  if (!g_conf->subsys.should_gather(ceph_subsys_bluestore, log_level))
-    return;
-  dout(log_level) << __func__ << " " << b
-                 << " " << std::hex << b->hash << std::dec << dendl;
-  _dump_blob_map(b->blob_map, log_level);
+  _dump_extent_map(o->extent_map, log_level);
 }
 
-void BlueStore::_dump_blob_map(BlobMap &bm, int log_level)
+void BlueStore::_dump_extent_map(ExtentMap &em, int log_level)
 {
-  for (auto& b : bm.blob_map) {
-    dout(log_level) << __func__ << "  " << b
-                   << " blob_bl " << b.get_encoded_length()
-                   << (b.is_dirty() ? " (dirty)" : "")
-                   << (b.is_undecoded() ? " (was undecoded)" : "")
+  uint64_t pos = 0;
+  for (auto& s : em.shards) {
+    dout(log_level) << __func__ << "  shard " << *s.shard_info
+                   << (s.loaded ? " (loaded)" : "")
+                   << (s.dirty ? " (dirty)" : "")
                    << dendl;
-    if (b.get_blob().has_csum()) {
+  }
+  for (auto& e : em.extent_map) {
+    dout(log_level) << __func__ << "  " << e << dendl;
+    assert(e.logical_offset >= pos);
+    pos = e.logical_offset + e.length;
+    const bluestore_blob_t& blob = e.blob->get_blob();
+    if (blob.has_csum()) {
       vector<uint64_t> v;
-      unsigned n = b.get_blob().get_csum_count();
+      unsigned n = blob.get_csum_count();
       for (unsigned i = 0; i < n; ++i)
-       v.push_back(b.get_blob().get_csum_item(i));
-      dout(log_level) << __func__ << "       csum: " << std::hex << v << std::dec
+       v.push_back(blob.get_csum_item(i));
+      dout(log_level) << __func__ << "      csum: " << std::hex << v << std::dec
                      << dendl;
     }
-    std::lock_guard<std::mutex> l(b.bc.cache->lock);
-    if (!b.bc.empty()) {
-      for (auto& i : b.bc.buffer_map) {
+    std::lock_guard<std::mutex> l(e.blob->shared_blob->bc.cache->lock);
+    if (!e.blob->shared_blob->bc.empty()) {
+      for (auto& i : e.blob->shared_blob->bc.buffer_map) {
        dout(log_level) << __func__ << "       0x" << std::hex << i.first
                        << "~" << i.second->length << std::dec
                        << " " << *i.second << dendl;
@@ -5957,6 +6807,7 @@ void BlueStore::_dump_blob_map(BlobMap &bm, int log_level)
   }
 }
 
+
 void BlueStore::_pad_zeros(
   bufferlist *bl, uint64_t *offset,
   uint64_t chunk_size)
@@ -6038,34 +6889,33 @@ void BlueStore::_do_write_small(
 
   // look for an existing mutable blob we can use
   BlobRef b = 0;
-  map<uint64_t,bluestore_lextent_t>::iterator ep = o->onode.seek_lextent(offset);
-  if (ep != o->onode.extent_map.begin()) {
+  boost::intrusive::set<Extent>::iterator ep =
+    o->extent_map.seek_lextent(offset);
+  if (ep != o->extent_map.extent_map.begin()) {
     --ep;
-    b = c->get_blob(o, ep->second.blob);
-    if (ep->first + b->get_blob().get_ondisk_length() <= offset) {
+    b = ep->blob;
+    if (ep->logical_offset + b->get_blob().get_ondisk_length() <= offset) {
       ++ep;
     }
   }
-  while (ep != o->onode.extent_map.end()) {
-    if (ep->first >= ep->second.offset + offset + length) {
+  while (ep != o->extent_map.extent_map.end()) {
+    if (ep->logical_offset >= ep->blob_offset + offset + length) {
       break;
     }
-    int64_t blob = ep->second.blob;
-    b = c->get_blob(o, blob);
+    b = ep->blob;
     if (!b->get_blob().is_mutable() || b->get_blob().is_compressed()) {
-      dout(20) << __func__ << " ignoring immutable " << blob << ": " << *b
-              << dendl;
+      dout(20) << __func__ << " ignoring immutable " << *b << dendl;
       ++ep;
       continue;
     }
-    if (ep->first % min_alloc_size != ep->second.offset % min_alloc_size) {
-      dout(20) << __func__ << " ignoring offset-skewed " << blob << ": " << *b
-              << dendl;
+    if (ep->logical_offset % min_alloc_size !=
+       ep->blob_offset % min_alloc_size) {
+      dout(20) << __func__ << " ignoring offset-skewed " << *b << dendl;
       ++ep;
       continue;
     }
-    uint64_t bstart = ep->first - ep->second.offset;
-    dout(20) << __func__ << " considering " << blob << ": " << *b
+    uint64_t bstart = ep->logical_offset - ep->blob_offset;
+    dout(20) << __func__ << " considering " << *b
             << " bstart 0x" << std::hex << bstart << std::dec << dendl;
 
     // can we pad our head/tail out with zeros?
@@ -6073,12 +6923,13 @@ void BlueStore::_do_write_small(
     // settings for csum enable/disable
     uint64_t chunk_size = b->get_blob().get_chunk_size(true, block_size);
     uint64_t head_pad = P2PHASE(offset, chunk_size);
-    if (head_pad && o->onode.has_any_lextents(offset - head_pad, chunk_size)) {
+    if (head_pad &&
+       o->extent_map.has_any_lextents(offset - head_pad, chunk_size)) {
       head_pad = 0;
     }
 
     uint64_t tail_pad = P2NPHASE(end, chunk_size);
-    if (tail_pad && o->onode.has_any_lextents(end, tail_pad)) {
+    if (tail_pad && o->extent_map.has_any_lextents(end, tail_pad)) {
       tail_pad = 0;
     }
 
@@ -6109,8 +6960,8 @@ void BlueStore::_do_write_small(
       dout(20) << __func__ << "  write to unused 0x" << std::hex
               << b_off << "~" << b_len
               << " pad 0x" << head_pad << " + 0x" << tail_pad
-              << std::dec << " of mutable " << blob << ": " << b << dendl;
-      assert(b->get_blob().is_unreferenced(b_off, b_len));
+              << std::dec << " of mutable " << *b << dendl;
+      assert(b->is_unreferenced(b_off, b_len));
       _buffer_cache_write(txc, b, b_off, padded,
                          wctx->buffered ? 0 : Buffer::FLAG_NOCACHE);
 
@@ -6121,15 +6972,12 @@ void BlueStore::_do_write_small(
                          &txc->ioc, wctx->buffered);
        });
       b->dirty_blob().calc_csum(b_off, padded);
-      dout(20) << __func__ << "  lexold 0x" << std::hex << offset << std::dec
-              << ": " << ep->second << dendl;
-      bluestore_lextent_t lex(blob, b_off + head_pad, length);
-      o->onode.set_lextent(offset, lex, &b->dirty_blob(), &wctx->lex_old);
-      b->dirty_blob().mark_used(lex.offset, lex.length, min_alloc_size);
-      txc->statfs_delta.stored() += lex.length;
-      dout(20) << __func__ << "  lex 0x" << std::hex << offset << std::dec
-              << ": " << lex << dendl;
-      dout(20) << __func__ << "  old " << blob << ": " << *b << dendl;
+      dout(20) << __func__ << "  lex old " << *ep << dendl;
+      Extent *le = o->extent_map.set_lextent(offset, b_off + head_pad, length, b,
+                                            &wctx->old_extents);
+      b->dirty_blob().mark_used(le->blob_offset, le->length, min_alloc_size);
+      txc->statfs_delta.stored() += le->length;
+      dout(20) << __func__ << "  lex " << *le << dendl;
       return;
     }
 
@@ -6191,19 +7039,17 @@ void BlueStore::_do_write_small(
        });
       assert(r == 0);
       if (b->get_blob().csum_type) {
-       txc->add_deferred_csum(o, blob, b_off, padded);
+       txc->add_deferred_csum(b, b_off, padded);
       }
       op->data.claim(padded);
       dout(20) << __func__ << "  wal write 0x" << std::hex << b_off << "~"
-              << b_len << std::dec << " of mutable " << blob << ": " << *b
+              << b_len << std::dec << " of mutable " << *b
               << " at " << op->extents << dendl;
-      bluestore_lextent_t lex(blob, offset - bstart, length);
-      o->onode.set_lextent(offset, lex, &b->dirty_blob(), &wctx->lex_old);
-      b->dirty_blob().mark_used(lex.offset, lex.length, min_alloc_size);
-      txc->statfs_delta.stored() += lex.length;
-      dout(20) << __func__ << "  lex 0x" << std::hex << offset
-              << std::dec << ": " << lex << dendl;
-      dout(20) << __func__ << "  old " << blob << ": " << *b << dendl;
+      Extent *le = o->extent_map.set_lextent(offset, offset - bstart, length, b,
+                                            &wctx->old_extents);
+      b->dirty_blob().mark_used(le->blob_offset, le->length, min_alloc_size);
+      txc->statfs_delta.stored() += le->length;
+      dout(20) << __func__ << "  lex " << *le << dendl;
       return;
     }
 
@@ -6211,17 +7057,16 @@ void BlueStore::_do_write_small(
   }
 
   // new blob.
-  b = o->blob_map.new_blob(c->cache);
+  b = c->new_blob();
   unsigned alloc_len = min_alloc_size;
   uint64_t b_off = P2PHASE(offset, alloc_len);
-  _buffer_cache_write(txc, b, b_off, bl, wctx->buffered ? 0 : Buffer::FLAG_NOCACHE);
+  _buffer_cache_write(txc, b, b_off, bl,
+                     wctx->buffered ? 0 : Buffer::FLAG_NOCACHE);
   _pad_zeros(&bl, &b_off, block_size);
-  bluestore_lextent_t lex(b->id, P2PHASE(offset, alloc_len), length);
-  o->onode.set_lextent(offset, lex, &b->dirty_blob(), &wctx->lex_old);
-  txc->statfs_delta.stored() += lex.length;
-  dout(20) << __func__ << "  lex 0x" << std::hex << offset << std::dec
-          << ": " << lex << dendl;
-  dout(20) << __func__ << "  new " << b->id << ": " << *b << dendl;
+  Extent *le = o->extent_map.set_lextent(offset, P2PHASE(offset, alloc_len),
+                                        length, b, &wctx->old_extents);
+  txc->statfs_delta.stored() += le->length;
+  dout(20) << __func__ << "  lex " << *le << dendl;
   wctx->write(b, alloc_len, b_off, bl, true);
   return;
 }
@@ -6243,18 +7088,15 @@ void BlueStore::_do_write_big(
           << " compress " << (int)wctx->compress
           << std::dec << dendl;
   while (length > 0) {
-    BlobRef b = o->blob_map.new_blob(c->cache);
+    BlobRef b = c->new_blob();
     auto l = MIN(max_blob_len, length);
     bufferlist t;
     blp.copy(l, t);
     _buffer_cache_write(txc, b, 0, t, wctx->buffered ? 0 : Buffer::FLAG_NOCACHE);
     wctx->write(b, l, 0, t, false);
-    bluestore_lextent_t lex(b->id, 0, l);
-    o->onode.set_lextent(offset, lex, &b->dirty_blob(), &wctx->lex_old);
+    Extent *le = o->extent_map.set_lextent(offset, 0, l, b, &wctx->old_extents);
     txc->statfs_delta.stored() += l;
-    dout(20) << __func__ << "  lex 0x" << std::hex << offset << std::dec << ": "
-            << o->onode.extent_map[offset] << dendl;
-    dout(20) << __func__ << "  blob " << *b << dendl;
+    dout(20) << __func__ << "  lex " << *le << dendl;
     offset += l;
     length -= l;
   }
@@ -6321,7 +7163,7 @@ int BlueStore::_do_alloc_write(
        logger->inc(l_bluestore_write_pad_bytes, newlen - rawlen);
        dout(20) << __func__ << hex << "  compressed 0x" << wi.blob_length
                 << " -> 0x" << rawlen << " => 0x" << newlen
-                << " with " << chdr.type
+                << " with " << (int)chdr.type
                 << dec << dendl;
        txc->statfs_delta.compressed() += rawlen;
        txc->statfs_delta.compressed_original() += l->length();
@@ -6335,7 +7177,7 @@ int BlueStore::_do_alloc_write(
         logger->inc(l_bluestore_compress_success_count);
       } else {
        dout(20) << __func__ << hex << "  compressed 0x" << l->length()
-                 << " -> 0x" << rawlen << " with " << chdr.type
+                 << " -> 0x" << rawlen << " with " << (int)chdr.type
                  << ", which is more than required 0x" << dstlen
                  << ", leaving uncompressed"
                  << dec << dendl;
@@ -6364,11 +7206,12 @@ int BlueStore::_do_alloc_write(
 
     need -= final_length;
     assert(r == 0);
+    bluestore_blob_t& dblob = b->dirty_blob();
     for (int i = 0; i < count; i++) {
       bluestore_pextent_t e = bluestore_pextent_t(extents[i]);
       txc->allocated.insert(e.offset, e.length);
       txc->statfs_delta.allocated() += e.length;
-      b->dirty_blob().extents.push_back(e);
+      dblob.extents.push_back(e);
       hint = e.end();
     }
 
@@ -6412,16 +7255,39 @@ void BlueStore::_wctx_finish(
   OnodeRef o,
   WriteContext *wctx)
 {
-  dout(10) << __func__ << " lex_old " << wctx->lex_old << dendl;
   set<pair<bool, BlobRef> > blobs2remove;
-  for (auto &lo : wctx->lex_old) {
-    bluestore_lextent_t& l = lo.second;
-    BlobRef b = c->get_blob(o, l.blob);
+  for (auto &lo : wctx->old_extents) {
+    dout(20) << __func__ << " lex_old " << lo << dendl;
+    BlobRef b = lo.blob;
+    const bluestore_blob_t& blob = b->get_blob();
     vector<bluestore_pextent_t> r;
-    bool compressed = b->get_blob().is_compressed();
-    if (o->onode.deref_lextent(lo.first, l, &b->dirty_blob(),
-                              min_alloc_size, &r)) {
-      blobs2remove.insert(std::make_pair(l.blob >= 0, b));
+    if (b->put_ref(lo.blob_offset, lo.length, min_alloc_size, &r)) {
+      if (blob.is_compressed()) {
+       txc->statfs_delta.compressed() -= blob.get_compressed_payload_length();
+      }
+    }
+    txc->statfs_delta.stored() -= lo.length;
+    if (blob.is_compressed()) {
+      txc->statfs_delta.compressed_original() -= lo.length;
+    }
+    if (!r.empty()) {
+      dout(20) << __func__ << "  blob release " << r << dendl;
+      if (blob.is_shared()) {
+       vector<bluestore_pextent_t> final;
+       if (!b->shared_blob->loaded) {
+         c->load_shared_blob(b->shared_blob);
+       }
+       for (auto e : r) {
+         vector<bluestore_pextent_t> cur;
+         b->shared_blob->shared_blob.ref_map.put(e.offset, e.length, &cur);
+         final.insert(final.end(), cur.begin(), cur.end());
+       }
+       dout(20) << __func__ << "  shared_blob release " << final
+                << " from " << *b->shared_blob << dendl;
+       txc->write_shared_blob(b->shared_blob);
+       r.clear();
+       r.swap(final);
+      }
     }
     // we can't invalidate our logical extents as we drop them because
     // other lextents (either in our onode or others) may still
@@ -6430,34 +7296,17 @@ void BlueStore::_wctx_finish(
     // that are no longer referenced but not deallocated (until they
     // age out of the cache naturally).
     b->discard_unallocated();
-    txc->statfs_delta.stored() -= l.length;
-    if (compressed) {
-      txc->statfs_delta.compressed_original() -= l.length;
-    }
     for (auto e : r) {
-      dout(20) << __func__ << " release " << e << dendl;
+      dout(20) << __func__ << "  release " << e << dendl;
       txc->released.insert(e.offset, e.length);
       txc->statfs_delta.allocated() -= e.length;
-      if (compressed) {
+      if (blob.is_compressed()) {
         txc->statfs_delta.compressed_allocated() -= e.length;
       }
     }
-    if (l.blob < 0) {
-      txc->write_bnode(o->bnode);
-    }
-  }
-  for (auto br : blobs2remove) {
-    dout(20) << __func__ << " rm blob " << *br.second << dendl;
-    txc->statfs_delta.compressed() -=
-      br.second->get_blob().get_compressed_payload_length();
-    if (br.first) {
-      o->blob_map.erase(br.second);
-    } else {
-      o->bnode->blob_map.erase(br.second);
-    }
   }
 
-  o->onode.compress_extent_map();
+  o->extent_map.compress_extent_map();
 }
 
 int BlueStore::_do_write(
@@ -6476,7 +7325,7 @@ int BlueStore::_do_write(
           << " 0x" << std::hex << offset << "~" << length
           << " - have 0x" << o->onode.size
           << " (" << std::dec << o->onode.size << ")"
-          << " bytes in " << o->onode.extent_map.size() << " lextents"
+          << " bytes"
           << dendl;
   _dump_onode(o);
 
@@ -6522,6 +7371,8 @@ int BlueStore::_do_write(
           << " comp_blob_size 0x" << std::hex << wctx.comp_blob_size
           << std::dec << dendl;
 
+  o->extent_map.fault_range(db, offset, length);
+
   bufferlist::iterator p = bl.begin();
   if (offset / min_alloc_size == (end - 1) / min_alloc_size &&
       (length != min_alloc_size)) {
@@ -6563,6 +7414,8 @@ int BlueStore::_do_write(
 
   _wctx_finish(txc, c, o, &wctx);
 
+  o->extent_map.dirty_range(txc->t, offset, length);
+
   if (end > o->onode.size) {
     dout(20) << __func__ << " extending size to 0x" << std::hex << end
             << std::dec << dendl;
@@ -6629,7 +7482,9 @@ int BlueStore::_do_zero(TransContext *txc,
   o->flush();
 
   WriteContext wctx;
-  o->onode.punch_hole(offset, length, &wctx.lex_old);
+  o->extent_map.fault_range(db, offset, length);
+  o->extent_map.punch_hole(offset, length, &wctx.old_extents);
+  o->extent_map.dirty_range(txc->t, offset, length);
   _wctx_finish(txc, c, o, &wctx);
 
   if (offset + length > o->onode.size) {
@@ -6650,6 +7505,7 @@ int BlueStore::_do_truncate(
 {
   dout(15) << __func__ << " " << c->cid << " " << o->oid
           << " 0x" << std::hex << offset << std::dec << dendl;
+  _dump_onode(o, 30);
 
   if (offset < o->onode.size) {
     // ensure any wal IO has completed before we truncate off any extents
@@ -6657,7 +7513,10 @@ int BlueStore::_do_truncate(
     o->flush();
 
     WriteContext wctx;
-    o->onode.punch_hole(offset, o->onode.size, &wctx.lex_old);
+    uint64_t length = o->onode.size - offset;
+    o->extent_map.fault_range(db, offset, length);
+    o->extent_map.punch_hole(offset, length, &wctx.old_extents);
+    o->extent_map.dirty_range(txc->t, offset, length);
     _wctx_finish(txc, c, o, &wctx);
   }
 
@@ -6696,6 +7555,9 @@ int BlueStore::_do_remove(
   o->exists = false;
   o->onode = bluestore_onode_t();
   txc->onodes.erase(o);
+  for (auto &s : o->extent_map.shards) {
+    txc->t->rmkey(PREFIX_OBJ, s.key);
+  }
   txc->t->rmkey(PREFIX_OBJ, o->key);
   return 0;
 }
@@ -6974,6 +7836,10 @@ int BlueStore::_clone(TransContext *txc,
         << " and " << newo->oid << dendl;
     return -EINVAL;
   }
+  oldo->extent_map.fault_range(db, 0, oldo->onode.size);
+  _dump_onode(oldo);
+
+  newo->extent_map.fault_range(db, 0, newo->onode.size);
 
   bufferlist bl;
   newo->exists = true;
@@ -6987,52 +7853,72 @@ int BlueStore::_clone(TransContext *txc,
     goto out;
 
   if (g_conf->bluestore_clone_cow) {
-    if (!oldo->onode.extent_map.empty()) {
-      if (!oldo->bnode) {
-       oldo->bnode = c->get_bnode(newo->oid.hobj.get_hash());
-      }
-      if (!newo->bnode) {
-       newo->bnode = oldo->bnode;
-      }
-      assert(newo->bnode == oldo->bnode);
-      // move blobs
-      map<int64_t,int64_t> moved_blobs;
-      for (auto& p : oldo->onode.extent_map) {
-        if (!p.second.is_shared()) {
-          BlobRef b;
-          if (moved_blobs.count(p.second.blob) == 0) {
-            b = oldo->blob_map.get(p.second.blob);
-            oldo->blob_map.erase(b);
-            newo->bnode->blob_map.claim(b);
-            moved_blobs[p.second.blob] = b->id;
-            dout(30) << __func__ << "  moving old onode blob " << p.second.blob
-                    << " to bnode blob " << b->id << dendl;
-            b->dirty_blob().clear_flag(bluestore_blob_t::FLAG_MUTABLE);
-            b->dirty_blob().set_flag(bluestore_blob_t::FLAG_HAS_REFMAP);
-          } else {
-            b = newo->bnode->blob_map.get(moved_blobs[p.second.blob]);
-          }
-          b->dirty_blob().get_ref(p.second.offset, p.second.length);
-        }
-      }
-      // update lextents
-      for (auto& p : oldo->onode.extent_map) {
-       if (moved_blobs.count(p.second.blob)) {
-         p.second.blob = -moved_blobs[p.second.blob];
+    // hmm, this could go into an ExtentMap::dup() method.
+    vector<BlobRef> id_to_blob(oldo->extent_map.extent_map.size());
+    for (auto &e : oldo->extent_map.extent_map) {
+      e.blob->last_encoded_id = -1;
+    }
+    int n = 0;
+    bool dirtied_oldo = false;
+    for (auto &e : oldo->extent_map.extent_map) {
+      BlobRef cb;  // clone blob
+      if (e.blob->last_encoded_id >= 0) {
+       // blob is already duped
+       cb = id_to_blob[e.blob->last_encoded_id];
+      } else {
+       // we need to dup the blob.
+       // make sure it is shared
+       const bluestore_blob_t& blob = e.blob->get_blob();
+       if (!blob.is_shared()) {
+         e.blob->dirty_blob().sbid = _assign_blobid(txc);
+         c->make_blob_shared(e.blob);
+         dirtied_oldo = true;  // fixme: overkill
+       } else if (!e.blob->shared_blob->loaded) {
+         c->load_shared_blob(e.blob->shared_blob);
+       }
+       cb = new Blob;
+       e.blob->last_encoded_id = n;
+       id_to_blob[n] = cb;
+       e.blob->dup(*cb);
+       if (cb->id >= 0) {
+         newo->extent_map.spanning_blob_map.insert(*cb);
+         cb->get();
+       }
+       // bump the extent refs on the copied blob's extents
+       for (auto p : blob.extents) {
+         if (p.is_valid()) {
+           e.blob->shared_blob->shared_blob.ref_map.get(p.offset, p.length);
+         }
+       }
+       txc->write_shared_blob(e.blob->shared_blob);
+       // ugly: duplicate deferred csum work, if any.
+       for (auto& dc : txc->deferred_csum) {
+         if (dc.blob == e.blob) {
+           dout(20) << __func__ << "  duplicating deferred csum for blob "
+                    << *e.blob << dendl;
+           txc->add_deferred_csum(cb, dc.b_off, dc.data);
+         }
        }
-       newo->onode.extent_map[p.first] = p.second;
-        assert(p.second.blob < 0);
-       newo->bnode->blob_map.get(-p.second.blob)->dirty_blob().get_ref(
-         p.second.offset, p.second.length);
-       txc->statfs_delta.stored() += p.second.length;
       }
-      _dump_onode(newo);
-      txc->write_bnode(newo->bnode);
-      if (!moved_blobs.empty()) {
-       txc->write_onode(oldo);
+      // dup extent
+      newo->extent_map.extent_map.insert(*new Extent(e.logical_offset,
+                                                    e.blob_offset,
+                                                    e.length, cb));
+      txc->statfs_delta.stored() += e.length;
+      if (e.blob->get_blob().is_compressed()) {
+       txc->statfs_delta.compressed_original() -= e.length;
       }
+      ++n;
+    }
+    if (dirtied_oldo) {
+      oldo->extent_map.dirty_range(txc->t, 0, oldo->onode.size); // overkill
+      txc->write_onode(oldo);
     }
     newo->onode.size = oldo->onode.size;
+    newo->extent_map.dirty_range(txc->t, 0, newo->onode.size);
+    txc->write_onode(newo);
+    _dump_onode(oldo);
+    _dump_onode(newo);
   } else {
     // read + write
     r = _do_read(c.get(), oldo, 0, oldo->onode.size, bl, 0);
@@ -7142,8 +8028,17 @@ int BlueStore::_rename(TransContext *txc,
   }
 
   txc->t->rmkey(PREFIX_OBJ, oldo->key);
-  txc->write_onode(oldo);
+
+  // rewrite shards
+  oldo->extent_map.fault_range(db, 0, oldo->onode.size);
+  for (auto &s : oldo->extent_map.shards) {
+    txc->t->rmkey(PREFIX_OBJ, s.key);
+    get_extent_shard_key(newo->key, s.offset, &s.key);
+    s.dirty = true;
+  }
+
   newo = oldo;
+  txc->write_onode(newo);
 
   // this adjusts oldo->{oid,key}, and reset oldo to a fresh empty
   // Onode in the old slot
@@ -7242,6 +8137,7 @@ int BlueStore::_split_collection(TransContext *txc,
   // blow away the caches.  FIXME.
   c->onode_map.clear();
   d->onode_map.clear();
+#warning assert that shared_blob cache (and others?) are also cleared..
 
   c->cnode.bits = bits;
   assert(d->cnode.bits == bits);
index 4dae5eb977f4e031d117d6923bb5d9c36b713c0a..264a770b5e3e89c2236a24a43077311d9a4d5bfa 100644 (file)
@@ -286,29 +286,114 @@ public:
     }
   };
 
-  struct BnodeSet;
+  struct SharedBlobSet;
+
+  /// in-memory shared blob state (incl cached buffers)
+  struct SharedBlob : public boost::intrusive::unordered_set_base_hook<> {
+    std::atomic_int nref = {0}; ///< reference count
+
+    // these are defined/set if the blob is marked 'shared'
+    uint64_t sbid = 0;          ///< shared blob id
+    string key;                 ///< key in kv store
+    SharedBlobSet *parent_set = 0;  ///< containing SharedBlobSet
+
+    // these are defined/set if the shared_blob is 'loaded'
+    bluestore_shared_blob_t shared_blob; ///< the actual shared state
+    bool loaded = false;        ///< whether shared_blob_t is loaded
+
+    BufferSpace bc;             ///< buffer cache
+
+    SharedBlob(uint64_t i, const string& k, Cache *c) : sbid(i), key(k), bc(c) {}
+    ~SharedBlob() {
+      assert(bc.empty());
+    }
+
+    friend void intrusive_ptr_add_ref(SharedBlob *b) { b->get(); }
+    friend void intrusive_ptr_release(SharedBlob *b) { b->put(); }
+
+    friend ostream& operator<<(ostream& out, const SharedBlob& sb);
+
+    void get() {
+      ++nref;
+    }
+    void put();
+
+    friend bool operator==(const SharedBlob &l, const SharedBlob &r) {
+      return l.sbid == r.sbid;
+    }
+    friend std::size_t hash_value(const SharedBlob &e) {
+      rjhash<uint32_t> h;
+      return h(e.sbid);
+    }
+  };
+  typedef boost::intrusive_ptr<SharedBlob> SharedBlobRef;
+
+  /// a lookup table of SharedBlobs
+  struct SharedBlobSet {
+    typedef boost::intrusive::unordered_set<SharedBlob>::bucket_type bucket_type;
+    typedef boost::intrusive::unordered_set<SharedBlob>::bucket_traits bucket_traits;
+
+    std::mutex lock;   ///< protect lookup, insertion, removal
+    int num_buckets;
+    vector<bucket_type> buckets;
+    boost::intrusive::unordered_set<SharedBlob> uset;
+
+    SharedBlob dummy;  ///< for lookups
+
+    explicit SharedBlobSet(unsigned n)
+      : num_buckets(n),
+       buckets(n),
+       uset(bucket_traits(buckets.data(), num_buckets)),
+       dummy(0, string(), nullptr) {
+      assert(n > 0);
+    }
+
+    SharedBlobRef lookup(uint64_t sbid);
+
+    void add(SharedBlob *sb) {
+      std::lock_guard<std::mutex> l(lock);
+      uset.insert(*sb);
+      sb->parent_set = this;
+    }
+
+    bool remove(SharedBlob *sb) {
+      std::lock_guard<std::mutex> l(lock);
+      if (sb->nref == 0) {
+       assert(sb->parent_set == this);
+       uset.erase(*sb);
+       return true;
+      }
+      return false;
+    }
+  };
 
   /// in-memory blob metadata and associated cached buffers (if any)
   struct Blob : public boost::intrusive::set_base_hook<> {
-    std::atomic_int nref;  ///< reference count
-    int64_t id = 0;          ///< id
-    BufferSpace bc;          ///< buffer cache
+    std::atomic_int nref = {0};     ///< reference count
+    int id = -1;                    ///< id, for spanning blobs only, >= 0
+    SharedBlobRef shared_blob;      ///< shared blob state (if any)
+
+    /// refs from this shard.  ephemeral if id<0, persisted if spanning.
+    bluestore_extent_ref_map_t ref_map;
+
+
+    int last_encoded_id = -1;       ///< (ephemeral) used during encoding only
 
   private:
     mutable bluestore_blob_t blob;  ///< decoded blob metadata
-    mutable bool undecoded = false; ///< true if blob_bl is newer than blob
     mutable bool dirty = true;      ///< true if blob is newer than blob_bl
     mutable bufferlist blob_bl;     ///< cached encoded blob
 
   public:
-    Blob(int64_t i, Cache *c) : nref(0), id(i), bc(c) {}
+    Blob() {}
     ~Blob() {
-      assert(bc.empty());
     }
 
     friend void intrusive_ptr_add_ref(Blob *b) { b->get(); }
     friend void intrusive_ptr_release(Blob *b) { b->put(); }
 
+    friend ostream& operator<<(ostream& out, const Blob &b);
+
     // comparators for intrusive_set
     friend bool operator<(const Blob &a, const Blob &b) {
       return a.id < b.id;
@@ -320,24 +405,23 @@ public:
       return a.id == b.id;
     }
 
-    friend ostream& operator<<(ostream& out, const Blob &b) {
-      return out << b.id << ":" << b.get_blob();
+    bool is_spanning() const {
+      return id >= 0;
+    }
+
+    void dup(Blob& o) {
+      o.id = id;
+      o.shared_blob = shared_blob;
+      o.ref_map = ref_map;
+      o.blob = blob;
+      o.dirty = dirty;
+      o.blob_bl = blob_bl;
     }
 
     const bluestore_blob_t& get_blob() const {
-      if (undecoded) {
-       bufferlist::iterator p = blob_bl.begin();
-       ::decode(blob, p);
-       undecoded = false;
-      }
       return blob;
     }
     bluestore_blob_t& dirty_blob() {
-      if (undecoded) {
-       bufferlist::iterator p = blob_bl.begin();
-       ::decode(blob, p);
-       undecoded = false;
-      }
       if (!dirty) {
        dirty = true;
        blob_bl.clear();
@@ -350,13 +434,20 @@ public:
     bool is_dirty() const {
       return dirty;
     }
-    bool is_undecoded() const {
-      return undecoded;
+
+    bool is_unreferenced(uint64_t offset, uint64_t length) const {
+      return !ref_map.intersects(offset, length);
     }
 
     /// discard buffers for unallocated regions
     void discard_unallocated();
 
+    /// get logical references
+    void get_ref(uint64_t offset, uint64_t length);
+    /// put logical references, and get back any released extents
+    bool put_ref(uint64_t offset, uint64_t length,  uint64_t min_alloc_size,
+                vector<bluestore_pextent_t> *r);
+
     void get() {
       ++nref;
     }
@@ -372,153 +463,137 @@ public:
       } else {
        assert(blob_bl.length());
       }
-      ::encode(blob_bl, bl);
+      bl.append(blob_bl);
     }
     void decode(bufferlist::iterator& p) {
-      ::decode(blob_bl, p);
-      undecoded = true;
+      bufferlist::iterator s = p;
+      ::decode(blob, p);
+      s.copy(p.get_off() - s.get_off(), blob_bl);
       dirty = false;
     }
   };
   typedef boost::intrusive_ptr<Blob> BlobRef;
+  typedef boost::intrusive::set<Blob> blob_map_t;
 
-  /// a map of blobs, indexed by int64_t
-  struct BlobMap {
-    typedef boost::intrusive::set<Blob> blob_map_t;
+  /// a logical extent, pointing to (some portion of) a blob
+  struct Extent : public boost::intrusive::set_base_hook<> {
+    uint32_t logical_offset = 0;      ///< logical offset
+    uint32_t blob_offset = 0;         ///< blob offset
+    uint32_t length = 0;              ///< length
+    BlobRef blob;                     ///< the blob with our data
 
-    blob_map_t blob_map;
+    explicit Extent() {}
+    explicit Extent(uint32_t lo) : logical_offset(lo) {}
+    Extent(uint32_t lo, uint32_t o, uint32_t l, BlobRef& b)
+      : logical_offset(lo), blob_offset(o), length(l), blob(b) {}
 
-    void encode(bufferlist& bl) const;
-    void decode(bufferlist::iterator& p, Cache *c);
-
-    bool empty() const {
-      return blob_map.empty();
+    // comparators for intrusive_set
+    friend bool operator<(const Extent &a, const Extent &b) {
+      return a.logical_offset < b.logical_offset;
     }
-
-    BlobRef get(int64_t id) {
-      Blob dummy(id, nullptr);
-      auto p = blob_map.find(dummy);
-      if (p != blob_map.end()) {
-       return &*p;
-      }
-      return nullptr;
+    friend bool operator>(const Extent &a, const Extent &b) {
+      return a.logical_offset > b.logical_offset;
     }
-
-    BlobRef new_blob(Cache *c) {
-      int64_t id = get_new_id();
-      Blob *b = new Blob(id, c);
-      b->get();
-      blob_map.insert(*b);
-      return b;
+    friend bool operator==(const Extent &a, const Extent &b) {
+      return a.logical_offset == b.logical_offset;
     }
 
-    void claim(BlobRef b) {
-      assert(b->id == 0);
-      b->id = get_new_id();
-      b->get();
-      blob_map.insert(*b);
+    bool blob_escapes_range(uint32_t o, uint32_t l) {
+      uint32_t bstart = logical_offset - blob_offset;
+      return (bstart < o ||
+             bstart + blob->get_blob().get_logical_length() > o + l);
     }
+  };
+  typedef boost::intrusive::set<Extent> extent_map_t;
 
-    void erase(BlobRef b) {
-      blob_map.erase(*b);
-      b->id = 0;
-      b->put();
-    }
+  friend ostream& operator<<(ostream& out, const Extent& e);
 
-    int64_t get_new_id() {
-      return blob_map.empty() ? 1 : blob_map.rbegin()->id + 1;
-    }
+  struct Collection;
+  struct Onode;
 
-    // must be called under protection of the Cache lock
-    void _clear() {
-      while (!blob_map.empty()) {
-       Blob *b = &*blob_map.begin();
-       b->bc._clear();
-       erase(b);
-      }
-    }
+  /// a sharded extent map, mapping offsets to lextents to blobs
+  struct ExtentMap {
+    Onode *onode;
+    extent_map_t extent_map;        ///< map of Extents to Blobs
+    blob_map_t spanning_blob_map;   ///< blobs that span shards
 
-    friend ostream& operator<<(ostream& out, const BlobMap& m) {
-      out << '{';
-      for (auto p = m.blob_map.begin(); p != m.blob_map.end(); ++p) {
-       if (p != m.blob_map.begin()) {
-         out << ',';
-       }
-       out << p->id << '=' << p->get_blob();
-      }
-      return out << '}';
-    }
-  };
+    bool faulted = false;    ///< for debug only
+
+    struct Shard {
+      string key;            ///< kv key
+      uint32_t offset;       ///< starting logical offset
+      bluestore_onode_t::shard_info *shard_info;
+      bool loaded = false;   ///< true if shard is loaded
+      bool dirty = false;    ///< true if shard is dirty and needs reencoding
+    };
+    vector<Shard> shards;    ///< shards
 
-  /// an in-memory extent-map, shared by a group of objects (w/ same hash value)
-  struct Bnode : public boost::intrusive::unordered_set_base_hook<> {
-    std::atomic_int nref;        ///< reference count
-    uint32_t hash;
-    string key;           ///< key under PREFIX_OBJ where we are stored
-    BnodeSet *bnode_set;  ///< reference to the containing set
+    bool inline_dirty = false;
+    bufferlist inline_bl;    ///< cached encoded map, if unsharded; empty=>dirty
 
-    BlobMap blob_map;
+    ExtentMap(Onode *o) : onode(o) {}
 
-    Bnode(uint32_t h, const string& k, BnodeSet *s)
-      : nref(0),
-       hash(h),
-       key(k),
-       bnode_set(s) {}
+    bool encode_some(uint32_t offset, uint32_t length, bufferlist& bl,
+                    unsigned *pn);
+    void decode_some(bufferlist& bl);
 
-    void get() {
-      ++nref;
-    }
-    void put();
+    void encode_spanning_blobs(bufferlist& bl);
+    void decode_spanning_blobs(Collection *c, bufferlist::iterator& p);
 
-    friend void intrusive_ptr_add_ref(Bnode *e) { e->get(); }
-    friend void intrusive_ptr_release(Bnode *e) { e->put(); }
+    BlobRef get_spanning_blob(int id);
 
-    friend bool operator==(const Bnode &l, const Bnode &r) {
-      return l.hash == r.hash;
-    }
-    friend std::size_t hash_value(const Bnode &e) {
-      return e.hash;
+    bool update(Onode *on, KeyValueDB::Transaction t, bool force);
+    void reshard(Onode *on);
+
+    /// initialize Shards from the onode
+    void init_shards(Onode *on, bool loaded, bool dirty);
+
+    /// return shard containing offset
+    vector<Shard>::iterator seek_shard(uint32_t offset) {
+      // fixme: we could do a binary search here
+      // we want the right-most shard that has an offset <= @offset.
+      vector<Shard>::iterator p = shards.begin();
+      while (p != shards.end() &&
+            p->offset <= offset) {
+       ++p;
+      }
+      if (p != shards.begin()) {
+       assert(p == shards.end() || p->offset > offset);
+       --p;
+       assert(p->offset <= offset);
+      }
+      return p;
     }
-  };
-  typedef boost::intrusive_ptr<Bnode> BnodeRef;
 
-  /// hash of Bnodes, by (object) hash value
-  struct BnodeSet {
-    typedef boost::intrusive::unordered_set<Bnode>::bucket_type bucket_type;
-    typedef boost::intrusive::unordered_set<Bnode>::bucket_traits bucket_traits;
+    /// ensure that a range of the map is loaded
+    void fault_range(KeyValueDB *db,
+                    uint32_t offset, uint32_t length);
 
-    std::mutex lock;
-    unsigned num_buckets;
-    vector<bucket_type> buckets;
-    boost::intrusive::unordered_set<Bnode> uset;
+    /// ensure a range of the map is marked dirty
+    void dirty_range(KeyValueDB::Transaction t,
+                    uint32_t offset, uint32_t length);
 
-    Bnode dummy;  ///< dummy entry used for lookups.  protected by lock.
+    /// find a lextent that includes offset
+    extent_map_t::iterator find_lextent(uint64_t offset);
 
-    explicit BnodeSet(unsigned n)
-      : num_buckets(n),
-       buckets(n),
-       uset(bucket_traits(buckets.data(), num_buckets)),
-       dummy(0, string(), NULL) {
-      assert(n > 0);
-    }
-    ~BnodeSet() {
-      assert(uset.empty());
-    }
+    /// seek to the first lextent including or after offset
+    extent_map_t::iterator seek_lextent(uint64_t offset);
 
-    BnodeRef get(uint32_t hash);
+    bool has_any_lextents(uint64_t offset, uint64_t length);
+
+    /// consolidate adjacent lextents in extent_map
+    int compress_extent_map();
+
+    /// punch a logical hole.  add lextents to deref to target list.
+    void punch_hole(uint64_t offset, uint64_t length,
+                   extent_map_t *old_extents);
+
+    /// put new lextent into lextent_map overwriting existing ones if
+    /// any and update references accordingly
+    Extent *set_lextent(uint64_t logical_offset,
+                       uint64_t offset, uint64_t length, BlobRef b,
+                       extent_map_t *old_extents);
 
-    void add(Bnode *b) {
-      std::lock_guard<std::mutex> l(lock);
-      uset.insert(*b);
-    }
-    bool remove(Bnode *b) {
-      std::lock_guard<std::mutex> l(lock);
-      if (b->nref == 0) {
-       uset.erase(*b);
-       return true;
-      }
-      return false;
-    }
   };
 
   struct OnodeSpace;
@@ -526,6 +601,7 @@ public:
   /// an in-memory object
   struct Onode {
     std::atomic_int nref;  ///< reference count
+    Collection *c;
 
     ghobject_t oid;
     string key;     ///< key under PREFIX_OBJ where we are stored
@@ -533,31 +609,23 @@ public:
     OnodeSpace *space;    ///< containing OnodeSpace
     boost::intrusive::list_member_hook<> lru_item;
 
-    BnodeRef bnode;  ///< ref to Bnode [optional]
-
     bluestore_onode_t onode;  ///< metadata stored as value in kv store
-    bool exists;
+    bool exists;              ///< true if object logically exists
 
-    BlobMap blob_map;       ///< local blobs (this onode onode)
+    ExtentMap extent_map;
 
     std::mutex flush_lock;  ///< protect flush_txns
     std::condition_variable flush_cond;   ///< wait here for unapplied txns
     set<TransContext*> flush_txns;   ///< committing or wal txns
 
-    Onode(OnodeSpace *s, const ghobject_t& o, const string& k)
+    Onode(OnodeSpace *s, Collection *c, const ghobject_t& o, const string& k)
       : nref(0),
+       c(c),
        oid(o),
        key(k),
        space(s),
-       exists(false) {
-    }
-
-    BlobRef get_blob(int64_t id) {
-      if (id < 0) {
-       assert(bnode);
-       return bnode->blob_map.get(-id);
-      }
-      return blob_map.get(id);
+       exists(false),
+       extent_map(this) {
     }
 
     void flush();
@@ -772,23 +840,33 @@ public:
 
     bool exists;
 
-    BnodeSet bnode_set;      ///< open Bnodes
+    SharedBlobSet shared_blob_set;      ///< open SharedBlobs
 
     // cache onodes on a per-collection basis to avoid lock
     // contention.
     OnodeSpace onode_map;
 
     OnodeRef get_onode(const ghobject_t& oid, bool create);
-    BnodeRef get_bnode(uint32_t hash);
 
-    BlobRef get_blob(OnodeRef& o, int64_t blob) {
-      if (blob < 0) {
-       if (!o->bnode) {
-         o->bnode = get_bnode(o->oid.hobj.get_hash());
-       }
-       return o->bnode->blob_map.get(-blob);
-      }
-      return o->blob_map.get(blob);
+    // the terminology is confusing here, sorry!
+    //
+    //  blob_t     shared_blob_t
+    //  !shared    unused                -> open
+    //  shared     !loaded               -> open + shared
+    //  shared     loaded                -> open + shared + loaded
+    //
+    // i.e.,
+    //  open = SharedBlob is instantiated
+    //  shared = blob_t shared flag is set; SharedBlob is hashed.
+    //  loaded = SharedBlob::shared_blob_t is loaded from kv store
+    void open_shared_blob(BlobRef b);
+    void load_shared_blob(SharedBlobRef sb);
+    void make_blob_shared(BlobRef b);
+
+    BlobRef new_blob() {
+      BlobRef b = new Blob;
+      b->shared_blob = new SharedBlob(0, string(), cache);
+      return b;
     }
 
     const coll_t &get_cid() override {
@@ -882,9 +960,9 @@ public:
 
     uint64_t ops, bytes;
 
-    set<OnodeRef> onodes;     ///< these onodes need to be updated/written
-    set<BnodeRef> bnodes;     ///< these bnodes need to be updated/written
-    set<BlobRef> blobs;       ///< these blobs need to be updated on io completion
+    set<OnodeRef> onodes;     ///< these need to be updated/written
+    set<SharedBlobRef> shared_blobs;  ///< these need to be updated/written
+    set<SharedBlobRef> shared_blobs_written; ///< update these on io completion
 
     KeyValueDB::Transaction t; ///< then we will commit this
     Context *oncommit;         ///< signal on commit
@@ -959,13 +1037,12 @@ public:
     utime_t start;
 
     struct DeferredCsum {
-      OnodeRef onode;
-      int64_t blob;
+      BlobRef blob;
       uint64_t b_off;
       bufferlist data;
 
-      DeferredCsum(OnodeRef& o, int64_t b, uint64_t bo, bufferlist& bl)
-       : onode(o), blob(b), b_off(bo), data(bl) {}
+      DeferredCsum(BlobRef& b, uint64_t bo, bufferlist& bl)
+       : blob(b), b_off(bo), data(bl) {}
     };
 
     list<DeferredCsum> deferred_csum;
@@ -991,12 +1068,12 @@ public:
     void write_onode(OnodeRef &o) {
       onodes.insert(o);
     }
-    void write_bnode(BnodeRef &e) {
-      bnodes.insert(e);
+    void write_shared_blob(SharedBlobRef &sb) {
+      shared_blobs.insert(sb);
     }
 
-    void add_deferred_csum(OnodeRef& o, int64_t b, uint64_t bo, bufferlist& bl) {
-      deferred_csum.emplace_back(TransContext::DeferredCsum(o, b, bo, bl));
+    void add_deferred_csum(BlobRef& b, uint64_t bo, bufferlist& bl) {
+      deferred_csum.emplace_back(TransContext::DeferredCsum(b, bo, bl));
     }
   };
 
@@ -1186,6 +1263,10 @@ private:
   uint64_t nid_last;
   uint64_t nid_max;
 
+  std::mutex blobid_lock;
+  uint64_t blobid_last = 0;
+  uint64_t blobid_max = 0;
+
   Throttle throttle_ops, throttle_bytes;          ///< submit to commit
   Throttle throttle_wal_ops, throttle_wal_bytes;  ///< submit to wal complete
 
@@ -1290,11 +1371,10 @@ private:
   void _reap_collections();
 
   void _assign_nid(TransContext *txc, OnodeRef o);
+  uint64_t _assign_blobid(TransContext *txc);
 
   void _dump_onode(OnodeRef o, int log_level=30);
-  void _dump_bnode(BnodeRef b, int log_level=30);
-  void _dump_blob_map(BlobMap &bm, int log_level);
-
+  void _dump_extent_map(ExtentMap& em, int log_level=30);
 
   TransContext *_txc_create(OpSequencer *osr);
   void _txc_update_store_statfs(TransContext *txc);
@@ -1331,11 +1411,10 @@ private:
   int _do_wal_op(TransContext *txc, bluestore_wal_op_t& wo);
   int _wal_replay();
 
-  // for fsck
-  int _fsck_verify_blob_map(
-    string what,
-    const BlobMap& blob_map,
-    map<int64_t,bluestore_extent_ref_map_t>& v,
+  int _fsck_check_extents(
+    const ghobject_t& oid,
+    const vector<bluestore_pextent_t>& extents,
+    bool compressed,
     boost::dynamic_bitset<> &used_blocks,
     store_statfs_t& expected_statfs);
 
@@ -1345,8 +1424,8 @@ private:
     uint64_t offset,
     bufferlist& bl,
     unsigned flags) {
-    b->bc.write(txc->seq, offset, bl, flags);
-    txc->blobs.insert(b);
+    b->shared_blob->bc.write(txc->seq, offset, bl, flags);
+    txc->shared_blobs_written.insert(b->shared_blob);
   }
 public:
   BlueStore(CephContext *cct, const string& path);
@@ -1571,7 +1650,7 @@ private:
     uint64_t comp_blob_size = 0; ///< target compressed blob size
     unsigned csum_order = 0;     ///< target checksum chunk order
 
-    vector<std::pair<uint64_t, bluestore_lextent_t> > lex_old; ///< must deref blobs
+    extent_map_t old_extents;       ///< must deref these blobs
 
     struct write_item {
       BlobRef b;
index f61ce43b9d33edd7bfc21f91fbfd04769dc012a8..e38ff30ab03a6ebe3509fbd96b75aa09d5d12747 100644 (file)
@@ -449,10 +449,10 @@ string bluestore_blob_t::get_flags_string(unsigned flags)
       s += '+';
     s += "has_unused";
   }
-  if (flags & FLAG_HAS_REFMAP) {
+  if (flags & FLAG_SHARED) {
     if (s.length())
       s += '+';
-    s += "has_refmap";
+    s += "shared";
   }
 
   return s;
@@ -463,6 +463,9 @@ void bluestore_blob_t::encode(bufferlist& bl) const
   ENCODE_START(1, 1, bl);
   small_encode(extents, bl);
   small_encode_varint(flags, bl);
+  if (is_shared()) {
+    small_encode_varint(sbid, bl);
+  }
   if (is_compressed()) {
     small_encode_varint_lowz(compressed_length_orig, bl);
     small_encode_varint_lowz(compressed_length, bl);
@@ -472,9 +475,6 @@ void bluestore_blob_t::encode(bufferlist& bl) const
     ::encode(csum_chunk_order, bl);
     small_encode_buf_lowz(csum_data, bl);
   }
-  if (has_refmap()) {
-    ::encode(ref_map, bl);
-  }
   if (has_unused()) {
     ::encode(unused_uint_t(unused.to_ullong()), bl);
   }
@@ -486,6 +486,9 @@ void bluestore_blob_t::decode(bufferlist::iterator& p)
   DECODE_START(1, p);
   small_decode(extents, p);
   small_decode_varint(flags, p);
+  if (is_shared()) {
+    small_decode_varint(sbid, p);
+  }
   if (is_compressed()) {
     small_decode_varint_lowz(compressed_length_orig, p);
     small_decode_varint_lowz(compressed_length, p);
@@ -500,9 +503,6 @@ void bluestore_blob_t::decode(bufferlist::iterator& p)
     csum_type = CSUM_NONE;
     csum_chunk_order = 0;
   }
-  if (has_refmap()) {
-    ::decode(ref_map, p);
-  }
   if (has_unused()) {
     unused_uint_t val;
     ::decode(val, p);
@@ -518,12 +518,12 @@ void bluestore_blob_t::dump(Formatter *f) const
     f->dump_object("extent", p);
   }
   f->close_section();
+  f->dump_unsigned("shared_blob_id", sbid);
   f->dump_unsigned("compressed_length_original", compressed_length_orig);
   f->dump_unsigned("compressed_length", compressed_length);
   f->dump_unsigned("flags", flags);
   f->dump_unsigned("csum_type", csum_type);
   f->dump_unsigned("csum_chunk_order", csum_chunk_order);
-  f->dump_object("ref_map", ref_map);
   f->open_array_section("csum_data");
   size_t n = get_csum_count();
   for (unsigned i = 0; i < n; ++i)
@@ -541,7 +541,6 @@ void bluestore_blob_t::generate_test_instances(list<bluestore_blob_t*>& ls)
   ls.push_back(new bluestore_blob_t);
   ls.back()->init_csum(CSUM_XXHASH32, 16, 65536);
   ls.back()->csum_data = buffer::claim_malloc(4, strdup("abcd"));
-  ls.back()->ref_map.get(3, 5);
   ls.back()->add_unused(0, 3, 4096);
   ls.back()->add_unused(8, 8, 4096);
   ls.back()->extents.emplace_back(bluestore_pextent_t(0x40100000, 0x10000));
@@ -552,8 +551,11 @@ void bluestore_blob_t::generate_test_instances(list<bluestore_blob_t*>& ls)
 
 ostream& operator<<(ostream& out, const bluestore_blob_t& o)
 {
-  out << "blob(" << o.extents
-      << " clen 0x" << std::hex
+  out << "blob(" << o.extents;
+  if (o.sbid) {
+    out << " sbid 0x" << std::hex << o.sbid << std::dec;
+  }
+  out << " clen 0x" << std::hex
       << o.compressed_length_orig
       << " -> 0x"
       << o.compressed_length
@@ -565,161 +567,12 @@ ostream& operator<<(ostream& out, const bluestore_blob_t& o)
     out << " " << o.get_csum_type_string(o.csum_type)
        << "/0x" << std::hex << (1ull << o.csum_chunk_order) << std::dec;
   }
-  if (!o.ref_map.empty()) {
-    out << " " << o.ref_map;
-  }
   if (o.has_unused())
     out << " unused=0x" << std::hex << o.unused.to_ullong() << std::dec;
   out << ")";
   return out;
 }
 
-void bluestore_blob_t::get_ref(
-  uint64_t offset,
-  uint64_t length)
-{
-  assert(has_refmap());
-  ref_map.get(offset, length);
-}
-
-bool bluestore_blob_t::put_ref(
-  uint64_t offset,
-  uint64_t length,
-  uint64_t min_release_size,
-  vector<bluestore_pextent_t> *r)
-{
-  assert(has_refmap());
-  return put_ref_external(
-    ref_map,
-    offset,
-    length,
-    min_release_size,
-    r);
-}
-
-
-bool bluestore_blob_t::put_ref_external(
-  bluestore_extent_ref_map_t& ref_map,
-  uint64_t offset,
-  uint64_t length,
-  uint64_t min_release_size,
-  vector<bluestore_pextent_t> *r)
-{
-  vector<bluestore_pextent_t> logical;
-  ref_map.put(offset, length, &logical);
-
-  r->clear();
-
-  // common case: all of it?
-  if (ref_map.empty()) {
-    uint64_t pos = 0;
-    for (auto& e : extents) {
-      if (e.is_valid()) {
-       r->push_back(e);
-      }
-      pos += e.length;
-    }
-    extents.resize(1);
-    extents[0].offset = bluestore_pextent_t::INVALID_OFFSET;
-    extents[0].length = pos;
-    return true;
-  }
-
-  // we cannot do partial deallocation on compressed blobs
-  if (has_flag(FLAG_COMPRESSED)) {
-    return false;
-  }
-
-  // we cannot release something smaller than our csum chunk size
-  if (has_csum() && get_csum_chunk_size() > min_release_size) {
-    min_release_size = get_csum_chunk_size();
-  }
-
-  // search from logical releases
-  for (auto le : logical) {
-    uint64_t r_off = le.offset;
-    auto p = ref_map.ref_map.lower_bound(le.offset);
-    if (p != ref_map.ref_map.begin()) {
-      --p;
-      r_off = p->first + p->second.length;
-      ++p;
-    } else {
-      r_off = 0;
-    }
-    uint64_t end;
-    if (p == ref_map.ref_map.end()) {
-      end = this->get_ondisk_length();
-    } else {
-      end = p->first;
-    }
-    r_off = ROUND_UP_TO(r_off, min_release_size);
-    end -= end % min_release_size;
-    if (r_off >= end) {
-      continue;
-    }
-    uint64_t r_len = end - r_off;
-
-    // cut it out of extents
-    struct vecbuilder {
-      vector<bluestore_pextent_t> v;
-      uint64_t invalid = 0;
-
-      void add_invalid(uint64_t length) {
-       invalid += length;
-      }
-      void flush() {
-       if (invalid) {
-         v.emplace_back(bluestore_pextent_t(bluestore_pextent_t::INVALID_OFFSET,
-                                            invalid));
-         invalid = 0;
-       }
-      }
-      void add(uint64_t offset, uint64_t length) {
-       if (offset == bluestore_pextent_t::INVALID_OFFSET) {
-         add_invalid(length);
-       } else {
-         flush();
-         v.emplace_back(bluestore_pextent_t(offset, length));
-       }
-      }
-    } vb;
-
-    assert(r_len > 0);
-    auto q = extents.begin();
-    assert(q != extents.end());
-    while (r_off >= q->length) {
-      vb.add(q->offset, q->length);
-      r_off -= q->length;
-      ++q;
-      assert(q != extents.end());
-    }
-    while (r_len > 0) {
-      uint64_t l = MIN(r_len, q->length - r_off);
-      if (q->is_valid()) {
-       r->push_back(bluestore_pextent_t(q->offset + r_off, l));
-      }
-      if (r_off) {
-       vb.add(q->offset, r_off);
-      }
-      vb.add_invalid(l);
-      if (r_off + l < q->length) {
-       vb.add(q->offset + r_off + l, q->length - (r_off + l));
-      }
-      r_len -= l;
-      r_off = 0;
-      ++q;
-      assert(q != extents.end() || r_len == 0);
-    }
-    while (q != extents.end()) {
-      vb.add(q->offset, q->length);
-      ++q;
-    }
-    vb.flush();
-    extents.swap(vb.v);
-  }
-  return false;
-}
-
 void bluestore_blob_t::calc_csum(uint64_t b_off, const bufferlist& bl)
 {
   switch (csum_type) {
@@ -788,75 +641,55 @@ int bluestore_blob_t::verify_csum(uint64_t b_off, const bufferlist& bl,
     return 0;
 }
 
-// bluestore_lextent_t
-void bluestore_lextent_t::encode(bufferlist& bl) const
+// bluestore_shared_blob_t
+void bluestore_shared_blob_t::encode(bufferlist& bl) const
 {
-  small_encode_signed_varint(blob, bl);
-  small_encode_varint_lowz(offset, bl);
-  small_encode_varint_lowz(length, bl);
+  ENCODE_START(1, 1, bl);
+  ::encode(ref_map, bl);
+  ENCODE_FINISH(bl);
 }
-void bluestore_lextent_t::decode(bufferlist::iterator& p)
+
+void bluestore_shared_blob_t::decode(bufferlist::iterator& p)
 {
-  small_decode_signed_varint(blob, p);
-  small_decode_varint_lowz(offset, p);
-  small_decode_varint_lowz(length, p);
+  DECODE_START(1, p);
+  ::decode(ref_map, p);
+  DECODE_FINISH(p);
 }
 
-void bluestore_lextent_t::dump(Formatter *f) const
+void bluestore_shared_blob_t::dump(Formatter *f) const
 {
-  f->dump_int("blob", blob);
-  f->dump_unsigned("offset", offset);
-  f->dump_unsigned("length", length);
+  f->dump_object("ref_map", ref_map);
+  f->open_array_section("objects");
+  /*for (auto &o : objects) {
+    f->dump_object("object", o);
+    }*/
+  f->close_section();
 }
 
-void bluestore_lextent_t::generate_test_instances(list<bluestore_lextent_t*>& ls)
+void bluestore_shared_blob_t::generate_test_instances(
+  list<bluestore_shared_blob_t*>& ls)
 {
-  ls.push_back(new bluestore_lextent_t);
-  ls.push_back(new bluestore_lextent_t(23232, 0, 4096));
-  ls.push_back(new bluestore_lextent_t(23232, 16384, 8192));
+  ls.push_back(new bluestore_shared_blob_t);
 }
 
-ostream& operator<<(ostream& out, const bluestore_lextent_t& lb)
+ostream& operator<<(ostream& out, const bluestore_shared_blob_t& sb)
 {
-  return out << "0x" << std::hex << lb.offset << "~" << lb.length << std::dec
-            << "->" << lb.blob;
+  out << "shared_blob(" << sb.ref_map << ")";
+  return out;
 }
 
 // bluestore_onode_t
-void small_encode(const map<uint64_t,bluestore_lextent_t>& extents, bufferlist& bl)
+
+void bluestore_onode_t::shard_info::dump(Formatter *f) const
 {
-  size_t n = extents.size();
-  small_encode_varint(n, bl);
-  if (n) {
-    auto p = extents.begin();
-    small_encode_varint_lowz(p->first, bl);
-    p->second.encode(bl);
-    uint64_t pos = p->first;
-    while (--n) {
-      ++p;
-      small_encode_varint_lowz((uint64_t)p->first - pos, bl);
-      p->second.encode(bl);
-      pos = p->first;
-    }
-  }
+  f->dump_unsigned("offset", offset);
+  f->dump_unsigned("bytes", bytes);
 }
 
-void small_decode(map<uint64_t,bluestore_lextent_t>& extents, bufferlist::iterator& p)
+ostream& operator<<(ostream& out, const bluestore_onode_t::shard_info& si)
 {
-  size_t n;
-  extents.clear();
-  small_decode_varint(n, p);
-  if (n) {
-    uint64_t pos;
-    small_decode_varint_lowz(pos, p);
-    extents[pos].decode(p);
-    while (--n) {
-      uint64_t delta;
-      small_decode_varint_lowz(delta, p);
-      pos += delta;
-      extents[pos].decode(p);
-    }
-  }
+  return out << std::hex << "0x" << si.offset << "(0x" << si.bytes << " bytes, "
+            << std::dec << si.extents << " extents)";
 }
 
 void bluestore_onode_t::encode(bufferlist& bl) const
@@ -865,8 +698,8 @@ void bluestore_onode_t::encode(bufferlist& bl) const
   ::encode(nid, bl);
   ::encode(size, bl);
   ::encode(attrs, bl);
-  small_encode(extent_map, bl);
   ::encode(omap_head, bl);
+  ::encode(extent_map_shards, bl);
   ::encode(expected_object_size, bl);
   ::encode(expected_write_size, bl);
   ::encode(alloc_hint_flags, bl);
@@ -879,8 +712,8 @@ void bluestore_onode_t::decode(bufferlist::iterator& p)
   ::decode(nid, p);
   ::decode(size, p);
   ::decode(attrs, p);
-  small_decode(extent_map, p);
   ::decode(omap_head, p);
+  ::decode(extent_map_shards, p);
   ::decode(expected_object_size, p);
   ::decode(expected_write_size, p);
   ::decode(alloc_hint_flags, p);
@@ -900,15 +733,12 @@ void bluestore_onode_t::dump(Formatter *f) const
     f->close_section();
   }
   f->close_section();
-  f->open_object_section("extent_map");
-  for (const auto& p : extent_map) {
-    f->open_object_section("extent");
-    f->dump_unsigned("logical_offset", p.first);
-    p.second.dump(f);
-    f->close_section();
+  f->dump_unsigned("omap_head", omap_head);
+  f->open_array_section("extent_map_shards");
+  for (auto si : extent_map_shards) {
+    f->dump_object("shard", si);
   }
   f->close_section();
-  f->dump_unsigned("omap_head", omap_head);
   f->dump_unsigned("expected_object_size", expected_object_size);
   f->dump_unsigned("expected_write_size", expected_write_size);
   f->dump_unsigned("alloc_hint_flags", alloc_hint_flags);
@@ -931,141 +761,7 @@ size_t bluestore_onode_t::get_preferred_csum_order() const
   return ctz(expected_write_size);
 }
 
-int bluestore_onode_t::compress_extent_map()
-{
-  if (extent_map.empty())
-    return 0;
-  int removed = 0;
-  auto p = extent_map.begin();
-  auto n = p;
-  for (++n; n != extent_map.end(); p = n++) {
-    while (n != extent_map.end() &&
-          p->first + p->second.length == n->first &&
-          p->second.blob == n->second.blob &&
-          p->second.offset + p->second.length == n->second.offset) {
-      p->second.length += n->second.length;
-      extent_map.erase(n++);
-      ++removed;
-    }
-    if (n == extent_map.end()) {
-      break;
-    }
-  }
-  return removed;
-}
-
-void bluestore_onode_t::punch_hole(
-  uint64_t offset,
-  uint64_t length,
-  vector<std::pair<uint64_t, bluestore_lextent_t> >*deref)
-{
-  auto p = seek_lextent(offset);
-  uint64_t end = offset + length;
-  while (p != extent_map.end()) {
-    if (p->first >= end) {
-      break;
-    }
-    if (p->first < offset) {
-      if (p->first + p->second.length > end) {
-       // split and deref middle
-       uint64_t front = offset - p->first;
-       deref->emplace_back(
-          std::make_pair(
-            offset,
-            bluestore_lextent_t(
-              p->second.blob,
-              p->second.offset + front,
-              length)));
-       extent_map[end] = bluestore_lextent_t(
-         p->second.blob,
-         p->second.offset + front + length,
-         p->second.length - front - length);
-       p->second.length = front;
-       break;
-      } else {
-       // deref tail
-       assert(p->first + p->second.length > offset); // else bug in seek_lextent
-       uint64_t keep = offset - p->first;
-       deref->emplace_back(
-          std::make_pair(
-            offset,
-            bluestore_lextent_t(
-              p->second.blob,
-              p->second.offset + keep,
-              p->second.length - keep)));
-       p->second.length = keep;
-       ++p;
-       continue;
-      }
-    }
-    if (p->first + p->second.length <= end) {
-      // deref whole lextent
-      deref->push_back(std::make_pair(p->first, p->second));
-      extent_map.erase(p++);
-      continue;
-    }
-    // deref head
-    uint64_t keep = (p->first + p->second.length) - end;
-    deref->emplace_back(
-      std::make_pair(
-        p->first,
-        bluestore_lextent_t(
-          p->second.blob,
-          p->second.offset,
-          p->second.length - keep)));
-    extent_map[end] = bluestore_lextent_t(
-      p->second.blob,
-      p->second.offset + p->second.length - keep,
-      keep);
-    extent_map.erase(p);
-    break;
-  }
-}
-
-void bluestore_onode_t::set_lextent(uint64_t offset,
-                   const bluestore_lextent_t& lext,
-                   bluestore_blob_t* b,
-                   vector<std::pair<uint64_t, bluestore_lextent_t> >*deref)
-{
-  punch_hole(offset, lext.length, deref);
-  extent_map[offset] = lext;
-  //increment reference for shared blobs only
-  if (b->has_refmap()) {
-    b->get_ref(lext.offset, lext.length);
-  }
-}
 
-bool bluestore_onode_t::deref_lextent( uint64_t offset,
-                         bluestore_lextent_t& lext,
-                         bluestore_blob_t* b,
-                         uint64_t min_alloc_size,
-                         vector<bluestore_pextent_t>* r)
-{
-  bool empty = false;
-  if (b->has_refmap()) {
-    empty = b->put_ref(lext.offset, lext.length, min_alloc_size, r);
-  } else {
-    bluestore_extent_ref_map_t temp_ref_map;
-    assert(offset >= lext.offset);
-    //determine the range in lextents map where specific blob can be referenced to.
-    uint64_t search_offset = offset - lext.offset;
-    uint64_t search_end = search_offset +
-                          (b->is_compressed() ?
-                             b->get_compressed_payload_original_length() :
-                             b->get_ondisk_length());
-    auto lp = seek_lextent(search_offset);
-    while (lp != extent_map.end() &&
-           lp->first < search_end) {
-      if (lp->second.blob == lext.blob) {
-        temp_ref_map.fill(lp->second.offset, lp->second.length);
-      }
-      ++lp;
-    }
-    temp_ref_map.get(lext.offset, lext.length); //insert a fake reference for the removed lextent
-    empty = b->put_ref_external( temp_ref_map, lext.offset, lext.length, min_alloc_size, r);
-  }
-  return empty;
-}
 // bluestore_wal_op_t
 
 void bluestore_wal_op_t::encode(bufferlist& bl) const
index ea227192ab7bfe8effa1ccce7cf3366600501c37..52c9eab9e30819f086cdbbfec72113f6392feae1 100644 (file)
@@ -175,7 +175,8 @@ struct bluestore_extent_ref_map_t {
     return ref_map.empty();
   }
 
-  //raw reference insertion that assumes no conflicts/interference with the existing references
+  // raw reference insertion that assumes no conflicts/interference
+  // with the existing references
   void fill(uint32_t offset, uint32_t len, int refs = 1) {
     auto p = ref_map.insert(
         map<uint32_t,record_t>::value_type(offset,
@@ -218,7 +219,7 @@ struct bluestore_blob_t {
     FLAG_COMPRESSED = 2,      ///< blob is compressed
     FLAG_CSUM = 4,            ///< blob has checksums
     FLAG_HAS_UNUSED = 8,      ///< blob has unused map
-    FLAG_HAS_REFMAP  = 16,    ///< blob has non-empty reference map
+    FLAG_SHARED = 16,         ///< blob is shared; see external SharedBlob
   };
   static string get_flags_string(unsigned flags);
 
@@ -286,6 +287,7 @@ struct bluestore_blob_t {
   }
 
   vector<bluestore_pextent_t> extents;///< raw data position on device
+  uint64_t sbid = 0;                  ///< shared blob id (if shared)
   uint32_t compressed_length_orig = 0;///< original length of compressed blob if any
   uint32_t compressed_length = 0;     ///< compressed length if any
   uint32_t flags = 0;                 ///< FLAG_*
@@ -293,7 +295,6 @@ struct bluestore_blob_t {
   uint8_t csum_type = CSUM_NONE;      ///< CSUM_*
   uint8_t csum_chunk_order = 0;       ///< csum block size is 1<<block_order bytes
 
-  bluestore_extent_ref_map_t ref_map; ///< references (empty when in onode)
   bufferptr csum_data;                ///< opaque vector of csum data
 
   typedef uint16_t unused_uint_t;
@@ -337,8 +338,8 @@ struct bluestore_blob_t {
   bool has_unused() const {
     return has_flag(FLAG_HAS_UNUSED);
   }
-  bool has_refmap() const {
-    return has_flag(FLAG_HAS_REFMAP);
+  bool is_shared() const {
+    return has_flag(FLAG_SHARED);
   }
 
   /// return chunk (i.e. min readable block) size for the blob
@@ -368,10 +369,6 @@ struct bluestore_blob_t {
     return p->offset + x_off;
   }
 
-  bool is_unreferenced(uint64_t offset, uint64_t length) const {
-    return !ref_map.intersects(offset, length);
-  }
-
   /// return true if the entire range is allocated (mapped to extents on disk)
   bool is_allocated(uint64_t b_off, uint64_t b_len) const {
     auto p = extents.begin();
@@ -448,16 +445,6 @@ struct bluestore_blob_t {
     }
   }
 
-  /// get logical references
-  void get_ref(uint64_t offset, uint64_t length);
-  /// put logical references, and get back any released extents
-  bool put_ref(uint64_t offset, uint64_t length,  uint64_t min_alloc_size,
-              vector<bluestore_pextent_t> *r);
-  /// put logical references using external ref_map, and get back any released extents
-  bool put_ref_external( bluestore_extent_ref_map_t& ref_map,
-               uint64_t offset, uint64_t length,  uint64_t min_alloc_size,
-               vector<bluestore_pextent_t> *r);
-
   int map(uint64_t x_off, uint64_t x_len,
           std::function<int(uint64_t,uint64_t)> f) const {
     auto p = extents.begin();
@@ -511,6 +498,13 @@ struct bluestore_blob_t {
     return len;
   }
 
+  uint32_t get_logical_length() const {
+    if (is_compressed()) {
+      return compressed_length_orig;
+    } else {
+      return get_ondisk_length();
+    }
+  }
 
   size_t get_csum_value_size() const {
     switch (csum_type) {
@@ -579,138 +573,72 @@ WRITE_CLASS_ENCODER(bluestore_blob_t)
 ostream& operator<<(ostream& out, const bluestore_blob_t& o);
 
 
-/// blob id: positive = local, negative = shared bnode
-typedef int64_t bluestore_blob_id_t;
-
-
-/// lextent: logical data block back by the extent
-struct bluestore_lextent_t {
-  bluestore_blob_id_t blob;  ///< blob
-  uint32_t offset;           ///< relative offset within the blob
-  uint32_t length;           ///< length within the blob
-
-  bluestore_lextent_t(bluestore_blob_id_t _blob = 0,
-                     uint32_t o = 0,
-                     uint32_t l = 0)
-    : blob(_blob),
-      offset(o),
-      length(l) {}
-
-  uint64_t end() const {
-    return offset + length;
-  }
-
-  bool is_shared() const {
-    return blob < 0;
-  }
+/// shared blob state
+struct bluestore_shared_blob_t {
+  bluestore_extent_ref_map_t ref_map;  ///< shared blob extents
+  //set<ghobject_t> objects;  ///< objects referencing these shared blocks (debug)
 
   void encode(bufferlist& bl) const;
   void decode(bufferlist::iterator& p);
-
   void dump(Formatter *f) const;
-  static void generate_test_instances(list<bluestore_lextent_t*>& o);
+  static void generate_test_instances(list<bluestore_shared_blob_t*>& ls);
+
+  bool empty() const {
+    return ref_map.empty();
+  }
 };
-WRITE_CLASS_ENCODER(bluestore_lextent_t)
+WRITE_CLASS_ENCODER(bluestore_shared_blob_t)
+
+ostream& operator<<(ostream& out, const bluestore_shared_blob_t& o);
 
-ostream& operator<<(ostream& out, const bluestore_lextent_t& o);
 
-typedef map<bluestore_blob_id_t, bluestore_blob_t> bluestore_blob_map_t;
+
+/// blob id: positive = local, negative = shared bnode
+typedef int64_t bluestore_blob_id_t;
 
 
 /// onode: per-object metadata
 struct bluestore_onode_t {
-  uint64_t nid;                        ///< numeric id (locally unique)
-  uint64_t size;                       ///< object size
+  uint64_t nid = 0;                    ///< numeric id (locally unique)
+  uint64_t size = 0;                   ///< object size
   map<string, bufferptr> attrs;        ///< attrs
-  map<uint64_t,bluestore_lextent_t> extent_map;  ///< extent refs
-  uint64_t omap_head;                  ///< id for omap root node
-
-  uint32_t expected_object_size;
-  uint32_t expected_write_size;
-  uint32_t alloc_hint_flags;
-
-  bluestore_onode_t()
-    : nid(0),
-      size(0),
-      omap_head(0),
-      expected_object_size(0),
-      expected_write_size(0),
-      alloc_hint_flags(0) {}
-
-  /// get preferred csum chunk size
-  size_t get_preferred_csum_order() const;
-
-  /// find a lextent that includes offset
-  map<uint64_t,bluestore_lextent_t>::iterator find_lextent(uint64_t offset) {
-    map<uint64_t,bluestore_lextent_t>::iterator fp =
-      extent_map.lower_bound(offset);
-    if (fp != extent_map.begin()) {
-      --fp;
-      if (fp->first + fp->second.length <= offset) {
-       ++fp;
-      }
-    }
-    if (fp != extent_map.end() && fp->first > offset)
-      return extent_map.end();  // extent is past offset
-    return fp;
-  }
-
-  /// seek to the first lextent including or after offset
-  map<uint64_t,bluestore_lextent_t>::iterator seek_lextent(uint64_t offset) {
-    map<uint64_t,bluestore_lextent_t>::iterator fp =
-      extent_map.lower_bound(offset);
-    if (fp != extent_map.begin()) {
-      --fp;
-      if (fp->first + fp->second.length <= offset) {
-       ++fp;
-      }
-    }
-    return fp;
-  }
+  uint64_t omap_head = 0;              ///< id for omap root node
 
-  bool has_any_lextents(uint64_t offset, uint64_t length) {
-    map<uint64_t,bluestore_lextent_t>::iterator fp =
-      extent_map.lower_bound(offset);
-    if (fp != extent_map.begin()) {
-      --fp;
-      if (fp->first + fp->second.length <= offset) {
-       ++fp;
-      }
+  struct shard_info {
+    uint32_t offset = 0;  ///< logical offset for start of shard
+    uint32_t bytes = 0;   ///< encoded bytes
+    uint32_t extents = 0; ///< extents
+    void encode(bufferlist& bl) const {
+      ::encode(offset, bl);
+      ::encode(bytes, bl);
+      ::encode(extents, bl);
     }
-    if (fp == extent_map.end() || fp->first >= offset + length) {
-      return false;
+    void decode(bufferlist::iterator& p) {
+      ::decode(offset, p);
+      ::decode(bytes, p);
+      ::decode(extents, p);
     }
-    return true;
-  }
-
-  /// consolidate adjacent lextents in extent_map
-  int compress_extent_map();
-
-  /// punch a logical hole.  add lextents to deref to target list.
-  void punch_hole(uint64_t offset, uint64_t length,
-                 vector<std::pair<uint64_t, bluestore_lextent_t> >*deref);
+    void dump(Formatter *f) const;
+  };
+  WRITE_CLASS_ENCODER(shard_info)
+  vector<shard_info> extent_map_shards; ///< extent map shards (if any)
 
-  /// put new lextent into lextent_map overwriting existing ones if any and update references accordingly
-  void set_lextent(uint64_t offset,
-                  const bluestore_lextent_t& lext,
-                  bluestore_blob_t* b,
-                  vector<std::pair<uint64_t, bluestore_lextent_t> >*deref);
+  uint32_t expected_object_size = 0;
+  uint32_t expected_write_size = 0;
+  uint32_t alloc_hint_flags = 0;
 
-  /// post process removed lextent to take care of blob references
-  /// returns true is underlying blob has to be released
-  bool deref_lextent(uint64_t offset,
-               bluestore_lextent_t& lext,
-               bluestore_blob_t* b,
-               uint64_t min_alloc_size,
-               vector<bluestore_pextent_t>* r);
+  /// get preferred csum chunk size
+  size_t get_preferred_csum_order() const;
 
   void encode(bufferlist& bl) const;
   void decode(bufferlist::iterator& p);
   void dump(Formatter *f) const;
   static void generate_test_instances(list<bluestore_onode_t*>& o);
 };
+WRITE_CLASS_ENCODER(bluestore_onode_t::shard_info)
 WRITE_CLASS_ENCODER(bluestore_onode_t)
 
+ostream& operator<<(ostream& out, const bluestore_onode_t::shard_info& si);
 
 /// writeahead-logged op
 struct bluestore_wal_op_t {
index 79c5db829e9a41246fe49ee8edd7a39b3abaafe7..f38055f995cc15333c16a350879bd24b16997347 100644 (file)
@@ -5414,7 +5414,7 @@ void store_statfs_t::dump(Formatter *f) const
 ostream& operator<<(ostream& out, const store_statfs_t &s)
 {
   out << std::hex
-      << " store_statfs(0x" << s.available
+      << "store_statfs(0x" << s.available
       << "/0x"  << s.total
       << ", stored 0x" << s.stored
       << "/0x"  << s.allocated
index cd1e7bf146ef9ea52c883d0afb5fec0c36fdf791..b883dd388f9b9b8f2a9714e0559898adadfb4b3d 100644 (file)
@@ -189,316 +189,6 @@ TEST(bluestore_extent_ref_map_t, intersects)
   ASSERT_FALSE(m.intersects(55, 1));
 }
 
-TEST(bluestore_blob_t, put_ref)
-{
-  unsigned mas = 4096;
-  unsigned mrs = 8192;
-
-  {
-    bluestore_blob_t b(bluestore_blob_t::FLAG_HAS_REFMAP);
-    vector<bluestore_pextent_t> r;
-    b.extents.push_back(bluestore_pextent_t(0, mas*2));
-    b.get_ref(0, mas*2);
-    ASSERT_TRUE(b.is_allocated(0, mas*2));
-    b.put_ref(0, mas*2, mrs, &r);
-    cout << "r " << r << " " << b << std::endl;
-    ASSERT_EQ(1u, r.size());
-    ASSERT_EQ(0u, r[0].offset);
-    ASSERT_EQ(mas*2, r[0].length);
-    ASSERT_FALSE(b.is_allocated(0, mas*2));
-    ASSERT_FALSE(b.is_allocated(0, mas));
-    ASSERT_FALSE(b.is_allocated(mas, 0));
-    ASSERT_FALSE(b.extents[0].is_valid());
-    ASSERT_EQ(mas*2, b.extents[0].length);
-  }
-  {
-    bluestore_blob_t b(bluestore_blob_t::FLAG_HAS_REFMAP);
-    vector<bluestore_pextent_t> r;
-    b.extents.push_back(bluestore_pextent_t(123, mas*2));
-    b.get_ref(0, mas*2);
-    b.put_ref(0, mas, mrs, &r);
-    cout << "r " << r << " " << b << std::endl;
-    ASSERT_EQ(0u, r.size());
-    ASSERT_TRUE(b.is_allocated(0, mas*2));
-    b.put_ref(mas, mas, mrs, &r);
-    cout << "r " << r << " " << b << std::endl;
-    ASSERT_EQ(1u, r.size());
-    ASSERT_EQ(123u, r[0].offset);
-    ASSERT_EQ(mas*2, r[0].length);
-    ASSERT_FALSE(b.is_allocated(0, mas*2));
-    ASSERT_FALSE(b.extents[0].is_valid());
-    ASSERT_EQ(mas*2, b.extents[0].length);
-  }
-  {
-    bluestore_blob_t b(bluestore_blob_t::FLAG_HAS_REFMAP);
-    vector<bluestore_pextent_t> r;
-    b.extents.push_back(bluestore_pextent_t(1, mas));
-    b.extents.push_back(bluestore_pextent_t(2, mas));
-    b.extents.push_back(bluestore_pextent_t(3, mas));
-    b.extents.push_back(bluestore_pextent_t(4, mas));
-    b.get_ref(0, mas*4);
-    b.put_ref(mas, mas, mrs, &r);
-    cout << "r " << r << " " << b << std::endl;
-    ASSERT_EQ(0u, r.size());
-    ASSERT_TRUE(b.is_allocated(0, mas*4));
-    ASSERT_TRUE(b.is_allocated(mas, mas));
-    b.put_ref(mas*2, mas, mrs, &r);
-    cout << "r " << r << " " << b << std::endl;
-    ASSERT_EQ(0u, r.size());
-    ASSERT_TRUE(b.is_allocated(mas*2, mas));
-    ASSERT_TRUE(b.is_allocated(0, mas*4));
-    b.put_ref(mas*3, mas, mrs, &r);
-    cout << "r " << r << " " << b << std::endl;
-    ASSERT_EQ(2u, r.size());
-    ASSERT_EQ(3u, r[0].offset);
-    ASSERT_EQ(mas, r[0].length);
-    ASSERT_EQ(4u, r[1].offset);
-    ASSERT_EQ(mas, r[1].length);
-    ASSERT_TRUE(b.is_allocated(0, mas*2));
-    ASSERT_FALSE(b.is_allocated(mas*2, mas*2));
-    ASSERT_TRUE(b.extents[0].is_valid());
-    ASSERT_TRUE(b.extents[1].is_valid());
-    ASSERT_FALSE(b.extents[2].is_valid());
-    ASSERT_EQ(3u, b.extents.size());
-  }
-  {
-    bluestore_blob_t b(bluestore_blob_t::FLAG_HAS_REFMAP);
-    vector<bluestore_pextent_t> r;
-    b.extents.push_back(bluestore_pextent_t(1, mas));
-    b.extents.push_back(bluestore_pextent_t(2, mas));
-    b.extents.push_back(bluestore_pextent_t(3, mas));
-    b.extents.push_back(bluestore_pextent_t(4, mas));
-    b.extents.push_back(bluestore_pextent_t(5, mas));
-    b.extents.push_back(bluestore_pextent_t(6, mas));
-    b.get_ref(0, mas*6);
-    b.put_ref(mas, mas, mrs, &r);
-    cout << "r " << r << " " << b << std::endl;
-    ASSERT_EQ(0u, r.size());
-    ASSERT_TRUE(b.is_allocated(0, mas*6));
-    b.put_ref(mas*2, mas, mrs, &r);
-    cout << "r " << r << " " << b << std::endl;
-    ASSERT_EQ(0u, r.size());
-    ASSERT_TRUE(b.is_allocated(0, mas*6));
-    b.put_ref(mas*3, mas, mrs, &r);
-    cout << "r " << r << " " << b << std::endl;
-    ASSERT_EQ(2u, r.size());
-    ASSERT_EQ(3u, r[0].offset);
-    ASSERT_EQ(mas, r[0].length);
-    ASSERT_EQ(4u, r[1].offset);
-    ASSERT_EQ(mas, r[1].length);
-    ASSERT_TRUE(b.is_allocated(0, mas*2));
-    ASSERT_FALSE(b.is_allocated(mas*2, mas*2));
-    ASSERT_TRUE(b.is_allocated(mas*4, mas*2));
-    ASSERT_EQ(5u, b.extents.size());
-    ASSERT_TRUE(b.extents[0].is_valid());
-    ASSERT_TRUE(b.extents[1].is_valid());
-    ASSERT_FALSE(b.extents[2].is_valid());
-    ASSERT_TRUE(b.extents[3].is_valid());
-    ASSERT_TRUE(b.extents[4].is_valid());
-  }
-  {
-    bluestore_blob_t b(bluestore_blob_t::FLAG_HAS_REFMAP);
-    vector<bluestore_pextent_t> r;
-    b.extents.push_back(bluestore_pextent_t(1, mas * 6));
-    b.get_ref(0, mas*6);
-    b.put_ref(mas, mas, mrs, &r);
-    cout << "r " << r << " " << b << std::endl;
-    ASSERT_EQ(0u, r.size());
-    ASSERT_TRUE(b.is_allocated(0, mas*6));
-    b.put_ref(mas*2, mas, mrs, &r);
-    cout << "r " << r << " " << b << std::endl;
-    ASSERT_EQ(0u, r.size());
-    ASSERT_TRUE(b.is_allocated(0, mas*6));
-    b.put_ref(mas*3, mas, mrs, &r);
-    cout << "r " << r << " " << b << std::endl;
-    ASSERT_EQ(1u, r.size());
-    ASSERT_EQ(0x2001u, r[0].offset);
-    ASSERT_EQ(mas*2, r[0].length);
-    ASSERT_TRUE(b.is_allocated(0, mas*2));
-    ASSERT_FALSE(b.is_allocated(mas*2, mas*2));
-    ASSERT_TRUE(b.is_allocated(mas*4, mas*2));
-    ASSERT_EQ(3u, b.extents.size());
-    ASSERT_TRUE(b.extents[0].is_valid());
-    ASSERT_FALSE(b.extents[1].is_valid());
-    ASSERT_TRUE(b.extents[2].is_valid());
-  }
-  {
-    bluestore_blob_t b(bluestore_blob_t::FLAG_HAS_REFMAP);
-    vector<bluestore_pextent_t> r;
-    b.extents.push_back(bluestore_pextent_t(1, mas * 4));
-    b.extents.push_back(bluestore_pextent_t(2, mas * 4));
-    b.extents.push_back(bluestore_pextent_t(3, mas * 4));
-    b.get_ref(0, mas*12);
-    b.put_ref(mas, mas, mrs, &r);
-    cout << "r " << r << " " << b << std::endl;
-    ASSERT_EQ(0u, r.size());
-    ASSERT_TRUE(b.is_allocated(0, mas*12));
-    b.put_ref(mas*9, mas, mrs, &r);
-    cout << "r " << r << " " << b << std::endl;
-    ASSERT_EQ(0u, r.size());
-    ASSERT_TRUE(b.is_allocated(0, mas*12));
-    b.put_ref(mas*2, mas*7, mrs, &r);
-    cout << "r " << r << " " << b << std::endl;
-    ASSERT_EQ(3u, r.size());
-    ASSERT_EQ(0x2001u, r[0].offset);
-    ASSERT_EQ(mas*2, r[0].length);
-    ASSERT_EQ(0x2u, r[1].offset);
-    ASSERT_EQ(mas*4, r[1].length);
-    ASSERT_EQ(0x3u, r[2].offset);
-    ASSERT_EQ(mas*2, r[2].length);
-    ASSERT_TRUE(b.is_allocated(0, mas*2));
-    ASSERT_FALSE(b.is_allocated(mas*2, mas*8));
-    ASSERT_TRUE(b.is_allocated(mas*10, mas*2));
-    ASSERT_EQ(3u, b.extents.size());
-    ASSERT_TRUE(b.extents[0].is_valid());
-    ASSERT_FALSE(b.extents[1].is_valid());
-    ASSERT_TRUE(b.extents[2].is_valid());
-  }
-  {
-    bluestore_blob_t b(bluestore_blob_t::FLAG_HAS_REFMAP);
-    vector<bluestore_pextent_t> r;
-    b.extents.push_back(bluestore_pextent_t(1, mas * 4));
-    b.extents.push_back(bluestore_pextent_t(2, mas * 4));
-    b.extents.push_back(bluestore_pextent_t(3, mas * 4));
-    b.get_ref(0, mas*12);
-    b.put_ref(mas, mas, mrs, &r);
-    cout << "r " << r << " " << b << std::endl;
-    ASSERT_EQ(0u, r.size());
-    ASSERT_TRUE(b.is_allocated(0, mas*12));
-    b.put_ref(mas*9, mas, mrs, &r);
-    cout << "r " << r << " " << b << std::endl;
-    ASSERT_EQ(0u, r.size());
-    ASSERT_TRUE(b.is_allocated(0, mas*12));
-    b.put_ref(mas*2, mas*7, mrs, &r);
-    cout << "r " << r << " " << b << std::endl;
-    ASSERT_EQ(3u, r.size());
-    ASSERT_EQ(0x2001u, r[0].offset);
-    ASSERT_EQ(mas*2, r[0].length);
-    ASSERT_EQ(0x2u, r[1].offset);
-    ASSERT_EQ(mas*4, r[1].length);
-    ASSERT_EQ(0x3u, r[2].offset);
-    ASSERT_EQ(mas*2, r[2].length);
-    ASSERT_TRUE(b.is_allocated(0, mas*2));
-    ASSERT_FALSE(b.is_allocated(mas*2, mas*8));
-    ASSERT_TRUE(b.is_allocated(mas*10, mas*2));
-    ASSERT_EQ(3u, b.extents.size());
-    ASSERT_TRUE(b.extents[0].is_valid());
-    ASSERT_FALSE(b.extents[1].is_valid());
-    ASSERT_TRUE(b.extents[2].is_valid());
-    b.put_ref(0, mas, mrs, &r);
-    cout << "r " << r << " " << b << std::endl;
-    ASSERT_EQ(1u, r.size());
-    ASSERT_EQ(0x1u, r[0].offset);
-    ASSERT_EQ(mas*2, r[0].length);
-    ASSERT_EQ(2u, b.extents.size());
-    ASSERT_FALSE(b.extents[0].is_valid());
-    ASSERT_TRUE(b.extents[1].is_valid());
-    b.put_ref(mas*10, mas*2, mrs, &r);
-    cout << "r " << r << " " << b << std::endl;
-    ASSERT_EQ(1u, r.size());
-    ASSERT_EQ(0x2003u, r[0].offset);
-    ASSERT_EQ(mas*2, r[0].length);
-    ASSERT_EQ(1u, b.extents.size());
-    ASSERT_FALSE(b.extents[0].is_valid());
-  }
-  {
-    bluestore_blob_t b(bluestore_blob_t::FLAG_HAS_REFMAP);
-    vector<bluestore_pextent_t> r;
-    b.extents.push_back(bluestore_pextent_t(1, mas * 4));
-    b.extents.push_back(bluestore_pextent_t(2, mas * 4));
-    b.extents.push_back(bluestore_pextent_t(3, mas * 4));
-    b.get_ref(0, mas*12);
-    b.put_ref(mas, mas, mrs, &r);
-    cout << "r " << r << " " << b << std::endl;
-    ASSERT_EQ(0u, r.size());
-    ASSERT_TRUE(b.is_allocated(0, mas*12));
-    b.put_ref(mas*9, mas, mrs, &r);
-    cout << "r " << r << " " << b << std::endl;
-    ASSERT_EQ(0u, r.size());
-    ASSERT_TRUE(b.is_allocated(0, mas*12));
-    b.put_ref(mas*2, mas*7, mrs, &r);
-    cout << "r " << r << " " << b << std::endl;
-    ASSERT_EQ(3u, r.size());
-    ASSERT_EQ(0x2001u, r[0].offset);
-    ASSERT_EQ(mas*2, r[0].length);
-    ASSERT_EQ(0x2u, r[1].offset);
-    ASSERT_EQ(mas*4, r[1].length);
-    ASSERT_EQ(0x3u, r[2].offset);
-    ASSERT_EQ(mas*2, r[2].length);
-    ASSERT_TRUE(b.is_allocated(0, mas*2));
-    ASSERT_FALSE(b.is_allocated(mas*2, mas*8));
-    ASSERT_TRUE(b.is_allocated(mas*10, mas*2));
-    ASSERT_EQ(3u, b.extents.size());
-    ASSERT_TRUE(b.extents[0].is_valid());
-    ASSERT_FALSE(b.extents[1].is_valid());
-    ASSERT_TRUE(b.extents[2].is_valid());
-    b.put_ref(mas*10, mas*2, mrs, &r);
-    cout << "r " << r << " " << b << std::endl;
-    ASSERT_EQ(1u, r.size());
-    ASSERT_EQ(0x2003u, r[0].offset);
-    ASSERT_EQ(mas*2, r[0].length);
-    ASSERT_EQ(2u, b.extents.size());
-    ASSERT_TRUE(b.extents[0].is_valid());
-    ASSERT_FALSE(b.extents[1].is_valid());
-    b.put_ref(0, mas, mrs, &r);
-    cout << "r " << r << " " << b << std::endl;
-    ASSERT_EQ(1u, r.size());
-    ASSERT_EQ(0x1u, r[0].offset);
-    ASSERT_EQ(mas*2, r[0].length);
-    ASSERT_EQ(1u, b.extents.size());
-    ASSERT_FALSE(b.extents[0].is_valid());
-  }
-  {
-    bluestore_blob_t b(bluestore_blob_t::FLAG_HAS_REFMAP);
-    vector<bluestore_pextent_t> r;
-    b.extents.push_back(bluestore_pextent_t(1, mas * 8));
-    b.get_ref(0, mas*8);
-    b.put_ref(0, mas, mrs, &r);
-    cout << "r " << r << " " << b << std::endl;
-    ASSERT_EQ(0u, r.size());
-    ASSERT_TRUE(b.is_allocated(0, mas*8));
-    b.put_ref(mas*7, mas, mrs, &r);
-    cout << "r " << r << " " << b << std::endl;
-    ASSERT_EQ(0u, r.size());
-    ASSERT_TRUE(b.is_allocated(0, mas*8));
-    b.put_ref(mas*2, mas, mrs, &r);
-    cout << "r " << r << " " << b << std::endl;
-    ASSERT_EQ(0u, r.size());
-    ASSERT_TRUE(b.is_allocated(0, 8));
-    b.put_ref(mas*3, mas*4, mrs, &r);
-    ASSERT_EQ(1u, r.size());
-    ASSERT_EQ(0x2001u, r[0].offset);
-    ASSERT_EQ(mas*6, r[0].length);
-    ASSERT_TRUE(b.is_allocated(0, mas*2));
-    ASSERT_FALSE(b.is_allocated(mas*2, mas*6));
-    ASSERT_EQ(2u, b.extents.size());
-    ASSERT_TRUE(b.extents[0].is_valid());
-    ASSERT_FALSE(b.extents[1].is_valid());
-    b.put_ref(mas, mas, mrs, &r);
-    cout << "r " << r << " " << b << std::endl;
-    ASSERT_EQ(1u, r.size());
-    ASSERT_EQ(0x1u, r[0].offset);
-    ASSERT_EQ(mas*2, r[0].length);
-    ASSERT_EQ(1u, b.extents.size());
-    ASSERT_FALSE(b.extents[0].is_valid());
-  }
-  // verify csum chunk size if factored in properly
-  {
-    bluestore_blob_t b(bluestore_blob_t::FLAG_HAS_REFMAP);
-    vector<bluestore_pextent_t> r;
-    b.extents.push_back(bluestore_pextent_t(0, mas*4));
-    b.init_csum(bluestore_blob_t::CSUM_CRC32C, 14, mas * 4);
-    b.get_ref(0, mas*4);
-    ASSERT_TRUE(b.is_allocated(0, mas*4));
-    b.put_ref(0, mas*3, mrs, &r);
-    cout << "r " << r << " " << b << std::endl;
-    ASSERT_EQ(0u, r.size());
-    ASSERT_TRUE(b.is_allocated(0, mas*4));
-    ASSERT_TRUE(b.extents[0].is_valid());
-    ASSERT_EQ(mas*4, b.extents[0].length);
-  }
-}
-
 TEST(bluestore_blob_t, calc_csum)
 {
   bufferlist bl;
@@ -609,443 +299,3 @@ TEST(bluestore_onode_t, get_preferred_csum_order)
   ASSERT_EQ(20u, on.get_preferred_csum_order());
 }
 
-TEST(bluestore_onode_t, find_lextent)
-{
-  bluestore_onode_t on;
-  ASSERT_EQ(on.extent_map.end(), on.find_lextent(0));
-  ASSERT_EQ(on.extent_map.end(), on.find_lextent(100));
-
-  on.extent_map[100] = bluestore_lextent_t(1, 0, 100);
-  map<uint64_t,bluestore_lextent_t>::iterator a = on.extent_map.find(100);
-  ASSERT_EQ(on.extent_map.end(), on.find_lextent(0));
-  ASSERT_EQ(on.extent_map.end(), on.find_lextent(99));
-  ASSERT_EQ(a, on.find_lextent(100));
-  ASSERT_EQ(a, on.find_lextent(101));
-  ASSERT_EQ(a, on.find_lextent(199));
-  ASSERT_EQ(on.extent_map.end(), on.find_lextent(200));
-
-  on.extent_map[200] = bluestore_lextent_t(2, 0, 100);
-  map<uint64_t,bluestore_lextent_t>::iterator b = on.extent_map.find(200);
-  ASSERT_EQ(on.extent_map.end(), on.find_lextent(0));
-  ASSERT_EQ(on.extent_map.end(), on.find_lextent(99));
-  ASSERT_EQ(a, on.find_lextent(100));
-  ASSERT_EQ(a, on.find_lextent(101));
-  ASSERT_EQ(a, on.find_lextent(199));
-  ASSERT_EQ(b, on.find_lextent(200));
-  ASSERT_EQ(b, on.find_lextent(299));
-  ASSERT_EQ(on.extent_map.end(), on.find_lextent(300));
-
-  on.extent_map[400] = bluestore_lextent_t(4, 0, 100);
-  map<uint64_t,bluestore_lextent_t>::iterator d = on.extent_map.find(400);
-  ASSERT_EQ(on.extent_map.end(), on.find_lextent(0));
-  ASSERT_EQ(on.extent_map.end(), on.find_lextent(99));
-  ASSERT_EQ(a, on.find_lextent(100));
-  ASSERT_EQ(a, on.find_lextent(101));
-  ASSERT_EQ(a, on.find_lextent(199));
-  ASSERT_EQ(b, on.find_lextent(200));
-  ASSERT_EQ(b, on.find_lextent(299));
-  ASSERT_EQ(on.extent_map.end(), on.find_lextent(300));
-  ASSERT_EQ(on.extent_map.end(), on.find_lextent(399));
-  ASSERT_EQ(d, on.find_lextent(400));
-  ASSERT_EQ(d, on.find_lextent(499));
-  ASSERT_EQ(on.extent_map.end(), on.find_lextent(500));
-}
-
-TEST(bluestore_onode_t, seek_lextent)
-{
-  bluestore_onode_t on;
-  ASSERT_EQ(on.extent_map.end(), on.seek_lextent(0));
-  ASSERT_EQ(on.extent_map.end(), on.seek_lextent(100));
-
-  on.extent_map[100] = bluestore_lextent_t(1, 0, 100);
-  map<uint64_t,bluestore_lextent_t>::iterator a = on.extent_map.find(100);
-  ASSERT_EQ(a, on.seek_lextent(0));
-  ASSERT_EQ(a, on.seek_lextent(99));
-  ASSERT_EQ(a, on.seek_lextent(100));
-  ASSERT_EQ(a, on.seek_lextent(101));
-  ASSERT_EQ(a, on.seek_lextent(199));
-  ASSERT_EQ(on.extent_map.end(), on.seek_lextent(200));
-
-  on.extent_map[200] = bluestore_lextent_t(2, 0, 100);
-  map<uint64_t,bluestore_lextent_t>::iterator b = on.extent_map.find(200);
-  ASSERT_EQ(a, on.seek_lextent(0));
-  ASSERT_EQ(a, on.seek_lextent(99));
-  ASSERT_EQ(a, on.seek_lextent(100));
-  ASSERT_EQ(a, on.seek_lextent(101));
-  ASSERT_EQ(a, on.seek_lextent(199));
-  ASSERT_EQ(b, on.seek_lextent(200));
-  ASSERT_EQ(b, on.seek_lextent(299));
-  ASSERT_EQ(on.extent_map.end(), on.seek_lextent(300));
-
-  on.extent_map[400] = bluestore_lextent_t(4, 0, 100);
-  map<uint64_t,bluestore_lextent_t>::iterator d = on.extent_map.find(400);
-  ASSERT_EQ(a, on.seek_lextent(0));
-  ASSERT_EQ(a, on.seek_lextent(99));
-  ASSERT_EQ(a, on.seek_lextent(100));
-  ASSERT_EQ(a, on.seek_lextent(101));
-  ASSERT_EQ(a, on.seek_lextent(199));
-  ASSERT_EQ(b, on.seek_lextent(200));
-  ASSERT_EQ(b, on.seek_lextent(299));
-  ASSERT_EQ(d, on.seek_lextent(300));
-  ASSERT_EQ(d, on.seek_lextent(399));
-  ASSERT_EQ(d, on.seek_lextent(400));
-  ASSERT_EQ(d, on.seek_lextent(499));
-  ASSERT_EQ(on.extent_map.end(), on.seek_lextent(500));
-}
-
-TEST(bluestore_onode_t, has_any_lextents)
-{
-  bluestore_onode_t on;
-  ASSERT_FALSE(on.has_any_lextents(0, 0));
-  ASSERT_FALSE(on.has_any_lextents(0, 1000));
-  ASSERT_FALSE(on.has_any_lextents(1000, 1000));
-
-  on.extent_map[100] = bluestore_lextent_t(1, 0, 100);
-  ASSERT_FALSE(on.has_any_lextents(0, 50));
-  ASSERT_FALSE(on.has_any_lextents(0, 100));
-  ASSERT_FALSE(on.has_any_lextents(50, 50));
-  ASSERT_TRUE(on.has_any_lextents(50, 51));
-  ASSERT_TRUE(on.has_any_lextents(50, 100051));
-  ASSERT_TRUE(on.has_any_lextents(100, 100));
-  ASSERT_TRUE(on.has_any_lextents(100, 1));
-  ASSERT_TRUE(on.has_any_lextents(199, 1));
-  ASSERT_TRUE(on.has_any_lextents(199, 2));
-  ASSERT_FALSE(on.has_any_lextents(200, 2));
-
-  on.extent_map[200] = bluestore_lextent_t(2, 0, 100);
-  ASSERT_TRUE(on.has_any_lextents(199, 1));
-  ASSERT_TRUE(on.has_any_lextents(199, 2));
-  ASSERT_TRUE(on.has_any_lextents(200, 2));
-  ASSERT_TRUE(on.has_any_lextents(200, 200));
-  ASSERT_TRUE(on.has_any_lextents(299, 1));
-  ASSERT_FALSE(on.has_any_lextents(300, 1));
-
-  on.extent_map[400] = bluestore_lextent_t(4, 0, 100);
-  ASSERT_TRUE(on.has_any_lextents(0, 10000));
-  ASSERT_TRUE(on.has_any_lextents(199, 1));
-  ASSERT_FALSE(on.has_any_lextents(300, 1));
-  ASSERT_FALSE(on.has_any_lextents(300, 100));
-  ASSERT_FALSE(on.has_any_lextents(399, 1));
-  ASSERT_TRUE(on.has_any_lextents(400, 1));
-  ASSERT_TRUE(on.has_any_lextents(400, 100));
-  ASSERT_TRUE(on.has_any_lextents(400, 1000));
-  ASSERT_TRUE(on.has_any_lextents(499, 1000));
-  ASSERT_FALSE(on.has_any_lextents(500, 1000));
-}
-
-TEST(bluestore_onode_t, compress_extent_map)
-{
-  bluestore_onode_t on;
-  vector<bluestore_lextent_t> r;
-  on.extent_map[0] = bluestore_lextent_t(1, 0, 100);
-  on.extent_map[100] = bluestore_lextent_t(2, 0, 100);
-  ASSERT_EQ(0, on.compress_extent_map());
-  ASSERT_EQ(2u, on.extent_map.size());
-
-  on.extent_map[200] = bluestore_lextent_t(2, 100, 100);
-  on.extent_map[300] = bluestore_lextent_t(2, 200, 100);
-  ASSERT_EQ(2, on.compress_extent_map());
-  ASSERT_EQ(2u, on.extent_map.size());
-
-  on.extent_map[200] = bluestore_lextent_t(3, 100, 100);
-  on.extent_map[300] = bluestore_lextent_t(2, 200, 100);
-  ASSERT_EQ(0, on.compress_extent_map());
-  ASSERT_EQ(4u, on.extent_map.size());
-
-  on.extent_map[400] = bluestore_lextent_t(2, 300, 100);
-  on.extent_map[500] = bluestore_lextent_t(2, 500, 100);
-  on.extent_map[600] = bluestore_lextent_t(2, 600, 100);
-  ASSERT_EQ(2, on.compress_extent_map());
-  ASSERT_EQ(5u, on.extent_map.size());
-
-  on.extent_map[400] = bluestore_lextent_t(2, 300, 100);
-  on.extent_map[500] = bluestore_lextent_t(2, 400, 100);
-  on.extent_map[700] = bluestore_lextent_t(2, 500, 100);
-  ASSERT_EQ(1, on.compress_extent_map());
-  ASSERT_EQ(6u, on.extent_map.size());
-}
-
-TEST(bluestore_onode_t, punch_hole)
-{
-  bluestore_onode_t on;
-  vector<std::pair<uint64_t, bluestore_lextent_t> > r;
-  on.extent_map[0] = bluestore_lextent_t(1, 0, 100);
-  on.extent_map[100] = bluestore_lextent_t(2, 0, 100);
-
-  on.punch_hole(0, 100, &r);
-  ASSERT_EQ(1u, on.extent_map.size());
-  ASSERT_EQ(1u, r.size());
-  ASSERT_EQ(1, r[0].second.blob);
-  ASSERT_EQ(0u, r[0].second.offset);
-  ASSERT_EQ(100u, r[0].second.length);
-  r.clear();
-
-  on.punch_hole(150, 10, &r);
-  ASSERT_EQ(2u, on.extent_map.size());
-  ASSERT_EQ(100u, on.extent_map.begin()->first);
-  ASSERT_EQ(0u, on.extent_map.begin()->second.offset);
-  ASSERT_EQ(50u, on.extent_map.begin()->second.length);
-  ASSERT_EQ(160u, on.extent_map.rbegin()->first);
-  ASSERT_EQ(60u, on.extent_map.rbegin()->second.offset);
-  ASSERT_EQ(40u, on.extent_map.rbegin()->second.length);
-  ASSERT_EQ(1u, r.size());
-  ASSERT_EQ(2, r[0].second.blob);
-  ASSERT_EQ(50u, r[0].second.offset);
-  ASSERT_EQ(10u, r[0].second.length);
-  r.clear();
-
-  on.punch_hole(140, 20, &r);
-  ASSERT_EQ(2u, on.extent_map.size());
-  ASSERT_EQ(100u, on.extent_map.begin()->first);
-  ASSERT_EQ(0u, on.extent_map.begin()->second.offset);
-  ASSERT_EQ(40u, on.extent_map.begin()->second.length);
-  ASSERT_EQ(160u, on.extent_map.rbegin()->first);
-  ASSERT_EQ(60u, on.extent_map.rbegin()->second.offset);
-  ASSERT_EQ(40u, on.extent_map.rbegin()->second.length);
-  ASSERT_EQ(1u, r.size());
-  ASSERT_EQ(2, r[0].second.blob);
-  ASSERT_EQ(40u, r[0].second.offset);
-  ASSERT_EQ(10u, r[0].second.length);
-  r.clear();
-
-  on.punch_hole(130, 40, &r);
-  ASSERT_EQ(2u, on.extent_map.size());
-  ASSERT_EQ(100u, on.extent_map.begin()->first);
-  ASSERT_EQ(0u, on.extent_map.begin()->second.offset);
-  ASSERT_EQ(30u, on.extent_map.begin()->second.length);
-  ASSERT_EQ(170u, on.extent_map.rbegin()->first);
-  ASSERT_EQ(70u, on.extent_map.rbegin()->second.offset);
-  ASSERT_EQ(30u, on.extent_map.rbegin()->second.length);
-  ASSERT_EQ(2u, r.size());
-  ASSERT_EQ(2, r[0].second.blob);
-  ASSERT_EQ(30u, r[0].second.offset);
-  ASSERT_EQ(10u, r[0].second.length);
-  ASSERT_EQ(2, r[1].second.blob);
-  ASSERT_EQ(60u, r[1].second.offset);
-  ASSERT_EQ(10u, r[1].second.length);
-  r.clear();
-
-  on.punch_hole(110, 10, &r);
-  ASSERT_EQ(3u, on.extent_map.size());
-  ASSERT_EQ(100u, on.extent_map.begin()->first);
-  ASSERT_EQ(0u, on.extent_map.begin()->second.offset);
-  ASSERT_EQ(10u, on.extent_map.begin()->second.length);
-  ASSERT_EQ(20u, on.extent_map[120].offset);
-  ASSERT_EQ(10u, on.extent_map[120].length);
-  ASSERT_EQ(170u, on.extent_map.rbegin()->first);
-  ASSERT_EQ(70u, on.extent_map.rbegin()->second.offset);
-  ASSERT_EQ(30u, on.extent_map.rbegin()->second.length);
-  ASSERT_EQ(1u, r.size());
-  ASSERT_EQ(2, r[0].second.blob);
-  ASSERT_EQ(10u, r[0].second.offset);
-  ASSERT_EQ(10u, r[0].second.length);
-  r.clear();
-
-  on.punch_hole(0, 1000, &r);
-  ASSERT_EQ(0u, on.extent_map.size());
-  ASSERT_EQ(3u, r.size());
-  ASSERT_EQ(2, r[0].second.blob);
-  ASSERT_EQ(0u, r[0].second.offset);
-  ASSERT_EQ(10u, r[0].second.length);
-  ASSERT_EQ(2, r[1].second.blob);
-  ASSERT_EQ(20u, r[1].second.offset);
-  ASSERT_EQ(10u, r[1].second.length);
-  ASSERT_EQ(2, r[2].second.blob);
-  ASSERT_EQ(70u, r[2].second.offset);
-  ASSERT_EQ(30u, r[2].second.length);
-  r.clear();
-}
-
-TEST(bluestore_onode_t, insert_remove_lextent)
-{
-  bluestore_onode_t on;
-  vector<std::pair<uint64_t, bluestore_lextent_t> > r;
-  vector<bluestore_pextent_t> rp;
-
-  bluestore_pextent_t pext1(1, 0x10000);
-  bluestore_pextent_t pext2(2, 0x10000);
-  bluestore_pextent_t pext3(3, 0x10000);
-
-  bluestore_blob_t blob(bluestore_blob_t::FLAG_HAS_REFMAP);
-  blob.extents.push_back(pext1);
-
-  bluestore_blob_t blob2, blob3;
-  blob2.clear_flag(bluestore_blob_t::FLAG_HAS_REFMAP);
-  blob2.extents.push_back(pext2);
-  blob3.clear_flag(bluestore_blob_t::FLAG_HAS_REFMAP);
-  blob3.extents.push_back(pext3);
-
-  bool empty;
-
-  bluestore_lextent_t lextent(1, 0, 100);
-  on.set_lextent(0, lextent, &blob, &r);
-
-  ASSERT_EQ(1u, on.extent_map.size());
-  ASSERT_EQ(0u, r.size());
-  ASSERT_TRUE(blob.ref_map.contains(0,100));
-  r.clear();
-
-  lextent.blob = 2;
-  lextent.offset = 1;
-  lextent.length = 99;
-  on.set_lextent(101, lextent, &blob2, &r);
-
-  ASSERT_EQ(2u, on.extent_map.size());
-  ASSERT_EQ(0u, r.size());
-  ASSERT_TRUE(blob.ref_map.contains(0,100));
-  ASSERT_TRUE(blob2.ref_map.empty());
-  r.clear();
-
-  //overwrite lextent/blob that doesn't have REF_MAP
-  lextent.blob = 3;
-  lextent.offset = 1;
-  lextent.length = 99;
-  on.set_lextent(101, lextent, &blob3, &r);
-
-  ASSERT_EQ(2u, on.extent_map.size());
-  ASSERT_EQ(1u, r.size());
-  ASSERT_EQ(2, r[0].second.blob);
-  ASSERT_EQ(1u, r[0].second.offset);
-  ASSERT_EQ(99u, r[0].second.length);
-  ASSERT_TRUE(blob.ref_map.contains(0,100));
-  ASSERT_TRUE(blob2.ref_map.empty());
-  ASSERT_TRUE(blob3.ref_map.empty());
-
-  //deref overwritten lextent
-  empty = on.deref_lextent(100, r[0].second, &blob2, 0x10000, &rp);
-  ASSERT_TRUE(empty);
-  ASSERT_EQ(1u, rp.size());
-  ASSERT_TRUE(pext2.offset == rp[0].offset);
-  ASSERT_TRUE(pext2.length == rp[0].length);
-
-  r.clear();
-  rp.clear();
-
-  //overwrite lextent/blob that has a REF_MAP with one that doesn't
-  lextent.blob = 2;
-  lextent.offset = 0;
-  lextent.length = 100;
-  blob2.extents.clear(); //for sure
-  blob2.extents.push_back(pext2);
-  on.set_lextent(0, lextent, &blob2, &r);
-
-  ASSERT_EQ(2u, on.extent_map.size());
-  ASSERT_EQ(1u, r.size());
-  ASSERT_EQ(1, r[0].second.blob);
-  ASSERT_EQ(0u, r[0].second.offset);
-  ASSERT_EQ(100u, r[0].second.length);
-  ASSERT_TRUE(blob.ref_map.contains(0,100));
-  ASSERT_TRUE(blob2.ref_map.empty());
-  ASSERT_TRUE(blob3.ref_map.empty());
-
-  //deref overwritten lextent
-  empty = on.deref_lextent(0, r[0].second, &blob, 0x10000, &rp);
-  ASSERT_TRUE(empty);
-  ASSERT_EQ(1u, rp.size());
-  ASSERT_TRUE(pext1.offset == rp[0].offset);
-  ASSERT_TRUE(pext1.length == rp[0].length);
-  ASSERT_TRUE(blob.ref_map.empty());
-
-  r.clear();
-  rp.clear();
-
-  //append an lextent pointing to already present blob 3
-  lextent.blob = 3;
-  lextent.offset = 200;
-  lextent.length = 50;
-  on.set_lextent(300, lextent, &blob3, &r);
-
-  ASSERT_EQ(3u, on.extent_map.size());
-  ASSERT_EQ(0u, r.size());
-  ASSERT_TRUE(blob3.ref_map.empty());
-
-  //deref lextent with underlying blob having multiple references (no ref_map case)
-  on.punch_hole(100, 100, &r);
-  ASSERT_EQ(1u, r.size());
-  ASSERT_EQ(3, r[0].second.blob);
-  ASSERT_EQ(1u, r[0].second.offset);
-  ASSERT_EQ(99u, r[0].second.length);
-
-  empty = on.deref_lextent(100, r[0].second, &blob3, 0x10000, &rp);
-  ASSERT_FALSE(empty);
-  ASSERT_EQ(0u, rp.size());
-
-  r.clear();
-  rp.clear();
-
-  //deref lextent with underlying blob having single reference (no ref_map case)
-  on.punch_hole(300, 100, &r);
-  ASSERT_EQ(1u, r.size());
-  ASSERT_EQ(3, r[0].second.blob);
-  ASSERT_EQ(200u, r[0].second.offset);
-  ASSERT_EQ(50u, r[0].second.length);
-
-  empty = on.deref_lextent(300, r[0].second, &blob3, 0x10000, &rp);
-  ASSERT_TRUE(empty);
-  ASSERT_EQ(1u, rp.size());
-  ASSERT_TRUE(pext3.offset == rp[0].offset);
-  ASSERT_TRUE(pext3.length == rp[0].length);
-  ASSERT_TRUE(blob3.ref_map.empty());
-
-  r.clear();
-  rp.clear();
-
-  //deref lextent partially (no ref_map case)
-  on.punch_hole(20, 10, &r);
-  ASSERT_EQ(2, r[0].second.blob);
-  ASSERT_EQ(20u, r[0].second.offset);
-  ASSERT_EQ(10u, r[0].second.length);
-
-  empty = on.deref_lextent(20, r[0].second, &blob2, 0x10000, &rp);
-  ASSERT_FALSE(empty);
-  ASSERT_EQ(0u, rp.size());
-
-  r.clear();
-  rp.clear();
-
-  //deref lextent partially once again(no ref_map case)
-  on.punch_hole(70, 10, &r);
-  ASSERT_EQ(1u, r.size());
-  ASSERT_EQ(2, r[0].second.blob);
-  ASSERT_EQ(70u, r[0].second.offset);
-  ASSERT_EQ(10u, r[0].second.length);
-
-  empty = on.deref_lextent(70, r[0].second, &blob2, 0x10000, &rp);
-  ASSERT_FALSE(empty);
-  ASSERT_EQ(0u, rp.size());
-
-  r.clear();
-  rp.clear();
-
-  //deref fragmented lextent totally (no ref_map case)
-  on.punch_hole(0, 100, &r);
-  ASSERT_EQ(3u, r.size());
-  ASSERT_EQ(2, r[0].second.blob);
-  ASSERT_EQ(0u, r[0].second.offset);
-  ASSERT_EQ(20u, r[0].second.length);
-  ASSERT_EQ(2, r[1].second.blob);
-  ASSERT_EQ(30u, r[1].second.offset);
-  ASSERT_EQ(40u, r[1].second.length);
-  ASSERT_EQ(2, r[2].second.blob);
-  ASSERT_EQ(80u, r[2].second.offset);
-  ASSERT_EQ(20u, r[2].second.length);
-
-  empty = on.deref_lextent(0, r[0].second, &blob2, 0x10000, &rp);
-  ASSERT_TRUE(empty);
-  ASSERT_EQ(1u, rp.size());
-  ASSERT_TRUE(pext2.offset == rp[0].offset);
-  ASSERT_TRUE(pext2.length == rp[0].length);
-  rp.clear();
-
-  empty = on.deref_lextent(30, r[1].second, &blob2, 0x10000, &rp);
-  ASSERT_TRUE(empty);
-  ASSERT_EQ(0u, rp.size()); //no more pextents for the blob, already deallocated above
-  rp.clear();
-
-  empty = on.deref_lextent(80, r[2].second, &blob2, 0x10000, &rp);
-  ASSERT_TRUE(empty);
-  ASSERT_EQ(0u, rp.size()); //no more pextents for the blob, already deallocated above
-
-  r.clear();
-  rp.clear();
-}