]> git.apps.os.sepia.ceph.com Git - ceph-ci.git/commitdiff
rgw/main: use AppMain in c++ main function
authorMatt Benjamin <mbenjamin@redhat.com>
Sat, 27 Aug 2022 17:11:30 +0000 (13:11 -0400)
committerMatt Benjamin <mbenjamin@redhat.com>
Fri, 9 Sep 2022 15:11:34 +0000 (11:11 -0400)
Consolidate standalone and library initialization into one
unit.

Signed-off-by: Matt Benjamin <mbenjamin@redhat.com>
src/rgw/librgw.cc
src/rgw/rgw_main.cc
src/rgw/rgw_main.h

index a07c942e89b60da7654522260124f58171c1f5bb..303a4f0351af8bf977c67e0dd12861e3e13a9330 100644 (file)
  *
  */
 
-#include "include/compat.h"
 #include <sys/types.h>
 #include <string.h>
 #include <chrono>
 
-#include "include/types.h"
 #include "include/rados/librgw.h"
-#include "rgw/rgw_acl_s3.h"
 #include "rgw_acl.h"
 
 #include "include/str_list.h"
-#include "include/stringify.h"
 #include "global/signal_handler.h"
-#include "common/config.h"
-#include "common/errno.h"
 #include "common/Timer.h"
 #include "common/TracepointProvider.h"
-#include "common/openssl_opts_handler.h"
-#include "common/Throttle.h"
 #include "common/WorkQueue.h"
 #include "common/ceph_argparse.h"
 #include "common/ceph_context.h"
 #include "common/common_init.h"
 #include "common/dout.h"
 
-#include "rgw_resolve.h"
 #include "rgw_op.h"
-#include "rgw_period_pusher.h"
-#include "rgw_realm_reloader.h"
 #include "rgw_rest.h"
-#include "rgw_rest_s3.h"
-#include "rgw_rest_swift.h"
-#include "rgw_rest_admin.h"
-#include "rgw_rest_info.h"
-#include "rgw_rest_usage.h"
-#include "rgw_rest_user.h"
-#include "rgw_rest_bucket.h"
-#include "rgw_rest_metadata.h"
-#include "rgw_rest_log.h"
-#include "rgw_rest_config.h"
-#include "rgw_rest_realm.h"
-#include "rgw_rest_sts.h"
-#include "rgw_rest_ratelimit.h"
-#include "rgw_swift_auth.h"
 #include "rgw_log.h"
-#include "rgw_tools.h"
 #include "rgw_frontend.h"
 #include "rgw_request.h"
 #include "rgw_process.h"
-#include "rgw_rest_user.h"
-#include "rgw_rest_s3.h"
-#include "rgw_os_lib.h"
 #include "rgw_auth.h"
-#include "rgw_auth_s3.h"
 #include "rgw_lib.h"
 #include "rgw_lib_frontend.h"
-#include "rgw_http_client.h"
-#include "rgw_http_client_curl.h"
-#include "rgw_kmip_client.h"
-#include "rgw_kmip_client_impl.h"
 #include "rgw_perf_counters.h"
-#ifdef WITH_RADOSGW_AMQP_ENDPOINT
-#include "rgw_amqp.h"
-#endif
-#ifdef WITH_RADOSGW_KAFKA_ENDPOINT
-#include "rgw_kafka.h"
-#endif
-#include "rgw_asio_frontend.h"
-#include "rgw_dmclock_scheduler_ctx.h"
-#include "rgw_lua.h"
-#ifdef WITH_RADOSGW_DBSTORE
-#include "rgw_sal_dbstore.h"
-#endif
-#include "rgw_lua_background.h"
-
-#include "services/svc_zone.h"
 #include "rgw_main.h"
 
 #include <errno.h>
