Skip to content

crimson/osd/pg_recovery: use OperationThrottler to throttle object pushes/pulls#62080

Merged
Matan-B merged 2 commits intoceph:mainfrom
xxhdx1985126:wip-70180
Apr 7, 2025
Merged

crimson/osd/pg_recovery: use OperationThrottler to throttle object pushes/pulls#62080
Matan-B merged 2 commits intoceph:mainfrom
xxhdx1985126:wip-70180

Conversation

@xxhdx1985126
Copy link
Contributor

@xxhdx1985126 xxhdx1985126 commented Mar 3, 2025

@Matan-B @athanatos This PR reverts 14463aa and use OperationThrottler to throttle recovery/backfill pushes/pulls instead of recovery/backfill operations.

Fixes: https://tracker.ceph.com/issues/70180
Signed-off-by: Xuehan Xu xuxuehan@qianxin.com

Contribution Guidelines

  • To sign and title your commits, please refer to Submitting Patches to Ceph.

  • If you are submitting a fix for a stable branch (e.g. "quincy"), please refer to Submitting Patches to Ceph - Backports for the proper workflow.

  • When filling out the below checklist, you may click boxes directly in the GitHub web UI. When entering or editing the entire PR message in the GitHub web UI editor, you may also select a checklist item by adding an x between the brackets: [x]. Spaces and capitalization matter when checking off items this way.

Checklist

  • Tracker (select at least one)
    • References tracker ticket
    • Very recent bug; references commit where it was introduced
    • New feature (ticket optional)
    • Doc update (no ticket needed)
    • Code cleanup (no ticket needed)
  • Component impact
    • Affects Dashboard, opened tracker ticket
    • Affects Orchestrator, opened tracker ticket
    • No impact that needs to be tracked
  • Documentation (select at least one)
    • Updates relevant documentation
    • No doc update is appropriate
  • Tests (select at least one)
Show available Jenkins commands

@mohit84
Copy link
Contributor

mohit84 commented Mar 4, 2025

@xxhdx1985126 I think here you are not using OperationThrottler so we can;t say it(backfill) is supporting throttle, IIUC you have implemented a threshold limit to push backfill ops.

using std::set;
using PglogBasedRecovery = crimson::osd::PglogBasedRecovery;

std::atomic<uint64_t> PGRecovery::ongoing_pushes = 0;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why does this need to be atomic?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry, that's a mistake, will correct it.

@athanatos
Copy link
Contributor

14463aa did have a problem in that there was no mechanism to release the throttle -- that would need to be fixed regardless.

This new mechanism doesn't actually perform the function that the throttle would. Because the throttle is an OSD (actually, shard)-wide mechanism across IO types (client, background recovery, etc), it can delay backfill operations to preserve capacity for client IO.

We may, also, want to limit the number of objects we push concurrently, but we'd also need to use the throttle in order to prioritize backfill against other IO types.

@xxhdx1985126
Copy link
Contributor Author

14463aa did have a problem in that there was no mechanism to release the throttle -- that would need to be fixed regardless.

This new mechanism doesn't actually perform the function that the throttle would. Because the throttle is an OSD (actually, shard)-wide mechanism across IO types (client, background recovery, etc), it can delay backfill operations to preserve capacity for client IO.

We may, also, want to limit the number of objects we push concurrently, but we'd also need to use the throttle in order to prioritize backfill against other IO types.

So I think we need both mechinary, right? Does the backfill throttle has to be OSD-wide or reactor-wide is enough? Thansk:-)

@athanatos
Copy link
Contributor

Reactor-wide should suffice. For now, I think this is the right direction. Let's revert the throttler change and introduce a reactor-wide push limit (should apply to log based recovery pushes as well, though that can be a follow up PR). We can reintroduce the throttler later on with more appropriate interfaces.

@Matan-B Matan-B moved this from Awaits review to In Progress in Crimson Mar 6, 2025
@mohit84
Copy link
Contributor

mohit84 commented Mar 6, 2025

@xxhdx1985126 Can you please refresh the pull request only revert the commit 14463aa via tracker bug https://tracker.ceph.com/issues/70180, for push limit per pg specific change we should use a separate tracker bug.

