q.f->dump_stream("enter_time") << enter_time;
q.f->open_array_section("past_intervals");
- pg->past_intervals.dump(q.f);
+ ps->past_intervals.dump(q.f);
q.f->close_section();
q.f->open_array_section("probing_osds");
DECLARE_LOCALS
psdout(10) << "Leaving Peering" << dendl;
context< PeeringMachine >().log_exit(state_name, enter_time);
- pg->state_clear(PG_STATE_PEERING);
- pg->clear_probe_targets();
+ ps->state_clear(PG_STATE_PEERING);
+ pl->clear_probe_targets();
utime_t dur = ceph_clock_now() - enter_time;
pl->get_peering_perf().tinc(rs_peering_latency, dur);
{
DECLARE_LOCALS
psdout(10) << "backfill has unfound, can't continue" << dendl;
- pg->state_set(PG_STATE_BACKFILL_UNFOUND);
- pg->state_clear(PG_STATE_BACKFILLING);
+ ps->state_set(PG_STATE_BACKFILL_UNFOUND);
+ ps->state_clear(PG_STATE_BACKFILLING);
cancel_backfill();
return transit<NotBackfilling>();
}
PeeringState::Backfilling::react(const RemoteReservationRevoked &)
{
DECLARE_LOCALS
- pg->state_set(PG_STATE_BACKFILL_WAIT);
+ ps->state_set(PG_STATE_BACKFILL_WAIT);
cancel_backfill();
if (pg->needs_backfill()) {
return transit<WaitLocalBackfillReserved>();
{
context< PeeringMachine >().log_enter(state_name);
DECLARE_LOCALS
- pg->state_clear(PG_STATE_REPAIR);
- pg->publish_stats_to_osd();
+ ps->state_clear(PG_STATE_REPAIR);
+ pl->publish_stats_to_osd();
}
boost::statechart::result
context< PeeringMachine >().log_exit(state_name, enter_time);
DECLARE_LOCALS
- pg->state_clear(PG_STATE_BACKFILL_UNFOUND);
+ ps->state_clear(PG_STATE_BACKFILL_UNFOUND);
utime_t dur = ceph_clock_now() - enter_time;
pl->get_peering_perf().tinc(rs_notbackfilling_latency, dur);
}
{
context< PeeringMachine >().log_enter(state_name);
DECLARE_LOCALS
- pg->publish_stats_to_osd();
+ pl->publish_stats_to_osd();
}
void PeeringState::NotRecovering::exit()
context< PeeringMachine >().log_exit(state_name, enter_time);
DECLARE_LOCALS
- pg->state_clear(PG_STATE_RECOVERY_UNFOUND);
+ ps->state_clear(PG_STATE_RECOVERY_UNFOUND);
utime_t dur = ceph_clock_now() - enter_time;
pl->get_peering_perf().tinc(rs_notrecovering_latency, dur);
}
// to whatever possibly different partition that is storing the database.
// update_osd_stat() from heartbeat will do this on a new
- // statfs using pg->primary_num_bytes.
+ // statfs using ps->primary_num_bytes.
uint64_t pending_adjustment = 0;
int64_t primary_num_bytes = evt.primary_num_bytes;
int64_t local_num_bytes = evt.local_num_bytes;
if (primary_num_bytes) {
// For erasure coded pool overestimate by a full stripe per object
// because we don't know how each objected rounded to the nearest stripe
- if (pg->pool.info.is_erasure()) {
+ if (ps->pool.info.is_erasure()) {
primary_num_bytes /= (int)pg->get_pgbackend()->get_ec_data_chunk_count();
- primary_num_bytes += pg->get_pgbackend()->get_ec_stripe_chunk_size() * pg->info.stats.stats.sum.num_objects;
+ primary_num_bytes += pg->get_pgbackend()->get_ec_stripe_chunk_size() * ps->info.stats.stats.sum.num_objects;
local_num_bytes /= (int)pg->get_pgbackend()->get_ec_data_chunk_count();
- local_num_bytes += pg->get_pgbackend()->get_ec_stripe_chunk_size() * pg->info.stats.stats.sum.num_objects;
+ local_num_bytes += pg->get_pgbackend()->get_ec_stripe_chunk_size() * ps->info.stats.stats.sum.num_objects;
}
pending_adjustment = pending_backfill(
context< PeeringMachine >().cct,
// That's why we don't immediately unlock
Mutex::Locker l(pg->osd->stat_lock);
osd_stat_t cur_stat = pg->osd->osd_stat;
- if (pg->cct->_conf->osd_debug_reject_backfill_probability > 0 &&
- (rand()%1000 < (pg->cct->_conf->osd_debug_reject_backfill_probability*1000.0))) {
+ if (ps->cct->_conf->osd_debug_reject_backfill_probability > 0 &&
+ (rand()%1000 < (ps->cct->_conf->osd_debug_reject_backfill_probability*1000.0))) {
psdout(10) << "backfill reservation rejected: failure injection"
<< dendl;
post_event(RejectRemoteReservation());
- } else if (!pg->cct->_conf->osd_debug_skip_full_check_in_backfill_reservation &&
+ } else if (!ps->cct->_conf->osd_debug_skip_full_check_in_backfill_reservation &&
pg->osd->tentative_backfill_full(pg, pending_adjustment, cur_stat)) {
psdout(10) << "backfill reservation rejected: backfill full"
<< dendl;
// to test the other backfill full check AND in case a corruption
// of num_bytes requires ignoring that value and trying the
// backfill anyway.
- if (primary_num_bytes && !pg->cct->_conf->osd_debug_skip_full_check_in_backfill_reservation)
+ if (primary_num_bytes && !ps->cct->_conf->osd_debug_skip_full_check_in_backfill_reservation)
pg->set_reserved_num_bytes(primary_num_bytes, local_num_bytes);
else
pg->clear_reserved_num_bytes();
// Use un-ec-adjusted bytes for stats.
- pg->info.stats.stats.sum.num_bytes = evt.local_num_bytes;
+ ps->info.stats.stats.sum.num_bytes = evt.local_num_bytes;
PGPeeringEventRef preempt;
- if (HAVE_FEATURE(pg->upacting_features, RECOVERY_RESERVATION_2)) {
+ if (HAVE_FEATURE(ps->upacting_features, RECOVERY_RESERVATION_2)) {
// older peers will interpret preemption as TOOFULL
preempt = std::make_shared<PGPeeringEvent>(
pl->get_osdmap_epoch(),
context< PeeringMachine >().log_enter(state_name);
DECLARE_LOCALS
- pg->state_clear(PG_STATE_RECOVERY_WAIT);
- pg->state_clear(PG_STATE_RECOVERY_TOOFULL);
- pg->state_set(PG_STATE_RECOVERING);
- ceph_assert(!pg->state_test(PG_STATE_ACTIVATING));
- pg->publish_stats_to_osd();
+ ps->state_clear(PG_STATE_RECOVERY_WAIT);
+ ps->state_clear(PG_STATE_RECOVERY_TOOFULL);
+ ps->state_set(PG_STATE_RECOVERING);
+ ceph_assert(!ps->state_test(PG_STATE_ACTIVATING));
+ pl->publish_stats_to_osd();
pg->queue_recovery();
}
DECLARE_LOCALS
utime_t dur = ceph_clock_now() - enter_time;
- pg->state_clear(PG_STATE_RECOVERING);
+ ps->state_clear(PG_STATE_RECOVERING);
pl->get_peering_perf().tinc(rs_recovering_latency, dur);
}
// if we finished backfill, all acting are active; recheck if
// DEGRADED | UNDERSIZED is appropriate.
- ceph_assert(!pg->acting_recovery_backfill.empty());
- if (pg->get_osdmap()->get_pg_size(context< PeeringMachine >().spgid.pgid) <=
- pg->acting_recovery_backfill.size()) {
- pg->state_clear(PG_STATE_FORCED_BACKFILL | PG_STATE_FORCED_RECOVERY);
- pg->publish_stats_to_osd();
+ ceph_assert(!ps->acting_recovery_backfill.empty());
+ if (ps->get_osdmap()->get_pg_size(context< PeeringMachine >().spgid.pgid) <=
+ ps->acting_recovery_backfill.size()) {
+ ps->state_clear(PG_STATE_FORCED_BACKFILL | PG_STATE_FORCED_RECOVERY);
+ pl->publish_stats_to_osd();
}
// adjust acting set? (e.g. because backfill completed...)
bool history_les_bound = false;
- if (pg->acting != pg->up && !pg->choose_acting(auth_log_shard,
+ if (ps->acting != ps->up && !pg->choose_acting(auth_log_shard,
true, &history_les_bound)) {
- ceph_assert(pg->want_acting.size());
- } else if (!pg->async_recovery_targets.empty()) {
+ ceph_assert(ps->want_acting.size());
+ } else if (!ps->async_recovery_targets.empty()) {
pg->choose_acting(auth_log_shard, true, &history_les_bound);
}
if (context< Active >().all_replicas_activated &&
- pg->async_recovery_targets.empty())
+ ps->async_recovery_targets.empty())
post_event(GoClean());
}
DECLARE_LOCALS
- if (pg->info.last_complete != pg->info.last_update) {
+ if (ps->info.last_complete != ps->info.last_update) {
ceph_abort();
}
Context *c = pg->finish_recovery();
context< PeeringMachine >().log_exit(state_name, enter_time);
DECLARE_LOCALS
- pg->state_clear(PG_STATE_CLEAN);
+ ps->state_clear(PG_STATE_CLEAN);
utime_t dur = ceph_clock_now() - enter_time;
pl->get_peering_perf().tinc(rs_clean_latency, dur);
}
NamedState(context< PeeringMachine >().state_history, "Started/Primary/Active"),
remote_shards_to_reserve_recovery(
unique_osd_shard_set(
- context< PeeringMachine >().pg->pg_whoami,
- context< PeeringMachine >().pg->acting_recovery_backfill)),
+ context< PeeringMachine >().state->pg_whoami,
+ context< PeeringMachine >().state->acting_recovery_backfill)),
remote_shards_to_reserve_backfill(
unique_osd_shard_set(
- context< PeeringMachine >().pg->pg_whoami,
- context< PeeringMachine >().pg->backfill_targets)),
+ context< PeeringMachine >().state->pg_whoami,
+ context< PeeringMachine >().state->backfill_targets)),
all_replicas_activated(false)
{
context< PeeringMachine >().log_enter(state_name);
DECLARE_LOCALS
ceph_assert(!ps->backfill_reserved);
- ceph_assert(pg->is_primary());
+ ceph_assert(ps->is_primary());
psdout(10) << "In Active, about to call activate" << dendl;
ps->start_flush(context< PeeringMachine >().get_cur_transaction());
pg->activate(*context< PeeringMachine >().get_cur_transaction(),
- pg->get_osdmap_epoch(),
+ ps->get_osdmap_epoch(),
*context< PeeringMachine >().get_query_map(),
context< PeeringMachine >().get_info_map(),
context< PeeringMachine >().get_recovery_ctx());
// everyone has to commit/ack before we are truly active
- pg->blocked_by.clear();
- for (set<pg_shard_t>::iterator p = pg->acting_recovery_backfill.begin();
- p != pg->acting_recovery_backfill.end();
+ ps->blocked_by.clear();
+ for (set<pg_shard_t>::iterator p = ps->acting_recovery_backfill.begin();
+ p != ps->acting_recovery_backfill.end();
++p) {
- if (p->shard != pg->pg_whoami.shard) {
- pg->blocked_by.insert(p->shard);
+ if (p->shard != ps->pg_whoami.shard) {
+ ps->blocked_by.insert(p->shard);
}
}
- pg->publish_stats_to_osd();
+ pl->publish_stats_to_osd();
psdout(10) << "Activate Finished" << dendl;
}
if (advmap.osdmap->require_osd_release >= CEPH_RELEASE_MIMIC) {
const auto& new_removed_snaps = advmap.osdmap->get_new_removed_snaps();
- auto i = new_removed_snaps.find(pg->info.pgid.pool());
+ auto i = new_removed_snaps.find(ps->info.pgid.pool());
if (i != new_removed_snaps.end()) {
bool bad = false;
for (auto j : i->second) {
decltype(pg->snap_trimq) added, overlap;
added.insert(j.first, j.second);
overlap.intersection_of(pg->snap_trimq, added);
- if (pg->last_require_osd_release < CEPH_RELEASE_MIMIC) {
- lderr(pg->cct) << __func__ << " removed_snaps already contains "
+ if (ps->last_require_osd_release < CEPH_RELEASE_MIMIC) {
+ lderr(ps->cct) << __func__ << " removed_snaps already contains "
<< overlap << ", but this is the first mimic+ osdmap,"
<< " so it's expected" << dendl;
} else {
- lderr(pg->cct) << __func__ << " removed_snaps already contains "
+ lderr(ps->cct) << __func__ << " removed_snaps already contains "
<< overlap << dendl;
bad = true;
}
pg->snap_trimq.insert(j.first, j.second);
}
}
- if (pg->last_require_osd_release < CEPH_RELEASE_MIMIC) {
+ if (ps->last_require_osd_release < CEPH_RELEASE_MIMIC) {
// at upgrade, we report *all* previously removed snaps as removed in
// the first mimic epoch. remove the ones we previously divined were
// removed (and subsequently purged) from the trimq.
- lderr(pg->cct) << __func__ << " first mimic map, filtering purged_snaps"
+ lderr(ps->cct) << __func__ << " first mimic map, filtering purged_snaps"
<< " from new removed_snaps" << dendl;
- pg->snap_trimq.subtract(pg->info.purged_snaps);
+ pg->snap_trimq.subtract(ps->info.purged_snaps);
}
- ldout(pg->cct,10) << __func__ << " new removed_snaps " << i->second
+ ldout(ps->cct,10) << __func__ << " new removed_snaps " << i->second
<< ", snap_trimq now " << pg->snap_trimq << dendl;
- ceph_assert(!bad || !pg->cct->_conf->osd_debug_verify_cached_snaps);
- pg->dirty_info = true;
- pg->dirty_big_info = true;
+ ceph_assert(!bad || !ps->cct->_conf->osd_debug_verify_cached_snaps);
+ ps->dirty_info = true;
+ ps->dirty_big_info = true;
}
const auto& new_purged_snaps = advmap.osdmap->get_new_purged_snaps();
- auto j = new_purged_snaps.find(pg->info.pgid.pool());
+ auto j = new_purged_snaps.find(ps->info.pgid.pool());
if (j != new_purged_snaps.end()) {
bool bad = false;
for (auto k : j->second) {
- if (!pg->info.purged_snaps.contains(k.first, k.second)) {
- decltype(pg->info.purged_snaps) rm, overlap;
+ if (!ps->info.purged_snaps.contains(k.first, k.second)) {
+ decltype(ps->info.purged_snaps) rm, overlap;
rm.insert(k.first, k.second);
- overlap.intersection_of(pg->info.purged_snaps, rm);
- lderr(pg->cct) << __func__ << " purged_snaps does not contain "
+ overlap.intersection_of(ps->info.purged_snaps, rm);
+ lderr(ps->cct) << __func__ << " purged_snaps does not contain "
<< rm << ", only " << overlap << dendl;
- pg->info.purged_snaps.subtract(overlap);
+ ps->info.purged_snaps.subtract(overlap);
// This can currently happen in the normal (if unlikely) course of
// events. Because adding snaps to purged_snaps does not increase
// the pg version or add a pg log entry, we don't reliably propagate
// propagated.
//bad = true;
} else {
- pg->info.purged_snaps.erase(k.first, k.second);
+ ps->info.purged_snaps.erase(k.first, k.second);
}
}
- ldout(pg->cct,10) << __func__ << " new purged_snaps " << j->second
- << ", now " << pg->info.purged_snaps << dendl;
- ceph_assert(!bad || !pg->cct->_conf->osd_debug_verify_cached_snaps);
- pg->dirty_info = true;
- pg->dirty_big_info = true;
+ ldout(ps->cct,10) << __func__ << " new purged_snaps " << j->second
+ << ", now " << ps->info.purged_snaps << dendl;
+ ceph_assert(!bad || !ps->cct->_conf->osd_debug_verify_cached_snaps);
+ ps->dirty_info = true;
+ ps->dirty_big_info = true;
}
- if (pg->dirty_big_info) {
+ if (ps->dirty_big_info) {
// share updated purged_snaps to mgr/mon so that we (a) stop reporting
// purged snaps and (b) perhaps share more snaps that we have purged
// but didn't fit in pg_stat_t.
need_publish = true;
pg->share_pg_info();
}
- } else if (!pg->pool.newly_removed_snaps.empty()) {
- pg->snap_trimq.union_of(pg->pool.newly_removed_snaps);
+ } else if (!ps->pool.newly_removed_snaps.empty()) {
+ pg->snap_trimq.union_of(ps->pool.newly_removed_snaps);
psdout(10) << *pg << " snap_trimq now " << pg->snap_trimq << dendl;
- pg->dirty_info = true;
- pg->dirty_big_info = true;
+ ps->dirty_info = true;
+ ps->dirty_big_info = true;
}
- for (size_t i = 0; i < pg->want_acting.size(); i++) {
- int osd = pg->want_acting[i];
+ for (size_t i = 0; i < ps->want_acting.size(); i++) {
+ int osd = ps->want_acting[i];
if (!advmap.osdmap->is_up(osd)) {
pg_shard_t osd_with_shard(osd, shard_id_t(i));
- ceph_assert(pg->is_acting(osd_with_shard) || pg->is_up(osd_with_shard));
+ ceph_assert(ps->is_acting(osd_with_shard) || ps->is_up(osd_with_shard));
}
}
/* Check for changes in pool size (if the acting set changed as a result,
* this does not matter) */
if (advmap.lastmap->get_pg_size(context< PeeringMachine >().spgid.pgid) !=
- pg->get_osdmap()->get_pg_size(context< PeeringMachine >().spgid.pgid)) {
- if (pg->get_osdmap()->get_pg_size(context< PeeringMachine >().spgid.pgid) <=
- pg->actingset.size()) {
- pg->state_clear(PG_STATE_UNDERSIZED);
+ ps->get_osdmap()->get_pg_size(context< PeeringMachine >().spgid.pgid)) {
+ if (ps->get_osdmap()->get_pg_size(context< PeeringMachine >().spgid.pgid) <=
+ ps->actingset.size()) {
+ ps->state_clear(PG_STATE_UNDERSIZED);
} else {
- pg->state_set(PG_STATE_UNDERSIZED);
+ ps->state_set(PG_STATE_UNDERSIZED);
}
// degraded changes will be detected by call from publish_stats_to_osd()
need_publish = true;
}
// if we haven't reported our PG stats in a long time, do so now.
- if (pg->info.stats.reported_epoch + pg->cct->_conf->osd_pg_stat_report_interval_max < advmap.osdmap->get_epoch()) {
- psdout(20) << "reporting stats to osd after " << (advmap.osdmap->get_epoch() - pg->info.stats.reported_epoch)
+ if (ps->info.stats.reported_epoch + ps->cct->_conf->osd_pg_stat_report_interval_max < advmap.osdmap->get_epoch()) {
+ psdout(20) << "reporting stats to osd after " << (advmap.osdmap->get_epoch() - ps->info.stats.reported_epoch)
<< " epochs" << dendl;
need_publish = true;
}
if (need_publish)
- pg->publish_stats_to_osd();
+ pl->publish_stats_to_osd();
return forward_event();
}
{
DECLARE_LOCALS
psdout(10) << "Active: handling ActMap" << dendl;
- ceph_assert(pg->is_primary());
+ ceph_assert(ps->is_primary());
if (pg->have_unfound()) {
// object may have become unfound
pg->discover_all_missing(*context< PeeringMachine >().get_query_map());
}
- if (pg->cct->_conf->osd_check_for_log_corruption)
+ if (ps->cct->_conf->osd_check_for_log_corruption)
pg->check_log_for_corruption(pg->osd->store);
- uint64_t unfound = pg->missing_loc.num_unfound();
+ uint64_t unfound = ps->missing_loc.num_unfound();
if (unfound > 0 &&
- pg->all_unfound_are_queried_or_lost(pg->get_osdmap())) {
- if (pg->cct->_conf->osd_auto_mark_unfound_lost) {
- pg->osd->clog->error() << context< PeeringMachine >().spgid.pgid << " has " << unfound
+ pg->all_unfound_are_queried_or_lost(ps->get_osdmap())) {
+ if (ps->cct->_conf->osd_auto_mark_unfound_lost) {
+ pl->get_clog().error() << context< PeeringMachine >().spgid.pgid << " has " << unfound
<< " objects unfound and apparently lost, would automatically "
<< "mark these objects lost but this feature is not yet implemented "
<< "(osd_auto_mark_unfound_lost)";
} else
- pg->osd->clog->error() << context< PeeringMachine >().spgid.pgid << " has "
+ pl->get_clog().error() << context< PeeringMachine >().spgid.pgid << " has "
<< unfound << " objects unfound and apparently lost";
}
- if (pg->is_active()) {
+ if (ps->is_active()) {
psdout(10) << "Active: kicking snap trim" << dendl;
pg->kick_snap_trim();
}
- if (pg->is_peered() &&
- !pg->is_clean() &&
- !pg->get_osdmap()->test_flag(CEPH_OSDMAP_NOBACKFILL) &&
- (!pg->get_osdmap()->test_flag(CEPH_OSDMAP_NOREBALANCE) || pg->is_degraded())) {
+ if (ps->is_peered() &&
+ !ps->is_clean() &&
+ !ps->get_osdmap()->test_flag(CEPH_OSDMAP_NOBACKFILL) &&
+ (!ps->get_osdmap()->test_flag(CEPH_OSDMAP_NOREBALANCE) || ps->is_degraded())) {
pg->queue_recovery();
}
return forward_event();
{
DECLARE_LOCALS
- ceph_assert(pg->is_primary());
- if (pg->peer_info.count(notevt.from)) {
+ ceph_assert(ps->is_primary());
+ if (ps->peer_info.count(notevt.from)) {
psdout(10) << "Active: got notify from " << notevt.from
<< ", already have info from that osd, ignoring"
<< dendl;
- } else if (pg->peer_purged.count(notevt.from)) {
+ } else if (ps->peer_purged.count(notevt.from)) {
psdout(10) << "Active: got notify from " << notevt.from
<< ", already purged that peer, ignoring"
<< dendl;
boost::statechart::result PeeringState::Active::react(const MTrim& trim)
{
DECLARE_LOCALS
- ceph_assert(pg->is_primary());
+ ceph_assert(ps->is_primary());
// peer is informing us of their last_complete_ondisk
- ldout(pg->cct,10) << " replica osd." << trim.from << " lcod " << trim.trim_to << dendl;
- pg->peer_last_complete_ondisk[pg_shard_t(trim.from, trim.shard)] = trim.trim_to;
+ ldout(ps->cct,10) << " replica osd." << trim.from << " lcod " << trim.trim_to << dendl;
+ ps->peer_last_complete_ondisk[pg_shard_t(trim.from, trim.shard)] = trim.trim_to;
// trim log when the pg is recovered
pg->calc_min_last_complete_ondisk();
boost::statechart::result PeeringState::Active::react(const MInfoRec& infoevt)
{
DECLARE_LOCALS
- ceph_assert(pg->is_primary());
+ ceph_assert(ps->is_primary());
- ceph_assert(!pg->acting_recovery_backfill.empty());
+ ceph_assert(!ps->acting_recovery_backfill.empty());
// don't update history (yet) if we are active and primary; the replica
// may be telling us they have activated (and committed) but we can't
// share that until _everyone_ does the same.
- if (pg->is_acting_recovery_backfill(infoevt.from) &&
- pg->peer_activated.count(infoevt.from) == 0) {
+ if (ps->is_acting_recovery_backfill(infoevt.from) &&
+ ps->peer_activated.count(infoevt.from) == 0) {
psdout(10) << " peer osd." << infoevt.from
<< " activated and committed" << dendl;
- pg->peer_activated.insert(infoevt.from);
- pg->blocked_by.erase(infoevt.from.shard);
- pg->publish_stats_to_osd();
- if (pg->peer_activated.size() == pg->acting_recovery_backfill.size()) {
+ ps->peer_activated.insert(infoevt.from);
+ ps->blocked_by.erase(infoevt.from.shard);
+ pl->publish_stats_to_osd();
+ if (ps->peer_activated.size() == ps->acting_recovery_backfill.size()) {
pg->all_activated_and_committed();
}
}
pg->proc_replica_log(
logevt.msg->info, logevt.msg->log, logevt.msg->missing, logevt.from);
bool got_missing = pg->search_for_missing(
- pg->peer_info[logevt.from],
- pg->peer_missing[logevt.from],
+ ps->peer_info[logevt.from],
+ ps->peer_missing[logevt.from],
logevt.from,
context< PeeringMachine >().get_recovery_ctx());
// If there are missing AND we are "fully" active then start recovery now
- if (got_missing && pg->state_test(PG_STATE_ACTIVE)) {
+ if (got_missing && ps->state_test(PG_STATE_ACTIVE)) {
post_event(DoRecovery());
}
return discard_event();
{
q.f->open_array_section("might_have_unfound");
- for (set<pg_shard_t>::iterator p = pg->might_have_unfound.begin();
- p != pg->might_have_unfound.end();
+ for (set<pg_shard_t>::iterator p = ps->might_have_unfound.begin();
+ p != ps->might_have_unfound.end();
++p) {
q.f->open_object_section("osd");
q.f->dump_stream("osd") << *p;
- if (pg->peer_missing.count(*p)) {
+ if (ps->peer_missing.count(*p)) {
q.f->dump_string("status", "already probed");
- } else if (pg->peer_missing_requested.count(*p)) {
+ } else if (ps->peer_missing_requested.count(*p)) {
q.f->dump_string("status", "querying");
- } else if (!pg->get_osdmap()->is_up(p->osd)) {
+ } else if (!ps->get_osdmap()->is_up(p->osd)) {
q.f->dump_string("status", "osd is down");
} else {
q.f->dump_string("status", "not queried");
all_replicas_activated = true;
- pg->state_clear(PG_STATE_ACTIVATING);
- pg->state_clear(PG_STATE_CREATING);
- pg->state_clear(PG_STATE_PREMERGE);
+ ps->state_clear(PG_STATE_ACTIVATING);
+ ps->state_clear(PG_STATE_CREATING);
+ ps->state_clear(PG_STATE_PREMERGE);
bool merge_target;
- if (pg->pool.info.is_pending_merge(pgid, &merge_target)) {
- pg->state_set(PG_STATE_PEERED);
- pg->state_set(PG_STATE_PREMERGE);
+ if (ps->pool.info.is_pending_merge(pgid, &merge_target)) {
+ ps->state_set(PG_STATE_PEERED);
+ ps->state_set(PG_STATE_PREMERGE);
- if (pg->actingset.size() != pg->get_osdmap()->get_pg_size(pgid)) {
+ if (ps->actingset.size() != ps->get_osdmap()->get_pg_size(pgid)) {
if (merge_target) {
pg_t src = pgid;
- src.set_ps(pg->pool.info.get_pg_num_pending());
+ src.set_ps(ps->pool.info.get_pg_num_pending());
assert(src.get_parent() == pgid);
pg->osd->set_not_ready_to_merge_target(pgid, src);
} else {
pg->osd->set_not_ready_to_merge_source(pgid);
}
}
- } else if (pg->acting.size() < pg->pool.info.min_size) {
- pg->state_set(PG_STATE_PEERED);
+ } else if (ps->acting.size() < ps->pool.info.min_size) {
+ ps->state_set(PG_STATE_PEERED);
} else {
- pg->state_set(PG_STATE_ACTIVE);
+ ps->state_set(PG_STATE_ACTIVE);
}
- if (pg->pool.info.has_flag(pg_pool_t::FLAG_CREATING)) {
+ if (ps->pool.info.has_flag(pg_pool_t::FLAG_CREATING)) {
pg->osd->send_pg_created(pgid);
}
- pg->info.history.last_epoch_started = pg->info.last_epoch_started;
- pg->info.history.last_interval_started = pg->info.last_interval_started;
- pg->dirty_info = true;
+ ps->info.history.last_epoch_started = ps->info.last_epoch_started;
+ ps->info.history.last_interval_started = ps->info.last_interval_started;
+ ps->dirty_info = true;
pg->share_pg_info();
- pg->publish_stats_to_osd();
+ pl->publish_stats_to_osd();
pg->check_local();
DECLARE_LOCALS
pl->cancel_local_background_io_reservation();
- pg->blocked_by.clear();
+ ps->blocked_by.clear();
ps->backfill_reserved = false;
pg->backfill_reserving = false;
ps->state_clear(PG_STATE_ACTIVATING);
psdout(10) << "received log from " << logevt.from << dendl;
ObjectStore::Transaction* t = context<PeeringMachine>().get_cur_transaction();
pg->merge_log(*t, logevt.msg->info, logevt.msg->log, logevt.from);
- ceph_assert(pg->pg_log.get_head() == pg->info.last_update);
+ ceph_assert(ps->pg_log.get_head() == ps->info.last_update);
return discard_event();
}
{
DECLARE_LOCALS
// primary is instructing us to trim
- pg->pg_log.trim(trim.trim_to, pg->info);
- pg->dirty_info = true;
+ ps->pg_log.trim(trim.trim_to, ps->info);
+ ps->dirty_info = true;
return discard_event();
}
boost::statechart::result PeeringState::ReplicaActive::react(const ActMap&)
{
DECLARE_LOCALS
- if (pg->should_send_notify() && pg->get_primary().osd >= 0) {
+ if (ps->should_send_notify() && ps->get_primary().osd >= 0) {
context< PeeringMachine >().send_notify(
- pg->get_primary(),
+ ps->get_primary(),
pg_notify_t(
- pg->get_primary().shard, pg->pg_whoami.shard,
- pg->get_osdmap_epoch(),
- pg->get_osdmap_epoch(),
- pg->info),
- pg->past_intervals);
+ ps->get_primary().shard, ps->pg_whoami.shard,
+ ps->get_osdmap_epoch(),
+ ps->get_osdmap_epoch(),
+ ps->info),
+ ps->past_intervals);
}
return discard_event();
}
DECLARE_LOCALS
- ceph_assert(!pg->is_peered());
- ceph_assert(!pg->is_peering());
- ceph_assert(!pg->is_primary());
+ ceph_assert(!ps->is_peered());
+ ceph_assert(!ps->is_peering());
+ ceph_assert(!ps->is_primary());
- if (!pg->get_osdmap()->have_pg_pool(pg->get_pgid().pool())) {
- ldout(pg->cct,10) << __func__ << " pool is deleted" << dendl;
+ if (!ps->get_osdmap()->have_pg_pool(ps->info.pgid.pgid.pool())) {
+ ldout(ps->cct,10) << __func__ << " pool is deleted" << dendl;
post_event(DeleteStart());
} else {
ps->start_flush(context< PeeringMachine >().get_cur_transaction());
ObjectStore::Transaction* t = context<PeeringMachine>().get_cur_transaction();
if (msg->info.last_backfill == hobject_t()) {
// restart backfill
- pg->info = msg->info;
- pg->on_info_history_change();
- pg->dirty_info = true;
- pg->dirty_big_info = true; // maybe.
+ ps->info = msg->info;
+ pl->on_info_history_change();
+ ps->dirty_info = true;
+ ps->dirty_big_info = true; // maybe.
PG::PGLogEntryHandler rollbacker{pg, t};
- pg->pg_log.reset_backfill_claim_log(msg->log, &rollbacker);
+ ps->pg_log.reset_backfill_claim_log(msg->log, &rollbacker);
- pg->pg_log.reset_backfill();
+ ps->pg_log.reset_backfill();
} else {
pg->merge_log(*t, msg->info, msg->log, logevt.from);
}
- ceph_assert(pg->pg_log.get_head() == pg->info.last_update);
+ ceph_assert(ps->pg_log.get_head() == ps->info.last_update);
post_event(Activate(logevt.msg->info.last_epoch_started));
return transit<ReplicaActive>();
DECLARE_LOCALS
psdout(10) << "got info from osd." << infoevt.from << " " << infoevt.info << dendl;
- if (pg->info.last_update > infoevt.info.last_update) {
+ if (ps->info.last_update > infoevt.info.last_update) {
// rewind divergent log entries
ObjectStore::Transaction* t = context<PeeringMachine>().get_cur_transaction();
pg->rewind_divergent_log(*t, infoevt.info.last_update);
- pg->info.stats = infoevt.info.stats;
- pg->info.hit_set = infoevt.info.hit_set;
+ ps->info.stats = infoevt.info.stats;
+ ps->info.hit_set = infoevt.info.hit_set;
}
- ceph_assert(infoevt.info.last_update == pg->info.last_update);
- ceph_assert(pg->pg_log.get_head() == pg->info.last_update);
+ ceph_assert(infoevt.info.last_update == ps->info.last_update);
+ ceph_assert(ps->pg_log.get_head() == ps->info.last_update);
post_event(Activate(infoevt.info.last_epoch_started));
return transit<ReplicaActive>();
boost::statechart::result PeeringState::Stray::react(const ActMap&)
{
DECLARE_LOCALS
- if (pg->should_send_notify() && pg->get_primary().osd >= 0) {
+ if (ps->should_send_notify() && ps->get_primary().osd >= 0) {
context< PeeringMachine >().send_notify(
- pg->get_primary(),
+ ps->get_primary(),
pg_notify_t(
- pg->get_primary().shard, pg->pg_whoami.shard,
- pg->get_osdmap_epoch(),
- pg->get_osdmap_epoch(),
- pg->info),
- pg->past_intervals);
+ ps->get_primary().shard, ps->pg_whoami.shard,
+ ps->get_osdmap_epoch(),
+ ps->get_osdmap_epoch(),
+ ps->info),
+ ps->past_intervals);
}
return discard_event();
}
{
context< PeeringMachine >().log_enter(state_name);
DECLARE_LOCALS
- pg->deleting = true;
+ ps->deleting = true;
ObjectStore::Transaction* t = context<PeeringMachine>().get_cur_transaction();
pg->on_removal(t);
- t->register_on_commit(new PG::C_DeleteMore(pg, pg->get_osdmap_epoch()));
+ t->register_on_commit(new PG::C_DeleteMore(pg, ps->get_osdmap_epoch()));
}
boost::statechart::result PeeringState::Deleting::react(
DECLARE_LOCALS
PastIntervals::PriorSet &prior_set = context< Peering >().prior_set;
- pg->blocked_by.clear();
+ ps->blocked_by.clear();
for (set<pg_shard_t>::const_iterator it = prior_set.probe.begin();
it != prior_set.probe.end();
++it) {
pg_shard_t peer = *it;
- if (peer == pg->pg_whoami) {
+ if (peer == ps->pg_whoami) {
continue;
}
- if (pg->peer_info.count(peer)) {
- psdout(10) << " have osd." << peer << " info " << pg->peer_info[peer] << dendl;
+ if (ps->peer_info.count(peer)) {
+ psdout(10) << " have osd." << peer << " info " << ps->peer_info[peer] << dendl;
continue;
}
if (peer_info_requested.count(peer)) {
psdout(10) << " already requested info from osd." << peer << dendl;
- pg->blocked_by.insert(peer.osd);
- } else if (!pg->get_osdmap()->is_up(peer.osd)) {
+ ps->blocked_by.insert(peer.osd);
+ } else if (!ps->get_osdmap()->is_up(peer.osd)) {
psdout(10) << " not querying info from down osd." << peer << dendl;
} else {
psdout(10) << " querying info from osd." << peer << dendl;
context< PeeringMachine >().send_query(
peer, pg_query_t(pg_query_t::INFO,
- it->shard, pg->pg_whoami.shard,
- pg->info.history,
- pg->get_osdmap_epoch()));
+ it->shard, ps->pg_whoami.shard,
+ ps->info.history,
+ ps->get_osdmap_epoch()));
peer_info_requested.insert(peer);
- pg->blocked_by.insert(peer.osd);
+ ps->blocked_by.insert(peer.osd);
}
}
- pg->publish_stats_to_osd();
+ pl->publish_stats_to_osd();
}
boost::statechart::result PeeringState::GetInfo::react(const MNotifyRec& infoevt)
set<pg_shard_t>::iterator p = peer_info_requested.find(infoevt.from);
if (p != peer_info_requested.end()) {
peer_info_requested.erase(p);
- pg->blocked_by.erase(infoevt.from.osd);
+ ps->blocked_by.erase(infoevt.from.osd);
}
- epoch_t old_start = pg->info.history.last_epoch_started;
+ epoch_t old_start = ps->info.history.last_epoch_started;
if (ps->proc_replica_info(
infoevt.from, infoevt.notify.info, infoevt.notify.epoch_sent)) {
// we got something new ...
PastIntervals::PriorSet &prior_set = context< Peering >().prior_set;
- if (old_start < pg->info.history.last_epoch_started) {
+ if (old_start < ps->info.history.last_epoch_started) {
psdout(10) << " last_epoch_started moved forward, rebuilding prior" << dendl;
prior_set = ps->build_prior();
++p) {
q.f->open_object_section("osd");
q.f->dump_stream("osd") << *p;
- if (pg->peer_info.count(*p)) {
+ if (ps->peer_info.count(*p)) {
q.f->open_object_section("got_info");
- pg->peer_info[*p].dump(q.f);
+ ps->peer_info[*p].dump(q.f);
q.f->close_section();
}
q.f->close_section();
DECLARE_LOCALS
utime_t dur = ceph_clock_now() - enter_time;
pl->get_peering_perf().tinc(rs_getinfo_latency, dur);
- pg->blocked_by.clear();
+ ps->blocked_by.clear();
}
/*------GetLog------------*/
// adjust acting?
if (!pg->choose_acting(auth_log_shard, false,
&context< Peering >().history_les_bound)) {
- if (!pg->want_acting.empty()) {
+ if (!ps->want_acting.empty()) {
post_event(NeedActingChange());
} else {
post_event(IsIncomplete());
}
// am i the best?
- if (auth_log_shard == pg->pg_whoami) {
+ if (auth_log_shard == ps->pg_whoami) {
post_event(GotLog());
return;
}
- const pg_info_t& best = pg->peer_info[auth_log_shard];
+ const pg_info_t& best = ps->peer_info[auth_log_shard];
// am i broken?
- if (pg->info.last_update < best.log_tail) {
+ if (ps->info.last_update < best.log_tail) {
psdout(10) << " not contiguous with osd." << auth_log_shard << ", down" << dendl;
post_event(IsIncomplete());
return;
}
// how much log to request?
- eversion_t request_log_from = pg->info.last_update;
- ceph_assert(!pg->acting_recovery_backfill.empty());
- for (set<pg_shard_t>::iterator p = pg->acting_recovery_backfill.begin();
- p != pg->acting_recovery_backfill.end();
+ eversion_t request_log_from = ps->info.last_update;
+ ceph_assert(!ps->acting_recovery_backfill.empty());
+ for (set<pg_shard_t>::iterator p = ps->acting_recovery_backfill.begin();
+ p != ps->acting_recovery_backfill.end();
++p) {
- if (*p == pg->pg_whoami) continue;
- pg_info_t& ri = pg->peer_info[*p];
- if (ri.last_update < pg->info.log_tail && ri.last_update >= best.log_tail &&
+ if (*p == ps->pg_whoami) continue;
+ pg_info_t& ri = ps->peer_info[*p];
+ if (ri.last_update < ps->info.log_tail && ri.last_update >= best.log_tail &&
ri.last_update < request_log_from)
request_log_from = ri.last_update;
}
auth_log_shard,
pg_query_t(
pg_query_t::LOG,
- auth_log_shard.shard, pg->pg_whoami.shard,
- request_log_from, pg->info.history,
- pg->get_osdmap_epoch()));
+ auth_log_shard.shard, ps->pg_whoami.shard,
+ request_log_from, ps->info.history,
+ ps->get_osdmap_epoch()));
- ceph_assert(pg->blocked_by.empty());
- pg->blocked_by.insert(auth_log_shard.osd);
- pg->publish_stats_to_osd();
+ ceph_assert(ps->blocked_by.empty());
+ ps->blocked_by.insert(auth_log_shard.osd);
+ pl->publish_stats_to_osd();
}
boost::statechart::result PeeringState::GetLog::react(const AdvMap& advmap)
DECLARE_LOCALS
utime_t dur = ceph_clock_now() - enter_time;
pl->get_peering_perf().tinc(rs_getlog_latency, dur);
- pg->blocked_by.clear();
+ ps->blocked_by.clear();
}
/*------WaitActingChange--------*/
DECLARE_LOCALS
OSDMapRef osdmap = advmap.osdmap;
- psdout(10) << "verifying no want_acting " << pg->want_acting << " targets didn't go down" << dendl;
- for (vector<int>::iterator p = pg->want_acting.begin(); p != pg->want_acting.end(); ++p) {
+ psdout(10) << "verifying no want_acting " << ps->want_acting << " targets didn't go down" << dendl;
+ for (vector<int>::iterator p = ps->want_acting.begin(); p != ps->want_acting.end(); ++p) {
if (!osdmap->is_up(*p)) {
psdout(10) << " want_acting target osd." << *p << " went down, resetting" << dendl;
post_event(advmap);
context< PeeringMachine >().log_enter(state_name);
DECLARE_LOCALS
- pg->state_clear(PG_STATE_PEERING);
- pg->state_set(PG_STATE_DOWN);
+ ps->state_clear(PG_STATE_PEERING);
+ ps->state_set(PG_STATE_DOWN);
auto &prior_set = context< Peering >().prior_set;
- ceph_assert(pg->blocked_by.empty());
- pg->blocked_by.insert(prior_set.down.begin(), prior_set.down.end());
- pg->publish_stats_to_osd();
+ ceph_assert(ps->blocked_by.empty());
+ ps->blocked_by.insert(prior_set.down.begin(), prior_set.down.end());
+ pl->publish_stats_to_osd();
}
void PeeringState::Down::exit()
DECLARE_LOCALS
- pg->state_clear(PG_STATE_DOWN);
+ ps->state_clear(PG_STATE_DOWN);
utime_t dur = ceph_clock_now() - enter_time;
pl->get_peering_perf().tinc(rs_down_latency, dur);
- pg->blocked_by.clear();
+ ps->blocked_by.clear();
}
boost::statechart::result PeeringState::Down::react(const QueryState& q)
{
DECLARE_LOCALS
- ceph_assert(pg->is_primary());
- epoch_t old_start = pg->info.history.last_epoch_started;
- if (!pg->peer_info.count(infoevt.from) &&
- pg->get_osdmap()->has_been_up_since(infoevt.from.osd, infoevt.notify.epoch_sent)) {
- pg->update_history(infoevt.notify.info.history);
+ ceph_assert(ps->is_primary());
+ epoch_t old_start = ps->info.history.last_epoch_started;
+ if (!ps->peer_info.count(infoevt.from) &&
+ ps->get_osdmap()->has_been_up_since(infoevt.from.osd, infoevt.notify.epoch_sent)) {
+ ps->update_history(infoevt.notify.info.history);
}
// if we got something new to make pg escape down state
- if (pg->info.history.last_epoch_started > old_start) {
+ if (ps->info.history.last_epoch_started > old_start) {
psdout(10) << " last_epoch_started moved forward, re-enter getinfo" << dendl;
- pg->state_clear(PG_STATE_DOWN);
- pg->state_set(PG_STATE_PEERING);
+ ps->state_clear(PG_STATE_DOWN);
+ ps->state_set(PG_STATE_PEERING);
return transit< GetInfo >();
}
context< PeeringMachine >().log_enter(state_name);
DECLARE_LOCALS
- pg->state_clear(PG_STATE_PEERING);
- pg->state_set(PG_STATE_INCOMPLETE);
+ ps->state_clear(PG_STATE_PEERING);
+ ps->state_set(PG_STATE_INCOMPLETE);
PastIntervals::PriorSet &prior_set = context< Peering >().prior_set;
- ceph_assert(pg->blocked_by.empty());
- pg->blocked_by.insert(prior_set.down.begin(), prior_set.down.end());
- pg->publish_stats_to_osd();
+ ceph_assert(ps->blocked_by.empty());
+ ps->blocked_by.insert(prior_set.down.begin(), prior_set.down.end());
+ pl->publish_stats_to_osd();
}
boost::statechart::result PeeringState::Incomplete::react(const AdvMap &advmap) {
DECLARE_LOCALS
- int64_t poolnum = pg->info.pgid.pool();
+ int64_t poolnum = ps->info.pgid.pool();
// Reset if min_size turn smaller than previous value, pg might now be able to go active
if (!advmap.osdmap->have_pg_pool(poolnum) ||
DECLARE_LOCALS
- pg->state_clear(PG_STATE_INCOMPLETE);
+ ps->state_clear(PG_STATE_INCOMPLETE);
utime_t dur = ceph_clock_now() - enter_time;
pl->get_peering_perf().tinc(rs_incomplete_latency, dur);
- pg->blocked_by.clear();
+ ps->blocked_by.clear();
}
/*------GetMissing--------*/
context< PeeringMachine >().log_enter(state_name);
DECLARE_LOCALS
- ceph_assert(!pg->acting_recovery_backfill.empty());
+ ceph_assert(!ps->acting_recovery_backfill.empty());
eversion_t since;
- for (set<pg_shard_t>::iterator i = pg->acting_recovery_backfill.begin();
- i != pg->acting_recovery_backfill.end();
+ for (set<pg_shard_t>::iterator i = ps->acting_recovery_backfill.begin();
+ i != ps->acting_recovery_backfill.end();
++i) {
- if (*i == pg->get_primary()) continue;
- const pg_info_t& pi = pg->peer_info[*i];
+ if (*i == ps->get_primary()) continue;
+ const pg_info_t& pi = ps->peer_info[*i];
// reset this so to make sure the pg_missing_t is initialized and
// has the correct semantics even if we don't need to get a
// missing set from a shard. This way later additions due to
// lost+unfound delete work properly.
- pg->peer_missing[*i].may_include_deletes = !pg->perform_deletes_during_peering();
+ ps->peer_missing[*i].may_include_deletes = !pg->perform_deletes_during_peering();
if (pi.is_empty())
continue; // no pg data, nothing divergent
- if (pi.last_update < pg->pg_log.get_tail()) {
+ if (pi.last_update < ps->pg_log.get_tail()) {
psdout(10) << " osd." << *i << " is not contiguous, will restart backfill" << dendl;
- pg->peer_missing[*i].clear();
+ ps->peer_missing[*i].clear();
continue;
}
if (pi.last_backfill == hobject_t()) {
psdout(10) << " osd." << *i << " will fully backfill; can infer empty missing set" << dendl;
- pg->peer_missing[*i].clear();
+ ps->peer_missing[*i].clear();
continue;
}
if (pi.last_update == pi.last_complete && // peer has no missing
- pi.last_update == pg->info.last_update) { // peer is up to date
+ pi.last_update == ps->info.last_update) { // peer is up to date
// replica has no missing and identical log as us. no need to
// pull anything.
// FIXME: we can do better here. if last_update==last_complete we
// can infer the rest!
psdout(10) << " osd." << *i << " has no missing, identical log" << dendl;
- pg->peer_missing[*i].clear();
+ ps->peer_missing[*i].clear();
continue;
}
// We pull the log from the peer's last_epoch_started to ensure we
// get enough log to detect divergent updates.
since.epoch = pi.last_epoch_started;
- ceph_assert(pi.last_update >= pg->info.log_tail); // or else choose_acting() did a bad thing
+ ceph_assert(pi.last_update >= ps->info.log_tail); // or else choose_acting() did a bad thing
if (pi.log_tail <= since) {
psdout(10) << " requesting log+missing since " << since << " from osd." << *i << dendl;
context< PeeringMachine >().send_query(
*i,
pg_query_t(
pg_query_t::LOG,
- i->shard, pg->pg_whoami.shard,
- since, pg->info.history,
- pg->get_osdmap_epoch()));
+ i->shard, ps->pg_whoami.shard,
+ since, ps->info.history,
+ ps->get_osdmap_epoch()));
} else {
psdout(10) << " requesting fulllog+missing from osd." << *i
<< " (want since " << since << " < log.tail "
context< PeeringMachine >().send_query(
*i, pg_query_t(
pg_query_t::FULLLOG,
- i->shard, pg->pg_whoami.shard,
- pg->info.history, pg->get_osdmap_epoch()));
+ i->shard, ps->pg_whoami.shard,
+ ps->info.history, ps->get_osdmap_epoch()));
}
peer_missing_requested.insert(*i);
- pg->blocked_by.insert(i->osd);
+ ps->blocked_by.insert(i->osd);
}
if (peer_missing_requested.empty()) {
- if (pg->need_up_thru) {
+ if (ps->need_up_thru) {
psdout(10) << " still need up_thru update before going active"
<< dendl;
post_event(NeedUpThru());
}
// all good!
- post_event(Activate(pg->get_osdmap_epoch()));
+ post_event(Activate(ps->get_osdmap_epoch()));
} else {
- pg->publish_stats_to_osd();
+ pl->publish_stats_to_osd();
}
}
pg->proc_replica_log(logevt.msg->info, logevt.msg->log, logevt.msg->missing, logevt.from);
if (peer_missing_requested.empty()) {
- if (pg->need_up_thru) {
+ if (ps->need_up_thru) {
psdout(10) << " still need up_thru update before going active"
<< dendl;
post_event(NeedUpThru());
} else {
psdout(10) << "Got last missing, don't need missing "
<< "posting Activate" << dendl;
- post_event(Activate(pg->get_osdmap_epoch()));
+ post_event(Activate(ps->get_osdmap_epoch()));
}
}
return discard_event();
++p) {
q.f->open_object_section("osd");
q.f->dump_stream("osd") << *p;
- if (pg->peer_missing.count(*p)) {
+ if (ps->peer_missing.count(*p)) {
q.f->open_object_section("got_missing");
- pg->peer_missing[*p].dump(q.f);
+ ps->peer_missing[*p].dump(q.f);
q.f->close_section();
}
q.f->close_section();
DECLARE_LOCALS
utime_t dur = ceph_clock_now() - enter_time;
pl->get_peering_perf().tinc(rs_getmissing_latency, dur);
- pg->blocked_by.clear();
+ ps->blocked_by.clear();
}
/*------WaitUpThru--------*/
boost::statechart::result PeeringState::WaitUpThru::react(const ActMap& am)
{
DECLARE_LOCALS
- if (!pg->need_up_thru) {
- post_event(Activate(pg->get_osdmap_epoch()));
+ if (!ps->need_up_thru) {
+ post_event(Activate(ps->get_osdmap_epoch()));
}
return forward_event();
}
{
DECLARE_LOCALS
psdout(10) << "Noting missing from osd." << logevt.from << dendl;
- pg->peer_missing[logevt.from].claim(logevt.msg->missing);
- pg->peer_info[logevt.from] = logevt.msg->info;
+ ps->peer_missing[logevt.from].claim(logevt.msg->missing);
+ ps->peer_info[logevt.from] = logevt.msg->info;
return discard_event();
}