diff options
Diffstat (limited to 'src/librbd/cache/pwl')
47 files changed, 10618 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..1e784f6b5 --- /dev/null +++ b/src/librbd/cache/pwl/AbstractWriteLog.cc @@ -0,0 +1,2187 @@ +// -*- 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 std; +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, false, false); + 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)) { + lderr(cct) << "can't find the existed pool file: " << m_log_pool_name + << ". error: " << cpp_strerror(-errno) << 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 image extents larger than 1M. This isn't strictly necessary but + * makes libpmemobj allocator's job easier and reduces pmemobj_defrag() cost. + * 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 */ + ceph_assert(cw_req->read_bl.length() <= cw_req->cmp_bl.length()); + ceph_assert(cw_req->read_bl.length() == cw_req->image_extents_summary.total_bytes); + bufferlist sub_cmp_bl; + sub_cmp_bl.substr_of(cw_req->cmp_bl, 0, cw_req->read_bl.length()); + if (sub_cmp_bl.contents_equal(cw_req->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 < sub_cmp_bl.length(); bl_index++) { + if (sub_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() << ", remain " + << m_ops_to_append.size() << 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) { + 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) { + 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) { + ldout(m_image_ctx.cct, 20) << "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 (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)); + ceph_assert(log_entry); + 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..1b537f32d --- /dev/null +++ b/src/librbd/cache/pwl/DiscardRequest.cc @@ -0,0 +1,160 @@ +// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- +// vim: ts=8 sw=2 smarttab + +#include <filesystem> + +#include "common/dout.h" +#include "common/errno.h" +#include "common/hostname.h" +#include "librbd/asio/ContextWQ.h" +#include "librbd/cache/pwl/DiscardRequest.h" + +#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 fs = std::filesystem; + +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..ab941df0f --- /dev/null +++ b/src/librbd/cache/pwl/ImageCacheState.cc @@ -0,0 +1,196 @@ +// -*- 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 { + +using namespace std; + +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..8a050eb79 --- /dev/null +++ b/src/librbd/cache/pwl/LogEntry.cc @@ -0,0 +1,135 @@ +// -*- 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 << ", 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..b3e7022b0 --- /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..e779802f0 --- /dev/null +++ b/src/librbd/cache/pwl/LogOperation.cc @@ -0,0 +1,312 @@ +// -*- 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 { + std::string op_name = is_writesame ? "(Write Same) " : "(Write) "; + os << op_name; + GenericWriteLogOperation::format(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); + 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..963331925 --- /dev/null +++ b/src/librbd/cache/pwl/Request.cc @@ -0,0 +1,562 @@ +// -*- 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__ << ": " + +using namespace std; + +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; + 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..4840b049e --- /dev/null +++ b/src/librbd/cache/pwl/Request.h @@ -0,0 +1,361 @@ +// -*- 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 */ + + 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(); + } + + 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; + std::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_end=" << 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..6d45e7a30 --- /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..c29305eec --- /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(std::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(std::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..0d8c93a24 --- /dev/null +++ b/src/librbd/cache/pwl/Types.h @@ -0,0 +1,445 @@ +// -*- 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(std::list<WriteLogCacheEntry*>& ls); +}; + +struct WriteLogPoolRoot { + #ifdef WITH_RBD_RWL + union { + struct { + uint8_t 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; /* TODO: remove it when changing disk format */ + #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; + uint32_t num_log_entries; + uint64_t first_free_entry; /* The free entry following the latest valid + * entry, which is going to be written */ + uint64_t first_valid_entry; /* The oldest valid entry to be retired */ + + #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(std::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..38e09c22a --- /dev/null +++ b/src/librbd/cache/pwl/rwl/LogEntry.cc @@ -0,0 +1,106 @@ +// -*- 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 + ceph_assert(cache_bp.length() == cache_bp.raw_length()); + buffer::ptr cloned_bp = cache_bp.begin_deep().get_ptr(cache_bp.length()); + 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..a6b81d55b --- /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..e922ba543 --- /dev/null +++ b/src/librbd/cache/pwl/rwl/WriteLog.cc @@ -0,0 +1,1011 @@ +// -*- 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 std; +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; + 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; + int r = -EINVAL; + 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 + << ". error: " << 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; + goto err_close_pool; + } + 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 + << ". pmemobj TX errno: " << pmemobj_tx_errno() << dendl; + r = -pmemobj_tx_errno(); + goto err_close_pool; + } 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; + goto err_close_pool; + } + 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; + goto err_close_pool; + } + 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; + +err_close_pool: + pmemobj_close(m_log_pool); + on_finish->complete(r); + return false; +} + +/* + * 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; + while (!m_log_entries.empty() && retiring_entries.size() < frees_per_tx && + this->can_retire_entry(m_log_entries.front())) { + auto entry = m_log_entries.front(); + 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); + } + } + } + } + + 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() << ", remain " + << m_ops_to_flush.size() << 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)) { + 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..61e39b7c1 --- /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..52f67ae20 --- /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(std::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..753b15b69 --- /dev/null +++ b/src/librbd/cache/pwl/ssd/WriteLog.cc @@ -0,0 +1,1160 @@ +// -*- 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 std; +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 = std::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 err_close_bdev; + } + 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 err_close_bdev; + } + 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 err_close_bdev; + } + + 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; + +err_close_bdev: + 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; // unused, no-op variable for SSD + bool appending = false; // unused, 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 |