]> git.proxmox.com Git - ceph.git/blobdiff - ceph/src/osd/PG.h
import quincy beta 17.1.0
[ceph.git] / ceph / src / osd / PG.h
index cad031c10861fb0f572051d6402ffa96f0a4cb76..59469602405707ac79535b0c712a77baea5fa294 100644 (file)
@@ -33,7 +33,6 @@
 
 #include "PGLog.h"
 #include "OSDMap.h"
-#include "messages/MOSDPGLog.h"
 #include "include/str_list.h"
 #include "PGBackend.h"
 #include "PGPeeringEvent.h"
@@ -57,14 +56,10 @@ class OSD;
 class OSDService;
 class OSDShard;
 class OSDShardPGSlot;
-class MOSDPGScan;
-class MOSDPGBackfill;
-class MOSDPGInfo;
 
 class PG;
 struct OpRequest;
 typedef OpRequest::Ref OpRequestRef;
-class MOSDPGLog;
 class DynamicPerfStats;
 class PgScrubber;
 
@@ -73,6 +68,7 @@ namespace Scrub {
   class ReplicaReservations;
   class LocalReservation;
   class ReservedByRemotePrimary;
+  enum class schedule_result_t;
 }
 
 #ifdef PG_DEBUG_REFS
@@ -166,26 +162,39 @@ class PGRecoveryStats {
  *
  */
 
+/// Facilitating scrub-realated object access to private PG data
+class ScrubberPasskey {
+private:
+  friend class Scrub::ReplicaReservations;
+  friend class PrimaryLogScrub;
+  ScrubberPasskey() {}
+  ScrubberPasskey(const ScrubberPasskey&) = default;
+  ScrubberPasskey& operator=(const ScrubberPasskey&) = delete;
+};
+
 class PG : public DoutPrefixProvider, public PeeringState::PeeringListener {
   friend struct NamedState;
   friend class PeeringState;
   friend class PgScrubber;
-  friend class PrimaryLogScrub;
-  friend class Scrub::ReplicaReservations;
-  friend class Scrub::LocalReservation;  // dout()-only friendship
-  friend class Scrub::ReservedByRemotePrimary;  //  dout()-only friendship
 
 public:
   const pg_shard_t pg_whoami;
   const spg_t pg_id;
 
+  /// the 'scrubber'. Will be allocated in the derivative (PrimaryLogPG) ctor,
+  /// and be removed only in the PrimaryLogPG destructor.
   std::unique_ptr<ScrubPgIF> m_scrubber;
 
   /// flags detailing scheduling/operation characteristics of the next scrub 
   requested_scrub_t m_planned_scrub;
+
   /// scrubbing state for both Primary & replicas
   bool is_scrub_active() const { return m_scrubber->is_scrub_active(); }
 
+  /// set when the scrub request is queued, and reset after scrubbing fully
+  /// cleaned up.
+  bool is_scrub_queued_or_active() const { return m_scrubber->is_queued_or_active(); }
+
 public:
   // -- members --
   const coll_t coll;
@@ -273,6 +282,31 @@ public:
       });
   }
 
+  static void add_objects_scrubbed_count(
+    int64_t count, pg_stat_t &stats) {
+    stats.objects_scrubbed += count;
+  }
+
+  void add_objects_scrubbed_count(int64_t count) {
+    recovery_state.update_stats(
+      [=](auto &history, auto &stats) {
+       add_objects_scrubbed_count(count, stats);
+       return true;
+      });
+  }
+
+  static void reset_objects_scrubbed(pg_stat_t &stats) {
+    stats.objects_scrubbed = 0;
+  }
+
+  void reset_objects_scrubbed() {
+    recovery_state.update_stats(
+      [=](auto &history, auto &stats) {
+  reset_objects_scrubbed(stats);
+  return true;
+      });
+  }
+
   bool is_deleting() const {
     return recovery_state.is_deleting();
   }
@@ -342,7 +376,6 @@ public:
     int acting_primary,
     const pg_history_t& history,
     const PastIntervals& pim,
-    bool backfill,
     ObjectStore::Transaction &t);
 
   /// read existing pg state off disk
@@ -380,28 +413,115 @@ public:
   void finish_split_stats(const object_stat_sum_t& stats,
                          ObjectStore::Transaction &t);
 
-  void scrub(epoch_t queued, ThreadPool::TPHandle &handle);
+  void scrub(epoch_t queued, ThreadPool::TPHandle& handle)
+  {
+    // a new scrub
+    forward_scrub_event(&ScrubPgIF::initiate_regular_scrub, queued, "StartScrub");
+  }
+
   /**
    *  a special version of PG::scrub(), which:
    *  - is initiated after repair, and
+   * (not true anymore:)
    *  - is not required to allocate local/remote OSD scrub resources
    */
