]> git.proxmox.com Git - ceph.git/blobdiff - ceph/src/librbd/io/ImageRequest.cc
import 15.2.0 Octopus source
[ceph.git] / ceph / src / librbd / io / ImageRequest.cc
index d6eb29fe601bf377af3ce2b35db175a7aef14164..977182eacd1c53426c4bbaa1dd3164d8f8f89f56 100644 (file)
 
 #define dout_subsys ceph_subsys_rbd
 #undef dout_prefix
-#define dout_prefix *_dout << "librbd::io::ImageRequest: " << this \
-                           << " " << __func__ << ": "
+#define dout_prefix *_dout << "librbd::io::ImageRequest: " << __func__ << ": "
 
 namespace librbd {
 namespace io {
 
+using librbd::util::data_object_name;
 using librbd::util::get_image_ctx;
 
 namespace {
 
+template <typename I>
+struct C_RBD_Readahead : public Context {
+  I *ictx;
+  uint64_t object_no;
+  uint64_t offset;
+  uint64_t length;
+
+  bufferlist read_data;
+  io::ExtentMap extent_map;
+
+  C_RBD_Readahead(I *ictx, uint64_t object_no, uint64_t offset, uint64_t length)
+    : ictx(ictx), object_no(object_no), offset(offset), length(length) {
+    ictx->readahead.inc_pending();
+  }
+
+  void finish(int r) override {
+    ldout(ictx->cct, 20) << "C_RBD_Readahead on "
+                         << data_object_name(ictx, object_no) << ": "
+                         << offset << "~" << length << dendl;
+    ictx->readahead.dec_pending();
+  }
+};
+
+template <typename I>
+void readahead(I *ictx, const Extents& image_extents) {
+  uint64_t total_bytes = 0;
+  for (auto& image_extent : image_extents) {
+    total_bytes += image_extent.second;
+  }
+
+  ictx->image_lock.lock_shared();
+  auto total_bytes_read = ictx->total_bytes_read.fetch_add(total_bytes);
+  bool abort = (
+    ictx->readahead_disable_after_bytes != 0 &&
+    total_bytes_read > ictx->readahead_disable_after_bytes);
+  if (abort) {
+    ictx->image_lock.unlock_shared();
+    return;
+  }
+
+  uint64_t image_size = ictx->get_image_size(ictx->snap_id);
+  auto snap_id = ictx->snap_id;
+  ictx->image_lock.unlock_shared();
+
+  auto readahead_extent = ictx->readahead.update(image_extents, image_size);
+  uint64_t readahead_offset = readahead_extent.first;
+  uint64_t readahead_length = readahead_extent.second;
+
+  if (readahead_length > 0) {
+    ldout(ictx->cct, 20) << "(readahead logical) " << readahead_offset << "~"
+                         << readahead_length << dendl;
+    LightweightObjectExtents readahead_object_extents;
+    Striper::file_to_extents(ictx->cct, &ictx->layout,
+                             readahead_offset, readahead_length, 0, 0,
+                             &readahead_object_extents);
+    for (auto& object_extent : readahead_object_extents) {
+      ldout(ictx->cct, 20) << "(readahead) "
+                           << data_object_name(ictx,
+                                               object_extent.object_no) << " "
+                           << object_extent.offset << "~"
+                           << object_extent.length << dendl;
+
+      auto req_comp = new C_RBD_Readahead<I>(ictx, object_extent.object_no,
+                                             object_extent.offset,
+                                             object_extent.length);
+      auto req = io::ObjectDispatchSpec::create_read(
+        ictx, io::OBJECT_DISPATCH_LAYER_NONE, object_extent.object_no,
+        object_extent.offset, object_extent.length, snap_id, 0, {},
+        &req_comp->read_data, &req_comp->extent_map, req_comp);
+      req->send();
+    }
+
+    ictx->perfcounter->inc(l_librbd_readahead);
+    ictx->perfcounter->inc(l_librbd_readahead_bytes, readahead_length);
+  }
+}
+
 template <typename I>
 struct C_UpdateTimestamp : public Context {
 public:
@@ -76,6 +153,10 @@ bool should_update_timestamp(const utime_t& now, const utime_t& timestamp,
 
 } // anonymous namespace
 
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::io::ImageRequest: " << this \
+                           << " " << __func__ << ": "
+
 template <typename I>
 void ImageRequest<I>::aio_read(I *ictx, AioCompletion *c,
                                Extents &&image_extents,
@@ -138,7 +219,6 @@ void ImageRequest<I>::aio_compare_and_write(I *ictx, AioCompletion *c,
   req.send();
 }
 
-
 template <typename I>
 void ImageRequest<I>::send() {
   I &image_ctx = this->m_image_ctx;
@@ -150,13 +230,16 @@ void ImageRequest<I>::send() {
   ldout(cct, 20) << get_request_type() << ": ictx=" << &image_ctx << ", "
                  << "completion=" << aio_comp << dendl;
 
-  aio_comp->get();
   int r = clip_request();
   if (r < 0) {
     m_aio_comp->fail(r);
     return;
   }
 
+  if (finish_request_early()) {
+    return;
+  }
+
   if (m_bypass_image_cache || m_image_ctx.image_cache == nullptr) {
     update_timestamp();
     send_request();
@@ -167,7 +250,7 @@ void ImageRequest<I>::send() {
 
 template <typename I>
 int ImageRequest<I>::clip_request() {
-  RWLock::RLocker snap_locker(m_image_ctx.snap_lock);
+  std::shared_lock image_locker{m_image_ctx.image_lock};
   for (auto &image_extent : m_image_extents) {
     auto clip_len = image_extent.second;
     int r = clip_io(get_image_ctx(&m_image_ctx), image_extent.first, &clip_len);
@@ -206,7 +289,7 @@ void ImageRequest<I>::update_timestamp() {
 
   utime_t ts = ceph_clock_now();
   {
-    RWLock::RLocker timestamp_locker(m_image_ctx.timestamp_lock);
+    std::shared_lock timestamp_locker{m_image_ctx.timestamp_lock};
     if(!should_update_timestamp(ts, std::invoke(get_timestamp_fn, m_image_ctx),
                                 update_interval)) {
       return;
@@ -214,7 +297,7 @@ void ImageRequest<I>::update_timestamp() {
   }
 
   {
-    RWLock::WLocker timestamp_locker(m_image_ctx.timestamp_lock);
+    std::unique_lock timestamp_locker{m_image_ctx.timestamp_lock};
     bool update = should_update_timestamp(
       ts, std::invoke(get_timestamp_fn, m_image_ctx), update_interval);
     if (!update) {
@@ -270,55 +353,43 @@ void ImageReadRequest<I>::send_request() {
     readahead(get_image_ctx(&image_ctx), image_extents);
   }
 
-  AioCompletion *aio_comp = this->m_aio_comp;
   librados::snap_t snap_id;
-  map<object_t,vector<ObjectExtent> > object_extents;
   uint64_t buffer_ofs = 0;
   {
     // prevent image size from changing between computing clip and recording
     // pending async operation
-    RWLock::RLocker snap_locker(image_ctx.snap_lock);
+    std::shared_lock image_locker{image_ctx.image_lock};
     snap_id = image_ctx.snap_id;
+  }
 
-    // map image extents to object extents
-    for (auto &extent : image_extents) {
-      if (extent.second == 0) {
-        continue;
-      }
-
-      Striper::file_to_extents(cct, image_ctx.format_string, &image_ctx.layout,
-                               extent.first, extent.second, 0, object_extents,
-                               buffer_ofs);
-      buffer_ofs += extent.second;
+  // map image extents to object extents
+  LightweightObjectExtents object_extents;
+  for (auto &extent : image_extents) {
+    if (extent.second == 0) {
+      continue;
     }
-  }
 
-  // pre-calculate the expected number of read requests
-  uint32_t request_count = 0;
-  for (auto &object_extent : object_extents) {
-    request_count += object_extent.second.size();
+    Striper::file_to_extents(cct, &image_ctx.layout, extent.first,
+                             extent.second, 0, buffer_ofs, &object_extents);
+    buffer_ofs += extent.second;
   }
-  aio_comp->set_request_count(request_count);
 
   // issue the requests
-  for (auto &object_extent : object_extents) {
-    for (auto &extent : object_extent.second) {
-      ldout(cct, 20) << "oid " << extent.oid << " " << extent.offset << "~"
-                     << extent.length << " from " << extent.buffer_extents
-                     << dendl;
-
-      auto req_comp = new io::ReadResult::C_ObjectReadRequest(
-        aio_comp, extent.offset, extent.length,
-        std::move(extent.buffer_extents));
-      auto req = ObjectDispatchSpec::create_read(
-        &image_ctx, OBJECT_DISPATCH_LAYER_NONE, extent.oid.name,
-        extent.objectno, extent.offset, extent.length, snap_id, m_op_flags,
-        this->m_trace, &req_comp->bl, &req_comp->extent_map, req_comp);
-      req->send();
-    }
-  }
+  AioCompletion *aio_comp = this->m_aio_comp;
+  aio_comp->set_request_count(object_extents.size());
+  for (auto &oe : object_extents) {
+    ldout(cct, 20) << data_object_name(&image_ctx, oe.object_no) << " "
+                   << oe.offset << "~" << oe.length << " from "
+                   << oe.buffer_extents << dendl;
 
-  aio_comp->put();
+    auto req_comp = new io::ReadResult::C_ObjectReadRequest(
+      aio_comp, oe.offset, oe.length, std::move(oe.buffer_extents));
+    auto req = ObjectDispatchSpec::create_read(
+      &image_ctx, OBJECT_DISPATCH_LAYER_NONE, oe.object_no, oe.offset,
+      oe.length, snap_id, m_op_flags, this->m_trace, &req_comp->bl,
+      &req_comp->extent_map, req_comp);
+    req->send();
+  }
 
   image_ctx.perfcounter->inc(l_librbd_rd);
   image_ctx.perfcounter->inc(l_librbd_rd_bytes, buffer_ofs);
@@ -339,50 +410,66 @@ void ImageReadRequest<I>::send_image_cache_request() {
                                   req_comp);
 }
 
+template <typename I>
+bool AbstractImageWriteRequest<I>::finish_request_early() {
+  AioCompletion *aio_comp = this->m_aio_comp;
+  {
+    std::shared_lock image_locker{this->m_image_ctx.image_lock};
+    if (this->m_image_ctx.snap_id != CEPH_NOSNAP || this->m_image_ctx.read_only) {
+      aio_comp->fail(-EROFS);
+      return true;
+    }
+  }
+  uint64_t total_bytes = 0;
+  for (auto& image_extent : this->m_image_extents) {
+    total_bytes += image_extent.second;
+  }
+  if (total_bytes == 0) {
+    aio_comp->set_request_count(0);
+    return true;
+  }
+  return false;
+}
+
 template <typename I>
 void AbstractImageWriteRequest<I>::send_request() {
   I &image_ctx = this->m_image_ctx;
   CephContext *cct = image_ctx.cct;
 
-  RWLock::RLocker md_locker(image_ctx.md_lock);
-
   bool journaling = false;
 
   AioCompletion *aio_comp = this->m_aio_comp;
-  uint64_t clip_len = 0;
-  ObjectExtents object_extents;
   ::SnapContext snapc;
   {
     // prevent image size from changing between computing clip and recording
     // pending async operation
-    RWLock::RLocker snap_locker(image_ctx.snap_lock);
-    if (image_ctx.snap_id != CEPH_NOSNAP || image_ctx.read_only) {
-      aio_comp->fail(-EROFS);
-      return;
-    }
-
-    for (auto &extent : this->m_image_extents) {
-      if (extent.second == 0) {
-        continue;
-      }
-
-      // map to object extents
-      Striper::file_to_extents(cct, image_ctx.format_string, &image_ctx.layout,
-                               extent.first, extent.second, 0, object_extents);
-      clip_len += extent.second;
-    }
+    std::shared_lock image_locker{image_ctx.image_lock};
 
     snapc = image_ctx.snapc;
     journaling = (image_ctx.journal != nullptr &&
                   image_ctx.journal->is_journal_appending());
   }
 
+  uint64_t clip_len = 0;
+  LightweightObjectExtents object_extents;
+  for (auto &extent : this->m_image_extents) {
+    if (extent.second == 0) {
+      continue;
+    }
+
+    // map to object extents
+    Striper::file_to_extents(cct, &image_ctx.layout, extent.first,
+                             extent.second, 0, clip_len, &object_extents);
+    clip_len += extent.second;
+  }
+
   int ret = prune_object_extents(&object_extents);
   if (ret < 0) {
     aio_comp->fail(ret);
     return;
   }
 
+  aio_comp->set_request_count(object_extents.size());
   if (!object_extents.empty()) {
     uint64_t journal_tid = 0;
     if (journaling) {
@@ -391,42 +478,35 @@ void AbstractImageWriteRequest<I>::send_request() {
       journal_tid = append_journal_event(m_synchronous);
     }
 
-    aio_comp->set_request_count(object_extents.size());
     send_object_requests(object_extents, snapc, journal_tid);
-  } else {
-    // no IO to perform -- fire completion
-    aio_comp->set_request_count(0);
   }
 
   update_stats(clip_len);
-  aio_comp->put();
 }
 
 template <typename I>
 void AbstractImageWriteRequest<I>::send_object_requests(
-    const ObjectExtents &object_extents, const ::SnapContext &snapc,
+    const LightweightObjectExtents &object_extents, const ::SnapContext &snapc,
     uint64_t journal_tid) {
   I &image_ctx = this->m_image_ctx;
   CephContext *cct = image_ctx.cct;
 
   AioCompletion *aio_comp = this->m_aio_comp;
-  for (ObjectExtents::const_iterator p = object_extents.begin();
-       p != object_extents.end(); ++p) {
-    ldout(cct, 20) << "oid " << p->oid << " " << p->offset << "~" << p->length
-                   << " from " << p->buffer_extents << dendl;
+  bool single_extent = (object_extents.size() == 1);
+  for (auto& oe : object_extents) {
+    ldout(cct, 20) << data_object_name(&image_ctx, oe.object_no) << " "
+                   << oe.offset << "~" << oe.length << " from "
+                   << oe.buffer_extents << dendl;
     C_AioRequest *req_comp = new C_AioRequest(aio_comp);
-    auto request = create_object_request(*p, snapc, journal_tid, req_comp);
-
-    // if journaling, stash the request for later; otherwise send
-    if (request != NULL) {
-      request->send();
-    }
+    auto request = create_object_request(oe, snapc, journal_tid, single_extent,
+                                         req_comp);
+    request->send();
   }
 }
 
 template <typename I>
-void ImageWriteRequest<I>::assemble_extent(const ObjectExtent &object_extent,
-                                           bufferlist *bl) {
+void ImageWriteRequest<I>::assemble_extent(
+    const LightweightObjectExtent &object_extent, bufferlist *bl) {
   for (auto q = object_extent.buffer_extents.begin();
        q != object_extent.buffer_extents.end(); ++q) {
     bufferlist sub_bl;
@@ -468,16 +548,22 @@ void ImageWriteRequest<I>::send_image_cache_request() {
 
 template <typename I>
 ObjectDispatchSpec *ImageWriteRequest<I>::create_object_request(
-    const ObjectExtent &object_extent, const ::SnapContext &snapc,
-    uint64_t journal_tid, Context *on_finish) {
+    const LightweightObjectExtent &object_extent, const ::SnapContext &snapc,
+    uint64_t journal_tid, bool single_extent, Context *on_finish) {
   I &image_ctx = this->m_image_ctx;
 
   bufferlist bl;
-  assemble_extent(object_extent, &bl);
+  if (single_extent && object_extent.buffer_extents.size() == 1) {
+    // optimization for single object/buffer extent writes
+    bl = std::move(m_bl);
+  } else {
+    assemble_extent(object_extent, &bl);
+  }
+
   auto req = ObjectDispatchSpec::create_write(
-    &image_ctx, OBJECT_DISPATCH_LAYER_NONE, object_extent.oid.name,
-    object_extent.objectno, object_extent.offset, std::move(bl), snapc,
-    m_op_flags, journal_tid, this->m_trace, on_finish);
+    &image_ctx, OBJECT_DISPATCH_LAYER_NONE, object_extent.object_no,
+    object_extent.offset, std::move(bl), snapc, m_op_flags, journal_tid,
+    this->m_trace, on_finish);
   return req;
 }
 
@@ -524,12 +610,12 @@ void ImageDiscardRequest<I>::send_image_cache_request() {
 
 template <typename I>
 ObjectDispatchSpec *ImageDiscardRequest<I>::create_object_request(
-    const ObjectExtent &object_extent, const ::SnapContext &snapc,
-    uint64_t journal_tid, Context *on_finish) {
+    const LightweightObjectExtent &object_extent, const ::SnapContext &snapc,
+    uint64_t journal_tid, bool single_extent, Context *on_finish) {
   I &image_ctx = this->m_image_ctx;
   auto req = ObjectDispatchSpec::create_discard(
-    &image_ctx, OBJECT_DISPATCH_LAYER_NONE, object_extent.oid.name,
-    object_extent.objectno, object_extent.offset, object_extent.length, snapc,
+    &image_ctx, OBJECT_DISPATCH_LAYER_NONE, object_extent.object_no,
+    object_extent.offset, object_extent.length, snapc,
     OBJECT_DISCARD_FLAG_DISABLE_CLONE_REMOVE, journal_tid, this->m_trace,
     on_finish);
   return req;
@@ -544,7 +630,7 @@ void ImageDiscardRequest<I>::update_stats(size_t length) {
 
 template <typename I>
 int ImageDiscardRequest<I>::prune_object_extents(
-    ObjectExtents* object_extents) const {
+    LightweightObjectExtents* object_extents) const {
   if (m_discard_granularity_bytes == 0) {
     return 0;
   }
@@ -560,7 +646,7 @@ int ImageDiscardRequest<I>::prune_object_extents(
                                             object_size);
   auto xform_lambda =
     [discard_granularity_bytes, object_size, &prune_required]
-    (ObjectExtent& object_extent) {
+    (LightweightObjectExtent& object_extent) {
       auto& offset = object_extent.offset;
       auto& length = object_extent.length;
       auto next_offset = offset + length;
@@ -583,7 +669,7 @@ int ImageDiscardRequest<I>::prune_object_extents(
   if (prune_required) {
     // one or more object extents were skipped
     auto remove_lambda =
-      [](const ObjectExtent& object_extent) {
+      [](const LightweightObjectExtent& object_extent) {
         return (object_extent.length == 0);
       };
     object_extents->erase(
@@ -600,7 +686,7 @@ void ImageFlushRequest<I>::send_request() {
 
   bool journaling = false;
   {
-    RWLock::RLocker snap_locker(image_ctx.snap_lock);
+    std::shared_lock image_locker{image_ctx.image_lock};
     journaling = (m_flush_source == FLUSH_SOURCE_USER &&
                   image_ctx.journal != nullptr &&
                   image_ctx.journal->is_journal_appending());
@@ -614,34 +700,24 @@ void ImageFlushRequest<I>::send_request() {
   // ensure no locks are held when flush is complete
   ctx = librbd::util::create_async_context_callback(image_ctx, ctx);
 
+  uint64_t journal_tid = 0;
   if (journaling) {
     // in-flight ops are flushed prior to closing the journal
-    uint64_t journal_tid = image_ctx.journal->append_io_event(
+    ceph_assert(image_ctx.journal != NULL);
+    journal_tid = image_ctx.journal->append_io_event(
       journal::EventEntry(journal::AioFlushEvent()), 0, 0, false, 0);
     image_ctx.journal->user_flushed();
-
-    ctx = new FunctionContext(
-      [&image_ctx, journal_tid, ctx](int r) {
-        image_ctx.journal->commit_io_event(journal_tid, r);
-        ctx->complete(r);
-      });
-    ctx = new FunctionContext(
-      [&image_ctx, journal_tid, ctx](int r) {
-        image_ctx.journal->flush_event(journal_tid, ctx);
-      });
-  } else {
-    // flush rbd cache only when journaling is not enabled
-    auto object_dispatch_spec = ObjectDispatchSpec::create_flush(
-      &image_ctx, OBJECT_DISPATCH_LAYER_NONE, m_flush_source, this->m_trace,
-      ctx);
-    ctx = new FunctionContext([object_dispatch_spec](int r) {
-        object_dispatch_spec->send();
-      });
   }
 
+  auto object_dispatch_spec = ObjectDispatchSpec::create_flush(
+    &image_ctx, OBJECT_DISPATCH_LAYER_NONE, m_flush_source, journal_tid,
+    this->m_trace, ctx);
+  ctx = new LambdaContext([object_dispatch_spec](int r) {
+      object_dispatch_spec->send();
+    });
+
   // ensure all in-flight IOs are settled if non-user flush request
   aio_comp->async_op.flush(ctx);
-  aio_comp->put();
 
   // might be flushing during image shutdown
   if (image_ctx.perfcounter != nullptr) {
@@ -695,27 +771,27 @@ void ImageWriteSameRequest<I>::send_image_cache_request() {
 
 template <typename I>
 ObjectDispatchSpec *ImageWriteSameRequest<I>::create_object_request(
-    const ObjectExtent &object_extent, const ::SnapContext &snapc,
-    uint64_t journal_tid, Context *on_finish) {
+    const LightweightObjectExtent &object_extent, const ::SnapContext &snapc,
+    uint64_t journal_tid, bool single_extent, Context *on_finish) {
   I &image_ctx = this->m_image_ctx;
 
   bufferlist bl;
   ObjectDispatchSpec *req;
 
   if (util::assemble_write_same_extent(object_extent, m_data_bl, &bl, false)) {
-    Extents buffer_extents{object_extent.buffer_extents};
+    auto buffer_extents{object_extent.buffer_extents};
 
     req = ObjectDispatchSpec::create_write_same(
-      &image_ctx, OBJECT_DISPATCH_LAYER_NONE, object_extent.oid.name,
-      object_extent.objectno, object_extent.offset, object_extent.length,
-      std::move(buffer_extents), std::move(bl), snapc, m_op_flags, journal_tid,
+      &image_ctx, OBJECT_DISPATCH_LAYER_NONE, object_extent.object_no,
+      object_extent.offset, object_extent.length, std::move(buffer_extents),
+      std::move(bl), snapc, m_op_flags, journal_tid,
       this->m_trace, on_finish);
     return req;
   }
   req = ObjectDispatchSpec::create_write(
-    &image_ctx, OBJECT_DISPATCH_LAYER_NONE, object_extent.oid.name,
-    object_extent.objectno, object_extent.offset, std::move(bl), snapc,
-    m_op_flags, journal_tid, this->m_trace, on_finish);
+    &image_ctx, OBJECT_DISPATCH_LAYER_NONE, object_extent.object_no,
+    object_extent.offset, std::move(bl), snapc, m_op_flags, journal_tid,
+    this->m_trace, on_finish);
   return req;
 }
 
@@ -746,7 +822,7 @@ uint64_t ImageCompareAndWriteRequest<I>::append_journal_event(
 
 template <typename I>
 void ImageCompareAndWriteRequest<I>::assemble_extent(
-  const ObjectExtent &object_extent, bufferlist *bl) {
+  const LightweightObjectExtent &object_extent, bufferlist *bl) {
   for (auto q = object_extent.buffer_extents.begin();
        q != object_extent.buffer_extents.end(); ++q) {
     bufferlist sub_bl;
@@ -770,9 +846,8 @@ void ImageCompareAndWriteRequest<I>::send_image_cache_request() {
 
 template <typename I>
 ObjectDispatchSpec *ImageCompareAndWriteRequest<I>::create_object_request(
-    const ObjectExtent &object_extent,
-    const ::SnapContext &snapc,
-    uint64_t journal_tid, Context *on_finish) {
+    const LightweightObjectExtent &object_extent, const ::SnapContext &snapc,
+    uint64_t journal_tid, bool single_extent, Context *on_finish) {
   I &image_ctx = this->m_image_ctx;
 
   // NOTE: safe to move m_cmp_bl since we only support this op against
@@ -780,10 +855,9 @@ ObjectDispatchSpec *ImageCompareAndWriteRequest<I>::create_object_request(
   bufferlist bl;
   assemble_extent(object_extent, &bl);
   auto req = ObjectDispatchSpec::create_compare_and_write(
-    &image_ctx, OBJECT_DISPATCH_LAYER_NONE, object_extent.oid.name,
-    object_extent.objectno, object_extent.offset, std::move(m_cmp_bl),
-    std::move(bl), snapc, m_mismatch_offset, m_op_flags, journal_tid,
-    this->m_trace, on_finish);
+    &image_ctx, OBJECT_DISPATCH_LAYER_NONE, object_extent.object_no,
+    object_extent.offset, std::move(m_cmp_bl), std::move(bl), snapc,
+    m_mismatch_offset, m_op_flags, journal_tid, this->m_trace, on_finish);
   return req;
 }
 
@@ -796,14 +870,14 @@ void ImageCompareAndWriteRequest<I>::update_stats(size_t length) {
 
 template <typename I>
 int ImageCompareAndWriteRequest<I>::prune_object_extents(
-    ObjectExtents* object_extents) const {
+    LightweightObjectExtents* object_extents) const {
   if (object_extents->size() > 1)
     return -EINVAL;
 
   I &image_ctx = this->m_image_ctx;
   uint64_t sector_size = 512ULL;
   uint64_t su = image_ctx.layout.stripe_unit;
-  ObjectExtent object_extent = object_extents->front();
+  auto& object_extent = object_extents->front();
   if (object_extent.offset % sector_size + object_extent.length > sector_size ||
       (su != 0 && (object_extent.offset % su + object_extent.length > su)))
     return -EINVAL;