Remove the class librbd::cache::WriteLogCache.
Signed-off-by: Li, Xiaoyan <xiaoyan.li@intel.com>
cache/pwl/Types.cc
cache/pwl/ReplicatedWriteLog.cc
cache/pwl/AbstractWriteLog.cc
- cache/pwl/ImageDispatch.cc
- cache/WriteLogCache.cc)
+ cache/WriteLogImageDispatch.cc)
endif()
add_library(rbd_api STATIC librbd.cc)
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-
-#include "WriteLogCache.h"
-#include "librbd/cache/pwl/ReplicatedWriteLog.h"
-#include "librbd/cache/pwl/ImageCacheState.h"
-
-#undef dout_subsys
-#define dout_subsys ceph_subsys_rbd_pwl
-#undef dout_prefix
-#define dout_prefix *_dout << "librbd::cache::WriteLogCache: " << this << " " \
- << __func__ << ": "
-
-namespace librbd {
-namespace cache {
-
-using namespace librbd::cache::pwl;
-
-template <typename I>
-WriteLogCache<I>::WriteLogCache(I &image_ctx, librbd::cache::pwl::ImageCacheState<I>* cache_state) {
- m_write_log = new librbd::cache::pwl::ReplicatedWriteLog<I>(image_ctx, cache_state);
-}
-
-template <typename I>
-WriteLogCache<I>::~WriteLogCache() {
- delete m_write_log;
-}
-
-template <typename I>
-void WriteLogCache<I>::aio_read(Extents&& image_extents,
- ceph::bufferlist* bl,
- int fadvise_flags,
- Context *on_finish) {
- m_write_log->read(std::move(image_extents), std::move(bl), fadvise_flags,
- on_finish);
-}
-
-template <typename I>
-void WriteLogCache<I>::aio_write(Extents &&image_extents,
- bufferlist&& bl,
- int fadvise_flags,
- Context *on_finish) {
- m_write_log->write(std::move(image_extents), std::move(bl),
- fadvise_flags, on_finish);
-}
-
-template <typename I>
-void WriteLogCache<I>::aio_discard(uint64_t offset, uint64_t length,
- uint32_t discard_granularity_bytes,
- Context *on_finish) {
- m_write_log->discard(offset, length, discard_granularity_bytes, on_finish);
-}
-
-template <typename I>
-void WriteLogCache<I>::aio_flush(io::FlushSource flush_source, Context *on_finish) {
- m_write_log->flush(flush_source, on_finish);
-}
-
-template <typename I>
-void WriteLogCache<I>::aio_writesame(uint64_t offset, uint64_t length,
- bufferlist&& bl, int fadvise_flags,
- Context *on_finish) {
- m_write_log->writesame(offset, length, std::move(bl), fadvise_flags,
- on_finish);
-}
-
-template <typename I>
-void WriteLogCache<I>::aio_compare_and_write(Extents &&image_extents,
- bufferlist&& cmp_bl,
- bufferlist&& bl,
- uint64_t *mismatch_offset,
- int fadvise_flags,
- Context *on_finish) {
- m_write_log->compare_and_write(std::move(image_extents), std::move(cmp_bl),
- std::move(bl), mismatch_offset, fadvise_flags,
- on_finish);
-}
-
-template <typename I>
-void WriteLogCache<I>::init(Context *on_finish) {
- m_write_log->init(on_finish);
-}
-
-template <typename I>
-void WriteLogCache<I>::shut_down(Context *on_finish) {
- m_write_log->shut_down(on_finish);
-}
-
-template <typename I>
-void WriteLogCache<I>::invalidate(Context *on_finish) {
- m_write_log->invalidate(on_finish);
-}
-
-template <typename I>
-void WriteLogCache<I>::flush(Context *on_finish) {
- m_write_log->flush(on_finish);
-}
-
-} // namespace cache
-} // namespace librbd
-
-template class librbd::cache::WriteLogCache<librbd::ImageCtx>;
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-
-#ifndef CEPH_LIBRBD_CACHE_WRITE_LOG_CACHE
-#define CEPH_LIBRBD_CACHE_WRITE_LOG_CACHE
-
-#include "librbd/cache/ImageCache.h"
-#include "librbd/io/Types.h"
-
-namespace librbd {
-
-struct ImageCtx;
-
-namespace cache {
-
-namespace pwl {
-template <typename> class AbstractWriteLog;
-template <typename> class ImageCacheState;
-}
-
-template <typename ImageCtxT>
-class WriteLogCache {
-public:
- typedef io::Extent Extent;
- typedef io::Extents Extents;
-
- WriteLogCache(ImageCtxT &image_ctx, librbd::cache::pwl::ImageCacheState<ImageCtxT>* cache_state);
- ~WriteLogCache();
- WriteLogCache(const WriteLogCache&) = delete;
- WriteLogCache &operator=(const WriteLogCache&) = delete;
-
- /// client AIO methods
- void aio_read(Extents&& image_extents, ceph::bufferlist *bl,
- int fadvise_flags, Context *on_finish) ;
- void aio_write(Extents&& image_extents, ceph::bufferlist&& bl,
- int fadvise_flags, Context *on_finish) ;
- void aio_discard(uint64_t offset, uint64_t length,
- uint32_t discard_granularity_bytes,
- Context *on_finish) ;
- void aio_flush(io::FlushSource flush_source, Context *on_finish) ;
- void aio_writesame(uint64_t offset, uint64_t length,
- ceph::bufferlist&& bl,
- int fadvise_flags, Context *on_finish) ;
- void aio_compare_and_write(Extents&& image_extents,
- ceph::bufferlist&& cmp_bl, ceph::bufferlist&& bl,
- uint64_t *mismatch_offset,int fadvise_flags,
- Context *on_finish) ;
-
- /// internal state methods
- void init(Context *on_finish) ;
- void shut_down(Context *on_finish) ;
- void invalidate(Context *on_finish) ;
- void flush(Context *on_finish) ;
-
- librbd::cache::pwl::AbstractWriteLog<ImageCtxT> *m_write_log;
-};
-
-} // namespace cache
-} // namespace librbd
-
-extern template class librbd::cache::WriteLogCache<librbd::ImageCtx>;
-
-#endif // CEPH_LIBRBD_CACHE_WRITE_LOG_CACHE
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "common/dout.h"
+#include "librbd/cache/pwl/AbstractWriteLog.h"
+#include "librbd/cache/pwl/ShutdownRequest.h"
+#include "librbd/cache/WriteLogImageDispatch.h"
+#include "librbd/ImageCtx.h"
+#include "librbd/io/AioCompletion.h"
+#include "librbd/io/Utils.h"
+
+#define dout_subsys ceph_subsys_rbd_pwl
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::cache::WriteLogImageDispatch: " << this << " " \
+ << __func__ << ": "
+
+namespace librbd {
+namespace cache {
+
+namespace {
+
+void start_in_flight_io(io::AioCompletion* aio_comp) {
+ if (!aio_comp->async_op.started()) {
+ aio_comp->start_op();
+ }
+}
+
+} // anonymous namespace
+
+template <typename I>
+void WriteLogImageDispatch<I>::shut_down(Context* on_finish) {
+ ceph_assert(m_image_cache != nullptr);
+
+ Context* ctx = new LambdaContext(
+ [this, on_finish](int r) {
+ m_image_cache = nullptr;
+ on_finish->complete(r);
+ });
+
+ cache::pwl::ShutdownRequest<I> *req = cache::pwl::ShutdownRequest<I>::create(
+ *m_image_ctx, m_image_cache, ctx);
+ req->send();
+}
+
+template <typename I>
+bool WriteLogImageDispatch<I>::read(
+ io::AioCompletion* aio_comp, io::Extents &&image_extents,
+ io::ReadResult &&read_result, IOContext io_context,
+ int op_flags, int read_flags,
+ const ZTracer::Trace &parent_trace, uint64_t tid,
+ std::atomic<uint32_t>* image_dispatch_flags,
+ io::DispatchResult* dispatch_result,
+ Context** on_finish, Context* on_dispatched) {
+ auto cct = m_image_ctx->cct;
+ ldout(cct, 20) << "image_extents=" << image_extents << dendl;
+
+ *dispatch_result = io::DISPATCH_RESULT_COMPLETE;
+ if (preprocess_length(aio_comp, image_extents)) {
+ return true;
+ }
+
+ start_in_flight_io(aio_comp);
+
+ aio_comp->set_request_count(1);
+ aio_comp->read_result = std::move(read_result);
+ uint64_t length = io::util::extents_length(image_extents);
+ aio_comp->read_result.set_clip_length(length);
+
+ auto *req_comp = new io::ReadResult::C_ImageReadRequest(
+ aio_comp, image_extents);
+
+ m_image_cache->read(std::move(image_extents),
+ &req_comp->bl, op_flags,
+ req_comp);
+ return true;
+}
+
+template <typename I>
+bool WriteLogImageDispatch<I>::write(
+ io::AioCompletion* aio_comp, io::Extents &&image_extents, bufferlist &&bl,
+ IOContext io_context, int op_flags, const ZTracer::Trace &parent_trace,
+ uint64_t tid, std::atomic<uint32_t>* image_dispatch_flags,
+ io::DispatchResult* dispatch_result,
+ Context** on_finish, Context* on_dispatched) {
+ auto cct = m_image_ctx->cct;
+ ldout(cct, 20) << "image_extents=" << image_extents << dendl;
+
+ *dispatch_result = io::DISPATCH_RESULT_COMPLETE;
+ if (preprocess_length(aio_comp, image_extents)) {
+ return true;
+ }
+
+ start_in_flight_io(aio_comp);
+
+ aio_comp->set_request_count(1);
+ io::C_AioRequest *req_comp = new io::C_AioRequest(aio_comp);
+ m_image_cache->write(std::move(image_extents),
+ std::move(bl), op_flags, req_comp);
+ return true;
+}
+
+template <typename I>
+bool WriteLogImageDispatch<I>::discard(
+ io::AioCompletion* aio_comp, io::Extents &&image_extents,
+ uint32_t discard_granularity_bytes, IOContext io_context,
+ const ZTracer::Trace &parent_trace,
+ uint64_t tid, std::atomic<uint32_t>* image_dispatch_flags,
+ io::DispatchResult* dispatch_result,
+ Context** on_finish, Context* on_dispatched) {
+ auto cct = m_image_ctx->cct;
+ ldout(cct, 20) << "image_extents=" << image_extents << dendl;
+
+ *dispatch_result = io::DISPATCH_RESULT_COMPLETE;
+ if (preprocess_length(aio_comp, image_extents)) {
+ return true;
+ }
+
+ start_in_flight_io(aio_comp);
+
+ aio_comp->set_request_count(image_extents.size());
+ for (auto &extent : image_extents) {
+ io::C_AioRequest *req_comp = new io::C_AioRequest(aio_comp);
+ m_image_cache->discard(extent.first, extent.second,
+ discard_granularity_bytes,
+ req_comp);
+ }
+ return true;
+}
+
+template <typename I>
+bool WriteLogImageDispatch<I>::write_same(
+ io::AioCompletion* aio_comp, io::Extents &&image_extents,
+ bufferlist &&bl, IOContext io_context,
+ int op_flags, const ZTracer::Trace &parent_trace, uint64_t tid,
+ std::atomic<uint32_t>* image_dispatch_flags,
+ io::DispatchResult* dispatch_result,
+ Context** on_finish, Context* on_dispatched) {
+ auto cct = m_image_ctx->cct;
+ ldout(cct, 20) << "image_extents=" << image_extents << dendl;
+
+ *dispatch_result = io::DISPATCH_RESULT_COMPLETE;
+ if (preprocess_length(aio_comp, image_extents)) {
+ return true;
+ }
+
+ start_in_flight_io(aio_comp);
+
+ aio_comp->set_request_count(image_extents.size());
+ for (auto &extent : image_extents) {
+ io::C_AioRequest *req_comp = new io::C_AioRequest(aio_comp);
+ m_image_cache->writesame(extent.first, extent.second,
+ std::move(bl), op_flags,
+ req_comp);
+ }
+ return true;
+}
+
+template <typename I>
+bool WriteLogImageDispatch<I>::compare_and_write(
+ io::AioCompletion* aio_comp, io::Extents &&image_extents, bufferlist &&cmp_bl,
+ bufferlist &&bl, uint64_t *mismatch_offset, IOContext io_context,
+ int op_flags, const ZTracer::Trace &parent_trace, uint64_t tid,
+ std::atomic<uint32_t>* image_dispatch_flags,
+ io::DispatchResult* dispatch_result,
+ Context** on_finish, Context* on_dispatched) {
+ auto cct = m_image_ctx->cct;
+ ldout(cct, 20) << "image_extents=" << image_extents << dendl;
+
+ *dispatch_result = io::DISPATCH_RESULT_COMPLETE;
+ if (preprocess_length(aio_comp, image_extents)) {
+ return true;
+ }
+
+ start_in_flight_io(aio_comp);
+
+ aio_comp->set_request_count(1);
+ io::C_AioRequest *req_comp = new io::C_AioRequest(aio_comp);
+ m_image_cache->compare_and_write(
+ std::move(image_extents), std::move(cmp_bl), std::move(bl),
+ mismatch_offset, op_flags, req_comp);
+ return true;
+}
+
+template <typename I>
+bool WriteLogImageDispatch<I>::flush(
+ io::AioCompletion* aio_comp, io::FlushSource flush_source,
+ const ZTracer::Trace &parent_trace, uint64_t tid,
+ std::atomic<uint32_t>* image_dispatch_flags,
+ io::DispatchResult* dispatch_result,
+ Context** on_finish, Context* on_dispatched) {
+ auto cct = m_image_ctx->cct;
+ ldout(cct, 20) << "tid=" << tid << dendl;
+
+ start_in_flight_io(aio_comp);
+
+ *dispatch_result = io::DISPATCH_RESULT_COMPLETE;
+
+ aio_comp->set_request_count(1);
+ io::C_AioRequest *req_comp = new io::C_AioRequest(aio_comp);
+ m_image_cache->flush(flush_source, req_comp);
+
+ return true;
+}
+
+template <typename I>
+bool WriteLogImageDispatch<I>::list_snaps(
+ io::AioCompletion* aio_comp, io::Extents&& image_extents,
+ io::SnapIds&& snap_ids,
+ int list_snaps_flags, io::SnapshotDelta* snapshot_delta,
+ const ZTracer::Trace &parent_trace, uint64_t tid,
+ std::atomic<uint32_t>* image_dispatch_flags,
+ io::DispatchResult* dispatch_result, Context** on_finish,
+ Context* on_dispatched) {
+ return false;
+}
+
+
+template <typename I>
+bool WriteLogImageDispatch<I>::preprocess_length(
+ io::AioCompletion* aio_comp, io::Extents &image_extents) const {
+ auto total_bytes = io::util::extents_length(image_extents);
+ if (total_bytes == 0) {
+ aio_comp->set_request_count(0);
+ return true;
+ }
+ return false;
+}
+
+} // namespace io
+} // namespace librbd
+
+template class librbd::cache::WriteLogImageDispatch<librbd::ImageCtx>;
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_WRITELOG_IMAGE_DISPATCH_H
+#define CEPH_LIBRBD_WRITELOG_IMAGE_DISPATCH_H
+
+#include "librbd/io/ImageDispatchInterface.h"
+#include "include/int_types.h"
+#include "include/buffer.h"
+#include "common/zipkin_trace.h"
+#include "librbd/io/ReadResult.h"
+#include "librbd/io/Types.h"
+
+struct Context;
+
+namespace librbd {
+
+struct ImageCtx;
+
+namespace cache {
+
+namespace pwl { template <typename> class AbstractWriteLog; }
+
+template <typename ImageCtxT>
+class WriteLogImageDispatch : public io::ImageDispatchInterface {
+public:
+ WriteLogImageDispatch(ImageCtxT* image_ctx,
+ pwl::AbstractWriteLog<ImageCtx> *image_cache) :
+ m_image_ctx(image_ctx), m_image_cache(image_cache) {
+ }
+
+ io::ImageDispatchLayer get_dispatch_layer() const override {
+ return io::IMAGE_DISPATCH_LAYER_WRITEBACK_CACHE;
+ }
+
+ void shut_down(Context* on_finish) override;
+
+ bool read(
+ io::AioCompletion* aio_comp, io::Extents &&image_extents,
+ io::ReadResult &&read_result, IOContext io_context,
+ int op_flags, int read_flags,
+ const ZTracer::Trace &parent_trace, uint64_t tid,
+ std::atomic<uint32_t>* image_dispatch_flags,
+ io::DispatchResult* dispatch_result, Context** on_finish,
+ Context* on_dispatched) override;
+ bool write(
+ io::AioCompletion* aio_comp, io::Extents &&image_extents, bufferlist &&bl,
+ IOContext io_context, int op_flags, const ZTracer::Trace &parent_trace,
+ uint64_t tid, std::atomic<uint32_t>* image_dispatch_flags,
+ io::DispatchResult* dispatch_result,
+ Context** on_finish, Context* on_dispatched) override;
+ bool discard(
+ io::AioCompletion* aio_comp, io::Extents &&image_extents,
+ uint32_t discard_granularity_bytes, IOContext io_context,
+ const ZTracer::Trace &parent_trace, uint64_t tid,
+ std::atomic<uint32_t>* image_dispatch_flags,
+ io::DispatchResult* dispatch_result,
+ Context** on_finish, Context* on_dispatched) override;
+ bool write_same(
+ io::AioCompletion* aio_comp, io::Extents &&image_extents, bufferlist &&bl,
+ IOContext io_context, int op_flags, const ZTracer::Trace &parent_trace,
+ uint64_t tid, std::atomic<uint32_t>* image_dispatch_flags,
+ io::DispatchResult* dispatch_result,
+ Context** on_finish, Context* on_dispatched) override;
+ bool compare_and_write(
+ io::AioCompletion* aio_comp, io::Extents &&image_extents,
+ bufferlist &&cmp_bl,
+ bufferlist &&bl, uint64_t *mismatch_offset,
+ IOContext io_context, int op_flags,
+ const ZTracer::Trace &parent_trace, uint64_t tid,
+ std::atomic<uint32_t>* image_dispatch_flags,
+ io::DispatchResult* dispatch_result,
+ Context** on_finish, Context* on_dispatched) override;
+ bool flush(
+ io::AioCompletion* aio_comp, io::FlushSource flush_source,
+ const ZTracer::Trace &parent_trace, uint64_t tid,
+ std::atomic<uint32_t>* image_dispatch_flags,
+ io::DispatchResult* dispatch_result,
+ Context** on_finish, Context* on_dispatched) override;
+ bool list_snaps(
+ io::AioCompletion* aio_comp, io::Extents&& image_extents,
+ io::SnapIds&& snap_ids, int list_snaps_flags,
+ io::SnapshotDelta* snapshot_delta,
+ const ZTracer::Trace &parent_trace, uint64_t tid,
+ std::atomic<uint32_t>* image_dispatch_flags,
+ io::DispatchResult* dispatch_result, Context** on_finish,
+ Context* on_dispatched) override;
+
+private:
+ ImageCtxT* m_image_ctx;
+ pwl::AbstractWriteLog<ImageCtx> *m_image_cache;
+
+ bool preprocess_length(
+ io::AioCompletion* aio_comp, io::Extents &image_extents) const;
+};
+
+} // namespace cache
+} // namespace librbd
+
+extern template class librbd::cache::WriteLogImageDispatch<librbd::ImageCtx>;
+
+#endif // CEPH_LIBRBD_WRITELOG_IMAGE_DISPATCH_H
#include "common/RWLock.h"
#include "common/WorkQueue.h"
#include "common/AsyncOpTracker.h"
-#include "librbd/cache/ImageCache.h"
#include "librbd/cache/ImageWriteback.h"
#include "librbd/Utils.h"
#include "librbd/BlockGuard.h"
#include "librbd/cache/pwl/LogOperation.h"
#include "librbd/cache/pwl/Request.h"
#include "librbd/cache/pwl/LogMap.h"
-#include "librbd/cache/pwl/Types.h"
#include <functional>
#include <list>
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-
-#include "common/dout.h"
-#include "librbd/cache/pwl/ImageDispatch.h"
-#include "librbd/cache/pwl/ShutdownRequest.h"
-#include "librbd/cache/WriteLogCache.h"
-#include "librbd/ImageCtx.h"
-#include "librbd/io/AioCompletion.h"
-#include "librbd/io/Utils.h"
-
-#define dout_subsys ceph_subsys_rbd_pwl
-#undef dout_prefix
-#define dout_prefix *_dout << "librbd::cache::pwl::ImageDispatch: " << this << " " \
- << __func__ << ": "
-
-namespace librbd {
-namespace cache {
-namespace pwl{
-
-namespace {
-
-void start_in_flight_io(io::AioCompletion* aio_comp) {
- if (!aio_comp->async_op.started()) {
- aio_comp->start_op();
- }
-}
-
-} // anonymous namespace
-
-template <typename I>
-void ImageDispatch<I>::shut_down(Context* on_finish) {
- ceph_assert(m_image_cache != nullptr);
-
- Context* ctx = new LambdaContext(
- [this, on_finish](int r) {
- m_image_cache = nullptr;
- on_finish->complete(r);
- });
-
- cache::pwl::ShutdownRequest<I> *req = cache::pwl::ShutdownRequest<I>::create(
- *m_image_ctx, m_image_cache, ctx);
- req->send();
-}
-
-template <typename I>
-bool ImageDispatch<I>::read(
- io::AioCompletion* aio_comp, io::Extents &&image_extents,
- io::ReadResult &&read_result, IOContext io_context,
- int op_flags, int read_flags,
- const ZTracer::Trace &parent_trace, uint64_t tid,
- std::atomic<uint32_t>* image_dispatch_flags,
- io::DispatchResult* dispatch_result,
- Context** on_finish, Context* on_dispatched) {
- auto cct = m_image_ctx->cct;
- ldout(cct, 20) << "image_extents=" << image_extents << dendl;
-
- *dispatch_result = io::DISPATCH_RESULT_COMPLETE;
- if (preprocess_length(aio_comp, image_extents)) {
- return true;
- }
-
- start_in_flight_io(aio_comp);
-
- aio_comp->set_request_count(1);
- aio_comp->read_result = std::move(read_result);
- uint64_t length = io::util::extents_length(image_extents);
- aio_comp->read_result.set_clip_length(length);
-
- auto *req_comp = new io::ReadResult::C_ImageReadRequest(
- aio_comp, image_extents);
-
- m_image_cache->aio_read(std::move(image_extents),
- &req_comp->bl, op_flags,
- req_comp);
- return true;
-}
-
-template <typename I>
-bool ImageDispatch<I>::write(
- io::AioCompletion* aio_comp, io::Extents &&image_extents, bufferlist &&bl,
- IOContext io_context, int op_flags, const ZTracer::Trace &parent_trace,
- uint64_t tid, std::atomic<uint32_t>* image_dispatch_flags,
- io::DispatchResult* dispatch_result,
- Context** on_finish, Context* on_dispatched) {
- auto cct = m_image_ctx->cct;
- ldout(cct, 20) << "image_extents=" << image_extents << dendl;
-
- *dispatch_result = io::DISPATCH_RESULT_COMPLETE;
- if (preprocess_length(aio_comp, image_extents)) {
- return true;
- }
-
- start_in_flight_io(aio_comp);
-
- aio_comp->set_request_count(1);
- io::C_AioRequest *req_comp = new io::C_AioRequest(aio_comp);
- m_image_cache->aio_write(std::move(image_extents),
- std::move(bl), op_flags, req_comp);
- return true;
-}
-
-template <typename I>
-bool ImageDispatch<I>::discard(
- io::AioCompletion* aio_comp, io::Extents &&image_extents,
- uint32_t discard_granularity_bytes, IOContext io_context,
- const ZTracer::Trace &parent_trace,
- uint64_t tid, std::atomic<uint32_t>* image_dispatch_flags,
- io::DispatchResult* dispatch_result,
- Context** on_finish, Context* on_dispatched) {
- auto cct = m_image_ctx->cct;
- ldout(cct, 20) << "image_extents=" << image_extents << dendl;
-
- *dispatch_result = io::DISPATCH_RESULT_COMPLETE;
- if (preprocess_length(aio_comp, image_extents)) {
- return true;
- }
-
- start_in_flight_io(aio_comp);
-
- aio_comp->set_request_count(image_extents.size());
- for (auto &extent : image_extents) {
- io::C_AioRequest *req_comp = new io::C_AioRequest(aio_comp);
- m_image_cache->aio_discard(extent.first, extent.second,
- discard_granularity_bytes,
- req_comp);
- }
- return true;
-}
-
-template <typename I>
-bool ImageDispatch<I>::write_same(
- io::AioCompletion* aio_comp, io::Extents &&image_extents,
- bufferlist &&bl, IOContext io_context,
- int op_flags, const ZTracer::Trace &parent_trace, uint64_t tid,
- std::atomic<uint32_t>* image_dispatch_flags,
- io::DispatchResult* dispatch_result,
- Context** on_finish, Context* on_dispatched) {
- auto cct = m_image_ctx->cct;
- ldout(cct, 20) << "image_extents=" << image_extents << dendl;
-
- *dispatch_result = io::DISPATCH_RESULT_COMPLETE;
- if (preprocess_length(aio_comp, image_extents)) {
- return true;
- }
-
- start_in_flight_io(aio_comp);
-
- aio_comp->set_request_count(image_extents.size());
- for (auto &extent : image_extents) {
- io::C_AioRequest *req_comp = new io::C_AioRequest(aio_comp);
- m_image_cache->aio_writesame(extent.first, extent.second,
- std::move(bl), op_flags,
- req_comp);
- }
- return true;
-}
-
-template <typename I>
-bool ImageDispatch<I>::compare_and_write(
- io::AioCompletion* aio_comp, io::Extents &&image_extents, bufferlist &&cmp_bl,
- bufferlist &&bl, uint64_t *mismatch_offset, IOContext io_context,
- int op_flags, const ZTracer::Trace &parent_trace, uint64_t tid,
- std::atomic<uint32_t>* image_dispatch_flags,
- io::DispatchResult* dispatch_result,
- Context** on_finish, Context* on_dispatched) {
- auto cct = m_image_ctx->cct;
- ldout(cct, 20) << "image_extents=" << image_extents << dendl;
-
- *dispatch_result = io::DISPATCH_RESULT_COMPLETE;
- if (preprocess_length(aio_comp, image_extents)) {
- return true;
- }
-
- start_in_flight_io(aio_comp);
-
- aio_comp->set_request_count(1);
- io::C_AioRequest *req_comp = new io::C_AioRequest(aio_comp);
- m_image_cache->aio_compare_and_write(
- std::move(image_extents), std::move(cmp_bl), std::move(bl),
- mismatch_offset, op_flags, req_comp);
- return true;
-}
-
-template <typename I>
-bool ImageDispatch<I>::flush(
- io::AioCompletion* aio_comp, io::FlushSource flush_source,
- const ZTracer::Trace &parent_trace, uint64_t tid,
- std::atomic<uint32_t>* image_dispatch_flags,
- io::DispatchResult* dispatch_result,
- Context** on_finish, Context* on_dispatched) {
- auto cct = m_image_ctx->cct;
- ldout(cct, 20) << "tid=" << tid << dendl;
-
- start_in_flight_io(aio_comp);
-
- *dispatch_result = io::DISPATCH_RESULT_COMPLETE;
-
- aio_comp->set_request_count(1);
- io::C_AioRequest *req_comp = new io::C_AioRequest(aio_comp);
- m_image_cache->aio_flush(flush_source, req_comp);
-
- return true;
-}
-
-template <typename I>
-bool ImageDispatch<I>::list_snaps(
- io::AioCompletion* aio_comp, io::Extents&& image_extents,
- io::SnapIds&& snap_ids,
- int list_snaps_flags, io::SnapshotDelta* snapshot_delta,
- const ZTracer::Trace &parent_trace, uint64_t tid,
- std::atomic<uint32_t>* image_dispatch_flags,
- io::DispatchResult* dispatch_result, Context** on_finish,
- Context* on_dispatched) {
- ceph_abort();
-}
-
-
-template <typename I>
-bool ImageDispatch<I>::preprocess_length(
- io::AioCompletion* aio_comp, io::Extents &image_extents) const {
- auto total_bytes = io::util::extents_length(image_extents);
- if (total_bytes == 0) {
- aio_comp->set_request_count(0);
- return true;
- }
- return false;
-}
-
-} // namespace pwl
-} // namespace io
-} // namespace librbd
-
-template class librbd::cache::pwl::ImageDispatch<librbd::ImageCtx>;
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-
-#ifndef CEPH_LIBRBD_PWL_IMAGE_DISPATCH_H
-#define CEPH_LIBRBD_PWL_IMAGE_DISPATCH_H
-
-#include "librbd/io/ImageDispatchInterface.h"
-#include "include/int_types.h"
-#include "include/buffer.h"
-#include "common/zipkin_trace.h"
-#include "librbd/io/ReadResult.h"
-#include "librbd/io/Types.h"
-
-struct Context;
-
-namespace librbd {
-
-struct ImageCtx;
-
-namespace cache {
-
-template<typename>
-class WriteLogCache;
-
-namespace pwl {
-
-template <typename ImageCtxT>
-class ImageDispatch : public io::ImageDispatchInterface {
-public:
- ImageDispatch(ImageCtxT* image_ctx,
- WriteLogCache<ImageCtx> *image_cache) :
- m_image_ctx(image_ctx), m_image_cache(image_cache) {
- }
-
- io::ImageDispatchLayer get_dispatch_layer() const override {
- return io::IMAGE_DISPATCH_LAYER_WRITEBACK_CACHE;
- }
-
- void shut_down(Context* on_finish) override;
-
- bool read(
- io::AioCompletion* aio_comp, io::Extents &&image_extents,
- io::ReadResult &&read_result, IOContext io_context,
- int op_flags, int read_flags,
- const ZTracer::Trace &parent_trace, uint64_t tid,
- std::atomic<uint32_t>* image_dispatch_flags,
- io::DispatchResult* dispatch_result, Context** on_finish,
- Context* on_dispatched) override;
- bool write(
- io::AioCompletion* aio_comp, io::Extents &&image_extents, bufferlist &&bl,
- IOContext io_context, int op_flags, const ZTracer::Trace &parent_trace,
- uint64_t tid, std::atomic<uint32_t>* image_dispatch_flags,
- io::DispatchResult* dispatch_result,
- Context** on_finish, Context* on_dispatched) override;
- bool discard(
- io::AioCompletion* aio_comp, io::Extents &&image_extents,
- uint32_t discard_granularity_bytes, IOContext io_context,
- const ZTracer::Trace &parent_trace, uint64_t tid,
- std::atomic<uint32_t>* image_dispatch_flags,
- io::DispatchResult* dispatch_result,
- Context** on_finish, Context* on_dispatched) override;
- bool write_same(
- io::AioCompletion* aio_comp, io::Extents &&image_extents, bufferlist &&bl,
- IOContext io_context, int op_flags, const ZTracer::Trace &parent_trace,
- uint64_t tid, std::atomic<uint32_t>* image_dispatch_flags,
- io::DispatchResult* dispatch_result,
- Context** on_finish, Context* on_dispatched) override;
- bool compare_and_write(
- io::AioCompletion* aio_comp, io::Extents &&image_extents,
- bufferlist &&cmp_bl,
- bufferlist &&bl, uint64_t *mismatch_offset,
- IOContext io_context, int op_flags,
- const ZTracer::Trace &parent_trace, uint64_t tid,
- std::atomic<uint32_t>* image_dispatch_flags,
- io::DispatchResult* dispatch_result,
- Context** on_finish, Context* on_dispatched) override;
- bool flush(
- io::AioCompletion* aio_comp, io::FlushSource flush_source,
- const ZTracer::Trace &parent_trace, uint64_t tid,
- std::atomic<uint32_t>* image_dispatch_flags,
- io::DispatchResult* dispatch_result,
- Context** on_finish, Context* on_dispatched) override;
- bool list_snaps(
- io::AioCompletion* aio_comp, io::Extents&& image_extents,
- io::SnapIds&& snap_ids, int list_snaps_flags,
- io::SnapshotDelta* snapshot_delta,
- const ZTracer::Trace &parent_trace, uint64_t tid,
- std::atomic<uint32_t>* image_dispatch_flags,
- io::DispatchResult* dispatch_result, Context** on_finish,
- Context* on_dispatched) override;
-
-private:
- ImageCtxT* m_image_ctx;
- cache::WriteLogCache<ImageCtx> *m_image_cache;
-
- bool preprocess_length(
- io::AioCompletion* aio_comp, io::Extents &image_extents) const;
-};
-
-} // namespace pwl
-} // namespace cache
-} // namespace librbd
-
-extern template class librbd::cache::pwl::ImageDispatch<librbd::ImageCtx>;
-
-#endif // CEPH_LIBRBD_PWL_IMAGE_DISPATCH_H
#if defined(WITH_RBD_RWL)
#include "librbd/cache/pwl/ImageCacheState.h"
-#include "librbd/cache/pwl/ImageDispatch.h"
-#include "librbd/cache/WriteLogCache.h"
+#include "librbd/cache/pwl/ReplicatedWriteLog.h"
+#include "librbd/cache/WriteLogImageDispatch.h"
#endif // WITH_RBD_RWL
#include "librbd/cache/Utils.h"
switch(cache_type) {
case cache::IMAGE_CACHE_TYPE_RWL:
m_image_cache =
- new librbd::cache::WriteLogCache<I>(m_image_ctx,
- cache_state);
+ new librbd::cache::pwl::ReplicatedWriteLog<I>(m_image_ctx,
+ cache_state);
break;
default:
delete cache_state;
}
// Register RWL dispatch
- auto image_dispatch = new cache::pwl::ImageDispatch<I>(&m_image_ctx, m_image_cache);
+ auto image_dispatch = new cache::WriteLogImageDispatch<I>(&m_image_ctx, m_image_cache);
m_image_ctx.io_image_dispatcher->register_dispatch(image_dispatch);
namespace cache {
-template<typename>
-class WriteLogCache;
-
namespace pwl {
+template<typename>
+class AbstractWriteLog;
+
template<typename>
class ImageCacheState;
InitRequest(ImageCtxT &image_ctx, Context *on_finish);
ImageCtxT &m_image_ctx;
- cache::WriteLogCache<ImageCtx> *m_image_cache;
+ AbstractWriteLog<ImageCtxT> *m_image_cache;
Context *m_on_finish;
int m_error_result;
// vim: ts=8 sw=2 smarttab
#include "librbd/cache/pwl/ShutdownRequest.h"
-#include "librbd/cache/WriteLogCache.h"
#include "librbd/ImageCtx.h"
#include "librbd/Utils.h"
#include "common/dout.h"
#include "librbd/asio/ContextWQ.h"
#include "librbd/cache/Types.h"
+#if defined(WITH_RBD_RWL)
+#include "librbd/cache/pwl/AbstractWriteLog.h"
+#endif // WITH_RBD_RWL
#define dout_subsys ceph_subsys_rbd_pwl
#undef dout_prefix
template <typename I>
ShutdownRequest<I>* ShutdownRequest<I>::create(
I &image_ctx,
- cache::WriteLogCache<ImageCtx> *image_cache,
+ AbstractWriteLog<I> *image_cache,
Context *on_finish) {
return new ShutdownRequest(image_ctx, image_cache, on_finish);
}
template <typename I>
ShutdownRequest<I>::ShutdownRequest(
I &image_ctx,
- cache::WriteLogCache<ImageCtx> *image_cache,
+ AbstractWriteLog<I> *image_cache,
Context *on_finish)
: m_image_ctx(image_ctx),
m_image_cache(image_cache),
template <typename I>
void ShutdownRequest<I>::send() {
+#if defined(WITH_RBD_RWL)
send_shutdown_image_cache();
+#else
+ finish();
+#endif // WITH_RBD_RWL
}
+#if defined(WITH_RBD_RWL)
template <typename I>
void ShutdownRequest<I>::send_shutdown_image_cache() {
CephContext *cct = m_image_ctx.cct;
finish();
}
+#endif // WITH_RBD_RWL
+
template <typename I>
void ShutdownRequest<I>::finish() {
m_on_finish->complete(m_error_result);
namespace cache {
-template<typename>
-class WriteLogCache;
-
namespace pwl {
+template<typename>
+class AbstractWriteLog;
+
template<typename>
class ImageCacheState;
public:
static ShutdownRequest* create(
ImageCtxT &image_ctx,
- cache::WriteLogCache<ImageCtx> *image_cache,
+ AbstractWriteLog<ImageCtxT> *image_cache,
Context *on_finish);
void send();
*/
ShutdownRequest(ImageCtxT &image_ctx,
- cache::WriteLogCache<ImageCtx> *image_cache,
+ AbstractWriteLog<ImageCtxT> *image_cache,
Context *on_finish);
ImageCtxT &m_image_ctx;
- cache::WriteLogCache<ImageCtx> *m_image_cache;
+ AbstractWriteLog<ImageCtxT> *m_image_cache;
Context *m_on_finish;
int m_error_result;
#include "common/AsyncOpTracker.h"
#include "common/dout.h"
#include "common/errno.h"
-#include "librbd/cache/pwl/ShutdownRequest.h"
#include "librbd/ExclusiveLock.h"
#include "librbd/ImageState.h"
#include "librbd/ImageWatcher.h"
#include "test/librbd/test_support.h"
#include "test/librbd/mock/MockImageCtx.h"
#include "include/rbd/librbd.hpp"
+#include "librbd/cache/pwl/AbstractWriteLog.h"
#include "librbd/cache/pwl/ImageCacheState.h"
#include "librbd/cache/pwl/Types.h"
#include "librbd/cache/ImageWriteback.h"
-#include "librbd/cache/WriteLogCache.h"
namespace librbd {
} // namespace util
} // namespace librbd
-#include "librbd/cache/WriteLogCache.cc"
#include "librbd/cache/pwl/AbstractWriteLog.cc"
#include "librbd/cache/pwl/ReplicatedWriteLog.cc"
namespace librbd {
namespace cache {
+namespace pwl {
using ::testing::_;
using ::testing::DoDefault;
using ::testing::Invoke;
struct TestMockCacheReplicatedWriteLog : public TestMockFixture {
- typedef WriteLogCache<librbd::MockImageCtx> MockReplicatedWriteLog;
+ typedef librbd::cache::pwl::ReplicatedWriteLog<librbd::MockImageCtx> MockReplicatedWriteLog;
typedef librbd::cache::pwl::ImageCacheState<librbd::MockImageCtx> MockImageCacheStateRWL;
MockImageCacheStateRWL *get_cache_state(MockImageCtx& mock_image_ctx) {
ASSERT_EQ(0, finish_ctx2.wait());
}
-TEST_F(TestMockCacheReplicatedWriteLog, aio_write) {
+TEST_F(TestMockCacheReplicatedWriteLog, write) {
librbd::ImageCtx *ictx;
ASSERT_EQ(0, open_image(m_image_name, &ictx));
bufferlist bl;
bl.append(std::string(4096, '1'));
int fadvise_flags = 0;
- rwl.aio_write(std::move(image_extents), std::move(bl), fadvise_flags, &finish_ctx2);
+ rwl.write(std::move(image_extents), std::move(bl), fadvise_flags, &finish_ctx2);
ASSERT_EQ(0, finish_ctx2.wait());
MockContextRWL finish_ctx3;
bl.append(std::string(4096, '1'));
bufferlist bl_copy = bl;
int fadvise_flags = 0;
- rwl.aio_write(std::move(image_extents), std::move(bl), fadvise_flags, &finish_ctx2);
+ rwl.write(std::move(image_extents), std::move(bl), fadvise_flags, &finish_ctx2);
ASSERT_EQ(0, finish_ctx2.wait());
MockContextRWL finish_ctx_flush;
ASSERT_EQ(0, finish_ctx3.wait());
}
-TEST_F(TestMockCacheReplicatedWriteLog, aio_flush_source_shutdown) {
+TEST_F(TestMockCacheReplicatedWriteLog, flush_source_shutdown) {
librbd::ImageCtx *ictx;
ASSERT_EQ(0, open_image(m_image_name, &ictx));
bufferlist bl;
bl.append(std::string(4096, '1'));
int fadvise_flags = 0;
- rwl.aio_write(std::move(image_extents), std::move(bl), fadvise_flags, &finish_ctx2);
+ rwl.write(std::move(image_extents), std::move(bl), fadvise_flags, &finish_ctx2);
ASSERT_EQ(0, finish_ctx2.wait());
MockContextRWL finish_ctx_flush;
expect_context_complete(finish_ctx_flush, 0);
- rwl.aio_flush(io::FLUSH_SOURCE_SHUTDOWN, &finish_ctx_flush);
+ rwl.flush(io::FLUSH_SOURCE_SHUTDOWN, &finish_ctx_flush);
ASSERT_EQ(0, finish_ctx_flush.wait());
MockContextRWL finish_ctx3;
ASSERT_EQ(0, finish_ctx3.wait());
}
-TEST_F(TestMockCacheReplicatedWriteLog, aio_flush_source_internal) {
+TEST_F(TestMockCacheReplicatedWriteLog, flush_source_internal) {
librbd::ImageCtx *ictx;
ASSERT_EQ(0, open_image(m_image_name, &ictx));
bufferlist bl;
bl.append(std::string(4096, '1'));
int fadvise_flags = 0;
- rwl.aio_write(std::move(image_extents), std::move(bl), fadvise_flags, &finish_ctx2);
+ rwl.write(std::move(image_extents), std::move(bl), fadvise_flags, &finish_ctx2);
ASSERT_EQ(0, finish_ctx2.wait());
MockContextRWL finish_ctx_flush;
expect_context_complete(finish_ctx_flush, 0);
- rwl.aio_flush(io::FLUSH_SOURCE_INTERNAL, &finish_ctx_flush);
+ rwl.flush(io::FLUSH_SOURCE_INTERNAL, &finish_ctx_flush);
ASSERT_EQ(0, finish_ctx_flush.wait());
MockContextRWL finish_ctx3;
ASSERT_EQ(0, finish_ctx3.wait());
}
-TEST_F(TestMockCacheReplicatedWriteLog, aio_flush_source_user) {
+TEST_F(TestMockCacheReplicatedWriteLog, flush_source_user) {
librbd::ImageCtx *ictx;
ASSERT_EQ(0, open_image(m_image_name, &ictx));
bufferlist bl;
bl.append(std::string(4096, '1'));
int fadvise_flags = 0;
- rwl.aio_write(std::move(image_extents), std::move(bl), fadvise_flags, &finish_ctx2);
+ rwl.write(std::move(image_extents), std::move(bl), fadvise_flags, &finish_ctx2);
ASSERT_EQ(0, finish_ctx2.wait());
usleep(10000);
MockContextRWL finish_ctx_flush;
expect_context_complete(finish_ctx_flush, 0);
- rwl.aio_flush(io::FLUSH_SOURCE_USER, &finish_ctx_flush);
+ rwl.flush(io::FLUSH_SOURCE_USER, &finish_ctx_flush);
ASSERT_EQ(0, finish_ctx_flush.wait());
MockContextRWL finish_ctx3;
ASSERT_EQ(0, finish_ctx3.wait());
}
-TEST_F(TestMockCacheReplicatedWriteLog, aio_read_hit_rwl_cache) {
+TEST_F(TestMockCacheReplicatedWriteLog, read_hit_rwl_cache) {
librbd::ImageCtx *ictx;
ASSERT_EQ(0, open_image(m_image_name, &ictx));
bl.append(std::string(4096, '1'));
bufferlist bl_copy = bl;
int fadvise_flags = 0;
- rwl.aio_write(std::move(image_extents), std::move(bl), fadvise_flags, &finish_ctx2);
+ rwl.write(std::move(image_extents), std::move(bl), fadvise_flags, &finish_ctx2);
ASSERT_EQ(0, finish_ctx2.wait());
MockContextRWL finish_ctx_read;
expect_context_complete(finish_ctx_read, 0);
Extents image_extents_read{{0, 4096}};
bufferlist read_bl;
- rwl.aio_read(std::move(image_extents_read), &read_bl, fadvise_flags, &finish_ctx_read);
+ rwl.read(std::move(image_extents_read), &read_bl, fadvise_flags, &finish_ctx_read);
ASSERT_EQ(0, finish_ctx_read.wait());
ASSERT_EQ(4096, read_bl.length());
ASSERT_TRUE(bl_copy.contents_equal(read_bl));
ASSERT_EQ(0, finish_ctx3.wait());
}
-TEST_F(TestMockCacheReplicatedWriteLog, aio_read_hit_part_rwl_cache) {
+TEST_F(TestMockCacheReplicatedWriteLog, read_hit_part_rwl_cache) {
librbd::ImageCtx *ictx;
ASSERT_EQ(0, open_image(m_image_name, &ictx));
bl.append(std::string(4096, '1'));
bufferlist bl_copy = bl;
int fadvise_flags = 0;
- rwl.aio_write(std::move(image_extents), std::move(bl), fadvise_flags, &finish_ctx2);
+ rwl.write(std::move(image_extents), std::move(bl), fadvise_flags, &finish_ctx2);
ASSERT_EQ(0, finish_ctx2.wait());
MockContextRWL finish_ctx_read;
bl_copy.begin(511).copy(4096-512, hit_bl);
expect_context_complete(finish_ctx_read, 512);
bufferlist read_bl;
- rwl.aio_read(std::move(image_extents_read), &read_bl, fadvise_flags, &finish_ctx_read);
+ rwl.read(std::move(image_extents_read), &read_bl, fadvise_flags, &finish_ctx_read);
ASSERT_EQ(512, finish_ctx_read.wait());
ASSERT_EQ(4096, read_bl.length());
bufferlist read_bl_hit;
ASSERT_EQ(0, finish_ctx3.wait());
}
-TEST_F(TestMockCacheReplicatedWriteLog, aio_read_miss_rwl_cache) {
+TEST_F(TestMockCacheReplicatedWriteLog, read_miss_rwl_cache) {
librbd::ImageCtx *ictx;
ASSERT_EQ(0, open_image(m_image_name, &ictx));
bufferlist bl;
bl.append(std::string(4096, '1'));
int fadvise_flags = 0;
- rwl.aio_write(std::move(image_extents), std::move(bl), fadvise_flags, &finish_ctx2);
+ rwl.write(std::move(image_extents), std::move(bl), fadvise_flags, &finish_ctx2);
ASSERT_EQ(0, finish_ctx2.wait());
MockContextRWL finish_ctx_read;
expect_context_complete(finish_ctx_read, 4096);
bufferlist read_bl;
ASSERT_EQ(0, read_bl.length());
- rwl.aio_read(std::move(image_extents_read), &read_bl, fadvise_flags, &finish_ctx_read);
+ rwl.read(std::move(image_extents_read), &read_bl, fadvise_flags, &finish_ctx_read);
ASSERT_EQ(4096, finish_ctx_read.wait());
ASSERT_EQ(4096, read_bl.length());
ASSERT_EQ(0, finish_ctx3.wait());
}
-TEST_F(TestMockCacheReplicatedWriteLog, aio_discard) {
+TEST_F(TestMockCacheReplicatedWriteLog, discard) {
librbd::ImageCtx *ictx;
ASSERT_EQ(0, open_image(m_image_name, &ictx));
bl.append(std::string(4096, '1'));
bufferlist bl_copy = bl;
int fadvise_flags = 0;
- rwl.aio_write(std::move(image_extents), std::move(bl), fadvise_flags, &finish_ctx2);
+ rwl.write(std::move(image_extents), std::move(bl), fadvise_flags, &finish_ctx2);
ASSERT_EQ(0, finish_ctx2.wait());
MockContextRWL finish_ctx_discard;
expect_context_complete(finish_ctx_discard, 0);
- rwl.aio_discard(0, 4096, 1, &finish_ctx_discard);
+ rwl.discard(0, 4096, 1, &finish_ctx_discard);
ASSERT_EQ(0, finish_ctx_discard.wait());
MockContextRWL finish_ctx_read;
bufferlist read_bl;
expect_context_complete(finish_ctx_read, 0);
- rwl.aio_read({{0, 4096}}, &read_bl, fadvise_flags, &finish_ctx_read);
+ rwl.read({{0, 4096}}, &read_bl, fadvise_flags, &finish_ctx_read);
ASSERT_EQ(0, finish_ctx_read.wait());
ASSERT_EQ(4096, read_bl.length());
ASSERT_TRUE(read_bl.is_zero());
ASSERT_EQ(0, finish_ctx3.wait());
}
-TEST_F(TestMockCacheReplicatedWriteLog, aio_writesame) {
+TEST_F(TestMockCacheReplicatedWriteLog, writesame) {
librbd::ImageCtx *ictx;
ASSERT_EQ(0, open_image(m_image_name, &ictx));
bl.append(std::string(512, '1'));
test_bl.append(std::string(4096, '1'));
int fadvise_flags = 0;
- rwl.aio_writesame(0, 4096, std::move(bl), fadvise_flags, &finish_ctx2);
+ rwl.writesame(0, 4096, std::move(bl), fadvise_flags, &finish_ctx2);
ASSERT_EQ(0, finish_ctx2.wait());
MockContextRWL finish_ctx_read;
bufferlist read_bl;
expect_context_complete(finish_ctx_read, 0);
- rwl.aio_read({{0, 4096}}, &read_bl, fadvise_flags, &finish_ctx_read);
+ rwl.read({{0, 4096}}, &read_bl, fadvise_flags, &finish_ctx_read);
ASSERT_EQ(0, finish_ctx_read.wait());
ASSERT_EQ(4096, read_bl.length());
ASSERT_TRUE(test_bl.contents_equal(read_bl));
bl.append(std::string(4096, '1'));
bufferlist bl_copy = bl;
int fadvise_flags = 0;
- rwl.aio_write(std::move(image_extents), std::move(bl), fadvise_flags, &finish_ctx2);
+ rwl.write(std::move(image_extents), std::move(bl), fadvise_flags, &finish_ctx2);
ASSERT_EQ(0, finish_ctx2.wait());
MockContextRWL finish_ctx_invalidate;
ASSERT_EQ(0, finish_ctx3.wait());
}
-TEST_F(TestMockCacheReplicatedWriteLog, aio_compare_and_write_compare_matched) {
+TEST_F(TestMockCacheReplicatedWriteLog, compare_and_write_compare_matched) {
librbd::ImageCtx *ictx;
ASSERT_EQ(0, open_image(m_image_name, &ictx));
bl1.append(std::string(4096, '1'));
bufferlist com_bl = bl1;
int fadvise_flags = 0;
- rwl.aio_write(std::move(image_extents), std::move(bl1), fadvise_flags, &finish_ctx2);
+ rwl.write(std::move(image_extents), std::move(bl1), fadvise_flags, &finish_ctx2);
ASSERT_EQ(0, finish_ctx2.wait());
MockContextRWL finish_ctx_cw;
bufferlist bl2_copy = bl2;
uint64_t mismatch_offset = -1;
expect_context_complete(finish_ctx_cw, 0);
- rwl.aio_compare_and_write({{0, 4096}}, std::move(com_bl), std::move(bl2),
+ rwl.compare_and_write({{0, 4096}}, std::move(com_bl), std::move(bl2),
&mismatch_offset, fadvise_flags, &finish_ctx_cw);
ASSERT_EQ(0, finish_ctx_cw.wait());
ASSERT_EQ(0, mismatch_offset);
MockContextRWL finish_ctx_read;
bufferlist read_bl;
expect_context_complete(finish_ctx_read, 0);
- rwl.aio_read({{0, 4096}}, &read_bl, fadvise_flags, &finish_ctx_read);
+ rwl.read({{0, 4096}}, &read_bl, fadvise_flags, &finish_ctx_read);
ASSERT_EQ(0, finish_ctx_read.wait());
ASSERT_EQ(4096, read_bl.length());
ASSERT_TRUE(bl2_copy.contents_equal(read_bl));
ASSERT_EQ(0, finish_ctx3.wait());
}
-TEST_F(TestMockCacheReplicatedWriteLog, aio_compare_and_write_compare_failed) {
+TEST_F(TestMockCacheReplicatedWriteLog, compare_and_write_compare_failed) {
librbd::ImageCtx *ictx;
ASSERT_EQ(0, open_image(m_image_name, &ictx));
bl1.append(std::string(4096, '1'));
bufferlist bl1_copy = bl1;
int fadvise_flags = 0;
- rwl.aio_write(std::move(image_extents), std::move(bl1), fadvise_flags, &finish_ctx2);
+ rwl.write(std::move(image_extents), std::move(bl1), fadvise_flags, &finish_ctx2);
ASSERT_EQ(0, finish_ctx2.wait());
MockContextRWL finish_ctx_cw;
bufferlist com_bl = bl2;
uint64_t mismatch_offset = -1;
expect_context_complete(finish_ctx_cw, -EILSEQ);
- rwl.aio_compare_and_write({{0, 4096}}, std::move(com_bl), std::move(bl2),
+ rwl.compare_and_write({{0, 4096}}, std::move(com_bl), std::move(bl2),
&mismatch_offset, fadvise_flags, &finish_ctx_cw);
ASSERT_EQ(-EILSEQ, finish_ctx_cw.wait());
ASSERT_EQ(0, mismatch_offset);
MockContextRWL finish_ctx_read;
bufferlist read_bl;
expect_context_complete(finish_ctx_read, 0);
- rwl.aio_read({{0, 4096}}, &read_bl, fadvise_flags, &finish_ctx_read);
+ rwl.read({{0, 4096}}, &read_bl, fadvise_flags, &finish_ctx_read);
ASSERT_EQ(0, finish_ctx_read.wait());
ASSERT_EQ(4096, read_bl.length());
ASSERT_TRUE(bl1_copy.contents_equal(read_bl));
ASSERT_EQ(0, finish_ctx3.wait());
}
+} // namespace pwl
} // namespace cache
} // namespace librbd