-  void recovery_scrub(epoch_t queued, ThreadPool::TPHandle &handle);
-  void replica_scrub(epoch_t queued, ThreadPool::TPHandle &handle);
-  void replica_scrub_resched(epoch_t queued, ThreadPool::TPHandle &handle);
-
-  /// Queues a PGScrubResourcesOK message. Will translate into 'RemotesReserved' FSM event
-  void scrub_send_resources_granted(epoch_t queued, ThreadPool::TPHandle &handle);
-  void scrub_send_resources_denied(epoch_t queued, ThreadPool::TPHandle &handle);
-  void scrub_send_scrub_resched(epoch_t queued, ThreadPool::TPHandle &handle);
-  void scrub_send_pushes_update(epoch_t queued, ThreadPool::TPHandle &handle);
-  void scrub_send_applied_update(epoch_t queued, ThreadPool::TPHandle &handle);
-  void scrub_send_unblocking(epoch_t epoch_queued, ThreadPool::TPHandle &handle);
-  void scrub_send_digest_update(epoch_t epoch_queued, ThreadPool::TPHandle &handle);
-  void scrub_send_replmaps_ready(epoch_t epoch_queued, ThreadPool::TPHandle &handle);
-  void scrub_send_replica_pushes(epoch_t queued, ThreadPool::TPHandle &handle);
-
-  void reg_next_scrub();
+  void recovery_scrub(epoch_t queued, ThreadPool::TPHandle& handle)
+  {
+    // a new scrub
+    forward_scrub_event(&ScrubPgIF::initiate_scrub_after_repair, queued,
+                       "AfterRepairScrub");
+  }
+
+  void replica_scrub(epoch_t queued,
+                    Scrub::act_token_t act_token,
+                    ThreadPool::TPHandle& handle);
+
+  void replica_scrub_resched(epoch_t queued,
+                            Scrub::act_token_t act_token,
+                            ThreadPool::TPHandle& handle)
+  {
+    forward_scrub_event(&ScrubPgIF::send_sched_replica, queued, act_token,
+                       "SchedReplica");
+  }
+
+  void scrub_send_resources_granted(epoch_t queued, ThreadPool::TPHandle& handle)
+  {
+    forward_scrub_event(&ScrubPgIF::send_remotes_reserved, queued, "RemotesReserved");
+  }
+
+  void scrub_send_resources_denied(epoch_t queued, ThreadPool::TPHandle& handle)
+  {
+    forward_scrub_event(&ScrubPgIF::send_reservation_failure, queued,
+                       "ReservationFailure");
+  }
+
+  void scrub_send_scrub_resched(epoch_t queued, ThreadPool::TPHandle& handle)
+  {
+    forward_scrub_event(&ScrubPgIF::send_scrub_resched, queued, "InternalSchedScrub");
+  }
+
+  void scrub_send_pushes_update(epoch_t queued, ThreadPool::TPHandle& handle)
+  {
+    forward_scrub_event(&ScrubPgIF::active_pushes_notification, queued,
+                       "ActivePushesUpd");
+  }
+
+  void scrub_send_applied_update(epoch_t queued, ThreadPool::TPHandle& handle)
+  {
+    forward_scrub_event(&ScrubPgIF::update_applied_notification, queued,
+                       "UpdatesApplied");
+  }
+
+  void scrub_send_unblocking(epoch_t queued, ThreadPool::TPHandle& handle)
+  {
+    forward_scrub_event(&ScrubPgIF::send_scrub_unblock, queued, "Unblocked");
+  }
+
+  void scrub_send_digest_update(epoch_t queued, ThreadPool::TPHandle& handle)
+  {
+    forward_scrub_event(&ScrubPgIF::digest_update_notification, queued, "DigestUpdate");
+  }
+
+  void scrub_send_local_map_ready(epoch_t queued, ThreadPool::TPHandle& handle)
+  {
+    forward_scrub_event(&ScrubPgIF::send_local_map_done, queued, "IntLocalMapDone");
+  }
+
+  void scrub_send_replmaps_ready(epoch_t queued, ThreadPool::TPHandle& handle)
+  {
+    forward_scrub_event(&ScrubPgIF::send_replica_maps_ready, queued, "GotReplicas");
+  }
+
+  void scrub_send_replica_pushes(epoch_t queued, ThreadPool::TPHandle& handle)
+  {
+    forward_scrub_event(&ScrubPgIF::send_replica_pushes_upd, queued,
+                       "ReplicaPushesUpd");
+  }
+
+  void scrub_send_maps_compared(epoch_t queued, ThreadPool::TPHandle& handle)
+  {
+    forward_scrub_event(&ScrubPgIF::send_maps_compared, queued, "MapsCompared");
+  }
+
+  void scrub_send_get_next_chunk(epoch_t queued, ThreadPool::TPHandle& handle)
+  {
+    forward_scrub_event(&ScrubPgIF::send_get_next_chunk, queued, "NextChunk");
+  }
+
+  void scrub_send_scrub_is_finished(epoch_t queued, ThreadPool::TPHandle& handle)
+  {
+    forward_scrub_event(&ScrubPgIF::send_scrub_is_finished, queued, "ScrubFinished");
+  }
+
+  void scrub_send_chunk_free(epoch_t queued, ThreadPool::TPHandle& handle)
+  {
+    forward_scrub_event(&ScrubPgIF::send_chunk_free, queued, "SelectedChunkFree");
+  }
+
+  void scrub_send_chunk_busy(epoch_t queued, ThreadPool::TPHandle& handle)
+  {
+    forward_scrub_event(&ScrubPgIF::send_chunk_busy, queued, "ChunkIsBusy");
+  }
 
   void queue_want_pg_temp(const std::vector<int> &wanted) override;
   void clear_want_pg_temp() override;