index 94d64d5430d6e8d1a61b95d480bbf1459836f217..bc07d2aadec83ff8c3c6ab5151500cf56d7ea33d 100644 (file)
@@ -174,6 +174,12 @@ void rgw::AppMain::init_frontends1(bool nfs)
   g_conf().early_expand_meta(rgw_frontends_str, &cerr);
   get_str_vec(rgw_frontends_str, ",", frontends);
 
+  if (!nfs) {
+    if (frontends.empty()) {
+      frontends.push_back("beast");
+    }
+  }
+
   for (auto &f : frontends) {
     if (f.find("beast") != string::npos) {
       have_http_frontend = true;
@@ -230,23 +236,51 @@ void rgw::AppMain::init_frontends1(bool nfs)
   ceph::crypto::init_openssl_engine_once();
 } /* init_frontends1 */
 
+void rgw::AppMain::init_numa()
+{
+  if (nfs) {
+    return;
+  }
+
+  int numa_node = g_conf().get_val<int64_t>("rgw_numa_node");
+  size_t numa_cpu_set_size = 0;
+  cpu_set_t numa_cpu_set;
+
+  if (numa_node >= 0) {
+    int r = get_numa_node_cpu_set(numa_node, &numa_cpu_set_size, &numa_cpu_set);
+    if (r < 0) {
+      dout(1) << __func__ << " unable to determine rgw numa node " << numa_node
+              << " CPUs" << dendl;
+      numa_node = -1;
+    } else {
+      r = set_cpu_affinity_all_threads(numa_cpu_set_size, &numa_cpu_set);
+      if (r < 0) {
+        derr << __func__ << " failed to set numa affinity: " << cpp_strerror(r)
+        << dendl;
+      }
+    }
+  } else {
+    dout(1) << __func__ << " not setting numa affinity" << dendl;
+  }
+} /* init_numa */
+
 void rgw::AppMain::init_storage()
 {
     auto run_gc =
-    g_conf()->rgw_enable_gc_threads &&
-    g_conf()->rgw_nfs_run_gc_threads;
+    (g_conf()->rgw_enable_gc_threads &&
+      ((!nfs) || (nfs && g_conf()->rgw_nfs_run_gc_threads)));
 
   auto run_lc =
-    g_conf()->rgw_enable_lc_threads &&
-    g_conf()->rgw_nfs_run_lc_threads;
+    (g_conf()->rgw_enable_lc_threads &&
+      ((!nfs) || (nfs && g_conf()->rgw_nfs_run_lc_threads)));
 
   auto run_quota =
-    g_conf()->rgw_enable_quota_threads &&
-    g_conf()->rgw_nfs_run_quota_threads;
+    (g_conf()->rgw_enable_quota_threads &&
+      ((!nfs) || (nfs && g_conf()->rgw_nfs_run_quota_threads)));
 
   auto run_sync =
-    g_conf()->rgw_run_sync_thread &&
-    g_conf()->rgw_nfs_run_sync_thread;
+    (g_conf()->rgw_run_sync_thread &&
+      ((!nfs) || (nfs && g_conf()->rgw_nfs_run_sync_thread)));
 
   StoreManager::Config cfg = StoreManager::get_config(false, g_ceph_context);
   store = StoreManager::get_storage(dpp, dpp->get_cct(),
@@ -255,7 +289,8 @@ void rgw::AppMain::init_storage()
           run_lc,
           run_quota,
           run_sync,
-          g_conf().get_val<bool>("rgw_dynamic_resharding"));
+          g_conf().get_val<bool>("rgw_dynamic_resharding"),
+          g_conf()->rgw_cache_enabled);
 
 } /* init_storage */
 
@@ -638,17 +673,15 @@ void rgw::AppMain::shutdown()
 #endif
 
   rgw_perf_stop(g_ceph_context);
-}
+} /* AppMain::shutdown */
 
 /*
  * start up the RADOS connection and then handle HTTP messages as they come in
  */
