return $(( $check >= $target && $check <= $target + $margin ? 0 : 1 ))
}
-FIND_UPACT='grep "pg[[]${PG}.*backfilling.*_update_calc_stats " $log | tail -1 | sed "s/.*[)] \([[][^ p]*\).*$/\1/"'
-FIND_FIRST='grep "pg[[]${PG}.*backfilling.*_update_calc_stats $which " $log | grep -F " ${UPACT}${addp}" | grep -v est | head -1 | sed "s/.* \([0-9]*\)$/\1/"'
-FIND_LAST='grep "pg[[]${PG}.*backfilling.*_update_calc_stats $which " $log | tail -1 | sed "s/.* \([0-9]*\)$/\1/"'
+FIND_UPACT='grep "pg[[]${PG}.*backfilling.*update_calc_stats " $log | tail -1 | sed "s/.*[)] \([[][^ p]*\).*$/\1/"'
+FIND_FIRST='grep "pg[[]${PG}.*backfilling.*update_calc_stats $which " $log | grep -F " ${UPACT}${addp}" | grep -v est | head -1 | sed "s/.* \([0-9]*\)$/\1/"'
+FIND_LAST='grep "pg[[]${PG}.*backfilling.*update_calc_stats $which " $log | tail -1 | sed "s/.* \([0-9]*\)$/\1/"'
function check() {
local dir=$1
return $(( $check >= $target && $check <= $target + $margin ? 0 : 1 ))
}
-FIND_UPACT='grep "pg[[]${PG}.*recovering.*_update_calc_stats " $log | tail -1 | sed "s/.*[)] \([[][^ p]*\).*$/\1/"'
-FIND_FIRST='grep "pg[[]${PG}.*recovering.*_update_calc_stats $which " $log | grep -F " ${UPACT}${addp}" | grep -v est | head -1 | sed "s/.* \([0-9]*\)$/\1/"'
-FIND_LAST='grep "pg[[]${PG}.*recovering.*_update_calc_stats $which " $log | tail -1 | sed "s/.* \([0-9]*\)$/\1/"'
+FIND_UPACT='grep "pg[[]${PG}.*recovering.*update_calc_stats " $log | tail -1 | sed "s/.*[)] \([[][^ p]*\).*$/\1/"'
+FIND_FIRST='grep "pg[[]${PG}.*recovering.*update_calc_stats $which " $log | grep -F " ${UPACT}${addp}" | grep -v est | head -1 | sed "s/.* \([0-9]*\)$/\1/"'
+FIND_LAST='grep "pg[[]${PG}.*recovering.*update_calc_stats $which " $log | tail -1 | sed "s/.* \([0-9]*\)$/\1/"'
function check() {
local dir=$1
local log=$dir/osd.${primary}.log
check $dir $PG $primary replicated 0 0 $misplaced 0 || return 1
- UPACT=$(grep "pg[[]${PG}.*recovering.*_update_calc_stats " $log | tail -1 | sed "s/.*[)] \([[][^ p]*\).*$/\1/")
+ UPACT=$(grep "pg[[]${PG}.*recovering.*update_calc_stats " $log | tail -1 | sed "s/.*[)] \([[][^ p]*\).*$/\1/")
# This is the value of set into MISSING_ON_PRIMARY
- FIRST=$(grep "pg[[]${PG}.*recovering.*_update_calc_stats shard $primary " $log | grep -F " $UPACT " | head -1 | sed "s/.* \([0-9]*\)$/\1/")
+ FIRST=$(grep "pg[[]${PG}.*recovering.*update_calc_stats shard $primary " $log | grep -F " $UPACT " | head -1 | sed "s/.* \([0-9]*\)$/\1/")
below_margin $FIRST $objects || return 1
- LAST=$(grep "pg[[]${PG}.*recovering.*_update_calc_stats shard $primary " $log | tail -1 | sed "s/.* \([0-9]*\)$/\1/")
+ LAST=$(grep "pg[[]${PG}.*recovering.*update_calc_stats shard $primary " $log | tail -1 | sed "s/.* \([0-9]*\)$/\1/")
above_margin $LAST 0 || return 1
delete_pool $poolname
pg_log.get_log().get_request(r, version, user_version, return_code));
}
-static bool find_shard(const set<pg_shard_t> & pgs, shard_id_t shard)
-{
- for (auto&p : pgs)
- if (p.shard == shard)
- return true;
- return false;
-}
-
-static pg_shard_t get_another_shard(const set<pg_shard_t> & pgs, pg_shard_t skip, shard_id_t shard)
-{
- for (auto&p : pgs) {
- if (p == skip)
- continue;
- if (p.shard == shard)
- return p;
- }
- return pg_shard_t();
-}
-
-void PG::_update_calc_stats()
-{
- info.stats.version = info.last_update;
- info.stats.created = info.history.epoch_created;
- info.stats.last_scrub = info.history.last_scrub;
- info.stats.last_scrub_stamp = info.history.last_scrub_stamp;
- info.stats.last_deep_scrub = info.history.last_deep_scrub;
- info.stats.last_deep_scrub_stamp = info.history.last_deep_scrub_stamp;
- info.stats.last_clean_scrub_stamp = info.history.last_clean_scrub_stamp;
- info.stats.last_epoch_clean = info.history.last_epoch_clean;
-
- info.stats.log_size = pg_log.get_head().version - pg_log.get_tail().version;
- info.stats.ondisk_log_size = info.stats.log_size;
- info.stats.log_start = pg_log.get_tail();
- info.stats.ondisk_log_start = pg_log.get_tail();
- info.stats.snaptrimq_len = snap_trimq.size();
-
- unsigned num_shards = get_osdmap()->get_pg_size(info.pgid.pgid);
-
- // In rare case that upset is too large (usually transient), use as target
- // for calculations below.
- unsigned target = std::max(num_shards, (unsigned)upset.size());
- // For undersized actingset may be larger with OSDs out
- unsigned nrep = std::max(actingset.size(), upset.size());
- // calc num_object_copies
- info.stats.stats.calc_copies(std::max(target, nrep));
- info.stats.stats.sum.num_objects_degraded = 0;
- info.stats.stats.sum.num_objects_unfound = 0;
- info.stats.stats.sum.num_objects_misplaced = 0;
- info.stats.avail_no_missing.clear();
- info.stats.object_location_counts.clear();
-
- if ((is_remapped() || is_undersized() || !is_clean()) && (is_peered() || is_activating())) {
- dout(20) << __func__ << " actingset " << actingset << " upset "
- << upset << " acting_recovery_backfill " << acting_recovery_backfill << dendl;
- dout(20) << __func__ << " acting " << acting << " up " << up << dendl;
-
- ceph_assert(!acting_recovery_backfill.empty());
-
- bool estimate = false;
-
- // NOTE: we only generate degraded, misplaced and unfound
- // values for the summation, not individual stat categories.
- int64_t num_objects = info.stats.stats.sum.num_objects;
-
- // Objects missing from up nodes, sorted by # objects.
- boost::container::flat_set<pair<int64_t,pg_shard_t>> missing_target_objects;
- // Objects missing from nodes not in up, sort by # objects
- boost::container::flat_set<pair<int64_t,pg_shard_t>> acting_source_objects;
-
- // Fill missing_target_objects/acting_source_objects
-
- {
- int64_t missing;
-
- // Primary first
- missing = pg_log.get_missing().num_missing();
- ceph_assert(acting_recovery_backfill.count(pg_whoami));
- if (upset.count(pg_whoami)) {
- missing_target_objects.emplace(missing, pg_whoami);
- } else {
- acting_source_objects.emplace(missing, pg_whoami);
- }
- info.stats.stats.sum.num_objects_missing_on_primary = missing;
- if (missing == 0)
- info.stats.avail_no_missing.push_back(pg_whoami);
- dout(20) << __func__ << " shard " << pg_whoami
- << " primary objects " << num_objects
- << " missing " << missing
- << dendl;
- }
-
- // All other peers
- for (auto& peer : peer_info) {
- // Primary should not be in the peer_info, skip if it is.
- if (peer.first == pg_whoami) continue;
- int64_t missing = 0;
- int64_t peer_num_objects = peer.second.stats.stats.sum.num_objects;
- // Backfill targets always track num_objects accurately
- // all other peers track missing accurately.
- if (is_backfill_targets(peer.first)) {
- missing = std::max((int64_t)0, num_objects - peer_num_objects);
- } else {
- if (peer_missing.count(peer.first)) {
- missing = peer_missing[peer.first].num_missing();
- } else {
- dout(20) << __func__ << " no peer_missing found for " << peer.first << dendl;
- if (is_recovering()) {
- estimate = true;
- }
- missing = std::max((int64_t)0, num_objects - peer_num_objects);
- }
- }
- if (upset.count(peer.first)) {
- missing_target_objects.emplace(missing, peer.first);
- } else if (actingset.count(peer.first)) {
- acting_source_objects.emplace(missing, peer.first);
- }
- peer.second.stats.stats.sum.num_objects_missing = missing;
- if (missing == 0)
- info.stats.avail_no_missing.push_back(peer.first);
- dout(20) << __func__ << " shard " << peer.first
- << " objects " << peer_num_objects
- << " missing " << missing
- << dendl;
- }
-
- // Compute object_location_counts
- for (auto& ml: missing_loc.get_missing_locs()) {
- info.stats.object_location_counts[ml.second]++;
- dout(30) << __func__ << " " << ml.first << " object_location_counts["
- << ml.second << "]=" << info.stats.object_location_counts[ml.second]
- << dendl;
- }
- int64_t not_missing = num_objects - missing_loc.get_missing_locs().size();
- if (not_missing) {
- // During recovery we know upset == actingset and is being populated
- // During backfill we know that all non-missing objects are in the actingset
- info.stats.object_location_counts[actingset] = not_missing;
- }
- dout(30) << __func__ << " object_location_counts["
- << upset << "]=" << info.stats.object_location_counts[upset]
- << dendl;
- dout(20) << __func__ << " object_location_counts "
- << info.stats.object_location_counts << dendl;
-
- // A misplaced object is not stored on the correct OSD
- int64_t misplaced = 0;
- // a degraded objects has fewer replicas or EC shards than the pool specifies.
- int64_t degraded = 0;
-
- if (is_recovering()) {
- for (auto& sml: missing_loc.get_missing_by_count()) {
- for (auto& ml: sml.second) {
- int missing_shards;
- if (sml.first == shard_id_t::NO_SHARD) {
- dout(20) << __func__ << " ml " << ml.second << " upset size " << upset.size() << " up " << ml.first.up << dendl;
- missing_shards = (int)upset.size() - ml.first.up;
- } else {
- // Handle shards not even in upset below
- if (!find_shard(upset, sml.first))
- continue;
- missing_shards = std::max(0, 1 - ml.first.up);
- dout(20) << __func__ << " shard " << sml.first << " ml " << ml.second << " missing shards " << missing_shards << dendl;
- }
- int odegraded = ml.second * missing_shards;
- // Copies on other osds but limited to the possible degraded
- int more_osds = std::min(missing_shards, ml.first.other);
- int omisplaced = ml.second * more_osds;
- ceph_assert(omisplaced <= odegraded);
- odegraded -= omisplaced;
-
- misplaced += omisplaced;
- degraded += odegraded;
- }
- }
-
- dout(20) << __func__ << " missing based degraded " << degraded << dendl;
- dout(20) << __func__ << " missing based misplaced " << misplaced << dendl;
-
- // Handle undersized case
- if (pool.info.is_replicated()) {
- // Add degraded for missing targets (num_objects missing)
- ceph_assert(target >= upset.size());
- unsigned needed = target - upset.size();
- degraded += num_objects * needed;
- } else {
- for (unsigned i = 0 ; i < num_shards; ++i) {
- shard_id_t shard(i);
-
- if (!find_shard(upset, shard)) {
- pg_shard_t pgs = get_another_shard(actingset, pg_shard_t(), shard);
-
- if (pgs != pg_shard_t()) {
- int64_t missing;
-
- if (pgs == pg_whoami)
- missing = info.stats.stats.sum.num_objects_missing_on_primary;
- else
- missing = peer_info[pgs].stats.stats.sum.num_objects_missing;
-
- degraded += missing;
- misplaced += std::max((int64_t)0, num_objects - missing);
- } else {
- // No shard anywhere
- degraded += num_objects;
- }
- }
- }
- }
- goto out;
- }
-
- // Handle undersized case
- if (pool.info.is_replicated()) {
- // Add to missing_target_objects
- ceph_assert(target >= missing_target_objects.size());
- unsigned needed = target - missing_target_objects.size();
- if (needed)
- missing_target_objects.emplace(num_objects * needed, pg_shard_t(pg_shard_t::NO_OSD));
- } else {
- for (unsigned i = 0 ; i < num_shards; ++i) {
- shard_id_t shard(i);
- bool found = false;
- for (const auto& t : missing_target_objects) {
- if (std::get<1>(t).shard == shard) {
- found = true;
- break;
- }
- }
- if (!found)
- missing_target_objects.emplace(num_objects, pg_shard_t(pg_shard_t::NO_OSD,shard));
- }
- }
-
- for (const auto& item : missing_target_objects)
- dout(20) << __func__ << " missing shard " << std::get<1>(item) << " missing= " << std::get<0>(item) << dendl;
- for (const auto& item : acting_source_objects)
- dout(20) << __func__ << " acting shard " << std::get<1>(item) << " missing= " << std::get<0>(item) << dendl;
-
- // Handle all objects not in missing for remapped
- // or backfill
- for (auto m = missing_target_objects.rbegin();
- m != missing_target_objects.rend(); ++m) {
-
- int64_t extra_missing = -1;
-
- if (pool.info.is_replicated()) {
- if (!acting_source_objects.empty()) {
- auto extra_copy = acting_source_objects.begin();
- extra_missing = std::get<0>(*extra_copy);
- acting_source_objects.erase(extra_copy);
- }
- } else { // Erasure coded
- // Use corresponding shard
- for (const auto& a : acting_source_objects) {
- if (std::get<1>(a).shard == std::get<1>(*m).shard) {
- extra_missing = std::get<0>(a);
- acting_source_objects.erase(a);
- break;
- }
- }
- }
-
- if (extra_missing >= 0 && std::get<0>(*m) >= extra_missing) {
- // We don't know which of the objects on the target
- // are part of extra_missing so assume are all degraded.
- misplaced += std::get<0>(*m) - extra_missing;
- degraded += extra_missing;
- } else {
- // 1. extra_missing == -1, more targets than sources so degraded
- // 2. extra_missing > std::get<0>(m), so that we know that some extra_missing
- // previously degraded are now present on the target.
- degraded += std::get<0>(*m);
- }
- }
- // If there are still acting that haven't been accounted for
- // then they are misplaced
- for (const auto& a : acting_source_objects) {
- int64_t extra_misplaced = std::max((int64_t)0, num_objects - std::get<0>(a));
- dout(20) << __func__ << " extra acting misplaced " << extra_misplaced << dendl;
- misplaced += extra_misplaced;
- }
-out:
- // NOTE: Tests use these messages to verify this code
- dout(20) << __func__ << " degraded " << degraded << (estimate ? " (est)": "") << dendl;
- dout(20) << __func__ << " misplaced " << misplaced << (estimate ? " (est)": "")<< dendl;
-
- info.stats.stats.sum.num_objects_degraded = degraded;
- info.stats.stats.sum.num_objects_unfound = get_num_unfound();
- info.stats.stats.sum.num_objects_misplaced = misplaced;
- }
-}
-
-void PG::_update_blocked_by()
-{
- // set a max on the number of blocking peers we report. if we go
- // over, report a random subset. keep the result sorted.
- unsigned keep = std::min<unsigned>(blocked_by.size(), cct->_conf->osd_max_pg_blocked_by);
- unsigned skip = blocked_by.size() - keep;
- info.stats.blocked_by.clear();
- info.stats.blocked_by.resize(keep);
- unsigned pos = 0;
- for (set<int>::iterator p = blocked_by.begin();
- p != blocked_by.end() && keep > 0;
- ++p) {
- if (skip > 0 && (rand() % (skip + keep) < skip)) {
- --skip;
- } else {
- info.stats.blocked_by[pos++] = *p;
- --keep;
- }
- }
-}
-
void PG::publish_stats_to_osd()
{
if (!is_primary())
pg_stats_publish_lock.Lock();
- if (info.stats.stats.sum.num_scrub_errors)
- state_set(PG_STATE_INCONSISTENT);
- else {
- state_clear(PG_STATE_INCONSISTENT);
- state_clear(PG_STATE_FAILED_REPAIR);
- }
-
- utime_t now = ceph_clock_now();
- if (info.stats.state != state) {
- info.stats.last_change = now;
- // Optimistic estimation, if we just find out an inactive PG,
- // assumt it is active till now.
- if (!(state & PG_STATE_ACTIVE) &&
- (info.stats.state & PG_STATE_ACTIVE))
- info.stats.last_active = now;
-
- if ((state & PG_STATE_ACTIVE) &&
- !(info.stats.state & PG_STATE_ACTIVE))
- info.stats.last_became_active = now;
- if ((state & (PG_STATE_ACTIVE|PG_STATE_PEERED)) &&
- !(info.stats.state & (PG_STATE_ACTIVE|PG_STATE_PEERED)))
- info.stats.last_became_peered = now;
- info.stats.state = state;
- }
-
- _update_calc_stats();
- if (info.stats.stats.sum.num_objects_degraded) {
- state_set(PG_STATE_DEGRADED);
- } else {
- state_clear(PG_STATE_DEGRADED);
- }
- _update_blocked_by();
-
- pg_stat_t pre_publish = info.stats;
- pre_publish.stats.add(unstable_stats);
- utime_t cutoff = now;
- cutoff -= cct->_conf->osd_pg_stat_report_interval_max;
-
- if (get_osdmap()->require_osd_release >= CEPH_RELEASE_MIMIC) {
- // share (some of) our purged_snaps via the pg_stats. limit # of intervals
- // because we don't want to make the pg_stat_t structures too expensive.
- unsigned max = cct->_conf->osd_max_snap_prune_intervals_per_epoch;
- unsigned num = 0;
- auto i = info.purged_snaps.begin();
- while (num < max && i != info.purged_snaps.end()) {
- pre_publish.purged_snaps.insert(i.get_start(), i.get_len());
- ++num;
- ++i;
- }
- dout(20) << __func__ << " reporting purged_snaps "
- << pre_publish.purged_snaps << dendl;
- }
-
- if (pg_stats_publish_valid && pre_publish == pg_stats_publish &&
- info.stats.last_fresh > cutoff) {
- dout(15) << "publish_stats_to_osd " << pg_stats_publish.reported_epoch
- << ": no change since " << info.stats.last_fresh << dendl;
- } else {
- // update our stat summary and timestamps
- info.stats.reported_epoch = get_osdmap_epoch();
- ++info.stats.reported_seq;
-
- info.stats.last_fresh = now;
-
- if (info.stats.state & PG_STATE_CLEAN)
- info.stats.last_clean = now;
- if (info.stats.state & PG_STATE_ACTIVE)
- info.stats.last_active = now;
- if (info.stats.state & (PG_STATE_ACTIVE|PG_STATE_PEERED))
- info.stats.last_peered = now;
- info.stats.last_unstale = now;
- if ((info.stats.state & PG_STATE_DEGRADED) == 0)
- info.stats.last_undegraded = now;
- if ((info.stats.state & PG_STATE_UNDERSIZED) == 0)
- info.stats.last_fullsized = now;
-
+ auto stats = recovery_state.prepare_stats_for_publish(
+ pg_stats_publish_valid,
+ pg_stats_publish,
+ unstable_stats);
+ if (stats) {
+ pg_stats_publish = stats.value();
pg_stats_publish_valid = true;
- pg_stats_publish = pre_publish;
-
- dout(15) << "publish_stats_to_osd " << pg_stats_publish.reported_epoch
- << ":" << pg_stats_publish.reported_seq << dendl;
}
+
pg_stats_publish_lock.Unlock();
}
void scrub_requested(bool deep, bool repair) override;
+ uint64_t get_snap_trimq_size() const override {
+ return snap_trimq.size();
+ }
+
void clear_publish_stats() override;
void clear_primary_state() override;
bool pg_stats_publish_valid;
pg_stat_t pg_stats_publish;
- void _update_calc_stats();
- void _update_blocked_by();
friend class TestOpsSocketHook;
void publish_stats_to_osd() override;
build_might_have_unfound();
- // Always call now so _update_calc_stats() will be accurate
+ // Always call now so update_calc_stats() will be accurate
discover_all_missing(query_map);
}
write_if_dirty(*t);
}
+void PeeringState::update_blocked_by()
+{
+ // set a max on the number of blocking peers we report. if we go
+ // over, report a random subset. keep the result sorted.
+ unsigned keep = std::min<unsigned>(
+ blocked_by.size(), cct->_conf->osd_max_pg_blocked_by);
+ unsigned skip = blocked_by.size() - keep;
+ info.stats.blocked_by.clear();
+ info.stats.blocked_by.resize(keep);
+ unsigned pos = 0;
+ for (set<int>::iterator p = blocked_by.begin();
+ p != blocked_by.end() && keep > 0;
+ ++p) {
+ if (skip > 0 && (rand() % (skip + keep) < skip)) {
+ --skip;
+ } else {
+ info.stats.blocked_by[pos++] = *p;
+ --keep;
+ }
+ }
+}
+
+static bool find_shard(const set<pg_shard_t> & pgs, shard_id_t shard)
+{
+ for (auto&p : pgs)
+ if (p.shard == shard)
+ return true;
+ return false;
+}
+
+static pg_shard_t get_another_shard(const set<pg_shard_t> & pgs, pg_shard_t skip, shard_id_t shard)
+{
+ for (auto&p : pgs) {
+ if (p == skip)
+ continue;
+ if (p.shard == shard)
+ return p;
+ }
+ return pg_shard_t();
+}
+
+void PeeringState::update_calc_stats()
+{
+ info.stats.version = info.last_update;
+ info.stats.created = info.history.epoch_created;
+ info.stats.last_scrub = info.history.last_scrub;
+ info.stats.last_scrub_stamp = info.history.last_scrub_stamp;
+ info.stats.last_deep_scrub = info.history.last_deep_scrub;
+ info.stats.last_deep_scrub_stamp = info.history.last_deep_scrub_stamp;
+ info.stats.last_clean_scrub_stamp = info.history.last_clean_scrub_stamp;
+ info.stats.last_epoch_clean = info.history.last_epoch_clean;
+
+ info.stats.log_size = pg_log.get_head().version - pg_log.get_tail().version;
+ info.stats.ondisk_log_size = info.stats.log_size;
+ info.stats.log_start = pg_log.get_tail();
+ info.stats.ondisk_log_start = pg_log.get_tail();
+ info.stats.snaptrimq_len = pl->get_snap_trimq_size();
+
+ unsigned num_shards = get_osdmap()->get_pg_size(info.pgid.pgid);
+
+ // In rare case that upset is too large (usually transient), use as target
+ // for calculations below.
+ unsigned target = std::max(num_shards, (unsigned)upset.size());
+ // For undersized actingset may be larger with OSDs out
+ unsigned nrep = std::max(actingset.size(), upset.size());
+ // calc num_object_copies
+ info.stats.stats.calc_copies(std::max(target, nrep));
+ info.stats.stats.sum.num_objects_degraded = 0;
+ info.stats.stats.sum.num_objects_unfound = 0;
+ info.stats.stats.sum.num_objects_misplaced = 0;
+ info.stats.avail_no_missing.clear();
+ info.stats.object_location_counts.clear();
+
+ if ((is_remapped() || is_undersized() || !is_clean()) &&
+ (is_peered()|| is_activating())) {
+ psdout(20) << __func__ << " actingset " << actingset << " upset "
+ << upset << " acting_recovery_backfill " << acting_recovery_backfill << dendl;
+
+ ceph_assert(!acting_recovery_backfill.empty());
+
+ bool estimate = false;
+
+ // NOTE: we only generate degraded, misplaced and unfound
+ // values for the summation, not individual stat categories.
+ int64_t num_objects = info.stats.stats.sum.num_objects;
+
+ // Objects missing from up nodes, sorted by # objects.
+ boost::container::flat_set<pair<int64_t,pg_shard_t>> missing_target_objects;
+ // Objects missing from nodes not in up, sort by # objects
+ boost::container::flat_set<pair<int64_t,pg_shard_t>> acting_source_objects;
+
+ // Fill missing_target_objects/acting_source_objects
+
+ {
+ int64_t missing;
+
+ // Primary first
+ missing = pg_log.get_missing().num_missing();
+ ceph_assert(acting_recovery_backfill.count(pg_whoami));
+ if (upset.count(pg_whoami)) {
+ missing_target_objects.emplace(missing, pg_whoami);
+ } else {
+ acting_source_objects.emplace(missing, pg_whoami);
+ }
+ info.stats.stats.sum.num_objects_missing_on_primary = missing;
+ if (missing == 0)
+ info.stats.avail_no_missing.push_back(pg_whoami);
+ psdout(20) << __func__ << " shard " << pg_whoami
+ << " primary objects " << num_objects
+ << " missing " << missing
+ << dendl;
+ }
+
+ // All other peers
+ for (auto& peer : peer_info) {
+ // Primary should not be in the peer_info, skip if it is.
+ if (peer.first == pg_whoami) continue;
+ int64_t missing = 0;
+ int64_t peer_num_objects = peer.second.stats.stats.sum.num_objects;
+ // Backfill targets always track num_objects accurately
+ // all other peers track missing accurately.
+ if (is_backfill_targets(peer.first)) {
+ missing = std::max((int64_t)0, num_objects - peer_num_objects);
+ } else {
+ if (peer_missing.count(peer.first)) {
+ missing = peer_missing[peer.first].num_missing();
+ } else {
+ psdout(20) << __func__ << " no peer_missing found for "
+ << peer.first << dendl;
+ if (is_recovering()) {
+ estimate = true;
+ }
+ missing = std::max((int64_t)0, num_objects - peer_num_objects);
+ }
+ }
+ if (upset.count(peer.first)) {
+ missing_target_objects.emplace(missing, peer.first);
+ } else if (actingset.count(peer.first)) {
+ acting_source_objects.emplace(missing, peer.first);
+ }
+ peer.second.stats.stats.sum.num_objects_missing = missing;
+ if (missing == 0)
+ info.stats.avail_no_missing.push_back(peer.first);
+ psdout(20) << __func__ << " shard " << peer.first
+ << " objects " << peer_num_objects
+ << " missing " << missing
+ << dendl;
+ }
+
+ // Compute object_location_counts
+ for (auto& ml: missing_loc.get_missing_locs()) {
+ info.stats.object_location_counts[ml.second]++;
+ psdout(30) << __func__ << " " << ml.first << " object_location_counts["
+ << ml.second << "]=" << info.stats.object_location_counts[ml.second]
+ << dendl;
+ }
+ int64_t not_missing = num_objects - missing_loc.get_missing_locs().size();
+ if (not_missing) {
+ // During recovery we know upset == actingset and is being populated
+ // During backfill we know that all non-missing objects are in the actingset
+ info.stats.object_location_counts[actingset] = not_missing;
+ }
+ psdout(30) << __func__ << " object_location_counts["
+ << upset << "]=" << info.stats.object_location_counts[upset]
+ << dendl;
+ psdout(20) << __func__ << " object_location_counts "
+ << info.stats.object_location_counts << dendl;
+
+ // A misplaced object is not stored on the correct OSD
+ int64_t misplaced = 0;
+ // a degraded objects has fewer replicas or EC shards than the pool specifies.
+ int64_t degraded = 0;
+
+ if (is_recovering()) {
+ for (auto& sml: missing_loc.get_missing_by_count()) {
+ for (auto& ml: sml.second) {
+ int missing_shards;
+ if (sml.first == shard_id_t::NO_SHARD) {
+ psdout(20) << __func__ << " ml " << ml.second
+ << " upset size " << upset.size()
+ << " up " << ml.first.up << dendl;
+ missing_shards = (int)upset.size() - ml.first.up;
+ } else {
+ // Handle shards not even in upset below
+ if (!find_shard(upset, sml.first))
+ continue;
+ missing_shards = std::max(0, 1 - ml.first.up);
+ psdout(20) << __func__
+ << " shard " << sml.first
+ << " ml " << ml.second
+ << " missing shards " << missing_shards << dendl;
+ }
+ int odegraded = ml.second * missing_shards;
+ // Copies on other osds but limited to the possible degraded
+ int more_osds = std::min(missing_shards, ml.first.other);
+ int omisplaced = ml.second * more_osds;
+ ceph_assert(omisplaced <= odegraded);
+ odegraded -= omisplaced;
+
+ misplaced += omisplaced;
+ degraded += odegraded;
+ }
+ }
+
+ psdout(20) << __func__ << " missing based degraded "
+ << degraded << dendl;
+ psdout(20) << __func__ << " missing based misplaced "
+ << misplaced << dendl;
+
+ // Handle undersized case
+ if (pool.info.is_replicated()) {
+ // Add degraded for missing targets (num_objects missing)
+ ceph_assert(target >= upset.size());
+ unsigned needed = target - upset.size();
+ degraded += num_objects * needed;
+ } else {
+ for (unsigned i = 0 ; i < num_shards; ++i) {
+ shard_id_t shard(i);
+
+ if (!find_shard(upset, shard)) {
+ pg_shard_t pgs = get_another_shard(actingset, pg_shard_t(), shard);
+
+ if (pgs != pg_shard_t()) {
+ int64_t missing;
+
+ if (pgs == pg_whoami)
+ missing = info.stats.stats.sum.num_objects_missing_on_primary;
+ else
+ missing = peer_info[pgs].stats.stats.sum.num_objects_missing;
+
+ degraded += missing;
+ misplaced += std::max((int64_t)0, num_objects - missing);
+ } else {
+ // No shard anywhere
+ degraded += num_objects;
+ }
+ }
+ }
+ }
+ goto out;
+ }
+
+ // Handle undersized case
+ if (pool.info.is_replicated()) {
+ // Add to missing_target_objects
+ ceph_assert(target >= missing_target_objects.size());
+ unsigned needed = target - missing_target_objects.size();
+ if (needed)
+ missing_target_objects.emplace(num_objects * needed, pg_shard_t(pg_shard_t::NO_OSD));
+ } else {
+ for (unsigned i = 0 ; i < num_shards; ++i) {
+ shard_id_t shard(i);
+ bool found = false;
+ for (const auto& t : missing_target_objects) {
+ if (std::get<1>(t).shard == shard) {
+ found = true;
+ break;
+ }
+ }
+ if (!found)
+ missing_target_objects.emplace(num_objects, pg_shard_t(pg_shard_t::NO_OSD,shard));
+ }
+ }
+
+ for (const auto& item : missing_target_objects)
+ psdout(20) << __func__ << " missing shard " << std::get<1>(item)
+ << " missing= " << std::get<0>(item) << dendl;
+ for (const auto& item : acting_source_objects)
+ psdout(20) << __func__ << " acting shard " << std::get<1>(item)
+ << " missing= " << std::get<0>(item) << dendl;
+
+ // Handle all objects not in missing for remapped
+ // or backfill
+ for (auto m = missing_target_objects.rbegin();
+ m != missing_target_objects.rend(); ++m) {
+
+ int64_t extra_missing = -1;
+
+ if (pool.info.is_replicated()) {
+ if (!acting_source_objects.empty()) {
+ auto extra_copy = acting_source_objects.begin();
+ extra_missing = std::get<0>(*extra_copy);
+ acting_source_objects.erase(extra_copy);
+ }
+ } else { // Erasure coded
+ // Use corresponding shard
+ for (const auto& a : acting_source_objects) {
+ if (std::get<1>(a).shard == std::get<1>(*m).shard) {
+ extra_missing = std::get<0>(a);
+ acting_source_objects.erase(a);
+ break;
+ }
+ }
+ }
+
+ if (extra_missing >= 0 && std::get<0>(*m) >= extra_missing) {
+ // We don't know which of the objects on the target
+ // are part of extra_missing so assume are all degraded.
+ misplaced += std::get<0>(*m) - extra_missing;
+ degraded += extra_missing;
+ } else {
+ // 1. extra_missing == -1, more targets than sources so degraded
+ // 2. extra_missing > std::get<0>(m), so that we know that some extra_missing
+ // previously degraded are now present on the target.
+ degraded += std::get<0>(*m);
+ }
+ }
+ // If there are still acting that haven't been accounted for
+ // then they are misplaced
+ for (const auto& a : acting_source_objects) {
+ int64_t extra_misplaced = std::max((int64_t)0, num_objects - std::get<0>(a));
+ psdout(20) << __func__ << " extra acting misplaced " << extra_misplaced
+ << dendl;
+ misplaced += extra_misplaced;
+ }
+out:
+ // NOTE: Tests use these messages to verify this code
+ psdout(20) << __func__ << " degraded " << degraded
+ << (estimate ? " (est)": "") << dendl;
+ psdout(20) << __func__ << " misplaced " << misplaced
+ << (estimate ? " (est)": "")<< dendl;
+
+ info.stats.stats.sum.num_objects_degraded = degraded;
+ info.stats.stats.sum.num_objects_unfound = get_num_unfound();
+ info.stats.stats.sum.num_objects_misplaced = misplaced;
+ }
+}
+
+std::optional<pg_stat_t> PeeringState::prepare_stats_for_publish(
+ bool pg_stats_publish_valid,
+ const pg_stat_t &pg_stats_publish,
+ const object_stat_collection_t &unstable_stats)
+{
+ if (info.stats.stats.sum.num_scrub_errors) {
+ state_set(PG_STATE_INCONSISTENT);
+ } else {
+ state_clear(PG_STATE_INCONSISTENT);
+ state_clear(PG_STATE_FAILED_REPAIR);
+ }
+
+ utime_t now = ceph_clock_now();
+ if (info.stats.state != state) {
+ info.stats.last_change = now;
+ // Optimistic estimation, if we just find out an inactive PG,
+ // assumt it is active till now.
+ if (!(state & PG_STATE_ACTIVE) &&
+ (info.stats.state & PG_STATE_ACTIVE))
+ info.stats.last_active = now;
+
+ if ((state & PG_STATE_ACTIVE) &&
+ !(info.stats.state & PG_STATE_ACTIVE))
+ info.stats.last_became_active = now;
+ if ((state & (PG_STATE_ACTIVE|PG_STATE_PEERED)) &&
+ !(info.stats.state & (PG_STATE_ACTIVE|PG_STATE_PEERED)))
+ info.stats.last_became_peered = now;
+ info.stats.state = state;
+ }
+
+ update_calc_stats();
+ if (info.stats.stats.sum.num_objects_degraded) {
+ state_set(PG_STATE_DEGRADED);
+ } else {
+ state_clear(PG_STATE_DEGRADED);
+ }
+ update_blocked_by();
+
+ pg_stat_t pre_publish = info.stats;
+ pre_publish.stats.add(unstable_stats);
+ utime_t cutoff = now;
+ cutoff -= cct->_conf->osd_pg_stat_report_interval_max;
+
+ if (get_osdmap()->require_osd_release >= CEPH_RELEASE_MIMIC) {
+ // share (some of) our purged_snaps via the pg_stats. limit # of intervals
+ // because we don't want to make the pg_stat_t structures too expensive.
+ unsigned max = cct->_conf->osd_max_snap_prune_intervals_per_epoch;
+ unsigned num = 0;
+ auto i = info.purged_snaps.begin();
+ while (num < max && i != info.purged_snaps.end()) {
+ pre_publish.purged_snaps.insert(i.get_start(), i.get_len());
+ ++num;
+ ++i;
+ }
+ psdout(20) << __func__ << " reporting purged_snaps "
+ << pre_publish.purged_snaps << dendl;
+ }
+
+ if (pg_stats_publish_valid && pre_publish == pg_stats_publish &&
+ info.stats.last_fresh > cutoff) {
+ psdout(15) << "publish_stats_to_osd " << pg_stats_publish.reported_epoch
+ << ": no change since " << info.stats.last_fresh << dendl;
+ return std::nullopt;
+ } else {
+ // update our stat summary and timestamps
+ info.stats.reported_epoch = get_osdmap_epoch();
+ ++info.stats.reported_seq;
+
+ info.stats.last_fresh = now;
+
+ if (info.stats.state & PG_STATE_CLEAN)
+ info.stats.last_clean = now;
+ if (info.stats.state & PG_STATE_ACTIVE)
+ info.stats.last_active = now;
+ if (info.stats.state & (PG_STATE_ACTIVE|PG_STATE_PEERED))
+ info.stats.last_peered = now;
+ info.stats.last_unstale = now;
+ if ((info.stats.state & PG_STATE_DEGRADED) == 0)
+ info.stats.last_undegraded = now;
+ if ((info.stats.state & PG_STATE_UNDERSIZED) == 0)
+ info.stats.last_fullsized = now;
+
+ psdout(15) << "publish_stats_to_osd " << pg_stats_publish.reported_epoch
+ << ":" << pg_stats_publish.reported_seq << dendl;
+ return std::make_optional(std::move(pre_publish));
+ }
+}
+
/*------------ Peering State Machine----------------*/
#undef dout_prefix
#define dout_prefix (context< PeeringMachine >().dpp->gen_prefix(*_dout) \
ceph_assert(ps->blocked_by.empty());
// Degraded?
- // _update_calc_stats(); TODOSAM
+ ps->update_calc_stats();
if (ps->info.stats.stats.sum.num_objects_degraded) {
ps->state_set(PG_STATE_DEGRADED);
} else {
virtual void on_info_history_change() = 0;
virtual void scrub_requested(bool deep, bool repair) = 0;
+ virtual uint64_t get_snap_trimq_size() const = 0;
+
virtual void send_cluster_message(
int osd, Message *m, epoch_t epoch, bool share_map_update=false) = 0;
virtual void send_pg_created(pg_t pgid) = 0;
void try_mark_clean();
+ void update_blocked_by();
+ void update_calc_stats();
public:
PeeringState(
CephContext *cct,
unsigned split_bits,
const pg_merge_meta_t& last_pg_merge_meta);
+ std::optional<pg_stat_t> prepare_stats_for_publish(
+ bool pg_stats_publish_valid,
+ const pg_stat_t &pg_stats_publish,
+ const object_stat_collection_t &unstable_stats);
+
void dump_history(Formatter *f) const {
state_history.dump(f);
}