std::ostream &operator<<(std::ostream &, const segments_info_t &);
/**
- * Callback interface for managing available segments
+ * Callback interface for journal trimming
*/
-class SegmentProvider {
+class JournalTrimmer {
public:
// get the committed journal head
virtual journal_seq_t get_journal_head() const = 0;
virtual void update_journal_tails(
journal_seq_t dirty_tail, journal_seq_t alloc_tail) = 0;
+ virtual ~JournalTrimmer() {}
+};
+
+/**
+ * Callback interface for managing available segments
+ */
+class SegmentProvider {
+public:
virtual const segment_info_t& get_seg_info(segment_id_t id) const = 0;
virtual segment_id_t allocate_segment(
};
-class AsyncCleaner : public SegmentProvider {
+class AsyncCleaner : public SegmentProvider, public JournalTrimmer {
public:
/// Config
struct config_t {
reclaim_gen_t gen,
SegmentProvider& sp,
SegmentSeqAllocator &ssa)
- : segment_allocator(segment_type_t::OOL, category, gen, sp, ssa),
+ : segment_allocator(nullptr, category, gen, sp, ssa),
record_submitter(crimson::common::get_conf<uint64_t>(
"seastore_journal_iodepth_limit"),
crimson::common::get_conf<uint64_t>(
namespace crimson::os::seastore::journal {
-JournalRef make_segmented(SegmentProvider &provider)
+JournalRef make_segmented(
+ SegmentProvider &provider,
+ JournalTrimmer &trimmer)
{
- return std::make_unique<SegmentedJournal>(provider);
+ return std::make_unique<SegmentedJournal>(provider, trimmer);
}
JournalRef make_circularbounded(
+ JournalTrimmer &trimmer,
crimson::os::seastore::random_block_device::RBMDevice* device,
std::string path)
{
- return std::make_unique<CircularBoundedJournal>(device, path);
+ return std::make_unique<CircularBoundedJournal>(trimmer, device, path);
}
}
class SegmentManagerGroup;
class SegmentProvider;
+class JournalTrimmer;
enum class journal_type_t {
SEGMENT_JOURNAL = 0,
class Journal {
public:
+ virtual JournalTrimmer &get_trimmer() = 0;
/**
* initializes journal for mkfs writes -- must run prior to calls
* to submit_record.
namespace journal {
-JournalRef make_segmented(SegmentProvider &provider);
+JournalRef make_segmented(
+ SegmentProvider &provider,
+ JournalTrimmer &trimmer);
JournalRef make_circularbounded(
+ JournalTrimmer &trimmer,
crimson::os::seastore::random_block_device::RBMDevice* device,
std::string path);
#include "crimson/common/errorator-loop.h"
#include "include/intarith.h"
+#include "crimson/os/seastore/async_cleaner.h"
#include "crimson/os/seastore/journal/circular_bounded_journal.h"
#include "crimson/os/seastore/logging.h"
<< ")";
}
-CircularBoundedJournal::CircularBoundedJournal(RBMDevice* device,
+CircularBoundedJournal::CircularBoundedJournal(
+ JournalTrimmer &trimmer,
+ RBMDevice* device,
const std::string &path)
- : device(device), path(path) {}
+ : trimmer(trimmer), device(device), path(path) {}
CircularBoundedJournal::mkfs_ret
CircularBoundedJournal::mkfs(const mkfs_config_t& config)
paddr,
write_result
};
+ trimmer.set_journal_head(write_result.start_seq);
return submit_result;
});
}
#include "crimson/os/seastore/random_block_manager/rbm_device.h"
#include <list>
-
namespace crimson::os::seastore::journal {
constexpr rbm_abs_addr CBJOURNAL_START_ADDRESS = 0;
}
};
- CircularBoundedJournal(RBMDevice* device, const std::string &path);
+ CircularBoundedJournal(
+ JournalTrimmer &trimmer, RBMDevice* device, const std::string &path);
~CircularBoundedJournal() {}
+ JournalTrimmer &get_trimmer() final {
+ return trimmer;
+ }
+
open_for_mkfs_ret open_for_mkfs() final;
open_for_mount_ret open_for_mount() final;
private:
cbj_header_t header;
+ JournalTrimmer &trimmer;
RBMDevice* device;
std::string path;
WritePipeline *write_pipeline = nullptr;
namespace crimson::os::seastore::journal {
SegmentAllocator::SegmentAllocator(
- segment_type_t type,
+ JournalTrimmer *trimmer,
data_category_t category,
reclaim_gen_t gen,
SegmentProvider &sp,
SegmentSeqAllocator &ssa)
: print_name{fmt::format("{}_G{}", category, gen)},
- type{type},
+ type{trimmer == nullptr ?
+ segment_type_t::OOL :
+ segment_type_t::JOURNAL},
category{category},
gen{gen},
segment_provider{sp},
sm_group{*sp.get_segment_manager_group()},
- segment_seq_allocator(ssa)
+ segment_seq_allocator(ssa),
+ trimmer{trimmer}
{
- ceph_assert(type != segment_type_t::NULL_SEG);
reset();
}
journal_seq_t dirty_tail;
journal_seq_t alloc_tail;
if (type == segment_type_t::JOURNAL) {
- dirty_tail = segment_provider.get_dirty_tail();
- alloc_tail = segment_provider.get_alloc_tail();
+ dirty_tail = trimmer->get_dirty_tail();
+ alloc_tail = trimmer->get_alloc_tail();
if (is_mkfs) {
ceph_assert(dirty_tail == JOURNAL_SEQ_NULL);
ceph_assert(alloc_tail == JOURNAL_SEQ_NULL);
namespace crimson::os::seastore {
class SegmentProvider;
+ class JournalTrimmer;
}
namespace crimson::os::seastore::journal {
crimson::ct_error::input_output_error>;
public:
- SegmentAllocator(segment_type_t type,
+ SegmentAllocator(JournalTrimmer *trimmer,
data_category_t category,
reclaim_gen_t gen,
SegmentProvider &sp,
seastore_off_t written_to;
SegmentSeqAllocator &segment_seq_allocator;
segment_nonce_t current_segment_nonce;
- //3. journal tail written to both segment_header_t and segment_tail_t
+ JournalTrimmer *trimmer;
};
/**
namespace crimson::os::seastore::journal {
SegmentedJournal::SegmentedJournal(
- SegmentProvider &segment_provider)
- : segment_provider(segment_provider),
- segment_seq_allocator(
+ SegmentProvider &segment_provider,
+ JournalTrimmer &trimmer)
+ : segment_seq_allocator(
new SegmentSeqAllocator(segment_type_t::JOURNAL)),
- journal_segment_allocator(segment_type_t::JOURNAL,
+ journal_segment_allocator(&trimmer,
data_category_t::METADATA,
0, // generation
segment_provider,
crimson::common::get_conf<double>(
"seastore_journal_batch_preferred_fullness"),
journal_segment_allocator),
- sm_group(*segment_provider.get_segment_manager_group())
+ sm_group(*segment_provider.get_segment_manager_group()),
+ trimmer{trimmer}
{
}
return scan_last_segment(last_segment_id, last_header
).safe_then([this, FNAME, segments=std::move(segments)] {
INFO("dirty_tail={}, alloc_tail={}",
- segment_provider.get_dirty_tail(),
- segment_provider.get_alloc_tail());
- auto journal_tail = segment_provider.get_journal_tail();
+ trimmer.get_dirty_tail(),
+ trimmer.get_alloc_tail());
+ auto journal_tail = trimmer.get_journal_tail();
auto journal_tail_paddr = journal_tail.offset;
ceph_assert(journal_tail != JOURNAL_SEQ_NULL);
ceph_assert(journal_tail_paddr != P_ADDR_NULL);
{
LOG_PREFIX(SegmentedJournal::scan_last_segment);
assert(segment_id == segment_header.physical_segment_id);
- segment_provider.update_journal_tails(
+ trimmer.update_journal_tails(
segment_header.dirty_tail, segment_header.alloc_tail);
auto seq = journal_seq_t{
segment_header.segment_seq,
DEBUG("got {}, at {}", tail_delta, start_seq);
ceph_assert(tail_delta.dirty_tail != JOURNAL_SEQ_NULL);
ceph_assert(tail_delta.alloc_tail != JOURNAL_SEQ_NULL);
- segment_provider.update_journal_tails(
+ trimmer.update_journal_tails(
tail_delta.dirty_tail, tail_delta.alloc_tail);
}
}
return handler(
locator,
delta,
- segment_provider.get_dirty_tail(),
- segment_provider.get_alloc_tail(),
+ trimmer.get_dirty_tail(),
+ trimmer.get_alloc_tail(),
modify_time
).safe_then([&stats, delta_type=delta.type](bool is_applied) {
if (is_applied) {
*/
class SegmentedJournal : public Journal {
public:
- SegmentedJournal(SegmentProvider &segment_provider);
+ SegmentedJournal(
+ SegmentProvider &segment_provider,
+ JournalTrimmer &trimmer);
~SegmentedJournal() {}
+ JournalTrimmer &get_trimmer() final {
+ return trimmer;
+ }
+
open_for_mkfs_ret open_for_mkfs() final;
open_for_mount_ret open_for_mount() final;
OrderingHandle &handle
);
- SegmentProvider& segment_provider;
SegmentSeqAllocatorRef segment_seq_allocator;
SegmentAllocator journal_segment_allocator;
RecordSubmitter record_submitter;
SegmentManagerGroup &sm_group;
+ JournalTrimmer &trimmer;
WritePipeline* write_pipeline = nullptr;
/// return ordered vector of segments to replay
).safe_then([this] {
return journal->open_for_mkfs();
}).safe_then([this](auto start_seq) {
- async_cleaner->update_journal_tails(start_seq, start_seq);
- async_cleaner->set_journal_head(start_seq);
+ journal->get_trimmer().update_journal_tails(start_seq, start_seq);
+ journal->get_trimmer().set_journal_head(start_seq);
return epm->open();
}).safe_then([this, FNAME]() {
return with_transaction_intr(
}).safe_then([this] {
return journal->open_for_mount();
}).safe_then([this](auto start_seq) {
- async_cleaner->set_journal_head(start_seq);
+ journal->get_trimmer().set_journal_head(start_seq);
return with_transaction_weak(
"mount",
[this](auto &t)
auto record = cache->prepare_record(
tref,
- async_cleaner->get_journal_head(),
- async_cleaner->get_dirty_tail());
+ journal->get_trimmer().get_journal_head(),
+ journal->get_trimmer().get_dirty_tail());
tref.get_handle().maybe_release_collection_lock();
).safe_then([this, FNAME, &tref](auto submit_result) mutable {
SUBDEBUGT(seastore_t, "committed with {}", tref, submit_result);
auto start_seq = submit_result.write_result.start_seq;
- async_cleaner->set_journal_head(start_seq);
+ journal->get_trimmer().set_journal_head(start_seq);
cache->complete_commit(
tref,
submit_result.record_block_base,
lba_manager->complete_transaction(tref, lba_to_clear, lba_to_link);
backref_manager->complete_transaction(tref, backref_to_clear, backref_to_link);
- async_cleaner->update_journal_tails(
+ journal->get_trimmer().update_journal_tails(
cache->get_oldest_dirty_from().value_or(start_seq),
cache->get_oldest_backref_dirty_from().value_or(start_seq));
return tref.get_handle().complete();
auto p_device_type = primary_device->get_device_type();
JournalRef journal;
if (p_device_type == device_type_t::SEGMENTED) {
- journal = journal::make_segmented(*async_cleaner);
+ journal = journal::make_segmented(*async_cleaner, *async_cleaner);
} else {
ceph_assert(p_device_type == device_type_t::RANDOM_BLOCK);
journal = journal::make_circularbounded(
+ *async_cleaner,
static_cast<random_block_device::RBMDevice*>(primary_device),
"");
async_cleaner->set_disable_trim(true);
using namespace crimson::os::seastore::lba_manager::btree;
struct btree_test_base :
- public seastar_test_suite_t, SegmentProvider {
+ public seastar_test_suite_t, SegmentProvider, JournalTrimmer {
segment_manager::EphemeralSegmentManagerRef segment_manager;
SegmentManagerGroupRef sms;
btree_test_base() = default;
/*
- * SegmentProvider interfaces
+ * JournalTrimmer interfaces
*/
journal_seq_t get_journal_head() const final { return dummy_tail; }
void update_journal_tails(journal_seq_t, journal_seq_t) final {}
+ /*
+ * SegmentProvider interfaces
+ */
const segment_info_t& get_seg_info(segment_id_t id) const final {
tmp_info = {};
tmp_info.seq = segment_seqs.at(id);
segment_manager::get_ephemeral_device_config(0, 1));
}).safe_then([this] {
sms.reset(new SegmentManagerGroup());
- journal = journal::make_segmented(*this);
+ journal = journal::make_segmented(*this, *this);
epm.reset(new ExtentPlacementManager());
cache.reset(new Cache(*epm));
#include <random>
#include "crimson/common/log.h"
-#include "crimson/os/seastore/seastore_types.h"
+#include "crimson/os/seastore/async_cleaner.h"
#include "crimson/os/seastore/journal.h"
#include "crimson/os/seastore/journal/circular_bounded_journal.h"
#include "crimson/os/seastore/random_block_manager.h"
#include "crimson/os/seastore/random_block_manager/rbm_device.h"
+#include "crimson/os/seastore/seastore_types.h"
#include "test/crimson/seastore/transaction_manager_test_state.h"
using namespace crimson;
}
};
-struct cbjournal_test_t : public seastar_test_suite_t
+struct cbjournal_test_t : public seastar_test_suite_t, JournalTrimmer
{
std::vector<entry_validator_t> entries;
std::unique_ptr<CircularBoundedJournal> cbj;
cbjournal_test_t() {
device = new random_block_device::TestMemory(CBTEST_DEFAULT_TEST_SIZE + CBTEST_DEFAULT_BLOCK_SIZE);
- cbj.reset(new CircularBoundedJournal(device, std::string()));
+ cbj.reset(new CircularBoundedJournal(*this, device, std::string()));
device_id_t d_id = 1 << (std::numeric_limits<device_id_t>::digits - 1);
config.block_size = CBTEST_DEFAULT_BLOCK_SIZE;
config.total_size = CBTEST_DEFAULT_TEST_SIZE;
cbj->set_write_pipeline(&pipeline);
}
+ /*
+ * JournalTrimmer interfaces
+ */
+ journal_seq_t get_journal_head() const {
+ return JOURNAL_SEQ_NULL;
+ }
+
+ journal_seq_t get_dirty_tail() const final {
+ return JOURNAL_SEQ_NULL;
+ }
+
+ journal_seq_t get_alloc_tail() const final {
+ return JOURNAL_SEQ_NULL;
+ }
+
+ void set_journal_head(journal_seq_t head) final {}
+
+ void update_journal_tails(
+ journal_seq_t dirty_tail,
+ journal_seq_t alloc_tail) final {}
+
seastar::future<> set_up_fut() final {
return seastar::now();
}
}
};
-struct journal_test_t : seastar_test_suite_t, SegmentProvider {
+struct journal_test_t : seastar_test_suite_t, SegmentProvider, JournalTrimmer {
segment_manager::EphemeralSegmentManagerRef segment_manager;
WritePipeline pipeline;
JournalRef journal;
journal_test_t() = default;
/*
- * SegmentProvider interfaces
+ * JournalTrimmer interfaces
*/
journal_seq_t get_journal_head() const final { return dummy_tail; }
void update_journal_tails(journal_seq_t, journal_seq_t) final {}
+ /*
+ * SegmentProvider interfaces
+ */
const segment_info_t& get_seg_info(segment_id_t id) const final {
tmp_info = {};
tmp_info.seq = segment_seqs.at(id);
block_size = segment_manager->get_block_size();
sms.reset(new SegmentManagerGroup());
next = segment_id_t(segment_manager->get_device_id(), 0);
- journal = journal::make_segmented(*this);
+ journal = journal::make_segmented(*this, *this);
journal->set_write_pipeline(&pipeline);
sms->add_segment_manager(segment_manager.get());
return journal->open_for_mkfs();
auto replay(T &&f) {
return journal->close(
).safe_then([this, f=std::move(f)]() mutable {
- journal = journal::make_segmented(*this);
+ journal = journal::make_segmented(*this, *this);
journal->set_write_pipeline(&pipeline);
return journal->replay(std::forward<T>(std::move(f)));
}).safe_then([this] {