]> git-server-git.apps.pok.os.sepia.ceph.com Git - ceph.git/commitdiff
crimson: refactor asok command
authorKefu Chai <kchai@redhat.com>
Sun, 16 Feb 2020 02:03:36 +0000 (10:03 +0800)
committerKefu Chai <kchai@redhat.com>
Sun, 16 Feb 2020 15:59:30 +0000 (23:59 +0800)
* do not define another iterator type, use `map::const_iterator`
  directly
* do not register hooks/commands with server block, register them
  one by one, much simpler this way.
* encapsulate the hook metadata in `AdminSocketHook`, so each
  `AdminSocketHook` instance is self-contained in the sense that
  we don't need to use an extra type for keeping track of them.

Signed-off-by: Kefu Chai <kchai@redhat.com>
src/crimson/admin/admin_socket.cc
src/crimson/admin/admin_socket.h
src/crimson/admin/osd_admin.cc
src/crimson/admin/osd_admin.h
src/crimson/osd/osd.cc
src/crimson/osd/osd.h

index 1f7fba8c75820d5eda6b882c88e88b8319576df5..47f5201c43ab39fbdb60045d8356bd28b32346e4 100644 (file)
@@ -31,120 +31,22 @@ seastar::logger& logger()
 
 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
  */
@@ -173,36 +75,28 @@ AdminSocket::maybe_parsed_t AdminSocket::parse_cmd(std::string cmd,
   }
 
   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 };
 }
 
 /*
@@ -212,29 +106,19 @@ bool AdminSocket::validate_command(const parsed_command_t& parsed,
                                    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(
@@ -262,9 +146,9 @@ seastar::future<> AdminSocket::execute_line(std::string cmdline,
        !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
@@ -350,8 +234,7 @@ seastar::future<> AdminSocket::start(const std::string& path)
   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) {
@@ -406,6 +289,9 @@ seastar::future<> AdminSocket::stop()
 
 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
@@ -428,6 +314,9 @@ class VersionHook final : public AdminSocketHook {
 */
 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
@@ -446,7 +335,10 @@ class HelpHook final : public AdminSocketHook {
   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,
@@ -456,9 +348,9 @@ class HelpHook final : public AdminSocketHook {
                                [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();
@@ -473,7 +365,11 @@ class GetdescsHook final : public AdminSocketHook {
   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,
@@ -483,10 +379,11 @@ class GetdescsHook final : public AdminSocketHook {
     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();
@@ -498,28 +395,13 @@ class GetdescsHook final : public AdminSocketHook {
 };
 
 /// 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
index 400b36aada3db0b936a05426513dfa9be484e7c7..0965ca7261c2515b59b8d2515aa3aba9bc90625d 100644 (file)
@@ -41,6 +41,11 @@ class AdminSocket;
  */
 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
    */
@@ -49,28 +54,11 @@ class AdminSocketHook {
        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;
@@ -116,16 +104,7 @@ class AdminSocket : public seastar::enable_lw_shared_from_this<AdminSocket> {
    * \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:
   /**
@@ -133,15 +112,9 @@ class AdminSocket : public seastar::enable_lw_shared_from_this<AdminSocket> {
    * 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>;
@@ -149,12 +122,7 @@ class AdminSocket : public seastar::enable_lw_shared_from_this<AdminSocket> {
   /**
    * 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);
@@ -180,13 +148,6 @@ class AdminSocket : public seastar::enable_lw_shared_from_this<AdminSocket> {
    */
   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
@@ -194,77 +155,29 @@ class AdminSocket : public seastar::enable_lw_shared_from_this<AdminSocket> {
    */
   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
index e271535f2d8e3b3e1d639ac393ad1adfae9eb019..f094e335e8f9d09e90b45e730c6646627e4dda2c 100644 (file)
@@ -2,6 +2,8 @@
 // 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>
@@ -27,290 +29,208 @@ namespace crimson::admin {
 
 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
index 7c79b796457aa1771ecd99e75aee7467d1d80473..2fd82fddb5e8e4bfecc3b57fddf1873f964528f6 100644 (file)
@@ -3,31 +3,19 @@
 #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
index 483777628a19596569dce8f1fb9bf1260203a392..cbb310ad3b4feb4fcf4ac4dbd633d2fcef9185bf 100644 (file)
@@ -74,7 +74,6 @@ OSD::OSD(int id, uint32_t nonce,
     // 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>();
@@ -407,17 +406,14 @@ seastar::future<> OSD::_send_alive()
 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>()));
   });
 }
 
@@ -428,7 +424,7 @@ seastar::future<> OSD::stop()
   state.set_stopping();
 
   return gate.close().then([this] {
-    return stop_asok_admin();
+    return asok->stop();
   }).then([this] {
     return heartbeat->stop();
   }).then([this] {
index 45dffb44fdcf4140c7713963333249b6517a5fae..aa28cd8ba1536e72510542543fe3457846c0b40d 100644 (file)
@@ -116,7 +116,6 @@ class OSD final : public crimson::net::Dispatcher,
 
   // admin-socket
   seastar::lw_shared_ptr<crimson::admin::AdminSocket> asok;
-  std::unique_ptr<crimson::admin::OsdAdmin> admin;
 
 public:
   OSD(int id, uint32_t nonce,
@@ -187,7 +186,6 @@ private:
 
   void check_osdmap_features();
 
-  seastar::future<> stop_asok_admin();
   seastar::future<> start_asok_admin();
 
 public:
@@ -218,7 +216,6 @@ public:
   seastar::future<> update_heartbeat_peers();
 
   friend class PGAdvanceMap;
-  friend class crimson::admin::OsdAdminImp;
 };
 
 }