pg_stats_valid(false),
osr(osd->osr_registry.lookup_or_create(p, (stringify(p)))),
finish_sync_event(NULL),
- finalizing_scrub(false),
- scrub_block_writes(false),
- scrub_active(false),
- scrub_reserved(false), scrub_reserve_failed(false),
- scrub_waiting_on(0),
- active_rep_scrub(0),
- scrub_is_chunky(false),
- scrub_errors(0),
- scrub_fixed(0),
active_pushes(0),
recovery_state(this)
{
log.reset_recovery_pointers();
- scrub_reserved_peers.clear();
+ scrubber.reserved_peers.clear();
osd->recovery_wq.dequeue(this);
osd->snap_trim_wq.dequeue(this);
}
/*
* when holding pg and sched_scrub_lock, then the states are:
* scheduling:
- * scrub_reserved = true
+ * scrubber.reserved = true
* scrub_rserved_peers includes whoami
* osd->scrub_pending++
* scheduling, replica declined:
- * scrub_reserved = true
- * scrub_reserved_peers includes -1
+ * scrubber.reserved = true
+ * scrubber.reserved_peers includes -1
* osd->scrub_pending++
* pending:
- * scrub_reserved = true
- * scrub_reserved_peers.size() == acting.size();
+ * scrubber.reserved = true
+ * scrubber.reserved_peers.size() == acting.size();
* pg on scrub_wq
* osd->scrub_pending++
* scrubbing:
- * scrub_reserved = false;
- * scrub_reserved_peers empty
- * osd->scrub_active++
+ * scrubber.reserved = false;
+ * scrubber.reserved_peers empty
+ * osd->scrubber.active++
*/
// returns false if waiting for a reply
}
bool ret = false;
- if (!scrub_reserved) {
- assert(scrub_reserved_peers.empty());
+ if (!scrubber.reserved) {
+ assert(scrubber.reserved_peers.empty());
if (osd->inc_scrubs_pending()) {
dout(20) << "sched_scrub: reserved locally, reserving replicas" << dendl;
- scrub_reserved = true;
- scrub_reserved_peers.insert(osd->whoami);
+ scrubber.reserved = true;
+ scrubber.reserved_peers.insert(osd->whoami);
scrub_reserve_replicas();
} else {
dout(20) << "sched_scrub: failed to reserve locally" << dendl;
}
}
- if (scrub_reserved) {
- if (scrub_reserve_failed) {
+ if (scrubber.reserved) {
+ if (scrubber.reserve_failed) {
dout(20) << "sched_scrub: failed, a peer declined" << dendl;
clear_scrub_reserved();
scrub_unreserve_replicas();
ret = true;
- } else if (scrub_reserved_peers.size() == acting.size()) {
+ } else if (scrubber.reserved_peers.size() == acting.size()) {
dout(20) << "sched_scrub: success, reserved self and replicas" << dendl;
queue_scrub();
ret = true;
} else {
- // none declined, since scrub_reserved is set
- dout(20) << "sched_scrub: reserved " << scrub_reserved_peers << ", waiting for replicas" << dendl;
+ // none declined, since scrubber.reserved is set
+ dout(20) << "sched_scrub: reserved " << scrubber.reserved_peers << ", waiting for replicas" << dendl;
}
}
dout(10) << " got osd." << from << " scrub map" << dendl;
bufferlist::iterator p = m->get_data().begin();
- if (scrub_is_chunky) { // chunky scrub
- scrub_received_maps[from].decode(p, info.pgid.pool());
- dout(10) << "map version is " << scrub_received_maps[from].valid_through << dendl;
+ if (scrubber.is_chunky) { // chunky scrub
+ scrubber.received_maps[from].decode(p, info.pgid.pool());
+ dout(10) << "map version is " << scrubber.received_maps[from].valid_through << dendl;
} else { // classic scrub
- if (scrub_received_maps.count(from)) {
+ if (scrubber.received_maps.count(from)) {
ScrubMap incoming;
incoming.decode(p, info.pgid.pool());
dout(10) << "from replica " << from << dendl;
dout(10) << "map version is " << incoming.valid_through << dendl;
- scrub_received_maps[from].merge_incr(incoming);
+ scrubber.received_maps[from].merge_incr(incoming);
} else {
- scrub_received_maps[from].decode(p, info.pgid.pool());
+ scrubber.received_maps[from].decode(p, info.pgid.pool());
}
}
- --scrub_waiting_on;
- scrub_waiting_on_whom.erase(from);
+ --scrubber.waiting_on;
+ scrubber.waiting_on_whom.erase(from);
- if (scrub_waiting_on == 0) {
- if (scrub_is_chunky) { // chunky scrub
+ if (scrubber.waiting_on == 0) {
+ if (scrubber.is_chunky) { // chunky scrub
osd->scrub_wq.queue(this);
} else { // classic scrub
- if (finalizing_scrub) { // incremental lists received
+ if (scrubber.finalizing) { // incremental lists received
osd->scrub_finalize_wq.queue(this);
} else { // initial lists received
- scrub_block_writes = true;
+ scrubber.block_writes = true;
if (last_update_applied == info.last_update) {
- finalizing_scrub = true;
+ scrubber.finalizing = true;
scrub_gather_replica_maps();
- ++scrub_waiting_on;
- scrub_waiting_on_whom.insert(osd->whoami);
+ ++scrubber.waiting_on;
+ scrubber.waiting_on_whom.insert(osd->whoami);
osd->scrub_wq.queue(this);
}
}
assert(m->get_header().type == MSG_OSD_SUBOP);
dout(7) << "sub_op_scrub_reserve" << dendl;
- if (scrub_reserved) {
+ if (scrubber.reserved) {
dout(10) << "Ignoring reserve request: Already reserved" << dendl;
return;
}
op->mark_started();
- scrub_reserved = osd->inc_scrubs_pending();
+ scrubber.reserved = osd->inc_scrubs_pending();
MOSDSubOpReply *reply = new MOSDSubOpReply(m, 0, get_osdmap()->get_epoch(), CEPH_OSD_FLAG_ACK);
- ::encode(scrub_reserved, reply->get_data());
+ ::encode(scrubber.reserved, reply->get_data());
osd->cluster_messenger->send_message(reply, m->get_connection());
}
assert(reply->get_header().type == MSG_OSD_SUBOPREPLY);
dout(7) << "sub_op_scrub_reserve_reply" << dendl;
- if (!scrub_reserved) {
+ if (!scrubber.reserved) {
dout(10) << "ignoring obsolete scrub reserve reply" << dendl;
return;
}
bool reserved;
::decode(reserved, p);
- if (scrub_reserved_peers.find(from) != scrub_reserved_peers.end()) {
+ if (scrubber.reserved_peers.find(from) != scrubber.reserved_peers.end()) {
dout(10) << " already had osd." << from << " reserved" << dendl;
} else {
if (reserved) {
dout(10) << " osd." << from << " scrub reserve = success" << dendl;
- scrub_reserved_peers.insert(from);
+ scrubber.reserved_peers.insert(from);
} else {
/* One decline stops this pg from being scheduled for scrubbing. */
dout(10) << " osd." << from << " scrub reserve = fail" << dendl;
- scrub_reserve_failed = true;
+ scrubber.reserve_failed = true;
}
sched_scrub();
}
dout(7) << "sub_op_scrub_stop" << dendl;
// see comment in sub_op_scrub_reserve
- scrub_reserved = false;
+ scrubber.reserved = false;
MOSDSubOpReply *reply = new MOSDSubOpReply(m, 0, get_osdmap()->get_epoch(), CEPH_OSD_FLAG_ACK);
osd->cluster_messenger->send_message(reply, m->get_connection());
void PG::clear_scrub_reserved()
{
osd->scrub_wq.dequeue(this);
- scrub_reserved_peers.clear();
- scrub_reserve_failed = false;
+ scrubber.reserved_peers.clear();
+ scrubber.reserve_failed = false;
- if (scrub_reserved) {
- scrub_reserved = false;
+ if (scrubber.reserved) {
+ scrubber.reserved = false;
osd->dec_scrubs_pending();
}
}
* If msg->scrub_from is not set, replica_scrub calls build_scrubmap to
* build a complete map (with the pg lock dropped).
*
- * If msg->scrub_from is set, replica_scrub sets finalizing_scrub.
+ * If msg->scrub_from is set, replica_scrub sets scrubber.finalizing.
* Similarly to scrub, if last_update_applied is behind info.last_update
* replica_scrub returns to be requeued by sub_op_modify_applied.
* replica_scrub then builds an incremental scrub map with the
*/
void PG::replica_scrub(MOSDRepScrub *msg)
{
- assert(!active_rep_scrub);
+ assert(!scrubber.active_rep_scrub);
dout(7) << "replica_scrub" << dendl;
if (msg->map_epoch < info.history.same_interval_since) {
- if (finalizing_scrub) {
+ if (scrubber.finalizing) {
dout(10) << "scrub pg changed, aborting" << dendl;
- finalizing_scrub = 0;
+ scrubber.finalizing = 0;
} else {
dout(10) << "replica_scrub discarding old replica_scrub from "
<< msg->map_epoch << " < " << info.history.same_interval_since
if (msg->chunky) { // chunky scrub
if (last_update_applied < msg->scrub_to) {
dout(10) << "waiting for last_update_applied to catch up" << dendl;
- active_rep_scrub = msg;
+ scrubber.active_rep_scrub = msg;
msg->get();
return;
}
if (active_pushes > 0) {
dout(10) << "waiting for active pushes to finish" << dendl;
- active_rep_scrub = msg;
+ scrubber.active_rep_scrub = msg;
msg->get();
return;
}
} else {
if (msg->scrub_from > eversion_t()) {
- if (finalizing_scrub) {
+ if (scrubber.finalizing) {
assert(last_update_applied == info.last_update);
assert(last_update_applied == msg->scrub_to);
} else {
- finalizing_scrub = 1;
+ scrubber.finalizing = 1;
if (last_update_applied != msg->scrub_to) {
- active_rep_scrub = msg;
+ scrubber.active_rep_scrub = msg;
msg->get();
return;
}
}
build_inc_scrub_map(map, msg->scrub_from);
- finalizing_scrub = 0;
+ scrubber.finalizing = 0;
} else {
build_scrub_map(map);
}
return;
}
- // when we're starting a scrub, we need to determine which type of scrub to do
- if (!scrub_active) {
+ // when the scrub is not active, we need to determine which type of scrub to do
+ if (!scrubber.active) {
OSDMapRef curmap = osd->get_osdmap();
- scrub_is_chunky = true;
+ scrubber.is_chunky = true;
for (unsigned i=1; i<acting.size(); i++) {
Connection *con = osd->cluster_messenger->get_connection(curmap->get_cluster_inst(acting[i]));
if (!(con->features & CEPH_FEATURE_CHUNKY_SCRUB)) {
dout(20) << "OSD " << acting[i]
<< " does not support chunky scrubs, falling back to classic"
<< dendl;
- scrub_is_chunky = false;
+ scrubber.is_chunky = false;
break;
}
}
- dout(10) << "starting a new " << (scrub_is_chunky ? "chunky" : "classic") << " scrub" << dendl;
+ dout(10) << "starting a new " << (scrubber.is_chunky ? "chunky" : "classic") << " scrub" << dendl;
}
- if (scrub_is_chunky) {
+ if (scrubber.is_chunky) {
chunky_scrub();
} else {
classic_scrub();
*
* Relevant variables:
*
- * scrub_waiting_on (int)
- * scrub_waiting_on_whom
+ * scrubber.waiting_on (int)
+ * scrubber.waiting_on_whom
* Number of people who still need to build an initial/incremental scrub map.
* This is decremented in sub_op_scrub_map.
*
*
* This is checked in op_applied.
*
- * scrub_block_writes
+ * scrubber.block_writes
* Flag to determine if writes are blocked.
*
* finalizing scrub
*/
void PG::classic_scrub()
{
- if (!scrub_active) {
+ if (!scrubber.active) {
dout(10) << "scrub start" << dendl;
- scrub_active = true;
+ scrubber.active = true;
update_stats();
- scrub_received_maps.clear();
- scrub_epoch_start = info.history.same_interval_since;
+ scrubber.received_maps.clear();
+ scrubber.epoch_start = info.history.same_interval_since;
osd->sched_scrub_lock.Lock();
- if (scrub_reserved) {
+ if (scrubber.reserved) {
--(osd->scrubs_pending);
assert(osd->scrubs_pending >= 0);
- scrub_reserved = false;
- scrub_reserved_peers.clear();
+ scrubber.reserved = false;
+ scrubber.reserved_peers.clear();
}
++(osd->scrubs_active);
osd->sched_scrub_lock.Unlock();
- /* scrub_waiting_on == 0 iff all replicas have sent the requested maps and
+ /* scrubber.waiting_on == 0 iff all replicas have sent the requested maps and
* the primary has done a final scrub (which in turn can only happen if
* last_update_applied == info.last_update)
*/
- scrub_waiting_on = acting.size();
- scrub_waiting_on_whom.insert(acting.begin(), acting.end());
+ scrubber.waiting_on = acting.size();
+ scrubber.waiting_on_whom.insert(acting.begin(), acting.end());
// request maps from replicas
for (unsigned i=1; i<acting.size(); i++) {
}
// Unlocks and relocks...
- primary_scrubmap = ScrubMap();
- build_scrub_map(primary_scrubmap);
+ scrubber.primary_scrubmap = ScrubMap();
+ build_scrub_map(scrubber.primary_scrubmap);
- if (scrub_epoch_start != info.history.same_interval_since) {
+ if (scrubber.epoch_start != info.history.same_interval_since) {
dout(10) << "scrub pg changed, aborting" << dendl;
scrub_clear_state();
scrub_unreserve_replicas();
return;
}
- --scrub_waiting_on;
- scrub_waiting_on_whom.erase(osd->whoami);
+ --scrubber.waiting_on;
+ scrubber.waiting_on_whom.erase(osd->whoami);
- if (scrub_waiting_on == 0) {
+ if (scrubber.waiting_on == 0) {
// the replicas have completed their scrub map, so lock out writes
- scrub_block_writes = true;
+ scrubber.block_writes = true;
} else {
dout(10) << "wait for replicas to build initial scrub map" << dendl;
return;
return;
}
- // fall through if last_update_applied == info.last_update and scrub_waiting_on == 0
+ // fall through if last_update_applied == info.last_update and scrubber.waiting_on == 0
// request incrementals from replicas
scrub_gather_replica_maps();
- ++scrub_waiting_on;
- scrub_waiting_on_whom.insert(osd->whoami);
+ ++scrubber.waiting_on;
+ scrubber.waiting_on_whom.insert(osd->whoami);
}
dout(10) << "clean up scrub" << dendl;
assert(last_update_applied == info.last_update);
- finalizing_scrub = true;
+ scrubber.finalizing = true;
- if (scrub_epoch_start != info.history.same_interval_since) {
+ if (scrubber.epoch_start != info.history.same_interval_since) {
dout(10) << "scrub pg changed, aborting" << dendl;
scrub_clear_state();
scrub_unreserve_replicas();
return;
}
- if (primary_scrubmap.valid_through != log.head) {
+ if (scrubber.primary_scrubmap.valid_through != log.head) {
ScrubMap incr;
- build_inc_scrub_map(incr, primary_scrubmap.valid_through);
- primary_scrubmap.merge_incr(incr);
+ build_inc_scrub_map(incr, scrubber.primary_scrubmap.valid_through);
+ scrubber.primary_scrubmap.merge_incr(incr);
}
- --scrub_waiting_on;
- scrub_waiting_on_whom.erase(osd->whoami);
- if (scrub_waiting_on == 0) {
+ --scrubber.waiting_on;
+ scrubber.waiting_on_whom.erase(osd->whoami);
+ if (scrubber.waiting_on == 0) {
assert(last_update_applied == info.last_update);
osd->scrub_finalize_wq.queue(this);
}
*
* This logic is encoded in the very linear state machine:
*
- * +---------------------+
- * ____________v_____________ |
- * | | |
- * | SCRUB_INACTIVE | |
- * |__________________________| |
- * | |
- * | +-------------+ |
- * ____________v___v_________ | |
- * | | | |
- * | SCRUB_NEW_CHUNK | | |
- * |__________________________| | |
- * | | |
- * ___________v_____________ | |
- * | | | |
- * | SCRUB_WAIT_PUSHES | | |
- * |_________________________| | |
- * | | |
- * ____________v_____________ | |
- * | | | |
- * | SCRUB_WAIT_LAST_UPDATE | | |
- * |__________________________| | |
- * | | |
- * ____________v_____________ | |
- * | | | |
- * | SCRUB_BUILD_MAP | | |
- * |__________________________| | |
- * | | |
- * ____________v_____________ | |
- * | | | |
- * | SCRUB_WAIT_REPLICAS | | |
- * |__________________________| | |
- * | | |
- * ____________v_____________ | |
- * | | | |
- * | SCRUB_COMPARE_MAPS | | |
- * |__________________________| | |
- * | | | |
- * | +-------------+ |
- * ____________v_____________ |
- * | | |
- * | SCRUB_FINISH | |
- * |__________________________| |
- * | |
- * +---------------------+
+ * +------------------+
+ * _________v__________ |
+ * | | |
+ * | INACTIVE | |
+ * |____________________| |
+ * | |
+ * | +----------+ |
+ * _________v___v______ | |
+ * | | | |
+ * | NEW_CHUNK | | |
+ * |____________________| | |
+ * | | |
+ * _________v__________ | |
+ * | | | |
+ * | WAIT_PUSHES | | |
+ * |____________________| | |
+ * | | |
+ * _________v__________ | |
+ * | | | |
+ * | WAIT_LAST_UPDATE | | |
+ * |____________________| | |
+ * | | |
+ * _________v__________ | |
+ * | | | |
+ * | BUILD_MAP | | |
+ * |____________________| | |
+ * | | |
+ * _________v__________ | |
+ * | | | |
+ * | WAIT_REPLICAS | | |
+ * |____________________| | |
+ * | | |
+ * _________v__________ | |
+ * | | | |
+ * | COMPARE_MAPS | | |
+ * |____________________| | |
+ * | | | |
+ * | +----------+ |
+ * _________v__________ |
+ * | | |
+ * | FINISH | |
+ * |____________________| |
+ * | |
+ * +------------------+
*
* The primary determines the last update from the subset by walking the log. If
* it sees a log entry pertaining to a file in the chunk, it tells the replicas
*
* In contrast to classic_scrub, chunky_scrub is entirely handled by scrub_wq.
*
- * scrub_state encodes the current state of the scrub (refer to state diagram
+ * scrubber.state encodes the current state of the scrub (refer to state diagram
* for details).
*/
void PG::chunky_scrub() {
// check for map changes
- if (is_chunky_scrub_active()) {
- if (scrub_epoch_start != info.history.same_interval_since) {
+ if (scrubber.is_chunky_scrub_active()) {
+ if (scrubber.epoch_start != info.history.same_interval_since) {
dout(10) << "scrub pg changed, aborting" << dendl;
scrub_clear_state();
scrub_unreserve_replicas();
int ret;
while (!done) {
- dout(20) << "scrub state " << scrub_string(scrub_state) << dendl;
+ dout(20) << "scrub state " << Scrubber::state_string(scrubber.state) << dendl;
- switch (scrub_state) {
- case SCRUB_INACTIVE:
+ switch (scrubber.state) {
+ case PG::Scrubber::INACTIVE:
dout(10) << "scrub start" << dendl;
update_stats();
- scrub_epoch_start = info.history.same_interval_since;
- scrub_active = true;
+ scrubber.epoch_start = info.history.same_interval_since;
+ scrubber.active = true;
osd->sched_scrub_lock.Lock();
- if (scrub_reserved) {
+ if (scrubber.reserved) {
--(osd->scrubs_pending);
assert(osd->scrubs_pending >= 0);
- scrub_reserved = false;
- scrub_reserved_peers.clear();
+ scrubber.reserved = false;
+ scrubber.reserved_peers.clear();
}
++(osd->scrubs_active);
osd->sched_scrub_lock.Unlock();
- scrub_start = hobject_t();
- scrub_state = SCRUB_NEW_CHUNK;
+ scrubber.start = hobject_t();
+ scrubber.state = PG::Scrubber::NEW_CHUNK;
break;
- case SCRUB_NEW_CHUNK:
- primary_scrubmap = ScrubMap();
- scrub_received_maps.clear();
+ case PG::Scrubber::NEW_CHUNK:
+ scrubber.primary_scrubmap = ScrubMap();
+ scrubber.received_maps.clear();
{
// list, and so forth.
bool boundary_found = false;
- hobject_t start = scrub_start;
+ hobject_t start = scrubber.start;
while (!boundary_found) {
vector<hobject_t> objects;
ret = osd->store->collection_list_partial(coll, start,
5, 5, 0,
- &objects, &scrub_end);
+ &objects, &scrubber.end);
assert(ret >= 0);
// in case we don't find a boundary: start again at the end
- start = scrub_end;
+ start = scrubber.end;
// special case: reached end of file store, implicitly a boundary
if (objects.size() == 0) {
}
// search backward from the end looking for a boundary
- objects.push_back(scrub_end);
+ objects.push_back(scrubber.end);
while (!boundary_found && objects.size() > 1) {
hobject_t end = objects.back();
objects.pop_back();
if (objects.back().get_filestore_key() != end.get_filestore_key()) {
- scrub_end = end;
+ scrubber.end = end;
boundary_found = true;
}
}
}
}
- scrub_block_writes = true;
+ scrubber.block_writes = true;
// walk the log to find the latest update that affects our chunk
- scrub_subset_last_update = eversion_t();
+ scrubber.subset_last_update = eversion_t();
for (list<pg_log_entry_t>::iterator p = log.log.begin();
p != log.log.end();
++p) {
- if (p->soid >= scrub_start && p->soid < scrub_end)
- scrub_subset_last_update = p->version;
+ if (p->soid >= scrubber.start && p->soid < scrubber.end)
+ scrubber.subset_last_update = p->version;
}
- // ask replicas to wait until last_update_applied >= scrub_subset_last_update and then scan
- scrub_waiting_on_whom.insert(osd->whoami);
- ++scrub_waiting_on;
+ // ask replicas to wait until last_update_applied >= scrubber.subset_last_update and then scan
+ scrubber.waiting_on_whom.insert(osd->whoami);
+ ++scrubber.waiting_on;
// request maps from replicas
for (unsigned i=1; i<acting.size(); i++) {
- _request_scrub_map(acting[i], scrub_subset_last_update,
- scrub_start, scrub_end);
- scrub_waiting_on_whom.insert(acting[i]);
- ++scrub_waiting_on;
+ _request_scrub_map(acting[i], scrubber.subset_last_update,
+ scrubber.start, scrubber.end);
+ scrubber.waiting_on_whom.insert(acting[i]);
+ ++scrubber.waiting_on;
}
- scrub_state = SCRUB_WAIT_PUSHES;
+ scrubber.state = PG::Scrubber::WAIT_PUSHES;
break;
- case SCRUB_WAIT_PUSHES:
+ case PG::Scrubber::WAIT_PUSHES:
if (active_pushes == 0) {
- scrub_state = SCRUB_WAIT_LAST_UPDATE;
+ scrubber.state = PG::Scrubber::WAIT_LAST_UPDATE;
} else {
dout(15) << "wait for pushes to apply" << dendl;
done = true;
}
break;
- case SCRUB_WAIT_LAST_UPDATE:
- if (last_update_applied >= scrub_subset_last_update) {
- scrub_state = SCRUB_BUILD_MAP;
+ case PG::Scrubber::WAIT_LAST_UPDATE:
+ if (last_update_applied >= scrubber.subset_last_update) {
+ scrubber.state = PG::Scrubber::BUILD_MAP;
} else {
// will be requeued by op_applied
dout(15) << "wait for writes to flush" << dendl;
}
break;
- case SCRUB_BUILD_MAP:
- assert(last_update_applied >= scrub_subset_last_update);
+ case PG::Scrubber::BUILD_MAP:
+ assert(last_update_applied >= scrubber.subset_last_update);
// build my own scrub map
- ret = build_scrub_map_chunk(primary_scrubmap, scrub_start, scrub_end);
+ ret = build_scrub_map_chunk(scrubber.primary_scrubmap, scrubber.start, scrubber.end);
if (ret < 0) {
dout(5) << "error building scrub map: " << ret << ", aborting" << dendl;
scrub_clear_state();
return;
}
- --scrub_waiting_on;
- scrub_waiting_on_whom.erase(osd->whoami);
+ --scrubber.waiting_on;
+ scrubber.waiting_on_whom.erase(osd->whoami);
- scrub_state = SCRUB_WAIT_REPLICAS;
+ scrubber.state = PG::Scrubber::WAIT_REPLICAS;
break;
- case SCRUB_WAIT_REPLICAS:
- if (scrub_waiting_on > 0) {
+ case PG::Scrubber::WAIT_REPLICAS:
+ if (scrubber.waiting_on > 0) {
// will be requeued by sub_op_scrub_map
dout(10) << "wait for replicas to build scrub map" << dendl;
done = true;
} else {
- scrub_state = SCRUB_COMPARE_MAPS;
+ scrubber.state = PG::Scrubber::COMPARE_MAPS;
}
break;
- case SCRUB_COMPARE_MAPS:
- assert(last_update_applied >= scrub_subset_last_update);
- assert(scrub_waiting_on == 0);
+ case PG::Scrubber::COMPARE_MAPS:
+ assert(last_update_applied >= scrubber.subset_last_update);
+ assert(scrubber.waiting_on == 0);
scrub_compare_maps();
- scrub_block_writes = false;
+ scrubber.block_writes = false;
// requeue the writes from the chunk that just finished
requeue_ops(waiting_for_active);
- if (scrub_end < hobject_t::get_max()) {
+ if (scrubber.end < hobject_t::get_max()) {
// schedule another leg of the scrub
- scrub_start = scrub_end;
+ scrubber.start = scrubber.end;
- scrub_state = SCRUB_NEW_CHUNK;
+ scrubber.state = PG::Scrubber::NEW_CHUNK;
osd->scrub_wq.queue(this);
done = true;
} else {
- scrub_state = SCRUB_FINISH;
+ scrubber.state = PG::Scrubber::FINISH;
}
break;
- case SCRUB_FINISH:
+ case PG::Scrubber::FINISH:
scrub_finish();
- scrub_state = SCRUB_INACTIVE;
+ scrubber.state = PG::Scrubber::INACTIVE;
done = true;
break;
requeue_ops(waiting_for_active);
- finalizing_scrub = false;
- scrub_block_writes = false;
- scrub_active = false;
- scrub_waiting_on = 0;
- scrub_waiting_on_whom.clear();
- if (active_rep_scrub) {
- active_rep_scrub->put();
- active_rep_scrub = NULL;
- }
- scrub_received_maps.clear();
-
- scrub_state = SCRUB_INACTIVE;
- scrub_start = hobject_t();
- scrub_end = hobject_t();
- scrub_subset_last_update = eversion_t();
- scrub_errors = 0;
- scrub_fixed = 0;
+ scrubber.reset();
// type-specific state clear
_scrub_clear_state();
}
bool PG::scrub_gather_replica_maps() {
- assert(scrub_waiting_on == 0);
+ assert(scrubber.waiting_on == 0);
assert(_lock.is_locked());
- for (map<int,ScrubMap>::iterator p = scrub_received_maps.begin();
- p != scrub_received_maps.end();
+ for (map<int,ScrubMap>::iterator p = scrubber.received_maps.begin();
+ p != scrubber.received_maps.end();
p++) {
- if (scrub_received_maps[p->first].valid_through != log.head) {
- scrub_waiting_on++;
- scrub_waiting_on_whom.insert(p->first);
+ if (scrubber.received_maps[p->first].valid_through != log.head) {
+ scrubber.waiting_on++;
+ scrubber.waiting_on_whom.insert(p->first);
// Need to request another incremental map
_request_scrub_map_classic(p->first, p->second.valid_through);
}
}
- if (scrub_waiting_on > 0) {
+ if (scrubber.waiting_on > 0) {
return false;
} else {
return true;
map<int,ScrubMap *> maps;
dout(2) << "scrub osd." << acting[0] << " has "
- << primary_scrubmap.objects.size() << " items" << dendl;
- maps[0] = &primary_scrubmap;
+ << scrubber.primary_scrubmap.objects.size() << " items" << dendl;
+ maps[0] = &scrubber.primary_scrubmap;
for (unsigned i=1; i<acting.size(); i++) {
dout(2) << "scrub osd." << acting[i] << " has "
- << scrub_received_maps[acting[i]].objects.size() << " items" << dendl;
- maps[i] = &scrub_received_maps[acting[i]];
+ << scrubber.received_maps[acting[i]].objects.size() << " items" << dendl;
+ maps[i] = &scrubber.received_maps[acting[i]];
}
_compare_scrubmaps(maps, missing, inconsistent, authoritative, ss);
}
// ok, do the pg-type specific scrubbing
- _scrub(primary_scrubmap);
+ _scrub(scrubber.primary_scrubmap);
}
void PG::scrub_finalize() {
assert(last_update_applied == info.last_update);
- if (scrub_epoch_start != info.history.same_interval_since) {
+ if (scrubber.epoch_start != info.history.same_interval_since) {
dout(10) << "scrub pg changed, aborting" << dendl;
scrub_clear_state();
scrub_unreserve_replicas();
// type-specific finish (can tally more errors)
_scrub_finish();
- if (scrub_errors == 0 || (repair && (scrub_errors - scrub_fixed) == 0))
+ if (scrubber.errors == 0 || (repair && (scrubber.errors - scrubber.fixed) == 0))
state_clear(PG_STATE_INCONSISTENT);
{
stringstream oss;
oss << info.pgid << " " << mode << " ";
- if (scrub_errors)
- oss << scrub_errors << " errors";
+ if (scrubber.errors)
+ oss << scrubber.errors << " errors";
else
oss << "ok";
if (repair)
- oss << ", " << scrub_fixed << " fixed";
+ oss << ", " << scrubber.fixed << " fixed";
oss << "\n";
- if (scrub_errors)
+ if (scrubber.errors)
osd->clog.error(oss);
else
osd->clog.info(oss);
{
q.f->open_object_section("scrub");
- q.f->dump_stream("scrub_epoch_start") << pg->scrub_epoch_start;
- q.f->dump_int("scrub_active", pg->scrub_active);
- q.f->dump_int("scrub_block_writes", pg->scrub_block_writes);
- q.f->dump_int("finalizing_scrub", pg->finalizing_scrub);
- q.f->dump_int("scrub_waiting_on", pg->scrub_waiting_on);
+ q.f->dump_stream("scrubber.epoch_start") << pg->scrubber.epoch_start;
+ q.f->dump_int("scrubber.active", pg->scrubber.active);
+ q.f->dump_int("scrubber.block_writes", pg->scrubber.block_writes);
+ q.f->dump_int("scrubber.finalizing", pg->scrubber.finalizing);
+ q.f->dump_int("scrubber.waiting_on", pg->scrubber.waiting_on);
{
- q.f->open_array_section("scrub_waiting_on_whom");
- for (set<int>::iterator p = pg->scrub_waiting_on_whom.begin();
- p != pg->scrub_waiting_on_whom.end();
+ q.f->open_array_section("scrubber.waiting_on_whom");
+ for (set<int>::iterator p = pg->scrubber.waiting_on_whom.begin();
+ p != pg->scrubber.waiting_on_whom.end();
++p) {
q.f->dump_int("osd", *p);
}
q.f->open_object_section("state");
q.f->dump_string("name", state_name);
q.f->dump_stream("enter_time") << enter_time;
- q.f->dump_int("finalizing_scrub", pg->finalizing_scrub);
+ q.f->dump_int("scrubber.finalizing", pg->scrubber.finalizing);
q.f->close_section();
return forward_event();
}