#include "Monitor.h"
#include "MDSMonitor.h"
#include "OSDMonitor.h"
+#include "MonitorStore.h"
#include "messages/MClientMount.h"
#include "messages/MClientUnmount.h"
+bool ClientMonitor::update_from_paxos()
+{
+ assert(paxos->is_active());
+
+ version_t paxosv = paxos->get_version();
+ dout(10) << "update_from_paxos paxosv " << paxosv
+ << ", my v " << client_map.version << endl;
+
+ if (paxosv == client_map.version) return true;
+ assert(paxosv >= client_map.version);
+
+ if (client_map.version == 0 && paxosv > 1 &&
+ mon->store->exists_bl_ss("clientmap","latest")) {
+ // starting up: load latest
+ dout(7) << "update_from_paxos startup: loading latest full clientmap" << endl;
+ bufferlist bl;
+ mon->store->get_bl_ss(bl, "clientmap", "latest");
+ int off = 0;
+ client_map._decode(bl, off);
+ }
+
+ // walk through incrementals
+ while (paxosv > client_map.version) {
+ bufferlist bl;
+ bool success = paxos->read(client_map.version+1, bl);
+ if (success) {
+ dout(7) << "update_from_paxos applying incremental " << client_map.version+1 << endl;
+ Incremental inc;
+ int off = 0;
+ inc._decode(bl, off);
+ client_map.apply_incremental(inc);
+
+ } else {
+ dout(7) << "update_from_paxos couldn't read incremental " << client_map.version+1 << endl;
+ return false;
+ }
+ }
+
+ // save latest
+ bufferlist bl;
+ client_map._encode(bl);
+ mon->store->put_bl_ss(bl, "clientmap", "latest");
+
+ return true;
+}
-void ClientMonitor::dispatch(Message *m)
+void ClientMonitor::create_pending()
{
- switch (m->get_type()) {
+ assert(mon->is_leader());
+ pending_inc = Incremental();
+ pending_inc.version = client_map.version + 1;
+ pending_inc.next_client = client_map.next_client;
+ dout(10) << "create_pending v " << pending_inc.version
+ << ", next is " << pending_inc.next_client
+ << endl;
+}
+
+void ClientMonitor::create_initial()
+{
+ dout(1) << "create_initial -- creating initial map" << endl;
+}
+
- case MSG_CLIENT_MOUNT:
- handle_client_mount((MClientMount*)m);
- break;
+void ClientMonitor::encode_pending(bufferlist &bl)
+{
+ assert(mon->is_leader());
+ dout(10) << "encode_pending v " << pending_inc.version
+ << ", next is " << pending_inc.next_client
+ << endl;
+
+ assert(paxos->get_version() + 1 == pending_inc.version);
+ pending_inc._encode(bl);
+}
+
+
+// -------
+
+
+bool ClientMonitor::preprocess_query(Message *m)
+{
+ dout(10) << "preprocess_query " << *m << " from " << m->get_source_inst() << endl;
+
+ switch (m->get_type()) {
+ case MSG_CLIENT_MOUNT:
+ {
+ // already mounted?
+ entity_addr_t addr = m->get_source_addr();
+ if (client_map.addr_client.count(addr)) {
+ int client = client_map.addr_client[addr];
+ dout(7) << " client" << client << " already mounted" << endl;
+ _mounted(client, (MClientMount*)m);
+ return true;
+ }
+ }
+ return false;
+
case MSG_CLIENT_UNMOUNT:
- handle_client_unmount((MClientUnmount*)m);
- break;
+ {
+ // already unmounted?
+ int client = m->get_source().num();
+ if (client_map.client_addr.count(client) == 0) {
+ dout(7) << " client" << client << " not mounted" << endl;
+ _unmounted((MClientUnmount*)m);
+ return true;
+ }
+ }
+ return false;
-
+
default:
assert(0);
- }
+ delete m;
+ return true;
+ }
}
-void ClientMonitor::handle_client_mount(MClientMount *m)
+bool ClientMonitor::prepare_update(Message *m)
{
- dout(7) << "client_mount from " << m->get_source_inst() << endl;
- assert(m->get_source().is_client());
- int from = m->get_source().num();
+ dout(10) << "prepare_update " << *m << " from " << m->get_source_inst() << endl;
- // choose a client id
- if (from < 0 ||
- (client_map.count(from) &&
- client_map[from] != m->get_source_addr())) {
- from = num_clients++;
- dout(10) << "client_mount assigned client" << from << endl;
- }
-
- client_map[from] = m->get_source_addr();
-
- // reply with latest mds map
- entity_inst_t to = m->get_source_inst();
- to.name = MSG_ADDR_CLIENT(from);
- mon->mdsmon->send_latest(to);
- mon->osdmon->send_latest(to);
- delete m;
-}
+ switch (m->get_type()) {
+ case MSG_CLIENT_MOUNT:
+ {
+ MClientMount *mount = (MClientMount*)m;
+ entity_addr_t addr = mount->addr;
+ int client = -1;
+ if (mount->get_source().is_client())
+ client = mount->get_source().num();
+
+ // choose a client id
+ if (client < 0 ||
+ (client_map.client_addr.count(client) &&
+ client_map.client_addr[client] != addr)) {
+ client = pending_inc.next_client;
+ dout(10) << "mount: assigned client" << client << " to " << addr << endl;
+ } else {
+ dout(10) << "mount: client" << client << " requested by " << addr << endl;
+ }
+
+ pending_inc.add_mount(client, addr);
+ paxos->wait_for_commit(new C_Mounted(this, client, mount));
+ }
+ return true;
-void ClientMonitor::handle_client_unmount(MClientUnmount *m)
-{
- dout(7) << "client_unmount from " << m->get_source()
- << " at " << m->get_source_inst() << endl;
- assert(m->get_source().is_client());
- int from = m->get_source().num();
-
- if (client_map.count(from)) {
- client_map.erase(from);
-
- if (client_map.empty() &&
- g_conf.mds_shutdown_on_last_unmount) {
- dout(1) << "last client unmounted" << endl;
- mon->do_stop();
+ case MSG_CLIENT_UNMOUNT:
+ {
+ MClientUnmount *unmount = (MClientUnmount*)m;
+ assert(unmount->inst.name.is_client());
+ int client = unmount->inst.name.num();
+
+ assert(client_map.client_addr.count(client));
+
+ pending_inc.add_unmount(client);
+ paxos->wait_for_commit(new C_Unmounted(this, unmount));
}
+ return true;
+
+ default:
+ assert(0);
+ delete m;
+ return false;
}
- // reply with (same) unmount message to ack
- mon->messenger->send_message(m, m->get_source_inst());
}
+// MOUNT
-/*
-void ClientMonitor::handle_mds_shutdown(Message *m)
-{
- assert(m->get_source().is_mds());
- int from = m->get_source().num();
- mdsmap.mds_inst.erase(from);
- mdsmap.all_mds.erase(from);
+void ClientMonitor::_mounted(int client, MClientMount *m)
+{
+ entity_inst_t to;
+ to.addr = m->addr;
+ to.name = MSG_ADDR_CLIENT(client);
- dout(7) << "mds_shutdown from " << m->get_source()
- << ", still have " << mdsmap.all_mds
- << endl;
-
- // tell someone?
- // fixme
+ dout(10) << "_mounted client" << client << " at " << to << endl;
+ // reply with latest mds, osd maps
+ mon->mdsmon->send_latest(to);
+ mon->osdmon->send_latest(0, to);
+
delete m;
}
-*/
-
-/*
-void ClientMonitor::bcast_latest_mds()
+void ClientMonitor::_unmounted(MClientUnmount *m)
{
- dout(10) << "bcast_latest_mds " << mdsmap.get_epoch() << endl;
+ dout(10) << "_unmounted " << m->inst << endl;
- // tell mds
- for (set<int>::iterator p = mdsmap.get_mds().begin();
- p != mdsmap.get_mds().end();
- p++) {
- if (mdsmap.is_down(*p)) continue;
- send_full(MSG_ADDR_MDS(*p), mdsmap.get_inst(*p));
+ // reply with (same) unmount message
+ mon->messenger->send_message(m, m->inst);
+
+ // auto-shutdown?
+ // (hack for fakesyn/newsyn, mostly)
+ if (mon->is_leader() &&
+ client_map.version > 1 &&
+ client_map.client_addr.empty() &&
+ g_conf.mon_stop_on_last_unmount) {
+ dout(1) << "last client unmounted" << endl;
+ mon->do_stop();
}
}
-*/
+
#include "mds/MDSMap.h"
+#include "PaxosService.h"
+
class Monitor;
+class Paxos;
+class MClientMount;
+class MClientUnmount;
+
+class ClientMonitor : public PaxosService {
+public:
+
+ struct Incremental {
+ version_t version;
+ uint32_t next_client;
+ map<int32_t, entity_addr_t> mount;
+ set<int32_t> unmount;
+
+ Incremental() : version(0), next_client() {}
+
+ bool is_empty() { return mount.empty() && unmount.empty(); }
+ void add_mount(uint32_t client, entity_addr_t addr) {
+ next_client = MAX(next_client, client+1);
+ mount[client] = addr;
+ }
+ void add_unmount(uint32_t client) {
+ assert(client < next_client);
+ if (mount.count(client))
+ mount.erase(client);
+ else
+ unmount.insert(client);
+ }
+
+ void _encode(bufferlist &bl) {
+ ::_encode(version, bl);
+ ::_encode(next_client, bl);
+ ::_encode(mount, bl);
+ ::_encode(unmount, bl);
+ }
+ void _decode(bufferlist &bl, int& off) {
+ ::_decode(version, bl, off);
+ ::_decode(next_client, bl, off);
+ ::_decode(mount, bl, off);
+ ::_decode(unmount, bl, off);
+ }
+ };
+
+ struct Map {
+ version_t version;
+ uint32_t next_client;
+ map<uint32_t,entity_addr_t> client_addr;
+ hash_map<entity_addr_t,uint32_t> addr_client;
+
+ Map() : version(0), next_client(0) {}
-class ClientMonitor : public Dispatcher {
- Monitor *mon;
- Messenger *messenger;
- Mutex &lock;
+ void reverse() {
+ addr_client.clear();
+ for (map<uint32_t,entity_addr_t>::iterator p = client_addr.begin();
+ p != client_addr.end();
+ ++p) {
+ addr_client[p->second] = p->first;
+ }
+ }
+ void apply_incremental(Incremental &inc) {
+ assert(inc.version == version+1);
+ version = inc.version;
+ next_client = inc.next_client;
+ for (map<int32_t, entity_addr_t>::iterator p = inc.mount.begin();
+ p != inc.mount.end();
+ ++p) {
+ client_addr[p->first] = p->second;
+ addr_client[p->second] = p->first;
+ }
+
+ for (set<int32_t>::iterator p = inc.unmount.begin();
+ p != inc.unmount.end();
+ ++p) {
+ assert(client_addr.count(*p));
+ addr_client.erase(client_addr[*p]);
+ client_addr.erase(*p);
+ }
+ }
- private:
- int num_clients;
- map<int,entity_addr_t> client_map;
+ void _encode(bufferlist &bl) {
+ ::_encode(version, bl);
+ ::_encode(next_client, bl);
+ ::_encode(client_addr, bl);
+ }
+ void _decode(bufferlist &bl, int& off) {
+ ::_decode(version, bl, off);
+ ::_decode(next_client, bl, off);
+ ::_decode(client_addr, bl, off);
+ reverse();
+ }
+ };
- void bcast_latest_mds();
+ class C_Mounted : public Context {
+ ClientMonitor *cmon;
+ int client;
+ MClientMount *m;
+ public:
+ C_Mounted(ClientMonitor *cm, int c, MClientMount *m_) :
+ cmon(cm), client(c), m(m_) {}
+ void finish(int r) {
+ if (r >= 0)
+ cmon->_mounted(client, m);
+ else
+ cmon->dispatch((Message*)m);
+ }
+ };
- //void accept_pending(); // accept pending, new map.
- //void send_incremental(epoch_t since, msg_addr_t dest);
+ class C_Unmounted : public Context {
+ ClientMonitor *cmon;
+ MClientUnmount *m;
+ public:
+ C_Unmounted(ClientMonitor *cm, MClientUnmount *m_) :
+ cmon(cm), m(m_) {}
+ void finish(int r) {
+ if (r >= 0)
+ cmon->_unmounted(m);
+ else
+ cmon->dispatch((Message*)m);
+ }
+ };
- void handle_client_mount(class MClientMount *m);
- void handle_client_unmount(class MClientUnmount *m);
+private:
+ Map client_map;
+
+ // leader
+ Incremental pending_inc;
+
+ void create_initial();
+ bool update_from_paxos();
+ void create_pending(); // prepare a new pending
+ void encode_pending(bufferlist &bl); // propose pending update to peers
+
+ void _mounted(int c, MClientMount *m);
+ void _unmounted(MClientUnmount *m);
+
+ bool preprocess_query(Message *m); // true if processed.
+ bool prepare_update(Message *m);
+
+
public:
- ClientMonitor(Monitor *mn, Messenger *m, Mutex& l) : mon(mn), messenger(m), lock(l),
- num_clients(0) { }
+ ClientMonitor(Monitor *mn, Paxos *p) : PaxosService(mn, p) { }
- void dispatch(Message *m);
- void tick(); // check state, take actions
+ //void tick(); // check state, take actions
+
};
#endif
#include "Monitor.h"
#include "common/Timer.h"
-
-#include "messages/MMonElectionPropose.h"
-#include "messages/MMonElectionAck.h"
-#include "messages/MMonElectionVictory.h"
+#include "MonitorStore.h"
+#include "messages/MMonElection.h"
#include "config.h"
#undef dout
-#define derr(l) if (l<=g_conf.debug || l<=g_conf.debug_mon) cerr << g_clock.now() << " mon" << mon->whoami << (mon->is_starting() ? (const char*)"(starting)":(mon->is_leader() ? (const char*)"(leader)":(mon->is_peon() ? (const char*)"(peon)":(const char*)"(?\?)"))) << ".elector "
-#define dout(l) if (l<=g_conf.debug || l<=g_conf.debug_mon) cout << g_clock.now() << " mon" << mon->whoami << (mon->is_starting() ? (const char*)"(starting)":(mon->is_leader() ? (const char*)"(leader)":(mon->is_peon() ? (const char*)"(peon)":(const char*)"(?\?)"))) << ".elector "
+#define derr(l) if (l<=g_conf.debug || l<=g_conf.debug_mon) cerr << g_clock.now() << " mon" << mon->whoami << (mon->is_starting() ? (const char*)"(starting)":(mon->is_leader() ? (const char*)"(leader)":(mon->is_peon() ? (const char*)"(peon)":(const char*)"(?\?)"))) << ".elector(" << epoch << ") "
+#define dout(l) if (l<=g_conf.debug || l<=g_conf.debug_mon) cout << g_clock.now() << " mon" << mon->whoami << (mon->is_starting() ? (const char*)"(starting)":(mon->is_leader() ? (const char*)"(leader)":(mon->is_peon() ? (const char*)"(peon)":(const char*)"(?\?)"))) << ".elector(" << epoch << ") "
-void Elector::start()
+void Elector::init()
{
- dout(5) << "start -- can i be leader?" << endl;
+ epoch = mon->store->get_int("mon_epoch");
+ if (!epoch)
+ epoch = 1;
+ dout(1) << "init, last seen epoch " << epoch << endl;
+}
+
+void Elector::shutdown()
+{
+ if (expire_event)
+ mon->timer.cancel_event(expire_event);
+}
+
+void Elector::bump_epoch(epoch_t e)
+{
+ dout(10) << "bump_epoch " << epoch << " to " << e << endl;
+ assert(epoch < e);
+ epoch = e;
+ mon->store->put_int(epoch, "mon_epoch");
+ // clear up some state
+ electing_me = false;
+ acked_me.clear();
leader_acked = -1;
+}
+
+void Elector::start()
+{
+ dout(5) << "start -- can i be leader?" << endl;
+
// start by trying to elect me
+ if (epoch % 2 == 0)
+ bump_epoch(epoch+1); // odd == election cycle
start_stamp = g_clock.now();
- acked_me.clear();
- acked_me.insert(whoami);
electing_me = true;
+ acked_me.insert(whoami);
// bcast to everyone else
for (int i=0; i<mon->monmap->num_mon; ++i) {
if (i == whoami) continue;
- mon->messenger->send_message(new MMonElectionPropose,
+ mon->messenger->send_message(new MMonElection(MMonElection::OP_PROPOSE, epoch),
mon->monmap->get_inst(i));
}
dout(5) << "defer to " << who << endl;
if (electing_me) {
+ // drop out
acked_me.clear();
electing_me = false;
}
// ack them
leader_acked = who;
ack_stamp = g_clock.now();
- mon->messenger->send_message(new MMonElectionAck,
+ mon->messenger->send_message(new MMonElection(MMonElection::OP_ACK, epoch),
mon->monmap->get_inst(who));
// set a timer
}
-class C_Mon_ElectionExpire : public Context {
- Elector *elector;
-public:
- C_Mon_ElectionExpire(Elector *e) : elector(e) { }
- void finish(int r) {
- elector->expire();
- }
-};
-
void Elector::reset_timer(double plus)
{
// set the timer
cancel_timer();
- expire_event = new C_Mon_ElectionExpire(this);
- g_timer.add_event_after(g_conf.mon_lease + plus,
- expire_event);
+ expire_event = new C_ElectionExpire(this);
+ mon->timer.add_event_after(g_conf.mon_lease + plus,
+ expire_event);
}
void Elector::cancel_timer()
{
- if (expire_event)
- g_timer.cancel_event(expire_event);
+ if (expire_event) {
+ mon->timer.cancel_event(expire_event);
+ expire_event = 0;
+ }
}
void Elector::expire()
{
leader_acked = -1;
electing_me = false;
-
+ set<int> quorum = acked_me;
+
cancel_timer();
-
+
+ assert(epoch % 2 == 1); // election
+ bump_epoch(epoch+1); // is over!
+
// tell everyone
- for (int i=0; i<mon->monmap->num_mon; ++i) {
- if (i == whoami) continue;
- mon->messenger->send_message(new MMonElectionVictory,
- mon->monmap->get_inst(i));
+ for (set<int>::iterator p = quorum.begin();
+ p != quorum.end();
+ ++p) {
+ if (*p == whoami) continue;
+ mon->messenger->send_message(new MMonElection(MMonElection::OP_VICTORY, epoch),
+ mon->monmap->get_inst(*p));
}
// tell monitor
- mon->win_election(acked_me);
+ mon->win_election(epoch, quorum);
}
-void Elector::handle_propose(MMonElectionPropose *m)
+void Elector::handle_propose(MMonElection *m)
{
dout(5) << "handle_propose from " << m->get_source() << endl;
int from = m->get_source().num();
- if (from > whoami) {
- if (leader_acked >= 0 && // we already acked someone
- leader_acked < from) { // who would win over them
+ assert(m->epoch % 2 == 1); // election
+ if (m->epoch > epoch) {
+ bump_epoch(m->epoch);
+ }
+ else if (m->epoch < epoch && // got an "old" propose,
+ epoch % 2 == 0 && // in a non-election cycle
+ mon->quorum.count(from) == 0) { // from someone outside the quorum
+ // a mon just started up, call a new election so they can rejoin!
+ dout(5) << " got propose from old epoch, " << m->get_source() << " must have just started" << endl;
+ start();
+ }
+
+ if (whoami < from) {
+ // i would win over them.
+ if (leader_acked >= 0) { // we already acked someone
+ assert(leader_acked < from); // and they still win, of course
dout(5) << "no, we already acked " << leader_acked << endl;
} else {
// wait, i should win!
delete m;
}
-void Elector::handle_ack(MMonElectionAck *m)
+void Elector::handle_ack(MMonElection *m)
{
dout(5) << "handle_ack from " << m->get_source() << endl;
int from = m->get_source().num();
+ assert(m->epoch % 2 == 1); // election
+ if (m->epoch > epoch) {
+ dout(5) << "woah, that's a newer epoch, i must have rebooted. bumping and re-starting!" << endl;
+ bump_epoch(m->epoch);
+ start();
+ delete m;
+ return;
+ }
+ assert(m->epoch == epoch);
+
if (electing_me) {
// thanks
acked_me.insert(from);
}
} else {
// ignore, i'm deferring already.
+ assert(leader_acked >= 0);
}
delete m;
}
-void Elector::handle_victory(MMonElectionVictory *m)
+
+void Elector::handle_victory(MMonElection *m)
{
dout(5) << "handle_victory from " << m->get_source() << endl;
int from = m->get_source().num();
+
+ assert(from < whoami);
+ assert(m->epoch % 2 == 0);
+ assert(m->epoch == epoch + 1); // i should have seen this election if i'm getting the victory.
+ bump_epoch(m->epoch);
- if (from < whoami) {
- // ok, fine, they win
- mon->lose_election(from);
-
- // cancel my timer
- cancel_timer();
- } else {
- // no, that makes no sense, i should win. start over!
- start();
- }
+ // they win
+ mon->lose_election(epoch, from);
+
+ // cancel my timer
+ cancel_timer();
}
void Elector::dispatch(Message *m)
{
switch (m->get_type()) {
- case MSG_MON_ELECTION_ACK:
- handle_ack((MMonElectionAck*)m);
- break;
-
- case MSG_MON_ELECTION_PROPOSE:
- handle_propose((MMonElectionPropose*)m);
- break;
-
- case MSG_MON_ELECTION_VICTORY:
- handle_victory((MMonElectionVictory*)m);
+
+ case MSG_MON_ELECTION:
+ {
+ MMonElection *em = (MMonElection*)m;
+
+ switch (em->op) {
+ case MMonElection::OP_PROPOSE:
+ handle_propose(em);
+ return;
+ }
+
+ if (em->epoch < epoch) {
+ dout(5) << "old epoch, dropping" << endl;
+ delete em;
+ break;
+ }
+
+ switch (em->op) {
+ case MMonElection::OP_ACK:
+ handle_ack(em);
+ return;
+ case MMonElection::OP_VICTORY:
+ handle_victory(em);
+ return;
+ default:
+ assert(0);
+ }
+ }
break;
- default:
+ default:
assert(0);
}
}
void reset_timer(double plus=0.0);
void cancel_timer();
+ epoch_t epoch; // latest epoch we've seen. odd == election, even == stable,
+
// electing me
bool electing_me;
utime_t start_stamp;
int leader_acked; // who i've acked
utime_t ack_stamp; // and when
- public:
-
+ void bump_epoch(epoch_t e=0); // i just saw a larger epoch
+
+ class C_ElectionExpire : public Context {
+ Elector *elector;
+ public:
+ C_ElectionExpire(Elector *e) : elector(e) { }
+ void finish(int r) {
+ elector->expire();
+ }
+ };
+
void start(); // start an electing me
void defer(int who);
void expire(); // timer goes off
void victory();
- void handle_propose(class MMonElectionPropose *m);
- void handle_ack(class MMonElectionAck *m);
- void handle_victory(class MMonElectionVictory *m);
-
+ void handle_propose(class MMonElection *m);
+ void handle_ack(class MMonElection *m);
+ void handle_victory(class MMonElection *m);
public:
- Elector(Monitor *m, int w) : mon(m), whoami(w) {
- // initialize all those values!
- // ...
- }
+ Elector(Monitor *m, int w) : mon(m), whoami(w),
+ expire_event(0),
+ epoch(0),
+ electing_me(false),
+ leader_acked(-1) { }
+
+ void init();
+ void shutdown();
void dispatch(Message *m);
+
+ void call_election() {
+ start();
+ }
+
};
#include "MDSMonitor.h"
#include "Monitor.h"
#include "MonitorStore.h"
+#include "OSDMonitor.h"
#include "messages/MMDSMap.h"
#include "messages/MMDSGetMap.h"
#include "messages/MMDSBeacon.h"
#include "messages/MMonCommand.h"
+#include "messages/MMonCommandAck.h"
+
+#include "messages/MGenericMessage.h"
+
#include "common/Timer.h"
-/********* MDS map **************/
+// my methods
-void MDSMonitor::dispatch(Message *m)
+void MDSMonitor::print_map(MDSMap &m)
{
- switch (m->get_type()) {
-
- case MSG_MDS_BEACON:
- handle_mds_beacon((MMDSBeacon*)m);
- break;
-
- case MSG_MDS_GETMAP:
- handle_mds_getmap((MMDSGetMap*)m);
- break;
-
- default:
- assert(0);
- }
-}
-
-
-
-void MDSMonitor::election_finished()
-{
- if (mon->is_leader()) {
-
- // FIXME be smarter later.
-
- if (g_conf.mkfs) {
- create_initial();
- save_map();
- } else {
- load_map();
- }
+ dout(7) << "print_map epoch " << m.get_epoch() << " num_mds " << g_conf.num_mds << endl;
+ entity_inst_t blank;
+ set<int> all;
+ m.get_mds_set(all);
+ for (set<int>::iterator p = all.begin();
+ p != all.end();
+ ++p) {
+ dout(7) << " mds" << *p << "." << m.mds_inc[*p]
+ << " : " << MDSMap::get_state_name(m.get_state(*p))
+ << " : " << (m.have_inst(*p) ? m.get_inst(*p) : blank)
+ << endl;
}
}
-void MDSMonitor::create_initial()
-{
- mdsmap.epoch = 0; // until everyone boots
- mdsmap.created = g_clock.now();
-
- mdsmap.encode(encoded_map);
- print_map();
-}
+// service methods
-void MDSMonitor::load_map()
+void MDSMonitor::create_initial()
{
- int r = mon->store->get_bl_ss(encoded_map, "mdsmap", "current");
- assert(r > 0);
- mdsmap.decode(encoded_map);
- dout(7) << "load_map epoch " << mdsmap.get_epoch() << endl;
+ dout(10) << "create_initial" << endl;
+ pending_mdsmap.created = g_clock.now();
+ print_map(pending_mdsmap);
}
-void MDSMonitor::save_map()
+bool MDSMonitor::update_from_paxos()
{
- dout(7) << "save_map epoch " << mdsmap.get_epoch() << endl;
-
- int r = mon->store->put_bl_ss(encoded_map, "mdsmap", "current");
- assert(r>=0);
+ assert(paxos->is_active());
+
+ version_t paxosv = paxos->get_version();
+ dout(10) << "update_from_paxos paxosv " << paxosv
+ << ", my e " << mdsmap.epoch << endl;
+
+ if (paxosv == mdsmap.epoch) return true;
+ assert(paxosv >= mdsmap.epoch);
+
+ // read and decode
+ mdsmap_bl.clear();
+ bool success = paxos->read(paxosv, mdsmap_bl);
+ assert(success);
+ dout(10) << "update_from_paxos got " << paxosv << endl;
+ mdsmap.decode(mdsmap_bl);
+
+ // new map
+ print_map(mdsmap);
+
+ // bcast map to mds, waiters
+ if (mon->is_leader())
+ bcast_latest_mds();
+ send_to_waiting();
+
+ // hackish: did all mds's shut down?
+ if (mon->is_leader() &&
+ g_conf.mon_stop_with_last_mds &&
+ mdsmap.get_epoch() > 1 &&
+ mdsmap.get_num_up_or_failed_mds() == 0)
+ mon->messenger->send_message(new MGenericMessage(MSG_SHUTDOWN),
+ mon->monmap->get_inst(mon->whoami));
+
+ return true;
}
-void MDSMonitor::print_map()
+void MDSMonitor::create_pending()
{
- dout(7) << "print_map epoch " << mdsmap.get_epoch() << " num_mds " << g_conf.num_mds << endl;
- entity_inst_t blank;
- set<int> all;
- mdsmap.get_mds_set(all);
- for (set<int>::iterator p = all.begin();
- p != all.end();
- ++p) {
- dout(7) << " mds" << *p << "." << mdsmap.mds_inc[*p]
- << " : " << MDSMap::get_state_name(mdsmap.get_state(*p))
- << " : " << (mdsmap.have_inst(*p) ? mdsmap.get_inst(*p) : blank)
- << endl;
- }
+ pending_mdsmap = mdsmap;
+ pending_mdsmap.epoch++;
+ dout(10) << "create_pending e" << pending_mdsmap.epoch << endl;
}
-void MDSMonitor::issue_map()
+void MDSMonitor::encode_pending(bufferlist &bl)
{
- mdsmap.inc_epoch();
- encoded_map.clear();
- mdsmap.encode(encoded_map);
-
- dout(7) << "issue_map epoch " << mdsmap.get_epoch() << endl;
-
- save_map();
- print_map();
+ dout(10) << "encode_pending e" << pending_mdsmap.epoch << endl;
- // bcast map
- bcast_latest_mds();
- send_current();
+ print_map(pending_mdsmap);
+
+ // apply to paxos
+ assert(paxos->get_version() + 1 == pending_mdsmap.epoch);
+ pending_mdsmap.encode(bl);
}
-void MDSMonitor::handle_command(MMonCommand *m, int& r, string& rs)
+bool MDSMonitor::preprocess_query(Message *m)
{
- stringstream ss;
- if (m->cmd.size() > 1) {
- if (m->cmd[1] == "stop" && m->cmd.size() > 2) {
- int who = atoi(m->cmd[2].c_str());
- if (mdsmap.is_active(who)) {
- r = 0;
- ss << "telling mds" << who << " to stop";
- getline(ss,rs);
+ dout(10) << "preprocess_query " << *m << " from " << m->get_source_inst() << endl;
- // hack
- mdsmap.mds_state[who] = MDSMap::STATE_STOPPING;
- issue_map();
+ switch (m->get_type()) {
+
+ case MSG_MDS_BEACON:
+ return preprocess_beacon((MMDSBeacon*)m);
+
+ case MSG_MDS_GETMAP:
+ send_full(m->get_source_inst());
+ return true;
- } else {
- ss << "mds" << who << " not active (" << mdsmap.get_state_name(mdsmap.get_state(who)) << ")";
- getline(ss,rs);
- }
- }
- else if (m->cmd[1] == "setnum" && m->cmd.size() > 2) {
- g_conf.num_mds = atoi(m->cmd[2].c_str());
- ss << "g_conf.num_mds = " << g_conf.num_mds << endl;
- getline(ss,rs);
- print_map();
- }
+ case MSG_MON_COMMAND:
+ return false;
+
+ default:
+ assert(0);
+ delete m;
+ return true;
}
}
-
-void MDSMonitor::handle_mds_beacon(MMDSBeacon *m)
+bool MDSMonitor::preprocess_beacon(MMDSBeacon *m)
{
- dout(12) << "mds_beacon " << *m
- << " from " << m->get_source()
- << " " << m->get_source_inst()
- << endl;
- int from = m->get_source().num();
+ dout(12) << "preprocess_beacon " << *m
+ << " from " << m->get_mds_inst()
+ << endl;
+
+ // fw to leader?
+ if (!mon->is_leader()) {
+ dout(10) << "fw to leader" << endl;
+ mon->messenger->send_message(m, mon->monmap->get_inst(mon->get_leader()));
+ return true;
+ }
+
+ // let's see.
+ int from = m->get_mds_inst().name.num();
int state = m->get_state();
version_t seq = m->get_seq();
- // initial boot?
- bool booted = false;
+ // can i handle this query without a map update?
- // choose an MDS id
- if (from >= 0) {
- // wants to be (or already is) a specific MDS.
- if (mdsmap.is_down(from)) {
- dout(10) << "mds_beacon assigning requested mds" << from << endl;
- booted = true;
- } else if (mdsmap.get_inst(from) != m->get_source_inst()) {
- dout(10) << "mds_beacon not assigning requested mds" << from
- << ", that mds is up and someone else" << endl;
- from = -1;
- }
- }
- if (from < 0) {
- // pick a failed mds?
- set<int> failed;
- mdsmap.get_failed_mds_set(failed);
- if (!failed.empty()) {
- from = *failed.begin();
- dout(10) << "mds_beacon assigned failed mds" << from << endl;
- booted = true;
- }
- }
- if (from < 0) {
- // ok, just pick any unused mds id.
- for (from=0; ; ++from) {
- if (mdsmap.is_dne(from) ||
- mdsmap.is_out(from)) {
- dout(10) << "mds_beacon assigned out|dne mds" << from << endl;
- booted = true;
- break;
- }
- }
+ // boot?
+ if (state == MDSMap::STATE_BOOT) {
+ // already booted?
+ int already = mdsmap.get_addr_rank(m->get_mds_inst().addr);
+ if (already < 0)
+ return false; // need to update map
+
+ // already booted. just reply to beacon, as per usual.
+ from = already;
}
-
- // old beacon?
+ // reply to beacon
if (mdsmap.mds_state_seq[from] > seq) {
dout(7) << "mds_beacon " << *m << " has old seq, ignoring" << endl;
delete m;
- return;
+ return true;
}
// reply to beacon?
if (state != MDSMap::STATE_OUT) {
last_beacon[from] = g_clock.now(); // note time
- messenger->send_message(new MMDSBeacon(state, seq),
- m->get_source_inst());
+ mon->messenger->send_message(new MMDSBeacon(m->get_mds_inst(), state, seq),
+ m->get_mds_inst());
}
+
+ // is there a state change here?
+ if (mdsmap.mds_state.count(from) == 0 ||
+ mdsmap.mds_state[from] != state)
+ return false; // yep, need to update map.
+
+ // we're done.
+ delete m;
+ return true;
+}
- // make sure it's in the map
- if (booted) {
- mdsmap.mds_inst[from].addr = m->get_source_addr();
- mdsmap.mds_inst[from].name = MSG_ADDR_MDS(from);
- mdsmap.mds_inc[from]++;
+bool MDSMonitor::prepare_update(Message *m)
+{
+ dout(10) << "prepare_update " << *m << endl;
- // someone (new) joined the cluster
- mdsmap.same_inst_since = mdsmap.epoch+1;
+ switch (m->get_type()) {
+
+ case MSG_MDS_BEACON:
+ return handle_beacon((MMDSBeacon*)m);
+
+ case MSG_MON_COMMAND:
+ return handle_command((MMonCommand*)m);
- // starting -> creating|starting|replay
- if (mdsmap.is_degraded() &&
- !mdsmap.is_failed(from)) {
- dout(10) << "mds_beacon currently degraded, mds" << from << " will be standby" << endl;
- state = MDSMap::STATE_STANDBY;
- }
- /*
- else if (from >= g_conf.num_mds) {
- dout(10) << "mds_beacon already have " << g_conf.num_mds << " mds's, standby (increase with 'mds setnum xxx')" << endl;
- state = MDSMap::STATE_STANDBY;
- }
- */
- else if (state == MDSMap::STATE_STARTING) {
+ default:
+ assert(0);
+ delete m;
+ }
+
+ return true;
+}
+
+bool MDSMonitor::should_propose_now()
+{
+ return true;
+}
+
+
+bool MDSMonitor::handle_beacon(MMDSBeacon *m)
+{
+ // -- this is an update --
+ dout(12) << "handle_beacon " << *m
+ << " from " << m->get_mds_inst()
+ << endl;
+ int from = m->get_mds_inst().name.num();
+ int state = m->get_state();
+ version_t seq = m->get_seq();
+
+ // boot?
+ if (state == MDSMap::STATE_BOOT) {
+ // assign a name.
+ if (from >= 0) {
+ // wants to be (or already is) a specific MDS.
if (mdsmap.is_failed(from)) {
- dout(10) << "mds_beacon will recover mds" << from << endl;
+ dout(10) << "mds_beacon boot: mds" << from << " was failed, replaying" << endl;
state = MDSMap::STATE_REPLAY;
- }
- else if (mdsmap.is_out(from)) {
- dout(10) << "mds_beacon will start mds" << from << endl;
+ } else if (mdsmap.is_out(from)) {
+ dout(10) << "mds_beacon boot: mds" << from << " was out, starting" << endl;
state = MDSMap::STATE_STARTING;
- }
- else {
- dout(10) << "mds_beacon will create mds" << from << endl;
- state = MDSMap::STATE_CREATING;
- }
+ } else if (!mdsmap.have_inst(from) || mdsmap.get_inst(from) != m->get_mds_inst()) {
+ dout(10) << "mds_beacon boot: mds" << from << " is someone else" << endl;
+ from = -1;
+ }
+ }
+ if (from < 0) {
+ from = pending_mdsmap.get_addr_rank(m->get_mds_inst().addr);
+ if (from >= 0) {
+ state = pending_mdsmap.mds_state[from];
+ dout(10) << "mds_beacon boot: already pending mds" << from
+ << " " << MDSMap::get_state_name(state) << endl;
+ delete m;
+ return false;
+ }
+ }
+ if (from < 0) {
+ // pick a failed mds?
+ set<int> failed;
+ pending_mdsmap.get_failed_mds_set(failed);
+ if (!failed.empty()) {
+ from = *failed.begin();
+ dout(10) << "mds_beacon boot: assigned failed mds" << from << endl;
+ state = MDSMap::STATE_REPLAY;
+ }
+ }
+ if (from < 0) {
+ // ok, just pick any unused mds id.
+ for (from=0; ; ++from) {
+ if (pending_mdsmap.is_dne(from)) {
+ dout(10) << "mds_beacon boot: assigned new mds" << from << endl;
+ state = MDSMap::STATE_CREATING;
+ break;
+ } else if (pending_mdsmap.is_out(from)) {
+ dout(10) << "mds_beacon boot: assigned out mds" << from << endl;
+ state = MDSMap::STATE_STARTING;
+ break;
+ }
+ }
+ }
+
+ assert(state != MDSMap::STATE_BOOT);
+
+ // put it in the map.
+ pending_mdsmap.mds_inst[from].addr = m->get_mds_inst().addr;
+ pending_mdsmap.mds_inst[from].name = MSG_ADDR_MDS(from);
+ pending_mdsmap.mds_inc[from]++;
+
+ // someone (new) has joined the cluster.
+ pending_mdsmap.same_inst_since = pending_mdsmap.epoch;
+
+ // if degraded, starting -> standby
+ if (pending_mdsmap.is_degraded() &&
+ state == MDSMap::STATE_STARTING) {
+ dout(10) << "mds_beacon boot: cluster degraded, mds" << from << " will be standby" << endl;
+ state = MDSMap::STATE_STANDBY;
}
}
// if creating -> active, go to standby instead
- if (state == MDSMap::STATE_ACTIVE && mdsmap.is_creating(from)) {
- mdsmap.mds_created.insert(from);
+ if (state == MDSMap::STATE_ACTIVE &&
+ mdsmap.is_creating(from)) {
+ pending_mdsmap.mds_created.insert(from);
dout(10) << "mds_beacon created mds" << from << endl;
if (mdsmap.is_degraded()) {
- dout(10) << "mds_beacon current degraded, marking mds" << from << " as standby" << endl;
+ dout(10) << "mds_beacon cluster degraded, marking mds" << from << " as standby" << endl;
state = MDSMap::STATE_STANDBY;
}
}
+ // update the map
+ dout(10) << "mds_beacon mds" << from << " " << MDSMap::get_state_name(mdsmap.mds_state[from])
+ << " -> " << MDSMap::get_state_name(state)
+ << endl;
- // did we update the map?
- if (mdsmap.mds_state.count(from) == 0 ||
- mdsmap.mds_state[from] != state) {
- // update mds state
- dout(10) << "mds_beacon mds" << from << " " << MDSMap::get_state_name(mdsmap.mds_state[from])
- << " -> " << MDSMap::get_state_name(state)
- << endl;
- // did someone leave the cluster?
- if (state == MDSMap::STATE_OUT && mdsmap.mds_state[from] != MDSMap::STATE_OUT)
- mdsmap.same_inst_since = mdsmap.epoch+1;
-
- // change the state
- mdsmap.mds_state[from] = state;
- if (mdsmap.is_up(from))
- mdsmap.mds_state_seq[from] = seq;
- else
- mdsmap.mds_state_seq.erase(from);
-
- issue_map();
- }
+ // did someone leave the cluster?
+ if (state == MDSMap::STATE_OUT &&
+ mdsmap.mds_state[from] != MDSMap::STATE_OUT)
+ pending_mdsmap.same_inst_since = pending_mdsmap.epoch;
+
+ // change the state
+ pending_mdsmap.mds_state[from] = state;
+ if (pending_mdsmap.is_up(from))
+ pending_mdsmap.mds_state_seq[from] = seq;
+ else
+ pending_mdsmap.mds_state_seq.erase(from);
+
+
+ dout(7) << "pending map now:" << endl;
+ print_map(pending_mdsmap);
+ paxos->wait_for_commit(new C_Updated(this, from, m));
+
+ return true;
+}
+
+
+void MDSMonitor::_updated(int from, MMDSBeacon *m)
+{
+ if (m->get_state() == MDSMap::STATE_BOOT) {
+ dout(10) << "_updated (booted) mds" << from << " " << *m << endl;
+ mon->osdmon->send_latest(0, mdsmap.get_inst(from));
+ } else {
+ dout(10) << "_updated mds" << from << " " << *m << endl;
+ }
delete m;
}
-void MDSMonitor::handle_mds_getmap(MMDSGetMap *m)
+
+bool MDSMonitor::handle_command(MMonCommand *m)
{
- dout(7) << "mds_getmap from " << m->get_source() << " " << m->get_source_inst() << endl;
- if (mdsmap.get_epoch() > 0)
- send_full(m->get_source_inst());
- else
- awaiting_map.push_back( m->get_source_inst() );
+ int r = -1;
+ string rs = "unrecognized command";
+ stringstream ss;
+
+ if (m->cmd.size() > 1) {
+ if (m->cmd[1] == "stop" && m->cmd.size() > 2) {
+ int who = atoi(m->cmd[2].c_str());
+ if (mdsmap.is_active(who)) {
+ r = 0;
+ ss << "telling mds" << who << " to stop";
+ getline(ss,rs);
+
+ pending_mdsmap.mds_state[who] = MDSMap::STATE_STOPPING;
+
+ } else {
+ ss << "mds" << who << " not active (" << mdsmap.get_state_name(mdsmap.get_state(who)) << ")";
+ getline(ss,rs);
+ }
+ }
+ /*
+ else if (m->cmd[1] == "setnum" && m->cmd.size() > 2) {
+ g_conf.num_mds = atoi(m->cmd[2].c_str());
+ ss << "g_conf.num_mds = " << g_conf.num_mds << endl;
+ getline(ss,rs);
+ print_map();
+ }
+ */
+ }
+
+ // reply
+ mon->messenger->send_message(new MMonCommandAck(r, rs), m->get_source_inst());
+ delete m;
+ return r >= 0;
}
+
void MDSMonitor::bcast_latest_mds()
{
dout(10) << "bcast_latest_mds " << mdsmap.get_epoch() << endl;
void MDSMonitor::send_full(entity_inst_t dest)
{
dout(11) << "send_full to " << dest << endl;
- messenger->send_message(new MMDSMap(&mdsmap), dest);
+ mon->messenger->send_message(new MMDSMap(&mdsmap), dest);
}
-void MDSMonitor::send_current()
+void MDSMonitor::send_to_waiting()
{
- dout(10) << "mds_send_current " << mdsmap.get_epoch() << endl;
- for (list<entity_inst_t>::iterator i = awaiting_map.begin();
- i != awaiting_map.end();
+ dout(10) << "send_to_waiting " << mdsmap.get_epoch() << endl;
+ for (list<entity_inst_t>::iterator i = waiting_for_map.begin();
+ i != waiting_for_map.end();
i++)
send_full(*i);
- awaiting_map.clear();
+ waiting_for_map.clear();
}
void MDSMonitor::send_latest(entity_inst_t dest)
{
- // FIXME: check if we're locked, etc.
- if (mdsmap.get_epoch() > 0)
+ if (paxos->is_readable())
send_full(dest);
else
- awaiting_map.push_back(dest);
+ waiting_for_map.push_back(dest);
}
{
// make sure mds's are still alive
utime_t now = g_clock.now();
+
+ // ...if i am an active leader
+ if (!mon->is_leader()) return;
+ if (!paxos->is_active()) return;
+
if (now > g_conf.mds_beacon_grace) {
utime_t cutoff = now;
cutoff -= g_conf.mds_beacon_grace;
<< endl;
// update map
- mdsmap.mds_state[*p] = newstate;
- mdsmap.mds_state_seq.erase(*p);
+ pending_mdsmap.mds_state[*p] = newstate;
+ pending_mdsmap.mds_state_seq.erase(*p);
changed = true;
}
} else {
}
}
- if (changed) {
- issue_map();
- }
+ if (changed)
+ propose_pending();
}
}
void MDSMonitor::do_stop()
{
+ // hrm...
+ if (!mon->is_leader() ||
+ !paxos->is_active()) {
+ dout(-10) << "do_stop can't stop right now, mdsmap not writeable" << endl;
+ return;
+ }
+
+ dout(10) << "do_stop stopping active mds nodes" << endl;
+
+ print_map(mdsmap);
for (map<int,int>::iterator p = mdsmap.mds_state.begin();
p != mdsmap.mds_state.end();
++p)
if (mdsmap.is_active(p->first))
- mdsmap.mds_state[p->first] = MDSMap::STATE_STOPPING;
+ pending_mdsmap.mds_state[p->first] = MDSMap::STATE_STOPPING;
- issue_map();
+ propose_pending();
}
#include "mds/MDSMap.h"
-class Monitor;
+#include "PaxosService.h"
-class MDSMonitor : public Dispatcher {
- Monitor *mon;
- Messenger *messenger;
- Mutex &lock;
+class MMDSBeacon;
- // mds maps
+class MDSMonitor : public PaxosService {
public:
- MDSMap mdsmap;
-
- private:
- bufferlist encoded_map;
-
- //map<epoch_t, bufferlist> inc_maps;
- //MDSMap::Incremental pending_inc;
+ // mds maps
+ MDSMap mdsmap; // current
+ bufferlist mdsmap_bl; // encoded
+
+ MDSMap pending_mdsmap; // current + pending updates
+
+ // my helpers
+ void print_map(MDSMap &m);
+
+ class C_Updated : public Context {
+ MDSMonitor *mm;
+ int mds;
+ MMDSBeacon *m;
+ public:
+ C_Updated(MDSMonitor *a, int b, MMDSBeacon *c) :
+ mm(a), mds(b), m(c) {}
+ void finish(int r) {
+ if (r >= 0)
+ mm->_updated(mds, m); // success
+ else
+ mm->dispatch((Message*)m); // try again
+ }
+ };
+
+
+ // service methods
+ void create_initial();
+ bool update_from_paxos();
+ void create_pending();
+ void encode_pending(bufferlist &bl);
- list<entity_inst_t> awaiting_map;
+ void _updated(int m, MMDSBeacon *m);
+
+ bool preprocess_query(Message *m); // true if processed.
+ bool prepare_update(Message *m);
+ bool should_propose_now();
+
+ bool preprocess_beacon(class MMDSBeacon *m);
+ bool handle_beacon(class MMDSBeacon *m);
+ bool handle_command(class MMonCommand *m);
// beacons
map<int, utime_t> last_beacon;
- bool is_alive(int mds);
+public:
+ MDSMonitor(Monitor *mn, Paxos *p) : PaxosService(mn, p) { }
+ // sending the map
+private:
+ list<entity_inst_t> waiting_for_map;
- // maps
- void create_initial();
- void send_current(); // send current map to waiters.
- void send_full(entity_inst_t dest);
void bcast_latest_mds();
+ void send_full(entity_inst_t dest);
+ void send_to_waiting();
- void issue_map();
-
- void save_map();
- void load_map();
- void print_map();
-
- //void accept_pending(); // accept pending, new map.
- //void send_incremental(epoch_t since, msg_addr_t dest);
-
- void handle_mds_state(class MMDSState *m);
- void handle_mds_beacon(class MMDSBeacon *m);
- //void handle_mds_failure(class MMDSFailure *m);
- void handle_mds_getmap(class MMDSGetMap *m);
-
-
-
- public:
- MDSMonitor(Monitor *mn, Messenger *m, Mutex& l) : mon(mn), messenger(m), lock(l) {
- }
-
- void dispatch(Message *m);
- void tick(); // check state, take actions
-
- void election_starting();
- void election_finished();
-
+public:
void send_latest(entity_inst_t dest);
- void handle_command(class MMonCommand *m, int& r, string& rs);
-
+ void tick(); // check state, take actions
void do_stop();
};
class MonMap {
public:
- epoch_t epoch; // what epoch of the osd cluster descriptor is this
+ epoch_t epoch; // what epoch/version of the monmap
int num_mon;
vector<entity_inst_t> mon_inst;
// choice should be stable, unless we explicitly ask for a new one.
int pick_mon(bool newmon=false) {
if (newmon || (last_mon < 0)) {
- last_mon = 0; //last_mon = rand() % num_mon;
+ last_mon = rand() % num_mon;
}
return last_mon;
}
_decode(mon_inst, blist, off);
}
+ // read from/write to a file
int write(char *fn) {
// encode
bufferlist bl;
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-// TODO: missing run() method, which creates the two main timers, refreshTimer and readTimer
-
-#include "Monitor.h"
-
-#include "osd/OSDMap.h"
-
-#include "MonitorStore.h"
-
-#include "msg/Message.h"
-#include "msg/Messenger.h"
-
-#include "messages/MPing.h"
-#include "messages/MPingAck.h"
-#include "messages/MGenericMessage.h"
-#include "messages/MMonCommand.h"
-#include "messages/MMonCommandAck.h"
-
-#include "messages/MMonPaxos.h"
-
-#include "common/Timer.h"
-#include "common/Clock.h"
-
-#include "OSDMonitor.h"
-#include "MDSMonitor.h"
-#include "ClientMonitor.h"
-
-#include "config.h"
-#undef dout
-#define dout(l) if (l<=g_conf.debug || l<=g_conf.debug_mon) cout << g_clock.now() << " mon" << whoami << (is_starting() ? (const char*)"(starting)":(is_leader() ? (const char*)"(leader)":(is_peon() ? (const char*)"(peon)":(const char*)"(?\?)"))) << " "
-#define derr(l) if (l<=g_conf.debug || l<=g_conf.debug_mon) cerr << g_clock.now() << " mon" << whoami << (is_starting() ? (const char*)"(starting)":(is_leader() ? (const char*)"(leader)":(is_peon() ? (const char*)"(peon)":(const char*)"(?\?)"))) << " "
-
-
-
-void Monitor::init()
-{
- lock.Lock();
-
- dout(1) << "init" << endl;
-
- // store
- char s[80];
- sprintf(s, "mondata/mon%d", whoami);
- store = new MonitorStore(s);
-
- if (g_conf.mkfs)
- store->mkfs();
-
- store->mount();
-
- // create
- osdmon = new OSDMonitor(this, messenger, lock);
- mdsmon = new MDSMonitor(this, messenger, lock);
- clientmon = new ClientMonitor(this, messenger, lock);
-
- // i'm ready!
- messenger->set_dispatcher(this);
-
- // start ticker
- reset_tick();
-
- // call election?
- if (monmap->num_mon > 1) {
- assert(monmap->num_mon != 2);
- call_election();
- } else {
- // we're standalone.
- set<int> q;
- q.insert(whoami);
- win_election(q);
- }
-
- lock.Unlock();
-}
-
-void Monitor::shutdown()
-{
- dout(1) << "shutdown" << endl;
-
- // cancel all events
- cancel_tick();
- timer.cancel_all();
- timer.join();
-
- // stop osds.
- for (set<int>::iterator it = osdmon->osdmap.get_osds().begin();
- it != osdmon->osdmap.get_osds().end();
- it++) {
- if (osdmon->osdmap.is_down(*it)) continue;
- dout(10) << "sending shutdown to osd" << *it << endl;
- messenger->send_message(new MGenericMessage(MSG_SHUTDOWN),
- osdmon->osdmap.get_inst(*it));
- }
- osdmon->mark_all_down();
-
- // monitors too.
- for (int i=0; i<monmap->num_mon; i++)
- if (i != whoami)
- messenger->send_message(new MGenericMessage(MSG_SHUTDOWN),
- monmap->get_inst(i));
-
- // unmount my local storage
- if (store)
- delete store;
-
- // clean up
- if (monmap) delete monmap;
- if (osdmon) delete osdmon;
- if (mdsmon) delete mdsmon;
- if (clientmon) delete clientmon;
-
- // die.
- messenger->shutdown();
- delete messenger;
-}
-
-
-void Monitor::call_election()
-{
- if (monmap->num_mon == 1) return;
-
- dout(10) << "call_election" << endl;
- state = STATE_STARTING;
-
- elector.start();
-
- osdmon->election_starting();
- //mdsmon->election_starting();
-}
-
-void Monitor::win_election(set<int>& active)
-{
- state = STATE_LEADER;
- leader = whoami;
- quorum = active;
- dout(10) << "win_election, quorum is " << quorum << endl;
-
- // init
- osdmon->election_finished();
- mdsmon->election_finished();
-
- // init paxos
- test_paxos.leader_start();
-}
-
-void Monitor::lose_election(int l)
-{
- state = STATE_PEON;
- leader = l;
- dout(10) << "lose_election, leader is mon" << leader << endl;
-}
-
-
-void Monitor::handle_command(MMonCommand *m)
-{
- dout(0) << "handle_command " << *m << endl;
-
- int r = -1;
- string rs = "unrecognized command";
-
- if (!m->cmd.empty()) {
- if (m->cmd[0] == "stop") {
- r = 0;
- rs = "stopping";
- do_stop();
- }
- else if (m->cmd[0] == "mds") {
- mdsmon->handle_command(m, r, rs);
- }
- else if (m->cmd[0] == "osd") {
-
- }
- }
-
- // reply
- messenger->send_message(new MMonCommandAck(r, rs), m->get_source_inst());
- delete m;
-}
-
-
-void Monitor::do_stop()
-{
- dout(0) << "do_stop -- shutting down" << endl;
- mdsmon->do_stop();
-}
-
-
-void Monitor::dispatch(Message *m)
-{
- lock.Lock();
- {
- switch (m->get_type()) {
-
- // misc
- case MSG_PING_ACK:
- handle_ping_ack((MPingAck*)m);
- break;
-
- case MSG_SHUTDOWN:
- assert(m->get_source().is_osd());
- osdmon->dispatch(m);
- break;
+ // -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+ // vim: ts=8 sw=2 smarttab
+ /*
+ * Ceph - scalable distributed file system
+ *
+ * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
+ *
+ * This is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License version 2.1, as published by the Free Software
+ * Foundation. See file COPYING.
+ *
+ */
+
+ // TODO: missing run() method, which creates the two main timers, refreshTimer and readTimer
+
+ #include "Monitor.h"
+
+ #include "osd/OSDMap.h"
+
+ #include "MonitorStore.h"
+
+ #include "msg/Message.h"
+ #include "msg/Messenger.h"
+
+ #include "messages/MPing.h"
+ #include "messages/MPingAck.h"
+ #include "messages/MGenericMessage.h"
+ #include "messages/MMonCommand.h"
+ #include "messages/MMonCommandAck.h"
+
+ #include "messages/MMonPaxos.h"
+
+ #include "common/Timer.h"
+ #include "common/Clock.h"
+
+ #include "OSDMonitor.h"
+ #include "MDSMonitor.h"
+ #include "ClientMonitor.h"
+
+ #include "config.h"
+ #undef dout
+ #define dout(l) if (l<=g_conf.debug || l<=g_conf.debug_mon) cout << g_clock.now() << " mon" << whoami << (is_starting() ? (const char*)"(starting)":(is_leader() ? (const char*)"(leader)":(is_peon() ? (const char*)"(peon)":(const char*)"(?\?)"))) << " "
+ #define derr(l) if (l<=g_conf.debug || l<=g_conf.debug_mon) cerr << g_clock.now() << " mon" << whoami << (is_starting() ? (const char*)"(starting)":(is_leader() ? (const char*)"(leader)":(is_peon() ? (const char*)"(peon)":(const char*)"(?\?)"))) << " "
+
+
+
+ void Monitor::init()
+ {
+ lock.Lock();
+
+ dout(1) << "init" << endl;
+
+ // store
+ char s[80];
+ sprintf(s, "mondata/mon%d", whoami);
+ store = new MonitorStore(s);
+
+ if (g_conf.mkfs)
+ store->mkfs();
+
+ store->mount();
+
+ // create
+ osdmon = new OSDMonitor(this, &paxos_osdmap);
+ mdsmon = new MDSMonitor(this, &paxos_mdsmap);
+ clientmon = new ClientMonitor(this, &paxos_clientmap);
+
+ // init paxos
+ paxos_test.init();
+ paxos_osdmap.init();
+ paxos_mdsmap.init();
+ paxos_clientmap.init();
+
+ // i'm ready!
+ messenger->set_dispatcher(this);
+
+ // start ticker
+ reset_tick();
+
+ // call election?
+ if (monmap->num_mon > 1) {
+ assert(monmap->num_mon != 2);
+ call_election();
+ } else {
+ // we're standalone.
+ set<int> q;
+ q.insert(whoami);
+ win_election(1, q);
+ }
+
+ lock.Unlock();
+ }
+
+ void Monitor::shutdown()
+ {
+ dout(1) << "shutdown" << endl;
+
+ elector.shutdown();
+
+ if (is_leader()) {
+ // stop osds.
+ for (set<int>::iterator it = osdmon->osdmap.get_osds().begin();
+ it != osdmon->osdmap.get_osds().end();
+ it++) {
+ if (osdmon->osdmap.is_down(*it)) continue;
+ dout(10) << "sending shutdown to osd" << *it << endl;
+ messenger->send_message(new MGenericMessage(MSG_SHUTDOWN),
+ osdmon->osdmap.get_inst(*it));
+ }
+ osdmon->mark_all_down();
+
+ // monitors too.
+ for (int i=0; i<monmap->num_mon; i++)
+ if (i != whoami)
+ messenger->send_message(new MGenericMessage(MSG_SHUTDOWN),
+ monmap->get_inst(i));
+ }
+
+ // cancel all events
+ cancel_tick();
+ timer.cancel_all();
+ timer.join();
+
+ // unmount my local storage
+ if (store)
+ delete store;
+
+ // clean up
+ if (osdmon) delete osdmon;
+ if (mdsmon) delete mdsmon;
+ if (clientmon) delete clientmon;
+
+ // die.
+ messenger->shutdown();
+ delete messenger;
+ }
+
+
+ void Monitor::call_election()
+ {
+ if (monmap->num_mon == 1) return;
+
+ dout(10) << "call_election" << endl;
+ state = STATE_STARTING;
+
+ // tell paxos
+ paxos_test.election_starting();
+ paxos_mdsmap.election_starting();
+ paxos_osdmap.election_starting();
+ paxos_clientmap.election_starting();
+
+ // call a new election
+ elector.call_election();
+ }
+
+ void Monitor::win_election(epoch_t epoch, set<int>& active)
+ {
+ state = STATE_LEADER;
+ leader = whoami;
+ mon_epoch = epoch;
+ quorum = active;
+ dout(10) << "win_election, epoch " << mon_epoch << " quorum is " << quorum << endl;
+
+ // init paxos
+ paxos_test.leader_init();
+ paxos_mdsmap.leader_init();
+ paxos_osdmap.leader_init();
+ paxos_clientmap.leader_init();
+
+ // init
+ osdmon->election_finished();
+ mdsmon->election_finished();
+ clientmon->election_finished();
+ }
+
+ void Monitor::lose_election(epoch_t epoch, int l)
+ {
+ state = STATE_PEON;
+ mon_epoch = epoch;
+ leader = l;
+ dout(10) << "lose_election, epoch " << mon_epoch << " leader is mon" << leader << endl;
+
+ // init paxos
+ paxos_test.peon_init();
+ paxos_mdsmap.peon_init();
+ paxos_osdmap.peon_init();
+ paxos_clientmap.peon_init();
+
+ // init
+ osdmon->election_finished();
+ mdsmon->election_finished();
+ clientmon->election_finished();
+ }
+
+
+ void Monitor::handle_command(MMonCommand *m)
+ {
+ dout(0) << "handle_command " << *m << endl;
+
+ int r = -1;
+ string rs = "unrecognized command";
+
+ if (!m->cmd.empty()) {
+ if (m->cmd[0] == "stop") {
+ r = 0;
+ rs = "stopping";
+ do_stop();
+ }
+ else if (m->cmd[0] == "mds") {
+ mdsmon->dispatch(m);
+ return;
+ }
+ else if (m->cmd[0] == "osd") {
+
+ }
+ }
+
+ // reply
+ messenger->send_message(new MMonCommandAck(r, rs), m->get_source_inst());
+ delete m;
+ }
+
+
+ void Monitor::do_stop()
+ {
+ dout(0) << "do_stop -- shutting down" << endl;
+ mdsmon->do_stop();
+ }
+
+
+ void Monitor::dispatch(Message *m)
+ {
+ lock.Lock();
+ {
+ switch (m->get_type()) {
+
+ // misc
+ case MSG_PING_ACK:
+ handle_ping_ack((MPingAck*)m);
+ break;
+
+ case MSG_SHUTDOWN:
+ if (m->get_source().is_osd())
+ osdmon->dispatch(m);
+ else
+ handle_shutdown(m);
+ break;
case MSG_MON_COMMAND:
handle_command((MMonCommand*)m);
case MSG_MDS_BEACON:
case MSG_MDS_GETMAP:
mdsmon->dispatch(m);
-
- // hackish: did all mds's shut down?
- if (g_conf.mon_stop_with_last_mds &&
- mdsmon->mdsmap.get_num_up_or_failed_mds() == 0)
- shutdown();
-
break;
// clients
// paxos
case MSG_MON_PAXOS:
- // send it to the right paxos instance
- switch (((MMonPaxos*)m)->machine_id) {
- case PAXOS_TEST:
- test_paxos.dispatch(m);
- break;
- case PAXOS_OSDMAP:
- //...
-
- default:
- assert(0);
+ {
+ MMonPaxos *pm = (MMonPaxos*)m;
+
+ // sanitize
+ if (pm->epoch > mon_epoch)
+ call_election();
+ if (pm->epoch != mon_epoch) {
+ delete pm;
+ break;
+ }
+
+ // send it to the right paxos instance
+ switch (pm->machine_id) {
+ case PAXOS_TEST:
+ paxos_test.dispatch(m);
+ break;
+ case PAXOS_OSDMAP:
+ paxos_osdmap.dispatch(m);
+ break;
+ case PAXOS_MDSMAP:
+ paxos_mdsmap.dispatch(m);
+ break;
+ case PAXOS_CLIENTMAP:
+ paxos_clientmap.dispatch(m);
+ break;
+ default:
+ assert(0);
+ }
}
break;
// elector messages
- case MSG_MON_ELECTION_PROPOSE:
- case MSG_MON_ELECTION_ACK:
- case MSG_MON_ELECTION_VICTORY:
+ case MSG_MON_ELECTION:
elector.dispatch(m);
break;
void Monitor::handle_shutdown(Message *m)
{
- dout(1) << "shutdown from " << m->get_source() << endl;
-
- shutdown();
+ assert(m->get_source().is_mon());
+ if (m->get_source().num() == get_leader()) {
+ dout(1) << "shutdown from leader " << m->get_source() << endl;
+ shutdown();
+ } else {
+ dout(1) << "ignoring shutdown from non-leader " << m->get_source() << endl;
+ }
delete m;
}
class MDSMonitor;
class ClientMonitor;
-#define PAXOS_TEST 0
-#define PAXOS_OSDMAP 1
-#define PAXOS_MDSMAP 2
-#define PAXOS_CLIENTMAP 3
class Monitor : public Dispatcher {
-protected:
+public:
// me
int whoami;
Messenger *messenger;
void reset_tick();
friend class C_Mon_Tick;
- // my local store
- //ObjectStore *store;
+ // -- local storage --
+public:
MonitorStore *store;
- const static int INO_ELECTOR = 1;
- const static int INO_MON_MAP = 2;
- const static int INO_OSD_MAP = 10;
- const static int INO_OSD_INC_MAP = 11;
- const static int INO_MDS_MAP = 20;
-
- // elector
- Elector elector;
- friend class Elector;
-
- epoch_t mon_epoch; // monitor epoch (election instance)
- set<int> quorum; // current active set of monitors (if !starting)
-
- //void call_election();
-
- // paxos
- Paxos test_paxos;
- friend class Paxos;
-
-
- // monitor state
+ // -- monitor state --
+private:
const static int STATE_STARTING = 0; // electing
const static int STATE_LEADER = 1;
const static int STATE_PEON = 2;
int state;
- int leader; // current leader (to best of knowledge)
- utime_t last_called_election; // [starting] last time i called an election
-
+public:
bool is_starting() { return state == STATE_STARTING; }
bool is_leader() { return state == STATE_LEADER; }
bool is_peon() { return state == STATE_PEON; }
- // my public services
+
+ // -- elector --
+private:
+ Elector elector;
+ friend class Elector;
+
+ epoch_t mon_epoch; // monitor epoch (election instance)
+ int leader; // current leader (to best of knowledge)
+ set<int> quorum; // current active set of monitors (if !starting)
+ utime_t last_called_election; // [starting] last time i called an election
+
+public:
+ epoch_t get_epoch() { return mon_epoch; }
+ int get_leader() { return leader; }
+ const set<int>& get_quorum() { return quorum; }
+
+ void call_election(); // initiate election
+ void win_election(epoch_t epoch, set<int>& q); // end election (called by Elector)
+ void lose_election(epoch_t epoch, int l); // end election (called by Elector)
+
+
+ // -- paxos --
+ Paxos paxos_test;
+ Paxos paxos_mdsmap;
+ Paxos paxos_osdmap;
+ Paxos paxos_clientmap;
+ friend class Paxos;
+
+
+ // -- services --
OSDMonitor *osdmon;
MDSMonitor *mdsmon;
ClientMonitor *clientmon;
- // messages
- void handle_shutdown(Message *m);
- void handle_ping_ack(class MPingAck *m);
- void handle_command(class MMonCommand *m);
-
friend class OSDMonitor;
friend class MDSMonitor;
friend class ClientMonitor;
- // initiate election
- void call_election();
- // end election (called by Elector)
- void win_election(set<int>& q);
- void lose_election(int l);
+ // messages
+ void handle_shutdown(Message *m);
+ void handle_ping_ack(class MPingAck *m);
+ void handle_command(class MMonCommand *m);
monmap(mm),
timer(lock), tick_timer(0),
store(0),
+
+ state(STATE_STARTING),
+
elector(this, w),
mon_epoch(0),
+ leader(0),
- test_paxos(this, w, PAXOS_TEST, "tester"), // tester state machine
+ paxos_test(this, w, PAXOS_TEST),
+ paxos_mdsmap(this, w, PAXOS_MDSMAP),
+ paxos_osdmap(this, w, PAXOS_OSDMAP),
+ paxos_clientmap(this, w, PAXOS_CLIENTMAP),
- state(STATE_STARTING),
- leader(0),
osdmon(0), mdsmon(0), clientmon(0)
{
}
-
void init();
void shutdown();
void dispatch(Message *m);
struct stat st;
int r = ::stat(fn, &st);
+ //dout(15) << "exists_bl stat " << fn << " r=" << r << " errno " << errno << " " << strerror(errno) << endl;
return r == 0;
}
#include "config.h"
#undef dout
-#define dout(l) if (l<=g_conf.debug || l<=g_conf.debug_mon) cout << g_clock.now() << " mon" << mon->whoami << (mon->is_starting() ? (const char*)"(starting)":(mon->is_leader() ? (const char*)"(leader)":(mon->is_peon() ? (const char*)"(peon)":(const char*)"(?\?)"))) << ".osd(" << (state == STATE_INIT ? (const char*)"init":(state == STATE_SYNC ? (const char*)"sync":(state == STATE_LOCK ? (const char*)"lock":(state == STATE_UPDATING ? (const char*)"updating":(const char*)"?\?")))) << ") e" << osdmap.get_epoch() << " "
-#define derr(l) if (l<=g_conf.debug || l<=g_conf.debug_mon) cerr << g_clock.now() << " mon" << mon->whoami << (mon->is_starting() ? (const char*)"(starting)":(mon->is_leader() ? (const char*)"(leader)":(mon->is_peon() ? (const char*)"(peon)":(const char*)"(?\?)"))) << ".osd(" << (state == STATE_INIT ? (const char*)"init":(state == STATE_SYNC ? (const char*)"sync":(state == STATE_LOCK ? (const char*)"lock":(state == STATE_UPDATING ? (const char*)"updating":(const char*)"?\?")))) << ") e" << osdmap.get_epoch() << " "
+#define dout(l) if (l<=g_conf.debug || l<=g_conf.debug_mon) cout << g_clock.now() << " mon" << mon->whoami << (mon->is_starting() ? (const char*)"(starting)":(mon->is_leader() ? (const char*)"(leader)":(mon->is_peon() ? (const char*)"(peon)":(const char*)"(?\?)"))) << ".osd(e" << osdmap.get_epoch() << ") "
+#define derr(l) if (l<=g_conf.debug || l<=g_conf.debug_mon) cerr << g_clock.now() << " mon" << mon->whoami << (mon->is_starting() ? (const char*)"(starting)":(mon->is_leader() ? (const char*)"(leader)":(mon->is_peon() ? (const char*)"(peon)":(const char*)"(?\?)"))) << ".osd(e" << osdmap.get_epoch() << ") "
+// FAKING
+
class C_Mon_FakeOSDFailure : public Context {
OSDMonitor *mon;
int osd;
}
};
+void OSDMonitor::fake_osd_failure(int osd, bool down)
+{
+ if (down) {
+ dout(1) << "fake_osd_failure DOWN osd" << osd << endl;
+ pending_inc.new_down[osd] = osdmap.osd_inst[osd];
+ } else {
+ dout(1) << "fake_osd_failure OUT osd" << osd << endl;
+ pending_inc.new_out.push_back(osd);
+ }
+ propose_pending();
+
+ // fixme
+ //bcast_latest_osd();
+ //bcast_latest_mds();
+}
void OSDMonitor::fake_osdmap_update()
{
dout(1) << "fake_osdmap_update" << endl;
- accept_pending();
+ propose_pending();
// tell a random osd
int osd = rand() % g_conf.num_osd;
- send_incremental(osdmap.get_epoch()-1, // ick! FIXME
- osdmap.get_inst(osd));
+ send_latest(0, osdmap.get_inst(osd));
}
pending_inc.new_out.push_back(r);
}
- accept_pending();
-
- // tell him!
- send_incremental(osdmap.get_epoch()-1, osdmap.get_inst(r));
-
- // do it again?
- /*
- if (g_conf.num_osd - d > 4 &&
- g_conf.num_osd - d > g_conf.num_osd/2)
- mon->timer.add_event_after(g_conf.fake_osdmap_expand,
- new C_Mon_Faker(this));
- */
+ propose_pending();
+ send_latest(0, osdmap.get_inst(r)); // after
}
-/*
-void OSDMonitor::init()
-{
- // start with blank map
-
- // load my last state from the store
- bufferlist bl;
- if (get_map_bl(0, bl)) { // FIXME
- // yay!
- osdmap.decode(bl);
- dout(1) << "init got epoch " << osdmap.get_epoch() << " from store" << endl;
-
- // set up pending_inc
- pending_inc.epoch = osdmap.get_epoch()+1;
- }
-}
-*/
-
-
-
-
/************ MAPS ****************/
-
void OSDMonitor::create_initial()
{
- dout(1) << "create_initial generating osdmap from g_conf" << endl;
+ assert(mon->is_leader());
+ assert(paxos->get_version() == 0);
+
+ dout(1) << "create_initial -- creating initial osdmap from g_conf" << endl;
// <HACK set up OSDMap from g_conf>
- osdmap.mon_epoch = mon->mon_epoch;
- osdmap.ctime = g_clock.now();
+ OSDMap newmap;
+ newmap.mon_epoch = mon->mon_epoch;
+ newmap.ctime = g_clock.now();
if (g_conf.osd_pg_bits) {
- osdmap.set_pg_num(1 << g_conf.osd_pg_bits);
+ newmap.set_pg_num(1 << g_conf.osd_pg_bits);
} else {
// 4 bits of pgs per osd.
- osdmap.set_pg_num(g_conf.num_osd << 4);
+ newmap.set_pg_num(g_conf.num_osd << 4);
}
- // start at epoch 0 until all osds boot
- //osdmap.inc_epoch(); // = 1
- //assert(osdmap.get_epoch() == 1);
+ // start at epoch 1 until all osds boot
+ newmap.inc_epoch(); // = 1
+ assert(newmap.get_epoch() == 1);
if (g_conf.num_osd >= 12) {
int ndom = g_conf.osd_max_rep;
int domid[ndom];
for (int i=0; i<ndom; i++) {
domain[i] = new UniformBucket(1, 0);
- domid[i] = osdmap.crush.add_bucket(domain[i]);
+ domid[i] = newmap.crush.add_bucket(domain[i]);
}
// add osds
int i = 0;
for (int dom=0; dom<ndom; dom++) {
for (int j=0; j<nper; j++) {
- osdmap.osds.insert(i);
+ newmap.osds.insert(i);
+ newmap.down_osds.insert(i); // initially DOWN
domain[dom]->add_item(i, 1.0);
//cerr << "osd" << i << " in domain " << dom << endl;
i++;
//cerr << "dom " << i << " w " << domain[i]->get_weight() << endl;
root->add_item(domid[i], domain[i]->get_weight());
}
- int nroot = osdmap.crush.add_bucket(root);
+ int nroot = newmap.crush.add_bucket(root);
// rules
// replication
for (int i=1; i<=ndom; i++) {
int r = CRUSH_REP_RULE(i);
- osdmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_TAKE, nroot));
- osdmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_CHOOSE, i, 1));
- osdmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_CHOOSE, 1, 0));
- osdmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_EMIT));
+ newmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_TAKE, nroot));
+ newmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_CHOOSE, i, 1));
+ newmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_CHOOSE, 1, 0));
+ newmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_EMIT));
}
// raid
for (int i=g_conf.osd_min_raid_width; i <= g_conf.osd_max_raid_width; i++) {
int r = CRUSH_RAID_RULE(i);
if (ndom >= i) {
- osdmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_TAKE, nroot));
- osdmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_CHOOSE_INDEP, i, 1));
- osdmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_CHOOSE_INDEP, 1, 0));
- osdmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_EMIT));
+ newmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_TAKE, nroot));
+ newmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_CHOOSE_INDEP, i, 1));
+ newmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_CHOOSE_INDEP, 1, 0));
+ newmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_EMIT));
} else {
- osdmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_TAKE, nroot));
- osdmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_CHOOSE_INDEP, i, 0));
- osdmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_EMIT));
+ newmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_TAKE, nroot));
+ newmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_CHOOSE_INDEP, i, 0));
+ newmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_EMIT));
}
}
// test
//vector<int> out;
- //osdmap.pg_to_osds(0x40200000110ULL, out);
+ //newmap.pg_to_osds(0x40200000110ULL, out);
} else {
// one bucket
Bucket *b = new UniformBucket(1, 0);
- int root = osdmap.crush.add_bucket(b);
+ int root = newmap.crush.add_bucket(b);
for (int i=0; i<g_conf.num_osd; i++) {
- osdmap.osds.insert(i);
+ newmap.osds.insert(i);
+ newmap.down_osds.insert(i);
b->add_item(i, 1.0);
}
// replication
for (int i=1; i<=g_conf.osd_max_rep; i++) {
int r = CRUSH_REP_RULE(i);
- osdmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_TAKE, root));
- osdmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_CHOOSE, i, 0));
- osdmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_EMIT));
+ newmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_TAKE, root));
+ newmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_CHOOSE, i, 0));
+ newmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_EMIT));
}
// raid
for (int i=g_conf.osd_min_raid_width; i <= g_conf.osd_max_raid_width; i++) {
int r = CRUSH_RAID_RULE(i);
- osdmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_TAKE, root));
- osdmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_CHOOSE_INDEP, i, 0));
- osdmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_EMIT));
+ newmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_TAKE, root));
+ newmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_CHOOSE_INDEP, i, 0));
+ newmap.crush.rules[r].steps.push_back(RuleStep(CRUSH_RULE_EMIT));
}
}
if (g_conf.mds_local_osd) {
// add mds osds, but don't put them in the crush mapping func
- for (int i=0; i<g_conf.num_mds; i++)
- osdmap.osds.insert(i+10000);
+ for (int i=0; i<g_conf.num_mds; i++) {
+ newmap.osds.insert(i+10000);
+ newmap.down_osds.insert(i+10000);
+ }
}
// </HACK>
dout(0) << "will fake osd" << i->first << " OUT after " << i->second << endl;
mon->timer.add_event_after(i->second, new C_Mon_FakeOSDFailure(this, i->first, 0));
}
-}
-
-bool OSDMonitor::get_map_bl(epoch_t epoch, bufferlist& bl)
-{
- if (!mon->store->exists_bl_sn("osdmap", epoch))
- return false;
- int r = mon->store->get_bl_sn(bl, "osdmap", epoch);
- assert(r > 0);
- return true;
+ // encode into pending incremental
+ newmap.encode(pending_inc.fullmap);
}
-bool OSDMonitor::get_inc_map_bl(epoch_t epoch, bufferlist& bl)
+bool OSDMonitor::update_from_paxos()
{
- if (!mon->store->exists_bl_sn("osdincmap", epoch))
- return false;
- int r = mon->store->get_bl_sn(bl, "osdincmap", epoch);
- assert(r > 0);
- return true;
-}
+ assert(paxos->is_active());
+ version_t paxosv = paxos->get_version();
+ dout(15) << "update_from_paxos paxos e " << paxosv
+ << ", my e " << osdmap.epoch << endl;
-void OSDMonitor::save_map()
-{
- bufferlist bl;
- osdmap.encode(bl);
+ if (paxosv == osdmap.epoch) return true;
+ assert(paxosv >= osdmap.epoch);
- mon->store->put_bl_sn(bl, "osdmap", osdmap.get_epoch());
- mon->store->put_int(osdmap.get_epoch(), "osd_epoch");
+ if (osdmap.epoch == 0 && paxosv > 1) {
+ // startup: just load latest full map
+ epoch_t lastfull = mon->store->get_int("osdmap_full","last_epoch");
+ if (lastfull) {
+ dout(7) << "update_from_paxos startup: loading latest full map e" << lastfull << endl;
+ bufferlist bl;
+ mon->store->get_bl_sn(bl, "osdmap_full", lastfull);
+ osdmap.decode(bl);
+ }
+ }
+
+ // walk through incrementals
+ while (paxosv > osdmap.epoch) {
+ bufferlist bl;
+ bool success = paxos->read(osdmap.epoch+1, bl);
+ assert(success);
+
+ dout(7) << "update_from_paxos applying incremental " << osdmap.epoch+1 << endl;
+ OSDMap::Incremental inc;
+ int off = 0;
+ inc.decode(bl, off);
+ osdmap.apply_incremental(inc);
+
+ // write out the full map, too.
+ bl.clear();
+ osdmap.encode(bl);
+ mon->store->put_bl_sn(bl, "osdmap_full", osdmap.epoch);
+ }
+ mon->store->put_int(osdmap.epoch, "osdmap_full","last_epoch");
+
+ // new map!
+ bcast_latest_mds();
+
+ return true;
}
-void OSDMonitor::save_inc_map(OSDMap::Incremental &inc)
+
+void OSDMonitor::create_pending()
{
- bufferlist bl;
- osdmap.encode(bl);
+ pending_inc = OSDMap::Incremental(osdmap.epoch+1);
+ dout(10) << "create_pending e " << pending_inc.epoch
+ << endl;
+}
- bufferlist incbl;
- inc.encode(incbl);
+void OSDMonitor::encode_pending(bufferlist &bl)
+{
+ dout(10) << "encode_pending e " << pending_inc.epoch
+ << endl;
+
+ // finish up pending_inc
+ pending_inc.ctime = g_clock.now();
+ pending_inc.mon_epoch = mon->mon_epoch;
+
+ // tell me about it
+ for (map<int,entity_inst_t>::iterator i = pending_inc.new_up.begin();
+ i != pending_inc.new_up.end();
+ i++) {
+ dout(0) << " osd" << i->first << " UP " << i->second << endl;
+ derr(0) << " osd" << i->first << " UP " << i->second << endl;
+ }
+ for (map<int,entity_inst_t>::iterator i = pending_inc.new_down.begin();
+ i != pending_inc.new_down.end();
+ i++) {
+ dout(0) << " osd" << i->first << " DOWN " << i->second << endl;
+ derr(0) << " osd" << i->first << " DOWN " << i->second << endl;
+ mon->messenger->mark_down(i->second.addr);
+ }
+ for (list<int>::iterator i = pending_inc.new_in.begin();
+ i != pending_inc.new_in.end();
+ i++) {
+ dout(0) << " osd" << *i << " IN" << endl;
+ derr(0) << " osd" << *i << " IN" << endl;
+ }
+ for (list<int>::iterator i = pending_inc.new_out.begin();
+ i != pending_inc.new_out.end();
+ i++) {
+ dout(0) << " osd" << *i << " OUT" << endl;
+ derr(0) << " osd" << *i << " OUT" << endl;
+ }
- mon->store->put_bl_sn(bl, "osdmap", osdmap.get_epoch());
- mon->store->put_bl_sn(incbl, "osdincmap", osdmap.get_epoch());
- mon->store->put_int(osdmap.get_epoch(), "osd_epoch");
+ // encode
+ assert(paxos->get_version() + 1 == pending_inc.epoch);
+ pending_inc.encode(bl);
}
+// -------------
-void OSDMonitor::dispatch(Message *m)
+bool OSDMonitor::preprocess_query(Message *m)
{
+ dout(10) << "preprocess_query " << *m << " from " << m->get_source_inst() << endl;
+
switch (m->get_type()) {
-
- // services
+ // READs
case MSG_OSD_GETMAP:
handle_osd_getmap((MOSDGetMap*)m);
- break;
+ return true;
+
+ // damp updates
case MSG_OSD_FAILURE:
- handle_osd_failure((MOSDFailure*)m);
- break;
+ return preprocess_failure((MOSDFailure*)m);
case MSG_OSD_BOOT:
- handle_osd_boot((MOSDBoot*)m);
- break;
+ return preprocess_boot((MOSDBoot*)m);
+ /*
case MSG_OSD_IN:
- handle_osd_in((MOSDIn*)m);
- break;
+ return preprocess_in((MOSDIn*)m);
case MSG_OSD_OUT:
- handle_osd_out((MOSDOut*)m);
- break;
-
- // replication
- case MSG_MON_OSDMAP_INFO:
- handle_info((MMonOSDMapInfo*)m);
- break;
- case MSG_MON_OSDMAP_LEASE:
- handle_lease((MMonOSDMapLease*)m);
- break;
- case MSG_MON_OSDMAP_LEASE_ACK:
- handle_lease_ack((MMonOSDMapLeaseAck*)m);
- break;
- case MSG_MON_OSDMAP_UPDATE_PREPARE:
- handle_update_prepare((MMonOSDMapUpdatePrepare*)m);
- break;
- case MSG_MON_OSDMAP_UPDATE_ACK:
- handle_update_ack((MMonOSDMapUpdateAck*)m);
- break;
- case MSG_MON_OSDMAP_UPDATE_COMMIT:
- handle_update_commit((MMonOSDMapUpdateCommit*)m);
- break;
+ return preprocess_out((MOSDOut*)m);
+ */
default:
assert(0);
+ delete m;
+ return true;
+ }
+}
+
+bool OSDMonitor::prepare_update(Message *m)
+{
+ dout(10) << "prepare_update " << *m << " from " << m->get_source_inst() << endl;
+
+ switch (m->get_type()) {
+ // damp updates
+ case MSG_OSD_FAILURE:
+ return prepare_failure((MOSDFailure*)m);
+ case MSG_OSD_BOOT:
+ return prepare_boot((MOSDBoot*)m);
+
+ /*
+ case MSG_OSD_IN:
+ return prepare_in((MOSDIn*)m);
+ case MSG_OSD_OUT:
+ return prepare_out((MOSDOut*)m);
+ */
+
+ default:
+ assert(0);
+ delete m;
}
+
+ return false;
+}
+
+bool OSDMonitor::should_propose_now()
+{
+ // don't propose initial map until _all_ osds boot.
+ //dout(10) << "should_propose_now " << pending_inc.new_up.size() << " vs " << osdmap.get_osds().size() << endl;
+ if (osdmap.epoch == 1 &&
+ pending_inc.new_up.size() < osdmap.get_osds().size())
+ return false; // not all up (yet)
+
+ // FIXME do somethihng smart here.
+ return true;
}
-void OSDMonitor::handle_osd_failure(MOSDFailure *m)
+// ---------------------------
+// READs
+
+void OSDMonitor::handle_osd_getmap(MOSDGetMap *m)
{
- dout(1) << "osd failure: " << m->get_failed() << " from " << m->get_source() << endl;
+ dout(7) << "osd_getmap from " << m->get_source() << " since " << m->get_since() << endl;
- // FIXME
- // take their word for it
- int from = m->get_failed().name.num();
- if (osdmap.is_up(from) &&
- (osdmap.osd_inst.count(from) == 0 ||
- osdmap.osd_inst[from] == m->get_failed())) {
- pending_inc.new_down[from] = m->get_failed();
-
- if (osdmap.is_in(from))
- down_pending_out[from] = g_clock.now();
-
- //awaiting_maps[pending_inc.epoch][m->get_source()] =
-
- accept_pending();
-
- send_incremental(m->get_epoch(), m->get_source_inst());
-
- send_waiting();
- bcast_latest_mds();
- }
+ //if (m->get_since())
+ send_incremental(m->get_since(), m->get_source_inst());
+ //else
+ //send_full(m->get_source_inst());
delete m;
}
-void OSDMonitor::fake_osd_failure(int osd, bool down)
+
+// ---------------------------
+// UPDATEs
+
+// failure --
+
+bool OSDMonitor::preprocess_failure(MOSDFailure *m)
{
- if (down) {
- dout(1) << "fake_osd_failure DOWN osd" << osd << endl;
- pending_inc.new_down[osd] = osdmap.osd_inst[osd];
- } else {
- dout(1) << "fake_osd_failure OUT osd" << osd << endl;
- pending_inc.new_out.push_back(osd);
+ int badboy = m->get_failed().name.num();
+
+ // weird?
+ if (!osdmap.have_inst(badboy)) {
+ dout(5) << "preprocess_failure dne(/dup?): " << m->get_failed() << ", from " << m->get_from() << endl;
+ send_incremental(m->get_epoch(), m->get_from());
+ return true;
}
- accept_pending();
- bcast_latest_osd();
- bcast_latest_mds();
+ if (osdmap.get_inst(badboy) != m->get_failed()) {
+ dout(5) << "preprocess_failure wrong osd: report " << m->get_failed() << " != map's " << osdmap.get_inst(badboy)
+ << ", from " << m->get_from() << endl;
+ send_incremental(m->get_epoch(), m->get_from());
+ return true;
+ }
+ // already reported?
+ if (osdmap.is_down(badboy)) {
+ dout(5) << "preprocess_failure dup: " << m->get_failed() << ", from " << m->get_from() << endl;
+ send_incremental(m->get_epoch(), m->get_from());
+ return true;
+ }
+
+ dout(10) << "preprocess_failure new: " << m->get_failed() << ", from " << m->get_from() << endl;
+ return false;
}
-void OSDMonitor::mark_all_down()
+bool OSDMonitor::prepare_failure(MOSDFailure *m)
{
- dout(7) << "mark_all_down" << endl;
+ dout(1) << "prepare_failure " << m->get_failed() << " from " << m->get_from() << endl;
+
+ // FIXME
+ // take their word for it
+ int badboy = m->get_failed().name.num();
+ assert(osdmap.is_up(badboy));
+ assert(osdmap.osd_inst[badboy] == m->get_failed());
+
+ pending_inc.new_down[badboy] = m->get_failed();
+
+ if (osdmap.is_in(badboy))
+ down_pending_out[badboy] = g_clock.now();
- for (set<int>::iterator it = osdmap.get_osds().begin();
- it != osdmap.get_osds().end();
- it++) {
- if (osdmap.is_down(*it)) continue;
- pending_inc.new_down[*it] = osdmap.get_inst(*it);
- }
- accept_pending();
+ paxos->wait_for_commit(new C_Reported(this, m));
+
+ return true;
}
+void OSDMonitor::_reported_failure(MOSDFailure *m)
+{
+ dout(7) << "_reported_failure on " << m->get_failed() << ", telling " << m->get_from() << endl;
+ send_latest(m->get_epoch(), m->get_from());
+}
+// boot --
-void OSDMonitor::handle_osd_boot(MOSDBoot *m)
+bool OSDMonitor::preprocess_boot(MOSDBoot *m)
{
- dout(7) << "osd_boot from " << m->get_source() << endl;
- assert(m->get_source().is_osd());
- int from = m->get_source().num();
+ assert(m->inst.name.is_osd());
+ int from = m->inst.name.num();
- if (osdmap.get_epoch() == 0) {
- // waiting for boot!
- osdmap.osd_inst[from] = m->get_source_inst();
-
- if (osdmap.osd_inst.size() == osdmap.osds.size()) {
- dout(-7) << "osd_boot all osds booted." << endl;
- osdmap.inc_epoch();
-
- save_map();
-
- pending_inc.epoch = osdmap.get_epoch()+1; // 2
-
- bcast_latest_osd();
- bcast_latest_mds();
- send_waiting();
- } else {
- dout(7) << "osd_boot waiting for "
- << (osdmap.osds.size() - osdmap.osd_inst.size())
- << " osds to boot" << endl;
- }
+ // already booted?
+ if (osdmap.is_up(from) &&
+ osdmap.get_inst(from) == m->inst) {
+ // yup.
+ dout(7) << "preprocess_boot dup from " << m->inst << endl;
+ _booted(m);
+ return true;
+ }
+
+ dout(10) << "preprocess_boot from " << m->inst << endl;
+ return false;
+}
+bool OSDMonitor::prepare_boot(MOSDBoot *m)
+{
+ dout(7) << "prepare_boot from " << m->inst << endl;
+ assert(m->inst.name.is_osd());
+ int from = m->inst.name.num();
+
+ // does this osd exist?
+ if (!osdmap.exists(from)) {
+ dout(1) << "boot from non-existent osd" << from << endl;
delete m;
- return;
+ return true;
}
-
+
// already up? mark down first?
if (osdmap.is_up(from)) {
+ assert(osdmap.get_inst(from) != m->inst); // preproces should have caught it
+
+ // mark previous guy down
pending_inc.new_down[from] = osdmap.osd_inst[from];
- accept_pending();
}
- // mark up.
- down_pending_out.erase(from);
- assert(osdmap.is_down(from));
- pending_inc.new_up[from] = m->get_source_inst();
+ // mark new guy up.
+ down_pending_out.erase(from); // if any
+ pending_inc.new_up[from] = m->inst;
// mark in?
if (osdmap.out_osds.count(from))
pending_inc.new_in.push_back(from);
- accept_pending();
-
- // the booting osd will spread word
- send_incremental(m->sb.current_epoch, m->get_source_inst());
- delete m;
+ // wait
+ paxos->wait_for_commit(new C_Booted(this, m));
- // tell mds
- bcast_latest_mds();
+ return true;
+}
+
+void OSDMonitor::_booted(MOSDBoot *m)
+{
+ dout(7) << "_booted " << m->inst << endl;
+ send_latest(m->sb.current_epoch, m->inst);
+ delete m;
}
+
+// in --
+
+/*
void OSDMonitor::handle_osd_in(MOSDIn *m)
{
dout(7) << "osd_in from " << m->get_source() << endl;
send_incremental(m->map_epoch, m->get_source_inst());
}
}
-
-void OSDMonitor::handle_osd_getmap(MOSDGetMap *m)
-{
- dout(7) << "osd_getmap from " << m->get_source() << " since " << m->get_since() << endl;
-
- if (osdmap.get_epoch() == 0) {
- awaiting_map[m->get_source()].first = m->get_source_inst();
- awaiting_map[m->get_source()].second = m->get_since();
- } else {
- //if (m->get_since())
- send_incremental(m->get_since(), m->get_source_inst());
- //else
- //send_full(m->get_source(), m->get_source_inst());
- }
- delete m;
-}
-
+*/
-void OSDMonitor::accept_pending()
-{
- dout(-10) << "accept_pending " << osdmap.get_epoch() << " -> " << pending_inc.epoch << endl;
- // accept pending into a new map!
- pending_inc.ctime = g_clock.now();
- pending_inc.mon_epoch = mon->mon_epoch;
+// ---------------
+// map helpers
- // advance!
- osdmap.apply_incremental(pending_inc);
-
- // save it.
- save_inc_map( pending_inc );
-
- // tell me about it
- for (map<int,entity_inst_t>::iterator i = pending_inc.new_up.begin();
- i != pending_inc.new_up.end();
- i++) {
- dout(0) << "osd" << i->first << " UP " << i->second << endl;
- derr(0) << "osd" << i->first << " UP " << i->second << endl;
- }
- for (map<int,entity_inst_t>::iterator i = pending_inc.new_down.begin();
- i != pending_inc.new_down.end();
- i++) {
- dout(0) << "osd" << i->first << " DOWN " << i->second << endl;
- derr(0) << "osd" << i->first << " DOWN " << i->second << endl;
- messenger->mark_down(i->second.addr);
- }
- for (list<int>::iterator i = pending_inc.new_in.begin();
- i != pending_inc.new_in.end();
- i++) {
- dout(0) << "osd" << *i << " IN" << endl;
- derr(0) << "osd" << *i << " IN" << endl;
- }
- for (list<int>::iterator i = pending_inc.new_out.begin();
- i != pending_inc.new_out.end();
- i++) {
- dout(0) << "osd" << *i << " OUT" << endl;
- derr(0) << "osd" << *i << " OUT" << endl;
- }
-
- // clear new pending
- OSDMap::Incremental next(osdmap.get_epoch() + 1);
- pending_inc = next;
-}
-
-void OSDMonitor::send_waiting()
+void OSDMonitor::send_to_waiting()
{
- dout(10) << "send_waiting " << osdmap.get_epoch() << endl;
+ dout(10) << "send_to_waiting " << osdmap.get_epoch() << endl;
for (map<entity_name_t,pair<entity_inst_t,epoch_t> >::iterator i = awaiting_map.begin();
i != awaiting_map.end();
}
}
-
-void OSDMonitor::send_latest(entity_inst_t who)
+void OSDMonitor::send_latest(epoch_t since, entity_inst_t who)
{
- // FIXME this is super naive
- if (osdmap.get_epoch() == 0) {
- awaiting_map[who.name].first = who;
- awaiting_map[who.name].second = 0;
+ if (paxos->is_readable()) {
+ dout(5) << "send_latest to " << who << " now" << endl;
+ if (since)
+ send_incremental(since, who);
+ else
+ send_full(who);
} else {
- send_full(who);
+ dout(5) << "send_latest to " << who << " later" << endl;
+ awaiting_map[who.name].first = who;
+ awaiting_map[who.name].second = since;
}
}
+
void OSDMonitor::send_full(entity_inst_t who)
{
- messenger->send_message(new MOSDMap(&osdmap), who);
+ dout(5) << "send_full to " << who << endl;
+ mon->messenger->send_message(new MOSDMap(&osdmap), who);
}
void OSDMonitor::send_incremental(epoch_t since, entity_inst_t dest)
{
- dout(5) << "osd_send_incremental " << since << " -> " << osdmap.get_epoch()
+ dout(5) << "send_incremental " << since << " -> " << osdmap.get_epoch()
<< " to " << dest << endl;
MOSDMap *m = new MOSDMap;
e > since;
e--) {
bufferlist bl;
- if (get_inc_map_bl(e, bl)) {
- dout(10) << "osd_send_incremental inc " << e << endl;
+ if (mon->store->get_bl_sn(bl, "osdmap", e) > 0) {
+ dout(20) << "send_incremental inc " << e << endl;
m->incremental_maps[e] = bl;
}
- else if (get_map_bl(e, bl)) {
- dout(10) << "osd_send_incremental full " << e << endl;
+ else if (mon->store->get_bl_sn(bl, "osdmap_full", e) > 0) {
+ dout(20) << "send_incremental full " << e << endl;
m->maps[e] = bl;
}
else {
}
}
- messenger->send_message(m, dest);
+ mon->messenger->send_message(m, dest);
}
-
void OSDMonitor::bcast_latest_mds()
{
epoch_t e = osdmap.get_epoch();
}
}
+void OSDMonitor::bcast_full_osd()
+{
+ epoch_t e = osdmap.get_epoch();
+ dout(1) << "bcast_full_osd epoch " << e << endl;
+
+ // tell osds
+ set<int> osds;
+ osdmap.get_all_osds(osds);
+ for (set<int>::iterator it = osds.begin();
+ it != osds.end();
+ it++) {
+ if (osdmap.is_down(*it)) continue;
+ send_full(osdmap.get_inst(*it));
+ }
+}
+
+
+// TICK
void OSDMonitor::tick()
pending_inc.new_out.push_back( *i );
}
if (!mark_out.empty()) {
- accept_pending();
-
- // hrmpf. bcast map for now. FIXME FIXME.
- bcast_latest_osd();
+ propose_pending();
}
}
-void OSDMonitor::election_starting()
-{
- dout(10) << "election_starting" << endl;
-}
-void OSDMonitor::election_finished()
-{
- dout(10) << "election_finished" << endl;
- if (mon->is_leader()) {
- if (g_conf.mkfs) {
- create_initial();
- save_map();
- } else {
- //
- epoch_t epoch = mon->store->get_int("osd_epoch");
- dout(10) << " last epoch was " << epoch << endl;
- bufferlist bl, blinc;
- int r = mon->store->get_bl_sn(bl, "osdmap", epoch);
- assert(r>0);
- osdmap.decode(bl);
- // pending_inc
- pending_inc.epoch = epoch+1;
- }
- }
+/*
+void OSDMonitor::init()
+{
+ // start with blank map
- /*
- state = STATE_INIT;
+ // load my last state from the store
+ bufferlist bl;
+ if (get_map_bl(0, bl)) { // FIXME
+ // yay!
+ osdmap.decode(bl);
+ dout(1) << "init got epoch " << osdmap.get_epoch() << " from store" << endl;
- // map?
- if (osdmap.get_epoch() == 0 &&
- mon->is_leader()) {
- create_initial();
+ // set up pending_inc
+ pending_inc.epoch = osdmap.get_epoch()+1;
}
+}
+*/
- if (mon->is_leader()) {
- // leader.
- if (mon->monmap->num_mon == 1) {
- // hmm, it's just me!
- state = STATE_SYNC;
- }
- }
- else if (mon->is_peon()) {
- // peon. send info
- //messenger->send_message(new MMonOSDMapInfo(osdmap.epoch, osdmap.mon_epoch),
- // mon->monmap->get_inst(mon->leader));
- }
- */
-}
+void OSDMonitor::mark_all_down()
+{
+ assert(mon->is_leader());
+ dout(7) << "mark_all_down" << endl;
-void OSDMonitor::handle_info(MMonOSDMapInfo *m)
-{
- dout(10) << "handle_info from " << m->get_source()
- << " epoch " << m->get_epoch() << " in mon_epoch " << m->get_mon_epoch()
- << endl;
-
- epoch_t epoch = m->get_epoch();
-
- // did they have anything?
- if (epoch > 0) {
- // make sure it's current.
- if (epoch == osdmap.get_epoch()) {
- if (osdmap.mon_epoch != m->get_mon_epoch()) {
- dout(10) << "handle_info had divergent epoch " << m->get_epoch()
- << ", mon_epoch " << m->get_mon_epoch() << " != " << osdmap.mon_epoch << endl;
- epoch--;
- }
- } else {
- bufferlist bl;
- get_map_bl(epoch, bl);
-
- OSDMap old;
- old.decode(bl);
-
- if (old.mon_epoch != m->get_mon_epoch()) {
- dout(10) << "handle_info had divergent epoch " << m->get_epoch()
- << ", mon_epoch " << m->get_mon_epoch() << " != " << old.mon_epoch << endl;
- epoch--;
- }
- }
+ for (set<int>::iterator it = osdmap.get_osds().begin();
+ it != osdmap.get_osds().end();
+ it++) {
+ if (osdmap.is_down(*it)) continue;
+ pending_inc.new_down[*it] = osdmap.get_inst(*it);
}
-
- // bring up to date
- if (epoch < osdmap.get_epoch())
- send_incremental(epoch, m->get_source_inst());
-
- delete m;
+
+ propose_pending();
}
-void OSDMonitor::issue_leases()
-{
- dout(10) << "issue_leases" << endl;
- assert(mon->is_leader());
- // set lease endpoint
- lease_expire = g_clock.now();
- lease_expire += g_conf.mon_lease;
- pending_ack.clear();
-
- for (set<int>::iterator i = mon->quorum.begin();
- i != mon->quorum.end();
- i++) {
- if (*i == mon->whoami) continue;
- messenger->send_message(new MMonOSDMapLease(osdmap.get_epoch(), lease_expire),
- mon->monmap->get_inst(*i));
- pending_ack.insert(*i);
- }
-}
-void OSDMonitor::handle_lease(MMonOSDMapLease *m)
-{
- if (m->get_epoch() != osdmap.get_epoch() + 1) {
- dout(10) << "map_lease from " << m->get_source()
- << " on epoch " << m->get_epoch() << ", but i am " << osdmap.get_epoch() << endl;
- assert(0);
- delete m;
- return;
- }
-
- dout(10) << "map_lease from " << m->get_source() << " expires " << lease_expire << endl;
- lease_expire = m->get_lease_expire();
-
- delete m;
-}
-void OSDMonitor::handle_lease_ack(MMonOSDMapLeaseAck *m)
-{
- // right epoch?
- if (m->get_epoch() != osdmap.get_epoch()) {
- dout(10) << "map_lease_ack from " << m->get_source()
- << " on old epoch " << m->get_epoch() << ", dropping" << endl;
- delete m;
- return;
- }
-
- // within time limit?
- if (g_clock.now() >= lease_expire) {
- dout(10) << "map_lease_ack from " << m->get_source()
- << ", but lease expired, calling election" << endl;
- mon->call_election();
- delete m;
- return;
- }
-
- assert(m->get_source().is_mon());
- int from = m->get_source().num();
- assert(pending_ack.count(from));
- pending_ack.erase(from);
- if (pending_ack.empty()) {
- dout(10) << "map_lease_ack from " << m->get_source()
- << ", last one" << endl;
- } else {
- dout(10) << "map_lease_ack from " << m->get_source()
- << ", still waiting on " << pending_ack << endl;
- }
-
- delete m;
-}
-void OSDMonitor::update_map()
-{
- // lock map
- state = STATE_UPDATING;
- pending_ack.clear();
-
- // set lease endpoint
- lease_expire += g_conf.mon_lease;
- // send prepare
- epoch_t epoch = osdmap.get_epoch();
- bufferlist map_bl, inc_map_bl;
- if (!get_inc_map_bl(epoch, inc_map_bl))
- get_map_bl(epoch, map_bl);
- for (set<int>::iterator i = mon->quorum.begin();
- i != mon->quorum.end();
- i++) {
- if (*i == mon->whoami) continue;
- messenger->send_message(new MMonOSDMapUpdatePrepare(epoch,
- map_bl, inc_map_bl),
- mon->monmap->get_inst(*i));
- pending_ack.insert(*i);
- }
-}
-void OSDMonitor::handle_update_prepare(MMonOSDMapUpdatePrepare *m)
-{
- dout(10) << "map_update_prepare from " << m->get_source() << " epoch " << m->get_epoch() << endl;
- // accept map
- assert(m->get_epoch() == osdmap.get_epoch() + 1);
-
- if (m->inc_map_bl.length()) {
- int off = 0;
- pending_inc.decode(m->inc_map_bl, off);
- accept_pending();
- } else {
- osdmap.decode(m->map_bl);
- }
-
- // state
- state = STATE_LOCK;
- //lease_expire = m->lease_expire;
-
- // ack
- messenger->send_message(new MMonOSDMapUpdateAck(osdmap.get_epoch()),
- m->get_source_inst());
- delete m;
-}
+/*
-void OSDMonitor::handle_update_ack(MMonOSDMapUpdateAck *m)
+
+void OSDMonitor::election_finished()
{
- /*
- // right epoch?
- if (m->get_epoch() != osdmap.get_epoch()) {
- dout(10) << "map_update_ack from " << m->get_source()
- << " on old epoch " << m->get_epoch() << ", dropping" << endl;
- delete m;
- return;
- }
+ dout(10) << "election_finished" << endl;
- // within time limit?
- if (g_clock.now() >= lease_expire) {
- dout(10) << "map_update_ack from " << m->get_source()
- << ", but lease expired, calling election" << endl;
- state = STATE_SYNC;
- mon->call_election();
- return;
- }
+ if (mon->is_leader()) {
+ if (g_conf.mkfs) {
+ create_initial();
+ save_map();
+ } else {
+ //
+ epoch_t epoch = mon->store->get_int("osd_epoch");
+ dout(10) << " last epoch was " << epoch << endl;
+ bufferlist bl, blinc;
+ int r = mon->store->get_bl_sn(bl, "osdmap_full", epoch);
+ assert(r>0);
+ osdmap.decode(bl);
- assert(m->get_source().is_mon());
- int from = m->get_source().num();
+ // pending_inc
+ pending_inc.epoch = epoch+1;
+ }
- assert(pending_lease_ack.count(from));
- pending_lease_ack.erase(from);
-
- if (pending_lease_ack.empty()) {
- dout(10) << "map_update_ack from " << m->get_source()
- << ", last one" << endl;
- state = STATE_SYNC;
-
- // send lease commit
- for (map<int>::iterator i = mon->quorum.begin();
- i != mon->quorum.end();
- i++) {
- if (i == mon->whoami) continue;
- messenger->send_message(new MMonOSDMapLeaseCommit(osdmap),
- MSG_ADDR_MON(*i), mon->monmap->get_inst(*i));
- }
- } else {
- dout(10) << "map_update_ack from " << m->get_source()
- << ", still waiting on " << pending_lease_ack << endl;
}
-*/
-}
-void OSDMonitor::handle_update_commit(MMonOSDMapUpdateCommit *m)
-{
}
+
+
+
+*/
#include "osd/OSDMap.h"
-class Monitor;
+#include "PaxosService.h"
-class OSDMonitor : public Dispatcher {
- Monitor *mon;
- Messenger *messenger;
- Mutex &lock;
+class Monitor;
+class MOSDBoot;
- // osd maps
+class OSDMonitor : public PaxosService {
public:
OSDMap osdmap;
private:
map<entity_name_t, pair<entity_inst_t, epoch_t> > awaiting_map;
-
- void create_initial();
- bool get_map_bl(epoch_t epoch, bufferlist &bl);
- bool get_inc_map_bl(epoch_t epoch, bufferlist &bl);
-
- void save_map();
- void save_inc_map(OSDMap::Incremental &inc);
// [leader]
OSDMap::Incremental pending_inc;
map<int,utime_t> down_pending_out; // osd down -> out
- set<int> pending_ack;
+ // svc
+ void create_initial();
+ bool update_from_paxos();
+ void create_pending(); // prepare a new pending
+ void encode_pending(bufferlist &bl);
- // we are distributed
- const static int STATE_INIT = 0; // startup
- const static int STATE_SYNC = 1; // sync map copy (readonly)
- const static int STATE_LOCK = 2; // [peon] map locked
- const static int STATE_UPDATING = 3; // [leader] map locked, waiting for peon ack
+ void handle_query(Message *m);
+ bool preprocess_query(Message *m); // true if processed.
+ bool prepare_update(Message *m);
+ bool should_propose_now();
- int state;
- utime_t lease_expire; // when lease expires
+ // ...
+ bool get_map_bl(epoch_t epoch, bufferlist &bl);
+ bool get_inc_map_bl(epoch_t epoch, bufferlist &bl);
- //void init();
-
- // maps
- void accept_pending(); // accept pending, new map.
- void send_waiting(); // send current map to waiters.
+ void send_to_waiting(); // send current map to waiters.
void send_full(entity_inst_t dest);
void send_incremental(epoch_t since, entity_inst_t dest);
void bcast_latest_mds();
void bcast_latest_osd();
-
- void update_map();
-
- void handle_osd_boot(class MOSDBoot *m);
- void handle_osd_in(class MOSDIn *m);
- void handle_osd_out(class MOSDOut *m);
- void handle_osd_failure(class MOSDFailure *m);
+ void bcast_full_osd();
+
void handle_osd_getmap(class MOSDGetMap *m);
- void handle_info(class MMonOSDMapInfo*);
- void handle_lease(class MMonOSDMapLease*);
- void handle_lease_ack(class MMonOSDMapLeaseAck*);
- void handle_update_prepare(class MMonOSDMapUpdatePrepare*);
- void handle_update_ack(class MMonOSDMapUpdateAck*);
- void handle_update_commit(class MMonOSDMapUpdateCommit*);
+ bool preprocess_failure(class MOSDFailure *m);
+ bool prepare_failure(class MOSDFailure *m);
+ void _reported_failure(MOSDFailure *m);
+
+ bool preprocess_boot(class MOSDBoot *m);
+ bool prepare_boot(class MOSDBoot *m);
+ void _booted(MOSDBoot *m);
+
+ class C_Booted : public Context {
+ OSDMonitor *cmon;
+ MOSDBoot *m;
+ public:
+ C_Booted(OSDMonitor *cm, MOSDBoot *m_) :
+ cmon(cm), m(m_) {}
+ void finish(int r) {
+ if (r >= 0)
+ cmon->_booted(m);
+ else
+ cmon->dispatch((Message*)m);
+ }
+ };
+ class C_Reported : public Context {
+ OSDMonitor *cmon;
+ MOSDFailure *m;
+ public:
+ C_Reported(OSDMonitor *cm, MOSDFailure *m_) :
+ cmon(cm), m(m_) {}
+ void finish(int r) {
+ if (r >= 0)
+ cmon->_reported_failure(m);
+ else
+ cmon->dispatch((Message*)m);
+ }
+ };
+
+ bool preprocess_in(class MOSDIn *m);
+ bool prepare_in(class MOSDIn *m);
+
+ bool preprocess_out(class MOSDOut *m);
+ bool prepare_out(class MOSDOut *m);
public:
- OSDMonitor(Monitor *mn, Messenger *m, Mutex& l) :
- mon(mn), messenger(m), lock(l),
- state(STATE_SYNC) {
- //init();
- }
+ OSDMonitor(Monitor *mn, Paxos *p) :
+ PaxosService(mn, p) { }
- void dispatch(Message *m);
void tick(); // check state, take actions
- void election_starting(); // abort whatever.
- void election_finished(); // reinitialize whatever.
-
- void issue_leases();
-
void mark_all_down();
- void send_latest(entity_inst_t i);
+ void send_latest(epoch_t since, entity_inst_t i);
void fake_osd_failure(int osd, bool down);
void fake_osdmap_update();
#include "config.h"
#undef dout
-#define derr(l) if (l<=g_conf.debug || l<=g_conf.debug_mon) cerr << g_clock.now() << " mon" << whoami << (mon->is_starting() ? (const char*)"(starting)":(mon->is_leader() ? (const char*)"(leader)":(mon->is_peon() ? (const char*)"(peon)":(const char*)"(?\?)"))) << ".paxos(" << machine_name << ") "
-#define dout(l) if (l<=g_conf.debug || l<=g_conf.debug_mon) cout << g_clock.now() << " mon" << whoami << (mon->is_starting() ? (const char*)"(starting)":(mon->is_leader() ? (const char*)"(leader)":(mon->is_peon() ? (const char*)"(peon)":(const char*)"(?\?)"))) << ".paxos(" << machine_name << ") "
+#define derr(l) if (l<=g_conf.debug || l<=g_conf.debug_mon) cerr << g_clock.now() << " mon" << whoami << (mon->is_starting() ? (const char*)"(starting)":(mon->is_leader() ? (const char*)"(leader)":(mon->is_peon() ? (const char*)"(peon)":(const char*)"(?\?)"))) << ".paxos(" << machine_name << " " << get_statename(state) << " lc " << last_committed << ") "
+#define dout(l) if (l<=g_conf.debug || l<=g_conf.debug_mon) cout << g_clock.now() << " mon" << whoami << (mon->is_starting() ? (const char*)"(starting)":(mon->is_leader() ? (const char*)"(leader)":(mon->is_peon() ? (const char*)"(peon)":(const char*)"(?\?)"))) << ".paxos(" << machine_name << " " << get_statename(state) << " lc " << last_committed << ") "
+void Paxos::init()
+{
+ // load paxos variables from stable storage
+ last_pn = mon->store->get_int(machine_name, "last_pn");
+ accepted_pn = mon->store->get_int(machine_name, "accepted_pn");
+ last_committed = mon->store->get_int(machine_name, "last_committed");
+
+ dout(10) << "init" << endl;
+}
+
// ---------------------------------
// PHASE 1
-// proposer
-
+// leader
void Paxos::collect(version_t oldpn)
{
+ // we're recoverying, it seems!
+ state = STATE_RECOVERING;
+ assert(mon->is_leader());
+
// reset the number of lasts received
+ uncommitted_v = 0;
+ uncommitted_pn = 0;
+ uncommitted_value.clear();
+
+ // look for uncommitted value
+ if (mon->store->exists_bl_sn(machine_name, last_committed+1)) {
+ uncommitted_v = last_committed+1;
+ uncommitted_pn = accepted_pn;
+ mon->store->get_bl_sn(uncommitted_value, machine_name, last_committed+1);
+ dout(10) << "learned uncommitted " << (last_committed+1)
+ << " (" << uncommitted_value.length() << " bytes) from myself"
+ << endl;
+ }
+
+ // pick new pn
accepted_pn = get_new_proposal_number(MAX(accepted_pn, oldpn));
accepted_pn_from = last_committed;
num_last = 1;
- old_accepted_pn = 0;
- old_accepted_value.clear();
-
dout(10) << "collect with pn " << accepted_pn << endl;
// send collect
- for (int i=0; i<mon->monmap->num_mon; ++i) {
- if (i == whoami) continue;
+ for (set<int>::const_iterator p = mon->get_quorum().begin();
+ p != mon->get_quorum().end();
+ ++p) {
+ if (*p == whoami) continue;
- MMonPaxos *collect = new MMonPaxos(MMonPaxos::OP_COLLECT, machine_id);
+ MMonPaxos *collect = new MMonPaxos(mon->get_epoch(), MMonPaxos::OP_COLLECT, machine_id);
collect->last_committed = last_committed;
collect->pn = accepted_pn;
- mon->messenger->send_message(collect, mon->monmap->get_inst(i));
+ mon->messenger->send_message(collect, mon->monmap->get_inst(*p));
}
+
}
+
+// peon
void Paxos::handle_collect(MMonPaxos *collect)
{
dout(10) << "handle_collect " << *collect << endl;
+ assert(mon->is_peon()); // mon epoch filter should catch strays
+
+ // we're recoverying, it seems!
+ state = STATE_RECOVERING;
+
// reply
- MMonPaxos *last = new MMonPaxos(MMonPaxos::OP_LAST, machine_id);
+ MMonPaxos *last = new MMonPaxos(mon->get_epoch(), MMonPaxos::OP_LAST, machine_id);
last->last_committed = last_committed;
// do we have an accepted but uncommitted value?
if (mon->store->exists_bl_sn(machine_name, last_committed+1)) {
mon->store->get_bl_sn(bl, machine_name, last_committed+1);
assert(bl.length() > 0);
- dout(10) << "sharing our accepted but uncommitted value for " << last_committed+1 << endl;
+ dout(10) << " sharing our accepted but uncommitted value for " << last_committed+1
+ << " (" << bl.length() << " bytes)" << endl;
last->values[last_committed+1] = bl;
- last->old_accepted_pn = accepted_pn;
+ last->uncommitted_pn = accepted_pn;
}
// can we accept this pn?
accepted_pn = collect->pn;
accepted_pn_from = collect->pn_from;
dout(10) << "accepting pn " << accepted_pn << " from " << accepted_pn_from << endl;
+ mon->store->put_int(accepted_pn, machine_name, "accepted_pn");
} else {
// don't accept!
dout(10) << "NOT accepting pn " << collect->pn << " from " << collect->pn_from
last->pn_from = accepted_pn_from;
// and share whatever data we have
- for (version_t v = collect->last_committed;
+ for (version_t v = collect->last_committed+1;
v <= last_committed;
v++) {
if (mon->store->exists_bl_sn(machine_name, v)) {
mon->store->get_bl_sn(last->values[v], machine_name, v);
- dout(10) << " sharing " << v << " "
- << last->values[v].length() << " bytes" << endl;
+ dout(10) << " sharing " << v << " ("
+ << last->values[v].length() << " bytes)" << endl;
}
}
}
+// leader
void Paxos::handle_last(MMonPaxos *last)
{
dout(10) << "handle_last " << *last << endl;
+ if (!mon->is_leader()) {
+ dout(10) << "not leader, dropping" << endl;
+ delete last;
+ return;
+ }
+
// share committed values?
if (last->last_committed < last_committed) {
// share committed values
dout(10) << "sending commit to " << last->get_source() << endl;
- MMonPaxos *commit = new MMonPaxos(MMonPaxos::OP_COMMIT, machine_id);
- for (version_t v = last->last_committed;
+ MMonPaxos *commit = new MMonPaxos(mon->get_epoch(), MMonPaxos::OP_COMMIT, machine_id);
+ for (version_t v = last->last_committed+1;
v <= last_committed;
v++) {
mon->store->get_bl_sn(commit->values[v], machine_name, v);
- dout(10) << "sharing " << v << " "
- << commit->values[v].length() << " bytes" << endl;
+ dout(10) << " sharing " << v << " ("
+ << commit->values[v].length() << " bytes)" << endl;
}
+ commit->last_committed = last_committed;
mon->messenger->send_message(commit, last->get_source_inst());
}
- // did we receive committed value?
+ // did we receive a committed value?
if (last->last_committed > last_committed) {
- for (version_t v = last_committed;
+ for (version_t v = last_committed+1;
v <= last->last_committed;
v++) {
mon->store->put_bl_sn(last->values[v], machine_name, v);
<< last->values[v].length() << " bytes" << endl;
}
last_committed = last->last_committed;
- mon->store->put_int(last_committed, machine_name, "last_commtted");
+ mon->store->put_int(last_committed, machine_name, "last_committed");
dout(10) << "last_committed now " << last_committed << endl;
}
// do they accept your pn?
- if (last->old_accepted_pn > accepted_pn) {
- dout(10) << "uh oh, they have a higher pn than us. pick a new one." << endl;
- collect(last->old_accepted_pn);
+ if (last->pn > accepted_pn) {
+ // no, try again.
+ dout(10) << " they had a higher pn than us, picking a new one." << endl;
+ collect(last->pn);
} else {
- // they accepted our pn. great.
+ // yes, they accepted our pn. great.
num_last++;
- dout(10) << "great, they accepted our pn, we now have " << num_last << endl;
+ dout(10) << " they accepted our pn, we now have "
+ << num_last << " peons" << endl;
// did this person send back an accepted but uncommitted value?
- if (last->old_accepted_pn &&
- last->old_accepted_pn > old_accepted_pn) {
- version_t v = last->last_committed+1;
- dout(10) << "we learned an old value for " << v << " pn " << last->old_accepted_pn;
- old_accepted_pn = last->old_accepted_pn;
- old_accepted_value = last->values[v];
+ if (last->uncommitted_pn &&
+ last->uncommitted_pn > uncommitted_pn) {
+ uncommitted_v = last->last_committed+1;
+ uncommitted_pn = last->uncommitted_pn;
+ uncommitted_value = last->values[uncommitted_v];
+ dout(10) << "we learned an uncommitted value for " << uncommitted_v
+ << " pn " << uncommitted_pn
+ << " " << uncommitted_value.length() << " bytes"
+ << endl;
}
- // do we have a majority?
- if (num_last == mon->monmap->num_mon/2+1) {
- // do this once.
+ // is that everyone?
+ if (num_last == mon->get_quorum().size()) {
+ // almost...
+ state = STATE_ACTIVE;
// did we learn an old value?
- if (old_accepted_value.length()) {
- dout(10) << "begin on old learned value" << endl;
- begin(old_accepted_value);
- }
+ if (uncommitted_v == last_committed+1 &&
+ uncommitted_value.length()) {
+ dout(10) << "that's everyone. begin on old learned value" << endl;
+ begin(uncommitted_value);
+ } else {
+ // active!
+ dout(10) << "that's everyone. active!" << endl;
+ extend_lease();
+
+ // wake people up
+ finish_contexts(waiting_for_active);
+ finish_contexts(waiting_for_readable);
+ finish_contexts(waiting_for_writeable);
+ }
}
}
}
+// leader
void Paxos::begin(bufferlist& v)
{
dout(10) << "begin for " << last_committed+1 << " "
- << new_value.length() << " bytes"
+ << v.length() << " bytes"
<< endl;
- // we must already have a majority for this to work.
- assert(num_last > mon->monmap->num_mon/2);
+ assert(mon->is_leader());
+ assert(is_active());
+ state = STATE_UPDATING;
+ // we must already have a majority for this to work.
+ assert(mon->get_quorum().size() == 1 ||
+ num_last > (unsigned)mon->monmap->num_mon/2);
+
// and no value, yet.
assert(new_value.length() == 0);
-
+
// accept it ourselves
- num_accepted = 1;
+ accepted.clear();
+ accepted.insert(whoami);
new_value = v;
mon->store->put_bl_sn(new_value, machine_name, last_committed+1);
- // ask others to accept it to!
- for (int i=0; i<mon->monmap->num_mon; ++i) {
- if (i == whoami) continue;
+ if (mon->get_quorum().size() == 1) {
+ // we're alone, take it easy
+ commit();
+ state = STATE_ACTIVE;
+ finish_contexts(waiting_for_active);
+ finish_contexts(waiting_for_commit);
+ finish_contexts(waiting_for_readable);
+ finish_contexts(waiting_for_writeable);
+ return;
+ }
- dout(10) << " sending begin to mon" << i << endl;
- MMonPaxos *begin = new MMonPaxos(MMonPaxos::OP_BEGIN, machine_id);
+ // ask others to accept it to!
+ for (set<int>::const_iterator p = mon->get_quorum().begin();
+ p != mon->get_quorum().end();
+ ++p) {
+ if (*p == whoami) continue;
+
+ dout(10) << " sending begin to mon" << *p << endl;
+ MMonPaxos *begin = new MMonPaxos(mon->get_epoch(), MMonPaxos::OP_BEGIN, machine_id);
begin->values[last_committed+1] = new_value;
+ begin->last_committed = last_committed;
begin->pn = accepted_pn;
- mon->messenger->send_message(begin, mon->monmap->get_inst(i));
+ mon->messenger->send_message(begin, mon->monmap->get_inst(*p));
}
+
+ // set timeout event
+ accept_timeout_event = new C_AcceptTimeout(this);
+ mon->timer.add_event_after(g_conf.mon_accept_timeout, accept_timeout_event);
}
+// peon
void Paxos::handle_begin(MMonPaxos *begin)
{
dout(10) << "handle_begin " << *begin << endl;
// can we accept this?
- if (begin->pn != accepted_pn) {
+ if (begin->pn < accepted_pn) {
dout(10) << " we accepted a higher pn " << accepted_pn << ", ignoring" << endl;
delete begin;
return;
}
+ assert(begin->pn == accepted_pn);
+ assert(begin->last_committed == last_committed);
+ // set state.
+ state = STATE_UPDATING;
+ lease_expire = utime_t(); // cancel lease
+
// yes.
version_t v = last_committed+1;
dout(10) << "accepting value for " << v << " pn " << accepted_pn << endl;
mon->store->put_bl_sn(begin->values[v], machine_name, v);
// reply
- MMonPaxos *accept = new MMonPaxos(MMonPaxos::OP_ACCEPT, machine_id);
+ MMonPaxos *accept = new MMonPaxos(mon->get_epoch(), MMonPaxos::OP_ACCEPT, machine_id);
accept->pn = accepted_pn;
accept->last_committed = last_committed;
mon->messenger->send_message(accept, begin->get_source_inst());
delete begin;
}
-
+// leader
void Paxos::handle_accept(MMonPaxos *accept)
{
dout(10) << "handle_accept " << *accept << endl;
-
+ int from = accept->get_source().num();
+
if (accept->pn != accepted_pn) {
// we accepted a higher pn, from some other leader
dout(10) << " we accepted a higher pn " << accepted_pn << ", ignoring" << endl;
delete accept;
return;
}
- if (accept->last_committed != last_committed) {
- dout(10) << " this is from an old round that's already committed, ignoring" << endl;
+ if (last_committed > 0 &&
+ accept->last_committed < last_committed-1) {
+ dout(10) << " this is from an old round, ignoring" << endl;
delete accept;
return;
}
+ assert(accept->last_committed == last_committed || // not committed
+ accept->last_committed == last_committed-1); // committed
- num_accepted++;
- dout(10) << "now " << num_accepted << " have accepted" << endl;
+ assert(state == STATE_UPDATING);
+ assert(accepted.count(from) == 0);
+ accepted.insert(from);
+ dout(10) << " now " << accepted << " have accepted" << endl;
// new majority?
- if (num_accepted == mon->monmap->num_mon/2+1) {
+ if (accepted.size() == (unsigned)mon->monmap->num_mon/2+1) {
// yay, commit!
- dout(10) << "we got a majority, committing too" << endl;
+ // note: this may happen before the lease is reextended (below)
+ dout(10) << " got majority, committing" << endl;
commit();
- }
+ }
+
+ // done?
+ if (accepted == mon->get_quorum()) {
+ dout(10) << " got quorum, done with update" << endl;
+ // cancel timeout event
+ mon->timer.cancel_event(accept_timeout_event);
+ accept_timeout_event = 0;
+ // yay!
+ state = STATE_ACTIVE;
+ extend_lease();
+
+ // wake people up
+ finish_contexts(waiting_for_active);
+ finish_contexts(waiting_for_commit);
+ finish_contexts(waiting_for_readable);
+ finish_contexts(waiting_for_writeable);
+ }
+}
+
+void Paxos::accept_timeout()
+{
+ dout(5) << "accept timeout, calling fresh election" << endl;
+ accept_timeout_event = 0;
+ assert(mon->is_leader());
+ assert(is_updating());
+ cancel_events();
+ mon->call_election();
}
void Paxos::commit()
mon->store->put_int(last_committed, machine_name, "last_committed");
// tell everyone
- for (int i=0; i<mon->monmap->num_mon; ++i) {
- if (i == whoami) continue;
+ for (set<int>::const_iterator p = mon->get_quorum().begin();
+ p != mon->get_quorum().end();
+ ++p) {
+ if (*p == whoami) continue;
- dout(10) << " sending commit to mon" << i << endl;
- MMonPaxos *commit = new MMonPaxos(MMonPaxos::OP_COMMIT, machine_id);
+ dout(10) << " sending commit to mon" << *p << endl;
+ MMonPaxos *commit = new MMonPaxos(mon->get_epoch(), MMonPaxos::OP_COMMIT, machine_id);
commit->values[last_committed] = new_value;
commit->pn = accepted_pn;
- mon->messenger->send_message(commit, mon->monmap->get_inst(i));
+ mon->messenger->send_message(commit, mon->monmap->get_inst(*p));
}
// get ready for a new round.
new_value.clear();
-
}
{
dout(10) << "handle_commit on " << commit->last_committed << endl;
+ if (!mon->is_peon()) {
+ dout(10) << "not a peon, dropping" << endl;
+ assert(0);
+ delete commit;
+ return;
+ }
+
// commit locally.
- last_committed = commit->last_committed;
- mon->store->put_bl_sn(commit->values[last_committed], machine_name, last_committed);
+ for (map<version_t,bufferlist>::iterator p = commit->values.begin();
+ p != commit->values.end();
+ ++p) {
+ assert(p->first == last_committed+1);
+ last_committed = p->first;
+ dout(10) << " storing " << last_committed << " (" << p->second.length() << " bytes)" << endl;
+ mon->store->put_bl_sn(p->second, machine_name, last_committed);
+ }
mon->store->put_int(last_committed, machine_name, "last_committed");
delete commit;
-}
+}
+
+void Paxos::extend_lease()
+{
+ assert(mon->is_leader());
+ assert(is_active());
+
+ lease_expire = g_clock.now();
+ lease_expire += g_conf.mon_lease;
+ acked_lease.clear();
+ acked_lease.insert(whoami);
+
+ dout(7) << "extend_lease now+" << g_conf.mon_lease << " (" << lease_expire << ")" << endl;
+
+ // bcast
+ for (set<int>::const_iterator p = mon->get_quorum().begin();
+ p != mon->get_quorum().end();
+ ++p) {
+ if (*p == whoami) continue;
+ MMonPaxos *lease = new MMonPaxos(mon->get_epoch(), MMonPaxos::OP_LEASE, machine_id);
+ lease->last_committed = last_committed;
+ lease->lease_expire = lease_expire;
+ mon->messenger->send_message(lease, mon->monmap->get_inst(*p));
+ }
+ // set timeout event.
+ // if old timeout is still in place, leave it.
+ if (!lease_ack_timeout_event) {
+ lease_ack_timeout_event = new C_LeaseAckTimeout(this);
+ mon->timer.add_event_after(g_conf.mon_lease_ack_timeout, lease_ack_timeout_event);
+ }
+
+ // set renew event
+ lease_renew_event = new C_LeaseRenew(this);
+ utime_t at = lease_expire;
+ at -= g_conf.mon_lease;
+ at += g_conf.mon_lease_renew_interval;
+ mon->timer.add_event_at(at, lease_renew_event);
+}
+
+
+// peon
+void Paxos::handle_lease(MMonPaxos *lease)
+{
+ // sanity
+ if (!mon->is_peon() ||
+ last_committed != lease->last_committed) {
+ dout(10) << "handle_lease i'm not a peon, or they're not the leader, or the last_committed doesn't match, dropping" << endl;
+ delete lease;
+ return;
+ }
+
+ // extend lease
+ if (lease_expire < lease->lease_expire)
+ lease_expire = lease->lease_expire;
+
+ state = STATE_ACTIVE;
+
+ dout(10) << "handle_lease on " << lease->last_committed
+ << " now " << lease_expire << endl;
+
+ // ack
+ MMonPaxos *ack = new MMonPaxos(mon->get_epoch(), MMonPaxos::OP_LEASE_ACK, machine_id);
+ ack->last_committed = last_committed;
+ ack->lease_expire = lease_expire;
+ mon->messenger->send_message(ack, lease->get_source_inst());
+
+ // (re)set timeout event.
+ if (lease_timeout_event)
+ mon->timer.cancel_event(lease_timeout_event);
+ lease_timeout_event = new C_LeaseTimeout(this);
+ mon->timer.add_event_after(g_conf.mon_lease_ack_timeout, lease_timeout_event);
+
+ // kick waiters
+ finish_contexts(waiting_for_active);
+ if (is_readable())
+ finish_contexts(waiting_for_readable);
+
+ delete lease;
+}
+
+void Paxos::handle_lease_ack(MMonPaxos *ack)
+{
+ int from = ack->get_source().num();
+
+ if (!lease_ack_timeout_event) {
+ dout(10) << "handle_lease_ack from " << ack->get_source() << " -- stray (probably since revoked)" << endl;
+ }
+ else if (acked_lease.count(from) == 0) {
+ acked_lease.insert(from);
+
+ if (acked_lease == mon->get_quorum()) {
+ // yay!
+ dout(10) << "handle_lease_ack from " << ack->get_source()
+ << " -- got everyone" << endl;
+ mon->timer.cancel_event(lease_ack_timeout_event);
+ lease_ack_timeout_event = 0;
+ } else {
+ dout(10) << "handle_lease_ack from " << ack->get_source()
+ << " -- still need "
+ << mon->get_quorum().size() - acked_lease.size()
+ << " more" << endl;
+ }
+ } else {
+ dout(10) << "handle_lease_ack from " << ack->get_source()
+ << " dup (lagging!), ignoring" << endl;
+ }
+
+ delete ack;
+}
+
+void Paxos::lease_ack_timeout()
+{
+ dout(5) << "lease_ack_timeout -- calling new election" << endl;
+ assert(mon->is_leader());
+ assert(is_active());
+
+ lease_ack_timeout_event = 0;
+ cancel_events();
+ mon->call_election();
+}
+
+void Paxos::lease_timeout()
+{
+ dout(5) << "lease_timeout -- calling new election" << endl;
+ assert(mon->is_peon());
+
+ lease_timeout_event = 0;
+ cancel_events();
+ mon->call_election();
+}
+
+void Paxos::lease_renew_timeout()
+{
+ lease_renew_event = 0;
+ extend_lease();
+}
/*
*/
version_t Paxos::get_new_proposal_number(version_t gt)
{
- // read last
- version_t last = mon->store->get_int("last_paxos_proposal");
- if (last < gt)
- last = gt;
+ if (last_pn < gt)
+ last_pn = gt;
- // update
- last /= 100;
- last++;
-
- // make it unique among all monitors.
- version_t pn = last*100 + (version_t)whoami;
+ // update. make it unique among all monitors.
+ last_pn /= 100;
+ last_pn++;
+ last_pn *= 100;
+ last_pn += (version_t)whoami;
// write
- mon->store->put_int(pn, "last_paxos_proposal");
+ mon->store->put_int(last_pn, machine_name, "last_pn");
- dout(10) << "get_new_proposal_number = " << pn << endl;
- return pn;
+ dout(10) << "get_new_proposal_number = " << last_pn << endl;
+ return last_pn;
}
-void Paxos::leader_start()
+void Paxos::cancel_events()
{
- dout(10) << "leader_start -- i am the leader, start paxos" << endl;
+ if (accept_timeout_event) {
+ mon->timer.cancel_event(accept_timeout_event);
+ accept_timeout_event = 0;
+ }
+ if (lease_renew_event) {
+ mon->timer.cancel_event(lease_renew_event);
+ lease_renew_event = 0;
+ }
+ if (lease_ack_timeout_event) {
+ mon->timer.cancel_event(lease_ack_timeout_event);
+ lease_ack_timeout_event = 0;
+ }
+ if (lease_timeout_event) {
+ mon->timer.cancel_event(lease_timeout_event);
+ lease_timeout_event = 0;
+ }
+}
+
+void Paxos::leader_init()
+{
+ if (mon->get_quorum().size() == 1) {
+ state = STATE_ACTIVE;
+ return;
+ }
+ cancel_events();
+ state = STATE_RECOVERING;
+ lease_expire = utime_t();
+ dout(10) << "leader_init -- starting paxos recovery" << endl;
collect(0);
}
+void Paxos::peon_init()
+{
+ cancel_events();
+ state = STATE_RECOVERING;
+ lease_expire = utime_t();
+ dout(10) << "peon_init -- i am a peon" << endl;
+
+ // no chance to write now!
+ finish_contexts(waiting_for_writeable, -1);
+ finish_contexts(waiting_for_commit, -1);
+}
+
+void Paxos::election_starting()
+{
+ dout(10) << "election_starting -- canceling timeouts" << endl;
+ cancel_events();
+ new_value.clear();
+
+ finish_contexts(waiting_for_commit, -1);
+}
+
void Paxos::dispatch(Message *m)
{
+ // election in progress?
+ if (mon->is_starting()) {
+ dout(5) << "election in progress, dropping " << *m << endl;
+ delete m;
+ return;
+ }
+
+ // check sanity
+ assert(mon->is_leader() ||
+ (mon->is_peon() && m->get_source().num() == mon->get_leader()));
+
switch (m->get_type()) {
-
+
case MSG_MON_PAXOS:
{
MMonPaxos *pm = (MMonPaxos*)m;
case MMonPaxos::OP_COLLECT:
handle_collect(pm);
break;
-
case MMonPaxos::OP_LAST:
handle_last(pm);
break;
-
case MMonPaxos::OP_BEGIN:
handle_begin(pm);
break;
-
case MMonPaxos::OP_ACCEPT:
handle_accept(pm);
break;
-
case MMonPaxos::OP_COMMIT:
handle_commit(pm);
break;
-
+ case MMonPaxos::OP_LEASE:
+ handle_lease(pm);
+ break;
+ case MMonPaxos::OP_LEASE_ACK:
+ handle_lease_ack(pm);
+ break;
default:
assert(0);
}
}
}
+
+
+
+// -----------------
+// service interface
+
+// -- READ --
+
+bool Paxos::is_readable()
+{
+ //dout(15) << "is_readable now=" << g_clock.now() << " lease_expire=" << lease_expire << endl;
+ return
+ (mon->is_peon() || mon->is_leader()) &&
+ is_active() &&
+ last_committed > 0 && // must have a value
+ (mon->get_quorum().size() == 1 || // alone, or
+ g_clock.now() < lease_expire); // have lease
+}
+
+bool Paxos::read(version_t v, bufferlist &bl)
+{
+ if (!is_readable())
+ return false;
+
+ if (!mon->store->get_bl_sn(bl, machine_name, v))
+ return false;
+ return true;
+}
+
+version_t Paxos::read_current(bufferlist &bl)
+{
+ if (!is_readable())
+ return 0;
+ if (read(last_committed, bl))
+ return last_committed;
+ return 0;
+}
+
+
+
+
+// -- WRITE --
+
+bool Paxos::is_writeable()
+{
+ if (mon->get_quorum().size() == 1) return true;
+ return
+ mon->is_leader() &&
+ is_active() &&
+ g_clock.now() < lease_expire;
+}
+
+bool Paxos::propose_new_value(bufferlist& bl, Context *oncommit)
+{
+ /*
+ // writeable?
+ if (!is_writeable()) {
+ dout(5) << "propose_new_value " << last_committed+1 << " " << bl.length() << " bytes"
+ << " -- not writeable" << endl;
+ if (oncommit) {
+ oncommit->finish(-1);
+ delete oncommit;
+ }
+ return false;
+ }
+ */
+
+ assert(mon->is_leader() && is_active());
+
+ // cancel lease renewal and timeout events.
+ cancel_events();
+
+ // ok!
+ dout(5) << "propose_new_value " << last_committed+1 << " " << bl.length() << " bytes" << endl;
+ if (oncommit)
+ waiting_for_commit.push_back(oncommit);
+ begin(bl);
+
+ return true;
+}
+
*/
+
+/*
+ * NOTE: This libary is based on the Paxos algorithm, but varies in a few key ways:
+ * 1- Only a single new value is generated at a time, simplifying the recovery logic.
+ * 2- Nodes track "committed" values, and share them generously (and trustingly)
+ * 3- A 'leasing' mechism is built-in, allowing nodes to determine when it is safe to
+ * "read" their copy of the last committed value.
+ *
+ * This provides a simple replication substrate that services can be built on top of.
+ */
+
#ifndef __MON_PAXOS_H
#define __MON_PAXOS_H
#include "include/types.h"
+#include "mon_types.h"
#include "include/buffer.h"
#include "msg/Message.h"
class Monitor;
class MMonPaxos;
+
// i am one state machine.
class Paxos {
Monitor *mon;
int machine_id;
const char *machine_name;
- // phase 1
+ friend class PaxosService;
+
+ // LEADER+PEON
+
+ // -- generic state --
+public:
+ const static int STATE_RECOVERING = 1; // leader|peon: recovering paxos state
+ const static int STATE_ACTIVE = 2; // leader|peon: idle. peon may or may not have valid lease
+ const static int STATE_UPDATING = 3; // leader|peon: updating to new value
+ const char *get_statename(int s) {
+ switch (s) {
+ case STATE_RECOVERING: return "recovering";
+ case STATE_ACTIVE: return "active";
+ case STATE_UPDATING: return "updating";
+ default: assert(0); return 0;
+ }
+ }
+
+private:
+ int state;
+
+public:
+ bool is_recovering() { return state == STATE_RECOVERING; }
+ bool is_active() { return state == STATE_ACTIVE; }
+ bool is_updating() { return state == STATE_UPDATING; }
+
+private:
+ // recovery (phase 1)
+ version_t last_pn;
version_t last_committed;
version_t accepted_pn;
version_t accepted_pn_from;
-
- // results from our last replies
- int num_last;
- version_t old_accepted_pn;
- bufferlist old_accepted_value;
- // phase 2
+ // active (phase 2)
+ utime_t lease_expire;
+ list<Context*> waiting_for_active;
+ list<Context*> waiting_for_readable;
+
+
+ // -- leader --
+ // recovery (paxos phase 1)
+ unsigned num_last;
+ version_t uncommitted_v;
+ version_t uncommitted_pn;
+ bufferlist uncommitted_value;
+
+ // active
+ set<int> acked_lease;
+ Context *lease_renew_event;
+ Context *lease_ack_timeout_event;
+ Context *lease_timeout_event;
+
+ // updating (paxos phase 2)
bufferlist new_value;
- int num_accepted;
-
+ set<int> accepted;
+
+ Context *accept_timeout_event;
+
+ list<Context*> waiting_for_writeable;
+ list<Context*> waiting_for_commit;
+
+ class C_AcceptTimeout : public Context {
+ Paxos *paxos;
+ public:
+ C_AcceptTimeout(Paxos *p) : paxos(p) {}
+ void finish(int r) {
+ paxos->accept_timeout();
+ }
+ };
+
+ class C_LeaseAckTimeout : public Context {
+ Paxos *paxos;
+ public:
+ C_LeaseAckTimeout(Paxos *p) : paxos(p) {}
+ void finish(int r) {
+ paxos->lease_ack_timeout();
+ }
+ };
+
+ class C_LeaseTimeout : public Context {
+ Paxos *paxos;
+ public:
+ C_LeaseTimeout(Paxos *p) : paxos(p) {}
+ void finish(int r) {
+ paxos->lease_timeout();
+ }
+ };
+
+ class C_LeaseRenew : public Context {
+ Paxos *paxos;
+ public:
+ C_LeaseRenew(Paxos *p) : paxos(p) {}
+ void finish(int r) {
+ paxos->lease_renew_timeout();
+ }
+ };
+
+
void collect(version_t oldpn);
void handle_collect(MMonPaxos*);
void handle_last(MMonPaxos*);
void begin(bufferlist& value);
void handle_begin(MMonPaxos*);
void handle_accept(MMonPaxos*);
+ void accept_timeout();
void commit();
void handle_commit(MMonPaxos*);
+ void extend_lease();
+ void handle_lease(MMonPaxos*);
+ void handle_lease_ack(MMonPaxos*);
+
+ void lease_ack_timeout(); // on leader, if lease isn't acked by all peons
+ void lease_renew_timeout(); // on leader, to renew the lease
+ void lease_timeout(); // on peon, if lease isn't extended
+
+ void cancel_events();
version_t get_new_proposal_number(version_t gt=0);
public:
Paxos(Monitor *m, int w,
- int mid,const char *mnm) : mon(m), whoami(w),
- machine_id(mid), machine_name(mnm) {
- }
+ int mid) : mon(m), whoami(w),
+ machine_id(mid),
+ machine_name(get_paxos_name(mid)),
+ state(STATE_RECOVERING),
+ lease_renew_event(0),
+ lease_ack_timeout_event(0),
+ lease_timeout_event(0),
+ accept_timeout_event(0) { }
void dispatch(Message *m);
- void leader_start();
+ void init();
+
+ void election_starting();
+ void leader_init();
+ void peon_init();
+
+
+ // -- service interface --
+ void wait_for_active(Context *c) {
+ assert(!is_active());
+ waiting_for_active.push_back(c);
+ }
+
+ // read
+ version_t get_version() { return last_committed; }
+ bool is_readable();
+ bool read(version_t v, bufferlist &bl);
+ version_t read_current(bufferlist &bl);
+ void wait_for_readable(Context *onreadable) {
+ assert(!is_readable());
+ waiting_for_readable.push_back(onreadable);
+ }
+
+ // write
+ bool is_leader();
+ bool is_writeable();
+ void wait_for_writeable(Context *c) {
+ assert(!is_writeable());
+ waiting_for_writeable.push_back(c);
+ }
+
+ bool propose_new_value(bufferlist& bl, Context *oncommit=0);
+ void wait_for_commit(Context *oncommit) {
+ waiting_for_commit.push_back(oncommit);
+ }
+ void wait_for_commit_front(Context *oncommit) {
+ waiting_for_commit.push_front(oncommit);
+ }
};
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
+ *
+ * This is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License version 2.1, as published by the Free Software
+ * Foundation. See file COPYING.
+ *
+ */
+
+#include "PaxosService.h"
+#include "common/Clock.h"
+#include "Monitor.h"
+
+
+
+#include "config.h"
+#undef dout
+#define dout(l) if (l<=g_conf.debug || l<=g_conf.debug_mon) cout << g_clock.now() << " mon" << mon->whoami << (mon->is_starting() ? (const char*)"(starting)":(mon->is_leader() ? (const char*)"(leader)":(mon->is_peon() ? (const char*)"(peon)":(const char*)"(?\?)"))) << ".paxosservice(" << get_paxos_name(paxos->machine_id) << ") "
+//#define derr(l) if (l<=g_conf.debug || l<=g_conf.debug_mon) cerr << g_clock.now() << " mon" << mon->whoami << (mon->is_starting() ? (const char*)"(starting)":(mon->is_leader() ? (const char*)"(leader)":(mon->is_peon() ? (const char*)"(peon)":(const char*)"(?\?)"))) << "." << get_paxos_name(paxos->machine_id) << " "
+
+
+
+
+void PaxosService::dispatch(Message *m)
+{
+ dout(10) << "dispatch " << *m << " from " << m->get_source_inst() << endl;
+
+ // make sure our map is readable and up to date
+ if (!paxos->is_readable()) {
+ dout(10) << " waiting for paxos -> readable" << endl;
+ paxos->wait_for_readable(new C_RetryMessage(this, m));
+ return;
+ }
+
+ // make sure service has latest from paxos.
+ update_from_paxos();
+
+ // preprocess
+ if (preprocess_query(m))
+ return; // easy!
+
+ // leader?
+ if (!mon->is_leader()) {
+ // fw to leader
+ dout(10) << " fw to leader mon" << mon->get_leader() << endl;
+ mon->messenger->send_message(m, mon->monmap->get_inst(mon->get_leader()));
+ return;
+ }
+
+ // writeable?
+ if (!paxos->is_writeable()) {
+ dout(10) << " waiting for paxos -> writeable" << endl;
+ paxos->wait_for_writeable(new C_RetryMessage(this, m));
+ return;
+ }
+
+ // update
+ if (prepare_update(m) &&
+ should_propose_now())
+ propose_pending();
+}
+
+void PaxosService::_commit()
+{
+ dout(7) << "_commit" << endl;
+ update_from_paxos(); // notify service of new paxos state
+
+ if (mon->is_leader()) {
+ dout(7) << "_commit creating new pending" << endl;
+ assert(have_pending == false);
+ create_pending();
+ have_pending = true;
+ }
+}
+
+
+void PaxosService::propose_pending()
+{
+ dout(10) << "propose_pending" << endl;
+ assert(have_pending);
+
+ // finish and encode
+ bufferlist bl;
+ encode_pending(bl);
+ have_pending = false;
+
+ // apply to paxos
+ paxos->wait_for_commit_front(new C_Commit(this));
+ paxos->propose_new_value(bl);
+}
+
+
+void PaxosService::election_finished()
+{
+ dout(10) << "election_finished" << endl;
+
+ if (have_pending &&
+ !mon->is_leader()) {
+ discard_pending();
+ have_pending = false;
+ }
+
+ // make sure we update our state
+ if (paxos->is_active())
+ _active();
+ else
+ paxos->wait_for_active(new C_Active(this));
+}
+
+void PaxosService::_active()
+{
+ dout(10) << "_active" << endl;
+ assert(paxos->is_active());
+
+ // pull latest from paxos
+ update_from_paxos();
+
+ // create pending state?
+ if (mon->is_leader()) {
+ if (!have_pending) {
+ create_pending();
+ have_pending = true;
+ }
+
+ if (g_conf.mkfs &&
+ paxos->get_version() == 0) {
+ create_initial();
+ propose_pending();
+ }
+ }
+}
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
+ *
+ * This is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License version 2.1, as published by the Free Software
+ * Foundation. See file COPYING.
+ *
+ */
+
+#ifndef __PAXOSSERVICE_H
+#define __PAXOSSERVICE_H
+
+#include "msg/Dispatcher.h"
+#include "include/Context.h"
+
+class Monitor;
+class Paxos;
+
+class PaxosService : public Dispatcher {
+protected:
+ Monitor *mon;
+ Paxos *paxos;
+
+ class C_RetryMessage : public Context {
+ PaxosService *svc;
+ Message *m;
+ public:
+ C_RetryMessage(PaxosService *s, Message *m_) : svc(s), m(m_) {}
+ void finish(int r) {
+ svc->dispatch(m);
+ }
+ };
+ class C_Active : public Context {
+ PaxosService *svc;
+ public:
+ C_Active(PaxosService *s) : svc(s) {}
+ void finish(int r) {
+ if (r >= 0)
+ svc->_active();
+ }
+ };
+ class C_Commit : public Context {
+ PaxosService *svc;
+ public:
+ C_Commit(PaxosService *s) : svc(s) {}
+ void finish(int r) {
+ if (r >= 0)
+ svc->_commit();
+ }
+ };
+ friend class C_Update;
+
+private:
+ bool have_pending;
+
+public:
+ PaxosService(Monitor *mn, Paxos *p) : mon(mn), paxos(p),
+ have_pending(false) { }
+
+ // i implement and you ignore
+ void dispatch(Message *m);
+ void election_finished();
+
+private:
+ void _active();
+ void _commit();
+
+public:
+ // i implement and you use
+ void propose_pending(); // propose current pending as new paxos state
+
+ // you implement
+ virtual bool update_from_paxos() = 0; // assimilate latest paxos state
+ virtual void create_pending() = 0; // [leader] create new pending structures
+ virtual void create_initial() = 0; // [leader] populate pending with initial state (1)
+ virtual void encode_pending(bufferlist& bl) = 0; // [leader] finish and encode pending for next paxos state
+ virtual void discard_pending() { } // [leader] discard pending
+
+ virtual bool preprocess_query(Message *m) = 0; // true if processed (e.g., read-only)
+ virtual bool prepare_update(Message *m) = 0;
+ virtual bool should_propose_now() { return true; }
+
+};
+
+#endif
+
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
+ *
+ * This is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License version 2.1, as published by the Free Software
+ * Foundation. See file COPYING.
+ *
+ */
+
+#ifndef __MON_TYPES_H
+#define __MON_TYPES_H
+
+#define PAXOS_TEST 0
+#define PAXOS_MDSMAP 1
+#define PAXOS_OSDMAP 2
+#define PAXOS_CLIENTMAP 3
+
+inline const char *get_paxos_name(int p) {
+ switch (p) {
+ case PAXOS_TEST: return "test";
+ case PAXOS_MDSMAP: return "mdsmap";
+ case PAXOS_OSDMAP: return "osdmap";
+ case PAXOS_CLIENTMAP: return "clientmap";
+ default: assert(0); return 0;
+ }
+}
+
+#endif