]> git.apps.os.sepia.ceph.com Git - ceph.git/commitdiff
librbd: move IO-related classes to new io namespace
authorJason Dillaman <dillaman@redhat.com>
Wed, 15 Feb 2017 20:38:39 +0000 (15:38 -0500)
committerJason Dillaman <dillaman@redhat.com>
Mon, 20 Feb 2017 13:19:18 +0000 (08:19 -0500)
Signed-off-by: Jason Dillaman <dillaman@redhat.com>
80 files changed:
src/librbd/AioCompletion.cc [deleted file]
src/librbd/AioCompletion.h [deleted file]
src/librbd/AioImageRequest.cc [deleted file]
src/librbd/AioImageRequest.h [deleted file]
src/librbd/AioImageRequestWQ.cc [deleted file]
src/librbd/AioImageRequestWQ.h [deleted file]
src/librbd/AioObjectRequest.cc [deleted file]
src/librbd/AioObjectRequest.h [deleted file]
src/librbd/CMakeLists.txt
src/librbd/CopyupRequest.cc [deleted file]
src/librbd/CopyupRequest.h [deleted file]
src/librbd/ExclusiveLock.cc
src/librbd/Group.cc
src/librbd/Group.h
src/librbd/ImageCtx.cc
src/librbd/ImageCtx.h
src/librbd/ImageWatcher.cc
src/librbd/Journal.cc
src/librbd/Journal.h
src/librbd/LibrbdWriteback.cc
src/librbd/cache/ImageWriteback.cc
src/librbd/exclusive_lock/PreReleaseRequest.cc
src/librbd/image/CloseRequest.cc
src/librbd/image/CloseRequest.h
src/librbd/image/CreateRequest.cc
src/librbd/image/RefreshRequest.cc
src/librbd/image/SetSnapRequest.cc
src/librbd/internal.cc
src/librbd/internal.h
src/librbd/io/AioCompletion.cc [new file with mode: 0644]
src/librbd/io/AioCompletion.h [new file with mode: 0644]
src/librbd/io/CopyupRequest.cc [new file with mode: 0644]
src/librbd/io/CopyupRequest.h [new file with mode: 0644]
src/librbd/io/ImageRequest.cc [new file with mode: 0644]
src/librbd/io/ImageRequest.h [new file with mode: 0644]
src/librbd/io/ImageRequestWQ.cc [new file with mode: 0644]
src/librbd/io/ImageRequestWQ.h [new file with mode: 0644]
src/librbd/io/ObjectRequest.cc [new file with mode: 0644]
src/librbd/io/ObjectRequest.h [new file with mode: 0644]
src/librbd/io/Types.h [new file with mode: 0644]
src/librbd/journal/Replay.cc
src/librbd/journal/Replay.h
src/librbd/librbd.cc
src/librbd/operation/DisableFeaturesRequest.cc
src/librbd/operation/EnableFeaturesRequest.cc
src/librbd/operation/FlattenRequest.cc
src/librbd/operation/ResizeRequest.cc
src/librbd/operation/SnapshotCreateRequest.cc
src/librbd/operation/SnapshotRollbackRequest.cc
src/librbd/operation/TrimRequest.cc
src/test/librbd/CMakeLists.txt
src/test/librbd/exclusive_lock/test_mock_PreReleaseRequest.cc
src/test/librbd/image/test_mock_RefreshRequest.cc
src/test/librbd/io/test_mock_ImageRequest.cc [new file with mode: 0644]
src/test/librbd/journal/test_Entries.cc
src/test/librbd/journal/test_Replay.cc
src/test/librbd/journal/test_mock_Replay.cc
src/test/librbd/mock/MockAioImageRequestWQ.h [deleted file]
src/test/librbd/mock/MockContextWQ.h
src/test/librbd/mock/MockImageCtx.h
src/test/librbd/mock/MockJournal.h
src/test/librbd/mock/io/MockImageRequestWQ.h [new file with mode: 0644]
src/test/librbd/operation/test_mock_DisableFeaturesRequest.cc
src/test/librbd/operation/test_mock_EnableFeaturesRequest.cc
src/test/librbd/operation/test_mock_ResizeRequest.cc
src/test/librbd/operation/test_mock_SnapshotCreateRequest.cc
src/test/librbd/operation/test_mock_SnapshotRollbackRequest.cc
src/test/librbd/test_ConsistencyGroups.cc [deleted file]
src/test/librbd/test_Groups.cc [new file with mode: 0644]
src/test/librbd/test_ImageWatcher.cc
src/test/librbd/test_fixture.cc
src/test/librbd/test_internal.cc
src/test/librbd/test_main.cc
src/test/librbd/test_mirroring.cc
src/test/librbd/test_mock_AioImageRequest.cc [deleted file]
src/test/librbd/test_mock_ExclusiveLock.cc
src/test/librbd/test_mock_Journal.cc
src/test/rbd_mirror/image_sync/test_mock_ObjectCopyRequest.cc
src/test/rbd_mirror/test_ImageReplayer.cc
src/test/rbd_mirror/test_ImageSync.cc

diff --git a/src/librbd/AioCompletion.cc b/src/librbd/AioCompletion.cc
deleted file mode 100644 (file)
index e42754b..0000000
+++ /dev/null
@@ -1,224 +0,0 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-
-#include <errno.h>
-
-#include "common/ceph_context.h"
-#include "common/dout.h"
-#include "common/errno.h"
-#include "common/perf_counters.h"
-#include "common/WorkQueue.h"
-
-#include "librbd/ImageCtx.h"
-#include "librbd/internal.h"
-
-#include "librbd/AioCompletion.h"
-#include "librbd/Journal.h"
-
-#ifdef WITH_LTTNG
-#include "tracing/librbd.h"
-#else
-#define tracepoint(...)
-#endif
-
-#define dout_subsys ceph_subsys_rbd
-#undef dout_prefix
-#define dout_prefix *_dout << "librbd::AioCompletion: "
-
-namespace librbd {
-
-int AioCompletion::wait_for_complete() {
-  tracepoint(librbd, aio_wait_for_complete_enter, this);
-  lock.Lock();
-  while (state != STATE_COMPLETE)
-    cond.Wait(lock);
-  lock.Unlock();
-  tracepoint(librbd, aio_wait_for_complete_exit, 0);
-  return 0;
-}
-
-void AioCompletion::finalize(ssize_t rval)
-{
-  assert(lock.is_locked());
-  assert(ictx != nullptr);
-  CephContext *cct = ictx->cct;
-
-  ldout(cct, 20) << this << " " << __func__ << ": r=" << rval << ", "
-                 << "read_buf=" << reinterpret_cast<void*>(read_buf) << ", "
-                 << "real_bl=" <<  reinterpret_cast<void*>(read_bl) << dendl;
-  if (rval >= 0 && aio_type == AIO_TYPE_READ) {
-    if (read_buf && !read_bl) {
-      destriper.assemble_result(cct, read_buf, read_buf_len);
-    } else {
-      // FIXME: make the destriper write directly into a buffer so
-      // that we avoid shuffling pointers and copying zeros around.
-      bufferlist bl;
-      destriper.assemble_result(cct, bl, true);
-
-      if (read_buf) {
-        assert(bl.length() == read_buf_len);
-        bl.copy(0, read_buf_len, read_buf);
-        ldout(cct, 20) << "copied resulting " << bl.length()
-                       << " bytes to " << (void*)read_buf << dendl;
-      }
-      if (read_bl) {
-        ldout(cct, 20) << " moving resulting " << bl.length()
-                       << " bytes to bl " << (void*)read_bl << dendl;
-        read_bl->claim(bl);
-      }
-    }
-  }
-}
-
-void AioCompletion::complete() {
-  assert(lock.is_locked());
-  assert(ictx != nullptr);
-  CephContext *cct = ictx->cct;
-
-  tracepoint(librbd, aio_complete_enter, this, rval);
-  utime_t elapsed;
-  elapsed = ceph_clock_now() - start_time;
-  switch (aio_type) {
-  case AIO_TYPE_OPEN:
-  case AIO_TYPE_CLOSE:
-    break;
-  case AIO_TYPE_READ:
-    ictx->perfcounter->tinc(l_librbd_rd_latency, elapsed); break;
-  case AIO_TYPE_WRITE:
-    ictx->perfcounter->tinc(l_librbd_wr_latency, elapsed); break;
-  case AIO_TYPE_DISCARD:
-    ictx->perfcounter->tinc(l_librbd_discard_latency, elapsed); break;
-  case AIO_TYPE_FLUSH:
-    ictx->perfcounter->tinc(l_librbd_aio_flush_latency, elapsed); break;
-  default:
-    lderr(cct) << "completed invalid aio_type: " << aio_type << dendl;
-    break;
-  }
-
-  // inform the journal that the op has successfully committed
-  if (journal_tid != 0) {
-    assert(ictx->journal != NULL);
-    ictx->journal->commit_io_event(journal_tid, rval);
-  }
-
-  state = STATE_CALLBACK;
-  if (complete_cb) {
-    lock.Unlock();
-    complete_cb(rbd_comp, complete_arg);
-    lock.Lock();
-  }
-
-  if (ictx && event_notify && ictx->event_socket.is_valid()) {
-    ictx->completed_reqs_lock.Lock();
-    ictx->completed_reqs.push_back(&m_xlist_item);
-    ictx->completed_reqs_lock.Unlock();
-    ictx->event_socket.notify();
-  }
-
-  state = STATE_COMPLETE;
-  cond.Signal();
-
-  // note: possible for image to be closed after op marked finished
-  if (async_op.started()) {
-    async_op.finish_op();
-  }
-  tracepoint(librbd, aio_complete_exit);
-}
-
-void AioCompletion::init_time(ImageCtx *i, aio_type_t t) {
-  Mutex::Locker locker(lock);
-  if (ictx == nullptr) {
-    ictx = i;
-    aio_type = t;
-    start_time = ceph_clock_now();
-  }
-}
-
-void AioCompletion::start_op(bool ignore_type) {
-  Mutex::Locker locker(lock);
-  assert(ictx != nullptr);
-  assert(!async_op.started());
-  if (state == STATE_PENDING && (ignore_type || aio_type != AIO_TYPE_FLUSH)) {
-    async_op.start_op(*ictx);
-  }
-}
-
-void AioCompletion::fail(int r)
-{
-  lock.Lock();
-  assert(ictx != nullptr);
-  CephContext *cct = ictx->cct;
-
-  lderr(cct) << this << " " << __func__ << ": " << cpp_strerror(r)
-             << dendl;
-  assert(pending_count == 0);
-  rval = r;
-  complete();
-  put_unlock();
-}
-
-void AioCompletion::set_request_count(uint32_t count) {
-  lock.Lock();
-  assert(ictx != nullptr);
-  CephContext *cct = ictx->cct;
-
-  ldout(cct, 20) << this << " " << __func__ << ": pending=" << count << dendl;
-  assert(pending_count == 0);
-  pending_count = count;
-  lock.Unlock();
-
-  // if no pending requests, completion will fire now
-  unblock();
-}
-
-void AioCompletion::complete_request(ssize_t r)
-{
-  lock.Lock();
-  assert(ictx != nullptr);
-  CephContext *cct = ictx->cct;
-
-  if (rval >= 0) {
-    if (r < 0 && r != -EEXIST)
-      rval = r;
-    else if (r > 0)
-      rval += r;
-  }
-  assert(pending_count);
-  int count = --pending_count;
-
-  ldout(cct, 20) << this << " " << __func__ << ": cb=" << complete_cb << ", "
-                 << "pending=" << pending_count << dendl;
-  if (!count && blockers == 0) {
-    finalize(rval);
-    complete();
-  }
-  put_unlock();
-}
-
-void AioCompletion::associate_journal_event(uint64_t tid) {
-  Mutex::Locker l(lock);
-  assert(state == STATE_PENDING);
-  journal_tid = tid;
-}
-
-bool AioCompletion::is_complete() {
-  tracepoint(librbd, aio_is_complete_enter, this);
-  bool done;
-  {
-    Mutex::Locker l(lock);
-    done = this->state == STATE_COMPLETE;
-  }
-  tracepoint(librbd, aio_is_complete_exit, done);
-  return done;
-}
-
-ssize_t AioCompletion::get_return_value() {
-  tracepoint(librbd, aio_get_return_value_enter, this);
-  lock.Lock();
-  ssize_t r = rval;
-  lock.Unlock();
-  tracepoint(librbd, aio_get_return_value_exit, r);
-  return r;
-}
-
-} // namespace librbd
diff --git a/src/librbd/AioCompletion.h b/src/librbd/AioCompletion.h
deleted file mode 100644 (file)
index 6a83677..0000000
+++ /dev/null
@@ -1,238 +0,0 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-
-#ifndef CEPH_LIBRBD_AIO_COMPLETION_H
-#define CEPH_LIBRBD_AIO_COMPLETION_H
-
-#include "common/Cond.h"
-#include "common/Mutex.h"
-#include "include/Context.h"
-#include "include/utime.h"
-#include "include/rbd/librbd.hpp"
-
-#include "librbd/AsyncOperation.h"
-#include "librbd/ImageCtx.h"
-
-#include "osdc/Striper.h"
-
-class CephContext;
-
-namespace librbd {
-
-typedef enum {
-  AIO_TYPE_NONE = 0,
-  AIO_TYPE_OPEN,
-  AIO_TYPE_CLOSE,
-  AIO_TYPE_READ,
-  AIO_TYPE_WRITE,
-  AIO_TYPE_DISCARD,
-  AIO_TYPE_FLUSH,
-} aio_type_t;
-
-typedef enum {
-  STATE_PENDING = 0,
-  STATE_CALLBACK,
-  STATE_COMPLETE,
-} aio_state_t;
-
-/**
- * AioCompletion is the overall completion for a single
- * rbd I/O request. It may be composed of many AioObjectRequests,
- * which each go to a single object.
- *
- * The retrying of individual requests is handled at a lower level,
- * so all AioCompletion cares about is the count of outstanding
- * requests. The number of expected individual requests should be
- * set initially using set_request_count() prior to issuing the
- * requests.  This ensures that the completion will not be completed
- * within the caller's thread of execution (instead via a librados
- * context or via a thread pool context for cache read hits).
- */
-struct AioCompletion {
-  mutable Mutex lock;
-  Cond cond;
-  aio_state_t state;
-  ssize_t rval;
-  callback_t complete_cb;
-  void *complete_arg;
-  rbd_completion_t rbd_comp;
-  uint32_t pending_count;   ///< number of requests
-  uint32_t blockers;
-  int ref;
-  bool released;
-  ImageCtx *ictx;
-  utime_t start_time;
-  aio_type_t aio_type;
-
-  Striper::StripedReadResult destriper;
-  bufferlist *read_bl;
-  char *read_buf;
-  size_t read_buf_len;
-
-  AsyncOperation async_op;
-
-  uint64_t journal_tid;
-  xlist<AioCompletion*>::item m_xlist_item;
-  bool event_notify;
-
-  template <typename T, void (T::*MF)(int)>
-  static void callback_adapter(completion_t cb, void *arg) {
-    AioCompletion *comp = reinterpret_cast<AioCompletion *>(cb);
-    T *t = reinterpret_cast<T *>(arg);
-    (t->*MF)(comp->get_return_value());
-    comp->release();
-  }
-
-  static AioCompletion *create(void *cb_arg, callback_t cb_complete,
-                               rbd_completion_t rbd_comp) {
-    AioCompletion *comp = new AioCompletion();
-    comp->set_complete_cb(cb_arg, cb_complete);
-    comp->rbd_comp = (rbd_comp != nullptr ? rbd_comp : comp);
-    return comp;
-  }
-
-  template <typename T, void (T::*MF)(int) = &T::complete>
-  static AioCompletion *create(T *obj) {
-    AioCompletion *comp = new AioCompletion();
-    comp->set_complete_cb(obj, &callback_adapter<T, MF>);
-    comp->rbd_comp = comp;
-    return comp;
-  }
-
-  template <typename T, void (T::*MF)(int) = &T::complete>
-  static AioCompletion *create_and_start(T *obj, ImageCtx *image_ctx,
-                                         aio_type_t type) {
-    AioCompletion *comp = create<T, MF>(obj);
-    comp->init_time(image_ctx, type);
-    comp->start_op();
-    return comp;
-  }
-
-  AioCompletion() : lock("AioCompletion::lock", true, false),
-                    state(STATE_PENDING), rval(0), complete_cb(NULL),
-                    complete_arg(NULL), rbd_comp(NULL),
-                    pending_count(0), blockers(1),
-                    ref(1), released(false), ictx(NULL),
-                    aio_type(AIO_TYPE_NONE),
-                    read_bl(NULL), read_buf(NULL), read_buf_len(0),
-                    journal_tid(0), m_xlist_item(this), event_notify(false) {
-  }
-
-  ~AioCompletion() {
-  }
-
-  int wait_for_complete();
-
-  void finalize(ssize_t rval);
-
-  inline bool is_initialized(aio_type_t type) const {
-    Mutex::Locker locker(lock);
-    return ((ictx != nullptr) && (aio_type == type));
-  }
-  inline bool is_started() const {
-    Mutex::Locker locker(lock);
-    return async_op.started();
-  }
-
-  void init_time(ImageCtx *i, aio_type_t t);
-  void start_op(bool ignore_type = false);
-  void fail(int r);
-
-  void complete();
-
-  void set_complete_cb(void *cb_arg, callback_t cb) {
-    complete_cb = cb;
-    complete_arg = cb_arg;
-  }
-
-  void set_request_count(uint32_t num);
-  void add_request() {
-    lock.Lock();
-    assert(pending_count > 0);
-    lock.Unlock();
-    get();
-  }
-  void complete_request(ssize_t r);
-
-  void associate_journal_event(uint64_t tid);
-
-  bool is_complete();
-
-  ssize_t get_return_value();
-
-  void get() {
-    lock.Lock();
-    assert(ref > 0);
-    ref++;
-    lock.Unlock();
-  }
-  void release() {
-    lock.Lock();
-    assert(!released);
-    released = true;
-    put_unlock();
-  }
-  void put() {
-    lock.Lock();
-    put_unlock();
-  }
-  void put_unlock() {
-    assert(ref > 0);
-    int n = --ref;
-    lock.Unlock();
-    if (!n) {
-      if (ictx) {
-        if (event_notify) {
-          ictx->completed_reqs_lock.Lock();
-          m_xlist_item.remove_myself();
-          ictx->completed_reqs_lock.Unlock();
-        }
-        if (aio_type == AIO_TYPE_CLOSE ||
-            (aio_type == AIO_TYPE_OPEN && rval < 0)) {
-          delete ictx;
-        }
-      }
-      delete this;
-    }
-  }
-
-  void block() {
-    Mutex::Locker l(lock);
-    ++blockers;
-  }
-  void unblock() {
-    Mutex::Locker l(lock);
-    assert(blockers > 0);
-    --blockers;
-    if (pending_count == 0 && blockers == 0) {
-      finalize(rval);
-      complete();
-    }
-  }
-
-  void set_event_notify(bool s) {
-    Mutex::Locker l(lock);
-    event_notify = s;
-  }
-
-  void *get_arg() {
-    return complete_arg;
-  }
-};
-
-class C_AioRequest : public Context {
-public:
-  C_AioRequest(AioCompletion *completion) : m_completion(completion) {
-    m_completion->add_request();
-  }
-  virtual ~C_AioRequest() {}
-  virtual void finish(int r) {
-    m_completion->complete_request(r);
-  }
-protected:
-  AioCompletion *m_completion;
-};
-
-} // namespace librbd
-
-#endif // CEPH_LIBRBD_AIO_COMPLETION_H
diff --git a/src/librbd/AioImageRequest.cc b/src/librbd/AioImageRequest.cc
deleted file mode 100644 (file)
index a262d8c..0000000
+++ /dev/null
@@ -1,733 +0,0 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-
-#include "librbd/AioImageRequest.h"
-#include "librbd/AioCompletion.h"
-#include "librbd/AioObjectRequest.h"
-#include "librbd/ImageCtx.h"
-#include "librbd/internal.h"
-#include "librbd/Journal.h"
-#include "librbd/Utils.h"
-#include "librbd/cache/ImageCache.h"
-#include "librbd/journal/Types.h"
-#include "include/rados/librados.hpp"
-#include "common/WorkQueue.h"
-#include "osdc/Striper.h"
-
-#define dout_subsys ceph_subsys_rbd
-#undef dout_prefix
-#define dout_prefix *_dout << "librbd::AioImageRequest: "
-
-namespace librbd {
-
-using util::get_image_ctx;
-
-namespace {
-
-template <typename ImageCtxT = ImageCtx>
-struct C_DiscardJournalCommit : public Context {
-  typedef std::vector<ObjectExtent> ObjectExtents;
-
-  ImageCtxT &image_ctx;
-  AioCompletion *aio_comp;
-  ObjectExtents object_extents;
-
-  C_DiscardJournalCommit(ImageCtxT &_image_ctx, AioCompletion *_aio_comp,
-                         const ObjectExtents &_object_extents, uint64_t tid)
-    : image_ctx(_image_ctx), aio_comp(_aio_comp),
-      object_extents(_object_extents) {
-    CephContext *cct = image_ctx.cct;
-    ldout(cct, 20) << this << " C_DiscardJournalCommit: "
-                   << "delaying cache discard until journal tid " << tid << " "
-                   << "safe" << dendl;
-
-    aio_comp->add_request();
-  }
-
-  void finish(int r) override {
-    CephContext *cct = image_ctx.cct;
-    ldout(cct, 20) << this << " C_DiscardJournalCommit: "
-                   << "journal committed: discarding from cache" << dendl;
-
-    Mutex::Locker cache_locker(image_ctx.cache_lock);
-    image_ctx.object_cacher->discard_set(image_ctx.object_set, object_extents);
-    aio_comp->complete_request(r);
-  }
-};
-
-template <typename ImageCtxT = ImageCtx>
-struct C_FlushJournalCommit : public Context {
-  ImageCtxT &image_ctx;
-  AioCompletion *aio_comp;
-
-  C_FlushJournalCommit(ImageCtxT &_image_ctx, AioCompletion *_aio_comp,
-                       uint64_t tid)
-    : image_ctx(_image_ctx), aio_comp(_aio_comp) {
-    CephContext *cct = image_ctx.cct;
-    ldout(cct, 20) << this << " C_FlushJournalCommit: "
-                   << "delaying flush until journal tid " << tid << " "
-                   << "safe" << dendl;
-
-    aio_comp->add_request();
-  }
-
-  void finish(int r) override {
-    CephContext *cct = image_ctx.cct;
-    ldout(cct, 20) << this << " C_FlushJournalCommit: journal committed"
-                   << dendl;
-    aio_comp->complete_request(r);
-  }
-};
-
-template <typename ImageCtxT>
-class C_AioRead : public C_AioRequest {
-public:
-  C_AioRead(AioCompletion *completion)
-    : C_AioRequest(completion), m_req(nullptr) {
-  }
-
-  void finish(int r) override {
-    m_completion->lock.Lock();
-    CephContext *cct = m_completion->ictx->cct;
-    ldout(cct, 10) << "C_AioRead::finish() " << this << " r = " << r << dendl;
-
-    if (r >= 0 || r == -ENOENT) { // this was a sparse_read operation
-      ldout(cct, 10) << " got " << m_req->get_extent_map()
-                     << " for " << m_req->get_buffer_extents()
-                     << " bl " << m_req->data().length() << dendl;
-      // reads from the parent don't populate the m_ext_map and the overlap
-      // may not be the full buffer.  compensate here by filling in m_ext_map
-      // with the read extent when it is empty.
-      if (m_req->get_extent_map().empty()) {
-        m_req->get_extent_map()[m_req->get_offset()] = m_req->data().length();
-      }
-
-      m_completion->destriper.add_partial_sparse_result(
-          cct, m_req->data(), m_req->get_extent_map(), m_req->get_offset(),
-          m_req->get_buffer_extents());
-      r = m_req->get_length();
-    }
-    m_completion->lock.Unlock();
-
-    C_AioRequest::finish(r);
-  }
-
-  void set_req(AioObjectRead<ImageCtxT> *req) {
-    m_req = req;
-  }
-private:
-  AioObjectRead<ImageCtxT> *m_req;
-};
-
-template <typename ImageCtxT>
-class C_ImageCacheRead : public C_AioRequest {
-public:
-  typedef std::vector<std::pair<uint64_t,uint64_t> > Extents;
-
-  C_ImageCacheRead(AioCompletion *completion, const Extents &image_extents)
-    : C_AioRequest(completion), m_image_extents(image_extents) {
-  }
-
-  inline bufferlist &get_data() {
-    return m_bl;
-  }
-
-protected:
-  void finish(int r) override {
-    CephContext *cct = m_completion->ictx->cct;
-    ldout(cct, 10) << "C_ImageCacheRead::finish() " << this << ": r=" << r
-                   << dendl;
-    if (r >= 0) {
-      size_t length = 0;
-      for (auto &image_extent : m_image_extents) {
-        length += image_extent.second;
-      }
-      assert(length == m_bl.length());
-
-      m_completion->lock.Lock();
-      m_completion->destriper.add_partial_result(cct, m_bl, m_image_extents);
-      m_completion->lock.Unlock();
-      r = length;
-    }
-    C_AioRequest::finish(r);
-  }
-
-private:
-  bufferlist m_bl;
-  Extents m_image_extents;
-};
-
-template <typename ImageCtxT>
-class C_ObjectCacheRead : public Context {
-public:
-  explicit C_ObjectCacheRead(ImageCtxT &ictx, AioObjectRead<ImageCtxT> *req)
-    : m_image_ctx(ictx), m_req(req), m_enqueued(false) {}
-
-  void complete(int r) override {
-    if (!m_enqueued) {
-      // cache_lock creates a lock ordering issue -- so re-execute this context
-      // outside the cache_lock
-      m_enqueued = true;
-      m_image_ctx.op_work_queue->queue(this, r);
-      return;
-    }
-    Context::complete(r);
-  }
-
-protected:
-  void finish(int r) override {
-    m_req->complete(r);
-  }
-
-private:
-  ImageCtxT &m_image_ctx;
-  AioObjectRead<ImageCtxT> *m_req;
-  bool m_enqueued;
-};
-
-} // anonymous namespace
-
-template <typename I>
-void AioImageRequest<I>::aio_read(I *ictx, AioCompletion *c,
-                                  Extents &&image_extents, char *buf,
-                                  bufferlist *pbl, int op_flags) {
-  AioImageRead<I> req(*ictx, c, std::move(image_extents), buf, pbl, op_flags);
-  req.send();
-}
-
-template <typename I>
-void AioImageRequest<I>::aio_write(I *ictx, AioCompletion *c, uint64_t off,
-                                   size_t len, const char *buf, int op_flags) {
-  AioImageWrite<I> req(*ictx, c, off, len, buf, op_flags);
-  req.send();
-}
-
-template <typename I>
-void AioImageRequest<I>::aio_write(I *ictx, AioCompletion *c,
-                                   Extents &&image_extents, bufferlist &&bl,
-                                   int op_flags) {
-  AioImageWrite<I> req(*ictx, c, std::move(image_extents), std::move(bl),
-                       op_flags);
-  req.send();
-}
-
-template <typename I>
-void AioImageRequest<I>::aio_discard(I *ictx, AioCompletion *c,
-                                     uint64_t off, uint64_t len) {
-  AioImageDiscard<I> req(*ictx, c, off, len);
-  req.send();
-}
-
-template <typename I>
-void AioImageRequest<I>::aio_flush(I *ictx, AioCompletion *c) {
-  AioImageFlush<I> req(*ictx, c);
-  req.send();
-}
-
-template <typename I>
-void AioImageRequest<I>::send() {
-  I &image_ctx = this->m_image_ctx;
-  assert(m_aio_comp->is_initialized(get_aio_type()));
-  assert(m_aio_comp->is_started() ^ (get_aio_type() == AIO_TYPE_FLUSH));
-
-  CephContext *cct = image_ctx.cct;
-  AioCompletion *aio_comp = this->m_aio_comp;
-  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 (m_bypass_image_cache || m_image_ctx.image_cache == nullptr) {
-    send_request();
-  } else {
-    send_image_cache_request();
-  }
-}
-
-template <typename I>
-int AioImageRequest<I>::clip_request() {
-  RWLock::RLocker snap_locker(m_image_ctx.snap_lock);
-  for (auto &image_extent : m_image_extents) {
-    size_t clip_len = image_extent.second;
-    int r = clip_io(get_image_ctx(&m_image_ctx), image_extent.first, &clip_len);
-    if (r < 0) {
-      return r;
-    }
-
-    image_extent.second = clip_len;
-  }
-  return 0;
-}
-
-template <typename I>
-void AioImageRequest<I>::fail(int r) {
-  AioCompletion *aio_comp = this->m_aio_comp;
-  aio_comp->get();
-  aio_comp->fail(r);
-}
-
-template <typename I>
-void AioImageRead<I>::send_request() {
-  I &image_ctx = this->m_image_ctx;
-  CephContext *cct = image_ctx.cct;
-
-  auto &image_extents = this->m_image_extents;
-  if (image_ctx.object_cacher && image_ctx.readahead_max_bytes > 0 &&
-      !(m_op_flags & LIBRADOS_OP_FLAG_FADVISE_RANDOM)) {
-    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);
-    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;
-    }
-  }
-
-  aio_comp->read_buf = m_buf;
-  aio_comp->read_buf_len = buffer_ofs;
-  aio_comp->read_bl = m_pbl;
-
-  // 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();
-  }
-  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;
-
-      C_AioRead<I> *req_comp = new C_AioRead<I>(aio_comp);
-      AioObjectRead<I> *req = AioObjectRead<I>::create(
-        &image_ctx, extent.oid.name, extent.objectno, extent.offset,
-        extent.length, extent.buffer_extents, snap_id, true, req_comp,
-        m_op_flags);
-      req_comp->set_req(req);
-
-      if (image_ctx.object_cacher) {
-        C_ObjectCacheRead<I> *cache_comp = new C_ObjectCacheRead<I>(image_ctx,
-                                                                    req);
-        image_ctx.aio_read_from_cache(extent.oid, extent.objectno,
-                                      &req->data(), extent.length,
-                                      extent.offset, cache_comp, m_op_flags);
-      } else {
-        req->send();
-      }
-    }
-  }
-
-  aio_comp->put();
-
-  image_ctx.perfcounter->inc(l_librbd_rd);
-  image_ctx.perfcounter->inc(l_librbd_rd_bytes, buffer_ofs);
-}
-
-template <typename I>
-void AioImageRead<I>::send_image_cache_request() {
-  I &image_ctx = this->m_image_ctx;
-  assert(image_ctx.image_cache != nullptr);
-
-  AioCompletion *aio_comp = this->m_aio_comp;
-  aio_comp->set_request_count(1);
-  C_ImageCacheRead<I> *req_comp = new C_ImageCacheRead<I>(
-    aio_comp, this->m_image_extents);
-  image_ctx.image_cache->aio_read(std::move(this->m_image_extents),
-                                  &req_comp->get_data(), m_op_flags,
-                                  req_comp);
-}
-
-template <typename I>
-void AbstractAioImageWrite<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;
-    }
-
-    snapc = image_ctx.snapc;
-    journaling = (image_ctx.journal != nullptr &&
-                  image_ctx.journal->is_journal_appending());
-  }
-
-  prune_object_extents(object_extents);
-
-  if (!object_extents.empty()) {
-    uint64_t journal_tid = 0;
-    aio_comp->set_request_count(
-      object_extents.size() + get_object_cache_request_count(journaling));
-
-    AioObjectRequests requests;
-    send_object_requests(object_extents, snapc,
-                         (journaling ? &requests : nullptr));
-
-    if (journaling) {
-      // in-flight ops are flushed prior to closing the journal
-      assert(image_ctx.journal != NULL);
-      journal_tid = append_journal_event(requests, m_synchronous);
-    }
-
-    if (image_ctx.object_cacher != NULL) {
-      send_object_cache_requests(object_extents, journal_tid);
-    }
-  } else {
-    // no IO to perform -- fire completion
-    aio_comp->unblock();
-  }
-
-  update_stats(clip_len);
-  aio_comp->put();
-}
-
-template <typename I>
-void AbstractAioImageWrite<I>::send_object_requests(
-    const ObjectExtents &object_extents, const ::SnapContext &snapc,
-    AioObjectRequests *aio_object_requests) {
-  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;
-    C_AioRequest *req_comp = new C_AioRequest(aio_comp);
-    AioObjectRequestHandle *request = create_object_request(*p, snapc,
-                                                            req_comp);
-
-    // if journaling, stash the request for later; otherwise send
-    if (request != NULL) {
-      if (aio_object_requests != NULL) {
-        aio_object_requests->push_back(request);
-      } else {
-        request->send();
-      }
-    }
-  }
-}
-
-template <typename I>
-void AioImageWrite<I>::assemble_extent(const ObjectExtent &object_extent,
-                                    bufferlist *bl) {
-  for (auto q = object_extent.buffer_extents.begin();
-       q != object_extent.buffer_extents.end(); ++q) {
-    bufferlist sub_bl;
-    sub_bl.substr_of(m_bl, q->first, q->second);
-    bl->claim_append(sub_bl);
-  }
-}
-
-template <typename I>
-uint64_t AioImageWrite<I>::append_journal_event(
-    const AioObjectRequests &requests, bool synchronous) {
-  I &image_ctx = this->m_image_ctx;
-
-  uint64_t tid = 0;
-  uint64_t buffer_offset = 0;
-  assert(!this->m_image_extents.empty());
-  for (auto &extent : this->m_image_extents) {
-    bufferlist sub_bl;
-    sub_bl.substr_of(m_bl, buffer_offset, extent.second);
-    buffer_offset += extent.second;
-
-    tid = image_ctx.journal->append_write_event(extent.first, extent.second,
-                                                sub_bl, requests, synchronous);
-  }
-
-  if (image_ctx.object_cacher == NULL) {
-    AioCompletion *aio_comp = this->m_aio_comp;
-    aio_comp->associate_journal_event(tid);
-  }
-  return tid;
-}
-
-template <typename I>
-void AioImageWrite<I>::send_image_cache_request() {
-  I &image_ctx = this->m_image_ctx;
-  assert(image_ctx.image_cache != nullptr);
-
-  AioCompletion *aio_comp = this->m_aio_comp;
-  aio_comp->set_request_count(1);
-  C_AioRequest *req_comp = new C_AioRequest(aio_comp);
-  image_ctx.image_cache->aio_write(std::move(this->m_image_extents),
-                                   std::move(m_bl), m_op_flags, req_comp);
-}
-
-template <typename I>
-void AioImageWrite<I>::send_object_cache_requests(const ObjectExtents &object_extents,
-                                                  uint64_t journal_tid) {
-  I &image_ctx = this->m_image_ctx;
-  for (auto p = object_extents.begin(); p != object_extents.end(); ++p) {
-    const ObjectExtent &object_extent = *p;
-
-    bufferlist bl;
-    assemble_extent(object_extent, &bl);
-
-    AioCompletion *aio_comp = this->m_aio_comp;
-    C_AioRequest *req_comp = new C_AioRequest(aio_comp);
-    image_ctx.write_to_cache(object_extent.oid, bl, object_extent.length,
-                             object_extent.offset, req_comp, m_op_flags,
-                               journal_tid);
-  }
-}
-
-template <typename I>
-void AioImageWrite<I>::send_object_requests(
-    const ObjectExtents &object_extents, const ::SnapContext &snapc,
-    AioObjectRequests *aio_object_requests) {
-  I &image_ctx = this->m_image_ctx;
-
-  // cache handles creating object requests during writeback
-  if (image_ctx.object_cacher == NULL) {
-    AbstractAioImageWrite<I>::send_object_requests(object_extents, snapc,
-                                                aio_object_requests);
-  }
-}
-
-template <typename I>
-AioObjectRequestHandle *AioImageWrite<I>::create_object_request(
-    const ObjectExtent &object_extent, const ::SnapContext &snapc,
-    Context *on_finish) {
-  I &image_ctx = this->m_image_ctx;
-  assert(image_ctx.object_cacher == NULL);
-
-  bufferlist bl;
-  assemble_extent(object_extent, &bl);
-  AioObjectRequest<I> *req = AioObjectRequest<I>::create_write(
-    &image_ctx, object_extent.oid.name, object_extent.objectno,
-    object_extent.offset, bl, snapc, on_finish, m_op_flags);
-  return req;
-}
-
-template <typename I>
-void AioImageWrite<I>::update_stats(size_t length) {
-  I &image_ctx = this->m_image_ctx;
-  image_ctx.perfcounter->inc(l_librbd_wr);
-  image_ctx.perfcounter->inc(l_librbd_wr_bytes, length);
-}
-
-template <typename I>
-uint64_t AioImageDiscard<I>::append_journal_event(
-    const AioObjectRequests &requests, bool synchronous) {
-  I &image_ctx = this->m_image_ctx;
-
-  uint64_t tid = 0;
-  assert(!this->m_image_extents.empty());
-  for (auto &extent : this->m_image_extents) {
-    journal::EventEntry event_entry(journal::AioDiscardEvent(extent.first,
-                                                             extent.second));
-    tid = image_ctx.journal->append_io_event(std::move(event_entry),
-                                             requests, extent.first,
-                                             extent.second, synchronous);
-  }
-
-  AioCompletion *aio_comp = this->m_aio_comp;
-  aio_comp->associate_journal_event(tid);
-  return tid;
-}
-
-template <typename I>
-void AioImageDiscard<I>::prune_object_extents(ObjectExtents &object_extents) {
-  I &image_ctx = this->m_image_ctx;
-  CephContext *cct = image_ctx.cct;
-  if (!cct->_conf->rbd_skip_partial_discard) {
-    return;
-  }
-
-  for (auto p = object_extents.begin(); p != object_extents.end(); ) {
-    if (p->offset + p->length < image_ctx.layout.object_size) {
-      ldout(cct, 20) << " oid " << p->oid << " " << p->offset << "~"
-                    << p->length << " from " << p->buffer_extents
-                    << ": skip partial discard" << dendl;
-      p = object_extents.erase(p);
-    } else {
-      ++p;
-    }
-  }
-}
-
-template <typename I>
-uint32_t AioImageDiscard<I>::get_object_cache_request_count(bool journaling) const {
-  // extra completion request is required for tracking journal commit
-  I &image_ctx = this->m_image_ctx;
-  return (image_ctx.object_cacher != nullptr && journaling ? 1 : 0);
-}
-
-template <typename I>
-void AioImageDiscard<I>::send_image_cache_request() {
-  I &image_ctx = this->m_image_ctx;
-  assert(image_ctx.image_cache != nullptr);
-
-  AioCompletion *aio_comp = this->m_aio_comp;
-  aio_comp->set_request_count(this->m_image_extents.size());
-  for (auto &extent : this->m_image_extents) {
-    C_AioRequest *req_comp = new C_AioRequest(aio_comp);
-    image_ctx.image_cache->aio_discard(extent.first, extent.second, req_comp);
-  }
-}
-
-template <typename I>
-void AioImageDiscard<I>::send_object_cache_requests(const ObjectExtents &object_extents,
-                                                    uint64_t journal_tid) {
-  I &image_ctx = this->m_image_ctx;
-  if (journal_tid == 0) {
-    Mutex::Locker cache_locker(image_ctx.cache_lock);
-    image_ctx.object_cacher->discard_set(image_ctx.object_set,
-                                         object_extents);
-  } else {
-    // cannot discard from cache until journal has committed
-    assert(image_ctx.journal != NULL);
-    AioCompletion *aio_comp = this->m_aio_comp;
-    image_ctx.journal->wait_event(
-      journal_tid, new C_DiscardJournalCommit<I>(image_ctx, aio_comp,
-                                                 object_extents, journal_tid));
-  }
-}
-
-template <typename I>
-AioObjectRequestHandle *AioImageDiscard<I>::create_object_request(
-    const ObjectExtent &object_extent, const ::SnapContext &snapc,
-    Context *on_finish) {
-  I &image_ctx = this->m_image_ctx;
-
-  AioObjectRequest<I> *req;
-  if (object_extent.length == image_ctx.layout.object_size) {
-    req = AioObjectRequest<I>::create_remove(
-      &image_ctx, object_extent.oid.name, object_extent.objectno, snapc,
-      on_finish);
-  } else if (object_extent.offset + object_extent.length ==
-               image_ctx.layout.object_size) {
-    req = AioObjectRequest<I>::create_truncate(
-      &image_ctx, object_extent.oid.name, object_extent.objectno,
-      object_extent.offset, snapc, on_finish);
-  } else {
-    req = AioObjectRequest<I>::create_zero(
-      &image_ctx, object_extent.oid.name, object_extent.objectno,
-      object_extent.offset, object_extent.length, snapc, on_finish);
-  }
-  return req;
-}
-
-template <typename I>
-void AioImageDiscard<I>::update_stats(size_t length) {
-  I &image_ctx = this->m_image_ctx;
-  image_ctx.perfcounter->inc(l_librbd_discard);
-  image_ctx.perfcounter->inc(l_librbd_discard_bytes, length);
-}
-
-template <typename I>
-void AioImageFlush<I>::send_request() {
-  I &image_ctx = this->m_image_ctx;
-  image_ctx.user_flushed();
-
-  bool journaling = false;
-  {
-    RWLock::RLocker snap_locker(image_ctx.snap_lock);
-    journaling = (image_ctx.journal != nullptr &&
-                  image_ctx.journal->is_journal_appending());
-  }
-
-  AioCompletion *aio_comp = this->m_aio_comp;
-  if (journaling) {
-    // in-flight ops are flushed prior to closing the journal
-    uint64_t journal_tid = image_ctx.journal->append_io_event(
-      journal::EventEntry(journal::AioFlushEvent()),
-      AioObjectRequests(), 0, 0, false);
-
-    aio_comp->set_request_count(1);
-    aio_comp->associate_journal_event(journal_tid);
-
-    FunctionContext *flush_ctx = new FunctionContext(
-      [aio_comp, &image_ctx, journal_tid] (int r) {
-        C_FlushJournalCommit<I> *ctx = new C_FlushJournalCommit<I>(image_ctx,
-                                                                 aio_comp,
-                                                                 journal_tid);
-        image_ctx.journal->flush_event(journal_tid, ctx);
-
-        // track flush op for block writes
-        aio_comp->start_op(true);
-        aio_comp->put();
-    });
-
-    image_ctx.flush_async_operations(flush_ctx);
-  } else {
-    // flush rbd cache only when journaling is not enabled
-    aio_comp->set_request_count(1);
-    C_AioRequest *req_comp = new C_AioRequest(aio_comp);
-    image_ctx.flush(req_comp);
-
-    aio_comp->start_op(true);
-    aio_comp->put();
-  }
-
-  image_ctx.perfcounter->inc(l_librbd_aio_flush);
-}
-
-template <typename I>
-void AioImageFlush<I>::send_image_cache_request() {
-  I &image_ctx = this->m_image_ctx;
-  assert(image_ctx.image_cache != nullptr);
-
-  AioCompletion *aio_comp = this->m_aio_comp;
-  aio_comp->set_request_count(1);
-  C_AioRequest *req_comp = new C_AioRequest(aio_comp);
-  image_ctx.image_cache->aio_flush(req_comp);
-}
-
-} // namespace librbd
-
-template class librbd::AioImageRequest<librbd::ImageCtx>;
-template class librbd::AbstractAioImageWrite<librbd::ImageCtx>;
-template class librbd::AioImageWrite<librbd::ImageCtx>;
-template class librbd::AioImageDiscard<librbd::ImageCtx>;
-template class librbd::AioImageFlush<librbd::ImageCtx>;
diff --git a/src/librbd/AioImageRequest.h b/src/librbd/AioImageRequest.h
deleted file mode 100644 (file)
index 18a2c9f..0000000
+++ /dev/null
@@ -1,276 +0,0 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-
-#ifndef CEPH_LIBRBD_AIO_IMAGE_REQUEST_H
-#define CEPH_LIBRBD_AIO_IMAGE_REQUEST_H
-
-#include "include/int_types.h"
-#include "include/buffer_fwd.h"
-#include "common/snap_types.h"
-#include "osd/osd_types.h"
-#include "librbd/AioCompletion.h"
-#include <list>
-#include <utility>
-#include <vector>
-
-namespace librbd {
-
-class AioCompletion;
-class AioObjectRequestHandle;
-class ImageCtx;
-
-template <typename ImageCtxT = ImageCtx>
-class AioImageRequest {
-public:
-  typedef std::vector<std::pair<uint64_t,uint64_t> > Extents;
-
-  virtual ~AioImageRequest() {}
-
-  static void aio_read(ImageCtxT *ictx, AioCompletion *c,
-                       Extents &&image_extents, char *buf, bufferlist *pbl,
-                       int op_flags);
-  static void aio_write(ImageCtxT *ictx, AioCompletion *c, uint64_t off,
-                        size_t len, const char *buf, int op_flags);
-  static void aio_write(ImageCtxT *ictx, AioCompletion *c,
-                        Extents &&image_extents, bufferlist &&bl, int op_flags);
-  static void aio_discard(ImageCtxT *ictx, AioCompletion *c, uint64_t off,
-                          uint64_t len);
-  static void aio_flush(ImageCtxT *ictx, AioCompletion *c);
-
-  virtual bool is_write_op() const {
-    return false;
-  }
-
-  void start_op() {
-    m_aio_comp->start_op();
-  }
-
-  void send();
-  void fail(int r);
-
-  void set_bypass_image_cache() {
-    m_bypass_image_cache = true;
-  }
-
-protected:
-  typedef std::list<AioObjectRequestHandle *> AioObjectRequests;
-
-  ImageCtxT &m_image_ctx;
-  AioCompletion *m_aio_comp;
-  Extents m_image_extents;
-  bool m_bypass_image_cache = false;
-
-  AioImageRequest(ImageCtxT &image_ctx, AioCompletion *aio_comp,
-                  Extents &&image_extents)
-    : m_image_ctx(image_ctx), m_aio_comp(aio_comp),
-      m_image_extents(image_extents) {
-  }
-
-  virtual int clip_request();
-  virtual void send_request() = 0;
-  virtual void send_image_cache_request() = 0;
-
-  virtual aio_type_t get_aio_type() const = 0;
-  virtual const char *get_request_type() const = 0;
-};
-
-template <typename ImageCtxT = ImageCtx>
-class AioImageRead : public AioImageRequest<ImageCtxT> {
-public:
-  using typename AioImageRequest<ImageCtxT>::Extents;
-
-  AioImageRead(ImageCtxT &image_ctx, AioCompletion *aio_comp,
-               Extents &&image_extents, char *buf, bufferlist *pbl,
-               int op_flags)
-    : AioImageRequest<ImageCtxT>(image_ctx, aio_comp, std::move(image_extents)),
-      m_buf(buf), m_pbl(pbl), m_op_flags(op_flags) {
-  }
-
-protected:
-  virtual void send_request() override;
-  virtual void send_image_cache_request() override;
-
-  virtual aio_type_t get_aio_type() const {
-    return AIO_TYPE_READ;
-  }
-  virtual const char *get_request_type() const {
-    return "aio_read";
-  }
-private:
-  char *m_buf;
-  bufferlist *m_pbl;
-  int m_op_flags;
-};
-
-template <typename ImageCtxT = ImageCtx>
-class AbstractAioImageWrite : public AioImageRequest<ImageCtxT> {
-public:
-  virtual bool is_write_op() const {
-    return true;
-  }
-
-  inline void flag_synchronous() {
-    m_synchronous = true;
-  }
-
-protected:
-  using typename AioImageRequest<ImageCtxT>::AioObjectRequests;
-  using typename AioImageRequest<ImageCtxT>::Extents;
-
-  typedef std::vector<ObjectExtent> ObjectExtents;
-
-  AbstractAioImageWrite(ImageCtxT &image_ctx, AioCompletion *aio_comp,
-                        Extents &&image_extents)
-    : AioImageRequest<ImageCtxT>(image_ctx, aio_comp, std::move(image_extents)),
-      m_synchronous(false) {
-  }
-
-  virtual void send_request();
-
-  virtual void prune_object_extents(ObjectExtents &object_extents) {
-  }
-  virtual uint32_t get_object_cache_request_count(bool journaling) const {
-    return 0;
-  }
-  virtual void send_object_cache_requests(const ObjectExtents &object_extents,
-                                          uint64_t journal_tid) = 0;
-
-  virtual void send_object_requests(const ObjectExtents &object_extents,
-                                    const ::SnapContext &snapc,
-                                    AioObjectRequests *aio_object_requests);
-  virtual AioObjectRequestHandle *create_object_request(
-      const ObjectExtent &object_extent, const ::SnapContext &snapc,
-      Context *on_finish) = 0;
-
-  virtual uint64_t append_journal_event(const AioObjectRequests &requests,
-                                        bool synchronous) = 0;
-  virtual void update_stats(size_t length) = 0;
-
-private:
-  bool m_synchronous;
-};
-
-template <typename ImageCtxT = ImageCtx>
-class AioImageWrite : public AbstractAioImageWrite<ImageCtxT> {
-public:
-  using typename AioImageRequest<ImageCtxT>::Extents;
-
-  AioImageWrite(ImageCtxT &image_ctx, AioCompletion *aio_comp, uint64_t off,
-                size_t len, const char *buf, int op_flags)
-    : AbstractAioImageWrite<ImageCtxT>(image_ctx, aio_comp, {{off, len}}),
-      m_op_flags(op_flags) {
-    m_bl.append(buf, len);
-  }
-  AioImageWrite(ImageCtxT &image_ctx, AioCompletion *aio_comp,
-                Extents &&image_extents, bufferlist &&bl, int op_flags)
-    : AbstractAioImageWrite<ImageCtxT>(image_ctx, aio_comp,
-                                       std::move(image_extents)),
-      m_bl(std::move(bl)), m_op_flags(op_flags) {
-  }
-
-protected:
-  using typename AioImageRequest<ImageCtxT>::AioObjectRequests;
-  using typename AbstractAioImageWrite<ImageCtxT>::ObjectExtents;
-
-  virtual aio_type_t get_aio_type() const {
-    return AIO_TYPE_WRITE;
-  }
-  virtual const char *get_request_type() const {
-    return "aio_write";
-  }
-
-  void assemble_extent(const ObjectExtent &object_extent, bufferlist *bl);
-
-  virtual void send_image_cache_request() override;
-
-  virtual void send_object_cache_requests(const ObjectExtents &object_extents,
-                                          uint64_t journal_tid);
-
-  virtual void send_object_requests(const ObjectExtents &object_extents,
-                                    const ::SnapContext &snapc,
-                                    AioObjectRequests *aio_object_requests);
-  virtual AioObjectRequestHandle *create_object_request(
-      const ObjectExtent &object_extent, const ::SnapContext &snapc,
-      Context *on_finish);
-
-  virtual uint64_t append_journal_event(const AioObjectRequests &requests,
-                                        bool synchronous);
-  virtual void update_stats(size_t length);
-private:
-  bufferlist m_bl;
-  int m_op_flags;
-};
-
-template <typename ImageCtxT = ImageCtx>
-class AioImageDiscard : public AbstractAioImageWrite<ImageCtxT> {
-public:
-  AioImageDiscard(ImageCtxT &image_ctx, AioCompletion *aio_comp, uint64_t off,
-                  uint64_t len)
-    : AbstractAioImageWrite<ImageCtxT>(image_ctx, aio_comp, {{off, len}}) {
-  }
-
-protected:
-  using typename AioImageRequest<ImageCtxT>::AioObjectRequests;
-  using typename AbstractAioImageWrite<ImageCtxT>::ObjectExtents;
-
-  virtual aio_type_t get_aio_type() const {
-    return AIO_TYPE_DISCARD;
-  }
-  virtual const char *get_request_type() const {
-    return "aio_discard";
-  }
-
-  virtual void prune_object_extents(ObjectExtents &object_extents) override;
-
-  virtual void send_image_cache_request() override;
-
-  virtual uint32_t get_object_cache_request_count(bool journaling) const override;
-  virtual void send_object_cache_requests(const ObjectExtents &object_extents,
-                                          uint64_t journal_tid);
-
-  virtual AioObjectRequestHandle *create_object_request(
-      const ObjectExtent &object_extent, const ::SnapContext &snapc,
-      Context *on_finish);
-
-  virtual uint64_t append_journal_event(const AioObjectRequests &requests,
-                                        bool synchronous);
-  virtual void update_stats(size_t length);
-};
-
-template <typename ImageCtxT = ImageCtx>
-class AioImageFlush : public AioImageRequest<ImageCtxT> {
-public:
-  AioImageFlush(ImageCtxT &image_ctx, AioCompletion *aio_comp)
-    : AioImageRequest<ImageCtxT>(image_ctx, aio_comp, {}) {
-  }
-
-  virtual bool is_write_op() const {
-    return true;
-  }
-
-protected:
-  using typename AioImageRequest<ImageCtxT>::AioObjectRequests;
-
-  virtual int clip_request() {
-    return 0;
-  }
-  virtual void send_request();
-  virtual void send_image_cache_request() override;
-
-  virtual aio_type_t get_aio_type() const {
-    return AIO_TYPE_FLUSH;
-  }
-  virtual const char *get_request_type() const {
-    return "aio_flush";
-  }
-};
-
-} // namespace librbd
-
-extern template class librbd::AioImageRequest<librbd::ImageCtx>;
-extern template class librbd::AbstractAioImageWrite<librbd::ImageCtx>;
-extern template class librbd::AioImageWrite<librbd::ImageCtx>;
-extern template class librbd::AioImageDiscard<librbd::ImageCtx>;
-extern template class librbd::AioImageFlush<librbd::ImageCtx>;
-
-#endif // CEPH_LIBRBD_AIO_IMAGE_REQUEST_H
diff --git a/src/librbd/AioImageRequestWQ.cc b/src/librbd/AioImageRequestWQ.cc
deleted file mode 100644 (file)
index 91b3dc7..0000000
+++ /dev/null
@@ -1,506 +0,0 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-
-#include "librbd/AioImageRequestWQ.h"
-#include "common/errno.h"
-#include "librbd/AioCompletion.h"
-#include "librbd/AioImageRequest.h"
-#include "librbd/ExclusiveLock.h"
-#include "librbd/exclusive_lock/Policy.h"
-#include "librbd/ImageCtx.h"
-#include "librbd/ImageState.h"
-#include "librbd/internal.h"
-#include "librbd/Utils.h"
-
-#define dout_subsys ceph_subsys_rbd
-#undef dout_prefix
-#define dout_prefix *_dout << "librbd::AioImageRequestWQ: "
-
-namespace librbd {
-
-AioImageRequestWQ::AioImageRequestWQ(ImageCtx *image_ctx, const string &name,
-                                     time_t ti, ThreadPool *tp)
-  : ThreadPool::PointerWQ<AioImageRequest<> >(name, ti, 0, tp),
-    m_image_ctx(*image_ctx),
-    m_lock(util::unique_lock_name("AioImageRequestWQ::m_lock", this)),
-    m_write_blockers(0), m_in_progress_writes(0), m_queued_reads(0),
-    m_queued_writes(0), m_in_flight_ops(0), m_refresh_in_progress(false),
-    m_shutdown(false), m_on_shutdown(nullptr) {
-  CephContext *cct = m_image_ctx.cct;
-  ldout(cct, 5) << this << " " << ": ictx=" << image_ctx << dendl;
-  tp->add_work_queue(this);
-}
-
-ssize_t AioImageRequestWQ::read(uint64_t off, uint64_t len, char *buf,
-                                int op_flags) {
-  CephContext *cct = m_image_ctx.cct;
-  ldout(cct, 20) << "read: ictx=" << &m_image_ctx << ", off=" << off << ", "
-                 << "len = " << len << dendl;
-
-  C_SaferCond cond;
-  AioCompletion *c = AioCompletion::create(&cond);
-  aio_read(c, off, len, buf, NULL, op_flags, false);
-  return cond.wait();
-}
-
-ssize_t AioImageRequestWQ::write(uint64_t off, uint64_t len, const char *buf,
-                                 int op_flags) {
-  CephContext *cct = m_image_ctx.cct;
-  ldout(cct, 20) << "write: ictx=" << &m_image_ctx << ", off=" << off << ", "
-                 << "len = " << len << dendl;
-
-  m_image_ctx.snap_lock.get_read();
-  int r = clip_io(util::get_image_ctx(&m_image_ctx), off, &len);
-  m_image_ctx.snap_lock.put_read();
-  if (r < 0) {
-    lderr(cct) << "invalid IO request: " << cpp_strerror(r) << dendl;
-    return r;
-  }
-
-  C_SaferCond cond;
-  AioCompletion *c = AioCompletion::create(&cond);
-  aio_write(c, off, len, buf, op_flags, false);
-
-  r = cond.wait();
-  if (r < 0) {
-    return r;
-  }
-  return len;
-}
-
-int AioImageRequestWQ::discard(uint64_t off, uint64_t len) {
-  CephContext *cct = m_image_ctx.cct;
-  ldout(cct, 20) << "discard: ictx=" << &m_image_ctx << ", off=" << off << ", "
-                 << "len = " << len << dendl;
-
-  m_image_ctx.snap_lock.get_read();
-  int r = clip_io(util::get_image_ctx(&m_image_ctx), off, &len);
-  m_image_ctx.snap_lock.put_read();
-  if (r < 0) {
-    lderr(cct) << "invalid IO request: " << cpp_strerror(r) << dendl;
-    return r;
-  }
-
-  C_SaferCond cond;
-  AioCompletion *c = AioCompletion::create(&cond);
-  aio_discard(c, off, len, false);
-
-  r = cond.wait();
-  if (r < 0) {
-    return r;
-  }
-  return len;
-}
-
-void AioImageRequestWQ::aio_read(AioCompletion *c, uint64_t off, uint64_t len,
-                                 char *buf, bufferlist *pbl, int op_flags,
-                                 bool native_async) {
-  c->init_time(&m_image_ctx, librbd::AIO_TYPE_READ);
-  CephContext *cct = m_image_ctx.cct;
-  ldout(cct, 20) << "aio_read: ictx=" << &m_image_ctx << ", "
-                 << "completion=" << c << ", off=" << off << ", "
-                 << "len=" << len << ", " << "flags=" << op_flags << dendl;
-
-  if (native_async && m_image_ctx.event_socket.is_valid()) {
-    c->set_event_notify(true);
-  }
-
-  if (!start_in_flight_op(c)) {
-    return;
-  }
-
-  RWLock::RLocker owner_locker(m_image_ctx.owner_lock);
-
-  // if journaling is enabled -- we need to replay the journal because
-  // it might contain an uncommitted write
-  bool lock_required;
-  {
-    RWLock::RLocker locker(m_lock);
-    lock_required = m_require_lock_on_read;
-  }
-
-  if (m_image_ctx.non_blocking_aio || writes_blocked() || !writes_empty() ||
-      lock_required) {
-    queue(new AioImageRead<>(m_image_ctx, c, {{off, len}}, buf, pbl, op_flags));
-  } else {
-    c->start_op();
-    AioImageRequest<>::aio_read(&m_image_ctx, c, {{off, len}}, buf, pbl,
-                                op_flags);
-    finish_in_flight_op();
-  }
-}
-
-void AioImageRequestWQ::aio_write(AioCompletion *c, uint64_t off, uint64_t len,
-                                  const char *buf, int op_flags,
-                                  bool native_async) {
-  c->init_time(&m_image_ctx, librbd::AIO_TYPE_WRITE);
-  CephContext *cct = m_image_ctx.cct;
-  ldout(cct, 20) << "aio_write: ictx=" << &m_image_ctx << ", "
-                 << "completion=" << c << ", off=" << off << ", "
-                 << "len=" << len << ", flags=" << op_flags << dendl;
-
-  if (native_async && m_image_ctx.event_socket.is_valid()) {
-    c->set_event_notify(true);
-  }
-
-  if (!start_in_flight_op(c)) {
-    return;
-  }
-
-  RWLock::RLocker owner_locker(m_image_ctx.owner_lock);
-  if (m_image_ctx.non_blocking_aio || writes_blocked()) {
-    queue(new AioImageWrite<>(m_image_ctx, c, off, len, buf, op_flags));
-  } else {
-    c->start_op();
-    AioImageRequest<>::aio_write(&m_image_ctx, c, off, len, buf, op_flags);
-    finish_in_flight_op();
-  }
-}
-
-void AioImageRequestWQ::aio_discard(AioCompletion *c, uint64_t off,
-                                    uint64_t len, bool native_async) {
-  c->init_time(&m_image_ctx, librbd::AIO_TYPE_DISCARD);
-  CephContext *cct = m_image_ctx.cct;
-  ldout(cct, 20) << "aio_discard: ictx=" << &m_image_ctx << ", "
-                 << "completion=" << c << ", off=" << off << ", len=" << len
-                 << dendl;
-
-  if (native_async && m_image_ctx.event_socket.is_valid()) {
-    c->set_event_notify(true);
-  }
-
-  if (!start_in_flight_op(c)) {
-    return;
-  }
-
-  RWLock::RLocker owner_locker(m_image_ctx.owner_lock);
-  if (m_image_ctx.non_blocking_aio || writes_blocked()) {
-    queue(new AioImageDiscard<>(m_image_ctx, c, off, len));
-  } else {
-    c->start_op();
-    AioImageRequest<>::aio_discard(&m_image_ctx, c, off, len);
-    finish_in_flight_op();
-  }
-}
-
-void AioImageRequestWQ::aio_flush(AioCompletion *c, bool native_async) {
-  c->init_time(&m_image_ctx, librbd::AIO_TYPE_FLUSH);
-  CephContext *cct = m_image_ctx.cct;
-  ldout(cct, 20) << "aio_flush: ictx=" << &m_image_ctx << ", "
-                 << "completion=" << c << dendl;
-
-  if (native_async && m_image_ctx.event_socket.is_valid()) {
-    c->set_event_notify(true);
-  }
-
-  if (!start_in_flight_op(c)) {
-    return;
-  }
-
-  RWLock::RLocker owner_locker(m_image_ctx.owner_lock);
-  if (m_image_ctx.non_blocking_aio || writes_blocked() || !writes_empty()) {
-    queue(new AioImageFlush<>(m_image_ctx, c));
-  } else {
-    AioImageRequest<>::aio_flush(&m_image_ctx, c);
-    finish_in_flight_op();
-  }
-}
-
-void AioImageRequestWQ::shut_down(Context *on_shutdown) {
-  assert(m_image_ctx.owner_lock.is_locked());
-
-  {
-    RWLock::WLocker locker(m_lock);
-    assert(!m_shutdown);
-    m_shutdown = true;
-
-    CephContext *cct = m_image_ctx.cct;
-    ldout(cct, 5) << __func__ << ": in_flight=" << m_in_flight_ops.read()
-                  << dendl;
-    if (m_in_flight_ops.read() > 0) {
-      m_on_shutdown = on_shutdown;
-      return;
-    }
-  }
-
-  // ensure that all in-flight IO is flushed
-  m_image_ctx.flush(on_shutdown);
-}
-
-bool AioImageRequestWQ::is_lock_request_needed() const {
-  RWLock::RLocker locker(m_lock);
-  return (m_queued_writes.read() > 0 ||
-          (m_require_lock_on_read && m_queued_reads.read() > 0));
-}
-
-int AioImageRequestWQ::block_writes() {
-  C_SaferCond cond_ctx;
-  block_writes(&cond_ctx);
-  return cond_ctx.wait();
-}
-
-void AioImageRequestWQ::block_writes(Context *on_blocked) {
-  assert(m_image_ctx.owner_lock.is_locked());
-  CephContext *cct = m_image_ctx.cct;
-
-  {
-    RWLock::WLocker locker(m_lock);
-    ++m_write_blockers;
-    ldout(cct, 5) << __func__ << ": " << &m_image_ctx << ", "
-                  << "num=" << m_write_blockers << dendl;
-    if (!m_write_blocker_contexts.empty() || m_in_progress_writes.read() > 0) {
-      m_write_blocker_contexts.push_back(on_blocked);
-      return;
-    }
-  }
-
-  // ensure that all in-flight IO is flushed
-  m_image_ctx.flush(on_blocked);
-}
-
-void AioImageRequestWQ::unblock_writes() {
-  CephContext *cct = m_image_ctx.cct;
-
-  bool wake_up = false;
-  {
-    RWLock::WLocker locker(m_lock);
-    assert(m_write_blockers > 0);
-    --m_write_blockers;
-
-    ldout(cct, 5) << __func__ << ": " << &m_image_ctx << ", "
-                  << "num=" << m_write_blockers << dendl;
-    if (m_write_blockers == 0) {
-      wake_up = true;
-    }
-  }
-
-  if (wake_up) {
-    signal();
-  }
-}
-
-void AioImageRequestWQ::set_require_lock_on_read() {
-  CephContext *cct = m_image_ctx.cct;
-  ldout(cct, 20) << __func__ << dendl;
-
-  RWLock::WLocker locker(m_lock);
-  m_require_lock_on_read = true;
-}
-
-void AioImageRequestWQ::clear_require_lock_on_read() {
-  CephContext *cct = m_image_ctx.cct;
-  ldout(cct, 20) << __func__ << dendl;
-
-  {
-    RWLock::WLocker locker(m_lock);
-    if (!m_require_lock_on_read) {
-      return;
-    }
-
-    m_require_lock_on_read = false;
-  }
-  signal();
-}
-
-void *AioImageRequestWQ::_void_dequeue() {
-  AioImageRequest<> *peek_item = front();
-
-  // no IO ops available or refresh in-progress (IO stalled)
-  if (peek_item == nullptr || m_refresh_in_progress) {
-    return nullptr;
-  }
-
-  bool refresh_required = m_image_ctx.state->is_refresh_required();
-  {
-    RWLock::RLocker locker(m_lock);
-    if (peek_item->is_write_op()) {
-      if (m_write_blockers > 0) {
-        return nullptr;
-      }
-
-      // refresh will requeue the op -- don't count it as in-progress
-      if (!refresh_required) {
-        m_in_progress_writes.inc();
-      }
-    } else if (m_require_lock_on_read) {
-      return nullptr;
-    }
-  }
-
-  AioImageRequest<> *item = reinterpret_cast<AioImageRequest<> *>(
-    ThreadPool::PointerWQ<AioImageRequest<> >::_void_dequeue());
-  assert(peek_item == item);
-
-  if (refresh_required) {
-    ldout(m_image_ctx.cct, 15) << "image refresh required: delaying IO " << item
-                               << dendl;
-
-    // stall IO until the refresh completes
-    m_refresh_in_progress = true;
-
-    get_pool_lock().Unlock();
-    m_image_ctx.state->refresh(new C_RefreshFinish(this, item));
-    get_pool_lock().Lock();
-    return nullptr;
-  }
-
-  item->start_op();
-  return item;
-}
-
-void AioImageRequestWQ::process(AioImageRequest<> *req) {
-  CephContext *cct = m_image_ctx.cct;
-  ldout(cct, 20) << __func__ << ": ictx=" << &m_image_ctx << ", "
-                 << "req=" << req << dendl;
-
-  req->send();
-
-  finish_queued_op(req);
-  if (req->is_write_op()) {
-    finish_in_progress_write();
-  }
-  delete req;
-
-  finish_in_flight_op();
-}
-
-void AioImageRequestWQ::finish_queued_op(AioImageRequest<> *req) {
-  RWLock::RLocker locker(m_lock);
-  if (req->is_write_op()) {
-    assert(m_queued_writes.read() > 0);
-    m_queued_writes.dec();
-  } else {
-    assert(m_queued_reads.read() > 0);
-    m_queued_reads.dec();
-  }
-}
-
-void AioImageRequestWQ::finish_in_progress_write() {
-  bool writes_blocked = false;
-  {
-    RWLock::RLocker locker(m_lock);
-    assert(m_in_progress_writes.read() > 0);
-    if (m_in_progress_writes.dec() == 0 &&
-        !m_write_blocker_contexts.empty()) {
-      writes_blocked = true;
-    }
-  }
-
-  if (writes_blocked) {
-    m_image_ctx.flush(new C_BlockedWrites(this));
-  }
-}
-
-int AioImageRequestWQ::start_in_flight_op(AioCompletion *c) {
-  RWLock::RLocker locker(m_lock);
-
-  if (m_shutdown) {
-    CephContext *cct = m_image_ctx.cct;
-    lderr(cct) << "IO received on closed image" << dendl;
-
-    c->fail(-ESHUTDOWN);
-    return false;
-  }
-
-  m_in_flight_ops.inc();
-  return true;
-}
-
-void AioImageRequestWQ::finish_in_flight_op() {
-  Context *on_shutdown;
-  {
-    RWLock::RLocker locker(m_lock);
-    if (m_in_flight_ops.dec() > 0 || !m_shutdown) {
-      return;
-    }
-    on_shutdown = m_on_shutdown;
-  }
-
-  CephContext *cct = m_image_ctx.cct;
-  ldout(cct, 5) << __func__ << ": completing shut down" << dendl;
-
-  assert(on_shutdown != nullptr);
-  m_image_ctx.flush(on_shutdown);
-}
-
-bool AioImageRequestWQ::is_lock_required() const {
-  assert(m_image_ctx.owner_lock.is_locked());
-  if (m_image_ctx.exclusive_lock == NULL) {
-    return false;
-  }
-
-  return (!m_image_ctx.exclusive_lock->is_lock_owner());
-}
-
-void AioImageRequestWQ::queue(AioImageRequest<> *req) {
-  CephContext *cct = m_image_ctx.cct;
-  ldout(cct, 20) << __func__ << ": ictx=" << &m_image_ctx << ", "
-                 << "req=" << req << dendl;
-
-  assert(m_image_ctx.owner_lock.is_locked());
-  bool write_op = req->is_write_op();
-  bool lock_required = (m_image_ctx.exclusive_lock != nullptr &&
-                        ((write_op && is_lock_required()) ||
-                          (!write_op && m_require_lock_on_read)));
-
-  if (lock_required && !m_image_ctx.get_exclusive_lock_policy()->may_auto_request_lock()) {
-    lderr(cct) << "op requires exclusive lock" << dendl;
-    req->fail(-EROFS);
-    delete req;
-    finish_in_flight_op();
-    return;
-  }
-
-  if (write_op) {
-    m_queued_writes.inc();
-  } else {
-    m_queued_reads.inc();
-  }
-
-  ThreadPool::PointerWQ<AioImageRequest<> >::queue(req);
-
-  if (lock_required) {
-    m_image_ctx.exclusive_lock->acquire_lock(nullptr);
-  }
-}
-
-void AioImageRequestWQ::handle_refreshed(int r, AioImageRequest<> *req) {
-  CephContext *cct = m_image_ctx.cct;
-  ldout(cct, 15) << "resuming IO after image refresh: r=" << r << ", "
-                 << "req=" << req << dendl;
-  if (r < 0) {
-    process_finish();
-    req->fail(r);
-    finish_queued_op(req);
-    delete req;
-    finish_in_flight_op();
-  } else {
-    // since IO was stalled for refresh -- original IO order is preserved
-    // if we requeue this op for work queue processing
-    requeue(req);
-  }
-
-  m_refresh_in_progress = false;
-  signal();
-
-  // refresh might have enabled exclusive lock -- IO stalled until
-  // we acquire the lock
-  RWLock::RLocker owner_locker(m_image_ctx.owner_lock);
-  if (is_lock_required() && is_lock_request_needed()) {
-    m_image_ctx.exclusive_lock->acquire_lock(nullptr);
-  }
-}
-
-void AioImageRequestWQ::handle_blocked_writes(int r) {
-  Contexts contexts;
-  {
-    RWLock::WLocker locker(m_lock);
-    contexts.swap(m_write_blocker_contexts);
-  }
-
-  for (auto ctx : contexts) {
-    ctx->complete(0);
-  }
-}
-
-} // namespace librbd
diff --git a/src/librbd/AioImageRequestWQ.h b/src/librbd/AioImageRequestWQ.h
deleted file mode 100644 (file)
index 65a70e1..0000000
+++ /dev/null
@@ -1,121 +0,0 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-
-#ifndef CEPH_LIBRBD_AIO_IMAGE_REQUEST_WQ_H
-#define CEPH_LIBRBD_AIO_IMAGE_REQUEST_WQ_H
-
-#include "include/Context.h"
-#include "include/atomic.h"
-#include "common/RWLock.h"
-#include "common/WorkQueue.h"
-#include <list>
-
-namespace librbd {
-
-class AioCompletion;
-template <typename> class AioImageRequest;
-class ImageCtx;
-
-class AioImageRequestWQ : protected ThreadPool::PointerWQ<AioImageRequest<ImageCtx> > {
-public:
-  AioImageRequestWQ(ImageCtx *image_ctx, const string &name, time_t ti,
-                    ThreadPool *tp);
-
-  ssize_t read(uint64_t off, uint64_t len, char *buf, int op_flags);
-  ssize_t write(uint64_t off, uint64_t len, const char *buf, int op_flags);
-  int discard(uint64_t off, uint64_t len);
-
-  void aio_read(AioCompletion *c, uint64_t off, uint64_t len, char *buf,
-                bufferlist *pbl, int op_flags, bool native_async=true);
-  void aio_write(AioCompletion *c, uint64_t off, uint64_t len, const char *buf,
-                 int op_flags, bool native_async=true);
-  void aio_discard(AioCompletion *c, uint64_t off, uint64_t len,
-                   bool native_async=true);
-  void aio_flush(AioCompletion *c, bool native_async=true);
-
-  using ThreadPool::PointerWQ<AioImageRequest<ImageCtx> >::drain;
-  using ThreadPool::PointerWQ<AioImageRequest<ImageCtx> >::empty;
-
-  void shut_down(Context *on_shutdown);
-
-  bool is_lock_request_needed() const;
-
-  inline bool writes_blocked() const {
-    RWLock::RLocker locker(m_lock);
-    return (m_write_blockers > 0);
-  }
-
-  int block_writes();
-  void block_writes(Context *on_blocked);
-  void unblock_writes();
-
-  void set_require_lock_on_read();
-  void clear_require_lock_on_read();
-
-protected:
-  virtual void *_void_dequeue();
-  virtual void process(AioImageRequest<ImageCtx> *req);
-
-private:
-  typedef std::list<Context *> Contexts;
-
-  struct C_RefreshFinish : public Context {
-    AioImageRequestWQ *aio_work_queue;
-    AioImageRequest<ImageCtx> *aio_image_request;
-
-    C_RefreshFinish(AioImageRequestWQ *aio_work_queue,
-                    AioImageRequest<ImageCtx> *aio_image_request)
-      : aio_work_queue(aio_work_queue), aio_image_request(aio_image_request) {
-    }
-    virtual void finish(int r) override {
-      aio_work_queue->handle_refreshed(r, aio_image_request);
-    }
-  };
-
-  struct C_BlockedWrites : public Context {
-    AioImageRequestWQ *aio_work_queue;
-    C_BlockedWrites(AioImageRequestWQ *_aio_work_queue)
-      : aio_work_queue(_aio_work_queue) {
-    }
-
-    virtual void finish(int r) {
-      aio_work_queue->handle_blocked_writes(r);
-    }
-  };
-
-  ImageCtx &m_image_ctx;
-  mutable RWLock m_lock;
-  Contexts m_write_blocker_contexts;
-  uint32_t m_write_blockers;
-  bool m_require_lock_on_read = false;
-  atomic_t m_in_progress_writes;
-  atomic_t m_queued_reads;
-  atomic_t m_queued_writes;
-  atomic_t m_in_flight_ops;
-
-  bool m_refresh_in_progress;
-
-  bool m_shutdown;
-  Context *m_on_shutdown;
-
-  inline bool writes_empty() const {
-    RWLock::RLocker locker(m_lock);
-    return (m_queued_writes.read() == 0);
-  }
-
-  void finish_queued_op(AioImageRequest<ImageCtx> *req);
-  void finish_in_progress_write();
-
-  int start_in_flight_op(AioCompletion *c);
-  void finish_in_flight_op();
-
-  bool is_lock_required() const;
-  void queue(AioImageRequest<ImageCtx> *req);
-
-  void handle_refreshed(int r, AioImageRequest<ImageCtx> *req);
-  void handle_blocked_writes(int r);
-};
-
-} // namespace librbd
-
-#endif // CEPH_LIBRBD_AIO_IMAGE_REQUEST_WQ_H
diff --git a/src/librbd/AioObjectRequest.cc b/src/librbd/AioObjectRequest.cc
deleted file mode 100644 (file)
index 3417eda..0000000
+++ /dev/null
@@ -1,626 +0,0 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-
-#include "common/ceph_context.h"
-#include "common/dout.h"
-#include "common/errno.h"
-#include "common/Mutex.h"
-#include "common/RWLock.h"
-#include "common/WorkQueue.h"
-#include "include/Context.h"
-
-#include "librbd/AioObjectRequest.h"
-#include "librbd/AioCompletion.h"
-#include "librbd/AioImageRequest.h"
-#include "librbd/CopyupRequest.h"
-#include "librbd/ExclusiveLock.h"
-#include "librbd/ImageCtx.h"
-#include "librbd/ObjectMap.h"
-#include "librbd/Utils.h"
-
-#include <boost/bind.hpp>
-#include <boost/optional.hpp>
-
-#define dout_subsys ceph_subsys_rbd
-#undef dout_prefix
-#define dout_prefix *_dout << "librbd::AioObjectRequest: "
-
-namespace librbd {
-
-template <typename I>
-AioObjectRequest<I>*
-AioObjectRequest<I>::create_remove(I *ictx, const std::string &oid,
-                                   uint64_t object_no,
-                                   const ::SnapContext &snapc,
-                                   Context *completion) {
-  return new AioObjectRemove(util::get_image_ctx(ictx), oid, object_no, snapc,
-                             completion);
-}
-
-template <typename I>
-AioObjectRequest<I>*
-AioObjectRequest<I>::create_truncate(I *ictx, const std::string &oid,
-                                     uint64_t object_no, uint64_t object_off,
-                                     const ::SnapContext &snapc,
-                                     Context *completion) {
-  return new AioObjectTruncate(util::get_image_ctx(ictx), oid, object_no,
-                               object_off, snapc, completion);
-}
-
-template <typename I>
-AioObjectRequest<I>*
-AioObjectRequest<I>::create_write(I *ictx, const std::string &oid,
-                                  uint64_t object_no, uint64_t object_off,
-                                  const ceph::bufferlist &data,
-                                  const ::SnapContext &snapc,
-                                  Context *completion, int op_flags) {
-  return new AioObjectWrite(util::get_image_ctx(ictx), oid, object_no,
-                            object_off, data, snapc, completion, op_flags);
-}
-
-template <typename I>
-AioObjectRequest<I>*
-AioObjectRequest<I>::create_zero(I *ictx, const std::string &oid,
-                                 uint64_t object_no, uint64_t object_off,
-                                 uint64_t object_len,
-                                 const ::SnapContext &snapc,
-                                 Context *completion) {
-  return new AioObjectZero(util::get_image_ctx(ictx), oid, object_no,
-                           object_off, object_len, snapc, completion);
-}
-
-template <typename I>
-AioObjectRequest<I>::AioObjectRequest(ImageCtx *ictx, const std::string &oid,
-                                      uint64_t objectno, uint64_t off,
-                                      uint64_t len, librados::snap_t snap_id,
-                                      Context *completion, bool hide_enoent)
-  : m_ictx(ictx), m_oid(oid), m_object_no(objectno), m_object_off(off),
-    m_object_len(len), m_snap_id(snap_id), m_completion(completion),
-    m_hide_enoent(hide_enoent) {
-
-  Striper::extent_to_file(m_ictx->cct, &m_ictx->layout, m_object_no,
-                          0, m_ictx->layout.object_size, m_parent_extents);
-
-  RWLock::RLocker snap_locker(m_ictx->snap_lock);
-  RWLock::RLocker parent_locker(m_ictx->parent_lock);
-  compute_parent_extents();
-}
-
-template <typename I>
-void AioObjectRequest<I>::complete(int r)
-{
-  if (should_complete(r)) {
-    ldout(m_ictx->cct, 20) << "complete " << this << dendl;
-    if (m_hide_enoent && r == -ENOENT) {
-      r = 0;
-    }
-    m_completion->complete(r);
-    delete this;
-  }
-}
-
-template <typename I>
-bool AioObjectRequest<I>::compute_parent_extents() {
-  assert(m_ictx->snap_lock.is_locked());
-  assert(m_ictx->parent_lock.is_locked());
-
-  uint64_t parent_overlap;
-  int r = m_ictx->get_parent_overlap(m_snap_id, &parent_overlap);
-  if (r < 0) {
-    // NOTE: it's possible for a snapshot to be deleted while we are
-    // still reading from it
-    lderr(m_ictx->cct) << this << " compute_parent_extents: failed to "
-                       << "retrieve parent overlap: " << cpp_strerror(r)
-                       << dendl;
-    m_has_parent = false;
-    m_parent_extents.clear();
-    return false;
-  }
-
-  uint64_t object_overlap =
-    m_ictx->prune_parent_extents(m_parent_extents, parent_overlap);
-  if (object_overlap > 0) {
-    ldout(m_ictx->cct, 20) << this << " compute_parent_extents: "
-                           << "overlap " << parent_overlap << " "
-                           << "extents " << m_parent_extents << dendl;
-    m_has_parent = !m_parent_extents.empty();
-    return true;
-  }
-  return false;
-}
-
-static inline bool is_copy_on_read(ImageCtx *ictx, librados::snap_t snap_id) {
-  assert(ictx->snap_lock.is_locked());
-  return (ictx->clone_copy_on_read &&
-          !ictx->read_only && snap_id == CEPH_NOSNAP &&
-          (ictx->exclusive_lock == nullptr ||
-           ictx->exclusive_lock->is_lock_owner()));
-}
-
-/** read **/
-
-template <typename I>
-AioObjectRead<I>::AioObjectRead(I *ictx, const std::string &oid,
-                                uint64_t objectno, uint64_t offset,
-                                uint64_t len,
-                                vector<pair<uint64_t,uint64_t> >& be,
-                                librados::snap_t snap_id, bool sparse,
-                                Context *completion, int op_flags)
-  : AioObjectRequest<I>(util::get_image_ctx(ictx), oid, objectno, offset, len,
-                        snap_id, completion, false),
-    m_buffer_extents(be), m_tried_parent(false), m_sparse(sparse),
-    m_op_flags(op_flags), m_state(LIBRBD_AIO_READ_FLAT) {
-  guard_read();
-}
-
-template <typename I>
-void AioObjectRead<I>::guard_read()
-{
-  ImageCtx *image_ctx = this->m_ictx;
-  RWLock::RLocker snap_locker(image_ctx->snap_lock);
-  RWLock::RLocker parent_locker(image_ctx->parent_lock);
-
-  if (this->has_parent()) {
-    ldout(image_ctx->cct, 20) << __func__ << " guarding read" << dendl;
-    m_state = LIBRBD_AIO_READ_GUARD;
-  }
-}
-
-template <typename I>
-bool AioObjectRead<I>::should_complete(int r)
-{
-  ImageCtx *image_ctx = this->m_ictx;
-  ldout(image_ctx->cct, 20) << "should_complete " << this << " "
-                            << this->m_oid << " "
-                            << this->m_object_off << "~" << this->m_object_len
-                            << " r = " << r << dendl;
-
-  bool finished = true;
-
-  switch (m_state) {
-  case LIBRBD_AIO_READ_GUARD:
-    ldout(image_ctx->cct, 20) << "should_complete " << this
-                              << " READ_CHECK_GUARD" << dendl;
-
-    // This is the step to read from parent
-    if (!m_tried_parent && r == -ENOENT) {
-      {
-        RWLock::RLocker snap_locker(image_ctx->snap_lock);
-        RWLock::RLocker parent_locker(image_ctx->parent_lock);
-        if (image_ctx->parent == NULL) {
-          ldout(image_ctx->cct, 20) << "parent is gone; do nothing" << dendl;
-          m_state = LIBRBD_AIO_READ_FLAT;
-          finished = false;
-          break;
-        }
-
-        // calculate reverse mapping onto the image
-        vector<pair<uint64_t,uint64_t> > parent_extents;
-        Striper::extent_to_file(image_ctx->cct, &image_ctx->layout,
-                                this->m_object_no, this->m_object_off,
-                                this->m_object_len, parent_extents);
-
-        uint64_t parent_overlap = 0;
-        uint64_t object_overlap = 0;
-        r = image_ctx->get_parent_overlap(this->m_snap_id, &parent_overlap);
-        if (r == 0) {
-          object_overlap = image_ctx->prune_parent_extents(parent_extents,
-                                                           parent_overlap);
-        }
-
-        if (object_overlap > 0) {
-          m_tried_parent = true;
-          if (is_copy_on_read(image_ctx, this->m_snap_id)) {
-            m_state = LIBRBD_AIO_READ_COPYUP;
-          }
-
-          read_from_parent(std::move(parent_extents));
-          finished = false;
-        }
-      }
-    }
-    break;
-  case LIBRBD_AIO_READ_COPYUP:
-    ldout(image_ctx->cct, 20) << "should_complete " << this << " READ_COPYUP"
-                              << dendl;
-    // This is the extra step for copy-on-read: kick off an asynchronous copyup.
-    // It is different from copy-on-write as asynchronous copyup will finish
-    // by itself so state won't go back to LIBRBD_AIO_READ_GUARD.
-
-    assert(m_tried_parent);
-    if (r > 0) {
-      // If read entire object from parent success and CoR is possible, kick
-      // off a asynchronous copyup. This approach minimizes the latency
-      // impact.
-      send_copyup();
-    }
-    break;
-  case LIBRBD_AIO_READ_FLAT:
-    ldout(image_ctx->cct, 20) << "should_complete " << this << " READ_FLAT"
-                              << dendl;
-    // The read content should be deposit in m_read_data
-    break;
-  default:
-    lderr(image_ctx->cct) << "invalid request state: " << m_state << dendl;
-    ceph_abort();
-  }
-
-  return finished;
-}
-
-template <typename I>
-void AioObjectRead<I>::send() {
-  ImageCtx *image_ctx = this->m_ictx;
-  ldout(image_ctx->cct, 20) << "send " << this << " " << this->m_oid << " "
-                            << this->m_object_off << "~" << this->m_object_len
-                            << dendl;
-
-  {
-    RWLock::RLocker snap_locker(image_ctx->snap_lock);
-
-    // send read request to parent if the object doesn't exist locally
-    if (image_ctx->object_map != nullptr &&
-        !image_ctx->object_map->object_may_exist(this->m_object_no)) {
-      image_ctx->op_work_queue->queue(util::create_context_callback<
-        AioObjectRequest<I> >(this), -ENOENT);
-      return;
-    }
-  }
-
-  librados::ObjectReadOperation op;
-  int flags = image_ctx->get_read_flags(this->m_snap_id);
-  if (m_sparse) {
-    op.sparse_read(this->m_object_off, this->m_object_len, &m_ext_map,
-                   &m_read_data, nullptr);
-  } else {
-    op.read(this->m_object_off, this->m_object_len, &m_read_data, nullptr);
-  }
-  op.set_op_flags2(m_op_flags);
-
-  librados::AioCompletion *rados_completion =
-    util::create_rados_ack_callback(this);
-  int r = image_ctx->data_ctx.aio_operate(this->m_oid, rados_completion, &op,
-                                          flags, nullptr);
-  assert(r == 0);
-
-  rados_completion->release();
-}
-
-template <typename I>
-void AioObjectRead<I>::send_copyup()
-{
-  ImageCtx *image_ctx = this->m_ictx;
-  {
-    RWLock::RLocker snap_locker(image_ctx->snap_lock);
-    RWLock::RLocker parent_locker(image_ctx->parent_lock);
-    if (!this->compute_parent_extents() ||
-        (image_ctx->exclusive_lock != nullptr &&
-         !image_ctx->exclusive_lock->is_lock_owner())) {
-      return;
-    }
-  }
-
-  Mutex::Locker copyup_locker(image_ctx->copyup_list_lock);
-  map<uint64_t, CopyupRequest*>::iterator it =
-    image_ctx->copyup_list.find(this->m_object_no);
-  if (it == image_ctx->copyup_list.end()) {
-    // create and kick off a CopyupRequest
-    CopyupRequest *new_req = new CopyupRequest(
-      image_ctx, this->m_oid, this->m_object_no,
-      std::move(this->m_parent_extents));
-    this->m_parent_extents.clear();
-
-    image_ctx->copyup_list[this->m_object_no] = new_req;
-    new_req->send();
-  }
-}
-
-template <typename I>
-void AioObjectRead<I>::read_from_parent(Extents&& parent_extents)
-{
-  ImageCtx *image_ctx = this->m_ictx;
-  AioCompletion *parent_completion = AioCompletion::create_and_start<
-    AioObjectRequest<I> >(this, image_ctx, AIO_TYPE_READ);
-
-  ldout(image_ctx->cct, 20) << "read_from_parent this = " << this
-                            << " parent completion " << parent_completion
-                            << " extents " << parent_extents
-                            << dendl;
-  AioImageRequest<>::aio_read(image_ctx->parent, parent_completion,
-                              std::move(parent_extents), nullptr, &m_read_data,
-                              0);
-}
-
-/** write **/
-
-AbstractAioObjectWrite::AbstractAioObjectWrite(ImageCtx *ictx,
-                                               const std::string &oid,
-                                               uint64_t object_no,
-                                               uint64_t object_off,
-                                               uint64_t len,
-                                               const ::SnapContext &snapc,
-                                               Context *completion,
-                                               bool hide_enoent)
-  : AioObjectRequest(ictx, oid, object_no, object_off, len, CEPH_NOSNAP,
-                     completion, hide_enoent),
-    m_state(LIBRBD_AIO_WRITE_FLAT), m_snap_seq(snapc.seq.val)
-{
-  m_snaps.insert(m_snaps.end(), snapc.snaps.begin(), snapc.snaps.end());
-}
-
-void AbstractAioObjectWrite::guard_write()
-{
-  if (has_parent()) {
-    m_state = LIBRBD_AIO_WRITE_GUARD;
-    m_write.assert_exists();
-    ldout(m_ictx->cct, 20) << __func__ << " guarding write" << dendl;
-  }
-}
-
-bool AbstractAioObjectWrite::should_complete(int r)
-{
-  ldout(m_ictx->cct, 20) << get_op_type() << " " << this << " " << m_oid
-                         << " " << m_object_off << "~" << m_object_len
-                         << " should_complete: r = " << r << dendl;
-
-  bool finished = true;
-  switch (m_state) {
-  case LIBRBD_AIO_WRITE_PRE:
-    ldout(m_ictx->cct, 20) << "WRITE_PRE" << dendl;
-    if (r < 0) {
-      return true;
-    }
-
-    send_write_op();
-    finished = false;
-    break;
-
-  case LIBRBD_AIO_WRITE_POST:
-    ldout(m_ictx->cct, 20) << "WRITE_POST" << dendl;
-    finished = true;
-    break;
-
-  case LIBRBD_AIO_WRITE_GUARD:
-    ldout(m_ictx->cct, 20) << "WRITE_CHECK_GUARD" << dendl;
-
-    if (r == -ENOENT) {
-      handle_write_guard();
-      finished = false;
-      break;
-    } else if (r < 0) {
-      // pass the error code to the finish context
-      m_state = LIBRBD_AIO_WRITE_ERROR;
-      complete(r);
-      finished = false;
-      break;
-    }
-
-    finished = send_post_object_map_update();
-    break;
-
-  case LIBRBD_AIO_WRITE_COPYUP:
-    ldout(m_ictx->cct, 20) << "WRITE_COPYUP" << dendl;
-    if (r < 0) {
-      m_state = LIBRBD_AIO_WRITE_ERROR;
-      complete(r);
-      finished = false;
-    } else {
-      finished = send_post_object_map_update();
-    }
-    break;
-
-  case LIBRBD_AIO_WRITE_FLAT:
-    ldout(m_ictx->cct, 20) << "WRITE_FLAT" << dendl;
-
-    finished = send_post_object_map_update();
-    break;
-
-  case LIBRBD_AIO_WRITE_ERROR:
-    assert(r < 0);
-    lderr(m_ictx->cct) << "WRITE_ERROR: " << cpp_strerror(r) << dendl;
-    break;
-
-  default:
-    lderr(m_ictx->cct) << "invalid request state: " << m_state << dendl;
-    ceph_abort();
-  }
-
-  return finished;
-}
-
-void AbstractAioObjectWrite::send() {
-  ldout(m_ictx->cct, 20) << "send " << get_op_type() << " " << this <<" "
-                         << m_oid << " " << m_object_off << "~"
-                         << m_object_len << dendl;
-  {
-    RWLock::RLocker snap_lock(m_ictx->snap_lock);
-    if (m_ictx->object_map == nullptr) {
-      m_object_exist = true;
-    } else {
-      // should have been flushed prior to releasing lock
-      assert(m_ictx->exclusive_lock->is_lock_owner());
-      m_object_exist = m_ictx->object_map->object_may_exist(m_object_no);
-    }
-  }
-
-  send_write();
-}
-
-void AbstractAioObjectWrite::send_pre_object_map_update() {
-  ldout(m_ictx->cct, 20) << __func__ << dendl;
-
-  {
-    RWLock::RLocker snap_lock(m_ictx->snap_lock);
-    if (m_ictx->object_map != nullptr) {
-      uint8_t new_state;
-      pre_object_map_update(&new_state);
-      RWLock::WLocker object_map_locker(m_ictx->object_map_lock);
-      ldout(m_ictx->cct, 20) << __func__ << this << " " << m_oid << " "
-                             << m_object_off << "~" << m_object_len
-                             << dendl;
-      m_state = LIBRBD_AIO_WRITE_PRE;
-
-      if (m_ictx->object_map->aio_update<AioObjectRequest>(
-            CEPH_NOSNAP, m_object_no, new_state, {}, this)) {
-        return;
-      }
-    }
-  }
-
-  send_write_op();
-}
-
-bool AbstractAioObjectWrite::send_post_object_map_update() {
-  RWLock::RLocker snap_locker(m_ictx->snap_lock);
-  if (m_ictx->object_map == nullptr || !post_object_map_update()) {
-    return true;
-  }
-
-  // should have been flushed prior to releasing lock
-  assert(m_ictx->exclusive_lock->is_lock_owner());
-
-  RWLock::WLocker object_map_locker(m_ictx->object_map_lock);
-  ldout(m_ictx->cct, 20) << __func__ << this << " " << m_oid << " "
-                         << m_object_off << "~" << m_object_len << dendl;
-  m_state = LIBRBD_AIO_WRITE_POST;
-
-  if (m_ictx->object_map->aio_update<AioObjectRequest>(
-        CEPH_NOSNAP, m_object_no, OBJECT_NONEXISTENT, OBJECT_PENDING, this)) {
-    return false;
-  }
-
-  return true;
-}
-
-void AbstractAioObjectWrite::send_write() {
-  ldout(m_ictx->cct, 20) << "send_write " << this << " " << m_oid << " "
-                        << m_object_off << "~" << m_object_len
-                         << " object exist " << m_object_exist << dendl;
-
-  if (!m_object_exist && has_parent()) {
-    m_state = LIBRBD_AIO_WRITE_GUARD;
-    handle_write_guard();
-  } else {
-    send_pre_object_map_update();
-  }
-}
-
-void AbstractAioObjectWrite::send_copyup()
-{
-  ldout(m_ictx->cct, 20) << "send_copyup " << this << " " << m_oid << " "
-                         << m_object_off << "~" << m_object_len << dendl;
-  m_state = LIBRBD_AIO_WRITE_COPYUP;
-
-  m_ictx->copyup_list_lock.Lock();
-  map<uint64_t, CopyupRequest*>::iterator it =
-    m_ictx->copyup_list.find(m_object_no);
-  if (it == m_ictx->copyup_list.end()) {
-    CopyupRequest *new_req = new CopyupRequest(m_ictx, m_oid,
-                                               m_object_no,
-                                               std::move(m_parent_extents));
-    m_parent_extents.clear();
-
-    // make sure to wait on this CopyupRequest
-    new_req->append_request(this);
-    m_ictx->copyup_list[m_object_no] = new_req;
-
-    m_ictx->copyup_list_lock.Unlock();
-    new_req->send();
-  } else {
-    it->second->append_request(this);
-    m_ictx->copyup_list_lock.Unlock();
-  }
-}
-void AbstractAioObjectWrite::send_write_op()
-{
-  m_state = LIBRBD_AIO_WRITE_FLAT;
-  if (m_guard) {
-    guard_write();
-  }
-
-  add_write_ops(&m_write);
-  assert(m_write.size() != 0);
-
-  librados::AioCompletion *rados_completion =
-    util::create_rados_safe_callback(this);
-  int r = m_ictx->data_ctx.aio_operate(m_oid, rados_completion, &m_write,
-                                       m_snap_seq, m_snaps);
-  assert(r == 0);
-  rados_completion->release();
-}
-void AbstractAioObjectWrite::handle_write_guard()
-{
-  bool has_parent;
-  {
-    RWLock::RLocker snap_locker(m_ictx->snap_lock);
-    RWLock::RLocker parent_locker(m_ictx->parent_lock);
-    has_parent = compute_parent_extents();
-  }
-  // If parent still exists, overlap might also have changed.
-  if (has_parent) {
-    send_copyup();
-  } else {
-    // parent may have disappeared -- send original write again
-    ldout(m_ictx->cct, 20) << "should_complete(" << this
-                           << "): parent overlap now 0" << dendl;
-    send_write();
-  }
-}
-
-void AioObjectWrite::add_write_ops(librados::ObjectWriteOperation *wr) {
-  RWLock::RLocker snap_locker(m_ictx->snap_lock);
-  if (m_ictx->enable_alloc_hint &&
-      (m_ictx->object_map == nullptr || !m_object_exist)) {
-    wr->set_alloc_hint(m_ictx->get_object_size(), m_ictx->get_object_size());
-  }
-
-  if (m_object_off == 0 && m_object_len == m_ictx->get_object_size()) {
-    wr->write_full(m_write_data);
-  } else {
-    wr->write(m_object_off, m_write_data);
-  }
-  wr->set_op_flags2(m_op_flags);
-}
-
-void AioObjectWrite::send_write() {
-  bool write_full = (m_object_off == 0 && m_object_len == m_ictx->get_object_size());
-  ldout(m_ictx->cct, 20) << "send_write " << this << " " << m_oid << " "
-                         << m_object_off << "~" << m_object_len
-                         << " object exist " << m_object_exist
-                         << " write_full " << write_full << dendl;
-  if (write_full && !has_parent()) {
-    m_guard = false;
-  }
-
-  AbstractAioObjectWrite::send_write();
-}
-
-void AioObjectRemove::guard_write() {
-  // do nothing to disable write guard only if deep-copyup not required
-  RWLock::RLocker snap_locker(m_ictx->snap_lock);
-  if (!m_ictx->snaps.empty()) {
-    AbstractAioObjectWrite::guard_write();
-  }
-}
-void AioObjectRemove::send_write() {
-  ldout(m_ictx->cct, 20) << "send_write " << this << " " << m_oid << " "
-                         << m_object_off << "~" << m_object_len << dendl;
-  send_pre_object_map_update();
-}
-
-void AioObjectTruncate::send_write() {
-  ldout(m_ictx->cct, 20) << "send_write " << this << " " << m_oid
-                         << " truncate " << m_object_off << dendl;
-  if (!m_object_exist && ! has_parent()) {
-    m_state = LIBRBD_AIO_WRITE_FLAT;
-    Context *ctx = util::create_context_callback<AioObjectRequest>(this);
-    m_ictx->op_work_queue->queue(ctx, 0);
-  } else {
-    AbstractAioObjectWrite::send_write();
-  }
-}
-
-} // namespace librbd
-
-template class librbd::AioObjectRequest<librbd::ImageCtx>;
-template class librbd::AioObjectRead<librbd::ImageCtx>;
diff --git a/src/librbd/AioObjectRequest.h b/src/librbd/AioObjectRequest.h
deleted file mode 100644 (file)
index 08fa870..0000000
+++ /dev/null
@@ -1,451 +0,0 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-
-#ifndef CEPH_LIBRBD_AIO_OBJECT_REQUEST_H
-#define CEPH_LIBRBD_AIO_OBJECT_REQUEST_H
-
-#include "include/int_types.h"
-
-#include <map>
-
-#include "common/snap_types.h"
-#include "include/buffer.h"
-#include "include/rados/librados.hpp"
-#include "librbd/ObjectMap.h"
-
-class Context;
-
-namespace librbd {
-
-struct AioCompletion;
-class AioObjectRemove;
-class AioObjectTruncate;
-class AioObjectWrite;
-class AioObjectZero;
-struct ImageCtx;
-class CopyupRequest;
-
-struct AioObjectRequestHandle {
-  virtual ~AioObjectRequestHandle() {
-  }
-
-  virtual void complete(int r) = 0;
-  virtual void send() = 0;
-};
-
-/**
- * This class represents an I/O operation to a single RBD data object.
- * Its subclasses encapsulate logic for dealing with special cases
- * for I/O due to layering.
- */
-template <typename ImageCtxT = ImageCtx>
-class AioObjectRequest : public AioObjectRequestHandle {
-public:
-  typedef std::vector<std::pair<uint64_t, uint64_t> > Extents;
-
-  static AioObjectRequest* create_remove(ImageCtxT *ictx,
-                                         const std::string &oid,
-                                         uint64_t object_no,
-                                         const ::SnapContext &snapc,
-                                         Context *completion);
-  static AioObjectRequest* create_truncate(ImageCtxT *ictx,
-                                           const std::string &oid,
-                                           uint64_t object_no,
-                                           uint64_t object_off,
-                                           const ::SnapContext &snapc,
-                                           Context *completion);
-  static AioObjectRequest* create_write(ImageCtxT *ictx, const std::string &oid,
-                                        uint64_t object_no,
-                                        uint64_t object_off,
-                                        const ceph::bufferlist &data,
-                                        const ::SnapContext &snapc,
-                                        Context *completion, int op_flags);
-  static AioObjectRequest* create_zero(ImageCtxT *ictx, const std::string &oid,
-                                       uint64_t object_no, uint64_t object_off,
-                                       uint64_t object_len,
-                                       const ::SnapContext &snapc,
-                                       Context *completion);
-
-  AioObjectRequest(ImageCtx *ictx, const std::string &oid,
-                   uint64_t objectno, uint64_t off, uint64_t len,
-                   librados::snap_t snap_id,
-                   Context *completion, bool hide_enoent);
-  virtual ~AioObjectRequest() {}
-
-  virtual void add_copyup_ops(librados::ObjectWriteOperation *wr) {};
-
-  void complete(int r);
-
-  virtual bool should_complete(int r) = 0;
-  virtual void send() = 0;
-
-  bool has_parent() const {
-    return m_has_parent;
-  }
-
-  virtual bool is_op_payload_empty() const {
-    return false;
-  }
-
-  virtual const char *get_op_type() const = 0;
-  virtual bool pre_object_map_update(uint8_t *new_state) = 0;
-
-protected:
-  bool compute_parent_extents();
-
-  ImageCtx *m_ictx;
-  std::string m_oid;
-  uint64_t m_object_no, m_object_off, m_object_len;
-  librados::snap_t m_snap_id;
-  Context *m_completion;
-  Extents m_parent_extents;
-  bool m_hide_enoent;
-
-private:
-  bool m_has_parent = false;
-};
-
-template <typename ImageCtxT = ImageCtx>
-class AioObjectRead : public AioObjectRequest<ImageCtxT> {
-public:
-  typedef std::vector<std::pair<uint64_t, uint64_t> > Extents;
-  typedef std::map<uint64_t, uint64_t> ExtentMap;
-
-  static AioObjectRead* create(ImageCtxT *ictx, const std::string &oid,
-                               uint64_t objectno, uint64_t offset,
-                               uint64_t len, Extents &buffer_extents,
-                               librados::snap_t snap_id, bool sparse,
-                               Context *completion, int op_flags) {
-    return new AioObjectRead(ictx, oid, objectno, offset, len, buffer_extents,
-                             snap_id, sparse, completion, op_flags);
-  }
-
-  AioObjectRead(ImageCtxT *ictx, const std::string &oid,
-                uint64_t objectno, uint64_t offset, uint64_t len,
-                Extents& buffer_extents, librados::snap_t snap_id, bool sparse,
-                Context *completion, int op_flags);
-
-  virtual bool should_complete(int r);
-  virtual void send();
-  void guard_read();
-
-  inline uint64_t get_offset() const {
-    return this->m_object_off;
-  }
-  inline uint64_t get_length() const {
-    return this->m_object_len;
-  }
-  ceph::bufferlist &data() {
-    return m_read_data;
-  }
-  const Extents &get_buffer_extents() const {
-    return m_buffer_extents;
-  }
-  ExtentMap &get_extent_map() {
-    return m_ext_map;
-  }
-
-  const char *get_op_type() const {
-    return "read";
-  }
-
-  bool pre_object_map_update(uint8_t *new_state) {
-    return false;
-  }
-
-private:
-  Extents m_buffer_extents;
-  bool m_tried_parent;
-  bool m_sparse;
-  int m_op_flags;
-  ceph::bufferlist m_read_data;
-  ExtentMap m_ext_map;
-
-  /**
-   * Reads go through the following state machine to deal with
-   * layering:
-   *
-   *                          need copyup
-   * LIBRBD_AIO_READ_GUARD ---------------> LIBRBD_AIO_READ_COPYUP
-   *           |                                       |
-   *           v                                       |
-   *         done <------------------------------------/
-   *           ^
-   *           |
-   * LIBRBD_AIO_READ_FLAT
-   *
-   * Reads start in LIBRBD_AIO_READ_GUARD or _FLAT, depending on
-   * whether there is a parent or not.
-   */
-  enum read_state_d {
-    LIBRBD_AIO_READ_GUARD,
-    LIBRBD_AIO_READ_COPYUP,
-    LIBRBD_AIO_READ_FLAT
-  };
-
-  read_state_d m_state;
-
-  void send_copyup();
-
-  void read_from_parent(Extents&& image_extents);
-};
-
-class AbstractAioObjectWrite : public AioObjectRequest<> {
-public:
-  AbstractAioObjectWrite(ImageCtx *ictx, const std::string &oid,
-                         uint64_t object_no, uint64_t object_off,
-                         uint64_t len, const ::SnapContext &snapc,
-                         Context *completion, bool hide_enoent);
-
-  virtual void add_copyup_ops(librados::ObjectWriteOperation *wr)
-  {
-    add_write_ops(wr);
-  }
-
-  virtual bool should_complete(int r);
-  virtual void send();
-
-  /**
-   * Writes go through the following state machine to deal with
-   * layering and the object map:
-   *
-   *   <start>
-   *      |
-   *      |\
-   *      | \       -or-
-   *      |  ---------------------------------> LIBRBD_AIO_WRITE_PRE
-   *      |                          .                            |
-   *      |                          .                            |
-   *      |                          .                            v
-   *      |                          . . .  . > LIBRBD_AIO_WRITE_FLAT. . .
-   *      |                                                       |      .
-   *      |                                                       |      .
-   *      |                                                       |      .
-   *      v                need copyup   (copyup performs pre)    |      .
-   * LIBRBD_AIO_WRITE_GUARD -----------> LIBRBD_AIO_WRITE_COPYUP  |      .
-   *  .       |                               |        .          |      .
-   *  .       |                               |        .          |      .
-   *  .       |                         /-----/        .          |      .
-   *  .       |                         |              .          |      .
-   *  .       \-------------------\     |     /-------------------/      .
-   *  .                           |     |     |        .                 .
-   *  .                           v     v     v        .                 .
-   *  .                       LIBRBD_AIO_WRITE_POST    .                 .
-   *  .                               |                .                 .
-   *  .                               |  . . . . . . . .                 .
-   *  .                               |  .                               .
-   *  .                               v  v                               .
-   *  . . . . . . . . . . . . . . > <finish> < . . . . . . . . . . . . . .
-   *
-   * The _PRE/_POST states are skipped if the object map is disabled.
-   * The write starts in _WRITE_GUARD or _FLAT depending on whether or not
-   * there is a parent overlap.
-   */
-protected:
-  enum write_state_d {
-    LIBRBD_AIO_WRITE_GUARD,
-    LIBRBD_AIO_WRITE_COPYUP,
-    LIBRBD_AIO_WRITE_FLAT,
-    LIBRBD_AIO_WRITE_PRE,
-    LIBRBD_AIO_WRITE_POST,
-    LIBRBD_AIO_WRITE_ERROR
-  };
-
-  write_state_d m_state;
-  librados::ObjectWriteOperation m_write;
-  uint64_t m_snap_seq;
-  std::vector<librados::snap_t> m_snaps;
-  bool m_object_exist;
-  bool m_guard = true;
-
-  virtual void add_write_ops(librados::ObjectWriteOperation *wr) = 0;
-  virtual void guard_write();
-  virtual bool post_object_map_update() {
-    return false;
-  }
-  virtual void send_write();
-  virtual void send_write_op();
-  virtual void handle_write_guard();
-
-  void send_pre_object_map_update();
-
-private:
-  bool send_post_object_map_update();
-  void send_copyup();
-};
-
-class AioObjectWrite : public AbstractAioObjectWrite {
-public:
-  AioObjectWrite(ImageCtx *ictx, const std::string &oid, uint64_t object_no,
-                 uint64_t object_off, const ceph::bufferlist &data,
-                 const ::SnapContext &snapc, Context *completion,
-                 int op_flags)
-    : AbstractAioObjectWrite(ictx, oid, object_no, object_off, data.length(),
-                             snapc, completion, false),
-      m_write_data(data), m_op_flags(op_flags) {
-  }
-
-  bool is_op_payload_empty() const {
-    return (m_write_data.length() == 0);
-  }
-
-  virtual const char *get_op_type() const {
-    return "write";
-  }
-
-  virtual bool pre_object_map_update(uint8_t *new_state) {
-    *new_state = OBJECT_EXISTS;
-    return true;
-  }
-
-protected:
-  virtual void add_write_ops(librados::ObjectWriteOperation *wr);
-
-  virtual void send_write();
-
-private:
-  ceph::bufferlist m_write_data;
-  int m_op_flags;
-};
-
-class AioObjectRemove : public AbstractAioObjectWrite {
-public:
-  AioObjectRemove(ImageCtx *ictx, const std::string &oid, uint64_t object_no,
-                  const ::SnapContext &snapc, Context *completion)
-    : AbstractAioObjectWrite(ictx, oid, object_no, 0, 0, snapc, completion,
-                             true),
-      m_object_state(OBJECT_NONEXISTENT) {
-  }
-
-  virtual const char* get_op_type() const {
-    if (has_parent()) {
-      return "remove (trunc)";
-    }
-    return "remove";
-  }
-
-  virtual bool pre_object_map_update(uint8_t *new_state) {
-    if (has_parent()) {
-      m_object_state = OBJECT_EXISTS;
-    } else {
-      m_object_state = OBJECT_PENDING;
-    }
-    *new_state = m_object_state;
-    return true;
-  }
-
-  virtual bool post_object_map_update() {
-    if (m_object_state == OBJECT_EXISTS) {
-      return false;
-    }
-    return true;
-  }
-
-  virtual void guard_write();
-  virtual void send_write();
-
-protected:
-  virtual void add_write_ops(librados::ObjectWriteOperation *wr) {
-    if (has_parent()) {
-      wr->truncate(0);
-    } else {
-      wr->remove();
-    }
-  }
-
-private:
-  uint8_t m_object_state;
-};
-
-class AioObjectTrim : public AbstractAioObjectWrite {
-public:
-  // we'd need to only conditionally specify if a post object map
-  // update is needed. pre update is decided as usual (by checking
-  // the state of the object in the map).
-  AioObjectTrim(ImageCtx *ictx, const std::string &oid, uint64_t object_no,
-                const ::SnapContext &snapc, Context *completion,
-                bool post_object_map_update)
-    : AbstractAioObjectWrite(ictx, oid, object_no, 0, 0, snapc, completion,
-                             true), m_post_object_map_update(post_object_map_update) { }
-
-  virtual const char* get_op_type() const {
-    return "remove (trim)";
-  }
-
-  virtual bool pre_object_map_update(uint8_t *new_state) {
-    *new_state = OBJECT_PENDING;
-    return true;
-  }
-
-  virtual bool post_object_map_update() {
-    return m_post_object_map_update;
-  }
-
-protected:
-  virtual void add_write_ops(librados::ObjectWriteOperation *wr) {
-    wr->remove();
-  }
-
-private:
-  bool m_post_object_map_update;
-};
-
-class AioObjectTruncate : public AbstractAioObjectWrite {
-public:
-  AioObjectTruncate(ImageCtx *ictx, const std::string &oid,
-                    uint64_t object_no, uint64_t object_off,
-                    const ::SnapContext &snapc, Context *completion)
-    : AbstractAioObjectWrite(ictx, oid, object_no, object_off, 0, snapc,
-                             completion, true) {
-  }
-
-  virtual const char* get_op_type() const {
-    return "truncate";
-  }
-
-  virtual bool pre_object_map_update(uint8_t *new_state) {
-    if (!m_object_exist && !has_parent())
-      *new_state = OBJECT_NONEXISTENT;
-    else
-      *new_state = OBJECT_EXISTS;
-    return true;
-  }
-
-  virtual void send_write();
-
-protected:
-  virtual void add_write_ops(librados::ObjectWriteOperation *wr) {
-    wr->truncate(m_object_off);
-  }
-};
-
-class AioObjectZero : public AbstractAioObjectWrite {
-public:
-  AioObjectZero(ImageCtx *ictx, const std::string &oid, uint64_t object_no,
-                uint64_t object_off, uint64_t object_len,
-                const ::SnapContext &snapc, Context *completion)
-    : AbstractAioObjectWrite(ictx, oid, object_no, object_off, object_len,
-                             snapc, completion, true) {
-  }
-
-  virtual const char* get_op_type() const {
-    return "zero";
-  }
-
-  virtual bool pre_object_map_update(uint8_t *new_state) {
-    *new_state = OBJECT_EXISTS;
-    return true;
-  }
-
-protected:
-  virtual void add_write_ops(librados::ObjectWriteOperation *wr) {
-    wr->zero(m_object_off, m_object_len);
-  }
-};
-
-} // namespace librbd
-
-extern template class librbd::AioObjectRequest<librbd::ImageCtx>;
-extern template class librbd::AioObjectRead<librbd::ImageCtx>;
-
-#endif // CEPH_LIBRBD_AIO_OBJECT_REQUEST_H
index e1739aa54af90801c19f6ebe2dd5e71d1f6f8fb8..e0fb674d0130d5649b86fe3f53fd344b637bd618 100644 (file)
@@ -4,14 +4,9 @@ add_library(rbd_types STATIC
   WatchNotifyTypes.cc)
 
 set(librbd_internal_srcs
-  AioCompletion.cc
-  AioImageRequest.cc
-  AioImageRequestWQ.cc
-  AioObjectRequest.cc
   AsyncObjectThrottle.cc
   AsyncOperation.cc
   AsyncRequest.cc
-  CopyupRequest.cc
   DiffIterate.cc
   ExclusiveLock.cc
   Group.cc
@@ -30,14 +25,6 @@ set(librbd_internal_srcs
   cache/ImageWriteback.cc
   cache/PassthroughImageCache.cc
   Watcher.cc
-  watcher/Types.cc
-  watcher/RewatchRequest.cc
-  managed_lock/AcquireRequest.cc
-  managed_lock/BreakRequest.cc
-  managed_lock/GetLockerRequest.cc
-  managed_lock/ReleaseRequest.cc
-  managed_lock/ReacquireRequest.cc
-  managed_lock/Utils.cc
   exclusive_lock/AutomaticPolicy.cc
   exclusive_lock/PreAcquireRequest.cc
   exclusive_lock/PostAcquireRequest.cc
@@ -51,6 +38,11 @@ set(librbd_internal_srcs
   image/SetFlagsRequest.cc
   image/SetSnapRequest.cc
   image_watcher/NotifyLockOwner.cc
+  io/AioCompletion.cc
+  io/CopyupRequest.cc
+  io/ImageRequest.cc
+  io/ImageRequestWQ.cc
+  io/ObjectRequest.cc
   journal/RemoveRequest.cc
   journal/CreateRequest.cc
   journal/OpenRequest.cc
@@ -58,6 +50,12 @@ set(librbd_internal_srcs
   journal/Replay.cc
   journal/StandardPolicy.cc
   journal/Utils.cc
+  managed_lock/AcquireRequest.cc
+  managed_lock/BreakRequest.cc
+  managed_lock/GetLockerRequest.cc
+  managed_lock/ReacquireRequest.cc
+  managed_lock/ReleaseRequest.cc
+  managed_lock/Utils.cc
   mirror/DisableRequest.cc
   mirror/EnableRequest.cc
   object_map/CreateRequest.cc
@@ -91,6 +89,8 @@ set(librbd_internal_srcs
   operation/SnapshotUnprotectRequest.cc
   operation/SnapshotLimitRequest.cc
   operation/TrimRequest.cc
+  watcher/RewatchRequest.cc
+  watcher/Types.cc
   ${CMAKE_SOURCE_DIR}/src/common/ContextCompletion.cc)
 
 add_library(rbd_api STATIC librbd.cc)
diff --git a/src/librbd/CopyupRequest.cc b/src/librbd/CopyupRequest.cc
deleted file mode 100644 (file)
index ae3f177..0000000
+++ /dev/null
@@ -1,366 +0,0 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-
-#include "common/ceph_context.h"
-#include "common/dout.h"
-#include "common/errno.h"
-#include "common/Mutex.h"
-
-#include "librbd/AioCompletion.h"
-#include "librbd/AioImageRequest.h"
-#include "librbd/AioObjectRequest.h"
-#include "librbd/AsyncObjectThrottle.h"
-#include "librbd/CopyupRequest.h"
-#include "librbd/ExclusiveLock.h"
-#include "librbd/ImageCtx.h"
-#include "librbd/ObjectMap.h"
-#include "librbd/Utils.h"
-
-#include <boost/bind.hpp>
-#include <boost/lambda/bind.hpp>
-#include <boost/lambda/construct.hpp>
-
-#define dout_subsys ceph_subsys_rbd
-#undef dout_prefix
-#define dout_prefix *_dout << "librbd::CopyupRequest: "
-
-namespace librbd {
-
-namespace {
-
-class UpdateObjectMap : public C_AsyncObjectThrottle<> {
-public:
-  UpdateObjectMap(AsyncObjectThrottle<> &throttle, ImageCtx *image_ctx,
-                  uint64_t object_no, const std::vector<uint64_t> *snap_ids,
-                  size_t snap_id_idx)
-    : C_AsyncObjectThrottle(throttle, *image_ctx),
-      m_object_no(object_no), m_snap_ids(*snap_ids), m_snap_id_idx(snap_id_idx)
-  {
-  }
-
-  int send() override {
-    uint64_t snap_id = m_snap_ids[m_snap_id_idx];
-    if (snap_id == CEPH_NOSNAP) {
-      RWLock::RLocker snap_locker(m_image_ctx.snap_lock);
-      RWLock::WLocker object_map_locker(m_image_ctx.object_map_lock);
-      assert(m_image_ctx.exclusive_lock->is_lock_owner());
-      assert(m_image_ctx.object_map != nullptr);
-      bool sent = m_image_ctx.object_map->aio_update<Context>(
-        CEPH_NOSNAP, m_object_no, OBJECT_EXISTS, {}, this);
-      return (sent ? 0 : 1);
-    }
-
-    uint8_t state = OBJECT_EXISTS;
-    if (m_image_ctx.test_features(RBD_FEATURE_FAST_DIFF) &&
-        m_snap_id_idx + 1 < m_snap_ids.size()) {
-      state = OBJECT_EXISTS_CLEAN;
-    }
-
-    RWLock::RLocker snap_locker(m_image_ctx.snap_lock);
-    RWLock::RLocker object_map_locker(m_image_ctx.object_map_lock);
-    if (m_image_ctx.object_map == nullptr) {
-      return 1;
-    }
-
-    bool sent = m_image_ctx.object_map->aio_update<Context>(
-      snap_id, m_object_no, state, {}, this);
-    assert(sent);
-    return 0;
-  }
-
-private:
-  uint64_t m_object_no;
-  const std::vector<uint64_t> &m_snap_ids;
-  size_t m_snap_id_idx;
-};
-
-} // anonymous namespace
-
-
-CopyupRequest::CopyupRequest(ImageCtx *ictx, const std::string &oid,
-                             uint64_t objectno, Extents &&image_extents)
-  : m_ictx(ictx), m_oid(oid), m_object_no(objectno),
-    m_image_extents(image_extents), m_state(STATE_READ_FROM_PARENT)
-{
-  m_async_op.start_op(*m_ictx);
-}
-
-CopyupRequest::~CopyupRequest() {
-  assert(m_pending_requests.empty());
-  m_async_op.finish_op();
-}
-
-void CopyupRequest::append_request(AioObjectRequest<> *req) {
-  ldout(m_ictx->cct, 20) << __func__ << " " << this << ": " << req << dendl;
-  m_pending_requests.push_back(req);
-}
-
-void CopyupRequest::complete_requests(int r) {
-  while (!m_pending_requests.empty()) {
-    vector<AioObjectRequest<> *>::iterator it = m_pending_requests.begin();
-    AioObjectRequest<> *req = *it;
-    ldout(m_ictx->cct, 20) << __func__ << " completing request " << req
-                           << dendl;
-    req->complete(r);
-    m_pending_requests.erase(it);
-  }
-}
-
-bool CopyupRequest::send_copyup() {
-  bool add_copyup_op = !m_copyup_data.is_zero();
-  bool copy_on_read = m_pending_requests.empty();
-  if (!add_copyup_op && copy_on_read) {
-    // copyup empty object to prevent future CoR attempts
-    m_copyup_data.clear();
-    add_copyup_op = true;
-  }
-
-  ldout(m_ictx->cct, 20) << __func__ << " " << this
-                         << ": oid " << m_oid << dendl;
-  m_state = STATE_COPYUP;
-
-  m_ictx->snap_lock.get_read();
-  ::SnapContext snapc = m_ictx->snapc;
-  m_ictx->snap_lock.put_read();
-
-  std::vector<librados::snap_t> snaps;
-
-  if (!copy_on_read) {
-    m_pending_copyups.inc();
-  }
-
-  int r;
-  if (copy_on_read || (!snapc.snaps.empty() && add_copyup_op)) {
-    assert(add_copyup_op);
-    add_copyup_op = false;
-
-    librados::ObjectWriteOperation copyup_op;
-    copyup_op.exec("rbd", "copyup", m_copyup_data);
-
-    // send only the copyup request with a blank snapshot context so that
-    // all snapshots are detected from the parent for this object.  If
-    // this is a CoW request, a second request will be created for the
-    // actual modification.
-    m_pending_copyups.inc();
-
-    ldout(m_ictx->cct, 20) << __func__ << " " << this << " copyup with "
-                           << "empty snapshot context" << dendl;
-    librados::AioCompletion *comp = util::create_rados_safe_callback(this);
-
-    librados::Rados rados(m_ictx->data_ctx);
-    r = rados.ioctx_create2(m_ictx->data_ctx.get_id(), m_data_ctx);
-    assert(r == 0);
-
-    r = m_data_ctx.aio_operate(m_oid, comp, &copyup_op, 0, snaps);
-    assert(r == 0);
-    comp->release();
-  }
-
-  if (!copy_on_read) {
-    librados::ObjectWriteOperation write_op;
-    if (add_copyup_op) {
-      // CoW did not need to handle existing snapshots
-      write_op.exec("rbd", "copyup", m_copyup_data);
-    }
-
-    // merge all pending write ops into this single RADOS op
-    for (size_t i=0; i<m_pending_requests.size(); ++i) {
-      AioObjectRequest<> *req = m_pending_requests[i];
-      ldout(m_ictx->cct, 20) << __func__ << " add_copyup_ops " << req
-                             << dendl;
-      req->add_copyup_ops(&write_op);
-    }
-    assert(write_op.size() != 0);
-
-    snaps.insert(snaps.end(), snapc.snaps.begin(), snapc.snaps.end());
-    librados::AioCompletion *comp = util::create_rados_safe_callback(this);
-    r = m_ictx->data_ctx.aio_operate(m_oid, comp, &write_op);
-    assert(r == 0);
-    comp->release();
-  }
-  return false;
-}
-
-bool CopyupRequest::is_copyup_required() {
-  bool noop = true;
-  for (const AioObjectRequest<> *req : m_pending_requests) {
-    if (!req->is_op_payload_empty()) {
-      noop = false;
-      break;
-    }
-  }
-
-  return (m_copyup_data.is_zero() && noop);
-}
-
-void CopyupRequest::send()
-{
-  m_state = STATE_READ_FROM_PARENT;
-  AioCompletion *comp = AioCompletion::create_and_start(
-    this, m_ictx, AIO_TYPE_READ);
-
-  ldout(m_ictx->cct, 20) << __func__ << " " << this
-                         << ": completion " << comp
-                         << ", oid " << m_oid
-                         << ", extents " << m_image_extents
-                         << dendl;
-  AioImageRequest<>::aio_read(m_ictx->parent, comp, std::move(m_image_extents),
-                              nullptr, &m_copyup_data, 0);
-}
-
-void CopyupRequest::complete(int r)
-{
-  if (should_complete(r)) {
-    complete_requests(r);
-    delete this;
-  }
-}
-
-bool CopyupRequest::should_complete(int r)
-{
-  CephContext *cct = m_ictx->cct;
-  ldout(cct, 20) << __func__ << " " << this
-                 << ": oid " << m_oid
-                 << ", r " << r << dendl;
-
-  uint64_t pending_copyups;
-  switch (m_state) {
-  case STATE_READ_FROM_PARENT:
-    ldout(cct, 20) << "READ_FROM_PARENT" << dendl;
-    remove_from_list();
-    if (r >= 0 || r == -ENOENT) {
-      if (is_copyup_required()) {
-        ldout(cct, 20) << __func__ << " " << this << " nop, skipping" << dendl;
-        return true;
-      }
-
-      return send_object_map_head();
-    }
-    break;
-
-  case STATE_OBJECT_MAP_HEAD:
-    ldout(cct, 20) << "OBJECT_MAP_HEAD" << dendl;
-    assert(r == 0);
-    return send_object_map();
-
-  case STATE_OBJECT_MAP:
-    ldout(cct, 20) << "OBJECT_MAP" << dendl;
-    assert(r == 0);
-    return send_copyup();
-
-  case STATE_COPYUP:
-    // invoked via a finisher in librados, so thread safe
-    pending_copyups = m_pending_copyups.dec();
-    ldout(cct, 20) << "COPYUP (" << pending_copyups << " pending)"
-                   << dendl;
-    if (r == -ENOENT) {
-      // hide the -ENOENT error if this is the last op
-      if (pending_copyups == 0) {
-        complete_requests(0);
-      }
-    } else if (r < 0) {
-      complete_requests(r);
-    }
-    return (pending_copyups == 0);
-
-  default:
-    lderr(cct) << "invalid state: " << m_state << dendl;
-    assert(false);
-    break;
-  }
-  return (r < 0);
-}
-
-void CopyupRequest::remove_from_list()
-{
-  Mutex::Locker l(m_ictx->copyup_list_lock);
-
-  map<uint64_t, CopyupRequest*>::iterator it =
-    m_ictx->copyup_list.find(m_object_no);
-  assert(it != m_ictx->copyup_list.end());
-  m_ictx->copyup_list.erase(it);
-}
-
-bool CopyupRequest::send_object_map_head() {
-  CephContext *cct = m_ictx->cct;
-  ldout(cct, 20) << __func__ << " " << this << dendl;
-
-  m_state = STATE_OBJECT_MAP_HEAD;
-
-  {
-    RWLock::RLocker owner_locker(m_ictx->owner_lock);
-    RWLock::RLocker snap_locker(m_ictx->snap_lock);
-    if (m_ictx->object_map != nullptr) {
-      bool copy_on_read = m_pending_requests.empty();
-      assert(m_ictx->exclusive_lock->is_lock_owner());
-
-      RWLock::WLocker object_map_locker(m_ictx->object_map_lock);
-      if (!m_ictx->snaps.empty()) {
-        m_snap_ids.insert(m_snap_ids.end(), m_ictx->snaps.begin(),
-                          m_ictx->snaps.end());
-      }
-      if (copy_on_read &&
-          (*m_ictx->object_map)[m_object_no] != OBJECT_EXISTS) {
-        m_snap_ids.insert(m_snap_ids.begin(), CEPH_NOSNAP);
-        object_map_locker.unlock();
-        snap_locker.unlock();
-        owner_locker.unlock();
-        return send_object_map();
-      }
-
-      bool may_update = false;
-      uint8_t new_state, current_state;
-
-      vector<AioObjectRequest<> *>::reverse_iterator r_it = m_pending_requests.rbegin();
-      for (; r_it != m_pending_requests.rend(); ++r_it) {
-        AioObjectRequest<> *req = *r_it;
-        if (!req->pre_object_map_update(&new_state)) {
-          continue;
-        }
-
-        current_state = (*m_ictx->object_map)[m_object_no];
-        ldout(cct, 20) << __func__ << " " << req->get_op_type() << " object no "
-                       << m_object_no << " current state "
-                       << stringify(static_cast<uint32_t>(current_state))
-                       << " new state " << stringify(static_cast<uint32_t>(new_state))
-                       << dendl;
-        may_update = true;
-        break;
-      }
-
-      if (may_update && (new_state != current_state) &&
-          m_ictx->object_map->aio_update<CopyupRequest>(
-            CEPH_NOSNAP, m_object_no, new_state, current_state, this)) {
-        return false;
-      }
-    }
-  }
-
-  return send_object_map();
-}
-
-bool CopyupRequest::send_object_map() {
-  // avoid possible recursive lock attempts
-  if (m_snap_ids.empty()) {
-    // no object map update required
-    return send_copyup();
-  } else {
-    // update object maps for HEAD and all existing snapshots
-    ldout(m_ictx->cct, 20) << __func__ << " " << this
-                           << ": oid " << m_oid << dendl;
-    m_state = STATE_OBJECT_MAP;
-
-    RWLock::RLocker owner_locker(m_ictx->owner_lock);
-    AsyncObjectThrottle<>::ContextFactory context_factory(
-      boost::lambda::bind(boost::lambda::new_ptr<UpdateObjectMap>(),
-      boost::lambda::_1, m_ictx, m_object_no, &m_snap_ids,
-      boost::lambda::_2));
-    AsyncObjectThrottle<> *throttle = new AsyncObjectThrottle<>(
-      NULL, *m_ictx, context_factory, util::create_context_callback(this),
-      NULL, 0, m_snap_ids.size());
-    throttle->start_ops(m_ictx->concurrent_management_ops);
-  }
-  return false;
-}
-
-} // namespace librbd
diff --git a/src/librbd/CopyupRequest.h b/src/librbd/CopyupRequest.h
deleted file mode 100644 (file)
index d661aaf..0000000
+++ /dev/null
@@ -1,97 +0,0 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-
-#ifndef CEPH_LIBRBD_COPYUP_REQUEST_H
-#define CEPH_LIBRBD_COPYUP_REQUEST_H
-
-#include "librbd/AsyncOperation.h"
-#include "include/int_types.h"
-#include "include/buffer.h"
-
-namespace librbd {
-
-struct AioCompletion;
-template <typename I> class AioObjectRequest;
-struct ImageCtx;
-
-class CopyupRequest {
-public:
-  typedef std::vector<std::pair<uint64_t, uint64_t> > Extents;
-
-  CopyupRequest(ImageCtx *ictx, const std::string &oid, uint64_t objectno,
-                Extents &&image_extents);
-  ~CopyupRequest();
-
-  void append_request(AioObjectRequest<ImageCtx> *req);
-
-  void send();
-
-  void complete(int r);
-
-private:
-  /**
-   * Copyup requests go through the following state machine to read from the
-   * parent image, update the object map, and copyup the object:
-   *
-   *
-   * @verbatim
-   *
-   *              <start>
-   *                 |
-   *                 v
-   *    . . .STATE_READ_FROM_PARENT. . .
-   *    . .          |                 .
-   *    . .          v                 .
-   *    . .  STATE_OBJECT_MAP_HEAD     v (copy on read /
-   *    . .          |                 .  no HEAD rev. update)
-   *    v v          v                 .
-   *    . .    STATE_OBJECT_MAP. . . . .
-   *    . .          |
-   *    . .          v
-   *    . . . . > STATE_COPYUP
-   *    .            |
-   *    .            v
-   *    . . . . > <finish>
-   *
-   * @endverbatim
-   *
-   * The _OBJECT_MAP state is skipped if the object map isn't enabled or if
-   * an object map update isn't required. The _COPYUP state is skipped if
-   * no data was read from the parent *and* there are no additional ops.
-   */
-  enum State {
-    STATE_READ_FROM_PARENT,
-    STATE_OBJECT_MAP_HEAD, // only update the HEAD revision
-    STATE_OBJECT_MAP,      // update HEAD+snaps (if any)
-    STATE_COPYUP
-  };
-
-  ImageCtx *m_ictx;
-  std::string m_oid;
-  uint64_t m_object_no;
-  Extents m_image_extents;
-  State m_state;
-  ceph::bufferlist m_copyup_data;
-  vector<AioObjectRequest<ImageCtx> *> m_pending_requests;
-  atomic_t m_pending_copyups;
-
-  AsyncOperation m_async_op;
-
-  std::vector<uint64_t> m_snap_ids;
-  librados::IoCtx m_data_ctx; // for empty SnapContext
-
-  void complete_requests(int r);
-
-  bool should_complete(int r);
-
-  void remove_from_list();
-
-  bool send_object_map_head();
-  bool send_object_map();
-  bool send_copyup();
-  bool is_copyup_required();
-};
-
-} // namespace librbd
-
-#endif // CEPH_LIBRBD_COPYUP_REQUEST_H
index 1a7a76ccb34c5bae735042ed5be6569e0dde5a3d..19ca6e42e1306cf66d9799396f4d1100ab7a9504 100644 (file)
@@ -2,12 +2,12 @@
 // vim: ts=8 sw=2 smarttab
 
 #include "librbd/ExclusiveLock.h"
-#include "librbd/AioImageRequestWQ.h"
 #include "librbd/ImageWatcher.h"
 #include "librbd/ImageState.h"
 #include "librbd/exclusive_lock/PreAcquireRequest.h"
 #include "librbd/exclusive_lock/PostAcquireRequest.h"
 #include "librbd/exclusive_lock/PreReleaseRequest.h"
+#include "librbd/io/ImageRequestWQ.h"
 #include "librbd/Utils.h"
 #include "common/Mutex.h"
 #include "common/dout.h"
@@ -83,8 +83,8 @@ void ExclusiveLock<I>::init(uint64_t features, Context *on_init) {
     ML<I>::set_state_initializing();
   }
 
-  m_image_ctx.aio_work_queue->block_writes(new C_InitComplete(this, features,
-                                                              on_init));
+  m_image_ctx.io_work_queue->block_writes(new C_InitComplete(this, features,
+                                                             on_init));
 }
 
 template <typename I>
@@ -116,7 +116,7 @@ void ExclusiveLock<I>::handle_init_complete(uint64_t features) {
   ldout(m_image_ctx.cct, 10) << "features=" << features << dendl;
 
   if ((features & RBD_FEATURE_JOURNALING) != 0) {
-    m_image_ctx.aio_work_queue->set_require_lock_on_read();
+    m_image_ctx.io_work_queue->set_require_lock_on_read();
   }
 
   Mutex::Locker locker(ML<I>::m_lock);
@@ -129,11 +129,11 @@ void ExclusiveLock<I>::shutdown_handler(int r, Context *on_finish) {
 
   {
     RWLock::WLocker owner_locker(m_image_ctx.owner_lock);
-    m_image_ctx.aio_work_queue->clear_require_lock_on_read();
+    m_image_ctx.io_work_queue->clear_require_lock_on_read();
     m_image_ctx.exclusive_lock = nullptr;
   }
 
-  m_image_ctx.aio_work_queue->unblock_writes();
+  m_image_ctx.io_work_queue->unblock_writes();
   m_image_ctx.image_watcher->flush(on_finish);
 }
 
@@ -237,8 +237,8 @@ void ExclusiveLock<I>::handle_post_acquired_lock(int r) {
 
   if (r >= 0) {
     m_image_ctx.image_watcher->notify_acquired_lock();
-    m_image_ctx.aio_work_queue->clear_require_lock_on_read();
-    m_image_ctx.aio_work_queue->unblock_writes();
+    m_image_ctx.io_work_queue->clear_require_lock_on_read();
+    m_image_ctx.io_work_queue->unblock_writes();
   }
 
   on_finish->complete(r);
@@ -270,7 +270,7 @@ void ExclusiveLock<I>::post_release_lock_handler(bool shutting_down, int r,
 
     if (r >= 0) {
       m_image_ctx.image_watcher->notify_released_lock();
-      if (m_image_ctx.aio_work_queue->is_lock_request_needed()) {
+      if (m_image_ctx.io_work_queue->is_lock_request_needed()) {
         // if we have blocked IO -- re-request the lock
         RWLock::RLocker owner_locker(m_image_ctx.owner_lock);
         ML<I>::acquire_lock(nullptr);
@@ -279,12 +279,12 @@ void ExclusiveLock<I>::post_release_lock_handler(bool shutting_down, int r,
   } else {
     {
       RWLock::WLocker owner_locker(m_image_ctx.owner_lock);
-      m_image_ctx.aio_work_queue->clear_require_lock_on_read();
+      m_image_ctx.io_work_queue->clear_require_lock_on_read();
       m_image_ctx.exclusive_lock = nullptr;
     }
 
     if (r >= 0) {
-      m_image_ctx.aio_work_queue->unblock_writes();
+      m_image_ctx.io_work_queue->unblock_writes();
     }
 
     m_image_ctx.image_watcher->notify_released_lock();
index fd37e9a349a8a16a5192107e533e5a7753cc2644..8e3d82b2e8cccd836bc4f34e22ee3865b309f7a2 100644 (file)
@@ -3,10 +3,10 @@
 
 #include "common/errno.h"
 
-#include "librbd/AioCompletion.h"
 #include "librbd/Group.h"
 #include "librbd/ImageState.h"
 #include "librbd/Utils.h"
+#include "librbd/io/AioCompletion.h"
 
 #define dout_subsys ceph_subsys_rbd
 #undef dout_prefix
index 9dcc9eb508f4dd663cd0ea1bee7563eef2278bce..87b638b9c8c9f17c69540cf49f1bf2404d630b44 100644 (file)
@@ -1,10 +1,18 @@
 // -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
 // vim: ts=8 sw=2 smarttab
+
 #ifndef CEPH_LIBRBD_GROUP_H
 #define CEPH_LIBRBD_GROUP_H
 
+#include "include/rados/librados.hpp"
+#include "include/rbd/librbd.hpp"
+#include <string>
+#include <vector>
+
 namespace librbd {
 
+struct ImageCtx;
+
 // Consistency groups functions
 int group_create(librados::IoCtx& io_ctx, const char *imgname);
 int group_remove(librados::IoCtx& io_ctx, const char *group_name);
@@ -16,6 +24,7 @@ int group_image_remove(librados::IoCtx& group_ioctx, const char *group_name,
 int group_image_list(librados::IoCtx& group_ioctx, const char *group_name,
                     std::vector<group_image_status_t> *images);
 int image_get_group(ImageCtx *ictx, group_spec_t *group_spec);
-}
+
+} // namespace librbd
 
 #endif // CEPH_LIBRBD_GROUP_H
index fa6724d4b85cf8cd11170bad503fda9759975cb3..d16c0a5725617757f6ddb9c4aa18d36915178ebc 100644 (file)
 #include "common/WorkQueue.h"
 #include "common/Timer.h"
 
-#include "librbd/AioImageRequestWQ.h"
-#include "librbd/AioCompletion.h"
 #include "librbd/AsyncOperation.h"
 #include "librbd/AsyncRequest.h"
 #include "librbd/ExclusiveLock.h"
-#include "librbd/exclusive_lock/AutomaticPolicy.h"
-#include "librbd/exclusive_lock/StandardPolicy.h"
 #include "librbd/internal.h"
 #include "librbd/ImageCtx.h"
 #include "librbd/ImageState.h"
 #include "librbd/ImageWatcher.h"
 #include "librbd/Journal.h"
-#include "librbd/journal/StandardPolicy.h"
 #include "librbd/LibrbdAdminSocketHook.h"
 #include "librbd/ObjectMap.h"
 #include "librbd/Operations.h"
 #include "librbd/operation/ResizeRequest.h"
 #include "librbd/Utils.h"
 #include "librbd/LibrbdWriteback.h"
+#include "librbd/exclusive_lock/AutomaticPolicy.h"
+#include "librbd/exclusive_lock/StandardPolicy.h"
+#include "librbd/io/AioCompletion.h"
+#include "librbd/io/ImageRequestWQ.h"
+#include "librbd/journal/StandardPolicy.h"
 
 #include "osdc/Striper.h"
 #include <boost/bind.hpp>
@@ -189,7 +189,7 @@ struct C_InvalidateCache : public Context {
       state(new ImageState<>(this)),
       operations(new Operations<>(*this)),
       exclusive_lock(nullptr), object_map(nullptr),
-      aio_work_queue(nullptr), op_work_queue(nullptr),
+      io_work_queue(nullptr), op_work_queue(nullptr),
       asok_hook(nullptr)
   {
     md_ctx.dup(p);
@@ -200,9 +200,9 @@ struct C_InvalidateCache : public Context {
     memset(&header, 0, sizeof(header));
 
     ThreadPool *thread_pool_singleton = get_thread_pool_instance(cct);
-    aio_work_queue = new AioImageRequestWQ(this, "librbd::aio_work_queue",
-                                           cct->_conf->rbd_op_thread_timeout,
-                                           thread_pool_singleton);
+    io_work_queue = new io::ImageRequestWQ(
+      this, "librbd::io_work_queue", cct->_conf->rbd_op_thread_timeout,
+      thread_pool_singleton);
     op_work_queue = new ContextWQ("librbd::op_work_queue",
                                   cct->_conf->rbd_op_thread_timeout,
                                   thread_pool_singleton);
@@ -242,12 +242,12 @@ struct C_InvalidateCache : public Context {
     md_ctx.aio_flush();
     data_ctx.aio_flush();
     op_work_queue->drain();
-    aio_work_queue->drain();
+    io_work_queue->drain();
 
     delete journal_policy;
     delete exclusive_lock_policy;
     delete op_work_queue;
-    delete aio_work_queue;
+    delete io_work_queue;
     delete operations;
     delete state;
   }
index f8c5b0c4288a7a73e46e9b615182e359d24f28dd..0f089c1683118d02546e2cacdc4245f68106ddc0 100644 (file)
@@ -38,10 +38,7 @@ class SafeTimer;
 
 namespace librbd {
 
-  class AioCompletion;
-  class AioImageRequestWQ;
   class AsyncOperation;
-  class CopyupRequest;
   template <typename> class ExclusiveLock;
   template <typename> class ImageState;
   template <typename> class ImageWatcher;
@@ -53,6 +50,11 @@ namespace librbd {
 
   namespace cache { struct ImageCache; }
   namespace exclusive_lock { struct Policy; }
+  namespace io {
+  class AioCompletion;
+  class ImageRequestWQ;
+  class CopyupRequest;
+  }
   namespace journal { struct Policy; }
 
   namespace operation {
@@ -136,7 +138,7 @@ namespace librbd {
     Readahead readahead;
     uint64_t total_bytes_read;
 
-    std::map<uint64_t, CopyupRequest*> copyup_list;
+    std::map<uint64_t, io::CopyupRequest*> copyup_list;
 
     xlist<AsyncOperation*> async_ops;
     xlist<AsyncRequest<>*> async_requests;
@@ -150,8 +152,8 @@ namespace librbd {
 
     xlist<operation::ResizeRequest<ImageCtx>*> resize_reqs;
 
-    AioImageRequestWQ *aio_work_queue;
-    xlist<AioCompletion*> completed_reqs;
+    io::ImageRequestWQ *io_work_queue;
+    xlist<io::AioCompletion*> completed_reqs;
     EventSocket event_socket;
 
     ContextWQ *op_work_queue;
index 27f76509519807516d54cf6cb5cf6bb87855f252..b38d0821174970ff95b98b6ecdf53b522da103eb 100644 (file)
@@ -2,7 +2,6 @@
 // vim: ts=8 sw=2 smarttab
 
 #include "librbd/ImageWatcher.h"
-#include "librbd/AioCompletion.h"
 #include "librbd/ExclusiveLock.h"
 #include "librbd/ImageCtx.h"
 #include "librbd/ImageState.h"
@@ -12,6 +11,7 @@
 #include "librbd/Utils.h"
 #include "librbd/exclusive_lock/Policy.h"
 #include "librbd/image_watcher/NotifyLockOwner.h"
+#include "librbd/io/AioCompletion.h"
 #include "include/encoding.h"
 #include "common/errno.h"
 #include "common/WorkQueue.h"
index ec0d7cb5125e525da5b7537cdb1fac2238174129..b959bbd72816b5f6b54f5ffc06b9bd6f671cf36d 100644 (file)
@@ -2,13 +2,8 @@
 // vim: ts=8 sw=2 smarttab
 
 #include "librbd/Journal.h"
-#include "librbd/AioImageRequestWQ.h"
-#include "librbd/AioObjectRequest.h"
 #include "librbd/ExclusiveLock.h"
 #include "librbd/ImageCtx.h"
-#include "librbd/journal/OpenRequest.h"
-#include "librbd/journal/PromoteRequest.h"
-#include "librbd/journal/Replay.h"
 #include "cls/journal/cls_journal_types.h"
 #include "journal/Journaler.h"
 #include "journal/Policy.h"
 #include "common/Timer.h"
 #include "common/WorkQueue.h"
 #include "include/rados/librados.hpp"
-#include "librbd/journal/RemoveRequest.h"
+#include "librbd/io/ImageRequestWQ.h"
+#include "librbd/io/ObjectRequest.h"
 #include "librbd/journal/CreateRequest.h"
+#include "librbd/journal/OpenRequest.h"
+#include "librbd/journal/PromoteRequest.h"
+#include "librbd/journal/RemoveRequest.h"
+#include "librbd/journal/Replay.h"
 
 #include <boost/scope_exit.hpp>
 #include <utility>
@@ -828,7 +828,7 @@ void Journal<I>::flush_commit_position(Context *on_finish) {
 template <typename I>
 uint64_t Journal<I>::append_write_event(uint64_t offset, size_t length,
                                         const bufferlist &bl,
-                                        const AioObjectRequests &requests,
+                                        const IOObjectRequests &requests,
                                         bool flush_entry) {
   assert(m_max_append_size > journal::AioWriteEvent::get_fixed_size());
   uint64_t max_write_data_size =
@@ -861,7 +861,7 @@ uint64_t Journal<I>::append_write_event(uint64_t offset, size_t length,
 
 template <typename I>
 uint64_t Journal<I>::append_io_event(journal::EventEntry &&event_entry,
-                                     const AioObjectRequests &requests,
+                                     const IOObjectRequests &requests,
                                      uint64_t offset, size_t length,
                                      bool flush_entry) {
   bufferlist bl;
@@ -874,7 +874,7 @@ uint64_t Journal<I>::append_io_event(journal::EventEntry &&event_entry,
 template <typename I>
 uint64_t Journal<I>::append_io_events(journal::EventType event_type,
                                       const Bufferlists &bufferlists,
-                                      const AioObjectRequests &requests,
+                                      const IOObjectRequests &requests,
                                       uint64_t offset, size_t length,
                                       bool flush_entry) {
   assert(!bufferlists.empty());
@@ -1547,7 +1547,7 @@ void Journal<I>::handle_io_event_safe(int r, uint64_t tid) {
                << "failed to commit IO event: "  << cpp_strerror(r) << dendl;
   }
 
-  AioObjectRequests aio_object_requests;
+  IOObjectRequests aio_object_requests;
   Contexts on_safe_contexts;
   {
     Mutex::Locker event_locker(m_event_lock);
@@ -1576,7 +1576,7 @@ void Journal<I>::handle_io_event_safe(int r, uint64_t tid) {
 
   ldout(cct, 20) << this << " " << __func__ << ": "
                  << "completing tid=" << tid << dendl;
-  for (AioObjectRequests::iterator it = aio_object_requests.begin();
+  for (IOObjectRequests::iterator it = aio_object_requests.begin();
        it != aio_object_requests.end(); ++it) {
     if (r < 0) {
       // don't send aio requests if the journal fails -- bubble error up
index a7fa1a5e5c4afb7c8d5366e44f168bda67af390a..8c5084aa1522e432933eaa847ac0e796b7e0cce4 100644 (file)
@@ -34,9 +34,9 @@ namespace librados {
 
 namespace librbd {
 
-struct AioObjectRequestHandle;
 class ImageCtx;
 
+namespace io { struct ObjectRequestHandle; }
 namespace journal { template <typename> class Replay; }
 
 template <typename ImageCtxT = ImageCtx>
@@ -90,7 +90,7 @@ public:
   static const std::string LOCAL_MIRROR_UUID;
   static const std::string ORPHAN_MIRROR_UUID;
 
-  typedef std::list<AioObjectRequestHandle *> AioObjectRequests;
+  typedef std::list<io::ObjectRequestHandle *> IOObjectRequests;
 
   Journal(ImageCtxT &image_ctx);
   ~Journal();
@@ -139,10 +139,10 @@ public:
 
   uint64_t append_write_event(uint64_t offset, size_t length,
                               const bufferlist &bl,
-                              const AioObjectRequests &requests,
+                              const IOObjectRequests &requests,
                               bool flush_entry);
   uint64_t append_io_event(journal::EventEntry &&event_entry,
-                           const AioObjectRequests &requests,
+                           const IOObjectRequests &requests,
                            uint64_t offset, size_t length,
                            bool flush_entry);
   void commit_io_event(uint64_t tid, int r);
@@ -192,7 +192,7 @@ private:
 
   struct Event {
     Futures futures;
-    AioObjectRequests aio_object_requests;
+    IOObjectRequests aio_object_requests;
     Contexts on_safe_contexts;
     ExtentInterval pending_extents;
     bool committed_io = false;
@@ -201,7 +201,7 @@ private:
 
     Event() {
     }
-    Event(const Futures &_futures, const AioObjectRequests &_requests,
+    Event(const Futures &_futures, const IOObjectRequests &_requests,
           uint64_t offset, size_t length)
       : futures(_futures), aio_object_requests(_requests) {
       if (length > 0) {
@@ -335,7 +335,7 @@ private:
 
   uint64_t append_io_events(journal::EventType event_type,
                             const Bufferlists &bufferlists,
-                            const AioObjectRequests &requests,
+                            const IOObjectRequests &requests,
                             uint64_t offset, size_t length, bool flush_entry);
   Future wait_event(Mutex &lock, uint64_t tid, Context *on_safe);
 
index b1e3826df4d34b263afa379d8c03e8fc26dcf057..5e53bdad0576d99ef7cf258d9b0512a39a19802d 100644 (file)
 #include "include/rados/librados.hpp"
 #include "include/rbd/librbd.hpp"
 
-#include "librbd/AioObjectRequest.h"
 #include "librbd/ExclusiveLock.h"
 #include "librbd/ImageCtx.h"
 #include "librbd/internal.h"
 #include "librbd/LibrbdWriteback.h"
-#include "librbd/AioCompletion.h"
 #include "librbd/ObjectMap.h"
 #include "librbd/Journal.h"
 #include "librbd/Utils.h"
+#include "librbd/io/AioCompletion.h"
+#include "librbd/io/ObjectRequest.h"
 
 #include "include/assert.h"
 
@@ -161,8 +161,8 @@ namespace librbd {
       assert(image_ctx->exclusive_lock->is_lock_owner());
 
       request_sent = true;
-      AioObjectWrite *req = new AioObjectWrite(image_ctx, oid, object_no, off,
-                                               bl, snapc, this, 0);
+      auto req = new io::ObjectWriteRequest(image_ctx, oid, object_no, off,
+                                            bl, snapc, this, 0);
       req->send();
     }
   };
@@ -271,8 +271,8 @@ namespace librbd {
                                               bl, snapc, req_comp,
                                              journal_tid));
     } else {
-      AioObjectWrite *req = new AioObjectWrite(m_ictx, oid.name, object_no,
-                                              off, bl, snapc, req_comp, 0);
+      auto req = new io::ObjectWriteRequest(m_ictx, oid.name, object_no,
+                                           off, bl, snapc, req_comp, 0);
       req->send();
     }
     return ++m_tid;
index a787f89472f70760fe0b6ea7a8b60e3517189d82..1fd167ce793c8ec4b56b4622e8106d8a80915351 100644 (file)
@@ -4,9 +4,9 @@
 #include "ImageWriteback.h"
 #include "include/buffer.h"
 #include "common/dout.h"
-#include "librbd/AioCompletion.h"
-#include "librbd/AioImageRequest.h"
 #include "librbd/ImageCtx.h"
+#include "librbd/io/AioCompletion.h"
+#include "librbd/io/ImageRequest.h"
 
 #define dout_subsys ceph_subsys_rbd
 #undef dout_prefix
@@ -26,11 +26,10 @@ void ImageWriteback<I>::aio_read(Extents &&image_extents, bufferlist *bl,
   ldout(cct, 20) << "image_extents=" << image_extents << ", "
                  << "on_finish=" << on_finish << dendl;
 
-  AioCompletion *aio_comp = AioCompletion::create_and_start(on_finish,
-                                                            &m_image_ctx,
-                                                            AIO_TYPE_READ);
-  AioImageRead<I> req(m_image_ctx, aio_comp, std::move(image_extents), nullptr,
-                      bl, fadvise_flags);
+  auto aio_comp = io::AioCompletion::create_and_start(on_finish, &m_image_ctx,
+                                                      io::AIO_TYPE_READ);
+  io::ImageReadRequest<I> req(m_image_ctx, aio_comp, std::move(image_extents),
+                              nullptr, bl, fadvise_flags);
   req.set_bypass_image_cache();
   req.send();
 }
@@ -43,11 +42,10 @@ void ImageWriteback<I>::aio_write(Extents &&image_extents,
   ldout(cct, 20) << "image_extents=" << image_extents << ", "
                  << "on_finish=" << on_finish << dendl;
 
-  AioCompletion *aio_comp = AioCompletion::create_and_start(on_finish,
-                                                            &m_image_ctx,
-                                                            AIO_TYPE_WRITE);
-  AioImageWrite<I> req(m_image_ctx, aio_comp, std::move(image_extents),
-                       std::move(bl), fadvise_flags);
+  auto aio_comp = io::AioCompletion::create_and_start(on_finish, &m_image_ctx,
+                                                      io::AIO_TYPE_WRITE);
+  io::ImageWriteRequest<I> req(m_image_ctx, aio_comp, std::move(image_extents),
+                               std::move(bl), fadvise_flags);
   req.set_bypass_image_cache();
   req.send();
 }
@@ -60,10 +58,9 @@ void ImageWriteback<I>::aio_discard(uint64_t offset, uint64_t length,
                  << "length=" << length << ", "
                 << "on_finish=" << on_finish << dendl;
 
-  AioCompletion *aio_comp = AioCompletion::create_and_start(on_finish,
-                                                            &m_image_ctx,
-                                                            AIO_TYPE_DISCARD);
-  AioImageDiscard<I> req(m_image_ctx, aio_comp, offset, length);
+  auto aio_comp = io::AioCompletion::create_and_start(on_finish, &m_image_ctx,
+                                                      io::AIO_TYPE_DISCARD);
+  io::ImageDiscardRequest<I> req(m_image_ctx, aio_comp, offset, length);
   req.set_bypass_image_cache();
   req.send();
 }
@@ -73,10 +70,9 @@ void ImageWriteback<I>::aio_flush(Context *on_finish) {
   CephContext *cct = m_image_ctx.cct;
   ldout(cct, 20) << "on_finish=" << on_finish << dendl;
 
-  AioCompletion *aio_comp = AioCompletion::create_and_start(on_finish,
-                                                            &m_image_ctx,
-                                                            AIO_TYPE_FLUSH);
-  AioImageFlush<I> req(m_image_ctx, aio_comp);
+  auto aio_comp = io::AioCompletion::create_and_start(on_finish, &m_image_ctx,
+                                                      io::AIO_TYPE_FLUSH);
+  io::ImageFlushRequest<I> req(m_image_ctx, aio_comp);
   req.set_bypass_image_cache();
   req.send();
 }
index 6d3382dd699a7499e5e1fb3c426b6a14d3d605a7..ee29a98584f587ceb7391aff6c2f2c12e8973a19 100644 (file)
@@ -4,13 +4,13 @@
 #include "librbd/exclusive_lock/PreReleaseRequest.h"
 #include "common/dout.h"
 #include "common/errno.h"
-#include "librbd/AioImageRequestWQ.h"
 #include "librbd/ExclusiveLock.h"
 #include "librbd/ImageState.h"
 #include "librbd/ImageWatcher.h"
 #include "librbd/Journal.h"
 #include "librbd/ObjectMap.h"
 #include "librbd/Utils.h"
+#include "librbd/io/ImageRequestWQ.h"
 
 #define dout_subsys ceph_subsys_rbd
 #undef dout_prefix
@@ -108,9 +108,9 @@ void PreReleaseRequest<I>::send_block_writes() {
   {
     RWLock::RLocker owner_locker(m_image_ctx.owner_lock);
     if (m_image_ctx.test_features(RBD_FEATURE_JOURNALING)) {
-      m_image_ctx.aio_work_queue->set_require_lock_on_read();
+      m_image_ctx.io_work_queue->set_require_lock_on_read();
     }
-    m_image_ctx.aio_work_queue->block_writes(ctx);
+    m_image_ctx.io_work_queue->block_writes(ctx);
   }
 }
 
@@ -125,7 +125,7 @@ void PreReleaseRequest<I>::handle_block_writes(int r) {
                << dendl;
   } else if (r < 0) {
     lderr(cct) << "failed to block writes: " << cpp_strerror(r) << dendl;
-    m_image_ctx.aio_work_queue->unblock_writes();
+    m_image_ctx.io_work_queue->unblock_writes();
     save_result(r);
     finish();
     return;
@@ -168,7 +168,7 @@ void PreReleaseRequest<I>::handle_invalidate_cache(int r) {
   } else if (r < 0 && r != -EBUSY) {
     lderr(cct) << "failed to invalidate cache: " << cpp_strerror(r)
                << dendl;
-    m_image_ctx.aio_work_queue->unblock_writes();
+    m_image_ctx.io_work_queue->unblock_writes();
     save_result(r);
     finish();
     return;
index fe87f9a6f016efa2462fba31b5b0087ec808137f..900164fccabd887fa35b45bb8375c2061021262e 100644 (file)
@@ -4,13 +4,13 @@
 #include "librbd/image/CloseRequest.h"
 #include "common/dout.h"
 #include "common/errno.h"
-#include "librbd/AioImageRequestWQ.h"
 #include "librbd/ExclusiveLock.h"
 #include "librbd/ImageCtx.h"
 #include "librbd/ImageState.h"
 #include "librbd/ImageWatcher.h"
 #include "librbd/ObjectMap.h"
 #include "librbd/Utils.h"
+#include "librbd/io/ImageRequestWQ.h"
 
 #define dout_subsys ceph_subsys_rbd
 #undef dout_prefix
@@ -61,7 +61,7 @@ void CloseRequest<I>::handle_shut_down_update_watchers(int r) {
 template <typename I>
 void CloseRequest<I>::send_unregister_image_watcher() {
   if (m_image_ctx->image_watcher == nullptr) {
-    send_shut_down_aio_queue();
+    send_shut_down_io_queue();
     return;
   }
 
@@ -84,21 +84,21 @@ void CloseRequest<I>::handle_unregister_image_watcher(int r) {
                << dendl;
   }
 
-  send_shut_down_aio_queue();
+  send_shut_down_io_queue();
 }
 
 template <typename I>
-void CloseRequest<I>::send_shut_down_aio_queue() {
+void CloseRequest<I>::send_shut_down_io_queue() {
   CephContext *cct = m_image_ctx->cct;
   ldout(cct, 10) << this << " " << __func__ << dendl;
 
   RWLock::RLocker owner_locker(m_image_ctx->owner_lock);
-  m_image_ctx->aio_work_queue->shut_down(create_context_callback<
-    CloseRequest<I>, &CloseRequest<I>::handle_shut_down_aio_queue>(this));
+  m_image_ctx->io_work_queue->shut_down(create_context_callback<
+    CloseRequest<I>, &CloseRequest<I>::handle_shut_down_io_queue>(this));
 }
 
 template <typename I>
-void CloseRequest<I>::handle_shut_down_aio_queue(int r) {
+void CloseRequest<I>::handle_shut_down_io_queue(int r) {
   CephContext *cct = m_image_ctx->cct;
   ldout(cct, 10) << this << " " << __func__ << ": r=" << r << dendl;
 
index e7d21845b7a7e2190fcee9f6e84fcb861f1635ff..c5cf2e1735e34c2dd498983dfd01aa47b2f5a7d6 100644 (file)
@@ -80,8 +80,8 @@ private:
   void send_unregister_image_watcher();
   void handle_unregister_image_watcher(int r);
 
-  void send_shut_down_aio_queue();
-  void handle_shut_down_aio_queue(int r);
+  void send_shut_down_io_queue();
+  void handle_shut_down_io_queue(int r);
 
   void send_shut_down_exclusive_lock();
   void handle_shut_down_exclusive_lock(int r);
index 63d9a06fa64d5962bc121ae05f1594bcc3d875ee..a453422fd5d66760c442068875b8fdd4c0c46c06 100644 (file)
@@ -8,12 +8,12 @@
 #include "include/assert.h"
 #include "librbd/Utils.h"
 #include "common/ceph_context.h"
-#include "librbd/AioCompletion.h"
 #include "librbd/Journal.h"
 #include "librbd/MirroringWatcher.h"
 #include "librbd/journal/CreateRequest.h"
 #include "librbd/journal/RemoveRequest.h"
 #include "librbd/mirror/EnableRequest.h"
+#include "librbd/io/AioCompletion.h"
 #include "journal/Journaler.h"
 
 #define dout_subsys ceph_subsys_rbd
index dabd094c6e947ac97d650d3043bb49dd752c29d0..bf64f9590c99d5a993792a9f306badce6601ce6d 100644 (file)
@@ -6,13 +6,13 @@
 #include "common/errno.h"
 #include "cls/lock/cls_lock_client.h"
 #include "cls/rbd/cls_rbd_client.h"
-#include "librbd/AioImageRequestWQ.h"
 #include "librbd/ExclusiveLock.h"
 #include "librbd/ImageCtx.h"
 #include "librbd/Journal.h"
 #include "librbd/ObjectMap.h"
 #include "librbd/Utils.h"
 #include "librbd/image/RefreshParentRequest.h"
+#include "librbd/io/ImageRequestWQ.h"
 #include "librbd/journal/Policy.h"
 
 #define dout_subsys ceph_subsys_rbd
@@ -644,7 +644,7 @@ void RefreshRequest<I>::send_v2_open_journal() {
         !journal_disabled_by_policy &&
         m_image_ctx.exclusive_lock != nullptr &&
         m_image_ctx.journal == nullptr) {
-      m_image_ctx.aio_work_queue->set_require_lock_on_read();
+      m_image_ctx.io_work_queue->set_require_lock_on_read();
     }
     send_v2_block_writes();
     return;
@@ -704,7 +704,7 @@ void RefreshRequest<I>::send_v2_block_writes() {
     RefreshRequest<I>, &RefreshRequest<I>::handle_v2_block_writes>(this);
 
   RWLock::RLocker owner_locker(m_image_ctx.owner_lock);
-  m_image_ctx.aio_work_queue->block_writes(ctx);
+  m_image_ctx.io_work_queue->block_writes(ctx);
 }
 
 template <typename I>
@@ -906,7 +906,7 @@ Context *RefreshRequest<I>::handle_v2_close_journal(int *result) {
   assert(m_blocked_writes);
   m_blocked_writes = false;
 
-  m_image_ctx.aio_work_queue->unblock_writes();
+  m_image_ctx.io_work_queue->unblock_writes();
   return send_v2_close_object_map();
 }
 
@@ -1083,7 +1083,7 @@ void RefreshRequest<I>::apply() {
       if (!m_image_ctx.test_features(RBD_FEATURE_JOURNALING,
                                      m_image_ctx.snap_lock)) {
         if (m_image_ctx.journal != nullptr) {
-          m_image_ctx.aio_work_queue->clear_require_lock_on_read();
+          m_image_ctx.io_work_queue->clear_require_lock_on_read();
         }
         std::swap(m_journal, m_image_ctx.journal);
       } else if (m_journal != nullptr) {
index def86f99a5d8e89f1ede452f90679137fad4f804..1b851eb748719b94f23010d9cb30bc65b971f12f 100644 (file)
@@ -4,12 +4,12 @@
 #include "librbd/image/SetSnapRequest.h"
 #include "common/dout.h"
 #include "common/errno.h"
-#include "librbd/AioImageRequestWQ.h"
 #include "librbd/ExclusiveLock.h"
 #include "librbd/ImageCtx.h"
 #include "librbd/ObjectMap.h"
 #include "librbd/Utils.h"
 #include "librbd/image/RefreshParentRequest.h"
+#include "librbd/io/ImageRequestWQ.h"
 
 #define dout_subsys ceph_subsys_rbd
 #undef dout_prefix
@@ -104,7 +104,7 @@ void SetSnapRequest<I>::send_block_writes() {
     klass, &klass::handle_block_writes>(this);
 
   RWLock::RLocker owner_locker(m_image_ctx.owner_lock);
-  m_image_ctx.aio_work_queue->block_writes(ctx);
+  m_image_ctx.io_work_queue->block_writes(ctx);
 }
 
 template <typename I>
@@ -348,7 +348,7 @@ int SetSnapRequest<I>::apply() {
 template <typename I>
 void SetSnapRequest<I>::finalize() {
   if (m_writes_blocked) {
-    m_image_ctx.aio_work_queue->unblock_writes();
+    m_image_ctx.io_work_queue->unblock_writes();
     m_writes_blocked = false;
   }
 }
index 67dfbea5debe2d82849e6cc974f1fa162b11499b..d7125b37dce31bdb1a7b7a35757a6f3a3f26d773 100644 (file)
 #include "cls/journal/cls_journal_types.h"
 #include "cls/journal/cls_journal_client.h"
 
-#include "librbd/AioCompletion.h"
-#include "librbd/AioImageRequest.h"
-#include "librbd/AioImageRequestWQ.h"
-#include "librbd/AioObjectRequest.h"
-#include "librbd/image/CreateRequest.h"
 #include "librbd/DiffIterate.h"
 #include "librbd/ExclusiveLock.h"
 #include "librbd/ImageCtx.h"
 #include "librbd/ImageState.h"
 #include "librbd/internal.h"
 #include "librbd/Journal.h"
-#include "librbd/journal/Types.h"
-#include "librbd/managed_lock/Types.h"
-#include "librbd/mirror/DisableRequest.h"
-#include "librbd/mirror/EnableRequest.h"
 #include "librbd/MirroringWatcher.h"
 #include "librbd/ObjectMap.h"
 #include "librbd/Operations.h"
 #include "librbd/Utils.h"
 #include "librbd/exclusive_lock/AutomaticPolicy.h"
 #include "librbd/exclusive_lock/StandardPolicy.h"
+#include "librbd/image/CreateRequest.h"
+#include "librbd/managed_lock/Types.h"
+#include "librbd/mirror/DisableRequest.h"
+#include "librbd/mirror/EnableRequest.h"
+#include "librbd/io/AioCompletion.h"
+#include "librbd/io/ImageRequest.h"
+#include "librbd/io/ImageRequestWQ.h"
+#include "librbd/io/ObjectRequest.h"
+#include "librbd/journal/Types.h"
 #include "librbd/operation/TrimRequest.h"
 
 #include "journal/Journaler.h"
@@ -2088,12 +2088,12 @@ void filter_out_mirror_watchers(ImageCtx *ictx,
       }
 
       Context *ctx = new C_CopyWrite(m_throttle, m_bl);
-      AioCompletion *comp = AioCompletion::create(ctx);
+      auto comp = io::AioCompletion::create(ctx);
 
       // coordinate through AIO WQ to ensure lock is acquired if needed
-      m_dest->aio_work_queue->aio_write(comp, m_offset, m_bl->length(),
-                                        m_bl->c_str(),
-                                        LIBRADOS_OP_FLAG_FADVISE_DONTNEED);
+      m_dest->io_work_queue->aio_write(comp, m_offset, m_bl->length(),
+                                       m_bl->c_str(),
+                                       LIBRADOS_OP_FLAG_FADVISE_DONTNEED);
     }
 
   private:
@@ -2146,10 +2146,10 @@ void filter_out_mirror_watchers(ImageCtx *ictx,
       uint64_t len = min(period, src_size - offset);
       bufferlist *bl = new bufferlist();
       Context *ctx = new C_CopyRead(&throttle, dest, offset, bl);
-      AioCompletion *comp = AioCompletion::create_and_start(ctx, src,
-                                                            AIO_TYPE_READ);
-      AioImageRequest<>::aio_read(src, comp, {{offset, len}}, nullptr, bl,
-                                  fadvise_flags);
+      auto comp = io::AioCompletion::create_and_start(ctx, src,
+                                                      io::AIO_TYPE_READ);
+      io::ImageRequest<>::aio_read(src, comp, {{offset, len}}, nullptr, bl,
+                                   fadvise_flags);
       prog_ctx.update_progress(offset, src_size);
     }
 
@@ -2334,7 +2334,7 @@ void filter_out_mirror_watchers(ImageCtx *ictx,
   void rbd_ctx_cb(completion_t cb, void *arg)
   {
     Context *ctx = reinterpret_cast<Context *>(arg);
-    AioCompletion *comp = reinterpret_cast<AioCompletion *>(cb);
+    auto comp = reinterpret_cast<io::AioCompletion *>(cb);
     ctx->complete(comp->get_return_value());
     comp->release();
   }
@@ -2372,9 +2372,10 @@ void filter_out_mirror_watchers(ImageCtx *ictx,
       bufferlist bl;
 
       C_SaferCond ctx;
-      AioCompletion *c = AioCompletion::create_and_start(&ctx, ictx,
-                                                         AIO_TYPE_READ);
-      AioImageRequest<>::aio_read(ictx, c, {{off, read_len}}, nullptr, &bl, 0);
+      auto c = io::AioCompletion::create_and_start(&ctx, ictx,
+                                                   io::AIO_TYPE_READ);
+      io::ImageRequest<>::aio_read(ictx, c, {{off, read_len}}, nullptr, &bl,
+                                      0);
 
       int ret = ctx.wait();
       if (ret < 0) {
@@ -2493,12 +2494,13 @@ void filter_out_mirror_watchers(ImageCtx *ictx,
     return r;
   }
 
-  int poll_io_events(ImageCtx *ictx, AioCompletion **comps, int numcomp)
+  int poll_io_events(ImageCtx *ictx, io::AioCompletion **comps, int numcomp)
   {
     if (numcomp <= 0)
       return -EINVAL;
     CephContext *cct = ictx->cct;
-    ldout(cct, 20) << __func__ << " " << ictx << " numcomp = " << numcomp << dendl;
+    ldout(cct, 20) << __func__ << " " << ictx << " numcomp = " << numcomp
+                   << dendl;
     int i = 0;
     Mutex::Locker l(ictx->completed_reqs_lock);
     while (i < numcomp) {
@@ -3369,7 +3371,7 @@ void filter_out_mirror_watchers(ImageCtx *ictx,
     uint64_t image_size = ictx->get_image_size(ictx->snap_id);
     ictx->snap_lock.put_read();
     ictx->md_lock.put_write();
-    
     pair<uint64_t, uint64_t> readahead_extent = ictx->readahead.update(image_extents, image_size);
     uint64_t readahead_offset = readahead_extent.first;
     uint64_t readahead_length = readahead_extent.second;
index 428c99ccd0871657a0ea58b7e53a848c534c89c5..22cd4bad2b630caba03880ee369c56d4c8ae9f1a 100644 (file)
@@ -52,8 +52,8 @@ enum {
 
 namespace librbd {
 
-  struct AioCompletion;
   struct ImageCtx;
+  namespace io { struct AioCompletion; }
 
   class NoOpProgressContext : public ProgressContext
   {
@@ -195,7 +195,7 @@ namespace librbd {
 
   int flush(ImageCtx *ictx);
   int invalidate_cache(ImageCtx *ictx);
-  int poll_io_events(ImageCtx *ictx, AioCompletion **comps, int numcomp);
+  int poll_io_events(ImageCtx *ictx, io::AioCompletion **comps, int numcomp);
   int metadata_list(ImageCtx *ictx, const string &last, uint64_t max, map<string, bufferlist> *pairs);
   int metadata_get(ImageCtx *ictx, const std::string &key, std::string *value);
 
diff --git a/src/librbd/io/AioCompletion.cc b/src/librbd/io/AioCompletion.cc
new file mode 100644 (file)
index 0000000..8d42e82
--- /dev/null
@@ -0,0 +1,227 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "librbd/io/AioCompletion.h"
+#include <errno.h>
+
+#include "common/ceph_context.h"
+#include "common/dout.h"
+#include "common/errno.h"
+#include "common/perf_counters.h"
+#include "common/WorkQueue.h"
+
+#include "librbd/ImageCtx.h"
+#include "librbd/internal.h"
+
+#include "librbd/Journal.h"
+
+#ifdef WITH_LTTNG
+#include "tracing/librbd.h"
+#else
+#define tracepoint(...)
+#endif
+
+#define dout_subsys ceph_subsys_rbd
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::io::AioCompletion: "
+
+namespace librbd {
+namespace io {
+
+int AioCompletion::wait_for_complete() {
+  tracepoint(librbd, aio_wait_for_complete_enter, this);
+  lock.Lock();
+  while (state != AIO_STATE_COMPLETE)
+    cond.Wait(lock);
+  lock.Unlock();
+  tracepoint(librbd, aio_wait_for_complete_exit, 0);
+  return 0;
+}
+
+void AioCompletion::finalize(ssize_t rval)
+{
+  assert(lock.is_locked());
+  assert(ictx != nullptr);
+  CephContext *cct = ictx->cct;
+
+  ldout(cct, 20) << this << " " << __func__ << ": r=" << rval << ", "
+                 << "read_buf=" << reinterpret_cast<void*>(read_buf) << ", "
+                 << "real_bl=" <<  reinterpret_cast<void*>(read_bl) << dendl;
+  if (rval >= 0 && aio_type == AIO_TYPE_READ) {
+    if (read_buf && !read_bl) {
+      destriper.assemble_result(cct, read_buf, read_buf_len);
+    } else {
+      // FIXME: make the destriper write directly into a buffer so
+      // that we avoid shuffling pointers and copying zeros around.
+      bufferlist bl;
+      destriper.assemble_result(cct, bl, true);
+
+      if (read_buf) {
+        assert(bl.length() == read_buf_len);
+        bl.copy(0, read_buf_len, read_buf);
+        ldout(cct, 20) << "copied resulting " << bl.length()
+                       << " bytes to " << (void*)read_buf << dendl;
+      }
+      if (read_bl) {
+        ldout(cct, 20) << " moving resulting " << bl.length()
+                       << " bytes to bl " << (void*)read_bl << dendl;
+        read_bl->claim(bl);
+      }
+    }
+  }
+}
+
+void AioCompletion::complete() {
+  assert(lock.is_locked());
+  assert(ictx != nullptr);
+  CephContext *cct = ictx->cct;
+
+  tracepoint(librbd, aio_complete_enter, this, rval);
+  utime_t elapsed;
+  elapsed = ceph_clock_now() - start_time;
+  switch (aio_type) {
+  case AIO_TYPE_OPEN:
+  case AIO_TYPE_CLOSE:
+    break;
+  case AIO_TYPE_READ:
+    ictx->perfcounter->tinc(l_librbd_rd_latency, elapsed); break;
+  case AIO_TYPE_WRITE:
+    ictx->perfcounter->tinc(l_librbd_wr_latency, elapsed); break;
+  case AIO_TYPE_DISCARD:
+    ictx->perfcounter->tinc(l_librbd_discard_latency, elapsed); break;
+  case AIO_TYPE_FLUSH:
+    ictx->perfcounter->tinc(l_librbd_aio_flush_latency, elapsed); break;
+  default:
+    lderr(cct) << "completed invalid aio_type: " << aio_type << dendl;
+    break;
+  }
+
+  // inform the journal that the op has successfully committed
+  if (journal_tid != 0) {
+    assert(ictx->journal != NULL);
+    ictx->journal->commit_io_event(journal_tid, rval);
+  }
+
+  state = AIO_STATE_CALLBACK;
+  if (complete_cb) {
+    lock.Unlock();
+    complete_cb(rbd_comp, complete_arg);
+    lock.Lock();
+  }
+
+  if (ictx && event_notify && ictx->event_socket.is_valid()) {
+    ictx->completed_reqs_lock.Lock();
+    ictx->completed_reqs.push_back(&m_xlist_item);
+    ictx->completed_reqs_lock.Unlock();
+    ictx->event_socket.notify();
+  }
+
+  state = AIO_STATE_COMPLETE;
+  cond.Signal();
+
+  // note: possible for image to be closed after op marked finished
+  if (async_op.started()) {
+    async_op.finish_op();
+  }
+  tracepoint(librbd, aio_complete_exit);
+}
+
+void AioCompletion::init_time(ImageCtx *i, aio_type_t t) {
+  Mutex::Locker locker(lock);
+  if (ictx == nullptr) {
+    ictx = i;
+    aio_type = t;
+    start_time = ceph_clock_now();
+  }
+}
+
+void AioCompletion::start_op(bool ignore_type) {
+  Mutex::Locker locker(lock);
+  assert(ictx != nullptr);
+  assert(!async_op.started());
+  if (state == AIO_STATE_PENDING &&
+      (ignore_type || aio_type != AIO_TYPE_FLUSH)) {
+    async_op.start_op(*ictx);
+  }
+}
+
+void AioCompletion::fail(int r)
+{
+  lock.Lock();
+  assert(ictx != nullptr);
+  CephContext *cct = ictx->cct;
+
+  lderr(cct) << this << " " << __func__ << ": " << cpp_strerror(r)
+             << dendl;
+  assert(pending_count == 0);
+  rval = r;
+  complete();
+  put_unlock();
+}
+
+void AioCompletion::set_request_count(uint32_t count) {
+  lock.Lock();
+  assert(ictx != nullptr);
+  CephContext *cct = ictx->cct;
+
+  ldout(cct, 20) << this << " " << __func__ << ": pending=" << count << dendl;
+  assert(pending_count == 0);
+  pending_count = count;
+  lock.Unlock();
+
+  // if no pending requests, completion will fire now
+  unblock();
+}
+
+void AioCompletion::complete_request(ssize_t r)
+{
+  lock.Lock();
+  assert(ictx != nullptr);
+  CephContext *cct = ictx->cct;
+
+  if (rval >= 0) {
+    if (r < 0 && r != -EEXIST)
+      rval = r;
+    else if (r > 0)
+      rval += r;
+  }
+  assert(pending_count);
+  int count = --pending_count;
+
+  ldout(cct, 20) << this << " " << __func__ << ": cb=" << complete_cb << ", "
+                 << "pending=" << pending_count << dendl;
+  if (!count && blockers == 0) {
+    finalize(rval);
+    complete();
+  }
+  put_unlock();
+}
+
+void AioCompletion::associate_journal_event(uint64_t tid) {
+  Mutex::Locker l(lock);
+  assert(state == AIO_STATE_PENDING);
+  journal_tid = tid;
+}
+
+bool AioCompletion::is_complete() {
+  tracepoint(librbd, aio_is_complete_enter, this);
+  bool done;
+  {
+    Mutex::Locker l(lock);
+    done = this->state == AIO_STATE_COMPLETE;
+  }
+  tracepoint(librbd, aio_is_complete_exit, done);
+  return done;
+}
+
+ssize_t AioCompletion::get_return_value() {
+  tracepoint(librbd, aio_get_return_value_enter, this);
+  lock.Lock();
+  ssize_t r = rval;
+  lock.Unlock();
+  tracepoint(librbd, aio_get_return_value_exit, r);
+  return r;
+}
+
+} // namespace io
+} // namespace librbd
diff --git a/src/librbd/io/AioCompletion.h b/src/librbd/io/AioCompletion.h
new file mode 100644 (file)
index 0000000..cce065f
--- /dev/null
@@ -0,0 +1,230 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_IO_AIO_COMPLETION_H
+#define CEPH_LIBRBD_IO_AIO_COMPLETION_H
+
+#include "common/Cond.h"
+#include "common/Mutex.h"
+#include "include/Context.h"
+#include "include/utime.h"
+#include "include/rbd/librbd.hpp"
+
+#include "librbd/AsyncOperation.h"
+#include "librbd/ImageCtx.h"
+#include "librbd/io/Types.h"
+#include "osdc/Striper.h"
+
+class CephContext;
+
+namespace librbd {
+namespace io {
+
+/**
+ * AioCompletion is the overall completion for a single
+ * rbd I/O request. It may be composed of many AioObjectRequests,
+ * which each go to a single object.
+ *
+ * The retrying of individual requests is handled at a lower level,
+ * so all AioCompletion cares about is the count of outstanding
+ * requests. The number of expected individual requests should be
+ * set initially using set_request_count() prior to issuing the
+ * requests.  This ensures that the completion will not be completed
+ * within the caller's thread of execution (instead via a librados
+ * context or via a thread pool context for cache read hits).
+ */
+struct AioCompletion {
+  typedef enum {
+    AIO_STATE_PENDING = 0,
+    AIO_STATE_CALLBACK,
+    AIO_STATE_COMPLETE,
+  } aio_state_t;
+
+  mutable Mutex lock;
+  Cond cond;
+  aio_state_t state;
+  ssize_t rval;
+  callback_t complete_cb;
+  void *complete_arg;
+  rbd_completion_t rbd_comp;
+  uint32_t pending_count;   ///< number of requests
+  uint32_t blockers;
+  int ref;
+  bool released;
+  ImageCtx *ictx;
+  utime_t start_time;
+  aio_type_t aio_type;
+
+  Striper::StripedReadResult destriper;
+  bufferlist *read_bl;
+  char *read_buf;
+  size_t read_buf_len;
+
+  AsyncOperation async_op;
+
+  uint64_t journal_tid;
+  xlist<AioCompletion*>::item m_xlist_item;
+  bool event_notify;
+
+  template <typename T, void (T::*MF)(int)>
+  static void callback_adapter(completion_t cb, void *arg) {
+    AioCompletion *comp = reinterpret_cast<AioCompletion *>(cb);
+    T *t = reinterpret_cast<T *>(arg);
+    (t->*MF)(comp->get_return_value());
+    comp->release();
+  }
+
+  static AioCompletion *create(void *cb_arg, callback_t cb_complete,
+                               rbd_completion_t rbd_comp) {
+    AioCompletion *comp = new AioCompletion();
+    comp->set_complete_cb(cb_arg, cb_complete);
+    comp->rbd_comp = (rbd_comp != nullptr ? rbd_comp : comp);
+    return comp;
+  }
+
+  template <typename T, void (T::*MF)(int) = &T::complete>
+  static AioCompletion *create(T *obj) {
+    AioCompletion *comp = new AioCompletion();
+    comp->set_complete_cb(obj, &callback_adapter<T, MF>);
+    comp->rbd_comp = comp;
+    return comp;
+  }
+
+  template <typename T, void (T::*MF)(int) = &T::complete>
+  static AioCompletion *create_and_start(T *obj, ImageCtx *image_ctx,
+                                         aio_type_t type) {
+    AioCompletion *comp = create<T, MF>(obj);
+    comp->init_time(image_ctx, type);
+    comp->start_op();
+    return comp;
+  }
+
+  AioCompletion() : lock("AioCompletion::lock", true, false),
+                    state(AIO_STATE_PENDING), rval(0), complete_cb(NULL),
+                    complete_arg(NULL), rbd_comp(NULL),
+                    pending_count(0), blockers(1),
+                    ref(1), released(false), ictx(NULL),
+                    aio_type(AIO_TYPE_NONE),
+                    read_bl(NULL), read_buf(NULL), read_buf_len(0),
+                    journal_tid(0), m_xlist_item(this), event_notify(false) {
+  }
+
+  ~AioCompletion() {
+  }
+
+  int wait_for_complete();
+
+  void finalize(ssize_t rval);
+
+  inline bool is_initialized(aio_type_t type) const {
+    Mutex::Locker locker(lock);
+    return ((ictx != nullptr) && (aio_type == type));
+  }
+  inline bool is_started() const {
+    Mutex::Locker locker(lock);
+    return async_op.started();
+  }
+
+  void init_time(ImageCtx *i, aio_type_t t);
+  void start_op(bool ignore_type = false);
+  void fail(int r);
+
+  void complete();
+
+  void set_complete_cb(void *cb_arg, callback_t cb) {
+    complete_cb = cb;
+    complete_arg = cb_arg;
+  }
+
+  void set_request_count(uint32_t num);
+  void add_request() {
+    lock.Lock();
+    assert(pending_count > 0);
+    lock.Unlock();
+    get();
+  }
+  void complete_request(ssize_t r);
+
+  void associate_journal_event(uint64_t tid);
+
+  bool is_complete();
+
+  ssize_t get_return_value();
+
+  void get() {
+    lock.Lock();
+    assert(ref > 0);
+    ref++;
+    lock.Unlock();
+  }
+  void release() {
+    lock.Lock();
+    assert(!released);
+    released = true;
+    put_unlock();
+  }
+  void put() {
+    lock.Lock();
+    put_unlock();
+  }
+  void put_unlock() {
+    assert(ref > 0);
+    int n = --ref;
+    lock.Unlock();
+    if (!n) {
+      if (ictx) {
+        if (event_notify) {
+          ictx->completed_reqs_lock.Lock();
+          m_xlist_item.remove_myself();
+          ictx->completed_reqs_lock.Unlock();
+        }
+        if (aio_type == AIO_TYPE_CLOSE ||
+            (aio_type == AIO_TYPE_OPEN && rval < 0)) {
+          delete ictx;
+        }
+      }
+      delete this;
+    }
+  }
+
+  void block() {
+    Mutex::Locker l(lock);
+    ++blockers;
+  }
+  void unblock() {
+    Mutex::Locker l(lock);
+    assert(blockers > 0);
+    --blockers;
+    if (pending_count == 0 && blockers == 0) {
+      finalize(rval);
+      complete();
+    }
+  }
+
+  void set_event_notify(bool s) {
+    Mutex::Locker l(lock);
+    event_notify = s;
+  }
+
+  void *get_arg() {
+    return complete_arg;
+  }
+};
+
+class C_AioRequest : public Context {
+public:
+  C_AioRequest(AioCompletion *completion) : m_completion(completion) {
+    m_completion->add_request();
+  }
+  virtual ~C_AioRequest() {}
+  virtual void finish(int r) {
+    m_completion->complete_request(r);
+  }
+protected:
+  AioCompletion *m_completion;
+};
+
+} // namespace io
+} // namespace librbd
+
+#endif // CEPH_LIBRBD_IO_AIO_COMPLETION_H
diff --git a/src/librbd/io/CopyupRequest.cc b/src/librbd/io/CopyupRequest.cc
new file mode 100644 (file)
index 0000000..b025a17
--- /dev/null
@@ -0,0 +1,368 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "librbd/io/CopyupRequest.h"
+#include "common/ceph_context.h"
+#include "common/dout.h"
+#include "common/errno.h"
+#include "common/Mutex.h"
+
+#include "librbd/AsyncObjectThrottle.h"
+#include "librbd/ExclusiveLock.h"
+#include "librbd/ImageCtx.h"
+#include "librbd/ObjectMap.h"
+#include "librbd/Utils.h"
+#include "librbd/io/AioCompletion.h"
+#include "librbd/io/ImageRequest.h"
+#include "librbd/io/ObjectRequest.h"
+
+#include <boost/bind.hpp>
+#include <boost/lambda/bind.hpp>
+#include <boost/lambda/construct.hpp>
+
+#define dout_subsys ceph_subsys_rbd
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::io::CopyupRequest: "
+
+namespace librbd {
+namespace io {
+
+namespace {
+
+class UpdateObjectMap : public C_AsyncObjectThrottle<> {
+public:
+  UpdateObjectMap(AsyncObjectThrottle<> &throttle, ImageCtx *image_ctx,
+                  uint64_t object_no, const std::vector<uint64_t> *snap_ids,
+                  size_t snap_id_idx)
+    : C_AsyncObjectThrottle(throttle, *image_ctx),
+      m_object_no(object_no), m_snap_ids(*snap_ids), m_snap_id_idx(snap_id_idx)
+  {
+  }
+
+  int send() override {
+    uint64_t snap_id = m_snap_ids[m_snap_id_idx];
+    if (snap_id == CEPH_NOSNAP) {
+      RWLock::RLocker snap_locker(m_image_ctx.snap_lock);
+      RWLock::WLocker object_map_locker(m_image_ctx.object_map_lock);
+      assert(m_image_ctx.exclusive_lock->is_lock_owner());
+      assert(m_image_ctx.object_map != nullptr);
+      bool sent = m_image_ctx.object_map->aio_update<Context>(
+        CEPH_NOSNAP, m_object_no, OBJECT_EXISTS, {}, this);
+      return (sent ? 0 : 1);
+    }
+
+    uint8_t state = OBJECT_EXISTS;
+    if (m_image_ctx.test_features(RBD_FEATURE_FAST_DIFF) &&
+        m_snap_id_idx + 1 < m_snap_ids.size()) {
+      state = OBJECT_EXISTS_CLEAN;
+    }
+
+    RWLock::RLocker snap_locker(m_image_ctx.snap_lock);
+    RWLock::RLocker object_map_locker(m_image_ctx.object_map_lock);
+    if (m_image_ctx.object_map == nullptr) {
+      return 1;
+    }
+
+    bool sent = m_image_ctx.object_map->aio_update<Context>(
+      snap_id, m_object_no, state, {}, this);
+    assert(sent);
+    return 0;
+  }
+
+private:
+  uint64_t m_object_no;
+  const std::vector<uint64_t> &m_snap_ids;
+  size_t m_snap_id_idx;
+};
+
+} // anonymous namespace
+
+
+CopyupRequest::CopyupRequest(ImageCtx *ictx, const std::string &oid,
+                             uint64_t objectno, Extents &&image_extents)
+  : m_ictx(ictx), m_oid(oid), m_object_no(objectno),
+    m_image_extents(image_extents), m_state(STATE_READ_FROM_PARENT)
+{
+  m_async_op.start_op(*m_ictx);
+}
+
+CopyupRequest::~CopyupRequest() {
+  assert(m_pending_requests.empty());
+  m_async_op.finish_op();
+}
+
+void CopyupRequest::append_request(ObjectRequest<> *req) {
+  ldout(m_ictx->cct, 20) << __func__ << " " << this << ": " << req << dendl;
+  m_pending_requests.push_back(req);
+}
+
+void CopyupRequest::complete_requests(int r) {
+  while (!m_pending_requests.empty()) {
+    vector<ObjectRequest<> *>::iterator it = m_pending_requests.begin();
+    ObjectRequest<> *req = *it;
+    ldout(m_ictx->cct, 20) << __func__ << " completing request " << req
+                           << dendl;
+    req->complete(r);
+    m_pending_requests.erase(it);
+  }
+}
+
+bool CopyupRequest::send_copyup() {
+  bool add_copyup_op = !m_copyup_data.is_zero();
+  bool copy_on_read = m_pending_requests.empty();
+  if (!add_copyup_op && copy_on_read) {
+    // copyup empty object to prevent future CoR attempts
+    m_copyup_data.clear();
+    add_copyup_op = true;
+  }
+
+  ldout(m_ictx->cct, 20) << __func__ << " " << this
+                         << ": oid " << m_oid << dendl;
+  m_state = STATE_COPYUP;
+
+  m_ictx->snap_lock.get_read();
+  ::SnapContext snapc = m_ictx->snapc;
+  m_ictx->snap_lock.put_read();
+
+  std::vector<librados::snap_t> snaps;
+
+  if (!copy_on_read) {
+    m_pending_copyups.inc();
+  }
+
+  int r;
+  if (copy_on_read || (!snapc.snaps.empty() && add_copyup_op)) {
+    assert(add_copyup_op);
+    add_copyup_op = false;
+
+    librados::ObjectWriteOperation copyup_op;
+    copyup_op.exec("rbd", "copyup", m_copyup_data);
+
+    // send only the copyup request with a blank snapshot context so that
+    // all snapshots are detected from the parent for this object.  If
+    // this is a CoW request, a second request will be created for the
+    // actual modification.
+    m_pending_copyups.inc();
+
+    ldout(m_ictx->cct, 20) << __func__ << " " << this << " copyup with "
+                           << "empty snapshot context" << dendl;
+    librados::AioCompletion *comp = util::create_rados_safe_callback(this);
+
+    librados::Rados rados(m_ictx->data_ctx);
+    r = rados.ioctx_create2(m_ictx->data_ctx.get_id(), m_data_ctx);
+    assert(r == 0);
+
+    r = m_data_ctx.aio_operate(m_oid, comp, &copyup_op, 0, snaps);
+    assert(r == 0);
+    comp->release();
+  }
+
+  if (!copy_on_read) {
+    librados::ObjectWriteOperation write_op;
+    if (add_copyup_op) {
+      // CoW did not need to handle existing snapshots
+      write_op.exec("rbd", "copyup", m_copyup_data);
+    }
+
+    // merge all pending write ops into this single RADOS op
+    for (size_t i=0; i<m_pending_requests.size(); ++i) {
+      ObjectRequest<> *req = m_pending_requests[i];
+      ldout(m_ictx->cct, 20) << __func__ << " add_copyup_ops " << req
+                             << dendl;
+      req->add_copyup_ops(&write_op);
+    }
+    assert(write_op.size() != 0);
+
+    snaps.insert(snaps.end(), snapc.snaps.begin(), snapc.snaps.end());
+    librados::AioCompletion *comp = util::create_rados_safe_callback(this);
+    r = m_ictx->data_ctx.aio_operate(m_oid, comp, &write_op);
+    assert(r == 0);
+    comp->release();
+  }
+  return false;
+}
+
+bool CopyupRequest::is_copyup_required() {
+  bool noop = true;
+  for (const ObjectRequest<> *req : m_pending_requests) {
+    if (!req->is_op_payload_empty()) {
+      noop = false;
+      break;
+    }
+  }
+
+  return (m_copyup_data.is_zero() && noop);
+}
+
+void CopyupRequest::send()
+{
+  m_state = STATE_READ_FROM_PARENT;
+  AioCompletion *comp = AioCompletion::create_and_start(
+    this, m_ictx, AIO_TYPE_READ);
+
+  ldout(m_ictx->cct, 20) << __func__ << " " << this
+                         << ": completion " << comp
+                         << ", oid " << m_oid
+                         << ", extents " << m_image_extents
+                         << dendl;
+  ImageRequest<>::aio_read(m_ictx->parent, comp, std::move(m_image_extents),
+                              nullptr, &m_copyup_data, 0);
+}
+
+void CopyupRequest::complete(int r)
+{
+  if (should_complete(r)) {
+    complete_requests(r);
+    delete this;
+  }
+}
+
+bool CopyupRequest::should_complete(int r)
+{
+  CephContext *cct = m_ictx->cct;
+  ldout(cct, 20) << __func__ << " " << this
+                 << ": oid " << m_oid
+                 << ", r " << r << dendl;
+
+  uint64_t pending_copyups;
+  switch (m_state) {
+  case STATE_READ_FROM_PARENT:
+    ldout(cct, 20) << "READ_FROM_PARENT" << dendl;
+    remove_from_list();
+    if (r >= 0 || r == -ENOENT) {
+      if (is_copyup_required()) {
+        ldout(cct, 20) << __func__ << " " << this << " nop, skipping" << dendl;
+        return true;
+      }
+
+      return send_object_map_head();
+    }
+    break;
+
+  case STATE_OBJECT_MAP_HEAD:
+    ldout(cct, 20) << "OBJECT_MAP_HEAD" << dendl;
+    assert(r == 0);
+    return send_object_map();
+
+  case STATE_OBJECT_MAP:
+    ldout(cct, 20) << "OBJECT_MAP" << dendl;
+    assert(r == 0);
+    return send_copyup();
+
+  case STATE_COPYUP:
+    // invoked via a finisher in librados, so thread safe
+    pending_copyups = m_pending_copyups.dec();
+    ldout(cct, 20) << "COPYUP (" << pending_copyups << " pending)"
+                   << dendl;
+    if (r == -ENOENT) {
+      // hide the -ENOENT error if this is the last op
+      if (pending_copyups == 0) {
+        complete_requests(0);
+      }
+    } else if (r < 0) {
+      complete_requests(r);
+    }
+    return (pending_copyups == 0);
+
+  default:
+    lderr(cct) << "invalid state: " << m_state << dendl;
+    assert(false);
+    break;
+  }
+  return (r < 0);
+}
+
+void CopyupRequest::remove_from_list()
+{
+  Mutex::Locker l(m_ictx->copyup_list_lock);
+
+  map<uint64_t, CopyupRequest*>::iterator it =
+    m_ictx->copyup_list.find(m_object_no);
+  assert(it != m_ictx->copyup_list.end());
+  m_ictx->copyup_list.erase(it);
+}
+
+bool CopyupRequest::send_object_map_head() {
+  CephContext *cct = m_ictx->cct;
+  ldout(cct, 20) << __func__ << " " << this << dendl;
+
+  m_state = STATE_OBJECT_MAP_HEAD;
+
+  {
+    RWLock::RLocker owner_locker(m_ictx->owner_lock);
+    RWLock::RLocker snap_locker(m_ictx->snap_lock);
+    if (m_ictx->object_map != nullptr) {
+      bool copy_on_read = m_pending_requests.empty();
+      assert(m_ictx->exclusive_lock->is_lock_owner());
+
+      RWLock::WLocker object_map_locker(m_ictx->object_map_lock);
+      if (!m_ictx->snaps.empty()) {
+        m_snap_ids.insert(m_snap_ids.end(), m_ictx->snaps.begin(),
+                          m_ictx->snaps.end());
+      }
+      if (copy_on_read &&
+          (*m_ictx->object_map)[m_object_no] != OBJECT_EXISTS) {
+        m_snap_ids.insert(m_snap_ids.begin(), CEPH_NOSNAP);
+        object_map_locker.unlock();
+        snap_locker.unlock();
+        owner_locker.unlock();
+        return send_object_map();
+      }
+
+      bool may_update = false;
+      uint8_t new_state, current_state;
+
+      vector<ObjectRequest<> *>::reverse_iterator r_it = m_pending_requests.rbegin();
+      for (; r_it != m_pending_requests.rend(); ++r_it) {
+        ObjectRequest<> *req = *r_it;
+        if (!req->pre_object_map_update(&new_state)) {
+          continue;
+        }
+
+        current_state = (*m_ictx->object_map)[m_object_no];
+        ldout(cct, 20) << __func__ << " " << req->get_op_type() << " object no "
+                       << m_object_no << " current state "
+                       << stringify(static_cast<uint32_t>(current_state))
+                       << " new state " << stringify(static_cast<uint32_t>(new_state))
+                       << dendl;
+        may_update = true;
+        break;
+      }
+
+      if (may_update && (new_state != current_state) &&
+          m_ictx->object_map->aio_update<CopyupRequest>(
+            CEPH_NOSNAP, m_object_no, new_state, current_state, this)) {
+        return false;
+      }
+    }
+  }
+
+  return send_object_map();
+}
+
+bool CopyupRequest::send_object_map() {
+  // avoid possible recursive lock attempts
+  if (m_snap_ids.empty()) {
+    // no object map update required
+    return send_copyup();
+  } else {
+    // update object maps for HEAD and all existing snapshots
+    ldout(m_ictx->cct, 20) << __func__ << " " << this
+                           << ": oid " << m_oid << dendl;
+    m_state = STATE_OBJECT_MAP;
+
+    RWLock::RLocker owner_locker(m_ictx->owner_lock);
+    AsyncObjectThrottle<>::ContextFactory context_factory(
+      boost::lambda::bind(boost::lambda::new_ptr<UpdateObjectMap>(),
+      boost::lambda::_1, m_ictx, m_object_no, &m_snap_ids,
+      boost::lambda::_2));
+    AsyncObjectThrottle<> *throttle = new AsyncObjectThrottle<>(
+      NULL, *m_ictx, context_factory, util::create_context_callback(this),
+      NULL, 0, m_snap_ids.size());
+    throttle->start_ops(m_ictx->concurrent_management_ops);
+  }
+  return false;
+}
+
+} // namespace io
+} // namespace librbd
diff --git a/src/librbd/io/CopyupRequest.h b/src/librbd/io/CopyupRequest.h
new file mode 100644 (file)
index 0000000..d018370
--- /dev/null
@@ -0,0 +1,104 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_IO_COPYUP_REQUEST_H
+#define CEPH_LIBRBD_IO_COPYUP_REQUEST_H
+
+#include "librbd/AsyncOperation.h"
+#include "include/atomic.h"
+#include "include/int_types.h"
+#include "include/rados/librados.hpp"
+#include "include/buffer.h"
+#include "librbd/io/Types.h"
+#include <string>
+#include <vector>
+
+namespace librbd {
+
+struct ImageCtx;
+
+namespace io {
+
+struct AioCompletion;
+template <typename I> class ObjectRequest;
+
+class CopyupRequest {
+public:
+  CopyupRequest(ImageCtx *ictx, const std::string &oid, uint64_t objectno,
+                Extents &&image_extents);
+  ~CopyupRequest();
+
+  void append_request(ObjectRequest<ImageCtx> *req);
+
+  void send();
+
+  void complete(int r);
+
+private:
+  /**
+   * Copyup requests go through the following state machine to read from the
+   * parent image, update the object map, and copyup the object:
+   *
+   *
+   * @verbatim
+   *
+   *              <start>
+   *                 |
+   *                 v
+   *    . . .STATE_READ_FROM_PARENT. . .
+   *    . .          |                 .
+   *    . .          v                 .
+   *    . .  STATE_OBJECT_MAP_HEAD     v (copy on read /
+   *    . .          |                 .  no HEAD rev. update)
+   *    v v          v                 .
+   *    . .    STATE_OBJECT_MAP. . . . .
+   *    . .          |
+   *    . .          v
+   *    . . . . > STATE_COPYUP
+   *    .            |
+   *    .            v
+   *    . . . . > <finish>
+   *
+   * @endverbatim
+   *
+   * The _OBJECT_MAP state is skipped if the object map isn't enabled or if
+   * an object map update isn't required. The _COPYUP state is skipped if
+   * no data was read from the parent *and* there are no additional ops.
+   */
+  enum State {
+    STATE_READ_FROM_PARENT,
+    STATE_OBJECT_MAP_HEAD, // only update the HEAD revision
+    STATE_OBJECT_MAP,      // update HEAD+snaps (if any)
+    STATE_COPYUP
+  };
+
+  ImageCtx *m_ictx;
+  std::string m_oid;
+  uint64_t m_object_no;
+  Extents m_image_extents;
+  State m_state;
+  ceph::bufferlist m_copyup_data;
+  std::vector<ObjectRequest<ImageCtx> *> m_pending_requests;
+  atomic_t m_pending_copyups;
+
+  AsyncOperation m_async_op;
+
+  std::vector<uint64_t> m_snap_ids;
+  librados::IoCtx m_data_ctx; // for empty SnapContext
+
+  void complete_requests(int r);
+
+  bool should_complete(int r);
+
+  void remove_from_list();
+
+  bool send_object_map_head();
+  bool send_object_map();
+  bool send_copyup();
+  bool is_copyup_required();
+};
+
+} // namespace io
+} // namespace librbd
+
+#endif // CEPH_LIBRBD_IO_COPYUP_REQUEST_H
diff --git a/src/librbd/io/ImageRequest.cc b/src/librbd/io/ImageRequest.cc
new file mode 100644 (file)
index 0000000..e11d194
--- /dev/null
@@ -0,0 +1,736 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "librbd/io/ImageRequest.h"
+#include "librbd/ImageCtx.h"
+#include "librbd/internal.h"
+#include "librbd/Journal.h"
+#include "librbd/Utils.h"
+#include "librbd/cache/ImageCache.h"
+#include "librbd/io/AioCompletion.h"
+#include "librbd/io/ObjectRequest.h"
+#include "librbd/journal/Types.h"
+#include "include/rados/librados.hpp"
+#include "common/WorkQueue.h"
+#include "osdc/Striper.h"
+
+#define dout_subsys ceph_subsys_rbd
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::io::ImageRequest: "
+
+namespace librbd {
+namespace io {
+
+using util::get_image_ctx;
+
+namespace {
+
+template <typename ImageCtxT = ImageCtx>
+struct C_DiscardJournalCommit : public Context {
+  typedef std::vector<ObjectExtent> ObjectExtents;
+
+  ImageCtxT &image_ctx;
+  AioCompletion *aio_comp;
+  ObjectExtents object_extents;
+
+  C_DiscardJournalCommit(ImageCtxT &_image_ctx, AioCompletion *_aio_comp,
+                         const ObjectExtents &_object_extents, uint64_t tid)
+    : image_ctx(_image_ctx), aio_comp(_aio_comp),
+      object_extents(_object_extents) {
+    CephContext *cct = image_ctx.cct;
+    ldout(cct, 20) << this << " C_DiscardJournalCommit: "
+                   << "delaying cache discard until journal tid " << tid << " "
+                   << "safe" << dendl;
+
+    aio_comp->add_request();
+  }
+
+  void finish(int r) override {
+    CephContext *cct = image_ctx.cct;
+    ldout(cct, 20) << this << " C_DiscardJournalCommit: "
+                   << "journal committed: discarding from cache" << dendl;
+
+    Mutex::Locker cache_locker(image_ctx.cache_lock);
+    image_ctx.object_cacher->discard_set(image_ctx.object_set, object_extents);
+    aio_comp->complete_request(r);
+  }
+};
+
+template <typename ImageCtxT = ImageCtx>
+struct C_FlushJournalCommit : public Context {
+  ImageCtxT &image_ctx;
+  AioCompletion *aio_comp;
+
+  C_FlushJournalCommit(ImageCtxT &_image_ctx, AioCompletion *_aio_comp,
+                       uint64_t tid)
+    : image_ctx(_image_ctx), aio_comp(_aio_comp) {
+    CephContext *cct = image_ctx.cct;
+    ldout(cct, 20) << this << " C_FlushJournalCommit: "
+                   << "delaying flush until journal tid " << tid << " "
+                   << "safe" << dendl;
+
+    aio_comp->add_request();
+  }
+
+  void finish(int r) override {
+    CephContext *cct = image_ctx.cct;
+    ldout(cct, 20) << this << " C_FlushJournalCommit: journal committed"
+                   << dendl;
+    aio_comp->complete_request(r);
+  }
+};
+
+template <typename ImageCtxT>
+class C_AioRead : public C_AioRequest {
+public:
+  C_AioRead(AioCompletion *completion)
+    : C_AioRequest(completion), m_req(nullptr) {
+  }
+
+  void finish(int r) override {
+    m_completion->lock.Lock();
+    CephContext *cct = m_completion->ictx->cct;
+    ldout(cct, 10) << "C_AioRead::finish() " << this << " r = " << r << dendl;
+
+    if (r >= 0 || r == -ENOENT) { // this was a sparse_read operation
+      ldout(cct, 10) << " got " << m_req->get_extent_map()
+                     << " for " << m_req->get_buffer_extents()
+                     << " bl " << m_req->data().length() << dendl;
+      // reads from the parent don't populate the m_ext_map and the overlap
+      // may not be the full buffer.  compensate here by filling in m_ext_map
+      // with the read extent when it is empty.
+      if (m_req->get_extent_map().empty()) {
+        m_req->get_extent_map()[m_req->get_offset()] = m_req->data().length();
+      }
+
+      m_completion->destriper.add_partial_sparse_result(
+          cct, m_req->data(), m_req->get_extent_map(), m_req->get_offset(),
+          m_req->get_buffer_extents());
+      r = m_req->get_length();
+    }
+    m_completion->lock.Unlock();
+
+    C_AioRequest::finish(r);
+  }
+
+  void set_req(ObjectReadRequest<ImageCtxT> *req) {
+    m_req = req;
+  }
+private:
+  ObjectReadRequest<ImageCtxT> *m_req;
+};
+
+template <typename ImageCtxT>
+class C_ImageCacheRead : public C_AioRequest {
+public:
+  typedef std::vector<std::pair<uint64_t,uint64_t> > Extents;
+
+  C_ImageCacheRead(AioCompletion *completion, const Extents &image_extents)
+    : C_AioRequest(completion), m_image_extents(image_extents) {
+  }
+
+  inline bufferlist &get_data() {
+    return m_bl;
+  }
+
+protected:
+  void finish(int r) override {
+    CephContext *cct = m_completion->ictx->cct;
+    ldout(cct, 10) << "C_ImageCacheRead::finish() " << this << ": r=" << r
+                   << dendl;
+    if (r >= 0) {
+      size_t length = 0;
+      for (auto &image_extent : m_image_extents) {
+        length += image_extent.second;
+      }
+      assert(length == m_bl.length());
+
+      m_completion->lock.Lock();
+      m_completion->destriper.add_partial_result(cct, m_bl, m_image_extents);
+      m_completion->lock.Unlock();
+      r = length;
+    }
+    C_AioRequest::finish(r);
+  }
+
+private:
+  bufferlist m_bl;
+  Extents m_image_extents;
+};
+
+template <typename ImageCtxT>
+class C_ObjectCacheRead : public Context {
+public:
+  explicit C_ObjectCacheRead(ImageCtxT &ictx, ObjectReadRequest<ImageCtxT> *req)
+    : m_image_ctx(ictx), m_req(req), m_enqueued(false) {}
+
+  void complete(int r) override {
+    if (!m_enqueued) {
+      // cache_lock creates a lock ordering issue -- so re-execute this context
+      // outside the cache_lock
+      m_enqueued = true;
+      m_image_ctx.op_work_queue->queue(this, r);
+      return;
+    }
+    Context::complete(r);
+  }
+
+protected:
+  void finish(int r) override {
+    m_req->complete(r);
+  }
+
+private:
+  ImageCtxT &m_image_ctx;
+  ObjectReadRequest<ImageCtxT> *m_req;
+  bool m_enqueued;
+};
+
+} // anonymous namespace
+
+template <typename I>
+void ImageRequest<I>::aio_read(I *ictx, AioCompletion *c,
+                               Extents &&image_extents, char *buf,
+                               bufferlist *pbl, int op_flags) {
+  ImageReadRequest<I> req(*ictx, c, std::move(image_extents), buf, pbl,
+                          op_flags);
+  req.send();
+}
+
+template <typename I>
+void ImageRequest<I>::aio_write(I *ictx, AioCompletion *c, uint64_t off,
+                                size_t len, const char *buf, int op_flags) {
+  ImageWriteRequest<I> req(*ictx, c, off, len, buf, op_flags);
+  req.send();
+}
+
+template <typename I>
+void ImageRequest<I>::aio_write(I *ictx, AioCompletion *c,
+                                Extents &&image_extents, bufferlist &&bl,
+                                int op_flags) {
+  ImageWriteRequest<I> req(*ictx, c, std::move(image_extents), std::move(bl),
+                           op_flags);
+  req.send();
+}
+
+template <typename I>
+void ImageRequest<I>::aio_discard(I *ictx, AioCompletion *c,
+                                  uint64_t off, uint64_t len) {
+  ImageDiscardRequest<I> req(*ictx, c, off, len);
+  req.send();
+}
+
+template <typename I>
+void ImageRequest<I>::aio_flush(I *ictx, AioCompletion *c) {
+  ImageFlushRequest<I> req(*ictx, c);
+  req.send();
+}
+
+template <typename I>
+void ImageRequest<I>::send() {
+  I &image_ctx = this->m_image_ctx;
+  assert(m_aio_comp->is_initialized(get_aio_type()));
+  assert(m_aio_comp->is_started() ^ (get_aio_type() == AIO_TYPE_FLUSH));
+
+  CephContext *cct = image_ctx.cct;
+  AioCompletion *aio_comp = this->m_aio_comp;
+  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 (m_bypass_image_cache || m_image_ctx.image_cache == nullptr) {
+    send_request();
+  } else {
+    send_image_cache_request();
+  }
+}
+
+template <typename I>
+int ImageRequest<I>::clip_request() {
+  RWLock::RLocker snap_locker(m_image_ctx.snap_lock);
+  for (auto &image_extent : m_image_extents) {
+    size_t clip_len = image_extent.second;
+    int r = clip_io(get_image_ctx(&m_image_ctx), image_extent.first, &clip_len);
+    if (r < 0) {
+      return r;
+    }
+
+    image_extent.second = clip_len;
+  }
+  return 0;
+}
+
+template <typename I>
+void ImageRequest<I>::fail(int r) {
+  AioCompletion *aio_comp = this->m_aio_comp;
+  aio_comp->get();
+  aio_comp->fail(r);
+}
+
+template <typename I>
+void ImageReadRequest<I>::send_request() {
+  I &image_ctx = this->m_image_ctx;
+  CephContext *cct = image_ctx.cct;
+
+  auto &image_extents = this->m_image_extents;
+  if (image_ctx.object_cacher && image_ctx.readahead_max_bytes > 0 &&
+      !(m_op_flags & LIBRADOS_OP_FLAG_FADVISE_RANDOM)) {
+    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);
+    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;
+    }
+  }
+
+  aio_comp->read_buf = m_buf;
+  aio_comp->read_buf_len = buffer_ofs;
+  aio_comp->read_bl = m_pbl;
+
+  // 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();
+  }
+  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;
+
+      C_AioRead<I> *req_comp = new C_AioRead<I>(aio_comp);
+      ObjectReadRequest<I> *req = ObjectReadRequest<I>::create(
+        &image_ctx, extent.oid.name, extent.objectno, extent.offset,
+        extent.length, extent.buffer_extents, snap_id, true, req_comp,
+        m_op_flags);
+      req_comp->set_req(req);
+
+      if (image_ctx.object_cacher) {
+        C_ObjectCacheRead<I> *cache_comp = new C_ObjectCacheRead<I>(image_ctx,
+                                                                    req);
+        image_ctx.aio_read_from_cache(extent.oid, extent.objectno,
+                                      &req->data(), extent.length,
+                                      extent.offset, cache_comp, m_op_flags);
+      } else {
+        req->send();
+      }
+    }
+  }
+
+  aio_comp->put();
+
+  image_ctx.perfcounter->inc(l_librbd_rd);
+  image_ctx.perfcounter->inc(l_librbd_rd_bytes, buffer_ofs);
+}
+
+template <typename I>
+void ImageReadRequest<I>::send_image_cache_request() {
+  I &image_ctx = this->m_image_ctx;
+  assert(image_ctx.image_cache != nullptr);
+
+  AioCompletion *aio_comp = this->m_aio_comp;
+  aio_comp->set_request_count(1);
+  C_ImageCacheRead<I> *req_comp = new C_ImageCacheRead<I>(
+    aio_comp, this->m_image_extents);
+  image_ctx.image_cache->aio_read(std::move(this->m_image_extents),
+                                  &req_comp->get_data(), m_op_flags,
+                                  req_comp);
+}
+
+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;
+    }
+
+    snapc = image_ctx.snapc;
+    journaling = (image_ctx.journal != nullptr &&
+                  image_ctx.journal->is_journal_appending());
+  }
+
+  prune_object_extents(object_extents);
+
+  if (!object_extents.empty()) {
+    uint64_t journal_tid = 0;
+    aio_comp->set_request_count(
+      object_extents.size() + get_object_cache_request_count(journaling));
+
+    ObjectRequests requests;
+    send_object_requests(object_extents, snapc,
+                         (journaling ? &requests : nullptr));
+
+    if (journaling) {
+      // in-flight ops are flushed prior to closing the journal
+      assert(image_ctx.journal != NULL);
+      journal_tid = append_journal_event(requests, m_synchronous);
+    }
+
+    if (image_ctx.object_cacher != NULL) {
+      send_object_cache_requests(object_extents, journal_tid);
+    }
+  } else {
+    // no IO to perform -- fire completion
+    aio_comp->unblock();
+  }
+
+  update_stats(clip_len);
+  aio_comp->put();
+}
+
+template <typename I>
+void AbstractImageWriteRequest<I>::send_object_requests(
+    const ObjectExtents &object_extents, const ::SnapContext &snapc,
+    ObjectRequests *object_requests) {
+  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;
+    C_AioRequest *req_comp = new C_AioRequest(aio_comp);
+    ObjectRequestHandle *request = create_object_request(*p, snapc,
+                                                            req_comp);
+
+    // if journaling, stash the request for later; otherwise send
+    if (request != NULL) {
+      if (object_requests != NULL) {
+        object_requests->push_back(request);
+      } else {
+        request->send();
+      }
+    }
+  }
+}
+
+template <typename I>
+void ImageWriteRequest<I>::assemble_extent(const ObjectExtent &object_extent,
+                                           bufferlist *bl) {
+  for (auto q = object_extent.buffer_extents.begin();
+       q != object_extent.buffer_extents.end(); ++q) {
+    bufferlist sub_bl;
+    sub_bl.substr_of(m_bl, q->first, q->second);
+    bl->claim_append(sub_bl);
+  }
+}
+
+template <typename I>
+uint64_t ImageWriteRequest<I>::append_journal_event(
+    const ObjectRequests &requests, bool synchronous) {
+  I &image_ctx = this->m_image_ctx;
+
+  uint64_t tid = 0;
+  uint64_t buffer_offset = 0;
+  assert(!this->m_image_extents.empty());
+  for (auto &extent : this->m_image_extents) {
+    bufferlist sub_bl;
+    sub_bl.substr_of(m_bl, buffer_offset, extent.second);
+    buffer_offset += extent.second;
+
+    tid = image_ctx.journal->append_write_event(extent.first, extent.second,
+                                                sub_bl, requests, synchronous);
+  }
+
+  if (image_ctx.object_cacher == NULL) {
+    AioCompletion *aio_comp = this->m_aio_comp;
+    aio_comp->associate_journal_event(tid);
+  }
+  return tid;
+}
+
+template <typename I>
+void ImageWriteRequest<I>::send_image_cache_request() {
+  I &image_ctx = this->m_image_ctx;
+  assert(image_ctx.image_cache != nullptr);
+
+  AioCompletion *aio_comp = this->m_aio_comp;
+  aio_comp->set_request_count(1);
+  C_AioRequest *req_comp = new C_AioRequest(aio_comp);
+  image_ctx.image_cache->aio_write(std::move(this->m_image_extents),
+                                   std::move(m_bl), m_op_flags, req_comp);
+}
+
+template <typename I>
+void ImageWriteRequest<I>::send_object_cache_requests(
+    const ObjectExtents &object_extents, uint64_t journal_tid) {
+  I &image_ctx = this->m_image_ctx;
+  for (auto p = object_extents.begin(); p != object_extents.end(); ++p) {
+    const ObjectExtent &object_extent = *p;
+
+    bufferlist bl;
+    assemble_extent(object_extent, &bl);
+
+    AioCompletion *aio_comp = this->m_aio_comp;
+    C_AioRequest *req_comp = new C_AioRequest(aio_comp);
+    image_ctx.write_to_cache(object_extent.oid, bl, object_extent.length,
+                             object_extent.offset, req_comp, m_op_flags,
+                             journal_tid);
+  }
+}
+
+template <typename I>
+void ImageWriteRequest<I>::send_object_requests(
+    const ObjectExtents &object_extents, const ::SnapContext &snapc,
+    ObjectRequests *object_requests) {
+  I &image_ctx = this->m_image_ctx;
+
+  // cache handles creating object requests during writeback
+  if (image_ctx.object_cacher == NULL) {
+    AbstractImageWriteRequest<I>::send_object_requests(object_extents, snapc,
+                                                       object_requests);
+  }
+}
+
+template <typename I>
+ObjectRequestHandle *ImageWriteRequest<I>::create_object_request(
+    const ObjectExtent &object_extent, const ::SnapContext &snapc,
+    Context *on_finish) {
+  I &image_ctx = this->m_image_ctx;
+  assert(image_ctx.object_cacher == NULL);
+
+  bufferlist bl;
+  assemble_extent(object_extent, &bl);
+  ObjectRequest<I> *req = ObjectRequest<I>::create_write(
+    &image_ctx, object_extent.oid.name, object_extent.objectno,
+    object_extent.offset, bl, snapc, on_finish, m_op_flags);
+  return req;
+}
+
+template <typename I>
+void ImageWriteRequest<I>::update_stats(size_t length) {
+  I &image_ctx = this->m_image_ctx;
+  image_ctx.perfcounter->inc(l_librbd_wr);
+  image_ctx.perfcounter->inc(l_librbd_wr_bytes, length);
+}
+
+template <typename I>
+uint64_t ImageDiscardRequest<I>::append_journal_event(
+    const ObjectRequests &requests, bool synchronous) {
+  I &image_ctx = this->m_image_ctx;
+
+  uint64_t tid = 0;
+  assert(!this->m_image_extents.empty());
+  for (auto &extent : this->m_image_extents) {
+    journal::EventEntry event_entry(journal::AioDiscardEvent(extent.first,
+                                                             extent.second));
+    tid = image_ctx.journal->append_io_event(std::move(event_entry),
+                                             requests, extent.first,
+                                             extent.second, synchronous);
+  }
+
+  AioCompletion *aio_comp = this->m_aio_comp;
+  aio_comp->associate_journal_event(tid);
+  return tid;
+}
+
+template <typename I>
+void ImageDiscardRequest<I>::prune_object_extents(ObjectExtents &object_extents) {
+  I &image_ctx = this->m_image_ctx;
+  CephContext *cct = image_ctx.cct;
+  if (!cct->_conf->rbd_skip_partial_discard) {
+    return;
+  }
+
+  for (auto p = object_extents.begin(); p != object_extents.end(); ) {
+    if (p->offset + p->length < image_ctx.layout.object_size) {
+      ldout(cct, 20) << " oid " << p->oid << " " << p->offset << "~"
+                    << p->length << " from " << p->buffer_extents
+                    << ": skip partial discard" << dendl;
+      p = object_extents.erase(p);
+    } else {
+      ++p;
+    }
+  }
+}
+
+template <typename I>
+uint32_t ImageDiscardRequest<I>::get_object_cache_request_count(bool journaling) const {
+  // extra completion request is required for tracking journal commit
+  I &image_ctx = this->m_image_ctx;
+  return (image_ctx.object_cacher != nullptr && journaling ? 1 : 0);
+}
+
+template <typename I>
+void ImageDiscardRequest<I>::send_image_cache_request() {
+  I &image_ctx = this->m_image_ctx;
+  assert(image_ctx.image_cache != nullptr);
+
+  AioCompletion *aio_comp = this->m_aio_comp;
+  aio_comp->set_request_count(this->m_image_extents.size());
+  for (auto &extent : this->m_image_extents) {
+    C_AioRequest *req_comp = new C_AioRequest(aio_comp);
+    image_ctx.image_cache->aio_discard(extent.first, extent.second, req_comp);
+  }
+}
+
+template <typename I>
+void ImageDiscardRequest<I>::send_object_cache_requests(
+    const ObjectExtents &object_extents, uint64_t journal_tid) {
+  I &image_ctx = this->m_image_ctx;
+  if (journal_tid == 0) {
+    Mutex::Locker cache_locker(image_ctx.cache_lock);
+    image_ctx.object_cacher->discard_set(image_ctx.object_set,
+                                         object_extents);
+  } else {
+    // cannot discard from cache until journal has committed
+    assert(image_ctx.journal != NULL);
+    AioCompletion *aio_comp = this->m_aio_comp;
+    image_ctx.journal->wait_event(
+      journal_tid, new C_DiscardJournalCommit<I>(image_ctx, aio_comp,
+                                                 object_extents, journal_tid));
+  }
+}
+
+template <typename I>
+ObjectRequestHandle *ImageDiscardRequest<I>::create_object_request(
+    const ObjectExtent &object_extent, const ::SnapContext &snapc,
+    Context *on_finish) {
+  I &image_ctx = this->m_image_ctx;
+
+  ObjectRequest<I> *req;
+  if (object_extent.length == image_ctx.layout.object_size) {
+    req = ObjectRequest<I>::create_remove(
+      &image_ctx, object_extent.oid.name, object_extent.objectno, snapc,
+      on_finish);
+  } else if (object_extent.offset + object_extent.length ==
+               image_ctx.layout.object_size) {
+    req = ObjectRequest<I>::create_truncate(
+      &image_ctx, object_extent.oid.name, object_extent.objectno,
+      object_extent.offset, snapc, on_finish);
+  } else {
+    req = ObjectRequest<I>::create_zero(
+      &image_ctx, object_extent.oid.name, object_extent.objectno,
+      object_extent.offset, object_extent.length, snapc, on_finish);
+  }
+  return req;
+}
+
+template <typename I>
+void ImageDiscardRequest<I>::update_stats(size_t length) {
+  I &image_ctx = this->m_image_ctx;
+  image_ctx.perfcounter->inc(l_librbd_discard);
+  image_ctx.perfcounter->inc(l_librbd_discard_bytes, length);
+}
+
+template <typename I>
+void ImageFlushRequest<I>::send_request() {
+  I &image_ctx = this->m_image_ctx;
+  image_ctx.user_flushed();
+
+  bool journaling = false;
+  {
+    RWLock::RLocker snap_locker(image_ctx.snap_lock);
+    journaling = (image_ctx.journal != nullptr &&
+                  image_ctx.journal->is_journal_appending());
+  }
+
+  AioCompletion *aio_comp = this->m_aio_comp;
+  if (journaling) {
+    // in-flight ops are flushed prior to closing the journal
+    uint64_t journal_tid = image_ctx.journal->append_io_event(
+      journal::EventEntry(journal::AioFlushEvent()),
+      ObjectRequests(), 0, 0, false);
+
+    aio_comp->set_request_count(1);
+    aio_comp->associate_journal_event(journal_tid);
+
+    FunctionContext *flush_ctx = new FunctionContext(
+      [aio_comp, &image_ctx, journal_tid] (int r) {
+        auto ctx = new C_FlushJournalCommit<I>(image_ctx, aio_comp,
+                                               journal_tid);
+        image_ctx.journal->flush_event(journal_tid, ctx);
+
+        // track flush op for block writes
+        aio_comp->start_op(true);
+        aio_comp->put();
+    });
+
+    image_ctx.flush_async_operations(flush_ctx);
+  } else {
+    // flush rbd cache only when journaling is not enabled
+    aio_comp->set_request_count(1);
+    C_AioRequest *req_comp = new C_AioRequest(aio_comp);
+    image_ctx.flush(req_comp);
+
+    aio_comp->start_op(true);
+    aio_comp->put();
+  }
+
+  image_ctx.perfcounter->inc(l_librbd_aio_flush);
+}
+
+template <typename I>
+void ImageFlushRequest<I>::send_image_cache_request() {
+  I &image_ctx = this->m_image_ctx;
+  assert(image_ctx.image_cache != nullptr);
+
+  AioCompletion *aio_comp = this->m_aio_comp;
+  aio_comp->set_request_count(1);
+  C_AioRequest *req_comp = new C_AioRequest(aio_comp);
+  image_ctx.image_cache->aio_flush(req_comp);
+}
+
+} // namespace io
+} // namespace librbd
+
+template class librbd::io::ImageRequest<librbd::ImageCtx>;
+template class librbd::io::ImageReadRequest<librbd::ImageCtx>;
+template class librbd::io::AbstractImageWriteRequest<librbd::ImageCtx>;
+template class librbd::io::ImageWriteRequest<librbd::ImageCtx>;
+template class librbd::io::ImageDiscardRequest<librbd::ImageCtx>;
+template class librbd::io::ImageFlushRequest<librbd::ImageCtx>;
diff --git a/src/librbd/io/ImageRequest.h b/src/librbd/io/ImageRequest.h
new file mode 100644 (file)
index 0000000..713425b
--- /dev/null
@@ -0,0 +1,280 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_IO_IMAGE_REQUEST_H
+#define CEPH_LIBRBD_IO_IMAGE_REQUEST_H
+
+#include "include/int_types.h"
+#include "include/buffer_fwd.h"
+#include "common/snap_types.h"
+#include "osd/osd_types.h"
+#include "librbd/io/AioCompletion.h"
+#include <list>
+#include <utility>
+#include <vector>
+
+namespace librbd {
+class ImageCtx;
+
+namespace io {
+
+class AioCompletion;
+class ObjectRequestHandle;
+
+template <typename ImageCtxT = ImageCtx>
+class ImageRequest {
+public:
+  typedef std::vector<std::pair<uint64_t,uint64_t> > Extents;
+
+  virtual ~ImageRequest() {}
+
+  static void aio_read(ImageCtxT *ictx, AioCompletion *c,
+                       Extents &&image_extents, char *buf, bufferlist *pbl,
+                       int op_flags);
+  static void aio_write(ImageCtxT *ictx, AioCompletion *c, uint64_t off,
+                        size_t len, const char *buf, int op_flags);
+  static void aio_write(ImageCtxT *ictx, AioCompletion *c,
+                        Extents &&image_extents, bufferlist &&bl, int op_flags);
+  static void aio_discard(ImageCtxT *ictx, AioCompletion *c, uint64_t off,
+                          uint64_t len);
+  static void aio_flush(ImageCtxT *ictx, AioCompletion *c);
+
+  virtual bool is_write_op() const {
+    return false;
+  }
+
+  void start_op() {
+    m_aio_comp->start_op();
+  }
+
+  void send();
+  void fail(int r);
+
+  void set_bypass_image_cache() {
+    m_bypass_image_cache = true;
+  }
+
+protected:
+  typedef std::list<ObjectRequestHandle *> ObjectRequests;
+
+  ImageCtxT &m_image_ctx;
+  AioCompletion *m_aio_comp;
+  Extents m_image_extents;
+  bool m_bypass_image_cache = false;
+
+  ImageRequest(ImageCtxT &image_ctx, AioCompletion *aio_comp,
+               Extents &&image_extents)
+    : m_image_ctx(image_ctx), m_aio_comp(aio_comp),
+      m_image_extents(image_extents) {
+  }
+
+  virtual int clip_request();
+  virtual void send_request() = 0;
+  virtual void send_image_cache_request() = 0;
+
+  virtual aio_type_t get_aio_type() const = 0;
+  virtual const char *get_request_type() const = 0;
+};
+
+template <typename ImageCtxT = ImageCtx>
+class ImageReadRequest : public ImageRequest<ImageCtxT> {
+public:
+  using typename ImageRequest<ImageCtxT>::Extents;
+
+  ImageReadRequest(ImageCtxT &image_ctx, AioCompletion *aio_comp,
+                   Extents &&image_extents, char *buf, bufferlist *pbl,
+                   int op_flags)
+    : ImageRequest<ImageCtxT>(image_ctx, aio_comp, std::move(image_extents)),
+      m_buf(buf), m_pbl(pbl), m_op_flags(op_flags) {
+  }
+
+protected:
+  virtual void send_request() override;
+  virtual void send_image_cache_request() override;
+
+  virtual aio_type_t get_aio_type() const {
+    return AIO_TYPE_READ;
+  }
+  virtual const char *get_request_type() const {
+    return "aio_read";
+  }
+private:
+  char *m_buf;
+  bufferlist *m_pbl;
+  int m_op_flags;
+};
+
+template <typename ImageCtxT = ImageCtx>
+class AbstractImageWriteRequest : public ImageRequest<ImageCtxT> {
+public:
+  virtual bool is_write_op() const {
+    return true;
+  }
+
+  inline void flag_synchronous() {
+    m_synchronous = true;
+  }
+
+protected:
+  using typename ImageRequest<ImageCtxT>::ObjectRequests;
+  using typename ImageRequest<ImageCtxT>::Extents;
+
+  typedef std::vector<ObjectExtent> ObjectExtents;
+
+  AbstractImageWriteRequest(ImageCtxT &image_ctx, AioCompletion *aio_comp,
+                            Extents &&image_extents)
+    : ImageRequest<ImageCtxT>(image_ctx, aio_comp, std::move(image_extents)),
+      m_synchronous(false) {
+  }
+
+  virtual void send_request();
+
+  virtual void prune_object_extents(ObjectExtents &object_extents) {
+  }
+  virtual uint32_t get_object_cache_request_count(bool journaling) const {
+    return 0;
+  }
+  virtual void send_object_cache_requests(const ObjectExtents &object_extents,
+                                          uint64_t journal_tid) = 0;
+
+  virtual void send_object_requests(const ObjectExtents &object_extents,
+                                    const ::SnapContext &snapc,
+                                    ObjectRequests *object_requests);
+  virtual ObjectRequestHandle *create_object_request(
+      const ObjectExtent &object_extent, const ::SnapContext &snapc,
+      Context *on_finish) = 0;
+
+  virtual uint64_t append_journal_event(const ObjectRequests &requests,
+                                        bool synchronous) = 0;
+  virtual void update_stats(size_t length) = 0;
+
+private:
+  bool m_synchronous;
+};
+
+template <typename ImageCtxT = ImageCtx>
+class ImageWriteRequest : public AbstractImageWriteRequest<ImageCtxT> {
+public:
+  using typename ImageRequest<ImageCtxT>::Extents;
+
+  ImageWriteRequest(ImageCtxT &image_ctx, AioCompletion *aio_comp, uint64_t off,
+                    size_t len, const char *buf, int op_flags)
+    : AbstractImageWriteRequest<ImageCtxT>(image_ctx, aio_comp, {{off, len}}),
+      m_op_flags(op_flags) {
+    m_bl.append(buf, len);
+  }
+  ImageWriteRequest(ImageCtxT &image_ctx, AioCompletion *aio_comp,
+                    Extents &&image_extents, bufferlist &&bl, int op_flags)
+    : AbstractImageWriteRequest<ImageCtxT>(image_ctx, aio_comp,
+                                           std::move(image_extents)),
+      m_bl(std::move(bl)), m_op_flags(op_flags) {
+  }
+
+protected:
+  using typename ImageRequest<ImageCtxT>::ObjectRequests;
+  using typename AbstractImageWriteRequest<ImageCtxT>::ObjectExtents;
+
+  virtual aio_type_t get_aio_type() const {
+    return AIO_TYPE_WRITE;
+  }
+  virtual const char *get_request_type() const {
+    return "aio_write";
+  }
+
+  void assemble_extent(const ObjectExtent &object_extent, bufferlist *bl);
+
+  virtual void send_image_cache_request() override;
+
+  virtual void send_object_cache_requests(const ObjectExtents &object_extents,
+                                          uint64_t journal_tid);
+
+  virtual void send_object_requests(const ObjectExtents &object_extents,
+                                    const ::SnapContext &snapc,
+                                    ObjectRequests *object_requests);
+  virtual ObjectRequestHandle *create_object_request(
+      const ObjectExtent &object_extent, const ::SnapContext &snapc,
+      Context *on_finish);
+
+  virtual uint64_t append_journal_event(const ObjectRequests &requests,
+                                        bool synchronous);
+  virtual void update_stats(size_t length);
+private:
+  bufferlist m_bl;
+  int m_op_flags;
+};
+
+template <typename ImageCtxT = ImageCtx>
+class ImageDiscardRequest : public AbstractImageWriteRequest<ImageCtxT> {
+public:
+  ImageDiscardRequest(ImageCtxT &image_ctx, AioCompletion *aio_comp,
+                      uint64_t off, uint64_t len)
+    : AbstractImageWriteRequest<ImageCtxT>(image_ctx, aio_comp, {{off, len}}) {
+  }
+
+protected:
+  using typename ImageRequest<ImageCtxT>::ObjectRequests;
+  using typename AbstractImageWriteRequest<ImageCtxT>::ObjectExtents;
+
+  virtual aio_type_t get_aio_type() const {
+    return AIO_TYPE_DISCARD;
+  }
+  virtual const char *get_request_type() const {
+    return "aio_discard";
+  }
+
+  virtual void prune_object_extents(ObjectExtents &object_extents) override;
+
+  virtual void send_image_cache_request() override;
+
+  virtual uint32_t get_object_cache_request_count(bool journaling) const override;
+  virtual void send_object_cache_requests(const ObjectExtents &object_extents,
+                                          uint64_t journal_tid);
+
+  virtual ObjectRequestHandle *create_object_request(
+      const ObjectExtent &object_extent, const ::SnapContext &snapc,
+      Context *on_finish);
+
+  virtual uint64_t append_journal_event(const ObjectRequests &requests,
+                                        bool synchronous);
+  virtual void update_stats(size_t length);
+};
+
+template <typename ImageCtxT = ImageCtx>
+class ImageFlushRequest : public ImageRequest<ImageCtxT> {
+public:
+  ImageFlushRequest(ImageCtxT &image_ctx, AioCompletion *aio_comp)
+    : ImageRequest<ImageCtxT>(image_ctx, aio_comp, {}) {
+  }
+
+  virtual bool is_write_op() const {
+    return true;
+  }
+
+protected:
+  using typename ImageRequest<ImageCtxT>::ObjectRequests;
+
+  virtual int clip_request() {
+    return 0;
+  }
+  virtual void send_request();
+  virtual void send_image_cache_request() override;
+
+  virtual aio_type_t get_aio_type() const {
+    return AIO_TYPE_FLUSH;
+  }
+  virtual const char *get_request_type() const {
+    return "aio_flush";
+  }
+};
+
+} // namespace io
+} // namespace librbd
+
+extern template class librbd::io::ImageRequest<librbd::ImageCtx>;
+extern template class librbd::io::ImageReadRequest<librbd::ImageCtx>;
+extern template class librbd::io::AbstractImageWriteRequest<librbd::ImageCtx>;
+extern template class librbd::io::ImageWriteRequest<librbd::ImageCtx>;
+extern template class librbd::io::ImageDiscardRequest<librbd::ImageCtx>;
+extern template class librbd::io::ImageFlushRequest<librbd::ImageCtx>;
+
+#endif // CEPH_LIBRBD_IO_IMAGE_REQUEST_H
diff --git a/src/librbd/io/ImageRequestWQ.cc b/src/librbd/io/ImageRequestWQ.cc
new file mode 100644 (file)
index 0000000..fcc9834
--- /dev/null
@@ -0,0 +1,509 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "librbd/io/ImageRequestWQ.h"
+#include "common/errno.h"
+#include "librbd/ExclusiveLock.h"
+#include "librbd/ImageCtx.h"
+#include "librbd/ImageState.h"
+#include "librbd/internal.h"
+#include "librbd/Utils.h"
+#include "librbd/exclusive_lock/Policy.h"
+#include "librbd/io/AioCompletion.h"
+#include "librbd/io/ImageRequest.h"
+
+#define dout_subsys ceph_subsys_rbd
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::io::ImageRequestWQ: "
+
+namespace librbd {
+namespace io {
+
+ImageRequestWQ::ImageRequestWQ(ImageCtx *image_ctx, const string &name,
+                               time_t ti, ThreadPool *tp)
+  : ThreadPool::PointerWQ<ImageRequest<> >(name, ti, 0, tp),
+    m_image_ctx(*image_ctx),
+    m_lock(util::unique_lock_name("ImageRequestWQ::m_lock", this)),
+    m_write_blockers(0), m_in_progress_writes(0), m_queued_reads(0),
+    m_queued_writes(0), m_in_flight_ops(0), m_refresh_in_progress(false),
+    m_shutdown(false), m_on_shutdown(nullptr) {
+  CephContext *cct = m_image_ctx.cct;
+  ldout(cct, 5) << this << " " << ": ictx=" << image_ctx << dendl;
+  tp->add_work_queue(this);
+}
+
+ssize_t ImageRequestWQ::read(uint64_t off, uint64_t len, char *buf,
+                             int op_flags) {
+  CephContext *cct = m_image_ctx.cct;
+  ldout(cct, 20) << "read: ictx=" << &m_image_ctx << ", off=" << off << ", "
+                 << "len = " << len << dendl;
+
+  C_SaferCond cond;
+  AioCompletion *c = AioCompletion::create(&cond);
+  aio_read(c, off, len, buf, NULL, op_flags, false);
+  return cond.wait();
+}
+
+ssize_t ImageRequestWQ::write(uint64_t off, uint64_t len, const char *buf,
+                              int op_flags) {
+  CephContext *cct = m_image_ctx.cct;
+  ldout(cct, 20) << "write: ictx=" << &m_image_ctx << ", off=" << off << ", "
+                 << "len = " << len << dendl;
+
+  m_image_ctx.snap_lock.get_read();
+  int r = clip_io(util::get_image_ctx(&m_image_ctx), off, &len);
+  m_image_ctx.snap_lock.put_read();
+  if (r < 0) {
+    lderr(cct) << "invalid IO request: " << cpp_strerror(r) << dendl;
+    return r;
+  }
+
+  C_SaferCond cond;
+  AioCompletion *c = AioCompletion::create(&cond);
+  aio_write(c, off, len, buf, op_flags, false);
+
+  r = cond.wait();
+  if (r < 0) {
+    return r;
+  }
+  return len;
+}
+
+int ImageRequestWQ::discard(uint64_t off, uint64_t len) {
+  CephContext *cct = m_image_ctx.cct;
+  ldout(cct, 20) << "discard: ictx=" << &m_image_ctx << ", off=" << off << ", "
+                 << "len = " << len << dendl;
+
+  m_image_ctx.snap_lock.get_read();
+  int r = clip_io(util::get_image_ctx(&m_image_ctx), off, &len);
+  m_image_ctx.snap_lock.put_read();
+  if (r < 0) {
+    lderr(cct) << "invalid IO request: " << cpp_strerror(r) << dendl;
+    return r;
+  }
+
+  C_SaferCond cond;
+  AioCompletion *c = AioCompletion::create(&cond);
+  aio_discard(c, off, len, false);
+
+  r = cond.wait();
+  if (r < 0) {
+    return r;
+  }
+  return len;
+}
+
+void ImageRequestWQ::aio_read(AioCompletion *c, uint64_t off, uint64_t len,
+                              char *buf, bufferlist *pbl, int op_flags,
+                              bool native_async) {
+  c->init_time(&m_image_ctx, AIO_TYPE_READ);
+  CephContext *cct = m_image_ctx.cct;
+  ldout(cct, 20) << "aio_read: ictx=" << &m_image_ctx << ", "
+                 << "completion=" << c << ", off=" << off << ", "
+                 << "len=" << len << ", " << "flags=" << op_flags << dendl;
+
+  if (native_async && m_image_ctx.event_socket.is_valid()) {
+    c->set_event_notify(true);
+  }
+
+  if (!start_in_flight_op(c)) {
+    return;
+  }
+
+  RWLock::RLocker owner_locker(m_image_ctx.owner_lock);
+
+  // if journaling is enabled -- we need to replay the journal because
+  // it might contain an uncommitted write
+  bool lock_required;
+  {
+    RWLock::RLocker locker(m_lock);
+    lock_required = m_require_lock_on_read;
+  }
+
+  if (m_image_ctx.non_blocking_aio || writes_blocked() || !writes_empty() ||
+      lock_required) {
+    queue(new ImageReadRequest<>(m_image_ctx, c, {{off, len}}, buf, pbl,
+                                 op_flags));
+  } else {
+    c->start_op();
+    ImageRequest<>::aio_read(&m_image_ctx, c, {{off, len}}, buf, pbl,
+                             op_flags);
+    finish_in_flight_op();
+  }
+}
+
+void ImageRequestWQ::aio_write(AioCompletion *c, uint64_t off, uint64_t len,
+                               const char *buf, int op_flags,
+                               bool native_async) {
+  c->init_time(&m_image_ctx, AIO_TYPE_WRITE);
+  CephContext *cct = m_image_ctx.cct;
+  ldout(cct, 20) << "aio_write: ictx=" << &m_image_ctx << ", "
+                 << "completion=" << c << ", off=" << off << ", "
+                 << "len=" << len << ", flags=" << op_flags << dendl;
+
+  if (native_async && m_image_ctx.event_socket.is_valid()) {
+    c->set_event_notify(true);
+  }
+
+  if (!start_in_flight_op(c)) {
+    return;
+  }
+
+  RWLock::RLocker owner_locker(m_image_ctx.owner_lock);
+  if (m_image_ctx.non_blocking_aio || writes_blocked()) {
+    queue(new ImageWriteRequest<>(m_image_ctx, c, off, len, buf, op_flags));
+  } else {
+    c->start_op();
+    ImageRequest<>::aio_write(&m_image_ctx, c, off, len, buf, op_flags);
+    finish_in_flight_op();
+  }
+}
+
+void ImageRequestWQ::aio_discard(AioCompletion *c, uint64_t off,
+                                 uint64_t len, bool native_async) {
+  c->init_time(&m_image_ctx, AIO_TYPE_DISCARD);
+  CephContext *cct = m_image_ctx.cct;
+  ldout(cct, 20) << "aio_discard: ictx=" << &m_image_ctx << ", "
+                 << "completion=" << c << ", off=" << off << ", len=" << len
+                 << dendl;
+
+  if (native_async && m_image_ctx.event_socket.is_valid()) {
+    c->set_event_notify(true);
+  }
+
+  if (!start_in_flight_op(c)) {
+    return;
+  }
+
+  RWLock::RLocker owner_locker(m_image_ctx.owner_lock);
+  if (m_image_ctx.non_blocking_aio || writes_blocked()) {
+    queue(new ImageDiscardRequest<>(m_image_ctx, c, off, len));
+  } else {
+    c->start_op();
+    ImageRequest<>::aio_discard(&m_image_ctx, c, off, len);
+    finish_in_flight_op();
+  }
+}
+
+void ImageRequestWQ::aio_flush(AioCompletion *c, bool native_async) {
+  c->init_time(&m_image_ctx, AIO_TYPE_FLUSH);
+  CephContext *cct = m_image_ctx.cct;
+  ldout(cct, 20) << "aio_flush: ictx=" << &m_image_ctx << ", "
+                 << "completion=" << c << dendl;
+
+  if (native_async && m_image_ctx.event_socket.is_valid()) {
+    c->set_event_notify(true);
+  }
+
+  if (!start_in_flight_op(c)) {
+    return;
+  }
+
+  RWLock::RLocker owner_locker(m_image_ctx.owner_lock);
+  if (m_image_ctx.non_blocking_aio || writes_blocked() || !writes_empty()) {
+    queue(new ImageFlushRequest<>(m_image_ctx, c));
+  } else {
+    ImageRequest<>::aio_flush(&m_image_ctx, c);
+    finish_in_flight_op();
+  }
+}
+
+void ImageRequestWQ::shut_down(Context *on_shutdown) {
+  assert(m_image_ctx.owner_lock.is_locked());
+
+  {
+    RWLock::WLocker locker(m_lock);
+    assert(!m_shutdown);
+    m_shutdown = true;
+
+    CephContext *cct = m_image_ctx.cct;
+    ldout(cct, 5) << __func__ << ": in_flight=" << m_in_flight_ops.read()
+                  << dendl;
+    if (m_in_flight_ops.read() > 0) {
+      m_on_shutdown = on_shutdown;
+      return;
+    }
+  }
+
+  // ensure that all in-flight IO is flushed
+  m_image_ctx.flush(on_shutdown);
+}
+
+bool ImageRequestWQ::is_lock_request_needed() const {
+  RWLock::RLocker locker(m_lock);
+  return (m_queued_writes.read() > 0 ||
+          (m_require_lock_on_read && m_queued_reads.read() > 0));
+}
+
+int ImageRequestWQ::block_writes() {
+  C_SaferCond cond_ctx;
+  block_writes(&cond_ctx);
+  return cond_ctx.wait();
+}
+
+void ImageRequestWQ::block_writes(Context *on_blocked) {
+  assert(m_image_ctx.owner_lock.is_locked());
+  CephContext *cct = m_image_ctx.cct;
+
+  {
+    RWLock::WLocker locker(m_lock);
+    ++m_write_blockers;
+    ldout(cct, 5) << __func__ << ": " << &m_image_ctx << ", "
+                  << "num=" << m_write_blockers << dendl;
+    if (!m_write_blocker_contexts.empty() || m_in_progress_writes.read() > 0) {
+      m_write_blocker_contexts.push_back(on_blocked);
+      return;
+    }
+  }
+
+  // ensure that all in-flight IO is flushed
+  m_image_ctx.flush(on_blocked);
+}
+
+void ImageRequestWQ::unblock_writes() {
+  CephContext *cct = m_image_ctx.cct;
+
+  bool wake_up = false;
+  {
+    RWLock::WLocker locker(m_lock);
+    assert(m_write_blockers > 0);
+    --m_write_blockers;
+
+    ldout(cct, 5) << __func__ << ": " << &m_image_ctx << ", "
+                  << "num=" << m_write_blockers << dendl;
+    if (m_write_blockers == 0) {
+      wake_up = true;
+    }
+  }
+
+  if (wake_up) {
+    signal();
+  }
+}
+
+void ImageRequestWQ::set_require_lock_on_read() {
+  CephContext *cct = m_image_ctx.cct;
+  ldout(cct, 20) << __func__ << dendl;
+
+  RWLock::WLocker locker(m_lock);
+  m_require_lock_on_read = true;
+}
+
+void ImageRequestWQ::clear_require_lock_on_read() {
+  CephContext *cct = m_image_ctx.cct;
+  ldout(cct, 20) << __func__ << dendl;
+
+  {
+    RWLock::WLocker locker(m_lock);
+    if (!m_require_lock_on_read) {
+      return;
+    }
+
+    m_require_lock_on_read = false;
+  }
+  signal();
+}
+
+void *ImageRequestWQ::_void_dequeue() {
+  ImageRequest<> *peek_item = front();
+
+  // no IO ops available or refresh in-progress (IO stalled)
+  if (peek_item == nullptr || m_refresh_in_progress) {
+    return nullptr;
+  }
+
+  bool refresh_required = m_image_ctx.state->is_refresh_required();
+  {
+    RWLock::RLocker locker(m_lock);
+    if (peek_item->is_write_op()) {
+      if (m_write_blockers > 0) {
+        return nullptr;
+      }
+
+      // refresh will requeue the op -- don't count it as in-progress
+      if (!refresh_required) {
+        m_in_progress_writes.inc();
+      }
+    } else if (m_require_lock_on_read) {
+      return nullptr;
+    }
+  }
+
+  ImageRequest<> *item = reinterpret_cast<ImageRequest<> *>(
+    ThreadPool::PointerWQ<ImageRequest<> >::_void_dequeue());
+  assert(peek_item == item);
+
+  if (refresh_required) {
+    ldout(m_image_ctx.cct, 15) << "image refresh required: delaying IO " << item
+                               << dendl;
+
+    // stall IO until the refresh completes
+    m_refresh_in_progress = true;
+
+    get_pool_lock().Unlock();
+    m_image_ctx.state->refresh(new C_RefreshFinish(this, item));
+    get_pool_lock().Lock();
+    return nullptr;
+  }
+
+  item->start_op();
+  return item;
+}
+
+void ImageRequestWQ::process(ImageRequest<> *req) {
+  CephContext *cct = m_image_ctx.cct;
+  ldout(cct, 20) << __func__ << ": ictx=" << &m_image_ctx << ", "
+                 << "req=" << req << dendl;
+
+  req->send();
+
+  finish_queued_op(req);
+  if (req->is_write_op()) {
+    finish_in_progress_write();
+  }
+  delete req;
+
+  finish_in_flight_op();
+}
+
+void ImageRequestWQ::finish_queued_op(ImageRequest<> *req) {
+  RWLock::RLocker locker(m_lock);
+  if (req->is_write_op()) {
+    assert(m_queued_writes.read() > 0);
+    m_queued_writes.dec();
+  } else {
+    assert(m_queued_reads.read() > 0);
+    m_queued_reads.dec();
+  }
+}
+
+void ImageRequestWQ::finish_in_progress_write() {
+  bool writes_blocked = false;
+  {
+    RWLock::RLocker locker(m_lock);
+    assert(m_in_progress_writes.read() > 0);
+    if (m_in_progress_writes.dec() == 0 &&
+        !m_write_blocker_contexts.empty()) {
+      writes_blocked = true;
+    }
+  }
+
+  if (writes_blocked) {
+    m_image_ctx.flush(new C_BlockedWrites(this));
+  }
+}
+
+int ImageRequestWQ::start_in_flight_op(AioCompletion *c) {
+  RWLock::RLocker locker(m_lock);
+
+  if (m_shutdown) {
+    CephContext *cct = m_image_ctx.cct;
+    lderr(cct) << "IO received on closed image" << dendl;
+
+    c->fail(-ESHUTDOWN);
+    return false;
+  }
+
+  m_in_flight_ops.inc();
+  return true;
+}
+
+void ImageRequestWQ::finish_in_flight_op() {
+  Context *on_shutdown;
+  {
+    RWLock::RLocker locker(m_lock);
+    if (m_in_flight_ops.dec() > 0 || !m_shutdown) {
+      return;
+    }
+    on_shutdown = m_on_shutdown;
+  }
+
+  CephContext *cct = m_image_ctx.cct;
+  ldout(cct, 5) << __func__ << ": completing shut down" << dendl;
+
+  assert(on_shutdown != nullptr);
+  m_image_ctx.flush(on_shutdown);
+}
+
+bool ImageRequestWQ::is_lock_required() const {
+  assert(m_image_ctx.owner_lock.is_locked());
+  if (m_image_ctx.exclusive_lock == NULL) {
+    return false;
+  }
+
+  return (!m_image_ctx.exclusive_lock->is_lock_owner());
+}
+
+void ImageRequestWQ::queue(ImageRequest<> *req) {
+  CephContext *cct = m_image_ctx.cct;
+  ldout(cct, 20) << __func__ << ": ictx=" << &m_image_ctx << ", "
+                 << "req=" << req << dendl;
+
+  assert(m_image_ctx.owner_lock.is_locked());
+  bool write_op = req->is_write_op();
+  bool lock_required = (m_image_ctx.exclusive_lock != nullptr &&
+                        ((write_op && is_lock_required()) ||
+                          (!write_op && m_require_lock_on_read)));
+
+  if (lock_required && !m_image_ctx.get_exclusive_lock_policy()->may_auto_request_lock()) {
+    lderr(cct) << "op requires exclusive lock" << dendl;
+    req->fail(-EROFS);
+    delete req;
+    finish_in_flight_op();
+    return;
+  }
+
+  if (write_op) {
+    m_queued_writes.inc();
+  } else {
+    m_queued_reads.inc();
+  }
+
+  ThreadPool::PointerWQ<ImageRequest<> >::queue(req);
+
+  if (lock_required) {
+    m_image_ctx.exclusive_lock->acquire_lock(nullptr);
+  }
+}
+
+void ImageRequestWQ::handle_refreshed(int r, ImageRequest<> *req) {
+  CephContext *cct = m_image_ctx.cct;
+  ldout(cct, 15) << "resuming IO after image refresh: r=" << r << ", "
+                 << "req=" << req << dendl;
+  if (r < 0) {
+    process_finish();
+    req->fail(r);
+    finish_queued_op(req);
+    delete req;
+    finish_in_flight_op();
+  } else {
+    // since IO was stalled for refresh -- original IO order is preserved
+    // if we requeue this op for work queue processing
+    requeue(req);
+  }
+
+  m_refresh_in_progress = false;
+  signal();
+
+  // refresh might have enabled exclusive lock -- IO stalled until
+  // we acquire the lock
+  RWLock::RLocker owner_locker(m_image_ctx.owner_lock);
+  if (is_lock_required() && is_lock_request_needed()) {
+    m_image_ctx.exclusive_lock->acquire_lock(nullptr);
+  }
+}
+
+void ImageRequestWQ::handle_blocked_writes(int r) {
+  Contexts contexts;
+  {
+    RWLock::WLocker locker(m_lock);
+    contexts.swap(m_write_blocker_contexts);
+  }
+
+  for (auto ctx : contexts) {
+    ctx->complete(0);
+  }
+}
+
+} // namespace io
+} // namespace librbd
diff --git a/src/librbd/io/ImageRequestWQ.h b/src/librbd/io/ImageRequestWQ.h
new file mode 100644 (file)
index 0000000..6e93ed1
--- /dev/null
@@ -0,0 +1,125 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_IO_IMAGE_REQUEST_WQ_H
+#define CEPH_LIBRBD_IO_IMAGE_REQUEST_WQ_H
+
+#include "include/Context.h"
+#include "include/atomic.h"
+#include "common/RWLock.h"
+#include "common/WorkQueue.h"
+#include <list>
+
+namespace librbd {
+
+class ImageCtx;
+
+namespace io {
+
+class AioCompletion;
+template <typename> class ImageRequest;
+
+class ImageRequestWQ : protected ThreadPool::PointerWQ<ImageRequest<ImageCtx> > {
+public:
+  ImageRequestWQ(ImageCtx *image_ctx, const string &name, time_t ti,
+                 ThreadPool *tp);
+
+  ssize_t read(uint64_t off, uint64_t len, char *buf, int op_flags);
+  ssize_t write(uint64_t off, uint64_t len, const char *buf, int op_flags);
+  int discard(uint64_t off, uint64_t len);
+
+  void aio_read(AioCompletion *c, uint64_t off, uint64_t len, char *buf,
+                bufferlist *pbl, int op_flags, bool native_async=true);
+  void aio_write(AioCompletion *c, uint64_t off, uint64_t len, const char *buf,
+                 int op_flags, bool native_async=true);
+  void aio_discard(AioCompletion *c, uint64_t off, uint64_t len,
+                   bool native_async=true);
+  void aio_flush(AioCompletion *c, bool native_async=true);
+
+  using ThreadPool::PointerWQ<ImageRequest<ImageCtx> >::drain;
+  using ThreadPool::PointerWQ<ImageRequest<ImageCtx> >::empty;
+
+  void shut_down(Context *on_shutdown);
+
+  bool is_lock_request_needed() const;
+
+  inline bool writes_blocked() const {
+    RWLock::RLocker locker(m_lock);
+    return (m_write_blockers > 0);
+  }
+
+  int block_writes();
+  void block_writes(Context *on_blocked);
+  void unblock_writes();
+
+  void set_require_lock_on_read();
+  void clear_require_lock_on_read();
+
+protected:
+  virtual void *_void_dequeue();
+  virtual void process(ImageRequest<ImageCtx> *req);
+
+private:
+  typedef std::list<Context *> Contexts;
+
+  struct C_RefreshFinish : public Context {
+    ImageRequestWQ *aio_work_queue;
+    ImageRequest<ImageCtx> *aio_image_request;
+
+    C_RefreshFinish(ImageRequestWQ *aio_work_queue,
+                    ImageRequest<ImageCtx> *aio_image_request)
+      : aio_work_queue(aio_work_queue), aio_image_request(aio_image_request) {
+    }
+    virtual void finish(int r) override {
+      aio_work_queue->handle_refreshed(r, aio_image_request);
+    }
+  };
+
+  struct C_BlockedWrites : public Context {
+    ImageRequestWQ *aio_work_queue;
+    C_BlockedWrites(ImageRequestWQ *_aio_work_queue)
+      : aio_work_queue(_aio_work_queue) {
+    }
+
+    virtual void finish(int r) {
+      aio_work_queue->handle_blocked_writes(r);
+    }
+  };
+
+  ImageCtx &m_image_ctx;
+  mutable RWLock m_lock;
+  Contexts m_write_blocker_contexts;
+  uint32_t m_write_blockers;
+  bool m_require_lock_on_read = false;
+  atomic_t m_in_progress_writes;
+  atomic_t m_queued_reads;
+  atomic_t m_queued_writes;
+  atomic_t m_in_flight_ops;
+
+  bool m_refresh_in_progress;
+
+  bool m_shutdown;
+  Context *m_on_shutdown;
+
+  inline bool writes_empty() const {
+    RWLock::RLocker locker(m_lock);
+    return (m_queued_writes.read() == 0);
+  }
+
+  void finish_queued_op(ImageRequest<ImageCtx> *req);
+  void finish_in_progress_write();
+
+  int start_in_flight_op(AioCompletion *c);
+  void finish_in_flight_op();
+
+  bool is_lock_required() const;
+  void queue(ImageRequest<ImageCtx> *req);
+
+  void handle_refreshed(int r, ImageRequest<ImageCtx> *req);
+  void handle_blocked_writes(int r);
+};
+
+} // namespace io
+} // namespace librbd
+
+#endif // CEPH_LIBRBD_IO_IMAGE_REQUEST_WQ_H
diff --git a/src/librbd/io/ObjectRequest.cc b/src/librbd/io/ObjectRequest.cc
new file mode 100644 (file)
index 0000000..8002c84
--- /dev/null
@@ -0,0 +1,626 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "librbd/io/ObjectRequest.h"
+#include "common/ceph_context.h"
+#include "common/dout.h"
+#include "common/errno.h"
+#include "common/Mutex.h"
+#include "common/RWLock.h"
+#include "common/WorkQueue.h"
+#include "include/Context.h"
+
+#include "librbd/ExclusiveLock.h"
+#include "librbd/ImageCtx.h"
+#include "librbd/ObjectMap.h"
+#include "librbd/Utils.h"
+#include "librbd/io/AioCompletion.h"
+#include "librbd/io/CopyupRequest.h"
+#include "librbd/io/ImageRequest.h"
+
+#include <boost/bind.hpp>
+#include <boost/optional.hpp>
+
+#define dout_subsys ceph_subsys_rbd
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::io::ObjectRequest: "
+
+namespace librbd {
+namespace io {
+
+template <typename I>
+ObjectRequest<I>*
+ObjectRequest<I>::create_remove(I *ictx, const std::string &oid,
+                                uint64_t object_no,
+                                const ::SnapContext &snapc,
+                                Context *completion) {
+  return new ObjectRemoveRequest(util::get_image_ctx(ictx), oid, object_no,
+                                 snapc, completion);
+}
+
+template <typename I>
+ObjectRequest<I>*
+ObjectRequest<I>::create_truncate(I *ictx, const std::string &oid,
+                                  uint64_t object_no, uint64_t object_off,
+                                  const ::SnapContext &snapc,
+                                  Context *completion) {
+  return new ObjectTruncateRequest(util::get_image_ctx(ictx), oid, object_no,
+                                   object_off, snapc, completion);
+}
+
+template <typename I>
+ObjectRequest<I>*
+ObjectRequest<I>::create_write(I *ictx, const std::string &oid,
+                               uint64_t object_no, uint64_t object_off,
+                               const ceph::bufferlist &data,
+                               const ::SnapContext &snapc,
+                               Context *completion, int op_flags) {
+  return new ObjectWriteRequest(util::get_image_ctx(ictx), oid, object_no,
+                                object_off, data, snapc, completion, op_flags);
+}
+
+template <typename I>
+ObjectRequest<I>*
+ObjectRequest<I>::create_zero(I *ictx, const std::string &oid,
+                              uint64_t object_no, uint64_t object_off,
+                              uint64_t object_len,
+                              const ::SnapContext &snapc,
+                              Context *completion) {
+  return new ObjectZeroRequest(util::get_image_ctx(ictx), oid, object_no,
+                               object_off, object_len, snapc, completion);
+}
+
+template <typename I>
+ObjectRequest<I>::ObjectRequest(ImageCtx *ictx, const std::string &oid,
+                                uint64_t objectno, uint64_t off,
+                                uint64_t len, librados::snap_t snap_id,
+                                Context *completion, bool hide_enoent)
+  : m_ictx(ictx), m_oid(oid), m_object_no(objectno), m_object_off(off),
+    m_object_len(len), m_snap_id(snap_id), m_completion(completion),
+    m_hide_enoent(hide_enoent) {
+
+  Striper::extent_to_file(m_ictx->cct, &m_ictx->layout, m_object_no,
+                          0, m_ictx->layout.object_size, m_parent_extents);
+
+  RWLock::RLocker snap_locker(m_ictx->snap_lock);
+  RWLock::RLocker parent_locker(m_ictx->parent_lock);
+  compute_parent_extents();
+}
+
+template <typename I>
+void ObjectRequest<I>::complete(int r)
+{
+  if (should_complete(r)) {
+    ldout(m_ictx->cct, 20) << "complete " << this << dendl;
+    if (m_hide_enoent && r == -ENOENT) {
+      r = 0;
+    }
+    m_completion->complete(r);
+    delete this;
+  }
+}
+
+template <typename I>
+bool ObjectRequest<I>::compute_parent_extents() {
+  assert(m_ictx->snap_lock.is_locked());
+  assert(m_ictx->parent_lock.is_locked());
+
+  uint64_t parent_overlap;
+  int r = m_ictx->get_parent_overlap(m_snap_id, &parent_overlap);
+  if (r < 0) {
+    // NOTE: it's possible for a snapshot to be deleted while we are
+    // still reading from it
+    lderr(m_ictx->cct) << this << " compute_parent_extents: failed to "
+                       << "retrieve parent overlap: " << cpp_strerror(r)
+                       << dendl;
+    m_has_parent = false;
+    m_parent_extents.clear();
+    return false;
+  }
+
+  uint64_t object_overlap = m_ictx->prune_parent_extents(
+    m_parent_extents, parent_overlap);
+  if (object_overlap > 0) {
+    ldout(m_ictx->cct, 20) << this << " compute_parent_extents: "
+                           << "overlap " << parent_overlap << " "
+                           << "extents " << m_parent_extents << dendl;
+    m_has_parent = !m_parent_extents.empty();
+    return true;
+  }
+  return false;
+}
+
+static inline bool is_copy_on_read(ImageCtx *ictx, librados::snap_t snap_id) {
+  assert(ictx->snap_lock.is_locked());
+  return (ictx->clone_copy_on_read &&
+          !ictx->read_only && snap_id == CEPH_NOSNAP &&
+          (ictx->exclusive_lock == nullptr ||
+           ictx->exclusive_lock->is_lock_owner()));
+}
+
+/** read **/
+
+template <typename I>
+ObjectReadRequest<I>::ObjectReadRequest(I *ictx, const std::string &oid,
+                                        uint64_t objectno, uint64_t offset,
+                                        uint64_t len, Extents& be,
+                                        librados::snap_t snap_id, bool sparse,
+                                        Context *completion, int op_flags)
+  : ObjectRequest<I>(util::get_image_ctx(ictx), oid, objectno, offset, len,
+                     snap_id, completion, false),
+    m_buffer_extents(be), m_tried_parent(false), m_sparse(sparse),
+    m_op_flags(op_flags), m_state(LIBRBD_AIO_READ_FLAT) {
+  guard_read();
+}
+
+template <typename I>
+void ObjectReadRequest<I>::guard_read()
+{
+  ImageCtx *image_ctx = this->m_ictx;
+  RWLock::RLocker snap_locker(image_ctx->snap_lock);
+  RWLock::RLocker parent_locker(image_ctx->parent_lock);
+
+  if (this->has_parent()) {
+    ldout(image_ctx->cct, 20) << __func__ << " guarding read" << dendl;
+    m_state = LIBRBD_AIO_READ_GUARD;
+  }
+}
+
+template <typename I>
+bool ObjectReadRequest<I>::should_complete(int r)
+{
+  ImageCtx *image_ctx = this->m_ictx;
+  ldout(image_ctx->cct, 20) << "should_complete " << this << " "
+                            << this->m_oid << " "
+                            << this->m_object_off << "~" << this->m_object_len
+                            << " r = " << r << dendl;
+
+  bool finished = true;
+
+  switch (m_state) {
+  case LIBRBD_AIO_READ_GUARD:
+    ldout(image_ctx->cct, 20) << "should_complete " << this
+                              << " READ_CHECK_GUARD" << dendl;
+
+    // This is the step to read from parent
+    if (!m_tried_parent && r == -ENOENT) {
+      {
+        RWLock::RLocker snap_locker(image_ctx->snap_lock);
+        RWLock::RLocker parent_locker(image_ctx->parent_lock);
+        if (image_ctx->parent == NULL) {
+          ldout(image_ctx->cct, 20) << "parent is gone; do nothing" << dendl;
+          m_state = LIBRBD_AIO_READ_FLAT;
+          finished = false;
+          break;
+        }
+
+        // calculate reverse mapping onto the image
+        vector<pair<uint64_t,uint64_t> > parent_extents;
+        Striper::extent_to_file(image_ctx->cct, &image_ctx->layout,
+                                this->m_object_no, this->m_object_off,
+                                this->m_object_len, parent_extents);
+
+        uint64_t parent_overlap = 0;
+        uint64_t object_overlap = 0;
+        r = image_ctx->get_parent_overlap(this->m_snap_id, &parent_overlap);
+        if (r == 0) {
+          object_overlap = image_ctx->prune_parent_extents(parent_extents,
+                                                           parent_overlap);
+        }
+
+        if (object_overlap > 0) {
+          m_tried_parent = true;
+          if (is_copy_on_read(image_ctx, this->m_snap_id)) {
+            m_state = LIBRBD_AIO_READ_COPYUP;
+          }
+
+          read_from_parent(std::move(parent_extents));
+          finished = false;
+        }
+      }
+    }
+    break;
+  case LIBRBD_AIO_READ_COPYUP:
+    ldout(image_ctx->cct, 20) << "should_complete " << this << " READ_COPYUP"
+                              << dendl;
+    // This is the extra step for copy-on-read: kick off an asynchronous copyup.
+    // It is different from copy-on-write as asynchronous copyup will finish
+    // by itself so state won't go back to LIBRBD_AIO_READ_GUARD.
+
+    assert(m_tried_parent);
+    if (r > 0) {
+      // If read entire object from parent success and CoR is possible, kick
+      // off a asynchronous copyup. This approach minimizes the latency
+      // impact.
+      send_copyup();
+    }
+    break;
+  case LIBRBD_AIO_READ_FLAT:
+    ldout(image_ctx->cct, 20) << "should_complete " << this << " READ_FLAT"
+                              << dendl;
+    // The read content should be deposit in m_read_data
+    break;
+  default:
+    lderr(image_ctx->cct) << "invalid request state: " << m_state << dendl;
+    ceph_abort();
+  }
+
+  return finished;
+}
+
+template <typename I>
+void ObjectReadRequest<I>::send() {
+  ImageCtx *image_ctx = this->m_ictx;
+  ldout(image_ctx->cct, 20) << "send " << this << " " << this->m_oid << " "
+                            << this->m_object_off << "~" << this->m_object_len
+                            << dendl;
+
+  {
+    RWLock::RLocker snap_locker(image_ctx->snap_lock);
+
+    // send read request to parent if the object doesn't exist locally
+    if (image_ctx->object_map != nullptr &&
+        !image_ctx->object_map->object_may_exist(this->m_object_no)) {
+      image_ctx->op_work_queue->queue(util::create_context_callback<
+        ObjectRequest<I> >(this), -ENOENT);
+      return;
+    }
+  }
+
+  librados::ObjectReadOperation op;
+  int flags = image_ctx->get_read_flags(this->m_snap_id);
+  if (m_sparse) {
+    op.sparse_read(this->m_object_off, this->m_object_len, &m_ext_map,
+                   &m_read_data, nullptr);
+  } else {
+    op.read(this->m_object_off, this->m_object_len, &m_read_data, nullptr);
+  }
+  op.set_op_flags2(m_op_flags);
+
+  librados::AioCompletion *rados_completion =
+    util::create_rados_ack_callback(this);
+  int r = image_ctx->data_ctx.aio_operate(this->m_oid, rados_completion, &op,
+                                          flags, nullptr);
+  assert(r == 0);
+
+  rados_completion->release();
+}
+
+template <typename I>
+void ObjectReadRequest<I>::send_copyup()
+{
+  ImageCtx *image_ctx = this->m_ictx;
+  {
+    RWLock::RLocker snap_locker(image_ctx->snap_lock);
+    RWLock::RLocker parent_locker(image_ctx->parent_lock);
+    if (!this->compute_parent_extents() ||
+        (image_ctx->exclusive_lock != nullptr &&
+         !image_ctx->exclusive_lock->is_lock_owner())) {
+      return;
+    }
+  }
+
+  Mutex::Locker copyup_locker(image_ctx->copyup_list_lock);
+  map<uint64_t, CopyupRequest*>::iterator it =
+    image_ctx->copyup_list.find(this->m_object_no);
+  if (it == image_ctx->copyup_list.end()) {
+    // create and kick off a CopyupRequest
+    CopyupRequest *new_req = new CopyupRequest(
+      image_ctx, this->m_oid, this->m_object_no,
+      std::move(this->m_parent_extents));
+    this->m_parent_extents.clear();
+
+    image_ctx->copyup_list[this->m_object_no] = new_req;
+    new_req->send();
+  }
+}
+
+template <typename I>
+void ObjectReadRequest<I>::read_from_parent(Extents&& parent_extents)
+{
+  ImageCtx *image_ctx = this->m_ictx;
+  AioCompletion *parent_completion = AioCompletion::create_and_start<
+    ObjectRequest<I> >(this, image_ctx, AIO_TYPE_READ);
+
+  ldout(image_ctx->cct, 20) << "read_from_parent this = " << this
+                            << " parent completion " << parent_completion
+                            << " extents " << parent_extents
+                            << dendl;
+  ImageRequest<>::aio_read(image_ctx->parent, parent_completion,
+                           std::move(parent_extents), nullptr, &m_read_data, 0);
+}
+
+/** write **/
+
+AbstractObjectWriteRequest::AbstractObjectWriteRequest(ImageCtx *ictx,
+                                                       const std::string &oid,
+                                                       uint64_t object_no,
+                                                       uint64_t object_off,
+                                                       uint64_t len,
+                                                       const ::SnapContext &snapc,
+                                                       Context *completion,
+                                                       bool hide_enoent)
+  : ObjectRequest(ictx, oid, object_no, object_off, len, CEPH_NOSNAP,
+                  completion, hide_enoent),
+    m_state(LIBRBD_AIO_WRITE_FLAT), m_snap_seq(snapc.seq.val)
+{
+  m_snaps.insert(m_snaps.end(), snapc.snaps.begin(), snapc.snaps.end());
+}
+
+void AbstractObjectWriteRequest::guard_write()
+{
+  if (has_parent()) {
+    m_state = LIBRBD_AIO_WRITE_GUARD;
+    m_write.assert_exists();
+    ldout(m_ictx->cct, 20) << __func__ << " guarding write" << dendl;
+  }
+}
+
+bool AbstractObjectWriteRequest::should_complete(int r)
+{
+  ldout(m_ictx->cct, 20) << get_op_type() << " " << this << " " << m_oid
+                         << " " << m_object_off << "~" << m_object_len
+                         << " should_complete: r = " << r << dendl;
+
+  bool finished = true;
+  switch (m_state) {
+  case LIBRBD_AIO_WRITE_PRE:
+    ldout(m_ictx->cct, 20) << "WRITE_PRE" << dendl;
+    if (r < 0) {
+      return true;
+    }
+
+    send_write_op();
+    finished = false;
+    break;
+
+  case LIBRBD_AIO_WRITE_POST:
+    ldout(m_ictx->cct, 20) << "WRITE_POST" << dendl;
+    finished = true;
+    break;
+
+  case LIBRBD_AIO_WRITE_GUARD:
+    ldout(m_ictx->cct, 20) << "WRITE_CHECK_GUARD" << dendl;
+
+    if (r == -ENOENT) {
+      handle_write_guard();
+      finished = false;
+      break;
+    } else if (r < 0) {
+      // pass the error code to the finish context
+      m_state = LIBRBD_AIO_WRITE_ERROR;
+      complete(r);
+      finished = false;
+      break;
+    }
+
+    finished = send_post_object_map_update();
+    break;
+
+  case LIBRBD_AIO_WRITE_COPYUP:
+    ldout(m_ictx->cct, 20) << "WRITE_COPYUP" << dendl;
+    if (r < 0) {
+      m_state = LIBRBD_AIO_WRITE_ERROR;
+      complete(r);
+      finished = false;
+    } else {
+      finished = send_post_object_map_update();
+    }
+    break;
+
+  case LIBRBD_AIO_WRITE_FLAT:
+    ldout(m_ictx->cct, 20) << "WRITE_FLAT" << dendl;
+
+    finished = send_post_object_map_update();
+    break;
+
+  case LIBRBD_AIO_WRITE_ERROR:
+    assert(r < 0);
+    lderr(m_ictx->cct) << "WRITE_ERROR: " << cpp_strerror(r) << dendl;
+    break;
+
+  default:
+    lderr(m_ictx->cct) << "invalid request state: " << m_state << dendl;
+    ceph_abort();
+  }
+
+  return finished;
+}
+
+void AbstractObjectWriteRequest::send() {
+  ldout(m_ictx->cct, 20) << "send " << get_op_type() << " " << this <<" "
+                         << m_oid << " " << m_object_off << "~"
+                         << m_object_len << dendl;
+  {
+    RWLock::RLocker snap_lock(m_ictx->snap_lock);
+    if (m_ictx->object_map == nullptr) {
+      m_object_exist = true;
+    } else {
+      // should have been flushed prior to releasing lock
+      assert(m_ictx->exclusive_lock->is_lock_owner());
+      m_object_exist = m_ictx->object_map->object_may_exist(m_object_no);
+    }
+  }
+
+  send_write();
+}
+
+void AbstractObjectWriteRequest::send_pre_object_map_update() {
+  ldout(m_ictx->cct, 20) << __func__ << dendl;
+
+  {
+    RWLock::RLocker snap_lock(m_ictx->snap_lock);
+    if (m_ictx->object_map != nullptr) {
+      uint8_t new_state;
+      pre_object_map_update(&new_state);
+      RWLock::WLocker object_map_locker(m_ictx->object_map_lock);
+      ldout(m_ictx->cct, 20) << __func__ << this << " " << m_oid << " "
+                             << m_object_off << "~" << m_object_len
+                             << dendl;
+      m_state = LIBRBD_AIO_WRITE_PRE;
+
+      if (m_ictx->object_map->aio_update<ObjectRequest>(
+            CEPH_NOSNAP, m_object_no, new_state, {}, this)) {
+        return;
+      }
+    }
+  }
+
+  send_write_op();
+}
+
+bool AbstractObjectWriteRequest::send_post_object_map_update() {
+  RWLock::RLocker snap_locker(m_ictx->snap_lock);
+  if (m_ictx->object_map == nullptr || !post_object_map_update()) {
+    return true;
+  }
+
+  // should have been flushed prior to releasing lock
+  assert(m_ictx->exclusive_lock->is_lock_owner());
+
+  RWLock::WLocker object_map_locker(m_ictx->object_map_lock);
+  ldout(m_ictx->cct, 20) << __func__ << this << " " << m_oid << " "
+                         << m_object_off << "~" << m_object_len << dendl;
+  m_state = LIBRBD_AIO_WRITE_POST;
+
+  if (m_ictx->object_map->aio_update<ObjectRequest>(
+        CEPH_NOSNAP, m_object_no, OBJECT_NONEXISTENT, OBJECT_PENDING, this)) {
+    return false;
+  }
+
+  return true;
+}
+
+void AbstractObjectWriteRequest::send_write() {
+  ldout(m_ictx->cct, 20) << "send_write " << this << " " << m_oid << " "
+                        << m_object_off << "~" << m_object_len
+                         << " object exist " << m_object_exist << dendl;
+
+  if (!m_object_exist && has_parent()) {
+    m_state = LIBRBD_AIO_WRITE_GUARD;
+    handle_write_guard();
+  } else {
+    send_pre_object_map_update();
+  }
+}
+
+void AbstractObjectWriteRequest::send_copyup()
+{
+  ldout(m_ictx->cct, 20) << "send_copyup " << this << " " << m_oid << " "
+                         << m_object_off << "~" << m_object_len << dendl;
+  m_state = LIBRBD_AIO_WRITE_COPYUP;
+
+  m_ictx->copyup_list_lock.Lock();
+  map<uint64_t, CopyupRequest*>::iterator it =
+    m_ictx->copyup_list.find(m_object_no);
+  if (it == m_ictx->copyup_list.end()) {
+    CopyupRequest *new_req = new CopyupRequest(m_ictx, m_oid,
+                                               m_object_no,
+                                               std::move(m_parent_extents));
+    m_parent_extents.clear();
+
+    // make sure to wait on this CopyupRequest
+    new_req->append_request(this);
+    m_ictx->copyup_list[m_object_no] = new_req;
+
+    m_ictx->copyup_list_lock.Unlock();
+    new_req->send();
+  } else {
+    it->second->append_request(this);
+    m_ictx->copyup_list_lock.Unlock();
+  }
+}
+void AbstractObjectWriteRequest::send_write_op()
+{
+  m_state = LIBRBD_AIO_WRITE_FLAT;
+  if (m_guard) {
+    guard_write();
+  }
+
+  add_write_ops(&m_write);
+  assert(m_write.size() != 0);
+
+  librados::AioCompletion *rados_completion =
+    util::create_rados_safe_callback(this);
+  int r = m_ictx->data_ctx.aio_operate(m_oid, rados_completion, &m_write,
+                                       m_snap_seq, m_snaps);
+  assert(r == 0);
+  rados_completion->release();
+}
+void AbstractObjectWriteRequest::handle_write_guard()
+{
+  bool has_parent;
+  {
+    RWLock::RLocker snap_locker(m_ictx->snap_lock);
+    RWLock::RLocker parent_locker(m_ictx->parent_lock);
+    has_parent = compute_parent_extents();
+  }
+  // If parent still exists, overlap might also have changed.
+  if (has_parent) {
+    send_copyup();
+  } else {
+    // parent may have disappeared -- send original write again
+    ldout(m_ictx->cct, 20) << "should_complete(" << this
+                           << "): parent overlap now 0" << dendl;
+    send_write();
+  }
+}
+
+void ObjectWriteRequest::add_write_ops(librados::ObjectWriteOperation *wr) {
+  RWLock::RLocker snap_locker(m_ictx->snap_lock);
+  if (m_ictx->enable_alloc_hint &&
+      (m_ictx->object_map == nullptr || !m_object_exist)) {
+    wr->set_alloc_hint(m_ictx->get_object_size(), m_ictx->get_object_size());
+  }
+
+  if (m_object_off == 0 && m_object_len == m_ictx->get_object_size()) {
+    wr->write_full(m_write_data);
+  } else {
+    wr->write(m_object_off, m_write_data);
+  }
+  wr->set_op_flags2(m_op_flags);
+}
+
+void ObjectWriteRequest::send_write() {
+  bool write_full = (m_object_off == 0 && m_object_len == m_ictx->get_object_size());
+  ldout(m_ictx->cct, 20) << "send_write " << this << " " << m_oid << " "
+                         << m_object_off << "~" << m_object_len
+                         << " object exist " << m_object_exist
+                         << " write_full " << write_full << dendl;
+  if (write_full && !has_parent()) {
+    m_guard = false;
+  }
+
+  AbstractObjectWriteRequest::send_write();
+}
+
+void ObjectRemoveRequest::guard_write() {
+  // do nothing to disable write guard only if deep-copyup not required
+  RWLock::RLocker snap_locker(m_ictx->snap_lock);
+  if (!m_ictx->snaps.empty()) {
+    AbstractObjectWriteRequest::guard_write();
+  }
+}
+void ObjectRemoveRequest::send_write() {
+  ldout(m_ictx->cct, 20) << "send_write " << this << " " << m_oid << " "
+                         << m_object_off << "~" << m_object_len << dendl;
+  send_pre_object_map_update();
+}
+
+void ObjectTruncateRequest::send_write() {
+  ldout(m_ictx->cct, 20) << "send_write " << this << " " << m_oid
+                         << " truncate " << m_object_off << dendl;
+  if (!m_object_exist && ! has_parent()) {
+    m_state = LIBRBD_AIO_WRITE_FLAT;
+    Context *ctx = util::create_context_callback<ObjectRequest>(this);
+    m_ictx->op_work_queue->queue(ctx, 0);
+  } else {
+    AbstractObjectWriteRequest::send_write();
+  }
+}
+
+} // namespace io
+} // namespace librbd
+
+template class librbd::io::ObjectRequest<librbd::ImageCtx>;
+template class librbd::io::ObjectReadRequest<librbd::ImageCtx>;
diff --git a/src/librbd/io/ObjectRequest.h b/src/librbd/io/ObjectRequest.h
new file mode 100644 (file)
index 0000000..102e476
--- /dev/null
@@ -0,0 +1,459 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_IO_OBJECT_REQUEST_H
+#define CEPH_LIBRBD_IO_OBJECT_REQUEST_H
+
+#include "include/int_types.h"
+
+#include <map>
+
+#include "common/snap_types.h"
+#include "include/buffer.h"
+#include "include/rados/librados.hpp"
+#include "librbd/ObjectMap.h"
+
+class Context;
+
+namespace librbd {
+
+struct ImageCtx;
+
+namespace io {
+
+struct AioCompletion;
+class CopyupRequest;
+class ObjectRemoveRequest;
+class ObjectTruncateRequest;
+class ObjectWriteRequest;
+class ObjectZeroRequest;
+
+struct ObjectRequestHandle {
+  virtual ~ObjectRequestHandle() {
+  }
+
+  virtual void complete(int r) = 0;
+  virtual void send() = 0;
+};
+
+/**
+ * This class represents an I/O operation to a single RBD data object.
+ * Its subclasses encapsulate logic for dealing with special cases
+ * for I/O due to layering.
+ */
+template <typename ImageCtxT = ImageCtx>
+class ObjectRequest : public ObjectRequestHandle {
+public:
+  typedef std::vector<std::pair<uint64_t, uint64_t> > Extents;
+
+  static ObjectRequest* create_remove(ImageCtxT *ictx,
+                                      const std::string &oid,
+                                      uint64_t object_no,
+                                      const ::SnapContext &snapc,
+                                      Context *completion);
+  static ObjectRequest* create_truncate(ImageCtxT *ictx,
+                                        const std::string &oid,
+                                        uint64_t object_no,
+                                        uint64_t object_off,
+                                        const ::SnapContext &snapc,
+                                        Context *completion);
+  static ObjectRequest* create_write(ImageCtxT *ictx, const std::string &oid,
+                                     uint64_t object_no,
+                                     uint64_t object_off,
+                                     const ceph::bufferlist &data,
+                                     const ::SnapContext &snapc,
+                                     Context *completion, int op_flags);
+  static ObjectRequest* create_zero(ImageCtxT *ictx, const std::string &oid,
+                                    uint64_t object_no, uint64_t object_off,
+                                    uint64_t object_len,
+                                    const ::SnapContext &snapc,
+                                    Context *completion);
+
+  ObjectRequest(ImageCtx *ictx, const std::string &oid,
+                uint64_t objectno, uint64_t off, uint64_t len,
+                librados::snap_t snap_id,
+                Context *completion, bool hide_enoent);
+  virtual ~ObjectRequest() {}
+
+  virtual void add_copyup_ops(librados::ObjectWriteOperation *wr) {};
+
+  void complete(int r);
+
+  virtual bool should_complete(int r) = 0;
+  virtual void send() = 0;
+
+  bool has_parent() const {
+    return m_has_parent;
+  }
+
+  virtual bool is_op_payload_empty() const {
+    return false;
+  }
+
+  virtual const char *get_op_type() const = 0;
+  virtual bool pre_object_map_update(uint8_t *new_state) = 0;
+
+protected:
+  bool compute_parent_extents();
+
+  ImageCtx *m_ictx;
+  std::string m_oid;
+  uint64_t m_object_no, m_object_off, m_object_len;
+  librados::snap_t m_snap_id;
+  Context *m_completion;
+  Extents m_parent_extents;
+  bool m_hide_enoent;
+
+private:
+  bool m_has_parent = false;
+};
+
+template <typename ImageCtxT = ImageCtx>
+class ObjectReadRequest : public ObjectRequest<ImageCtxT> {
+public:
+  typedef std::vector<std::pair<uint64_t, uint64_t> > Extents;
+  typedef std::map<uint64_t, uint64_t> ExtentMap;
+
+  static ObjectReadRequest* create(ImageCtxT *ictx, const std::string &oid,
+                                   uint64_t objectno, uint64_t offset,
+                                   uint64_t len, Extents &buffer_extents,
+                                   librados::snap_t snap_id, bool sparse,
+                                   Context *completion, int op_flags) {
+    return new ObjectReadRequest(ictx, oid, objectno, offset, len,
+                                 buffer_extents, snap_id, sparse, completion,
+                                 op_flags);
+  }
+
+  ObjectReadRequest(ImageCtxT *ictx, const std::string &oid,
+                    uint64_t objectno, uint64_t offset, uint64_t len,
+                    Extents& buffer_extents, librados::snap_t snap_id,
+                    bool sparse, Context *completion, int op_flags);
+
+  virtual bool should_complete(int r);
+  virtual void send();
+  void guard_read();
+
+  inline uint64_t get_offset() const {
+    return this->m_object_off;
+  }
+  inline uint64_t get_length() const {
+    return this->m_object_len;
+  }
+  ceph::bufferlist &data() {
+    return m_read_data;
+  }
+  const Extents &get_buffer_extents() const {
+    return m_buffer_extents;
+  }
+  ExtentMap &get_extent_map() {
+    return m_ext_map;
+  }
+
+  const char *get_op_type() const {
+    return "read";
+  }
+
+  bool pre_object_map_update(uint8_t *new_state) {
+    return false;
+  }
+
+private:
+  Extents m_buffer_extents;
+  bool m_tried_parent;
+  bool m_sparse;
+  int m_op_flags;
+  ceph::bufferlist m_read_data;
+  ExtentMap m_ext_map;
+
+  /**
+   * Reads go through the following state machine to deal with
+   * layering:
+   *
+   *                          need copyup
+   * LIBRBD_AIO_READ_GUARD ---------------> LIBRBD_AIO_READ_COPYUP
+   *           |                                       |
+   *           v                                       |
+   *         done <------------------------------------/
+   *           ^
+   *           |
+   * LIBRBD_AIO_READ_FLAT
+   *
+   * Reads start in LIBRBD_AIO_READ_GUARD or _FLAT, depending on
+   * whether there is a parent or not.
+   */
+  enum read_state_d {
+    LIBRBD_AIO_READ_GUARD,
+    LIBRBD_AIO_READ_COPYUP,
+    LIBRBD_AIO_READ_FLAT
+  };
+
+  read_state_d m_state;
+
+  void send_copyup();
+
+  void read_from_parent(Extents&& image_extents);
+};
+
+class AbstractObjectWriteRequest : public ObjectRequest<> {
+public:
+  AbstractObjectWriteRequest(ImageCtx *ictx, const std::string &oid,
+                             uint64_t object_no, uint64_t object_off,
+                             uint64_t len, const ::SnapContext &snapc,
+                             Context *completion, bool hide_enoent);
+
+  virtual void add_copyup_ops(librados::ObjectWriteOperation *wr)
+  {
+    add_write_ops(wr);
+  }
+
+  virtual bool should_complete(int r);
+  virtual void send();
+
+  /**
+   * Writes go through the following state machine to deal with
+   * layering and the object map:
+   *
+   *   <start>
+   *      |
+   *      |\
+   *      | \       -or-
+   *      |  ---------------------------------> LIBRBD_AIO_WRITE_PRE
+   *      |                          .                            |
+   *      |                          .                            |
+   *      |                          .                            v
+   *      |                          . . .  . > LIBRBD_AIO_WRITE_FLAT. . .
+   *      |                                                       |      .
+   *      |                                                       |      .
+   *      |                                                       |      .
+   *      v                need copyup   (copyup performs pre)    |      .
+   * LIBRBD_AIO_WRITE_GUARD -----------> LIBRBD_AIO_WRITE_COPYUP  |      .
+   *  .       |                               |        .          |      .
+   *  .       |                               |        .          |      .
+   *  .       |                         /-----/        .          |      .
+   *  .       |                         |              .          |      .
+   *  .       \-------------------\     |     /-------------------/      .
+   *  .                           |     |     |        .                 .
+   *  .                           v     v     v        .                 .
+   *  .                       LIBRBD_AIO_WRITE_POST    .                 .
+   *  .                               |                .                 .
+   *  .                               |  . . . . . . . .                 .
+   *  .                               |  .                               .
+   *  .                               v  v                               .
+   *  . . . . . . . . . . . . . . > <finish> < . . . . . . . . . . . . . .
+   *
+   * The _PRE/_POST states are skipped if the object map is disabled.
+   * The write starts in _WRITE_GUARD or _FLAT depending on whether or not
+   * there is a parent overlap.
+   */
+protected:
+  enum write_state_d {
+    LIBRBD_AIO_WRITE_GUARD,
+    LIBRBD_AIO_WRITE_COPYUP,
+    LIBRBD_AIO_WRITE_FLAT,
+    LIBRBD_AIO_WRITE_PRE,
+    LIBRBD_AIO_WRITE_POST,
+    LIBRBD_AIO_WRITE_ERROR
+  };
+
+  write_state_d m_state;
+  librados::ObjectWriteOperation m_write;
+  uint64_t m_snap_seq;
+  std::vector<librados::snap_t> m_snaps;
+  bool m_object_exist;
+  bool m_guard = true;
+
+  virtual void add_write_ops(librados::ObjectWriteOperation *wr) = 0;
+  virtual void guard_write();
+  virtual bool post_object_map_update() {
+    return false;
+  }
+  virtual void send_write();
+  virtual void send_write_op();
+  virtual void handle_write_guard();
+
+  void send_pre_object_map_update();
+
+private:
+  bool send_post_object_map_update();
+  void send_copyup();
+};
+
+class ObjectWriteRequest : public AbstractObjectWriteRequest {
+public:
+  ObjectWriteRequest(ImageCtx *ictx, const std::string &oid, uint64_t object_no,
+                     uint64_t object_off, const ceph::bufferlist &data,
+                     const ::SnapContext &snapc, Context *completion,
+                     int op_flags)
+    : AbstractObjectWriteRequest(ictx, oid, object_no, object_off,
+                                 data.length(), snapc, completion, false),
+      m_write_data(data), m_op_flags(op_flags) {
+  }
+
+  bool is_op_payload_empty() const {
+    return (m_write_data.length() == 0);
+  }
+
+  virtual const char *get_op_type() const {
+    return "write";
+  }
+
+  virtual bool pre_object_map_update(uint8_t *new_state) {
+    *new_state = OBJECT_EXISTS;
+    return true;
+  }
+
+protected:
+  virtual void add_write_ops(librados::ObjectWriteOperation *wr);
+
+  virtual void send_write();
+
+private:
+  ceph::bufferlist m_write_data;
+  int m_op_flags;
+};
+
+class ObjectRemoveRequest : public AbstractObjectWriteRequest {
+public:
+  ObjectRemoveRequest(ImageCtx *ictx, const std::string &oid,
+                      uint64_t object_no, const ::SnapContext &snapc,
+                      Context *completion)
+    : AbstractObjectWriteRequest(ictx, oid, object_no, 0, 0, snapc, completion,
+                                 true),
+      m_object_state(OBJECT_NONEXISTENT) {
+  }
+
+  virtual const char* get_op_type() const {
+    if (has_parent()) {
+      return "remove (trunc)";
+    }
+    return "remove";
+  }
+
+  virtual bool pre_object_map_update(uint8_t *new_state) {
+    if (has_parent()) {
+      m_object_state = OBJECT_EXISTS;
+    } else {
+      m_object_state = OBJECT_PENDING;
+    }
+    *new_state = m_object_state;
+    return true;
+  }
+
+  virtual bool post_object_map_update() {
+    if (m_object_state == OBJECT_EXISTS) {
+      return false;
+    }
+    return true;
+  }
+
+  virtual void guard_write();
+  virtual void send_write();
+
+protected:
+  virtual void add_write_ops(librados::ObjectWriteOperation *wr) {
+    if (has_parent()) {
+      wr->truncate(0);
+    } else {
+      wr->remove();
+    }
+  }
+
+private:
+  uint8_t m_object_state;
+};
+
+class ObjectTrimRequest : public AbstractObjectWriteRequest {
+public:
+  // we'd need to only conditionally specify if a post object map
+  // update is needed. pre update is decided as usual (by checking
+  // the state of the object in the map).
+  ObjectTrimRequest(ImageCtx *ictx, const std::string &oid, uint64_t object_no,
+                    const ::SnapContext &snapc, Context *completion,
+                    bool post_object_map_update)
+    : AbstractObjectWriteRequest(ictx, oid, object_no, 0, 0, snapc, completion,
+                                 true),
+      m_post_object_map_update(post_object_map_update) {
+  }
+
+  virtual const char* get_op_type() const {
+    return "remove (trim)";
+  }
+
+  virtual bool pre_object_map_update(uint8_t *new_state) {
+    *new_state = OBJECT_PENDING;
+    return true;
+  }
+
+  virtual bool post_object_map_update() {
+    return m_post_object_map_update;
+  }
+
+protected:
+  virtual void add_write_ops(librados::ObjectWriteOperation *wr) {
+    wr->remove();
+  }
+
+private:
+  bool m_post_object_map_update;
+};
+
+class ObjectTruncateRequest : public AbstractObjectWriteRequest {
+public:
+  ObjectTruncateRequest(ImageCtx *ictx, const std::string &oid,
+                        uint64_t object_no, uint64_t object_off,
+                        const ::SnapContext &snapc, Context *completion)
+    : AbstractObjectWriteRequest(ictx, oid, object_no, object_off, 0, snapc,
+                                 completion, true) {
+  }
+
+  virtual const char* get_op_type() const {
+    return "truncate";
+  }
+
+  virtual bool pre_object_map_update(uint8_t *new_state) {
+    if (!m_object_exist && !has_parent())
+      *new_state = OBJECT_NONEXISTENT;
+    else
+      *new_state = OBJECT_EXISTS;
+    return true;
+  }
+
+  virtual void send_write();
+
+protected:
+  virtual void add_write_ops(librados::ObjectWriteOperation *wr) {
+    wr->truncate(m_object_off);
+  }
+};
+
+class ObjectZeroRequest : public AbstractObjectWriteRequest {
+public:
+  ObjectZeroRequest(ImageCtx *ictx, const std::string &oid, uint64_t object_no,
+                    uint64_t object_off, uint64_t object_len,
+                    const ::SnapContext &snapc, Context *completion)
+    : AbstractObjectWriteRequest(ictx, oid, object_no, object_off, object_len,
+                                 snapc, completion, true) {
+  }
+
+  virtual const char* get_op_type() const {
+    return "zero";
+  }
+
+  virtual bool pre_object_map_update(uint8_t *new_state) {
+    *new_state = OBJECT_EXISTS;
+    return true;
+  }
+
+protected:
+  virtual void add_write_ops(librados::ObjectWriteOperation *wr) {
+    wr->zero(m_object_off, m_object_len);
+  }
+};
+
+} // namespace io
+} // namespace librbd
+
+extern template class librbd::io::ObjectRequest<librbd::ImageCtx>;
+extern template class librbd::io::ObjectReadRequest<librbd::ImageCtx>;
+
+#endif // CEPH_LIBRBD_IO_OBJECT_REQUEST_H
diff --git a/src/librbd/io/Types.h b/src/librbd/io/Types.h
new file mode 100644 (file)
index 0000000..f80df9f
--- /dev/null
@@ -0,0 +1,31 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_IO_TYPES_H
+#define CEPH_LIBRBD_IO_TYPES_H
+
+#include "include/int_types.h"
+#include <map>
+#include <vector>
+
+namespace librbd {
+namespace io {
+
+typedef enum {
+  AIO_TYPE_NONE = 0,
+  AIO_TYPE_OPEN,
+  AIO_TYPE_CLOSE,
+  AIO_TYPE_READ,
+  AIO_TYPE_WRITE,
+  AIO_TYPE_DISCARD,
+  AIO_TYPE_FLUSH,
+} aio_type_t;
+
+typedef std::vector<std::pair<uint64_t, uint64_t> > Extents;
+typedef std::map<uint64_t, uint64_t> ExtentMap;
+
+} // namespace io
+} // namespace librbd
+
+#endif // CEPH_LIBRBD_IO_TYPES_H
+
index 413465b2382849d41f77065a3c488f7f822fb72a..3049bb8638865600796f563180b958d9c17b1a78 100644 (file)
@@ -5,13 +5,13 @@
 #include "common/dout.h"
 #include "common/errno.h"
 #include "common/WorkQueue.h"
-#include "librbd/AioCompletion.h"
-#include "librbd/AioImageRequest.h"
 #include "librbd/ImageCtx.h"
 #include "librbd/ImageState.h"
 #include "librbd/internal.h"
 #include "librbd/Operations.h"
 #include "librbd/Utils.h"
+#include "librbd/io/AioCompletion.h"
+#include "librbd/io/ImageRequest.h"
 
 #define dout_subsys ceph_subsys_rbd
 #undef dout_prefix
@@ -202,7 +202,7 @@ void Replay<I>::shut_down(bool cancel_ops, Context *on_finish) {
   CephContext *cct = m_image_ctx.cct;
   ldout(cct, 20) << dendl;
 
-  AioCompletion *flush_comp = nullptr;
+  io::AioCompletion *flush_comp = nullptr;
   on_finish = util::create_async_context_callback(
     m_image_ctx, on_finish);
 
@@ -245,7 +245,7 @@ void Replay<I>::shut_down(bool cancel_ops, Context *on_finish) {
   // execute the following outside of lock scope
   if (flush_comp != nullptr) {
     RWLock::RLocker owner_locker(m_image_ctx.owner_lock);
-    AioImageRequest<I>::aio_flush(&m_image_ctx, flush_comp);
+    io::ImageRequest<I>::aio_flush(&m_image_ctx, flush_comp);
   }
   if (on_finish != nullptr) {
     on_finish->complete(0);
@@ -254,7 +254,7 @@ void Replay<I>::shut_down(bool cancel_ops, Context *on_finish) {
 
 template <typename I>
 void Replay<I>::flush(Context *on_finish) {
-  AioCompletion *aio_comp;
+  io::AioCompletion *aio_comp;
   {
     Mutex::Locker locker(m_lock);
     aio_comp = create_aio_flush_completion(
@@ -262,7 +262,7 @@ void Replay<I>::flush(Context *on_finish) {
   }
 
   RWLock::RLocker owner_locker(m_image_ctx.owner_lock);
-  AioImageRequest<I>::aio_flush(&m_image_ctx, aio_comp);
+  io::ImageRequest<I>::aio_flush(&m_image_ctx, aio_comp);
 }
 
 template <typename I>
@@ -311,17 +311,17 @@ void Replay<I>::handle_event(const journal::AioDiscardEvent &event,
   ldout(cct, 20) << ": AIO discard event" << dendl;
 
   bool flush_required;
-  AioCompletion *aio_comp = create_aio_modify_completion(on_ready, on_safe,
-                                                         AIO_TYPE_DISCARD,
-                                                         &flush_required);
-  AioImageRequest<I>::aio_discard(&m_image_ctx, aio_comp, event.offset,
-                                  event.length);
+  auto aio_comp = create_aio_modify_completion(on_ready, on_safe,
+                                               io::AIO_TYPE_DISCARD,
+                                               &flush_required);
+  io::ImageRequest<I>::aio_discard(&m_image_ctx, aio_comp, event.offset,
+                                   event.length);
   if (flush_required) {
     m_lock.Lock();
-    AioCompletion *flush_comp = create_aio_flush_completion(nullptr);
+    auto flush_comp = create_aio_flush_completion(nullptr);
     m_lock.Unlock();
 
-    AioImageRequest<I>::aio_flush(&m_image_ctx, flush_comp);
+    io::ImageRequest<I>::aio_flush(&m_image_ctx, flush_comp);
   }
 }
 
@@ -333,17 +333,17 @@ void Replay<I>::handle_event(const journal::AioWriteEvent &event,
 
   bufferlist data = event.data;
   bool flush_required;
-  AioCompletion *aio_comp = create_aio_modify_completion(on_ready, on_safe,
-                                                         AIO_TYPE_WRITE,
-                                                         &flush_required);
-  AioImageRequest<I>::aio_write(&m_image_ctx, aio_comp, event.offset,
-                                event.length, data.c_str(), 0);
+  auto aio_comp = create_aio_modify_completion(on_ready, on_safe,
+                                               io::AIO_TYPE_WRITE,
+                                               &flush_required);
+  io::ImageRequest<I>::aio_write(&m_image_ctx, aio_comp, event.offset,
+                                 event.length, data.c_str(), 0);
   if (flush_required) {
     m_lock.Lock();
-    AioCompletion *flush_comp = create_aio_flush_completion(nullptr);
+    auto flush_comp = create_aio_flush_completion(nullptr);
     m_lock.Unlock();
 
-    AioImageRequest<I>::aio_flush(&m_image_ctx, flush_comp);
+    io::ImageRequest<I>::aio_flush(&m_image_ctx, flush_comp);
   }
 }
 
@@ -353,12 +353,12 @@ void Replay<I>::handle_event(const journal::AioFlushEvent &event,
   CephContext *cct = m_image_ctx.cct;
   ldout(cct, 20) << ": AIO flush event" << dendl;
 
-  AioCompletion *aio_comp;
+  io::AioCompletion *aio_comp;
   {
     Mutex::Locker locker(m_lock);
     aio_comp = create_aio_flush_completion(on_safe);
   }
-  AioImageRequest<I>::aio_flush(&m_image_ctx, aio_comp);
+  io::ImageRequest<I>::aio_flush(&m_image_ctx, aio_comp);
 
   on_ready->complete(0);
 }
@@ -922,10 +922,10 @@ void Replay<I>::handle_op_complete(uint64_t op_tid, int r) {
 }
 
 template <typename I>
-AioCompletion *Replay<I>::create_aio_modify_completion(Context *on_ready,
-                                                       Context *on_safe,
-                                                       aio_type_t aio_type,
-                                                       bool *flush_required) {
+io::AioCompletion *
+Replay<I>::create_aio_modify_completion(Context *on_ready, Context *on_safe,
+                                        io::aio_type_t aio_type,
+                                        bool *flush_required) {
   Mutex::Locker locker(m_lock);
   CephContext *cct = m_image_ctx.cct;
   assert(m_on_aio_ready == nullptr);
@@ -959,23 +959,23 @@ AioCompletion *Replay<I>::create_aio_modify_completion(Context *on_ready,
   // when the modification is ACKed by librbd, we can process the next
   // event. when flushed, the completion of the next flush will fire the
   // on_safe callback
-  AioCompletion *aio_comp = AioCompletion::create_and_start<Context>(
+  auto aio_comp = io::AioCompletion::create_and_start<Context>(
     new C_AioModifyComplete(this, on_ready, on_safe),
     util::get_image_ctx(&m_image_ctx), aio_type);
   return aio_comp;
 }
 
 template <typename I>
-AioCompletion *Replay<I>::create_aio_flush_completion(Context *on_safe) {
+io::AioCompletion *Replay<I>::create_aio_flush_completion(Context *on_safe) {
   assert(m_lock.is_locked());
 
   ++m_in_flight_aio_flush;
 
   // associate all prior write/discard ops to this flush request
-  AioCompletion *aio_comp = AioCompletion::create_and_start<Context>(
+  auto aio_comp = io::AioCompletion::create_and_start<Context>(
       new C_AioFlushComplete(this, on_safe,
                              std::move(m_aio_modify_unsafe_contexts)),
-      util::get_image_ctx(&m_image_ctx), AIO_TYPE_FLUSH);
+      util::get_image_ctx(&m_image_ctx), io::AIO_TYPE_FLUSH);
   m_aio_modify_unsafe_contexts.clear();
   return aio_comp;
 }
index 5b7b7e07925dc0d3015c432b38f2d979830caf32..6b7fdd67b7427758c97bf14ca191cce4c3910b82 100644 (file)
@@ -8,7 +8,7 @@
 #include "include/buffer_fwd.h"
 #include "include/Context.h"
 #include "common/Mutex.h"
-#include "librbd/AioCompletion.h"
+#include "librbd/io/Types.h"
 #include "librbd/journal/Types.h"
 #include <boost/variant.hpp>
 #include <list>
@@ -17,8 +17,8 @@
 
 namespace librbd {
 
-class AioCompletion;
 class ImageCtx;
+namespace io { struct AioCompletion; }
 
 namespace journal {
 
@@ -176,12 +176,12 @@ private:
                                       Context *on_safe, OpEvent **op_event);
   void handle_op_complete(uint64_t op_tid, int r);
 
-  AioCompletion *create_aio_modify_completion(Context *on_ready,
-                                              Context *on_safe,
-                                              aio_type_t aio_type,
-                                              bool *flush_required);
-  AioCompletion *create_aio_flush_completion(Context *on_safe);
-  void handle_aio_completion(AioCompletion *aio_comp);
+  io::AioCompletion *create_aio_modify_completion(Context *on_ready,
+                                                  Context *on_safe,
+                                                  io::aio_type_t aio_type,
+                                                  bool *flush_required);
+  io::AioCompletion *create_aio_flush_completion(Context *on_safe);
+  void handle_aio_completion(io::AioCompletion *aio_comp);
 
 };
 
index f9ac00300687073f6e9325cab5d799aa6f1f3490..b8936f9af027de4bed93a2a4b6bd35e448278fc1 100644 (file)
@@ -21,8 +21,6 @@
 #include "common/TracepointProvider.h"
 #include "include/Context.h"
 
-#include "librbd/AioCompletion.h"
-#include "librbd/AioImageRequestWQ.h"
 #include "cls/rbd/cls_rbd_client.h"
 #include "cls/rbd/cls_rbd_types.h"
 #include "librbd/Group.h"
@@ -30,6 +28,8 @@
 #include "librbd/ImageState.h"
 #include "librbd/internal.h"
 #include "librbd/Operations.h"
+#include "librbd/io/AioCompletion.h"
+#include "librbd/io/ImageRequestWQ.h"
 #include <algorithm>
 #include <string>
 #include <vector>
@@ -63,19 +63,19 @@ CephContext* get_cct(IoCtx &io_ctx) {
   return reinterpret_cast<CephContext*>(io_ctx.cct());
 }
 
-librbd::AioCompletion* get_aio_completion(librbd::RBD::AioCompletion *comp) {
-  return reinterpret_cast<librbd::AioCompletion *>(comp->pc);
+librbd::io::AioCompletion* get_aio_completion(librbd::RBD::AioCompletion *comp) {
+  return reinterpret_cast<librbd::io::AioCompletion *>(comp->pc);
 }
 
 struct C_OpenComplete : public Context {
   librbd::ImageCtx *ictx;
-  librbd::AioCompletion* comp;
+  librbd::io::AioCompletion* comp;
   void **ictxp;
   bool reopen;
-  C_OpenComplete(librbd::ImageCtx *ictx, librbd::AioCompletion* comp,
+  C_OpenComplete(librbd::ImageCtx *ictx, librbd::io::AioCompletion* comp,
                 void **ictxp, bool reopen = false)
     : ictx(ictx), comp(comp), ictxp(ictxp), reopen(reopen) {
-    comp->init_time(ictx, librbd::AIO_TYPE_OPEN);
+    comp->init_time(ictx, librbd::io::AIO_TYPE_OPEN);
     comp->get();
   }
   void finish(int r) override {
@@ -97,9 +97,10 @@ struct C_OpenComplete : public Context {
 
 struct C_OpenAfterCloseComplete : public Context {
   librbd::ImageCtx *ictx;
-  librbd::AioCompletion* comp;
+  librbd::io::AioCompletion* comp;
   void **ictxp;
-  C_OpenAfterCloseComplete(librbd::ImageCtx *ictx, librbd::AioCompletion* comp,
+  C_OpenAfterCloseComplete(librbd::ImageCtx *ictx,
+                           librbd::io::AioCompletion* comp,
                           void **ictxp)
     : ictx(ictx), comp(comp), ictxp(ictxp) {
   }
@@ -112,10 +113,10 @@ struct C_OpenAfterCloseComplete : public Context {
 
 struct C_CloseComplete : public Context {
   CephContext *cct;
-  librbd::AioCompletion* comp;
-  C_CloseComplete(librbd::ImageCtx *ictx, librbd::AioCompletion* comp)
+  librbd::io::AioCompletion* comp;
+  C_CloseComplete(librbd::ImageCtx *ictx, librbd::io::AioCompletion* comp)
     : cct(ictx->cct), comp(comp) {
-    comp->init_time(ictx, librbd::AIO_TYPE_CLOSE);
+    comp->init_time(ictx, librbd::io::AIO_TYPE_CLOSE);
     comp->get();
   }
   void finish(int r) override {
@@ -536,37 +537,37 @@ namespace librbd {
 
   RBD::AioCompletion::AioCompletion(void *cb_arg, callback_t complete_cb)
   {
-    pc = reinterpret_cast<void*>(librbd::AioCompletion::create(
+    pc = reinterpret_cast<void*>(librbd::io::AioCompletion::create(
       cb_arg, complete_cb, this));
   }
 
   bool RBD::AioCompletion::is_complete()
   {
-    librbd::AioCompletion *c = (librbd::AioCompletion *)pc;
+    librbd::io::AioCompletion *c = (librbd::io::AioCompletion *)pc;
     return c->is_complete();
   }
 
   int RBD::AioCompletion::wait_for_complete()
   {
-    librbd::AioCompletion *c = (librbd::AioCompletion *)pc;
+    librbd::io::AioCompletion *c = (librbd::io::AioCompletion *)pc;
     return c->wait_for_complete();
   }
 
   ssize_t RBD::AioCompletion::get_return_value()
   {
-    librbd::AioCompletion *c = (librbd::AioCompletion *)pc;
+    librbd::io::AioCompletion *c = (librbd::io::AioCompletion *)pc;
     return c->get_return_value();
   }
 
   void *RBD::AioCompletion::get_arg()
   {
-    librbd::AioCompletion *c = (librbd::AioCompletion *)pc;
+    librbd::io::AioCompletion *c = (librbd::io::AioCompletion *)pc;
     return c->get_arg();
   }
 
   void RBD::AioCompletion::release()
   {
-    librbd::AioCompletion *c = (librbd::AioCompletion *)pc;
+    librbd::io::AioCompletion *c = (librbd::io::AioCompletion *)pc;
     c->release();
     delete this;
   }
@@ -1217,7 +1218,7 @@ namespace librbd {
     tracepoint(librbd, read_enter, ictx, ictx->name.c_str(), ictx->snap_name.c_str(), ictx->read_only, ofs, len);
     bufferptr ptr(len);
     bl.push_back(std::move(ptr));
-    int r = ictx->aio_work_queue->read(ofs, len, bl.c_str(), 0);
+    int r = ictx->io_work_queue->read(ofs, len, bl.c_str(), 0);
     tracepoint(librbd, read_exit, r);
     return r;
   }
@@ -1229,7 +1230,7 @@ namespace librbd {
                ictx->read_only, ofs, len, op_flags);
     bufferptr ptr(len);
     bl.push_back(std::move(ptr));
-    int r = ictx->aio_work_queue->read(ofs, len, bl.c_str(), op_flags);
+    int r = ictx->io_work_queue->read(ofs, len, bl.c_str(), op_flags);
     tracepoint(librbd, read_exit, r);
     return r;
   }
@@ -1295,7 +1296,7 @@ namespace librbd {
       tracepoint(librbd, write_exit, -EINVAL);
       return -EINVAL;
     }
-    int r = ictx->aio_work_queue->write(ofs, len, bl.c_str(), 0);
+    int r = ictx->io_work_queue->write(ofs, len, bl.c_str(), 0);
     tracepoint(librbd, write_exit, r);
     return r;
   }
@@ -1309,7 +1310,7 @@ namespace librbd {
       tracepoint(librbd, write_exit, -EINVAL);
       return -EINVAL;
     }
-    int r = ictx->aio_work_queue->write(ofs, len, bl.c_str(), op_flags);
+    int r = ictx->io_work_queue->write(ofs, len, bl.c_str(), op_flags);
     tracepoint(librbd, write_exit, r);
     return r;
   }
@@ -1322,7 +1323,7 @@ namespace librbd {
         tracepoint(librbd, discard_exit, -EINVAL);
         return -EINVAL;
     }
-    int r = ictx->aio_work_queue->discard(ofs, len);
+    int r = ictx->io_work_queue->discard(ofs, len);
     tracepoint(librbd, discard_exit, r);
     return r;
   }
@@ -1336,8 +1337,8 @@ namespace librbd {
       tracepoint(librbd, aio_write_exit, -EINVAL);
       return -EINVAL;
     }
-    ictx->aio_work_queue->aio_write(get_aio_completion(c), off, len, bl.c_str(),
-                                    0);
+    ictx->io_work_queue->aio_write(get_aio_completion(c), off, len, bl.c_str(),
+                                   0);
     tracepoint(librbd, aio_write_exit, 0);
     return 0;
   }
@@ -1352,8 +1353,8 @@ namespace librbd {
       tracepoint(librbd, aio_write_exit, -EINVAL);
       return -EINVAL;
     }
-    ictx->aio_work_queue->aio_write(get_aio_completion(c), off, len, bl.c_str(),
-                                    op_flags);
+    ictx->io_work_queue->aio_write(get_aio_completion(c), off, len, bl.c_str(),
+                                   op_flags);
     tracepoint(librbd, aio_write_exit, 0);
     return 0;
   }
@@ -1362,7 +1363,7 @@ namespace librbd {
   {
     ImageCtx *ictx = (ImageCtx *)ctx;
     tracepoint(librbd, aio_discard_enter, ictx, ictx->name.c_str(), ictx->snap_name.c_str(), ictx->read_only, off, len, c->pc);
-    ictx->aio_work_queue->aio_discard(get_aio_completion(c), off, len);
+    ictx->io_work_queue->aio_discard(get_aio_completion(c), off, len);
     tracepoint(librbd, aio_discard_exit, 0);
     return 0;
   }
@@ -1374,8 +1375,8 @@ namespace librbd {
     tracepoint(librbd, aio_read_enter, ictx, ictx->name.c_str(), ictx->snap_name.c_str(), ictx->read_only, off, len, bl.c_str(), c->pc);
     ldout(ictx->cct, 10) << "Image::aio_read() buf=" << (void *)bl.c_str() << "~"
                         << (void *)(bl.c_str() + len - 1) << dendl;
-    ictx->aio_work_queue->aio_read(get_aio_completion(c), off, len, NULL, &bl,
-                                   0);
+    ictx->io_work_queue->aio_read(get_aio_completion(c), off, len, NULL, &bl,
+                                  0);
     tracepoint(librbd, aio_read_exit, 0);
     return 0;
   }
@@ -1388,8 +1389,8 @@ namespace librbd {
                ictx->read_only, off, len, bl.c_str(), c->pc, op_flags);
     ldout(ictx->cct, 10) << "Image::aio_read() buf=" << (void *)bl.c_str() << "~"
                         << (void *)(bl.c_str() + len - 1) << dendl;
-    ictx->aio_work_queue->aio_read(get_aio_completion(c), off, len, NULL, &bl,
-                                   op_flags);
+    ictx->io_work_queue->aio_read(get_aio_completion(c), off, len, NULL, &bl,
+                                  op_flags);
     tracepoint(librbd, aio_read_exit, 0);
     return 0;
   }
@@ -1407,7 +1408,7 @@ namespace librbd {
   {
     ImageCtx *ictx = (ImageCtx *)ctx;
     tracepoint(librbd, aio_flush_enter, ictx, ictx->name.c_str(), ictx->snap_name.c_str(), ictx->read_only, c->pc);
-    ictx->aio_work_queue->aio_flush(get_aio_completion(c));
+    ictx->io_work_queue->aio_flush(get_aio_completion(c));
     tracepoint(librbd, aio_flush_exit, 0);
     return 0;
   }
@@ -1423,7 +1424,7 @@ namespace librbd {
 
   int Image::poll_io_events(RBD::AioCompletion **comps, int numcomp)
   {
-    AioCompletion *cs[numcomp];
+    io::AioCompletion *cs[numcomp];
     ImageCtx *ictx = (ImageCtx *)ctx;
     tracepoint(librbd, poll_io_events_enter, ictx, numcomp);
     int r = librbd::poll_io_events(ictx, cs, numcomp);
@@ -2781,7 +2782,7 @@ extern "C" ssize_t rbd_read(rbd_image_t image, uint64_t ofs, size_t len,
 {
   librbd::ImageCtx *ictx = (librbd::ImageCtx *)image;
   tracepoint(librbd, read_enter, ictx, ictx->name.c_str(), ictx->snap_name.c_str(), ictx->read_only, ofs, len);
-  int r = ictx->aio_work_queue->read(ofs, len, buf, 0);
+  int r = ictx->io_work_queue->read(ofs, len, buf, 0);
   tracepoint(librbd, read_exit, r);
   return r;
 }
@@ -2792,7 +2793,7 @@ extern "C" ssize_t rbd_read2(rbd_image_t image, uint64_t ofs, size_t len,
   librbd::ImageCtx *ictx = (librbd::ImageCtx *)image;
   tracepoint(librbd, read2_enter, ictx, ictx->name.c_str(),
              ictx->snap_name.c_str(), ictx->read_only, ofs, len, op_flags);
-  int r = ictx->aio_work_queue->read(ofs, len, buf, op_flags);
+  int r = ictx->io_work_queue->read(ofs, len, buf, op_flags);
   tracepoint(librbd, read_exit, r);
   return r;
 }
@@ -2859,7 +2860,7 @@ extern "C" ssize_t rbd_write(rbd_image_t image, uint64_t ofs, size_t len,
 {
   librbd::ImageCtx *ictx = (librbd::ImageCtx *)image;
   tracepoint(librbd, write_enter, ictx, ictx->name.c_str(), ictx->snap_name.c_str(), ictx->read_only, ofs, len, buf);
-  int r = ictx->aio_work_queue->write(ofs, len, buf, 0);
+  int r = ictx->io_work_queue->write(ofs, len, buf, 0);
   tracepoint(librbd, write_exit, r);
   return r;
 }
@@ -2870,7 +2871,7 @@ extern "C" ssize_t rbd_write2(rbd_image_t image, uint64_t ofs, size_t len,
   librbd::ImageCtx *ictx = (librbd::ImageCtx *)image;
   tracepoint(librbd, write2_enter, ictx, ictx->name.c_str(),
              ictx->snap_name.c_str(), ictx->read_only, ofs, len, buf, op_flags);
-  int r = ictx->aio_work_queue->write(ofs, len, buf, op_flags);
+  int r = ictx->io_work_queue->write(ofs, len, buf, op_flags);
   tracepoint(librbd, write_exit, r);
   return r;
 }
@@ -2880,7 +2881,7 @@ extern "C" int rbd_discard(rbd_image_t image, uint64_t ofs, uint64_t len)
 {
   librbd::ImageCtx *ictx = (librbd::ImageCtx *)image;
   tracepoint(librbd, discard_enter, ictx, ictx->name.c_str(), ictx->snap_name.c_str(), ictx->read_only, ofs, len);
-  int r = ictx->aio_work_queue->discard(ofs, len);
+  int r = ictx->io_work_queue->discard(ofs, len);
   tracepoint(librbd, discard_exit, r);
   return r;
 }
@@ -2901,7 +2902,7 @@ extern "C" int rbd_aio_write(rbd_image_t image, uint64_t off, size_t len,
   librbd::ImageCtx *ictx = (librbd::ImageCtx *)image;
   librbd::RBD::AioCompletion *comp = (librbd::RBD::AioCompletion *)c;
   tracepoint(librbd, aio_write_enter, ictx, ictx->name.c_str(), ictx->snap_name.c_str(), ictx->read_only, off, len, buf, comp->pc);
-  ictx->aio_work_queue->aio_write(get_aio_completion(comp), off, len, buf, 0);
+  ictx->io_work_queue->aio_write(get_aio_completion(comp), off, len, buf, 0);
   tracepoint(librbd, aio_write_exit, 0);
   return 0;
 }
@@ -2913,8 +2914,8 @@ extern "C" int rbd_aio_write2(rbd_image_t image, uint64_t off, size_t len,
   librbd::RBD::AioCompletion *comp = (librbd::RBD::AioCompletion *)c;
   tracepoint(librbd, aio_write2_enter, ictx, ictx->name.c_str(), ictx->snap_name.c_str(),
              ictx->read_only, off, len, buf, comp->pc, op_flags);
-  ictx->aio_work_queue->aio_write(get_aio_completion(comp), off, len, buf,
-                                  op_flags);
+  ictx->io_work_queue->aio_write(get_aio_completion(comp), off, len, buf,
+                                 op_flags);
   tracepoint(librbd, aio_write_exit, 0);
   return 0;
 }
@@ -2926,7 +2927,7 @@ extern "C" int rbd_aio_discard(rbd_image_t image, uint64_t off, uint64_t len,
   librbd::ImageCtx *ictx = (librbd::ImageCtx *)image;
   librbd::RBD::AioCompletion *comp = (librbd::RBD::AioCompletion *)c;
   tracepoint(librbd, aio_discard_enter, ictx, ictx->name.c_str(), ictx->snap_name.c_str(), ictx->read_only, off, len, comp->pc);
-  ictx->aio_work_queue->aio_discard(get_aio_completion(comp), off, len);
+  ictx->io_work_queue->aio_discard(get_aio_completion(comp), off, len);
   tracepoint(librbd, aio_discard_exit, 0);
   return 0;
 }
@@ -2937,8 +2938,8 @@ extern "C" int rbd_aio_read(rbd_image_t image, uint64_t off, size_t len,
   librbd::ImageCtx *ictx = (librbd::ImageCtx *)image;
   librbd::RBD::AioCompletion *comp = (librbd::RBD::AioCompletion *)c;
   tracepoint(librbd, aio_read_enter, ictx, ictx->name.c_str(), ictx->snap_name.c_str(), ictx->read_only, off, len, buf, comp->pc);
-  ictx->aio_work_queue->aio_read(get_aio_completion(comp), off, len, buf, NULL,
-                                 0);
+  ictx->io_work_queue->aio_read(get_aio_completion(comp), off, len, buf, NULL,
+                                0);
   tracepoint(librbd, aio_read_exit, 0);
   return 0;
 }
@@ -2950,8 +2951,8 @@ extern "C" int rbd_aio_read2(rbd_image_t image, uint64_t off, size_t len,
   librbd::RBD::AioCompletion *comp = (librbd::RBD::AioCompletion *)c;
   tracepoint(librbd, aio_read2_enter, ictx, ictx->name.c_str(), ictx->snap_name.c_str(),
              ictx->read_only, off, len, buf, comp->pc, op_flags);
-  ictx->aio_work_queue->aio_read(get_aio_completion(comp), off, len, buf, NULL,
-                                 op_flags);
+  ictx->io_work_queue->aio_read(get_aio_completion(comp), off, len, buf, NULL,
+                                op_flags);
   tracepoint(librbd, aio_read_exit, 0);
   return 0;
 }
@@ -2970,7 +2971,7 @@ extern "C" int rbd_aio_flush(rbd_image_t image, rbd_completion_t c)
   librbd::ImageCtx *ictx = (librbd::ImageCtx *)image;
   librbd::RBD::AioCompletion *comp = (librbd::RBD::AioCompletion *)c;
   tracepoint(librbd, aio_flush_enter, ictx, ictx->name.c_str(), ictx->snap_name.c_str(), ictx->read_only, comp->pc);
-  ictx->aio_work_queue->aio_flush(get_aio_completion(comp));
+  ictx->io_work_queue->aio_flush(get_aio_completion(comp));
   tracepoint(librbd, aio_flush_exit, 0);
   return 0;
 }
@@ -2987,7 +2988,7 @@ extern "C" int rbd_invalidate_cache(rbd_image_t image)
 extern "C" int rbd_poll_io_events(rbd_image_t image, rbd_completion_t *comps, int numcomp)
 {
   librbd::ImageCtx *ictx = (librbd::ImageCtx *)image;
-  librbd::AioCompletion *cs[numcomp];
+  librbd::io::AioCompletion *cs[numcomp];
   tracepoint(librbd, poll_io_events_enter, ictx, numcomp);
   int r = librbd::poll_io_events(ictx, cs, numcomp);
   tracepoint(librbd, poll_io_events_exit, r);
index b7e9bc473c4c7bc307c2709db47c4e70869bef2c..504f530570021ffa631f0ed8325569430a2c3e34 100644 (file)
@@ -5,13 +5,13 @@
 #include "common/dout.h"
 #include "common/errno.h"
 #include "cls/rbd/cls_rbd_client.h"
-#include "librbd/AioImageRequestWQ.h"
 #include "librbd/ExclusiveLock.h"
 #include "librbd/ImageCtx.h"
 #include "librbd/ImageState.h"
 #include "librbd/Journal.h"
 #include "librbd/Utils.h"
 #include "librbd/image/SetFlagsRequest.h"
+#include "librbd/io/ImageRequestWQ.h"
 #include "librbd/journal/RemoveRequest.h"
 #include "librbd/mirror/DisableRequest.h"
 #include "librbd/object_map/RemoveRequest.h"
@@ -93,7 +93,7 @@ void DisableFeaturesRequest<I>::send_block_writes() {
   ldout(cct, 20) << this << " " << __func__ << dendl;
 
   RWLock::WLocker locker(image_ctx.owner_lock);
-  image_ctx.aio_work_queue->block_writes(create_context_callback<
+  image_ctx.io_work_queue->block_writes(create_context_callback<
     DisableFeaturesRequest<I>,
     &DisableFeaturesRequest<I>::handle_block_writes>(this));
 }
@@ -624,7 +624,7 @@ Context *DisableFeaturesRequest<I>::handle_finish(int r) {
       image_ctx.exclusive_lock->unblock_requests();
     }
 
-    image_ctx.aio_work_queue->unblock_writes();
+    image_ctx.io_work_queue->unblock_writes();
   }
   image_ctx.state->handle_prepare_lock_complete();
 
index b1521f8cba7380ab04566f501858358fbddef247..621357b4802627cba46b9629e8ef075e2220ef96 100644 (file)
@@ -4,13 +4,13 @@
 #include "librbd/operation/EnableFeaturesRequest.h"
 #include "common/dout.h"
 #include "common/errno.h"
-#include "librbd/AioImageRequestWQ.h"
 #include "librbd/ExclusiveLock.h"
 #include "librbd/ImageCtx.h"
 #include "librbd/ImageState.h"
 #include "librbd/Journal.h"
 #include "librbd/Utils.h"
 #include "librbd/image/SetFlagsRequest.h"
+#include "librbd/io/ImageRequestWQ.h"
 #include "librbd/journal/CreateRequest.h"
 #include "librbd/mirror/EnableRequest.h"
 #include "librbd/object_map/CreateRequest.h"
@@ -91,7 +91,7 @@ void EnableFeaturesRequest<I>::send_block_writes() {
   ldout(cct, 20) << this << " " << __func__ << dendl;
 
   RWLock::WLocker locker(image_ctx.owner_lock);
-  image_ctx.aio_work_queue->block_writes(create_context_callback<
+  image_ctx.io_work_queue->block_writes(create_context_callback<
     EnableFeaturesRequest<I>,
     &EnableFeaturesRequest<I>::handle_block_writes>(this));
 }
@@ -471,7 +471,7 @@ Context *EnableFeaturesRequest<I>::handle_finish(int r) {
       image_ctx.exclusive_lock->unblock_requests();
     }
     if (m_writes_blocked) {
-      image_ctx.aio_work_queue->unblock_writes();
+      image_ctx.io_work_queue->unblock_writes();
     }
   }
   image_ctx.state->handle_prepare_lock_complete();
index 70d358ab3c046799083de70eebd651df12764cd5..2cfa1ad1c13ae375155e85e3ea42d21db5cf522c 100644 (file)
@@ -2,10 +2,10 @@
 // vim: ts=8 sw=2 smarttab
 
 #include "librbd/operation/FlattenRequest.h"
-#include "librbd/AioObjectRequest.h"
 #include "librbd/AsyncObjectThrottle.h"
 #include "librbd/ExclusiveLock.h"
 #include "librbd/ImageCtx.h"
+#include "librbd/io/ObjectRequest.h"
 #include "common/dout.h"
 #include "common/errno.h"
 #include <boost/lambda/bind.hpp>
@@ -41,8 +41,8 @@ public:
 
     bufferlist bl;
     string oid = image_ctx.get_object_name(m_object_no);
-    AioObjectWrite *req = new AioObjectWrite(&image_ctx, oid, m_object_no, 0,
-                                             bl, m_snapc, this, 0);
+    auto req = new io::ObjectWriteRequest(&image_ctx, oid, m_object_no, 0,
+                                          bl, m_snapc, this, 0);
     if (!req->has_parent()) {
       // stop early if the parent went away - it just means
       // another flatten finished first or the image was resized
index 078c268defaa3a8ea978e5bd77f162717a54a071..2512dd623ef599eff29caff298bc1339fd99774c 100644 (file)
@@ -2,12 +2,12 @@
 // vim: ts=8 sw=2 smarttab
 
 #include "librbd/operation/ResizeRequest.h"
-#include "librbd/AioImageRequestWQ.h"
 #include "librbd/ExclusiveLock.h"
 #include "librbd/ImageCtx.h"
 #include "librbd/internal.h"
 #include "librbd/ObjectMap.h"
 #include "librbd/Utils.h"
+#include "librbd/io/ImageRequestWQ.h"
 #include "librbd/operation/TrimRequest.h"
 #include "common/dout.h"
 #include "common/errno.h"
@@ -92,7 +92,7 @@ void ResizeRequest<I>::send_pre_block_writes() {
   CephContext *cct = image_ctx.cct;
   ldout(cct, 5) << this << " " << __func__ << dendl;
 
-  image_ctx.aio_work_queue->block_writes(create_context_callback<
+  image_ctx.io_work_queue->block_writes(create_context_callback<
     ResizeRequest<I>, &ResizeRequest<I>::handle_pre_block_writes>(this));
 }
 
@@ -104,7 +104,7 @@ Context *ResizeRequest<I>::handle_pre_block_writes(int *result) {
 
   if (*result < 0) {
     lderr(cct) << "failed to block writes: " << cpp_strerror(*result) << dendl;
-    image_ctx.aio_work_queue->unblock_writes();
+    image_ctx.io_work_queue->unblock_writes();
     return this->create_context_finisher(*result);
   }
 
@@ -140,7 +140,7 @@ Context *ResizeRequest<I>::handle_append_op_event(int *result) {
   if (*result < 0) {
     lderr(cct) << "failed to commit journal entry: " << cpp_strerror(*result)
                << dendl;
-    image_ctx.aio_work_queue->unblock_writes();
+    image_ctx.io_work_queue->unblock_writes();
     return this->create_context_finisher(*result);
   }
 
@@ -252,7 +252,7 @@ Context *ResizeRequest<I>::send_grow_object_map() {
     RWLock::WLocker snap_locker(image_ctx.snap_lock);
     m_shrink_size_visible = true;
   }
-  image_ctx.aio_work_queue->unblock_writes();
+  image_ctx.io_work_queue->unblock_writes();
 
   if (m_original_size == m_new_size) {
     return this->create_context_finisher(0);
@@ -346,7 +346,7 @@ void ResizeRequest<I>::send_post_block_writes() {
   ldout(cct, 5) << this << " " << __func__ << dendl;
 
   RWLock::RLocker owner_locker(image_ctx.owner_lock);
-  image_ctx.aio_work_queue->block_writes(create_context_callback<
+  image_ctx.io_work_queue->block_writes(create_context_callback<
     ResizeRequest<I>, &ResizeRequest<I>::handle_post_block_writes>(this));
 }
 
@@ -357,7 +357,7 @@ Context *ResizeRequest<I>::handle_post_block_writes(int *result) {
   ldout(cct, 5) << this << " " << __func__ << ": r=" << *result << dendl;
 
   if (*result < 0) {
-    image_ctx.aio_work_queue->unblock_writes();
+    image_ctx.io_work_queue->unblock_writes();
     lderr(cct) << "failed to block writes prior to header update: "
                << cpp_strerror(*result) << dendl;
     return this->create_context_finisher(*result);
@@ -408,7 +408,7 @@ Context *ResizeRequest<I>::handle_update_header(int *result) {
   if (*result < 0) {
     lderr(cct) << "failed to update image header: " << cpp_strerror(*result)
                << dendl;
-    image_ctx.aio_work_queue->unblock_writes();
+    image_ctx.io_work_queue->unblock_writes();
     return this->create_context_finisher(*result);
   }
 
@@ -440,7 +440,7 @@ void ResizeRequest<I>::update_size_and_overlap() {
   }
 
   // blocked by POST_BLOCK_WRITES state
-  image_ctx.aio_work_queue->unblock_writes();
+  image_ctx.io_work_queue->unblock_writes();
 }
 
 } // namespace operation
index ce816bb71cf4eb6ce6962f330cacbbcf10f94c18..afec2896bb88f1f1cc817d4c8e371354add19a04 100644 (file)
@@ -5,11 +5,11 @@
 #include "librbd/operation/SnapshotCreateRequest.h"
 #include "common/dout.h"
 #include "common/errno.h"
-#include "librbd/AioImageRequestWQ.h"
 #include "librbd/ExclusiveLock.h"
 #include "librbd/ImageCtx.h"
 #include "librbd/ObjectMap.h"
 #include "librbd/Utils.h"
+#include "librbd/io/ImageRequestWQ.h"
 
 #define dout_subsys ceph_subsys_rbd
 #undef dout_prefix
@@ -68,7 +68,7 @@ void SnapshotCreateRequest<I>::send_suspend_aio() {
   CephContext *cct = image_ctx.cct;
   ldout(cct, 5) << this << " " << __func__ << dendl;
 
-  image_ctx.aio_work_queue->block_writes(create_context_callback<
+  image_ctx.io_work_queue->block_writes(create_context_callback<
     SnapshotCreateRequest<I>,
     &SnapshotCreateRequest<I>::handle_suspend_aio>(this));
 }
@@ -81,7 +81,7 @@ Context *SnapshotCreateRequest<I>::handle_suspend_aio(int *result) {
 
   if (*result < 0) {
     lderr(cct) << "failed to block writes: " << cpp_strerror(*result) << dendl;
-    image_ctx.aio_work_queue->unblock_writes();
+    image_ctx.io_work_queue->unblock_writes();
     return this->create_context_finisher(*result);
   }
 
@@ -110,7 +110,7 @@ Context *SnapshotCreateRequest<I>::handle_append_op_event(int *result) {
   ldout(cct, 5) << this << " " << __func__ << ": r=" << *result << dendl;
 
   if (*result < 0) {
-    image_ctx.aio_work_queue->unblock_writes();
+    image_ctx.io_work_queue->unblock_writes();
     lderr(cct) << "failed to commit journal entry: " << cpp_strerror(*result)
                << dendl;
     return this->create_context_finisher(*result);
@@ -142,7 +142,7 @@ Context *SnapshotCreateRequest<I>::handle_allocate_snap_id(int *result) {
 
   if (*result < 0) {
     save_result(result);
-    image_ctx.aio_work_queue->unblock_writes();
+    image_ctx.io_work_queue->unblock_writes();
     lderr(cct) << "failed to allocate snapshot id: " << cpp_strerror(*result)
                << dendl;
     return this->create_context_finisher(*result);
@@ -214,7 +214,7 @@ Context *SnapshotCreateRequest<I>::send_create_object_map() {
   if (image_ctx.object_map == nullptr || m_skip_object_map) {
     image_ctx.snap_lock.put_read();
 
-    image_ctx.aio_work_queue->unblock_writes();
+    image_ctx.io_work_queue->unblock_writes();
     return this->create_context_finisher(0);
   }
 
@@ -240,7 +240,7 @@ Context *SnapshotCreateRequest<I>::handle_create_object_map(int *result) {
 
   assert(*result == 0);
 
-  image_ctx.aio_work_queue->unblock_writes();
+  image_ctx.io_work_queue->unblock_writes();
   return this->create_context_finisher(0);
 }
 
@@ -268,7 +268,7 @@ Context *SnapshotCreateRequest<I>::handle_release_snap_id(int *result) {
   assert(m_ret_val < 0);
   *result = m_ret_val;
 
-  image_ctx.aio_work_queue->unblock_writes();
+  image_ctx.io_work_queue->unblock_writes();
   return this->create_context_finisher(m_ret_val);
 }
 
index a12edfd4e10f89f6ac9bbe75e8083a03d9e5050b..492b74c426a8fbc4a41472941ddac30c4ca4c50d 100644 (file)
@@ -5,11 +5,11 @@
 #include "include/rados/librados.hpp"
 #include "common/dout.h"
 #include "common/errno.h"
-#include "librbd/AioImageRequestWQ.h"
 #include "librbd/AsyncObjectThrottle.h"
 #include "librbd/ImageCtx.h"
 #include "librbd/ObjectMap.h"
 #include "librbd/Utils.h"
+#include "librbd/io/ImageRequestWQ.h"
 #include "librbd/operation/ResizeRequest.h"
 #include "osdc/Striper.h"
 #include <boost/lambda/bind.hpp>
@@ -77,7 +77,7 @@ template <typename I>
 SnapshotRollbackRequest<I>::~SnapshotRollbackRequest() {
   I &image_ctx = this->m_image_ctx;
   if (m_blocking_writes) {
-    image_ctx.aio_work_queue->unblock_writes();
+    image_ctx.io_work_queue->unblock_writes();
   }
   delete m_object_map;
 }
@@ -94,7 +94,7 @@ void SnapshotRollbackRequest<I>::send_block_writes() {
   ldout(cct, 5) << this << " " << __func__ << dendl;
 
   m_blocking_writes = true;
-  image_ctx.aio_work_queue->block_writes(create_context_callback<
+  image_ctx.io_work_queue->block_writes(create_context_callback<
     SnapshotRollbackRequest<I>,
     &SnapshotRollbackRequest<I>::handle_block_writes>(this));
 }
index b3a30883e166a5886f8c9a5f350414bab7601ca1..972c6aaf62dd02f75ae1ea5d456fa32b754b5006 100644 (file)
@@ -3,12 +3,12 @@
 
 #include "librbd/operation/TrimRequest.h"
 #include "librbd/AsyncObjectThrottle.h"
-#include "librbd/AioObjectRequest.h"
 #include "librbd/ExclusiveLock.h"
 #include "librbd/ImageCtx.h"
 #include "librbd/internal.h"
 #include "librbd/ObjectMap.h"
 #include "librbd/Utils.h"
+#include "librbd/io/ObjectRequest.h"
 #include "common/ContextCompletion.h"
 #include "common/dout.h"
 #include "common/errno.h"
@@ -45,8 +45,8 @@ public:
     string oid = image_ctx.get_object_name(m_object_no);
     ldout(image_ctx.cct, 10) << "removing (with copyup) " << oid << dendl;
 
-    AioObjectRequest<> *req = new AioObjectTrim(&image_ctx, oid, m_object_no,
-                                                m_snapc, this, false);
+    auto req = new io::ObjectTrimRequest(&image_ctx, oid, m_object_no,
+                                         m_snapc, this, false);
     req->send();
     return 0;
   }
@@ -413,13 +413,13 @@ void TrimRequest<I>::send_clean_boundary() {
     ldout(cct, 20) << " ex " << *p << dendl;
     Context *req_comp = new C_ContextCompletion(*completion);
 
-    AioObjectRequest<> *req;
+    io::ObjectRequest<> *req;
     if (p->offset == 0) {
-      req = new AioObjectTrim(&image_ctx, p->oid.name, p->objectno, snapc,
-                              req_comp, true);
+      req = new io::ObjectTrimRequest(&image_ctx, p->oid.name, p->objectno,
+                                      snapc, req_comp, true);
     } else {
-      req = new AioObjectTruncate(&image_ctx, p->oid.name, p->objectno,
-                                  p->offset, snapc, req_comp);
+      req = new io::ObjectTruncateRequest(&image_ctx, p->oid.name, p->objectno,
+                                          p->offset, snapc, req_comp);
     }
     req->send();
   }
index 3d946fdc194d3f1fb110f94b2ab0bc4143210ac5..105d98be9dacc0634317280433511fe1d9f2ce49 100644 (file)
@@ -24,10 +24,9 @@ set_target_properties(rbd_test_mock PROPERTIES COMPILE_FLAGS
 # doesn't use add_ceph_test because it is called by run-rbd-unit-tests.sh
 set(unittest_librbd_srcs
   test_BlockGuard.cc
-  test_ConsistencyGroups.cc
+  test_Groups.cc
   test_main.cc
   test_mock_fixture.cc
-  test_mock_AioImageRequest.cc
   test_mock_ExclusiveLock.cc
   test_mock_Journal.cc
   test_mock_ManagedLock.cc
@@ -36,6 +35,7 @@ set(unittest_librbd_srcs
   exclusive_lock/test_mock_PostAcquireRequest.cc
   exclusive_lock/test_mock_PreReleaseRequest.cc
   image/test_mock_RefreshRequest.cc
+  io/test_mock_ImageRequest.cc
   journal/test_mock_OpenRequest.cc
   journal/test_mock_PromoteRequest.cc
   journal/test_mock_Replay.cc
index a2fda18bf20fb141ac97fb9f2f1bd694e4c9754b..72249d186c30d6e7946e7b44e445e1aa2cd83d63 100644 (file)
@@ -53,7 +53,7 @@ public:
   }
 
   void expect_set_require_lock_on_read(MockImageCtx &mock_image_ctx) {
-    EXPECT_CALL(*mock_image_ctx.aio_work_queue, set_require_lock_on_read());
+    EXPECT_CALL(*mock_image_ctx.io_work_queue, set_require_lock_on_read());
   }
 
   void expect_block_writes(MockImageCtx &mock_image_ctx, int r) {
@@ -62,12 +62,12 @@ public:
     if ((mock_image_ctx.features & RBD_FEATURE_JOURNALING) != 0) {
       expect_set_require_lock_on_read(mock_image_ctx);
     }
-    EXPECT_CALL(*mock_image_ctx.aio_work_queue, block_writes(_))
+    EXPECT_CALL(*mock_image_ctx.io_work_queue, block_writes(_))
                   .WillOnce(CompleteContext(r, mock_image_ctx.image_ctx->op_work_queue));
   }
 
   void expect_unblock_writes(MockImageCtx &mock_image_ctx) {
-    EXPECT_CALL(*mock_image_ctx.aio_work_queue, unblock_writes());
+    EXPECT_CALL(*mock_image_ctx.io_work_queue, unblock_writes());
   }
 
   void expect_cancel_op_requests(MockImageCtx &mock_image_ctx, int r) {
index e9c20434def3686546c931b7ca59499f0c730d27..59932f2f850924762354a55a8bf19246c97d5117 100644 (file)
@@ -98,11 +98,11 @@ public:
   typedef RefreshParentRequest<MockRefreshImageCtx> MockRefreshParentRequest;
 
   void expect_set_require_lock_on_read(MockRefreshImageCtx &mock_image_ctx) {
-    EXPECT_CALL(*mock_image_ctx.aio_work_queue, set_require_lock_on_read());
+    EXPECT_CALL(*mock_image_ctx.io_work_queue, set_require_lock_on_read());
   }
 
   void expect_clear_require_lock_on_read(MockRefreshImageCtx &mock_image_ctx) {
-    EXPECT_CALL(*mock_image_ctx.aio_work_queue, clear_require_lock_on_read());
+    EXPECT_CALL(*mock_image_ctx.io_work_queue, clear_require_lock_on_read());
   }
 
   void expect_v1_read_header(MockRefreshImageCtx &mock_image_ctx, int r) {
@@ -325,12 +325,12 @@ public:
   }
 
   void expect_block_writes(MockImageCtx &mock_image_ctx, int r) {
-    EXPECT_CALL(*mock_image_ctx.aio_work_queue, block_writes(_))
+    EXPECT_CALL(*mock_image_ctx.io_work_queue, block_writes(_))
                   .WillOnce(CompleteContext(r, mock_image_ctx.image_ctx->op_work_queue));
   }
 
   void expect_unblock_writes(MockImageCtx &mock_image_ctx) {
-    EXPECT_CALL(*mock_image_ctx.aio_work_queue, unblock_writes())
+    EXPECT_CALL(*mock_image_ctx.io_work_queue, unblock_writes())
                   .Times(1);
   }
 
diff --git a/src/test/librbd/io/test_mock_ImageRequest.cc b/src/test/librbd/io/test_mock_ImageRequest.cc
new file mode 100644 (file)
index 0000000..55dd754
--- /dev/null
@@ -0,0 +1,267 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "test/librbd/test_mock_fixture.h"
+#include "test/librbd/test_support.h"
+#include "test/librbd/mock/MockImageCtx.h"
+#include "test/librbd/mock/MockJournal.h"
+#include "test/librbd/mock/cache/MockImageCache.h"
+#include "librbd/io/ImageRequest.h"
+#include "librbd/io/ObjectRequest.h"
+
+namespace librbd {
+namespace {
+
+struct MockTestImageCtx : public MockImageCtx {
+  MockTestImageCtx(ImageCtx &image_ctx) : MockImageCtx(image_ctx) {
+  }
+};
+
+} // anonymous namespace
+
+namespace util {
+
+inline ImageCtx *get_image_ctx(MockTestImageCtx *image_ctx) {
+  return image_ctx->image_ctx;
+}
+
+} // namespace util
+
+namespace io {
+
+template <>
+struct ObjectRequest<librbd::MockTestImageCtx> : public ObjectRequestHandle {
+  static ObjectRequest* s_instance;
+  Context *on_finish = nullptr;
+
+  static ObjectRequest* create_remove(librbd::MockTestImageCtx *ictx,
+                                      const std::string &oid,
+                                      uint64_t object_no,
+                                      const ::SnapContext &snapc,
+                                      Context *completion) {
+    assert(s_instance != nullptr);
+    s_instance->on_finish = completion;
+    return s_instance;
+  }
+
+  static ObjectRequest* create_truncate(librbd::MockTestImageCtx *ictx,
+                                        const std::string &oid,
+                                        uint64_t object_no,
+                                        uint64_t object_off,
+                                        const ::SnapContext &snapc,
+                                        Context *completion) {
+    assert(s_instance != nullptr);
+    s_instance->on_finish = completion;
+    return s_instance;
+  }
+
+  static ObjectRequest* create_write(librbd::MockTestImageCtx *ictx,
+                                     const std::string &oid,
+                                     uint64_t object_no,
+                                     uint64_t object_off,
+                                     const ceph::bufferlist &data,
+                                     const ::SnapContext &snapc,
+                                     Context *completion, int op_flags) {
+    assert(s_instance != nullptr);
+    s_instance->on_finish = completion;
+    return s_instance;
+  }
+
+  static ObjectRequest* create_zero(librbd::MockTestImageCtx *ictx,
+                                    const std::string &oid,
+                                    uint64_t object_no, uint64_t object_off,
+                                    uint64_t object_len,
+                                    const ::SnapContext &snapc,
+                                    Context *completion) {
+    assert(s_instance != nullptr);
+    s_instance->on_finish = completion;
+    return s_instance;
+  }
+
+  ObjectRequest() {
+    assert(s_instance == nullptr);
+    s_instance = this;
+  }
+  ~ObjectRequest() {
+    s_instance = nullptr;
+  }
+
+  MOCK_METHOD1(complete, void(int));
+  MOCK_METHOD0(send, void());
+};
+
+template <>
+struct ObjectReadRequest<librbd::MockTestImageCtx> : public ObjectRequest<librbd::MockTestImageCtx> {
+  typedef std::vector<std::pair<uint64_t, uint64_t> > Extents;
+  typedef std::map<uint64_t, uint64_t> ExtentMap;
+
+  static ObjectReadRequest* s_instance;
+
+  static ObjectReadRequest* create(librbd::MockTestImageCtx *ictx,
+                                   const std::string &oid,
+                                   uint64_t objectno, uint64_t offset,
+                                   uint64_t len, Extents &buffer_extents,
+                                   librados::snap_t snap_id, bool sparse,
+                                   Context *completion, int op_flags) {
+    assert(s_instance != nullptr);
+    s_instance->on_finish = completion;
+    return s_instance;
+  }
+
+  ObjectReadRequest() {
+    assert(s_instance == nullptr);
+    s_instance = this;
+  }
+  ~ObjectReadRequest() {
+    s_instance = nullptr;
+  }
+
+  MOCK_CONST_METHOD0(get_offset, uint64_t());
+  MOCK_CONST_METHOD0(get_length, uint64_t());
+  MOCK_METHOD0(data, ceph::bufferlist &());
+  MOCK_CONST_METHOD0(get_buffer_extents, const Extents &());
+  MOCK_METHOD0(get_extent_map, ExtentMap &());
+
+};
+
+ObjectRequest<librbd::MockTestImageCtx>* ObjectRequest<librbd::MockTestImageCtx>::s_instance = nullptr;
+ObjectReadRequest<librbd::MockTestImageCtx>* ObjectReadRequest<librbd::MockTestImageCtx>::s_instance = nullptr;
+
+} // namespace io
+} // namespace librbd
+
+#include "librbd/io/ImageRequest.cc"
+
+namespace librbd {
+namespace io {
+
+using ::testing::_;
+using ::testing::InSequence;
+using ::testing::Invoke;
+using ::testing::Return;
+using ::testing::WithArg;
+
+struct TestMockIoImageRequest : public TestMockFixture {
+  typedef ImageRequest<librbd::MockTestImageCtx> MockImageRequest;
+  typedef ImageWriteRequest<librbd::MockTestImageCtx> MockImageWriteRequest;
+  typedef ImageDiscardRequest<librbd::MockTestImageCtx> MockImageDiscardRequest;
+  typedef ImageFlushRequest<librbd::MockTestImageCtx> MockImageFlushRequest;
+  typedef ObjectRequest<librbd::MockTestImageCtx> MockObjectRequest;
+  typedef ObjectReadRequest<librbd::MockTestImageCtx> MockObjectReadRequest;
+
+  void expect_is_journal_appending(MockJournal &mock_journal, bool appending) {
+    EXPECT_CALL(mock_journal, is_journal_appending())
+      .WillOnce(Return(appending));
+  }
+
+  void expect_write_to_cache(MockImageCtx &mock_image_ctx,
+                             const object_t &object,
+                             uint64_t offset, uint64_t length,
+                             uint64_t journal_tid, int r) {
+    EXPECT_CALL(mock_image_ctx, write_to_cache(object, _, length, offset, _, _,
+                journal_tid))
+      .WillOnce(WithArg<4>(CompleteContext(r, mock_image_ctx.image_ctx->op_work_queue)));
+  }
+
+  void expect_object_request_send(MockImageCtx &mock_image_ctx,
+                                  MockObjectRequest &mock_object_request,
+                                  int r) {
+    EXPECT_CALL(mock_object_request, send())
+      .WillOnce(Invoke([&mock_image_ctx, &mock_object_request, r]() {
+                  mock_image_ctx.image_ctx->op_work_queue->queue(
+                    mock_object_request.on_finish, r);
+                }));
+  }
+
+  void expect_user_flushed(MockImageCtx &mock_image_ctx) {
+    EXPECT_CALL(mock_image_ctx, user_flushed());
+  }
+
+  void expect_flush(MockImageCtx &mock_image_ctx, int r) {
+    EXPECT_CALL(mock_image_ctx, flush(_))
+      .WillOnce(CompleteContext(r, mock_image_ctx.image_ctx->op_work_queue));
+  }
+};
+
+TEST_F(TestMockIoImageRequest, AioWriteJournalAppendDisabled) {
+  REQUIRE_FEATURE(RBD_FEATURE_JOURNALING);
+
+  librbd::ImageCtx *ictx;
+  ASSERT_EQ(0, open_image(m_image_name, &ictx));
+
+  MockObjectRequest mock_aio_object_request;
+  MockTestImageCtx mock_image_ctx(*ictx);
+  MockJournal mock_journal;
+  mock_image_ctx.journal = &mock_journal;
+
+  InSequence seq;
+  expect_is_journal_appending(mock_journal, false);
+  expect_write_to_cache(mock_image_ctx, ictx->get_object_name(0),
+                        0, 1, 0, 0);
+
+  C_SaferCond aio_comp_ctx;
+  AioCompletion *aio_comp = AioCompletion::create_and_start(
+    &aio_comp_ctx, ictx, AIO_TYPE_WRITE);
+  MockImageWriteRequest mock_aio_image_write(mock_image_ctx, aio_comp, 0, 1, "1",
+                                         0);
+  {
+    RWLock::RLocker owner_locker(mock_image_ctx.owner_lock);
+    mock_aio_image_write.send();
+  }
+  ASSERT_EQ(0, aio_comp_ctx.wait());
+}
+
+TEST_F(TestMockIoImageRequest, AioDiscardJournalAppendDisabled) {
+  REQUIRE_FEATURE(RBD_FEATURE_JOURNALING);
+
+  librbd::ImageCtx *ictx;
+  ASSERT_EQ(0, open_image(m_image_name, &ictx));
+
+  MockObjectRequest mock_aio_object_request;
+  MockTestImageCtx mock_image_ctx(*ictx);
+  MockJournal mock_journal;
+  mock_image_ctx.journal = &mock_journal;
+
+  InSequence seq;
+  expect_is_journal_appending(mock_journal, false);
+  expect_object_request_send(mock_image_ctx, mock_aio_object_request, 0);
+
+  C_SaferCond aio_comp_ctx;
+  AioCompletion *aio_comp = AioCompletion::create_and_start(
+    &aio_comp_ctx, ictx, AIO_TYPE_DISCARD);
+  MockImageDiscardRequest mock_aio_image_discard(mock_image_ctx, aio_comp, 0, 1);
+  {
+    RWLock::RLocker owner_locker(mock_image_ctx.owner_lock);
+    mock_aio_image_discard.send();
+  }
+  ASSERT_EQ(0, aio_comp_ctx.wait());
+}
+
+TEST_F(TestMockIoImageRequest, AioFlushJournalAppendDisabled) {
+  REQUIRE_FEATURE(RBD_FEATURE_JOURNALING);
+
+  librbd::ImageCtx *ictx;
+  ASSERT_EQ(0, open_image(m_image_name, &ictx));
+
+  MockTestImageCtx mock_image_ctx(*ictx);
+  MockJournal mock_journal;
+  mock_image_ctx.journal = &mock_journal;
+
+  InSequence seq;
+  expect_user_flushed(mock_image_ctx);
+  expect_is_journal_appending(mock_journal, false);
+  expect_flush(mock_image_ctx, 0);
+
+  C_SaferCond aio_comp_ctx;
+  AioCompletion *aio_comp = AioCompletion::create_and_start(
+    &aio_comp_ctx, ictx, AIO_TYPE_FLUSH);
+  MockImageFlushRequest mock_aio_image_flush(mock_image_ctx, aio_comp);
+  {
+    RWLock::RLocker owner_locker(mock_image_ctx.owner_lock);
+    mock_aio_image_flush.send();
+  }
+  ASSERT_EQ(0, aio_comp_ctx.wait());
+}
+
+} // namespace io
+} // namespace librbd
index 21fd17c594ca8380e6dad99b09813311cf02fb74..8506a6f7e4e181117c78d2893f434629610bc3ec 100644 (file)
@@ -3,10 +3,10 @@
 
 #include "test/librbd/test_fixture.h"
 #include "test/librbd/test_support.h"
-#include "librbd/AioCompletion.h"
-#include "librbd/AioImageRequestWQ.h"
 #include "librbd/internal.h"
 #include "librbd/Journal.h"
+#include "librbd/io/AioCompletion.h"
+#include "librbd/io/ImageRequestWQ.h"
 #include "librbd/journal/Types.h"
 #include "journal/Journaler.h"
 #include "journal/ReplayEntry.h"
@@ -127,9 +127,9 @@ TEST_F(TestJournalEntries, AioWrite) {
 
   std::string buffer(512, '1');
   C_SaferCond cond_ctx;
-  librbd::AioCompletion *c = librbd::AioCompletion::create(&cond_ctx);
+  auto c = librbd::io::AioCompletion::create(&cond_ctx);
   c->get();
-  ictx->aio_work_queue->aio_write(c, 123, buffer.size(), buffer.c_str(), 0);
+  ictx->io_work_queue->aio_write(c, 123, buffer.size(), buffer.c_str(), 0);
   ASSERT_EQ(0, c->wait_for_complete());
   c->put();
 
@@ -170,9 +170,9 @@ TEST_F(TestJournalEntries, AioDiscard) {
   ASSERT_TRUE(journaler != NULL);
 
   C_SaferCond cond_ctx;
-  librbd::AioCompletion *c = librbd::AioCompletion::create(&cond_ctx);
+  auto c = librbd::io::AioCompletion::create(&cond_ctx);
   c->get();
-  ictx->aio_work_queue->aio_discard(c, 123, 234);
+  ictx->io_work_queue->aio_discard(c, 123, 234);
   ASSERT_EQ(0, c->wait_for_complete());
   c->put();
 
@@ -203,9 +203,9 @@ TEST_F(TestJournalEntries, AioFlush) {
   ASSERT_TRUE(journaler != NULL);
 
   C_SaferCond cond_ctx;
-  librbd::AioCompletion *c = librbd::AioCompletion::create(&cond_ctx);
+  auto c = librbd::io::AioCompletion::create(&cond_ctx);
   c->get();
-  ictx->aio_work_queue->aio_flush(c);
+  ictx->io_work_queue->aio_flush(c);
   ASSERT_EQ(0, c->wait_for_complete());
   c->put();
 
index 23a99ef07b3a0f682bd2dedf29c649a56d30ce77..e2d4425f7b888d91328c5f67da605b319e296f31 100644 (file)
@@ -8,9 +8,6 @@
 #include "cls/journal/cls_journal_types.h"
 #include "cls/journal/cls_journal_client.h"
 #include "journal/Journaler.h"
-#include "librbd/AioCompletion.h"
-#include "librbd/AioImageRequest.h"
-#include "librbd/AioImageRequestWQ.h"
 #include "librbd/ExclusiveLock.h"
 #include "librbd/ImageCtx.h"
 #include "librbd/ImageState.h"
@@ -18,6 +15,9 @@
 #include "librbd/internal.h"
 #include "librbd/Journal.h"
 #include "librbd/Operations.h"
+#include "librbd/io/AioCompletion.h"
+#include "librbd/io/ImageRequest.h"
+#include "librbd/io/ImageRequestWQ.h"
 #include "librbd/journal/Types.h"
 
 void register_test_journal_replay() {
@@ -46,7 +46,7 @@ public:
   void inject_into_journal(librbd::ImageCtx *ictx, T event) {
     C_SaferCond ctx;
     librbd::journal::EventEntry event_entry(event);
-    librbd::Journal<>::AioObjectRequests requests;
+    librbd::Journal<>::IOObjectRequests requests;
     {
       RWLock::RLocker owner_locker(ictx->owner_lock);
       uint64_t tid = ictx->journal->append_io_event(std::move(event_entry),
@@ -109,21 +109,21 @@ TEST_F(TestJournalReplay, AioDiscardEvent) {
   ictx->features &= ~RBD_FEATURE_JOURNALING;
 
   std::string payload(4096, '1');
-  librbd::AioCompletion *aio_comp = new librbd::AioCompletion();
-  ictx->aio_work_queue->aio_write(aio_comp, 0, payload.size(), payload.c_str(),
-                                  0);
+  auto aio_comp = new librbd::io::AioCompletion();
+  ictx->io_work_queue->aio_write(aio_comp, 0, payload.size(), payload.c_str(),
+                                 0);
   ASSERT_EQ(0, aio_comp->wait_for_complete());
   aio_comp->release();
 
-  aio_comp = new librbd::AioCompletion();
-  ictx->aio_work_queue->aio_flush(aio_comp);
+  aio_comp = new librbd::io::AioCompletion();
+  ictx->io_work_queue->aio_flush(aio_comp);
   ASSERT_EQ(0, aio_comp->wait_for_complete());
   aio_comp->release();
 
   std::string read_payload(4096, '\0');
-  aio_comp = new librbd::AioCompletion();
-  ictx->aio_work_queue->aio_read(aio_comp, 0, read_payload.size(),
-                                 &read_payload[0], NULL, 0);
+  aio_comp = new librbd::io::AioCompletion();
+  ictx->io_work_queue->aio_read(aio_comp, 0, read_payload.size(),
+                                &read_payload[0], NULL, 0);
   ASSERT_EQ(0, aio_comp->wait_for_complete());
   aio_comp->release();
   ASSERT_EQ(payload, read_payload);
@@ -146,9 +146,9 @@ TEST_F(TestJournalReplay, AioDiscardEvent) {
   ASSERT_EQ(0, open_image(m_image_name, &ictx));
   ASSERT_EQ(0, when_acquired_lock(ictx));
 
-  aio_comp = new librbd::AioCompletion();
-  ictx->aio_work_queue->aio_read(aio_comp, 0, read_payload.size(),
-                                 &read_payload[0], NULL, 0);
+  aio_comp = new librbd::io::AioCompletion();
+  ictx->io_work_queue->aio_read(aio_comp, 0, read_payload.size(),
+                                &read_payload[0], NULL, 0);
   ASSERT_EQ(0, aio_comp->wait_for_complete());
   aio_comp->release();
   if (ictx->cct->_conf->rbd_skip_partial_discard) {
@@ -178,8 +178,8 @@ TEST_F(TestJournalReplay, AioDiscardEvent) {
   ASSERT_EQ(1, current_entry);
 
   // verify lock ordering constraints
-  aio_comp = new librbd::AioCompletion();
-  ictx->aio_work_queue->aio_discard(aio_comp, 0, read_payload.size());
+  aio_comp = new librbd::io::AioCompletion();
+  ictx->io_work_queue->aio_discard(aio_comp, 0, read_payload.size());
   ASSERT_EQ(0, aio_comp->wait_for_complete());
   aio_comp->release();
 }
@@ -209,9 +209,9 @@ TEST_F(TestJournalReplay, AioWriteEvent) {
   ASSERT_EQ(0, when_acquired_lock(ictx));
 
   std::string read_payload(4096, '\0');
-  librbd::AioCompletion *aio_comp = new librbd::AioCompletion();
-  ictx->aio_work_queue->aio_read(aio_comp, 0, read_payload.size(),
-                                 &read_payload[0], NULL, 0);
+  auto aio_comp = new librbd::io::AioCompletion();
+  ictx->io_work_queue->aio_read(aio_comp, 0, read_payload.size(),
+                                &read_payload[0], NULL, 0);
   ASSERT_EQ(0, aio_comp->wait_for_complete());
   aio_comp->release();
   ASSERT_EQ(payload, read_payload);
@@ -237,9 +237,9 @@ TEST_F(TestJournalReplay, AioWriteEvent) {
   ASSERT_EQ(1, current_entry);
 
   // verify lock ordering constraints
-  aio_comp = new librbd::AioCompletion();
-  ictx->aio_work_queue->aio_write(aio_comp, 0, payload.size(), payload.c_str(),
-                                  0);
+  aio_comp = new librbd::io::AioCompletion();
+  ictx->io_work_queue->aio_write(aio_comp, 0, payload.size(), payload.c_str(),
+                                 0);
   ASSERT_EQ(0, aio_comp->wait_for_complete());
   aio_comp->release();
 }
@@ -284,8 +284,8 @@ TEST_F(TestJournalReplay, AioFlushEvent) {
   ASSERT_EQ(1, current_entry);
 
   // verify lock ordering constraints
-  librbd::AioCompletion *aio_comp = new librbd::AioCompletion();
-  ictx->aio_work_queue->aio_flush(aio_comp);
+  auto aio_comp = new librbd::io::AioCompletion();
+  ictx->io_work_queue->aio_flush(aio_comp);
   ASSERT_EQ(0, aio_comp->wait_for_complete());
   aio_comp->release();
 }
@@ -782,14 +782,14 @@ TEST_F(TestJournalReplay, ObjectPosition) {
   get_journal_commit_position(ictx, &initial_tag, &initial_entry);
 
   std::string payload(4096, '1');
-  librbd::AioCompletion *aio_comp = new librbd::AioCompletion();
-  ictx->aio_work_queue->aio_write(aio_comp, 0, payload.size(), payload.c_str(),
-                                  0);
+  auto aio_comp = new librbd::io::AioCompletion();
+  ictx->io_work_queue->aio_write(aio_comp, 0, payload.size(), payload.c_str(),
+                                 0);
   ASSERT_EQ(0, aio_comp->wait_for_complete());
   aio_comp->release();
 
-  aio_comp = new librbd::AioCompletion();
-  ictx->aio_work_queue->aio_flush(aio_comp);
+  aio_comp = new librbd::io::AioCompletion();
+  ictx->io_work_queue->aio_flush(aio_comp);
   ASSERT_EQ(0, aio_comp->wait_for_complete());
   aio_comp->release();
 
@@ -802,14 +802,14 @@ TEST_F(TestJournalReplay, ObjectPosition) {
 
   // write again
 
-  aio_comp = new librbd::AioCompletion();
-  ictx->aio_work_queue->aio_write(aio_comp, 0, payload.size(), payload.c_str(),
-                                  0);
+  aio_comp = new librbd::io::AioCompletion();
+  ictx->io_work_queue->aio_write(aio_comp, 0, payload.size(), payload.c_str(),
+                                 0);
   ASSERT_EQ(0, aio_comp->wait_for_complete());
   aio_comp->release();
 
-  aio_comp = new librbd::AioCompletion();
-  ictx->aio_work_queue->aio_flush(aio_comp);
+  aio_comp = new librbd::io::AioCompletion();
+  ictx->io_work_queue->aio_flush(aio_comp);
   ASSERT_EQ(0, aio_comp->wait_for_complete());
   aio_comp->release();
 
index 6f26ccc8f87b92608da6da874554f3051e6da11d..a3847e3a69e4d3bc7904b2a562a4b78283510fb9 100644 (file)
@@ -4,7 +4,7 @@
 #include "test/librbd/test_mock_fixture.h"
 #include "test/librbd/test_support.h"
 #include "test/librbd/mock/MockImageCtx.h"
-#include "librbd/AioImageRequest.h"
+#include "librbd/io/ImageRequest.h"
 #include "librbd/journal/Replay.h"
 #include "librbd/journal/Types.h"
 #include "gmock/gmock.h"
@@ -22,14 +22,17 @@ struct MockReplayImageCtx : public MockImageCtx {
 
 } // anonymous namespace
 
+namespace io {
+
 template <>
-struct AioImageRequest<MockReplayImageCtx> {
-  static AioImageRequest *s_instance;
+struct ImageRequest<MockReplayImageCtx> {
+  static ImageRequest *s_instance;
 
   MOCK_METHOD5(aio_write, void(AioCompletion *c, uint64_t off, size_t len,
                                const char *buf, int op_flags));
-  static void aio_write(MockReplayImageCtx *ictx, AioCompletion *c, uint64_t off,
-                        size_t len, const char *buf, int op_flags) {
+  static void aio_write(MockReplayImageCtx *ictx, AioCompletion *c,
+                        uint64_t off, size_t len, const char *buf,
+                        int op_flags) {
     assert(s_instance != nullptr);
     s_instance->aio_write(c, off, len, buf, op_flags);
   }
@@ -47,12 +50,14 @@ struct AioImageRequest<MockReplayImageCtx> {
     s_instance->aio_flush(c);
   }
 
-  AioImageRequest() {
+  ImageRequest() {
     s_instance = this;
   }
 };
 
-AioImageRequest<MockReplayImageCtx> *AioImageRequest<MockReplayImageCtx>::s_instance = nullptr;
+ImageRequest<MockReplayImageCtx> *ImageRequest<MockReplayImageCtx>::s_instance = nullptr;
+
+} // namespace io
 
 namespace util {
 
@@ -88,7 +93,7 @@ ACTION_P2(NotifyInvoke, lock, cond) {
 ACTION_P2(CompleteAioCompletion, r, image_ctx) {
   image_ctx->op_work_queue->queue(new FunctionContext([this, arg0](int r) {
       arg0->get();
-      arg0->init_time(image_ctx, librbd::AIO_TYPE_NONE);
+      arg0->init_time(image_ctx, librbd::io::AIO_TYPE_NONE);
       arg0->set_request_count(1);
       arg0->complete_request(r);
     }), r);
@@ -99,35 +104,35 @@ namespace journal {
 
 class TestMockJournalReplay : public TestMockFixture {
 public:
-  typedef AioImageRequest<MockReplayImageCtx> MockAioImageRequest;
+  typedef io::ImageRequest<MockReplayImageCtx> MockIoImageRequest;
   typedef Replay<MockReplayImageCtx> MockJournalReplay;
 
   TestMockJournalReplay() : m_invoke_lock("m_invoke_lock") {
   }
 
-  void expect_aio_discard(MockAioImageRequest &mock_aio_image_request,
-                          AioCompletion **aio_comp, uint64_t off,
+  void expect_aio_discard(MockIoImageRequest &mock_io_image_request,
+                          io::AioCompletion **aio_comp, uint64_t off,
                           uint64_t len) {
-    EXPECT_CALL(mock_aio_image_request, aio_discard(_, off, len))
+    EXPECT_CALL(mock_io_image_request, aio_discard(_, off, len))
                   .WillOnce(SaveArg<0>(aio_comp));
   }
 
-  void expect_aio_flush(MockAioImageRequest &mock_aio_image_request,
-                        AioCompletion **aio_comp) {
-    EXPECT_CALL(mock_aio_image_request, aio_flush(_))
+  void expect_aio_flush(MockIoImageRequest &mock_io_image_request,
+                        io::AioCompletion **aio_comp) {
+    EXPECT_CALL(mock_io_image_request, aio_flush(_))
                   .WillOnce(SaveArg<0>(aio_comp));
   }
 
   void expect_aio_flush(MockReplayImageCtx &mock_image_ctx,
-                        MockAioImageRequest &mock_aio_image_request, int r) {
-    EXPECT_CALL(mock_aio_image_request, aio_flush(_))
+                        MockIoImageRequest &mock_io_image_request, int r) {
+    EXPECT_CALL(mock_io_image_request, aio_flush(_))
                   .WillOnce(CompleteAioCompletion(r, mock_image_ctx.image_ctx));
   }
 
-  void expect_aio_write(MockAioImageRequest &mock_aio_image_request,
-                        AioCompletion **aio_comp, uint64_t off,
+  void expect_aio_write(MockIoImageRequest &mock_io_image_request,
+                        io::AioCompletion **aio_comp, uint64_t off,
                         uint64_t len, const char *data) {
-    EXPECT_CALL(mock_aio_image_request,
+    EXPECT_CALL(mock_io_image_request,
                 aio_write(_, off, len, CStrEq(data), _))
                   .WillOnce(SaveArg<0>(aio_comp));
   }
@@ -250,10 +255,10 @@ public:
     mock_journal_replay.process(event_entry, on_ready, on_safe);
   }
 
-  void when_complete(MockReplayImageCtx &mock_image_ctx, AioCompletion *aio_comp,
-                     int r) {
+  void when_complete(MockReplayImageCtx &mock_image_ctx,
+                     io::AioCompletion *aio_comp, int r) {
     aio_comp->get();
-    aio_comp->init_time(mock_image_ctx.image_ctx, librbd::AIO_TYPE_NONE);
+    aio_comp->init_time(mock_image_ctx.image_ctx, librbd::io::AIO_TYPE_NONE);
     aio_comp->set_request_count(1);
     aio_comp->complete_request(r);
   }
@@ -303,14 +308,14 @@ TEST_F(TestMockJournalReplay, AioDiscard) {
 
   MockReplayImageCtx mock_image_ctx(*ictx);
   MockJournalReplay mock_journal_replay(mock_image_ctx);
-  MockAioImageRequest mock_aio_image_request;
+  MockIoImageRequest mock_io_image_request;
   expect_op_work_queue(mock_image_ctx);
 
   InSequence seq;
-  AioCompletion *aio_comp;
+  io::AioCompletion *aio_comp;
   C_SaferCond on_ready;
   C_SaferCond on_safe;
-  expect_aio_discard(mock_aio_image_request, &aio_comp, 123, 456);
+  expect_aio_discard(mock_io_image_request, &aio_comp, 123, 456);
   when_process(mock_journal_replay,
                EventEntry{AioDiscardEvent(123, 456)},
                &on_ready, &on_safe);
@@ -318,7 +323,7 @@ TEST_F(TestMockJournalReplay, AioDiscard) {
   when_complete(mock_image_ctx, aio_comp, 0);
   ASSERT_EQ(0, on_ready.wait());
 
-  expect_aio_flush(mock_image_ctx, mock_aio_image_request, 0);
+  expect_aio_flush(mock_image_ctx, mock_io_image_request, 0);
   ASSERT_EQ(0, when_shut_down(mock_journal_replay, false));
   ASSERT_EQ(0, on_safe.wait());
 }
@@ -331,14 +336,14 @@ TEST_F(TestMockJournalReplay, AioWrite) {
 
   MockReplayImageCtx mock_image_ctx(*ictx);
   MockJournalReplay mock_journal_replay(mock_image_ctx);
-  MockAioImageRequest mock_aio_image_request;
+  MockIoImageRequest mock_io_image_request;
   expect_op_work_queue(mock_image_ctx);
 
   InSequence seq;
-  AioCompletion *aio_comp;
+  io::AioCompletion *aio_comp;
   C_SaferCond on_ready;
   C_SaferCond on_safe;
-  expect_aio_write(mock_aio_image_request, &aio_comp, 123, 456, "test");
+  expect_aio_write(mock_io_image_request, &aio_comp, 123, 456, "test");
   when_process(mock_journal_replay,
                EventEntry{AioWriteEvent(123, 456, to_bl("test"))},
                &on_ready, &on_safe);
@@ -346,7 +351,7 @@ TEST_F(TestMockJournalReplay, AioWrite) {
   when_complete(mock_image_ctx, aio_comp, 0);
   ASSERT_EQ(0, on_ready.wait());
 
-  expect_aio_flush(mock_image_ctx, mock_aio_image_request, 0);
+  expect_aio_flush(mock_image_ctx, mock_io_image_request, 0);
   ASSERT_EQ(0, when_shut_down(mock_journal_replay, false));
   ASSERT_EQ(0, on_safe.wait());
 }
@@ -359,14 +364,14 @@ TEST_F(TestMockJournalReplay, AioFlush) {
 
   MockReplayImageCtx mock_image_ctx(*ictx);
   MockJournalReplay mock_journal_replay(mock_image_ctx);
-  MockAioImageRequest mock_aio_image_request;
+  MockIoImageRequest mock_io_image_request;
   expect_op_work_queue(mock_image_ctx);
 
   InSequence seq;
-  AioCompletion *aio_comp;
+  io::AioCompletion *aio_comp;
   C_SaferCond on_ready;
   C_SaferCond on_safe;
-  expect_aio_flush(mock_aio_image_request, &aio_comp);
+  expect_aio_flush(mock_io_image_request, &aio_comp);
   when_process(mock_journal_replay, EventEntry{AioFlushEvent()},
                &on_ready, &on_safe);
 
@@ -385,14 +390,14 @@ TEST_F(TestMockJournalReplay, IOError) {
 
   MockReplayImageCtx mock_image_ctx(*ictx);
   MockJournalReplay mock_journal_replay(mock_image_ctx);
-  MockAioImageRequest mock_aio_image_request;
+  MockIoImageRequest mock_io_image_request;
   expect_op_work_queue(mock_image_ctx);
 
   InSequence seq;
-  AioCompletion *aio_comp;
+  io::AioCompletion *aio_comp;
   C_SaferCond on_ready;
   C_SaferCond on_safe;
-  expect_aio_discard(mock_aio_image_request, &aio_comp, 123, 456);
+  expect_aio_discard(mock_io_image_request, &aio_comp, 123, 456);
   when_process(mock_journal_replay,
                EventEntry{AioDiscardEvent(123, 456)},
                &on_ready, &on_safe);
@@ -400,7 +405,7 @@ TEST_F(TestMockJournalReplay, IOError) {
   when_complete(mock_image_ctx, aio_comp, -EINVAL);
   ASSERT_EQ(-EINVAL, on_safe.wait());
 
-  expect_aio_flush(mock_image_ctx, mock_aio_image_request, 0);
+  expect_aio_flush(mock_image_ctx, mock_io_image_request, 0);
   ASSERT_EQ(0, when_shut_down(mock_journal_replay, false));
   ASSERT_EQ(0, on_ready.wait());
 }
@@ -413,19 +418,19 @@ TEST_F(TestMockJournalReplay, SoftFlushIO) {
 
   MockReplayImageCtx mock_image_ctx(*ictx);
   MockJournalReplay mock_journal_replay(mock_image_ctx);
-  MockAioImageRequest mock_aio_image_request;
+  MockIoImageRequest mock_io_image_request;
   expect_op_work_queue(mock_image_ctx);
 
   InSequence seq;
   const size_t io_count = 32;
   C_SaferCond on_safes[io_count];
   for (size_t i = 0; i < io_count; ++i) {
-    AioCompletion *aio_comp;
-    AioCompletion *flush_comp = nullptr;
+    io::AioCompletion *aio_comp;
+    io::AioCompletion *flush_comp = nullptr;
     C_SaferCond on_ready;
-    expect_aio_discard(mock_aio_image_request, &aio_comp, 123, 456);
+    expect_aio_discard(mock_io_image_request, &aio_comp, 123, 456);
     if (i == io_count - 1) {
-      expect_aio_flush(mock_aio_image_request, &flush_comp);
+      expect_aio_flush(mock_io_image_request, &flush_comp);
     }
     when_process(mock_journal_replay,
                  EventEntry{AioDiscardEvent(123, 456)},
@@ -452,20 +457,20 @@ TEST_F(TestMockJournalReplay, PauseIO) {
 
   MockReplayImageCtx mock_image_ctx(*ictx);
   MockJournalReplay mock_journal_replay(mock_image_ctx);
-  MockAioImageRequest mock_aio_image_request;
+  MockIoImageRequest mock_io_image_request;
   expect_op_work_queue(mock_image_ctx);
 
   InSequence seq;
   const size_t io_count = 64;
-  std::list<AioCompletion *> flush_comps;
+  std::list<io::AioCompletion *> flush_comps;
   C_SaferCond on_safes[io_count];
   for (size_t i = 0; i < io_count; ++i) {
-    AioCompletion *aio_comp;
+    io::AioCompletion *aio_comp;
     C_SaferCond on_ready;
-    expect_aio_write(mock_aio_image_request, &aio_comp, 123, 456, "test");
+    expect_aio_write(mock_io_image_request, &aio_comp, 123, 456, "test");
     if ((i + 1) % 32 == 0) {
       flush_comps.push_back(nullptr);
-      expect_aio_flush(mock_aio_image_request, &flush_comps.back());
+      expect_aio_flush(mock_io_image_request, &flush_comps.back());
     }
     when_process(mock_journal_replay,
                  EventEntry{AioWriteEvent(123, 456, to_bl("test"))},
@@ -493,14 +498,14 @@ TEST_F(TestMockJournalReplay, Flush) {
 
   MockReplayImageCtx mock_image_ctx(*ictx);
   MockJournalReplay mock_journal_replay(mock_image_ctx);
-  MockAioImageRequest mock_aio_image_request;
+  MockIoImageRequest mock_io_image_request;
   expect_op_work_queue(mock_image_ctx);
 
   InSequence seq;
-  AioCompletion *aio_comp = nullptr;
+  io::AioCompletion *aio_comp = nullptr;
   C_SaferCond on_ready;
   C_SaferCond on_safe;
-  expect_aio_discard(mock_aio_image_request, &aio_comp, 123, 456);
+  expect_aio_discard(mock_io_image_request, &aio_comp, 123, 456);
   when_process(mock_journal_replay,
                EventEntry{AioDiscardEvent(123, 456)},
                &on_ready, &on_safe);
@@ -508,7 +513,7 @@ TEST_F(TestMockJournalReplay, Flush) {
   when_complete(mock_image_ctx, aio_comp, 0);
   ASSERT_EQ(0, on_ready.wait());
 
-  expect_aio_flush(mock_image_ctx, mock_aio_image_request, 0);
+  expect_aio_flush(mock_image_ctx, mock_io_image_request, 0);
   ASSERT_EQ(0, when_flush(mock_journal_replay));
   ASSERT_EQ(0, on_safe.wait());
 }
diff --git a/src/test/librbd/mock/MockAioImageRequestWQ.h b/src/test/librbd/mock/MockAioImageRequestWQ.h
deleted file mode 100644 (file)
index b5b5aae..0000000
+++ /dev/null
@@ -1,25 +0,0 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-
-#ifndef CEPH_TEST_LIBRBD_MOCK_AIO_IMAGE_REQUEST_WQ_H
-#define CEPH_TEST_LIBRBD_MOCK_AIO_IMAGE_REQUEST_WQ_H
-
-#include "gmock/gmock.h"
-
-class Context;
-
-namespace librbd {
-
-struct MockAioImageRequestWQ {
-  MOCK_METHOD1(block_writes, void(Context *));
-  MOCK_METHOD0(unblock_writes, void());
-
-  MOCK_METHOD0(set_require_lock_on_read, void());
-  MOCK_METHOD0(clear_require_lock_on_read, void());
-
-  MOCK_CONST_METHOD0(is_lock_request_needed, bool());
-};
-
-} // namespace librbd
-
-#endif // CEPH_TEST_LIBRBD_MOCK_AIO_IMAGE_REQUEST_WQ_H
index de6ecdaa5cb716b9e32b7b99700b8f1e517535d2..f900b627df3e24a56a5f00ea15e491ed01e52886 100644 (file)
@@ -6,6 +6,8 @@
 
 #include "gmock/gmock.h"
 
+struct Context;
+
 namespace librbd {
 
 struct MockContextWQ {
index b7983facb08a9d1f10a6391d295d3161f01d49ee..1b2354df1563c440948f7171a81157244dcb4894 100644 (file)
@@ -5,7 +5,6 @@
 #define CEPH_TEST_LIBRBD_MOCK_IMAGE_CTX_H
 
 #include "include/rados/librados.hpp"
-#include "test/librbd/mock/MockAioImageRequestWQ.h"
 #include "test/librbd/mock/MockContextWQ.h"
 #include "test/librbd/mock/MockExclusiveLock.h"
 #include "test/librbd/mock/MockImageState.h"
@@ -14,6 +13,7 @@
 #include "test/librbd/mock/MockObjectMap.h"
 #include "test/librbd/mock/MockOperations.h"
 #include "test/librbd/mock/MockReadahead.h"
+#include "test/librbd/mock/io/MockImageRequestWQ.h"
 #include "common/RWLock.h"
 #include "common/WorkQueue.h"
 #include "librbd/ImageCtx.h"
@@ -75,7 +75,7 @@ struct MockImageCtx {
       format_string(image_ctx.format_string),
       group_spec(image_ctx.group_spec),
       layout(image_ctx.layout),
-      aio_work_queue(new MockAioImageRequestWQ()),
+      io_work_queue(new io::MockImageRequestWQ()),
       op_work_queue(new MockContextWQ()),
       readahead_max_bytes(image_ctx.readahead_max_bytes),
       parent(NULL), operations(new MockOperations()),
@@ -116,7 +116,7 @@ struct MockImageCtx {
     delete operations;
     delete image_watcher;
     delete op_work_queue;
-    delete aio_work_queue;
+    delete io_work_queue;
   }
 
   void wait_for_async_requests() {
@@ -244,7 +244,7 @@ struct MockImageCtx {
   xlist<AsyncRequest<MockImageCtx>*> async_requests;
   std::list<Context*> async_requests_waiters;
 
-  MockAioImageRequestWQ *aio_work_queue;
+  io::MockImageRequestWQ *io_work_queue;
   MockContextWQ *op_work_queue;
 
   cache::MockImageCache *image_cache = nullptr;
index 2a4b44d076a8f73487ae94a3de65ea32b96491d3..a31a80386e34a963c15537f0a0b150d1df44cd0b 100644 (file)
 
 namespace librbd {
 
-struct AioObjectRequestHandle;
 struct ImageCtx;
+namespace io { struct ObjectRequestHandle; }
 
 struct MockJournal {
-  typedef std::list<AioObjectRequestHandle *> AioObjectRequests;
+  typedef std::list<io::ObjectRequestHandle *> ObjectRequests;
 
   static MockJournal *s_instance;
   static MockJournal *get_instance() {
@@ -54,12 +54,12 @@ struct MockJournal {
 
   MOCK_METHOD5(append_write_event, uint64_t(uint64_t, size_t,
                                             const bufferlist &,
-                                            const AioObjectRequests &, bool));
+                                            const ObjectRequests &, bool));
   MOCK_METHOD5(append_io_event_mock, uint64_t(const journal::EventEntry&,
-                                              const AioObjectRequests &,
+                                              const ObjectRequests &,
                                               uint64_t, size_t, bool));
   uint64_t append_io_event(journal::EventEntry &&event_entry,
-                           const AioObjectRequests &requests,
+                           const ObjectRequests &requests,
                            uint64_t offset, size_t length,
                            bool flush_entry) {
     // googlemock doesn't support move semantics
diff --git a/src/test/librbd/mock/io/MockImageRequestWQ.h b/src/test/librbd/mock/io/MockImageRequestWQ.h
new file mode 100644 (file)
index 0000000..972b75f
--- /dev/null
@@ -0,0 +1,27 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_TEST_LIBRBD_MOCK_IO_IMAGE_REQUEST_WQ_H
+#define CEPH_TEST_LIBRBD_MOCK_IO_IMAGE_REQUEST_WQ_H
+
+#include "gmock/gmock.h"
+
+class Context;
+
+namespace librbd {
+namespace io {
+
+struct MockImageRequestWQ {
+  MOCK_METHOD1(block_writes, void(Context *));
+  MOCK_METHOD0(unblock_writes, void());
+
+  MOCK_METHOD0(set_require_lock_on_read, void());
+  MOCK_METHOD0(clear_require_lock_on_read, void());
+
+  MOCK_CONST_METHOD0(is_lock_request_needed, bool());
+};
+
+} // namespace io
+} // namespace librbd
+
+#endif // CEPH_TEST_LIBRBD_MOCK_IO_IMAGE_REQUEST_WQ_H
index 53e4d6beaf79de652356c86515e9a1eb053da3f3..4d451c71545ea13e61c8e0072d7f6e857bfde95d 100644 (file)
@@ -6,9 +6,9 @@
 #include "test/librbd/mock/MockImageCtx.h"
 #include "test/librbd/mock/MockJournalPolicy.h"
 #include "cls/rbd/cls_rbd_client.h"
-#include "librbd/AioCompletion.h"
 #include "librbd/internal.h"
 #include "librbd/image/SetFlagsRequest.h"
+#include "librbd/io/AioCompletion.h"
 #include "librbd/mirror/DisableRequest.h"
 #include "librbd/journal/RemoveRequest.h"
 #include "librbd/journal/StandardPolicy.h"
@@ -203,12 +203,12 @@ public:
   }
 
   void expect_block_writes(MockOperationImageCtx &mock_image_ctx) {
-    EXPECT_CALL(*mock_image_ctx.aio_work_queue, block_writes(_))
+    EXPECT_CALL(*mock_image_ctx.io_work_queue, block_writes(_))
       .WillOnce(CompleteContext(0, mock_image_ctx.image_ctx->op_work_queue));
   }
 
   void expect_unblock_writes(MockOperationImageCtx &mock_image_ctx) {
-    EXPECT_CALL(*mock_image_ctx.aio_work_queue, unblock_writes()).Times(1);
+    EXPECT_CALL(*mock_image_ctx.io_work_queue, unblock_writes()).Times(1);
   }
 
   void expect_verify_lock_ownership(MockOperationImageCtx &mock_image_ctx) {
index 90ee5f79fce02d5152e01b0e00b748e01d6679c9..5085d04dfca6bcdc9a0b93acad4752bea9f8b51b 100644 (file)
@@ -5,10 +5,10 @@
 #include "test/librbd/test_support.h"
 #include "test/librbd/mock/MockImageCtx.h"
 #include "cls/rbd/cls_rbd_client.h"
-#include "librbd/AioCompletion.h"
 #include "librbd/Operations.h"
 #include "librbd/internal.h"
 #include "librbd/image/SetFlagsRequest.h"
+#include "librbd/io/AioCompletion.h"
 #include "librbd/mirror/EnableRequest.h"
 #include "librbd/journal/CreateRequest.h"
 #include "librbd/journal/Types.h"
@@ -212,12 +212,12 @@ public:
   }
 
   void expect_block_writes(MockOperationImageCtx &mock_image_ctx) {
-    EXPECT_CALL(*mock_image_ctx.aio_work_queue, block_writes(_))
+    EXPECT_CALL(*mock_image_ctx.io_work_queue, block_writes(_))
       .WillOnce(CompleteContext(0, mock_image_ctx.image_ctx->op_work_queue));
   }
 
   void expect_unblock_writes(MockOperationImageCtx &mock_image_ctx) {
-    EXPECT_CALL(*mock_image_ctx.aio_work_queue, unblock_writes()).Times(1);
+    EXPECT_CALL(*mock_image_ctx.io_work_queue, unblock_writes()).Times(1);
   }
 
   void expect_verify_lock_ownership(MockOperationImageCtx &mock_image_ctx) {
index 090a1609943bd6e995f909e686d83d3a9f47a4aa..851f34a108f1ca90f88eb1281e3b3d81469f5180 100644 (file)
@@ -62,12 +62,12 @@ public:
   typedef TrimRequest<MockImageCtx> MockTrimRequest;
 
   void expect_block_writes(MockImageCtx &mock_image_ctx, int r) {
-    EXPECT_CALL(*mock_image_ctx.aio_work_queue, block_writes(_))
+    EXPECT_CALL(*mock_image_ctx.io_work_queue, block_writes(_))
                   .WillOnce(CompleteContext(r, mock_image_ctx.image_ctx->op_work_queue));
   }
 
   void expect_unblock_writes(MockImageCtx &mock_image_ctx) {
-    EXPECT_CALL(*mock_image_ctx.aio_work_queue, unblock_writes())
+    EXPECT_CALL(*mock_image_ctx.io_work_queue, unblock_writes())
                   .Times(1);
   }
 
index dada4a552b712fedeaf2c8334a5063edabc1ca82..3fce272a6bd1609f7c8eb873d973b74ba0ada86d 100644 (file)
@@ -31,7 +31,7 @@ public:
   typedef SnapshotCreateRequest<MockImageCtx> MockSnapshotCreateRequest;
 
   void expect_block_writes(MockImageCtx &mock_image_ctx) {
-    EXPECT_CALL(*mock_image_ctx.aio_work_queue, block_writes(_))
+    EXPECT_CALL(*mock_image_ctx.io_work_queue, block_writes(_))
                   .WillOnce(CompleteContext(0, mock_image_ctx.image_ctx->op_work_queue));
   }
 
@@ -93,7 +93,7 @@ public:
   }
 
   void expect_unblock_writes(MockImageCtx &mock_image_ctx) {
-    EXPECT_CALL(*mock_image_ctx.aio_work_queue, unblock_writes())
+    EXPECT_CALL(*mock_image_ctx.io_work_queue, unblock_writes())
                   .Times(1);
   }
 
index 42cedf8a16a48de7f25dca79d78f042dd2c7ef64..a65206c42b965720abb399c5898852a397ef52d5 100644 (file)
@@ -88,12 +88,12 @@ public:
   typedef ResizeRequest<MockOperationImageCtx> MockResizeRequest;
 
   void expect_block_writes(MockOperationImageCtx &mock_image_ctx, int r) {
-    EXPECT_CALL(*mock_image_ctx.aio_work_queue, block_writes(_))
+    EXPECT_CALL(*mock_image_ctx.io_work_queue, block_writes(_))
                   .WillOnce(CompleteContext(r, mock_image_ctx.image_ctx->op_work_queue));
   }
 
   void expect_unblock_writes(MockOperationImageCtx &mock_image_ctx) {
-    EXPECT_CALL(*mock_image_ctx.aio_work_queue, unblock_writes())
+    EXPECT_CALL(*mock_image_ctx.io_work_queue, unblock_writes())
                   .Times(1);
   }
 
diff --git a/src/test/librbd/test_ConsistencyGroups.cc b/src/test/librbd/test_ConsistencyGroups.cc
deleted file mode 100644 (file)
index db30fe6..0000000
+++ /dev/null
@@ -1,91 +0,0 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-#include "test/librbd/test_fixture.h"
-#include "test/librbd/test_support.h"
-#include "include/int_types.h"
-#include "include/stringify.h"
-#include "include/rados/librados.h"
-#include "include/rbd/librbd.hpp"
-#include "common/Cond.h"
-#include "common/errno.h"
-#include "common/Mutex.h"
-#include "common/RWLock.h"
-#include "cls/lock/cls_lock_client.h"
-#include "cls/lock/cls_lock_types.h"
-#include "librbd/AioCompletion.h"
-#include "librbd/AioImageRequestWQ.h"
-#include "librbd/internal.h"
-#include "librbd/ImageCtx.h"
-#include "librbd/ImageWatcher.h"
-#include "librbd/WatchNotifyTypes.h"
-#include "test/librados/test.h"
-#include "gtest/gtest.h"
-#include <boost/assign/std/set.hpp>
-#include <boost/assign/std/map.hpp>
-#include <boost/bind.hpp>
-#include <boost/scope_exit.hpp>
-#include <boost/thread/thread.hpp>
-#include <iostream>
-#include <map>
-#include <set>
-#include <sstream>
-#include <vector>
-
-using namespace ceph;
-using namespace boost::assign;
-using namespace librbd::watch_notify;
-
-void register_test_consistency_groups() {
-}
-
-class TestLibCG : public TestFixture {
-
-};
-
-TEST_F(TestLibCG, group_create)
-{
-  librados::IoCtx ioctx;
-  ASSERT_EQ(0, _rados.ioctx_create(_pool_name.c_str(), ioctx));
-
-  librbd::RBD rbd;
-  ASSERT_EQ(0, rbd.group_create(ioctx, "mygroup"));
-
-  vector<string> groups;
-  ASSERT_EQ(0, rbd.group_list(ioctx, &groups));
-  ASSERT_EQ(1U, groups.size());
-  ASSERT_EQ("mygroup", groups[0]);
-
-  ASSERT_EQ(0, rbd.group_remove(ioctx, "mygroup"));
-
-  groups.clear();
-  ASSERT_EQ(0, rbd.group_list(ioctx, &groups));
-  ASSERT_EQ(0U, groups.size());
-}
-
-TEST_F(TestLibCG, add_image)
-{
-  librados::IoCtx ioctx;
-  ASSERT_EQ(0, _rados.ioctx_create(_pool_name.c_str(), ioctx));
-
-  const char *group_name = "mycg";
-  const char *image_name = "myimage";
-  librbd::RBD rbd;
-  ASSERT_EQ(0, rbd.group_create(ioctx, group_name));
-  int order = 14;
-  ASSERT_EQ(0, rbd.create2(ioctx, image_name, 65535,
-                          RBD_FEATURE_LAYERING, &order)); // Specified features make image of new format.
-
-  ASSERT_EQ(0, rbd.group_image_add(ioctx, group_name, ioctx, image_name));
-
-  vector<librbd::group_image_status_t> images;
-  ASSERT_EQ(0, rbd.group_image_list(ioctx, group_name, &images));
-  ASSERT_EQ(1U, images.size());
-  ASSERT_EQ("myimage", images[0].name);
-  ASSERT_EQ(ioctx.get_id(), images[0].pool);
-
-  ASSERT_EQ(0, rbd.group_image_remove(ioctx, group_name, ioctx, image_name));
-
-  images.clear();
-  ASSERT_EQ(0, rbd.group_image_list(ioctx, group_name, &images));
-  ASSERT_EQ(0U, images.size());
-}
diff --git a/src/test/librbd/test_Groups.cc b/src/test/librbd/test_Groups.cc
new file mode 100644 (file)
index 0000000..39f3095
--- /dev/null
@@ -0,0 +1,91 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+#include "test/librbd/test_fixture.h"
+#include "test/librbd/test_support.h"
+#include "include/int_types.h"
+#include "include/stringify.h"
+#include "include/rados/librados.h"
+#include "include/rbd/librbd.hpp"
+#include "common/Cond.h"
+#include "common/errno.h"
+#include "common/Mutex.h"
+#include "common/RWLock.h"
+#include "cls/lock/cls_lock_client.h"
+#include "cls/lock/cls_lock_types.h"
+#include "librbd/internal.h"
+#include "librbd/ImageCtx.h"
+#include "librbd/ImageWatcher.h"
+#include "librbd/WatchNotifyTypes.h"
+#include "librbd/io/AioCompletion.h"
+#include "librbd/io/ImageRequestWQ.h"
+#include "test/librados/test.h"
+#include "gtest/gtest.h"
+#include <boost/assign/std/set.hpp>
+#include <boost/assign/std/map.hpp>
+#include <boost/bind.hpp>
+#include <boost/scope_exit.hpp>
+#include <boost/thread/thread.hpp>
+#include <iostream>
+#include <map>
+#include <set>
+#include <sstream>
+#include <vector>
+
+using namespace ceph;
+using namespace boost::assign;
+using namespace librbd::watch_notify;
+
+void register_test_groups() {
+}
+
+class TestLibCG : public TestFixture {
+
+};
+
+TEST_F(TestLibCG, group_create)
+{
+  librados::IoCtx ioctx;
+  ASSERT_EQ(0, _rados.ioctx_create(_pool_name.c_str(), ioctx));
+
+  librbd::RBD rbd;
+  ASSERT_EQ(0, rbd.group_create(ioctx, "mygroup"));
+
+  vector<string> groups;
+  ASSERT_EQ(0, rbd.group_list(ioctx, &groups));
+  ASSERT_EQ(1U, groups.size());
+  ASSERT_EQ("mygroup", groups[0]);
+
+  ASSERT_EQ(0, rbd.group_remove(ioctx, "mygroup"));
+
+  groups.clear();
+  ASSERT_EQ(0, rbd.group_list(ioctx, &groups));
+  ASSERT_EQ(0U, groups.size());
+}
+
+TEST_F(TestLibCG, add_image)
+{
+  librados::IoCtx ioctx;
+  ASSERT_EQ(0, _rados.ioctx_create(_pool_name.c_str(), ioctx));
+
+  const char *group_name = "mycg";
+  const char *image_name = "myimage";
+  librbd::RBD rbd;
+  ASSERT_EQ(0, rbd.group_create(ioctx, group_name));
+  int order = 14;
+  ASSERT_EQ(0, rbd.create2(ioctx, image_name, 65535,
+                          RBD_FEATURE_LAYERING, &order)); // Specified features make image of new format.
+
+  ASSERT_EQ(0, rbd.group_image_add(ioctx, group_name, ioctx, image_name));
+
+  vector<librbd::group_image_status_t> images;
+  ASSERT_EQ(0, rbd.group_image_list(ioctx, group_name, &images));
+  ASSERT_EQ(1U, images.size());
+  ASSERT_EQ("myimage", images[0].name);
+  ASSERT_EQ(ioctx.get_id(), images[0].pool);
+
+  ASSERT_EQ(0, rbd.group_image_remove(ioctx, group_name, ioctx, image_name));
+
+  images.clear();
+  ASSERT_EQ(0, rbd.group_image_list(ioctx, group_name, &images));
+  ASSERT_EQ(0U, images.size());
+}
index 740d030a618ce715bd0895f1bdf2ffdeec50fa5f..24d420898ad6abd4e92d8120cfdd19d9093e839e 100644 (file)
 #include "common/RWLock.h"
 #include "cls/lock/cls_lock_client.h"
 #include "cls/lock/cls_lock_types.h"
-#include "librbd/AioCompletion.h"
-#include "librbd/AioImageRequestWQ.h"
 #include "librbd/internal.h"
 #include "librbd/ImageCtx.h"
 #include "librbd/ImageWatcher.h"
 #include "librbd/WatchNotifyTypes.h"
+#include "librbd/io/AioCompletion.h"
+#include "librbd/io/ImageRequestWQ.h"
 #include "test/librados/test.h"
 #include "gtest/gtest.h"
 #include <boost/assign/std/set.hpp>
index 7b7dc05a7c2a83fb12e95541a2d76fcdf8a49ee6..bf3e149dfc84ee95e14ae2222ad3395e926a7248 100644 (file)
@@ -3,11 +3,11 @@
 #include "test/librbd/test_fixture.h"
 #include "test/librbd/test_support.h"
 #include "include/stringify.h"
-#include "librbd/AioImageRequestWQ.h"
 #include "librbd/ExclusiveLock.h"
 #include "librbd/ImageState.h"
 #include "librbd/ImageWatcher.h"
 #include "librbd/Operations.h"
+#include "librbd/io/ImageRequestWQ.h"
 #include "cls/lock/cls_lock_client.h"
 #include "cls/lock/cls_lock_types.h"
 #include "cls/rbd/cls_rbd_types.h"
@@ -123,7 +123,7 @@ int TestFixture::unlock_image() {
 }
 
 int TestFixture::acquire_exclusive_lock(librbd::ImageCtx &ictx) {
-  int r = ictx.aio_work_queue->write(0, 0, "", 0);
+  int r = ictx.io_work_queue->write(0, 0, "", 0);
   if (r != 0) {
     return r;
   }
index fc192959773eba17a04c49959a26b5ead329ee21..743559be78d32c182083ba7626d6c948ba1db03f 100644 (file)
@@ -3,15 +3,15 @@
 #include "cls/rbd/cls_rbd_types.h"
 #include "test/librbd/test_fixture.h"
 #include "test/librbd/test_support.h"
-#include "librbd/AioCompletion.h"
-#include "librbd/AioImageRequest.h"
-#include "librbd/AioImageRequestWQ.h"
 #include "librbd/ExclusiveLock.h"
 #include "librbd/ImageState.h"
 #include "librbd/ImageWatcher.h"
 #include "librbd/internal.h"
 #include "librbd/ObjectMap.h"
 #include "librbd/Operations.h"
+#include "librbd/io/AioCompletion.h"
+#include "librbd/io/ImageRequest.h"
+#include "librbd/io/ImageRequestWQ.h"
 #include <boost/scope_exit.hpp>
 #include <boost/assign/list_of.hpp>
 #include <utility>
@@ -272,9 +272,9 @@ TEST_F(TestInternal, AioWriteRequestsLock) {
 
   std::string buffer(256, '1');
   Context *ctx = new DummyContext();
-  librbd::AioCompletion *c = librbd::AioCompletion::create(ctx);
+  auto c = librbd::io::AioCompletion::create(ctx);
   c->get();
-  ictx->aio_work_queue->aio_write(c, 0, buffer.size(), buffer.c_str(), 0);
+  ictx->io_work_queue->aio_write(c, 0, buffer.size(), buffer.c_str(), 0);
 
   bool is_owner;
   ASSERT_EQ(0, librbd::is_exclusive_lock_owner(ictx, &is_owner));
@@ -294,9 +294,9 @@ TEST_F(TestInternal, AioDiscardRequestsLock) {
   ASSERT_EQ(0, lock_image(*ictx, LOCK_EXCLUSIVE, "manually locked"));
 
   Context *ctx = new DummyContext();
-  librbd::AioCompletion *c = librbd::AioCompletion::create(ctx);
+  auto c = librbd::io::AioCompletion::create(ctx);
   c->get();
-  ictx->aio_work_queue->aio_discard(c, 0, 256);
+  ictx->io_work_queue->aio_discard(c, 0, 256);
 
   bool is_owner;
   ASSERT_EQ(0, librbd::is_exclusive_lock_owner(ictx, &is_owner));
@@ -493,7 +493,7 @@ TEST_F(TestInternal, SnapshotCopyup)
 
   bufferlist bl;
   bl.append(std::string(256, '1'));
-  ASSERT_EQ(256, ictx->aio_work_queue->write(0, bl.length(), bl.c_str(), 0));
+  ASSERT_EQ(256, ictx->io_work_queue->write(0, bl.length(), bl.c_str(), 0));
 
   ASSERT_EQ(0, snap_create(*ictx, "snap1"));
   ASSERT_EQ(0, ictx->operations->snap_protect("snap1"));
@@ -512,7 +512,7 @@ TEST_F(TestInternal, SnapshotCopyup)
   ASSERT_EQ(0, snap_create(*ictx2, "snap1"));
   ASSERT_EQ(0, snap_create(*ictx2, "snap2"));
 
-  ASSERT_EQ(256, ictx2->aio_work_queue->write(256, bl.length(), bl.c_str(), 0));
+  ASSERT_EQ(256, ictx2->io_work_queue->write(256, bl.length(), bl.c_str(), 0));
 
   librados::IoCtx snap_ctx;
   snap_ctx.dup(ictx2->data_ctx);
@@ -541,10 +541,10 @@ TEST_F(TestInternal, SnapshotCopyup)
     const char *snap_name = it->empty() ? NULL : it->c_str();
     ASSERT_EQ(0, librbd::snap_set(ictx2, snap_name));
 
-    ASSERT_EQ(256, ictx2->aio_work_queue->read(0, 256, read_bl.c_str(), 0));
+    ASSERT_EQ(256, ictx2->io_work_queue->read(0, 256, read_bl.c_str(), 0));
     ASSERT_TRUE(bl.contents_equal(read_bl));
 
-    ASSERT_EQ(256, ictx2->aio_work_queue->read(256, 256, read_bl.c_str(), 0));
+    ASSERT_EQ(256, ictx2->io_work_queue->read(256, 256, read_bl.c_str(), 0));
     if (snap_name == NULL) {
       ASSERT_TRUE(bl.contents_equal(read_bl));
     } else {
@@ -589,8 +589,8 @@ TEST_F(TestInternal, ResizeCopyup)
   bufferlist bl;
   bl.append(std::string(4096, '1'));
   for (size_t i = 0; i < m_image_size; i += bl.length()) {
-    ASSERT_EQ(bl.length(), ictx->aio_work_queue->write(i, bl.length(),
-                                                       bl.c_str(), 0));
+    ASSERT_EQ(bl.length(), ictx->io_work_queue->write(i, bl.length(),
+                                                      bl.c_str(), 0));
   }
 
   ASSERT_EQ(0, snap_create(*ictx, "snap1"));
@@ -623,8 +623,8 @@ TEST_F(TestInternal, ResizeCopyup)
   }
 
   for (size_t i = 2 << order; i < m_image_size; i += bl.length()) {
-    ASSERT_EQ(bl.length(), ictx2->aio_work_queue->read(i, bl.length(),
-                                                       read_bl.c_str(), 0));
+    ASSERT_EQ(bl.length(), ictx2->io_work_queue->read(i, bl.length(),
+                                                      read_bl.c_str(), 0));
     ASSERT_TRUE(bl.contents_equal(read_bl));
   }
 }
@@ -651,8 +651,8 @@ TEST_F(TestInternal, DiscardCopyup)
   bufferlist bl;
   bl.append(std::string(4096, '1'));
   for (size_t i = 0; i < m_image_size; i += bl.length()) {
-    ASSERT_EQ(bl.length(), ictx->aio_work_queue->write(i, bl.length(),
-                                                       bl.c_str(), 0));
+    ASSERT_EQ(bl.length(), ictx->io_work_queue->write(i, bl.length(),
+                                                      bl.c_str(), 0));
   }
 
   ASSERT_EQ(0, snap_create(*ictx, "snap1"));
@@ -672,7 +672,7 @@ TEST_F(TestInternal, DiscardCopyup)
   read_bl.push_back(read_ptr);
 
   ASSERT_EQ(static_cast<int>(m_image_size - 64),
-            ictx2->aio_work_queue->discard(32, m_image_size - 64));
+            ictx2->io_work_queue->discard(32, m_image_size - 64));
   ASSERT_EQ(0, librbd::snap_set(ictx2, "snap1"));
 
   {
@@ -682,8 +682,8 @@ TEST_F(TestInternal, DiscardCopyup)
   }
 
   for (size_t i = 0; i < m_image_size; i += bl.length()) {
-    ASSERT_EQ(bl.length(), ictx2->aio_work_queue->read(i, bl.length(),
-                                                       read_bl.c_str(), 0));
+    ASSERT_EQ(bl.length(), ictx2->io_work_queue->read(i, bl.length(),
+                                                      read_bl.c_str(), 0));
     ASSERT_TRUE(bl.contents_equal(read_bl));
   }
 }
@@ -695,12 +695,12 @@ TEST_F(TestInternal, ShrinkFlushesCache) {
   std::string buffer(4096, '1');
 
   // ensure write-path is initialized
-  ictx->aio_work_queue->write(0, buffer.size(), buffer.c_str(), 0);
+  ictx->io_work_queue->write(0, buffer.size(), buffer.c_str(), 0);
 
   C_SaferCond cond_ctx;
-  librbd::AioCompletion *c = librbd::AioCompletion::create(&cond_ctx);
+  auto c = librbd::io::AioCompletion::create(&cond_ctx);
   c->get();
-  ictx->aio_work_queue->aio_write(c, 0, buffer.size(), buffer.c_str(), 0);
+  ictx->io_work_queue->aio_write(c, 0, buffer.size(), buffer.c_str(), 0);
 
   librbd::NoOpProgressContext no_op;
   ASSERT_EQ(0, ictx->operations->resize(m_image_size >> 1, true, no_op));
@@ -790,7 +790,7 @@ TEST_F(TestInternal, WriteFullCopyup) {
   bufferlist bl;
   bl.append(std::string(1 << ictx->order, '1'));
   ASSERT_EQ(bl.length(),
-            ictx->aio_work_queue->write(0, bl.length(), bl.c_str(), 0));
+            ictx->io_work_queue->write(0, bl.length(), bl.c_str(), 0));
   ASSERT_EQ(0, librbd::flush(ictx));
 
   ASSERT_EQ(0, create_snapshot("snap1", true));
@@ -819,7 +819,7 @@ TEST_F(TestInternal, WriteFullCopyup) {
   bufferlist write_full_bl;
   write_full_bl.append(std::string(1 << ictx2->order, '2'));
   ASSERT_EQ(write_full_bl.length(),
-            ictx2->aio_work_queue->write(0, write_full_bl.length(),
+            ictx2->io_work_queue->write(0, write_full_bl.length(),
             write_full_bl.c_str(), 0));
 
   ASSERT_EQ(0, ictx2->operations->flatten(no_op));
@@ -828,13 +828,13 @@ TEST_F(TestInternal, WriteFullCopyup) {
   bufferlist read_bl;
   read_bl.push_back(read_ptr);
 
-  ASSERT_EQ(read_bl.length(), ictx2->aio_work_queue->read(0, read_bl.length(),
-                                                          read_bl.c_str(), 0));
+  ASSERT_EQ(read_bl.length(), ictx2->io_work_queue->read(0, read_bl.length(),
+                                                         read_bl.c_str(), 0));
   ASSERT_TRUE(write_full_bl.contents_equal(read_bl));
 
   ASSERT_EQ(0, librbd::snap_set(ictx2, "snap1"));
-  ASSERT_EQ(read_bl.length(), ictx2->aio_work_queue->read(0, read_bl.length(),
-                                                          read_bl.c_str(), 0));
+  ASSERT_EQ(read_bl.length(), ictx2->io_work_queue->read(0, read_bl.length(),
+                                                         read_bl.c_str(), 0));
   ASSERT_TRUE(bl.contents_equal(read_bl));
 }
 
index 8835162d8f024a792b81c3c95d56ad2bf09704f1..0aa6aeeb81c9603ac4d07b1569fa97aa40292007 100644 (file)
@@ -10,6 +10,7 @@
 
 extern void register_test_librbd();
 #ifdef TEST_LIBRBD_INTERNALS
+extern void register_test_groups();
 extern void register_test_image_watcher();
 extern void register_test_internal();
 extern void register_test_journal_entries();
@@ -18,13 +19,13 @@ extern void register_test_object_map();
 extern void register_test_operations();
 extern void register_test_mirroring();
 extern void register_test_mirroring_watcher();
-extern void register_test_consistency_groups();
 #endif // TEST_LIBRBD_INTERNALS
 
 int main(int argc, char **argv)
 {
   register_test_librbd();
 #ifdef TEST_LIBRBD_INTERNALS
+  register_test_groups();
   register_test_image_watcher();
   register_test_internal();
   register_test_journal_entries();
@@ -33,7 +34,6 @@ int main(int argc, char **argv)
   register_test_operations();
   register_test_mirroring();
   register_test_mirroring_watcher();
-  register_test_consistency_groups();
 #endif // TEST_LIBRBD_INTERNALS
 
   ::testing::InitGoogleTest(&argc, argv);
index b2c0cff5ae53be1680f2f55169a41a3a66705f56..8e70f383b238c4990a33af624539ee7f369210a7 100644 (file)
  */
 #include "test/librbd/test_fixture.h"
 #include "test/librbd/test_support.h"
-#include "librbd/AioCompletion.h"
-#include "librbd/AioImageRequest.h"
-#include "librbd/AioImageRequestWQ.h"
 #include "librbd/ExclusiveLock.h"
 #include "librbd/ImageState.h"
 #include "librbd/ImageWatcher.h"
 #include "librbd/internal.h"
 #include "librbd/ObjectMap.h"
 #include "librbd/Operations.h"
+#include "librbd/io/AioCompletion.h"
+#include "librbd/io/ImageRequest.h"
+#include "librbd/io/ImageRequestWQ.h"
 #include "librbd/journal/Types.h"
 #include "journal/Journaler.h"
 #include "journal/Settings.h"
diff --git a/src/test/librbd/test_mock_AioImageRequest.cc b/src/test/librbd/test_mock_AioImageRequest.cc
deleted file mode 100644 (file)
index aafe2b6..0000000
+++ /dev/null
@@ -1,262 +0,0 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-
-#include "test/librbd/test_mock_fixture.h"
-#include "test/librbd/test_support.h"
-#include "test/librbd/mock/MockImageCtx.h"
-#include "test/librbd/mock/MockJournal.h"
-#include "test/librbd/mock/cache/MockImageCache.h"
-#include "librbd/AioImageRequest.h"
-#include "librbd/AioObjectRequest.h"
-
-namespace librbd {
-namespace {
-
-struct MockTestImageCtx : public MockImageCtx {
-  MockTestImageCtx(ImageCtx &image_ctx) : MockImageCtx(image_ctx) {
-  }
-};
-
-} // anonymous namespace
-
-namespace util {
-
-inline ImageCtx *get_image_ctx(MockTestImageCtx *image_ctx) {
-  return image_ctx->image_ctx;
-}
-
-} // namespace util
-
-template <>
-struct AioObjectRequest<librbd::MockTestImageCtx> : public AioObjectRequestHandle {
-  static AioObjectRequest* s_instance;
-  Context *on_finish = nullptr;
-
-  static AioObjectRequest* create_remove(librbd::MockTestImageCtx *ictx,
-                                         const std::string &oid,
-                                         uint64_t object_no,
-                                         const ::SnapContext &snapc,
-                                         Context *completion) {
-    assert(s_instance != nullptr);
-    s_instance->on_finish = completion;
-    return s_instance;
-  }
-
-  static AioObjectRequest* create_truncate(librbd::MockTestImageCtx *ictx,
-                                           const std::string &oid,
-                                           uint64_t object_no,
-                                           uint64_t object_off,
-                                           const ::SnapContext &snapc,
-                                           Context *completion) {
-    assert(s_instance != nullptr);
-    s_instance->on_finish = completion;
-    return s_instance;
-  }
-
-  static AioObjectRequest* create_write(librbd::MockTestImageCtx *ictx,
-                                        const std::string &oid,
-                                        uint64_t object_no,
-                                        uint64_t object_off,
-                                        const ceph::bufferlist &data,
-                                        const ::SnapContext &snapc,
-                                        Context *completion, int op_flags) {
-    assert(s_instance != nullptr);
-    s_instance->on_finish = completion;
-    return s_instance;
-  }
-
-  static AioObjectRequest* create_zero(librbd::MockTestImageCtx *ictx,
-                                       const std::string &oid,
-                                       uint64_t object_no, uint64_t object_off,
-                                       uint64_t object_len,
-                                       const ::SnapContext &snapc,
-                                       Context *completion) {
-    assert(s_instance != nullptr);
-    s_instance->on_finish = completion;
-    return s_instance;
-  }
-
-  AioObjectRequest() {
-    assert(s_instance == nullptr);
-    s_instance = this;
-  }
-  ~AioObjectRequest() {
-    s_instance = nullptr;
-  }
-
-  MOCK_METHOD1(complete, void(int));
-  MOCK_METHOD0(send, void());
-};
-
-template <>
-struct AioObjectRead<librbd::MockTestImageCtx> : public AioObjectRequest<librbd::MockTestImageCtx> {
-  typedef std::vector<std::pair<uint64_t, uint64_t> > Extents;
-  typedef std::map<uint64_t, uint64_t> ExtentMap;
-
-  static AioObjectRead* s_instance;
-
-  static AioObjectRead* create(librbd::MockTestImageCtx *ictx,
-                               const std::string &oid,
-                               uint64_t objectno, uint64_t offset,
-                               uint64_t len, Extents &buffer_extents,
-                               librados::snap_t snap_id, bool sparse,
-                               Context *completion, int op_flags) {
-    assert(s_instance != nullptr);
-    s_instance->on_finish = completion;
-    return s_instance;
-  }
-
-  AioObjectRead() {
-    assert(s_instance == nullptr);
-    s_instance = this;
-  }
-  ~AioObjectRead() {
-    s_instance = nullptr;
-  }
-
-  MOCK_CONST_METHOD0(get_offset, uint64_t());
-  MOCK_CONST_METHOD0(get_length, uint64_t());
-  MOCK_METHOD0(data, ceph::bufferlist &());
-  MOCK_CONST_METHOD0(get_buffer_extents, const Extents &());
-  MOCK_METHOD0(get_extent_map, ExtentMap &());
-
-};
-
-AioObjectRequest<librbd::MockTestImageCtx>* AioObjectRequest<librbd::MockTestImageCtx>::s_instance = nullptr;
-AioObjectRead<librbd::MockTestImageCtx>* AioObjectRead<librbd::MockTestImageCtx>::s_instance = nullptr;
-
-} // namespace librbd
-
-#include "librbd/AioImageRequest.cc"
-
-namespace librbd {
-
-using ::testing::_;
-using ::testing::InSequence;
-using ::testing::Invoke;
-using ::testing::Return;
-using ::testing::WithArg;
-
-struct TestMockAioImageRequest : public TestMockFixture {
-  typedef AioImageRequest<librbd::MockTestImageCtx> MockAioImageRequest;
-  typedef AioImageWrite<librbd::MockTestImageCtx> MockAioImageWrite;
-  typedef AioImageDiscard<librbd::MockTestImageCtx> MockAioImageDiscard;
-  typedef AioImageFlush<librbd::MockTestImageCtx> MockAioImageFlush;
-  typedef AioObjectRequest<librbd::MockTestImageCtx> MockAioObjectRequest;
-  typedef AioObjectRead<librbd::MockTestImageCtx> MockAioObjectRead;
-
-  void expect_is_journal_appending(MockJournal &mock_journal, bool appending) {
-    EXPECT_CALL(mock_journal, is_journal_appending())
-      .WillOnce(Return(appending));
-  }
-
-  void expect_write_to_cache(MockImageCtx &mock_image_ctx,
-                             const object_t &object,
-                             uint64_t offset, uint64_t length,
-                             uint64_t journal_tid, int r) {
-    EXPECT_CALL(mock_image_ctx, write_to_cache(object, _, length, offset, _, _,
-                journal_tid))
-      .WillOnce(WithArg<4>(CompleteContext(r, mock_image_ctx.image_ctx->op_work_queue)));
-  }
-
-  void expect_object_request_send(MockImageCtx &mock_image_ctx,
-                                  MockAioObjectRequest &mock_object_request,
-                                  int r) {
-    EXPECT_CALL(mock_object_request, send())
-      .WillOnce(Invoke([&mock_image_ctx, &mock_object_request, r]() {
-                  mock_image_ctx.image_ctx->op_work_queue->queue(
-                    mock_object_request.on_finish, r);
-                }));
-  }
-
-  void expect_user_flushed(MockImageCtx &mock_image_ctx) {
-    EXPECT_CALL(mock_image_ctx, user_flushed());
-  }
-
-  void expect_flush(MockImageCtx &mock_image_ctx, int r) {
-    EXPECT_CALL(mock_image_ctx, flush(_))
-      .WillOnce(CompleteContext(r, mock_image_ctx.image_ctx->op_work_queue));
-  }
-};
-
-TEST_F(TestMockAioImageRequest, AioWriteJournalAppendDisabled) {
-  REQUIRE_FEATURE(RBD_FEATURE_JOURNALING);
-
-  librbd::ImageCtx *ictx;
-  ASSERT_EQ(0, open_image(m_image_name, &ictx));
-
-  MockAioObjectRequest mock_aio_object_request;
-  MockTestImageCtx mock_image_ctx(*ictx);
-  MockJournal mock_journal;
-  mock_image_ctx.journal = &mock_journal;
-
-  InSequence seq;
-  expect_is_journal_appending(mock_journal, false);
-  expect_write_to_cache(mock_image_ctx, ictx->get_object_name(0),
-                        0, 1, 0, 0);
-
-  C_SaferCond aio_comp_ctx;
-  AioCompletion *aio_comp = AioCompletion::create_and_start(
-    &aio_comp_ctx, ictx, AIO_TYPE_WRITE);
-  MockAioImageWrite mock_aio_image_write(mock_image_ctx, aio_comp, 0, 1, "1",
-                                         0);
-  {
-    RWLock::RLocker owner_locker(mock_image_ctx.owner_lock);
-    mock_aio_image_write.send();
-  }
-  ASSERT_EQ(0, aio_comp_ctx.wait());
-}
-
-TEST_F(TestMockAioImageRequest, AioDiscardJournalAppendDisabled) {
-  REQUIRE_FEATURE(RBD_FEATURE_JOURNALING);
-
-  librbd::ImageCtx *ictx;
-  ASSERT_EQ(0, open_image(m_image_name, &ictx));
-
-  MockAioObjectRequest mock_aio_object_request;
-  MockTestImageCtx mock_image_ctx(*ictx);
-  MockJournal mock_journal;
-  mock_image_ctx.journal = &mock_journal;
-
-  InSequence seq;
-  expect_is_journal_appending(mock_journal, false);
-  expect_object_request_send(mock_image_ctx, mock_aio_object_request, 0);
-
-  C_SaferCond aio_comp_ctx;
-  AioCompletion *aio_comp = AioCompletion::create_and_start(
-    &aio_comp_ctx, ictx, AIO_TYPE_DISCARD);
-  MockAioImageDiscard mock_aio_image_discard(mock_image_ctx, aio_comp, 0, 1);
-  {
-    RWLock::RLocker owner_locker(mock_image_ctx.owner_lock);
-    mock_aio_image_discard.send();
-  }
-  ASSERT_EQ(0, aio_comp_ctx.wait());
-}
-
-TEST_F(TestMockAioImageRequest, AioFlushJournalAppendDisabled) {
-  REQUIRE_FEATURE(RBD_FEATURE_JOURNALING);
-
-  librbd::ImageCtx *ictx;
-  ASSERT_EQ(0, open_image(m_image_name, &ictx));
-
-  MockTestImageCtx mock_image_ctx(*ictx);
-  MockJournal mock_journal;
-  mock_image_ctx.journal = &mock_journal;
-
-  InSequence seq;
-  expect_user_flushed(mock_image_ctx);
-  expect_is_journal_appending(mock_journal, false);
-  expect_flush(mock_image_ctx, 0);
-
-  C_SaferCond aio_comp_ctx;
-  AioCompletion *aio_comp = AioCompletion::create_and_start(
-    &aio_comp_ctx, ictx, AIO_TYPE_FLUSH);
-  MockAioImageFlush mock_aio_image_flush(mock_image_ctx, aio_comp);
-  {
-    RWLock::RLocker owner_locker(mock_image_ctx.owner_lock);
-    mock_aio_image_flush.send();
-  }
-  ASSERT_EQ(0, aio_comp_ctx.wait());
-}
-
-} // namespace librbd
index 68ceda2779774f747ec6d81973033a2998ae5e94..f593f834803e5438ff77e83b2731959ff3809055 100644 (file)
@@ -214,15 +214,15 @@ public:
   }
 
   void expect_set_require_lock_on_read(MockExclusiveLockImageCtx &mock_image_ctx) {
-    EXPECT_CALL(*mock_image_ctx.aio_work_queue, set_require_lock_on_read());
+    EXPECT_CALL(*mock_image_ctx.io_work_queue, set_require_lock_on_read());
   }
 
   void expect_clear_require_lock_on_read(MockExclusiveLockImageCtx &mock_image_ctx) {
-    EXPECT_CALL(*mock_image_ctx.aio_work_queue, clear_require_lock_on_read());
+    EXPECT_CALL(*mock_image_ctx.io_work_queue, clear_require_lock_on_read());
   }
 
   void expect_block_writes(MockExclusiveLockImageCtx &mock_image_ctx) {
-    EXPECT_CALL(*mock_image_ctx.aio_work_queue, block_writes(_))
+    EXPECT_CALL(*mock_image_ctx.io_work_queue, block_writes(_))
                   .WillOnce(CompleteContext(0, mock_image_ctx.image_ctx->op_work_queue));
     if ((mock_image_ctx.features & RBD_FEATURE_JOURNALING) != 0) {
       expect_set_require_lock_on_read(mock_image_ctx);
@@ -231,7 +231,7 @@ public:
 
   void expect_unblock_writes(MockExclusiveLockImageCtx &mock_image_ctx) {
     expect_clear_require_lock_on_read(mock_image_ctx);
-    EXPECT_CALL(*mock_image_ctx.aio_work_queue, unblock_writes());
+    EXPECT_CALL(*mock_image_ctx.io_work_queue, unblock_writes());
   }
 
   void expect_prepare_lock_complete(MockExclusiveLockImageCtx &mock_image_ctx) {
@@ -272,7 +272,7 @@ public:
   }
 
   void expect_is_lock_request_needed(MockExclusiveLockImageCtx &mock_image_ctx, bool ret) {
-    EXPECT_CALL(*mock_image_ctx.aio_work_queue, is_lock_request_needed())
+    EXPECT_CALL(*mock_image_ctx.io_work_queue, is_lock_request_needed())
                   .WillRepeatedly(Return(ret));
   }
 
index 04847e0fe03e6fe87ca8f72c3edd69d26d09b5b6..c7a6ea3041701b088fa330518289559136378eb8 100644 (file)
 #include "common/Mutex.h"
 #include "cls/journal/cls_journal_types.h"
 #include "journal/Journaler.h"
-#include "librbd/AioCompletion.h"
-#include "librbd/AioObjectRequest.h"
 #include "librbd/Journal.h"
 #include "librbd/Utils.h"
+#include "librbd/io/AioCompletion.h"
+#include "librbd/io/ObjectRequest.h"
 #include "librbd/journal/Replay.h"
 #include "librbd/journal/RemoveRequest.h"
 #include "librbd/journal/CreateRequest.h"
@@ -426,9 +426,9 @@ public:
 
   uint64_t when_append_io_event(MockJournalImageCtx &mock_image_ctx,
                                 MockJournal &mock_journal,
-                                AioObjectRequest<> *object_request = nullptr) {
+                                io::ObjectRequest<> *object_request = nullptr) {
     RWLock::RLocker owner_locker(mock_image_ctx.owner_lock);
-    MockJournal::AioObjectRequests object_requests;
+    MockJournal::IOObjectRequests object_requests;
     if (object_request != nullptr) {
       object_requests.push_back(object_request);
     }
@@ -1018,7 +1018,7 @@ TEST_F(TestMockJournal, EventCommitError) {
   };
 
   C_SaferCond object_request_ctx;
-  AioObjectRemove *object_request = new AioObjectRemove(
+  auto object_request = new io::ObjectRemoveRequest(
     ictx, "oid", 0, {}, &object_request_ctx);
 
   ::journal::MockFuture mock_future;
@@ -1059,7 +1059,7 @@ TEST_F(TestMockJournal, EventCommitErrorWithPendingWriteback) {
   };
 
   C_SaferCond object_request_ctx;
-  AioObjectRemove *object_request = new AioObjectRemove(
+  auto object_request = new io::ObjectRemoveRequest(
     ictx, "oid", 0, {}, &object_request_ctx);
 
   ::journal::MockFuture mock_future;
index e1177bf0680027e85f8bf4e5b6c397b3150806ab..8c5e92e3f219844fd0e3cd44196afc2996644949 100644 (file)
@@ -5,11 +5,11 @@
 #include "include/interval_set.h"
 #include "include/rbd/librbd.hpp"
 #include "include/rbd/object_map_types.h"
-#include "librbd/AioImageRequestWQ.h"
 #include "librbd/ImageCtx.h"
 #include "librbd/ImageState.h"
 #include "librbd/internal.h"
 #include "librbd/Operations.h"
+#include "librbd/io/ImageRequestWQ.h"
 #include "test/librados_test_stub/MockTestMemIoCtxImpl.h"
 #include "test/librbd/mock/MockImageCtx.h"
 #include "tools/rbd_mirror/Threads.h"
@@ -55,7 +55,7 @@ void scribble(librbd::ImageCtx *image_ctx, int num_ops, size_t max_size,
     bufferlist bl;
     bl.append(std::string(len, '1'));
 
-    int r = image_ctx->aio_work_queue->write(off, len, bl.c_str(), 0);
+    int r = image_ctx->io_work_queue->write(off, len, bl.c_str(), 0);
     ASSERT_EQ(static_cast<int>(len), r);
 
     interval_set<uint64_t> w;
@@ -265,16 +265,16 @@ public:
 
       bufferlist remote_bl;
       remote_bl.append(std::string(object_size, '1'));
-      r = m_remote_image_ctx->aio_work_queue->read(0, object_size,
-                                                   remote_bl.c_str(), 0);
+      r = m_remote_image_ctx->io_work_queue->read(0, object_size,
+                                                  remote_bl.c_str(), 0);
       if (r < 0) {
         return r;
       }
 
       bufferlist local_bl;
       local_bl.append(std::string(object_size, '1'));
-      r = m_local_image_ctx->aio_work_queue->read(0, object_size,
-                                                  local_bl.c_str(), 0);
+      r = m_local_image_ctx->io_work_queue->read(0, object_size,
+                                                 local_bl.c_str(), 0);
       if (r < 0) {
         return r;
       }
@@ -485,7 +485,7 @@ TEST_F(TestMockImageSyncObjectCopyRequest, Trim) {
 
   // trim the object
   uint64_t trim_offset = rand() % one.range_end();
-  ASSERT_LE(0, m_remote_image_ctx->aio_work_queue->discard(
+  ASSERT_LE(0, m_remote_image_ctx->io_work_queue->discard(
     trim_offset, one.range_end() - trim_offset));
   ASSERT_EQ(0, create_snap("sync"));
 
@@ -529,7 +529,7 @@ TEST_F(TestMockImageSyncObjectCopyRequest, Remove) {
 
   // remove the object
   uint64_t object_size = 1 << m_remote_image_ctx->order;
-  ASSERT_LE(0, m_remote_image_ctx->aio_work_queue->discard(0, object_size));
+  ASSERT_LE(0, m_remote_image_ctx->io_work_queue->discard(0, object_size));
   ASSERT_EQ(0, create_snap("sync"));
   librbd::MockTestImageCtx mock_remote_image_ctx(*m_remote_image_ctx);
   librbd::MockTestImageCtx mock_local_image_ctx(*m_local_image_ctx);
index 644b41ce7523acda7ec75f094171e581d2c8333c..29ab47ff63ae6d23c390b3f8c1587bc60f9570db 100644 (file)
@@ -23,8 +23,6 @@
 #include "cls/rbd/cls_rbd_types.h"
 #include "cls/rbd/cls_rbd_client.h"
 #include "journal/Journaler.h"
-#include "librbd/AioCompletion.h"
-#include "librbd/AioImageRequestWQ.h"
 #include "librbd/ExclusiveLock.h"
 #include "librbd/ImageCtx.h"
 #include "librbd/ImageState.h"
@@ -32,6 +30,8 @@
 #include "librbd/Operations.h"
 #include "librbd/Utils.h"
 #include "librbd/internal.h"
+#include "librbd/io/AioCompletion.h"
+#include "librbd/io/ImageRequestWQ.h"
 #include "tools/rbd_mirror/types.h"
 #include "tools/rbd_mirror/ImageReplayer.h"
 #include "tools/rbd_mirror/ImageSyncThrottler.h"
@@ -306,7 +306,7 @@ public:
                        size_t len)
   {
     size_t written;
-    written = ictx->aio_work_queue->write(off, len, test_data, 0);
+    written = ictx->io_work_queue->write(off, len, test_data, 0);
     printf("wrote: %d\n", (int)written);
     ASSERT_EQ(len, written);
   }
@@ -318,7 +318,7 @@ public:
     char *result = (char *)malloc(len + 1);
 
     ASSERT_NE(static_cast<char *>(NULL), result);
-    read = ictx->aio_work_queue->read(off, len, result, 0);
+    read = ictx->io_work_queue->read(off, len, result, 0);
     printf("read: %d\n", (int)read);
     ASSERT_EQ(len, static_cast<size_t>(read));
     result[len] = '\0';
@@ -339,9 +339,9 @@ public:
   void flush(librbd::ImageCtx *ictx)
   {
     C_SaferCond aio_flush_ctx;
-    librbd::AioCompletion *c = librbd::AioCompletion::create(&aio_flush_ctx);
+    auto c = librbd::io::AioCompletion::create(&aio_flush_ctx);
     c->get();
-    ictx->aio_work_queue->aio_flush(c);
+    ictx->io_work_queue->aio_flush(c);
     ASSERT_EQ(0, c->wait_for_complete());
     c->put();
 
index 2b79e6ba2da77efccd19a22566f215b590a9a6c2..de68bddfd12bcb39fcb0a78020f0a0dc13286ebd 100644 (file)
@@ -6,12 +6,12 @@
 #include "include/rbd/librbd.hpp"
 #include "journal/Journaler.h"
 #include "journal/Settings.h"
-#include "librbd/AioImageRequestWQ.h"
 #include "librbd/ExclusiveLock.h"
 #include "librbd/ImageCtx.h"
 #include "librbd/ImageState.h"
 #include "librbd/internal.h"
 #include "librbd/Operations.h"
+#include "librbd/io/ImageRequestWQ.h"
 #include "librbd/journal/Types.h"
 #include "tools/rbd_mirror/ImageSync.h"
 #include "tools/rbd_mirror/Threads.h"
@@ -32,11 +32,11 @@ void scribble(librbd::ImageCtx *image_ctx, int num_ops, size_t max_size)
     uint64_t len = 1 + rand() % max_size;
 
     if (rand() % 4 == 0) {
-      ASSERT_EQ((int)len, image_ctx->aio_work_queue->discard(off, len));
+      ASSERT_EQ((int)len, image_ctx->io_work_queue->discard(off, len));
     } else {
       std::string str(len, '1');
-      ASSERT_EQ((int)len, image_ctx->aio_work_queue->write(off, len,
-                                                           str.c_str(), 0));
+      ASSERT_EQ((int)len, image_ctx->io_work_queue->write(off, len,
+                                                          str.c_str(), 0));
     }
   }
 
@@ -128,9 +128,9 @@ TEST_F(TestImageSync, Simple) {
 
   for (uint64_t offset = 0; offset < m_remote_image_ctx->size;
        offset += object_size) {
-    ASSERT_LE(0, m_remote_image_ctx->aio_work_queue->read(
+    ASSERT_LE(0, m_remote_image_ctx->io_work_queue->read(
                    offset, object_size, read_remote_bl.c_str(), 0));
-    ASSERT_LE(0, m_local_image_ctx->aio_work_queue->read(
+    ASSERT_LE(0, m_local_image_ctx->io_work_queue->read(
                    offset, object_size, read_local_bl.c_str(), 0));
     ASSERT_TRUE(read_remote_bl.contents_equal(read_local_bl));
   }
@@ -144,8 +144,8 @@ TEST_F(TestImageSync, Resize) {
   uint64_t len = object_size / 10;
 
   std::string str(len, '1');
-  ASSERT_EQ((int)len, m_remote_image_ctx->aio_work_queue->write(off, len,
-                                                                str.c_str(), 0));
+  ASSERT_EQ((int)len, m_remote_image_ctx->io_work_queue->write(off, len,
+                                                               str.c_str(), 0));
   {
     RWLock::RLocker owner_locker(m_remote_image_ctx->owner_lock);
     ASSERT_EQ(0, m_remote_image_ctx->flush());
@@ -168,9 +168,9 @@ TEST_F(TestImageSync, Resize) {
   bufferlist read_local_bl;
   read_local_bl.append(std::string(len, '\0'));
 
-  ASSERT_LE(0, m_remote_image_ctx->aio_work_queue->read(
+  ASSERT_LE(0, m_remote_image_ctx->io_work_queue->read(
               off, len, read_remote_bl.c_str(), 0));
-  ASSERT_LE(0, m_local_image_ctx->aio_work_queue->read(
+  ASSERT_LE(0, m_local_image_ctx->io_work_queue->read(
               off, len, read_local_bl.c_str(), 0));
 
   ASSERT_TRUE(read_remote_bl.contents_equal(read_local_bl));
@@ -184,8 +184,8 @@ TEST_F(TestImageSync, Discard) {
   uint64_t len = object_size / 10;
 
   std::string str(len, '1');
-  ASSERT_EQ((int)len, m_remote_image_ctx->aio_work_queue->write(off, len,
-                                                                str.c_str(), 0));
+  ASSERT_EQ((int)len, m_remote_image_ctx->io_work_queue->write(off, len,
+                                                               str.c_str(), 0));
   {
     RWLock::RLocker owner_locker(m_remote_image_ctx->owner_lock);
     ASSERT_EQ(0, m_remote_image_ctx->flush());
@@ -193,8 +193,8 @@ TEST_F(TestImageSync, Discard) {
 
   ASSERT_EQ(0, create_snap(m_remote_image_ctx, "snap", nullptr));
 
-  ASSERT_EQ((int)len - 2, m_remote_image_ctx->aio_work_queue->discard(off + 1,
-                                                                      len - 2));
+  ASSERT_EQ((int)len - 2, m_remote_image_ctx->io_work_queue->discard(off + 1,
+                                                                     len - 2));
   {
     RWLock::RLocker owner_locker(m_remote_image_ctx->owner_lock);
     ASSERT_EQ(0, m_remote_image_ctx->flush());
@@ -210,9 +210,9 @@ TEST_F(TestImageSync, Discard) {
   bufferlist read_local_bl;
   read_local_bl.append(std::string(object_size, '\0'));
 
-  ASSERT_LE(0, m_remote_image_ctx->aio_work_queue->read(
+  ASSERT_LE(0, m_remote_image_ctx->io_work_queue->read(
               off, len, read_remote_bl.c_str(), 0));
-  ASSERT_LE(0, m_local_image_ctx->aio_work_queue->read(
+  ASSERT_LE(0, m_local_image_ctx->io_work_queue->read(
               off, len, read_local_bl.c_str(), 0));
 
   ASSERT_TRUE(read_remote_bl.contents_equal(read_local_bl));
@@ -280,9 +280,9 @@ TEST_F(TestImageSync, SnapshotStress) {
     ASSERT_EQ(remote_size, local_size);
 
     for (uint64_t offset = 0; offset < remote_size; offset += object_size) {
-      ASSERT_LE(0, m_remote_image_ctx->aio_work_queue->read(
+      ASSERT_LE(0, m_remote_image_ctx->io_work_queue->read(
                      offset, object_size, read_remote_bl.c_str(), 0));
-      ASSERT_LE(0, m_local_image_ctx->aio_work_queue->read(
+      ASSERT_LE(0, m_local_image_ctx->io_work_queue->read(
                      offset, object_size, read_local_bl.c_str(), 0));
       ASSERT_TRUE(read_remote_bl.contents_equal(read_local_bl));
     }