namespace crimson::admin {
-// Hooks table - iterator support
-
-AdminHooksIter AdminSocket::begin() const
-{
- AdminHooksIter it{ *this };
- return it;
-}
-
-AdminHooksIter AdminSocket::end() const
-{
- AdminHooksIter it{ *this, true };
- return it;
-}
-
-/**
- * note that 'end-flag' is an optional parameter, and is only used internally
- * (to signal the end() state).
- */
-AdminHooksIter::AdminHooksIter(const AdminSocket& master, bool end_flag)
- : m_master{ master }, m_end_marker{ end_flag }
-{
- if (end_flag) {
- // create the equivalent of servers.end();
- m_miter = m_master.servers.cend();
- } else {
- m_miter = m_master.servers.cbegin();
- m_siter = m_miter->second.m_hooks.begin();
- }
-}
-
-AdminHooksIter& AdminHooksIter::operator++()
-{
- if (!m_end_marker) {
- ++m_siter;
- if (m_siter == m_miter->second.m_hooks.end()) {
- // move to the next server-block
- m_miter++;
- if (m_miter == m_master.servers.end()) {
- m_end_marker = true;
- return *this;
- }
-
- m_siter = m_miter->second.m_hooks.begin();
- }
- }
- return *this;
-}
-
-seastar::future<AsokRegistrationRes> AdminSocket::register_server(
- hook_server_tag server_tag, const std::vector<AsokServiceDef>& apis_served)
-{
- return seastar::with_lock(
- servers_tbl_rwlock,
- [this, server_tag, &apis_served]() -> AsokRegistrationRes {
- auto ne = servers.try_emplace(server_tag, apis_served);
- // was this server tag already registered?
- if (!ne.second) {
- return {};
- }
- logger().info("register_server(): pid:{} ^:{} (tag: {})", (int)getpid(),
- (uint64_t)(this), (uint64_t)(server_tag));
- return this->shared_from_this();
- });
-}
-
-seastar::future<> AdminSocket::unregister_server(hook_server_tag server_tag)
+seastar::future<>
+AdminSocket::register_command(std::unique_ptr<AdminSocketHook>&& hook)
{
- logger().debug("{}: pid:{} server tag: {})", __func__, (int)getpid(),
- (uint64_t)(server_tag));
-
- return seastar::with_lock(servers_tbl_rwlock, [this, server_tag] {
- if (auto erased = servers.erase(server_tag); erased == 0) {
- logger().warn("unregister_server(): unregistering a "
- "non-existing registration (tag: {})",
- (uint64_t)(server_tag));
+ return seastar::with_lock(servers_tbl_rwlock,
+ [this, hook = std::move(hook)]() mutable {
+ auto prefix = hook->prefix;
+ auto [it, added] = hooks.emplace(prefix, std::move(hook));
+ // was this server tag already registered?
+ assert(added);
+ if (added) {
+ logger().info("register_command(): {})", it->first);
}
+ return seastar::now();
});
}
-seastar::future<> AdminSocket::unregister_server(hook_server_tag server_tag,
- AdminSocketRef&& server_ref)
-{
- return unregister_server(server_tag).then([ref=std::move(server_ref)] {
- // reducing the ref-count on us (the ASOK server) by discarding server_ref:
- }).finally([server_tag] {
- logger().debug("unregister_server: done {}", (uint64_t)(server_tag));
- });
-}
-
-AdminSocket::maybe_service_def_t AdminSocket::locate_subcmd(
- std::string match) const
-{
- while (!match.empty()) {
- // try locating this sub-sequence of the incoming command
- for (auto& [tag, srv] : servers) {
- for (auto& api : srv.m_hooks) {
- if (api.command == match) {
- logger().debug("{}: located {} w/ server {}", __func__, match, tag);
- return &api;
- }
- }
- }
- // drop right-most word
- size_t pos = match.rfind(' ');
- if (pos == match.npos) {
- match.clear(); // we fail
- return {};
- } else {
- match.resize(pos);
- }
- }
- return {};
-}
-
/*
* Note: parse_cmd() is executed with servers_tbl_rwlock held as shared
*/
}
string format;
- string match;
- std::size_t full_command_seq; // the full sequence, before we start chipping
- // away the end
+ string prefix;
try {
cmd_getval(cmdmap, "format", format);
- cmd_getval(cmdmap, "prefix", match);
- full_command_seq = match.length();
+ cmd_getval(cmdmap, "prefix", prefix);
} catch (const bad_cmd_get& e) {
logger().error("{}: invalid syntax: {}", __func__, cmd);
out.append("error: command syntax: missing 'prefix'"s);
return maybe_parsed_t{ std::nullopt };
}
- if (match.empty()) {
+ if (prefix.empty()) {
// no command identified
out.append("error: no command identified"s);
return maybe_parsed_t{ std::nullopt };
}
// match the incoming op-code to one of the registered APIs
- auto parsed_cmd = locate_subcmd(match);
- if (!parsed_cmd.has_value()) {
+ if (auto found = hooks.find(prefix); found != hooks.end()) {
+ return parsed_command_t{ cmdmap, format, *found->second };
+ } else {
return maybe_parsed_t{ std::nullopt };
}
- return parsed_command_t{ match,
- cmdmap,
- format,
- parsed_cmd.value(),
- parsed_cmd.value()->hook,
- full_command_seq };
}
/*
const std::string& command_text,
ceph::bufferlist& out) const
{
- // did we receive any arguments apart from the command word(s)?
- if (parsed.cmd_seq_len == parsed.cmd.length())
- return true;
-
logger().info("{}: validating {} against:{}", __func__, command_text,
- parsed.api->cmddesc);
+ parsed.hook.desc);
stringstream os; // for possible validation error messages
- try {
- // validate_cmd throws on some syntax errors
- if (validate_cmd(nullptr, parsed.api->cmddesc, parsed.parameters, os)) {
- return true;
- }
- } catch (std::exception& e) {
- logger().error("{}: validation failure ({} : {}) {}", __func__,
- command_text, parsed.cmd, e.what());
+ if (validate_cmd(nullptr, std::string{parsed.hook.desc}, parsed.parameters, os)) {
+ return true;
+ } else {
+ os << "error: command validation failure ";
+ logger().error("{}: validation failure (incoming:{}) {}", __func__,
+ command_text, os.str());
+ out.append(os);
+ return false;
}
-
- os << "error: command validation failure ";
- logger().error("{}: validation failure (incoming:{}) {}", __func__,
- command_text, os.str());
- out.append(os);
- return false;
}
seastar::future<> AdminSocket::finalize_response(
!validate_command(*parsed, cmdline, err)) {
return finalize_response(out, std::move(err));
}
- return parsed->hook->call(parsed->cmd,
- parsed->format,
- parsed->parameters).then(
+ return parsed->hook.call(parsed->hook.prefix,
+ parsed->format,
+ parsed->parameters).then(
[this, &out](auto result) {
// add 'failed' to the contents of out_buf? not what
// happens in the old code
logger().debug("{}: asok socket path={}", __func__, path);
auto sock_path = seastar::socket_address{ seastar::unix_domain_addr{ path } };
- std::ignore = register_admin_hooks().then([this, sock_path](
- AsokRegistrationRes reg_res) {
+ std::ignore = register_admin_hooks().then([this, sock_path] {
return seastar::do_with(
seastar::engine().listen(sock_path),
[this](seastar::server_socket& lstn) {
class VersionHook final : public AdminSocketHook {
public:
+ VersionHook()
+ : AdminSocketHook{"version", "version", "get ceph version"}
+ {}
seastar::future<bufferlist> call(std::string_view,
std::string_view format,
const cmdmap_t&) const final
*/
class GitVersionHook final : public AdminSocketHook {
public:
+ GitVersionHook()
+ : AdminSocketHook{"git_version", "git_version", "get git sha1"}
+ {}
seastar::future<bufferlist> call(std::string_view command,
std::string_view format,
const cmdmap_t&) const final
const AdminSocket& m_as;
public:
- explicit HelpHook(const AdminSocket& as) : m_as{as} {}
+ explicit HelpHook(const AdminSocket& as) :
+ AdminSocketHook{"help", "help", "list available commands"},
+ m_as{as}
+ {}
seastar::future<bufferlist> call(std::string_view command,
std::string_view format,
[this, format] {
unique_ptr<Formatter> f{Formatter::create(format, "json-pretty", "json-pretty")};
f->open_object_section("help");
- for (const auto& hook : m_as) {
+ for (const auto& [prefix, hook] : m_as) {
if (!hook->help.empty()) {
- f->dump_string(hook->command.c_str(), hook->help);
+ f->dump_string(prefix.data(), hook->help);
}
}
f->close_section();
const AdminSocket& m_as;
public:
- explicit GetdescsHook(const AdminSocket& as) : m_as{ as } {}
+ explicit GetdescsHook(const AdminSocket& as) :
+ AdminSocketHook{"get_command_descriptions",
+ "get_command_descriptions",
+ "list available commands"},
+ m_as{ as } {}
seastar::future<bufferlist> call(std::string_view command,
std::string_view format,
return seastar::with_shared(m_as.servers_tbl_rwlock, [this, f=move(f)] {
int cmdnum = 0;
f->open_object_section("command_descriptions");
- for (const auto& hook : m_as) {
+ for (const auto& [prefix, hook] : m_as) {
auto secname = fmt::format("cmd {:>03}", cmdnum);
dump_cmd_and_help_to_json(f.get(), CEPH_FEATURES_ALL, secname,
- hook->cmddesc, hook->help);
+ std::string{hook->desc},
+ std::string{hook->help});
cmdnum++;
}
f->close_section();
};
/// the hooks that are served directly by the admin_socket server
-seastar::future<AsokRegistrationRes> AdminSocket::register_admin_hooks()
+seastar::future<> AdminSocket::register_admin_hooks()
{
- version_hook = std::make_unique<VersionHook>();
- git_ver_hook = std::make_unique<GitVersionHook>();
- help_hook = std::make_unique<HelpHook>(*this);
- getdescs_hook = std::make_unique<GetdescsHook>(*this);
-
- // clang-format off
- static const std::vector<AsokServiceDef> internal_hooks_tbl{
- AsokServiceDef{"version", "version", version_hook.get(), "get ceph version"}
- , AsokServiceDef{"git_version", "git_version", git_ver_hook.get(), "get git sha1"}
- , AsokServiceDef{"help", "help", help_hook.get(), "list available commands"}
- , AsokServiceDef{"get_command_descriptions", "get_command_descriptions",
- getdescs_hook.get(), "list available commands"}
- };
- // clang-format on
-
- // server_registration() returns a shared pointer to the AdminSocket
- // server, i.e. to us. As we already have shared ownership of this object,
- // we do not need it.
- return register_server(AdminSocket::hook_server_tag{ this },
- internal_hooks_tbl);
+ return seastar::when_all_succeed(
+ register_command(std::make_unique<VersionHook>()),
+ register_command(std::make_unique<GitVersionHook>()),
+ register_command(std::make_unique<HelpHook>(*this)),
+ register_command(std::make_unique<GetdescsHook>(*this)));
}
} // namespace crimson::admin
*/
class AdminSocketHook {
public:
+ AdminSocketHook(std::string_view prefix,
+ std::string_view desc,
+ std::string_view help) :
+ prefix{prefix}, desc{desc}, help{help}
+ {}
/**
* \retval 'false' for hook execution errors
*/
std::string_view format,
const cmdmap_t& cmdmap) const = 0;
virtual ~AdminSocketHook() {}
+ const std::string_view prefix;
+ const std::string_view desc;
+ const std::string_view help;
};
-/**
- * The details of a single API in a server's hooks block
- */
-struct AsokServiceDef {
- const std::string command; ///< the sequence of words that should be used
- const std::string cmddesc; ///< the command syntax
- const AdminSocketHook* hook;
- const std::string help; ///< help message
-};
-
-class AdminHooksIter; ///< an iterator over all APIs in all server blocks
-
-/// a ref-count owner of the AdminSocket, used to guarantee its existence until
-/// all server-blocks are unregistered
-using AdminSocketRef = seastar::lw_shared_ptr<AdminSocket>;
-
-using AsokRegistrationRes =
- std::optional<AdminSocketRef>; // holding the server alive until after our
- // unregistration
-
class AdminSocket : public seastar::enable_lw_shared_from_this<AdminSocket> {
public:
AdminSocket() = default;
* \retval a shared ptr to the asok server itself, or nullopt if
* a block with same tag is already registered.
*/
- seastar::future<AsokRegistrationRes> register_server(
- hook_server_tag server_tag, const std::vector<AsokServiceDef>& apis_served);
-
- /**
- * unregister all hooks registered by this hooks-server. The caller
- * gives up on its shared-ownership of the asok server once the
- * deregistration is complete.
- */
- seastar::future<> unregister_server(hook_server_tag server_tag,
- AdminSocketRef&& server_ref);
+ seastar::future<> register_command(std::unique_ptr<AdminSocketHook>&& hook);
private:
/**
* the registered APIs collection.
*/
struct parsed_command_t {
- std::string cmd;
cmdmap_t parameters;
std::string format;
- const AsokServiceDef* api;
- const AdminSocketHook* hook;
- /**
- * the length of the whole command-sequence under the 'prefix' header
- */
- std::size_t cmd_seq_len;
+ const AdminSocketHook& hook;
};
// and the shorthand:
using maybe_parsed_t = std::optional<AdminSocket::parsed_command_t>;
/**
* Registering the APIs that are served directly by the admin_socket server.
*/
- seastar::future<AsokRegistrationRes> register_admin_hooks();
-
- /**
- * unregister all hooks registered by this hooks-server
- */
- seastar::future<> unregister_server(hook_server_tag server_tag);
+ seastar::future<> register_admin_hooks();
seastar::future<> handle_client(seastar::input_stream<char>& inp,
seastar::output_stream<char>& out);
*/
seastar::gate arrivals_gate;
- std::unique_ptr<AdminSocketHook> version_hook;
- std::unique_ptr<AdminSocketHook> git_ver_hook;
- std::unique_ptr<AdminSocketHook> the0_hook;
- std::unique_ptr<AdminSocketHook> help_hook;
- std::unique_ptr<AdminSocketHook> getdescs_hook;
- std::unique_ptr<AdminSocketHook> test_throw_hook; // for dev unit-tests
-
/**
* parse the incoming command line into the sequence of words that identifies
* the API, and into its arguments. Locate the command string in the
*/
maybe_parsed_t parse_cmd(std::string command_text, bufferlist& out);
- struct server_block {
- server_block(const std::vector<AsokServiceDef>& hooks) : m_hooks{ hooks } {}
- const std::vector<AsokServiceDef>& m_hooks;
- };
-
/**
* The servers table is protected by a rw-lock, to be acquired exclusively
* only when registering or removing a server.
* The lock is locked-shared when executing any hook.
*/
mutable seastar::shared_mutex servers_tbl_rwlock;
- std::map<hook_server_tag, server_block> servers;
-
- using maybe_service_def_t = std::optional<const AsokServiceDef*>;
-
- /**
- * Find the longest subset of words in 'match' that is a registered API (in
- * any of the servers' control blocks).
- * locate_subcmd() is expected to be called with the servers table RW-lock
- * held.
- */
- maybe_service_def_t locate_subcmd(std::string match) const;
+ using hooks_t = std::map<std::string_view, std::unique_ptr<AdminSocketHook>>;
+ hooks_t hooks;
public:
/**
* iterator support
*/
- AdminHooksIter begin() const;
- AdminHooksIter end() const;
-
- using ServersListIt = std::map<hook_server_tag, server_block>::const_iterator;
- using ServerApiIt = std::vector<AsokServiceDef>::const_iterator;
+ hooks_t::const_iterator begin() const {
+ return hooks.cbegin();
+ }
+ hooks_t::const_iterator end() const {
+ return hooks.cend();
+ }
friend class AdminSocketTest;
friend class HelpHook;
friend class GetdescsHook;
- friend class AdminHooksIter;
-};
-
-/**
- * An iterator over all registered APIs.
- */
-struct AdminHooksIter
- : public std::iterator<std::output_iterator_tag, AsokServiceDef> {
- public:
- explicit AdminHooksIter(const AdminSocket& master, bool end_flag = false);
-
- ~AdminHooksIter() = default;
-
- const AsokServiceDef* operator*() const
- {
- return &(*m_siter);
- }
-
- /**
- * The (in)equality test is only used to compare to 'end'.
- */
- bool operator!=(const AdminHooksIter& other) const
- {
- return m_end_marker != other.m_end_marker;
- }
-
- AdminHooksIter& operator++();
-
- private:
- const AdminSocket& m_master;
- AdminSocket::ServersListIt m_miter;
- AdminSocket::ServerApiIt m_siter;
- bool m_end_marker;
-
- friend class AdminSocket;
};
} // namespace crimson::admin
// vim: ts=8 sw=2 smarttab
#include "crimson/admin/osd_admin.h"
+#include <string>
+#include <string_view>
#include <fmt/format.h>
#include <seastar/core/do_with.hh>
using crimson::common::local_conf;
-/**
- * the hooks and states needed to handle OSD asok requests
- */
-class OsdAdminImp {
- friend class OsdAdmin;
- friend class OsdAdminHookBase;
-
- OSD* m_osd;
-
- // shared-ownership of the socket server itself, to guarantee its existence
- // until we have a chance to remove our registration:
- AsokRegistrationRes m_socket_server;
-
- /**
- * Common code for all OSD admin hooks.
- * Adds access to the owning OSD.
- */
- class OsdAdminHookBase : public AdminSocketHook {
- protected:
- explicit OsdAdminHookBase(OsdAdminImp& master) : m_osd_admin{ master } {}
- struct tell_result_t {
- int ret = 0;
- string err;
- };
- /// the specific command implementation
- virtual seastar::future<tell_result_t> tell(const cmdmap_t& cmdmap,
- Formatter* f) const = 0;
- seastar::future<bufferlist> call(std::string_view command,
- std::string_view format,
- const cmdmap_t& cmdmap) const final
- {
- unique_ptr<Formatter> f{Formatter::create(format, "json-pretty", "json-pretty")};
- auto ret = seastar::do_with(std::move(f), [cmdmap, this] (auto& f) {
- Formatter* formatter = f.get();
- return tell(cmdmap, formatter).then([formatter](auto result) {
- bufferlist out;
- if (auto& [ret, err] = result; ret < 0) {
- out.append(fmt::format("ERROR: {}\n", cpp_strerror(ret)));
- out.append(err);
- } else {
- formatter->flush(out);
- }
- return seastar::make_ready_future<bufferlist>(std::move(out));
- });
- });
- return ret;
- }
- OsdAdminImp& m_osd_admin;
- };
-
- /**
- * An OSD admin hook: OSD status
- */
- class OsdStatusHook : public OsdAdminHookBase {
- public:
- explicit OsdStatusHook(OsdAdminImp& master) : OsdAdminHookBase(master){};
- seastar::future<tell_result_t> tell(const cmdmap_t&,
- Formatter* f) const final
- {
- f->open_object_section("status");
- f->dump_stream("cluster_fsid")
- << m_osd_admin.osd_superblock().cluster_fsid;
- f->dump_stream("osd_fsid") << m_osd_admin.osd_superblock().osd_fsid;
- f->dump_unsigned("whoami", m_osd_admin.osd_superblock().whoami);
- f->dump_string("state", m_osd_admin.osd_state_name());
- f->dump_unsigned("oldest_map", m_osd_admin.osd_superblock().oldest_map);
- f->dump_unsigned("newest_map", m_osd_admin.osd_superblock().newest_map);
- f->dump_unsigned("num_pgs", m_osd_admin.m_osd->pg_map.get_pgs().size());
- f->close_section();
- return seastar::make_ready_future<tell_result_t>();
- }
- };
-
- /**
- * An OSD admin hook: send beacon
- */
- class SendBeaconHook : public OsdAdminHookBase {
- public:
- explicit SendBeaconHook(OsdAdminImp& master) : OsdAdminHookBase(master){};
- seastar::future<tell_result_t> tell(const cmdmap_t&, Formatter*) const final
- {
- if (m_osd_admin.get_osd_state().is_active()) {
- return m_osd_admin.m_osd->send_beacon().then([] {
- return seastar::make_ready_future<tell_result_t>();
- });
- } else {
- return seastar::make_ready_future<tell_result_t>();
- }
- }
- };
-
- /**
- * A CephContext admin hook: listing the configuration values
- */
- class ConfigShowHook : public OsdAdminHookBase {
- public:
- explicit ConfigShowHook(OsdAdminImp& master)
- : OsdAdminHookBase(master) {}
- seastar::future<tell_result_t> tell(const cmdmap_t&, Formatter* f) const final
- {
- f->open_object_section("config_show");
- local_conf().show_config(f);
- f->close_section();
- return seastar::make_ready_future<tell_result_t>();
- }
- };
+template <class Hook, class... Args>
+std::unique_ptr<AdminSocketHook> make_asok_hook(Args&&... args)
+{
+ return std::make_unique<Hook>(std::forward<Args>(args)...);
+}
- /**
- * A CephContext admin hook: fetching the value of a specific
- * configuration item
- */
- class ConfigGetHook : public OsdAdminHookBase {
- public:
- explicit ConfigGetHook(OsdAdminImp& master)
- : OsdAdminHookBase(master) {}
- seastar::future<tell_result_t> tell(const cmdmap_t& cmdmap,
- ceph::Formatter* f) const final
- {
- std::string var;
- if (!cmd_getval(cmdmap, "var", var)) {
- // should have been caught by 'validate()'
- return seastar::make_ready_future<tell_result_t>(
- tell_result_t{-EINVAL, "syntax error: 'config get <var>'"});
- }
- try {
- f->open_object_section("config_get");
- std::string conf_val =
- local_conf().get_val<std::string>(var);
- f->dump_string(var.c_str(), conf_val.c_str());
- f->close_section();
- return seastar::make_ready_future<tell_result_t>();
- } catch (const boost::bad_get&) {
- return seastar::make_ready_future<tell_result_t>(
- tell_result_t{-EINVAL, fmt::format("unrecognized option {}", var)});
- }
- }
+class OsdAdminHookBase : public AdminSocketHook {
+protected:
+ OsdAdminHookBase(std::string_view prefix,
+ std::string_view desc,
+ std::string_view help)
+ : AdminSocketHook(prefix, desc, help) {}
+ struct tell_result_t {
+ int ret = 0;
+ std::string err;
};
-
- /**
- * A CephContext admin hook: setting the value of a specific configuration
- * item (a real example: {"prefix": "config set", "var":"debug_osd", "val":
- * ["30/20"]} )
- */
- class ConfigSetHook : public OsdAdminHookBase {
- public:
- explicit ConfigSetHook(OsdAdminImp& master)
- : OsdAdminHookBase(master) {}
-
- seastar::future<tell_result_t> tell(const cmdmap_t& cmdmap,
- ceph::Formatter* f) const final
- {
- std::string var;
- std::vector<std::string> new_val;
- if (!cmd_getval(cmdmap, "var", var) ||
- !cmd_getval(cmdmap, "val", new_val)) {
- return seastar::make_ready_future<tell_result_t>(
- tell_result_t{-EINVAL, "syntax error: 'config set <var> <value>'"});
- }
- // val may be multiple words
- string valstr = str_join(new_val, " ");
- return local_conf().set_val(var, valstr).then([f] {
- f->open_object_section("config_set");
- f->dump_string("success", "");
- f->close_section();
- return seastar::make_ready_future<tell_result_t>();
- }).handle_exception_type([](std::invalid_argument& e) {
- return seastar::make_ready_future<tell_result_t>(
- tell_result_t{-EINVAL, e.what()});
+ /// the specific command implementation
+ virtual seastar::future<tell_result_t> tell(const cmdmap_t& cmdmap,
+ Formatter* f) const = 0;
+ seastar::future<bufferlist> call(std::string_view prefix,
+ std::string_view format,
+ const cmdmap_t& cmdmap) const final
+ {
+ unique_ptr<Formatter> f{Formatter::create(format, "json-pretty", "json-pretty")};
+ auto ret = seastar::do_with(std::move(f), [cmdmap, this] (auto& f) {
+ Formatter* formatter = f.get();
+ return tell(cmdmap, formatter).then([formatter](auto result) {
+ bufferlist out;
+ if (auto& [ret, err] = result; ret < 0) {
+ out.append(fmt::format("ERROR: {}\n", cpp_strerror(ret)));
+ out.append(err);
+ } else {
+ formatter->flush(out);
+ }
+ return seastar::make_ready_future<bufferlist>(std::move(out));
});
- }
- };
-
- /**
- * A CephContext admin hook: calling assert (if allowed by
- * 'debug_asok_assert_abort')
- */
- class AssertAlwaysHook : public OsdAdminHookBase {
- public:
- explicit AssertAlwaysHook(OsdAdminImp& master)
- : OsdAdminHookBase(master) {}
- seastar::future<tell_result_t> tell(const cmdmap_t&,
- ceph::Formatter*) const final
- {
- if (local_conf().get_val<bool>("debug_asok_assert_abort")) {
- ceph_assert_always(0);
- return seastar::make_ready_future<tell_result_t>();
- } else {
- return seastar::make_ready_future<tell_result_t>(
- tell_result_t{-EPERM, "configuration set to disallow asok assert"});
- }
- }
- };
+ });
+ return ret;
+ }
+};
- /**
- * provide the hooks with access to OSD internals
- */
- const OSDSuperblock& osd_superblock() const
+/**
+ * An OSD admin hook: OSD status
+ */
+class OsdStatusHook : public OsdAdminHookBase {
+public:
+ explicit OsdStatusHook(crimson::osd::OSD& osd) :
+ OsdAdminHookBase("status", "status", "OSD status"),
+ osd(osd)
+ {}
+ seastar::future<tell_result_t> tell(const cmdmap_t&,
+ Formatter* f) const final
{
- return m_osd->superblock;
+ f->open_object_section("status");
+ osd.dump_status(f);
+ f->close_section();
+ return seastar::make_ready_future<tell_result_t>();
}
+private:
+ crimson::osd::OSD& osd;
+};
+template std::unique_ptr<AdminSocketHook>
+make_asok_hook<OsdStatusHook>(crimson::osd::OSD& osd);
- OSDState get_osd_state() const
+/**
+ * An OSD admin hook: send beacon
+ */
+class SendBeaconHook : public OsdAdminHookBase {
+public:
+ explicit SendBeaconHook(crimson::osd::OSD& osd) :
+ OsdAdminHookBase("send_beacon",
+ "send_beacon",
+ "send OSD beacon to mon immediately"),
+ osd(osd)
+ {}
+ seastar::future<tell_result_t> tell(const cmdmap_t&, Formatter*) const final
{
- return m_osd->state;
+ return osd.send_beacon().then([] {
+ return seastar::make_ready_future<tell_result_t>();
+ });
}
+private:
+ crimson::osd::OSD& osd;
+};
+template std::unique_ptr<AdminSocketHook>
+make_asok_hook<SendBeaconHook>(crimson::osd::OSD& osd);
- string_view osd_state_name() const
+/**
+ * A CephContext admin hook: listing the configuration values
+ */
+class ConfigShowHook : public OsdAdminHookBase {
+public:
+ explicit ConfigShowHook() :
+ OsdAdminHookBase("config show",
+ "config show",
+ "dump current config settings")
+ {}
+ seastar::future<tell_result_t> tell(const cmdmap_t&, Formatter* f) const final
{
- return m_osd->state.to_string();
+ f->open_object_section("config_show");
+ local_conf().show_config(f);
+ f->close_section();
+ return seastar::make_ready_future<tell_result_t>();
}
+};
+template std::unique_ptr<AdminSocketHook> make_asok_hook<ConfigShowHook>();
- OsdStatusHook osd_status_hook{*this};
- SendBeaconHook send_beacon_hook{*this};
- ConfigShowHook config_show_hook{*this};
- ConfigGetHook config_get_hook{*this};
- ConfigSetHook config_set_hook{*this};
- AssertAlwaysHook assert_hook{*this};
-
- public:
- OsdAdminImp(OSD* osd)
- : m_osd{ osd }
+/**
+ * A CephContext admin hook: fetching the value of a specific
+ * configuration item
+ */
+class ConfigGetHook : public OsdAdminHookBase {
+public:
+ ConfigGetHook() :
+ OsdAdminHookBase("config get",
+ "config get name=var,type=CephString",
+ "config get <field>: get the config value")
{}
-
- ~OsdAdminImp() = default;
-
- seastar::future<> register_admin_commands()
+ seastar::future<tell_result_t> tell(const cmdmap_t& cmdmap,
+ ceph::Formatter* f) const final
{
- static const std::vector<AsokServiceDef> hooks_tbl{
- // clang-format off
- AsokServiceDef{"status", "status", &osd_status_hook,
- "OSD status"}
- , AsokServiceDef{"send_beacon", "send_beacon", &send_beacon_hook,
- "send OSD beacon to mon immediately"}
- , AsokServiceDef{"config show", "config show", &config_show_hook,
- "dump current config settings" }
- , AsokServiceDef{"config get", "config get name=var,type=CephString",
- &config_get_hook,
- "config get <field>: get the config value" }
- , AsokServiceDef{"config set",
- "config set name=var,type=CephString name=val,type=CephString,n=N",
- &config_set_hook,
- "config set <field> <val> [<val> ...]: set a config variable" }
- , AsokServiceDef{ "assert", "assert", &assert_hook, "asserts" }
- // clang-format on
- };
-
- return m_osd->asok
- ->register_server(AdminSocket::hook_server_tag{ this }, hooks_tbl)
- .then([this](AsokRegistrationRes res) { m_socket_server = res; });
+ std::string var;
+ if (!cmd_getval(cmdmap, "var", var)) {
+ // should have been caught by 'validate()'
+ return seastar::make_ready_future<tell_result_t>(
+ tell_result_t{-EINVAL, "syntax error: 'config get <var>'"});
+ }
+ try {
+ f->open_object_section("config_get");
+ std::string conf_val =
+ local_conf().get_val<std::string>(var);
+ f->dump_string(var.c_str(), conf_val.c_str());
+ f->close_section();
+ return seastar::make_ready_future<tell_result_t>();
+ } catch (const boost::bad_get&) {
+ return seastar::make_ready_future<tell_result_t>(
+ tell_result_t{-EINVAL, fmt::format("unrecognized option {}", var)});
+ }
}
+};
+template std::unique_ptr<AdminSocketHook> make_asok_hook<ConfigGetHook>();
- seastar::future<> unregister_admin_commands()
+/**
+ * A CephContext admin hook: setting the value of a specific configuration
+ * item (a real example: {"prefix": "config set", "var":"debug_osd", "val":
+ * ["30/20"]} )
+ */
+class ConfigSetHook : public OsdAdminHookBase {
+public:
+ ConfigSetHook() :
+ OsdAdminHookBase("config set",
+ "config set name=var,type=CephString name=val,type=CephString,n=N",
+ "config set <field> <val> [<val> ...]: set a config variable")
+ {}
+ seastar::future<tell_result_t> tell(const cmdmap_t& cmdmap,
+ ceph::Formatter* f) const final
{
- if (!m_socket_server.has_value()) {
- logger().warn("{}: OSD asok APIs already removed", __func__);
- return seastar::now();
+ std::string var;
+ std::vector<std::string> new_val;
+ if (!cmd_getval(cmdmap, "var", var) ||
+ !cmd_getval(cmdmap, "val", new_val)) {
+ return seastar::make_ready_future<tell_result_t>(
+ tell_result_t{-EINVAL, "syntax error: 'config set <var> <value>'"});
}
-
- AdminSocketRef srv{ std::move(m_socket_server.value()) };
- m_socket_server.reset();
-
- return m_osd->asok->unregister_server(AdminSocket::hook_server_tag{ this },
- std::move(srv));
+ // val may be multiple words
+ string valstr = str_join(new_val, " ");
+ return local_conf().set_val(var, valstr).then([f] {
+ f->open_object_section("config_set");
+ f->dump_string("success", "");
+ f->close_section();
+ return seastar::make_ready_future<tell_result_t>();
+ }).handle_exception_type([](std::invalid_argument& e) {
+ return seastar::make_ready_future<tell_result_t>(
+ tell_result_t{-EINVAL, e.what()});
+ });
}
};
+template std::unique_ptr<AdminSocketHook> make_asok_hook<ConfigSetHook>();
-//
-// some PIMPL details:
-//
-OsdAdmin::OsdAdmin(OSD* osd)
- : m_imp{ std::make_unique<crimson::admin::OsdAdminImp>(osd) }
-{}
-
-seastar::future<> OsdAdmin::register_admin_commands()
-{
- return m_imp->register_admin_commands();
-}
-
-seastar::future<> OsdAdmin::unregister_admin_commands()
-{
- return seastar::do_with(std::move(m_imp), [](auto& detached_imp) {
- return detached_imp->unregister_admin_commands();
- });
-}
+/**
+ * A CephContext admin hook: calling assert (if allowed by
+ * 'debug_asok_assert_abort')
+ */
+class AssertAlwaysHook : public OsdAdminHookBase {
+public:
+ AssertAlwaysHook() :
+ OsdAdminHookBase("assert",
+ "assert",
+ "asserts")
+ {}
+ seastar::future<tell_result_t> tell(const cmdmap_t&,
+ ceph::Formatter*) const final
+ {
+ if (local_conf().get_val<bool>("debug_asok_assert_abort")) {
+ ceph_assert_always(0);
+ return seastar::make_ready_future<tell_result_t>();
+ } else {
+ return seastar::make_ready_future<tell_result_t>(
+ tell_result_t{-EPERM, "configuration set to disallow asok assert"});
+ }
+ }
+};
+template std::unique_ptr<AdminSocketHook> make_asok_hook<AssertAlwaysHook>();
-OsdAdmin::~OsdAdmin() = default;
-} // namespace crimson::admin
+} // namespace crimson::admin
#pragma once
#include <memory>
-#include <seastar/core/future.hh>
-namespace crimson::osd {
-class OSD;
-}
+#include "admin_socket.h"
namespace crimson::admin {
-class OsdAdminImp;
-/**
- * \brief implementation of the configuration-related 'admin_socket' API of
- * (Crimson) OSD
- *
- * Main functionality:
- * - fetching OSD status data
- * - ...
- */
-class OsdAdmin {
- public:
- OsdAdmin(crimson::osd::OSD* osd);
- ~OsdAdmin();
- seastar::future<> register_admin_commands();
- seastar::future<> unregister_admin_commands();
- private:
- std::unique_ptr<crimson::admin::OsdAdminImp> m_imp;
-};
+class OsdStatusHook;
+class SendBeaconHook;
+class ConfigShowHook;
+class ConfigGetHook;
+class ConfigSetHook;
+class AssertAlwaysHook;
+
+template<class Hook, class... Args>
+std::unique_ptr<AdminSocketHook> make_asok_hook(Args&&... args);
} // namespace crimson::admin
// do this in background
heartbeat_timer{[this] { (void)update_heartbeat_peers(); }},
asok{seastar::make_lw_shared<crimson::admin::AdminSocket>()},
- admin{std::make_unique<crimson::admin::OsdAdmin>(this)},
osdmap_gate("OSD::osdmap_gate", std::make_optional(std::ref(shard_services)))
{
osdmaps[0] = boost::make_local_shared<OSDMap>();
seastar::future<> OSD::start_asok_admin()
{
auto asok_path = local_conf().get_val<std::string>("admin_socket");
-
+ using namespace crimson::admin;
return asok->start(asok_path).then([this] {
- // register OSD-specific admin-socket hooks
- return admin->register_admin_commands();
- });
-}
-
-seastar::future<> OSD::stop_asok_admin()
-{
- return admin->unregister_admin_commands().then([this] {
- return asok->stop();
+ return seastar::when_all_succeed(
+ asok->register_command(make_asok_hook<OsdStatusHook>(*this)),
+ asok->register_command(make_asok_hook<SendBeaconHook>(*this)),
+ asok->register_command(make_asok_hook<ConfigShowHook>()),
+ asok->register_command(make_asok_hook<ConfigGetHook>()),
+ asok->register_command(make_asok_hook<ConfigSetHook>()));
});
}
state.set_stopping();
return gate.close().then([this] {
- return stop_asok_admin();
+ return asok->stop();
}).then([this] {
return heartbeat->stop();
}).then([this] {
// admin-socket
seastar::lw_shared_ptr<crimson::admin::AdminSocket> asok;
- std::unique_ptr<crimson::admin::OsdAdmin> admin;
public:
OSD(int id, uint32_t nonce,
void check_osdmap_features();
- seastar::future<> stop_asok_admin();
seastar::future<> start_asok_admin();
public:
seastar::future<> update_heartbeat_peers();
friend class PGAdvanceMap;
- friend class crimson::admin::OsdAdminImp;
};
}