-int main(int argc, const char **argv)
-{
+int main (int argc, char *argv[])
+{ 
+  int r{0};
 
-  multimap<string, RGWFrontendConfig*> fe_map;
-  vector<RGWFrontendConfig*> configs;
-  
   // dout() messages will be sent to stderr, but FCGX wants messages on stdout
   // Redirect stderr to stdout.
   TEMP_FAILURE_RETRY(close(STDERR_FILENO));
@@ -687,82 +720,11 @@ int main(int argc, const char **argv)
   auto cct = rgw_global_init(&defaults, args, CEPH_ENTITY_TYPE_CLIENT,
                             CODE_ENVIRONMENT_DAEMON, flags);
 
-  // First, let's determine which frontends are configured.
-  vector<string> frontends;
-  string rgw_frontends_str = g_conf().get_val<string>("rgw_frontends");
-  g_conf().early_expand_meta(rgw_frontends_str, &cerr);
-  get_str_vec(rgw_frontends_str, ",", frontends);
-  if (frontends.empty()) {
-    frontends.push_back("beast");
-  }
-  for (vector<string>::iterator iter = frontends.begin(); iter != frontends.end(); ++iter) {
-    string& f = *iter;
-
-    if (f.find("beast") != string::npos) {
-      if (f.find("port") != string::npos) {
-        // check for the most common ws problems
-        if ((f.find("port=") == string::npos) ||
-            (f.find("port= ") != string::npos)) {
-          derr << "WARNING: radosgw frontend config found unexpected spacing around 'port' "
-               << "(ensure frontend port parameter has the form 'port=80' with no spaces "
-               << "before or after '=')" << dendl;
-        }
-      }
-    }
-
-    RGWFrontendConfig *config = new RGWFrontendConfig(f);
-    int r = config->init();
-    if (r < 0) {
-      delete config;
-      cerr << "ERROR: failed to init config: " << f << std::endl;
-      return EINVAL;
-    }
-
-    configs.push_back(config);
-
-    string framework = config->get_framework();
-    fe_map.insert(pair<string, RGWFrontendConfig*>(framework, config));
-  } /* for each frontend */
-
-  int numa_node = g_conf().get_val<int64_t>("rgw_numa_node");
-  size_t numa_cpu_set_size = 0;
-  cpu_set_t numa_cpu_set;
-
-  if (numa_node >= 0) {
-    int r = get_numa_node_cpu_set(numa_node, &numa_cpu_set_size, &numa_cpu_set);
-    if (r < 0) {
-      dout(1) << __func__ << " unable to determine rgw numa node " << numa_node
-              << " CPUs" << dendl;
-      numa_node = -1;
-    } else {
-      r = set_cpu_affinity_all_threads(numa_cpu_set_size, &numa_cpu_set);
-      if (r < 0) {
-        derr << __func__ << " failed to set numa affinity: " << cpp_strerror(r)
-        << dendl;
-      }
-    }
-  } else {
-    dout(1) << __func__ << " not setting numa affinity" << dendl;
-  }
-
-  // maintain existing region root pool for new multisite objects
-  if (!g_conf()->rgw_region_root_pool.empty()) {
-    const char *root_pool = g_conf()->rgw_region_root_pool.c_str();
-    if (g_conf()->rgw_zonegroup_root_pool.empty()) {
-      g_conf().set_val_or_die("rgw_zonegroup_root_pool", root_pool);
-    }
-    if (g_conf()->rgw_period_root_pool.empty()) {
-      g_conf().set_val_or_die("rgw_period_root_pool", root_pool);
-    }
-    if (g_conf()->rgw_realm_root_pool.empty()) {
-      g_conf().set_val_or_die("rgw_realm_root_pool", root_pool);
-    }
-  }
+  DoutPrefix dp(cct.get(), dout_subsys, "rgw main: ");
+  rgw::AppMain main(&dp);
 
-  // for region -> zonegroup conversion (must happen before common_init_finish())
-  if (!g_conf()->rgw_region.empty() && g_conf()->rgw_zonegroup.empty()) {
-    g_conf().set_val_or_die("rgw_zonegroup", g_conf()->rgw_region.c_str());
-  }
+  main.init_frontends1(false /* nfs */);
+  main.init_numa();
 
   if (g_conf()->daemonize) {
     global_init_daemonize(g_ceph_context);
@@ -774,39 +736,25 @@ int main(int argc, const char **argv)
   init_timer.add_event_after(g_conf()->rgw_init_timeout, new C_InitTimeout);
   mutex.unlock();
 
-  ceph::crypto::init_openssl_engine_once();
-
   common_init_finish(g_ceph_context);
-
   init_async_signal_handler();
 
-  TracepointProvider::initialize<rgw_rados_tracepoint_traits>(g_ceph_context);
-  TracepointProvider::initialize<rgw_op_tracepoint_traits>(g_ceph_context);
-
-  const DoutPrefix dp(cct.get(), dout_subsys, "rgw main: ");
-  int r = rgw_tools_init(&dp, g_ceph_context);
+  /* XXXX check locations thru sighandler_alrm */
+  register_async_signal_handler(SIGHUP, rgw_sighup_handler);
+  r = signal_fd_init();
   if (r < 0) {
-    derr << "ERROR: unable to initialize rgw tools" << dendl;
-    return -r;
+    derr << "ERROR: unable to initialize signal fds" << dendl;
+  exit(1);
   }
-  tracing::rgw::tracer.init("rgw");
-  rgw_init_resolver();
-  rgw::curl::setup_curl(fe_map);
-  rgw_http_client_init(g_ceph_context);
-  rgw_kmip_client_init(*new RGWKMIPManagerImpl(g_ceph_context));
-  
-  StoreManager::Config cfg = StoreManager::get_config(false, g_ceph_context);
 
-  rgw::sal::Store* store =
-    StoreManager::get_storage(&dp, g_ceph_context,
-                                cfg,
-                                g_conf()->rgw_enable_gc_threads,
-                                g_conf()->rgw_enable_lc_threads,
-                                g_conf()->rgw_enable_quota_threads,
-                                g_conf()->rgw_run_sync_thread,
-                                g_conf().get_val<bool>("rgw_dynamic_resharding"),
-                                g_conf()->rgw_cache_enabled);
-  if (!store) {
+  register_async_signal_handler(SIGTERM, handle_sigterm);
+  register_async_signal_handler(SIGINT, handle_sigterm);
+  register_async_signal_handler(SIGUSR1, handle_sigterm);
+  sighandler_alrm = signal(SIGALRM, godown_alarm);
+
+
+  main.init_storage();
+  if (! main.get_store()) {
     mutex.lock();
     init_timer.cancel_all_events();
     init_timer.shutdown();
@@ -815,298 +763,22 @@ int main(int argc, const char **argv)
     derr << "Couldn't init storage provider (RADOS)" << dendl;
     return EIO;
   }
-  r = rgw_perf_start(g_ceph_context);
-  if (r < 0) {
-    derr << "ERROR: failed starting rgw perf" << dendl;
-    return -r;
-  }
 
-  rgw_rest_init(g_ceph_context, store->get_zone()->get_zonegroup());
+  main.init_perfcounters();
+  main.init_http_clients();
+  main.cond_init_apis();
 
   mutex.lock();
   init_timer.cancel_all_events();
   init_timer.shutdown();
   mutex.unlock();
 
-  rgw_log_usage_init(g_ceph_context, store);
-
-  RGWREST rest;
-
-  vector<string> apis;
-  get_str_vec(g_conf()->rgw_enable_apis, apis);
-
-  map<string, bool> apis_map;
-  for (vector<string>::iterator li = apis.begin(); li != apis.end(); ++li) {
-    apis_map[*li] = true;
-  }
-
-  /* warn about insecure keystone secret config options */
-  if (!(g_ceph_context->_conf->rgw_keystone_admin_token.empty() ||
-       g_ceph_context->_conf->rgw_keystone_admin_password.empty())) {
-    dout(0) << "WARNING: rgw_keystone_admin_token and rgw_keystone_admin_password should be avoided as they can expose secrets.  Prefer the new rgw_keystone_admin_token_path and rgw_keystone_admin_password_path options, which read their secrets from files." << dendl;
-  }
-
-  // S3 website mode is a specialization of S3
-  const bool s3website_enabled = apis_map.count("s3website") > 0;
-  const bool sts_enabled = apis_map.count("sts") > 0;
-  const bool iam_enabled = apis_map.count("iam") > 0;
-  const bool pubsub_enabled = apis_map.count("pubsub") > 0 || apis_map.count("notifications") > 0;
-  // Swift API entrypoint could placed in the root instead of S3
-  const bool swift_at_root = g_conf()->rgw_swift_url_prefix == "/";
-  if (apis_map.count("s3") > 0 || s3website_enabled) {
-    if (! swift_at_root) {
-      rest.register_default_mgr(set_logging(rest_filter(store, RGW_REST_S3,
-                                                        new RGWRESTMgr_S3(s3website_enabled, sts_enabled, iam_enabled, pubsub_enabled))));
-    } else {
-      derr << "Cannot have the S3 or S3 Website enabled together with "
-           << "Swift API placed in the root of hierarchy" << dendl;
-      return EINVAL;
-    }
-  }
-
-  if (pubsub_enabled) {
-#ifdef WITH_RADOSGW_AMQP_ENDPOINT
-    if (!rgw::amqp::init(cct.get())) {
-        dout(1) << "ERROR: failed to initialize AMQP manager" << dendl;
-    }
-#endif
-#ifdef WITH_RADOSGW_KAFKA_ENDPOINT
-    if (!rgw::kafka::init(cct.get())) {
-        dout(1) << "ERROR: failed to initialize Kafka manager" << dendl;
-    }
-#endif
-  }
-
-  const auto& luarocks_path = g_conf().get_val<std::string>("rgw_luarocks_location");
-  if (luarocks_path.empty()) {
-    store->set_luarocks_path("");
-  } else {
-    store->set_luarocks_path(luarocks_path+"/"+g_conf()->name.to_str());
-  }
-#ifdef WITH_RADOSGW_LUA_PACKAGES
-  rgw::lua::packages_t failed_packages;
-  std::string output;
-  r = rgw::lua::install_packages(&dp, store, null_yield, failed_packages, output);
-  if (r < 0) {
-    dout(1) << "WARNING: failed to install lua packages from allowlist" << dendl;
-  }
-  if (!output.empty()) {
-    dout(10) << "INFO: lua packages installation output: \n" << output << dendl; 
-  }
-  for (const auto& p : failed_packages) {
-    dout(5) << "WARNING: failed to install lua package: " << p << " from allowlist" << dendl;
-  }
-#endif
-
-  if (apis_map.count("swift") > 0) {
-    RGWRESTMgr_SWIFT* const swift_resource = new RGWRESTMgr_SWIFT;
-
-    if (! g_conf()->rgw_cross_domain_policy.empty()) {
-      swift_resource->register_resource("crossdomain.xml",
-                          set_logging(new RGWRESTMgr_SWIFT_CrossDomain));
-    }
-
-    swift_resource->register_resource("healthcheck",
-                          set_logging(new RGWRESTMgr_SWIFT_HealthCheck));
-
-    swift_resource->register_resource("info",
-                          set_logging(new RGWRESTMgr_SWIFT_Info));
-
-    if (! swift_at_root) {
-      rest.register_resource(g_conf()->rgw_swift_url_prefix,
-                          set_logging(rest_filter(store, RGW_REST_SWIFT,
-                                                  swift_resource)));
-    } else {
-      if (store->get_zone()->get_zonegroup().get_zone_count() > 1) {
-        derr << "Placing Swift API in the root of URL hierarchy while running"
-             << " multi-site configuration requires another instance of RadosGW"
-             << " with S3 API enabled!" << dendl;
-      }
-
-      rest.register_default_mgr(set_logging(swift_resource));
-    }
-  }
-
-  if (apis_map.count("swift_auth") > 0) {
-    rest.register_resource(g_conf()->rgw_swift_auth_entry,
-               set_logging(new RGWRESTMgr_SWIFT_Auth));
-  }
-
-  if (apis_map.count("admin") > 0) {
-    RGWRESTMgr_Admin *admin_resource = new RGWRESTMgr_Admin;
-    admin_resource->register_resource("info", new RGWRESTMgr_Info);
-    admin_resource->register_resource("usage", new RGWRESTMgr_Usage);
-    admin_resource->register_resource("user", new RGWRESTMgr_User);
-  
-    /* Register store-specific admin APIs */
-    store->register_admin_apis(admin_resource);
-    rest.register_resource(g_conf()->rgw_admin_entry, admin_resource);
-  }
-
-  /* Initialize the registry of auth strategies which will coordinate
-   * the dynamic reconfiguration. */
-  rgw::auth::ImplicitTenants implicit_tenant_context{g_conf()};
-  g_conf().add_observer(&implicit_tenant_context);
-  auto auth_registry = \
-    rgw::auth::StrategyRegistry::create(g_ceph_context, implicit_tenant_context, store);
-
-  /* Header custom behavior */
-  rest.register_x_headers(g_conf()->rgw_log_http_headers);
-
-  if (cct->_conf.get_val<std::string>("rgw_scheduler_type") == "dmclock" &&
-      !cct->check_experimental_feature_enabled("dmclock")){
-    derr << "dmclock scheduler type is experimental and needs to be"
-        << "set in the option enable experimental data corrupting features"
-        << dendl;
-    return EINVAL;
-  }
-
-  rgw::dmclock::SchedulerCtx sched_ctx{cct.get()};
-
-  OpsLogManifold *olog = new OpsLogManifold();
-  ActiveRateLimiter ratelimiting{cct.get()};
-  ratelimiting.start();
-
-  if (!g_conf()->rgw_ops_log_socket_path.empty()) {
-    OpsLogSocket* olog_socket = new OpsLogSocket(g_ceph_context, g_conf()->rgw_ops_log_data_backlog);
-    olog_socket->init(g_conf()->rgw_ops_log_socket_path);
-    olog->add_sink(olog_socket);
-  }
-  if (!g_conf()->rgw_ops_log_file_path.empty()) {
-    ops_log_file = new OpsLogFile(g_ceph_context, g_conf()->rgw_ops_log_file_path, g_conf()->rgw_ops_log_data_backlog);
-    ops_log_file->start();
-    olog->add_sink(ops_log_file);
-  }
-  register_async_signal_handler(SIGHUP, rgw_sighup_handler);
-  olog->add_sink(new OpsLogRados(store));
-
-  r = signal_fd_init();
-  if (r < 0) {
-    derr << "ERROR: unable to initialize signal fds" << dendl;
-    exit(1);
-  }
-
-  register_async_signal_handler(SIGTERM, handle_sigterm);
-  register_async_signal_handler(SIGINT, handle_sigterm);
-  register_async_signal_handler(SIGUSR1, handle_sigterm);
-  sighandler_alrm = signal(SIGALRM, godown_alarm);
-
-  map<string, string> service_map_meta;
-  service_map_meta["pid"] = stringify(getpid());
-
-  vector<RGWFrontend *> fes;
-
-  string frontend_defs_str = g_conf().get_val<string>("rgw_frontend_defaults");
-
-  vector<string> frontends_def;
-  get_str_vec(frontend_defs_str, ",", frontends_def);
-
-  map<string, std::unique_ptr<RGWFrontendConfig> > fe_def_map;
-  for (auto& f : frontends_def) {
-    RGWFrontendConfig *config = new RGWFrontendConfig(f);
-    int r = config->init();
-    if (r < 0) {
-      delete config;
-      cerr << "ERROR: failed to init default config: " << f << std::endl;
-      return EINVAL;
-    }
-
-    fe_def_map[config->get_framework()].reset(config);
-  }
-
-  int fe_count = 0;
-
-  std::unique_ptr<rgw::lua::Background> lua_background;
-  if (store->get_name() == "rados") { /* Supported for only RadosStore */
-    lua_background = std::make_unique<rgw::lua::Background>(store, cct.get(), store->get_luarocks_path());
-    lua_background->start();
-  }
-
-  for (multimap<string, RGWFrontendConfig *>::iterator fiter = fe_map.begin();
-       fiter != fe_map.end(); ++fiter, ++fe_count) {
-    RGWFrontendConfig *config = fiter->second;
-    string framework = config->get_framework();
-
-    auto def_iter = fe_def_map.find(framework);
-    if (def_iter != fe_def_map.end()) {
-      config->set_default_config(*def_iter->second);
-    }
-
-    RGWFrontend *fe = NULL;
-
-    if (framework == "loadgen") {
-      int port;
-      config->get_val("port", 80, &port);
-      std::string uri_prefix;
-      config->get_val("prefix", "", &uri_prefix);
-
-      RGWProcessEnv env = { store, &rest, olog, port, uri_prefix, 
-                            auth_registry, &ratelimiting, lua_background.get()};
-
-      fe = new RGWLoadGenFrontend(env, config);
-    }
-    else if (framework == "beast") {
-      int port;
-      config->get_val("port", 80, &port);
-      std::string uri_prefix;
-      config->get_val("prefix", "", &uri_prefix);
-      RGWProcessEnv env{ store, &rest, olog, port, uri_prefix, 
-                         auth_registry, &ratelimiting, lua_background.get()};
-      fe = new RGWAsioFrontend(env, config, sched_ctx);
-    }
-
-    service_map_meta["frontend_type#" + stringify(fe_count)] = framework;
-    service_map_meta["frontend_config#" + stringify(fe_count)] = config->get_config();
-
-    if (fe == NULL) {
-      dout(0) << "WARNING: skipping unknown framework: " << framework << dendl;
-      continue;
-    }
-
-    dout(0) << "starting handler: " << fiter->first << dendl;
-    int r = fe->init();
-    if (r < 0) {
-      derr << "ERROR: failed initializing frontend" << dendl;
-      return -r;
-    }
-    r = fe->run();
-    if (r < 0) {
-      derr << "ERROR: failed run" << dendl;
-      return -r;
-    }
-
-    fes.push_back(fe);
-  }
-
-  r = store->register_to_service_map(&dp, "rgw", service_map_meta);
-  if (r < 0) {
-    derr << "ERROR: failed to register to service map: " << cpp_strerror(-r) << dendl;
-
-    /* ignore error */
-  }
-
-
-  std::unique_ptr<RGWRealmReloader> reloader;
-  std::unique_ptr<RGWPeriodPusher> pusher;
-  std::unique_ptr<RGWFrontendPauser> fe_pauser;
-  std::unique_ptr<RGWRealmWatcher> realm_watcher;
-  std::unique_ptr<RGWPauser> rgw_pauser;
-  if (store->get_name() == "rados") {
-    // add a watcher to respond to realm configuration changes
-    pusher = std::make_unique<RGWPeriodPusher>(&dp, store, null_yield);
-    fe_pauser = std::make_unique<RGWFrontendPauser>(fes, implicit_tenant_context, pusher.get());
-    rgw_pauser = std::make_unique<RGWPauser>();
-    rgw_pauser->add_pauser(fe_pauser.get());
-    if (lua_background) {
-      rgw_pauser->add_pauser(lua_background.get());
-    }
-    reloader = std::make_unique<RGWRealmReloader>(store, service_map_meta, rgw_pauser.get());
-
-    realm_watcher = std::make_unique<RGWRealmWatcher>(&dp, g_ceph_context,
-                                 static_cast<rgw::sal::RadosStore*>(store)->svc()->zone->get_realm());
-    realm_watcher->add_watcher(RGWRealmNotify::Reload, *reloader);
-    realm_watcher->add_watcher(RGWRealmNotify::ZonesNeedPeriod, *pusher.get());
-  }
+  main.init_ldap();
+  main.init_opslog();
+  main.init_tracepoints();
+  main.init_frontends2(nullptr /* RGWLib */);
+  main.init_notification_endpoints();
+  main.init_lua();
 
 #if defined(HAVE_SYS_PRCTL_H)
   if (prctl(PR_SET_DUMPABLE, 1) == -1) {
@@ -1117,29 +789,7 @@ int main(int argc, const char **argv)
   wait_shutdown();
 
   derr << "shutting down" << dendl;
-
-  if (store->get_name() == "rados") {
-    reloader.reset(); // stop the realm reloader
-  }
-
-  for (vector<RGWFrontend *>::iterator liter = fes.begin(); liter != fes.end();
-       ++liter) {
-    RGWFrontend *fe = *liter;
-    fe->stop();
-  }
-
-  for (vector<RGWFrontend *>::iterator liter = fes.begin(); liter != fes.end();
-       ++liter) {
-    RGWFrontend *fe = *liter;
-    fe->join();
-    delete fe;
-  }
-
-  for (vector<RGWFrontendConfig *>::iterator liter = configs.begin();
-       liter != configs.end(); ++liter) {
-    RGWFrontendConfig *fec = *liter;
-    delete fec;
-  }
+  main.shutdown();
 
   unregister_async_signal_handler(SIGHUP, rgw_sighup_handler);
   unregister_async_signal_handler(SIGTERM, handle_sigterm);
@@ -1147,31 +797,6 @@ int main(int argc, const char **argv)
   unregister_async_signal_handler(SIGUSR1, handle_sigterm);
   shutdown_async_signal_handler();
 
-  rgw_log_usage_finalize();
-  delete olog;
-
-  if (lua_background) {
-    lua_background->shutdown();
-  }
-
-  StoreManager::close_storage(store);
-  rgw::auth::s3::LDAPEngine::shutdown();
-  rgw_tools_cleanup();
-  rgw_shutdown_resolver();
-  rgw_http_client_cleanup();
-  rgw_kmip_client_cleanup();
-  rgw::curl::cleanup_curl();
-  g_conf().remove_observer(&implicit_tenant_context);
-#ifdef WITH_RADOSGW_AMQP_ENDPOINT
-  rgw::amqp::shutdown();
-#endif
-#ifdef WITH_RADOSGW_KAFKA_ENDPOINT
-  rgw::kafka::shutdown();
-#endif
-
-
-  rgw_perf_stop(g_ceph_context);
-
   dout(1) << "final shutdown" << dendl;
 
   signal_fd_finalize();
index 432c58bb49085fa10ebb6eb777d1c4d74f80adac..c56e638397f7a0727d74f6aa53082e4c1827bc34 100644 (file)
@@ -78,6 +78,7 @@ namespace rgw {
     rgw::sal::Store* store;
     DoutPrefixProvider* dpp;
 
+    friend int main(int, char*[]); // doesnt work gcc-12.1
     friend class RGWLib;
 
   public:
@@ -87,7 +88,12 @@ namespace rgw {
 
     void shutdown();
 
+    rgw::sal::Store* get_store() {
+      return store;
+    }
+
     void init_frontends1(bool nfs = false);
+    void init_numa();
     void init_storage();
     void init_perfcounters();
     void init_http_clients();