#include <vector>
#include "common/entity_name.h"
+#include "include/encoding.h"
/////////////////////// Types ///////////////////////
class CephInitParameters
{
public:
explicit CephInitParameters(uint32_t module_type_);
- std::list<std::string> get_conf_files() const;
uint32_t module_type;
EntityName name;
bool no_config_file = false;
+
+ void encode(ceph::buffer::list& bl) const {
+ ENCODE_START(1, 1, bl);
+ encode(module_type, bl);
+ encode(name, bl);
+ encode(no_config_file, bl);
+ ENCODE_FINISH(bl);
+ }
+ void decode(ceph::buffer::list::const_iterator& bl) {
+ DECODE_START(1, bl);
+ decode(module_type, bl);
+ decode(name, bl);
+ decode(no_config_file, bl);
+ DECODE_FINISH(bl);
+ }
};
+WRITE_CLASS_ENCODER(CephInitParameters)
/////////////////////// Functions ///////////////////////
extern void string_to_vec(std::vector<std::string>& args, std::string argstr);
default: 6
flags:
- startup
+- name: crimson_seastar_smp
+ type: uint
+ level: advanced
+ desc: Number of seastar reactor threads to use for the osd
+ default: 1
+ flags:
+ - startup
- name: crimson_alien_thread_cpu_cores
type: str
level: advanced
ec_backend.cc
heartbeat.cc
main.cc
+ main_config_bootstrap_helpers.cc
osd.cc
osd_meta.cc
pg.cc
#include "crimson/mon/MonClient.h"
#include "crimson/net/Messenger.h"
#include "crimson/osd/stop_signal.h"
+#include "crimson/osd/main_config_bootstrap_helpers.h"
#include "global/pidfile.h"
#include "osd.h"
using namespace std::literals;
namespace bpo = boost::program_options;
+using crimson::common::local_conf;
+using crimson::common::sharded_conf;
+using crimson::common::sharded_perf_coll;
-seastar::logger& logger() {
- return crimson::get_logger(ceph_subsys_osd);
-}
-
-void usage(const char* prog) {
- std::cout << "usage: " << prog << std::endl;
- generic_server_usage();
-}
-
-auto partition_args(int argc, const char *argv[])
+static seastar::logger& logger()
{
- auto seastar_n_early_args = [=] {
- class DummyTracker : public ConfigTracker {
- bool is_tracking(const std::string& name) const override {
- return false;
- }
- };
- DummyTracker dt;
- ConfigValues config_values;
- md_config_t config(config_values, dt, true);
- std::vector<const char *> seastar_n_early_args{argv, argv + argc};
- // pull off the stuff from seastar_n_early_args that we'll give to local_conf
- config.parse_argv(config_values, dt, seastar_n_early_args, CONF_CMDLINE);
- return seastar_n_early_args;
- }();
-
- // Now that we have a filtered seastar_n_early_args, populate config_proxy_args without them
- std::vector<const char*> config_proxy_args;
- std::set_difference(argv, argv + argc,
- std::begin(seastar_n_early_args), std::end(seastar_n_early_args),
- std::back_inserter(config_proxy_args));
- return make_pair(std::move(config_proxy_args), std::move(seastar_n_early_args));
+ return crimson::get_logger(ceph_subsys_osd);
}
-using crimson::common::local_conf;
-
seastar::future<> make_keyring()
{
const auto path = local_conf().get_val<std::string>("keyring");
});
}
-static uint64_t get_nonce()
-{
- if (auto pid = getpid(); pid == 1 || std::getenv("CEPH_USE_RANDOM_NONCE")) {
- // we're running in a container; use a random number instead!
- std::random_device rd;
- std::default_random_engine rng{rd()};
- return std::uniform_int_distribution<uint64_t>{}(rng);
- } else {
- return pid;
- }
-}
-
-seastar::future<> fetch_config()
-{
- // i don't have any client before joining the cluster, so no need to have
- // a proper auth handler
- class DummyAuthHandler : public crimson::common::AuthHandler {
- public:
- void handle_authentication(const EntityName& name,
- const AuthCapsInfo& caps)
- {}
- };
- return seastar::async([] {
- auto auth_handler = std::make_unique<DummyAuthHandler>();
- auto msgr = crimson::net::Messenger::create(entity_name_t::CLIENT(),
- "temp_mon_client",
- get_nonce());
- crimson::mon::Client monc{*msgr, *auth_handler};
- msgr->set_auth_client(&monc);
- msgr->start({&monc}).get();
- auto stop_msgr = seastar::defer([&] {
- // stop msgr here also, in case monc fails to start.
- msgr->stop();
- msgr->shutdown().get();
- });
- monc.start().handle_exception([] (auto ep) {
- fmt::print(std::cerr, "FATAL: unable to connect to cluster: {}\n", ep);
- return seastar::make_exception_future<>(ep);
- }).get();
- auto stop_monc = seastar::defer([&] {
- // unregister me from msgr first.
- msgr->stop();
- monc.stop().get();
- });
- monc.sub_want("config", 0, 0);
- monc.renew_subs().get();
- // wait for monmap and config
- monc.wait_for_config().get();
- local_conf().set_val("fsid", monc.get_fsid().to_string()).get();
- });
-}
-
-static void override_seastar_opts(std::vector<const char*>& args)
-{
- if (auto found = std::find_if(std::begin(args), std::end(args),
- [] (auto* arg) { return "--smp"sv == arg; });
- found == std::end(args)) {
- // TODO: we don't have a way to communicate the resource requirements
- // with the deployment tools, like cephadm and rook, which don't set, for
- // instance, aio-max-nr for us. but we should fix this, once crimson is able
- // to run on a multi-core system, i.e., once m-to-n problem is resolved.
- std::cout << "warn: added seastar option --smp 1" << std::endl;
- args.emplace_back("--smp");
- args.emplace_back("1");
- }
-}
-
static std::ofstream maybe_set_logger()
{
std::ofstream log_file_stream;
int main(int argc, const char* argv[])
{
+ auto early_config_result = crimson::osd::get_early_config(argc, argv);
+ if (!early_config_result.has_value()) {
+ int r = early_config_result.error();
+ std::cerr << "do_early_config returned error: " << r << std::endl;
+ return r;
+ }
+ auto &early_config = early_config_result.value();
+
+ auto seastar_n_early_args = early_config.get_early_args();
+ auto config_proxy_args = early_config.get_ceph_args();
+
seastar::app_template::config app_cfg;
app_cfg.name = "Crimson";
app_cfg.auto_handle_sigint_sigterm = false;
("prometheus_prefix", bpo::value<std::string>()->default_value("osd"),
"Prometheus metrics prefix");
- auto [config_proxy_args, seastar_n_early_args] = partition_args(argc, argv);
- if (ceph_argparse_need_usage(seastar_n_early_args)) {
- usage(argv[0]);
- }
- override_seastar_opts(seastar_n_early_args);
- std::string cluster_name{"ceph"};
- std::string conf_file_list;
- // ceph_argparse_early_args() could _exit(), while local_conf() won't ready
- // until it's started. so do the boilerplate-settings parsing here.
- auto init_params = ceph_argparse_early_args(seastar_n_early_args,
- CEPH_ENTITY_TYPE_OSD,
- &cluster_name,
- &conf_file_list);
- using crimson::common::sharded_conf;
- using crimson::common::sharded_perf_coll;
try {
- return app.run(seastar_n_early_args.size(), const_cast<char**>(seastar_n_early_args.data()),
- [&, &config_proxy_args=config_proxy_args] {
+ return app.run(
+ seastar_n_early_args.size(),
+ const_cast<char**>(seastar_n_early_args.data()),
+ [&] {
auto& config = app.configuration();
return seastar::async([&] {
try {
seastar::log_level::trace
);
}
- sharded_conf().start(init_params.name, cluster_name).get();
+ sharded_conf().start(
+ early_config.init_params.name, early_config.cluster_name).get();
local_conf().start().get();
auto stop_conf = seastar::deferred_stop(sharded_conf());
sharded_perf_coll().start().get();
auto stop_perf_coll = seastar::deferred_stop(sharded_perf_coll());
- local_conf().parse_config_files(conf_file_list).get();
+ local_conf().parse_config_files(early_config.conf_file_list).get();
local_conf().parse_env().get();
local_conf().parse_argv(config_proxy_args).get();
auto log_file_stream = maybe_set_logger();
}
const int whoami = std::stoi(local_conf()->name.get_id());
- const auto nonce = get_nonce();
+ const auto nonce = crimson::osd::get_nonce();
crimson::net::MessengerRef cluster_msgr, client_msgr;
crimson::net::MessengerRef hb_front_msgr, hb_back_msgr;
for (auto [msgr, name] : {make_pair(std::ref(cluster_msgr), "cluster"s),
if (local_conf()->no_mon_config) {
logger().info("bypassing the config fetch due to --no-mon-config");
} else {
- fetch_config().get();
+ crimson::osd::populate_config_from_mon().get();
}
if (config.count("mkfs")) {
auto osd_uuid = local_conf().get_val<uuid_d>("osd_uuid");
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "crimson/osd/main_config_bootstrap_helpers.h"
+
+#include <seastar/core/print.hh>
+#include <seastar/core/prometheus.hh>
+#include <seastar/core/thread.hh>
+#include <seastar/http/httpd.hh>
+#include <seastar/net/inet_address.hh>
+#include <seastar/util/closeable.hh>
+#include <seastar/util/defer.hh>
+#include <seastar/util/std-compat.hh>
+
+#include "common/ceph_argparse.h"
+#include "common/config_tracker.h"
+#include "crimson/common/buffer_io.h"
+#include "crimson/common/config_proxy.h"
+#include "crimson/common/fatal_signal.h"
+#include "crimson/mon/MonClient.h"
+#include "crimson/net/Messenger.h"
+#include "crimson/osd/main_config_bootstrap_helpers.h"
+
+using namespace std::literals;
+using crimson::common::local_conf;
+using crimson::common::sharded_conf;
+using crimson::common::sharded_perf_coll;
+
+static seastar::logger& logger()
+{
+ return crimson::get_logger(ceph_subsys_osd);
+}
+
+namespace crimson::osd {
+
+void usage(const char* prog)
+{
+ std::cout << "usage: " << prog << std::endl;
+ generic_server_usage();
+}
+
+
+seastar::future<> populate_config_from_mon()
+{
+ logger().info("populating config from monitor");
+ // i don't have any client before joining the cluster, so no need to have
+ // a proper auth handler
+ class DummyAuthHandler : public crimson::common::AuthHandler {
+ public:
+ void handle_authentication(const EntityName& name,
+ const AuthCapsInfo& caps)
+ {}
+ };
+ return seastar::async([] {
+ auto auth_handler = std::make_unique<DummyAuthHandler>();
+ auto msgr = crimson::net::Messenger::create(entity_name_t::CLIENT(),
+ "temp_mon_client",
+ get_nonce());
+ crimson::mon::Client monc{*msgr, *auth_handler};
+ msgr->set_auth_client(&monc);
+ msgr->start({&monc}).get();
+ auto stop_msgr = seastar::defer([&] {
+ msgr->stop();
+ msgr->shutdown().get();
+ });
+ monc.start().handle_exception([] (auto ep) {
+ fmt::print(std::cerr, "FATAL: unable to connect to cluster: {}\n", ep);
+ return seastar::make_exception_future<>(ep);
+ }).get();
+ auto stop_monc = seastar::defer([&] {
+ monc.stop().get();
+ });
+ monc.sub_want("config", 0, 0);
+ monc.renew_subs().get();
+ // wait for monmap and config
+ monc.wait_for_config().get();
+ auto fsid = monc.get_fsid().to_string();
+ local_conf().set_val("fsid", fsid).get();
+ logger().debug("{}: got config from monitor, fsid {}", __func__, fsid);
+ });
+}
+
+static tl::expected<early_config_t, int>
+_get_early_config(int argc, const char *argv[])
+{
+ early_config_t ret;
+
+ // pull off ceph configs the stuff from early_args
+ std::vector<const char *> early_args;
+ early_args.insert(
+ std::end(early_args),
+ argv, argv + argc);
+
+ ret.init_params = ceph_argparse_early_args(
+ early_args,
+ CEPH_ENTITY_TYPE_OSD,
+ &ret.cluster_name,
+ &ret.conf_file_list);
+
+ if (ceph_argparse_need_usage(early_args)) {
+ usage(argv[0]);
+ exit(0);
+ }
+
+ seastar::app_template::config app_cfg;
+ app_cfg.name = "Crimson-startup";
+ app_cfg.auto_handle_sigint_sigterm = false;
+ seastar::app_template app(std::move(app_cfg));
+ const char *bootstrap_args[] = { argv[0], "--smp", "1" };
+ int r = app.run(
+ sizeof(bootstrap_args) / sizeof(bootstrap_args[0]),
+ const_cast<char**>(bootstrap_args),
+ [argc, argv, &ret, &early_args] {
+ return seastar::async([argc, argv, &ret, &early_args] {
+ seastar::global_logger_registry().set_all_loggers_level(
+ seastar::log_level::debug);
+ sharded_conf().start(
+ ret.init_params.name, ret.cluster_name).get();
+ local_conf().start().get();
+ auto stop_conf = seastar::deferred_stop(sharded_conf());
+
+ sharded_perf_coll().start().get();
+ auto stop_perf_coll = seastar::deferred_stop(sharded_perf_coll());
+
+ local_conf().parse_env().get();
+ local_conf().parse_argv(early_args).get();
+ local_conf().parse_config_files(ret.conf_file_list).get();
+
+ if (local_conf()->no_mon_config) {
+ logger().info("bypassing the config fetch due to --no-mon-config");
+ } else {
+ populate_config_from_mon().get();
+ }
+
+ // get ceph configs
+ std::set_difference(
+ argv, argv + argc,
+ std::begin(early_args),
+ std::end(early_args),
+ std::back_inserter(ret.ceph_args));
+
+ ret.early_args.insert(
+ std::end(ret.early_args),
+ std::begin(early_args),
+ std::end(early_args));
+
+ if (auto found = std::find_if(
+ std::begin(early_args),
+ std::end(early_args),
+ [](auto* arg) { return "--smp"sv == arg; });
+ found == std::end(early_args)) {
+
+ // Set --smp based on crimson_seastar_smp config option
+ ret.early_args.emplace_back("--smp");
+
+ auto smp_config = local_conf().get_val<uint64_t>(
+ "crimson_seastar_smp");
+
+ ret.early_args.emplace_back(fmt::format("{}", smp_config));
+ logger().info("get_early_config: set --smp {}", smp_config);
+ }
+ return 0;
+ });
+ });
+ if (r < 0) {
+ return tl::unexpected(r);
+ }
+ return ret;
+}
+
+/* get_early_config handles obtaining config parameters required prior
+ * to reactor startup. Most deployment mechanisms (cephadm for one)
+ * rely on pulling configs from the monitor rather than shipping around
+ * config files, so this process needs to support pulling config options
+ * from the monitors.
+ *
+ * Of particular interest are config params related to the seastar
+ * reactor itself which can't be modified after the reactor has been
+ * started -- like the number of cores to use (smp::count). Contacting
+ * the monitors, however, requires a MonClient, which in turn needs a
+ * running reactor.
+ *
+ * Unfortunately, seastar doesn't clean up thread local state
+ * associated with seastar::smp task queues etc, so we can't
+ * start a reactor, stop it, and restart it in the same thread
+ * without an impractical amount of cleanup in seastar.
+ *
+ * More unfortunately, starting a reactor in a seperate thread
+ * and then joining the thread still doesn't avoid all global state,
+ * I observed tasks from the previous reactor incarnation nevertheless
+ * continuing to run in the new one resulting in a crash as they access
+ * freed memory.
+ *
+ * The approach taken here, therefore, is to actually fork, start a
+ * reactor in the child process, encode the resulting early_config_t,
+ * and send it back to the parent process.
+ */
+tl::expected<early_config_t, int>
+get_early_config(int argc, const char *argv[])
+{
+ int pipes[2];
+ int r = pipe2(pipes, 0);
+ if (r < 0) {
+ std::cerr << "get_early_config: failed to create pipes: "
+ << -errno << std::endl;
+ return tl::unexpected(-errno);
+ }
+
+ pid_t worker = fork();
+ if (worker < 0) {
+ close(pipes[0]);
+ close(pipes[1]);
+ std::cerr << "get_early_config: failed to fork: "
+ << -errno << std::endl;
+ return tl::unexpected(-errno);
+ } else if (worker == 0) { // child
+ close(pipes[0]);
+ auto ret = _get_early_config(argc, argv);
+ if (ret.has_value()) {
+ bufferlist bl;
+ ::encode(ret.value(), bl);
+ r = bl.write_fd(pipes[1]);
+ close(pipes[1]);
+ if (r < 0) {
+ std::cerr << "get_early_config: child failed to write_fd: "
+ << r << std::endl;
+ exit(-r);
+ } else {
+ exit(0);
+ }
+ } else {
+ std::cerr << "get_early_config: child failed: "
+ << -ret.error() << std::endl;
+ exit(-ret.error());
+ }
+ return tl::unexpected(-1);
+ } else { // parent
+ close(pipes[1]);
+
+ bufferlist bl;
+ early_config_t ret;
+ while ((r = bl.read_fd(pipes[0], 1024)) > 0);
+ close(pipes[0]);
+
+ // ignore error, we'll propogate error based on read and decode
+ waitpid(worker, nullptr, 0);
+
+ if (r < 0) {
+ std::cerr << "get_early_config: parent failed to read from pipe: "
+ << r << std::endl;
+ return tl::unexpected(r);
+ }
+ try {
+ auto bliter = bl.cbegin();
+ ::decode(ret, bliter);
+ return ret;
+ } catch (...) {
+ std::cerr << "get_early_config: parent failed to decode" << std::endl;
+ return tl::unexpected(-EINVAL);
+ }
+ }
+}
+
+}
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:nil -*-
+// vim: ts=8 sw=2 smarttab
+
+#pragma once
+
+#include <sys/types.h>
+#include <unistd.h>
+
+#include <iostream>
+#include <fstream>
+#include <random>
+
+#include <seastar/core/future.hh>
+
+#include "common/ceph_argparse.h"
+#include "include/expected.hpp"
+
+namespace crimson::osd {
+
+void usage(const char* prog);
+
+inline uint64_t get_nonce()
+{
+ if (auto pid = getpid(); pid == 1 || std::getenv("CEPH_USE_RANDOM_NONCE")) {
+ // we're running in a container; use a random number instead!
+ std::random_device rd;
+ std::default_random_engine rng{rd()};
+ return std::uniform_int_distribution<uint64_t>{}(rng);
+ } else {
+ return pid;
+ }
+}
+
+seastar::future<> populate_config_from_mon();
+
+struct early_config_t {
+ std::vector<std::string> early_args;
+ std::vector<std::string> ceph_args;
+
+ std::string cluster_name{"ceph"};
+ std::string conf_file_list;
+ CephInitParameters init_params{CEPH_ENTITY_TYPE_OSD};
+
+ /// Returned vector must not outlive in
+ auto to_ptr_vector(const std::vector<std::string> &in) {
+ std::vector<const char *> ret;
+ ret.reserve(in.size());
+ std::transform(
+ std::begin(in), std::end(in),
+ std::back_inserter(ret),
+ [](const auto &str) { return str.c_str(); });
+ return ret;
+ }
+
+ std::vector<const char *> get_early_args() {
+ return to_ptr_vector(early_args);
+ }
+
+ std::vector<const char *> get_ceph_args() {
+ return to_ptr_vector(ceph_args);
+ }
+
+ void encode(ceph::buffer::list& bl) const {
+ ENCODE_START(1, 1, bl);
+ encode(early_args, bl);
+ encode(ceph_args, bl);
+ encode(cluster_name, bl);
+ encode(conf_file_list, bl);
+ encode(init_params, bl);
+ ENCODE_FINISH(bl);
+ }
+
+ void decode(ceph::buffer::list::const_iterator& bl) {
+ DECODE_START(1, bl);
+ decode(early_args, bl);
+ decode(ceph_args, bl);
+ decode(cluster_name, bl);
+ decode(conf_file_list, bl);
+ decode(init_params, bl);
+ DECODE_FINISH(bl);
+ }
+};
+
+/**
+ * get_early_config
+ *
+ * Compile initial configuration information from command line arguments,
+ * config files, and monitors.
+ *
+ * This implementation forks off a worker process to do this work and must
+ * therefore be called very early in main(). (See implementation for an
+ * explanation).
+ */
+tl::expected<early_config_t, int>
+get_early_config(int argc, const char *argv[]);
+
+}
+
+WRITE_CLASS_ENCODER(crimson::osd::early_config_t)
bottom_cpu=$(( osd * crimson_smp ))
top_cpu=$(( bottom_cpu + crimson_smp - 1 ))
# set a single CPU nodes for each osd
- extra_seastar_args="--smp $crimson_smp --cpuset $bottom_cpu-$top_cpu"
+ extra_seastar_args="--cpuset $bottom_cpu-$top_cpu"
if [ "$debug" -ne 0 ]; then
extra_seastar_args+=" --debug"
fi
fi
fi
+if [ "$crimson" -eq 1 ]; then
+ $CEPH_BIN/ceph -c $conf_fn config set osd crimson_seastar_smp $crimson_smp
+fi
+
if [ $CEPH_NUM_MGR -gt 0 ]; then
start_mgr
fi