@@ -417,6 +537,10 @@ public:
 
   void on_info_history_change() override;
 
+  void on_primary_status_change(bool was_primary, bool now_primary) override;
+
+  void reschedule_scrub() override;
+
   void scrub_requested(scrub_level_t scrub_level, scrub_type_t scrub_type) override;
 
   uint64_t get_snap_trimq_size() const override {
@@ -533,14 +657,14 @@ public:
   void dump_pgstate_history(ceph::Formatter *f);
   void dump_missing(ceph::Formatter *f);
 
-  void get_pg_stats(std::function<void(const pg_stat_t&, epoch_t lec)> f);
-  void with_heartbeat_peers(std::function<void(int)> f);
+  void with_pg_stats(std::function<void(const pg_stat_t&, epoch_t lec)>&& f);
+  void with_heartbeat_peers(std::function<void(int)>&& f);
 
   void shutdown();
   virtual void on_shutdown() = 0;
 
   bool get_must_scrub() const;
-  bool sched_scrub();
+  Scrub::schedule_result_t sched_scrub();
 
   unsigned int scrub_requeue_priority(Scrub::scrub_prio_t with_priority, unsigned int suggested_priority) const;
   /// the version that refers to flags_.priority
@@ -570,7 +694,14 @@ private:
                                  requested_scrub_t& planned) const;
 
   using ScrubAPI = void (ScrubPgIF::*)(epoch_t epoch_queued);
-  void forward_scrub_event(ScrubAPI fn, epoch_t epoch_queued);
+  void forward_scrub_event(ScrubAPI fn, epoch_t epoch_queued, std::string_view desc);
+  // and for events that carry a meaningful 'activation token'
+  using ScrubSafeAPI = void (ScrubPgIF::*)(epoch_t epoch_queued,
+                                          Scrub::act_token_t act_token);
+  void forward_scrub_event(ScrubSafeAPI fn,
+                          epoch_t epoch_queued,
+                          Scrub::act_token_t act_token,
+                          std::string_view desc);
 
 public:
   virtual void do_request(
@@ -604,8 +735,6 @@ public:
     void complete(int r) override;
   };
 
-  void _delete_some(ObjectStore::Transaction *t);
-
   virtual void set_dynamic_perf_stats_queries(
     const std::list<OSDPerfMetricQuery> &queries) {
   }
@@ -707,7 +836,6 @@ protected:
   /* You should not use these items without taking their respective queue locks
    * (if they have one) */
   xlist<PG*>::item stat_queue_item;
-  bool scrub_queued;
   bool recovery_queued;
 
   int recovery_ops_active;
@@ -719,7 +847,7 @@ protected:
 public:
   bool dne() { return info.dne(); }
 
-  virtual void send_cluster_message(
+  void send_cluster_message(
     int osd, MessageRef m, epoch_t epoch, bool share_map_update) override;
 
 protected:
@@ -947,8 +1075,7 @@ protected:
   // publish stats
   ceph::mutex pg_stats_publish_lock =
     ceph::make_mutex("PG::pg_stats_publish_lock");
-  bool pg_stats_publish_valid;
-  pg_stat_t pg_stats_publish;
+  std::optional<pg_stat_t> pg_stats_publish;
 
   friend class TestOpsSocketHook;
   void publish_stats_to_osd() override;
@@ -1146,7 +1273,7 @@ protected:
   void do_pending_flush();
 
 public:
-  virtual void prepare_write(
+  void prepare_write(
     pg_info_t &info,
     pg_info_t &last_written_info,
     PastIntervals &past_intervals,
@@ -1239,6 +1366,18 @@ protected:
 
   // ref to recovery_state.info
   const pg_info_t &info;
+
+
+// ScrubberPasskey getters:
+public:
+  const pg_info_t& get_pg_info(ScrubberPasskey) const {
+    return info;
+  }
+
+  OSDService* get_pg_osd(ScrubberPasskey) const {
+    return osd;
+  }
+
 };
 
 #endif