summaryrefslogtreecommitdiffstats
path: root/src/librbd/cache/pwl
diff options
context:
space:
mode:
Diffstat (limited to 'src/librbd/cache/pwl')
-rw-r--r--src/librbd/cache/pwl/AbstractWriteLog.cc2195
-rw-r--r--src/librbd/cache/pwl/AbstractWriteLog.h410
-rw-r--r--src/librbd/cache/pwl/Builder.h61
-rw-r--r--src/librbd/cache/pwl/DiscardRequest.cc164
-rw-r--r--src/librbd/cache/pwl/DiscardRequest.h90
-rw-r--r--src/librbd/cache/pwl/ImageCacheState.cc194
-rw-r--r--src/librbd/cache/pwl/ImageCacheState.h86
-rw-r--r--src/librbd/cache/pwl/InitRequest.cc226
-rw-r--r--src/librbd/cache/pwl/InitRequest.h105
-rw-r--r--src/librbd/cache/pwl/LogEntry.cc140
-rw-r--r--src/librbd/cache/pwl/LogEntry.h280
-rw-r--r--src/librbd/cache/pwl/LogMap.cc278
-rw-r--r--src/librbd/cache/pwl/LogMap.h81
-rw-r--r--src/librbd/cache/pwl/LogOperation.cc316
-rw-r--r--src/librbd/cache/pwl/LogOperation.h224
-rw-r--r--src/librbd/cache/pwl/ReadRequest.h45
-rw-r--r--src/librbd/cache/pwl/Request.cc561
-rw-r--r--src/librbd/cache/pwl/Request.h374
-rw-r--r--src/librbd/cache/pwl/ShutdownRequest.cc161
-rw-r--r--src/librbd/cache/pwl/ShutdownRequest.h95
-rw-r--r--src/librbd/cache/pwl/SyncPoint.cc109
-rw-r--r--src/librbd/cache/pwl/SyncPoint.h69
-rw-r--r--src/librbd/cache/pwl/Types.cc185
-rw-r--r--src/librbd/cache/pwl/Types.h444
-rw-r--r--src/librbd/cache/pwl/rwl/Builder.h107
-rw-r--r--src/librbd/cache/pwl/rwl/LogEntry.cc105
-rw-r--r--src/librbd/cache/pwl/rwl/LogEntry.h68
-rw-r--r--src/librbd/cache/pwl/rwl/LogOperation.cc39
-rw-r--r--src/librbd/cache/pwl/rwl/LogOperation.h55
-rw-r--r--src/librbd/cache/pwl/rwl/ReadRequest.cc70
-rw-r--r--src/librbd/cache/pwl/rwl/ReadRequest.h34
-rw-r--r--src/librbd/cache/pwl/rwl/Request.cc86
-rw-r--r--src/librbd/cache/pwl/rwl/Request.h90
-rw-r--r--src/librbd/cache/pwl/rwl/WriteLog.cc1014
-rw-r--r--src/librbd/cache/pwl/rwl/WriteLog.h124
-rw-r--r--src/librbd/cache/pwl/ssd/Builder.h108
-rw-r--r--src/librbd/cache/pwl/ssd/LogEntry.cc63
-rw-r--r--src/librbd/cache/pwl/ssd/LogEntry.h75
-rw-r--r--src/librbd/cache/pwl/ssd/LogOperation.cc36
-rw-r--r--src/librbd/cache/pwl/ssd/LogOperation.h35
-rw-r--r--src/librbd/cache/pwl/ssd/ReadRequest.cc92
-rw-r--r--src/librbd/cache/pwl/ssd/ReadRequest.h34
-rw-r--r--src/librbd/cache/pwl/ssd/Request.cc63
-rw-r--r--src/librbd/cache/pwl/ssd/Request.h92
-rw-r--r--src/librbd/cache/pwl/ssd/Types.h51
-rw-r--r--src/librbd/cache/pwl/ssd/WriteLog.cc1158
-rw-r--r--src/librbd/cache/pwl/ssd/WriteLog.h156
47 files changed, 10648 insertions, 0 deletions
diff --git a/src/librbd/cache/pwl/AbstractWriteLog.cc b/src/librbd/cache/pwl/AbstractWriteLog.cc
new file mode 100644
index 000000000..6f017a9c3
--- /dev/null
+++ b/src/librbd/cache/pwl/AbstractWriteLog.cc
@@ -0,0 +1,2195 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "AbstractWriteLog.h"
+#include "include/buffer.h"
+#include "include/Context.h"
+#include "include/ceph_assert.h"
+#include "common/deleter.h"
+#include "common/dout.h"
+#include "common/environment.h"
+#include "common/errno.h"
+#include "common/hostname.h"
+#include "common/WorkQueue.h"
+#include "common/Timer.h"
+#include "common/perf_counters.h"
+#include "librbd/ImageCtx.h"
+#include "librbd/asio/ContextWQ.h"
+#include "librbd/cache/pwl/ImageCacheState.h"
+#include "librbd/cache/pwl/LogEntry.h"
+#include "librbd/plugin/Api.h"
+#include <map>
+#include <vector>
+
+#undef dout_subsys
+#define dout_subsys ceph_subsys_rbd_pwl
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::cache::pwl::AbstractWriteLog: " << this \
+ << " " << __func__ << ": "
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+
+using namespace librbd::cache::pwl;
+
+typedef AbstractWriteLog<ImageCtx>::Extent Extent;
+typedef AbstractWriteLog<ImageCtx>::Extents Extents;
+
+template <typename I>
+AbstractWriteLog<I>::AbstractWriteLog(
+ I &image_ctx, librbd::cache::pwl::ImageCacheState<I>* cache_state,
+ Builder<This> *builder, cache::ImageWritebackInterface& image_writeback,
+ plugin::Api<I>& plugin_api)
+ : m_builder(builder),
+ m_write_log_guard(image_ctx.cct),
+ m_flush_guard(image_ctx.cct),
+ m_flush_guard_lock(ceph::make_mutex(pwl::unique_lock_name(
+ "librbd::cache::pwl::AbstractWriteLog::m_flush_guard_lock", this))),
+ m_deferred_dispatch_lock(ceph::make_mutex(pwl::unique_lock_name(
+ "librbd::cache::pwl::AbstractWriteLog::m_deferred_dispatch_lock", this))),
+ m_blockguard_lock(ceph::make_mutex(pwl::unique_lock_name(
+ "librbd::cache::pwl::AbstractWriteLog::m_blockguard_lock", this))),
+ m_thread_pool(
+ image_ctx.cct, "librbd::cache::pwl::AbstractWriteLog::thread_pool",
+ "tp_pwl", 4, ""),
+ m_cache_state(cache_state),
+ m_image_ctx(image_ctx),
+ m_log_pool_size(DEFAULT_POOL_SIZE),
+ m_image_writeback(image_writeback),
+ m_plugin_api(plugin_api),
+ m_log_retire_lock(ceph::make_mutex(pwl::unique_lock_name(
+ "librbd::cache::pwl::AbstractWriteLog::m_log_retire_lock", this))),
+ m_entry_reader_lock("librbd::cache::pwl::AbstractWriteLog::m_entry_reader_lock"),
+ m_log_append_lock(ceph::make_mutex(pwl::unique_lock_name(
+ "librbd::cache::pwl::AbstractWriteLog::m_log_append_lock", this))),
+ m_lock(ceph::make_mutex(pwl::unique_lock_name(
+ "librbd::cache::pwl::AbstractWriteLog::m_lock", this))),
+ m_blocks_to_log_entries(image_ctx.cct),
+ m_work_queue("librbd::cache::pwl::ReplicatedWriteLog::work_queue",
+ ceph::make_timespan(
+ image_ctx.config.template get_val<uint64_t>(
+ "rbd_op_thread_timeout")),
+ &m_thread_pool)
+{
+ CephContext *cct = m_image_ctx.cct;
+ m_plugin_api.get_image_timer_instance(cct, &m_timer, &m_timer_lock);
+}
+
+template <typename I>
+AbstractWriteLog<I>::~AbstractWriteLog() {
+ ldout(m_image_ctx.cct, 15) << "enter" << dendl;
+ {
+ std::lock_guard timer_locker(*m_timer_lock);
+ std::lock_guard locker(m_lock);
+ m_timer->cancel_event(m_timer_ctx);
+ m_thread_pool.stop();
+ ceph_assert(m_deferred_ios.size() == 0);
+ ceph_assert(m_ops_to_flush.size() == 0);
+ ceph_assert(m_ops_to_append.size() == 0);
+ ceph_assert(m_flush_ops_in_flight == 0);
+
+ delete m_cache_state;
+ m_cache_state = nullptr;
+ }
+ ldout(m_image_ctx.cct, 15) << "exit" << dendl;
+}
+
+template <typename I>
+void AbstractWriteLog<I>::perf_start(std::string name) {
+ PerfCountersBuilder plb(m_image_ctx.cct, name, l_librbd_pwl_first,
+ l_librbd_pwl_last);
+
+ // Latency axis configuration for op histograms, values are in nanoseconds
+ PerfHistogramCommon::axis_config_d op_hist_x_axis_config{
+ "Latency (nsec)",
+ PerfHistogramCommon::SCALE_LOG2, ///< Latency in logarithmic scale
+ 0, ///< Start at 0
+ 5000, ///< Quantization unit is 5usec
+ 16, ///< Ranges into the mS
+ };
+
+ // Syncpoint logentry number x-axis configuration for op histograms
+ PerfHistogramCommon::axis_config_d sp_logentry_number_config{
+ "logentry number",
+ PerfHistogramCommon::SCALE_LINEAR, // log entry number in linear scale
+ 0, // Start at 0
+ 1, // Quantization unit is 1
+ 260, // Up to 260 > (MAX_WRITES_PER_SYNC_POINT)
+ };
+
+ // Syncpoint bytes number y-axis configuration for op histogram
+ PerfHistogramCommon::axis_config_d sp_bytes_number_config{
+ "Number of SyncPoint",
+ PerfHistogramCommon::SCALE_LOG2, // Request size in logarithmic scale
+ 0, // Start at 0
+ 512, // Quantization unit is 512
+ 17, // Writes up to 8M >= MAX_BYTES_PER_SYNC_POINT
+ };
+
+ // Op size axis configuration for op histogram y axis, values are in bytes
+ PerfHistogramCommon::axis_config_d op_hist_y_axis_config{
+ "Request size (bytes)",
+ PerfHistogramCommon::SCALE_LOG2, ///< Request size in logarithmic scale
+ 0, ///< Start at 0
+ 512, ///< Quantization unit is 512 bytes
+ 16, ///< Writes up to >32k
+ };
+
+ // Num items configuration for op histogram y axis, values are in items
+ PerfHistogramCommon::axis_config_d op_hist_y_axis_count_config{
+ "Number of items",
+ PerfHistogramCommon::SCALE_LINEAR, ///< Request size in linear scale
+ 0, ///< Start at 0
+ 1, ///< Quantization unit is 1
+ 32, ///< Writes up to >32k
+ };
+
+ plb.add_u64_counter(l_librbd_pwl_rd_req, "rd", "Reads");
+ plb.add_u64_counter(l_librbd_pwl_rd_bytes, "rd_bytes", "Data size in reads");
+ plb.add_time_avg(l_librbd_pwl_rd_latency, "rd_latency", "Latency of reads");
+
+ plb.add_u64_counter(l_librbd_pwl_rd_hit_req, "hit_rd", "Reads completely hitting RWL");
+ plb.add_u64_counter(l_librbd_pwl_rd_hit_bytes, "rd_hit_bytes", "Bytes read from RWL");
+ plb.add_time_avg(l_librbd_pwl_rd_hit_latency, "hit_rd_latency", "Latency of read hits");
+
+ plb.add_u64_counter(l_librbd_pwl_rd_part_hit_req, "part_hit_rd", "reads partially hitting RWL");
+
+ plb.add_u64_counter_histogram(
+ l_librbd_pwl_syncpoint_hist, "syncpoint_logentry_bytes_histogram",
+ sp_logentry_number_config, sp_bytes_number_config,
+ "Histogram of syncpoint's logentry numbers vs bytes number");
+
+ plb.add_u64_counter(l_librbd_pwl_wr_req, "wr", "Writes");
+ plb.add_u64_counter(l_librbd_pwl_wr_bytes, "wr_bytes", "Data size in writes");
+ plb.add_u64_counter(l_librbd_pwl_wr_req_def, "wr_def", "Writes deferred for resources");
+ plb.add_u64_counter(l_librbd_pwl_wr_req_def_lanes, "wr_def_lanes", "Writes deferred for lanes");
+ plb.add_u64_counter(l_librbd_pwl_wr_req_def_log, "wr_def_log", "Writes deferred for log entries");
+ plb.add_u64_counter(l_librbd_pwl_wr_req_def_buf, "wr_def_buf", "Writes deferred for buffers");
+ plb.add_u64_counter(l_librbd_pwl_wr_req_overlap, "wr_overlap", "Writes overlapping with prior in-progress writes");
+ plb.add_u64_counter(l_librbd_pwl_wr_req_queued, "wr_q_barrier", "Writes queued for prior barriers (aio_flush)");
+
+ plb.add_u64_counter(l_librbd_pwl_log_ops, "log_ops", "Log appends");
+ plb.add_u64_avg(l_librbd_pwl_log_op_bytes, "log_op_bytes", "Average log append bytes");
+
+ plb.add_time_avg(
+ l_librbd_pwl_req_arr_to_all_t, "req_arr_to_all_t",
+ "Average arrival to allocation time (time deferred for overlap)");
+ plb.add_time_avg(
+ l_librbd_pwl_req_arr_to_dis_t, "req_arr_to_dis_t",
+ "Average arrival to dispatch time (includes time deferred for overlaps and allocation)");
+ plb.add_time_avg(
+ l_librbd_pwl_req_all_to_dis_t, "req_all_to_dis_t",
+ "Average allocation to dispatch time (time deferred for log resources)");
+ plb.add_time_avg(
+ l_librbd_pwl_wr_latency, "wr_latency",
+ "Latency of writes (persistent completion)");
+ plb.add_u64_counter_histogram(
+ l_librbd_pwl_wr_latency_hist, "wr_latency_bytes_histogram",
+ op_hist_x_axis_config, op_hist_y_axis_config,
+ "Histogram of write request latency (nanoseconds) vs. bytes written");
+ plb.add_time_avg(
+ l_librbd_pwl_wr_caller_latency, "caller_wr_latency",
+ "Latency of write completion to caller");
+ plb.add_time_avg(
+ l_librbd_pwl_nowait_req_arr_to_all_t, "req_arr_to_all_nw_t",
+ "Average arrival to allocation time (time deferred for overlap)");
+ plb.add_time_avg(
+ l_librbd_pwl_nowait_req_arr_to_dis_t, "req_arr_to_dis_nw_t",
+ "Average arrival to dispatch time (includes time deferred for overlaps and allocation)");
+ plb.add_time_avg(
+ l_librbd_pwl_nowait_req_all_to_dis_t, "req_all_to_dis_nw_t",
+ "Average allocation to dispatch time (time deferred for log resources)");
+ plb.add_time_avg(
+ l_librbd_pwl_nowait_wr_latency, "wr_latency_nw",
+ "Latency of writes (persistent completion) not deferred for free space");
+ plb.add_u64_counter_histogram(
+ l_librbd_pwl_nowait_wr_latency_hist, "wr_latency_nw_bytes_histogram",
+ op_hist_x_axis_config, op_hist_y_axis_config,
+ "Histogram of write request latency (nanoseconds) vs. bytes written for writes not deferred for free space");
+ plb.add_time_avg(
+ l_librbd_pwl_nowait_wr_caller_latency, "caller_wr_latency_nw",
+ "Latency of write completion to callerfor writes not deferred for free space");
+ plb.add_time_avg(l_librbd_pwl_log_op_alloc_t, "op_alloc_t", "Average buffer pmemobj_reserve() time");
+ plb.add_u64_counter_histogram(
+ l_librbd_pwl_log_op_alloc_t_hist, "op_alloc_t_bytes_histogram",
+ op_hist_x_axis_config, op_hist_y_axis_config,
+ "Histogram of buffer pmemobj_reserve() time (nanoseconds) vs. bytes written");
+ plb.add_time_avg(l_librbd_pwl_log_op_dis_to_buf_t, "op_dis_to_buf_t", "Average dispatch to buffer persist time");
+ plb.add_time_avg(l_librbd_pwl_log_op_dis_to_app_t, "op_dis_to_app_t", "Average dispatch to log append time");
+ plb.add_time_avg(l_librbd_pwl_log_op_dis_to_cmp_t, "op_dis_to_cmp_t", "Average dispatch to persist completion time");
+ plb.add_u64_counter_histogram(
+ l_librbd_pwl_log_op_dis_to_cmp_t_hist, "op_dis_to_cmp_t_bytes_histogram",
+ op_hist_x_axis_config, op_hist_y_axis_config,
+ "Histogram of op dispatch to persist complete time (nanoseconds) vs. bytes written");
+
+ plb.add_time_avg(
+ l_librbd_pwl_log_op_buf_to_app_t, "op_buf_to_app_t",
+ "Average buffer persist to log append time (write data persist/replicate + wait for append time)");
+ plb.add_time_avg(
+ l_librbd_pwl_log_op_buf_to_bufc_t, "op_buf_to_bufc_t",
+ "Average buffer persist time (write data persist/replicate time)");
+ plb.add_u64_counter_histogram(
+ l_librbd_pwl_log_op_buf_to_bufc_t_hist, "op_buf_to_bufc_t_bytes_histogram",
+ op_hist_x_axis_config, op_hist_y_axis_config,
+ "Histogram of write buffer persist time (nanoseconds) vs. bytes written");
+ plb.add_time_avg(
+ l_librbd_pwl_log_op_app_to_cmp_t, "op_app_to_cmp_t",
+ "Average log append to persist complete time (log entry append/replicate + wait for complete time)");
+ plb.add_time_avg(
+ l_librbd_pwl_log_op_app_to_appc_t, "op_app_to_appc_t",
+ "Average log append to persist complete time (log entry append/replicate time)");
+ plb.add_u64_counter_histogram(
+ l_librbd_pwl_log_op_app_to_appc_t_hist, "op_app_to_appc_t_bytes_histogram",
+ op_hist_x_axis_config, op_hist_y_axis_config,
+ "Histogram of log append persist time (nanoseconds) (vs. op bytes)");
+
+ plb.add_u64_counter(l_librbd_pwl_discard, "discard", "Discards");
+ plb.add_u64_counter(l_librbd_pwl_discard_bytes, "discard_bytes", "Bytes discarded");
+ plb.add_time_avg(l_librbd_pwl_discard_latency, "discard_lat", "Discard latency");
+
+ plb.add_u64_counter(l_librbd_pwl_aio_flush, "aio_flush", "AIO flush (flush to RWL)");
+ plb.add_u64_counter(l_librbd_pwl_aio_flush_def, "aio_flush_def", "AIO flushes deferred for resources");
+ plb.add_time_avg(l_librbd_pwl_aio_flush_latency, "aio_flush_lat", "AIO flush latency");
+
+ plb.add_u64_counter(l_librbd_pwl_ws,"ws", "Write Sames");
+ plb.add_u64_counter(l_librbd_pwl_ws_bytes, "ws_bytes", "Write Same bytes to image");
+ plb.add_time_avg(l_librbd_pwl_ws_latency, "ws_lat", "Write Same latency");
+
+ plb.add_u64_counter(l_librbd_pwl_cmp, "cmp", "Compare and Write requests");
+ plb.add_u64_counter(l_librbd_pwl_cmp_bytes, "cmp_bytes", "Compare and Write bytes compared/written");
+ plb.add_time_avg(l_librbd_pwl_cmp_latency, "cmp_lat", "Compare and Write latecy");
+ plb.add_u64_counter(l_librbd_pwl_cmp_fails, "cmp_fails", "Compare and Write compare fails");
+
+ plb.add_u64_counter(l_librbd_pwl_internal_flush, "internal_flush", "Flush RWL (write back to OSD)");
+ plb.add_time_avg(l_librbd_pwl_writeback_latency, "writeback_lat", "write back to OSD latency");
+ plb.add_u64_counter(l_librbd_pwl_invalidate_cache, "invalidate", "Invalidate RWL");
+ plb.add_u64_counter(l_librbd_pwl_invalidate_discard_cache, "discard", "Discard and invalidate RWL");
+
+ plb.add_time_avg(l_librbd_pwl_append_tx_t, "append_tx_lat", "Log append transaction latency");
+ plb.add_u64_counter_histogram(
+ l_librbd_pwl_append_tx_t_hist, "append_tx_lat_histogram",
+ op_hist_x_axis_config, op_hist_y_axis_count_config,
+ "Histogram of log append transaction time (nanoseconds) vs. entries appended");
+ plb.add_time_avg(l_librbd_pwl_retire_tx_t, "retire_tx_lat", "Log retire transaction latency");
+ plb.add_u64_counter_histogram(
+ l_librbd_pwl_retire_tx_t_hist, "retire_tx_lat_histogram",
+ op_hist_x_axis_config, op_hist_y_axis_count_config,
+ "Histogram of log retire transaction time (nanoseconds) vs. entries retired");
+
+ m_perfcounter = plb.create_perf_counters();
+ m_image_ctx.cct->get_perfcounters_collection()->add(m_perfcounter);
+}
+
+template <typename I>
+void AbstractWriteLog<I>::perf_stop() {
+ ceph_assert(m_perfcounter);
+ m_image_ctx.cct->get_perfcounters_collection()->remove(m_perfcounter);
+ delete m_perfcounter;
+}
+
+template <typename I>
+void AbstractWriteLog<I>::log_perf() {
+ bufferlist bl;
+ Formatter *f = Formatter::create("json-pretty");
+ bl.append("Perf dump follows\n--- Begin perf dump ---\n");
+ bl.append("{\n");
+ stringstream ss;
+ utime_t now = ceph_clock_now();
+ ss << "\"test_time\": \"" << now << "\",";
+ ss << "\"image\": \"" << m_image_ctx.name << "\",";
+ bl.append(ss);
+ bl.append("\"stats\": ");
+ m_image_ctx.cct->get_perfcounters_collection()->dump_formatted(f, 0);
+ f->flush(bl);
+ bl.append(",\n\"histograms\": ");
+ m_image_ctx.cct->get_perfcounters_collection()->dump_formatted_histograms(f, 0);
+ f->flush(bl);
+ delete f;
+ bl.append("}\n--- End perf dump ---\n");
+ bl.append('\0');
+ ldout(m_image_ctx.cct, 1) << bl.c_str() << dendl;
+}
+
+template <typename I>
+void AbstractWriteLog<I>::periodic_stats() {
+ std::unique_lock locker(m_lock);
+ ldout(m_image_ctx.cct, 5) << "STATS: m_log_entries=" << m_log_entries.size()
+ << ", m_dirty_log_entries=" << m_dirty_log_entries.size()
+ << ", m_free_log_entries=" << m_free_log_entries
+ << ", m_bytes_allocated=" << m_bytes_allocated
+ << ", m_bytes_cached=" << m_bytes_cached
+ << ", m_bytes_dirty=" << m_bytes_dirty
+ << ", bytes available=" << m_bytes_allocated_cap - m_bytes_allocated
+ << ", m_first_valid_entry=" << m_first_valid_entry
+ << ", m_first_free_entry=" << m_first_free_entry
+ << ", m_current_sync_gen=" << m_current_sync_gen
+ << ", m_flushed_sync_gen=" << m_flushed_sync_gen
+ << dendl;
+
+ update_image_cache_state();
+ write_image_cache_state(locker);
+}
+
+template <typename I>
+void AbstractWriteLog<I>::arm_periodic_stats() {
+ ceph_assert(ceph_mutex_is_locked(*m_timer_lock));
+ m_timer_ctx = new LambdaContext([this](int r) {
+ /* m_timer_lock is held */
+ periodic_stats();
+ arm_periodic_stats();
+ });
+ m_timer->add_event_after(LOG_STATS_INTERVAL_SECONDS, m_timer_ctx);
+}
+
+template <typename I>
+void AbstractWriteLog<I>::update_entries(std::shared_ptr<GenericLogEntry> *log_entry,
+ WriteLogCacheEntry *cache_entry, std::map<uint64_t, bool> &missing_sync_points,
+ std::map<uint64_t, std::shared_ptr<SyncPointLogEntry>> &sync_point_entries,
+ uint64_t entry_index) {
+ bool writer = cache_entry->is_writer();
+ if (cache_entry->is_sync_point()) {
+ ldout(m_image_ctx.cct, 20) << "Entry " << entry_index
+ << " is a sync point. cache_entry=[" << *cache_entry << "]" << dendl;
+ auto sync_point_entry = std::make_shared<SyncPointLogEntry>(cache_entry->sync_gen_number);
+ *log_entry = sync_point_entry;
+ sync_point_entries[cache_entry->sync_gen_number] = sync_point_entry;
+ missing_sync_points.erase(cache_entry->sync_gen_number);
+ m_current_sync_gen = cache_entry->sync_gen_number;
+ } else if (cache_entry->is_write()) {
+ ldout(m_image_ctx.cct, 20) << "Entry " << entry_index
+ << " is a write. cache_entry=[" << *cache_entry << "]" << dendl;
+ auto write_entry =
+ m_builder->create_write_log_entry(nullptr, cache_entry->image_offset_bytes, cache_entry->write_bytes);
+ write_data_to_buffer(write_entry, cache_entry);
+ *log_entry = write_entry;
+ } else if (cache_entry->is_writesame()) {
+ ldout(m_image_ctx.cct, 20) << "Entry " << entry_index
+ << " is a write same. cache_entry=[" << *cache_entry << "]" << dendl;
+ auto ws_entry =
+ m_builder->create_writesame_log_entry(nullptr, cache_entry->image_offset_bytes,
+ cache_entry->write_bytes, cache_entry->ws_datalen);
+ write_data_to_buffer(ws_entry, cache_entry);
+ *log_entry = ws_entry;
+ } else if (cache_entry->is_discard()) {
+ ldout(m_image_ctx.cct, 20) << "Entry " << entry_index
+ << " is a discard. cache_entry=[" << *cache_entry << "]" << dendl;
+ auto discard_entry =
+ std::make_shared<DiscardLogEntry>(nullptr, cache_entry->image_offset_bytes, cache_entry->write_bytes,
+ m_discard_granularity_bytes);
+ *log_entry = discard_entry;
+ } else {
+ lderr(m_image_ctx.cct) << "Unexpected entry type in entry " << entry_index
+ << ", cache_entry=[" << *cache_entry << "]" << dendl;
+ }
+
+ if (writer) {
+ ldout(m_image_ctx.cct, 20) << "Entry " << entry_index
+ << " writes. cache_entry=[" << *cache_entry << "]" << dendl;
+ if (!sync_point_entries[cache_entry->sync_gen_number]) {
+ missing_sync_points[cache_entry->sync_gen_number] = true;
+ }
+ }
+}
+
+template <typename I>
+void AbstractWriteLog<I>::update_sync_points(std::map<uint64_t, bool> &missing_sync_points,
+ std::map<uint64_t, std::shared_ptr<SyncPointLogEntry>> &sync_point_entries,
+ DeferredContexts &later) {
+ /* Create missing sync points. These must not be appended until the
+ * entry reload is complete and the write map is up to
+ * date. Currently this is handled by the deferred contexts object
+ * passed to new_sync_point(). These contexts won't be completed
+ * until this function returns. */
+ for (auto &kv : missing_sync_points) {
+ ldout(m_image_ctx.cct, 5) << "Adding sync point " << kv.first << dendl;
+ if (0 == m_current_sync_gen) {
+ /* The unlikely case where the log contains writing entries, but no sync
+ * points (e.g. because they were all retired) */
+ m_current_sync_gen = kv.first-1;
+ }
+ ceph_assert(kv.first == m_current_sync_gen+1);
+ init_flush_new_sync_point(later);
+ ceph_assert(kv.first == m_current_sync_gen);
+ sync_point_entries[kv.first] = m_current_sync_point->log_entry;;
+ }
+
+ /*
+ * Iterate over the log entries again (this time via the global
+ * entries list), connecting write entries to their sync points and
+ * updating the sync point stats.
+ *
+ * Add writes to the write log map.
+ */
+ std::shared_ptr<SyncPointLogEntry> previous_sync_point_entry = nullptr;
+ for (auto &log_entry : m_log_entries) {
+ if ((log_entry->write_bytes() > 0) || (log_entry->bytes_dirty() > 0)) {
+ /* This entry is one of the types that write */
+ auto gen_write_entry = static_pointer_cast<GenericWriteLogEntry>(log_entry);
+ if (gen_write_entry) {
+ auto sync_point_entry = sync_point_entries[gen_write_entry->ram_entry.sync_gen_number];
+ if (!sync_point_entry) {
+ lderr(m_image_ctx.cct) << "Sync point missing for entry=[" << *gen_write_entry << "]" << dendl;
+ ceph_assert(false);
+ } else {
+ gen_write_entry->sync_point_entry = sync_point_entry;
+ sync_point_entry->writes++;
+ sync_point_entry->bytes += gen_write_entry->ram_entry.write_bytes;
+ sync_point_entry->writes_completed++;
+ m_blocks_to_log_entries.add_log_entry(gen_write_entry);
+ /* This entry is only dirty if its sync gen number is > the flushed
+ * sync gen number from the root object. */
+ if (gen_write_entry->ram_entry.sync_gen_number > m_flushed_sync_gen) {
+ m_dirty_log_entries.push_back(log_entry);
+ m_bytes_dirty += gen_write_entry->bytes_dirty();
+ } else {
+ gen_write_entry->set_flushed(true);
+ sync_point_entry->writes_flushed++;
+ }
+
+ /* calc m_bytes_allocated & m_bytes_cached */
+ inc_allocated_cached_bytes(log_entry);
+ }
+ }
+ } else {
+ /* This entry is sync point entry */
+ auto sync_point_entry = static_pointer_cast<SyncPointLogEntry>(log_entry);
+ if (sync_point_entry) {
+ if (previous_sync_point_entry) {
+ previous_sync_point_entry->next_sync_point_entry = sync_point_entry;
+ if (previous_sync_point_entry->ram_entry.sync_gen_number > m_flushed_sync_gen) {
+ sync_point_entry->prior_sync_point_flushed = false;
+ ceph_assert(!previous_sync_point_entry->prior_sync_point_flushed ||
+ (0 == previous_sync_point_entry->writes) ||
+ (previous_sync_point_entry->writes >= previous_sync_point_entry->writes_flushed));
+ } else {
+ sync_point_entry->prior_sync_point_flushed = true;
+ ceph_assert(previous_sync_point_entry->prior_sync_point_flushed);
+ ceph_assert(previous_sync_point_entry->writes == previous_sync_point_entry->writes_flushed);
+ }
+ } else {
+ /* There are no previous sync points, so we'll consider them flushed */
+ sync_point_entry->prior_sync_point_flushed = true;
+ }
+ previous_sync_point_entry = sync_point_entry;
+ ldout(m_image_ctx.cct, 10) << "Loaded to sync point=[" << *sync_point_entry << dendl;
+ }
+ }
+ }
+ if (0 == m_current_sync_gen) {
+ /* If a re-opened log was completely flushed, we'll have found no sync point entries here,
+ * and not advanced m_current_sync_gen. Here we ensure it starts past the last flushed sync
+ * point recorded in the log. */
+ m_current_sync_gen = m_flushed_sync_gen;
+ }
+}
+
+template <typename I>
+void AbstractWriteLog<I>::pwl_init(Context *on_finish, DeferredContexts &later) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 20) << dendl;
+ ceph_assert(m_cache_state);
+ std::lock_guard locker(m_lock);
+ ceph_assert(!m_initialized);
+ ldout(cct,5) << "image name: " << m_image_ctx.name << " id: " << m_image_ctx.id << dendl;
+
+ if (!m_cache_state->present) {
+ m_cache_state->host = ceph_get_short_hostname();
+ m_cache_state->size = m_image_ctx.config.template get_val<uint64_t>(
+ "rbd_persistent_cache_size");
+
+ string path = m_image_ctx.config.template get_val<string>(
+ "rbd_persistent_cache_path");
+ std::string pool_name = m_image_ctx.md_ctx.get_pool_name();
+ m_cache_state->path = path + "/rbd-pwl." + pool_name + "." + m_image_ctx.id + ".pool";
+ }
+
+ ldout(cct,5) << "pwl_size: " << m_cache_state->size << dendl;
+ ldout(cct,5) << "pwl_path: " << m_cache_state->path << dendl;
+
+ m_log_pool_name = m_cache_state->path;
+ m_log_pool_size = max(m_cache_state->size, MIN_POOL_SIZE);
+ m_log_pool_size = p2align(m_log_pool_size, POOL_SIZE_ALIGN);
+ ldout(cct, 5) << "pool " << m_log_pool_name << " size " << m_log_pool_size
+ << " (adjusted from " << m_cache_state->size << ")" << dendl;
+
+ if ((!m_cache_state->present) &&
+ (access(m_log_pool_name.c_str(), F_OK) == 0)) {
+ ldout(cct, 5) << "There's an existing pool file " << m_log_pool_name
+ << ", While there's no cache in the image metatata." << dendl;
+ if (remove(m_log_pool_name.c_str()) != 0) {
+ lderr(cct) << "Failed to remove the pool file " << m_log_pool_name
+ << dendl;
+ on_finish->complete(-errno);
+ return;
+ } else {
+ ldout(cct, 5) << "Removed the existing pool file." << dendl;
+ }
+ } else if ((m_cache_state->present) &&
+ (access(m_log_pool_name.c_str(), F_OK) != 0)) {
+ ldout(cct, 5) << "Can't find the existed pool file " << m_log_pool_name << dendl;
+ on_finish->complete(-errno);
+ return;
+ }
+
+ bool succeeded = initialize_pool(on_finish, later);
+ if (!succeeded) {
+ return ;
+ }
+
+ ldout(cct,1) << "pool " << m_log_pool_name << " has " << m_total_log_entries
+ << " log entries, " << m_free_log_entries << " of which are free."
+ << " first_valid=" << m_first_valid_entry
+ << ", first_free=" << m_first_free_entry
+ << ", flushed_sync_gen=" << m_flushed_sync_gen
+ << ", m_current_sync_gen=" << m_current_sync_gen << dendl;
+ if (m_first_free_entry == m_first_valid_entry) {
+ ldout(cct,1) << "write log is empty" << dendl;
+ m_cache_state->empty = true;
+ }
+
+ /* Start the sync point following the last one seen in the
+ * log. Flush the last sync point created during the loading of the
+ * existing log entries. */
+ init_flush_new_sync_point(later);
+ ldout(cct,20) << "new sync point = [" << m_current_sync_point << "]" << dendl;
+
+ m_initialized = true;
+ // Start the thread
+ m_thread_pool.start();
+
+ /* Do these after we drop lock */
+ later.add(new LambdaContext([this](int r) {
+ /* Log stats for the first time */
+ periodic_stats();
+ /* Arm periodic stats logging for the first time */
+ std::lock_guard timer_locker(*m_timer_lock);
+ arm_periodic_stats();
+ }));
+ m_image_ctx.op_work_queue->queue(on_finish, 0);
+}
+
+template <typename I>
+void AbstractWriteLog<I>::write_image_cache_state(std::unique_lock<ceph::mutex>& locker) {
+ using klass = AbstractWriteLog<I>;
+ Context *ctx = util::create_context_callback<
+ klass, &klass::handle_write_image_cache_state>(this);
+ m_cache_state->write_image_cache_state(locker, ctx);
+}
+
+template <typename I>
+void AbstractWriteLog<I>::update_image_cache_state() {
+ ldout(m_image_ctx.cct, 10) << dendl;
+
+ ceph_assert(ceph_mutex_is_locked_by_me(m_lock));
+ m_cache_state->allocated_bytes = m_bytes_allocated;
+ m_cache_state->cached_bytes = m_bytes_cached;
+ m_cache_state->dirty_bytes = m_bytes_dirty;
+ m_cache_state->free_bytes = m_bytes_allocated_cap - m_bytes_allocated;
+ m_cache_state->hits_full = m_perfcounter->get(l_librbd_pwl_rd_hit_req);
+ m_cache_state->hits_partial = m_perfcounter->get(l_librbd_pwl_rd_part_hit_req);
+ m_cache_state->misses = m_perfcounter->get(l_librbd_pwl_rd_req) -
+ m_cache_state->hits_full - m_cache_state->hits_partial;
+ m_cache_state->hit_bytes = m_perfcounter->get(l_librbd_pwl_rd_hit_bytes);
+ m_cache_state->miss_bytes = m_perfcounter->get(l_librbd_pwl_rd_bytes) -
+ m_cache_state->hit_bytes;
+}
+
+template <typename I>
+void AbstractWriteLog<I>::handle_write_image_cache_state(int r) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << "r=" << r << dendl;
+
+ if (r < 0) {
+ lderr(cct) << "failed to update image cache state: " << cpp_strerror(r)
+ << dendl;
+ return;
+ }
+}
+
+template <typename I>
+void AbstractWriteLog<I>::init(Context *on_finish) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 20) << dendl;
+ auto pname = std::string("librbd-pwl-") + m_image_ctx.id +
+ std::string("-") + m_image_ctx.md_ctx.get_pool_name() +
+ std::string("-") + m_image_ctx.name;
+ perf_start(pname);
+
+ ceph_assert(!m_initialized);
+
+ Context *ctx = new LambdaContext(
+ [this, on_finish](int r) {
+ if (r >= 0) {
+ std::unique_lock locker(m_lock);
+ update_image_cache_state();
+ m_cache_state->write_image_cache_state(locker, on_finish);
+ } else {
+ on_finish->complete(r);
+ }
+ });
+
+ DeferredContexts later;
+ pwl_init(ctx, later);
+}
+
+template <typename I>
+void AbstractWriteLog<I>::shut_down(Context *on_finish) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 20) << dendl;
+
+ ldout(cct,5) << "image name: " << m_image_ctx.name << " id: " << m_image_ctx.id << dendl;
+
+ Context *ctx = new LambdaContext(
+ [this, on_finish](int r) {
+ if (m_perfcounter) {
+ perf_stop();
+ }
+ ldout(m_image_ctx.cct, 6) << "shutdown complete" << dendl;
+ m_image_ctx.op_work_queue->queue(on_finish, r);
+ });
+ ctx = new LambdaContext(
+ [this, ctx](int r) {
+ ldout(m_image_ctx.cct, 6) << "image cache cleaned" << dendl;
+ Context *next_ctx = override_ctx(r, ctx);
+ periodic_stats();
+
+ std::unique_lock locker(m_lock);
+ check_image_cache_state_clean();
+ m_wake_up_enabled = false;
+ m_log_entries.clear();
+ m_cache_state->clean = true;
+ m_cache_state->empty = true;
+ remove_pool_file();
+ update_image_cache_state();
+ m_cache_state->write_image_cache_state(locker, next_ctx);
+ });
+ ctx = new LambdaContext(
+ [this, ctx](int r) {
+ Context *next_ctx = override_ctx(r, ctx);
+ ldout(m_image_ctx.cct, 6) << "waiting for in flight operations" << dendl;
+ // Wait for in progress IOs to complete
+ next_ctx = util::create_async_context_callback(&m_work_queue, next_ctx);
+ m_async_op_tracker.wait_for_ops(next_ctx);
+ });
+ ctx = new LambdaContext(
+ [this, ctx](int r) {
+ Context *next_ctx = override_ctx(r, ctx);
+ {
+ /* Sync with process_writeback_dirty_entries() */
+ RWLock::WLocker entry_reader_wlocker(m_entry_reader_lock);
+ m_shutting_down = true;
+ /* Flush all writes to OSDs (unless disabled) and wait for all
+ in-progress flush writes to complete */
+ ldout(m_image_ctx.cct, 6) << "flushing" << dendl;
+ periodic_stats();
+ }
+ flush_dirty_entries(next_ctx);
+ });
+ ctx = new LambdaContext(
+ [this, ctx](int r) {
+ ldout(m_image_ctx.cct, 6) << "Done internal_flush in shutdown" << dendl;
+ m_work_queue.queue(ctx, r);
+ });
+ /* Complete all in-flight writes before shutting down */
+ ldout(m_image_ctx.cct, 6) << "internal_flush in shutdown" << dendl;
+ internal_flush(false, ctx);
+}
+
+template <typename I>
+void AbstractWriteLog<I>::read(Extents&& image_extents,
+ ceph::bufferlist* bl,
+ int fadvise_flags, Context *on_finish) {
+ CephContext *cct = m_image_ctx.cct;
+ utime_t now = ceph_clock_now();
+
+ on_finish = new LambdaContext(
+ [this, on_finish](int r) {
+ m_async_op_tracker.finish_op();
+ on_finish->complete(r);
+ });
+ C_ReadRequest *read_ctx = m_builder->create_read_request(
+ cct, now, m_perfcounter, bl, on_finish);
+ ldout(cct, 20) << "name: " << m_image_ctx.name << " id: " << m_image_ctx.id
+ << "image_extents=" << image_extents << ", "
+ << "bl=" << bl << ", "
+ << "on_finish=" << on_finish << dendl;
+
+ ceph_assert(m_initialized);
+ bl->clear();
+ m_perfcounter->inc(l_librbd_pwl_rd_req, 1);
+
+ std::vector<std::shared_ptr<GenericWriteLogEntry>> log_entries_to_read;
+ std::vector<bufferlist*> bls_to_read;
+
+ m_async_op_tracker.start_op();
+ Context *ctx = new LambdaContext(
+ [this, read_ctx, fadvise_flags](int r) {
+ if (read_ctx->miss_extents.empty()) {
+ /* All of this read comes from RWL */
+ read_ctx->complete(0);
+ } else {
+ /* Pass the read misses on to the layer below RWL */
+ m_image_writeback.aio_read(
+ std::move(read_ctx->miss_extents), &read_ctx->miss_bl,
+ fadvise_flags, read_ctx);
+ }
+ });
+
+ /*
+ * The strategy here is to look up all the WriteLogMapEntries that overlap
+ * this read, and iterate through those to separate this read into hits and
+ * misses. A new Extents object is produced here with Extents for each miss
+ * region. The miss Extents is then passed on to the read cache below RWL. We
+ * also produce an ImageExtentBufs for all the extents (hit or miss) in this
+ * read. When the read from the lower cache layer completes, we iterate
+ * through the ImageExtentBufs and insert buffers for each cache hit at the
+ * appropriate spot in the bufferlist returned from below for the miss
+ * read. The buffers we insert here refer directly to regions of various
+ * write log entry data buffers.
+ *
+ * Locking: These buffer objects hold a reference on the write log entries
+ * they refer to. Log entries can't be retired until there are no references.
+ * The GenericWriteLogEntry references are released by the buffer destructor.
+ */
+ for (auto &extent : image_extents) {
+ uint64_t extent_offset = 0;
+ RWLock::RLocker entry_reader_locker(m_entry_reader_lock);
+ WriteLogMapEntries map_entries = m_blocks_to_log_entries.find_map_entries(
+ block_extent(extent));
+ for (auto &map_entry : map_entries) {
+ Extent entry_image_extent(pwl::image_extent(map_entry.block_extent));
+ /* If this map entry starts after the current image extent offset ... */
+ if (entry_image_extent.first > extent.first + extent_offset) {
+ /* ... add range before map_entry to miss extents */
+ uint64_t miss_extent_start = extent.first + extent_offset;
+ uint64_t miss_extent_length = entry_image_extent.first -
+ miss_extent_start;
+ Extent miss_extent(miss_extent_start, miss_extent_length);
+ read_ctx->miss_extents.push_back(miss_extent);
+ /* Add miss range to read extents */
+ auto miss_extent_buf = std::make_shared<ImageExtentBuf>(miss_extent);
+ read_ctx->read_extents.push_back(miss_extent_buf);
+ extent_offset += miss_extent_length;
+ }
+ ceph_assert(entry_image_extent.first <= extent.first + extent_offset);
+ uint64_t entry_offset = 0;
+ /* If this map entry starts before the current image extent offset ... */
+ if (entry_image_extent.first < extent.first + extent_offset) {
+ /* ... compute offset into log entry for this read extent */
+ entry_offset = (extent.first + extent_offset) - entry_image_extent.first;
+ }
+ /* This read hit ends at the end of the extent or the end of the log
+ entry, whichever is less. */
+ uint64_t entry_hit_length = min(entry_image_extent.second - entry_offset,
+ extent.second - extent_offset);
+ Extent hit_extent(entry_image_extent.first, entry_hit_length);
+ if (0 == map_entry.log_entry->write_bytes() &&
+ 0 < map_entry.log_entry->bytes_dirty()) {
+ /* discard log entry */
+ ldout(cct, 20) << "discard log entry" << dendl;
+ auto discard_entry = map_entry.log_entry;
+ ldout(cct, 20) << "read hit on discard entry: log_entry="
+ << *discard_entry
+ << dendl;
+ /* Discards read as zero, so we'll construct a bufferlist of zeros */
+ bufferlist zero_bl;
+ zero_bl.append_zero(entry_hit_length);
+ /* Add hit extent to read extents */
+ auto hit_extent_buf = std::make_shared<ImageExtentBuf>(
+ hit_extent, zero_bl);
+ read_ctx->read_extents.push_back(hit_extent_buf);
+ } else {
+ ldout(cct, 20) << "write or writesame log entry" << dendl;
+ /* write and writesame log entry */
+ /* Offset of the map entry into the log entry's buffer */
+ uint64_t map_entry_buffer_offset = entry_image_extent.first -
+ map_entry.log_entry->ram_entry.image_offset_bytes;
+ /* Offset into the log entry buffer of this read hit */
+ uint64_t read_buffer_offset = map_entry_buffer_offset + entry_offset;
+ /* Create buffer object referring to pmem pool for this read hit */
+ collect_read_extents(
+ read_buffer_offset, map_entry, log_entries_to_read, bls_to_read,
+ entry_hit_length, hit_extent, read_ctx);
+ }
+ /* Exclude RWL hit range from buffer and extent */
+ extent_offset += entry_hit_length;
+ ldout(cct, 20) << map_entry << dendl;
+ }
+ /* If the last map entry didn't consume the entire image extent ... */
+ if (extent.second > extent_offset) {
+ /* ... add the rest of this extent to miss extents */
+ uint64_t miss_extent_start = extent.first + extent_offset;
+ uint64_t miss_extent_length = extent.second - extent_offset;
+ Extent miss_extent(miss_extent_start, miss_extent_length);
+ read_ctx->miss_extents.push_back(miss_extent);
+ /* Add miss range to read extents */
+ auto miss_extent_buf = std::make_shared<ImageExtentBuf>(miss_extent);
+ read_ctx->read_extents.push_back(miss_extent_buf);
+ extent_offset += miss_extent_length;
+ }
+ }
+
+ ldout(cct, 20) << "miss_extents=" << read_ctx->miss_extents << ", "
+ << "miss_bl=" << read_ctx->miss_bl << dendl;
+
+ complete_read(log_entries_to_read, bls_to_read, ctx);
+}
+
+template <typename I>
+void AbstractWriteLog<I>::write(Extents &&image_extents,
+ bufferlist&& bl,
+ int fadvise_flags,
+ Context *on_finish) {
+ CephContext *cct = m_image_ctx.cct;
+
+ ldout(cct, 20) << "aio_write" << dendl;
+
+ utime_t now = ceph_clock_now();
+ m_perfcounter->inc(l_librbd_pwl_wr_req, 1);
+
+ ceph_assert(m_initialized);
+
+ /* Split images because PMDK's space management is not perfect, there are
+ * fragment problems. The larger the block size difference of the block,
+ * the easier the fragmentation problem will occur, resulting in the
+ * remaining space can not be allocated in large size. We plan to manage
+ * pmem space and allocation by ourselves in the future.
+ */
+ Extents split_image_extents;
+ uint64_t max_extent_size = get_max_extent();
+ if (max_extent_size != 0) {
+ for (auto extent : image_extents) {
+ if (extent.second > max_extent_size) {
+ uint64_t off = extent.first;
+ uint64_t extent_bytes = extent.second;
+ for (int i = 0; extent_bytes != 0; ++i) {
+ Extent _ext;
+ _ext.first = off + i * max_extent_size;
+ _ext.second = std::min(max_extent_size, extent_bytes);
+ extent_bytes = extent_bytes - _ext.second ;
+ split_image_extents.emplace_back(_ext);
+ }
+ } else {
+ split_image_extents.emplace_back(extent);
+ }
+ }
+ } else {
+ split_image_extents = image_extents;
+ }
+
+ C_WriteRequestT *write_req =
+ m_builder->create_write_request(*this, now, std::move(split_image_extents),
+ std::move(bl), fadvise_flags, m_lock,
+ m_perfcounter, on_finish);
+ m_perfcounter->inc(l_librbd_pwl_wr_bytes,
+ write_req->image_extents_summary.total_bytes);
+
+ /* The lambda below will be called when the block guard for all
+ * blocks affected by this write is obtained */
+ GuardedRequestFunctionContext *guarded_ctx =
+ new GuardedRequestFunctionContext([this,
+ write_req](GuardedRequestFunctionContext &guard_ctx) {
+ write_req->blockguard_acquired(guard_ctx);
+ alloc_and_dispatch_io_req(write_req);
+ });
+
+ detain_guarded_request(write_req, guarded_ctx, false);
+}
+
+template <typename I>
+void AbstractWriteLog<I>::discard(uint64_t offset, uint64_t length,
+ uint32_t discard_granularity_bytes,
+ Context *on_finish) {
+ CephContext *cct = m_image_ctx.cct;
+
+ ldout(cct, 20) << dendl;
+
+ utime_t now = ceph_clock_now();
+ m_perfcounter->inc(l_librbd_pwl_discard, 1);
+ Extents discard_extents = {{offset, length}};
+ m_discard_granularity_bytes = discard_granularity_bytes;
+
+ ceph_assert(m_initialized);
+
+ auto *discard_req =
+ new C_DiscardRequestT(*this, now, std::move(discard_extents), discard_granularity_bytes,
+ m_lock, m_perfcounter, on_finish);
+
+ /* The lambda below will be called when the block guard for all
+ * blocks affected by this write is obtained */
+ GuardedRequestFunctionContext *guarded_ctx =
+ new GuardedRequestFunctionContext([this, discard_req](GuardedRequestFunctionContext &guard_ctx) {
+ discard_req->blockguard_acquired(guard_ctx);
+ alloc_and_dispatch_io_req(discard_req);
+ });
+
+ detain_guarded_request(discard_req, guarded_ctx, false);
+}
+
+/**
+ * Aio_flush completes when all previously completed writes are
+ * flushed to persistent cache. We make a best-effort attempt to also
+ * defer until all in-progress writes complete, but we may not know
+ * about all of the writes the application considers in-progress yet,
+ * due to uncertainty in the IO submission workq (multiple WQ threads
+ * may allow out-of-order submission).
+ *
+ * This flush operation will not wait for writes deferred for overlap
+ * in the block guard.
+ */
+template <typename I>
+void AbstractWriteLog<I>::flush(io::FlushSource flush_source, Context *on_finish) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 20) << "on_finish=" << on_finish << " flush_source=" << flush_source << dendl;
+
+ if (io::FLUSH_SOURCE_SHUTDOWN == flush_source || io::FLUSH_SOURCE_INTERNAL == flush_source ||
+ io::FLUSH_SOURCE_WRITE_BLOCK == flush_source) {
+ internal_flush(false, on_finish);
+ return;
+ }
+ m_perfcounter->inc(l_librbd_pwl_aio_flush, 1);
+
+ /* May be called even if initialization fails */
+ if (!m_initialized) {
+ ldout(cct, 05) << "never initialized" << dendl;
+ /* Deadlock if completed here */
+ m_image_ctx.op_work_queue->queue(on_finish, 0);
+ return;
+ }
+
+ {
+ std::shared_lock image_locker(m_image_ctx.image_lock);
+ if (m_image_ctx.snap_id != CEPH_NOSNAP || m_image_ctx.read_only) {
+ on_finish->complete(-EROFS);
+ return;
+ }
+ }
+
+ auto flush_req = make_flush_req(on_finish);
+
+ GuardedRequestFunctionContext *guarded_ctx =
+ new GuardedRequestFunctionContext([this, flush_req](GuardedRequestFunctionContext &guard_ctx) {
+ ldout(m_image_ctx.cct, 20) << "flush_req=" << flush_req << " cell=" << guard_ctx.cell << dendl;
+ ceph_assert(guard_ctx.cell);
+ flush_req->detained = guard_ctx.state.detained;
+ /* We don't call flush_req->set_cell(), because the block guard will be released here */
+ {
+ DeferredContexts post_unlock; /* Do these when the lock below is released */
+ std::lock_guard locker(m_lock);
+
+ if (!m_persist_on_flush && m_persist_on_write_until_flush) {
+ m_persist_on_flush = true;
+ ldout(m_image_ctx.cct, 5) << "now persisting on flush" << dendl;
+ }
+
+ /*
+ * Create a new sync point if there have been writes since the last
+ * one.
+ *
+ * We do not flush the caches below the RWL here.
+ */
+ flush_new_sync_point_if_needed(flush_req, post_unlock);
+ }
+
+ release_guarded_request(guard_ctx.cell);
+ });
+
+ detain_guarded_request(flush_req, guarded_ctx, true);
+}
+
+template <typename I>
+void AbstractWriteLog<I>::writesame(uint64_t offset, uint64_t length,
+ bufferlist&& bl, int fadvise_flags,
+ Context *on_finish) {
+ CephContext *cct = m_image_ctx.cct;
+
+ ldout(cct, 20) << "aio_writesame" << dendl;
+
+ utime_t now = ceph_clock_now();
+ Extents ws_extents = {{offset, length}};
+ m_perfcounter->inc(l_librbd_pwl_ws, 1);
+ ceph_assert(m_initialized);
+
+ /* A write same request is also a write request. The key difference is the
+ * write same data buffer is shorter than the extent of the request. The full
+ * extent will be used in the block guard, and appear in
+ * m_blocks_to_log_entries_map. The data buffer allocated for the WS is only
+ * as long as the length of the bl here, which is the pattern that's repeated
+ * in the image for the entire length of this WS. Read hits and flushing of
+ * write sames are different than normal writes. */
+ C_WriteSameRequestT *ws_req =
+ m_builder->create_writesame_request(*this, now, std::move(ws_extents), std::move(bl),
+ fadvise_flags, m_lock, m_perfcounter, on_finish);
+ m_perfcounter->inc(l_librbd_pwl_ws_bytes, ws_req->image_extents_summary.total_bytes);
+
+ /* The lambda below will be called when the block guard for all
+ * blocks affected by this write is obtained */
+ GuardedRequestFunctionContext *guarded_ctx =
+ new GuardedRequestFunctionContext([this, ws_req](GuardedRequestFunctionContext &guard_ctx) {
+ ws_req->blockguard_acquired(guard_ctx);
+ alloc_and_dispatch_io_req(ws_req);
+ });
+
+ detain_guarded_request(ws_req, guarded_ctx, false);
+}
+
+template <typename I>
+void AbstractWriteLog<I>::compare_and_write(Extents &&image_extents,
+ bufferlist&& cmp_bl,
+ bufferlist&& bl,
+ uint64_t *mismatch_offset,
+ int fadvise_flags,
+ Context *on_finish) {
+ ldout(m_image_ctx.cct, 20) << dendl;
+
+ utime_t now = ceph_clock_now();
+ m_perfcounter->inc(l_librbd_pwl_cmp, 1);
+ ceph_assert(m_initialized);
+
+ /* A compare and write request is also a write request. We only allocate
+ * resources and dispatch this write request if the compare phase
+ * succeeds. */
+ C_WriteRequestT *cw_req =
+ m_builder->create_comp_and_write_request(
+ *this, now, std::move(image_extents), std::move(cmp_bl), std::move(bl),
+ mismatch_offset, fadvise_flags, m_lock, m_perfcounter, on_finish);
+ m_perfcounter->inc(l_librbd_pwl_cmp_bytes, cw_req->image_extents_summary.total_bytes);
+
+ /* The lambda below will be called when the block guard for all
+ * blocks affected by this write is obtained */
+ GuardedRequestFunctionContext *guarded_ctx =
+ new GuardedRequestFunctionContext([this, cw_req](GuardedRequestFunctionContext &guard_ctx) {
+ cw_req->blockguard_acquired(guard_ctx);
+
+ auto read_complete_ctx = new LambdaContext(
+ [this, cw_req](int r) {
+ ldout(m_image_ctx.cct, 20) << "name: " << m_image_ctx.name << " id: " << m_image_ctx.id
+ << "cw_req=" << cw_req << dendl;
+
+ /* Compare read_bl to cmp_bl to determine if this will produce a write */
+ buffer::list aligned_read_bl;
+ if (cw_req->cmp_bl.length() < cw_req->read_bl.length()) {
+ aligned_read_bl.substr_of(cw_req->read_bl, 0, cw_req->cmp_bl.length());
+ }
+ if (cw_req->cmp_bl.contents_equal(cw_req->read_bl) ||
+ cw_req->cmp_bl.contents_equal(aligned_read_bl)) {
+ /* Compare phase succeeds. Begin write */
+ ldout(m_image_ctx.cct, 5) << " cw_req=" << cw_req << " compare matched" << dendl;
+ cw_req->compare_succeeded = true;
+ *cw_req->mismatch_offset = 0;
+ /* Continue with this request as a write. Blockguard release and
+ * user request completion handled as if this were a plain
+ * write. */
+ alloc_and_dispatch_io_req(cw_req);
+ } else {
+ /* Compare phase fails. Comp-and write ends now. */
+ ldout(m_image_ctx.cct, 15) << " cw_req=" << cw_req << " compare failed" << dendl;
+ /* Bufferlist doesn't tell us where they differed, so we'll have to determine that here */
+ uint64_t bl_index = 0;
+ for (bl_index = 0; bl_index < cw_req->cmp_bl.length(); bl_index++) {
+ if (cw_req->cmp_bl[bl_index] != cw_req->read_bl[bl_index]) {
+ ldout(m_image_ctx.cct, 15) << " cw_req=" << cw_req << " mismatch at " << bl_index << dendl;
+ break;
+ }
+ }
+ cw_req->compare_succeeded = false;
+ *cw_req->mismatch_offset = bl_index;
+ cw_req->complete_user_request(-EILSEQ);
+ cw_req->release_cell();
+ cw_req->complete(0);
+ }
+ });
+
+ /* Read phase of comp-and-write must read through RWL */
+ Extents image_extents_copy = cw_req->image_extents;
+ read(std::move(image_extents_copy), &cw_req->read_bl, cw_req->fadvise_flags, read_complete_ctx);
+ });
+
+ detain_guarded_request(cw_req, guarded_ctx, false);
+}
+
+template <typename I>
+void AbstractWriteLog<I>::flush(Context *on_finish) {
+ internal_flush(false, on_finish);
+}
+
+template <typename I>
+void AbstractWriteLog<I>::invalidate(Context *on_finish) {
+ internal_flush(true, on_finish);
+}
+
+template <typename I>
+CephContext *AbstractWriteLog<I>::get_context() {
+ return m_image_ctx.cct;
+}
+
+template <typename I>
+BlockGuardCell* AbstractWriteLog<I>::detain_guarded_request_helper(GuardedRequest &req)
+{
+ CephContext *cct = m_image_ctx.cct;
+ BlockGuardCell *cell;
+
+ ceph_assert(ceph_mutex_is_locked_by_me(m_blockguard_lock));
+ ldout(cct, 20) << dendl;
+
+ int r = m_write_log_guard.detain(req.block_extent, &req, &cell);
+ ceph_assert(r>=0);
+ if (r > 0) {
+ ldout(cct, 20) << "detaining guarded request due to in-flight requests: "
+ << "req=" << req << dendl;
+ return nullptr;
+ }
+
+ ldout(cct, 20) << "in-flight request cell: " << cell << dendl;
+ return cell;
+}
+
+template <typename I>
+BlockGuardCell* AbstractWriteLog<I>::detain_guarded_request_barrier_helper(
+ GuardedRequest &req)
+{
+ BlockGuardCell *cell = nullptr;
+
+ ceph_assert(ceph_mutex_is_locked_by_me(m_blockguard_lock));
+ ldout(m_image_ctx.cct, 20) << dendl;
+
+ if (m_barrier_in_progress) {
+ req.guard_ctx->state.queued = true;
+ m_awaiting_barrier.push_back(req);
+ } else {
+ bool barrier = req.guard_ctx->state.barrier;
+ if (barrier) {
+ m_barrier_in_progress = true;
+ req.guard_ctx->state.current_barrier = true;
+ }
+ cell = detain_guarded_request_helper(req);
+ if (barrier) {
+ /* Only non-null if the barrier acquires the guard now */
+ m_barrier_cell = cell;
+ }
+ }
+
+ return cell;
+}
+
+template <typename I>
+void AbstractWriteLog<I>::detain_guarded_request(
+ C_BlockIORequestT *request,
+ GuardedRequestFunctionContext *guarded_ctx,
+ bool is_barrier)
+{
+ BlockExtent extent;
+ if (request) {
+ extent = request->image_extents_summary.block_extent();
+ } else {
+ extent = block_extent(whole_volume_extent());
+ }
+ auto req = GuardedRequest(extent, guarded_ctx, is_barrier);
+ BlockGuardCell *cell = nullptr;
+
+ ldout(m_image_ctx.cct, 20) << dendl;
+ {
+ std::lock_guard locker(m_blockguard_lock);
+ cell = detain_guarded_request_barrier_helper(req);
+ }
+ if (cell) {
+ req.guard_ctx->cell = cell;
+ req.guard_ctx->complete(0);
+ }
+}
+
+template <typename I>
+void AbstractWriteLog<I>::release_guarded_request(BlockGuardCell *released_cell)
+{
+ CephContext *cct = m_image_ctx.cct;
+ WriteLogGuard::BlockOperations block_reqs;
+ ldout(cct, 20) << "released_cell=" << released_cell << dendl;
+
+ {
+ std::lock_guard locker(m_blockguard_lock);
+ m_write_log_guard.release(released_cell, &block_reqs);
+
+ for (auto &req : block_reqs) {
+ req.guard_ctx->state.detained = true;
+ BlockGuardCell *detained_cell = detain_guarded_request_helper(req);
+ if (detained_cell) {
+ if (req.guard_ctx->state.current_barrier) {
+ /* The current barrier is acquiring the block guard, so now we know its cell */
+ m_barrier_cell = detained_cell;
+ /* detained_cell could be == released_cell here */
+ ldout(cct, 20) << "current barrier cell=" << detained_cell << " req=" << req << dendl;
+ }
+ req.guard_ctx->cell = detained_cell;
+ m_work_queue.queue(req.guard_ctx);
+ }
+ }
+
+ if (m_barrier_in_progress && (released_cell == m_barrier_cell)) {
+ ldout(cct, 20) << "current barrier released cell=" << released_cell << dendl;
+ /* The released cell is the current barrier request */
+ m_barrier_in_progress = false;
+ m_barrier_cell = nullptr;
+ /* Move waiting requests into the blockguard. Stop if there's another barrier */
+ while (!m_barrier_in_progress && !m_awaiting_barrier.empty()) {
+ auto &req = m_awaiting_barrier.front();
+ ldout(cct, 20) << "submitting queued request to blockguard: " << req << dendl;
+ BlockGuardCell *detained_cell = detain_guarded_request_barrier_helper(req);
+ if (detained_cell) {
+ req.guard_ctx->cell = detained_cell;
+ m_work_queue.queue(req.guard_ctx);
+ }
+ m_awaiting_barrier.pop_front();
+ }
+ }
+ }
+
+ ldout(cct, 20) << "exit" << dendl;
+}
+
+template <typename I>
+void AbstractWriteLog<I>::append_scheduled(GenericLogOperations &ops, bool &ops_remain,
+ bool &appending, bool isRWL)
+{
+ const unsigned long int OPS_APPENDED = isRWL ? MAX_ALLOC_PER_TRANSACTION
+ : MAX_WRITES_PER_SYNC_POINT;
+ {
+ std::lock_guard locker(m_lock);
+ if (!appending && m_appending) {
+ /* Another thread is appending */
+ ldout(m_image_ctx.cct, 15) << "Another thread is appending" << dendl;
+ return;
+ }
+ if (m_ops_to_append.size()) {
+ appending = true;
+ m_appending = true;
+ auto last_in_batch = m_ops_to_append.begin();
+ unsigned int ops_to_append = m_ops_to_append.size();
+ if (ops_to_append > OPS_APPENDED) {
+ ops_to_append = OPS_APPENDED;
+ }
+ std::advance(last_in_batch, ops_to_append);
+ ops.splice(ops.end(), m_ops_to_append, m_ops_to_append.begin(), last_in_batch);
+ ops_remain = true; /* Always check again before leaving */
+ ldout(m_image_ctx.cct, 20) << "appending " << ops.size() << ", "
+ << m_ops_to_append.size() << " remain" << dendl;
+ } else if (isRWL) {
+ ops_remain = false;
+ if (appending) {
+ appending = false;
+ m_appending = false;
+ }
+ }
+ }
+}
+
+template <typename I>
+void AbstractWriteLog<I>::schedule_append(GenericLogOperationsVector &ops, C_BlockIORequestT *req)
+{
+ GenericLogOperations to_append(ops.begin(), ops.end());
+
+ schedule_append_ops(to_append, req);
+}
+
+template <typename I>
+void AbstractWriteLog<I>::schedule_append(GenericLogOperationSharedPtr op, C_BlockIORequestT *req)
+{
+ GenericLogOperations to_append { op };
+
+ schedule_append_ops(to_append, req);
+}
+
+/*
+ * Complete a set of write ops with the result of append_op_entries.
+ */
+template <typename I>
+void AbstractWriteLog<I>::complete_op_log_entries(GenericLogOperations &&ops,
+ const int result)
+{
+ GenericLogEntries dirty_entries;
+ int published_reserves = 0;
+ ldout(m_image_ctx.cct, 20) << __func__ << ": completing" << dendl;
+ for (auto &op : ops) {
+ utime_t now = ceph_clock_now();
+ auto log_entry = op->get_log_entry();
+ log_entry->completed = true;
+ if (op->is_writing_op()) {
+ op->mark_log_entry_completed();
+ dirty_entries.push_back(log_entry);
+ }
+ if (log_entry->is_write_entry()) {
+ release_ram(log_entry);
+ }
+ if (op->reserved_allocated()) {
+ published_reserves++;
+ }
+ {
+ std::lock_guard locker(m_lock);
+ m_unpublished_reserves -= published_reserves;
+ m_dirty_log_entries.splice(m_dirty_log_entries.end(), dirty_entries);
+ }
+ op->complete(result);
+ m_perfcounter->tinc(l_librbd_pwl_log_op_dis_to_app_t,
+ op->log_append_start_time - op->dispatch_time);
+ m_perfcounter->tinc(l_librbd_pwl_log_op_dis_to_cmp_t, now - op->dispatch_time);
+ m_perfcounter->hinc(l_librbd_pwl_log_op_dis_to_cmp_t_hist,
+ utime_t(now - op->dispatch_time).to_nsec(),
+ log_entry->ram_entry.write_bytes);
+ utime_t app_lat = op->log_append_comp_time - op->log_append_start_time;
+ m_perfcounter->tinc(l_librbd_pwl_log_op_app_to_appc_t, app_lat);
+ m_perfcounter->hinc(l_librbd_pwl_log_op_app_to_appc_t_hist, app_lat.to_nsec(),
+ log_entry->ram_entry.write_bytes);
+ m_perfcounter->tinc(l_librbd_pwl_log_op_app_to_cmp_t, now - op->log_append_start_time);
+ }
+ // New entries may be flushable
+ {
+ std::lock_guard locker(m_lock);
+ wake_up();
+ }
+}
+
+/**
+ * Dispatch as many deferred writes as possible
+ */
+template <typename I>
+void AbstractWriteLog<I>::dispatch_deferred_writes(void)
+{
+ C_BlockIORequestT *front_req = nullptr; /* req still on front of deferred list */
+ C_BlockIORequestT *allocated_req = nullptr; /* req that was allocated, and is now off the list */
+ bool allocated = false; /* front_req allocate succeeded */
+ bool cleared_dispatching_flag = false;
+
+ /* If we can't become the dispatcher, we'll exit */
+ {
+ std::lock_guard locker(m_lock);
+ if (m_dispatching_deferred_ops ||
+ !m_deferred_ios.size()) {
+ return;
+ }
+ m_dispatching_deferred_ops = true;
+ }
+
+ /* There are ops to dispatch, and this should be the only thread dispatching them */
+ {
+ std::lock_guard deferred_dispatch(m_deferred_dispatch_lock);
+ do {
+ {
+ std::lock_guard locker(m_lock);
+ ceph_assert(m_dispatching_deferred_ops);
+ if (allocated) {
+ /* On the 2..n-1 th time we get lock, front_req->alloc_resources() will
+ * have succeeded, and we'll need to pop it off the deferred ops list
+ * here. */
+ ceph_assert(front_req);
+ ceph_assert(!allocated_req);
+ m_deferred_ios.pop_front();
+ allocated_req = front_req;
+ front_req = nullptr;
+ allocated = false;
+ }
+ ceph_assert(!allocated);
+ if (!allocated && front_req) {
+ /* front_req->alloc_resources() failed on the last iteration.
+ * We'll stop dispatching. */
+ wake_up();
+ front_req = nullptr;
+ ceph_assert(!cleared_dispatching_flag);
+ m_dispatching_deferred_ops = false;
+ cleared_dispatching_flag = true;
+ } else {
+ ceph_assert(!front_req);
+ if (m_deferred_ios.size()) {
+ /* New allocation candidate */
+ front_req = m_deferred_ios.front();
+ } else {
+ ceph_assert(!cleared_dispatching_flag);
+ m_dispatching_deferred_ops = false;
+ cleared_dispatching_flag = true;
+ }
+ }
+ }
+ /* Try allocating for front_req before we decide what to do with allocated_req
+ * (if any) */
+ if (front_req) {
+ ceph_assert(!cleared_dispatching_flag);
+ allocated = front_req->alloc_resources();
+ }
+ if (allocated_req && front_req && allocated) {
+ /* Push dispatch of the first allocated req to a wq */
+ m_work_queue.queue(new LambdaContext(
+ [allocated_req](int r) {
+ allocated_req->dispatch();
+ }), 0);
+ allocated_req = nullptr;
+ }
+ ceph_assert(!(allocated_req && front_req && allocated));
+
+ /* Continue while we're still considering the front of the deferred ops list */
+ } while (front_req);
+ ceph_assert(!allocated);
+ }
+ ceph_assert(cleared_dispatching_flag);
+
+ /* If any deferred requests were allocated, the last one will still be in allocated_req */
+ if (allocated_req) {
+ allocated_req->dispatch();
+ }
+}
+
+/**
+ * Returns the lanes used by this write, and attempts to dispatch the next
+ * deferred write
+ */
+template <typename I>
+void AbstractWriteLog<I>::release_write_lanes(C_BlockIORequestT *req)
+{
+ {
+ std::lock_guard locker(m_lock);
+ m_free_lanes += req->image_extents.size();
+ }
+ dispatch_deferred_writes();
+}
+
+/**
+ * Attempts to allocate log resources for a write. Write is dispatched if
+ * resources are available, or queued if they aren't.
+ */
+template <typename I>
+void AbstractWriteLog<I>::alloc_and_dispatch_io_req(C_BlockIORequestT *req)
+{
+ bool dispatch_here = false;
+
+ {
+ /* If there are already deferred writes, queue behind them for resources */
+ {
+ std::lock_guard locker(m_lock);
+ dispatch_here = m_deferred_ios.empty();
+ // Only flush req's total_bytes is the max uint64
+ if (req->image_extents_summary.total_bytes ==
+ std::numeric_limits<uint64_t>::max() &&
+ static_cast<C_FlushRequestT *>(req)->internal == true) {
+ dispatch_here = true;
+ }
+ }
+ if (dispatch_here) {
+ dispatch_here = req->alloc_resources();
+ }
+ if (dispatch_here) {
+ ldout(m_image_ctx.cct, 20) << "dispatching" << dendl;
+ req->dispatch();
+ } else {
+ req->deferred();
+ {
+ std::lock_guard locker(m_lock);
+ m_deferred_ios.push_back(req);
+ }
+ ldout(m_image_ctx.cct, 20) << "deferred IOs: " << m_deferred_ios.size() << dendl;
+ dispatch_deferred_writes();
+ }
+ }
+}
+
+template <typename I>
+bool AbstractWriteLog<I>::check_allocation(
+ C_BlockIORequestT *req, uint64_t bytes_cached, uint64_t bytes_dirtied,
+ uint64_t bytes_allocated, uint32_t num_lanes, uint32_t num_log_entries,
+ uint32_t num_unpublished_reserves) {
+ bool alloc_succeeds = true;
+ bool no_space = false;
+ {
+ std::lock_guard locker(m_lock);
+ if (m_free_lanes < num_lanes) {
+ req->set_io_waited_for_lanes(true);
+ ldout(m_image_ctx.cct, 20) << "not enough free lanes (need "
+ << num_lanes
+ << ", have " << m_free_lanes << ") "
+ << *req << dendl;
+ alloc_succeeds = false;
+ /* This isn't considered a "no space" alloc fail. Lanes are a throttling mechanism. */
+ }
+ if (m_free_log_entries < num_log_entries) {
+ req->set_io_waited_for_entries(true);
+ ldout(m_image_ctx.cct, 20) << "not enough free entries (need "
+ << num_log_entries
+ << ", have " << m_free_log_entries << ") "
+ << *req << dendl;
+ alloc_succeeds = false;
+ no_space = true; /* Entries must be retired */
+ }
+ /* Don't attempt buffer allocate if we've exceeded the "full" threshold */
+ if (m_bytes_allocated + bytes_allocated > m_bytes_allocated_cap) {
+ if (!req->has_io_waited_for_buffers()) {
+ req->set_io_waited_for_buffers(true);
+ ldout(m_image_ctx.cct, 5) << "Waiting for allocation cap (cap="
+ << m_bytes_allocated_cap
+ << ", allocated=" << m_bytes_allocated
+ << ") in write [" << *req << "]" << dendl;
+ }
+ alloc_succeeds = false;
+ no_space = true; /* Entries must be retired */
+ }
+ }
+
+ if (alloc_succeeds) {
+ reserve_cache(req, alloc_succeeds, no_space);
+ }
+
+ if (alloc_succeeds) {
+ std::unique_lock locker(m_lock);
+ /* We need one free log entry per extent (each is a separate entry), and
+ * one free "lane" for remote replication. */
+ if ((m_free_lanes >= num_lanes) &&
+ (m_free_log_entries >= num_log_entries) &&
+ (m_bytes_allocated_cap >= m_bytes_allocated + bytes_allocated)) {
+ m_free_lanes -= num_lanes;
+ m_free_log_entries -= num_log_entries;
+ m_unpublished_reserves += num_unpublished_reserves;
+ m_bytes_allocated += bytes_allocated;
+ m_bytes_cached += bytes_cached;
+ m_bytes_dirty += bytes_dirtied;
+ if (req->has_io_waited_for_buffers()) {
+ req->set_io_waited_for_buffers(false);
+ }
+ if (m_cache_state->clean && bytes_dirtied > 0) {
+ m_cache_state->clean = false;
+ update_image_cache_state();
+ write_image_cache_state(locker);
+ }
+ } else {
+ alloc_succeeds = false;
+ }
+ }
+
+ if (!alloc_succeeds && no_space) {
+ /* Expedite flushing and/or retiring */
+ std::lock_guard locker(m_lock);
+ m_alloc_failed_since_retire = true;
+ m_last_alloc_fail = ceph_clock_now();
+ }
+
+ return alloc_succeeds;
+}
+
+template <typename I>
+C_FlushRequest<AbstractWriteLog<I>>* AbstractWriteLog<I>::make_flush_req(Context *on_finish) {
+ utime_t flush_begins = ceph_clock_now();
+ bufferlist bl;
+ auto *flush_req =
+ new C_FlushRequestT(*this, flush_begins, Extents({whole_volume_extent()}),
+ std::move(bl), 0, m_lock, m_perfcounter, on_finish);
+
+ return flush_req;
+}
+
+template <typename I>
+void AbstractWriteLog<I>::wake_up() {
+ CephContext *cct = m_image_ctx.cct;
+ ceph_assert(ceph_mutex_is_locked_by_me(m_lock));
+
+ if (!m_wake_up_enabled) {
+ // wake_up is disabled during shutdown after flushing completes
+ ldout(m_image_ctx.cct, 6) << "deferred processing disabled" << dendl;
+ return;
+ }
+
+ if (m_wake_up_requested && m_wake_up_scheduled) {
+ return;
+ }
+
+ ldout(cct, 20) << dendl;
+
+ /* Wake-up can be requested while it's already scheduled */
+ m_wake_up_requested = true;
+
+ /* Wake-up cannot be scheduled if it's already scheduled */
+ if (m_wake_up_scheduled) {
+ return;
+ }
+ m_wake_up_scheduled = true;
+ m_async_process_work++;
+ m_async_op_tracker.start_op();
+ m_work_queue.queue(new LambdaContext(
+ [this](int r) {
+ process_work();
+ m_async_op_tracker.finish_op();
+ m_async_process_work--;
+ }), 0);
+}
+
+template <typename I>
+bool AbstractWriteLog<I>::can_flush_entry(std::shared_ptr<GenericLogEntry> log_entry) {
+ CephContext *cct = m_image_ctx.cct;
+
+ ldout(cct, 20) << "" << dendl;
+ ceph_assert(ceph_mutex_is_locked_by_me(m_lock));
+
+ if (m_invalidating) {
+ return true;
+ }
+
+ /* For OWB we can flush entries with the same sync gen number (write between
+ * aio_flush() calls) concurrently. Here we'll consider an entry flushable if
+ * its sync gen number is <= the lowest sync gen number carried by all the
+ * entries currently flushing.
+ *
+ * If the entry considered here bears a sync gen number lower than a
+ * previously flushed entry, the application had to have submitted the write
+ * bearing the higher gen number before the write with the lower gen number
+ * completed. So, flushing these concurrently is OK.
+ *
+ * If the entry considered here bears a sync gen number higher than a
+ * currently flushing entry, the write with the lower gen number may have
+ * completed to the application before the write with the higher sync gen
+ * number was submitted, and the application may rely on that completion
+ * order for volume consistency. In this case the entry will not be
+ * considered flushable until all the entries bearing lower sync gen numbers
+ * finish flushing.
+ */
+
+ if (m_flush_ops_in_flight &&
+ (log_entry->ram_entry.sync_gen_number > m_lowest_flushing_sync_gen)) {
+ return false;
+ }
+
+ return (log_entry->can_writeback() &&
+ (m_flush_ops_in_flight <= IN_FLIGHT_FLUSH_WRITE_LIMIT) &&
+ (m_flush_bytes_in_flight <= IN_FLIGHT_FLUSH_BYTES_LIMIT));
+}
+
+template <typename I>
+void AbstractWriteLog<I>::detain_flush_guard_request(std::shared_ptr<GenericLogEntry> log_entry,
+ GuardedRequestFunctionContext *guarded_ctx) {
+ ldout(m_image_ctx.cct, 20) << dendl;
+
+ BlockExtent extent;
+ if (log_entry->is_sync_point()) {
+ extent = block_extent(whole_volume_extent());
+ } else {
+ extent = log_entry->ram_entry.block_extent();
+ }
+
+ auto req = GuardedRequest(extent, guarded_ctx, false);
+ BlockGuardCell *cell = nullptr;
+
+ {
+ std::lock_guard locker(m_flush_guard_lock);
+ m_flush_guard.detain(req.block_extent, &req, &cell);
+ }
+ if (cell) {
+ req.guard_ctx->cell = cell;
+ m_image_ctx.op_work_queue->queue(req.guard_ctx, 0);
+ }
+}
+
+template <typename I>
+Context* AbstractWriteLog<I>::construct_flush_entry(std::shared_ptr<GenericLogEntry> log_entry,
+ bool invalidating) {
+ ldout(m_image_ctx.cct, 20) << "" << dendl;
+
+ /* Flush write completion action */
+ utime_t writeback_start_time = ceph_clock_now();
+ Context *ctx = new LambdaContext(
+ [this, log_entry, writeback_start_time, invalidating](int r) {
+ utime_t writeback_comp_time = ceph_clock_now();
+ m_perfcounter->tinc(l_librbd_pwl_writeback_latency,
+ writeback_comp_time - writeback_start_time);
+ {
+ std::lock_guard locker(m_lock);
+ if (r < 0) {
+ lderr(m_image_ctx.cct) << "failed to flush log entry"
+ << cpp_strerror(r) << dendl;
+ m_dirty_log_entries.push_front(log_entry);
+ } else {
+ ceph_assert(m_bytes_dirty >= log_entry->bytes_dirty());
+ log_entry->set_flushed(true);
+ m_bytes_dirty -= log_entry->bytes_dirty();
+ sync_point_writer_flushed(log_entry->get_sync_point_entry());
+ ldout(m_image_ctx.cct, 20) << "flushed: " << log_entry
+ << " invalidating=" << invalidating
+ << dendl;
+ }
+ m_flush_ops_in_flight -= 1;
+ m_flush_bytes_in_flight -= log_entry->ram_entry.write_bytes;
+ wake_up();
+ }
+ });
+ /* Flush through lower cache before completing */
+ ctx = new LambdaContext(
+ [this, ctx, log_entry](int r) {
+ {
+
+ WriteLogGuard::BlockOperations block_reqs;
+ BlockGuardCell *detained_cell = nullptr;
+
+ std::lock_guard locker{m_flush_guard_lock};
+ m_flush_guard.release(log_entry->m_cell, &block_reqs);
+
+ for (auto &req : block_reqs) {
+ m_flush_guard.detain(req.block_extent, &req, &detained_cell);
+ if (detained_cell) {
+ req.guard_ctx->cell = detained_cell;
+ m_image_ctx.op_work_queue->queue(req.guard_ctx, 0);
+ }
+ }
+ }
+
+ if (r < 0) {
+ lderr(m_image_ctx.cct) << "failed to flush log entry"
+ << cpp_strerror(r) << dendl;
+ ctx->complete(r);
+ } else {
+ m_image_writeback.aio_flush(io::FLUSH_SOURCE_WRITEBACK, ctx);
+ }
+ });
+ return ctx;
+}
+
+template <typename I>
+void AbstractWriteLog<I>::process_writeback_dirty_entries() {
+ CephContext *cct = m_image_ctx.cct;
+ bool all_clean = false;
+ int flushed = 0;
+ bool has_write_entry = false;
+ bool need_update_state = false;
+
+ ldout(cct, 20) << "Look for dirty entries" << dendl;
+ {
+ DeferredContexts post_unlock;
+ GenericLogEntries entries_to_flush;
+
+ std::shared_lock entry_reader_locker(m_entry_reader_lock);
+ std::lock_guard locker(m_lock);
+ while (flushed < IN_FLIGHT_FLUSH_WRITE_LIMIT) {
+ if (m_shutting_down) {
+ ldout(cct, 5) << "Flush during shutdown supressed" << dendl;
+ /* Do flush complete only when all flush ops are finished */
+ all_clean = !m_flush_ops_in_flight;
+ break;
+ }
+ if (m_dirty_log_entries.empty()) {
+ ldout(cct, 20) << "Nothing new to flush" << dendl;
+ /* Do flush complete only when all flush ops are finished */
+ all_clean = !m_flush_ops_in_flight;
+ if (!m_cache_state->clean && all_clean) {
+ m_cache_state->clean = true;
+ update_image_cache_state();
+ need_update_state = true;
+ }
+ break;
+ }
+
+ auto candidate = m_dirty_log_entries.front();
+ bool flushable = can_flush_entry(candidate);
+ if (flushable) {
+ entries_to_flush.push_back(candidate);
+ flushed++;
+ if (!has_write_entry)
+ has_write_entry = candidate->is_write_entry();
+ m_dirty_log_entries.pop_front();
+
+ // To track candidate, we should add m_flush_ops_in_flight in here
+ {
+ if (!m_flush_ops_in_flight ||
+ (candidate->ram_entry.sync_gen_number < m_lowest_flushing_sync_gen)) {
+ m_lowest_flushing_sync_gen = candidate->ram_entry.sync_gen_number;
+ }
+ m_flush_ops_in_flight += 1;
+ /* For write same this is the bytes affected by the flush op, not the bytes transferred */
+ m_flush_bytes_in_flight += candidate->ram_entry.write_bytes;
+ }
+ } else {
+ ldout(cct, 20) << "Next dirty entry isn't flushable yet" << dendl;
+ break;
+ }
+ }
+
+ construct_flush_entries(entries_to_flush, post_unlock, has_write_entry);
+ }
+ if (need_update_state) {
+ std::unique_lock locker(m_lock);
+ write_image_cache_state(locker);
+ }
+
+ if (all_clean) {
+ /* All flushing complete, drain outside lock */
+ Contexts flush_contexts;
+ {
+ std::lock_guard locker(m_lock);
+ flush_contexts.swap(m_flush_complete_contexts);
+ }
+ finish_contexts(m_image_ctx.cct, flush_contexts, 0);
+ }
+}
+
+/* Returns true if the specified SyncPointLogEntry is considered flushed, and
+ * the log will be updated to reflect this. */
+template <typename I>
+bool AbstractWriteLog<I>::handle_flushed_sync_point(std::shared_ptr<SyncPointLogEntry> log_entry)
+{
+ ceph_assert(ceph_mutex_is_locked_by_me(m_lock));
+ ceph_assert(log_entry);
+
+ if ((log_entry->writes_flushed == log_entry->writes) &&
+ log_entry->completed && log_entry->prior_sync_point_flushed &&
+ log_entry->next_sync_point_entry) {
+ ldout(m_image_ctx.cct, 20) << "All writes flushed up to sync point="
+ << *log_entry << dendl;
+ log_entry->next_sync_point_entry->prior_sync_point_flushed = true;
+ /* Don't move the flushed sync gen num backwards. */
+ if (m_flushed_sync_gen < log_entry->ram_entry.sync_gen_number) {
+ m_flushed_sync_gen = log_entry->ram_entry.sync_gen_number;
+ }
+ m_async_op_tracker.start_op();
+ m_work_queue.queue(new LambdaContext(
+ [this, next = std::move(log_entry->next_sync_point_entry)](int r) {
+ bool handled_by_next;
+ {
+ std::lock_guard locker(m_lock);
+ handled_by_next = handle_flushed_sync_point(std::move(next));
+ }
+ if (!handled_by_next) {
+ persist_last_flushed_sync_gen();
+ }
+ m_async_op_tracker.finish_op();
+ }));
+ return true;
+ }
+ return false;
+}
+
+template <typename I>
+void AbstractWriteLog<I>::sync_point_writer_flushed(std::shared_ptr<SyncPointLogEntry> log_entry)
+{
+ ceph_assert(ceph_mutex_is_locked_by_me(m_lock));
+ ceph_assert(log_entry);
+ log_entry->writes_flushed++;
+
+ /* If this entry might be completely flushed, look closer */
+ if ((log_entry->writes_flushed == log_entry->writes) && log_entry->completed) {
+ ldout(m_image_ctx.cct, 15) << "All writes flushed for sync point="
+ << *log_entry << dendl;
+ handle_flushed_sync_point(log_entry);
+ }
+}
+
+/* Make a new sync point and flush the previous during initialization, when there may or may
+ * not be a previous sync point */
+template <typename I>
+void AbstractWriteLog<I>::init_flush_new_sync_point(DeferredContexts &later) {
+ ceph_assert(ceph_mutex_is_locked_by_me(m_lock));
+ ceph_assert(!m_initialized); /* Don't use this after init */
+
+ if (!m_current_sync_point) {
+ /* First sync point since start */
+ new_sync_point(later);
+ } else {
+ flush_new_sync_point(nullptr, later);
+ }
+}
+
+/**
+ * Begin a new sync point
+ */
+template <typename I>
+void AbstractWriteLog<I>::new_sync_point(DeferredContexts &later) {
+ CephContext *cct = m_image_ctx.cct;
+ std::shared_ptr<SyncPoint> old_sync_point = m_current_sync_point;
+ std::shared_ptr<SyncPoint> new_sync_point;
+ ldout(cct, 20) << dendl;
+
+ ceph_assert(ceph_mutex_is_locked_by_me(m_lock));
+
+ /* The first time this is called, if this is a newly created log,
+ * this makes the first sync gen number we'll use 1. On the first
+ * call for a re-opened log m_current_sync_gen will be the highest
+ * gen number from all the sync point entries found in the re-opened
+ * log, and this advances to the next sync gen number. */
+ ++m_current_sync_gen;
+
+ new_sync_point = std::make_shared<SyncPoint>(m_current_sync_gen, cct);
+ m_current_sync_point = new_sync_point;
+
+ /* If this log has been re-opened, old_sync_point will initially be
+ * nullptr, but m_current_sync_gen may not be zero. */
+ if (old_sync_point) {
+ new_sync_point->setup_earlier_sync_point(old_sync_point, m_last_op_sequence_num);
+ m_perfcounter->hinc(l_librbd_pwl_syncpoint_hist,
+ old_sync_point->log_entry->writes,
+ old_sync_point->log_entry->bytes);
+ /* This sync point will acquire no more sub-ops. Activation needs
+ * to acquire m_lock, so defer to later*/
+ later.add(new LambdaContext(
+ [old_sync_point](int r) {
+ old_sync_point->prior_persisted_gather_activate();
+ }));
+ }
+
+ new_sync_point->prior_persisted_gather_set_finisher();
+
+ if (old_sync_point) {
+ ldout(cct,6) << "new sync point = [" << *m_current_sync_point
+ << "], prior = [" << *old_sync_point << "]" << dendl;
+ } else {
+ ldout(cct,6) << "first sync point = [" << *m_current_sync_point
+ << "]" << dendl;
+ }
+}
+
+template <typename I>
+void AbstractWriteLog<I>::flush_new_sync_point(C_FlushRequestT *flush_req,
+ DeferredContexts &later) {
+ ceph_assert(ceph_mutex_is_locked_by_me(m_lock));
+
+ if (!flush_req) {
+ m_async_null_flush_finish++;
+ m_async_op_tracker.start_op();
+ Context *flush_ctx = new LambdaContext([this](int r) {
+ m_async_null_flush_finish--;
+ m_async_op_tracker.finish_op();
+ });
+ flush_req = make_flush_req(flush_ctx);
+ flush_req->internal = true;
+ }
+
+ /* Add a new sync point. */
+ new_sync_point(later);
+ std::shared_ptr<SyncPoint> to_append = m_current_sync_point->earlier_sync_point;
+ ceph_assert(to_append);
+
+ /* This flush request will append/persist the (now) previous sync point */
+ flush_req->to_append = to_append;
+
+ /* When the m_sync_point_persist Gather completes this sync point can be
+ * appended. The only sub for this Gather is the finisher Context for
+ * m_prior_log_entries_persisted, which records the result of the Gather in
+ * the sync point, and completes. TODO: Do we still need both of these
+ * Gathers?*/
+ Context * ctx = new LambdaContext([this, flush_req](int r) {
+ ldout(m_image_ctx.cct, 20) << "Flush req=" << flush_req
+ << " sync point =" << flush_req->to_append
+ << ". Ready to persist." << dendl;
+ alloc_and_dispatch_io_req(flush_req);
+ });
+ to_append->persist_gather_set_finisher(ctx);
+
+ /* The m_sync_point_persist Gather has all the subs it will ever have, and
+ * now has its finisher. If the sub is already complete, activation will
+ * complete the Gather. The finisher will acquire m_lock, so we'll activate
+ * this when we release m_lock.*/
+ later.add(new LambdaContext([to_append](int r) {
+ to_append->persist_gather_activate();
+ }));
+
+ /* The flush request completes when the sync point persists */
+ to_append->add_in_on_persisted_ctxs(flush_req);
+}
+
+template <typename I>
+void AbstractWriteLog<I>::flush_new_sync_point_if_needed(C_FlushRequestT *flush_req,
+ DeferredContexts &later) {
+ ceph_assert(ceph_mutex_is_locked_by_me(m_lock));
+
+ /* If there have been writes since the last sync point ... */
+ if (m_current_sync_point->log_entry->writes) {
+ flush_new_sync_point(flush_req, later);
+ } else {
+ /* There have been no writes to the current sync point. */
+ if (m_current_sync_point->earlier_sync_point) {
+ /* If previous sync point hasn't completed, complete this flush
+ * with the earlier sync point. No alloc or dispatch needed. */
+ m_current_sync_point->earlier_sync_point->on_sync_point_persisted.push_back(flush_req);
+ } else {
+ /* The previous sync point has already completed and been
+ * appended. The current sync point has no writes, so this flush
+ * has nothing to wait for. This flush completes now. */
+ later.add(flush_req);
+ }
+ }
+}
+
+/*
+ * RWL internal flush - will actually flush the RWL.
+ *
+ * User flushes should arrive at aio_flush(), and only flush prior
+ * writes to all log replicas.
+ *
+ * Librbd internal flushes will arrive at flush(invalidate=false,
+ * discard=false), and traverse the block guard to ensure in-flight writes are
+ * flushed.
+ */
+template <typename I>
+void AbstractWriteLog<I>::flush_dirty_entries(Context *on_finish) {
+ CephContext *cct = m_image_ctx.cct;
+ bool all_clean;
+ bool flushing;
+ bool stop_flushing;
+
+ {
+ std::unique_lock locker(m_lock);
+ flushing = (0 != m_flush_ops_in_flight);
+ all_clean = m_dirty_log_entries.empty();
+ stop_flushing = (m_shutting_down);
+ if (!m_cache_state->clean && all_clean && !flushing) {
+ m_cache_state->clean = true;
+ update_image_cache_state();
+ write_image_cache_state(locker);
+ }
+ }
+
+ if (!flushing && (all_clean || stop_flushing)) {
+ /* Complete without holding m_lock */
+ if (all_clean) {
+ ldout(cct, 20) << "no dirty entries" << dendl;
+ } else {
+ ldout(cct, 5) << "flush during shutdown suppressed" << dendl;
+ }
+ on_finish->complete(0);
+ } else {
+ if (all_clean) {
+ ldout(cct, 5) << "flush ops still in progress" << dendl;
+ } else {
+ ldout(cct, 20) << "dirty entries remain" << dendl;
+ }
+ std::lock_guard locker(m_lock);
+ /* on_finish can't be completed yet */
+ m_flush_complete_contexts.push_back(new LambdaContext(
+ [this, on_finish](int r) {
+ flush_dirty_entries(on_finish);
+ }));
+ wake_up();
+ }
+}
+
+template <typename I>
+void AbstractWriteLog<I>::internal_flush(bool invalidate, Context *on_finish) {
+ ldout(m_image_ctx.cct, 20) << "invalidate=" << invalidate << dendl;
+
+ if (m_perfcounter) {
+ if (invalidate) {
+ m_perfcounter->inc(l_librbd_pwl_invalidate_cache, 1);
+ } else {
+ m_perfcounter->inc(l_librbd_pwl_internal_flush, 1);
+ }
+ }
+
+ /* May be called even if initialization fails */
+ if (!m_initialized) {
+ ldout(m_image_ctx.cct, 05) << "never initialized" << dendl;
+ /* Deadlock if completed here */
+ m_image_ctx.op_work_queue->queue(on_finish, 0);
+ return;
+ }
+
+ /* Flush/invalidate must pass through block guard to ensure all layers of
+ * cache are consistently flush/invalidated. This ensures no in-flight write leaves
+ * some layers with valid regions, which may later produce inconsistent read
+ * results. */
+ GuardedRequestFunctionContext *guarded_ctx =
+ new GuardedRequestFunctionContext(
+ [this, on_finish, invalidate](GuardedRequestFunctionContext &guard_ctx) {
+ DeferredContexts on_exit;
+ ldout(m_image_ctx.cct, 20) << "cell=" << guard_ctx.cell << dendl;
+ ceph_assert(guard_ctx.cell);
+
+ Context *ctx = new LambdaContext(
+ [this, cell=guard_ctx.cell, invalidate, on_finish](int r) {
+ std::lock_guard locker(m_lock);
+ m_invalidating = false;
+ ldout(m_image_ctx.cct, 6) << "Done flush/invalidating (invalidate="
+ << invalidate << ")" << dendl;
+ if (m_log_entries.size()) {
+ ldout(m_image_ctx.cct, 1) << "m_log_entries.size()="
+ << m_log_entries.size() << ", "
+ << "front()=" << *m_log_entries.front()
+ << dendl;
+ }
+ if (invalidate) {
+ ceph_assert(m_log_entries.size() == 0);
+ }
+ ceph_assert(m_dirty_log_entries.size() == 0);
+ m_image_ctx.op_work_queue->queue(on_finish, r);
+ release_guarded_request(cell);
+ });
+ ctx = new LambdaContext(
+ [this, ctx, invalidate](int r) {
+ Context *next_ctx = ctx;
+ ldout(m_image_ctx.cct, 6) << "flush_dirty_entries finished" << dendl;
+ if (r < 0) {
+ /* Override on_finish status with this error */
+ next_ctx = new LambdaContext([r, ctx](int _r) {
+ ctx->complete(r);
+ });
+ }
+ if (invalidate) {
+ {
+ std::lock_guard locker(m_lock);
+ ceph_assert(m_dirty_log_entries.size() == 0);
+ ceph_assert(!m_invalidating);
+ ldout(m_image_ctx.cct, 6) << "Invalidating" << dendl;
+ m_invalidating = true;
+ }
+ /* Discards all RWL entries */
+ while (retire_entries(MAX_ALLOC_PER_TRANSACTION)) { }
+ next_ctx->complete(0);
+ } else {
+ {
+ std::lock_guard locker(m_lock);
+ ceph_assert(m_dirty_log_entries.size() == 0);
+ ceph_assert(!m_invalidating);
+ }
+ m_image_writeback.aio_flush(io::FLUSH_SOURCE_WRITEBACK, next_ctx);
+ }
+ });
+ ctx = new LambdaContext(
+ [this, ctx](int r) {
+ flush_dirty_entries(ctx);
+ });
+ std::lock_guard locker(m_lock);
+ /* Even if we're throwing everything away, but we want the last entry to
+ * be a sync point so we can cleanly resume.
+ *
+ * Also, the blockguard only guarantees the replication of this op
+ * can't overlap with prior ops. It doesn't guarantee those are all
+ * completed and eligible for flush & retire, which we require here.
+ */
+ auto flush_req = make_flush_req(ctx);
+ flush_new_sync_point_if_needed(flush_req, on_exit);
+ });
+ detain_guarded_request(nullptr, guarded_ctx, true);
+}
+
+template <typename I>
+void AbstractWriteLog<I>::add_into_log_map(GenericWriteLogEntries &log_entries,
+ C_BlockIORequestT *req) {
+ req->copy_cache();
+ m_blocks_to_log_entries.add_log_entries(log_entries);
+}
+
+template <typename I>
+bool AbstractWriteLog<I>::can_retire_entry(std::shared_ptr<GenericLogEntry> log_entry) {
+ CephContext *cct = m_image_ctx.cct;
+
+ ldout(cct, 20) << dendl;
+ ceph_assert(ceph_mutex_is_locked_by_me(m_lock));
+ return log_entry->can_retire();
+}
+
+template <typename I>
+void AbstractWriteLog<I>::check_image_cache_state_clean() {
+ ceph_assert(m_deferred_ios.empty());
+ ceph_assert(m_ops_to_append.empty());;
+ ceph_assert(m_async_flush_ops == 0);
+ ceph_assert(m_async_append_ops == 0);
+ ceph_assert(m_dirty_log_entries.empty());
+ ceph_assert(m_ops_to_flush.empty());
+ ceph_assert(m_flush_ops_in_flight == 0);
+ ceph_assert(m_flush_bytes_in_flight == 0);
+ ceph_assert(m_bytes_dirty == 0);
+ ceph_assert(m_work_queue.empty());
+}
+
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+template class librbd::cache::pwl::AbstractWriteLog<librbd::ImageCtx>;
diff --git a/src/librbd/cache/pwl/AbstractWriteLog.h b/src/librbd/cache/pwl/AbstractWriteLog.h
new file mode 100644
index 000000000..ffe299c37
--- /dev/null
+++ b/src/librbd/cache/pwl/AbstractWriteLog.h
@@ -0,0 +1,410 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_CACHE_PARENT_WRITE_LOG
+#define CEPH_LIBRBD_CACHE_PARENT_WRITE_LOG
+
+#include "common/Timer.h"
+#include "common/RWLock.h"
+#include "common/WorkQueue.h"
+#include "common/AsyncOpTracker.h"
+#include "librbd/cache/ImageWriteback.h"
+#include "librbd/Utils.h"
+#include "librbd/BlockGuard.h"
+#include "librbd/cache/Types.h"
+#include "librbd/cache/pwl/LogOperation.h"
+#include "librbd/cache/pwl/ReadRequest.h"
+#include "librbd/cache/pwl/Request.h"
+#include "librbd/cache/pwl/LogMap.h"
+#include "librbd/cache/pwl/Builder.h"
+#include <functional>
+#include <list>
+
+class Context;
+
+namespace librbd {
+
+struct ImageCtx;
+
+namespace plugin { template <typename> struct Api; }
+
+namespace cache {
+namespace pwl {
+
+class GenericLogEntry;
+class GenericWriteLogEntry;
+class SyncPointLogEntry;
+class WriteLogEntry;
+struct WriteLogCacheEntry;
+
+typedef std::list<std::shared_ptr<WriteLogEntry>> WriteLogEntries;
+typedef std::list<std::shared_ptr<GenericLogEntry>> GenericLogEntries;
+typedef std::list<std::shared_ptr<GenericWriteLogEntry>> GenericWriteLogEntries;
+typedef std::vector<std::shared_ptr<GenericLogEntry>> GenericLogEntriesVector;
+
+typedef LogMapEntries<GenericWriteLogEntry> WriteLogMapEntries;
+typedef LogMap<GenericWriteLogEntry> WriteLogMap;
+
+/**** Write log entries end ****/
+
+typedef librbd::BlockGuard<GuardedRequest> WriteLogGuard;
+
+class DeferredContexts;
+template <typename>
+class ImageCacheState;
+
+template<typename T>
+class Builder;
+
+template <typename T>
+struct C_BlockIORequest;
+
+template <typename T>
+struct C_WriteRequest;
+
+using GenericLogOperations = std::list<GenericLogOperationSharedPtr>;
+
+
+template <typename ImageCtxT>
+class AbstractWriteLog {
+public:
+ typedef io::Extent Extent;
+ typedef io::Extents Extents;
+ using This = AbstractWriteLog<ImageCtxT>;
+ Builder<This> *m_builder;
+
+ AbstractWriteLog(ImageCtxT &image_ctx,
+ librbd::cache::pwl::ImageCacheState<ImageCtxT>* cache_state,
+ Builder<This> *builder,
+ cache::ImageWritebackInterface& image_writeback,
+ plugin::Api<ImageCtxT>& plugin_api);
+ virtual ~AbstractWriteLog();
+ AbstractWriteLog(const AbstractWriteLog&) = delete;
+ AbstractWriteLog &operator=(const AbstractWriteLog&) = delete;
+
+ /// IO methods
+ void read(
+ Extents&& image_extents, ceph::bufferlist *bl,
+ int fadvise_flags, Context *on_finish);
+ void write(
+ Extents&& image_extents, ceph::bufferlist&& bl,
+ int fadvise_flags,
+ Context *on_finish);
+ void discard(
+ uint64_t offset, uint64_t length,
+ uint32_t discard_granularity_bytes,
+ Context *on_finish);
+ void flush(
+ io::FlushSource flush_source, Context *on_finish);
+ void writesame(
+ uint64_t offset, uint64_t length,
+ ceph::bufferlist&& bl,
+ int fadvise_flags, Context *on_finish);
+ void 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);
+
+ using C_WriteRequestT = pwl::C_WriteRequest<This>;
+ using C_BlockIORequestT = pwl::C_BlockIORequest<This>;
+ using C_FlushRequestT = pwl::C_FlushRequest<This>;
+ using C_DiscardRequestT = pwl::C_DiscardRequest<This>;
+ using C_WriteSameRequestT = pwl::C_WriteSameRequest<This>;
+
+ CephContext * get_context();
+ void release_guarded_request(BlockGuardCell *cell);
+ void release_write_lanes(C_BlockIORequestT *req);
+ virtual bool alloc_resources(C_BlockIORequestT *req) = 0;
+ virtual void setup_schedule_append(
+ pwl::GenericLogOperationsVector &ops, bool do_early_flush,
+ C_BlockIORequestT *req) = 0;
+ void schedule_append(pwl::GenericLogOperationsVector &ops, C_BlockIORequestT *req = nullptr);
+ void schedule_append(pwl::GenericLogOperationSharedPtr op, C_BlockIORequestT *req = nullptr);
+ void flush_new_sync_point(C_FlushRequestT *flush_req,
+ pwl::DeferredContexts &later);
+
+ std::shared_ptr<pwl::SyncPoint> get_current_sync_point() {
+ return m_current_sync_point;
+ }
+ bool get_persist_on_flush() {
+ return m_persist_on_flush;
+ }
+ void inc_last_op_sequence_num() {
+ m_perfcounter->inc(l_librbd_pwl_log_ops, 1);
+ ++m_last_op_sequence_num;
+ }
+ uint64_t get_last_op_sequence_num() {
+ return m_last_op_sequence_num;
+ }
+ uint64_t get_current_sync_gen() {
+ return m_current_sync_gen;
+ }
+ unsigned int get_free_lanes() {
+ return m_free_lanes;
+ }
+ uint32_t get_free_log_entries() {
+ return m_free_log_entries;
+ }
+ void add_into_log_map(pwl::GenericWriteLogEntries &log_entries,
+ C_BlockIORequestT *req);
+ virtual void complete_user_request(Context *&user_req, int r) = 0;
+ virtual void copy_bl_to_buffer(
+ WriteRequestResources *resources,
+ std::unique_ptr<WriteLogOperationSet> &op_set) {}
+
+private:
+ typedef std::list<pwl::C_WriteRequest<This> *> C_WriteRequests;
+ typedef std::list<pwl::C_BlockIORequest<This> *> C_BlockIORequests;
+
+ std::atomic<bool> m_initialized = {false};
+
+ uint64_t m_bytes_dirty = 0; /* Total bytes yet to flush to RBD */
+ utime_t m_last_alloc_fail; /* Entry or buffer allocation fail seen */
+
+ pwl::WriteLogGuard m_write_log_guard;
+
+ /* Starts at 0 for a new write log. Incremented on every flush. */
+ uint64_t m_current_sync_gen = 0;
+ /* Starts at 0 on each sync gen increase. Incremented before applied
+ to an operation */
+ uint64_t m_last_op_sequence_num = 0;
+
+ bool m_persist_on_write_until_flush = true;
+
+ pwl::WriteLogGuard m_flush_guard;
+ mutable ceph::mutex m_flush_guard_lock;
+
+ /* Debug counters for the places m_async_op_tracker is used */
+ std::atomic<int> m_async_complete_ops = {0};
+ std::atomic<int> m_async_null_flush_finish = {0};
+ std::atomic<int> m_async_process_work = {0};
+
+ /* Hold m_deferred_dispatch_lock while consuming from m_deferred_ios. */
+ mutable ceph::mutex m_deferred_dispatch_lock;
+
+ /* Used in release/detain to make BlockGuard preserve submission order */
+ mutable ceph::mutex m_blockguard_lock;
+
+ /* Use m_blockguard_lock for the following 3 things */
+ bool m_barrier_in_progress = false;
+ BlockGuardCell *m_barrier_cell = nullptr;
+
+ bool m_wake_up_enabled = true;
+
+ Contexts m_flush_complete_contexts;
+
+ std::shared_ptr<pwl::SyncPoint> m_current_sync_point = nullptr;
+ bool m_persist_on_flush = false; //If false, persist each write before completion
+
+ int m_flush_ops_in_flight = 0;
+ int m_flush_bytes_in_flight = 0;
+ uint64_t m_lowest_flushing_sync_gen = 0;
+
+ /* Writes that have left the block guard, but are waiting for resources */
+ C_BlockIORequests m_deferred_ios;
+ /* Throttle writes concurrently allocating & replicating */
+ unsigned int m_free_lanes = pwl::MAX_CONCURRENT_WRITES;
+
+ SafeTimer *m_timer = nullptr; /* Used with m_timer_lock */
+ mutable ceph::mutex *m_timer_lock = nullptr; /* Used with and by m_timer */
+ Context *m_timer_ctx = nullptr;
+
+ ThreadPool m_thread_pool;
+
+ uint32_t m_discard_granularity_bytes;
+
+ BlockGuardCell* detain_guarded_request_helper(pwl::GuardedRequest &req);
+ BlockGuardCell* detain_guarded_request_barrier_helper(
+ pwl::GuardedRequest &req);
+ void detain_guarded_request(C_BlockIORequestT *request,
+ pwl::GuardedRequestFunctionContext *guarded_ctx,
+ bool is_barrier);
+ void perf_start(const std::string name);
+ void perf_stop();
+ void log_perf();
+ void periodic_stats();
+ void arm_periodic_stats();
+
+ void pwl_init(Context *on_finish, pwl::DeferredContexts &later);
+ void check_image_cache_state_clean();
+
+ void flush_dirty_entries(Context *on_finish);
+ bool can_flush_entry(const std::shared_ptr<pwl::GenericLogEntry> log_entry);
+ bool handle_flushed_sync_point(
+ std::shared_ptr<pwl::SyncPointLogEntry> log_entry);
+ void sync_point_writer_flushed(
+ std::shared_ptr<pwl::SyncPointLogEntry> log_entry);
+
+ void init_flush_new_sync_point(pwl::DeferredContexts &later);
+ void new_sync_point(pwl::DeferredContexts &later);
+ pwl::C_FlushRequest<AbstractWriteLog<ImageCtxT>>* make_flush_req(
+ Context *on_finish);
+ void flush_new_sync_point_if_needed(C_FlushRequestT *flush_req,
+ pwl::DeferredContexts &later);
+
+ void alloc_and_dispatch_io_req(C_BlockIORequestT *write_req);
+ void schedule_complete_op_log_entries(pwl::GenericLogOperations &&ops,
+ const int r);
+ void internal_flush(bool invalidate, Context *on_finish);
+
+protected:
+ librbd::cache::pwl::ImageCacheState<ImageCtxT>* m_cache_state = nullptr;
+
+ std::atomic<bool> m_shutting_down = {false};
+ std::atomic<bool> m_invalidating = {false};
+
+ ImageCtxT &m_image_ctx;
+
+ std::string m_log_pool_name;
+ uint64_t m_log_pool_size;
+
+ uint32_t m_total_log_entries = 0;
+ uint32_t m_free_log_entries = 0;
+
+ std::atomic<uint64_t> m_bytes_allocated = {0}; /* Total bytes allocated in write buffers */
+ uint64_t m_bytes_cached = 0; /* Total bytes used in write buffers */
+ uint64_t m_bytes_allocated_cap = 0;
+
+ std::atomic<bool> m_alloc_failed_since_retire = {false};
+
+ cache::ImageWritebackInterface& m_image_writeback;
+ plugin::Api<ImageCtxT>& m_plugin_api;
+
+ /*
+ * When m_first_free_entry == m_first_valid_entry, the log is
+ * empty. There is always at least one free entry, which can't be
+ * used.
+ */
+ uint64_t m_first_free_entry = 0; /* Entries from here to m_first_valid_entry-1 are free */
+ uint64_t m_first_valid_entry = 0; /* Entries from here to m_first_free_entry-1 are valid */
+
+ /* All writes bearing this and all prior sync gen numbers are flushed */
+ uint64_t m_flushed_sync_gen = 0;
+
+ AsyncOpTracker m_async_op_tracker;
+ /* Debug counters for the places m_async_op_tracker is used */
+ std::atomic<int> m_async_flush_ops = {0};
+ std::atomic<int> m_async_append_ops = {0};
+
+ /* Acquire locks in order declared here */
+
+ mutable ceph::mutex m_log_retire_lock;
+ /* Hold a read lock on m_entry_reader_lock to add readers to log entry
+ * bufs. Hold a write lock to prevent readers from being added (e.g. when
+ * removing log entrys from the map). No lock required to remove readers. */
+ mutable RWLock m_entry_reader_lock;
+ /* Hold m_log_append_lock while appending or retiring log entries. */
+ mutable ceph::mutex m_log_append_lock;
+ /* Used for most synchronization */
+ mutable ceph::mutex m_lock;
+
+ /* Use m_blockguard_lock for the following 3 things */
+ pwl::WriteLogGuard::BlockOperations m_awaiting_barrier;
+
+ bool m_wake_up_requested = false;
+ bool m_wake_up_scheduled = false;
+ bool m_appending = false;
+ bool m_dispatching_deferred_ops = false;
+
+ pwl::GenericLogOperations m_ops_to_flush; /* Write ops needing flush in local log */
+ pwl::GenericLogOperations m_ops_to_append; /* Write ops needing event append in local log */
+
+ pwl::WriteLogMap m_blocks_to_log_entries;
+
+ /* New entries are at the back. Oldest at the front */
+ pwl::GenericLogEntries m_log_entries;
+ pwl::GenericLogEntries m_dirty_log_entries;
+
+ PerfCounters *m_perfcounter = nullptr;
+
+ unsigned int m_unpublished_reserves = 0;
+
+ ContextWQ m_work_queue;
+
+ void wake_up();
+
+ void update_entries(
+ std::shared_ptr<pwl::GenericLogEntry> *log_entry,
+ pwl::WriteLogCacheEntry *cache_entry,
+ std::map<uint64_t, bool> &missing_sync_points,
+ std::map<uint64_t,
+ std::shared_ptr<pwl::SyncPointLogEntry>> &sync_point_entries,
+ uint64_t entry_index);
+ void update_sync_points(
+ std::map<uint64_t, bool> &missing_sync_points,
+ std::map<uint64_t,
+ std::shared_ptr<pwl::SyncPointLogEntry>> &sync_point_entries,
+ pwl::DeferredContexts &later);
+ virtual void inc_allocated_cached_bytes(
+ std::shared_ptr<pwl::GenericLogEntry> log_entry) = 0;
+ Context *construct_flush_entry(
+ const std::shared_ptr<pwl::GenericLogEntry> log_entry, bool invalidating);
+ void detain_flush_guard_request(std::shared_ptr<GenericLogEntry> log_entry,
+ GuardedRequestFunctionContext *guarded_ctx);
+ void process_writeback_dirty_entries();
+ bool can_retire_entry(const std::shared_ptr<pwl::GenericLogEntry> log_entry);
+
+ void dispatch_deferred_writes(void);
+ void complete_op_log_entries(pwl::GenericLogOperations &&ops, const int r);
+
+ bool check_allocation(
+ C_BlockIORequestT *req, uint64_t bytes_cached, uint64_t bytes_dirtied,
+ uint64_t bytes_allocated, uint32_t num_lanes, uint32_t num_log_entries,
+ uint32_t num_unpublished_reserves);
+ void append_scheduled(
+ pwl::GenericLogOperations &ops, bool &ops_remain, bool &appending,
+ bool isRWL=false);
+
+ virtual void process_work() = 0;
+ virtual void append_scheduled_ops(void) = 0;
+ virtual void schedule_append_ops(pwl::GenericLogOperations &ops, C_BlockIORequestT *req) = 0;
+ virtual void remove_pool_file() = 0;
+ virtual bool initialize_pool(Context *on_finish,
+ pwl::DeferredContexts &later) = 0;
+ virtual void collect_read_extents(
+ uint64_t read_buffer_offset, LogMapEntry<GenericWriteLogEntry> map_entry,
+ std::vector<std::shared_ptr<GenericWriteLogEntry>> &log_entries_to_read,
+ std::vector<bufferlist*> &bls_to_read, uint64_t entry_hit_length,
+ Extent hit_extent, pwl::C_ReadRequest *read_ctx) = 0;
+ virtual void complete_read(
+ std::vector<std::shared_ptr<GenericWriteLogEntry>> &log_entries_to_read,
+ std::vector<bufferlist*> &bls_to_read, Context *ctx) = 0;
+ virtual void write_data_to_buffer(
+ std::shared_ptr<pwl::WriteLogEntry> ws_entry,
+ pwl::WriteLogCacheEntry *cache_entry) {}
+ virtual void release_ram(
+ const std::shared_ptr<pwl::GenericLogEntry> log_entry) {}
+ virtual void alloc_op_log_entries(pwl::GenericLogOperations &ops) {}
+ virtual bool retire_entries(const unsigned long int frees_per_tx) {
+ return false;
+ }
+ virtual void schedule_flush_and_append(
+ pwl::GenericLogOperationsVector &ops) {}
+ virtual void persist_last_flushed_sync_gen() {}
+ virtual void reserve_cache(C_BlockIORequestT *req, bool &alloc_succeeds,
+ bool &no_space) {}
+ virtual void construct_flush_entries(pwl::GenericLogEntries entries_to_flush,
+ DeferredContexts &post_unlock,
+ bool has_write_entry) = 0;
+ virtual uint64_t get_max_extent() {
+ return 0;
+ }
+ void update_image_cache_state(void);
+ void write_image_cache_state(std::unique_lock<ceph::mutex>& locker);
+ void handle_write_image_cache_state(int r);
+};
+
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+extern template class librbd::cache::pwl::AbstractWriteLog<librbd::ImageCtx>;
+
+#endif // CEPH_LIBRBD_CACHE_PARENT_WRITE_LOG
diff --git a/src/librbd/cache/pwl/Builder.h b/src/librbd/cache/pwl/Builder.h
new file mode 100644
index 000000000..9db28ea68
--- /dev/null
+++ b/src/librbd/cache/pwl/Builder.h
@@ -0,0 +1,61 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_CACHE_PWL_BUILDER_H
+#define CEPH_LIBRBD_CACHE_PWL_BUILDER_H
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+
+template <typename T>
+class Builder {
+public:
+ virtual ~Builder() {}
+ virtual std::shared_ptr<WriteLogEntry> create_write_log_entry(
+ uint64_t image_offset_bytes, uint64_t write_bytes) = 0;
+ virtual std::shared_ptr<WriteLogEntry> create_write_log_entry(
+ std::shared_ptr<SyncPointLogEntry> sync_point_entry,
+ uint64_t image_offset_bytes, uint64_t write_bytes) = 0;
+ virtual std::shared_ptr<WriteLogEntry> create_writesame_log_entry(
+ uint64_t image_offset_bytes, uint64_t write_bytes,
+ uint32_t data_length) = 0;
+ virtual std::shared_ptr<WriteLogEntry> create_writesame_log_entry(
+ std::shared_ptr<SyncPointLogEntry> sync_point_entry,
+ uint64_t image_offset_bytes, uint64_t write_bytes,
+ uint32_t data_length) = 0;
+ virtual C_WriteRequest<T> *create_write_request(
+ T &pwl, utime_t arrived, io::Extents &&image_extents,
+ bufferlist&& bl, const int fadvise_flags, ceph::mutex &lock,
+ PerfCounters *perfcounter, Context *user_req) = 0;
+ virtual C_WriteSameRequest<T> *create_writesame_request(
+ T &pwl, utime_t arrived, io::Extents &&image_extents,
+ bufferlist&& bl, const int fadvise_flags, ceph::mutex &lock,
+ PerfCounters *perfcounter, Context *user_req) = 0;
+ virtual C_WriteRequest<T> *create_comp_and_write_request(
+ T &pwl, utime_t arrived, io::Extents &&image_extents,
+ bufferlist&& cmp_bl, bufferlist&& bl, uint64_t *mismatch_offset,
+ const int fadvise_flags, ceph::mutex &lock,
+ PerfCounters *perfcounter, Context *user_req) = 0;
+ virtual std::shared_ptr<WriteLogOperation> create_write_log_operation(
+ WriteLogOperationSet &set, uint64_t image_offset_bytes,
+ uint64_t write_bytes, CephContext *cct,
+ std::shared_ptr<WriteLogEntry> write_log_entry) = 0;
+ virtual std::shared_ptr<WriteLogOperation> create_write_log_operation(
+ WriteLogOperationSet &set, uint64_t image_offset_bytes,
+ uint64_t write_bytes, uint32_t data_len, CephContext *cct,
+ std::shared_ptr<WriteLogEntry> writesame_log_entry) = 0;
+ virtual std::shared_ptr<pwl::DiscardLogOperation> create_discard_log_operation(
+ std::shared_ptr<SyncPoint> sync_point, uint64_t image_offset_bytes,
+ uint64_t write_bytes, uint32_t discard_granularity_bytes,
+ utime_t dispatch_time, PerfCounters *perfcounter, CephContext *cct) = 0;
+ virtual C_ReadRequest *create_read_request(CephContext *cct, utime_t arrived,
+ PerfCounters *perfcounter, ceph::bufferlist *bl, Context *on_finish) = 0;
+
+};
+
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+#endif // CEPH_LIBRBD_CACHE_PWL_BUILDER_H
diff --git a/src/librbd/cache/pwl/DiscardRequest.cc b/src/librbd/cache/pwl/DiscardRequest.cc
new file mode 100644
index 000000000..eaf24137d
--- /dev/null
+++ b/src/librbd/cache/pwl/DiscardRequest.cc
@@ -0,0 +1,164 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "common/dout.h"
+#include "common/errno.h"
+#include "common/hostname.h"
+#include "librbd/asio/ContextWQ.h"
+#include "librbd/cache/pwl/DiscardRequest.h"
+
+#if __has_include(<filesystem>)
+#include <filesystem>
+namespace fs = std::filesystem;
+#elif __has_include(<experimental/filesystem>)
+#include <experimental/filesystem>
+namespace fs = std::experimental::filesystem;
+#endif
+
+#include "librbd/cache/pwl/ImageCacheState.h"
+
+#include "librbd/cache/Types.h"
+#include "librbd/io/ImageDispatcherInterface.h"
+#include "librbd/ImageCtx.h"
+#include "librbd/Utils.h"
+
+
+#define dout_subsys ceph_subsys_rbd_pwl
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::cache::pwl:DiscardRequest: " \
+ << this << " " << __func__ << ": "
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+
+using librbd::util::create_async_context_callback;
+using librbd::util::create_context_callback;
+
+template <typename I>
+DiscardRequest<I>* DiscardRequest<I>::create(
+ I &image_ctx,
+ plugin::Api<I>& plugin_api,
+ Context *on_finish) {
+ return new DiscardRequest(image_ctx, plugin_api, on_finish);
+}
+
+template <typename I>
+DiscardRequest<I>::DiscardRequest(
+ I &image_ctx,
+ plugin::Api<I>& plugin_api,
+ Context *on_finish)
+ : m_image_ctx(image_ctx),
+ m_plugin_api(plugin_api),
+ m_on_finish(create_async_context_callback(image_ctx, on_finish)),
+ m_error_result(0) {
+}
+
+template <typename I>
+void DiscardRequest<I>::send() {
+ delete_image_cache_file();
+}
+
+template <typename I>
+void DiscardRequest<I>::delete_image_cache_file() {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << dendl;
+
+ m_cache_state = ImageCacheState<I>::get_image_cache_state(&m_image_ctx, m_plugin_api);
+ if (!m_cache_state) {
+ remove_feature_bit();
+ return;
+ }
+ if (m_cache_state->present &&
+ !m_cache_state->host.compare(ceph_get_short_hostname()) &&
+ fs::exists(m_cache_state->path)) {
+ std::error_code ec;
+ fs::remove(m_cache_state->path, ec);
+ if (ec) {
+ lderr(cct) << "failed to remove persistent cache file: " << ec.message()
+ << dendl;
+ // not fatal
+ }
+ }
+
+ remove_image_cache_state();
+}
+
+template <typename I>
+void DiscardRequest<I>::remove_image_cache_state() {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << dendl;
+
+ using klass = DiscardRequest<I>;
+ Context *ctx = create_context_callback<klass, &klass::handle_remove_image_cache_state>(
+ this);
+
+ m_cache_state->clear_image_cache_state(ctx);
+}
+
+template <typename I>
+void DiscardRequest<I>::handle_remove_image_cache_state(int r) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << dendl;
+
+ if (r < 0) {
+ lderr(cct) << "failed to remove the image cache state: " << cpp_strerror(r)
+ << dendl;
+ save_result(r);
+ finish();
+ return;
+ }
+
+ remove_feature_bit();
+}
+
+template <typename I>
+void DiscardRequest<I>::remove_feature_bit() {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << dendl;
+
+ uint64_t new_features = m_image_ctx.features & ~RBD_FEATURE_DIRTY_CACHE;
+ uint64_t features_mask = RBD_FEATURE_DIRTY_CACHE;
+ ldout(cct, 10) << "old_features=" << m_image_ctx.features
+ << ", new_features=" << new_features
+ << ", features_mask=" << features_mask
+ << dendl;
+
+ int r = librbd::cls_client::set_features(&m_image_ctx.md_ctx, m_image_ctx.header_oid,
+ new_features, features_mask);
+ m_image_ctx.features &= ~RBD_FEATURE_DIRTY_CACHE;
+ using klass = DiscardRequest<I>;
+ Context *ctx = create_context_callback<klass, &klass::handle_remove_feature_bit>(
+ this);
+ ctx->complete(r);
+}
+
+template <typename I>
+void DiscardRequest<I>::handle_remove_feature_bit(int r) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << dendl;
+
+ if (r < 0) {
+ lderr(cct) << "failed to remove the feature bit: " << cpp_strerror(r)
+ << dendl;
+ save_result(r);
+ }
+ finish();
+}
+
+template <typename I>
+void DiscardRequest<I>::finish() {
+ if (m_cache_state) {
+ delete m_cache_state;
+ m_cache_state = nullptr;
+ }
+
+ m_on_finish->complete(m_error_result);
+ delete this;
+}
+
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+template class librbd::cache::pwl::DiscardRequest<librbd::ImageCtx>;
diff --git a/src/librbd/cache/pwl/DiscardRequest.h b/src/librbd/cache/pwl/DiscardRequest.h
new file mode 100644
index 000000000..c896369fe
--- /dev/null
+++ b/src/librbd/cache/pwl/DiscardRequest.h
@@ -0,0 +1,90 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_CACHE_PWL_SHUTDOWN_REQUEST_H
+#define CEPH_LIBRBD_CACHE_PWL_SHUTDOWN_REQUEST_H
+
+class Context;
+
+namespace librbd {
+
+class ImageCtx;
+namespace plugin { template <typename> struct Api; }
+
+namespace cache {
+
+namespace pwl {
+
+template<typename>
+class ImageCacheState;
+
+template <typename ImageCtxT = ImageCtx>
+class DiscardRequest {
+public:
+ static DiscardRequest* create(
+ ImageCtxT &image_ctx,
+ plugin::Api<ImageCtxT>& plugin_api,
+ Context *on_finish);
+
+ void send();
+
+private:
+
+ /**
+ * @verbatim
+ *
+ * Shutdown request goes through the following state machine:
+ *
+ * <start>
+ * |
+ * v
+ * REMOVE_IMAGE_CACHE_FILE
+ * |
+ * v
+ * REMOVE_IMAGE_CACHE_STATE
+ * |
+ * v
+ * REMOVE_IMAGE_FEATURE_BIT
+ * |
+ * v
+ * <finish>
+ *
+ * @endverbatim
+ */
+
+ DiscardRequest(ImageCtxT &image_ctx,
+ plugin::Api<ImageCtxT>& plugin_api,
+ Context *on_finish);
+
+ ImageCtxT &m_image_ctx;
+ ImageCacheState<ImageCtxT>* m_cache_state;
+ plugin::Api<ImageCtxT>& m_plugin_api;
+ Context *m_on_finish;
+
+ int m_error_result;
+
+ void delete_image_cache_file();
+
+ void remove_image_cache_state();
+ void handle_remove_image_cache_state(int r);
+
+ void remove_feature_bit();
+ void handle_remove_feature_bit(int r);
+
+ void finish();
+
+ void save_result(int result) {
+ if (m_error_result == 0 && result < 0) {
+ m_error_result = result;
+ }
+ }
+
+};
+
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+extern template class librbd::cache::pwl::DiscardRequest<librbd::ImageCtx>;
+
+#endif // CEPH_LIBRBD_CACHE_PWL_SHUTDOWN_REQUEST_H
diff --git a/src/librbd/cache/pwl/ImageCacheState.cc b/src/librbd/cache/pwl/ImageCacheState.cc
new file mode 100644
index 000000000..2bd6e1132
--- /dev/null
+++ b/src/librbd/cache/pwl/ImageCacheState.cc
@@ -0,0 +1,194 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "librbd/cache/Types.h"
+#include "librbd/cache/Utils.h"
+#include "librbd/cache/pwl/ImageCacheState.h"
+#include "librbd/ImageCtx.h"
+#include "librbd/Operations.h"
+#include "common/config_proxy.h"
+#include "common/environment.h"
+#include "common/hostname.h"
+#include "librbd/plugin/Api.h"
+
+#undef dout_subsys
+#define dout_subsys ceph_subsys_rbd_pwl
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::cache::pwl::ImageCacheState: " \
+ << __func__ << ": "
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+
+template <typename I>
+void ImageCacheState<I>::init_from_config() {
+ ldout(m_image_ctx->cct, 20) << dendl;
+
+ present = false;
+ empty = true;
+ clean = true;
+ host = "";
+ path = "";
+ ConfigProxy &config = m_image_ctx->config;
+ mode = config.get_val<std::string>("rbd_persistent_cache_mode");
+ size = 0;
+}
+
+template <typename I>
+bool ImageCacheState<I>::init_from_metadata(json_spirit::mValue& json_root) {
+ ldout(m_image_ctx->cct, 20) << dendl;
+
+ try {
+ auto& o = json_root.get_obj();
+ present = o["present"].get_bool();
+ empty = o["empty"].get_bool();
+ clean = o["clean"].get_bool();
+ host = o["host"].get_str();
+ path = o["path"].get_str();
+ mode = o["mode"].get_str();
+ size = o["size"].get_uint64();
+ } catch (std::runtime_error& e) {
+ lderr(m_image_ctx->cct) << "failed to parse cache state: " << e.what()
+ << dendl;
+ return false;
+ }
+
+ return true;
+}
+
+template <typename I>
+void ImageCacheState<I>::write_image_cache_state(std::unique_lock<ceph::mutex>& locker,
+ Context *on_finish) {
+ ceph_assert(ceph_mutex_is_locked_by_me(*locker.mutex()));
+ stats_timestamp = ceph_clock_now();
+ json_spirit::mObject o;
+ o["present"] = present;
+ o["empty"] = empty;
+ o["clean"] = clean;
+ o["host"] = host;
+ o["path"] = path;
+ o["mode"] = mode;
+ o["size"] = size;
+ o["stats_timestamp"] = stats_timestamp.sec();
+ o["allocated_bytes"] = allocated_bytes;
+ o["cached_bytes"] = cached_bytes;
+ o["dirty_bytes"] = dirty_bytes;
+ o["free_bytes"] = free_bytes;
+ o["hits_full"] = hits_full;
+ o["hits_partial"] = hits_partial;
+ o["misses"] = misses;
+ o["hit_bytes"] = hit_bytes;
+ o["miss_bytes"] = miss_bytes;
+ std::string image_state_json = json_spirit::write(o);
+ locker.unlock();
+
+ std::shared_lock owner_lock{m_image_ctx->owner_lock};
+ ldout(m_image_ctx->cct, 20) << __func__ << " Store state: "
+ << image_state_json << dendl;
+ m_plugin_api.execute_image_metadata_set(m_image_ctx, PERSISTENT_CACHE_STATE,
+ image_state_json, on_finish);
+}
+
+template <typename I>
+void ImageCacheState<I>::clear_image_cache_state(Context *on_finish) {
+ std::shared_lock owner_lock{m_image_ctx->owner_lock};
+ ldout(m_image_ctx->cct, 20) << __func__ << " Remove state: " << dendl;
+ m_plugin_api.execute_image_metadata_remove(
+ m_image_ctx, PERSISTENT_CACHE_STATE, on_finish);
+}
+
+template <typename I>
+ImageCacheState<I>* ImageCacheState<I>::create_image_cache_state(
+ I* image_ctx, plugin::Api<I>& plugin_api, int &r) {
+ std::string cache_state_str;
+ ImageCacheState<I>* cache_state = nullptr;
+
+ r = 0;
+ bool dirty_cache = plugin_api.test_image_features(image_ctx, RBD_FEATURE_DIRTY_CACHE);
+ if (dirty_cache) {
+ cls_client::metadata_get(&image_ctx->md_ctx, image_ctx->header_oid,
+ PERSISTENT_CACHE_STATE, &cache_state_str);
+ }
+
+ ldout(image_ctx->cct, 20) << "image_cache_state: " << cache_state_str << dendl;
+
+ bool pwl_enabled = cache::util::is_pwl_enabled(*image_ctx);
+ bool cache_desired = pwl_enabled;
+ cache_desired &= !image_ctx->read_only;
+ cache_desired &= !plugin_api.test_image_features(image_ctx, RBD_FEATURE_MIGRATING);
+ cache_desired &= !plugin_api.test_image_features(image_ctx, RBD_FEATURE_JOURNALING);
+ cache_desired &= !image_ctx->old_format;
+
+ if (!dirty_cache && !cache_desired) {
+ ldout(image_ctx->cct, 20) << "Do not desire to use image cache." << dendl;
+ } else if (dirty_cache && !cache_desired) {
+ lderr(image_ctx->cct) << "There's a dirty cache, but RWL cache is disabled."
+ << dendl;
+ r = -EINVAL;
+ }else if ((!dirty_cache || cache_state_str.empty()) && cache_desired) {
+ cache_state = new ImageCacheState<I>(image_ctx, plugin_api);
+ cache_state->init_from_config();
+ } else {
+ ceph_assert(!cache_state_str.empty());
+ json_spirit::mValue json_root;
+ if (!json_spirit::read(cache_state_str.c_str(), json_root)) {
+ lderr(image_ctx->cct) << "failed to parse cache state" << dendl;
+ r = -EINVAL;
+ return nullptr;
+ }
+ cache_state = new ImageCacheState<I>(image_ctx, plugin_api);
+ if (!cache_state->init_from_metadata(json_root)) {
+ delete cache_state;
+ r = -EINVAL;
+ return nullptr;
+ }
+ if (!cache_state->present) {
+ cache_state->init_from_config();
+ }
+ }
+ return cache_state;
+}
+
+template <typename I>
+ImageCacheState<I>* ImageCacheState<I>::get_image_cache_state(
+ I* image_ctx, plugin::Api<I>& plugin_api) {
+ ImageCacheState<I>* cache_state = nullptr;
+ string cache_state_str;
+ cls_client::metadata_get(&image_ctx->md_ctx, image_ctx->header_oid,
+ PERSISTENT_CACHE_STATE, &cache_state_str);
+ if (!cache_state_str.empty()) {
+ // ignore errors, best effort
+ cache_state = new ImageCacheState<I>(image_ctx, plugin_api);
+ json_spirit::mValue json_root;
+ if (!json_spirit::read(cache_state_str.c_str(), json_root)) {
+ lderr(image_ctx->cct) << "failed to parse cache state" << dendl;
+ } else {
+ cache_state->init_from_metadata(json_root);
+ }
+ }
+ return cache_state;
+}
+
+template <typename I>
+bool ImageCacheState<I>::is_valid() {
+ if (this->present &&
+ (host.compare(ceph_get_short_hostname()) != 0)) {
+ auto cleanstring = "dirty";
+ if (this->clean) {
+ cleanstring = "clean";
+ }
+ lderr(m_image_ctx->cct) << "An image cache (RWL) remains on another host "
+ << host << " which is " << cleanstring
+ << ". Flush/close the image there to remove the "
+ << "image cache" << dendl;
+ return false;
+ }
+ return true;
+}
+
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+template class librbd::cache::pwl::ImageCacheState<librbd::ImageCtx>;
diff --git a/src/librbd/cache/pwl/ImageCacheState.h b/src/librbd/cache/pwl/ImageCacheState.h
new file mode 100644
index 000000000..5be5f73ac
--- /dev/null
+++ b/src/librbd/cache/pwl/ImageCacheState.h
@@ -0,0 +1,86 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_CACHE_RWL_IMAGE_CACHE_STATE_H
+#define CEPH_LIBRBD_CACHE_RWL_IMAGE_CACHE_STATE_H
+
+#include "json_spirit/json_spirit.h"
+#include "librbd/ImageCtx.h"
+#include "librbd/cache/Types.h"
+#include <string>
+
+namespace ceph {
+ class Formatter;
+}
+
+namespace librbd {
+
+namespace plugin { template <typename> struct Api; }
+
+namespace cache {
+namespace pwl {
+
+template <typename ImageCtxT = ImageCtx>
+class ImageCacheState {
+private:
+ ImageCtxT* m_image_ctx;
+ plugin::Api<ImageCtxT>& m_plugin_api;
+public:
+ bool present = false;
+ bool empty = true;
+ bool clean = true;
+ std::string host;
+ std::string path;
+ std::string mode;
+ uint64_t size = 0;
+ /* After reloading, the following data does not need to be read,
+ * but recalculated. */
+ utime_t stats_timestamp;
+ uint64_t allocated_bytes = 0;
+ uint64_t cached_bytes = 0;
+ uint64_t dirty_bytes = 0;
+ uint64_t free_bytes = 0;
+ uint64_t hits_full = 0;
+ uint64_t hits_partial = 0;
+ uint64_t misses = 0;
+ uint64_t hit_bytes = 0;
+ uint64_t miss_bytes = 0;
+
+ ImageCacheState(ImageCtxT* image_ctx, plugin::Api<ImageCtxT>& plugin_api)
+ : m_image_ctx(image_ctx), m_plugin_api(plugin_api) {}
+
+ ~ImageCacheState() {}
+
+ ImageCacheType get_image_cache_mode() const {
+ if (mode == "rwl") {
+ return IMAGE_CACHE_TYPE_RWL;
+ } else if (mode == "ssd") {
+ return IMAGE_CACHE_TYPE_SSD;
+ }
+ return IMAGE_CACHE_TYPE_UNKNOWN;
+ }
+
+ void init_from_config();
+ bool init_from_metadata(json_spirit::mValue& json_root);
+
+ void write_image_cache_state(std::unique_lock<ceph::mutex>& locker,
+ Context *on_finish);
+
+ void clear_image_cache_state(Context *on_finish);
+
+ static ImageCacheState<ImageCtxT>* create_image_cache_state(
+ ImageCtxT* image_ctx, plugin::Api<ImageCtxT>& plugin_api, int &r);
+
+ static ImageCacheState<ImageCtxT>* get_image_cache_state(
+ ImageCtxT* image_ctx, plugin::Api<ImageCtxT>& plugin_api);
+
+ bool is_valid();
+};
+
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+extern template class librbd::cache::pwl::ImageCacheState<librbd::ImageCtx>;
+
+#endif // CEPH_LIBRBD_CACHE_RWL_IMAGE_CACHE_STATE_H
diff --git a/src/librbd/cache/pwl/InitRequest.cc b/src/librbd/cache/pwl/InitRequest.cc
new file mode 100644
index 000000000..65dac8b46
--- /dev/null
+++ b/src/librbd/cache/pwl/InitRequest.cc
@@ -0,0 +1,226 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "librbd/cache/pwl/InitRequest.h"
+#include "librbd/io/ImageDispatcher.h"
+#include "librbd/Utils.h"
+#include "common/dout.h"
+#include "common/errno.h"
+#include "librbd/asio/ContextWQ.h"
+
+#include "librbd/cache/pwl/ImageCacheState.h"
+#include "librbd/cache/WriteLogImageDispatch.h"
+#include "librbd/cache/ImageWriteback.h"
+#ifdef WITH_RBD_RWL
+#include "librbd/cache/pwl/rwl/WriteLog.h"
+#endif
+
+#ifdef WITH_RBD_SSD_CACHE
+#include "librbd/cache/pwl/ssd/WriteLog.h"
+#endif
+
+#include "librbd/cache/Utils.h"
+#include "librbd/ImageCtx.h"
+#include "librbd/plugin/Api.h"
+
+#define dout_subsys ceph_subsys_rbd_pwl
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::cache::pwl:InitRequest " \
+ << this << " " << __func__ << ": "
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+
+using librbd::util::create_async_context_callback;
+using librbd::util::create_context_callback;
+
+template <typename I>
+InitRequest<I>* InitRequest<I>::create(
+ I &image_ctx,
+ cache::ImageWritebackInterface& image_writeback,
+ plugin::Api<I>& plugin_api,
+ Context *on_finish) {
+ return new InitRequest(image_ctx, image_writeback, plugin_api, on_finish);
+}
+
+template <typename I>
+InitRequest<I>::InitRequest(
+ I &image_ctx,
+ cache::ImageWritebackInterface& image_writeback,
+ plugin::Api<I>& plugin_api,
+ Context *on_finish)
+ : m_image_ctx(image_ctx),
+ m_image_writeback(image_writeback),
+ m_plugin_api(plugin_api),
+ m_on_finish(create_async_context_callback(image_ctx, on_finish)),
+ m_error_result(0) {
+}
+
+template <typename I>
+void InitRequest<I>::send() {
+ get_image_cache_state();
+}
+
+template <typename I>
+void InitRequest<I>::get_image_cache_state() {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << dendl;
+
+ int r;
+ auto cache_state = ImageCacheState<I>::create_image_cache_state(
+ &m_image_ctx, m_plugin_api, r);
+
+ if (r < 0 || !cache_state) {
+ save_result(r);
+ finish();
+ return;
+ } else if (!cache_state->is_valid()) {
+ delete cache_state;
+ cache_state = nullptr;
+ lderr(cct) << "failed to get image cache state: " << cpp_strerror(r)
+ << dendl;
+ save_result(-ENOENT);
+ finish();
+ return;
+ }
+
+ auto mode = cache_state->get_image_cache_mode();
+ switch (mode) {
+ #ifdef WITH_RBD_RWL
+ case cache::IMAGE_CACHE_TYPE_RWL:
+ m_image_cache =
+ new librbd::cache::pwl::rwl::WriteLog<I>(m_image_ctx,
+ cache_state,
+ m_image_writeback,
+ m_plugin_api);
+ break;
+ #endif
+ #ifdef WITH_RBD_SSD_CACHE
+ case cache::IMAGE_CACHE_TYPE_SSD:
+ m_image_cache =
+ new librbd::cache::pwl::ssd::WriteLog<I>(m_image_ctx,
+ cache_state,
+ m_image_writeback,
+ m_plugin_api);
+ break;
+ #endif
+ default:
+ delete cache_state;
+ cache_state = nullptr;
+ save_result(-ENOENT);
+ finish();
+ return;
+ }
+
+ init_image_cache();
+}
+
+template <typename I>
+void InitRequest<I>::init_image_cache() {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << dendl;
+
+ using klass = InitRequest<I>;
+ Context *ctx = create_async_context_callback(m_image_ctx,
+ create_context_callback<klass, &klass::handle_init_image_cache>(this));
+ m_image_cache->init(ctx);
+}
+
+template <typename I>
+void InitRequest<I>::handle_init_image_cache(int r) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << dendl;
+
+ if (r < 0) {
+ lderr(cct) << "failed to init image cache: " << cpp_strerror(r)
+ << dendl;
+ delete m_image_cache;
+ m_image_cache = nullptr;
+ save_result(r);
+ finish();
+ return;
+ }
+ set_feature_bit();
+}
+
+template <typename I>
+void InitRequest<I>::set_feature_bit() {
+ CephContext *cct = m_image_ctx.cct;
+
+ uint64_t new_features = m_image_ctx.features | RBD_FEATURE_DIRTY_CACHE;
+ uint64_t features_mask = RBD_FEATURE_DIRTY_CACHE;
+ ldout(cct, 10) << "old_features=" << m_image_ctx.features
+ << ", new_features=" << new_features
+ << ", features_mask=" << features_mask
+ << dendl;
+
+ int r = librbd::cls_client::set_features(&m_image_ctx.md_ctx,
+ m_image_ctx.header_oid,
+ new_features, features_mask);
+ m_image_ctx.features |= RBD_FEATURE_DIRTY_CACHE;
+ using klass = InitRequest<I>;
+ Context *ctx = create_context_callback<klass, &klass::handle_set_feature_bit>(
+ this);
+ ctx->complete(r);
+}
+
+template <typename I>
+void InitRequest<I>::handle_set_feature_bit(int r) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << "r=" << r << dendl;
+
+ if (r < 0) {
+ lderr(cct) << "failed to set feature bit: " << cpp_strerror(r)
+ << dendl;
+ save_result(r);
+
+ shutdown_image_cache();
+ }
+
+ // Register RWL dispatch
+ auto image_dispatch = new cache::WriteLogImageDispatch<I>(
+ &m_image_ctx, m_image_cache, m_plugin_api);
+
+ m_image_ctx.io_image_dispatcher->register_dispatch(image_dispatch);
+
+ finish();
+}
+
+template <typename I>
+void InitRequest<I>::shutdown_image_cache() {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << dendl;
+
+ using klass = InitRequest<I>;
+ Context *ctx = create_context_callback<
+ klass, &klass::handle_shutdown_image_cache>(this);
+ m_image_cache->shut_down(ctx);
+}
+
+template <typename I>
+void InitRequest<I>::handle_shutdown_image_cache(int r) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << dendl;
+
+ if (r < 0) {
+ lderr(cct) << "failed to close image cache: " << cpp_strerror(r)
+ << dendl;
+ }
+ delete m_image_cache;
+ m_image_cache = nullptr;
+
+ finish();
+}
+
+template <typename I>
+void InitRequest<I>::finish() {
+ m_on_finish->complete(m_error_result);
+ delete this;
+}
+
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+template class librbd::cache::pwl::InitRequest<librbd::ImageCtx>;
diff --git a/src/librbd/cache/pwl/InitRequest.h b/src/librbd/cache/pwl/InitRequest.h
new file mode 100644
index 000000000..56e63425e
--- /dev/null
+++ b/src/librbd/cache/pwl/InitRequest.h
@@ -0,0 +1,105 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_CACHE_RWL_INIT_REQUEST_H
+#define CEPH_LIBRBD_CACHE_RWL_INIT_REQUEST_H
+
+class Context;
+
+namespace librbd {
+
+class ImageCtx;
+
+namespace io { class ImageDispatchInterface; }
+
+namespace plugin { template <typename> struct Api; }
+
+namespace cache {
+
+class ImageWritebackInterface;
+
+namespace pwl {
+
+template<typename>
+class AbstractWriteLog;
+
+template<typename>
+class ImageCacheState;
+
+template <typename ImageCtxT = ImageCtx>
+class InitRequest {
+public:
+ static InitRequest* create(
+ ImageCtxT &image_ctx,
+ librbd::cache::ImageWritebackInterface& image_writeback,
+ plugin::Api<ImageCtxT>& plugin_api,
+ Context *on_finish);
+
+ void send();
+
+private:
+
+ /**
+ * @verbatim
+ *
+ * Init request goes through the following state machine:
+ *
+ * <start>
+ * |
+ * v
+ * GET_IMAGE_CACHE_STATE
+ * |
+ * v
+ * INIT_IMAGE_CACHE
+ * |
+ * v
+ * SET_FEATURE_BIT * * * > CLOSE_IMAGE_CACHE
+ * | |
+ * v |
+ * <finish> <-------------------/
+ *
+ * @endverbatim
+ */
+
+ InitRequest(ImageCtxT &image_ctx,
+ librbd::cache::ImageWritebackInterface& image_writeback,
+ plugin::Api<ImageCtxT>& plugin_api,
+ Context *on_finish);
+
+ ImageCtxT &m_image_ctx;
+ librbd::cache::ImageWritebackInterface& m_image_writeback;
+ plugin::Api<ImageCtxT>& m_plugin_api;
+ AbstractWriteLog<ImageCtxT> *m_image_cache;
+ Context *m_on_finish;
+
+ int m_error_result;
+
+ bool is_pwl_enabled();
+
+ void get_image_cache_state();
+
+ void init_image_cache();
+ void handle_init_image_cache(int r);
+
+ void set_feature_bit();
+ void handle_set_feature_bit(int r);
+
+ void shutdown_image_cache();
+ void handle_shutdown_image_cache(int r);
+
+ void finish();
+
+ void save_result(int result) {
+ if (m_error_result == 0 && result < 0) {
+ m_error_result = result;
+ }
+ }
+};
+
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+extern template class librbd::cache::pwl::InitRequest<librbd::ImageCtx>;
+
+#endif // CEPH_LIBRBD_CACHE_RWL_INIT_REQUEST_H
diff --git a/src/librbd/cache/pwl/LogEntry.cc b/src/librbd/cache/pwl/LogEntry.cc
new file mode 100644
index 000000000..504d21051
--- /dev/null
+++ b/src/librbd/cache/pwl/LogEntry.cc
@@ -0,0 +1,140 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include <iostream>
+#include "LogEntry.h"
+#include "librbd/cache/ImageWriteback.h"
+
+#define dout_subsys ceph_subsys_rbd_pwl
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::cache::pwl::LogEntry: " << this << " " \
+ << __func__ << ": "
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+
+std::ostream& GenericLogEntry::format(std::ostream &os) const {
+ os << "ram_entry=[" << ram_entry << "], "
+ << "cache_entry=" << (void*)cache_entry << ", "
+ << "log_entry_index=" << log_entry_index << ", "
+ << "completed=" << completed;
+ return os;
+}
+
+std::ostream &operator<<(std::ostream &os,
+ const GenericLogEntry &entry) {
+ return entry.format(os);
+}
+
+std::ostream& SyncPointLogEntry::format(std::ostream &os) const {
+ os << "(Sync Point) ";
+ GenericLogEntry::format(os);
+ os << ", "
+ << "writes=" << writes << ", "
+ << "bytes=" << bytes << ", "
+ << "writes_completed=" << writes_completed << ", "
+ << "writes_flushed=" << writes_flushed << ", "
+ << "prior_sync_point_flushed=" << prior_sync_point_flushed << ", "
+ << "next_sync_point_entry=" << next_sync_point_entry;
+ return os;
+}
+
+std::ostream &operator<<(std::ostream &os,
+ const SyncPointLogEntry &entry) {
+ return entry.format(os);
+}
+
+bool GenericWriteLogEntry::can_writeback() const {
+ return (this->completed &&
+ (ram_entry.is_sequenced() ||
+ (sync_point_entry &&
+ sync_point_entry->completed)));
+}
+
+std::ostream& GenericWriteLogEntry::format(std::ostream &os) const {
+ GenericLogEntry::format(os);
+ os << ", "
+ << "sync_point_entry=[";
+ if (sync_point_entry) {
+ os << *sync_point_entry;
+ } else {
+ os << "nullptr";
+ }
+ os << "], "
+ << "referring_map_entries=" << referring_map_entries;
+ return os;
+}
+
+std::ostream &operator<<(std::ostream &os,
+ const GenericWriteLogEntry &entry) {
+ return entry.format(os);
+}
+
+void WriteLogEntry::init(bool has_data,
+ uint64_t current_sync_gen,
+ uint64_t last_op_sequence_num, bool persist_on_flush) {
+ ram_entry.set_has_data(has_data);
+ ram_entry.sync_gen_number = current_sync_gen;
+ if (persist_on_flush) {
+ /* Persist on flush. Sequence #0 is never used. */
+ ram_entry.write_sequence_number = 0;
+ } else {
+ /* Persist on write */
+ ram_entry.write_sequence_number = last_op_sequence_num;
+ ram_entry.set_sequenced(true);
+ }
+ ram_entry.set_sync_point(false);
+ ram_entry.set_discard(false);
+}
+
+std::ostream& WriteLogEntry::format(std::ostream &os) const {
+ os << "(Write) ";
+ GenericWriteLogEntry::format(os);
+ os << ", "
+ << "cache_buffer=" << (void*)cache_buffer << ", ";
+ os << "cache_bp=" << cache_bp << ", ";
+ os << "cache_bl=" << cache_bl << ", ";
+ os << "bl_refs=" << bl_refs;
+ return os;
+}
+
+std::ostream &operator<<(std::ostream &os,
+ const WriteLogEntry &entry) {
+ return entry.format(os);
+}
+
+void DiscardLogEntry::writeback(
+ librbd::cache::ImageWritebackInterface &image_writeback, Context *ctx) {
+ image_writeback.aio_discard(ram_entry.image_offset_bytes,
+ ram_entry.write_bytes,
+ m_discard_granularity_bytes, ctx);
+}
+
+void DiscardLogEntry::init(uint64_t current_sync_gen, bool persist_on_flush,
+ uint64_t last_op_sequence_num) {
+ ram_entry.sync_gen_number = current_sync_gen;
+ if (persist_on_flush) {
+ /* Persist on flush. Sequence #0 is never used. */
+ ram_entry.write_sequence_number = 0;
+ } else {
+ /* Persist on write */
+ ram_entry.write_sequence_number = last_op_sequence_num;
+ ram_entry.set_sequenced(true);
+ }
+}
+
+std::ostream &DiscardLogEntry::format(std::ostream &os) const {
+ os << "(Discard) ";
+ GenericWriteLogEntry::format(os);
+ return os;
+}
+
+std::ostream &operator<<(std::ostream &os,
+ const DiscardLogEntry &entry) {
+ return entry.format(os);
+}
+
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
diff --git a/src/librbd/cache/pwl/LogEntry.h b/src/librbd/cache/pwl/LogEntry.h
new file mode 100644
index 000000000..ecaca0b7b
--- /dev/null
+++ b/src/librbd/cache/pwl/LogEntry.h
@@ -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_CACHE_PWL_LOG_ENTRY_H
+#define CEPH_LIBRBD_CACHE_PWL_LOG_ENTRY_H
+
+#include "common/ceph_mutex.h"
+#include "librbd/Utils.h"
+#include "librbd/cache/pwl/Types.h"
+#include <atomic>
+#include <memory>
+
+namespace librbd {
+namespace cache {
+class ImageWritebackInterface;
+namespace pwl {
+
+class SyncPointLogEntry;
+class GenericWriteLogEntry;
+class WriteLogEntry;
+
+typedef std::list<std::shared_ptr<GenericWriteLogEntry>> GenericWriteLogEntries;
+
+class GenericLogEntry {
+public:
+ WriteLogCacheEntry ram_entry;
+ WriteLogCacheEntry *cache_entry = nullptr;
+ uint64_t log_entry_index = 0;
+ bool completed = false;
+ BlockGuardCell* m_cell = nullptr;
+ GenericLogEntry(uint64_t image_offset_bytes = 0, uint64_t write_bytes = 0)
+ : ram_entry(image_offset_bytes, write_bytes) {
+ };
+ virtual ~GenericLogEntry() { };
+ GenericLogEntry(const GenericLogEntry&) = delete;
+ GenericLogEntry &operator=(const GenericLogEntry&) = delete;
+ virtual bool can_writeback() const {
+ return false;
+ }
+ virtual bool can_retire() const {
+ return false;
+ }
+ virtual void set_flushed(bool flushed) {
+ ceph_assert(false);
+ }
+ virtual unsigned int write_bytes() const {
+ return 0;
+ };
+ virtual unsigned int bytes_dirty() const {
+ return 0;
+ };
+ virtual std::shared_ptr<SyncPointLogEntry> get_sync_point_entry() {
+ return nullptr;
+ }
+ virtual void writeback(librbd::cache::ImageWritebackInterface &image_writeback,
+ Context *ctx) {
+ ceph_assert(false);
+ };
+ virtual void writeback_bl(librbd::cache::ImageWritebackInterface &image_writeback,
+ Context *ctx, ceph::bufferlist &&bl) {
+ ceph_assert(false);
+ }
+ virtual bool is_write_entry() const {
+ return false;
+ }
+ virtual bool is_writesame_entry() const {
+ return false;
+ }
+ virtual bool is_sync_point() const {
+ return false;
+ }
+ virtual unsigned int get_aligned_data_size() const {
+ return 0;
+ }
+ virtual void remove_cache_bl() {}
+ virtual std::ostream& format(std::ostream &os) const;
+ friend std::ostream &operator<<(std::ostream &os,
+ const GenericLogEntry &entry);
+};
+
+class SyncPointLogEntry : public GenericLogEntry {
+public:
+ /* Writing entries using this sync gen number */
+ std::atomic<unsigned int> writes = {0};
+ /* Total bytes for all writing entries using this sync gen number */
+ std::atomic<uint64_t> bytes = {0};
+ /* Writing entries using this sync gen number that have completed */
+ std::atomic<unsigned int> writes_completed = {0};
+ /* Writing entries using this sync gen number that have completed flushing to the writeback interface */
+ std::atomic<unsigned int> writes_flushed = {0};
+ /* All writing entries using all prior sync gen numbers have been flushed */
+ std::atomic<bool> prior_sync_point_flushed = {true};
+ std::shared_ptr<SyncPointLogEntry> next_sync_point_entry = nullptr;
+ SyncPointLogEntry(uint64_t sync_gen_number) {
+ ram_entry.sync_gen_number = sync_gen_number;
+ ram_entry.set_sync_point(true);
+ };
+ ~SyncPointLogEntry() override {};
+ SyncPointLogEntry(const SyncPointLogEntry&) = delete;
+ SyncPointLogEntry &operator=(const SyncPointLogEntry&) = delete;
+ bool can_retire() const override {
+ return this->completed;
+ }
+ bool is_sync_point() const override {
+ return true;
+ }
+ std::ostream& format(std::ostream &os) const;
+ friend std::ostream &operator<<(std::ostream &os,
+ const SyncPointLogEntry &entry);
+};
+
+class GenericWriteLogEntry : public GenericLogEntry {
+public:
+ uint32_t referring_map_entries = 0;
+ std::shared_ptr<SyncPointLogEntry> sync_point_entry;
+ GenericWriteLogEntry(std::shared_ptr<SyncPointLogEntry> sync_point_entry,
+ uint64_t image_offset_bytes, uint64_t write_bytes)
+ : GenericLogEntry(image_offset_bytes, write_bytes), sync_point_entry(sync_point_entry) { }
+ GenericWriteLogEntry(uint64_t image_offset_bytes, uint64_t write_bytes)
+ : GenericLogEntry(image_offset_bytes, write_bytes), sync_point_entry(nullptr) { }
+ ~GenericWriteLogEntry() override {};
+ GenericWriteLogEntry(const GenericWriteLogEntry&) = delete;
+ GenericWriteLogEntry &operator=(const GenericWriteLogEntry&) = delete;
+ unsigned int write_bytes() const override {
+ /* The valid bytes in this ops data buffer. Discard and WS override. */
+ return ram_entry.write_bytes;
+ };
+ unsigned int bytes_dirty() const override {
+ /* The bytes in the image this op makes dirty. Discard and WS override. */
+ return write_bytes();
+ };
+ BlockExtent block_extent() {
+ return ram_entry.block_extent();
+ }
+ uint32_t get_map_ref() {
+ return(referring_map_entries);
+ }
+ void inc_map_ref() { referring_map_entries++; }
+ void dec_map_ref() { referring_map_entries--; }
+ bool can_writeback() const override;
+ std::shared_ptr<SyncPointLogEntry> get_sync_point_entry() override {
+ return sync_point_entry;
+ }
+ virtual void copy_cache_bl(bufferlist *out_bl) = 0;
+ void set_flushed(bool flushed) override {
+ m_flushed = flushed;
+ }
+ bool get_flushed() const {
+ return m_flushed;
+ }
+ std::ostream &format(std::ostream &os) const;
+ friend std::ostream &operator<<(std::ostream &os,
+ const GenericWriteLogEntry &entry);
+
+private:
+ bool m_flushed = false; /* or invalidated */
+};
+
+class WriteLogEntry : public GenericWriteLogEntry {
+protected:
+ bool is_writesame = false;
+ buffer::ptr cache_bp;
+ buffer::list cache_bl;
+ std::atomic<int> bl_refs = {0}; /* The refs held on cache_bp by cache_bl */
+ /* Used in WriteLogEntry::get_cache_bl() to syncronize between threads making entries readable */
+ mutable ceph::mutex m_entry_bl_lock;
+
+ virtual void init_cache_bp() {}
+
+ virtual void init_bl(buffer::ptr &bp, buffer::list &bl) {}
+public:
+ uint8_t *cache_buffer = nullptr;
+ WriteLogEntry(std::shared_ptr<SyncPointLogEntry> sync_point_entry,
+ uint64_t image_offset_bytes, uint64_t write_bytes)
+ : GenericWriteLogEntry(sync_point_entry, image_offset_bytes, write_bytes),
+ m_entry_bl_lock(ceph::make_mutex(pwl::unique_lock_name(
+ "librbd::cache::pwl::WriteLogEntry::m_entry_bl_lock", this)))
+ { }
+ WriteLogEntry(uint64_t image_offset_bytes, uint64_t write_bytes)
+ : GenericWriteLogEntry(nullptr, image_offset_bytes, write_bytes),
+ m_entry_bl_lock(ceph::make_mutex(pwl::unique_lock_name(
+ "librbd::cache::pwl::WriteLogEntry::m_entry_bl_lock", this)))
+ { }
+ WriteLogEntry(std::shared_ptr<SyncPointLogEntry> sync_point_entry,
+ uint64_t image_offset_bytes, uint64_t write_bytes,
+ uint32_t data_length)
+ : WriteLogEntry(sync_point_entry, image_offset_bytes, write_bytes) {
+ ram_entry.set_writesame(true);
+ ram_entry.ws_datalen = data_length;
+ is_writesame = true;
+ };
+ WriteLogEntry(uint64_t image_offset_bytes, uint64_t write_bytes,
+ uint32_t data_length)
+ : WriteLogEntry(nullptr, image_offset_bytes, write_bytes) {
+ ram_entry.set_writesame(true);
+ ram_entry.ws_datalen = data_length;
+ is_writesame = true;
+ };
+ ~WriteLogEntry() override {};
+ WriteLogEntry(const WriteLogEntry&) = delete;
+ WriteLogEntry &operator=(const WriteLogEntry&) = delete;
+ unsigned int write_bytes() const override {
+ // The valid bytes in this ops data buffer.
+ if(is_writesame) {
+ return ram_entry.ws_datalen;
+ }
+ return ram_entry.write_bytes;
+ };
+ unsigned int bytes_dirty() const override {
+ // The bytes in the image this op makes dirty.
+ return ram_entry.write_bytes;
+ };
+ void init(bool has_data,
+ uint64_t current_sync_gen, uint64_t last_op_sequence_num, bool persist_on_flush);
+ virtual void init_cache_buffer(std::vector<WriteBufferAllocation>::iterator allocation) {}
+ virtual void init_cache_bl(bufferlist &src_bl, uint64_t off, uint64_t len) {}
+ /* Returns a ref to a bl containing bufferptrs to the entry cache buffer */
+ virtual buffer::list &get_cache_bl() = 0;
+
+ BlockExtent block_extent();
+ virtual unsigned int reader_count() const = 0;
+ /* Constructs a new bl containing copies of cache_bp */
+ bool can_retire() const override {
+ return (this->completed && this->get_flushed() && (0 == reader_count()));
+ }
+ bool is_write_entry() const override {
+ return true;
+ }
+ bool is_writesame_entry() const override {
+ return is_writesame;
+ }
+ std::ostream &format(std::ostream &os) const;
+ friend std::ostream &operator<<(std::ostream &os,
+ const WriteLogEntry &entry);
+};
+
+class DiscardLogEntry : public GenericWriteLogEntry {
+public:
+ DiscardLogEntry(std::shared_ptr<SyncPointLogEntry> sync_point_entry,
+ uint64_t image_offset_bytes, uint64_t write_bytes,
+ uint32_t discard_granularity_bytes)
+ : GenericWriteLogEntry(sync_point_entry, image_offset_bytes, write_bytes),
+ m_discard_granularity_bytes(discard_granularity_bytes) {
+ ram_entry.set_discard(true);
+ };
+ DiscardLogEntry(uint64_t image_offset_bytes, uint64_t write_bytes)
+ : GenericWriteLogEntry(nullptr, image_offset_bytes, write_bytes) {
+ ram_entry.set_discard(true);
+ };
+ DiscardLogEntry(const DiscardLogEntry&) = delete;
+ DiscardLogEntry &operator=(const DiscardLogEntry&) = delete;
+ unsigned int write_bytes() const override {
+ /* The valid bytes in this ops data buffer. */
+ return 0;
+ };
+ unsigned int bytes_dirty() const override {
+ /* The bytes in the image this op makes dirty. */
+ return ram_entry.write_bytes;
+ };
+ bool can_retire() const override {
+ return this->completed;
+ }
+ void copy_cache_bl(bufferlist *out_bl) override {
+ ceph_assert(false);
+ }
+ void writeback(librbd::cache::ImageWritebackInterface &image_writeback,
+ Context *ctx) override;
+ void init(uint64_t current_sync_gen, bool persist_on_flush, uint64_t last_op_sequence_num);
+ std::ostream &format(std::ostream &os) const;
+ friend std::ostream &operator<<(std::ostream &os,
+ const DiscardLogEntry &entry);
+private:
+ uint32_t m_discard_granularity_bytes;
+};
+
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+#endif // CEPH_LIBRBD_CACHE_PWL_LOG_ENTRY_H
diff --git a/src/librbd/cache/pwl/LogMap.cc b/src/librbd/cache/pwl/LogMap.cc
new file mode 100644
index 000000000..a2e6d65eb
--- /dev/null
+++ b/src/librbd/cache/pwl/LogMap.cc
@@ -0,0 +1,278 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "LogMap.h"
+#include "include/ceph_assert.h"
+#include "librbd/Utils.h"
+#include "librbd/cache/pwl/LogEntry.h"
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+
+#define dout_subsys ceph_subsys_rbd_pwl
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::cache::pwl::LogMap: " << this << " " \
+ << __func__ << ": "
+template <typename T>
+std::ostream &operator<<(std::ostream &os,
+ LogMapEntry<T> &e) {
+ os << "block_extent=" << e.block_extent << ", "
+ << "log_entry=[" << e.log_entry << "]";
+ return os;
+}
+
+template <typename T>
+LogMapEntry<T>::LogMapEntry(const BlockExtent block_extent,
+ std::shared_ptr<T> log_entry)
+ : block_extent(block_extent) , log_entry(log_entry) {
+}
+
+template <typename T>
+LogMapEntry<T>::LogMapEntry(std::shared_ptr<T> log_entry)
+ : block_extent(log_entry->block_extent()) , log_entry(log_entry) {
+}
+
+template <typename T>
+LogMap<T>::LogMap(CephContext *cct)
+ : m_cct(cct),
+ m_lock(ceph::make_mutex(pwl::unique_lock_name(
+ "librbd::cache::pwl::LogMap::m_lock", this))) {
+}
+
+/**
+ * Add a write log entry to the map. Subsequent queries for blocks
+ * within this log entry's extent will find this log entry. Portions
+ * of prior write log entries overlapping with this log entry will
+ * be replaced in the map by this log entry.
+ *
+ * The map_entries field of the log entry object will be updated to
+ * contain this map entry.
+ *
+ * The map_entries fields of all log entries overlapping with this
+ * entry will be updated to remove the regions that overlap with
+ * this.
+ */
+template <typename T>
+void LogMap<T>::add_log_entry(std::shared_ptr<T> log_entry) {
+ std::lock_guard locker(m_lock);
+ add_log_entry_locked(log_entry);
+}
+
+template <typename T>
+void LogMap<T>::add_log_entries(std::list<std::shared_ptr<T>> &log_entries) {
+ std::lock_guard locker(m_lock);
+ ldout(m_cct, 20) << dendl;
+ for (auto &log_entry : log_entries) {
+ add_log_entry_locked(log_entry);
+ }
+}
+
+/**
+ * Remove any map entries that refer to the supplied write log
+ * entry.
+ */
+template <typename T>
+void LogMap<T>::remove_log_entry(std::shared_ptr<T> log_entry) {
+ std::lock_guard locker(m_lock);
+ remove_log_entry_locked(log_entry);
+}
+
+template <typename T>
+void LogMap<T>::remove_log_entries(std::list<std::shared_ptr<T>> &log_entries) {
+ std::lock_guard locker(m_lock);
+ ldout(m_cct, 20) << dendl;
+ for (auto &log_entry : log_entries) {
+ remove_log_entry_locked(log_entry);
+ }
+}
+
+/**
+ * Returns the list of all write log entries that overlap the specified block
+ * extent. This doesn't tell you which portions of these entries overlap the
+ * extent, or each other. For that, use find_map_entries(). A log entry may
+ * appear in the list more than once, if multiple map entries refer to it
+ * (e.g. the middle of that write log entry has been overwritten).
+ */
+template <typename T>
+std::list<std::shared_ptr<T>> LogMap<T>::find_log_entries(BlockExtent block_extent) {
+ std::lock_guard locker(m_lock);
+ ldout(m_cct, 20) << dendl;
+ return find_log_entries_locked(block_extent);
+}
+
+/**
+ * Returns the list of all write log map entries that overlap the
+ * specified block extent.
+ */
+template <typename T>
+LogMapEntries<T> LogMap<T>::find_map_entries(BlockExtent block_extent) {
+ std::lock_guard locker(m_lock);
+ ldout(m_cct, 20) << dendl;
+ return find_map_entries_locked(block_extent);
+}
+
+template <typename T>
+void LogMap<T>::add_log_entry_locked(std::shared_ptr<T> log_entry) {
+ LogMapEntry<T> map_entry(log_entry);
+ ldout(m_cct, 20) << "block_extent=" << map_entry.block_extent
+ << dendl;
+ ceph_assert(ceph_mutex_is_locked_by_me(m_lock));
+ LogMapEntries<T> overlap_entries = find_map_entries_locked(map_entry.block_extent);
+ for (auto &entry : overlap_entries) {
+ ldout(m_cct, 20) << entry << dendl;
+ if (map_entry.block_extent.block_start <= entry.block_extent.block_start) {
+ if (map_entry.block_extent.block_end >= entry.block_extent.block_end) {
+ ldout(m_cct, 20) << "map entry completely occluded by new log entry" << dendl;
+ remove_map_entry_locked(entry);
+ } else {
+ ceph_assert(map_entry.block_extent.block_end < entry.block_extent.block_end);
+ /* The new entry occludes the beginning of the old entry */
+ BlockExtent adjusted_extent(map_entry.block_extent.block_end,
+ entry.block_extent.block_end);
+ adjust_map_entry_locked(entry, adjusted_extent);
+ }
+ } else {
+ if (map_entry.block_extent.block_end >= entry.block_extent.block_end) {
+ /* The new entry occludes the end of the old entry */
+ BlockExtent adjusted_extent(entry.block_extent.block_start,
+ map_entry.block_extent.block_start);
+ adjust_map_entry_locked(entry, adjusted_extent);
+ } else {
+ /* The new entry splits the old entry */
+ split_map_entry_locked(entry, map_entry.block_extent);
+ }
+ }
+ }
+ add_map_entry_locked(map_entry);
+}
+
+template <typename T>
+void LogMap<T>::remove_log_entry_locked(std::shared_ptr<T> log_entry) {
+ ldout(m_cct, 20) << "*log_entry=" << *log_entry << dendl;
+ ceph_assert(ceph_mutex_is_locked_by_me(m_lock));
+
+ LogMapEntries<T> possible_hits = find_map_entries_locked(log_entry->block_extent());
+ for (auto &possible_hit : possible_hits) {
+ if (possible_hit.log_entry == log_entry) {
+ /* This map entry refers to the specified log entry */
+ remove_map_entry_locked(possible_hit);
+ }
+ }
+}
+
+template <typename T>
+void LogMap<T>::add_map_entry_locked(LogMapEntry<T> &map_entry) {
+ ceph_assert(map_entry.log_entry);
+ m_block_to_log_entry_map.insert(map_entry);
+ map_entry.log_entry->inc_map_ref();
+}
+
+template <typename T>
+void LogMap<T>::remove_map_entry_locked(LogMapEntry<T> &map_entry) {
+ auto it = m_block_to_log_entry_map.find(map_entry);
+ ceph_assert(it != m_block_to_log_entry_map.end());
+
+ LogMapEntry<T> erased = *it;
+ m_block_to_log_entry_map.erase(it);
+ erased.log_entry->dec_map_ref();
+ if (0 == erased.log_entry->get_map_ref()) {
+ ldout(m_cct, 20) << "log entry has zero map entries: " << erased.log_entry << dendl;
+ }
+}
+
+template <typename T>
+void LogMap<T>::adjust_map_entry_locked(LogMapEntry<T> &map_entry, BlockExtent &new_extent) {
+ auto it = m_block_to_log_entry_map.find(map_entry);
+ ceph_assert(it != m_block_to_log_entry_map.end());
+
+ LogMapEntry<T> adjusted = *it;
+ m_block_to_log_entry_map.erase(it);
+
+ m_block_to_log_entry_map.insert(LogMapEntry<T>(new_extent, adjusted.log_entry));
+}
+
+template <typename T>
+void LogMap<T>::split_map_entry_locked(LogMapEntry<T> &map_entry, BlockExtent &removed_extent) {
+ auto it = m_block_to_log_entry_map.find(map_entry);
+ ceph_assert(it != m_block_to_log_entry_map.end());
+
+ LogMapEntry<T> split = *it;
+ m_block_to_log_entry_map.erase(it);
+
+ BlockExtent left_extent(split.block_extent.block_start,
+ removed_extent.block_start);
+ m_block_to_log_entry_map.insert(LogMapEntry<T>(left_extent, split.log_entry));
+
+ BlockExtent right_extent(removed_extent.block_end,
+ split.block_extent.block_end);
+ m_block_to_log_entry_map.insert(LogMapEntry<T>(right_extent, split.log_entry));
+
+ split.log_entry->inc_map_ref();
+}
+
+template <typename T>
+std::list<std::shared_ptr<T>> LogMap<T>::find_log_entries_locked(const BlockExtent &block_extent) {
+ std::list<std::shared_ptr<T>> overlaps;
+ ldout(m_cct, 20) << "block_extent=" << block_extent << dendl;
+
+ ceph_assert(ceph_mutex_is_locked_by_me(m_lock));
+ LogMapEntries<T> map_entries = find_map_entries_locked(block_extent);
+ for (auto &map_entry : map_entries) {
+ overlaps.emplace_back(map_entry.log_entry);
+ }
+ return overlaps;
+}
+
+/**
+ * TODO: Generalize this to do some arbitrary thing to each map
+ * extent, instead of returning a list.
+ */
+template <typename T>
+LogMapEntries<T> LogMap<T>::find_map_entries_locked(const BlockExtent &block_extent) {
+ LogMapEntries<T> overlaps;
+
+ ldout(m_cct, 20) << "block_extent=" << block_extent << dendl;
+ ceph_assert(ceph_mutex_is_locked_by_me(m_lock));
+ auto p = m_block_to_log_entry_map.equal_range(LogMapEntry<T>(block_extent));
+ ldout(m_cct, 20) << "count=" << std::distance(p.first, p.second) << dendl;
+ for ( auto i = p.first; i != p.second; ++i ) {
+ LogMapEntry<T> entry = *i;
+ overlaps.emplace_back(entry);
+ ldout(m_cct, 20) << entry << dendl;
+ }
+ return overlaps;
+}
+
+/* We map block extents to write log entries, or portions of write log
+ * entries. These are both represented by a WriteLogMapEntry. When a
+ * GenericWriteLogEntry is added to this map, a WriteLogMapEntry is created to
+ * represent the entire block extent of the GenericWriteLogEntry, and the
+ * WriteLogMapEntry is added to the set.
+ *
+ * The set must not contain overlapping WriteLogMapEntrys. WriteLogMapEntrys
+ * in the set that overlap with one being added are adjusted (shrunk, split,
+ * or removed) before the new entry is added.
+ *
+ * This comparison works despite the ambiguity because we ensure the set
+ * contains no overlapping entries. This comparison works to find entries
+ * that overlap with a given block extent because equal_range() returns the
+ * first entry in which the extent doesn't end before the given extent
+ * starts, and the last entry for which the extent starts before the given
+ * extent ends (the first entry that the key is less than, and the last entry
+ * that is less than the key).
+ */
+template <typename T>
+bool LogMap<T>::LogMapEntryCompare::operator()(const LogMapEntry<T> &lhs,
+ const LogMapEntry<T> &rhs) const {
+ if (lhs.block_extent.block_end <= rhs.block_extent.block_start) {
+ return true;
+ }
+ return false;
+}
+
+} //namespace pwl
+} //namespace cache
+} //namespace librbd
+
+template class librbd::cache::pwl::LogMap<librbd::cache::pwl::GenericWriteLogEntry>;
diff --git a/src/librbd/cache/pwl/LogMap.h b/src/librbd/cache/pwl/LogMap.h
new file mode 100644
index 000000000..a05307896
--- /dev/null
+++ b/src/librbd/cache/pwl/LogMap.h
@@ -0,0 +1,81 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_CACHE_RWL_LOG_MAP_H
+#define CEPH_LIBRBD_CACHE_RWL_LOG_MAP_H
+
+#include "librbd/BlockGuard.h"
+#include <list>
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+
+/**
+ * WriteLogMap: maps block extents to GenericWriteLogEntries
+ *
+ * A WriteLogMapEntry (based on LogMapEntry) refers to a portion of a GenericWriteLogEntry
+ */
+template <typename T>
+class LogMapEntry {
+public:
+ BlockExtent block_extent;
+ std::shared_ptr<T> log_entry;
+
+ LogMapEntry(BlockExtent block_extent,
+ std::shared_ptr<T> log_entry = nullptr);
+ LogMapEntry(std::shared_ptr<T> log_entry);
+
+ template <typename U>
+ friend std::ostream &operator<<(std::ostream &os,
+ LogMapEntry<U> &e);
+};
+
+template <typename T>
+using LogMapEntries = std::list<LogMapEntry<T>>;
+
+template <typename T>
+class LogMap {
+public:
+ LogMap(CephContext *cct);
+ LogMap(const LogMap&) = delete;
+ LogMap &operator=(const LogMap&) = delete;
+
+ void add_log_entry(std::shared_ptr<T> log_entry);
+ void add_log_entries(std::list<std::shared_ptr<T>> &log_entries);
+ void remove_log_entry(std::shared_ptr<T> log_entry);
+ void remove_log_entries(std::list<std::shared_ptr<T>> &log_entries);
+ std::list<std::shared_ptr<T>> find_log_entries(BlockExtent block_extent);
+ LogMapEntries<T> find_map_entries(BlockExtent block_extent);
+
+private:
+ void add_log_entry_locked(std::shared_ptr<T> log_entry);
+ void remove_log_entry_locked(std::shared_ptr<T> log_entry);
+ void add_map_entry_locked(LogMapEntry<T> &map_entry);
+ void remove_map_entry_locked(LogMapEntry<T> &map_entry);
+ void adjust_map_entry_locked(LogMapEntry<T> &map_entry, BlockExtent &new_extent);
+ void split_map_entry_locked(LogMapEntry<T> &map_entry, BlockExtent &removed_extent);
+ std::list<std::shared_ptr<T>> find_log_entries_locked(const BlockExtent &block_extent);
+ LogMapEntries<T> find_map_entries_locked(const BlockExtent &block_extent);
+
+ using LogMapEntryT = LogMapEntry<T>;
+
+ class LogMapEntryCompare {
+ public:
+ bool operator()(const LogMapEntryT &lhs,
+ const LogMapEntryT &rhs) const;
+ };
+
+ using BlockExtentToLogMapEntries = std::set<LogMapEntryT,
+ LogMapEntryCompare>;
+
+ CephContext *m_cct;
+ ceph::mutex m_lock;
+ BlockExtentToLogMapEntries m_block_to_log_entry_map;
+};
+
+} //namespace pwl
+} //namespace cache
+} //namespace librbd
+
+#endif //CEPH_LIBRBD_CACHE_RWL_LOG_MAP_H
diff --git a/src/librbd/cache/pwl/LogOperation.cc b/src/librbd/cache/pwl/LogOperation.cc
new file mode 100644
index 000000000..4fc13a91a
--- /dev/null
+++ b/src/librbd/cache/pwl/LogOperation.cc
@@ -0,0 +1,316 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include <iostream>
+#include "LogOperation.h"
+#include "librbd/cache/pwl/Types.h"
+
+#define dout_subsys ceph_subsys_rbd_pwl
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::cache::pwl::LogOperation: " << this \
+ << " " << __func__ << ": "
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+
+GenericLogOperation::GenericLogOperation(utime_t dispatch_time,
+ PerfCounters *perfcounter)
+ : m_perfcounter(perfcounter), dispatch_time(dispatch_time) {
+}
+
+std::ostream& GenericLogOperation::format(std::ostream &os) const {
+ os << "dispatch_time=[" << dispatch_time << "], "
+ << "buf_persist_start_time=[" << buf_persist_start_time << "], "
+ << "buf_persist_comp_time=[" << buf_persist_comp_time << "], "
+ << "log_append_start_time=[" << log_append_start_time << "], "
+ << "log_append_comp_time=[" << log_append_comp_time << "], ";
+ return os;
+}
+
+std::ostream &operator<<(std::ostream &os,
+ const GenericLogOperation &op) {
+ return op.format(os);
+}
+
+SyncPointLogOperation::SyncPointLogOperation(ceph::mutex &lock,
+ std::shared_ptr<SyncPoint> sync_point,
+ utime_t dispatch_time,
+ PerfCounters *perfcounter,
+ CephContext *cct)
+ : GenericLogOperation(dispatch_time, perfcounter), m_cct(cct), m_lock(lock),
+ sync_point(sync_point) {
+}
+
+SyncPointLogOperation::~SyncPointLogOperation() { }
+
+std::ostream &SyncPointLogOperation::format(std::ostream &os) const {
+ os << "(Sync Point) ";
+ GenericLogOperation::format(os);
+ os << ", "
+ << "sync_point=[" << *sync_point << "]";
+ return os;
+}
+
+std::ostream &operator<<(std::ostream &os,
+ const SyncPointLogOperation &op) {
+ return op.format(os);
+}
+
+std::vector<Context*> SyncPointLogOperation::append_sync_point() {
+ std::vector<Context*> appending_contexts;
+ std::lock_guard locker(m_lock);
+ if (!sync_point->appending) {
+ sync_point->appending = true;
+ }
+ appending_contexts.swap(sync_point->on_sync_point_appending);
+ return appending_contexts;
+}
+
+void SyncPointLogOperation::clear_earlier_sync_point() {
+ std::lock_guard locker(m_lock);
+ ceph_assert(sync_point->later_sync_point);
+ ceph_assert(sync_point->later_sync_point->earlier_sync_point == sync_point);
+ sync_point->later_sync_point->earlier_sync_point = nullptr;
+ sync_point->later_sync_point = nullptr;
+}
+
+std::vector<Context*> SyncPointLogOperation::swap_on_sync_point_persisted() {
+ std::lock_guard locker(m_lock);
+ std::vector<Context*> persisted_contexts;
+ persisted_contexts.swap(sync_point->on_sync_point_persisted);
+ return persisted_contexts;
+}
+
+void SyncPointLogOperation::appending() {
+ ceph_assert(sync_point);
+ ldout(m_cct, 20) << "Sync point op=[" << *this
+ << "] appending" << dendl;
+ auto appending_contexts = append_sync_point();
+ for (auto &ctx : appending_contexts) {
+ ctx->complete(0);
+ }
+}
+
+void SyncPointLogOperation::complete(int result) {
+ ceph_assert(sync_point);
+ ldout(m_cct, 20) << "Sync point op =[" << *this
+ << "] completed" << dendl;
+ clear_earlier_sync_point();
+
+ /* Do append now in case completion occurred before the
+ * normal append callback executed, and to handle
+ * on_append work that was queued after the sync point
+ * entered the appending state. */
+ appending();
+ auto persisted_contexts = swap_on_sync_point_persisted();
+ for (auto &ctx : persisted_contexts) {
+ ctx->complete(result);
+ }
+}
+
+GenericWriteLogOperation::GenericWriteLogOperation(std::shared_ptr<SyncPoint> sync_point,
+ utime_t dispatch_time,
+ PerfCounters *perfcounter,
+ CephContext *cct)
+ : GenericLogOperation(dispatch_time, perfcounter),
+ m_lock(ceph::make_mutex(pwl::unique_lock_name(
+ "librbd::cache::pwl::GenericWriteLogOperation::m_lock", this))),
+ m_cct(cct),
+ sync_point(sync_point) {
+}
+
+GenericWriteLogOperation::~GenericWriteLogOperation() { }
+
+std::ostream &GenericWriteLogOperation::format(std::ostream &os) const {
+ GenericLogOperation::format(os);
+ return os;
+}
+
+std::ostream &operator<<(std::ostream &os,
+ const GenericWriteLogOperation &op) {
+ return op.format(os);
+}
+
+/* Called when the write log operation is appending and its log position is guaranteed */
+void GenericWriteLogOperation::appending() {
+ Context *on_append = nullptr;
+ ldout(m_cct, 20) << __func__ << " " << this << dendl;
+ {
+ std::lock_guard locker(m_lock);
+ on_append = on_write_append;
+ on_write_append = nullptr;
+ }
+ if (on_append) {
+ ldout(m_cct, 20) << __func__ << " " << this << " on_append=" << on_append << dendl;
+ on_append->complete(0);
+ }
+}
+
+/* Called when the write log operation is completed in all log replicas */
+void GenericWriteLogOperation::complete(int result) {
+ appending();
+ Context *on_persist = nullptr;
+ ldout(m_cct, 20) << __func__ << " " << this << dendl;
+ {
+ std::lock_guard locker(m_lock);
+ on_persist = on_write_persist;
+ on_write_persist = nullptr;
+ }
+ if (on_persist) {
+ ldout(m_cct, 20) << __func__ << " " << this << " on_persist=" << on_persist
+ << dendl;
+ on_persist->complete(result);
+ }
+}
+
+WriteLogOperation::WriteLogOperation(
+ WriteLogOperationSet &set, uint64_t image_offset_bytes,
+ uint64_t write_bytes, CephContext *cct,
+ std::shared_ptr<WriteLogEntry> write_log_entry)
+ : GenericWriteLogOperation(set.sync_point, set.dispatch_time,
+ set.perfcounter, cct),
+ log_entry(write_log_entry) {
+ on_write_append = set.extent_ops_appending->new_sub();
+ on_write_persist = set.extent_ops_persist->new_sub();
+ log_entry->sync_point_entry->writes++;
+ log_entry->sync_point_entry->bytes += write_bytes;
+}
+
+WriteLogOperation::WriteLogOperation(WriteLogOperationSet &set,
+ uint64_t image_offset_bytes,
+ uint64_t write_bytes,
+ uint32_t data_len,
+ CephContext *cct,
+ std::shared_ptr<WriteLogEntry> writesame_log_entry)
+ : WriteLogOperation(set, image_offset_bytes, write_bytes, cct,
+ writesame_log_entry) {
+ is_writesame = true;
+}
+
+WriteLogOperation::~WriteLogOperation() { }
+
+void WriteLogOperation::init(bool has_data, std::vector<WriteBufferAllocation>::iterator allocation,
+ uint64_t current_sync_gen,
+ uint64_t last_op_sequence_num,
+ bufferlist &write_req_bl, uint64_t buffer_offset,
+ bool persist_on_flush) {
+ log_entry->init(has_data, current_sync_gen, last_op_sequence_num,
+ persist_on_flush);
+ buffer_alloc = &(*allocation);
+ bl.substr_of(write_req_bl, buffer_offset, log_entry->write_bytes());
+ log_entry->init_cache_bl(write_req_bl, buffer_offset,
+ log_entry->write_bytes());
+}
+
+std::ostream &WriteLogOperation::format(std::ostream &os) const {
+ string op_name = is_writesame ? "(Write Same) " : "(Write) ";
+ os << op_name;
+ GenericWriteLogOperation::format(os);
+ os << ", ";
+ if (log_entry) {
+ os << "log_entry=[" << *log_entry << "], ";
+ } else {
+ os << "log_entry=nullptr, ";
+ }
+ os << "bl=[" << bl << "],"
+ << "buffer_alloc=" << buffer_alloc;
+ return os;
+}
+
+std::ostream &operator<<(std::ostream &os,
+ const WriteLogOperation &op) {
+ return op.format(os);
+}
+
+
+void WriteLogOperation::complete(int result) {
+ GenericWriteLogOperation::complete(result);
+ m_perfcounter->tinc(l_librbd_pwl_log_op_dis_to_buf_t,
+ buf_persist_start_time - dispatch_time);
+ utime_t buf_persist_lat = buf_persist_comp_time - buf_persist_start_time;
+ m_perfcounter->tinc(l_librbd_pwl_log_op_buf_to_bufc_t, buf_persist_lat);
+ m_perfcounter->hinc(l_librbd_pwl_log_op_buf_to_bufc_t_hist,
+ buf_persist_lat.to_nsec(),
+ log_entry->ram_entry.write_bytes);
+ m_perfcounter->tinc(l_librbd_pwl_log_op_buf_to_app_t,
+ log_append_start_time - buf_persist_start_time);
+}
+
+WriteLogOperationSet::WriteLogOperationSet(utime_t dispatched, PerfCounters *perfcounter, std::shared_ptr<SyncPoint> sync_point,
+ bool persist_on_flush, CephContext *cct, Context *on_finish)
+ : m_cct(cct), m_on_finish(on_finish),
+ persist_on_flush(persist_on_flush),
+ dispatch_time(dispatched),
+ perfcounter(perfcounter),
+ sync_point(sync_point) {
+ on_ops_appending = sync_point->prior_persisted_gather_new_sub();
+ on_ops_persist = nullptr;
+ extent_ops_persist =
+ new C_Gather(m_cct,
+ new LambdaContext( [this](int r) {
+ ldout(this->m_cct,20) << __func__ << " " << this << " m_extent_ops_persist completed" << dendl;
+ if (on_ops_persist) {
+ on_ops_persist->complete(r);
+ }
+ m_on_finish->complete(r);
+ }));
+ auto appending_persist_sub = extent_ops_persist->new_sub();
+ extent_ops_appending =
+ new C_Gather(m_cct,
+ new LambdaContext( [this, appending_persist_sub](int r) {
+ ldout(this->m_cct, 20) << __func__ << " " << this << " m_extent_ops_appending completed" << dendl;
+ on_ops_appending->complete(r);
+ appending_persist_sub->complete(r);
+ }));
+}
+
+WriteLogOperationSet::~WriteLogOperationSet() { }
+
+std::ostream &operator<<(std::ostream &os,
+ const WriteLogOperationSet &s) {
+ os << "cell=" << (void*)s.cell << ", "
+ << "extent_ops_appending=[" << s.extent_ops_appending << ", "
+ << "extent_ops_persist=[" << s.extent_ops_persist << "]";
+ return os;
+}
+
+DiscardLogOperation::DiscardLogOperation(std::shared_ptr<SyncPoint> sync_point,
+ uint64_t image_offset_bytes,
+ uint64_t write_bytes,
+ uint32_t discard_granularity_bytes,
+ utime_t dispatch_time,
+ PerfCounters *perfcounter,
+ CephContext *cct)
+ : GenericWriteLogOperation(sync_point, dispatch_time, perfcounter, cct),
+ log_entry(std::make_shared<DiscardLogEntry>(sync_point->log_entry,
+ image_offset_bytes,
+ write_bytes,
+ discard_granularity_bytes)) {
+ on_write_persist = nullptr;
+ log_entry->sync_point_entry->writes++;
+ log_entry->sync_point_entry->bytes += write_bytes;
+}
+
+DiscardLogOperation::~DiscardLogOperation() { }
+
+std::ostream &DiscardLogOperation::format(std::ostream &os) const {
+ os << "(Discard) ";
+ GenericWriteLogOperation::format(os);
+ os << ", ";
+ if (log_entry) {
+ os << "log_entry=[" << *log_entry << "], ";
+ } else {
+ os << "log_entry=nullptr, ";
+ }
+ return os;
+}
+
+std::ostream &operator<<(std::ostream &os,
+ const DiscardLogOperation &op) {
+ return op.format(os);
+}
+
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
diff --git a/src/librbd/cache/pwl/LogOperation.h b/src/librbd/cache/pwl/LogOperation.h
new file mode 100644
index 000000000..15befe05f
--- /dev/null
+++ b/src/librbd/cache/pwl/LogOperation.h
@@ -0,0 +1,224 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_CACHE_RWL_LOG_OPERATION_H
+#define CEPH_LIBRBD_CACHE_RWL_LOG_OPERATION_H
+
+#include "include/utime.h"
+#include "librbd/cache/pwl/LogEntry.h"
+#include "librbd/cache/pwl/SyncPoint.h"
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+
+struct WriteBufferAllocation;
+
+class WriteLogOperationSet;
+
+class WriteLogOperation;
+
+class GenericWriteLogOperation;
+
+class SyncPointLogOperation;
+
+class GenericLogOperation;
+
+template <typename T>
+class AbstractWriteLog;
+
+using GenericLogOperationSharedPtr = std::shared_ptr<GenericLogOperation>;
+
+using GenericLogOperationsVector = std::vector<GenericLogOperationSharedPtr>;
+
+class GenericLogOperation {
+protected:
+ PerfCounters *m_perfcounter = nullptr;
+public:
+ utime_t dispatch_time; // When op created
+ utime_t buf_persist_start_time; // When buffer persist begins
+ utime_t buf_persist_comp_time; // When buffer persist completes
+ utime_t log_append_start_time; // When log append begins
+ utime_t log_append_comp_time; // When log append completes
+ GenericLogOperation(utime_t dispatch_time, PerfCounters *perfcounter);
+ virtual ~GenericLogOperation() { };
+ GenericLogOperation(const GenericLogOperation&) = delete;
+ GenericLogOperation &operator=(const GenericLogOperation&) = delete;
+ virtual std::ostream &format(std::ostream &os) const;
+ friend std::ostream &operator<<(std::ostream &os,
+ const GenericLogOperation &op);
+ virtual const std::shared_ptr<GenericLogEntry> get_log_entry() = 0;
+ virtual void appending() = 0;
+ virtual void complete(int r) = 0;
+ virtual void mark_log_entry_completed() {};
+ virtual bool reserved_allocated() const {
+ return false;
+ }
+ virtual bool is_writing_op() const {
+ return false;
+ }
+ virtual void init_op(uint64_t current_sync_gen, bool persist_on_flush,
+ uint64_t last_op_sequence_num, Context *write_persist,
+ Context *write_append) {};
+ virtual void copy_bl_to_cache_buffer(
+ std::vector<WriteBufferAllocation>::iterator allocation) {};
+};
+
+class SyncPointLogOperation : public GenericLogOperation {
+private:
+ CephContext *m_cct;
+ ceph::mutex &m_lock;
+ std::vector<Context*> append_sync_point();
+ void clear_earlier_sync_point();
+ std::vector<Context*> swap_on_sync_point_persisted();
+public:
+ std::shared_ptr<SyncPoint> sync_point;
+ SyncPointLogOperation(ceph::mutex &lock,
+ std::shared_ptr<SyncPoint> sync_point,
+ utime_t dispatch_time,
+ PerfCounters *perfcounter,
+ CephContext *cct);
+ ~SyncPointLogOperation() override;
+ SyncPointLogOperation(const SyncPointLogOperation&) = delete;
+ SyncPointLogOperation &operator=(const SyncPointLogOperation&) = delete;
+ std::ostream &format(std::ostream &os) const;
+ friend std::ostream &operator<<(std::ostream &os,
+ const SyncPointLogOperation &op);
+ const std::shared_ptr<GenericLogEntry> get_log_entry() override {
+ return sync_point->log_entry;
+ }
+ void appending() override;
+ void complete(int r) override;
+};
+
+class GenericWriteLogOperation : public GenericLogOperation {
+protected:
+ ceph::mutex m_lock;
+ CephContext *m_cct;
+public:
+ std::shared_ptr<SyncPoint> sync_point;
+ Context *on_write_append = nullptr; /* Completion for things waiting on this
+ * write's position in the log to be
+ * guaranteed */
+ Context *on_write_persist = nullptr; /* Completion for things waiting on this
+ * write to persist */
+ GenericWriteLogOperation(std::shared_ptr<SyncPoint> sync_point,
+ utime_t dispatch_time,
+ PerfCounters *perfcounter,
+ CephContext *cct);
+ ~GenericWriteLogOperation() override;
+ GenericWriteLogOperation(const GenericWriteLogOperation&) = delete;
+ GenericWriteLogOperation &operator=(const GenericWriteLogOperation&) = delete;
+ std::ostream &format(std::ostream &os) const;
+ friend std::ostream &operator<<(std::ostream &os,
+ const GenericWriteLogOperation &op);
+ void mark_log_entry_completed() override{
+ sync_point->log_entry->writes_completed++;
+ }
+ bool reserved_allocated() const override {
+ return true;
+ }
+ bool is_writing_op() const override {
+ return true;
+ }
+ void appending() override;
+ void complete(int r) override;
+};
+
+class WriteLogOperation : public GenericWriteLogOperation {
+public:
+ using GenericWriteLogOperation::m_lock;
+ using GenericWriteLogOperation::sync_point;
+ using GenericWriteLogOperation::on_write_append;
+ using GenericWriteLogOperation::on_write_persist;
+ std::shared_ptr<WriteLogEntry> log_entry;
+ bufferlist bl;
+ bool is_writesame = false;
+ WriteBufferAllocation *buffer_alloc = nullptr;
+ WriteLogOperation(WriteLogOperationSet &set,
+ uint64_t image_offset_bytes,
+ uint64_t write_bytes, CephContext *cct,
+ std::shared_ptr<WriteLogEntry> write_log_entry);
+ WriteLogOperation(WriteLogOperationSet &set,
+ uint64_t image_offset_bytes,
+ uint64_t write_bytes, uint32_t data_len,
+ CephContext *cct,
+ std::shared_ptr<WriteLogEntry> writesame_log_entry);
+ ~WriteLogOperation() override;
+ WriteLogOperation(const WriteLogOperation&) = delete;
+ WriteLogOperation &operator=(const WriteLogOperation&) = delete;
+ void init(bool has_data,
+ std::vector<WriteBufferAllocation>::iterator allocation,
+ uint64_t current_sync_gen, uint64_t last_op_sequence_num,
+ bufferlist &write_req_bl, uint64_t buffer_offset,
+ bool persist_on_flush);
+ std::ostream &format(std::ostream &os) const;
+ friend std::ostream &operator<<(std::ostream &os,
+ const WriteLogOperation &op);
+ const std::shared_ptr<GenericLogEntry> get_log_entry() override {
+ return log_entry;
+ }
+
+ void complete(int r) override;
+};
+
+
+class WriteLogOperationSet {
+private:
+ CephContext *m_cct;
+ Context *m_on_finish;
+public:
+ bool persist_on_flush;
+ BlockGuardCell *cell;
+ C_Gather *extent_ops_appending;
+ Context *on_ops_appending;
+ C_Gather *extent_ops_persist;
+ Context *on_ops_persist;
+ GenericLogOperationsVector operations;
+ utime_t dispatch_time; /* When set created */
+ PerfCounters *perfcounter = nullptr;
+ std::shared_ptr<SyncPoint> sync_point;
+ WriteLogOperationSet(utime_t dispatched, PerfCounters *perfcounter,
+ std::shared_ptr<SyncPoint> sync_point,
+ const bool persist_on_flush, CephContext *cct,
+ Context *on_finish);
+ ~WriteLogOperationSet();
+ WriteLogOperationSet(const WriteLogOperationSet&) = delete;
+ WriteLogOperationSet &operator=(const WriteLogOperationSet&) = delete;
+ friend std::ostream &operator<<(std::ostream &os,
+ const WriteLogOperationSet &s);
+};
+
+class DiscardLogOperation : public GenericWriteLogOperation {
+public:
+ using GenericWriteLogOperation::m_lock;
+ using GenericWriteLogOperation::sync_point;
+ using GenericWriteLogOperation::on_write_append;
+ using GenericWriteLogOperation::on_write_persist;
+ std::shared_ptr<DiscardLogEntry> log_entry;
+ DiscardLogOperation(std::shared_ptr<SyncPoint> sync_point,
+ uint64_t image_offset_bytes,
+ uint64_t write_bytes,
+ uint32_t discard_granularity_bytes,
+ utime_t dispatch_time,
+ PerfCounters *perfcounter,
+ CephContext *cct);
+ ~DiscardLogOperation() override;
+ DiscardLogOperation(const DiscardLogOperation&) = delete;
+ DiscardLogOperation &operator=(const DiscardLogOperation&) = delete;
+ const std::shared_ptr<GenericLogEntry> get_log_entry() override {
+ return log_entry;
+ }
+ bool reserved_allocated() const override {
+ return false;
+ }
+ std::ostream &format(std::ostream &os) const;
+ friend std::ostream &operator<<(std::ostream &os,
+ const DiscardLogOperation &op);
+};
+
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+#endif // CEPH_LIBRBD_CACHE_RWL_LOG_OPERATION_H
diff --git a/src/librbd/cache/pwl/ReadRequest.h b/src/librbd/cache/pwl/ReadRequest.h
new file mode 100644
index 000000000..d4b2aee5b
--- /dev/null
+++ b/src/librbd/cache/pwl/ReadRequest.h
@@ -0,0 +1,45 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_CACHE_RWL_READ_REQUEST_H
+#define CEPH_LIBRBD_CACHE_RWL_READ_REQUEST_H
+
+#include "include/Context.h"
+#include "librbd/cache/pwl/Types.h"
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+
+typedef std::vector<std::shared_ptr<pwl::ImageExtentBuf>> ImageExtentBufs;
+
+class C_ReadRequest : public Context {
+public:
+ io::Extents miss_extents; // move back to caller
+ ImageExtentBufs read_extents;
+ bufferlist miss_bl;
+
+ C_ReadRequest(
+ CephContext *cct, utime_t arrived, PerfCounters *perfcounter,
+ bufferlist *out_bl, Context *on_finish)
+ : m_cct(cct), m_on_finish(on_finish), m_out_bl(out_bl),
+ m_arrived_time(arrived), m_perfcounter(perfcounter) {}
+ ~C_ReadRequest() {}
+
+ const char *get_name() const {
+ return "C_ReadRequest";
+ }
+
+protected:
+ CephContext *m_cct;
+ Context *m_on_finish;
+ bufferlist *m_out_bl;
+ utime_t m_arrived_time;
+ PerfCounters *m_perfcounter;
+};
+
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+#endif // CEPH_LIBRBD_CACHE_RWL_READ_REQUEST_H
diff --git a/src/librbd/cache/pwl/Request.cc b/src/librbd/cache/pwl/Request.cc
new file mode 100644
index 000000000..8159b121a
--- /dev/null
+++ b/src/librbd/cache/pwl/Request.cc
@@ -0,0 +1,561 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "Request.h"
+#include "librbd/BlockGuard.h"
+#include "librbd/cache/pwl/LogEntry.h"
+#include "librbd/cache/pwl/AbstractWriteLog.h"
+
+#define dout_subsys ceph_subsys_rbd_pwl
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::cache::pwl::Request: " << this << " " \
+ << __func__ << ": "
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+
+template <typename T>
+C_BlockIORequest<T>::C_BlockIORequest(T &pwl, const utime_t arrived, io::Extents &&extents,
+ bufferlist&& bl, const int fadvise_flags, Context *user_req)
+ : pwl(pwl), image_extents(std::move(extents)),
+ bl(std::move(bl)), fadvise_flags(fadvise_flags),
+ user_req(user_req), image_extents_summary(image_extents), m_arrived_time(arrived) {
+ ldout(pwl.get_context(), 99) << this << dendl;
+}
+
+template <typename T>
+C_BlockIORequest<T>::~C_BlockIORequest() {
+ ldout(pwl.get_context(), 99) << this << dendl;
+ ceph_assert(m_cell_released || !m_cell);
+}
+
+template <typename T>
+std::ostream &operator<<(std::ostream &os,
+ const C_BlockIORequest<T> &req) {
+ os << "image_extents=[" << req.image_extents << "], "
+ << "image_extents_summary=[" << req.image_extents_summary << "], "
+ << "bl=" << req.bl << ", "
+ << "user_req=" << req.user_req << ", "
+ << "m_user_req_completed=" << req.m_user_req_completed << ", "
+ << "m_deferred=" << req.m_deferred << ", "
+ << "detained=" << req.detained << ", "
+ << "waited_lanes=" << req.waited_lanes << ", "
+ << "waited_entries=" << req.waited_entries << ", "
+ << "waited_buffers=" << req.waited_buffers << "";
+ return os;
+}
+
+template <typename T>
+void C_BlockIORequest<T>::set_cell(BlockGuardCell *cell) {
+ ldout(pwl.get_context(), 20) << this << " cell=" << cell << dendl;
+ ceph_assert(cell);
+ ceph_assert(!m_cell);
+ m_cell = cell;
+}
+
+template <typename T>
+BlockGuardCell *C_BlockIORequest<T>::get_cell(void) {
+ ldout(pwl.get_context(), 20) << this << " cell=" << m_cell << dendl;
+ return m_cell;
+}
+
+template <typename T>
+void C_BlockIORequest<T>::release_cell() {
+ ldout(pwl.get_context(), 20) << this << " cell=" << m_cell << dendl;
+ ceph_assert(m_cell);
+ bool initial = false;
+ if (m_cell_released.compare_exchange_strong(initial, true)) {
+ pwl.release_guarded_request(m_cell);
+ } else {
+ ldout(pwl.get_context(), 5) << "cell " << m_cell << " already released for " << this << dendl;
+ }
+}
+
+template <typename T>
+void C_BlockIORequest<T>::complete_user_request(int r) {
+ bool initial = false;
+ if (m_user_req_completed.compare_exchange_strong(initial, true)) {
+ ldout(pwl.get_context(), 15) << this << " completing user req" << dendl;
+ m_user_req_completed_time = ceph_clock_now();
+ pwl.complete_user_request(user_req, r);
+ } else {
+ ldout(pwl.get_context(), 20) << this << " user req already completed" << dendl;
+ }
+}
+
+template <typename T>
+void C_BlockIORequest<T>::finish(int r) {
+ ldout(pwl.get_context(), 20) << this << dendl;
+
+ complete_user_request(r);
+ bool initial = false;
+ if (m_finish_called.compare_exchange_strong(initial, true)) {
+ ldout(pwl.get_context(), 15) << this << " finishing" << dendl;
+ finish_req(0);
+ } else {
+ ldout(pwl.get_context(), 20) << this << " already finished" << dendl;
+ ceph_assert(0);
+ }
+}
+
+template <typename T>
+void C_BlockIORequest<T>::deferred() {
+ bool initial = false;
+ if (m_deferred.compare_exchange_strong(initial, true)) {
+ deferred_handler();
+ }
+}
+
+template <typename T>
+C_WriteRequest<T>::C_WriteRequest(T &pwl, const utime_t arrived, io::Extents &&image_extents,
+ bufferlist&& bl, const int fadvise_flags, ceph::mutex &lock,
+ PerfCounters *perfcounter, Context *user_req)
+ : C_BlockIORequest<T>(pwl, arrived, std::move(image_extents), std::move(bl), fadvise_flags, user_req),
+ m_perfcounter(perfcounter), m_lock(lock) {
+ ldout(pwl.get_context(), 99) << this << dendl;
+}
+
+template <typename T>
+C_WriteRequest<T>::C_WriteRequest(T &pwl, const utime_t arrived, io::Extents &&image_extents,
+ bufferlist&& cmp_bl, bufferlist&& bl, uint64_t *mismatch_offset,
+ int fadvise_flags, ceph::mutex &lock, PerfCounters *perfcounter,
+ Context *user_req)
+ : C_BlockIORequest<T>(pwl, arrived, std::move(image_extents), std::move(bl), fadvise_flags, user_req),
+ mismatch_offset(mismatch_offset), cmp_bl(std::move(cmp_bl)),
+ m_perfcounter(perfcounter), m_lock(lock) {
+ is_comp_and_write = true;
+ ldout(pwl.get_context(), 20) << dendl;
+}
+
+template <typename T>
+C_WriteRequest<T>::~C_WriteRequest() {
+ ldout(pwl.get_context(), 99) << this << dendl;
+}
+
+template <typename T>
+std::ostream &operator<<(std::ostream &os,
+ const C_WriteRequest<T> &req) {
+ os << (C_BlockIORequest<T>&)req
+ << " m_resources.allocated=" << req.m_resources.allocated;
+ if (req.op_set) {
+ os << "op_set=" << *req.op_set;
+ }
+ return os;
+}
+
+template <typename T>
+void C_WriteRequest<T>::blockguard_acquired(GuardedRequestFunctionContext &guard_ctx) {
+ ldout(pwl.get_context(), 20) << __func__ << " write_req=" << this << " cell=" << guard_ctx.cell << dendl;
+
+ ceph_assert(guard_ctx.cell);
+ this->detained = guard_ctx.state.detained; /* overlapped */
+ this->m_queued = guard_ctx.state.queued; /* queued behind at least one barrier */
+ this->set_cell(guard_ctx.cell);
+}
+
+template <typename T>
+void C_WriteRequest<T>::finish_req(int r) {
+ ldout(pwl.get_context(), 15) << "write_req=" << this << " cell=" << this->get_cell() << dendl;
+
+ /* Completed to caller by here (in finish(), which calls this) */
+ utime_t now = ceph_clock_now();
+ if(is_comp_and_write && !compare_succeeded) {
+ update_req_stats(now);
+ return;
+ }
+ pwl.release_write_lanes(this);
+ ceph_assert(m_resources.allocated);
+ m_resources.allocated = false;
+ this->release_cell(); /* TODO: Consider doing this in appending state */
+ update_req_stats(now);
+}
+
+template <typename T>
+std::shared_ptr<WriteLogOperation> C_WriteRequest<T>::create_operation(
+ uint64_t offset, uint64_t len) {
+ return pwl.m_builder->create_write_log_operation(
+ *op_set, offset, len, pwl.get_context(),
+ pwl.m_builder->create_write_log_entry(op_set->sync_point->log_entry, offset, len));
+}
+
+template <typename T>
+void C_WriteRequest<T>::setup_log_operations(DeferredContexts &on_exit) {
+ GenericWriteLogEntries log_entries;
+ {
+ std::lock_guard locker(m_lock);
+ std::shared_ptr<SyncPoint> current_sync_point = pwl.get_current_sync_point();
+ if ((!pwl.get_persist_on_flush() && current_sync_point->log_entry->writes_completed) ||
+ (current_sync_point->log_entry->writes > MAX_WRITES_PER_SYNC_POINT) ||
+ (current_sync_point->log_entry->bytes > MAX_BYTES_PER_SYNC_POINT)) {
+ /* Create new sync point and persist the previous one. This sequenced
+ * write will bear a sync gen number shared with no already completed
+ * writes. A group of sequenced writes may be safely flushed concurrently
+ * if they all arrived before any of them completed. We'll insert one on
+ * an aio_flush() from the application. Here we're inserting one to cap
+ * the number of bytes and writes per sync point. When the application is
+ * not issuing flushes, we insert sync points to record some observed
+ * write concurrency information that enables us to safely issue >1 flush
+ * write (for writes observed here to have been in flight simultaneously)
+ * at a time in persist-on-write mode.
+ */
+ pwl.flush_new_sync_point(nullptr, on_exit);
+ current_sync_point = pwl.get_current_sync_point();
+ }
+ uint64_t current_sync_gen = pwl.get_current_sync_gen();
+ op_set =
+ make_unique<WriteLogOperationSet>(this->m_dispatched_time,
+ m_perfcounter,
+ current_sync_point,
+ pwl.get_persist_on_flush(),
+ pwl.get_context(), this);
+ ldout(pwl.get_context(), 20) << "write_req=" << *this << " op_set=" << op_set.get() << dendl;
+ ceph_assert(m_resources.allocated);
+ /* op_set->operations initialized differently for plain write or write same */
+ auto allocation = m_resources.buffers.begin();
+ uint64_t buffer_offset = 0;
+ for (auto &extent : this->image_extents) {
+ /* operation->on_write_persist connected to m_prior_log_entries_persisted Gather */
+ auto operation = this->create_operation(extent.first, extent.second);
+ this->op_set->operations.emplace_back(operation);
+
+ /* A WS is also a write */
+ ldout(pwl.get_context(), 20) << "write_req=" << *this << " op_set=" << op_set.get()
+ << " operation=" << operation << dendl;
+ log_entries.emplace_back(operation->log_entry);
+ if (!op_set->persist_on_flush) {
+ pwl.inc_last_op_sequence_num();
+ }
+ operation->init(true, allocation, current_sync_gen,
+ pwl.get_last_op_sequence_num(), this->bl, buffer_offset, op_set->persist_on_flush);
+ buffer_offset += operation->log_entry->write_bytes();
+ ldout(pwl.get_context(), 20) << "operation=[" << *operation << "]" << dendl;
+ allocation++;
+ }
+ }
+ /* All extent ops subs created */
+ op_set->extent_ops_appending->activate();
+ op_set->extent_ops_persist->activate();
+
+ pwl.add_into_log_map(log_entries, this);
+}
+
+template <typename T>
+void C_WriteRequest<T>::copy_cache() {
+ pwl.copy_bl_to_buffer(&m_resources, op_set);
+}
+
+template <typename T>
+bool C_WriteRequest<T>::append_write_request(std::shared_ptr<SyncPoint> sync_point) {
+ std::lock_guard locker(m_lock);
+ auto write_req_sp = this;
+ if (sync_point->earlier_sync_point) {
+ Context *schedule_append_ctx = new LambdaContext([write_req_sp](int r) {
+ write_req_sp->schedule_append();
+ });
+ sync_point->earlier_sync_point->on_sync_point_appending.push_back(schedule_append_ctx);
+ return true;
+ }
+ return false;
+}
+
+template <typename T>
+void C_WriteRequest<T>::schedule_append() {
+ ceph_assert(++m_appended == 1);
+ pwl.setup_schedule_append(this->op_set->operations, m_do_early_flush, this);
+}
+
+/**
+ * Attempts to allocate log resources for a write. Returns true if successful.
+ *
+ * Resources include 1 lane per extent, 1 log entry per extent, and the payload
+ * data space for each extent.
+ *
+ * Lanes are released after the write persists via release_write_lanes()
+ */
+template <typename T>
+bool C_WriteRequest<T>::alloc_resources() {
+ this->allocated_time = ceph_clock_now();
+ return pwl.alloc_resources(this);
+}
+
+/**
+ * Takes custody of write_req. Resources must already be allocated.
+ *
+ * Locking:
+ * Acquires lock
+ */
+template <typename T>
+void C_WriteRequest<T>::dispatch()
+{
+ CephContext *cct = pwl.get_context();
+ DeferredContexts on_exit;
+ utime_t now = ceph_clock_now();
+ this->m_dispatched_time = now;
+
+ ldout(cct, 15) << "write_req=" << this << " cell=" << this->get_cell() << dendl;
+ this->setup_log_operations(on_exit);
+
+ bool append_deferred = false;
+ if (!op_set->persist_on_flush &&
+ append_write_request(op_set->sync_point)) {
+ /* In persist-on-write mode, we defer the append of this write until the
+ * previous sync point is appending (meaning all the writes before it are
+ * persisted and that previous sync point can now appear in the
+ * log). Since we insert sync points in persist-on-write mode when writes
+ * have already completed to the current sync point, this limits us to
+ * one inserted sync point in flight at a time, and gives the next
+ * inserted sync point some time to accumulate a few writes if they
+ * arrive soon. Without this we can insert an absurd number of sync
+ * points, each with one or two writes. That uses a lot of log entries,
+ * and limits flushing to very few writes at a time. */
+ m_do_early_flush = false;
+ append_deferred = true;
+ } else {
+ /* The prior sync point is done, so we'll schedule append here. If this is
+ * persist-on-write, and probably still the caller's thread, we'll use this
+ * caller's thread to perform the persist & replication of the payload
+ * buffer. */
+ m_do_early_flush =
+ !(this->detained || this->m_queued || this->m_deferred || op_set->persist_on_flush);
+ }
+ if (!append_deferred) {
+ this->schedule_append();
+ }
+}
+
+template <typename T>
+C_FlushRequest<T>::C_FlushRequest(T &pwl, const utime_t arrived,
+ io::Extents &&image_extents,
+ bufferlist&& bl, const int fadvise_flags,
+ ceph::mutex &lock, PerfCounters *perfcounter,
+ Context *user_req)
+ : C_BlockIORequest<T>(pwl, arrived, std::move(image_extents), std::move(bl),
+ fadvise_flags, user_req),
+ m_lock(lock), m_perfcounter(perfcounter) {
+ ldout(pwl.get_context(), 20) << this << dendl;
+}
+
+template <typename T>
+void C_FlushRequest<T>::finish_req(int r) {
+ ldout(pwl.get_context(), 20) << "flush_req=" << this
+ << " cell=" << this->get_cell() << dendl;
+ /* Block guard already released */
+ ceph_assert(!this->get_cell());
+
+ /* Completed to caller by here */
+ utime_t now = ceph_clock_now();
+ m_perfcounter->tinc(l_librbd_pwl_aio_flush_latency, now - this->m_arrived_time);
+}
+
+template <typename T>
+bool C_FlushRequest<T>::alloc_resources() {
+ ldout(pwl.get_context(), 20) << "req type=" << get_name() << " "
+ << "req=[" << *this << "]" << dendl;
+ return pwl.alloc_resources(this);
+}
+
+template <typename T>
+void C_FlushRequest<T>::dispatch() {
+ utime_t now = ceph_clock_now();
+ ldout(pwl.get_context(), 20) << "req type=" << get_name() << " "
+ << "req=[" << *this << "]" << dendl;
+ ceph_assert(this->m_resources.allocated);
+ this->m_dispatched_time = now;
+
+ op = std::make_shared<SyncPointLogOperation>(m_lock,
+ to_append,
+ now,
+ m_perfcounter,
+ pwl.get_context());
+
+ m_perfcounter->inc(l_librbd_pwl_log_ops, 1);
+ pwl.schedule_append(op);
+}
+
+template <typename T>
+void C_FlushRequest<T>::setup_buffer_resources(
+ uint64_t *bytes_cached, uint64_t *bytes_dirtied, uint64_t *bytes_allocated,
+ uint64_t *number_lanes, uint64_t *number_log_entries,
+ uint64_t *number_unpublished_reserves) {
+ *number_log_entries = 1;
+}
+
+template <typename T>
+std::ostream &operator<<(std::ostream &os,
+ const C_FlushRequest<T> &req) {
+ os << (C_BlockIORequest<T>&)req
+ << " m_resources.allocated=" << req.m_resources.allocated;
+ return os;
+}
+
+template <typename T>
+C_DiscardRequest<T>::C_DiscardRequest(T &pwl, const utime_t arrived, io::Extents &&image_extents,
+ uint32_t discard_granularity_bytes, ceph::mutex &lock,
+ PerfCounters *perfcounter, Context *user_req)
+ : C_BlockIORequest<T>(pwl, arrived, std::move(image_extents), bufferlist(), 0, user_req),
+ m_discard_granularity_bytes(discard_granularity_bytes),
+ m_lock(lock),
+ m_perfcounter(perfcounter) {
+ ldout(pwl.get_context(), 20) << this << dendl;
+}
+
+template <typename T>
+C_DiscardRequest<T>::~C_DiscardRequest() {
+ ldout(pwl.get_context(), 20) << this << dendl;
+}
+
+template <typename T>
+bool C_DiscardRequest<T>::alloc_resources() {
+ ldout(pwl.get_context(), 20) << "req type=" << get_name() << " "
+ << "req=[" << *this << "]" << dendl;
+ return pwl.alloc_resources(this);
+}
+
+template <typename T>
+void C_DiscardRequest<T>::setup_log_operations() {
+ std::lock_guard locker(m_lock);
+ GenericWriteLogEntries log_entries;
+ for (auto &extent : this->image_extents) {
+ op = pwl.m_builder->create_discard_log_operation(
+ pwl.get_current_sync_point(), extent.first, extent.second,
+ m_discard_granularity_bytes, this->m_dispatched_time, m_perfcounter,
+ pwl.get_context());
+ log_entries.emplace_back(op->log_entry);
+ break;
+ }
+ uint64_t current_sync_gen = pwl.get_current_sync_gen();
+ bool persist_on_flush = pwl.get_persist_on_flush();
+ if (!persist_on_flush) {
+ pwl.inc_last_op_sequence_num();
+ }
+ auto discard_req = this;
+ Context *on_write_append = pwl.get_current_sync_point()->prior_persisted_gather_new_sub();
+
+ Context *on_write_persist = new LambdaContext(
+ [this, discard_req](int r) {
+ ldout(pwl.get_context(), 20) << "discard_req=" << discard_req
+ << " cell=" << discard_req->get_cell() << dendl;
+ ceph_assert(discard_req->get_cell());
+ discard_req->complete_user_request(r);
+ discard_req->release_cell();
+ });
+ op->init_op(current_sync_gen, persist_on_flush, pwl.get_last_op_sequence_num(),
+ on_write_persist, on_write_append);
+ pwl.add_into_log_map(log_entries, this);
+}
+
+template <typename T>
+void C_DiscardRequest<T>::dispatch() {
+ utime_t now = ceph_clock_now();
+ ldout(pwl.get_context(), 20) << "req type=" << get_name() << " "
+ << "req=[" << *this << "]" << dendl;
+ ceph_assert(this->m_resources.allocated);
+ this->m_dispatched_time = now;
+ setup_log_operations();
+ m_perfcounter->inc(l_librbd_pwl_log_ops, 1);
+ pwl.schedule_append(op);
+}
+
+template <typename T>
+void C_DiscardRequest<T>::setup_buffer_resources(
+ uint64_t *bytes_cached, uint64_t *bytes_dirtied, uint64_t *bytes_allocated,
+ uint64_t *number_lanes, uint64_t *number_log_entries,
+ uint64_t *number_unpublished_reserves) {
+ *number_log_entries = 1;
+ /* No bytes are allocated for a discard, but we count the discarded bytes
+ * as dirty. This means it's possible to have more bytes dirty than
+ * there are bytes cached or allocated. */
+ for (auto &extent : this->image_extents) {
+ *bytes_dirtied = extent.second;
+ break;
+ }
+}
+
+template <typename T>
+void C_DiscardRequest<T>::blockguard_acquired(GuardedRequestFunctionContext &guard_ctx) {
+ ldout(pwl.get_context(), 20) << " cell=" << guard_ctx.cell << dendl;
+
+ ceph_assert(guard_ctx.cell);
+ this->detained = guard_ctx.state.detained; /* overlapped */
+ this->set_cell(guard_ctx.cell);
+}
+
+template <typename T>
+std::ostream &operator<<(std::ostream &os,
+ const C_DiscardRequest<T> &req) {
+ os << (C_BlockIORequest<T>&)req;
+ if (req.op) {
+ os << " op=[" << *req.op << "]";
+ } else {
+ os << " op=nullptr";
+ }
+ return os;
+}
+
+template <typename T>
+C_WriteSameRequest<T>::C_WriteSameRequest(
+ T &pwl, const utime_t arrived, io::Extents &&image_extents,
+ bufferlist&& bl, const int fadvise_flags, ceph::mutex &lock,
+ PerfCounters *perfcounter, Context *user_req)
+ : C_WriteRequest<T>(pwl, arrived, std::move(image_extents), std::move(bl),
+ fadvise_flags, lock, perfcounter, user_req) {
+ ldout(pwl.get_context(), 20) << this << dendl;
+}
+
+template <typename T>
+C_WriteSameRequest<T>::~C_WriteSameRequest() {
+ ldout(pwl.get_context(), 20) << this << dendl;
+}
+
+template <typename T>
+void C_WriteSameRequest<T>::update_req_stats(utime_t &now) {
+ /* Write same stats excluded from most write stats
+ * because the read phase will make them look like slow writes in
+ * those histograms. */
+ ldout(pwl.get_context(), 20) << this << dendl;
+ utime_t comp_latency = now - this->m_arrived_time;
+ this->m_perfcounter->tinc(l_librbd_pwl_ws_latency, comp_latency);
+}
+
+template <typename T>
+std::shared_ptr<WriteLogOperation> C_WriteSameRequest<T>::create_operation(
+ uint64_t offset, uint64_t len) {
+ ceph_assert(this->image_extents.size() == 1);
+ WriteLogOperationSet &set = *this->op_set.get();
+ return pwl.m_builder->create_write_log_operation(
+ *this->op_set.get(), offset, len, this->bl.length(), pwl.get_context(),
+ pwl.m_builder->create_writesame_log_entry(set.sync_point->log_entry, offset,
+ len, this->bl.length()));
+}
+
+template <typename T>
+std::ostream &operator<<(std::ostream &os,
+ const C_WriteSameRequest<T> &req) {
+ os << (C_WriteRequest<T>&)req;
+ return os;
+}
+
+template <typename T>
+void C_WriteRequest<T>::update_req_stats(utime_t &now) {
+ /* Compare-and-write stats. Compare-and-write excluded from most write
+ * stats because the read phase will make them look like slow writes in
+ * those histograms. */
+ if(is_comp_and_write) {
+ if (!compare_succeeded) {
+ this->m_perfcounter->inc(l_librbd_pwl_cmp_fails, 1);
+ }
+ utime_t comp_latency = now - this->m_arrived_time;
+ this->m_perfcounter->tinc(l_librbd_pwl_cmp_latency, comp_latency);
+ }
+}
+
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+template class librbd::cache::pwl::C_BlockIORequest<librbd::cache::pwl::AbstractWriteLog<librbd::ImageCtx> >;
+template class librbd::cache::pwl::C_WriteRequest<librbd::cache::pwl::AbstractWriteLog<librbd::ImageCtx> >;
+template class librbd::cache::pwl::C_FlushRequest<librbd::cache::pwl::AbstractWriteLog<librbd::ImageCtx> >;
+template class librbd::cache::pwl::C_DiscardRequest<librbd::cache::pwl::AbstractWriteLog<librbd::ImageCtx> >;
+template class librbd::cache::pwl::C_WriteSameRequest<librbd::cache::pwl::AbstractWriteLog<librbd::ImageCtx> >;
diff --git a/src/librbd/cache/pwl/Request.h b/src/librbd/cache/pwl/Request.h
new file mode 100644
index 000000000..7953c2887
--- /dev/null
+++ b/src/librbd/cache/pwl/Request.h
@@ -0,0 +1,374 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_CACHE_PWL_REQUEST_H
+#define CEPH_LIBRBD_CACHE_PWL_REQUEST_H
+
+#include "include/Context.h"
+#include "librbd/cache/pwl/Types.h"
+#include "librbd/cache/pwl/LogOperation.h"
+
+namespace librbd {
+class BlockGuardCell;
+
+namespace cache {
+namespace pwl {
+
+class GuardedRequestFunctionContext;
+
+struct WriteRequestResources {
+ bool allocated = false;
+ std::vector<WriteBufferAllocation> buffers;
+};
+
+/**
+ * A request that can be deferred in a BlockGuard to sequence
+ * overlapping operations.
+ * This is the custodian of the BlockGuard cell for this IO, and the
+ * state information about the progress of this IO. This object lives
+ * until the IO is persisted in all (live) log replicas. User request
+ * may be completed from here before the IO persists.
+ */
+template <typename T>
+class C_BlockIORequest : public Context {
+public:
+ T &pwl;
+ io::Extents image_extents;
+ bufferlist bl;
+ int fadvise_flags;
+ Context *user_req; /* User write request */
+ ExtentsSummary<io::Extents> image_extents_summary;
+ bool detained = false; /* Detained in blockguard (overlapped with a prior IO) */
+ utime_t allocated_time; /* When allocation began */
+ bool waited_lanes = false; /* This IO waited for free persist/replicate lanes */
+ bool waited_entries = false; /* This IO waited for free log entries */
+ bool waited_buffers = false; /* This IO waited for data buffers (pmemobj_reserve() failed) */
+
+ C_BlockIORequest(T &pwl, const utime_t arrived, io::Extents &&extents,
+ bufferlist&& bl, const int fadvise_flags, Context *user_req);
+ ~C_BlockIORequest() override;
+ C_BlockIORequest(const C_BlockIORequest&) = delete;
+ C_BlockIORequest &operator=(const C_BlockIORequest&) = delete;
+
+ void set_cell(BlockGuardCell *cell);
+ BlockGuardCell *get_cell(void);
+ void release_cell();
+
+ void complete_user_request(int r);
+ void finish(int r);
+ virtual void finish_req(int r) = 0;
+
+ virtual bool alloc_resources() = 0;
+
+ void deferred();
+
+ virtual void deferred_handler() = 0;
+
+ virtual void dispatch() = 0;
+
+ virtual void copy_cache() {};
+
+ virtual const char *get_name() const {
+ return "C_BlockIORequest";
+ }
+ uint64_t get_image_extents_size() {
+ return image_extents.size();
+ }
+ void set_io_waited_for_lanes(bool waited) {
+ waited_lanes = waited;
+ }
+ void set_io_waited_for_entries(bool waited) {
+ waited_entries = waited;
+ }
+ void set_io_waited_for_buffers(bool waited) {
+ waited_buffers = waited;
+ }
+ bool has_io_waited_for_buffers() {
+ return waited_buffers;
+ }
+ std::vector<WriteBufferAllocation>& get_resources_buffers() {
+ return m_resources.buffers;
+ }
+
+ void set_allocated(bool allocated) {
+ if (allocated) {
+ m_resources.allocated = true;
+ } else {
+ m_resources.buffers.clear();
+ }
+ }
+
+ virtual void setup_buffer_resources(
+ uint64_t *bytes_cached, uint64_t *bytes_dirtied, uint64_t *bytes_allocated,
+ uint64_t *number_lanes, uint64_t *number_log_entries,
+ uint64_t *number_unpublished_reserves) = 0;
+
+protected:
+ utime_t m_arrived_time;
+ utime_t m_dispatched_time; /* When dispatch began */
+ utime_t m_user_req_completed_time;
+ std::atomic<bool> m_deferred = {false}; /* Deferred because this or a prior IO had to wait for write resources */
+ WriteRequestResources m_resources;
+
+private:
+ std::atomic<bool> m_user_req_completed = {false};
+ std::atomic<bool> m_finish_called = {false};
+ std::atomic<bool> m_cell_released = {false};
+ BlockGuardCell* m_cell = nullptr;
+
+ template <typename U>
+ friend std::ostream &operator<<(std::ostream &os,
+ const C_BlockIORequest<U> &req);
+};
+
+/**
+ * This is the custodian of the BlockGuard cell for this write. Block
+ * guard is not released until the write persists everywhere (this is
+ * how we guarantee to each log replica that they will never see
+ * overlapping writes).
+ */
+template <typename T>
+class C_WriteRequest : public C_BlockIORequest<T> {
+public:
+ using C_BlockIORequest<T>::pwl;
+ bool compare_succeeded = false;
+ uint64_t *mismatch_offset;
+ bufferlist cmp_bl;
+ bufferlist read_bl;
+ bool is_comp_and_write = false;
+ unique_ptr<WriteLogOperationSet> op_set = nullptr;
+
+ C_WriteRequest(T &pwl, const utime_t arrived, io::Extents &&image_extents,
+ bufferlist&& bl, const int fadvise_flags, ceph::mutex &lock,
+ PerfCounters *perfcounter, Context *user_req);
+
+ C_WriteRequest(T &pwl, const utime_t arrived, io::Extents &&image_extents,
+ bufferlist&& cmp_bl, bufferlist&& bl, uint64_t *mismatch_offset,
+ int fadvise_flags, ceph::mutex &lock, PerfCounters *perfcounter,
+ Context *user_req);
+
+ ~C_WriteRequest() override;
+
+ void blockguard_acquired(GuardedRequestFunctionContext &guard_ctx);
+
+ /* Common finish to plain write and compare-and-write (if it writes) */
+ void finish_req(int r) override;
+
+ /* Compare and write will override this */
+ virtual void update_req_stats(utime_t &now);
+
+ bool alloc_resources() override;
+
+ void deferred_handler() override { }
+
+ void dispatch() override;
+
+ void copy_cache() override;
+
+ virtual std::shared_ptr<WriteLogOperation> create_operation(uint64_t offset,
+ uint64_t len);
+
+ virtual void setup_log_operations(DeferredContexts &on_exit);
+
+ bool append_write_request(std::shared_ptr<SyncPoint> sync_point);
+
+ virtual void schedule_append();
+
+ const char *get_name() const override {
+ return "C_WriteRequest";
+ }
+
+protected:
+ using C_BlockIORequest<T>::m_resources;
+ PerfCounters *m_perfcounter = nullptr;
+
+private:
+ bool m_do_early_flush = false;
+ std::atomic<int> m_appended = {0};
+ bool m_queued = false;
+ ceph::mutex &m_lock;
+ template <typename U>
+ friend std::ostream &operator<<(std::ostream &os,
+ const C_WriteRequest<U> &req);
+};
+
+/**
+ * This is the custodian of the BlockGuard cell for this
+ * aio_flush. Block guard is released as soon as the new
+ * sync point (if required) is created. Subsequent IOs can
+ * proceed while this flush waits for prior IOs to complete
+ * and any required sync points to be persisted.
+ */
+template <typename T>
+class C_FlushRequest : public C_BlockIORequest<T> {
+public:
+ using C_BlockIORequest<T>::pwl;
+ bool internal = false;
+ std::shared_ptr<SyncPoint> to_append;
+
+ C_FlushRequest(T &pwl, const utime_t arrived,
+ io::Extents &&image_extents,
+ bufferlist&& bl, const int fadvise_flags,
+ ceph::mutex &lock, PerfCounters *perfcounter,
+ Context *user_req);
+
+ ~C_FlushRequest() override {}
+
+ bool alloc_resources() override;
+
+ void dispatch() override;
+
+ const char *get_name() const override {
+ return "C_FlushRequest";
+ }
+
+ void setup_buffer_resources(
+ uint64_t *bytes_cached, uint64_t *bytes_dirtied,
+ uint64_t *bytes_allocated, uint64_t *number_lanes,
+ uint64_t *number_log_entries,
+ uint64_t *number_unpublished_reserves) override;
+private:
+ std::shared_ptr<SyncPointLogOperation> op;
+ ceph::mutex &m_lock;
+ PerfCounters *m_perfcounter = nullptr;
+
+ void finish_req(int r) override;
+ void deferred_handler() override {
+ m_perfcounter->inc(l_librbd_pwl_aio_flush_def, 1);
+ }
+
+ template <typename U>
+ friend std::ostream &operator<<(std::ostream &os,
+ const C_FlushRequest<U> &req);
+};
+
+/**
+ * This is the custodian of the BlockGuard cell for this discard. As in the
+ * case of write, the block guard is not released until the discard persists
+ * everywhere.
+ */
+template <typename T>
+class C_DiscardRequest : public C_BlockIORequest<T> {
+public:
+ using C_BlockIORequest<T>::pwl;
+ std::shared_ptr<DiscardLogOperation> op;
+
+ C_DiscardRequest(T &pwl, const utime_t arrived, io::Extents &&image_extents,
+ uint32_t discard_granularity_bytes, ceph::mutex &lock,
+ PerfCounters *perfcounter, Context *user_req);
+
+ ~C_DiscardRequest() override;
+ void finish_req(int r) override {}
+
+ bool alloc_resources() override;
+
+ void deferred_handler() override { }
+
+ void setup_log_operations();
+
+ void dispatch() override;
+
+ void blockguard_acquired(GuardedRequestFunctionContext &guard_ctx);
+
+ const char *get_name() const override {
+ return "C_DiscardRequest";
+ }
+ void setup_buffer_resources(
+ uint64_t *bytes_cached, uint64_t *bytes_dirtied, uint64_t *bytes_allocated,
+ uint64_t *number_lanes, uint64_t *number_log_entries,
+ uint64_t *number_unpublished_reserves) override;
+private:
+ uint32_t m_discard_granularity_bytes;
+ ceph::mutex &m_lock;
+ PerfCounters *m_perfcounter = nullptr;
+ template <typename U>
+ friend std::ostream &operator<<(std::ostream &os,
+ const C_DiscardRequest<U> &req);
+};
+
+/**
+ * This is the custodian of the BlockGuard cell for this write same.
+ *
+ * A writesame allocates and persists a data buffer like a write, but the
+ * data buffer is usually much shorter than the write same.
+ */
+template <typename T>
+class C_WriteSameRequest : public C_WriteRequest<T> {
+public:
+ using C_BlockIORequest<T>::pwl;
+ C_WriteSameRequest(T &pwl, const utime_t arrived, io::Extents &&image_extents,
+ bufferlist&& bl, const int fadvise_flags, ceph::mutex &lock,
+ PerfCounters *perfcounter, Context *user_req);
+
+ ~C_WriteSameRequest() override;
+
+ void update_req_stats(utime_t &now) override;
+
+ std::shared_ptr<WriteLogOperation> create_operation(uint64_t offset, uint64_t len) override;
+
+ const char *get_name() const override {
+ return "C_WriteSameRequest";
+ }
+
+ template<typename U>
+ friend std::ostream &operator<<(std::ostream &os,
+ const C_WriteSameRequest<U> &req);
+};
+
+struct BlockGuardReqState {
+ bool barrier = false; /* This is a barrier request */
+ bool current_barrier = false; /* This is the currently active barrier */
+ bool detained = false;
+ bool queued = false; /* Queued for barrier */
+ friend std::ostream &operator<<(std::ostream &os,
+ const BlockGuardReqState &r) {
+ os << "barrier=" << r.barrier << ", "
+ << "current_barrier=" << r.current_barrier << ", "
+ << "detained=" << r.detained << ", "
+ << "queued=" << r.queued;
+ return os;
+ }
+};
+
+class GuardedRequestFunctionContext : public Context {
+public:
+ BlockGuardCell *cell = nullptr;
+ BlockGuardReqState state;
+ GuardedRequestFunctionContext(boost::function<void(GuardedRequestFunctionContext&)> &&callback)
+ : m_callback(std::move(callback)){ }
+ ~GuardedRequestFunctionContext(void) override { };
+ GuardedRequestFunctionContext(const GuardedRequestFunctionContext&) = delete;
+ GuardedRequestFunctionContext &operator=(const GuardedRequestFunctionContext&) = delete;
+
+private:
+ boost::function<void(GuardedRequestFunctionContext&)> m_callback;
+ void finish(int r) override {
+ ceph_assert(cell);
+ m_callback(*this);
+ }
+};
+
+class GuardedRequest {
+public:
+ const BlockExtent block_extent;
+ GuardedRequestFunctionContext *guard_ctx; /* Work to do when guard on range obtained */
+
+ GuardedRequest(const BlockExtent block_extent,
+ GuardedRequestFunctionContext *on_guard_acquire, bool barrier = false)
+ : block_extent(block_extent), guard_ctx(on_guard_acquire) {
+ guard_ctx->state.barrier = barrier;
+ }
+ friend std::ostream &operator<<(std::ostream &os,
+ const GuardedRequest &r) {
+ os << "guard_ctx->state=[" << r.guard_ctx->state << "], "
+ << "block_extent.block_start=" << r.block_extent.block_start << ", "
+ << "block_extent.block_start=" << r.block_extent.block_end;
+ return os;
+ }
+};
+
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+#endif // CEPH_LIBRBD_CACHE_PWL_REQUEST_H
diff --git a/src/librbd/cache/pwl/ShutdownRequest.cc b/src/librbd/cache/pwl/ShutdownRequest.cc
new file mode 100644
index 000000000..e022328ba
--- /dev/null
+++ b/src/librbd/cache/pwl/ShutdownRequest.cc
@@ -0,0 +1,161 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "librbd/cache/pwl/ShutdownRequest.h"
+#include "librbd/ImageCtx.h"
+#include "librbd/Utils.h"
+#include "common/dout.h"
+#include "common/errno.h"
+#include "librbd/Operations.h"
+#include "librbd/asio/ContextWQ.h"
+#include "librbd/cache/Types.h"
+
+#include "librbd/cache/pwl/AbstractWriteLog.h"
+#include "librbd/plugin/Api.h"
+
+#define dout_subsys ceph_subsys_rbd_pwl
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::cache::pwl:ShutdownRequest: " \
+ << this << " " << __func__ << ": "
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+
+using librbd::util::create_async_context_callback;
+using librbd::util::create_context_callback;
+
+template <typename I>
+ShutdownRequest<I>* ShutdownRequest<I>::create(
+ I &image_ctx,
+ AbstractWriteLog<I> *image_cache,
+ plugin::Api<I>& plugin_api,
+ Context *on_finish) {
+ return new ShutdownRequest(image_ctx, image_cache, plugin_api, on_finish);
+}
+
+template <typename I>
+ShutdownRequest<I>::ShutdownRequest(
+ I &image_ctx,
+ AbstractWriteLog<I> *image_cache,
+ plugin::Api<I>& plugin_api,
+ Context *on_finish)
+ : m_image_ctx(image_ctx),
+ m_image_cache(image_cache),
+ m_plugin_api(plugin_api),
+ m_on_finish(create_async_context_callback(image_ctx, on_finish)),
+ m_error_result(0) {
+}
+
+template <typename I>
+void ShutdownRequest<I>::send() {
+ send_shutdown_image_cache();
+}
+
+template <typename I>
+void ShutdownRequest<I>::send_shutdown_image_cache() {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << dendl;
+
+ if (m_image_cache == nullptr) {
+ finish();
+ return;
+ }
+
+ using klass = ShutdownRequest<I>;
+ Context *ctx = create_context_callback<klass, &klass::handle_shutdown_image_cache>(
+ this);
+
+ m_image_cache->shut_down(ctx);
+}
+
+template <typename I>
+void ShutdownRequest<I>::handle_shutdown_image_cache(int r) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << dendl;
+
+ if (r < 0) {
+ lderr(cct) << "failed to shut down the image cache: " << cpp_strerror(r)
+ << dendl;
+ save_result(r);
+ finish();
+ return;
+ } else {
+ delete m_image_cache;
+ m_image_cache = nullptr;
+ }
+ send_remove_feature_bit();
+}
+
+template <typename I>
+void ShutdownRequest<I>::send_remove_feature_bit() {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << dendl;
+
+ uint64_t new_features = m_image_ctx.features & ~RBD_FEATURE_DIRTY_CACHE;
+ uint64_t features_mask = RBD_FEATURE_DIRTY_CACHE;
+ ldout(cct, 10) << "old_features=" << m_image_ctx.features
+ << ", new_features=" << new_features
+ << ", features_mask=" << features_mask
+ << dendl;
+
+ int r = librbd::cls_client::set_features(&m_image_ctx.md_ctx, m_image_ctx.header_oid,
+ new_features, features_mask);
+ m_image_ctx.features &= ~RBD_FEATURE_DIRTY_CACHE;
+ using klass = ShutdownRequest<I>;
+ Context *ctx = create_context_callback<klass, &klass::handle_remove_feature_bit>(
+ this);
+ ctx->complete(r);
+}
+
+template <typename I>
+void ShutdownRequest<I>::handle_remove_feature_bit(int r) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << dendl;
+
+ if (r < 0) {
+ lderr(cct) << "failed to remove the feature bit: " << cpp_strerror(r)
+ << dendl;
+ save_result(r);
+ finish();
+ return;
+ }
+ send_remove_image_cache_state();
+}
+
+template <typename I>
+void ShutdownRequest<I>::send_remove_image_cache_state() {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << dendl;
+
+ using klass = ShutdownRequest<I>;
+ Context *ctx = create_context_callback<klass, &klass::handle_remove_image_cache_state>(
+ this);
+ std::shared_lock owner_lock{m_image_ctx.owner_lock};
+ m_plugin_api.execute_image_metadata_remove(&m_image_ctx, PERSISTENT_CACHE_STATE, ctx);
+}
+
+template <typename I>
+void ShutdownRequest<I>::handle_remove_image_cache_state(int r) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 10) << dendl;
+
+ if (r < 0) {
+ lderr(cct) << "failed to remove the image cache state: " << cpp_strerror(r)
+ << dendl;
+ save_result(r);
+ }
+ finish();
+}
+
+template <typename I>
+void ShutdownRequest<I>::finish() {
+ m_on_finish->complete(m_error_result);
+ delete this;
+}
+
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+template class librbd::cache::pwl::ShutdownRequest<librbd::ImageCtx>;
diff --git a/src/librbd/cache/pwl/ShutdownRequest.h b/src/librbd/cache/pwl/ShutdownRequest.h
new file mode 100644
index 000000000..dafac9e9c
--- /dev/null
+++ b/src/librbd/cache/pwl/ShutdownRequest.h
@@ -0,0 +1,95 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_CACHE_RWL_SHUTDOWN_REQUEST_H
+#define CEPH_LIBRBD_CACHE_RWL_SHUTDOWN_REQUEST_H
+
+class Context;
+
+namespace librbd {
+
+class ImageCtx;
+
+namespace plugin { template <typename> struct Api; }
+
+namespace cache {
+namespace pwl {
+
+template<typename>
+class AbstractWriteLog;
+
+template<typename>
+class ImageCacheState;
+
+template <typename ImageCtxT = ImageCtx>
+class ShutdownRequest {
+public:
+ static ShutdownRequest* create(
+ ImageCtxT &image_ctx,
+ AbstractWriteLog<ImageCtxT> *image_cache,
+ plugin::Api<ImageCtxT>& plugin_api,
+ Context *on_finish);
+
+ void send();
+
+private:
+
+ /**
+ * @verbatim
+ *
+ * Shutdown request goes through the following state machine:
+ *
+ * <start>
+ * |
+ * v
+ * SHUTDOWN_IMAGE_CACHE
+ * |
+ * v
+ * REMOVE_IMAGE_FEATURE_BIT
+ * |
+ * v
+ * REMOVE_IMAGE_CACHE_STATE
+ * |
+ * v
+ * <finish>
+ *
+ * @endverbatim
+ */
+
+ ShutdownRequest(ImageCtxT &image_ctx,
+ AbstractWriteLog<ImageCtxT> *image_cache,
+ plugin::Api<ImageCtxT>& plugin_api,
+ Context *on_finish);
+
+ ImageCtxT &m_image_ctx;
+ AbstractWriteLog<ImageCtxT> *m_image_cache;
+ plugin::Api<ImageCtxT>& m_plugin_api;
+ Context *m_on_finish;
+
+ int m_error_result;
+
+ void send_shutdown_image_cache();
+ void handle_shutdown_image_cache(int r);
+
+ void send_remove_feature_bit();
+ void handle_remove_feature_bit(int r);
+
+ void send_remove_image_cache_state();
+ void handle_remove_image_cache_state(int r);
+
+ void finish();
+
+ void save_result(int result) {
+ if (m_error_result == 0 && result < 0) {
+ m_error_result = result;
+ }
+ }
+};
+
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+extern template class librbd::cache::pwl::ShutdownRequest<librbd::ImageCtx>;
+
+#endif // CEPH_LIBRBD_CACHE_RWL_SHUTDOWN_REQUEST_H
diff --git a/src/librbd/cache/pwl/SyncPoint.cc b/src/librbd/cache/pwl/SyncPoint.cc
new file mode 100644
index 000000000..8fb2f8205
--- /dev/null
+++ b/src/librbd/cache/pwl/SyncPoint.cc
@@ -0,0 +1,109 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "SyncPoint.h"
+
+#define dout_subsys ceph_subsys_rbd_pwl
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::cache::pwl::SyncPoint: " << this << " " \
+ << __func__ << ": "
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+
+SyncPoint::SyncPoint(uint64_t sync_gen_num, CephContext *cct)
+ : log_entry(std::make_shared<SyncPointLogEntry>(sync_gen_num)), m_cct(cct) {
+ m_prior_log_entries_persisted = new C_Gather(cct, nullptr);
+ m_sync_point_persist = new C_Gather(cct, nullptr);
+ on_sync_point_appending.reserve(MAX_WRITES_PER_SYNC_POINT + 2);
+ on_sync_point_persisted.reserve(MAX_WRITES_PER_SYNC_POINT + 2);
+ ldout(m_cct, 20) << "sync point " << sync_gen_num << dendl;
+}
+
+SyncPoint::~SyncPoint() {
+ ceph_assert(on_sync_point_appending.empty());
+ ceph_assert(on_sync_point_persisted.empty());
+ ceph_assert(!earlier_sync_point);
+}
+
+std::ostream &operator<<(std::ostream &os,
+ const SyncPoint &p) {
+ os << "log_entry=[" << *p.log_entry << "], "
+ << "earlier_sync_point=" << p.earlier_sync_point << ", "
+ << "later_sync_point=" << p.later_sync_point << ", "
+ << "m_final_op_sequence_num=" << p.m_final_op_sequence_num << ", "
+ << "m_prior_log_entries_persisted=" << p.m_prior_log_entries_persisted << ", "
+ << "m_prior_log_entries_persisted_complete=" << p.m_prior_log_entries_persisted_complete << ", "
+ << "m_append_scheduled=" << p.m_append_scheduled << ", "
+ << "appending=" << p.appending << ", "
+ << "on_sync_point_appending=" << p.on_sync_point_appending.size() << ", "
+ << "on_sync_point_persisted=" << p.on_sync_point_persisted.size() << "";
+ return os;
+}
+
+void SyncPoint::persist_gather_set_finisher(Context *ctx) {
+ m_append_scheduled = true;
+ /* All prior sync points that are still in this list must already be scheduled for append */
+ std::shared_ptr<SyncPoint> previous = earlier_sync_point;
+ while (previous) {
+ ceph_assert(previous->m_append_scheduled);
+ previous = previous->earlier_sync_point;
+ }
+
+ m_sync_point_persist->set_finisher(ctx);
+}
+
+void SyncPoint::persist_gather_activate() {
+ m_sync_point_persist->activate();
+}
+
+Context* SyncPoint::persist_gather_new_sub() {
+ return m_sync_point_persist->new_sub();
+}
+
+void SyncPoint::prior_persisted_gather_activate() {
+ m_prior_log_entries_persisted->activate();
+}
+
+Context* SyncPoint::prior_persisted_gather_new_sub() {
+ return m_prior_log_entries_persisted->new_sub();
+}
+
+void SyncPoint::prior_persisted_gather_set_finisher() {
+ Context *sync_point_persist_ready = persist_gather_new_sub();
+ std::shared_ptr<SyncPoint> sp = shared_from_this();
+ m_prior_log_entries_persisted->
+ set_finisher(new LambdaContext([this, sp, sync_point_persist_ready](int r) {
+ ldout(m_cct, 20) << "Prior log entries persisted for sync point =["
+ << sp << "]" << dendl;
+ sp->m_prior_log_entries_persisted_result = r;
+ sp->m_prior_log_entries_persisted_complete = true;
+ sync_point_persist_ready->complete(r);
+ }));
+}
+
+void SyncPoint::add_in_on_persisted_ctxs(Context* ctx) {
+ on_sync_point_persisted.push_back(ctx);
+}
+
+void SyncPoint::add_in_on_appending_ctxs(Context* ctx) {
+ on_sync_point_appending.push_back(ctx);
+}
+
+void SyncPoint::setup_earlier_sync_point(std::shared_ptr<SyncPoint> sync_point,
+ uint64_t last_op_sequence_num) {
+ earlier_sync_point = sync_point;
+ log_entry->prior_sync_point_flushed = false;
+ earlier_sync_point->log_entry->next_sync_point_entry = log_entry;
+ earlier_sync_point->later_sync_point = shared_from_this();
+ earlier_sync_point->m_final_op_sequence_num = last_op_sequence_num;
+ if (!earlier_sync_point->appending) {
+ /* Append of new sync point deferred until old sync point is appending */
+ earlier_sync_point->add_in_on_appending_ctxs(prior_persisted_gather_new_sub());
+ }
+}
+
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
diff --git a/src/librbd/cache/pwl/SyncPoint.h b/src/librbd/cache/pwl/SyncPoint.h
new file mode 100644
index 000000000..424e3730e
--- /dev/null
+++ b/src/librbd/cache/pwl/SyncPoint.h
@@ -0,0 +1,69 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_CACHE_RWL_SYNC_POINT_H
+#define CEPH_LIBRBD_CACHE_RWL_SYNC_POINT_H
+
+#include "librbd/ImageCtx.h"
+#include "librbd/cache/pwl/LogEntry.h"
+#include "librbd/cache/pwl/Types.h"
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+
+class SyncPoint: public std::enable_shared_from_this<SyncPoint> {
+public:
+ std::shared_ptr<SyncPointLogEntry> log_entry;
+ /* Use lock for earlier/later links */
+ std::shared_ptr<SyncPoint> earlier_sync_point; /* NULL if earlier has completed */
+ std::shared_ptr<SyncPoint> later_sync_point;
+ bool appending = false;
+ /* Signal these when this sync point is appending to the log, and its order
+ * of appearance is guaranteed. One of these is is a sub-operation of the
+ * next sync point's m_prior_log_entries_persisted Gather. */
+ std::vector<Context*> on_sync_point_appending;
+ /* Signal these when this sync point is appended and persisted. User
+ * aio_flush() calls are added to this. */
+ std::vector<Context*> on_sync_point_persisted;
+
+ SyncPoint(uint64_t sync_gen_num, CephContext *cct);
+ ~SyncPoint();
+ SyncPoint(const SyncPoint&) = delete;
+ SyncPoint &operator=(const SyncPoint&) = delete;
+ void persist_gather_activate();
+ Context* persist_gather_new_sub();
+ void persist_gather_set_finisher(Context *ctx);
+ void prior_persisted_gather_activate();
+ Context* prior_persisted_gather_new_sub();
+ void prior_persisted_gather_set_finisher();
+ void add_in_on_persisted_ctxs(Context* cxt);
+ void add_in_on_appending_ctxs(Context* cxt);
+ void setup_earlier_sync_point(std::shared_ptr<SyncPoint> sync_point,
+ uint64_t last_op_sequence_num);
+private:
+ CephContext *m_cct;
+ bool m_append_scheduled = false;
+ uint64_t m_final_op_sequence_num = 0;
+ /* A sync point can't appear in the log until all the writes bearing
+ * it and all the prior sync points have been appended and
+ * persisted.
+ *
+ * Writes bearing this sync gen number and the prior sync point will be
+ * sub-ops of this Gather. This sync point will not be appended until all
+ * these complete to the point where their persist order is guaranteed. */
+ C_Gather *m_prior_log_entries_persisted;
+ /* The finisher for this will append the sync point to the log. The finisher
+ * for m_prior_log_entries_persisted will be a sub-op of this. */
+ C_Gather *m_sync_point_persist;
+ int m_prior_log_entries_persisted_result = 0;
+ int m_prior_log_entries_persisted_complete = false;
+ friend std::ostream &operator<<(std::ostream &os,
+ const SyncPoint &p);
+};
+
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+#endif // CEPH_LIBRBD_CACHE_RWL_SYNC_POINT_H
diff --git a/src/librbd/cache/pwl/Types.cc b/src/librbd/cache/pwl/Types.cc
new file mode 100644
index 000000000..505f5d57b
--- /dev/null
+++ b/src/librbd/cache/pwl/Types.cc
@@ -0,0 +1,185 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include <iostream>
+#include "Types.h"
+#include "common/ceph_context.h"
+#include "include/Context.h"
+#include "include/stringify.h"
+
+#define dout_subsys ceph_subsys_rbd_pwl
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::cache::pwl::Types: " << this << " " \
+ << __func__ << ": "
+using ceph::Formatter;
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+
+DeferredContexts::~DeferredContexts() {
+ finish_contexts(nullptr, contexts, 0);
+}
+
+void DeferredContexts::add(Context* ctx) {
+ contexts.push_back(ctx);
+}
+
+/*
+ * A BlockExtent identifies a range by first and last.
+ *
+ * An Extent ("image extent") identifies a range by start and length.
+ *
+ * The ImageDispatch interface is defined in terms of image extents, and
+ * requires no alignment of the beginning or end of the extent. We
+ * convert between image and block extents here using a "block size"
+ * of 1.
+ */
+BlockExtent convert_to_block_extent(uint64_t offset_bytes, uint64_t length_bytes)
+{
+ return BlockExtent(offset_bytes,
+ offset_bytes + length_bytes);
+}
+
+BlockExtent WriteLogCacheEntry::block_extent() {
+ return convert_to_block_extent(image_offset_bytes, write_bytes);
+}
+
+uint64_t WriteLogCacheEntry::get_offset_bytes() {
+ return image_offset_bytes;
+}
+
+uint64_t WriteLogCacheEntry::get_write_bytes() {
+ return write_bytes;
+}
+
+#ifdef WITH_RBD_SSD_CACHE
+void WriteLogCacheEntry::dump(Formatter *f) const {
+ f->dump_unsigned("sync_gen_number", sync_gen_number);
+ f->dump_unsigned("write_sequence_number", write_sequence_number);
+ f->dump_unsigned("image_offset_bytes", image_offset_bytes);
+ f->dump_unsigned("write_bytes", write_bytes);
+ f->dump_unsigned("write_data_pos", write_data_pos);
+ f->dump_bool("entry_valid", is_entry_valid());
+ f->dump_bool("sync_point", is_sync_point());
+ f->dump_bool("sequenced", is_sequenced());
+ f->dump_bool("has_data", has_data());
+ f->dump_bool("discard", is_discard());
+ f->dump_bool("writesame", is_writesame());
+ f->dump_unsigned("ws_datalen", ws_datalen);
+ f->dump_unsigned("entry_index", entry_index);
+}
+
+void WriteLogCacheEntry::generate_test_instances(list<WriteLogCacheEntry*>& ls) {
+ ls.push_back(new WriteLogCacheEntry());
+ ls.push_back(new WriteLogCacheEntry);
+ ls.back()->sync_gen_number = 1;
+ ls.back()->write_sequence_number = 1;
+ ls.back()->image_offset_bytes = 1;
+ ls.back()->write_bytes = 1;
+ ls.back()->write_data_pos = 1;
+ ls.back()->set_entry_valid(true);
+ ls.back()->set_sync_point(true);
+ ls.back()->set_sequenced(true);
+ ls.back()->set_has_data(true);
+ ls.back()->set_discard(true);
+ ls.back()->set_writesame(true);
+ ls.back()->ws_datalen = 1;
+ ls.back()->entry_index = 1;
+}
+
+void WriteLogPoolRoot::dump(Formatter *f) const {
+ f->dump_unsigned("layout_version", layout_version);
+ f->dump_unsigned("cur_sync_gen", cur_sync_gen);
+ f->dump_unsigned("pool_size", pool_size);
+ f->dump_unsigned("flushed_sync_gen", flushed_sync_gen);
+ f->dump_unsigned("block_size", block_size);
+ f->dump_unsigned("num_log_entries", num_log_entries);
+ f->dump_unsigned("first_free_entry", first_free_entry);
+ f->dump_unsigned("first_valid_entry", first_valid_entry);
+}
+
+void WriteLogPoolRoot::generate_test_instances(list<WriteLogPoolRoot*>& ls) {
+ ls.push_back(new WriteLogPoolRoot());
+ ls.push_back(new WriteLogPoolRoot);
+ ls.back()->layout_version = 2;
+ ls.back()->cur_sync_gen = 1;
+ ls.back()->pool_size = 1024;
+ ls.back()->flushed_sync_gen = 1;
+ ls.back()->block_size = 4096;
+ ls.back()->num_log_entries = 10000000;
+ ls.back()->first_free_entry = 1;
+ ls.back()->first_valid_entry = 0;
+}
+#endif
+
+std::ostream& operator<<(std::ostream& os,
+ const WriteLogCacheEntry &entry) {
+ os << "entry_valid=" << entry.is_entry_valid() << ", "
+ << "sync_point=" << entry.is_sync_point() << ", "
+ << "sequenced=" << entry.is_sequenced() << ", "
+ << "has_data=" << entry.has_data() << ", "
+ << "discard=" << entry.is_discard() << ", "
+ << "writesame=" << entry.is_writesame() << ", "
+ << "sync_gen_number=" << entry.sync_gen_number << ", "
+ << "write_sequence_number=" << entry.write_sequence_number << ", "
+ << "image_offset_bytes=" << entry.image_offset_bytes << ", "
+ << "write_bytes=" << entry.write_bytes << ", "
+ << "ws_datalen=" << entry.ws_datalen << ", "
+ << "entry_index=" << entry.entry_index;
+ return os;
+}
+
+template <typename ExtentsType>
+ExtentsSummary<ExtentsType>::ExtentsSummary(const ExtentsType &extents)
+ : total_bytes(0), first_image_byte(0), last_image_byte(0)
+{
+ if (extents.empty()) return;
+ /* These extents refer to image offsets between first_image_byte
+ * and last_image_byte, inclusive, but we don't guarantee here
+ * that they address all of those bytes. There may be gaps. */
+ first_image_byte = extents.front().first;
+ last_image_byte = first_image_byte + extents.front().second;
+ for (auto &extent : extents) {
+ /* Ignore zero length extents */
+ if (extent.second) {
+ total_bytes += extent.second;
+ if (extent.first < first_image_byte) {
+ first_image_byte = extent.first;
+ }
+ if ((extent.first + extent.second) > last_image_byte) {
+ last_image_byte = extent.first + extent.second;
+ }
+ }
+ }
+}
+
+io::Extent whole_volume_extent() {
+ return io::Extent({0, std::numeric_limits<uint64_t>::max()});
+}
+
+BlockExtent block_extent(const io::Extent& image_extent) {
+ return convert_to_block_extent(image_extent.first, image_extent.second);
+}
+
+Context * override_ctx(int r, Context *ctx) {
+ if (r < 0) {
+ /* Override next_ctx status with this error */
+ return new LambdaContext(
+ [r, ctx](int _r) {
+ ctx->complete(r);
+ });
+ } else {
+ return ctx;
+ }
+}
+
+std::string unique_lock_name(const std::string &name, void *address) {
+ return name + " (" + stringify(address) + ")";
+}
+
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+template class librbd::cache::pwl::ExtentsSummary<librbd::io::Extents>;
diff --git a/src/librbd/cache/pwl/Types.h b/src/librbd/cache/pwl/Types.h
new file mode 100644
index 000000000..dc067612b
--- /dev/null
+++ b/src/librbd/cache/pwl/Types.h
@@ -0,0 +1,444 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_CACHE_PWL_TYPES_H
+#define CEPH_LIBRBD_CACHE_PWL_TYPES_H
+
+#include "acconfig.h"
+
+#ifdef WITH_RBD_RWL
+#include "libpmemobj.h"
+#endif
+
+#include <vector>
+#include "librbd/BlockGuard.h"
+#include "librbd/io/Types.h"
+
+namespace ceph {
+class Formatter;
+}
+
+class Context;
+
+enum {
+ l_librbd_pwl_first = 26500,
+
+ // All read requests
+ l_librbd_pwl_rd_req, // read requests
+ l_librbd_pwl_rd_bytes, // bytes read
+ l_librbd_pwl_rd_latency, // average req completion latency
+
+ // Read requests completed from RWL (no misses)
+ l_librbd_pwl_rd_hit_req, // read requests
+ l_librbd_pwl_rd_hit_bytes, // bytes read
+ l_librbd_pwl_rd_hit_latency, // average req completion latency
+
+ // Reed requests with hit and miss extents
+ l_librbd_pwl_rd_part_hit_req, // read ops
+
+ // Per SyncPoint's LogEntry number and write bytes distribution
+ l_librbd_pwl_syncpoint_hist,
+
+ // All write requests
+ l_librbd_pwl_wr_req, // write requests
+ l_librbd_pwl_wr_bytes, // bytes written
+ l_librbd_pwl_wr_req_def, // write requests deferred for resources
+ l_librbd_pwl_wr_req_def_lanes, // write requests deferred for lanes
+ l_librbd_pwl_wr_req_def_log, // write requests deferred for log entries
+ l_librbd_pwl_wr_req_def_buf, // write requests deferred for buffer space
+ l_librbd_pwl_wr_req_overlap, // write requests detained for overlap
+ l_librbd_pwl_wr_req_queued, // write requests queued for prior barrier
+
+ // Write log operations (1 .. n per request that appends to the log)
+ l_librbd_pwl_log_ops, // log append ops
+ l_librbd_pwl_log_op_bytes, // average bytes written per log op
+
+ /*
+
+ Req and op average latencies to the beginning of and over various phases:
+
+ +------------------------------+------+-------------------------------+
+ | Phase | Name | Description |
+ +------------------------------+------+-------------------------------+
+ | Arrive at RWL | arr |Arrives as a request |
+ +------------------------------+------+-------------------------------+
+ | Allocate resources | all |time spent in block guard for |
+ | | |overlap sequencing occurs |
+ | | |before this point |
+ +------------------------------+------+-------------------------------+
+ | Dispatch | dis |Op lifetime begins here. time |
+ | | |spent in allocation waiting for|
+ | | |resources occurs before this |
+ | | |point |
+ +------------------------------+------+-------------------------------+
+ | Payload buffer persist and | buf |time spent queued for |
+ |replicate | |replication occurs before here |
+ +------------------------------+------+-------------------------------+
+ | Payload buffer persist | bufc |bufc - buf is just the persist |
+ |complete | |time |
+ +------------------------------+------+-------------------------------+
+ | Log append | app |time spent queued for append |
+ | | |occurs before here |
+ +------------------------------+------+-------------------------------+
+ | Append complete | appc |appc - app is just the time |
+ | | |spent in the append operation |
+ +------------------------------+------+-------------------------------+
+ | Complete | cmp |write persisted, replicated, |
+ | | |and globally visible |
+ +------------------------------+------+-------------------------------+
+
+ */
+
+ /* Request times */
+ l_librbd_pwl_req_arr_to_all_t, // arrival to allocation elapsed time - same as time deferred in block guard
+ l_librbd_pwl_req_arr_to_dis_t, // arrival to dispatch elapsed time
+ l_librbd_pwl_req_all_to_dis_t, // Time spent allocating or waiting to allocate resources
+ l_librbd_pwl_wr_latency, // average req (persist) completion latency
+ l_librbd_pwl_wr_latency_hist, // Histogram of write req (persist) completion latency vs. bytes written
+ l_librbd_pwl_wr_caller_latency, // average req completion (to caller) latency
+
+ /* Request times for requests that never waited for space*/
+ l_librbd_pwl_nowait_req_arr_to_all_t, // arrival to allocation elapsed time - same as time deferred in block guard
+ l_librbd_pwl_nowait_req_arr_to_dis_t, // arrival to dispatch elapsed time
+ l_librbd_pwl_nowait_req_all_to_dis_t, // Time spent allocating or waiting to allocate resources
+ l_librbd_pwl_nowait_wr_latency, // average req (persist) completion latency
+ l_librbd_pwl_nowait_wr_latency_hist, // Histogram of write req (persist) completion latency vs. bytes written
+ l_librbd_pwl_nowait_wr_caller_latency, // average req completion (to caller) latency
+
+ /* Log operation times */
+ l_librbd_pwl_log_op_alloc_t, // elapsed time of pmemobj_reserve()
+ l_librbd_pwl_log_op_alloc_t_hist, // Histogram of elapsed time of pmemobj_reserve()
+
+ l_librbd_pwl_log_op_dis_to_buf_t, // dispatch to buffer persist elapsed time
+ l_librbd_pwl_log_op_dis_to_app_t, // dispatch to log append elapsed time
+ l_librbd_pwl_log_op_dis_to_cmp_t, // dispatch to persist completion elapsed time
+ l_librbd_pwl_log_op_dis_to_cmp_t_hist, // Histogram of dispatch to persist completion elapsed time
+
+ l_librbd_pwl_log_op_buf_to_app_t, // data buf persist + append wait time
+ l_librbd_pwl_log_op_buf_to_bufc_t,// data buf persist / replicate elapsed time
+ l_librbd_pwl_log_op_buf_to_bufc_t_hist,// data buf persist time vs bytes histogram
+ l_librbd_pwl_log_op_app_to_cmp_t, // log entry append + completion wait time
+ l_librbd_pwl_log_op_app_to_appc_t, // log entry append / replicate elapsed time
+ l_librbd_pwl_log_op_app_to_appc_t_hist, // log entry append time (vs. op bytes) histogram
+
+ l_librbd_pwl_discard,
+ l_librbd_pwl_discard_bytes,
+ l_librbd_pwl_discard_latency,
+
+ l_librbd_pwl_aio_flush,
+ l_librbd_pwl_aio_flush_def,
+ l_librbd_pwl_aio_flush_latency,
+ l_librbd_pwl_ws,
+ l_librbd_pwl_ws_bytes, // Bytes modified by write same, probably much larger than WS payload bytes
+ l_librbd_pwl_ws_latency,
+
+ l_librbd_pwl_cmp,
+ l_librbd_pwl_cmp_bytes,
+ l_librbd_pwl_cmp_latency,
+ l_librbd_pwl_cmp_fails,
+
+ l_librbd_pwl_internal_flush,
+ l_librbd_pwl_writeback_latency,
+ l_librbd_pwl_invalidate_cache,
+ l_librbd_pwl_invalidate_discard_cache,
+
+ l_librbd_pwl_append_tx_t,
+ l_librbd_pwl_retire_tx_t,
+ l_librbd_pwl_append_tx_t_hist,
+ l_librbd_pwl_retire_tx_t_hist,
+
+ l_librbd_pwl_last,
+};
+
+enum {
+ WRITE_LOG_CACHE_ENTRY_VALID = 1U << 0, /* if 0, this entry is free */
+ WRITE_LOG_CACHE_ENTRY_SYNC_POINT = 1U << 1, /* No data. No write sequence number.
+ Marks sync point for this sync gen number */
+ WRITE_LOG_CACHE_ENTRY_SEQUENCED = 1U << 2, /* write sequence number is valid */
+ WRITE_LOG_CACHE_ENTRY_HAS_DATA = 1U << 3, /* write_data field is valid (else ignore) */
+ WRITE_LOG_CACHE_ENTRY_DISCARD = 1U << 4, /* has_data will be 0 if this is a discard */
+ WRITE_LOG_CACHE_ENTRY_WRITESAME = 1U << 5, /* ws_datalen indicates length of data at write_bytes */
+};
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+
+class ImageExtentBuf;
+
+const int IN_FLIGHT_FLUSH_WRITE_LIMIT = 64;
+const int IN_FLIGHT_FLUSH_BYTES_LIMIT = (1 * 1024 * 1024);
+
+/* Limit work between sync points */
+const uint64_t MAX_WRITES_PER_SYNC_POINT = 256;
+const uint64_t MAX_BYTES_PER_SYNC_POINT = (1024 * 1024 * 8);
+
+const uint32_t MIN_WRITE_ALLOC_SIZE = 512;
+const uint32_t MIN_WRITE_ALLOC_SSD_SIZE = 4096;
+const uint32_t LOG_STATS_INTERVAL_SECONDS = 5;
+
+/**** Write log entries ****/
+const unsigned long int MAX_ALLOC_PER_TRANSACTION = 8;
+const unsigned long int MAX_FREE_PER_TRANSACTION = 1;
+const unsigned int MAX_CONCURRENT_WRITES = (1024 * 1024);
+
+const uint64_t DEFAULT_POOL_SIZE = 1u<<30;
+const uint64_t MIN_POOL_SIZE = DEFAULT_POOL_SIZE;
+const uint64_t POOL_SIZE_ALIGN = 1 << 20;
+constexpr double USABLE_SIZE = (7.0 / 10);
+const uint64_t BLOCK_ALLOC_OVERHEAD_BYTES = 16;
+const uint8_t RWL_LAYOUT_VERSION = 1;
+const uint8_t SSD_LAYOUT_VERSION = 1;
+const uint64_t MAX_LOG_ENTRIES = (1024 * 1024);
+const double AGGRESSIVE_RETIRE_HIGH_WATER = 0.75;
+const double RETIRE_HIGH_WATER = 0.50;
+const double RETIRE_LOW_WATER = 0.40;
+const int RETIRE_BATCH_TIME_LIMIT_MS = 250;
+const uint64_t CONTROL_BLOCK_MAX_LOG_ENTRIES = 32;
+const uint64_t SPAN_MAX_DATA_LEN = (16 * 1024 * 1024);
+
+/* offset of ring on SSD */
+const uint64_t DATA_RING_BUFFER_OFFSET = 8192;
+
+/* Defer a set of Contexts until destruct/exit. Used for deferring
+ * work on a given thread until a required lock is dropped. */
+class DeferredContexts {
+private:
+ std::vector<Context*> contexts;
+public:
+ ~DeferredContexts();
+ void add(Context* ctx);
+};
+
+/* Pmem structures */
+#ifdef WITH_RBD_RWL
+POBJ_LAYOUT_BEGIN(rbd_pwl);
+POBJ_LAYOUT_ROOT(rbd_pwl, struct WriteLogPoolRoot);
+POBJ_LAYOUT_TOID(rbd_pwl, uint8_t);
+POBJ_LAYOUT_TOID(rbd_pwl, struct WriteLogCacheEntry);
+POBJ_LAYOUT_END(rbd_pwl);
+#endif
+
+struct WriteLogCacheEntry {
+ uint64_t sync_gen_number = 0;
+ uint64_t write_sequence_number = 0;
+ uint64_t image_offset_bytes;
+ uint64_t write_bytes;
+ #ifdef WITH_RBD_RWL
+ TOID(uint8_t) write_data;
+ #endif
+ #ifdef WITH_RBD_SSD_CACHE
+ uint64_t write_data_pos = 0; /* SSD data offset */
+ #endif
+ uint8_t flags = 0;
+ uint32_t ws_datalen = 0; /* Length of data buffer (writesame only) */
+ uint32_t entry_index = 0; /* For debug consistency check. Can be removed if
+ * we need the space */
+ WriteLogCacheEntry(uint64_t image_offset_bytes=0, uint64_t write_bytes=0)
+ : image_offset_bytes(image_offset_bytes), write_bytes(write_bytes) {}
+ BlockExtent block_extent();
+ uint64_t get_offset_bytes();
+ uint64_t get_write_bytes();
+ bool is_entry_valid() const {
+ return flags & WRITE_LOG_CACHE_ENTRY_VALID;
+ }
+ bool is_sync_point() const {
+ return flags & WRITE_LOG_CACHE_ENTRY_SYNC_POINT;
+ }
+ bool is_sequenced() const {
+ return flags & WRITE_LOG_CACHE_ENTRY_SEQUENCED;
+ }
+ bool has_data() const {
+ return flags & WRITE_LOG_CACHE_ENTRY_HAS_DATA;
+ }
+ bool is_discard() const {
+ return flags & WRITE_LOG_CACHE_ENTRY_DISCARD;
+ }
+ bool is_writesame() const {
+ return flags & WRITE_LOG_CACHE_ENTRY_WRITESAME;
+ }
+ bool is_write() const {
+ /* Log entry is a basic write */
+ return !is_sync_point() && !is_discard() && !is_writesame();
+ }
+ bool is_writer() const {
+ /* Log entry is any type that writes data */
+ return is_write() || is_discard() || is_writesame();
+ }
+ void set_entry_valid(bool flag) {
+ if (flag) {
+ flags |= WRITE_LOG_CACHE_ENTRY_VALID;
+ } else {
+ flags &= ~WRITE_LOG_CACHE_ENTRY_VALID;
+ }
+ }
+ void set_sync_point(bool flag) {
+ if (flag) {
+ flags |= WRITE_LOG_CACHE_ENTRY_SYNC_POINT;
+ } else {
+ flags &= ~WRITE_LOG_CACHE_ENTRY_SYNC_POINT;
+ }
+ }
+ void set_sequenced(bool flag) {
+ if (flag) {
+ flags |= WRITE_LOG_CACHE_ENTRY_SEQUENCED;
+ } else {
+ flags &= ~WRITE_LOG_CACHE_ENTRY_SEQUENCED;
+ }
+ }
+ void set_has_data(bool flag) {
+ if (flag) {
+ flags |= WRITE_LOG_CACHE_ENTRY_HAS_DATA;
+ } else {
+ flags &= ~WRITE_LOG_CACHE_ENTRY_HAS_DATA;
+ }
+ }
+ void set_discard(bool flag) {
+ if (flag) {
+ flags |= WRITE_LOG_CACHE_ENTRY_DISCARD;
+ } else {
+ flags &= ~WRITE_LOG_CACHE_ENTRY_DISCARD;
+ }
+ }
+ void set_writesame(bool flag) {
+ if (flag) {
+ flags |= WRITE_LOG_CACHE_ENTRY_WRITESAME;
+ } else {
+ flags &= ~WRITE_LOG_CACHE_ENTRY_WRITESAME;
+ }
+ }
+ friend std::ostream& operator<<(std::ostream& os,
+ const WriteLogCacheEntry &entry);
+ #ifdef WITH_RBD_SSD_CACHE
+ DENC(WriteLogCacheEntry, v, p) {
+ DENC_START(1, 1, p);
+ denc(v.sync_gen_number, p);
+ denc(v.write_sequence_number, p);
+ denc(v.image_offset_bytes, p);
+ denc(v.write_bytes, p);
+ denc(v.write_data_pos, p);
+ denc(v.flags, p);
+ denc(v.ws_datalen, p);
+ denc(v.entry_index, p);
+ DENC_FINISH(p);
+ }
+ #endif
+ void dump(ceph::Formatter *f) const;
+ static void generate_test_instances(list<WriteLogCacheEntry*>& ls);
+};
+
+struct WriteLogPoolRoot {
+ #ifdef WITH_RBD_RWL
+ union {
+ struct {
+ uint8_t layout_version; /* Version of this structure (RWL_LAYOUT_VERSION) */
+ };
+ uint64_t _u64;
+ } header;
+ TOID(struct WriteLogCacheEntry) log_entries; /* contiguous array of log entries */
+ #endif
+ #ifdef WITH_RBD_SSD_CACHE
+ uint64_t layout_version = 0;
+ uint64_t cur_sync_gen = 0;
+ #endif
+ uint64_t pool_size;
+ uint64_t flushed_sync_gen; /* All writing entries with this or a lower
+ * sync gen number are flushed. */
+ uint32_t block_size; /* block size */
+ uint32_t num_log_entries;
+ uint64_t first_free_entry; /* Entry following the newest valid entry */
+ uint64_t first_valid_entry; /* Index of the oldest valid entry in the log */
+
+ #ifdef WITH_RBD_SSD_CACHE
+ DENC(WriteLogPoolRoot, v, p) {
+ DENC_START(1, 1, p);
+ denc(v.layout_version, p);
+ denc(v.cur_sync_gen, p);
+ denc(v.pool_size, p);
+ denc(v.flushed_sync_gen, p);
+ denc(v.block_size, p);
+ denc(v.num_log_entries, p);
+ denc(v.first_free_entry, p);
+ denc(v.first_valid_entry, p);
+ DENC_FINISH(p);
+ }
+ #endif
+
+ void dump(ceph::Formatter *f) const;
+ static void generate_test_instances(list<WriteLogPoolRoot*>& ls);
+};
+
+struct WriteBufferAllocation {
+ unsigned int allocation_size = 0;
+ #ifdef WITH_RBD_RWL
+ pobj_action buffer_alloc_action;
+ TOID(uint8_t) buffer_oid = OID_NULL;
+ #endif
+ bool allocated = false;
+ utime_t allocation_lat;
+};
+
+static inline io::Extent image_extent(const BlockExtent& block_extent) {
+ return io::Extent(block_extent.block_start,
+ block_extent.block_end - block_extent.block_start);
+}
+
+template <typename ExtentsType>
+class ExtentsSummary {
+public:
+ uint64_t total_bytes;
+ uint64_t first_image_byte;
+ uint64_t last_image_byte;
+ explicit ExtentsSummary(const ExtentsType &extents);
+ friend std::ostream &operator<<(std::ostream &os,
+ const ExtentsSummary &s) {
+ os << "total_bytes=" << s.total_bytes << ", "
+ << "first_image_byte=" << s.first_image_byte << ", "
+ << "last_image_byte=" << s.last_image_byte << "";
+ return os;
+ }
+ BlockExtent block_extent() {
+ return BlockExtent(first_image_byte, last_image_byte);
+ }
+ io::Extent image_extent() {
+ return librbd::cache::pwl::image_extent(block_extent());
+ }
+};
+
+io::Extent whole_volume_extent();
+
+BlockExtent block_extent(const io::Extent& image_extent);
+
+Context * override_ctx(int r, Context *ctx);
+
+class ImageExtentBuf : public io::Extent {
+public:
+ bufferlist m_bl;
+ bool need_to_truncate;
+ int truncate_offset;
+ bool writesame;
+ ImageExtentBuf() {}
+ ImageExtentBuf(io::Extent extent,
+ bool need_to_truncate = false, uint64_t truncate_offset = 0,
+ bool writesame = false)
+ : io::Extent(extent), need_to_truncate(need_to_truncate),
+ truncate_offset(truncate_offset), writesame(writesame) {}
+ ImageExtentBuf(io::Extent extent, bufferlist bl,
+ bool need_to_truncate = false, uint64_t truncate_offset = 0,
+ bool writesame = false)
+ : io::Extent(extent), m_bl(bl), need_to_truncate(need_to_truncate),
+ truncate_offset(truncate_offset), writesame(writesame) {}
+};
+
+std::string unique_lock_name(const std::string &name, void *address);
+
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+#ifdef WITH_RBD_SSD_CACHE
+WRITE_CLASS_DENC(librbd::cache::pwl::WriteLogCacheEntry)
+WRITE_CLASS_DENC(librbd::cache::pwl::WriteLogPoolRoot)
+#endif
+
+#endif // CEPH_LIBRBD_CACHE_PWL_TYPES_H
diff --git a/src/librbd/cache/pwl/rwl/Builder.h b/src/librbd/cache/pwl/rwl/Builder.h
new file mode 100644
index 000000000..c13c7b5ae
--- /dev/null
+++ b/src/librbd/cache/pwl/rwl/Builder.h
@@ -0,0 +1,107 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_CACHE_PWL_RWL_BUILDER_H
+#define CEPH_LIBRBD_CACHE_PWL_RWL_BUILDER_H
+
+#include <iostream>
+#include "LogEntry.h"
+#include "ReadRequest.h"
+#include "Request.h"
+#include "LogOperation.h"
+
+#include "librbd/cache/ImageWriteback.h"
+#include "librbd/cache/pwl/Builder.h"
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+namespace rwl {
+
+template <typename T>
+class Builder : public pwl::Builder<T> {
+public:
+ std::shared_ptr<pwl::WriteLogEntry> create_write_log_entry(
+ uint64_t image_offset_bytes, uint64_t write_bytes) override {
+ return std::make_shared<WriteLogEntry>(image_offset_bytes, write_bytes);
+ }
+ std::shared_ptr<pwl::WriteLogEntry> create_write_log_entry(
+ std::shared_ptr<SyncPointLogEntry> sync_point_entry,
+ uint64_t image_offset_bytes, uint64_t write_bytes) override {
+ return std::make_shared<WriteLogEntry>(
+ sync_point_entry, image_offset_bytes, write_bytes);
+ }
+ std::shared_ptr<pwl::WriteLogEntry> create_writesame_log_entry(
+ uint64_t image_offset_bytes, uint64_t write_bytes,
+ uint32_t data_length) override {
+ return std::make_shared<WriteSameLogEntry>(
+ image_offset_bytes, write_bytes, data_length);
+ }
+ std::shared_ptr<pwl::WriteLogEntry> create_writesame_log_entry(
+ std::shared_ptr<SyncPointLogEntry> sync_point_entry,
+ uint64_t image_offset_bytes, uint64_t write_bytes,
+ uint32_t data_length) override {
+ return std::make_shared<WriteSameLogEntry>(
+ sync_point_entry, image_offset_bytes, write_bytes, data_length);
+ }
+ pwl::C_WriteRequest<T> *create_write_request(
+ T &pwl, utime_t arrived, io::Extents &&image_extents,
+ bufferlist&& bl, const int fadvise_flags, ceph::mutex &lock,
+ PerfCounters *perfcounter, Context *user_req) override {
+ return new C_WriteRequest<T>(
+ pwl, arrived, std::move(image_extents), std::move(bl),
+ fadvise_flags, lock, perfcounter, user_req);
+ }
+ pwl::C_WriteSameRequest<T> *create_writesame_request(
+ T &pwl, utime_t arrived, io::Extents &&image_extents,
+ bufferlist&& bl, const int fadvise_flags, ceph::mutex &lock,
+ PerfCounters *perfcounter, Context *user_req) override {
+ return new C_WriteSameRequest<T>(
+ pwl, arrived, std::move(image_extents), std::move(bl),
+ fadvise_flags, lock, perfcounter, user_req);
+ }
+ pwl::C_WriteRequest<T> *create_comp_and_write_request(
+ T &pwl, utime_t arrived, io::Extents &&image_extents,
+ bufferlist&& cmp_bl, bufferlist&& bl, uint64_t *mismatch_offset,
+ const int fadvise_flags, ceph::mutex &lock,
+ PerfCounters *perfcounter, Context *user_req) override {
+ return new rwl::C_CompAndWriteRequest<T>(
+ pwl, arrived, std::move(image_extents), std::move(cmp_bl),
+ std::move(bl), mismatch_offset, fadvise_flags,
+ lock, perfcounter, user_req);
+ }
+ std::shared_ptr<pwl::WriteLogOperation> create_write_log_operation(
+ WriteLogOperationSet &set, uint64_t image_offset_bytes,
+ uint64_t write_bytes, CephContext *cct,
+ std::shared_ptr<pwl::WriteLogEntry> write_log_entry) {
+ return std::make_shared<WriteLogOperation>(
+ set, image_offset_bytes, write_bytes, cct, write_log_entry);
+ }
+ std::shared_ptr<pwl::WriteLogOperation> create_write_log_operation(
+ WriteLogOperationSet &set, uint64_t image_offset_bytes,
+ uint64_t write_bytes, uint32_t data_len, CephContext *cct,
+ std::shared_ptr<pwl::WriteLogEntry> writesame_log_entry) {
+ return std::make_shared<WriteLogOperation>(
+ set, image_offset_bytes, write_bytes, data_len, cct,
+ writesame_log_entry);
+ }
+ std::shared_ptr<pwl::DiscardLogOperation> create_discard_log_operation(
+ std::shared_ptr<SyncPoint> sync_point, uint64_t image_offset_bytes,
+ uint64_t write_bytes, uint32_t discard_granularity_bytes,
+ utime_t dispatch_time, PerfCounters *perfcounter, CephContext *cct) {
+ return std::make_shared<DiscardLogOperation>(
+ sync_point, image_offset_bytes, write_bytes, discard_granularity_bytes,
+ dispatch_time, perfcounter, cct);
+ }
+ C_ReadRequest *create_read_request(CephContext *cct, utime_t arrived,
+ PerfCounters *perfcounter, ceph::bufferlist *bl, Context *on_finish) {
+ return new C_ReadRequest(cct, arrived, perfcounter, bl, on_finish);
+ }
+};
+
+} // namespace rwl
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+#endif // CEPH_LIBRBD_CACHE_PWL_RWL_BUILDER_H
diff --git a/src/librbd/cache/pwl/rwl/LogEntry.cc b/src/librbd/cache/pwl/rwl/LogEntry.cc
new file mode 100644
index 000000000..056701fb5
--- /dev/null
+++ b/src/librbd/cache/pwl/rwl/LogEntry.cc
@@ -0,0 +1,105 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "librbd/cache/ImageWriteback.h"
+#include "LogEntry.h"
+
+#define dout_subsys ceph_subsys_rbd_pwl
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::cache::pwl::rwl::WriteLogEntry: " \
+ << this << " " << __func__ << ": "
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+namespace rwl {
+
+void WriteLogEntry::writeback(
+ librbd::cache::ImageWritebackInterface &image_writeback, Context *ctx) {
+ /* Pass a copy of the pmem buffer to ImageWriteback (which may hang on to the
+ * bl even after flush()). */
+ bufferlist entry_bl;
+ buffer::list entry_bl_copy;
+ copy_cache_bl(&entry_bl_copy);
+ entry_bl_copy.begin(0).copy(write_bytes(), entry_bl);
+ image_writeback.aio_write({{ram_entry.image_offset_bytes,
+ ram_entry.write_bytes}},
+ std::move(entry_bl), 0, ctx);
+}
+
+void WriteLogEntry::init_cache_bp() {
+ ceph_assert(!this->cache_bp.have_raw());
+ cache_bp = buffer::ptr(buffer::create_static(this->write_bytes(),
+ (char*)this->cache_buffer));
+}
+
+void WriteLogEntry::init_bl(buffer::ptr &bp, buffer::list &bl) {
+ if(!is_writesame) {
+ bl.append(bp);
+ return;
+ }
+ for (uint64_t i = 0; i < ram_entry.write_bytes / ram_entry.ws_datalen; i++) {
+ bl.append(bp);
+ }
+ int trailing_partial = ram_entry.write_bytes % ram_entry.ws_datalen;
+ if (trailing_partial) {
+ bl.append(bp, 0, trailing_partial);
+ }
+}
+
+void WriteLogEntry::init_cache_buffer(
+ std::vector<WriteBufferAllocation>::iterator allocation) {
+ this->ram_entry.write_data = allocation->buffer_oid;
+ ceph_assert(!TOID_IS_NULL(this->ram_entry.write_data));
+ cache_buffer = D_RW(this->ram_entry.write_data);
+}
+
+buffer::list& WriteLogEntry::get_cache_bl() {
+ if (0 == bl_refs) {
+ std::lock_guard locker(m_entry_bl_lock);
+ if (0 == bl_refs) {
+ //init pmem bufferlist
+ cache_bl.clear();
+ init_cache_bp();
+ ceph_assert(cache_bp.have_raw());
+ int before_bl = cache_bp.raw_nref();
+ this->init_bl(cache_bp, cache_bl);
+ int after_bl = cache_bp.raw_nref();
+ bl_refs = after_bl - before_bl;
+ }
+ ceph_assert(0 != bl_refs);
+ }
+ return cache_bl;
+}
+
+void WriteLogEntry::copy_cache_bl(bufferlist *out_bl) {
+ this->get_cache_bl();
+ // cache_bp is now initialized
+ buffer::ptr cloned_bp(cache_bp.clone());
+ out_bl->clear();
+ this->init_bl(cloned_bp, *out_bl);
+}
+
+unsigned int WriteLogEntry::reader_count() const {
+ if (cache_bp.have_raw()) {
+ return (cache_bp.raw_nref() - bl_refs - 1);
+ } else {
+ return 0;
+ }
+}
+
+void WriteSameLogEntry::writeback(
+ librbd::cache::ImageWritebackInterface &image_writeback, Context *ctx) {
+ bufferlist entry_bl;
+ buffer::list entry_bl_copy;
+ copy_cache_bl(&entry_bl_copy);
+ entry_bl_copy.begin(0).copy(write_bytes(), entry_bl);
+ image_writeback.aio_writesame(ram_entry.image_offset_bytes,
+ ram_entry.write_bytes,
+ std::move(entry_bl), 0, ctx);
+}
+
+} // namespace rwl
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
diff --git a/src/librbd/cache/pwl/rwl/LogEntry.h b/src/librbd/cache/pwl/rwl/LogEntry.h
new file mode 100644
index 000000000..a4675c5fb
--- /dev/null
+++ b/src/librbd/cache/pwl/rwl/LogEntry.h
@@ -0,0 +1,68 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_CACHE_PWL_RWL_LOG_ENTRY_H
+#define CEPH_LIBRBD_CACHE_PWL_RWL_LOG_ENTRY_H
+
+#include "librbd/cache/pwl/LogEntry.h"
+
+namespace librbd {
+namespace cache {
+class ImageWritebackInterface;
+namespace pwl {
+namespace rwl {
+
+class WriteLogEntry : public pwl::WriteLogEntry {
+public:
+ WriteLogEntry(std::shared_ptr<SyncPointLogEntry> sync_point_entry,
+ uint64_t image_offset_bytes, uint64_t write_bytes)
+ : pwl::WriteLogEntry(sync_point_entry, image_offset_bytes, write_bytes) {}
+ WriteLogEntry(uint64_t image_offset_bytes, uint64_t write_bytes)
+ : pwl::WriteLogEntry(image_offset_bytes, write_bytes) {}
+ WriteLogEntry(std::shared_ptr<SyncPointLogEntry> sync_point_entry,
+ uint64_t image_offset_bytes, uint64_t write_bytes,
+ uint32_t data_length)
+ : pwl::WriteLogEntry(sync_point_entry, image_offset_bytes, write_bytes,
+ data_length) {}
+ WriteLogEntry(uint64_t image_offset_bytes, uint64_t write_bytes,
+ uint32_t data_length)
+ : pwl::WriteLogEntry(image_offset_bytes, write_bytes, data_length) {}
+ ~WriteLogEntry() {}
+ WriteLogEntry(const WriteLogEntry&) = delete;
+ WriteLogEntry &operator=(const WriteLogEntry&) = delete;
+
+ void writeback(librbd::cache::ImageWritebackInterface &image_writeback,
+ Context *ctx) override;
+ void init_cache_bp() override;
+ void init_bl(buffer::ptr &bp, buffer::list &bl) override;
+ void init_cache_buffer(
+ std::vector<WriteBufferAllocation>::iterator allocation) override;
+ buffer::list &get_cache_bl() override;
+ void copy_cache_bl(bufferlist *out_bl) override;
+ unsigned int reader_count() const override;
+};
+
+class WriteSameLogEntry : public WriteLogEntry {
+public:
+ WriteSameLogEntry(std::shared_ptr<SyncPointLogEntry> sync_point_entry,
+ uint64_t image_offset_bytes, uint64_t write_bytes,
+ uint32_t data_length)
+ : WriteLogEntry(sync_point_entry, image_offset_bytes, write_bytes,
+ data_length) {}
+ WriteSameLogEntry(uint64_t image_offset_bytes, uint64_t write_bytes,
+ uint32_t data_length)
+ : WriteLogEntry(image_offset_bytes, write_bytes, data_length) {}
+ ~WriteSameLogEntry() {}
+ WriteSameLogEntry(const WriteSameLogEntry&) = delete;
+ WriteSameLogEntry &operator=(const WriteSameLogEntry&) = delete;
+
+ void writeback(librbd::cache::ImageWritebackInterface &image_writeback,
+ Context *ctx) override;
+};
+
+} // namespace rwl
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+#endif // CEPH_LIBRBD_CACHE_PWL_RWL_LOG_ENTRY_H
diff --git a/src/librbd/cache/pwl/rwl/LogOperation.cc b/src/librbd/cache/pwl/rwl/LogOperation.cc
new file mode 100644
index 000000000..53fb917b2
--- /dev/null
+++ b/src/librbd/cache/pwl/rwl/LogOperation.cc
@@ -0,0 +1,39 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "LogOperation.h"
+
+#define dout_subsys ceph_subsys_rbd_pwl
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::cache::pwl::rwl::LogOperation: " \
+ << this << " " << __func__ << ": "
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+namespace rwl {
+
+void WriteLogOperation::copy_bl_to_cache_buffer(
+ std::vector<WriteBufferAllocation>::iterator allocation) {
+ /* operation is a shared_ptr, so write_op is only good as long as operation is
+ * in scope */
+ bufferlist::iterator i(&bl);
+ m_perfcounter->inc(l_librbd_pwl_log_op_bytes, log_entry->write_bytes());
+ ldout(m_cct, 20) << bl << dendl;
+ log_entry->init_cache_buffer(allocation);
+ i.copy((unsigned)log_entry->write_bytes(), (char*)log_entry->cache_buffer);
+}
+
+void DiscardLogOperation::init_op(
+ uint64_t current_sync_gen, bool persist_on_flush,
+ uint64_t last_op_sequence_num, Context *write_persist,
+ Context *write_append) {
+ log_entry->init(current_sync_gen, persist_on_flush, last_op_sequence_num);
+ this->on_write_append = write_append;
+ this->on_write_persist = write_persist;
+}
+
+} // namespace rwl
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
diff --git a/src/librbd/cache/pwl/rwl/LogOperation.h b/src/librbd/cache/pwl/rwl/LogOperation.h
new file mode 100644
index 000000000..874ac77fb
--- /dev/null
+++ b/src/librbd/cache/pwl/rwl/LogOperation.h
@@ -0,0 +1,55 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_CACHE_PWL_RWL_LOG_OPERATION_H
+#define CEPH_LIBRBD_CACHE_PWL_RWL_LOG_OPERATION_H
+
+#include "librbd/cache/pwl/LogOperation.h"
+
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+namespace rwl {
+
+class WriteLogOperation : public pwl::WriteLogOperation {
+public:
+ WriteLogOperation(
+ WriteLogOperationSet &set, uint64_t image_offset_bytes,
+ uint64_t write_bytes, CephContext *cct,
+ std::shared_ptr<pwl::WriteLogEntry> write_log_entry)
+ : pwl::WriteLogOperation(set, image_offset_bytes, write_bytes, cct,
+ write_log_entry) {}
+
+ WriteLogOperation(
+ WriteLogOperationSet &set, uint64_t image_offset_bytes,
+ uint64_t write_bytes, uint32_t data_len, CephContext *cct,
+ std::shared_ptr<pwl::WriteLogEntry> writesame_log_entry)
+ : pwl::WriteLogOperation(set, image_offset_bytes, write_bytes, cct,
+ writesame_log_entry) {}
+
+ void copy_bl_to_cache_buffer(
+ std::vector<WriteBufferAllocation>::iterator allocation) override;
+};
+
+class DiscardLogOperation : public pwl::DiscardLogOperation {
+public:
+ DiscardLogOperation(
+ std::shared_ptr<SyncPoint> sync_point, uint64_t image_offset_bytes,
+ uint64_t write_bytes, uint32_t discard_granularity_bytes,
+ utime_t dispatch_time, PerfCounters *perfcounter, CephContext *cct)
+ : pwl::DiscardLogOperation(sync_point, image_offset_bytes, write_bytes,
+ discard_granularity_bytes, dispatch_time,
+ perfcounter, cct) {}
+ void init_op(
+ uint64_t current_sync_gen, bool persist_on_flush,
+ uint64_t last_op_sequence_num, Context *write_persist,
+ Context *write_append) override;
+};
+
+} // namespace rwl
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+#endif // CEPH_LIBRBD_CACHE_PWL_RWL_LOG_OPERATION_H
diff --git a/src/librbd/cache/pwl/rwl/ReadRequest.cc b/src/librbd/cache/pwl/rwl/ReadRequest.cc
new file mode 100644
index 000000000..f91f8e5a7
--- /dev/null
+++ b/src/librbd/cache/pwl/rwl/ReadRequest.cc
@@ -0,0 +1,70 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "ReadRequest.h"
+
+#define dout_subsys ceph_subsys_rbd_pwl
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::cache::pwl::rwl::ReadRequest: " << this << " " \
+ << __func__ << ": "
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+namespace rwl {
+
+void C_ReadRequest::finish(int r) {
+ ldout(m_cct, 20) << "(" << get_name() << "): r=" << r << dendl;
+ int hits = 0;
+ int misses = 0;
+ int hit_bytes = 0;
+ int miss_bytes = 0;
+ if (r >= 0) {
+ /*
+ * At this point the miss read has completed. We'll iterate through
+ * read_extents and produce *m_out_bl by assembling pieces of miss_bl
+ * and the individual hit extent bufs in the read extents that represent
+ * hits.
+ */
+ uint64_t miss_bl_offset = 0;
+ for (auto extent : read_extents) {
+ if (extent->m_bl.length()) {
+ /* This was a hit */
+ ceph_assert(extent->second == extent->m_bl.length());
+ ++hits;
+ hit_bytes += extent->second;
+ m_out_bl->claim_append(extent->m_bl);
+ } else {
+ /* This was a miss. */
+ ++misses;
+ miss_bytes += extent->second;
+ bufferlist miss_extent_bl;
+ miss_extent_bl.substr_of(miss_bl, miss_bl_offset, extent->second);
+ /* Add this read miss bufferlist to the output bufferlist */
+ m_out_bl->claim_append(miss_extent_bl);
+ /* Consume these bytes in the read miss bufferlist */
+ miss_bl_offset += extent->second;
+ }
+ }
+ }
+ ldout(m_cct, 20) << "(" << get_name() << "): r=" << r << " bl=" << *m_out_bl << dendl;
+ utime_t now = ceph_clock_now();
+ ceph_assert((int)m_out_bl->length() == hit_bytes + miss_bytes);
+ m_on_finish->complete(r);
+ m_perfcounter->inc(l_librbd_pwl_rd_bytes, hit_bytes + miss_bytes);
+ m_perfcounter->inc(l_librbd_pwl_rd_hit_bytes, hit_bytes);
+ m_perfcounter->tinc(l_librbd_pwl_rd_latency, now - m_arrived_time);
+ if (!misses) {
+ m_perfcounter->inc(l_librbd_pwl_rd_hit_req, 1);
+ m_perfcounter->tinc(l_librbd_pwl_rd_hit_latency, now - m_arrived_time);
+ } else {
+ if (hits) {
+ m_perfcounter->inc(l_librbd_pwl_rd_part_hit_req, 1);
+ }
+ }
+}
+
+} // namespace rwl
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
diff --git a/src/librbd/cache/pwl/rwl/ReadRequest.h b/src/librbd/cache/pwl/rwl/ReadRequest.h
new file mode 100644
index 000000000..25168e83b
--- /dev/null
+++ b/src/librbd/cache/pwl/rwl/ReadRequest.h
@@ -0,0 +1,34 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_CACHE_PWL_RWL_READ_REQUEST_H
+#define CEPH_LIBRBD_CACHE_PWL_RWL_READ_REQUEST_H
+
+#include "librbd/cache/pwl/ReadRequest.h"
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+namespace rwl {
+
+typedef std::vector<pwl::ImageExtentBuf> ImageExtentBufs;
+
+class C_ReadRequest : public pwl::C_ReadRequest {
+protected:
+ using pwl::C_ReadRequest::m_cct;
+ using pwl::C_ReadRequest::m_on_finish;
+ using pwl::C_ReadRequest::m_out_bl;
+ using pwl::C_ReadRequest::m_arrived_time;
+ using pwl::C_ReadRequest::m_perfcounter;
+public:
+ C_ReadRequest(CephContext *cct, utime_t arrived, PerfCounters *perfcounter, bufferlist *out_bl, Context *on_finish)
+ : pwl::C_ReadRequest(cct, arrived, perfcounter, out_bl, on_finish) {}
+ void finish(int r) override;
+};
+
+} // namespace rwl
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+#endif // CEPH_LIBRBD_CACHE_PWL_RWL_READ_REQUEST_H
diff --git a/src/librbd/cache/pwl/rwl/Request.cc b/src/librbd/cache/pwl/rwl/Request.cc
new file mode 100644
index 000000000..091581272
--- /dev/null
+++ b/src/librbd/cache/pwl/rwl/Request.cc
@@ -0,0 +1,86 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "Request.h"
+#include "librbd/cache/pwl/AbstractWriteLog.h"
+
+#define dout_subsys ceph_subsys_rbd_pwl
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::cache::pwl::rwl::Request: " << this \
+ << " " << __func__ << ": "
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+namespace rwl {
+
+template <typename T>
+void C_WriteRequest<T>::setup_buffer_resources(
+ uint64_t *bytes_cached, uint64_t *bytes_dirtied, uint64_t *bytes_allocated,
+ uint64_t *number_lanes, uint64_t *number_log_entries,
+ uint64_t *number_unpublished_reserves) {
+
+ ceph_assert(!this->m_resources.allocated);
+
+ auto image_extents_size = this->image_extents.size();
+ this->m_resources.buffers.reserve(image_extents_size);
+
+ *bytes_cached = 0;
+ *bytes_allocated = 0;
+ *number_lanes = image_extents_size;
+ *number_log_entries = image_extents_size;
+ *number_unpublished_reserves = image_extents_size;
+
+ for (auto &extent : this->image_extents) {
+ this->m_resources.buffers.emplace_back();
+ struct WriteBufferAllocation &buffer = this->m_resources.buffers.back();
+ buffer.allocation_size = MIN_WRITE_ALLOC_SIZE;
+ buffer.allocated = false;
+ *bytes_cached += extent.second;
+ if (extent.second > buffer.allocation_size) {
+ buffer.allocation_size = extent.second;
+ }
+ *bytes_allocated += buffer.allocation_size;
+ }
+ *bytes_dirtied = *bytes_cached;
+}
+
+template <typename T>
+std::ostream &operator<<(std::ostream &os,
+ const C_CompAndWriteRequest<T> &req) {
+ os << (C_WriteRequest<T>&)req
+ << "cmp_bl=" << req.cmp_bl << ", "
+ << "read_bl=" << req.read_bl << ", "
+ << "compare_succeeded=" << req.compare_succeeded << ", "
+ << "mismatch_offset=" << req.mismatch_offset;
+ return os;
+}
+
+template <typename T>
+void C_WriteSameRequest<T>::setup_buffer_resources(
+ uint64_t *bytes_cached, uint64_t *bytes_dirtied, uint64_t *bytes_allocated,
+ uint64_t *number_lanes, uint64_t *number_log_entries,
+ uint64_t *number_unpublished_reserves) {
+ ceph_assert(this->image_extents.size() == 1);
+ *number_log_entries = 1;
+ *bytes_dirtied += this->image_extents[0].second;
+ auto pattern_length = this->bl.length();
+ this->m_resources.buffers.emplace_back();
+ struct WriteBufferAllocation &buffer = this->m_resources.buffers.back();
+ buffer.allocation_size = MIN_WRITE_ALLOC_SIZE;
+ buffer.allocated = false;
+ *bytes_cached += pattern_length;
+ if (pattern_length > buffer.allocation_size) {
+ buffer.allocation_size = pattern_length;
+ }
+ *bytes_allocated += buffer.allocation_size;
+}
+
+} // namespace rwl
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+template class librbd::cache::pwl::rwl::C_WriteRequest<librbd::cache::pwl::AbstractWriteLog<librbd::ImageCtx> >;
+template class librbd::cache::pwl::rwl::C_WriteSameRequest<librbd::cache::pwl::AbstractWriteLog<librbd::ImageCtx> >;
+template class librbd::cache::pwl::rwl::C_CompAndWriteRequest<librbd::cache::pwl::AbstractWriteLog<librbd::ImageCtx> >;
diff --git a/src/librbd/cache/pwl/rwl/Request.h b/src/librbd/cache/pwl/rwl/Request.h
new file mode 100644
index 000000000..0a5c610d6
--- /dev/null
+++ b/src/librbd/cache/pwl/rwl/Request.h
@@ -0,0 +1,90 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_CACHE_RWL_REQUEST_H
+#define CEPH_LIBRBD_CACHE_RWL_REQUEST_H
+
+#include "librbd/cache/pwl/Request.h"
+
+namespace librbd {
+class BlockGuardCell;
+
+namespace cache {
+namespace pwl {
+namespace rwl {
+
+template <typename T>
+class C_WriteRequest : public pwl::C_WriteRequest<T> {
+public:
+ C_WriteRequest(
+ T &pwl, const utime_t arrived, io::Extents &&image_extents,
+ bufferlist&& cmp_bl, bufferlist&& bl, uint64_t *mismatch_offset,
+ const int fadvise_flags, ceph::mutex &lock,
+ PerfCounters *perfcounter, Context *user_req)
+ : pwl::C_WriteRequest<T>(
+ pwl, arrived, std::move(image_extents), std::move(cmp_bl),
+ std::move(bl), mismatch_offset, fadvise_flags,
+ lock, perfcounter, user_req) {}
+
+ C_WriteRequest(
+ T &pwl, const utime_t arrived, io::Extents &&image_extents,
+ bufferlist&& bl, const int fadvise_flags, ceph::mutex &lock,
+ PerfCounters *perfcounter, Context *user_req)
+ : pwl::C_WriteRequest<T>(
+ pwl, arrived, std::move(image_extents), std::move(bl),
+ fadvise_flags, lock, perfcounter, user_req) {}
+protected:
+ //Plain writes will allocate one buffer per request extent
+ void setup_buffer_resources(
+ uint64_t *bytes_cached, uint64_t *bytes_dirtied,
+ uint64_t *bytes_allocated, uint64_t *number_lanes,
+ uint64_t *number_log_entries,
+ uint64_t *number_unpublished_reserves) override;
+};
+
+template <typename T>
+class C_CompAndWriteRequest : public C_WriteRequest<T> {
+public:
+ C_CompAndWriteRequest(
+ T &pwl, const utime_t arrived, io::Extents &&image_extents,
+ bufferlist&& cmp_bl, bufferlist&& bl, uint64_t *mismatch_offset,
+ const int fadvise_flags, ceph::mutex &lock,
+ PerfCounters *perfcounter, Context *user_req)
+ : C_WriteRequest<T>(
+ pwl, arrived, std::move(image_extents), std::move(cmp_bl),
+ std::move(bl), mismatch_offset, fadvise_flags,
+ lock, perfcounter, user_req) {}
+
+ const char *get_name() const override {
+ return "C_CompAndWriteRequest";
+ }
+ template <typename U>
+ friend std::ostream &operator<<(std::ostream &os,
+ const C_CompAndWriteRequest<U> &req);
+};
+
+template <typename T>
+class C_WriteSameRequest : public pwl::C_WriteSameRequest<T> {
+public:
+ C_WriteSameRequest(
+ T &pwl, const utime_t arrived, io::Extents &&image_extents,
+ bufferlist&& bl, const int fadvise_flags, ceph::mutex &lock,
+ PerfCounters *perfcounter, Context *user_req)
+ : pwl::C_WriteSameRequest<T>(
+ pwl, arrived, std::move(image_extents), std::move(bl), fadvise_flags,
+ lock, perfcounter, user_req) {}
+
+ void setup_buffer_resources(
+ uint64_t *bytes_cached, uint64_t *bytes_dirtied,
+ uint64_t *bytes_allocated, uint64_t *number_lanes,
+ uint64_t *number_log_entries,
+ uint64_t *number_unpublished_reserves) override;
+
+};
+
+} // namespace rwl
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+#endif // CEPH_LIBRBD_CACHE_RWL_REQUEST_H
diff --git a/src/librbd/cache/pwl/rwl/WriteLog.cc b/src/librbd/cache/pwl/rwl/WriteLog.cc
new file mode 100644
index 000000000..dd623c9ad
--- /dev/null
+++ b/src/librbd/cache/pwl/rwl/WriteLog.cc
@@ -0,0 +1,1014 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "WriteLog.h"
+#include "include/buffer.h"
+#include "include/Context.h"
+#include "include/ceph_assert.h"
+#include "common/deleter.h"
+#include "common/dout.h"
+#include "common/environment.h"
+#include "common/errno.h"
+#include "common/WorkQueue.h"
+#include "common/Timer.h"
+#include "common/perf_counters.h"
+#include "librbd/ImageCtx.h"
+#include "librbd/asio/ContextWQ.h"
+#include "librbd/cache/pwl/ImageCacheState.h"
+#include "librbd/cache/pwl/LogEntry.h"
+#include "librbd/plugin/Api.h"
+#include <map>
+#include <vector>
+
+#undef dout_subsys
+#define dout_subsys ceph_subsys_rbd_pwl
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::cache::pwl::rwl::WriteLog: " << this \
+ << " " << __func__ << ": "
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+using namespace librbd::cache::pwl;
+namespace rwl {
+
+const unsigned long int OPS_APPENDED_TOGETHER = MAX_ALLOC_PER_TRANSACTION;
+
+template <typename I>
+Builder<AbstractWriteLog<I>>* WriteLog<I>::create_builder() {
+ m_builderobj = new Builder<This>();
+ return m_builderobj;
+}
+
+template <typename I>
+WriteLog<I>::WriteLog(
+ I &image_ctx, librbd::cache::pwl::ImageCacheState<I>* cache_state,
+ ImageWritebackInterface& image_writeback,
+ plugin::Api<I>& plugin_api)
+: AbstractWriteLog<I>(image_ctx, cache_state, create_builder(), image_writeback,
+ plugin_api),
+ m_pwl_pool_layout_name(POBJ_LAYOUT_NAME(rbd_pwl))
+{
+}
+
+template <typename I>
+WriteLog<I>::~WriteLog() {
+ m_log_pool = nullptr;
+ delete m_builderobj;
+}
+
+template <typename I>
+void WriteLog<I>::collect_read_extents(
+ uint64_t read_buffer_offset, LogMapEntry<GenericWriteLogEntry> map_entry,
+ std::vector<std::shared_ptr<GenericWriteLogEntry>> &log_entries_to_read,
+ std::vector<bufferlist*> &bls_to_read, uint64_t entry_hit_length,
+ Extent hit_extent, pwl::C_ReadRequest *read_ctx) {
+ /* Make a bl for this hit extent. This will add references to the
+ * write_entry->pmem_bp */
+ buffer::list hit_bl;
+
+ /* Create buffer object referring to pmem pool for this read hit */
+ auto write_entry = map_entry.log_entry;
+
+ buffer::list entry_bl_copy;
+ write_entry->copy_cache_bl(&entry_bl_copy);
+ entry_bl_copy.begin(read_buffer_offset).copy(entry_hit_length, hit_bl);
+ ceph_assert(hit_bl.length() == entry_hit_length);
+
+ /* Add hit extent to read extents */
+ auto hit_extent_buf = std::make_shared<ImageExtentBuf>(hit_extent, hit_bl);
+ read_ctx->read_extents.push_back(hit_extent_buf);
+}
+
+template <typename I>
+void WriteLog<I>::complete_read(
+ std::vector<std::shared_ptr<GenericWriteLogEntry>> &log_entries_to_read,
+ std::vector<bufferlist*> &bls_to_read, Context *ctx) {
+ ctx->complete(0);
+}
+
+/*
+ * Allocate the (already reserved) write log entries for a set of operations.
+ *
+ * Locking:
+ * Acquires lock
+ */
+template <typename I>
+void WriteLog<I>::alloc_op_log_entries(GenericLogOperations &ops)
+{
+ TOID(struct WriteLogPoolRoot) pool_root;
+ pool_root = POBJ_ROOT(m_log_pool, struct WriteLogPoolRoot);
+ struct WriteLogCacheEntry *pmem_log_entries = D_RW(D_RW(pool_root)->log_entries);
+
+ ceph_assert(ceph_mutex_is_locked_by_me(this->m_log_append_lock));
+
+ /* Allocate the (already reserved) log entries */
+ std::unique_lock locker(m_lock);
+
+ for (auto &operation : ops) {
+ uint32_t entry_index = this->m_first_free_entry;
+ this->m_first_free_entry = (this->m_first_free_entry + 1) % this->m_total_log_entries;
+ auto &log_entry = operation->get_log_entry();
+ log_entry->log_entry_index = entry_index;
+ log_entry->ram_entry.entry_index = entry_index;
+ log_entry->cache_entry = &pmem_log_entries[entry_index];
+ log_entry->ram_entry.set_entry_valid(true);
+ m_log_entries.push_back(log_entry);
+ ldout(m_image_ctx.cct, 20) << "operation=[" << *operation << "]" << dendl;
+ }
+ if (m_cache_state->empty && !m_log_entries.empty()) {
+ m_cache_state->empty = false;
+ this->update_image_cache_state();
+ this->write_image_cache_state(locker);
+ }
+}
+
+/*
+ * Write and persist the (already allocated) write log entries and
+ * data buffer allocations for a set of ops. The data buffer for each
+ * of these must already have been persisted to its reserved area.
+ */
+template <typename I>
+int WriteLog<I>::append_op_log_entries(GenericLogOperations &ops)
+{
+ CephContext *cct = m_image_ctx.cct;
+ GenericLogOperationsVector entries_to_flush;
+ TOID(struct WriteLogPoolRoot) pool_root;
+ pool_root = POBJ_ROOT(m_log_pool, struct WriteLogPoolRoot);
+ int ret = 0;
+
+ ceph_assert(ceph_mutex_is_locked_by_me(this->m_log_append_lock));
+
+ if (ops.empty()) {
+ return 0;
+ }
+ entries_to_flush.reserve(OPS_APPENDED_TOGETHER);
+
+ /* Write log entries to ring and persist */
+ utime_t now = ceph_clock_now();
+ for (auto &operation : ops) {
+ if (!entries_to_flush.empty()) {
+ /* Flush these and reset the list if the current entry wraps to the
+ * tail of the ring */
+ if (entries_to_flush.back()->get_log_entry()->log_entry_index >
+ operation->get_log_entry()->log_entry_index) {
+ ldout(m_image_ctx.cct, 20) << "entries to flush wrap around the end of the ring at "
+ << "operation=[" << *operation << "]" << dendl;
+ flush_op_log_entries(entries_to_flush);
+ entries_to_flush.clear();
+ now = ceph_clock_now();
+ }
+ }
+ ldout(m_image_ctx.cct, 20) << "Copying entry for operation at index="
+ << operation->get_log_entry()->log_entry_index << " "
+ << "from " << &operation->get_log_entry()->ram_entry << " "
+ << "to " << operation->get_log_entry()->cache_entry << " "
+ << "operation=[" << *operation << "]" << dendl;
+ ldout(m_image_ctx.cct, 05) << "APPENDING: index="
+ << operation->get_log_entry()->log_entry_index << " "
+ << "operation=[" << *operation << "]" << dendl;
+ operation->log_append_start_time = now;
+ *operation->get_log_entry()->cache_entry = operation->get_log_entry()->ram_entry;
+ ldout(m_image_ctx.cct, 20) << "APPENDING: index="
+ << operation->get_log_entry()->log_entry_index << " "
+ << "pmem_entry=[" << *operation->get_log_entry()->cache_entry
+ << "]" << dendl;
+ entries_to_flush.push_back(operation);
+ }
+ flush_op_log_entries(entries_to_flush);
+
+ /* Drain once for all */
+ pmemobj_drain(m_log_pool);
+
+ /*
+ * Atomically advance the log head pointer and publish the
+ * allocations for all the data buffers they refer to.
+ */
+ utime_t tx_start = ceph_clock_now();
+ TX_BEGIN(m_log_pool) {
+ D_RW(pool_root)->first_free_entry = this->m_first_free_entry;
+ for (auto &operation : ops) {
+ if (operation->reserved_allocated()) {
+ auto write_op = (std::shared_ptr<WriteLogOperation>&) operation;
+ pmemobj_tx_publish(&write_op->buffer_alloc->buffer_alloc_action, 1);
+ } else {
+ ldout(m_image_ctx.cct, 20) << "skipping non-write op: " << *operation << dendl;
+ }
+ }
+ } TX_ONCOMMIT {
+ } TX_ONABORT {
+ lderr(cct) << "failed to commit " << ops.size()
+ << " log entries (" << this->m_log_pool_name << ")" << dendl;
+ ceph_assert(false);
+ ret = -EIO;
+ } TX_FINALLY {
+ } TX_END;
+
+ utime_t tx_end = ceph_clock_now();
+ m_perfcounter->tinc(l_librbd_pwl_append_tx_t, tx_end - tx_start);
+ m_perfcounter->hinc(
+ l_librbd_pwl_append_tx_t_hist, utime_t(tx_end - tx_start).to_nsec(), ops.size());
+ for (auto &operation : ops) {
+ operation->log_append_comp_time = tx_end;
+ }
+
+ return ret;
+}
+
+/*
+ * Flush the persistent write log entries set of ops. The entries must
+ * be contiguous in persistent memory.
+ */
+template <typename I>
+void WriteLog<I>::flush_op_log_entries(GenericLogOperationsVector &ops)
+{
+ if (ops.empty()) {
+ return;
+ }
+
+ if (ops.size() > 1) {
+ ceph_assert(ops.front()->get_log_entry()->cache_entry < ops.back()->get_log_entry()->cache_entry);
+ }
+
+ ldout(m_image_ctx.cct, 20) << "entry count=" << ops.size() << " "
+ << "start address="
+ << ops.front()->get_log_entry()->cache_entry << " "
+ << "bytes="
+ << ops.size() * sizeof(*(ops.front()->get_log_entry()->cache_entry))
+ << dendl;
+ pmemobj_flush(m_log_pool,
+ ops.front()->get_log_entry()->cache_entry,
+ ops.size() * sizeof(*(ops.front()->get_log_entry()->cache_entry)));
+}
+
+template <typename I>
+void WriteLog<I>::remove_pool_file() {
+ if (m_log_pool) {
+ ldout(m_image_ctx.cct, 6) << "closing pmem pool" << dendl;
+ pmemobj_close(m_log_pool);
+ }
+ if (m_cache_state->clean) {
+ ldout(m_image_ctx.cct, 5) << "Removing empty pool file: " << this->m_log_pool_name << dendl;
+ if (remove(this->m_log_pool_name.c_str()) != 0) {
+ lderr(m_image_ctx.cct) << "failed to remove empty pool \"" << this->m_log_pool_name << "\": "
+ << pmemobj_errormsg() << dendl;
+ } else {
+ m_cache_state->present = false;
+ }
+ } else {
+ ldout(m_image_ctx.cct, 5) << "Not removing pool file: " << this->m_log_pool_name << dendl;
+ }
+}
+
+template <typename I>
+bool WriteLog<I>::initialize_pool(Context *on_finish, pwl::DeferredContexts &later) {
+ CephContext *cct = m_image_ctx.cct;
+ TOID(struct WriteLogPoolRoot) pool_root;
+ ceph_assert(ceph_mutex_is_locked_by_me(m_lock));
+ if (access(this->m_log_pool_name.c_str(), F_OK) != 0) {
+ if ((m_log_pool =
+ pmemobj_create(this->m_log_pool_name.c_str(),
+ this->m_pwl_pool_layout_name,
+ this->m_log_pool_size,
+ (S_IWUSR | S_IRUSR))) == NULL) {
+ lderr(cct) << "failed to create pool (" << this->m_log_pool_name << ")"
+ << pmemobj_errormsg() << dendl;
+ m_cache_state->present = false;
+ m_cache_state->clean = true;
+ m_cache_state->empty = true;
+ /* TODO: filter/replace errnos that are meaningless to the caller */
+ on_finish->complete(-errno);
+ return false;
+ }
+ m_cache_state->present = true;
+ m_cache_state->clean = true;
+ m_cache_state->empty = true;
+ pool_root = POBJ_ROOT(m_log_pool, struct WriteLogPoolRoot);
+
+ /* new pool, calculate and store metadata */
+ size_t effective_pool_size = (size_t)(this->m_log_pool_size * USABLE_SIZE);
+ size_t small_write_size = MIN_WRITE_ALLOC_SIZE + BLOCK_ALLOC_OVERHEAD_BYTES + sizeof(struct WriteLogCacheEntry);
+ uint64_t num_small_writes = (uint64_t)(effective_pool_size / small_write_size);
+ if (num_small_writes > MAX_LOG_ENTRIES) {
+ num_small_writes = MAX_LOG_ENTRIES;
+ }
+ if (num_small_writes <= 2) {
+ lderr(cct) << "num_small_writes needs to > 2" << dendl;
+ on_finish->complete(-EINVAL);
+ return false;
+ }
+ this->m_bytes_allocated_cap = effective_pool_size;
+ /* Log ring empty */
+ m_first_free_entry = 0;
+ m_first_valid_entry = 0;
+ TX_BEGIN(m_log_pool) {
+ TX_ADD(pool_root);
+ D_RW(pool_root)->header.layout_version = RWL_LAYOUT_VERSION;
+ D_RW(pool_root)->log_entries =
+ TX_ZALLOC(struct WriteLogCacheEntry,
+ sizeof(struct WriteLogCacheEntry) * num_small_writes);
+ D_RW(pool_root)->pool_size = this->m_log_pool_size;
+ D_RW(pool_root)->flushed_sync_gen = this->m_flushed_sync_gen;
+ D_RW(pool_root)->block_size = MIN_WRITE_ALLOC_SIZE;
+ D_RW(pool_root)->num_log_entries = num_small_writes;
+ D_RW(pool_root)->first_free_entry = m_first_free_entry;
+ D_RW(pool_root)->first_valid_entry = m_first_valid_entry;
+ } TX_ONCOMMIT {
+ this->m_total_log_entries = D_RO(pool_root)->num_log_entries;
+ this->m_free_log_entries = D_RO(pool_root)->num_log_entries - 1; // leave one free
+ } TX_ONABORT {
+ this->m_total_log_entries = 0;
+ this->m_free_log_entries = 0;
+ lderr(cct) << "failed to initialize pool (" << this->m_log_pool_name << ")" << dendl;
+ on_finish->complete(-pmemobj_tx_errno());
+ return false;
+ } TX_FINALLY {
+ } TX_END;
+ } else {
+ ceph_assert(m_cache_state->present);
+ /* Open existing pool */
+ if ((m_log_pool =
+ pmemobj_open(this->m_log_pool_name.c_str(),
+ this->m_pwl_pool_layout_name)) == NULL) {
+ lderr(cct) << "failed to open pool (" << this->m_log_pool_name << "): "
+ << pmemobj_errormsg() << dendl;
+ on_finish->complete(-errno);
+ return false;
+ }
+ pool_root = POBJ_ROOT(m_log_pool, struct WriteLogPoolRoot);
+ if (D_RO(pool_root)->header.layout_version != RWL_LAYOUT_VERSION) {
+ // TODO: will handle upgrading version in the future
+ lderr(cct) << "Pool layout version is "
+ << D_RO(pool_root)->header.layout_version
+ << " expected " << RWL_LAYOUT_VERSION << dendl;
+ on_finish->complete(-EINVAL);
+ return false;
+ }
+ if (D_RO(pool_root)->block_size != MIN_WRITE_ALLOC_SIZE) {
+ lderr(cct) << "Pool block size is " << D_RO(pool_root)->block_size
+ << " expected " << MIN_WRITE_ALLOC_SIZE << dendl;
+ on_finish->complete(-EINVAL);
+ return false;
+ }
+ this->m_log_pool_size = D_RO(pool_root)->pool_size;
+ this->m_flushed_sync_gen = D_RO(pool_root)->flushed_sync_gen;
+ this->m_total_log_entries = D_RO(pool_root)->num_log_entries;
+ m_first_free_entry = D_RO(pool_root)->first_free_entry;
+ m_first_valid_entry = D_RO(pool_root)->first_valid_entry;
+ if (m_first_free_entry < m_first_valid_entry) {
+ /* Valid entries wrap around the end of the ring, so first_free is lower
+ * than first_valid. If first_valid was == first_free+1, the entry at
+ * first_free would be empty. The last entry is never used, so in
+ * that case there would be zero free log entries. */
+ this->m_free_log_entries = this->m_total_log_entries - (m_first_valid_entry - m_first_free_entry) -1;
+ } else {
+ /* first_valid is <= first_free. If they are == we have zero valid log
+ * entries, and n-1 free log entries */
+ this->m_free_log_entries = this->m_total_log_entries - (m_first_free_entry - m_first_valid_entry) -1;
+ }
+ size_t effective_pool_size = (size_t)(this->m_log_pool_size * USABLE_SIZE);
+ this->m_bytes_allocated_cap = effective_pool_size;
+ load_existing_entries(later);
+ m_cache_state->clean = this->m_dirty_log_entries.empty();
+ m_cache_state->empty = m_log_entries.empty();
+ }
+ return true;
+}
+
+/*
+ * Loads the log entries from an existing log.
+ *
+ * Creates the in-memory structures to represent the state of the
+ * re-opened log.
+ *
+ * Finds the last appended sync point, and any sync points referred to
+ * in log entries, but missing from the log. These missing sync points
+ * are created and scheduled for append. Some rudimentary consistency
+ * checking is done.
+ *
+ * Rebuilds the m_blocks_to_log_entries map, to make log entries
+ * readable.
+ *
+ * Places all writes on the dirty entries list, which causes them all
+ * to be flushed.
+ *
+ */
+
+template <typename I>
+void WriteLog<I>::load_existing_entries(DeferredContexts &later) {
+ TOID(struct WriteLogPoolRoot) pool_root;
+ pool_root = POBJ_ROOT(m_log_pool, struct WriteLogPoolRoot);
+ struct WriteLogCacheEntry *pmem_log_entries = D_RW(D_RW(pool_root)->log_entries);
+ uint64_t entry_index = m_first_valid_entry;
+ /* The map below allows us to find sync point log entries by sync
+ * gen number, which is necessary so write entries can be linked to
+ * their sync points. */
+ std::map<uint64_t, std::shared_ptr<SyncPointLogEntry>> sync_point_entries;
+ /* The map below tracks sync points referred to in writes but not
+ * appearing in the sync_point_entries map. We'll use this to
+ * determine which sync points are missing and need to be
+ * created. */
+ std::map<uint64_t, bool> missing_sync_points;
+
+ /*
+ * Read the existing log entries. Construct an in-memory log entry
+ * object of the appropriate type for each. Add these to the global
+ * log entries list.
+ *
+ * Write entries will not link to their sync points yet. We'll do
+ * that in the next pass. Here we'll accumulate a map of sync point
+ * gen numbers that are referred to in writes but do not appearing in
+ * the log.
+ */
+ while (entry_index != m_first_free_entry) {
+ WriteLogCacheEntry *pmem_entry = &pmem_log_entries[entry_index];
+ std::shared_ptr<GenericLogEntry> log_entry = nullptr;
+ ceph_assert(pmem_entry->entry_index == entry_index);
+
+ this->update_entries(&log_entry, pmem_entry, missing_sync_points,
+ sync_point_entries, entry_index);
+
+ log_entry->ram_entry = *pmem_entry;
+ log_entry->cache_entry = pmem_entry;
+ log_entry->log_entry_index = entry_index;
+ log_entry->completed = true;
+
+ m_log_entries.push_back(log_entry);
+
+ entry_index = (entry_index + 1) % this->m_total_log_entries;
+ }
+
+ this->update_sync_points(missing_sync_points, sync_point_entries, later);
+}
+
+template <typename I>
+void WriteLog<I>::inc_allocated_cached_bytes(
+ std::shared_ptr<pwl::GenericLogEntry> log_entry) {
+ if (log_entry->is_write_entry()) {
+ this->m_bytes_allocated += std::max(log_entry->write_bytes(), MIN_WRITE_ALLOC_SIZE);
+ this->m_bytes_cached += log_entry->write_bytes();
+ }
+}
+
+template <typename I>
+void WriteLog<I>::write_data_to_buffer(
+ std::shared_ptr<pwl::WriteLogEntry> ws_entry,
+ WriteLogCacheEntry *pmem_entry) {
+ ws_entry->cache_buffer = D_RW(pmem_entry->write_data);
+}
+
+/**
+ * Retire up to MAX_ALLOC_PER_TRANSACTION of the oldest log entries
+ * that are eligible to be retired. Returns true if anything was
+ * retired.
+ */
+template <typename I>
+bool WriteLog<I>::retire_entries(const unsigned long int frees_per_tx) {
+ CephContext *cct = m_image_ctx.cct;
+ GenericLogEntriesVector retiring_entries;
+ uint32_t initial_first_valid_entry;
+ uint32_t first_valid_entry;
+
+ std::lock_guard retire_locker(this->m_log_retire_lock);
+ ldout(cct, 20) << "Look for entries to retire" << dendl;
+ {
+ /* Entry readers can't be added while we hold m_entry_reader_lock */
+ RWLock::WLocker entry_reader_locker(this->m_entry_reader_lock);
+ std::lock_guard locker(m_lock);
+ initial_first_valid_entry = this->m_first_valid_entry;
+ first_valid_entry = this->m_first_valid_entry;
+ auto entry = m_log_entries.front();
+ while (!m_log_entries.empty() &&
+ retiring_entries.size() < frees_per_tx &&
+ this->can_retire_entry(entry)) {
+ if (entry->log_entry_index != first_valid_entry) {
+ lderr(cct) << "Retiring entry index (" << entry->log_entry_index
+ << ") and first valid log entry index (" << first_valid_entry
+ << ") must be ==." << dendl;
+ }
+ ceph_assert(entry->log_entry_index == first_valid_entry);
+ first_valid_entry = (first_valid_entry + 1) % this->m_total_log_entries;
+ m_log_entries.pop_front();
+ retiring_entries.push_back(entry);
+ /* Remove entry from map so there will be no more readers */
+ if ((entry->write_bytes() > 0) || (entry->bytes_dirty() > 0)) {
+ auto gen_write_entry = static_pointer_cast<GenericWriteLogEntry>(entry);
+ if (gen_write_entry) {
+ this->m_blocks_to_log_entries.remove_log_entry(gen_write_entry);
+ }
+ }
+ entry = m_log_entries.front();
+ }
+ }
+
+ if (retiring_entries.size()) {
+ ldout(cct, 20) << "Retiring " << retiring_entries.size() << " entries" << dendl;
+ TOID(struct WriteLogPoolRoot) pool_root;
+ pool_root = POBJ_ROOT(m_log_pool, struct WriteLogPoolRoot);
+
+ utime_t tx_start;
+ utime_t tx_end;
+ /* Advance first valid entry and release buffers */
+ {
+ uint64_t flushed_sync_gen;
+ std::lock_guard append_locker(this->m_log_append_lock);
+ {
+ std::lock_guard locker(m_lock);
+ flushed_sync_gen = this->m_flushed_sync_gen;
+ }
+
+ tx_start = ceph_clock_now();
+ TX_BEGIN(m_log_pool) {
+ if (D_RO(pool_root)->flushed_sync_gen < flushed_sync_gen) {
+ ldout(m_image_ctx.cct, 20) << "flushed_sync_gen in log updated from "
+ << D_RO(pool_root)->flushed_sync_gen << " to "
+ << flushed_sync_gen << dendl;
+ D_RW(pool_root)->flushed_sync_gen = flushed_sync_gen;
+ }
+ D_RW(pool_root)->first_valid_entry = first_valid_entry;
+ for (auto &entry: retiring_entries) {
+ if (entry->write_bytes()) {
+ ldout(cct, 20) << "Freeing " << entry->ram_entry.write_data.oid.pool_uuid_lo
+ << "." << entry->ram_entry.write_data.oid.off << dendl;
+ TX_FREE(entry->ram_entry.write_data);
+ } else {
+ ldout(cct, 20) << "Retiring non-write: " << *entry << dendl;
+ }
+ }
+ } TX_ONCOMMIT {
+ } TX_ONABORT {
+ lderr(cct) << "failed to commit free of" << retiring_entries.size()
+ << " log entries (" << this->m_log_pool_name << ")" << dendl;
+ ceph_assert(false);
+ } TX_FINALLY {
+ } TX_END;
+ tx_end = ceph_clock_now();
+ }
+ m_perfcounter->tinc(l_librbd_pwl_retire_tx_t, tx_end - tx_start);
+ m_perfcounter->hinc(l_librbd_pwl_retire_tx_t_hist, utime_t(tx_end - tx_start).to_nsec(),
+ retiring_entries.size());
+
+ bool need_update_state = false;
+ /* Update runtime copy of first_valid, and free entries counts */
+ {
+ std::lock_guard locker(m_lock);
+
+ ceph_assert(this->m_first_valid_entry == initial_first_valid_entry);
+ this->m_first_valid_entry = first_valid_entry;
+ this->m_free_log_entries += retiring_entries.size();
+ if (!m_cache_state->empty && m_log_entries.empty()) {
+ m_cache_state->empty = true;
+ this->update_image_cache_state();
+ need_update_state = true;
+ }
+ for (auto &entry: retiring_entries) {
+ if (entry->write_bytes()) {
+ ceph_assert(this->m_bytes_cached >= entry->write_bytes());
+ this->m_bytes_cached -= entry->write_bytes();
+ uint64_t entry_allocation_size = entry->write_bytes();
+ if (entry_allocation_size < MIN_WRITE_ALLOC_SIZE) {
+ entry_allocation_size = MIN_WRITE_ALLOC_SIZE;
+ }
+ ceph_assert(this->m_bytes_allocated >= entry_allocation_size);
+ this->m_bytes_allocated -= entry_allocation_size;
+ }
+ }
+ this->m_alloc_failed_since_retire = false;
+ this->wake_up();
+ }
+ if (need_update_state) {
+ std::unique_lock locker(m_lock);
+ this->write_image_cache_state(locker);
+ }
+ } else {
+ ldout(cct, 20) << "Nothing to retire" << dendl;
+ return false;
+ }
+ return true;
+}
+
+template <typename I>
+void WriteLog<I>::construct_flush_entries(pwl::GenericLogEntries entries_to_flush,
+ DeferredContexts &post_unlock,
+ bool has_write_entry) {
+ bool invalidating = this->m_invalidating; // snapshot so we behave consistently
+
+ for (auto &log_entry : entries_to_flush) {
+ GuardedRequestFunctionContext *guarded_ctx =
+ new GuardedRequestFunctionContext([this, log_entry, invalidating]
+ (GuardedRequestFunctionContext &guard_ctx) {
+ log_entry->m_cell = guard_ctx.cell;
+ Context *ctx = this->construct_flush_entry(log_entry, invalidating);
+
+ if (!invalidating) {
+ ctx = new LambdaContext(
+ [this, log_entry, ctx](int r) {
+ m_image_ctx.op_work_queue->queue(new LambdaContext(
+ [this, log_entry, ctx](int r) {
+ ldout(m_image_ctx.cct, 15) << "flushing:" << log_entry
+ << " " << *log_entry << dendl;
+ log_entry->writeback(this->m_image_writeback, ctx);
+ }), 0);
+ });
+ }
+
+ ctx->complete(0);
+ });
+ this->detain_flush_guard_request(log_entry, guarded_ctx);
+ }
+}
+
+const unsigned long int ops_flushed_together = 4;
+/*
+ * Performs the pmem buffer flush on all scheduled ops, then schedules
+ * the log event append operation for all of them.
+ */
+template <typename I>
+void WriteLog<I>::flush_then_append_scheduled_ops(void)
+{
+ GenericLogOperations ops;
+ bool ops_remain = false;
+ ldout(m_image_ctx.cct, 20) << dendl;
+ do {
+ {
+ ops.clear();
+ std::lock_guard locker(m_lock);
+ if (m_ops_to_flush.size()) {
+ auto last_in_batch = m_ops_to_flush.begin();
+ unsigned int ops_to_flush = m_ops_to_flush.size();
+ if (ops_to_flush > ops_flushed_together) {
+ ops_to_flush = ops_flushed_together;
+ }
+ ldout(m_image_ctx.cct, 20) << "should flush " << ops_to_flush << dendl;
+ std::advance(last_in_batch, ops_to_flush);
+ ops.splice(ops.end(), m_ops_to_flush, m_ops_to_flush.begin(), last_in_batch);
+ ops_remain = !m_ops_to_flush.empty();
+ ldout(m_image_ctx.cct, 20) << "flushing " << ops.size() << ", "
+ << m_ops_to_flush.size() << " remain" << dendl;
+ } else {
+ ops_remain = false;
+ }
+ }
+ if (ops_remain) {
+ enlist_op_flusher();
+ }
+
+ /* Ops subsequently scheduled for flush may finish before these,
+ * which is fine. We're unconcerned with completion order until we
+ * get to the log message append step. */
+ if (ops.size()) {
+ flush_pmem_buffer(ops);
+ schedule_append_ops(ops, nullptr);
+ }
+ } while (ops_remain);
+ append_scheduled_ops();
+}
+
+/*
+ * Performs the log event append operation for all of the scheduled
+ * events.
+ */
+template <typename I>
+void WriteLog<I>::append_scheduled_ops(void) {
+ GenericLogOperations ops;
+ int append_result = 0;
+ bool ops_remain = false;
+ bool appending = false; /* true if we set m_appending */
+ ldout(m_image_ctx.cct, 20) << dendl;
+ do {
+ ops.clear();
+ this->append_scheduled(ops, ops_remain, appending, true);
+
+ if (ops.size()) {
+ std::lock_guard locker(this->m_log_append_lock);
+ alloc_op_log_entries(ops);
+ append_result = append_op_log_entries(ops);
+ }
+
+ int num_ops = ops.size();
+ if (num_ops) {
+ /* New entries may be flushable. Completion will wake up flusher. */
+ this->complete_op_log_entries(std::move(ops), append_result);
+ }
+ } while (ops_remain);
+}
+
+template <typename I>
+void WriteLog<I>::enlist_op_flusher()
+{
+ this->m_async_flush_ops++;
+ this->m_async_op_tracker.start_op();
+ Context *flush_ctx = new LambdaContext([this](int r) {
+ flush_then_append_scheduled_ops();
+ this->m_async_flush_ops--;
+ this->m_async_op_tracker.finish_op();
+ });
+ this->m_work_queue.queue(flush_ctx);
+}
+
+template <typename I>
+void WriteLog<I>::setup_schedule_append(
+ pwl::GenericLogOperationsVector &ops, bool do_early_flush,
+ C_BlockIORequestT *req) {
+ if (do_early_flush) {
+ /* This caller is waiting for persist, so we'll use their thread to
+ * expedite it */
+ flush_pmem_buffer(ops);
+ this->schedule_append(ops);
+ } else {
+ /* This is probably not still the caller's thread, so do the payload
+ * flushing/replicating later. */
+ schedule_flush_and_append(ops);
+ }
+}
+
+/*
+ * Takes custody of ops. They'll all get their log entries appended,
+ * and have their on_write_persist contexts completed once they and
+ * all prior log entries are persisted everywhere.
+ */
+template <typename I>
+void WriteLog<I>::schedule_append_ops(GenericLogOperations &ops, C_BlockIORequestT *req)
+{
+ bool need_finisher;
+ GenericLogOperationsVector appending;
+
+ std::copy(std::begin(ops), std::end(ops), std::back_inserter(appending));
+ {
+ std::lock_guard locker(m_lock);
+
+ need_finisher = this->m_ops_to_append.empty() && !this->m_appending;
+ this->m_ops_to_append.splice(this->m_ops_to_append.end(), ops);
+ }
+
+ if (need_finisher) {
+ //enlist op appender
+ this->m_async_append_ops++;
+ this->m_async_op_tracker.start_op();
+ Context *append_ctx = new LambdaContext([this](int r) {
+ append_scheduled_ops();
+ this->m_async_append_ops--;
+ this->m_async_op_tracker.finish_op();
+ });
+ this->m_work_queue.queue(append_ctx);
+ }
+
+ for (auto &op : appending) {
+ op->appending();
+ }
+}
+
+/*
+ * Takes custody of ops. They'll all get their pmem blocks flushed,
+ * then get their log entries appended.
+ */
+template <typename I>
+void WriteLog<I>::schedule_flush_and_append(GenericLogOperationsVector &ops)
+{
+ GenericLogOperations to_flush(ops.begin(), ops.end());
+ bool need_finisher;
+ ldout(m_image_ctx.cct, 20) << dendl;
+ {
+ std::lock_guard locker(m_lock);
+
+ need_finisher = m_ops_to_flush.empty();
+ m_ops_to_flush.splice(m_ops_to_flush.end(), to_flush);
+ }
+
+ if (need_finisher) {
+ enlist_op_flusher();
+ }
+}
+
+template <typename I>
+void WriteLog<I>::process_work() {
+ CephContext *cct = m_image_ctx.cct;
+ int max_iterations = 4;
+ bool wake_up_requested = false;
+ uint64_t aggressive_high_water_bytes = this->m_bytes_allocated_cap * AGGRESSIVE_RETIRE_HIGH_WATER;
+ uint64_t high_water_bytes = this->m_bytes_allocated_cap * RETIRE_HIGH_WATER;
+ uint64_t low_water_bytes = this->m_bytes_allocated_cap * RETIRE_LOW_WATER;
+ uint64_t aggressive_high_water_entries = this->m_total_log_entries * AGGRESSIVE_RETIRE_HIGH_WATER;
+ uint64_t high_water_entries = this->m_total_log_entries * RETIRE_HIGH_WATER;
+ uint64_t low_water_entries = this->m_total_log_entries * RETIRE_LOW_WATER;
+
+ ldout(cct, 20) << dendl;
+
+ do {
+ {
+ std::lock_guard locker(m_lock);
+ this->m_wake_up_requested = false;
+ }
+ if (this->m_alloc_failed_since_retire || this->m_invalidating ||
+ this->m_bytes_allocated > high_water_bytes ||
+ (m_log_entries.size() > high_water_entries)) {
+ int retired = 0;
+ utime_t started = ceph_clock_now();
+ ldout(m_image_ctx.cct, 10) << "alloc_fail=" << this->m_alloc_failed_since_retire
+ << ", allocated > high_water="
+ << (this->m_bytes_allocated > high_water_bytes)
+ << ", allocated_entries > high_water="
+ << (m_log_entries.size() > high_water_entries)
+ << dendl;
+ while (this->m_alloc_failed_since_retire || this->m_invalidating ||
+ (this->m_bytes_allocated > high_water_bytes) ||
+ (m_log_entries.size() > high_water_entries) ||
+ (((this->m_bytes_allocated > low_water_bytes) ||
+ (m_log_entries.size() > low_water_entries)) &&
+ (utime_t(ceph_clock_now() - started).to_msec() < RETIRE_BATCH_TIME_LIMIT_MS))) {
+ if (!retire_entries((this->m_shutting_down || this->m_invalidating ||
+ (this->m_bytes_allocated > aggressive_high_water_bytes) ||
+ (m_log_entries.size() > aggressive_high_water_entries) ||
+ this->m_alloc_failed_since_retire)
+ ? MAX_ALLOC_PER_TRANSACTION
+ : MAX_FREE_PER_TRANSACTION)) {
+ break;
+ }
+ retired++;
+ this->dispatch_deferred_writes();
+ this->process_writeback_dirty_entries();
+ }
+ ldout(m_image_ctx.cct, 10) << "Retired " << retired << " times" << dendl;
+ }
+ this->dispatch_deferred_writes();
+ this->process_writeback_dirty_entries();
+
+ {
+ std::lock_guard locker(m_lock);
+ wake_up_requested = this->m_wake_up_requested;
+ }
+ } while (wake_up_requested && --max_iterations > 0);
+
+ {
+ std::lock_guard locker(m_lock);
+ this->m_wake_up_scheduled = false;
+ /* Reschedule if it's still requested */
+ if (this->m_wake_up_requested) {
+ this->wake_up();
+ }
+ }
+}
+
+/*
+ * Flush the pmem regions for the data blocks of a set of operations
+ *
+ * V is expected to be GenericLogOperations<I>, or GenericLogOperationsVector<I>
+ */
+template <typename I>
+template <typename V>
+void WriteLog<I>::flush_pmem_buffer(V& ops)
+{
+ utime_t now = ceph_clock_now();
+ for (auto &operation : ops) {
+ if (operation->reserved_allocated()) {
+ operation->buf_persist_start_time = now;
+ } else {
+ ldout(m_image_ctx.cct, 20) << "skipping non-write op: "
+ << *operation << dendl;
+ }
+ }
+
+ for (auto &operation : ops) {
+ if(operation->is_writing_op()) {
+ auto log_entry = static_pointer_cast<WriteLogEntry>(operation->get_log_entry());
+ pmemobj_flush(m_log_pool, log_entry->cache_buffer, log_entry->write_bytes());
+ }
+ }
+
+ /* Drain once for all */
+ pmemobj_drain(m_log_pool);
+
+ now = ceph_clock_now();
+ for (auto &operation : ops) {
+ if (operation->reserved_allocated()) {
+ operation->buf_persist_comp_time = now;
+ } else {
+ ldout(m_image_ctx.cct, 20) << "skipping non-write op: "
+ << *operation << dendl;
+ }
+ }
+}
+
+/**
+ * Update/persist the last flushed sync point in the log
+ */
+template <typename I>
+void WriteLog<I>::persist_last_flushed_sync_gen()
+{
+ TOID(struct WriteLogPoolRoot) pool_root;
+ pool_root = POBJ_ROOT(m_log_pool, struct WriteLogPoolRoot);
+ uint64_t flushed_sync_gen;
+
+ std::lock_guard append_locker(this->m_log_append_lock);
+ {
+ std::lock_guard locker(m_lock);
+ flushed_sync_gen = this->m_flushed_sync_gen;
+ }
+
+ if (D_RO(pool_root)->flushed_sync_gen < flushed_sync_gen) {
+ ldout(m_image_ctx.cct, 15) << "flushed_sync_gen in log updated from "
+ << D_RO(pool_root)->flushed_sync_gen << " to "
+ << flushed_sync_gen << dendl;
+ TX_BEGIN(m_log_pool) {
+ D_RW(pool_root)->flushed_sync_gen = flushed_sync_gen;
+ } TX_ONCOMMIT {
+ } TX_ONABORT {
+ lderr(m_image_ctx.cct) << "failed to commit update of flushed sync point" << dendl;
+ ceph_assert(false);
+ } TX_FINALLY {
+ } TX_END;
+ }
+}
+
+template <typename I>
+void WriteLog<I>::reserve_cache(C_BlockIORequestT *req,
+ bool &alloc_succeeds, bool &no_space) {
+ std::vector<WriteBufferAllocation>& buffers = req->get_resources_buffers();
+ for (auto &buffer : buffers) {
+ utime_t before_reserve = ceph_clock_now();
+ buffer.buffer_oid = pmemobj_reserve(m_log_pool,
+ &buffer.buffer_alloc_action,
+ buffer.allocation_size,
+ 0 /* Object type */);
+ buffer.allocation_lat = ceph_clock_now() - before_reserve;
+ if (TOID_IS_NULL(buffer.buffer_oid)) {
+ if (!req->has_io_waited_for_buffers()) {
+ req->set_io_waited_for_buffers(true);
+ }
+ ldout(m_image_ctx.cct, 5) << "can't allocate all data buffers: "
+ << pmemobj_errormsg() << ". "
+ << *req << dendl;
+ alloc_succeeds = false;
+ no_space = true; /* Entries need to be retired */
+
+ if (this->m_free_log_entries == this->m_total_log_entries - 1) {
+ /* When the cache is empty, there is still no space to allocate.
+ * Defragment. */
+ pmemobj_defrag(m_log_pool, NULL, 0, NULL);
+ }
+ break;
+ } else {
+ buffer.allocated = true;
+ }
+ ldout(m_image_ctx.cct, 20) << "Allocated " << buffer.buffer_oid.oid.pool_uuid_lo
+ << "." << buffer.buffer_oid.oid.off
+ << ", size=" << buffer.allocation_size << dendl;
+ }
+}
+
+template<typename I>
+void WriteLog<I>::copy_bl_to_buffer(
+ WriteRequestResources *resources, std::unique_ptr<WriteLogOperationSet> &op_set) {
+ auto allocation = resources->buffers.begin();
+ for (auto &operation : op_set->operations) {
+ operation->copy_bl_to_cache_buffer(allocation);
+ allocation++;
+ }
+}
+
+template <typename I>
+bool WriteLog<I>::alloc_resources(C_BlockIORequestT *req) {
+ bool alloc_succeeds = true;
+ uint64_t bytes_allocated = 0;
+ uint64_t bytes_cached = 0;
+ uint64_t bytes_dirtied = 0;
+ uint64_t num_lanes = 0;
+ uint64_t num_unpublished_reserves = 0;
+ uint64_t num_log_entries = 0;
+
+ ldout(m_image_ctx.cct, 20) << dendl;
+ // Setup buffer, and get all the number of required resources
+ req->setup_buffer_resources(&bytes_cached, &bytes_dirtied, &bytes_allocated,
+ &num_lanes, &num_log_entries, &num_unpublished_reserves);
+
+ alloc_succeeds = this->check_allocation(req, bytes_cached, bytes_dirtied,
+ bytes_allocated, num_lanes, num_log_entries,
+ num_unpublished_reserves);
+
+ std::vector<WriteBufferAllocation>& buffers = req->get_resources_buffers();
+ if (!alloc_succeeds) {
+ /* On alloc failure, free any buffers we did allocate */
+ for (auto &buffer : buffers) {
+ if (buffer.allocated) {
+ pmemobj_cancel(m_log_pool, &buffer.buffer_alloc_action, 1);
+ }
+ }
+ }
+
+ req->set_allocated(alloc_succeeds);
+ return alloc_succeeds;
+}
+
+template <typename I>
+void WriteLog<I>::complete_user_request(Context *&user_req, int r) {
+ user_req->complete(r);
+ // Set user_req as null as it is deleted
+ user_req = nullptr;
+}
+
+} // namespace rwl
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+template class librbd::cache::pwl::rwl::WriteLog<librbd::ImageCtx>;
diff --git a/src/librbd/cache/pwl/rwl/WriteLog.h b/src/librbd/cache/pwl/rwl/WriteLog.h
new file mode 100644
index 000000000..5083a2568
--- /dev/null
+++ b/src/librbd/cache/pwl/rwl/WriteLog.h
@@ -0,0 +1,124 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_CACHE_REPLICATED_WRITE_LOG
+#define CEPH_LIBRBD_CACHE_REPLICATED_WRITE_LOG
+
+#include <functional>
+#include <libpmemobj.h>
+#include <list>
+#include "common/Timer.h"
+#include "common/RWLock.h"
+#include "common/WorkQueue.h"
+#include "common/AsyncOpTracker.h"
+#include "librbd/cache/ImageWriteback.h"
+#include "librbd/Utils.h"
+#include "librbd/BlockGuard.h"
+#include "librbd/cache/Types.h"
+#include "librbd/cache/pwl/AbstractWriteLog.h"
+#include "librbd/cache/pwl/LogMap.h"
+#include "librbd/cache/pwl/LogOperation.h"
+#include "librbd/cache/pwl/Request.h"
+#include "librbd/cache/pwl/rwl/Builder.h"
+
+class Context;
+
+namespace librbd {
+
+struct ImageCtx;
+
+namespace cache {
+namespace pwl {
+namespace rwl {
+
+template <typename ImageCtxT>
+class WriteLog : public AbstractWriteLog<ImageCtxT> {
+public:
+ WriteLog(
+ ImageCtxT &image_ctx, librbd::cache::pwl::ImageCacheState<ImageCtxT>* cache_state,
+ ImageWritebackInterface& image_writeback,
+ plugin::Api<ImageCtxT>& plugin_api);
+ ~WriteLog();
+ WriteLog(const WriteLog&) = delete;
+ WriteLog &operator=(const WriteLog&) = delete;
+
+ typedef io::Extent Extent;
+ using This = AbstractWriteLog<ImageCtxT>;
+ using C_WriteRequestT = pwl::C_WriteRequest<This>;
+ using C_WriteSameRequestT = pwl::C_WriteSameRequest<This>;
+
+ void copy_bl_to_buffer(
+ WriteRequestResources *resources, std::unique_ptr<WriteLogOperationSet> &op_set) override;
+ void complete_user_request(Context *&user_req, int r) override;
+private:
+ using C_BlockIORequestT = pwl::C_BlockIORequest<This>;
+ using C_FlushRequestT = pwl::C_FlushRequest<This>;
+ using C_DiscardRequestT = pwl::C_DiscardRequest<This>;
+
+ PMEMobjpool *m_log_pool = nullptr;
+ Builder<This> *m_builderobj;
+ const char* m_pwl_pool_layout_name;
+ const uint64_t MAX_EXTENT_SIZE = 1048576;
+
+ Builder<This>* create_builder();
+ void remove_pool_file();
+ void load_existing_entries(pwl::DeferredContexts &later);
+ void alloc_op_log_entries(pwl::GenericLogOperations &ops);
+ int append_op_log_entries(pwl::GenericLogOperations &ops);
+ void flush_then_append_scheduled_ops(void);
+ void enlist_op_flusher();
+ void flush_op_log_entries(pwl::GenericLogOperationsVector &ops);
+ template <typename V>
+ void flush_pmem_buffer(V& ops);
+ void inc_allocated_cached_bytes(
+ std::shared_ptr<pwl::GenericLogEntry> log_entry) override;
+protected:
+ using AbstractWriteLog<ImageCtxT>::m_lock;
+ using AbstractWriteLog<ImageCtxT>::m_log_entries;
+ using AbstractWriteLog<ImageCtxT>::m_image_ctx;
+ using AbstractWriteLog<ImageCtxT>::m_perfcounter;
+ using AbstractWriteLog<ImageCtxT>::m_ops_to_flush;
+ using AbstractWriteLog<ImageCtxT>::m_cache_state;
+ using AbstractWriteLog<ImageCtxT>::m_first_free_entry;
+ using AbstractWriteLog<ImageCtxT>::m_first_valid_entry;
+
+ void process_work() override;
+ void schedule_append_ops(pwl::GenericLogOperations &ops, C_BlockIORequestT *req) override;
+ void append_scheduled_ops(void) override;
+ void reserve_cache(C_BlockIORequestT *req,
+ bool &alloc_succeeds, bool &no_space) override;
+ void collect_read_extents(
+ uint64_t read_buffer_offset, LogMapEntry<GenericWriteLogEntry> map_entry,
+ std::vector<std::shared_ptr<GenericWriteLogEntry>> &log_entries_to_read,
+ std::vector<bufferlist*> &bls_to_read, uint64_t entry_hit_length,
+ Extent hit_extent, pwl::C_ReadRequest *read_ctx) override;
+ void complete_read(
+ std::vector<std::shared_ptr<GenericWriteLogEntry>> &log_entries_to_read,
+ std::vector<bufferlist*> &bls_to_read, Context *ctx) override;
+ bool retire_entries(const unsigned long int frees_per_tx) override;
+ void persist_last_flushed_sync_gen() override;
+ bool alloc_resources(C_BlockIORequestT *req) override;
+ void schedule_flush_and_append(pwl::GenericLogOperationsVector &ops) override;
+ void setup_schedule_append(
+ pwl::GenericLogOperationsVector &ops, bool do_early_flush,
+ C_BlockIORequestT *req) override;
+ void construct_flush_entries(pwl::GenericLogEntries entries_to_flush,
+ DeferredContexts &post_unlock,
+ bool has_write_entry) override;
+ bool initialize_pool(Context *on_finish, pwl::DeferredContexts &later) override;
+ void write_data_to_buffer(
+ std::shared_ptr<pwl::WriteLogEntry> ws_entry,
+ pwl::WriteLogCacheEntry *pmem_entry) override;
+ uint64_t get_max_extent() override {
+ return MAX_EXTENT_SIZE;
+ }
+};
+
+} // namespace rwl
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+extern template class librbd::cache::pwl::rwl::WriteLog<librbd::ImageCtx>;
+
+#endif // CEPH_LIBRBD_CACHE_REPLICATED_WRITE_LOG
diff --git a/src/librbd/cache/pwl/ssd/Builder.h b/src/librbd/cache/pwl/ssd/Builder.h
new file mode 100644
index 000000000..07b3fb869
--- /dev/null
+++ b/src/librbd/cache/pwl/ssd/Builder.h
@@ -0,0 +1,108 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_CACHE_PWL_SSD_BUILDER_H
+#define CEPH_LIBRBD_CACHE_PWL_SSD_BUILDER_H
+
+#include <iostream>
+#include "LogEntry.h"
+#include "ReadRequest.h"
+#include "Request.h"
+#include "LogOperation.h"
+
+#include "librbd/cache/ImageWriteback.h"
+#include "librbd/cache/pwl/Builder.h"
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+namespace ssd {
+
+template <typename T>
+class Builder : public pwl::Builder<T> {
+public:
+ std::shared_ptr<pwl::WriteLogEntry> create_write_log_entry(
+ uint64_t image_offset_bytes, uint64_t write_bytes) override {
+ return std::make_shared<WriteLogEntry>(image_offset_bytes, write_bytes);
+ }
+ std::shared_ptr<pwl::WriteLogEntry> create_write_log_entry(
+ std::shared_ptr<SyncPointLogEntry> sync_point_entry,
+ uint64_t image_offset_bytes, uint64_t write_bytes) override {
+ return std::make_shared<WriteLogEntry>(
+ sync_point_entry, image_offset_bytes, write_bytes);
+ }
+ std::shared_ptr<pwl::WriteLogEntry> create_writesame_log_entry(
+ uint64_t image_offset_bytes, uint64_t write_bytes,
+ uint32_t data_length) override {
+ return std::make_shared<WriteSameLogEntry>(
+ image_offset_bytes, write_bytes, data_length);
+ }
+ std::shared_ptr<pwl::WriteLogEntry> create_writesame_log_entry(
+ std::shared_ptr<SyncPointLogEntry> sync_point_entry,
+ uint64_t image_offset_bytes, uint64_t write_bytes,
+ uint32_t data_length) override {
+ return std::make_shared<WriteSameLogEntry>(
+ sync_point_entry, image_offset_bytes, write_bytes, data_length);
+ }
+ pwl::C_WriteRequest<T> *create_write_request(
+ T &pwl, utime_t arrived, io::Extents &&image_extents,
+ bufferlist&& bl, const int fadvise_flags, ceph::mutex &lock,
+ PerfCounters *perfcounter, Context *user_req) override {
+ return new C_WriteRequest<T>(
+ pwl, arrived, std::move(image_extents), std::move(bl),
+ fadvise_flags, lock, perfcounter, user_req);
+ }
+ pwl::C_WriteSameRequest<T> *create_writesame_request(
+ T &pwl, utime_t arrived, io::Extents &&image_extents,
+ bufferlist&& bl, const int fadvise_flags, ceph::mutex &lock,
+ PerfCounters *perfcounter, Context *user_req) override {
+ return new C_WriteSameRequest<T>(
+ pwl, arrived, std::move(image_extents), std::move(bl),
+ fadvise_flags, lock, perfcounter, user_req);
+ }
+ pwl::C_WriteRequest<T> *create_comp_and_write_request(
+ T &pwl, utime_t arrived, io::Extents &&image_extents,
+ bufferlist&& cmp_bl, bufferlist&& bl, uint64_t *mismatch_offset,
+ const int fadvise_flags, ceph::mutex &lock,
+ PerfCounters *perfcounter, Context *user_req) override {
+ return new C_CompAndWriteRequest<T>(
+ pwl, arrived, std::move(image_extents), std::move(cmp_bl),
+ std::move(bl), mismatch_offset, fadvise_flags,
+ lock, perfcounter, user_req);
+ }
+ std::shared_ptr<pwl::WriteLogOperation> create_write_log_operation(
+ WriteLogOperationSet &set, uint64_t image_offset_bytes,
+ uint64_t write_bytes, CephContext *cct,
+ std::shared_ptr<pwl::WriteLogEntry> write_log_entry) {
+ return std::make_shared<WriteLogOperation>(
+ set, image_offset_bytes, write_bytes, cct, write_log_entry);
+ }
+ std::shared_ptr<pwl::WriteLogOperation> create_write_log_operation(
+ WriteLogOperationSet &set, uint64_t image_offset_bytes,
+ uint64_t write_bytes, uint32_t data_len, CephContext *cct,
+ std::shared_ptr<pwl::WriteLogEntry> writesame_log_entry) {
+ return std::make_shared<WriteLogOperation>(
+ set, image_offset_bytes, write_bytes, data_len, cct,
+ writesame_log_entry);
+ }
+ std::shared_ptr<pwl::DiscardLogOperation> create_discard_log_operation(
+ std::shared_ptr<SyncPoint> sync_point, uint64_t image_offset_bytes,
+ uint64_t write_bytes, uint32_t discard_granularity_bytes,
+ utime_t dispatch_time, PerfCounters *perfcounter, CephContext *cct) {
+ return std::make_shared<DiscardLogOperation>(
+ sync_point, image_offset_bytes, write_bytes, discard_granularity_bytes,
+ dispatch_time, perfcounter, cct);
+ }
+ C_ReadRequest *create_read_request(CephContext *cct, utime_t arrived,
+ PerfCounters *perfcounter, ceph::bufferlist *bl, Context *on_finish) {
+ return new C_ReadRequest(cct, arrived, perfcounter, bl, on_finish);
+ }
+};
+
+
+} // namespace ssd
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+#endif // CEPH_LIBRBD_CACHE_PWL_SSD_BUILDER_H
diff --git a/src/librbd/cache/pwl/ssd/LogEntry.cc b/src/librbd/cache/pwl/ssd/LogEntry.cc
new file mode 100644
index 000000000..0e6edd87b
--- /dev/null
+++ b/src/librbd/cache/pwl/ssd/LogEntry.cc
@@ -0,0 +1,63 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "librbd/cache/ImageWriteback.h"
+#include "librbd/cache/pwl/ssd/LogEntry.h"
+
+#define dout_subsys ceph_subsys_rbd_pwl
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::cache::pwl::ssd::WriteLogEntry: " \
+ << this << " " << __func__ << ": "
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+namespace ssd {
+
+void WriteLogEntry::init_cache_bl(
+ bufferlist &src_bl, uint64_t off, uint64_t len) {
+ cache_bl.clear();
+ cache_bl.substr_of(src_bl, off, len);
+}
+
+buffer::list& WriteLogEntry::get_cache_bl() {
+ return cache_bl;
+}
+
+void WriteLogEntry::copy_cache_bl(bufferlist *out) {
+ std::lock_guard locker(m_entry_bl_lock);
+ *out = cache_bl;
+}
+
+void WriteLogEntry::remove_cache_bl() {
+ std::lock_guard locker(m_entry_bl_lock);
+ cache_bl.clear();
+}
+
+unsigned int WriteLogEntry::get_aligned_data_size() const {
+ if (cache_bl.length()) {
+ return round_up_to(cache_bl.length(), MIN_WRITE_ALLOC_SSD_SIZE);
+ }
+ return round_up_to(write_bytes(), MIN_WRITE_ALLOC_SSD_SIZE);
+}
+
+void WriteLogEntry::writeback_bl(
+ librbd::cache::ImageWritebackInterface &image_writeback,
+ Context *ctx, ceph::bufferlist&& bl) {
+ image_writeback.aio_write({{ram_entry.image_offset_bytes,
+ ram_entry.write_bytes}},
+ std::move(bl), 0, ctx);
+}
+
+void WriteSameLogEntry::writeback_bl(
+ librbd::cache::ImageWritebackInterface &image_writeback,
+ Context *ctx, ceph::bufferlist &&bl) {
+ image_writeback.aio_writesame(ram_entry.image_offset_bytes,
+ ram_entry.write_bytes,
+ std::move(bl), 0, ctx);
+}
+
+} // namespace ssd
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
diff --git a/src/librbd/cache/pwl/ssd/LogEntry.h b/src/librbd/cache/pwl/ssd/LogEntry.h
new file mode 100644
index 000000000..8e26f661f
--- /dev/null
+++ b/src/librbd/cache/pwl/ssd/LogEntry.h
@@ -0,0 +1,75 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// // vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_CACHE_PWL_SSD_LOG_ENTRY_H
+#define CEPH_LIBRBD_CACHE_PWL_SSD_LOG_ENTRY_H
+
+#include "librbd/cache/pwl/LogEntry.h"
+
+namespace librbd {
+namespace cache {
+class ImageWritebackInterface;
+namespace pwl {
+namespace ssd {
+
+class WriteLogEntry : public pwl::WriteLogEntry {
+public:
+ WriteLogEntry(
+ std::shared_ptr<SyncPointLogEntry> sync_point_entry,
+ uint64_t image_offset_bytes, uint64_t write_bytes)
+ : pwl::WriteLogEntry(sync_point_entry, image_offset_bytes, write_bytes) {}
+ WriteLogEntry(
+ uint64_t image_offset_bytes, uint64_t write_bytes)
+ : pwl::WriteLogEntry(image_offset_bytes, write_bytes) {}
+ WriteLogEntry(
+ std::shared_ptr<SyncPointLogEntry> sync_point_entry,
+ uint64_t image_offset_bytes, uint64_t write_bytes,
+ uint32_t data_length)
+ : pwl::WriteLogEntry(sync_point_entry, image_offset_bytes,
+ write_bytes, data_length) {}
+ WriteLogEntry(
+ uint64_t image_offset_bytes, uint64_t write_bytes,
+ uint32_t data_length)
+ : pwl::WriteLogEntry(image_offset_bytes, write_bytes, data_length) {}
+ ~WriteLogEntry() {}
+ WriteLogEntry(const WriteLogEntry&) = delete;
+ WriteLogEntry &operator=(const WriteLogEntry&) = delete;
+ void writeback_bl(librbd::cache::ImageWritebackInterface &image_writeback,
+ Context *ctx, ceph::bufferlist &&bl) override;
+ void init_cache_bl(bufferlist &src_bl, uint64_t off, uint64_t len) override;
+ buffer::list &get_cache_bl() override;
+ void copy_cache_bl(bufferlist *out) override;
+ void remove_cache_bl() override;
+ unsigned int get_aligned_data_size() const override;
+ void inc_bl_refs() { bl_refs++; };
+ void dec_bl_refs() { bl_refs--; };
+ unsigned int reader_count() const override {
+ return bl_refs;
+ }
+};
+
+class WriteSameLogEntry : public WriteLogEntry {
+public:
+ WriteSameLogEntry(
+ std::shared_ptr<SyncPointLogEntry> sync_point_entry,
+ uint64_t image_offset_bytes, uint64_t write_bytes,
+ uint32_t data_length)
+ : WriteLogEntry(sync_point_entry, image_offset_bytes,
+ write_bytes, data_length) {}
+ WriteSameLogEntry(
+ uint64_t image_offset_bytes, uint64_t write_bytes,
+ uint32_t data_length)
+ : WriteLogEntry(image_offset_bytes, write_bytes, data_length) {}
+ ~WriteSameLogEntry() {}
+ WriteSameLogEntry(const WriteSameLogEntry&) = delete;
+ WriteSameLogEntry &operator=(const WriteSameLogEntry&) = delete;
+ void writeback_bl(librbd::cache::ImageWritebackInterface &image_writeback,
+ Context *ctx, ceph::bufferlist &&bl) override;
+};
+
+} // namespace ssd
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+#endif // CEPH_LIBRBD_CACHE_PWL_SSD_LOG_ENTRY_H
diff --git a/src/librbd/cache/pwl/ssd/LogOperation.cc b/src/librbd/cache/pwl/ssd/LogOperation.cc
new file mode 100644
index 000000000..c8080e37d
--- /dev/null
+++ b/src/librbd/cache/pwl/ssd/LogOperation.cc
@@ -0,0 +1,36 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "LogOperation.h"
+
+#define dout_subsys ceph_subsys_rbd_pwl
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::cache::pwl::ssd::LogOperation: " \
+ << this << " " << __func__ << ": "
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+namespace ssd {
+
+void DiscardLogOperation::init_op(
+ uint64_t current_sync_gen, bool persist_on_flush,
+ uint64_t last_op_sequence_num, Context *write_persist,
+ Context *write_append) {
+ log_entry->init(current_sync_gen, persist_on_flush, last_op_sequence_num);
+ if (persist_on_flush) {
+ this->on_write_append = new LambdaContext(
+ [write_persist, write_append] (int r) {
+ write_append->complete(r);
+ write_persist->complete(r);
+ });
+ } else {
+ this->on_write_append = write_append;
+ this->on_write_persist = write_persist;
+ }
+}
+
+} // namespace ssd
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
diff --git a/src/librbd/cache/pwl/ssd/LogOperation.h b/src/librbd/cache/pwl/ssd/LogOperation.h
new file mode 100644
index 000000000..dbc89aa73
--- /dev/null
+++ b/src/librbd/cache/pwl/ssd/LogOperation.h
@@ -0,0 +1,35 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_CACHE_PWL_SSD_LOG_OPERATION_H
+#define CEPH_LIBRBD_CACHE_PWL_SSD_LOG_OPERATION_H
+
+#include "librbd/cache/pwl/LogOperation.h"
+
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+namespace ssd {
+
+class DiscardLogOperation : public pwl::DiscardLogOperation {
+public:
+ DiscardLogOperation(
+ std::shared_ptr<SyncPoint> sync_point, uint64_t image_offset_bytes,
+ uint64_t write_bytes, uint32_t discard_granularity_bytes,
+ utime_t dispatch_time, PerfCounters *perfcounter, CephContext *cct)
+ : pwl::DiscardLogOperation(sync_point, image_offset_bytes, write_bytes,
+ discard_granularity_bytes, dispatch_time,
+ perfcounter, cct) {}
+ void init_op(
+ uint64_t current_sync_gen, bool persist_on_flush,
+ uint64_t last_op_sequence_num, Context *write_persist,
+ Context *write_append) override;
+};
+
+} // namespace ssd
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+#endif // CEPH_LIBRBD_CACHE_PWL_SSD_LOG_OPERATION_H
diff --git a/src/librbd/cache/pwl/ssd/ReadRequest.cc b/src/librbd/cache/pwl/ssd/ReadRequest.cc
new file mode 100644
index 000000000..1a80a8d8c
--- /dev/null
+++ b/src/librbd/cache/pwl/ssd/ReadRequest.cc
@@ -0,0 +1,92 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "ReadRequest.h"
+
+#define dout_subsys ceph_subsys_rbd_pwl
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::cache::pwl::ssd::ReadRequest: " << this << " " \
+ << __func__ << ": "
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+namespace ssd {
+
+void C_ReadRequest::finish(int r) {
+ ldout(m_cct, 20) << "(" << get_name() << "): r=" << r << dendl;
+ int hits = 0;
+ int misses = 0;
+ int hit_bytes = 0;
+ int miss_bytes = 0;
+ if (r >= 0) {
+ /*
+ * At this point the miss read has completed. We'll iterate through
+ * m_read_extents and produce *m_out_bl by assembling pieces of m_miss_bl
+ * and the individual hit extent bufs in the read extents that represent
+ * hits.
+ */
+ uint64_t miss_bl_offset = 0;
+ for (auto extent : read_extents) {
+ if (extent->m_bl.length()) {
+ /* This was a hit */
+ bufferlist data_bl;
+ if (extent->writesame) {
+ int data_len = extent->m_bl.length();
+ int read_buffer_offset = extent->truncate_offset;
+ if (extent->need_to_truncate && extent->truncate_offset >= data_len) {
+ read_buffer_offset = (extent->truncate_offset) % data_len;
+ }
+ // build data and truncate
+ bufferlist temp_bl;
+ uint64_t total_left_bytes = read_buffer_offset + extent->second;
+ while (total_left_bytes > 0) {
+ temp_bl.append(extent->m_bl);
+ total_left_bytes = total_left_bytes - data_len;
+ }
+ data_bl.substr_of(temp_bl, read_buffer_offset, extent->second);
+ m_out_bl->claim_append(data_bl);
+ } else if (extent->need_to_truncate) {
+ assert(extent->m_bl.length() >= extent->truncate_offset + extent->second);
+ data_bl.substr_of(extent->m_bl, extent->truncate_offset, extent->second);
+ m_out_bl->claim_append(data_bl);
+ } else {
+ assert(extent->second == extent->m_bl.length());
+ m_out_bl->claim_append(extent->m_bl);
+ }
+ ++hits;
+ hit_bytes += extent->second;
+ } else {
+ /* This was a miss. */
+ ++misses;
+ miss_bytes += extent->second;
+ bufferlist miss_extent_bl;
+ miss_extent_bl.substr_of(miss_bl, miss_bl_offset, extent->second);
+ /* Add this read miss bufferlist to the output bufferlist */
+ m_out_bl->claim_append(miss_extent_bl);
+ /* Consume these bytes in the read miss bufferlist */
+ miss_bl_offset += extent->second;
+ }
+ }
+ }
+ ldout(m_cct, 20) << "(" << get_name() << "): r=" << r << " bl=" << *m_out_bl << dendl;
+ utime_t now = ceph_clock_now();
+ ceph_assert((int)m_out_bl->length() == hit_bytes + miss_bytes);
+ m_on_finish->complete(r);
+ m_perfcounter->inc(l_librbd_pwl_rd_bytes, hit_bytes + miss_bytes);
+ m_perfcounter->inc(l_librbd_pwl_rd_hit_bytes, hit_bytes);
+ m_perfcounter->tinc(l_librbd_pwl_rd_latency, now - m_arrived_time);
+ if (!misses) {
+ m_perfcounter->inc(l_librbd_pwl_rd_hit_req, 1);
+ m_perfcounter->tinc(l_librbd_pwl_rd_hit_latency, now - m_arrived_time);
+ } else {
+ if (hits) {
+ m_perfcounter->inc(l_librbd_pwl_rd_part_hit_req, 1);
+ }
+ }
+}
+
+} // namespace ssd
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
diff --git a/src/librbd/cache/pwl/ssd/ReadRequest.h b/src/librbd/cache/pwl/ssd/ReadRequest.h
new file mode 100644
index 000000000..345c4aa65
--- /dev/null
+++ b/src/librbd/cache/pwl/ssd/ReadRequest.h
@@ -0,0 +1,34 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_CACHE_PWL_SSD_READ_REQUEST_H
+#define CEPH_LIBRBD_CACHE_PWL_SSD_READ_REQUEST_H
+
+#include "librbd/cache/pwl/ReadRequest.h"
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+namespace ssd {
+
+typedef std::vector<pwl::ImageExtentBuf> ImageExtentBufs;
+
+class C_ReadRequest : public pwl::C_ReadRequest {
+protected:
+ using pwl::C_ReadRequest::m_cct;
+ using pwl::C_ReadRequest::m_on_finish;
+ using pwl::C_ReadRequest::m_out_bl;
+ using pwl::C_ReadRequest::m_arrived_time;
+ using pwl::C_ReadRequest::m_perfcounter;
+public:
+ C_ReadRequest(CephContext *cct, utime_t arrived, PerfCounters *perfcounter, bufferlist *out_bl, Context *on_finish)
+ : pwl::C_ReadRequest(cct, arrived, perfcounter, out_bl, on_finish) {}
+ void finish(int r) override;
+};
+
+} // namespace ssd
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+#endif // CEPH_LIBRBD_CACHE_PWL_SSD_READ_REQUEST_H
diff --git a/src/librbd/cache/pwl/ssd/Request.cc b/src/librbd/cache/pwl/ssd/Request.cc
new file mode 100644
index 000000000..e92e547c8
--- /dev/null
+++ b/src/librbd/cache/pwl/ssd/Request.cc
@@ -0,0 +1,63 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "Request.h"
+
+#define dout_subsys ceph_subsys_rbd_pwl
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::cache::pwl::ssd::Request: " << this << " " \
+ << __func__ << ": "
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+namespace ssd {
+
+template <typename T>
+void C_WriteRequest<T>::setup_buffer_resources(
+ uint64_t *bytes_cached, uint64_t *bytes_dirtied, uint64_t *bytes_allocated,
+ uint64_t *number_lanes, uint64_t *number_log_entries,
+ uint64_t *number_unpublished_reserves) {
+
+ *bytes_cached = 0;
+ *bytes_allocated = 0;
+ *number_log_entries = this->image_extents.size();
+
+ for (auto &extent : this->image_extents) {
+ *bytes_cached += extent.second;
+ *bytes_allocated += round_up_to(extent.second, MIN_WRITE_ALLOC_SSD_SIZE);
+ }
+ *bytes_dirtied = *bytes_cached;
+}
+
+template <typename T>
+std::ostream &operator<<(std::ostream &os,
+ const C_CompAndWriteRequest<T> &req) {
+ os << (C_WriteRequest<T>&)req
+ << "cmp_bl=" << req.cmp_bl << ", "
+ << "read_bl=" << req.read_bl << ", "
+ << "compare_succeeded=" << req.compare_succeeded << ", "
+ << "mismatch_offset=" << req.mismatch_offset;
+ return os;
+}
+
+template <typename T>
+void C_WriteSameRequest<T>::setup_buffer_resources(
+ uint64_t *bytes_cached, uint64_t *bytes_dirtied, uint64_t *bytes_allocated,
+ uint64_t *number_lanes, uint64_t *number_log_entries,
+ uint64_t *number_unpublished_reserves) {
+ ceph_assert(this->image_extents.size() == 1);
+ *number_log_entries = 1;
+ *bytes_dirtied = this->image_extents[0].second;
+ *bytes_cached = this->bl.length();
+ *bytes_allocated = round_up_to(*bytes_cached, MIN_WRITE_ALLOC_SSD_SIZE);
+}
+
+} // namespace ssd
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+template class librbd::cache::pwl::ssd::C_WriteRequest<librbd::cache::pwl::AbstractWriteLog<librbd::ImageCtx> >;
+template class librbd::cache::pwl::ssd::C_WriteSameRequest<librbd::cache::pwl::AbstractWriteLog<librbd::ImageCtx> >;
+template class librbd::cache::pwl::ssd::C_CompAndWriteRequest<librbd::cache::pwl::AbstractWriteLog<librbd::ImageCtx> >;
diff --git a/src/librbd/cache/pwl/ssd/Request.h b/src/librbd/cache/pwl/ssd/Request.h
new file mode 100644
index 000000000..9bb3e85b9
--- /dev/null
+++ b/src/librbd/cache/pwl/ssd/Request.h
@@ -0,0 +1,92 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_CACHE_SSD_REQUEST_H
+#define CEPH_LIBRBD_CACHE_SSD_REQUEST_H
+
+#include "librbd/cache/pwl/Request.h"
+
+namespace librbd {
+class BlockGuardCell;
+
+namespace cache {
+namespace pwl {
+
+template<typename T>
+class AbstractWriteLog;
+
+namespace ssd {
+
+template <typename T>
+class C_WriteRequest : public pwl::C_WriteRequest<T> {
+public:
+ C_WriteRequest(
+ T &pwl, const utime_t arrived, io::Extents &&image_extents,
+ bufferlist&& cmp_bl, bufferlist&& bl, uint64_t *mismatch_offset,
+ const int fadvise_flags, ceph::mutex &lock,
+ PerfCounters *perfcounter, Context *user_req)
+ : pwl::C_WriteRequest<T>(
+ pwl, arrived, std::move(image_extents), std::move(cmp_bl),
+ std::move(bl), mismatch_offset, fadvise_flags,
+ lock, perfcounter, user_req) {}
+
+ C_WriteRequest(
+ T &pwl, const utime_t arrived, io::Extents &&image_extents,
+ bufferlist&& bl, const int fadvise_flags, ceph::mutex &lock,
+ PerfCounters *perfcounter, Context *user_req)
+ : pwl::C_WriteRequest<T>(
+ pwl, arrived, std::move(image_extents), std::move(bl),
+ fadvise_flags, lock, perfcounter, user_req) {}
+protected:
+ void setup_buffer_resources(
+ uint64_t *bytes_cached, uint64_t *bytes_dirtied,
+ uint64_t *bytes_allocated, uint64_t *number_lanes,
+ uint64_t *number_log_entries,
+ uint64_t *number_unpublished_reserves) override;
+};
+
+template <typename T>
+class C_CompAndWriteRequest : public C_WriteRequest<T> {
+public:
+ C_CompAndWriteRequest(
+ T &pwl, const utime_t arrived, io::Extents &&image_extents,
+ bufferlist&& cmp_bl, bufferlist&& bl, uint64_t *mismatch_offset,
+ const int fadvise_flags, ceph::mutex &lock,
+ PerfCounters *perfcounter, Context *user_req)
+ : C_WriteRequest<T>(
+ pwl, arrived, std::move(image_extents), std::move(cmp_bl),
+ std::move(bl), mismatch_offset,fadvise_flags,
+ lock, perfcounter, user_req) {}
+
+ const char *get_name() const override {
+ return "C_CompAndWriteRequest";
+ }
+ template <typename U>
+ friend std::ostream &operator<<(std::ostream &os,
+ const C_CompAndWriteRequest<U> &req);
+};
+
+template <typename T>
+class C_WriteSameRequest : public pwl::C_WriteSameRequest<T> {
+public:
+ C_WriteSameRequest(
+ T &pwl, const utime_t arrived, io::Extents &&image_extents,
+ bufferlist&& bl, const int fadvise_flags, ceph::mutex &lock,
+ PerfCounters *perfcounter, Context *user_req)
+ : pwl::C_WriteSameRequest<T>(
+ pwl, arrived, std::move(image_extents), std::move(bl), fadvise_flags,
+ lock, perfcounter, user_req) {}
+
+ void setup_buffer_resources(
+ uint64_t *bytes_cached, uint64_t *bytes_dirtied,
+ uint64_t *bytes_allocated, uint64_t *number_lanes,
+ uint64_t *number_log_entries,
+ uint64_t *number_unpublished_reserves) override;
+};
+
+} // namespace ssd
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+#endif // CEPH_LIBRBD_CACHE_SSD_REQUEST_H
diff --git a/src/librbd/cache/pwl/ssd/Types.h b/src/librbd/cache/pwl/ssd/Types.h
new file mode 100644
index 000000000..3ebad1fd9
--- /dev/null
+++ b/src/librbd/cache/pwl/ssd/Types.h
@@ -0,0 +1,51 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_CACHE_SSD_TYPES_H
+#define CEPH_LIBRBD_CACHE_SSD_TYPES_H
+
+#include "acconfig.h"
+
+#include "librbd/io/Types.h"
+#include "librbd/cache/pwl/Types.h"
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+namespace ssd {
+
+struct SuperBlock{
+ WriteLogPoolRoot root;
+
+ DENC(SuperBlock, v, p) {
+ DENC_START(1, 1, p);
+ denc(v.root, p);
+ DENC_FINISH(p);
+ }
+
+ void dump(Formatter *f) const {
+ f->dump_object("super", root);
+ }
+
+ static void generate_test_instances(list<SuperBlock*>& ls) {
+ ls.push_back(new SuperBlock());
+ ls.push_back(new SuperBlock);
+ ls.back()->root.layout_version = 3;
+ ls.back()->root.cur_sync_gen = 1;
+ ls.back()->root.pool_size = 10737418240;
+ ls.back()->root.flushed_sync_gen = 1;
+ ls.back()->root.block_size = 4096;
+ ls.back()->root.num_log_entries = 0;
+ ls.back()->root.first_free_entry = 30601;
+ ls.back()->root.first_valid_entry = 2;
+ }
+};
+
+} // namespace ssd
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+WRITE_CLASS_DENC(librbd::cache::pwl::ssd::SuperBlock)
+
+#endif // CEPH_LIBRBD_CACHE_SSD_TYPES_H
diff --git a/src/librbd/cache/pwl/ssd/WriteLog.cc b/src/librbd/cache/pwl/ssd/WriteLog.cc
new file mode 100644
index 000000000..00626506a
--- /dev/null
+++ b/src/librbd/cache/pwl/ssd/WriteLog.cc
@@ -0,0 +1,1158 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "WriteLog.h"
+#include "include/buffer.h"
+#include "include/Context.h"
+#include "include/ceph_assert.h"
+#include "common/deleter.h"
+#include "common/dout.h"
+#include "common/environment.h"
+#include "common/errno.h"
+#include "common/WorkQueue.h"
+#include "common/Timer.h"
+#include "common/perf_counters.h"
+#include "librbd/ImageCtx.h"
+#include "librbd/asio/ContextWQ.h"
+#include "librbd/cache/pwl/ImageCacheState.h"
+#include "librbd/cache/pwl/LogEntry.h"
+#include <map>
+#include <vector>
+
+#undef dout_subsys
+#define dout_subsys ceph_subsys_rbd_pwl
+#undef dout_prefix
+#define dout_prefix *_dout << "librbd::cache::pwl::ssd::WriteLog: " \
+ << this << " " << __func__ << ": "
+
+namespace librbd {
+namespace cache {
+namespace pwl {
+namespace ssd {
+
+using namespace librbd::cache::pwl;
+
+static bool is_valid_pool_root(const WriteLogPoolRoot& root) {
+ return root.pool_size % MIN_WRITE_ALLOC_SSD_SIZE == 0 &&
+ root.first_valid_entry >= DATA_RING_BUFFER_OFFSET &&
+ root.first_valid_entry < root.pool_size &&
+ root.first_valid_entry % MIN_WRITE_ALLOC_SSD_SIZE == 0 &&
+ root.first_free_entry >= DATA_RING_BUFFER_OFFSET &&
+ root.first_free_entry < root.pool_size &&
+ root.first_free_entry % MIN_WRITE_ALLOC_SSD_SIZE == 0;
+}
+
+template <typename I>
+Builder<AbstractWriteLog<I>>* WriteLog<I>::create_builder() {
+ m_builderobj = new Builder<This>();
+ return m_builderobj;
+}
+
+template <typename I>
+WriteLog<I>::WriteLog(
+ I &image_ctx, librbd::cache::pwl::ImageCacheState<I>* cache_state,
+ cache::ImageWritebackInterface& image_writeback,
+ plugin::Api<I>& plugin_api)
+ : AbstractWriteLog<I>(image_ctx, cache_state, create_builder(),
+ image_writeback, plugin_api)
+{
+}
+
+template <typename I>
+WriteLog<I>::~WriteLog() {
+ delete m_builderobj;
+}
+
+template <typename I>
+void WriteLog<I>::collect_read_extents(
+ uint64_t read_buffer_offset, LogMapEntry<GenericWriteLogEntry> map_entry,
+ std::vector<std::shared_ptr<GenericWriteLogEntry>> &log_entries_to_read,
+ std::vector<bufferlist*> &bls_to_read,
+ uint64_t entry_hit_length, Extent hit_extent,
+ pwl::C_ReadRequest *read_ctx) {
+ // Make a bl for this hit extent. This will add references to the
+ // write_entry->cache_bl */
+ ldout(m_image_ctx.cct, 5) << dendl;
+ auto write_entry = static_pointer_cast<WriteLogEntry>(map_entry.log_entry);
+ buffer::list hit_bl;
+ write_entry->copy_cache_bl(&hit_bl);
+ bool writesame = write_entry->is_writesame_entry();
+ auto hit_extent_buf = std::make_shared<ImageExtentBuf>(
+ hit_extent, hit_bl, true, read_buffer_offset, writesame);
+ read_ctx->read_extents.push_back(hit_extent_buf);
+
+ if (!hit_bl.length()) {
+ ldout(m_image_ctx.cct, 5) << "didn't hit RAM" << dendl;
+ auto read_extent = read_ctx->read_extents.back();
+ write_entry->inc_bl_refs();
+ log_entries_to_read.push_back(std::move(write_entry));
+ bls_to_read.push_back(&read_extent->m_bl);
+ }
+}
+
+template <typename I>
+void WriteLog<I>::complete_read(
+ std::vector<std::shared_ptr<GenericWriteLogEntry>> &log_entries_to_read,
+ std::vector<bufferlist*> &bls_to_read,
+ Context *ctx) {
+ if (!log_entries_to_read.empty()) {
+ aio_read_data_blocks(log_entries_to_read, bls_to_read, ctx);
+ } else {
+ ctx->complete(0);
+ }
+}
+
+template <typename I>
+int WriteLog<I>::create_and_open_bdev() {
+ CephContext *cct = m_image_ctx.cct;
+
+ bdev = BlockDevice::create(cct, this->m_log_pool_name, aio_cache_cb,
+ nullptr, nullptr, nullptr);
+ int r = bdev->open(this->m_log_pool_name);
+ if (r < 0) {
+ lderr(cct) << "failed to open bdev" << dendl;
+ delete bdev;
+ return r;
+ }
+
+ ceph_assert(this->m_log_pool_size % MIN_WRITE_ALLOC_SSD_SIZE == 0);
+ if (bdev->get_size() != this->m_log_pool_size) {
+ lderr(cct) << "size mismatch: bdev size " << bdev->get_size()
+ << " (block size " << bdev->get_block_size()
+ << ") != pool size " << this->m_log_pool_size << dendl;
+ bdev->close();
+ delete bdev;
+ return -EINVAL;
+ }
+
+ return 0;
+}
+
+template <typename I>
+bool WriteLog<I>::initialize_pool(Context *on_finish,
+ pwl::DeferredContexts &later) {
+ int r;
+ CephContext *cct = m_image_ctx.cct;
+
+ ceph_assert(ceph_mutex_is_locked_by_me(m_lock));
+ if (access(this->m_log_pool_name.c_str(), F_OK) != 0) {
+ int fd = ::open(this->m_log_pool_name.c_str(), O_RDWR|O_CREAT, 0644);
+ bool succeed = true;
+ if (fd >= 0) {
+ if (truncate(this->m_log_pool_name.c_str(),
+ this->m_log_pool_size) != 0) {
+ succeed = false;
+ }
+ ::close(fd);
+ } else {
+ succeed = false;
+ }
+ if (!succeed) {
+ m_cache_state->present = false;
+ m_cache_state->clean = true;
+ m_cache_state->empty = true;
+ /* TODO: filter/replace errnos that are meaningless to the caller */
+ on_finish->complete(-errno);
+ return false;
+ }
+
+ r = create_and_open_bdev();
+ if (r < 0) {
+ on_finish->complete(r);
+ return false;
+ }
+ m_cache_state->present = true;
+ m_cache_state->clean = true;
+ m_cache_state->empty = true;
+ /* new pool, calculate and store metadata */
+
+ /* Keep ring buffer at least MIN_WRITE_ALLOC_SSD_SIZE bytes free.
+ * In this way, when all ring buffer spaces are allocated,
+ * m_first_free_entry and m_first_valid_entry will not be equal.
+ * Equal only means the cache is empty. */
+ this->m_bytes_allocated_cap = this->m_log_pool_size -
+ DATA_RING_BUFFER_OFFSET - MIN_WRITE_ALLOC_SSD_SIZE;
+ /* Log ring empty */
+ m_first_free_entry = DATA_RING_BUFFER_OFFSET;
+ m_first_valid_entry = DATA_RING_BUFFER_OFFSET;
+
+ auto new_root = std::make_shared<WriteLogPoolRoot>(pool_root);
+ new_root->layout_version = SSD_LAYOUT_VERSION;
+ new_root->pool_size = this->m_log_pool_size;
+ new_root->flushed_sync_gen = this->m_flushed_sync_gen;
+ new_root->block_size = MIN_WRITE_ALLOC_SSD_SIZE;
+ new_root->first_free_entry = m_first_free_entry;
+ new_root->first_valid_entry = m_first_valid_entry;
+ new_root->num_log_entries = 0;
+ pool_root = *new_root;
+
+ r = update_pool_root_sync(new_root);
+ if (r != 0) {
+ lderr(cct) << "failed to initialize pool ("
+ << this->m_log_pool_name << ")" << dendl;
+ bdev->close();
+ delete bdev;
+ on_finish->complete(r);
+ return false;
+ }
+ } else {
+ ceph_assert(m_cache_state->present);
+ r = create_and_open_bdev();
+ if (r < 0) {
+ on_finish->complete(r);
+ return false;
+ }
+
+ bufferlist bl;
+ SuperBlock superblock;
+ ::IOContext ioctx(cct, nullptr);
+ r = bdev->read(0, MIN_WRITE_ALLOC_SSD_SIZE, &bl, &ioctx, false);
+ if (r < 0) {
+ lderr(cct) << "Read ssd cache superblock failed " << dendl;
+ goto error_handle;
+ }
+ auto p = bl.cbegin();
+ decode(superblock, p);
+ pool_root = superblock.root;
+ ldout(cct, 1) << "Decoded root: pool_size=" << pool_root.pool_size
+ << " first_valid_entry=" << pool_root.first_valid_entry
+ << " first_free_entry=" << pool_root.first_free_entry
+ << " flushed_sync_gen=" << pool_root.flushed_sync_gen
+ << dendl;
+ ceph_assert(is_valid_pool_root(pool_root));
+ if (pool_root.layout_version != SSD_LAYOUT_VERSION) {
+ lderr(cct) << "Pool layout version is "
+ << pool_root.layout_version
+ << " expected " << SSD_LAYOUT_VERSION
+ << dendl;
+ goto error_handle;
+ }
+ if (pool_root.block_size != MIN_WRITE_ALLOC_SSD_SIZE) {
+ lderr(cct) << "Pool block size is " << pool_root.block_size
+ << " expected " << MIN_WRITE_ALLOC_SSD_SIZE
+ << dendl;
+ goto error_handle;
+ }
+
+ this->m_log_pool_size = pool_root.pool_size;
+ this->m_flushed_sync_gen = pool_root.flushed_sync_gen;
+ this->m_first_valid_entry = pool_root.first_valid_entry;
+ this->m_first_free_entry = pool_root.first_free_entry;
+ this->m_bytes_allocated_cap = this->m_log_pool_size -
+ DATA_RING_BUFFER_OFFSET -
+ MIN_WRITE_ALLOC_SSD_SIZE;
+
+ load_existing_entries(later);
+ m_cache_state->clean = this->m_dirty_log_entries.empty();
+ m_cache_state->empty = m_log_entries.empty();
+ }
+ return true;
+
+error_handle:
+ bdev->close();
+ delete bdev;
+ on_finish->complete(-EINVAL);
+ return false;
+}
+
+template <typename I>
+void WriteLog<I>::remove_pool_file() {
+ ceph_assert(bdev);
+ bdev->close();
+ delete bdev;
+ bdev = nullptr;
+ ldout(m_image_ctx.cct, 5) << "block device is closed" << dendl;
+
+ if (m_cache_state->clean) {
+ ldout(m_image_ctx.cct, 5) << "Removing empty pool file: "
+ << this->m_log_pool_name << dendl;
+ if (remove(this->m_log_pool_name.c_str()) != 0) {
+ lderr(m_image_ctx.cct) << "failed to remove empty pool \""
+ << this->m_log_pool_name << "\": " << dendl;
+ } else {
+ m_cache_state->present = false;
+ }
+ } else {
+ ldout(m_image_ctx.cct, 5) << "Not removing pool file: "
+ << this->m_log_pool_name << dendl;
+ }
+}
+
+template <typename I>
+void WriteLog<I>::load_existing_entries(pwl::DeferredContexts &later) {
+ CephContext *cct = m_image_ctx.cct;
+ std::map<uint64_t, std::shared_ptr<SyncPointLogEntry>> sync_point_entries;
+ std::map<uint64_t, bool> missing_sync_points;
+
+ // Iterate through the log_entries and append all the write_bytes
+ // of each entry to fetch the pos of next 4k of log_entries. Iterate
+ // through the log entries and append them to the in-memory vector
+ for (uint64_t next_log_pos = this->m_first_valid_entry;
+ next_log_pos != this->m_first_free_entry; ) {
+ // read the entries from SSD cache and decode
+ bufferlist bl_entries;
+ ::IOContext ioctx_entry(cct, nullptr);
+ bdev->read(next_log_pos, MIN_WRITE_ALLOC_SSD_SIZE, &bl_entries,
+ &ioctx_entry, false);
+ std::vector<WriteLogCacheEntry> ssd_log_entries;
+ auto pl = bl_entries.cbegin();
+ decode(ssd_log_entries, pl);
+ ldout(cct, 5) << "decoded ssd log entries" << dendl;
+ uint64_t curr_log_pos = next_log_pos;
+ std::shared_ptr<GenericLogEntry> log_entry = nullptr;
+
+ for (auto it = ssd_log_entries.begin(); it != ssd_log_entries.end(); ++it) {
+ this->update_entries(&log_entry, &*it, missing_sync_points,
+ sync_point_entries, curr_log_pos);
+ log_entry->ram_entry = *it;
+ log_entry->log_entry_index = curr_log_pos;
+ log_entry->completed = true;
+ m_log_entries.push_back(log_entry);
+ next_log_pos += round_up_to(it->write_bytes, MIN_WRITE_ALLOC_SSD_SIZE);
+ }
+ // along with the write_bytes, add control block size too
+ next_log_pos += MIN_WRITE_ALLOC_SSD_SIZE;
+ if (next_log_pos >= this->m_log_pool_size) {
+ next_log_pos = next_log_pos % this->m_log_pool_size + DATA_RING_BUFFER_OFFSET;
+ }
+ }
+ this->update_sync_points(missing_sync_points, sync_point_entries, later);
+ if (m_first_valid_entry > m_first_free_entry) {
+ m_bytes_allocated = this->m_log_pool_size - m_first_valid_entry +
+ m_first_free_entry - DATA_RING_BUFFER_OFFSET;
+ } else {
+ m_bytes_allocated = m_first_free_entry - m_first_valid_entry;
+ }
+}
+
+// For SSD we don't calc m_bytes_allocated in this
+template <typename I>
+void WriteLog<I>::inc_allocated_cached_bytes(
+ std::shared_ptr<pwl::GenericLogEntry> log_entry) {
+ if (log_entry->is_write_entry()) {
+ this->m_bytes_cached += log_entry->write_bytes();
+ }
+}
+
+template <typename I>
+bool WriteLog<I>::alloc_resources(C_BlockIORequestT *req) {
+ bool alloc_succeeds = true;
+ uint64_t bytes_allocated = 0;
+ uint64_t bytes_cached = 0;
+ uint64_t bytes_dirtied = 0;
+ uint64_t num_lanes = 0;
+ uint64_t num_unpublished_reserves = 0;
+ uint64_t num_log_entries = 0;
+
+ // Setup buffer, and get all the number of required resources
+ req->setup_buffer_resources(&bytes_cached, &bytes_dirtied, &bytes_allocated,
+ &num_lanes, &num_log_entries,
+ &num_unpublished_reserves);
+
+ ceph_assert(!num_lanes);
+ if (num_log_entries) {
+ bytes_allocated += num_log_entries * MIN_WRITE_ALLOC_SSD_SIZE;
+ num_log_entries = 0;
+ }
+ ceph_assert(!num_unpublished_reserves);
+
+ alloc_succeeds = this->check_allocation(req, bytes_cached, bytes_dirtied,
+ bytes_allocated, num_lanes,
+ num_log_entries,
+ num_unpublished_reserves);
+ req->set_allocated(alloc_succeeds);
+ return alloc_succeeds;
+}
+
+template <typename I>
+bool WriteLog<I>::has_sync_point_logs(GenericLogOperations &ops) {
+ for (auto &op : ops) {
+ if (op->get_log_entry()->is_sync_point()) {
+ return true;
+ break;
+ }
+ }
+ return false;
+}
+
+template<typename I>
+void WriteLog<I>::enlist_op_appender() {
+ this->m_async_append_ops++;
+ this->m_async_op_tracker.start_op();
+ Context *append_ctx = new LambdaContext([this](int r) {
+ append_scheduled_ops();
+ });
+ this->m_work_queue.queue(append_ctx);
+}
+/*
+ * Takes custody of ops. They'll all get their log entries appended,
+ * and have their on_write_persist contexts completed once they and
+ * all prior log entries are persisted everywhere.
+ */
+template<typename I>
+void WriteLog<I>::schedule_append_ops(GenericLogOperations &ops, C_BlockIORequestT *req) {
+ bool need_finisher = false;
+ GenericLogOperationsVector appending;
+
+ std::copy(std::begin(ops), std::end(ops), std::back_inserter(appending));
+ {
+ std::lock_guard locker(m_lock);
+
+ bool persist_on_flush = this->get_persist_on_flush();
+ need_finisher = !this->m_appending &&
+ ((this->m_ops_to_append.size() >= CONTROL_BLOCK_MAX_LOG_ENTRIES) ||
+ !persist_on_flush);
+
+ // Only flush logs into SSD when there is internal/external flush request
+ if (!need_finisher) {
+ need_finisher = has_sync_point_logs(ops);
+ }
+ this->m_ops_to_append.splice(this->m_ops_to_append.end(), ops);
+
+ // To preserve the order of overlapping IOs, release_cell() may be
+ // called only after the ops are added to m_ops_to_append.
+ // As soon as m_lock is released, the appended ops can be picked up
+ // by append_scheduled_ops() in another thread and req can be freed.
+ if (req != nullptr) {
+ if (persist_on_flush) {
+ req->complete_user_request(0);
+ }
+ req->release_cell();
+ }
+ }
+
+ if (need_finisher) {
+ this->enlist_op_appender();
+ }
+
+ for (auto &op : appending) {
+ op->appending();
+ }
+}
+
+template <typename I>
+void WriteLog<I>::setup_schedule_append(pwl::GenericLogOperationsVector &ops,
+ bool do_early_flush,
+ C_BlockIORequestT *req) {
+ this->schedule_append(ops, req);
+}
+
+template <typename I>
+void WriteLog<I>::append_scheduled_ops(void) {
+ GenericLogOperations ops;
+ ldout(m_image_ctx.cct, 20) << dendl;
+
+ bool ops_remain = false; //no-op variable for SSD
+ bool appending = false; //no-op variable for SSD
+ this->append_scheduled(ops, ops_remain, appending);
+
+ if (ops.size()) {
+ alloc_op_log_entries(ops);
+ append_op_log_entries(ops);
+ } else {
+ this->m_async_append_ops--;
+ this->m_async_op_tracker.finish_op();
+ }
+}
+
+/*
+ * Write and persist the (already allocated) write log entries and
+ * data buffer allocations for a set of ops. The data buffer for each
+ * of these must already have been persisted to its reserved area.
+ */
+template <typename I>
+void WriteLog<I>::append_op_log_entries(GenericLogOperations &ops) {
+ ceph_assert(!ops.empty());
+ ldout(m_image_ctx.cct, 20) << dendl;
+ Context *ctx = new LambdaContext([this, ops](int r) {
+ assert(r == 0);
+ ldout(m_image_ctx.cct, 20) << "Finished root update " << dendl;
+
+ auto captured_ops = std::move(ops);
+ this->complete_op_log_entries(std::move(captured_ops), r);
+
+ bool need_finisher = false;
+ {
+ std::lock_guard locker1(m_lock);
+ bool persist_on_flush = this->get_persist_on_flush();
+ need_finisher = ((this->m_ops_to_append.size() >= CONTROL_BLOCK_MAX_LOG_ENTRIES) ||
+ !persist_on_flush);
+
+ if (!need_finisher) {
+ need_finisher = has_sync_point_logs(this->m_ops_to_append);
+ }
+ }
+
+ if (need_finisher) {
+ this->enlist_op_appender();
+ }
+ this->m_async_update_superblock--;
+ this->m_async_op_tracker.finish_op();
+ });
+ uint64_t *new_first_free_entry = new(uint64_t);
+ Context *append_ctx = new LambdaContext(
+ [this, new_first_free_entry, ops, ctx](int r) {
+ std::shared_ptr<WriteLogPoolRoot> new_root;
+ {
+ ldout(m_image_ctx.cct, 20) << "Finished appending at "
+ << *new_first_free_entry << dendl;
+ utime_t now = ceph_clock_now();
+ for (auto &operation : ops) {
+ operation->log_append_comp_time = now;
+ }
+
+ std::lock_guard locker(this->m_log_append_lock);
+ std::lock_guard locker1(m_lock);
+ assert(this->m_appending);
+ this->m_appending = false;
+ new_root = std::make_shared<WriteLogPoolRoot>(pool_root);
+ pool_root.first_free_entry = *new_first_free_entry;
+ new_root->first_free_entry = *new_first_free_entry;
+ delete new_first_free_entry;
+ schedule_update_root(new_root, ctx);
+ }
+ this->m_async_append_ops--;
+ this->m_async_op_tracker.finish_op();
+ });
+ // Append logs and update first_free_update
+ append_ops(ops, append_ctx, new_first_free_entry);
+
+ if (ops.size()) {
+ this->dispatch_deferred_writes();
+ }
+}
+
+template <typename I>
+void WriteLog<I>::release_ram(std::shared_ptr<GenericLogEntry> log_entry) {
+ log_entry->remove_cache_bl();
+}
+
+template <typename I>
+void WriteLog<I>::alloc_op_log_entries(GenericLogOperations &ops) {
+ std::unique_lock locker(m_lock);
+
+ for (auto &operation : ops) {
+ auto &log_entry = operation->get_log_entry();
+ log_entry->ram_entry.set_entry_valid(true);
+ m_log_entries.push_back(log_entry);
+ ldout(m_image_ctx.cct, 20) << "operation=[" << *operation << "]" << dendl;
+ }
+ if (m_cache_state->empty && !m_log_entries.empty()) {
+ m_cache_state->empty = false;
+ this->update_image_cache_state();
+ this->write_image_cache_state(locker);
+ }
+}
+
+template <typename I>
+void WriteLog<I>::construct_flush_entries(pwl::GenericLogEntries entries_to_flush,
+ DeferredContexts &post_unlock,
+ bool has_write_entry) {
+ // snapshot so we behave consistently
+ bool invalidating = this->m_invalidating;
+
+ if (invalidating || !has_write_entry) {
+ for (auto &log_entry : entries_to_flush) {
+ GuardedRequestFunctionContext *guarded_ctx =
+ new GuardedRequestFunctionContext([this, log_entry, invalidating]
+ (GuardedRequestFunctionContext &guard_ctx) {
+ log_entry->m_cell = guard_ctx.cell;
+ Context *ctx = this->construct_flush_entry(log_entry, invalidating);
+
+ if (!invalidating) {
+ ctx = new LambdaContext([this, log_entry, ctx](int r) {
+ m_image_ctx.op_work_queue->queue(new LambdaContext(
+ [this, log_entry, ctx](int r) {
+ ldout(m_image_ctx.cct, 15) << "flushing:" << log_entry
+ << " " << *log_entry << dendl;
+ log_entry->writeback(this->m_image_writeback, ctx);
+ }), 0);
+ });
+ }
+ ctx->complete(0);
+ });
+ this->detain_flush_guard_request(log_entry, guarded_ctx);
+ }
+ } else {
+ int count = entries_to_flush.size();
+ std::vector<std::shared_ptr<GenericWriteLogEntry>> write_entries;
+ std::vector<bufferlist *> read_bls;
+
+ write_entries.reserve(count);
+ read_bls.reserve(count);
+
+ for (auto &log_entry : entries_to_flush) {
+ if (log_entry->is_write_entry()) {
+ bufferlist *bl = new bufferlist;
+ auto write_entry = static_pointer_cast<WriteLogEntry>(log_entry);
+ write_entry->inc_bl_refs();
+ write_entries.push_back(write_entry);
+ read_bls.push_back(bl);
+ }
+ }
+
+ Context *ctx = new LambdaContext(
+ [this, entries_to_flush, read_bls](int r) {
+ int i = 0;
+ GuardedRequestFunctionContext *guarded_ctx = nullptr;
+
+ for (auto &log_entry : entries_to_flush) {
+ if (log_entry->is_write_entry()) {
+ bufferlist captured_entry_bl;
+ captured_entry_bl.claim_append(*read_bls[i]);
+ delete read_bls[i++];
+
+ guarded_ctx = new GuardedRequestFunctionContext([this, log_entry, captured_entry_bl]
+ (GuardedRequestFunctionContext &guard_ctx) {
+ log_entry->m_cell = guard_ctx.cell;
+ Context *ctx = this->construct_flush_entry(log_entry, false);
+
+ m_image_ctx.op_work_queue->queue(new LambdaContext(
+ [this, log_entry, entry_bl=std::move(captured_entry_bl), ctx](int r) {
+ auto captured_entry_bl = std::move(entry_bl);
+ ldout(m_image_ctx.cct, 15) << "flushing:" << log_entry
+ << " " << *log_entry << dendl;
+ log_entry->writeback_bl(this->m_image_writeback, ctx,
+ std::move(captured_entry_bl));
+ }), 0);
+ });
+ } else {
+ guarded_ctx = new GuardedRequestFunctionContext([this, log_entry]
+ (GuardedRequestFunctionContext &guard_ctx) {
+ log_entry->m_cell = guard_ctx.cell;
+ Context *ctx = this->construct_flush_entry(log_entry, false);
+ m_image_ctx.op_work_queue->queue(new LambdaContext(
+ [this, log_entry, ctx](int r) {
+ ldout(m_image_ctx.cct, 15) << "flushing:" << log_entry
+ << " " << *log_entry << dendl;
+ log_entry->writeback(this->m_image_writeback, ctx);
+ }), 0);
+ });
+ }
+ this->detain_flush_guard_request(log_entry, guarded_ctx);
+ }
+ });
+
+ aio_read_data_blocks(write_entries, read_bls, ctx);
+ }
+}
+
+template <typename I>
+void WriteLog<I>::process_work() {
+ CephContext *cct = m_image_ctx.cct;
+ int max_iterations = 4;
+ bool wake_up_requested = false;
+ uint64_t aggressive_high_water_bytes =
+ this->m_bytes_allocated_cap * AGGRESSIVE_RETIRE_HIGH_WATER;
+ uint64_t high_water_bytes = this->m_bytes_allocated_cap * RETIRE_HIGH_WATER;
+
+ ldout(cct, 20) << dendl;
+
+ do {
+ {
+ std::lock_guard locker(m_lock);
+ this->m_wake_up_requested = false;
+ }
+ if (this->m_alloc_failed_since_retire || (this->m_shutting_down) ||
+ this->m_invalidating || m_bytes_allocated > high_water_bytes) {
+ ldout(m_image_ctx.cct, 10) << "alloc_fail=" << this->m_alloc_failed_since_retire
+ << ", allocated > high_water="
+ << (m_bytes_allocated > high_water_bytes)
+ << dendl;
+ retire_entries((this->m_shutting_down || this->m_invalidating ||
+ m_bytes_allocated > aggressive_high_water_bytes)
+ ? MAX_ALLOC_PER_TRANSACTION : MAX_FREE_PER_TRANSACTION);
+ }
+ this->dispatch_deferred_writes();
+ this->process_writeback_dirty_entries();
+ {
+ std::lock_guard locker(m_lock);
+ wake_up_requested = this->m_wake_up_requested;
+ }
+ } while (wake_up_requested && --max_iterations > 0);
+
+ {
+ std::lock_guard locker(m_lock);
+ this->m_wake_up_scheduled = false;
+ // Reschedule if it's still requested
+ if (this->m_wake_up_requested) {
+ this->wake_up();
+ }
+ }
+}
+
+/**
+ * Retire up to MAX_ALLOC_PER_TRANSACTION of the oldest log entries
+ * that are eligible to be retired. Returns true if anything was
+ * retired.
+ *
+*/
+template <typename I>
+bool WriteLog<I>::retire_entries(const unsigned long int frees_per_tx) {
+ CephContext *cct = m_image_ctx.cct;
+ GenericLogEntriesVector retiring_entries;
+ uint64_t initial_first_valid_entry;
+ uint64_t first_valid_entry;
+
+ std::lock_guard retire_locker(this->m_log_retire_lock);
+ ldout(cct, 20) << "Look for entries to retire" << dendl;
+ {
+ // Entry readers can't be added while we hold m_entry_reader_lock
+ RWLock::WLocker entry_reader_locker(this->m_entry_reader_lock);
+ std::lock_guard locker(m_lock);
+ initial_first_valid_entry = m_first_valid_entry;
+ first_valid_entry = m_first_valid_entry;
+ while (retiring_entries.size() < frees_per_tx && !m_log_entries.empty()) {
+ GenericLogEntriesVector retiring_subentries;
+ uint64_t control_block_pos = m_log_entries.front()->log_entry_index;
+ uint64_t data_length = 0;
+ for (auto it = m_log_entries.begin(); it != m_log_entries.end(); ++it) {
+ if (this->can_retire_entry(*it)) {
+ // log_entry_index is valid after appending to SSD
+ if ((*it)->log_entry_index != control_block_pos) {
+ ldout(cct, 20) << "Old log_entry_index is " << control_block_pos
+ << ",New log_entry_index is "
+ << (*it)->log_entry_index
+ << ",data length is " << data_length << dendl;
+ ldout(cct, 20) << "The log entry is " << *(*it) << dendl;
+ if ((*it)->log_entry_index < control_block_pos) {
+ ceph_assert((*it)->log_entry_index ==
+ (control_block_pos + data_length + MIN_WRITE_ALLOC_SSD_SIZE) %
+ this->m_log_pool_size + DATA_RING_BUFFER_OFFSET);
+ } else {
+ ceph_assert((*it)->log_entry_index == control_block_pos +
+ data_length + MIN_WRITE_ALLOC_SSD_SIZE);
+ }
+ break;
+ } else {
+ retiring_subentries.push_back(*it);
+ if ((*it)->is_write_entry()) {
+ data_length += (*it)->get_aligned_data_size();
+ }
+ }
+ } else {
+ retiring_subentries.clear();
+ break;
+ }
+ }
+ // SSD: retiring_subentries in a span
+ if (!retiring_subentries.empty()) {
+ for (auto it = retiring_subentries.begin();
+ it != retiring_subentries.end(); it++) {
+ ceph_assert(m_log_entries.front() == *it);
+ m_log_entries.pop_front();
+ if ((*it)->write_bytes() > 0 || (*it)->bytes_dirty() > 0) {
+ auto gen_write_entry = static_pointer_cast<GenericWriteLogEntry>(*it);
+ if (gen_write_entry) {
+ this->m_blocks_to_log_entries.remove_log_entry(gen_write_entry);
+ }
+ }
+ }
+
+ ldout(cct, 20) << "span with " << retiring_subentries.size()
+ << " entries: control_block_pos=" << control_block_pos
+ << " data_length=" << data_length
+ << dendl;
+ retiring_entries.insert(
+ retiring_entries.end(), retiring_subentries.begin(),
+ retiring_subentries.end());
+
+ first_valid_entry = control_block_pos + data_length +
+ MIN_WRITE_ALLOC_SSD_SIZE;
+ if (first_valid_entry >= this->m_log_pool_size) {
+ first_valid_entry = first_valid_entry % this->m_log_pool_size +
+ DATA_RING_BUFFER_OFFSET;
+ }
+ } else {
+ break;
+ }
+ }
+ }
+ if (retiring_entries.size()) {
+ ldout(cct, 20) << "Retiring " << retiring_entries.size() << " entries"
+ << dendl;
+
+ // Advance first valid entry and release buffers
+ uint64_t flushed_sync_gen;
+ std::lock_guard append_locker(this->m_log_append_lock);
+ {
+ std::lock_guard locker(m_lock);
+ flushed_sync_gen = this->m_flushed_sync_gen;
+ }
+
+ ceph_assert(first_valid_entry != initial_first_valid_entry);
+ auto new_root = std::make_shared<WriteLogPoolRoot>(pool_root);
+ new_root->flushed_sync_gen = flushed_sync_gen;
+ new_root->first_valid_entry = first_valid_entry;
+ pool_root.flushed_sync_gen = flushed_sync_gen;
+ pool_root.first_valid_entry = first_valid_entry;
+
+ Context *ctx = new LambdaContext(
+ [this, first_valid_entry, initial_first_valid_entry,
+ retiring_entries](int r) {
+ uint64_t allocated_bytes = 0;
+ uint64_t cached_bytes = 0;
+ uint64_t former_log_pos = 0;
+ for (auto &entry : retiring_entries) {
+ ceph_assert(entry->log_entry_index != 0);
+ if (entry->log_entry_index != former_log_pos ) {
+ // Space for control blocks
+ allocated_bytes += MIN_WRITE_ALLOC_SSD_SIZE;
+ former_log_pos = entry->log_entry_index;
+ }
+ if (entry->is_write_entry()) {
+ cached_bytes += entry->write_bytes();
+ // space for userdata
+ allocated_bytes += entry->get_aligned_data_size();
+ }
+ }
+ bool need_update_state = false;
+ {
+ std::lock_guard locker(m_lock);
+ m_first_valid_entry = first_valid_entry;
+ ceph_assert(m_first_valid_entry % MIN_WRITE_ALLOC_SSD_SIZE == 0);
+ ceph_assert(this->m_bytes_allocated >= allocated_bytes);
+ this->m_bytes_allocated -= allocated_bytes;
+ ceph_assert(this->m_bytes_cached >= cached_bytes);
+ this->m_bytes_cached -= cached_bytes;
+ if (!m_cache_state->empty && m_log_entries.empty()) {
+ m_cache_state->empty = true;
+ this->update_image_cache_state();
+ need_update_state = true;
+ }
+
+ ldout(m_image_ctx.cct, 20)
+ << "Finished root update: " << "initial_first_valid_entry="
+ << initial_first_valid_entry << ", " << "m_first_valid_entry="
+ << m_first_valid_entry << "," << "release space = "
+ << allocated_bytes << "," << "m_bytes_allocated="
+ << m_bytes_allocated << "," << "release cached space="
+ << cached_bytes << "," << "m_bytes_cached="
+ << this->m_bytes_cached << dendl;
+
+ this->m_alloc_failed_since_retire = false;
+ this->wake_up();
+ }
+ if (need_update_state) {
+ std::unique_lock locker(m_lock);
+ this->write_image_cache_state(locker);
+ }
+
+ this->dispatch_deferred_writes();
+ this->process_writeback_dirty_entries();
+ m_async_update_superblock--;
+ this->m_async_op_tracker.finish_op();
+ });
+
+ std::lock_guard locker(m_lock);
+ schedule_update_root(new_root, ctx);
+ } else {
+ ldout(cct, 20) << "Nothing to retire" << dendl;
+ return false;
+ }
+ return true;
+}
+
+template <typename I>
+void WriteLog<I>::append_ops(GenericLogOperations &ops, Context *ctx,
+ uint64_t* new_first_free_entry) {
+ GenericLogEntriesVector log_entries;
+ CephContext *cct = m_image_ctx.cct;
+ uint64_t span_payload_len = 0;
+ uint64_t bytes_to_free = 0;
+ ldout(cct, 20) << "Appending " << ops.size() << " log entries." << dendl;
+
+ *new_first_free_entry = pool_root.first_free_entry;
+ AioTransContext* aio = new AioTransContext(cct, ctx);
+
+ utime_t now = ceph_clock_now();
+ for (auto &operation : ops) {
+ operation->log_append_start_time = now;
+ auto log_entry = operation->get_log_entry();
+
+ if (log_entries.size() == CONTROL_BLOCK_MAX_LOG_ENTRIES ||
+ span_payload_len >= SPAN_MAX_DATA_LEN) {
+ if (log_entries.size() > 1) {
+ bytes_to_free += (log_entries.size() - 1) * MIN_WRITE_ALLOC_SSD_SIZE;
+ }
+ write_log_entries(log_entries, aio, new_first_free_entry);
+ log_entries.clear();
+ span_payload_len = 0;
+ }
+ log_entries.push_back(log_entry);
+ span_payload_len += log_entry->write_bytes();
+ }
+ if (!span_payload_len || !log_entries.empty()) {
+ if (log_entries.size() > 1) {
+ bytes_to_free += (log_entries.size() - 1) * MIN_WRITE_ALLOC_SSD_SIZE;
+ }
+ write_log_entries(log_entries, aio, new_first_free_entry);
+ }
+
+ {
+ std::lock_guard locker1(m_lock);
+ m_first_free_entry = *new_first_free_entry;
+ m_bytes_allocated -= bytes_to_free;
+ }
+
+ bdev->aio_submit(&aio->ioc);
+}
+
+template <typename I>
+void WriteLog<I>::write_log_entries(GenericLogEntriesVector log_entries,
+ AioTransContext *aio, uint64_t *pos) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(m_image_ctx.cct, 20) << "pos=" << *pos << dendl;
+ ceph_assert(*pos >= DATA_RING_BUFFER_OFFSET &&
+ *pos < this->m_log_pool_size &&
+ *pos % MIN_WRITE_ALLOC_SSD_SIZE == 0);
+
+ // The first block is for log entries
+ uint64_t control_block_pos = *pos;
+ *pos += MIN_WRITE_ALLOC_SSD_SIZE;
+ if (*pos == this->m_log_pool_size) {
+ *pos = DATA_RING_BUFFER_OFFSET;
+ }
+
+ std::vector<WriteLogCacheEntry> persist_log_entries;
+ bufferlist data_bl;
+ for (auto &log_entry : log_entries) {
+ log_entry->log_entry_index = control_block_pos;
+ // Append data buffer for write operations
+ if (log_entry->is_write_entry()) {
+ auto write_entry = static_pointer_cast<WriteLogEntry>(log_entry);
+ auto cache_bl = write_entry->get_cache_bl();
+ auto align_size = write_entry->get_aligned_data_size();
+ data_bl.append(cache_bl);
+ data_bl.append_zero(align_size - cache_bl.length());
+
+ write_entry->ram_entry.write_data_pos = *pos;
+ *pos += align_size;
+ if (*pos >= this->m_log_pool_size) {
+ *pos = *pos % this->m_log_pool_size + DATA_RING_BUFFER_OFFSET;
+ }
+ }
+ // push_back _after_ setting write_data_pos
+ persist_log_entries.push_back(log_entry->ram_entry);
+ }
+
+ //aio write
+ bufferlist bl;
+ encode(persist_log_entries, bl);
+ ceph_assert(bl.length() <= MIN_WRITE_ALLOC_SSD_SIZE);
+ bl.append_zero(MIN_WRITE_ALLOC_SSD_SIZE - bl.length());
+ bl.append(data_bl);
+ ceph_assert(bl.length() % MIN_WRITE_ALLOC_SSD_SIZE == 0);
+ if (control_block_pos + bl.length() > this->m_log_pool_size) {
+ //exceeds border, need to split
+ uint64_t size = bl.length();
+ bufferlist bl1;
+ bl.splice(0, this->m_log_pool_size - control_block_pos, &bl1);
+ ceph_assert(bl.length() == (size - bl1.length()));
+
+ ldout(cct, 20) << "write " << control_block_pos << "~"
+ << size << " spans boundary, split into "
+ << control_block_pos << "~" << bl1.length()
+ << " and " << DATA_RING_BUFFER_OFFSET << "~"
+ << bl.length() << dendl;
+ bdev->aio_write(control_block_pos, bl1, &aio->ioc, false,
+ WRITE_LIFE_NOT_SET);
+ bdev->aio_write(DATA_RING_BUFFER_OFFSET, bl, &aio->ioc, false,
+ WRITE_LIFE_NOT_SET);
+ } else {
+ ldout(cct, 20) << "write " << control_block_pos << "~"
+ << bl.length() << dendl;
+ bdev->aio_write(control_block_pos, bl, &aio->ioc, false,
+ WRITE_LIFE_NOT_SET);
+ }
+}
+
+template <typename I>
+void WriteLog<I>::schedule_update_root(
+ std::shared_ptr<WriteLogPoolRoot> root, Context *ctx) {
+ CephContext *cct = m_image_ctx.cct;
+ ldout(cct, 15) << "New root: pool_size=" << root->pool_size
+ << " first_valid_entry=" << root->first_valid_entry
+ << " first_free_entry=" << root->first_free_entry
+ << " flushed_sync_gen=" << root->flushed_sync_gen
+ << dendl;
+ ceph_assert(is_valid_pool_root(*root));
+
+ bool need_finisher;
+ {
+ ceph_assert(ceph_mutex_is_locked_by_me(m_lock));
+ need_finisher = m_poolroot_to_update.empty() && !m_updating_pool_root;
+ std::shared_ptr<WriteLogPoolRootUpdate> entry =
+ std::make_shared<WriteLogPoolRootUpdate>(root, ctx);
+ this->m_async_update_superblock++;
+ this->m_async_op_tracker.start_op();
+ m_poolroot_to_update.emplace_back(entry);
+ }
+ if (need_finisher) {
+ enlist_op_update_root();
+ }
+}
+
+template <typename I>
+void WriteLog<I>::enlist_op_update_root() {
+ Context *append_ctx = new LambdaContext([this](int r) {
+ update_root_scheduled_ops();
+ });
+ this->m_work_queue.queue(append_ctx);
+}
+
+template <typename I>
+void WriteLog<I>::update_root_scheduled_ops() {
+ ldout(m_image_ctx.cct, 20) << dendl;
+
+ std::shared_ptr<WriteLogPoolRoot> root;
+ WriteLogPoolRootUpdateList root_updates;
+ Context *ctx = nullptr;
+ {
+ std::lock_guard locker(m_lock);
+ if (m_updating_pool_root) {
+ /* Another thread is appending */
+ ldout(m_image_ctx.cct, 15) << "Another thread is updating pool root"
+ << dendl;
+ return;
+ }
+ if (m_poolroot_to_update.size()) {
+ m_updating_pool_root = true;
+ root_updates.swap(m_poolroot_to_update);
+ }
+ }
+ ceph_assert(!root_updates.empty());
+ ldout(m_image_ctx.cct, 15) << "Update root number: " << root_updates.size()
+ << dendl;
+ // We just update the last one, and call all the completions.
+ auto entry = root_updates.back();
+ root = entry->root;
+
+ ctx = new LambdaContext([this, updates = std::move(root_updates)](int r) {
+ ldout(m_image_ctx.cct, 15) << "Start to callback." << dendl;
+ for (auto it = updates.begin(); it != updates.end(); it++) {
+ Context *it_ctx = (*it)->ctx;
+ it_ctx->complete(r);
+ }
+ });
+ Context *append_ctx = new LambdaContext([this, ctx](int r) {
+ ldout(m_image_ctx.cct, 15) << "Finish the update of pool root." << dendl;
+ bool need_finisher = false;;
+ assert(r == 0);
+ {
+ std::lock_guard locker(m_lock);
+ m_updating_pool_root = false;
+ need_finisher = !m_poolroot_to_update.empty();
+ }
+ if (need_finisher) {
+ enlist_op_update_root();
+ }
+ ctx->complete(r);
+ });
+ AioTransContext* aio = new AioTransContext(m_image_ctx.cct, append_ctx);
+ update_pool_root(root, aio);
+}
+
+template <typename I>
+void WriteLog<I>::update_pool_root(std::shared_ptr<WriteLogPoolRoot> root,
+ AioTransContext *aio) {
+ bufferlist bl;
+ SuperBlock superblock;
+ superblock.root = *root;
+ encode(superblock, bl);
+ bl.append_zero(MIN_WRITE_ALLOC_SSD_SIZE - bl.length());
+ ceph_assert(bl.length() % MIN_WRITE_ALLOC_SSD_SIZE == 0);
+ bdev->aio_write(0, bl, &aio->ioc, false, WRITE_LIFE_NOT_SET);
+ bdev->aio_submit(&aio->ioc);
+}
+
+template <typename I>
+int WriteLog<I>::update_pool_root_sync(
+ std::shared_ptr<WriteLogPoolRoot> root) {
+ bufferlist bl;
+ SuperBlock superblock;
+ superblock.root = *root;
+ encode(superblock, bl);
+ bl.append_zero(MIN_WRITE_ALLOC_SSD_SIZE - bl.length());
+ ceph_assert(bl.length() % MIN_WRITE_ALLOC_SSD_SIZE == 0);
+ return bdev->write(0, bl, false);
+}
+
+template <typename I>
+void WriteLog<I>::aio_read_data_block(std::shared_ptr<GenericWriteLogEntry> log_entry,
+ bufferlist *bl, Context *ctx) {
+ std::vector<std::shared_ptr<GenericWriteLogEntry>> log_entries = {std::move(log_entry)};
+ std::vector<bufferlist *> bls {bl};
+ aio_read_data_blocks(log_entries, bls, ctx);
+}
+
+template <typename I>
+void WriteLog<I>::aio_read_data_blocks(
+ std::vector<std::shared_ptr<GenericWriteLogEntry>> &log_entries,
+ std::vector<bufferlist *> &bls, Context *ctx) {
+ ceph_assert(log_entries.size() == bls.size());
+
+ //get the valid part
+ Context *read_ctx = new LambdaContext(
+ [log_entries, bls, ctx](int r) {
+ for (unsigned int i = 0; i < log_entries.size(); i++) {
+ bufferlist valid_data_bl;
+ auto write_entry = static_pointer_cast<WriteLogEntry>(log_entries[i]);
+ auto length = write_entry->ram_entry.is_write() ? write_entry->ram_entry.write_bytes
+ : write_entry->ram_entry.ws_datalen;
+
+ valid_data_bl.substr_of(*bls[i], 0, length);
+ bls[i]->clear();
+ bls[i]->append(valid_data_bl);
+ write_entry->dec_bl_refs();
+ }
+ ctx->complete(r);
+ });
+
+ CephContext *cct = m_image_ctx.cct;
+ AioTransContext *aio = new AioTransContext(cct, read_ctx);
+ for (unsigned int i = 0; i < log_entries.size(); i++) {
+ WriteLogCacheEntry *log_entry = &log_entries[i]->ram_entry;
+
+ ceph_assert(log_entry->is_write() || log_entry->is_writesame());
+ uint64_t len = log_entry->is_write() ? log_entry->write_bytes :
+ log_entry->ws_datalen;
+ uint64_t align_len = round_up_to(len, MIN_WRITE_ALLOC_SSD_SIZE);
+
+ ldout(cct, 20) << "entry i=" << i << " " << log_entry->write_data_pos
+ << "~" << len << dendl;
+ ceph_assert(log_entry->write_data_pos >= DATA_RING_BUFFER_OFFSET &&
+ log_entry->write_data_pos < pool_root.pool_size);
+ ceph_assert(align_len);
+ if (log_entry->write_data_pos + align_len > pool_root.pool_size) {
+ // spans boundary, need to split
+ uint64_t len1 = pool_root.pool_size - log_entry->write_data_pos;
+ uint64_t len2 = align_len - len1;
+
+ ldout(cct, 20) << "read " << log_entry->write_data_pos << "~"
+ << align_len << " spans boundary, split into "
+ << log_entry->write_data_pos << "~" << len1
+ << " and " << DATA_RING_BUFFER_OFFSET << "~"
+ << len2 << dendl;
+ bdev->aio_read(log_entry->write_data_pos, len1, bls[i], &aio->ioc);
+ bdev->aio_read(DATA_RING_BUFFER_OFFSET, len2, bls[i], &aio->ioc);
+ } else {
+ ldout(cct, 20) << "read " << log_entry->write_data_pos << "~"
+ << align_len << dendl;
+ bdev->aio_read(log_entry->write_data_pos, align_len, bls[i], &aio->ioc);
+ }
+ }
+ bdev->aio_submit(&aio->ioc);
+}
+
+template <typename I>
+void WriteLog<I>::complete_user_request(Context *&user_req, int r) {
+ m_image_ctx.op_work_queue->queue(user_req, r);
+}
+
+} // namespace ssd
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+template class librbd::cache::pwl::ssd::WriteLog<librbd::ImageCtx>;
diff --git a/src/librbd/cache/pwl/ssd/WriteLog.h b/src/librbd/cache/pwl/ssd/WriteLog.h
new file mode 100644
index 000000000..69cc36662
--- /dev/null
+++ b/src/librbd/cache/pwl/ssd/WriteLog.h
@@ -0,0 +1,156 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_LIBRBD_CACHE_PWL_SSD_WRITE_LOG
+#define CEPH_LIBRBD_CACHE_PWL_SSD_WRITE_LOG
+
+#include "blk/BlockDevice.h"
+#include "common/AsyncOpTracker.h"
+#include "common/Checksummer.h"
+#include "common/environment.h"
+#include "common/RWLock.h"
+#include "common/WorkQueue.h"
+#include "librbd/BlockGuard.h"
+#include "librbd/Utils.h"
+#include "librbd/cache/ImageWriteback.h"
+#include "librbd/cache/Types.h"
+#include "librbd/cache/pwl/AbstractWriteLog.h"
+#include "librbd/cache/pwl/LogMap.h"
+#include "librbd/cache/pwl/LogOperation.h"
+#include "librbd/cache/pwl/Request.h"
+#include "librbd/cache/pwl/ssd/Builder.h"
+#include "librbd/cache/pwl/ssd/Types.h"
+#include <functional>
+#include <list>
+
+namespace librbd {
+
+struct ImageCtx;
+
+namespace cache {
+namespace pwl {
+namespace ssd {
+
+template <typename ImageCtxT>
+class WriteLog : public AbstractWriteLog<ImageCtxT> {
+public:
+ WriteLog(ImageCtxT &image_ctx,
+ librbd::cache::pwl::ImageCacheState<ImageCtxT>* cache_state,
+ cache::ImageWritebackInterface& image_writeback,
+ plugin::Api<ImageCtxT>& plugin_api);
+ ~WriteLog();
+ WriteLog(const WriteLog&) = delete;
+ WriteLog &operator=(const WriteLog&) = delete;
+
+ typedef io::Extent Extent;
+ using This = AbstractWriteLog<ImageCtxT>;
+ using C_BlockIORequestT = pwl::C_BlockIORequest<This>;
+ using C_WriteRequestT = pwl::C_WriteRequest<This>;
+ using C_WriteSameRequestT = pwl::C_WriteSameRequest<This>;
+
+ bool alloc_resources(C_BlockIORequestT *req) override;
+ void setup_schedule_append(
+ pwl::GenericLogOperationsVector &ops, bool do_early_flush,
+ C_BlockIORequestT *req) override;
+ void complete_user_request(Context *&user_req, int r) override;
+
+protected:
+ using AbstractWriteLog<ImageCtxT>::m_lock;
+ using AbstractWriteLog<ImageCtxT>::m_log_entries;
+ using AbstractWriteLog<ImageCtxT>::m_image_ctx;
+ using AbstractWriteLog<ImageCtxT>::m_cache_state;
+ using AbstractWriteLog<ImageCtxT>::m_first_free_entry;
+ using AbstractWriteLog<ImageCtxT>::m_first_valid_entry;
+ using AbstractWriteLog<ImageCtxT>::m_bytes_allocated;
+
+ bool initialize_pool(Context *on_finish,
+ pwl::DeferredContexts &later) override;
+ void process_work() override;
+ void append_scheduled_ops(void) override;
+ void schedule_append_ops(pwl::GenericLogOperations &ops, C_BlockIORequestT *req) override;
+ void remove_pool_file() override;
+ void release_ram(std::shared_ptr<GenericLogEntry> log_entry) override;
+
+private:
+ class AioTransContext {
+ public:
+ Context *on_finish;
+ ::IOContext ioc;
+ explicit AioTransContext(CephContext* cct, Context *cb)
+ : on_finish(cb), ioc(cct, this) {}
+
+ ~AioTransContext(){}
+
+ void aio_finish() {
+ on_finish->complete(ioc.get_return_value());
+ delete this;
+ }
+ }; //class AioTransContext
+
+ struct WriteLogPoolRootUpdate {
+ std::shared_ptr<pwl::WriteLogPoolRoot> root;
+ Context *ctx;
+ WriteLogPoolRootUpdate(std::shared_ptr<pwl::WriteLogPoolRoot> r,
+ Context* c)
+ : root(r), ctx(c) {}
+ };
+
+ using WriteLogPoolRootUpdateList = std::list<std::shared_ptr<WriteLogPoolRootUpdate>>;
+ WriteLogPoolRootUpdateList m_poolroot_to_update; /* pool root list to update to SSD */
+ bool m_updating_pool_root = false;
+
+ std::atomic<int> m_async_update_superblock = {0};
+ BlockDevice *bdev = nullptr;
+ pwl::WriteLogPoolRoot pool_root;
+ Builder<This> *m_builderobj;
+
+ Builder<This>* create_builder();
+ int create_and_open_bdev();
+ void load_existing_entries(pwl::DeferredContexts &later);
+ void inc_allocated_cached_bytes(
+ std::shared_ptr<pwl::GenericLogEntry> log_entry) override;
+ void collect_read_extents(
+ uint64_t read_buffer_offset, LogMapEntry<GenericWriteLogEntry> map_entry,
+ std::vector<std::shared_ptr<GenericWriteLogEntry>> &log_entries_to_read,
+ std::vector<bufferlist*> &bls_to_read, uint64_t entry_hit_length,
+ Extent hit_extent, pwl::C_ReadRequest *read_ctx) override;
+ void complete_read(
+ std::vector<std::shared_ptr<GenericWriteLogEntry>> &log_entries_to_read,
+ std::vector<bufferlist*> &bls_to_read, Context *ctx) override;
+ void enlist_op_appender();
+ bool retire_entries(const unsigned long int frees_per_tx);
+ bool has_sync_point_logs(GenericLogOperations &ops);
+ void append_op_log_entries(GenericLogOperations &ops);
+ void alloc_op_log_entries(GenericLogOperations &ops);
+ void construct_flush_entries(pwl::GenericLogEntries entires_to_flush,
+ DeferredContexts &post_unlock,
+ bool has_write_entry) override;
+ void append_ops(GenericLogOperations &ops, Context *ctx,
+ uint64_t* new_first_free_entry);
+ void write_log_entries(GenericLogEntriesVector log_entries,
+ AioTransContext *aio, uint64_t *pos);
+ void schedule_update_root(std::shared_ptr<WriteLogPoolRoot> root,
+ Context *ctx);
+ void enlist_op_update_root();
+ void update_root_scheduled_ops();
+ int update_pool_root_sync(std::shared_ptr<pwl::WriteLogPoolRoot> root);
+ void update_pool_root(std::shared_ptr<WriteLogPoolRoot> root,
+ AioTransContext *aio);
+ void aio_read_data_block(std::shared_ptr<GenericWriteLogEntry> log_entry,
+ bufferlist *bl, Context *ctx);
+ void aio_read_data_blocks(std::vector<std::shared_ptr<GenericWriteLogEntry>> &log_entries,
+ std::vector<bufferlist *> &bls, Context *ctx);
+ static void aio_cache_cb(void *priv, void *priv2) {
+ AioTransContext *c = static_cast<AioTransContext*>(priv2);
+ c->aio_finish();
+ }
+};//class WriteLog
+
+} // namespace ssd
+} // namespace pwl
+} // namespace cache
+} // namespace librbd
+
+extern template class librbd::cache::pwl::ssd::WriteLog<librbd::ImageCtx>;
+
+#endif // CEPH_LIBRBD_CACHE_PWL_SSD_WRITE_LOG