@Matan-B
Copy link
Contributor

Matan-B commented Mar 6, 2025

@xxhdx1985126 Can you please refresh the pull request only revert the commit 14463aa via tracker bug https://tracker.ceph.com/issues/70180, for push limit per pg specific change we should use a separate tracker bug.

Let's keep this PR as is (with the revert and the new implementation).
If you need the revert earlier than the time this PR gets merged - feel free to push the revert as a separate PR so @xxhdx1985126 could rebase later.

@xxhdx1985126
Copy link
Contributor Author

@Matan-B @athanatos I've just modified the code as suggested, please take a look, again. Thanks:-)

@mohit84
Copy link
Contributor

mohit84 commented Mar 12, 2025

osd_recovery_max_active

@Matan-B

I might be wrong but i believe instead of using budget_available during Enqueuing we should use it on_backfill_reserved.
I am also planning to use throttle in the same function, i think if we call budget_available in the function backfill_reserved
it should be similar to classic OSD.

@xxhdx1985126
Copy link
Contributor Author

osd_recovery_max_active

@Matan-B

I might be wrong but i believe instead of using budget_available during Enqueuing we should use it on_backfill_reserved. I am also planning to use throttle in the same function, i think if we call budget_available in the function backfill_reserved it should be similar to classic OSD.

The purpose here is to throttle the maximum in-flight pushes, on_backfill_reserved happens only when the backfill is reserved at the pg level, it doesn't provide the information about the start/finish of the pushes, which, I think, makes it an inapprioprate place to do the throttling.

@mohit84
Copy link
Contributor

mohit84 commented Mar 12, 2025

osd_recovery_max_active

If we do check the classic OSD source code how do we control active recovery ops? Before enqueue an object into scheduler(via function _maybe_queue_recovery) we do check the current active value if it is less than max value we do enqueue an object otherwise ignore it. I believe the behavior should be same in crimson also.

@Matan-B Matan-B moved this from In Progress to Awaits review in Crimson Mar 12, 2025
Copy link
Contributor

@Matan-B Matan-B left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This seems to work well, I've left a few comments.

When the actual throttle usage (wip #62238) is introduced and will support all the other interfaces - we could decide if we would like to keep the budget_available limit added here as well. (cc @mohit84)

Comment on lines +49 to +51
auto max_to_start = std::max(
max,
crimson::common::get_conf<uint64_t>("crimson_max_active_push_per_reactor"));
Copy link
Contributor

@Matan-B Matan-B Mar 12, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think that it's confusing that we support both osd_recovery_max_single_start and crimson_max_active_push_per_reactor.

My suggestion would be to divide osd_recovery_max_single_start by smp::count to get the number of active pushes allowed for each reactor instead of introducing the new opiton.

      return pg->get_recovery_handler()->start_recovery_ops(
	trigger,
	*this,
	crimson::common::local_conf()->osd_recovery_max_single_start);
    });
