]> git.proxmox.com Git - ceph.git/blobdiff - ceph/src/crimson/osd/osd.h
update ceph source to reef 18.1.2
[ceph.git] / ceph / src / crimson / osd / osd.h
index 7970d8c9fe38a2605a3d34a50dd13cf11a2a381c..b3933e80c268cba7e4433a362867ca7136a49705 100644 (file)
@@ -3,6 +3,7 @@
 
 #pragma once
 
+#include <seastar/core/abort_source.hh>
 #include <seastar/core/future.hh>
 #include <seastar/core/shared_future.hh>
 #include <seastar/core/gate.hh>
 #include "crimson/common/gated.h"
 #include "crimson/admin/admin_socket.h"
 #include "crimson/common/simple_lru.h"
-#include "crimson/common/shared_lru.h"
 #include "crimson/mgr/client.h"
 #include "crimson/net/Dispatcher.h"
 #include "crimson/osd/osdmap_service.h"
-#include "crimson/osd/state.h"
-#include "crimson/osd/shard_services.h"
+#include "crimson/osd/pg_shard_manager.h"
 #include "crimson/osd/osdmap_gate.h"
 #include "crimson/osd/pg_map.h"
 #include "crimson/osd/osd_operations/peering_event.h"
@@ -37,7 +36,6 @@ class MOSDMap;
 class MOSDRepOpReply;
 class MOSDRepOp;
 class MOSDScrub2;
-class OSDMap;
 class OSDMeta;
 class Heartbeat;
 
