only add encapsulation. alien store will be implement later.
exception handling will be implement later.
Signed-off-by: chunmei Liu <chunmei.liu@intel.com>
cyan_store.cc
cyan_collection.cc
cyan_object.cc
+ futurized_store.cc
${PROJECT_SOURCE_DIR}/src/os/Transaction.cc)
target_link_libraries(crimson-os
crimson)
seastar::future<CyanStore::omap_values_t>
CyanStore::omap_get_values(CollectionRef c,
const ghobject_t& oid,
- std::vector<std::string>&& keys)
+ const omap_keys_t& keys)
{
logger().debug("{} {} {}",
__func__, c->cid, oid);
{
const coll_t &cid = i.get_cid(op->cid);
const ghobject_t &oid = i.get_oid(op->oid);
- std::set<std::string> keys;
+ omap_keys_t keys;
i.decode_keyset(keys);
r = _omap_rmkeys(cid, oid, keys);
}
int CyanStore::_omap_rmkeys(
const coll_t& cid,
const ghobject_t& oid,
- const std::set<std::string> &aset)
+ const omap_keys_t& aset)
{
logger().debug(
"{} {} {} {} keys",
#include "osd/osd_types.h"
#include "include/uuid.h"
+#include "futurized_store.h"
+
namespace ceph::os {
class Collection;
class Transaction;
// a just-enough store for reading/writing the superblock
-class CyanStore {
+class CyanStore final : public FuturizedStore {
constexpr static unsigned MAX_KEYS_PER_OMAP_GET_CALL = 32;
const std::string path;
uuid_d osd_fsid;
public:
- template <class ConcreteExceptionT>
- class Exception : public std::logic_error {
- public:
- using std::logic_error::logic_error;
-
- // Throwing an exception isn't the sole way to signalize an error
- // with it. This approach nicely fits cold, infrequent issues but
- // when applied to a hot one (like ENOENT on write path), it will
- // likely hurt performance.
- // Alternative approach for hot errors is to create exception_ptr
- // on our own and place it in the future via make_exception_future.
- // When ::handle_exception is called, handler would inspect stored
- // exception whether it's hot-or-cold before rethrowing it.
- // The main advantage is both types flow through very similar path
- // based on future::handle_exception.
- static bool is_class_of(const std::exception_ptr& ep) {
- // Seastar offers hacks for making throwing lock-less but stack
- // unwinding still can be a problem so painful to justify going
- // with non-standard, obscure things like this one.
- return *ep.__cxa_exception_type() == typeid(ConcreteExceptionT);
- }
- };
-
- struct EnoentException : public Exception<EnoentException> {
- using Exception<EnoentException>::Exception;
- };
CyanStore(const std::string& path);
- ~CyanStore();
+ ~CyanStore() final;
- seastar::future<> mount();
- seastar::future<> umount();
+ seastar::future<> mount() final;
+ seastar::future<> umount() final;
- seastar::future<> mkfs();
+ seastar::future<> mkfs() final;
seastar::future<ceph::bufferlist> read(CollectionRef c,
const ghobject_t& oid,
uint64_t offset,
size_t len,
- uint32_t op_flags = 0);
+ uint32_t op_flags = 0) final;
seastar::future<ceph::bufferptr> get_attr(CollectionRef c,
const ghobject_t& oid,
- std::string_view name);
- using attrs_t = std::map<std::string, ceph::bufferptr, std::less<>>;
- seastar::future<attrs_t> get_attrs(CollectionRef c, const ghobject_t& oid);
+ std::string_view name) final;
+ seastar::future<attrs_t> get_attrs(CollectionRef c,
+ const ghobject_t& oid) final;
- using omap_values_t = std::map<std::string,ceph::bufferlist, std::less<>>;
seastar::future<omap_values_t> omap_get_values(
CollectionRef c,
const ghobject_t& oid,
- std::vector<std::string>&& keys);
+ const omap_keys_t& keys) final;
seastar::future<std::vector<ghobject_t>, ghobject_t> list_objects(
CollectionRef c,
const ghobject_t& start,
const ghobject_t& end,
- uint64_t limit);
+ uint64_t limit) final;
/// Retrieves paged set of values > start (if present)
seastar::future<bool, omap_values_t> omap_get_values(
CollectionRef c, ///< [in] collection
const ghobject_t &oid, ///< [in] oid
const std::optional<std::string> &start ///< [in] start, empty for begin
- ); ///< @return <done, values> values.empty() iff done
+ ) final; ///< @return <done, values> values.empty() iff done
- CollectionRef create_new_collection(const coll_t& cid);
- CollectionRef open_collection(const coll_t& cid);
- std::vector<coll_t> list_collections();
+ CollectionRef create_new_collection(const coll_t& cid) final;
+ CollectionRef open_collection(const coll_t& cid) final;
+ std::vector<coll_t> list_collections() final;
seastar::future<> do_transaction(CollectionRef ch,
- Transaction&& txn);
+ Transaction&& txn) final;
void write_meta(const std::string& key,
- const std::string& value);
- int read_meta(const std::string& key, std::string* value);
- uuid_d get_fsid() const;
+ const std::string& value) final;
+ int read_meta(const std::string& key, std::string* value) final;
+ uuid_d get_fsid() const final;
private:
int _remove(const coll_t& cid, const ghobject_t& oid);
int _omap_rmkeys(
const coll_t& cid,
const ghobject_t& oid,
- const std::set<std::string> &aset);
+ const omap_keys_t& aset);
int _omap_rmkeyrange(
const coll_t& cid,
const ghobject_t& oid,
--- /dev/null
+#include "futurized_store.h"
+#include "cyan_store.h"
+
+namespace ceph::os {
+
+std::unique_ptr<FuturizedStore> FuturizedStore::create(const std::string& type,
+ const std::string& data)
+{
+ if (type == "memstore") {
+ return std::make_unique<ceph::os::CyanStore>(data);
+ }
+
+ return nullptr;
+}
+
+}
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#pragma once
+
+#include <string>
+#include <unordered_map>
+#include <map>
+#include <typeinfo>
+#include <vector>
+
+#include <seastar/core/future.hh>
+
+#include "include/buffer_fwd.h"
+#include "include/uuid.h"
+#include "osd/osd_types.h"
+
+namespace ceph::os {
+
+class Collection;
+class Transaction;
+
+class FuturizedStore {
+
+public:
+ template <class ConcreteExceptionT>
+ class Exception : public std::logic_error {
+ public:
+ using std::logic_error::logic_error;
+
+ // Throwing an exception isn't the sole way to signalize an error
+ // with it. This approach nicely fits cold, infrequent issues but
+ // when applied to a hot one (like ENOENT on write path), it will
+ // likely hurt performance.
+ // Alternative approach for hot errors is to create exception_ptr
+ // on our own and place it in the future via make_exception_future.
+ // When ::handle_exception is called, handler would inspect stored
+ // exception whether it's hot-or-cold before rethrowing it.
+ // The main advantage is both types flow through very similar path
+ // based on future::handle_exception.
+ static bool is_class_of(const std::exception_ptr& ep) {
+ // Seastar offers hacks for making throwing lock-less but stack
+ // unwinding still can be a problem so painful to justify going
+ // with non-standard, obscure things like this one.
+ return *ep.__cxa_exception_type() == typeid(ConcreteExceptionT);
+ }
+ };
+
+ struct EnoentException : public Exception<EnoentException> {
+ using Exception<EnoentException>::Exception;
+ };
+ static std::unique_ptr<FuturizedStore> create(const std::string& type,
+ const std::string& data);
+ FuturizedStore() = default;
+ virtual ~FuturizedStore() = default;
+
+ // no copying
+ explicit FuturizedStore(const FuturizedStore& o) = delete;
+ const FuturizedStore& operator=(const FuturizedStore& o) = delete;
+
+ virtual seastar::future<> mount() = 0;
+ virtual seastar::future<> umount() = 0;
+
+ virtual seastar::future<> mkfs() = 0;
+ using CollectionRef = boost::intrusive_ptr<Collection>;
+ virtual seastar::future<ceph::bufferlist> read(CollectionRef c,
+ const ghobject_t& oid,
+ uint64_t offset,
+ size_t len,
+ uint32_t op_flags = 0) = 0;
+ virtual seastar::future<ceph::bufferptr> get_attr(CollectionRef c,
+ const ghobject_t& oid,
+ std::string_view name) = 0;
+
+ using attrs_t = std::map<std::string, ceph::bufferptr, std::less<>>;
+ virtual seastar::future<attrs_t> get_attrs(CollectionRef c,
+ const ghobject_t& oid) = 0;
+ using omap_values_t = std::map<std::string, bufferlist, std::less<>>;
+ using omap_keys_t = std::set<std::string>;
+ virtual seastar::future<omap_values_t> omap_get_values(
+ CollectionRef c,
+ const ghobject_t& oid,
+ const omap_keys_t& keys) = 0;
+ virtual seastar::future<std::vector<ghobject_t>, ghobject_t> list_objects(
+ CollectionRef c,
+ const ghobject_t& start,
+ const ghobject_t& end,
+ uint64_t limit) = 0;
+ virtual seastar::future<bool, omap_values_t> omap_get_values(
+ CollectionRef c, ///< [in] collection
+ const ghobject_t &oid, ///< [in] oid
+ const std::optional<std::string> &start ///< [in] start, empty for begin
+ ) = 0; ///< @return <done, values> values.empty() iff done
+
+ virtual CollectionRef create_new_collection(const coll_t& cid) = 0;
+ virtual CollectionRef open_collection(const coll_t& cid) = 0;
+ virtual std::vector<coll_t> list_collections() = 0;
+
+ virtual seastar::future<> do_transaction(CollectionRef ch,
+ Transaction&& txn) = 0;
+
+ virtual void write_meta(const std::string& key,
+ const std::string& value) = 0;
+ virtual int read_meta(const std::string& key, std::string* value) = 0;
+ virtual uuid_d get_fsid() const = 0;
+};
+
+}
ECBackend::ECBackend(shard_id_t shard,
ECBackend::CollectionRef coll,
- ceph::os::CyanStore* store,
+ ceph::os::FuturizedStore* store,
const ec_profile_t&,
uint64_t)
: PGBackend{shard, coll, store}
{
public:
ECBackend(shard_id_t shard,
- CollectionRef, ceph::os::CyanStore*,
+ CollectionRef, ceph::os::FuturizedStore*,
const ec_profile_t& ec_profile,
uint64_t stripe_width);
private:
uint64_t len,
uint32_t flags) override;
CollectionRef coll;
- ceph::os::CyanStore* store;
+ ceph::os::FuturizedStore* store;
};
#include "crimson/net/Messenger.h"
#include "crimson/os/cyan_collection.h"
#include "crimson/os/cyan_object.h"
-#include "crimson/os/cyan_store.h"
+#include "crimson/os/futurized_store.h"
#include "os/Transaction.h"
#include "crimson/osd/heartbeat.h"
#include "crimson/osd/osd_meta.h"
}
using ceph::common::local_conf;
-using ceph::os::CyanStore;
+using ceph::os::FuturizedStore;
OSD::OSD(int id, uint32_t nonce,
ceph::net::Messenger& cluster_msgr,
mgrc{new ceph::mgr::Client{public_msgr, *this}},
heartbeat{new Heartbeat{*this, *monc, hb_front_msgr, hb_back_msgr}},
heartbeat_timer{[this] { update_heartbeat_peers(); }},
- store{std::make_unique<ceph::os::CyanStore>(
+ store{ceph::os::FuturizedStore::create(
+ local_conf().get_val<std::string>("osd_objectstore"),
local_conf().get_val<std::string>("osd_data"))},
shard_services{cluster_msgr, public_msgr, *monc, *mgrc, *store}
{
}
namespace ceph::os {
- class CyanStore;
+ class FuturizedStore;
struct Collection;
class Transaction;
}
SimpleLRU<epoch_t, bufferlist, false> map_bl_cache;
cached_map_t osdmap;
// TODO: use a wrapper for ObjectStore
- std::unique_ptr<ceph::os::CyanStore> store;
+ std::unique_ptr<ceph::os::FuturizedStore> store;
std::unique_ptr<OSDMeta> meta_coll;
OSDState state;
#include <fmt/format.h>
#include "crimson/os/cyan_collection.h"
-#include "crimson/os/cyan_store.h"
+#include "crimson/os/futurized_store.h"
#include "os/Transaction.h"
void OSDMeta::create(ceph::os::Transaction& t)
#include "osd/osd_types.h"
namespace ceph::os {
- class CyanStore;
+ class FuturizedStore;
class Collection;
class Transaction;
}
class OSDMeta {
template<typename T> using Ref = boost::intrusive_ptr<T>;
- ceph::os::CyanStore* store;
+ ceph::os::FuturizedStore* store;
Ref<ceph::os::Collection> coll;
public:
OSDMeta(Ref<ceph::os::Collection> coll,
- ceph::os::CyanStore* store)
+ ceph::os::FuturizedStore* store)
: store{store}, coll{coll}
{}
#include "crimson/net/Connection.h"
#include "crimson/net/Messenger.h"
#include "crimson/os/cyan_collection.h"
-#include "crimson/os/cyan_store.h"
+#include "crimson/os/futurized_store.h"
#include "os/Transaction.h"
#include "crimson/osd/exceptions.h"
#include "crimson/osd/pg_meta.h"
new_acting_primary, history, pi, backfill, t);
}
-seastar::future<> PG::read_state(ceph::os::CyanStore* store)
+seastar::future<> PG::read_state(ceph::os::FuturizedStore* store)
{
coll_ref = store->open_collection(coll_t(pgid));
return PGMeta{store, pgid}.load().then(
}
namespace ceph::os {
- class CyanStore;
+ class FuturizedStore;
}
class PG : public boost::intrusive_ref_counter<
bool backfill,
ObjectStore::Transaction &t);
- seastar::future<> read_state(ceph::os::CyanStore* store);
+ seastar::future<> read_state(ceph::os::FuturizedStore* store);
void do_peering_event(
const boost::statechart::event_base &evt,
#include "crimson/os/cyan_collection.h"
#include "crimson/os/cyan_object.h"
-#include "crimson/os/cyan_store.h"
+#include "crimson/os/futurized_store.h"
#include "replicated_backend.h"
#include "ec_backend.h"
#include "exceptions.h"
std::unique_ptr<PGBackend> PGBackend::create(const spg_t pgid,
const pg_pool_t& pool,
ceph::os::CollectionRef coll,
- ceph::os::CyanStore* store,
+ ceph::os::FuturizedStore* store,
const ec_profile_t& ec_profile)
{
switch (pool.type) {
PGBackend::PGBackend(shard_id_t shard,
CollectionRef coll,
- ceph::os::CyanStore* store)
+ ceph::os::FuturizedStore* store)
: shard{shard},
coll{coll},
store{store}
OI_ATTR).then_wrapped([oid, this](auto fut) {
if (fut.failed()) {
auto ep = std::move(fut).get_exception();
- if (!ceph::os::CyanStore::EnoentException::is_class_of(ep)) {
+ if (!ceph::os::FuturizedStore::EnoentException::is_class_of(ep)) {
std::rethrow_exception(ep);
}
return seastar::make_ready_future<cached_os_t>(
std::unique_ptr<SnapSet> snapset;
if (fut.failed()) {
auto ep = std::move(fut).get_exception();
- if (!ceph::os::CyanStore::EnoentException::is_class_of(ep)) {
+ if (!ceph::os::FuturizedStore::EnoentException::is_class_of(ep)) {
std::rethrow_exception(ep);
} else {
snapset = std::make_unique<SnapSet>();
#include <string>
#include <boost/smart_ptr/local_shared_ptr.hpp>
-#include "crimson/os/cyan_store.h"
+#include "crimson/os/futurized_store.h"
+#include "crimson/os/cyan_collection.h"
#include "crimson/common/shared_lru.h"
#include "os/Transaction.h"
#include "osd/osd_types.h"
using ec_profile_t = std::map<std::string, std::string>;
public:
- PGBackend(shard_id_t shard, CollectionRef coll, ceph::os::CyanStore* store);
+ PGBackend(shard_id_t shard, CollectionRef coll, ceph::os::FuturizedStore* store);
virtual ~PGBackend() = default;
static std::unique_ptr<PGBackend> create(const spg_t pgid,
const pg_pool_t& pool,
ceph::os::CollectionRef coll,
- ceph::os::CyanStore* store,
+ ceph::os::FuturizedStore* store,
const ec_profile_t& ec_profile);
using cached_os_t = boost::local_shared_ptr<ObjectState>;
seastar::future<cached_os_t> get_object_state(const hobject_t& oid);
protected:
const shard_id_t shard;
CollectionRef coll;
- ceph::os::CyanStore* store;
+ ceph::os::FuturizedStore* store;
private:
using cached_ss_t = boost::local_shared_ptr<SnapSet>;
#include <string_view>
#include "crimson/os/cyan_collection.h"
-#include "crimson/os/cyan_store.h"
+#include "crimson/os/futurized_store.h"
// prefix pgmeta_oid keys with _ so that PGLog::read_log_and_missing() can
// easily skip them
+using ceph::os::FuturizedStore;
-using ceph::os::CyanStore;
-
-PGMeta::PGMeta(CyanStore* store, spg_t pgid)
+PGMeta::PGMeta(FuturizedStore* store, spg_t pgid)
: store{store},
pgid{pgid}
{}
namespace {
template<typename T>
- std::optional<T> find_value(const CyanStore::omap_values_t& values,
+ std::optional<T> find_value(const FuturizedStore::omap_values_t& values,
string_view key)
{
auto found = values.find(key);
seastar::future<epoch_t> PGMeta::get_epoch()
{
auto ch = store->open_collection(coll_t{pgid});
+ std::set<std::string> keys{infover_key.data(),
+ epoch_key.data()};
return store->omap_get_values(ch,
pgid.make_pgmeta_oid(),
- {string{infover_key},
- string{epoch_key}}).then(
+ keys).then(
[](auto&& values) {
{
// sanity check
seastar::future<pg_info_t, PastIntervals> PGMeta::load()
{
auto ch = store->open_collection(coll_t{pgid});
+ std::set<std::string> keys{infover_key.data(),
+ info_key.data(),
+ biginfo_key.data(),
+ fastinfo_key.data()};
return store->omap_get_values(ch,
pgid.make_pgmeta_oid(),
- {string{infover_key},
- string{info_key},
- string{biginfo_key},
- string{fastinfo_key}}).then(
+ keys).then(
[this](auto&& values) {
{
// sanity check
#include "osd/osd_types.h"
namespace ceph::os {
- class CyanStore;
+ class FuturizedStore;
}
/// PG related metadata
class PGMeta
{
- ceph::os::CyanStore* store;
+ ceph::os::FuturizedStore* store;
const spg_t pgid;
public:
- PGMeta(ceph::os::CyanStore *store, spg_t pgid);
+ PGMeta(ceph::os::FuturizedStore *store, spg_t pgid);
seastar::future<epoch_t> get_epoch();
seastar::future<pg_info_t, PastIntervals> load();
};
#include "crimson/os/cyan_collection.h"
#include "crimson/os/cyan_object.h"
-#include "crimson/os/cyan_store.h"
+#include "crimson/os/futurized_store.h"
ReplicatedBackend::ReplicatedBackend(shard_id_t shard,
ReplicatedBackend::CollectionRef coll,
- ceph::os::CyanStore* store)
+ ceph::os::FuturizedStore* store)
: PGBackend{shard, coll, store}
{}
public:
ReplicatedBackend(shard_id_t shard,
CollectionRef coll,
- ceph::os::CyanStore* store);
+ ceph::os::FuturizedStore* store);
private:
seastar::future<ceph::bufferlist> _read(const hobject_t& hoid,
uint64_t off,
ceph::net::Messenger &public_msgr,
ceph::mon::Client &monc,
ceph::mgr::Client &mgrc,
- ceph::os::CyanStore &store)
+ ceph::os::FuturizedStore &store)
: cluster_msgr(cluster_msgr),
public_msgr(public_msgr),
monc(monc),
}
namespace ceph::os {
- class CyanStore;
+ class FuturizedStore;
}
class PerfCounters;
ceph::net::Messenger &public_msgr;
ceph::mon::Client &monc;
ceph::mgr::Client &mgrc;
- ceph::os::CyanStore &store;
+ ceph::os::FuturizedStore &store;
CephContext cct;
ceph::net::Messenger &public_msgr,
ceph::mon::Client &monc,
ceph::mgr::Client &mgrc,
- ceph::os::CyanStore &store);
+ ceph::os::FuturizedStore &store);
seastar::future<> send_to_osd(
int peer,
MessageRef m,
epoch_t from_epoch);
- ceph::os::CyanStore &get_store() {
+ ceph::os::FuturizedStore &get_store() {
return store;
}
#ifdef WITH_SEASTAR
#include <seastar/core/future.hh>
-#include "crimson/os/cyan_store.h"
+#include "crimson/os/futurized_store.h"
#include "crimson/os/cyan_collection.h"
#endif
#ifdef WITH_SEASTAR
seastar::future<> read_log_and_missing_crimson(
- ceph::os::CyanStore &store,
+ ceph::os::FuturizedStore &store,
ceph::os::CollectionRef ch,
const pg_info_t &info,
ghobject_t pgmeta_oid
}
template <typename missing_type>
- struct CyanStoreLogReader {
- ceph::os::CyanStore &store;
+ struct FuturizedStoreLogReader {
+ ceph::os::FuturizedStore &store;
ceph::os::CollectionRef ch;
const pg_info_t &info;
IndexedLog &log;
on_disk_can_rollback_to = info.last_update;
missing.may_include_deletes = false;
- auto reader = std::unique_ptr<CyanStoreLogReader>(this);
+ auto reader = std::unique_ptr<FuturizedStoreLogReader>(this);
return seastar::repeat(
[this]() {
return store.omap_get_values(ch, pgmeta_oid, next).then(
[this](
- bool done, ceph::os::CyanStore::omap_values_t values) {
+ bool done, ceph::os::FuturizedStore::omap_values_t values) {
for (auto &&p : values) {
process_entry(p);
}
template <typename missing_type>
static seastar::future<> read_log_and_missing_crimson(
- ceph::os::CyanStore &store,
+ ceph::os::FuturizedStore &store,
ceph::os::CollectionRef ch,
const pg_info_t &info,
IndexedLog &log,
ldpp_dout(dpp, 20) << "read_log_and_missing coll "
<< ch->cid
<< " " << pgmeta_oid << dendl;
- return (new CyanStoreLogReader<missing_type>{
+ return (new FuturizedStoreLogReader<missing_type>{
store, ch, info, log, missing, pgmeta_oid, dpp})->start();
}