PGRecovery::start_recovery_ops(
  RecoveryBackend::RecoveryBlockingEvent::TriggerI& trigger,
  PglogBasedRecovery &recover_op,
  size_t osd_max_to_start)
{
  ..
  auto max_active_pushes_per_reactor = osd_max_to_start / seastar::smp::count
  DEBUG(...)

What do you think?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yeah, will do.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@Matan-B Hi, matan, after a second thought, max_active_pushes_per_reactor limits the active pushes like osd_recovery_max_active does, so maybe auto max_active_pushes_per_reactor = osd_recovery_max_active / seastar::smp::count is better? What do you think?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Seems correct.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Seems correct.

I'm also trying to refactor this PR, changing OperationThrottle to work at the push/pull level as opposed to the current way, limiting the active recoveries/pushes for both pglog-based recoveries and backfills in the same shard. What do you think?

Copy link
Contributor

@Matan-B Matan-B Mar 31, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It does make sense to introduce both osd_recovery_max_single_start and osd_recovery_max_active in one go as their logic is somewhat intertwined (OSDService::_maybe_queue_recovery())

Is it a follow up to Mohit's #62238 or a different version of it? (cc @mohit84)

Copy link
Contributor Author

@xxhdx1985126 xxhdx1985126 Mar 31, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Um, I mean that, at present, the OperationThrottler is used to throttle the pglog-based recovery operation which can contain multiple object pushes/pulls, as shown in the code:

return this->template with_blocking_event<OperationThrottler::BlockingEvent>(
[ref, this] (auto&& trigger) {
return ss.with_throttle_while(
std::move(trigger),
this, get_scheduler_params(), [this] {
return interruptor::with_interruption([this] {
return do_recovery();
}, [](std::exception_ptr) {
return seastar::make_ready_future<bool>(false);
}, pg, epoch_started);
});
});

I intend to turn to use OperationThrottler to throttle the number of active pushes/pulls instead of recovery operations, so both pglog-based recoveries and backfills in the same shard can be throttled together. I think this is orthogonal to #62238 , which tries to throttle recovery/backfill operations, am I right?

Copy link
Contributor Author

@xxhdx1985126 xxhdx1985126 Mar 31, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@Matan-B I mean something like the following, in which all single object recoveries in the same shard are throttled together:

diff --git a/src/crimson/osd/osd_operation.cc b/src/crimson/osd/osd_operation.cc
index 97800970976..e552c3482b0 100644
--- a/src/crimson/osd/osd_operation.cc
+++ b/src/crimson/osd/osd_operation.cc
@@ -158,8 +158,7 @@ OperationThrottler::OperationThrottler(ConfigProxy &conf)
 
 void OperationThrottler::wake()
 {
-  while ((!max_in_progress || in_progress < max_in_progress) &&
-        !scheduler->empty()) {
+  while (avaiable() && !scheduler->empty()) {
     auto item = scheduler->dequeue();
     item.wake.set_value();
     ++in_progress;
diff --git a/src/crimson/osd/osd_operation.h b/src/crimson/osd/osd_operation.h
index af983eae4ce..d619b0753f5 100644
--- a/src/crimson/osd/osd_operation.h
+++ b/src/crimson/osd/osd_operation.h
@@ -334,6 +334,22 @@ public:
                          const std::set<std::string> &changed) final;
   void update_from_config(const ConfigProxy &conf);
 
+  bool available() const {
+    return !max_in_progress || in_progress < max_in_progress;
+  }
+
+  template <typename F>
+  auto with_throttle(
+    crimson::osd::scheduler::params_t params,
+    F &&f) {
+    if (!max_in_progress) return f();
+    return acquire_throttle(params)
+      .then(std::forward<F>(f))
+      .then([this] {
+       release_throttle();
+      });
+  }
+
   template <class OpT, class... Args>
   seastar::future<> with_throttle_while(
     BlockingEvent::Trigger<OpT>&& trigger,
diff --git a/src/crimson/osd/pg_recovery.cc b/src/crimson/osd/pg_recovery.cc
index 8d5841758e3..e40adc25031 100644
--- a/src/crimson/osd/pg_recovery.cc
+++ b/src/crimson/osd/pg_recovery.cc
@@ -319,7 +319,7 @@ PGRecovery::recover_missing(
     } else {
       return recovering.wait_track_blocking(
        trigger,
-       pg->get_recovery_backend()->recover_object(soid, need)
+       recover_object(soid, need)
        .handle_exception_interruptible(
          [=, this, soid = std::move(soid)] (auto e) {
          on_failed_recover({ pg->get_pg_whoami() }, soid, need);
@@ -367,7 +367,7 @@ RecoveryBackend::interruptible_future<> PGRecovery::prep_object_replica_pushes(
     logger().info("{} {} v {}, new recovery", __func__, soid, need);
     return recovering.wait_track_blocking(
       trigger,
-      pg->get_recovery_backend()->recover_object(soid, need)
+      recover_object(soid, need)
       .handle_exception_interruptible(
        [=, this, soid = std::move(soid)] (auto e) {
        on_failed_recover({ pg->get_pg_whoami() }, soid, need);
@@ -516,6 +516,23 @@ void PGRecovery::request_primary_scan(
   });
 }
 
+PGRecovery::interruptible_future<>
+PGRecovery::recover_object(
+  const hobject_t &soid,
+  eversion_t need)
+{
+  crimson::osd::scheduler::params_t params =
+    {1, 0, crimson::osd::scheduler::scheduler_class_t::background_best_effort};
+  auto &ss = pg->get_shard_services();
+  return ss.with_throttle(
+    std::move(params),
+    [this, soid, need] {
+    auto backend = pg->get_recovery_backend();
+    assert(backend);
+    return backend->recover_object(soid, need);
+  });
+}
+
 void PGRecovery::enqueue_push(
   const hobject_t& obj,
   const eversion_t& v,
@@ -527,7 +544,7 @@ void PGRecovery::enqueue_push(
   if (!added)
     return;
   peering_state.prepare_backfill_for_missing(obj, v, peers);
-  std::ignore = pg->get_recovery_backend()->recover_object(obj, v).\
+  std::ignore = recover_object(obj, v).\
   handle_exception_interruptible([] (auto) {
     ceph_abort_msg("got exception on backfill's push");
     return seastar::make_ready_future<>();
@@ -605,8 +622,8 @@ void PGRecovery::update_peers_last_backfill(
 
 bool PGRecovery::budget_available() const
 {
-  // TODO: the limits!
-  return true;
+  auto &ss = pg->get_shard_services();
+  return ss.available();
 }
 
 void PGRecovery::on_pg_clean()
diff --git a/src/crimson/osd/pg_recovery.h b/src/crimson/osd/pg_recovery.h
index 9d4a4874402..2b4a88bc5d9 100644
--- a/src/crimson/osd/pg_recovery.h
+++ b/src/crimson/osd/pg_recovery.h
@@ -97,6 +97,10 @@ private:
   friend class ReplicatedRecoveryBackend;
   friend class crimson::osd::UrgentRecovery;
 
+  interruptible_future<> recover_object(
+    const hobject_t &soid,
+    eversion_t need);
+
   // backfill begin
   std::unique_ptr<crimson::osd::BackfillState> backfill_state;
   std::map<pg_shard_t,

What do you think?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree that we should be able to throttle based on the number of active pushes/pulls instead of recovery operations. Although I'm not sure this is orthogonal to #62238 - based on the latest discussion there - it seems that OperationThrottler should be used to throttle pushed/pulls as well and not the based on the entire backfill interval operation. Right?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think so:-)

if (!added)
return;
peering_state.prepare_backfill_for_missing(obj, v, peers);
ongoing_pushes++;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should probably log ongoing_pushes (increases/decreases) somewhere to keep track of it

{
// TODO: the limits!
return true;
auto max = crimson::common::get_conf<
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Continuing my previous, I think we should be able to reuse osd_recovery_max_single_start here.
Let's also add a log line here with the three counters/limits to be aware that throttling has failed.

"max_active_push_per_reactor {}, max_active_push_per_osd {}, ongoing_pushes {}"

std::unique_ptr<crimson::osd::BackfillState> backfill_state;
std::map<pg_shard_t,
MURef<MOSDPGBackfillRemove>> backfill_drop_requests;
static thread_local uint64_t ongoing_pushes;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This should work, ack.

@Matan-B Matan-B moved this from Awaits review to In Progress in Crimson Mar 12, 2025
@mohit84
Copy link
Contributor

mohit84 commented Mar 12, 2025

This seems to work well, I've left a few comments.

When the actual throttle usage (wip #62238) is introduced and will support all the other interfaces - we could decide if we would like to keep the budget_available limit added here as well. (cc @mohit84)

Ack

@Matan-B Matan-B changed the title crimson/osd/pg_recovery: throttle backfills on the per-pg basis crimson/osd/pg_recovery: throttle active pushes on per-reactor basis Mar 31, 2025
@xxhdx1985126
Copy link
Contributor Author

jenkins test make check

@Matan-B
Copy link
Contributor

Matan-B commented Apr 1, 2025

I feel it is not a right place(during update_on_peers) to handle OperationThrottle for backfilling operation. Here we are acquiring a throttler for specific sub operation recover_object not for complete backfill operation . I think for backfilling also we need to do the same as we are doing for BackgroundRecovery.

I still haven't reviewed the PR, but recover_object could be a good place to acquire the throttler as it's common to Backfiil/PGLog-based (with all it's variants: Replica/Primary/UrgentRecovery) recovery. That means that once we are properly throttling pushes/pulls in recover_object we control any recovery ongoing. The only downside is that we won't be able to differentiate between Backfill or pglog based recovery ops - though I'm ok with this as I like the generalization. @mohit84, what do you think?

@Matan-B Matan-B moved this from In Progress to Awaits review in Crimson Apr 1, 2025
Copy link
Contributor

@Matan-B Matan-B left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Following up to my comment above, seems that now PGRecovery::recover_object acquires the throttle and then internally uses ReplicatedRecoveryBackend::recover_object.

Since backfill enqueue uses ReplicatedRecoveryBackend::recover_object directly, can we add a wrapper that would be used by both recovery types prior to actually calling ReplicatedRecoveryBackend::recover_object?

Could be something like recover_object_with_throttle - as we would still need the non throttling recover object for UrgentRecovery probably.

What do you think?

CC @mohit84.

@Matan-B Matan-B requested a review from amathuria April 1, 2025 12:44
@xxhdx1985126
Copy link
Contributor Author

Following up to my comment above, seems that now PGRecovery::recover_object acquires the throttle and then internally uses ReplicatedRecoveryBackend::recover_object.

Since backfill enqueue uses ReplicatedRecoveryBackend::recover_object directly, can we add a wrapper that would be used by both recovery types prior to actually calling ReplicatedRecoveryBackend::recover_object?

Could be something like recover_object_with_throttle - as we would still need the non throttling recover object for UrgentRecovery probably.

What do you think?

Ah, yes, will do.

CC @mohit84.

@mohit84
Copy link
Contributor

mohit84 commented Apr 1, 2025

recover_object

I have checked the code i think we can go with (recover_object) , this is more cleaner as compare to previous approach.

@xxhdx1985126
Copy link
Contributor Author

Following up to my comment above, seems that now PGRecovery::recover_object acquires the throttle and then internally uses ReplicatedRecoveryBackend::recover_object.

Since backfill enqueue uses ReplicatedRecoveryBackend::recover_object directly, can we add a wrapper that would be used by both recovery types prior to actually calling ReplicatedRecoveryBackend::recover_object?

Could be something like recover_object_with_throttle - as we would still need the non throttling recover object for UrgentRecovery probably.

What do you think?

Done, please take a look, again. Thanks:-)

Copy link
Contributor

@Matan-B Matan-B left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As mentioned above, backfill uses ReplicatedRecoveryBackend::recover_object directly (without PGRecovery::recover_missing) - the changes here won't affect backfill.

The with_throttle param should be moved to the recovery backend to be used by both pglog and backfill.

}, [](std::exception_ptr) {
return seastar::make_ready_future<bool>(false);
}, pg, epoch_started).then([](bool goon) {
if (goon) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would suggest changing start_recovery_ops to return done (instead of !done)
And change goon to recovery_done:

if (recovery_done) {
  return seastar::stop_iteration::yes
} else {
 ..

Otherwise, the reader must go to the definition of do_recovery to understand what goon is.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, will do.

@xxhdx1985126
Copy link
Contributor Author

As mentioned above, backfill uses ReplicatedRecoveryBackend::recover_object directly (without PGRecovery::recover_missing) - the changes here won't affect backfill.

Um, I'm a bit confused. IIUC, backfill calls PGRecovery::enqueue_push which further calls ReplicatedRecoveryBackend::recover_object to do the pushes. So I changed PGRecovery::enqueue_push to call PGRecovery::recover_object_with_throttle so that backfill pushes can also be throttled. Am I missing something?

The with_throttle param should be moved to the recovery backend to be used by both pglog and backfill.

Copy link
Contributor

@Matan-B Matan-B left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Um, I'm a bit confused. IIUC, backfill calls PGRecovery::enqueue_push which further calls ReplicatedRecoveryBackend::recover_object to do the pushes. So I changed PGRecovery::enqueue_push to call PGRecovery::recover_object_with_throttle so that backfill pushes can also be throttled. Am I missing something?

My mistake! You are right. Overall lgtm, looks like an elegant way to throttle both any recovery operation.
Only comments are cleanup and previous comment regarding naming.


do {
if (!backfill_listener().budget_available()) {
DEBUGDPP("throttle failed, turning to Waiting", pg());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we still need to check for budget_available here with the new approach? IIUC budget_available can be removed now as the with_throttle (from recover_object_with_throttle) will handle available()

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am getting crash after merging this pull request with mclock pull request, it seems interrupt_cond is not valid during process recover_object via interruptible

(gdb) bt
#0 0x00007fbb73cabe0c in __pthread_kill_implementation () from /lib64/libc.so.6
#1 0x00007fbb73c5eb46 in raise () from /lib64/libc.so.6
#2 0x0000000001b6ee1a in reraise_fatal (signum=6) at /ceph-src/ceph/src/crimson/common/fatal_signal.cc:41
#3 FatalSignal::signal_entry (signum=6, info=) at /ceph-src/ceph/src/crimson/common/fatal_signal.cc:62
#4
#5 0x00007fbb73cabe0c in __pthread_kill_implementation () from /lib64/libc.so.6
#6 0x00007fbb73c5eb46 in raise () from /lib64/libc.so.6
#7 0x00007fbb73c48833 in abort () from /lib64/libc.so.6
#8 0x00000000011d74cf in ceph::__ceph_assert_fail (assertion=, file=,
line=, func=) at /ceph-src/ceph/src/crimson/common/assert.cc:27
#9 0x00000000011d752a in ceph::__ceph_assert_fail (ctx=...) at /ceph-src/ceph/src/crimson/common/assert.cc:14
#10 0x00000000016fbb5e in crimson::interruptible::interruptible_future_detail<crimson::osd::IOInterruptCondition, seastar::future >::then_interruptible<ReplicatedRecoveryBackend::recover_object(const hobject_t&, eversion_t)::<lambda()> > (
func=..., this=0x7fff4e9fcb90) at /ceph-src/ceph/src/crimson/common/interruptible_future.h:485
#11 ReplicatedRecoveryBackend::recover_object (this=, soid=..., need=...)
at /ceph-src/ceph/src/crimson/osd/replicated_recovery_backend.cc:60
#12 0x00000000016ab375 in operator() (__closure=0x600000bac2d0) at /ceph-src/ceph/src/crimson/osd/pg_recovery.cc:536
#13 std::__invoke_impl<crimson::interruptible::interruptible_future_detail<crimson::osd::IOInterruptCondition, seastar::future<> >, PGRecovery::recover_object_with_throttle(const hobject_t&, eversion_t)::<lambda()>&> (__f=...)
at /opt/rh/gcc-toolset-13/root/usr/include/c++/13/bits/invoke.h:61
#14 std::__invoke<PGRecovery::recover_object_with_throttle(const hobject_t&, eversion_t)::<lambda()>&> (__fn=...)
at /opt/rh/gcc-toolset-13/root/usr/include/c++/13/bits/invoke.h:97
#15 std::invoke<PGRecovery::recover_object_with_throttle(const hobject_t&, eversion_t)::<lambda()>&> (__fn=...)
at /opt/rh/gcc-toolset-13/root/usr/include/c++/13/functional:114
#16 seastar::internal::future_invoke<PGRecovery::recover_object_with_throttle(const hobject_t&, eversion_t)::<lambda()>&, seastar::internal::monostate> (v=..., func=...) at /ceph-src/ceph/src/seastar/include/seastar/core/future.hh:1150
#17 operator() (__closure=) at /ceph-src/ceph/src/seastar/include/seastar/core/future.hh:1466
--Type for more, q to quit, c to continue without paging--
#18 seastar::futurize<crimson::interruptible::interruptible_future_detail<crimson::osd::IOInterruptCondition, seastar::future > >::satisfy_with_result_of<seastar::internal::promise_base_with_type, seastar::future<>::then_impl_nrvo<PGRecovery::recover_object_with_throttle(const hobject_t&, eversion_t)::<lambda()>, crimson::interruptible::interruptible_future_detail<crimson::osd::IOInterruptCondition, seastar::future<> > >(PGRecovery::recover_object_with_throttle(const hobject_t&, eversion_t)::<lambda()>&&)::<lambda(pr_type&&, PGRecovery::recover_object_with_throttle(const hobject_t&, eversion_t)::<lambda()>&, seastar::future<>::future_state&&)>::<lambda()> > (func=..., pr=...)
at /ceph-src/ceph/src/crimson/common/interruptible_future.h:1664
#19 operator() (__closure=, state=..., func=..., pr=...)
at /ceph-src/ceph/src/seastar/include/seastar/core/future.hh:1462
#20 seastar::continuation<seastar::internal::promise_base_with_type, PGRecovery::recover_object_with_throttle(const hobject_t&, eversion_t)::<lambda()>, seastar::future<>::then_impl_nrvo<PGRecovery::recover_object_with_throttle(const hobject_t&, eversion_t)::<lambda()>, crimson::interruptible::interruptible_future_detail<crimson::osd::IOInterruptCondition, seastar::future<> > >(PGRecovery::recover_object_with_throttle(const hobject_t&, eversion_t)::<lambda()>&&)::<lambda(pr_type&&, PGRecovery::recover_object_with_throttle(const hobject_t&, eversion_t)::<lambda()>&, seastar::future<>::future_state&&)>, void>::run_and_dispose(void) (this=0x600000bac2a0) at /ceph-src/ceph/src/seastar/include/seastar/core/future.hh:728
#21 0x0000000002b57558 in seastar::reactor::run_tasks (this=this@entry=0x600000106000, tq=...)
at /ceph-src/ceph/src/seastar/src/core/reactor.cc:2608
#22 0x0000000002b579fd in seastar::reactor::run_some_tasks (this=this@entry=0x600000106000)
at /ceph-src/ceph/src/seastar/src/core/reactor.cc:3080
#23 0x0000000002b9140e in seastar::reactor::run_some_tasks (this=0x600000106000)
at /ceph-src/ceph/src/seastar/include/seastar/core/circular_buffer_fixed_capacity.hh:213
#24 seastar::reactor::do_run (this=0x600000106000) at /ceph-src/ceph/src/seastar/src/core/reactor.cc:3248
#25 0x0000000002b92119 in seastar::reactor::run (this=)
at /ceph-src/ceph/src/seastar/src/core/reactor.cc:3138
#26 0x0000000002aff736 in seastar::app_template::run_deprecated(int, char**, std::function<void ()>&&) (
this=this@entry=0x7fff4e9fd790, ac=ac@entry=10, av=av@entry=0x3c12f7c0, func=...)
at /ceph-src/ceph/src/seastar/src/core/app-template.cc:276
--Type for more, q to quit, c to continue without paging--
#27 0x0000000002afffef in seastar::app_template::run(int, char**, std::function<seastar::future ()>&&) (
this=this@entry=0x7fff4e9fd790, ac=10, av=0x3c12f7c0, func=...)
at /ceph-src/ceph/src/seastar/src/core/app-template.cc:167
#28 0x0000000001309f85 in main (argc=, argv=)
at /opt/rh/gcc-toolset-13/root/usr/include/c++/13/bits/stl_vector.h:992

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Copy link
Contributor

@Matan-B Matan-B Apr 3, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for sharing this one @mohit84. Next steps could probably be to see if you can reproduce this without your additional branch - so we would know if the issue is in this PR or not.
If it's related to this PR, please share the reproducer.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please ignore the issue is in my patch.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@mohit84, thanks for verifying this! Should we keep #62238 open?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I have closed the request.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@xxhdx1985126, That means that we check for budget_available prior to scanning but not actually taking the throttle when scanning, the throttle will be only taken for actual individual object pushes. Right?

Yes, that's right.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Merged - follow up tracker opened: https://tracker.ceph.com/issues/70808 (@xxhdx1985126)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Merged - follow up tracker opened: https://tracker.ceph.com/issues/70808 (@xxhdx1985126)

Got it:-)

Copy link
Contributor

@Matan-B Matan-B left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lgtm, only comment is regarding naming above - moving this one to be tested in the meanwhile.

@Matan-B Matan-B moved this from Awaits review to Needs QA in Crimson Apr 3, 2025
@Matan-B
Copy link
Contributor

Matan-B commented Apr 3, 2025

PGRecovery::recover_object_with_throttle(
const hobject_t &soid,
eversion_t need)
{
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we can push a log here to capture the details about ongoing operation so that it would be easier to analyze logs in case of throttle vs non throttle.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, will do.

pushes/pulls

Instead of throttling recovery/backfill operations

Fixes: https://tracker.ceph.com/issues/70180
Signed-off-by: Xuehan Xu <xuxuehan@qianxin.com>
@xxhdx1985126
Copy link
Contributor Author

All done.

@Matan-B
Copy link
Contributor

Matan-B commented Apr 7, 2025

https://pulpito.ceph.com/matan-2025-04-06_13:17:00-crimson-rados-wip-matanb-crimson-testing-62080-distro-crimson-smithi/

8226288, 8226340 - https://tracker.ceph.com/issues/69612
8226384 - https://tracker.ceph.com/issues/67972
8226362 - https://tracker.ceph.com/issues/70502
8226405 - https://tracker.ceph.com/issues/69439
8226342, 8226378 -  https://tracker.ceph.com/issues/69406

@Matan-B Matan-B merged commit 9cb8feb into ceph:main Apr 7, 2025
12 checks passed
@Matan-B Matan-B moved this from Needs QA to Merged in Crimson Apr 7, 2025
Matan-B added a commit to Matan-B/ceph that referenced this pull request Apr 15, 2025
When ceph#62080 tested version was
diffrent from the one that got merged.
The untested chage was changing the boolean returned from start_recovery_ops.
While the seastar::repeat loop in BackgroundRecoveryT<T>::start() was
changed accordignly, other do_recovery() return cases were not
considered.

start_recovery_ops used by do_recovery should return whether the
iteration (i.e recovery) keep going.

Signed-off-by: Matan Breizman <mbreizma@redhat.com>
Matan-B added a commit to Matan-B/ceph that referenced this pull request Apr 15, 2025
When ceph#62080 tested version was
diffrent from the one that got merged.
The untested chage was changing the boolean returned from start_recovery_ops.
While the seastar::repeat loop in BackgroundRecoveryT<T>::start() was
changed accordignly, other do_recovery() return cases were not
considered.

See Tested / Merged here: https://github.com/Matan-B/ceph/pull/2/files

start_recovery_ops used by do_recovery should return whether the
iteration (i.e recovery) keep going.

Signed-off-by: Matan Breizman <mbreizma@redhat.com>
Matan-B added a commit to Matan-B/ceph that referenced this pull request Apr 15, 2025
ceph#62080 tested version was **different**
from the one that got merged.
The untested change was changing the boolean returned from start_recovery_ops.
While the seastar::repeat loop in BackgroundRecoveryT<T>::start() was changed accordingly,
other do_recovery() return cases were not considered.

See Tested / Merged here: https://github.com/Matan-B/ceph/pull/2/files

start_recovery_ops used by do_recovery should return whether the iteration (i.e recovery) keep going.

_Note: This has caused a regression in our suite_

Signed-off-by: Matan Breizman <mbreizma@redhat.com>
Matan-B added a commit to ceph/ceph-ci that referenced this pull request Apr 15, 2025
ceph/ceph#62080 tested version was **different**
from the one that got merged.
The untested change was changing the boolean returned from start_recovery_ops.
While the seastar::repeat loop in BackgroundRecoveryT<T>::start() was changed accordingly,
other do_recovery() return cases were not considered.

See Tested / Merged here: https://github.com/Matan-B/ceph/pull/2/files

start_recovery_ops used by do_recovery should return whether the iteration (i.e recovery) keep going.

_Note: This has caused a regression in our suite_

Signed-off-by: Matan Breizman <mbreizma@redhat.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

Status: Merged (Pre Tentacle Freeze)

Development

Successfully merging this pull request may close these issues.

4 participants