@@ -61,32 +59,30 @@ namespace crimson::osd {
 class PG;
 
 class OSD final : public crimson::net::Dispatcher,
-                 private OSDMapService,
                  private crimson::common::AuthHandler,
                  private crimson::mgr::WithStats {
   const int whoami;
   const uint32_t nonce;
+  seastar::abort_source& abort_source;
   seastar::timer<seastar::lowres_clock> beacon_timer;
   // talk with osd
   crimson::net::MessengerRef cluster_msgr;
   // talk with client/mon/mgr
   crimson::net::MessengerRef public_msgr;
+
+  // HB Messengers
+  crimson::net::MessengerRef hb_front_msgr;
+  crimson::net::MessengerRef hb_back_msgr;
+
   std::unique_ptr<crimson::mon::Client> monc;
   std::unique_ptr<crimson::mgr::Client> mgrc;
 
-  SharedLRU<epoch_t, OSDMap> osdmaps;
-  SimpleLRU<epoch_t, bufferlist, false> map_bl_cache;
-  cached_map_t osdmap;
   // TODO: use a wrapper for ObjectStore
+  OSDMapService::cached_map_t osdmap;
   crimson::os::FuturizedStore& store;
-  std::unique_ptr<OSDMeta> meta_coll;
-
-  OSDState state;
 
   /// _first_ epoch we were marked up (after this process started)
   epoch_t boot_epoch = 0;
-  /// _most_recent_ epoch we were marked up
-  epoch_t up_epoch = 0;
   //< epoch we last did a bind to new ip:ports
   epoch_t bind_epoch = 0;
   //< since when there is no more pending pg creates from mon
@@ -106,13 +102,13 @@ class OSD final : public crimson::net::Dispatcher,
   osd_stat_t osd_stat;
   uint32_t osd_stat_seq = 0;
   void update_stats();
-  MessageURef get_stats() const final;
+  seastar::future<MessageURef> get_stats() const final;
 
   // AuthHandler methods
   void handle_authentication(const EntityName& name,
                             const AuthCapsInfo& caps) final;
 
-  crimson::osd::ShardServices shard_services;
+  crimson::osd::PGShardManager pg_shard_manager;
 
   std::unique_ptr<Heartbeat> heartbeat;
   seastar::timer<seastar::lowres_clock> tick_timer;
@@ -122,6 +118,7 @@ class OSD final : public crimson::net::Dispatcher,
 
 public:
   OSD(int id, uint32_t nonce,
+      seastar::abort_source& abort_source,
       crimson::os::FuturizedStore& store,
       crimson::net::MessengerRef cluster_msgr,
       crimson::net::MessengerRef client_msgr,
@@ -129,49 +126,39 @@ public:
       crimson::net::MessengerRef hb_back_msgr);
   ~OSD() final;
 
-  seastar::future<> mkfs(uuid_d osd_uuid, uuid_d cluster_fsid);
+  seastar::future<> open_meta_coll();
+  static seastar::future<OSDMeta> open_or_create_meta_coll(
+    crimson::os::FuturizedStore &store
+  );
+  static seastar::future<> mkfs(
+    crimson::os::FuturizedStore &store,
+    unsigned whoami,
+    uuid_d osd_uuid,
+    uuid_d cluster_fsid,
+    std::string osdspec_affinity);
 
   seastar::future<> start();
   seastar::future<> stop();
 
   void dump_status(Formatter*) const;
-  void dump_pg_state_history(Formatter*) const;
   void print(std::ostream&) const;
 
-  seastar::future<> send_incremental_map(crimson::net::ConnectionRef conn,
-                                        epoch_t first);
-
   /// @return the seq id of the pg stats being sent
   uint64_t send_pg_stats();
 
 private:
-  seastar::future<> _write_superblock();
-  seastar::future<> _write_key_meta();
+  static seastar::future<> _write_superblock(
+    crimson::os::FuturizedStore &store,
+    OSDMeta meta,
+    OSDSuperblock superblock);
+  static seastar::future<> _write_key_meta(
+    crimson::os::FuturizedStore &store
+  );
   seastar::future<> start_boot();
   seastar::future<> _preboot(version_t oldest_osdmap, version_t newest_osdmap);
   seastar::future<> _send_boot();
   seastar::future<> _add_me_to_crush();
 
-  seastar::future<Ref<PG>> make_pg(cached_map_t create_map,
-                                  spg_t pgid,
-                                  bool do_create);
-  seastar::future<Ref<PG>> load_pg(spg_t pgid);
-  seastar::future<> load_pgs();
-
-  // OSDMapService methods
-  epoch_t get_up_epoch() const final {
-    return up_epoch;
-  }
-  seastar::future<cached_map_t> get_map(epoch_t e) final;
-  cached_map_t get_map() const final;
-  seastar::future<std::unique_ptr<OSDMap>> load_map(epoch_t e);
-  seastar::future<bufferlist> load_map_bl(epoch_t e);
-  seastar::future<std::map<epoch_t, bufferlist>>
-  load_map_bls(epoch_t first, epoch_t last);
-  void store_map_bl(ceph::os::Transaction& t,
-                    epoch_t e, bufferlist&& bl);
-  seastar::future<> store_maps(ceph::os::Transaction& t,
-                               epoch_t start, Ref<MOSDMap> m);
   seastar::future<> osdmap_subscribe(version_t epoch, bool force_request);
 
   void write_superblock(ceph::os::Transaction& t);
@@ -179,11 +166,10 @@ private:
 
   bool require_mon_peer(crimson::net::Connection *conn, Ref<Message> m);
 
-  seastar::future<Ref<PG>> handle_pg_create_info(
-    std::unique_ptr<PGCreateInfo> info);
-
   seastar::future<> handle_osd_map(crimson::net::ConnectionRef conn,
                                    Ref<MOSDMap> m);
+  seastar::future<> handle_pg_create(crimson::net::ConnectionRef conn,
+                                    Ref<MOSDPGCreate2> m);
   seastar::future<> handle_osd_op(crimson::net::ConnectionRef conn,
                                  Ref<MOSDOp> m);
   seastar::future<> handle_rep_op(crimson::net::ConnectionRef conn,
@@ -208,18 +194,20 @@ private:
   seastar::future<> handle_command(crimson::net::ConnectionRef conn,
                                   Ref<MCommand> m);
   seastar::future<> start_asok_admin();
-
+  seastar::future<> handle_update_log_missing(crimson::net::ConnectionRef conn,
+                                              Ref<MOSDPGUpdateLogMissing> m);
+  seastar::future<> handle_update_log_missing_reply(
+    crimson::net::ConnectionRef conn,
+    Ref<MOSDPGUpdateLogMissingReply> m);
 public:
-  OSDMapGate osdmap_gate;
-
+  auto &get_pg_shard_manager() {
+    return pg_shard_manager;
+  }
   ShardServices &get_shard_services() {
-    return shard_services;
+    return pg_shard_manager.get_shard_services();
   }
 
-  seastar::future<> consume_map(epoch_t epoch);
-
 private:
-  PGMap pg_map;
   crimson::common::Gated gate;
 
   seastar::promise<> stop_acked;
@@ -230,20 +218,10 @@ private:
   bool should_restart() const;
   seastar::future<> restart();
   seastar::future<> shutdown();
-  void update_heartbeat_peers();
+  seastar::future<> update_heartbeat_peers();
   friend class PGAdvanceMap;
 
-  RemotePeeringEvent::OSDPipeline peering_request_osd_pipeline;
-  friend class RemotePeeringEvent;
-
 public:
-  blocking_future<Ref<PG>> get_or_create_pg(
-    spg_t pgid,
-    epoch_t epoch,
-    std::unique_ptr<PGCreateInfo> info);
-  blocking_future<Ref<PG>> wait_for_pg(
-    spg_t pgid);
-  Ref<PG> get_pg(spg_t pgid);
   seastar::future<> send_beacon();
 
 private:
@@ -257,3 +235,7 @@ inline std::ostream& operator<<(std::ostream& out, const OSD& osd) {
 }
 
 }
+
+#if FMT_VERSION >= 90000
+template <> struct fmt::formatter<crimson::osd::OSD> : fmt::ostream_formatter {};
+#endif