]> git.apps.os.sepia.ceph.com Git - ceph.git/commitdiff
mon: Single-paxos and key/value store support
authorJoao Eduardo Luis <joao.luis@inktank.com>
Mon, 11 Jun 2012 13:55:21 +0000 (14:55 +0100)
committerJoao Eduardo Luis <joao.luis@inktank.com>
Thu, 21 Feb 2013 18:02:14 +0000 (18:02 +0000)
We are converting the monitor subsystem to a Single-Paxos architecture,
backed by a key/value store. The previous architecture used a Paxos
instance for each Paxos Service, backed by a nasty Monitor Store that
provided few to no consistency guarantees whatsoever, which led to a fair
amount of workarounds.

Changes:

* Paxos:
  - Add k/v store support
  - Add documentation describing the new Paxos storage layout and behavior
  - Get rid of the stashing code, which was used as a consistency point
    mechanism (we no longer need it, because of our k/v store)
  - Debug level of 30 will output json-formatted transaction dumps
  - Allows for proposal queueing, to be proposed in the same order as
    they were queued.
  - No more 'is_leader()' function, using instead the Monitor's for
    enhanced simplicity.
  - Add 'is_lease_valid()' function.
  - Disregard 'stashed versions'
  - Make the paxos 'state' variable a bit-map, so we lock the proposal
    mechanism while maintaining the state [5].
  - Related notes: [3]

* PaxosService:
  - Add k/v store support, creating wrappers to be used by the services
  - Add documentation
  - Support single-paxos behavior, creating wrappers to be used by the
    services and service-specific version
  - Rearrange variables so they are neatly organized in the beginning of
    the class
  - Add a trim_to() function to be used by the services, instead of letting
    them rely on Paxos::trim_to(), which is no longer adequate to the job
    at hand
  - Debug level of 30 will output json-formatted transaction dumps
  - Support proposal queueing, taking it into consideration when
    assessing the current state of the service (active, writeable,
    readable, ...)
  - Redefine the conditions for 'is_{active,readable,writeable}()' given
    the new single-paxos approach, with proposal queueing [1].
  - Use our own waiting_for_* callback lists, which now must be
    dissociated from their Paxos counterparts [2].
  - Related notes: [3], [4]

* Monitor:
  - Add k/v store support
  - Use only one Paxos instance and pass it down to each service instance
  - Crank up CEPH_MON_PROTOCOL to 10

* {Auth,Log,MDS,Monmap,OSD,PG}Monitor:
  - Add k/v store support
  - Add single-paxos support

* AuthMonitor:
  - Don't always propose full versions: if the KeyServer doesn't have
    keys, we cannot propose a full version. This should only happen when
    we start with a brand new store and we are creating the first
    pending proposal, and if we were to commit a full version filled
    with nothing but a big void of nothingness, we could eventually end
    up with a corrupted version.

* Elector:
  - Add k/v store support
  - Add single-paxos support

* ceph-mon:
  - Use the monitor's k/v store instead of MonitorStore

* MMonPaxos:
  - remove the machine_id field: This field was used to identify from/to
    which paxos service a given message belonged. We no longer have a Paxos
    for each service, so this field became obsolete.

Notes:

[1] Redefine the conditions for 'is_{active,readable,writeable}()' on
    the PaxosService class, to be used with single-paxos and proposal
    queueing:

  We should not rely on the Paxos::is_*() functions, since they do not apply
  directly to the PaxosService.

  All the PaxosService classes share the same Paxos class, but they do not
  rely on its values. Each service only relies, uses and updates its own
  values on the k/v store. Thus, we may have a given service (e.g., the
  OSDMonitor) proposing a new value, hence updating or waiting to update its
  store, and we may still consider the LogMonitor as being able to read and
  write its own values on the k/v store. In a nutshell, different services
  do not overlap on their access to their own store when it comes to reading,
  and since the Paxos will queue their updates and deal with them in a FIFO
  order, their updates won't overlap either.

  Therefore, the conditions for the PaxosService::is_{active,readable,
  writeable} differ from those on the Paxos::is_{active,readable,writeable}.

  * PaxosService::is_active() - the PaxosService will be considered as
  active iff it is not proposing and the Paxos is not recovering. This
  means that a given PaxosService (e.g., the OSDMonitor) may be considered
  as being active even though some other service (e.g., the LogMonitor) is
  proposing a new value and the Paxos is on the UPDATING state. This means
  that the OSDMonitor will be able to read its own versions and queue any
  changes on to the Paxos. However, if the Paxos is on state RECOVERING,
  we cannot be considered as active.

  * PaxosService::is_writeable() - We will be able to propose new values
  iff we are the Leader, we have a valid lease, and we are not already
  proposing. If we are proposing, we must wait for our proposal to finish
  in order to proceed with writing to our k/v store; otherwise we could
  incur in assuming that our last committed version was, say, 10; then
  assign map epochs/versions taking that into consideration, make changes
  to the store based on those values, just to come to smash previously
  proposed values on the store. We really don't want that. To be fair,
  there was a chance we could assume we were always writable, but there
  may be unforeseen consequences to this; so we take the conservative
  approach here for now, and we will relax it in the future if we believe
  it to be fruitful.

  * PaxosService::is_readable() - We will be readable iff we are not
  proposing and the Paxos is not recovering; if our last committed version
  exists; and if we are either a cluster of one or we have a valid lease.

[2] Use own waiting_for_* callback lists on PaxosService, which now must
    be dissociated from their Paxos counterparts:

  We were relying on Paxos to wait for state changes, but since our state
  became somewhat independent from the Paxos state, we have to deal with
  callbacks waiting for 'readable', 'writable' or 'active' on different
  terms than those that Paxos provide.

  So, basically, we will take one of two approaches when it comes to waiting:

  * If we are proposing, queue ourselves on our own list, waiting for the
  proposal to finish;
  * Otherwise, the cause for the need to wait comes from Paxos, so queue
  the callback directly on Paxos.

  This approach means that we must make sure to check our desired state
  whenever the callback is fired up, and re-queue ourselves if the state
  didn't quite change (or if it changed but our waiting condition result
  didn't). For instance, if we were waiting for a proposal to finish due to
  a failed 'is_active()', we will need to recheck if we are active before
  continuing once the callback is fired. This is mainly because we may have
  finished our proposal, but a new Election may have been called and the
  Paxos may not be active.

[3] Propose everything in the queue before bootstrapping, but don't
    allow new proposals:

  The MonmapMonitor may issue bootstraps once it is updated. We must ensure
  that we propose every single pending proposal before we actually do it.

  However, ee don't want to propose if we are going to bootstrap; otherwise,
  we may end up losing proposals.

[4] Handle the case when first_committed_version equals 0 on a
    PaxosService

  In a nutshell, the services do not set the first committed version, as
  they consider it as a SEP (Somebody Else's Problem). They do rely on it
  though, and we, the PaxosService, must ensure that it contains a valid
  value (that is, higher than zero) at all times.

  Since we will only have a first_committed version equal to zero once,
  and that is before the service's first proposal, we are safe to simply
  read the variable from the store and assign the first_committed the same
  value as the last_committed iff the first_committed version is zero.

  This also affects trimming, since trimming relies on the first_committed
  version as the lower bound for version trimming. Even though the k/v store
  will gracefully ignore any problem from trying to remove non-existent
  versions, the main issue would still stand: we'd be removing a non-existent
  version and that just doesn't make any sense.

[5] 'lock' paxos when we are running some internal proposals

  Force the paxos services to wait for us to complete whatever we are
  doing before they can proceed.  This is required because on certain
  occasions we might need to run internal proposals, not affected to any of
  the paxos services (for instance, when learning an old value), and we need
  them to stay put, or they might incur in erroneous state and crash the
  monitor.

  This could have been done with an extra bool, but there was no point
  in creating a new variable when we can just as easily reuse the
  'state' variable for our twisted interests.

Fixes: #4175
Signed-off-by: Joao Eduardo Luis <joao.luis@inktank.com>
21 files changed:
src/ceph_mon.cc
src/messages/MMonPaxos.h
src/mon/AuthMonitor.cc
src/mon/AuthMonitor.h
src/mon/Elector.cc
src/mon/LogMonitor.cc
src/mon/LogMonitor.h
src/mon/MDSMonitor.cc
src/mon/MDSMonitor.h
src/mon/Monitor.cc
src/mon/Monitor.h
src/mon/MonmapMonitor.cc
src/mon/MonmapMonitor.h
src/mon/OSDMonitor.cc
src/mon/OSDMonitor.h
src/mon/PGMonitor.cc
src/mon/PGMonitor.h
src/mon/Paxos.cc
src/mon/Paxos.h
src/mon/PaxosService.cc
src/mon/PaxosService.h

index 3e3edf9b90af932fe650f9e2b42e6c402a3b910f..9ca75f27039d3615c8aa5c1bec20857f47cc45e4 100644 (file)
@@ -25,7 +25,7 @@ using namespace std;
 
 #include "mon/MonMap.h"
 #include "mon/Monitor.h"
-#include "mon/MonitorStore.h"
+#include "mon/MonitorDBStore.h"
 #include "mon/MonClient.h"
 
 #include "msg/Messenger.h"
@@ -202,7 +202,7 @@ int main(int argc, const char **argv)
     }
 
     // go
-    MonitorStore store(g_conf->mon_data);
+    MonitorDBStore store(g_conf->mon_data);
     Monitor mon(g_ceph_context, g_conf->name.get_id(), &store, 0, &monmap);
     int r = mon.mkfs(osdmapbl);
     if (r < 0) {
@@ -214,16 +214,11 @@ int main(int argc, const char **argv)
     return 0;
   }
 
-  MonitorStore store(g_conf->mon_data);
-  err = store.mount();
-  if (err < 0) {
-    cerr << "problem opening monitor store in " << g_conf->mon_data << ": " << cpp_strerror(err) << std::endl;
-    exit(1);
-  }
+  MonitorDBStore store(g_conf->mon_data);
 
   bufferlist magicbl;
-  err = store.get_bl_ss(magicbl, "magic", 0);
-  if (err < 0) {
+  err = store.get(Monitor::MONITOR_NAME, "magic", magicbl);
+  if (!magicbl.length()) {
     cerr << "unable to read magic from mon data.. did you run mkcephfs?" << std::endl;
     exit(1);
   }
@@ -251,7 +246,7 @@ int main(int argc, const char **argv)
     }
 
     // get next version
-    version_t v = store.get_int("monmap", "last_committed");
+    version_t v = store.get("monmap", "last_committed");
     cout << "last committed monmap epoch is " << v << ", injected map will be " << (v+1) << std::endl;
     v++;
 
@@ -268,10 +263,12 @@ int main(int argc, const char **argv)
     ::encode(v, final);
     ::encode(mapbl, final);
 
+    MonitorDBStore::Transaction t;
     // save it
-    store.put_bl_sn(mapbl, "monmap", v);
-    store.put_bl_ss(final, "monmap", "latest");
-    store.put_int(v, "monmap", "last_committed");
+    t.put("monmap", v, mapbl);
+    t.put("monmap", "latest", final);
+    t.put("monmap", "last_committed", v);
+    store.apply_transaction(t);
 
     cout << "done." << std::endl;
     exit(0);
@@ -283,14 +280,14 @@ int main(int argc, const char **argv)
   {
     bufferlist mapbl;
     bufferlist latest;
-    store.get_bl_ss_safe(latest, "monmap", "latest");
+    store.get("monmap", "latest", latest);
     if (latest.length() > 0) {
       bufferlist::iterator p = latest.begin();
       version_t v;
       ::decode(v, p);
       ::decode(mapbl, p);
     } else {
-      store.get_bl_ss_safe(mapbl, "mkfs", "monmap");
+      store.get("mkfs", "monmap", mapbl);
       if (mapbl.length() == 0) {
        cerr << "mon fs missing 'monmap/latest' and 'mkfs/monmap'" << std::endl;
        exit(1);
@@ -403,7 +400,8 @@ int main(int argc, const char **argv)
     return 1;
 
   // start monitor
-  mon = new Monitor(g_ceph_context, g_conf->name.get_id(), &store, messenger, &monmap);
+  mon = new Monitor(g_ceph_context, g_conf->name.get_id(), &store, 
+                   messenger, &monmap);
 
   err = mon->preinit();
   if (err < 0)
@@ -431,7 +429,6 @@ int main(int argc, const char **argv)
 
   shutdown_async_signal_handler();
 
-  store.umount();
   delete mon;
   delete messenger;
   delete client_throttler;
index af1c9c6be85135fcbaed2a952ceecb4aeb4fb884..206586ccd215667cc8beaaa119f1f31c5e63006a 100644 (file)
@@ -49,7 +49,6 @@ class MMonPaxos : public Message {
 
   epoch_t epoch;   // monitor epoch
   __s32 op;          // paxos op
-  __s32 machine_id;  // which state machine?
 
   version_t first_committed;  // i've committed to
   version_t last_committed;  // i've committed to
@@ -65,10 +64,10 @@ class MMonPaxos : public Message {
   map<version_t,bufferlist> values;
 
   MMonPaxos() : Message(MSG_MON_PAXOS, HEAD_VERSION, COMPAT_VERSION) { }
-  MMonPaxos(epoch_t e, int o, int mid, utime_t now) : 
+  MMonPaxos(epoch_t e, int o, utime_t now) : 
     Message(MSG_MON_PAXOS, HEAD_VERSION, COMPAT_VERSION),
     epoch(e),
-    op(o), machine_id(mid),
+    op(o),
     first_committed(0), last_committed(0), pn_from(0), pn(0), uncommitted_pn(0),
     sent_timestamp(now),
     latest_version(0) {
@@ -81,8 +80,7 @@ public:
   const char *get_type_name() const { return "paxos"; }
   
   void print(ostream& out) const {
-    out << "paxos(" << get_paxos_name(machine_id)
-       << " " << get_opname(op) 
+    out << "paxos(" << get_opname(op) 
        << " lc " << last_committed
        << " fc " << first_committed
        << " pn " << pn << " opn " << uncommitted_pn;
@@ -96,7 +94,6 @@ public:
       header.version = 0;
     ::encode(epoch, payload);
     ::encode(op, payload);
-    ::encode(machine_id, payload);
     ::encode(first_committed, payload);
     ::encode(last_committed, payload);
     ::encode(pn_from, payload);
@@ -113,7 +110,6 @@ public:
     bufferlist::iterator p = payload.begin();
     ::decode(epoch, p);
     ::decode(op, p);
-    ::decode(machine_id, p);
     ::decode(first_committed, p);
     ::decode(last_committed, p);
     ::decode(pn_from, p);   
index 49e69b0fcde3f79822bcc479b06ff04461898062..7e93dff88d1c796b2dd6bf2861cfff62e3f21de4 100644 (file)
@@ -14,9 +14,9 @@
 
 #include <sstream>
 
-#include "AuthMonitor.h"
-#include "Monitor.h"
-#include "MonitorStore.h"
+#include "mon/AuthMonitor.h"
+#include "mon/Monitor.h"
+#include "mon/MonitorDBStore.h"
 
 #include "messages/MMonCommand.h"
 #include "messages/MAuth.h"
 
 #include "common/config.h"
 #include "include/assert.h"
+#include <sstream>
 
 #define dout_subsys ceph_subsys_mon
 #undef dout_prefix
-#define dout_prefix _prefix(_dout, mon, paxos->get_version())
+#define dout_prefix _prefix(_dout, mon, get_version())
 static ostream& _prefix(std::ostream *_dout, Monitor *mon, version_t v) {
   return *_dout << "mon." << mon->name << "@" << mon->rank
                << "(" << mon->get_state_name()
@@ -67,8 +68,7 @@ void AuthMonitor::check_rotate()
 
 void AuthMonitor::tick() 
 {
-  if (!paxos->is_active())
-    return;
+  if (!is_active()) return;
 
   update_from_paxos();
   dout(10) << *this << dendl;
@@ -96,7 +96,8 @@ void AuthMonitor::create_initial()
 
   KeyRing keyring;
   bufferlist bl;
-  mon->store->get_bl_ss_safe(bl, "mkfs", "keyring");
+  int ret = mon->store->get("mkfs", "keyring", bl);
+  assert(ret == 0);
   bufferlist::iterator p = bl.begin();
   ::decode(keyring, p);
 
@@ -112,30 +113,41 @@ void AuthMonitor::create_initial()
 
 void AuthMonitor::update_from_paxos()
 {
-  dout(10) << "update_from_paxos()" << dendl;
-  version_t paxosv = paxos->get_version();
+  dout(10) << __func__ << dendl;
+  version_t version = get_version();
   version_t keys_ver = mon->key_server.get_ver();
-  if (paxosv == keys_ver)
+  if (version == keys_ver)
     return;
-  assert(paxosv >= keys_ver);
+  assert(version >= keys_ver);
+
+  version_t latest_full = get_version_latest_full();
 
-  if (keys_ver != paxos->get_stashed_version()) {
-    bufferlist latest;
-    keys_ver = paxos->get_stashed(latest);
-    dout(7) << "update_from_paxos loading summary e" << keys_ver << dendl;
-    bufferlist::iterator p = latest.begin();
+  dout(10) << __func__ << " version " << version << " keys ver " << keys_ver
+           << " latest " << latest_full << dendl;
+
+  if ((latest_full > 0) && (latest_full > keys_ver)) {
+    bufferlist latest_bl;
+    int err = get_version_full(latest_full, latest_bl);
+    assert(err == 0);
+    assert(latest_bl.length() != 0);
+    dout(7) << __func__ << " loading summary e " << latest_full << dendl;
+    dout(7) << __func__ << " latest length " << latest_bl.length() << dendl;
+    bufferlist::iterator p = latest_bl.begin();
     __u8 struct_v;
     ::decode(struct_v, p);
     ::decode(max_global_id, p);
     ::decode(mon->key_server, p);
-    mon->key_server.set_ver(keys_ver);
-  } 
+    mon->key_server.set_ver(latest_full);
+    keys_ver = latest_full;
+  }
+
+  dout(10) << __func__ << " key server version " << mon->key_server.get_ver() << dendl;
 
   // walk through incrementals
-  while (paxosv > keys_ver) {
+  while (version > keys_ver) {
     bufferlist bl;
-    bool success = paxos->read(keys_ver+1, bl);
-    assert(success);
+    int ret = get_version(keys_ver+1, bl);
+    assert(ret == 0);
 
     // reset if we are moving to initial state.  we will normally have
     // keys in here temporarily for bootstrapping that we need to
@@ -143,6 +155,9 @@ void AuthMonitor::update_from_paxos()
     if (keys_ver == 0) 
       mon->key_server.clear_secrets();
 
+    dout(20) << __func__ << " walking through version " << (keys_ver+1)
+             << " len " << bl.length() << dendl;
+
     bufferlist::iterator p = bl.begin();
     __u8 v;
     ::decode(v, p);
@@ -169,7 +184,9 @@ void AuthMonitor::update_from_paxos()
     mon->key_server.set_ver(keys_ver);
 
     if (keys_ver == 1) {
-      mon->store->erase_ss("mkfs", "keyring");
+      MonitorDBStore::Transaction t;
+      t.erase("mkfs", "keyring");
+      mon->store->apply_transaction(t);
     }
   }
 
@@ -178,19 +195,20 @@ void AuthMonitor::update_from_paxos()
 
   dout(10) << "update_from_paxos() last_allocated_id=" << last_allocated_id
           << " max_global_id=" << max_global_id << dendl;
-
+  /*
   bufferlist bl;
   __u8 v = 1;
   ::encode(v, bl);
   ::encode(max_global_id, bl);
   Mutex::Locker l(mon->key_server.get_lock());
   ::encode(mon->key_server, bl);
-  paxos->stash_latest(paxosv, bl);
-
+  paxos->stash_latest(version, bl);
+  */
   unsigned max = g_conf->paxos_max_join_drift * 2;
   if (mon->is_leader() &&
-      paxosv > max)
-    paxos->trim_to(paxosv - max);
+      version > max)
+    trim_to(version - max);
 }
 
 void AuthMonitor::increase_max_global_id()
@@ -213,18 +231,38 @@ bool AuthMonitor::should_propose(double& delay)
 void AuthMonitor::create_pending()
 {
   pending_auth.clear();
-  dout(10) << "create_pending v " << (paxos->get_version() + 1) << dendl;
+  dout(10) << "create_pending v " << (get_version() + 1) << dendl;
 }
 
-void AuthMonitor::encode_pending(bufferlist &bl)
+void AuthMonitor::encode_pending(MonitorDBStore::Transaction *t)
 {
-  dout(10) << "encode_pending v " << (paxos->get_version() + 1) << dendl;
+  dout(10) << __func__ << " v " << (get_version() + 1) << dendl;
+
+  bufferlist bl;
+
   __u8 v = 1;
   ::encode(v, bl);
-  for (vector<Incremental>::iterator p = pending_auth.begin();
-       p != pending_auth.end();
-       p++)
+  vector<Incremental>::iterator p;
+  for (p = pending_auth.begin(); p != pending_auth.end(); p++)
     p->encode(bl, mon->get_quorum_features());
+
+  version_t version = get_version() + 1;
+  put_version(t, version, bl);
+  put_last_committed(t, version);
+
+  bufferlist full_bl;
+  Mutex::Locker l(mon->key_server.get_lock());
+  if (mon->key_server.has_secrets()) {
+    dout(10) << __func__ << " key server has secrets!" << dendl;
+    v = 1;
+    ::encode(v, full_bl);
+    ::encode(max_global_id, full_bl);
+    ::encode(mon->key_server, full_bl);
+
+    put_version_full(t, version, full_bl);
+    put_version_latest_full(t, version);
+  } else
+    dout(10) << __func__ << " key server has no secrets; do not put them in tx" << dendl;
 }
 
 bool AuthMonitor::preprocess_query(PaxosServiceMessage *m)
@@ -393,7 +431,7 @@ bool AuthMonitor::prep_auth(MAuth *m, bool paxos_writable)
        MMonGlobalID *req = new MMonGlobalID();
        req->old_max_id = max_global_id;
        mon->messenger->send_message(req, mon->monmap->get_inst(leader));
-       paxos->wait_for_commit(new C_RetryMessage(this, m));
+       wait_for_finished_proposal(new C_RetryMessage(this, m));
        return true;
       }
 
@@ -419,7 +457,8 @@ bool AuthMonitor::prep_auth(MAuth *m, bool paxos_writable)
       ret = s->auth_handler->handle_request(indata, response_bl, s->global_id, caps_info, &auid);
     }
     if (ret == -EIO) {
-      paxos->wait_for_active(new C_RetryMessage(this, m));
+      wait_for_active(new C_RetryMessage(this,m));
+      //paxos->wait_for_active(new C_RetryMessage(this, m));
       goto done;
     }
     if (caps_info.caps.length()) {
@@ -468,7 +507,7 @@ bool AuthMonitor::preprocess_command(MMonCommand *m)
     if (!session ||
        (!session->caps.get_allow_all() &&
         !mon->_allowed_command(session, m->cmd))) {
-      mon->reply_command(m, -EACCES, "access denied", rdata, paxos->get_version());
+      mon->reply_command(m, -EACCES, "access denied", rdata, get_version());
       return true;
     }
 
@@ -552,7 +591,7 @@ bool AuthMonitor::preprocess_command(MMonCommand *m)
  done:
   string rs;
   getline(ss, rs, '\0');
-  mon->reply_command(m, r, rs, rdata, paxos->get_version());
+  mon->reply_command(m, r, rs, rdata, get_version());
   return true;
 }
 
@@ -587,7 +626,7 @@ bool AuthMonitor::prepare_command(MMonCommand *m)
   if (!session ||
       (!session->caps.get_allow_all() &&
        !mon->_allowed_command(session, m->cmd))) {
-    mon->reply_command(m, -EACCES, "access denied", rdata, paxos->get_version());
+    mon->reply_command(m, -EACCES, "access denied", rdata, get_version());
     return true;
   }
 
@@ -607,7 +646,8 @@ bool AuthMonitor::prepare_command(MMonCommand *m)
       import_keyring(keyring);
       ss << "imported keyring";
       getline(ss, rs);
-      paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+      wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
+      //paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, get_version()));
       return true;
     }
     else if (m->cmd[1] == "add" && m->cmd.size() >= 3) {
@@ -655,7 +695,8 @@ bool AuthMonitor::prepare_command(MMonCommand *m)
 
       ss << "added key for " << auth_inc.name;
       getline(ss, rs);
-      paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+      wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
+      //paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, get_version()));
       return true;
     }
     else if ((m->cmd[1] == "get-or-create-key" ||
@@ -705,7 +746,8 @@ bool AuthMonitor::prepare_command(MMonCommand *m)
          ::decode(auth_inc, q);
          if (auth_inc.op == KeyServerData::AUTH_INC_ADD &&
              auth_inc.name == entity) {
-           paxos->wait_for_commit(new C_RetryMessage(this, m));
+           wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
+           //paxos->wait_for_commit(new C_RetryMessage(this, m));
            return true;
          }
        }
@@ -730,7 +772,8 @@ bool AuthMonitor::prepare_command(MMonCommand *m)
       }
 
       getline(ss, rs);
-      paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, rdata, paxos->get_version()));
+      wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, rdata, get_version()));
+      //paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, get_version()));
       return true;
     }
     else if (m->cmd[1] == "caps" && m->cmd.size() >= 3) {
@@ -756,7 +799,8 @@ bool AuthMonitor::prepare_command(MMonCommand *m)
 
       ss << "updated caps for " << auth_inc.name;
       getline(ss, rs);
-      paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+      wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
+      //paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, get_version()));
       return true;     
     }
     else if (m->cmd[1] == "del" && m->cmd.size() >= 3) {
@@ -778,7 +822,8 @@ bool AuthMonitor::prepare_command(MMonCommand *m)
 
       ss << "updated";
       getline(ss, rs);
-      paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+      wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
+      //paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, get_version()));
       return true;
     }
     else {
@@ -790,7 +835,7 @@ bool AuthMonitor::prepare_command(MMonCommand *m)
 
 done:
   getline(ss, rs, '\0');
-  mon->reply_command(m, err, rs, rdata, paxos->get_version());
+  mon->reply_command(m, err, rs, rdata, get_version());
   return false;
 }
 
index 92e7937322310de19f2749124c79d0b5ef23fb35..f84e42bef985cf6216620e869294e4344bd85e8c 100644 (file)
@@ -22,8 +22,9 @@ using namespace std;
 #include "include/ceph_features.h"
 #include "include/types.h"
 #include "msg/Messenger.h"
-#include "PaxosService.h"
+#include "mon/PaxosService.h"
 #include "mon/Monitor.h"
+#include "mon/MonitorDBStore.h"
 
 class MMonCommand;
 class MAuth;
@@ -131,7 +132,8 @@ private:
   bool prepare_global_id(MMonGlobalID *m);
   void increase_max_global_id();
   uint64_t assign_global_id(MAuth *m, bool should_increase_max);
-  void encode_pending(bufferlist &bl);  // propose pending update to peers
+  // propose pending update to peers
+  void encode_pending(MonitorDBStore::Transaction *t);
 
   bool preprocess_query(PaxosServiceMessage *m);  // true if processed.
   bool prepare_update(PaxosServiceMessage *m);
@@ -143,7 +145,10 @@ private:
 
   void check_rotate();
  public:
-  AuthMonitor(Monitor *mn, Paxos *p) : PaxosService(mn, p), last_rotating_ver(0), max_global_id(0), last_allocated_id(0) {}
+  AuthMonitor(Monitor *mn, Paxos *p, const string& service_name)
+    : PaxosService(mn, p, service_name), last_rotating_ver(0),
+      max_global_id(0), last_allocated_id(0) {}
+
   void pre_auth(MAuth *m);
   
   void tick();  // check state, take actions
index 199eaeae5384a3af705a1b2319c5730780ab48a5..867ed8db02bb04da125c22814668fdc14d332c9b 100644 (file)
@@ -16,7 +16,7 @@
 #include "Monitor.h"
 
 #include "common/Timer.h"
-#include "MonitorStore.h"
+#include "MonitorDBStore.h"
 #include "MonmapMonitor.h"
 #include "messages/MMonElection.h"
 
@@ -35,7 +35,7 @@ static ostream& _prefix(std::ostream *_dout, Monitor *mon, epoch_t epoch) {
 
 void Elector::init()
 {
-  epoch = mon->store->get_int("election_epoch");
+  epoch = mon->store->get(Monitor::MONITOR_NAME, "election_epoch");
   if (!epoch)
     epoch = 1;
   dout(1) << "init, last seen epoch " << epoch << dendl;
@@ -52,7 +52,9 @@ void Elector::bump_epoch(epoch_t e)
   dout(10) << "bump_epoch " << epoch << " to " << e << dendl;
   assert(epoch <= e);
   epoch = e;
-  mon->store->put_int(epoch, "election_epoch");
+  MonitorDBStore::Transaction t;
+  t.put(Monitor::MONITOR_NAME, "election_epoch", epoch);
+  mon->store->apply_transaction(t);
 
   // clear up some state
   electing_me = false;
@@ -337,8 +339,11 @@ void Elector::dispatch(Message *m)
                << ", taking it"
                << dendl;
        mon->monmap->decode(em->monmap_bl);
-       mon->store->put_bl_sn(em->monmap_bl, "monmap", mon->monmap->epoch);
-       mon->monmon()->paxos->stash_latest(mon->monmap->epoch, em->monmap_bl);
+        MonitorDBStore::Transaction t;
+        t.put("monmap", mon->monmap->epoch, em->monmap_bl);
+        t.put("monmap", "last_committed", mon->monmap->epoch);
+        mon->store->apply_transaction(t);
+       //mon->monmon()->paxos->stash_latest(mon->monmap->epoch, em->monmap_bl);
        mon->bootstrap();
        m->put();
        delete peermap;
index 4bdc2cb97040046956f1f25544d3eff42714376f..9ff7f50949d48a3ef590486a46a8eecd9864f151 100644 (file)
@@ -17,7 +17,7 @@
 
 #include "LogMonitor.h"
 #include "Monitor.h"
-#include "MonitorStore.h"
+#include "MonitorDBStore.h"
 
 #include "messages/MMonCommand.h"
 #include "messages/MLog.h"
 #include "common/errno.h"
 #include "common/config.h"
 #include "include/assert.h"
+#include <sstream>
 
 #define dout_subsys ceph_subsys_mon
 #undef dout_prefix
-#define dout_prefix _prefix(_dout, mon, paxos->get_version())
+#define dout_prefix _prefix(_dout, mon, get_version())
 static ostream& _prefix(std::ostream *_dout, Monitor *mon, version_t v) {
   return *_dout << "mon." << mon->name << "@" << mon->rank
                << "(" << mon->get_state_name()
@@ -68,8 +69,7 @@ ostream& operator<<(ostream& out, LogMonitor& pm)
 
 void LogMonitor::tick() 
 {
-  if (!paxos->is_active())
-    return;
+  if (!is_active()) return;
 
   update_from_paxos();
   dout(10) << *this << dendl;
@@ -94,26 +94,34 @@ void LogMonitor::create_initial()
 
 void LogMonitor::update_from_paxos()
 {
-  version_t paxosv = paxos->get_version();
-  if (paxosv == summary.version)
+  dout(10) << __func__ << dendl;
+  version_t version = get_version();
+  dout(10) << __func__ << " version " << version
+           << " summary v " << summary.version << dendl;
+  if (version == summary.version)
     return;
-  assert(paxosv >= summary.version);
+  assert(version >= summary.version);
 
   bufferlist blog;
 
-  if (summary.version != paxos->get_stashed_version()) {
-    bufferlist latest;
-    version_t v = paxos->get_stashed(latest);
-    dout(7) << "update_from_paxos loading summary e" << v << dendl;
-    bufferlist::iterator p = latest.begin();
-    ::decode(summary, p);
-  } 
+  version_t latest_full = get_version_latest_full();
+  dout(10) << __func__ << " latest full " << latest_full << dendl;
+  if ((latest_full > 0) && (latest_full > summary.version)) {
+      bufferlist latest_bl;
+      get_version_full(latest_full, latest_bl);
+      assert(latest_bl.length() != 0);
+      dout(7) << __func__ << " loading summary e" << latest_full << dendl;
+      bufferlist::iterator p = latest_bl.begin();
+      ::decode(summary, p);
+      dout(7) << __func__ << " loaded summary e" << summary.version << dendl;
+  }
 
   // walk through incrementals
-  while (paxosv > summary.version) {
+  while (version > summary.version) {
     bufferlist bl;
-    bool success = paxos->read(summary.version+1, bl);
-    assert(success);
+    int err = get_version(summary.version+1, bl);
+    assert(err == 0);
+    assert(bl.length());
 
     bufferlist::iterator p = bl.begin();
     __u8 v;
@@ -140,9 +148,6 @@ void LogMonitor::update_from_paxos()
     summary.version++;
   }
 
-  bufferlist bl;
-  ::encode(summary, bl);
-  paxos->stash_latest(paxosv, bl);
 
   if (blog.length()) {
     int fd = ::open(g_conf->mon_cluster_log_file.c_str(), O_WRONLY|O_APPEND|O_CREAT, 0600);
@@ -161,28 +166,49 @@ void LogMonitor::update_from_paxos()
 
   // trim
   unsigned max = g_conf->mon_max_log_epochs;
-  if (mon->is_leader() && paxosv > max)
-    paxos->trim_to(paxosv - max);
+  if (mon->is_leader() && version > max)
+    trim_to(version - max);
 
   check_subs();
 }
 
+void LogMonitor::store_do_append(MonitorDBStore::Transaction *t,
+    const string& key, bufferlist& bl)
+{
+  bufferlist existing_bl;
+  int err = get_value(key, existing_bl);
+  assert(err == 0);
+
+  existing_bl.append(bl);
+  put_value(t, key, existing_bl);
+}
+
 void LogMonitor::create_pending()
 {
   pending_log.clear();
   pending_summary = summary;
-  dout(10) << "create_pending v " << (paxos->get_version() + 1) << dendl;
+  dout(10) << "create_pending v " << (get_version() + 1) << dendl;
 }
 
-void LogMonitor::encode_pending(bufferlist &bl)
+void LogMonitor::encode_pending(MonitorDBStore::Transaction *t)
 {
-  dout(10) << "encode_pending v " << (paxos->get_version() + 1) << dendl;
+  version_t version = get_version() + 1;
+  bufferlist bl;
+  dout(10) << __func__ << " v" << version << dendl;
   __u8 v = 1;
   ::encode(v, bl);
-  for (multimap<utime_t,LogEntry>::iterator p = pending_log.begin();
-       p != pending_log.end();
-       p++)
+  multimap<utime_t,LogEntry>::iterator p;
+  for (p = pending_log.begin(); p != pending_log.end(); p++)
     p->second.encode(bl);
+
+  bufferlist summary_bl;
+  ::encode(summary, summary_bl);
+
+  put_version(t, version, bl);
+  put_last_committed(t, version);
+
+  put_version_full(t, version, summary_bl);
+  put_version_latest_full(t, version);
 }
 
 bool LogMonitor::preprocess_query(PaxosServiceMessage *m)
@@ -303,7 +329,7 @@ bool LogMonitor::preprocess_command(MMonCommand *m)
   if (r != -1) {
     string rs;
     getline(ss, rs);
-    mon->reply_command(m, r, rs, rdata, paxos->get_version());
+    mon->reply_command(m, r, rs, rdata, get_version());
     return true;
   } else
     return false;
@@ -320,7 +346,7 @@ bool LogMonitor::prepare_command(MMonCommand *m)
   ss << "unrecognized command";
 
   getline(ss, rs);
-  mon->reply_command(m, err, rs, paxos->get_version());
+  mon->reply_command(m, err, rs, get_version());
   return false;
 }
 
@@ -439,17 +465,17 @@ void LogMonitor::_create_sub_incremental(MLog *mlog, int level, version_t sv)
   dout(10) << __func__ << " level " << level << " ver " << sv 
          << " cur summary ver " << summary.version << dendl; 
 
-  if (sv < paxos->get_first_committed()) {
+  if (sv < get_first_committed()) {
     dout(10) << __func__ << " skipped from " << sv
-            << " to first_committed " << paxos->get_first_committed() << dendl;
+            << " to first_committed " << get_first_committed() << dendl;
     LogEntry le;
     le.stamp = ceph_clock_now(NULL);
     le.type = CLOG_WARN;
     ostringstream ss;
-    ss << "skipped log messages from " << sv << " to " << paxos->get_first_committed();
+    ss << "skipped log messages from " << sv << " to " << get_first_committed();
     le.msg = ss.str();
     mlog->entries.push_back(le);
-    sv = paxos->get_first_committed();
+    sv = get_first_committed();
   }
 
   version_t summary_ver = summary.version;
index 49fecc6f2ee2ad5a6898c4ea21469bf75b5fe9ed..489b0d72bb804b85fdbf0c25312703df5453f5e8 100644 (file)
@@ -36,8 +36,8 @@ private:
   void create_initial();
   void update_from_paxos();
   void create_pending();  // prepare a new pending
-  void encode_pending(bufferlist &bl);  // propose pending update to peers
-
+  // propose pending update to peers
+  void encode_pending(MonitorDBStore::Transaction *t);
   bool preprocess_query(PaxosServiceMessage *m);  // true if processed.
   bool prepare_update(PaxosServiceMessage *m);
 
@@ -67,8 +67,12 @@ private:
   bool _create_sub_summary(MLog *mlog, int level);
   void _create_sub_incremental(MLog *mlog, int level, version_t sv);
 
+  void store_do_append(MonitorDBStore::Transaction *t,
+                      const string& key, bufferlist& bl);
+
  public:
-  LogMonitor(Monitor *mn, Paxos *p) : PaxosService(mn, p) { }
+  LogMonitor(Monitor *mn, Paxos *p, const string& service_name) 
+    : PaxosService(mn, p, service_name) { }
   
   void tick();  // check state, take actions
 
index 18209e6508fb34593bb1f7dfd02a16414855074a..f3027e3493a36f12b6c6c8b0dc8531fb5723ef07 100644 (file)
@@ -15,7 +15,7 @@
 
 #include "MDSMonitor.h"
 #include "Monitor.h"
-#include "MonitorStore.h"
+#include "MonitorDBStore.h"
 #include "OSDMonitor.h"
 
 #include "common/strtol.h"
@@ -36,6 +36,8 @@
 #include "common/config.h"
 #include "include/assert.h"
 
+#include "MonitorDBStore.h"
+
 #define dout_subsys ceph_subsys_mon
 #undef dout_prefix
 #define dout_prefix _prefix(_dout, mon, mdsmap)
@@ -83,24 +85,21 @@ void MDSMonitor::create_initial()
 
 void MDSMonitor::update_from_paxos()
 {
-  version_t paxosv = paxos->get_version();
-  if (paxosv == mdsmap.epoch)
+  version_t version = get_version();
+  if (version == mdsmap.epoch)
     return;
-  assert(paxosv >= mdsmap.epoch);
+  assert(version >= mdsmap.epoch);
 
-  dout(10) << "update_from_paxos paxosv " << paxosv 
+  dout(10) << __func__ << " version " << version
           << ", my e " << mdsmap.epoch << dendl;
 
   // read and decode
   mdsmap_bl.clear();
-  bool success = paxos->read(paxosv, mdsmap_bl);
-  assert(success);
-  dout(10) << "update_from_paxos  got " << paxosv << dendl;
+  get_version(version, mdsmap_bl);
+  assert(mdsmap_bl.length() > 0);
+  dout(10) << __func__ << " got " << version << dendl;
   mdsmap.decode(mdsmap_bl);
 
-  // save as 'latest', too.
-  paxos->stash_latest(paxosv, mdsmap_bl);
-
   // new map
   dout(4) << "new map" << dendl;
   print_map(mdsmap, 0);
@@ -116,7 +115,7 @@ void MDSMonitor::create_pending()
   dout(10) << "create_pending e" << pending_mdsmap.epoch << dendl;
 }
 
-void MDSMonitor::encode_pending(bufferlist &bl)
+void MDSMonitor::encode_pending(MonitorDBStore::Transaction *t)
 {
   dout(10) << "encode_pending e" << pending_mdsmap.epoch << dendl;
 
@@ -125,8 +124,13 @@ void MDSMonitor::encode_pending(bufferlist &bl)
   //print_map(pending_mdsmap);
 
   // apply to paxos
-  assert(paxos->get_version() + 1 == pending_mdsmap.epoch);
-  pending_mdsmap.encode(bl);
+  assert(get_version() + 1 == pending_mdsmap.epoch);
+  bufferlist mdsmap_bl;
+  pending_mdsmap.encode(mdsmap_bl);
+
+  /* put everything in the transaction */
+  put_version(t, pending_mdsmap.epoch, mdsmap_bl);
+  put_last_committed(t, pending_mdsmap.epoch);
 }
 
 void MDSMonitor::update_logger()
@@ -519,7 +523,7 @@ bool MDSMonitor::preprocess_command(MMonCommand *m)
       (!session->caps.get_allow_all() &&
        !session->caps.check_privileges(PAXOS_MDSMAP, MON_CAP_R) &&
        !mon->_allowed_command(session, m->cmd))) {
-    mon->reply_command(m, -EACCES, "access denied", rdata, paxos->get_version());
+    mon->reply_command(m, -EACCES, "access denied", rdata, get_version());
     return true;
   }
 
@@ -555,7 +559,7 @@ bool MDSMonitor::preprocess_command(MMonCommand *m)
       MDSMap *p = &mdsmap;
       if (epoch) {
        bufferlist b;
-       mon->store->get_bl_sn_safe(b, "mdsmap", epoch);
+       get_version(epoch, b);
        if (!b.length()) {
          p = 0;
          r = -ENOENT;
@@ -597,7 +601,7 @@ bool MDSMonitor::preprocess_command(MMonCommand *m)
        }
        epoch_t e = l;
        bufferlist b;
-       mon->store->get_bl_sn_safe(b,"mdsmap",e);
+       get_version(e, b);
        if (!b.length()) {
          r = -ENOENT;
        } else {
@@ -622,7 +626,7 @@ bool MDSMonitor::preprocess_command(MMonCommand *m)
        for (map<uint64_t, MDSMap::mds_info_t>::const_iterator i = mds_info.begin();
             i != mds_info.end();
             ++i) {
-         mon->send_command(i->second.get_inst(), m->cmd, paxos->get_version());
+         mon->send_command(i->second.get_inst(), m->cmd, get_version());
          r = 0;
        }
        if (r == -ENOENT) {
@@ -636,7 +640,7 @@ bool MDSMonitor::preprocess_command(MMonCommand *m)
        m->cmd.erase(m->cmd.begin()); //done with target num now
        if (!errno && who >= 0) {
          if (mdsmap.is_up(who)) {
-           mon->send_command(mdsmap.get_inst(who), m->cmd, paxos->get_version());
+           mon->send_command(mdsmap.get_inst(who), m->cmd, get_version());
            r = 0;
            ss << "ok";
          } else {
@@ -666,7 +670,7 @@ bool MDSMonitor::preprocess_command(MMonCommand *m)
   if (r != -1) {
     string rs;
     getline(ss, rs);
-    mon->reply_command(m, r, rs, rdata, paxos->get_version());
+    mon->reply_command(m, r, rs, rdata, get_version());
     return true;
   } else
     return false;
@@ -712,7 +716,7 @@ int MDSMonitor::cluster_fail(std::ostream &ss)
     ss << "mdsmap must be marked DOWN first ('mds cluster_down')";
     return -EPERM;
   }
-  if (pending_mdsmap.up.size() && !mon->osdmon()->paxos->is_writeable()) {
+  if (pending_mdsmap.up.size() && !mon->osdmon()->is_writeable()) {
     ss << "osdmap not writeable, can't blacklist up mds's";
     return -EAGAIN;
   }
@@ -752,7 +756,7 @@ bool MDSMonitor::prepare_command(MMonCommand *m)
       (!session->caps.get_allow_all() &&
        !session->caps.check_privileges(PAXOS_MDSMAP, MON_CAP_W) &&
        !mon->_allowed_command(session, m->cmd))) {
-    mon->reply_command(m, -EACCES, "access denied", rdata, paxos->get_version());
+    mon->reply_command(m, -EACCES, "access denied", rdata, get_version());
     return true;
   }
 
@@ -802,7 +806,7 @@ bool MDSMonitor::prepare_command(MMonCommand *m)
        map.epoch = pending_mdsmap.epoch;  // make sure epoch is correct
        pending_mdsmap = map;
        string rs = "set mds map";
-       paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+       paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, get_version()));
        return true;
       } else
        ss << "next mdsmap epoch " << pending_mdsmap.epoch << " != " << e;
@@ -824,7 +828,7 @@ bool MDSMonitor::prepare_command(MMonCommand *m)
        ss << "set mds gid " << gid << " to state " << state << " " << ceph_mds_state_name(state);
        string rs;
        getline(ss, rs);
-       paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+       paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, get_version()));
        return true;
       }
     }
@@ -849,7 +853,7 @@ bool MDSMonitor::prepare_command(MMonCommand *m)
        ss << "removed mds gid " << gid;
        string rs;
        getline(ss, rs);
-       paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+       paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, get_version()));
        return true;
       }
     }
@@ -862,7 +866,7 @@ bool MDSMonitor::prepare_command(MMonCommand *m)
       ss << "removed failed mds." << w;
       string rs;
       getline(ss, rs);
-      paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+      paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, get_version()));
       return true;
     }
     else if (m->cmd[1] == "cluster_fail") {
@@ -943,7 +947,7 @@ bool MDSMonitor::prepare_command(MMonCommand *m)
        ss << "new fs with metadata pool " << metadata << " and data pool " << data;
        string rs;
        getline(ss, rs);
-       paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+       paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, get_version()));
        return true;
       }
     }    
@@ -956,11 +960,11 @@ bool MDSMonitor::prepare_command(MMonCommand *m)
 
   if (r >= 0) {
     // success.. delay reply
-    paxos->wait_for_commit(new Monitor::C_Command(mon, m, r, rs, paxos->get_version()));
+    paxos->wait_for_commit(new Monitor::C_Command(mon, m, r, rs, get_version()));
     return true;
   } else {
     // reply immediately
-    mon->reply_command(m, r, rs, rdata, paxos->get_version());
+    mon->reply_command(m, r, rs, rdata, get_version());
     return false;
   }
 }
@@ -995,8 +999,7 @@ void MDSMonitor::tick()
 {
   // make sure mds's are still alive
   // ...if i am an active leader
-  if (!paxos->is_active())
-    return;
+  if (!is_active()) return;
 
   update_from_paxos();
   dout(10) << mdsmap << dendl;
@@ -1050,7 +1053,7 @@ void MDSMonitor::tick()
     }
   }
 
-  if (mon->osdmon()->paxos->is_writeable()) {
+  if (mon->osdmon()->is_writeable()) {
 
     bool propose_osdmap = false;
 
@@ -1255,7 +1258,7 @@ void MDSMonitor::do_stop()
 {
   // hrm...
   if (!mon->is_leader() ||
-      !paxos->is_active()) {
+      !is_active()) {
     dout(0) << "do_stop can't stop right now, mdsmap not writeable" << dendl;
     return;
   }
index d852785fa909ff905a6d30a6ebf0ff51874e125f..e8ccebea1fee8916a27830386bb7cc7df199c8f1 100644 (file)
@@ -74,7 +74,7 @@ class MDSMonitor : public PaxosService {
   void create_initial();
   void update_from_paxos();
   void create_pending(); 
-  void encode_pending(bufferlist &bl);
+  void encode_pending(MonitorDBStore::Transaction *t);
 
   void update_logger();
 
@@ -111,8 +111,8 @@ class MDSMonitor : public PaxosService {
   bool try_standby_replay(MDSMap::mds_info_t& finfo, MDSMap::mds_info_t& ainfo);
 
 public:
-  MDSMonitor(Monitor *mn, Paxos *p)
-    : PaxosService(mn, p)
+  MDSMonitor(Monitor *mn, Paxos *p, string service_name)
+    : PaxosService(mn, p, service_name)
   {
   }
 
index cde3eb44d96d493d62f5f78096996f8f69ea4c15..aba6e53c2edf73e05ed833f13468dd428a44cf8d 100644 (file)
@@ -23,7 +23,7 @@
 
 #include "osd/OSDMap.h"
 
-#include "MonitorStore.h"
+#include "MonitorDBStore.h"
 
 #include "msg/Messenger.h"
 
@@ -81,6 +81,8 @@ static ostream& _prefix(std::ostream *_dout, const Monitor *mon) {
                << "(" << mon->get_state_name() << ") e" << mon->monmap->get_epoch() << " ";
 }
 
+const string Monitor::MONITOR_NAME = "monitor";
+
 long parse_pos_long(const char *s, ostream *pss)
 {
   if (*s == '-' || *s == '+') {
@@ -103,8 +105,8 @@ long parse_pos_long(const char *s, ostream *pss)
   return r;
 }
 
-
-Monitor::Monitor(CephContext* cct_, string nm, MonitorStore *s, Messenger *m, MonMap *map) :
+Monitor::Monitor(CephContext* cct_, string nm, MonitorDBStore *s,
+                Messenger *m, MonMap *map) :
   Dispatcher(cct_),
   name(nm),
   rank(-1), 
@@ -141,13 +143,14 @@ Monitor::Monitor(CephContext* cct_, string nm, MonitorStore *s, Messenger *m, Mo
 {
   rank = -1;
 
-  paxos_service[PAXOS_PGMAP] = new PGMonitor(this, add_paxos(PAXOS_PGMAP));
-  paxos_service[PAXOS_OSDMAP] = new OSDMonitor(this, add_paxos(PAXOS_OSDMAP));
-  // mdsmap should be added to the paxos vector after the osdmap
-  paxos_service[PAXOS_MDSMAP] = new MDSMonitor(this, add_paxos(PAXOS_MDSMAP));
-  paxos_service[PAXOS_LOG] = new LogMonitor(this, add_paxos(PAXOS_LOG));
-  paxos_service[PAXOS_MONMAP] = new MonmapMonitor(this, add_paxos(PAXOS_MONMAP));
-  paxos_service[PAXOS_AUTH] = new AuthMonitor(this, add_paxos(PAXOS_AUTH));
+  paxos = new Paxos(this, "paxos");
+
+  paxos_service[PAXOS_MDSMAP] = new MDSMonitor(this, paxos, "mdsmap");
+  paxos_service[PAXOS_MONMAP] = new MonmapMonitor(this, paxos, "monmap");
+  paxos_service[PAXOS_OSDMAP] = new OSDMonitor(this, paxos, "osdmap");
+  paxos_service[PAXOS_PGMAP] = new PGMonitor(this, paxos, "pgmap");
+  paxos_service[PAXOS_LOG] = new LogMonitor(this, paxos, "logm");
+  paxos_service[PAXOS_AUTH] = new AuthMonitor(this, paxos, "auth");
 
   mon_caps = new MonCaps();
   mon_caps->set_allow_all(true);
@@ -156,24 +159,6 @@ Monitor::Monitor(CephContext* cct_, string nm, MonitorStore *s, Messenger *m, Mo
   exited_quorum = ceph_clock_now(g_ceph_context);
 }
 
-Paxos *Monitor::add_paxos(int type)
-{
-  Paxos *p = new Paxos(this, type);
-  paxos.push_back(p);
-  return p;
-}
-
-Paxos *Monitor::get_paxos_by_name(const string& name)
-{
-  for (list<Paxos*>::iterator p = paxos.begin();
-       p != paxos.end();
-       ++p) {
-    if ((*p)->machine_name == name)
-      return *p;
-  }
-  return NULL;
-}
-
 PaxosService *Monitor::get_paxos_service_by_name(const string& name)
 {
   if (name == "mdsmap")
@@ -197,8 +182,7 @@ Monitor::~Monitor()
 {
   for (vector<PaxosService*>::iterator p = paxos_service.begin(); p != paxos_service.end(); p++)
     delete *p;
-  for (list<Paxos*>::iterator p = paxos.begin(); p != paxos.end(); p++)
-    delete *p;
+  delete paxos;
   assert(session_map.sessions.empty());
   delete mon_caps;
 }
@@ -263,13 +247,13 @@ CompatSet Monitor::get_legacy_features()
                   ceph_mon_feature_incompat);
 }
 
-int Monitor::check_features(MonitorStore *store)
+int Monitor::check_features(MonitorDBStore *store)
 {
   CompatSet required = get_supported_features();
   CompatSet ondisk;
 
   bufferlist features;
-  store->get_bl_ss_safe(features, COMPAT_SET_LOC, 0);
+  store->get(MONITOR_NAME, COMPAT_SET_LOC, features);
   if (features.length() == 0) {
     generic_dout(0) << "WARNING: mon fs missing feature list.\n"
            << "Assuming it is old-style and introducing one." << dendl;
@@ -280,7 +264,9 @@ int Monitor::check_features(MonitorStore *store)
 
     bufferlist bl;
     ondisk.encode(bl);
-    store->put_bl_ss(bl, COMPAT_SET_LOC, 0);
+    MonitorDBStore::Transaction t;
+    t.put(MONITOR_NAME, COMPAT_SET_LOC, bl);
+    store->apply_transaction(t);
   } else {
     bufferlist::iterator it = features.begin();
     ondisk.decode(it);
@@ -298,7 +284,7 @@ int Monitor::check_features(MonitorStore *store)
 void Monitor::read_features()
 {
   bufferlist bl;
-  store->get_bl_ss_safe(bl, COMPAT_SET_LOC, 0);
+  store->get(MONITOR_NAME, COMPAT_SET_LOC, bl);
   assert(bl.length());
 
   bufferlist::iterator p = bl.begin();
@@ -306,11 +292,11 @@ void Monitor::read_features()
   dout(10) << "features " << features << dendl;
 }
 
-void Monitor::write_features()
+void Monitor::write_features(MonitorDBStore::Transaction &t)
 {
   bufferlist bl;
   features.encode(bl);
-  store->put_bl_ss(bl, COMPAT_SET_LOC, 0);
+  t.put(MONITOR_NAME, COMPAT_SET_LOC, bl);
 }
 
 int Monitor::preinit()
@@ -370,7 +356,7 @@ int Monitor::preinit()
   read_features();
 
   // have we ever joined a quorum?
-  has_ever_joined = store->exists_bl_ss("joined");
+  has_ever_joined = (store->get(MONITOR_NAME, "joined") != 0);
   dout(10) << "has_ever_joined = " << (int)has_ever_joined << dendl;
 
   if (!has_ever_joined) {
@@ -388,21 +374,20 @@ int Monitor::preinit()
     }
   }
 
+  paxos->init();
   // init paxos
-  for (list<Paxos*>::iterator it = paxos.begin(); it != paxos.end(); ++it) {
-    (*it)->init();
-    if ((*it)->is_consistent()) {
-      int i = (*it)->machine_id;
+  for (int i = 0; i < PAXOS_NUM; ++i) {
+    if (paxos->is_consistent()) {
       paxos_service[i]->update_from_paxos();
     } // else we don't do anything; handle_probe_reply will detect it's slurping
   }
 
   // we need to bootstrap authentication keys so we can form an
   // initial quorum.
-  if (authmon()->paxos->get_version() == 0) {
+  if (authmon()->get_version() == 0) {
     dout(10) << "loading initial keyring to bootstrap authentication for mkfs" << dendl;
     bufferlist bl;
-    store->get_bl_ss_safe(bl, "mkfs", "keyring");
+    store->get("mkfs", "keyring", bl);
     KeyRing keyring;
     bufferlist::iterator p = bl.begin();
     ::decode(keyring, p);
@@ -639,8 +624,8 @@ void Monitor::reset()
   quorum.clear();
   outside_quorum.clear();
 
-  for (list<Paxos*>::iterator p = paxos.begin(); p != paxos.end(); p++)
-    (*p)->restart();
+  paxos->restart();
+
   for (vector<PaxosService*>::iterator p = paxos_service.begin(); p != paxos_service.end(); p++)
     (*p)->restart();
 }
@@ -710,6 +695,9 @@ void Monitor::handle_probe(MMonProbe *m)
   }
 }
 
+/**
+ * @todo fix this. This is going to cause trouble.
+ */
 void Monitor::handle_probe_probe(MMonProbe *m)
 {
   dout(10) << "handle_probe_probe " << m->get_source_inst() << *m << dendl;
@@ -717,8 +705,7 @@ void Monitor::handle_probe_probe(MMonProbe *m)
   r->name = name;
   r->quorum = quorum;
   monmap->encode(r->monmap_bl, m->get_connection()->get_features());
-  for (list<Paxos*>::iterator p = paxos.begin(); p != paxos.end(); ++p)
-    r->paxos_versions[(*p)->get_machine_name()] = (*p)->get_version();
+  r->paxos_versions[paxos->get_name()] = paxos->get_version();
   messenger->send_message(r, m->get_connection());
 
   // did we discover a peer here?
@@ -726,7 +713,6 @@ void Monitor::handle_probe_probe(MMonProbe *m)
     dout(1) << " adding peer " << m->get_source_addr() << " to list of hints" << dendl;
     extra_probe_peers.insert(m->get_source_addr());
   }
-
   m->put();
 }
 
@@ -794,25 +780,24 @@ void Monitor::handle_probe_reply(MMonProbe *m)
     for (map<string,version_t>::iterator p = m->paxos_versions.begin();
         p != m->paxos_versions.end();
         ++p) {
-      Paxos *pax = get_paxos_by_name(p->first);
-      if (!pax) {
+      if (!paxos) {
        dout(0) << " peer has paxos machine " << p->first << " but i don't... weird" << dendl;
        continue;  // weird!
       }
-      if (pax->is_slurping()) {
+      if (paxos->is_slurping()) {
         dout(10) << " My paxos machine " << p->first
                  << " is currently slurping, so that will continue. Peer has v "
                  << p->second << dendl;
         ok = false;
-      } else if (pax->get_version() + g_conf->paxos_max_join_drift < p->second) {
+      } else if (paxos->get_version() + g_conf->paxos_max_join_drift < p->second) {
        dout(10) << " peer paxos machine " << p->first << " v " << p->second
-                << " vs my v " << pax->get_version()
+                << " vs my v " << paxos->get_version()
                 << " (too far ahead)"
                 << dendl;
        ok = false;
       } else {
        dout(10) << " peer paxos machine " << p->first << " v " << p->second
-                << " vs my v " << pax->get_version()
+                << " vs my v " << paxos->get_version()
                 << " (ok)"
                 << dendl;
       }
@@ -853,7 +838,6 @@ void Monitor::handle_probe_reply(MMonProbe *m)
       dout(10) << " that's not yet enough for a new quorum, waiting" << dendl;
     }
   }
-
   m->put();
 }
 
@@ -874,6 +858,7 @@ void Monitor::slurp()
 {
   dout(10) << "slurp " << slurp_source << " " << slurp_versions << dendl;
 
+  /*
   reset_probe_timeout();
 
   state = STATE_SLURPING;
@@ -925,10 +910,13 @@ void Monitor::slurp()
 
   dout(10) << "done slurping" << dendl;
   bootstrap();
+  */
 }
 
 MMonProbe *Monitor::fill_probe_data(MMonProbe *m, Paxos *pax)
 {
+  dout(10) << __func__ << *m << dendl;
+  /*
   MMonProbe *r = new MMonProbe(monmap->fsid, MMonProbe::OP_DATA, name, has_ever_joined);
   r->machine_name = m->machine_name;
   r->oldest_version = pax->get_first_committed();
@@ -937,37 +925,40 @@ MMonProbe *Monitor::fill_probe_data(MMonProbe *m, Paxos *pax)
   version_t v = MAX(pax->get_first_committed(), m->newest_version + 1);
   int len = 0;
   for (; v <= pax->get_version(); v++) {
-    store->get_bl_sn_safe(r->paxos_values[m->machine_name][v], m->machine_name.c_str(), v);
-    len += r->paxos_values[m->machine_name][v].length();
+    len += store->get(m->machine_name.c_str(), v, 
+                      r->paxos_values[m->machine_name][v]);
+
     for (list<string>::iterator p = pax->extra_state_dirs.begin();
          p != pax->extra_state_dirs.end();
          ++p) {
-      store->get_bl_sn_safe(r->paxos_values[*p][v], p->c_str(), v);
-      len += r->paxos_values[*p][v].length();
+      len += store->get(p->c_str(), v, r->paxos_values[*p][v]);
     }
     if (len >= g_conf->mon_slurp_bytes)
       break;
   }
 
   return r;
+  */
+  return NULL;
 }
 
 void Monitor::handle_probe_slurp(MMonProbe *m)
 {
   dout(10) << "handle_probe_slurp " << *m << dendl;
-
+  /*
   Paxos *pax = get_paxos_by_name(m->machine_name);
   assert(pax);
 
   MMonProbe *r = fill_probe_data(m, pax);
   messenger->send_message(r, m->get_connection());
+  */
   m->put();
 }
 
 void Monitor::handle_probe_slurp_latest(MMonProbe *m)
 {
   dout(10) << "handle_probe_slurp_latest " << *m << dendl;
-
+  /*
   Paxos *pax = get_paxos_by_name(m->machine_name);
   assert(pax);
 
@@ -975,13 +966,14 @@ void Monitor::handle_probe_slurp_latest(MMonProbe *m)
   r->latest_version = pax->get_stashed(r->latest_value);
 
   messenger->send_message(r, m->get_connection());
+  */
   m->put();
 }
 
 void Monitor::handle_probe_data(MMonProbe *m)
 {
   dout(10) << "handle_probe_data " << *m << dendl;
-
+  /*
   Paxos *pax = get_paxos_by_name(m->machine_name);
   assert(pax);
 
@@ -995,15 +987,20 @@ void Monitor::handle_probe_data(MMonProbe *m)
 
   // store any new stuff
   if (m->paxos_values.size()) {
-    for (map<string, map<version_t, bufferlist> >::iterator p = m->paxos_values.begin();
-        p != m->paxos_values.end();
-        ++p) {
-      store->put_bl_sn_map(p->first.c_str(), p->second.begin(), p->second.end());
+    map<string, map<version_t,bufferlist> >::iterator p;
+   
+    // bundle everything on a single transaction
+    MonitorDBStore::Transaction t;
+    
+    for (p = m->paxos_values.begin(); p != m->paxos_values.end(); ++p) {
+      store->put(&t, p->first.c_str(), p->second.begin(), p->second.end());
     }
 
     pax->last_committed = m->paxos_values.begin()->second.rbegin()->first;
-    store->put_int(pax->last_committed, m->machine_name.c_str(),
-                  "last_committed");
+    store->put(&t, m->machine_name.c_str(), 
+               "last_committed", pax->last_committed);
+
+    store->apply_transaction(t);
   }
 
   // latest?
@@ -1014,6 +1011,7 @@ void Monitor::handle_probe_data(MMonProbe *m)
   m->put();
 
   slurp();
+  */
 }
 
 void Monitor::start_election()
@@ -1072,8 +1070,7 @@ void Monitor::win_election(epoch_t epoch, set<int>& active, uint64_t features)
   clog.info() << "mon." << name << "@" << rank
                << " won leader election with quorum " << quorum << "\n";
 
-  for (list<Paxos*>::iterator p = paxos.begin(); p != paxos.end(); p++)
-    (*p)->leader_init();
+  paxos->leader_init();
   for (vector<PaxosService*>::iterator p = paxos_service.begin(); p != paxos_service.end(); p++)
     (*p)->election_finished();
 
@@ -1093,8 +1090,7 @@ void Monitor::lose_election(epoch_t epoch, set<int> &q, int l, uint64_t features
   dout(10) << "lose_election, epoch " << epoch << " leader is mon" << leader
           << " quorum is " << quorum << " features are " << quorum_features << dendl;
 
-  for (list<Paxos*>::iterator p = paxos.begin(); p != paxos.end(); p++)
-    (*p)->peon_init();
+  paxos->peon_init();
   for (vector<PaxosService*>::iterator p = paxos_service.begin(); p != paxos_service.end(); p++)
     (*p)->election_finished();
 
@@ -2084,14 +2080,14 @@ bool Monitor::_ms_dispatch(Message *m)
          break;
        }
 
-       // send it to the right paxos instance
-       assert(pm->machine_id < PAXOS_NUM);
-       Paxos *p = get_paxos_by_name(get_paxos_name(pm->machine_id));
-       p->dispatch((PaxosServiceMessage*)m);
+       paxos->dispatch((PaxosServiceMessage*)m);
 
        // make sure service finds out about any state changes
-       if (p->is_active())
-         paxos_service[p->machine_id]->update_from_paxos();
+       if (paxos->is_active()) {
+         vector<PaxosService*>::iterator service_it = paxos_service.begin();
+         for ( ; service_it != paxos_service.end(); ++service_it)
+           (*service_it)->update_from_paxos();
+       }
       }
       break;
 
@@ -2573,13 +2569,13 @@ void Monitor::handle_get_version(MMonGetVersion *m)
   reply->handle = m->handle;
   if (m->what == "mdsmap") {
     reply->version = mdsmon()->mdsmap.get_epoch();
-    reply->oldest_version = mdsmon()->paxos->get_first_committed();
+    reply->oldest_version = mdsmon()->get_first_committed();
   } else if (m->what == "osdmap") {
     reply->version = osdmon()->osdmap.get_epoch();
-    reply->oldest_version = osdmon()->paxos->get_first_committed();
+    reply->oldest_version = osdmon()->get_first_committed();
   } else if (m->what == "monmap") {
     reply->version = monmap->get_epoch();
-    reply->oldest_version = monmon()->paxos->get_first_committed();
+    reply->oldest_version = monmon()->get_first_committed();
   } else {
     derr << "invalid map type " << m->what << dendl;
   }
@@ -2730,9 +2726,12 @@ int Monitor::check_fsid()
   ss << monmap->get_fsid();
   string us = ss.str();
   bufferlist ebl;
-  int r = store->get_bl_ss(ebl, "cluster_uuid", 0);
-  if (r < 0)
-    return r;
+
+  if (!store->exists(MONITOR_NAME, "cluster_uuid"))
+    return -ENOENT;
+
+  int r = store->get(MONITOR_NAME, "cluster_uuid", ebl);
+  assert(r == 0);
 
   string es(ebl.c_str(), ebl.length());
 
@@ -2753,6 +2752,14 @@ int Monitor::check_fsid()
 }
 
 int Monitor::write_fsid()
+{
+  MonitorDBStore::Transaction t;
+  int r = write_fsid(t);
+  store->apply_transaction(t);
+  return r;
+}
+
+int Monitor::write_fsid(MonitorDBStore::Transaction &t)
 {
   ostringstream ss;
   ss << monmap->get_fsid() << "\n";
@@ -2760,7 +2767,8 @@ int Monitor::write_fsid()
 
   bufferlist b;
   b.append(us);
-  store->put_bl_ss(b, "cluster_uuid", 0);
+
+  t.put(MONITOR_NAME, "cluster_uuid", b);
   return 0;
 }
 
@@ -2770,12 +2778,7 @@ int Monitor::write_fsid()
  */
 int Monitor::mkfs(bufferlist& osdmapbl)
 {
-  // create it
-  int err = store->mkfs();
-  if (err) {
-    derr << "store->mkfs failed with: " << cpp_strerror(err) << dendl;
-    return err;
-  }
+  MonitorDBStore::Transaction t;
 
   // verify cluster fsid
   int r = check_fsid();
@@ -2785,17 +2788,17 @@ int Monitor::mkfs(bufferlist& osdmapbl)
   bufferlist magicbl;
   magicbl.append(CEPH_MON_ONDISK_MAGIC);
   magicbl.append("\n");
-  store->put_bl_ss(magicbl, "magic", 0);
+  t.put(MONITOR_NAME, "magic", magicbl);
 
 
   features = get_supported_features();
-  write_features();
+  write_features(t);
 
   // save monmap, osdmap, keyring.
   bufferlist monmapbl;
   monmap->encode(monmapbl, CEPH_FEATURES_ALL);
   monmap->set_epoch(0);     // must be 0 to avoid confusing first MonmapMonitor::update_from_paxos()
-  store->put_bl_ss(monmapbl, "mkfs", "monmap");
+  t.put("mkfs", "monmap", monmapbl);
 
   if (osdmapbl.length()) {
     // make sure it's a valid osdmap
@@ -2807,7 +2810,7 @@ int Monitor::mkfs(bufferlist& osdmapbl)
       derr << "error decoding provided osdmap: " << e.what() << dendl;
       return -EINVAL;
     }
-    store->put_bl_ss(osdmapbl, "mkfs", "osdmap");
+    t.put("mkfs", "osdmap", osdmapbl);
   }
 
   KeyRing keyring;
@@ -2828,13 +2831,9 @@ int Monitor::mkfs(bufferlist& osdmapbl)
 
   bufferlist keyringbl;
   keyring.encode_plaintext(keyringbl);
-  store->put_bl_ss(keyringbl, "mkfs", "keyring");
-
-  // sync and write out fsid to indicate completion.
-  store->sync();
-  r = write_fsid();
-  if (r < 0)
-    return r;
+  t.put("mkfs", "keyring", keyringbl);
+  write_fsid(t);
+  store->apply_transaction(t);
 
   return 0;
 }
index e98ef5f8695a9285658e7d55687254fa33668165..3d2e35b9ad17022c3acf6907a386e1bc5a7c3500 100644 (file)
 #include "perfglue/heap_profiler.h"
 
 #include "messages/MMonCommand.h"
+#include "mon/MonitorDBStore.h"
 
 #include <memory>
 #include <errno.h>
 
 
-#define CEPH_MON_PROTOCOL     9 /* cluster internal */
+#define CEPH_MON_PROTOCOL     10 /* cluster internal */
 
 
 enum {
@@ -79,8 +80,6 @@ enum {
   l_cluster_last,
 };
 
-class MonitorStore;
-
 class PaxosService;
 
 class PerfCounters;
@@ -136,7 +135,8 @@ private:
 
   // -- local storage --
 public:
-  MonitorStore *store;
+  MonitorDBStore *store;
+  static const string MONITOR_NAME;
 
   // -- monitor state --
 private:
@@ -175,6 +175,7 @@ public:
 
   // -- elector --
 private:
+  Paxos *paxos;
   Elector elector;
   friend class Elector;
   
@@ -298,12 +299,11 @@ public:
 
   void update_logger();
 
-  // -- paxos -- These vector indices are matched
-  list<Paxos*> paxos;
+  /**
+   * Vector holding the Services serviced by this Monitor.
+   */
   vector<PaxosService*> paxos_service;
 
-  Paxos *add_paxos(int type);
-  Paxos *get_paxos_by_name(const string& name);
   PaxosService *get_paxos_service_by_name(const string& name);
 
   class PGMonitor *pgmon() {
@@ -490,13 +490,14 @@ public:
   static CompatSet get_supported_features();
   static CompatSet get_legacy_features();
   void read_features();
-  void write_features();
+  void write_features(MonitorDBStore::Transaction &t);
 
  public:
-  Monitor(CephContext *cct_, string nm, MonitorStore *s, Messenger *m, MonMap *map);
+  Monitor(CephContext *cct_, string nm, MonitorDBStore *s,
+         Messenger *m, MonMap *map);
   ~Monitor();
 
-  static int check_features(MonitorStore *store);
+  static int check_features(MonitorDBStore *store);
 
   int preinit();
   int init();
@@ -522,6 +523,7 @@ public:
    * @return 0 on success, or negative error code
    */
   int write_fsid();
+  int write_fsid(MonitorDBStore::Transaction &t);
 
   void do_admin_command(std::string command, std::string args, ostream& ss);
 
index 196edeee476d955940e80bf29a69852d2d93c480..7586edc456e03c70f3a326ae9c90c694834e93f8 100644 (file)
@@ -14,7 +14,7 @@
 
 #include "MonmapMonitor.h"
 #include "Monitor.h"
-#include "MonitorStore.h"
+#include "MonitorDBStore.h"
 
 #include "messages/MMonCommand.h"
 #include "messages/MMonJoin.h"
@@ -47,43 +47,71 @@ void MonmapMonitor::create_initial()
 
 void MonmapMonitor::update_from_paxos()
 {
-  version_t paxosv = paxos->get_version();
-  if (paxosv <= paxos->get_stashed_version() &&
-      paxosv <= mon->monmap->get_epoch())
+  version_t version = get_version();
+  if (version <= mon->monmap->get_epoch())
     return;
 
-  dout(10) << "update_from_paxos paxosv " << paxosv
+  dout(10) << __func__ << " version " << version
           << ", my v " << mon->monmap->epoch << dendl;
   
-  version_t orig_latest = paxos->get_stashed_version();
-  bool need_restart = paxosv != mon->monmap->get_epoch();  
-  
-  if (paxosv > 0 && (mon->monmap->get_epoch() == 0 ||
-                    paxos->get_stashed_version() != paxosv)) {
-    bufferlist latest;
-    version_t v = paxos->get_stashed(latest);
-    if (v) {
-      mon->monmap->decode(latest);
-    }
+  /* It becomes clear here that we used the stashed version as a consistency
+   * mechanism. Take the 'if' we use: if our latest committed version is
+   * greater than 0 (i.e., exists one), and this version is different from
+   * our stashed version, then we will take the stashed monmap as our owm.
+   * 
+   * This is cleary to address the case in which we have a failure during
+   * the old MonitorStore updates. If a stashed version exists and it has
+   * a grater value than the last committed version, it means something
+   * went awry, and we did stashed a version (either after updating paxos
+   * and before proposing a new value, or during paxos itself) but it
+   * never became the last committed (for instance, because the system failed
+   * in the mean time).
+   *
+   * We no longer need to address these concerns. We are using transactions
+   * now and it should be the Paxos applying them. If the Paxos applies a
+   * transaction with the value we proposed, then it will be consistent
+   * with the Paxos values themselves. No need to hack our way in the
+   * store and create stashed versions to handle inconsistencies that are
+   * addressed by our MonitorDBStore.
+   *
+   * NOTE: this is not entirely true for the remaining services. In this one,
+   * the MonmapMonitor, we don't keep incrementals and each version is a full
+   * monmap. In the remaining services however, we keep mostly incrementals and
+   * we used to stash full versions of each map/summary. We still do it. We
+   * just don't need to do it here. Just check the code below and compare it
+   * with the code further down the line where we 'get' the latest committed
+   * version: it's the same code.
+   *
+  version_t latest_full = get_version_latest_full();
+  if ((latest_full > 0) && (latest_full > mon->monmap->get_epoch())) {
+    bufferlist latest_bl;
+    int err = get_version_full(latest_full, latest_bl);
+    assert(err == 0);
+    dout(7) << __func__ << " loading latest full monmap v"
+           << latest_full << dendl;
+    if (latest_bl.length() > 0)
+      mon->monmap->decode(latest_bl);
   }
+   */
+  bool need_restart = version != mon->monmap->get_epoch();  
 
-  if (paxosv > mon->monmap->get_epoch()) {
-    // read and decode
-    monmap_bl.clear();
-    bool success = paxos->read(paxosv, monmap_bl);
-    assert(success);
-    dout(10) << "update_from_paxos got " << paxosv << dendl;
-    mon->monmap->decode(monmap_bl);
+  // read and decode
+  monmap_bl.clear();
+  int ret = get_version(version, monmap_bl);
+  assert(ret == 0);
 
-    // save the bufferlist version in the paxos instance as well
-    paxos->stash_latest(paxosv, monmap_bl);
+  dout(10) << "update_from_paxos got " << version << dendl;
+  mon->monmap->decode(monmap_bl);
 
-    if (orig_latest == 0)
-      mon->store->erase_ss("mkfs", "monmap");
+  if (exists_key("mfks", get_service_name())) {
+    MonitorDBStore::Transaction t;
+    erase_mkfs(&t);
+    mon->store->apply_transaction(t);
   }
 
-  if (need_restart)
-    mon->bootstrap();
+  if (need_restart) {
+    paxos->prepare_bootstrap();
+  }
 }
 
 void MonmapMonitor::create_pending()
@@ -94,21 +122,34 @@ void MonmapMonitor::create_pending()
   dout(10) << "create_pending monmap epoch " << pending_map.epoch << dendl;
 }
 
-void MonmapMonitor::encode_pending(bufferlist& bl)
+void MonmapMonitor::encode_pending(MonitorDBStore::Transaction *t)
 {
   dout(10) << "encode_pending epoch " << pending_map.epoch << dendl;
 
   assert(mon->monmap->epoch + 1 == pending_map.epoch ||
         pending_map.epoch == 1);  // special case mkfs!
+  bufferlist bl;
   pending_map.encode(bl, mon->get_quorum_features());
+
+  put_version(t, pending_map.epoch, bl);
+  put_last_committed(t, pending_map.epoch);
 }
 
 void MonmapMonitor::on_active()
 {
-  if (paxos->get_version() >= 1 && !mon->has_ever_joined) {
+  if (get_version() >= 1 && !mon->has_ever_joined) {
     // make note of the fact that i was, once, part of the quorum.
     dout(10) << "noting that i was, once, part of an active quorum." << dendl;
-    mon->store->put_int(1, "joined");
+
+    /* This is some form of nasty in-breeding we have between the MonmapMonitor
+       and the Monitor itself. We should find a way to get rid of it given our
+       new architecture. Until then, stick with it since we are a
+       single-threaded process and, truth be told, no one else relies on this
+       thing besides us.
+     */
+    MonitorDBStore::Transaction t;
+    t.put(Monitor::MONITOR_NAME, "joined", 1);
+    mon->store->apply_transaction(t);
     mon->has_ever_joined = true;
   }
 
@@ -153,7 +194,7 @@ bool MonmapMonitor::preprocess_command(MMonCommand *m)
       (!session->caps.get_allow_all() &&
        !session->caps.check_privileges(PAXOS_MONMAP, MON_CAP_R) &&
        !mon->_allowed_command(session, m->cmd))) {
-    mon->reply_command(m, -EACCES, "access denied", paxos->get_version());
+    mon->reply_command(m, -EACCES, "access denied", get_version());
     return true;
   }
 
@@ -283,7 +324,7 @@ bool MonmapMonitor::preprocess_command(MMonCommand *m)
     string rs;
     getline(ss, rs);
 
-    mon->reply_command(m, r, rs, rdata, paxos->get_version());
+    mon->reply_command(m, r, rs, rdata, get_version());
     return true;
   } else
     return false;
@@ -318,7 +359,7 @@ bool MonmapMonitor::prepare_command(MMonCommand *m)
       (!session->caps.get_allow_all() &&
        !session->caps.check_privileges(PAXOS_MONMAP, MON_CAP_R) &&
        !mon->_allowed_command(session, m->cmd))) {
-    mon->reply_command(m, -EACCES, "access denied", paxos->get_version());
+    mon->reply_command(m, -EACCES, "access denied", get_version());
     return true;
   }
 
@@ -352,7 +393,7 @@ bool MonmapMonitor::prepare_command(MMonCommand *m)
       pending_map.last_changed = ceph_clock_now(g_ceph_context);
       ss << "added mon." << name << " at " << addr;
       getline(ss, rs);
-      paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+      paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, get_version()));
       return true;
     }
     else if (m->cmd.size() == 3 && m->cmd[1] == "remove") {
@@ -369,7 +410,7 @@ bool MonmapMonitor::prepare_command(MMonCommand *m)
       ss << "removed mon." << name << " at " << addr << ", there are now " << pending_map.size() << " monitors" ;
       getline(ss, rs);
       // send reply immediately in case we get removed
-      mon->reply_command(m, 0, rs, paxos->get_version());
+      mon->reply_command(m, 0, rs, get_version());
       return true;
     }
     else
@@ -379,7 +420,7 @@ bool MonmapMonitor::prepare_command(MMonCommand *m)
   
 out:
   getline(ss, rs);
-  mon->reply_command(m, err, rs, paxos->get_version());
+  mon->reply_command(m, err, rs, get_version());
   return false;
 }
 
index 2cd4413e510ea1235b71858c82e9870c5a0627a0..b3a5ab5c01dc7b9dd26c3700fa54c4690871199c 100644 (file)
@@ -29,6 +29,7 @@ using namespace std;
 
 #include "PaxosService.h"
 #include "MonMap.h"
+#include "MonitorDBStore.h"
 
 class MMonGetMap;
 class MMonMap;
@@ -37,8 +38,8 @@ class MMonJoin;
 
 class MonmapMonitor : public PaxosService {
  public:
-  MonmapMonitor(Monitor *mn, Paxos *p
-    : PaxosService(mn, p)
+  MonmapMonitor(Monitor *mn, Paxos *p, const string& service_name)
+    : PaxosService(mn, p, service_name)
   {
   }
   MonMap pending_map; //the pending map awaiting passage
@@ -49,7 +50,7 @@ class MonmapMonitor : public PaxosService {
 
   void create_pending();
 
-  void encode_pending(bufferlist& bl);
+  void encode_pending(MonitorDBStore::Transaction *t);
 
   void on_active();
 
index ded713aff51c0103e16651e246fa1499d0631aee..a9f13ae34849ff356c2838423c773d20b3b82a86 100644 (file)
@@ -19,7 +19,7 @@
 #include "MDSMonitor.h"
 #include "PGMonitor.h"
 
-#include "MonitorStore.h"
+#include "MonitorDBStore.h"
 
 #include "crush/CrushWrapper.h"
 #include "crush/CrushTester.h"
@@ -59,14 +59,6 @@ static ostream& _prefix(std::ostream *_dout, Monitor *mon, OSDMap& osdmap) {
 
 
 /************ MAPS ****************/
-OSDMonitor::OSDMonitor(Monitor *mn, Paxos *p)
-  : PaxosService(mn, p),
-    thrash_map(0), thrash_last_up_osd(-1)
-{
-  // we need to trim this too
-  p->add_extra_state_dir("osdmap_full");
-}
-
 
 void OSDMonitor::create_initial()
 {
@@ -75,7 +67,8 @@ void OSDMonitor::create_initial()
   OSDMap newmap;
 
   bufferlist bl;
-  mon->store->get_bl_ss_safe(bl, "mkfs", "osdmap");
+
+  get_mkfs(bl);
   if (bl.length()) {
     newmap.decode(bl);
     newmap.set_fsid(mon->monmap->fsid);
@@ -92,45 +85,62 @@ void OSDMonitor::create_initial()
 
 void OSDMonitor::update_from_paxos()
 {
-  version_t paxosv = paxos->get_version();
-  if (paxosv == osdmap.epoch)
+  version_t version = get_version();
+  if (version == osdmap.epoch)
     return;
-  assert(paxosv >= osdmap.epoch);
+  assert(version >= osdmap.epoch);
 
-  dout(15) << "update_from_paxos paxos e " << paxosv 
+  dout(15) << "update_from_paxos paxos e " << version
           << ", my e " << osdmap.epoch << dendl;
 
-  if (osdmap.epoch != paxos->get_stashed_version()) {
-    bufferlist latest;
-    version_t v = paxos->get_stashed(latest);
-    dout(7) << "update_from_paxos loading latest full map e" << v << dendl;
-    osdmap.decode(latest);
-  } 
-  
+
+  /* We no longer have stashed versions. Maybe we can do this by reading
+   * from a full map? Maybe we should keep the last full map version on a key
+   * as well (say, osdmap_full_version), and consider that the last_committed
+   * always contains incrementals, and maybe a full version if
+   * osdmap_full_version == last_committed
+   *
+   * This ^^^^ sounds about right. Do it. We should then change the
+   * 'get_stashed_version()' to 'get_full_version(version_t ver)', which should
+   * then be read iif
+   *   (osdmap.epoch != osd_full_version)
+   *   && (osdmap.epoch <= osdmap_full_version)
+   */
+  version_t latest_full = get_version_latest_full();
+  if ((latest_full > 0) && (latest_full > osdmap.epoch)) {
+    bufferlist latest_bl;
+    get_version_full(latest_full, latest_bl);
+    assert(latest_bl.length() != 0);
+    dout(7) << __func__ << " loading latest full map e" << latest_full << dendl;
+    osdmap.decode(latest_bl);
+  }
+
   // walk through incrementals
-  bufferlist bl;
-  while (paxosv > osdmap.epoch) {
-    bool success = paxos->read(osdmap.epoch+1, bl);
-    assert(success);
+  MonitorDBStore::Transaction t;
+  while (version > osdmap.epoch) {
+    bufferlist inc_bl;
+    int err = get_version(osdmap.epoch+1, inc_bl);
+    assert(err == 0);
+    assert(inc_bl.length());
     
     dout(7) << "update_from_paxos  applying incremental " << osdmap.epoch+1 << dendl;
-    OSDMap::Incremental inc(bl);
+    OSDMap::Incremental inc(inc_bl);
     osdmap.apply_incremental(inc);
 
     // write out the full map for all past epochs
-    bl.clear();
-    osdmap.encode(bl);
-    mon->store->put_bl_sn(bl, "osdmap_full", osdmap.epoch);
+    bufferlist full_bl;
+    osdmap.encode(full_bl);
+    put_version_full(&t, osdmap.epoch, full_bl);
 
     // share
     dout(1) << osdmap << dendl;
 
-    if (osdmap.epoch == 1)
-      mon->store->erase_ss("mkfs", "osdmap");
+    if (osdmap.epoch == 1) {
+      erase_mkfs(&t);
+    }
   }
-
-  // save latest
-  paxos->stash_latest(paxosv, bl);
+  if (!t.empty())
+    mon->store->apply_transaction(t);
 
   // populate down -> out map
   for (int o = 0; o < osdmap.get_max_osd(); o++)
@@ -417,8 +427,11 @@ void OSDMonitor::create_pending()
   remove_down_pg_temp();
 }
 
-
-void OSDMonitor::encode_pending(bufferlist &bl)
+/**
+ * @note receiving a transaction in this function gives a fair amount of
+ * freedom to the service implementation if it does need it. It shouldn't.
+ */
+void OSDMonitor::encode_pending(MonitorDBStore::Transaction *t)
 {
   dout(10) << "encode_pending e " << pending_inc.epoch
           << dendl;
@@ -426,6 +439,8 @@ void OSDMonitor::encode_pending(bufferlist &bl)
   // finalize up pending_inc
   pending_inc.modified = ceph_clock_now(g_ceph_context);
 
+  bufferlist bl;
+
   // tell me about it
   for (map<int32_t,uint8_t>::iterator i = pending_inc.new_state.begin();
        i != pending_inc.new_state.end();
@@ -439,7 +454,8 @@ void OSDMonitor::encode_pending(bufferlist &bl)
   for (map<int32_t,entity_addr_t>::iterator i = pending_inc.new_up_client.begin();
        i != pending_inc.new_up_client.end();
        i++) { 
-    dout(2) << " osd." << i->first << " UP " << i->second << dendl; //FIXME: insert cluster addresses too
+    //FIXME: insert cluster addresses too
+    dout(2) << " osd." << i->first << " UP " << i->second << dendl;
   }
   for (map<int32_t,uint32_t>::iterator i = pending_inc.new_weight.begin();
        i != pending_inc.new_weight.end();
@@ -454,8 +470,17 @@ void OSDMonitor::encode_pending(bufferlist &bl)
   }
 
   // encode
-  assert(paxos->get_version() + 1 == pending_inc.epoch);
+  assert(get_version() + 1 == pending_inc.epoch);
   ::encode(pending_inc, bl, CEPH_FEATURES_ALL);
+
+  /* put everything in the transaction */
+  put_version(t, pending_inc.epoch, bl);
+  put_last_committed(t, pending_inc.epoch);
+
+  bufferlist osdmap_bl;
+  osdmap.encode(osdmap_bl);
+  put_version_full(t, pending_inc.epoch, osdmap_bl);
+  put_version_latest_full(t, pending_inc.epoch);
 }
 
 
@@ -953,11 +978,11 @@ bool OSDMonitor::prepare_boot(MOSDBoot *m)
       // mark previous guy down
       pending_inc.new_state[from] = CEPH_OSD_UP;
     }
-    paxos->wait_for_commit(new C_RetryMessage(this, m));
+    wait_for_finished_proposal(new C_RetryMessage(this, m));
   } else if (pending_inc.new_up_client.count(from)) { //FIXME: should this be using new_up_client?
     // already prepared, just wait
     dout(7) << "prepare_boot already prepared, waiting on " << m->get_orig_source_addr() << dendl;
-    paxos->wait_for_commit(new C_RetryMessage(this, m));
+    wait_for_finished_proposal(new C_RetryMessage(this, m));
   } else {
     // mark new guy up.
     pending_inc.new_up_client[from] = m->get_orig_source_addr();
@@ -1031,7 +1056,7 @@ bool OSDMonitor::prepare_boot(MOSDBoot *m)
     pending_inc.new_xinfo[from] = xi;
 
     // wait
-    paxos->wait_for_commit(new C_Booted(this, m));
+    wait_for_finished_proposal(new C_Booted(this, m));
   }
   return true;
 }
@@ -1099,7 +1124,7 @@ bool OSDMonitor::prepare_alive(MOSDAlive *m)
   dout(7) << "prepare_alive want up_thru " << m->want << " have " << m->version
          << " from " << m->get_orig_source_inst() << dendl;
   pending_inc.new_up_thru[from] = m->version;  // set to the latest map the OSD has
-  paxos->wait_for_commit(new C_ReplyMap(this, m, m->version));
+  wait_for_finished_proposal(new C_ReplyMap(this, m, m->version));
   return true;
 }
 
@@ -1165,7 +1190,7 @@ bool OSDMonitor::prepare_pgtemp(MOSDPGTemp *m)
   for (map<pg_t,vector<int> >::iterator p = m->pg_temp.begin(); p != m->pg_temp.end(); p++)
     pending_inc.new_pg_temp[p->first] = p->second;
   pending_inc.new_up_thru[from] = m->map_epoch;   // set up_thru too, so the osd doesn't have to ask again
-  paxos->wait_for_commit(new C_ReplyMap(this, m, m->map_epoch));
+  wait_for_finished_proposal(new C_ReplyMap(this, m, m->map_epoch));
   return true;
 }
 
@@ -1279,7 +1304,7 @@ void OSDMonitor::send_to_waiting()
 
 void OSDMonitor::send_latest(PaxosServiceMessage *m, epoch_t start)
 {
-  if (paxos->is_readable()) {
+  if (is_readable()) {
     dout(5) << "send_latest to " << m->get_orig_source_inst()
            << " start " << start << dendl;
     if (start == 0)
@@ -1298,7 +1323,7 @@ void OSDMonitor::send_latest(PaxosServiceMessage *m, epoch_t start)
 MOSDMap *OSDMonitor::build_latest_full()
 {
   MOSDMap *r = new MOSDMap(mon->monmap->fsid, &osdmap);
-  r->oldest_map = paxos->get_first_committed();
+  r->oldest_map = get_first_committed();
   r->newest_map = osdmap.get_epoch();
   return r;
 }
@@ -1307,23 +1332,27 @@ MOSDMap *OSDMonitor::build_incremental(epoch_t from, epoch_t to)
 {
   dout(10) << "build_incremental [" << from << ".." << to << "]" << dendl;
   MOSDMap *m = new MOSDMap(mon->monmap->fsid);
-  m->oldest_map = paxos->get_first_committed();
+  m->oldest_map = get_first_committed();
   m->newest_map = osdmap.get_epoch();
 
-  for (epoch_t e = to;
-       e >= from && e > 0;
-       e--) {
+  for (epoch_t e = to; e >= from && e > 0; e--) {
     bufferlist bl;
-    if (mon->store->get_bl_sn(bl, "osdmap", e) > 0) {
-      dout(20) << "build_incremental    inc " << e << " " << bl.length() << " bytes" << dendl;
+    get_version(e, bl);
+    if (bl.length() > 0) {
+      // if (get_version(e, bl) > 0) {
+      dout(20) << "build_incremental    inc " << e << " "
+              << bl.length() << " bytes" << dendl;
       m->incremental_maps[e] = bl;
-    } 
-    else if (mon->store->get_bl_sn(bl, "osdmap_full", e) > 0) {
-      dout(20) << "build_incremental   full " << e << " " << bl.length() << " bytes" << dendl;
+    } else {
+      get_version("full", e, bl);
+      if (bl.length() > 0) {
+      //else if (get_version("full", e, bl) > 0) {
+      dout(20) << "build_incremental   full " << e << " "
+              << bl.length() << " bytes" << dendl;
       m->maps[e] = bl;
-    }
-    else {
-      assert(0);  // we should have all maps.
+      } else {
+       assert(0);  // we should have all maps.
+      }
     }
   }
   return m;
@@ -1335,18 +1364,28 @@ void OSDMonitor::send_full(PaxosServiceMessage *m)
   mon->send_reply(m, build_latest_full());
 }
 
+/* TBH, I'm fairly certain these two functions could somehow be using a single
+ * helper function to do the heavy lifting. As this is not our main focus right
+ * now, I'm leaving it to the next near-future iteration over the services'
+ * code. We should not forget it though.
+ *
+ * TODO: create a helper function and get rid of the duplicated code.
+ */
 void OSDMonitor::send_incremental(PaxosServiceMessage *req, epoch_t first)
 {
   dout(5) << "send_incremental [" << first << ".." << osdmap.get_epoch() << "]"
          << " to " << req->get_orig_source_inst() << dendl;
-  if (first < paxos->get_first_committed()) {
-    first = paxos->get_first_committed();
+  if (first < get_first_committed()) {
+    first = get_first_committed();
     bufferlist bl;
-    mon->store->get_bl_sn_safe(bl, "osdmap_full", first);
+    get_version("full", first, bl);
     assert(bl.length());
-    dout(20) << "send_incremental starting with base full " << first << " " << bl.length() << " bytes" << dendl;
+
+    dout(20) << "send_incremental starting with base full "
+            << first << " " << bl.length() << " bytes" << dendl;
+
     MOSDMap *m = new MOSDMap(osdmap.get_fsid());
-    m->oldest_map = paxos->get_first_committed();
+    m->oldest_map = first;
     m->newest_map = osdmap.get_epoch();
     m->maps[first] = bl;
     mon->send_reply(req, m);
@@ -1357,7 +1396,7 @@ void OSDMonitor::send_incremental(PaxosServiceMessage *req, epoch_t first)
   // started.
   epoch_t last = MIN(first + g_conf->osd_map_message_max, osdmap.get_epoch());
   MOSDMap *m = build_incremental(first, last);
-  m->oldest_map = paxos->get_first_committed();
+  m->oldest_map = get_first_committed();
   m->newest_map = osdmap.get_epoch();
   mon->send_reply(req, m);
 }
@@ -1367,14 +1406,17 @@ void OSDMonitor::send_incremental(epoch_t first, entity_inst_t& dest, bool oneti
   dout(5) << "send_incremental [" << first << ".." << osdmap.get_epoch() << "]"
          << " to " << dest << dendl;
 
-  if (first < paxos->get_first_committed()) {
-    first = paxos->get_first_committed();
+  if (first < get_first_committed()) {
+    first = get_first_committed();
     bufferlist bl;
-    mon->store->get_bl_sn_safe(bl, "osdmap_full", first);
+    get_version("full", first, bl);
     assert(bl.length());
-    dout(20) << "send_incremental starting with base full " << first << " " << bl.length() << " bytes" << dendl;
+    
+    dout(20) << "send_incremental starting with base full "
+            << first << " " << bl.length() << " bytes" << dendl;
+
     MOSDMap *m = new MOSDMap(osdmap.get_fsid());
-    m->oldest_map = paxos->get_first_committed();
+    m->oldest_map = first;
     m->newest_map = osdmap.get_epoch();
     m->maps[first] = bl;
     mon->messenger->send_message(m, dest);
@@ -1435,8 +1477,7 @@ void OSDMonitor::check_sub(Subscription *sub)
 
 void OSDMonitor::tick()
 {
-  if (!paxos->is_active())
-    return;
+  if (!is_active()) return;
 
   update_from_paxos();
   dout(10) << osdmap << dendl;
@@ -1535,7 +1576,7 @@ void OSDMonitor::tick()
   }
 
   //if map full setting has changed, get that info out there!
-  if (mon->pgmon()->paxos->is_readable()) {
+  if (mon->pgmon()->is_readable()) {
     if (!mon->pgmon()->pg_map.full_osds.empty()) {
       dout(5) << "There are full osds, setting full flag" << dendl;
       add_flag(CEPH_OSDMAP_FULL);
@@ -1577,23 +1618,22 @@ void OSDMonitor::tick()
       !pending_inc.new_pg_temp.empty())  // also propose if we adjusted pg_temp
     propose_pending();
 
-  if (mon->pgmon()->paxos->is_readable() &&
+  if (mon->pgmon()->is_readable() &&
       mon->pgmon()->pg_map.creating_pgs.empty()) {
     epoch_t floor = mon->pgmon()->pg_map.calc_min_last_epoch_clean();
     dout(10) << " min_last_epoch_clean " << floor << dendl;
     unsigned min = g_conf->mon_min_osdmap_epochs;
-    if (floor + min > paxos->get_version()) {
-      if (min < paxos->get_version())
-       floor = paxos->get_version() - min;
+    if (floor + min > get_version()) {
+      if (min < get_version())
+       floor = get_version() - min;
       else
        floor = 0;
     }
-    if (floor > paxos->get_first_committed())
-      paxos->trim_to(floor);
+    if (floor > get_first_committed())
+      trim_to(floor); // we are now responsible for trimming our own versions.
   }    
 }
 
-
 void OSDMonitor::handle_osd_timeouts(const utime_t &now,
                                     std::map<int,utime_t> &last_osd_report)
 {
@@ -1712,7 +1752,7 @@ bool OSDMonitor::preprocess_command(MMonCommand *m)
       (!session->caps.get_allow_all() &&
        !session->caps.check_privileges(PAXOS_OSDMAP, MON_CAP_R) &&
        !mon->_allowed_command(session, m->cmd))) {
-    mon->reply_command(m, -EACCES, "access denied", rdata, paxos->get_version());
+    mon->reply_command(m, -EACCES, "access denied", rdata, get_version());
     return true;
   }
 
@@ -1753,7 +1793,7 @@ bool OSDMonitor::preprocess_command(MMonCommand *m)
       OSDMap *p = &osdmap;
       if (epoch) {
        bufferlist b;
-       mon->store->get_bl_sn_safe(b,"osdmap_full", epoch);
+       get_version("full", epoch, b);
        if (!b.length()) {
          p = 0;
          r = -ENOENT;
@@ -1859,7 +1899,7 @@ bool OSDMonitor::preprocess_command(MMonCommand *m)
        m->cmd.erase(m->cmd.begin()); //and now we're done with the target num
        for (int i = 0; i < osdmap.get_max_osd(); ++i)
          if (osdmap.is_up(i))
-           mon->send_command(osdmap.get_inst(i), m->cmd, paxos->get_version());
+           mon->send_command(osdmap.get_inst(i), m->cmd, get_version());
        r = 0;
        ss << "ok";
       } else {
@@ -1870,7 +1910,7 @@ bool OSDMonitor::preprocess_command(MMonCommand *m)
          r = -EINVAL;
        } else {
          if (osdmap.is_up(who)) {
-           mon->send_command(osdmap.get_inst(who), m->cmd, paxos->get_version());
+           mon->send_command(osdmap.get_inst(who), m->cmd, get_version());
            r = 0;
            ss << "ok";
          } else {
@@ -1969,7 +2009,7 @@ bool OSDMonitor::preprocess_command(MMonCommand *m)
   if (r != -1) {
     string rs;
     getline(ss, rs);
-    mon->reply_command(m, r, rs, rdata, paxos->get_version());
+    mon->reply_command(m, r, rs, rdata, get_version());
     return true;
   } else
     return false;
@@ -2033,7 +2073,7 @@ bool OSDMonitor::prepare_set_flag(MMonCommand *m, int flag)
     pending_inc.new_flags = osdmap.get_flags();
   pending_inc.new_flags |= flag;
   ss << "set " << OSDMap::get_flag_string(flag);
-  paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, ss.str(), paxos->get_version()));
+  wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, ss.str(), get_version()));
   return true;
 }
 
@@ -2044,7 +2084,7 @@ bool OSDMonitor::prepare_unset_flag(MMonCommand *m, int flag)
     pending_inc.new_flags = osdmap.get_flags();
   pending_inc.new_flags &= ~flag;
   ss << "unset " << OSDMap::get_flag_string(flag);
-  paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, ss.str(), paxos->get_version()));
+  wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, ss.str(), get_version()));
   return true;
 }
 
@@ -2097,7 +2137,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
       (!session->caps.get_allow_all() &&
        !session->caps.check_privileges(PAXOS_OSDMAP, MON_CAP_W) &&
        !mon->_allowed_command(session, m->cmd))) {
-    mon->reply_command(m, -EACCES, "access denied", paxos->get_version());
+    mon->reply_command(m, -EACCES, "access denied", get_version());
     return true;
   }
 
@@ -2126,7 +2166,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
 
       pending_inc.crush = data;
       string rs = "set crush map";
-      paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+      wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
       return true;
     }
     else if (m->cmd.size() >= 5 && m->cmd[1] == "crush" && m->cmd[2] == "set") {
@@ -2182,7 +2222,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
          ss << "updated item id " << id << " name '" << name << "' weight " << weight
             << " at location " << loc << " to crush map";
          getline(ss, rs);
-         paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+         wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
          return true;
        }
       } while (false);
@@ -2230,7 +2270,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
          ss << "create-or-move updating item id " << id << " name '" << name << "' weight " << weight
             << " at location " << loc << " to crush map";
          getline(ss, rs);
-         paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+         wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
          return true;
        }
       } while (false);
@@ -2267,7 +2307,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
            pending_inc.crush.clear();
            newcrush.encode(pending_inc.crush);
            getline(ss, rs);
-           paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+           wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
            return true;
          }
        } else {
@@ -2304,7 +2344,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
          newcrush.encode(pending_inc.crush);
          ss << "removed item id " << id << " name '" << m->cmd[3] << "' from crush map";
          getline(ss, rs);
-         paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+         wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
          return true;
        }
       } while (false);
@@ -2342,7 +2382,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
          ss << "reweighted item id " << id << " name '" << m->cmd[3] << "' to " << w
             << " in crush map";
          getline(ss, rs);
-         paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+         wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
          return true;
        }
       } while (false);
@@ -2376,7 +2416,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
        newcrush.encode(pending_inc.crush);
        ss << "adjusted tunables profile to " << m->cmd[3];
        getline(ss, rs);
-       paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+       wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
        return true;
       }
     }
@@ -2396,7 +2436,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
       pending_inc.new_max_osd = newmax;
       ss << "set new max_osd = " << pending_inc.new_max_osd;
       getline(ss, rs);
-      paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+      wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
       return true;
     }
     else if (m->cmd[1] == "pause") {
@@ -2471,7 +2511,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
       }
       if (any) {
        getline(ss, rs);
-       paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+       wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
        return true;
       }
     }
@@ -2494,7 +2534,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
       }
       if (any) {
        getline(ss, rs);
-       paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+       wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
        return true;
       }
     }
@@ -2517,7 +2557,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
       }
       if (any) {
        getline(ss, rs);
-       paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+       wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
        return true;
       } 
     }
@@ -2542,7 +2582,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
          pending_inc.new_weight[osd] = ww;
          ss << "reweighted osd." << osd << " to " << w << " (" << ios::hex << ww << ios::dec << ")";
          getline(ss, rs);
-         paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+         wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
          return true;
        }
       }
@@ -2563,7 +2603,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
        pending_inc.new_lost[osd] = e;
        ss << "marked osd lost in epoch " << e;
        getline(ss, rs);
-       paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+       wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
        return true;
       }
     }
@@ -2589,7 +2629,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
        i = pending_inc.identify_osd(uuid);
        if (i >= 0) {
          // osd is about to exist
-         paxos->wait_for_commit(new C_RetryMessage(this, m));
+         wait_for_finished_proposal(new C_RetryMessage(this, m));
          return true;
        }
       }
@@ -2617,7 +2657,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
        pending_inc.new_uuid[i] = uuid;
       ss << i;
       getline(ss, rs);
-      paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+      wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
       return true;
     }
     else if (m->cmd[1] == "rm" && m->cmd.size() >= 3) {
@@ -2642,7 +2682,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
       }
       if (any) {
        getline(ss, rs);
-       paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+       wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
        return true;
       }
     }
@@ -2661,7 +2701,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
        pending_inc.new_blacklist[addr] = expires;
        ss << "blacklisting " << addr << " until " << expires << " (" << d << " sec)";
        getline(ss, rs);
-       paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+       wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
        return true;
       } else if (m->cmd[2] == "rm") {
        if (osdmap.is_blacklisted(addr) || 
@@ -2672,7 +2712,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
            pending_inc.new_blacklist.erase(addr);
          ss << "un-blacklisting " << addr;
          getline(ss, rs);
-         paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+         wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
          return true;
        }
        ss << addr << " isn't blacklisted";
@@ -2705,7 +2745,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
            pp->set_snap_epoch(pending_inc.epoch);
            ss << "created pool " << m->cmd[3] << " snap " << snapname;
            getline(ss, rs);
-           paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+           wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
            return true;
          }
        }
@@ -2735,7 +2775,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
            pp->set_snap_epoch(pending_inc.epoch);
            ss << "removed pool " << m->cmd[3] << " snap " << snapname;
            getline(ss, rs);
-           paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+           wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
            return true;
          }
        }
@@ -2797,7 +2837,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
          ss << "pool '" << m->cmd[3] << "' created";
        }
        getline(ss, rs);
-       paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+       wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
        return true;
       } else if (m->cmd[2] == "delete" && m->cmd.size() >= 4) {
        // osd pool delete <poolname> <poolname again> --yes-i-really-really-mean-it
@@ -2820,7 +2860,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
        if (ret == 0)
          ss << "pool '" << m->cmd[3] << "' deleted";
        getline(ss, rs);
-       paxos->wait_for_commit(new Monitor::C_Command(mon, m, ret, rs, paxos->get_version()));
+       wait_for_finished_proposal(new Monitor::C_Command(mon, m, ret, rs, get_version()));
        return true;
       } else if (m->cmd[2] == "rename" && m->cmd.size() == 5) {
        int64_t pool = osdmap.lookup_pg_pool_name(m->cmd[3].c_str());
@@ -2839,7 +2879,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
               << cpp_strerror(ret);
          }
          getline(ss, rs);
-         paxos->wait_for_commit(new Monitor::C_Command(mon, m, ret, rs, paxos->get_version()));
+         wait_for_finished_proposal(new Monitor::C_Command(mon, m, ret, rs, get_version()));
          return true;
        }
       } else if (m->cmd[2] == "set") {
@@ -2867,7 +2907,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
              pending_inc.new_pools[pool].last_change = pending_inc.epoch;
              ss << "set pool " << pool << " size to " << n;
              getline(ss, rs);
-             paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+             wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
              return true;
            } else if (m->cmd[4] == "min_size") {
              if (pending_inc.new_pools.count(pool) == 0)
@@ -2876,7 +2916,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
              pending_inc.new_pools[pool].last_change = pending_inc.epoch;
              ss << "set pool " << pool << " min_size to " << n;
              getline(ss, rs);
-             paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+             wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
              return true;
            } else if (m->cmd[4] == "crash_replay_interval") {
              if (pending_inc.new_pools.count(pool) == 0)
@@ -2884,7 +2924,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
              pending_inc.new_pools[pool].crash_replay_interval = n;
              ss << "set pool " << pool << " to crash_replay_interval to " << n;
              getline(ss, rs);
-             paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+             wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
              return true;
            } else if (m->cmd[4] == "pg_num") {
              if (m->cmd.size() < 7 ||
@@ -2904,7 +2944,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
                pending_inc.new_pools[pool].last_change = pending_inc.epoch;
                ss << "set pool " << pool << " pg_num to " << n;
                getline(ss, rs);
-               paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+               wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
                return true;
              }
            } else if (m->cmd[4] == "pgp_num") {
@@ -2920,7 +2960,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
                pending_inc.new_pools[pool].last_change = pending_inc.epoch;
                ss << "set pool " << pool << " pgp_num to " << n;
                getline(ss, rs);
-               paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+               wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
                return true;
              }
            } else if (m->cmd[4] == "crush_ruleset") {
@@ -2931,7 +2971,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
                pending_inc.new_pools[pool].last_change = pending_inc.epoch;
                ss << "set pool " << pool << " crush_ruleset to " << n;
                getline(ss, rs);
-               paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+               wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
                return true;
              } else {
                ss << "crush ruleset " << n << " does not exist";
@@ -3011,7 +3051,7 @@ bool OSDMonitor::prepare_command(MMonCommand *m)
       } else {
        ss << "SUCCESSFUL reweight-by-utilization: " << out_str;
        getline(ss, rs);
-       paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+        wait_for_finished_proposal(new Monitor::C_Command(mon, m, 0, rs, get_version()));
        return true;
       }
     }
@@ -3036,7 +3076,7 @@ out:
   getline(ss, rs);
   if (err < 0 && rs.length() == 0)
     rs = cpp_strerror(err);
-  mon->reply_command(m, err, rs, paxos->get_version());
+  mon->reply_command(m, err, rs, get_version());
   return ret;
 }
 
@@ -3222,7 +3262,7 @@ bool OSDMonitor::prepare_pool_op(MPoolOp *m)
   }
 
  out:
-  paxos->wait_for_commit(new OSDMonitor::C_PoolOp(this, m, ret, pending_inc.epoch, &reply_data));
+  wait_for_finished_proposal(new OSDMonitor::C_PoolOp(this, m, ret, pending_inc.epoch, &reply_data));
   propose_pending();
   return false;
 }
@@ -3230,7 +3270,7 @@ bool OSDMonitor::prepare_pool_op(MPoolOp *m)
 bool OSDMonitor::prepare_pool_op_create(MPoolOp *m)
 {
   int err = prepare_new_pool(m);
-  paxos->wait_for_commit(new OSDMonitor::C_PoolOp(this, m, err, pending_inc.epoch));
+  wait_for_finished_proposal(new OSDMonitor::C_PoolOp(this, m, err, pending_inc.epoch));
   return true;
 }
 
@@ -3278,7 +3318,7 @@ int OSDMonitor::_prepare_rename_pool(uint64_t pool, string newname)
 bool OSDMonitor::prepare_pool_op_delete(MPoolOp *m)
 {
   int ret = _prepare_remove_pool(m->pool);
-  paxos->wait_for_commit(new OSDMonitor::C_PoolOp(this, m, ret, pending_inc.epoch));
+  wait_for_finished_proposal(new OSDMonitor::C_PoolOp(this, m, ret, pending_inc.epoch));
   return true;
 }
 
@@ -3296,7 +3336,7 @@ bool OSDMonitor::prepare_pool_op_auid(MPoolOp *m)
       if (pending_inc.new_pools.count(m->pool) == 0)
        pending_inc.new_pools[m->pool] = *(osdmap.get_pg_pool(m->pool));
       pending_inc.new_pools[m->pool].auid = m->auid;
-      paxos->wait_for_commit(new OSDMonitor::C_PoolOp(this, m, 0, pending_inc.epoch));
+      wait_for_finished_proposal(new OSDMonitor::C_PoolOp(this, m, 0, pending_inc.epoch));
       return true;
     }
   }
@@ -3311,7 +3351,7 @@ void OSDMonitor::_pool_op_reply(MPoolOp *m, int ret, epoch_t epoch, bufferlist *
 {
   dout(20) << "_pool_op_reply " << ret << dendl;
   MPoolOpReply *reply = new MPoolOpReply(m->fsid, m->get_tid(),
-                                        ret, epoch, paxos->get_version(), blp);
+                                        ret, epoch, get_version(), blp);
   mon->send_reply(m, reply);
   m->put();
 }
index f53b6285abb2c4763e7adf53e03535dd85ec0bed..7d327e4df90f8a5527258283012d8bb570d390ea 100644 (file)
@@ -137,7 +137,7 @@ public:
 private:
   void update_from_paxos();
   void create_pending();  // prepare a new pending
-  void encode_pending(bufferlist &bl);
+  void encode_pending(MonitorDBStore::Transaction *t);
   void on_active();
 
   void update_msgr_features();
@@ -259,7 +259,9 @@ private:
   bool prepare_remove_snaps(class MRemoveSnaps *m);
 
  public:
-  OSDMonitor(Monitor *mn, Paxos *p);
+  OSDMonitor(Monitor *mn, Paxos *p, string service_name)
+  : PaxosService(mn, p, service_name),
+    thrash_map(0), thrash_last_up_osd(-1) { }
 
   void tick();  // check state, take actions
 
index 95954b7253bfd58727fa17e1b30b1e32d387438a..056bd98f6457dc65785e84285f58ddcbce22857c 100644 (file)
@@ -17,7 +17,7 @@
 #include "Monitor.h"
 #include "MDSMonitor.h"
 #include "OSDMonitor.h"
-#include "MonitorStore.h"
+#include "MonitorDBStore.h"
 
 #include "messages/MPGStats.h"
 #include "messages/MPGStatsAck.h"
@@ -50,13 +50,6 @@ static ostream& _prefix(std::ostream *_dout, const Monitor *mon, const PGMap& pg
                << ").pg v" << pg_map.version << " ";
 }
 
-PGMonitor::PGMonitor(Monitor *mn, Paxos *p)
-  : PaxosService(mn, p),
-    need_check_down_pgs(false)
-{ }
-
-PGMonitor::~PGMonitor() {}
-
 /*
  Tick function to update the map based on performance every N seconds
 */
@@ -116,8 +109,7 @@ void PGMonitor::update_logger()
 
 void PGMonitor::tick() 
 {
-  if (!paxos->is_active())
-    return;
+  if (!is_active()) return;
 
   update_from_paxos();
   handle_osd_timeouts();
@@ -143,35 +135,41 @@ void PGMonitor::create_initial()
 
 void PGMonitor::update_from_paxos()
 {
-  version_t paxosv = paxos->get_version();
-  if (paxosv == pg_map.version)
+  version_t version = get_version();
+  if (version == pg_map.version)
     return;
-  assert(paxosv >= pg_map.version);
-
-  if (pg_map.version != paxos->get_stashed_version()) {
-    bufferlist latest;
-    version_t v = paxos->get_stashed(latest);
-    dout(7) << "update_from_paxos loading latest full pgmap v" << v << dendl;
+  assert(version >= pg_map.version);
+
+  /* Obtain latest full pgmap version, if available and whose version is
+   * greater than the current pgmap's version.
+   */
+  version_t latest_full = get_version_latest_full();
+  if ((latest_full > 0) && (latest_full > pg_map.version)) {
+    bufferlist latest_bl;
+    int err = get_version_full(latest_full, latest_bl);
+    assert(err == 0);
+    dout(7) << __func__ << " loading latest full pgmap v"
+           << latest_full << dendl;
     try {
       PGMap tmp_pg_map;
-      bufferlist::iterator p = latest.begin();
+      bufferlist::iterator p = latest_bl.begin();
       tmp_pg_map.decode(p);
       pg_map = tmp_pg_map;
-    }
-    catch (const std::exception &e) {
-      dout(0) << "update_from_paxos: error parsing update: "
+    } catch (const std::exception& e) {
+      dout(0) << __func__ << ": error parsing update: "
              << e.what() << dendl;
       assert(0 == "update_from_paxos: error parsing update");
       return;
     }
-  } 
+  }
 
   // walk through incrementals
   utime_t now(ceph_clock_now(g_ceph_context));
-  while (paxosv > pg_map.version) {
+  while (version > pg_map.version) {
     bufferlist bl;
-    bool success = paxos->read(pg_map.version+1, bl);
-    assert(success);
+    int err = get_version(pg_map.version+1, bl);
+    assert(err == 0);
+    assert(bl.length());
 
     dout(7) << "update_from_paxos  applying incremental " << pg_map.version+1 << dendl;
     PGMap::Incremental inc;
@@ -194,26 +192,25 @@ void PGMonitor::update_from_paxos()
       last_sent_pg_create.clear();  // reset pg_create throttle timer
   }
 
-  assert(paxosv == pg_map.version);
-
-  // save latest
-  bufferlist bl;
-  pg_map.encode(bl, mon->get_quorum_features());
-  paxos->stash_latest(paxosv, bl);
+  assert(version == pg_map.version);
 
+  /* If we dump the summaries onto the k/v store, they hardly would be useful
+   * without a tool created with reading them in mind.
+   * Comment this out until we decide what is the best course of action.
+   *
   // dump pgmap summaries?  (useful for debugging)
   if (0) {
     stringstream ds;
     pg_map.dump(ds);
     bufferlist d;
     d.append(ds);
-    mon->store->put_bl_sn(d, "pgmap_dump", paxosv);
+    mon->store->put_bl_sn(d, "pgmap_dump", version);
   }
+  */
 
   unsigned max = g_conf->mon_max_pgmap_epochs;
-  if (mon->is_leader() &&
-      paxosv > max)
-    paxos->trim_to(paxosv - max);
+  if (mon->is_leader() && (version > max))
+    trim_to(version - max);
 
   send_pg_creates();
 
@@ -231,7 +228,7 @@ void PGMonitor::handle_osd_timeouts()
     return;
   }
 
-  if (mon->osdmon()->paxos->is_writeable())
+  if (mon->osdmon()->is_writeable())
     mon->osdmon()->handle_osd_timeouts(now, last_osd_report);
 }
 
@@ -254,12 +251,24 @@ void PGMonitor::create_pending()
   dout(10) << "create_pending v " << pending_inc.version << dendl;
 }
 
-void PGMonitor::encode_pending(bufferlist &bl)
+void PGMonitor::encode_pending(MonitorDBStore::Transaction *t)
 {
-  dout(10) << "encode_pending v " << pending_inc.version << dendl;
-  assert(paxos->get_version() + 1 == pending_inc.version);
+  version_t version = pending_inc.version;
+  dout(10) << __func__ << " v " << version << dendl;
+  assert(get_version() + 1 == version);
   pending_inc.stamp = ceph_clock_now(g_ceph_context);
+
+  bufferlist bl;
   pending_inc.encode(bl, mon->get_quorum_features());
+
+  bufferlist full_bl;
+  pg_map.encode(full_bl, mon->get_quorum_features());
+
+  put_version(t, version, bl);
+  put_last_committed(t, version);
+
+  put_version_full(t, version, full_bl);
+  put_version_latest_full(t, version);
 }
 
 bool PGMonitor::preprocess_query(PaxosServiceMessage *m)
@@ -324,7 +333,7 @@ void PGMonitor::handle_statfs(MStatfs *statfs)
   }
 
   // fill out stfs
-  reply = new MStatfsReply(mon->monmap->fsid, statfs->get_tid(), paxos->get_version());
+  reply = new MStatfsReply(mon->monmap->fsid, statfs->get_tid(), get_version());
 
   // these are in KB.
   reply->h.st.kb = pg_map.osd_sum.kb;
@@ -356,7 +365,7 @@ bool PGMonitor::preprocess_getpoolstats(MGetPoolStats *m)
     goto out;
   }
   
-  reply = new MGetPoolStatsReply(m->fsid, m->get_tid(), paxos->get_version());
+  reply = new MGetPoolStatsReply(m->fsid, m->get_tid(), get_version());
 
   for (list<string>::iterator p = m->pools.begin();
        p != m->pools.end();
@@ -396,7 +405,7 @@ bool PGMonitor::preprocess_pg_stats(MPGStats *stats)
   // First, just see if they need a new osdmap. But
   // only if they've had the map for a while.
   if (stats->had_map_for > 30.0 && 
-      mon->osdmon()->paxos->is_readable() &&
+      mon->osdmon()->is_readable() &&
       stats->epoch < mon->osdmon()->osdmap.get_epoch())
     mon->osdmon()->send_latest_now_nodelete(stats, stats->epoch+1);
 
@@ -562,15 +571,15 @@ void PGMonitor::check_osd_map(epoch_t epoch)
     return;
   }
 
-  if (!mon->osdmon()->paxos->is_readable()) {
+  if (!mon->osdmon()->is_readable()) {
     dout(10) << "check_osd_map -- osdmap not readable, waiting" << dendl;
-    mon->osdmon()->paxos->wait_for_readable(new RetryCheckOSDMap(this, epoch));
+    mon->osdmon()->wait_for_readable(new RetryCheckOSDMap(this, epoch));
     return;
   }
 
-  if (!paxos->is_writeable()) {
+  if (!is_writeable()) {
     dout(10) << "check_osd_map -- pgmap not writeable, waiting" << dendl;
-    paxos->wait_for_writeable(new RetryCheckOSDMap(this, epoch));
+    wait_for_writeable(new RetryCheckOSDMap(this, epoch));
     return;
   }
 
@@ -580,7 +589,8 @@ void PGMonitor::check_osd_map(epoch_t epoch)
        e++) {
     dout(10) << "check_osd_map applying osdmap e" << e << " to pg_map" << dendl;
     bufferlist bl;
-    mon->store->get_bl_sn_safe(bl, "osdmap", e);
+    mon->osdmon()->get_version(e, bl);
+
     assert(bl.length());
     OSDMap::Incremental inc(bl);
     for (map<int32_t,uint32_t>::iterator p = inc.new_weight.begin();
@@ -889,7 +899,7 @@ bool PGMonitor::preprocess_command(MMonCommand *m)
       (!session->caps.get_allow_all() &&
        !session->caps.check_privileges(PAXOS_PGMAP, MON_CAP_R) &&
        !mon->_allowed_command(session, m->cmd))) {
-    mon->reply_command(m, -EACCES, "access denied", rdata, paxos->get_version());
+    mon->reply_command(m, -EACCES, "access denied", rdata, get_version());
     return true;
   }
 
@@ -1084,7 +1094,7 @@ bool PGMonitor::preprocess_command(MMonCommand *m)
   if (r != -1) {
     string rs;
     getline(ss, rs);
-    mon->reply_command(m, r, rs, rdata, paxos->get_version());
+    mon->reply_command(m, r, rs, rdata, get_version());
     return true;
   } else
     return false;
@@ -1104,7 +1114,7 @@ bool PGMonitor::prepare_command(MMonCommand *m)
       (!session->caps.get_allow_all() &&
        !session->caps.check_privileges(PAXOS_PGMAP, MON_CAP_W) &&
        !mon->_allowed_command(session, m->cmd))) {
-    mon->reply_command(m, -EACCES, "access denied", paxos->get_version());
+    mon->reply_command(m, -EACCES, "access denied", get_version());
     return true;
   }
 
@@ -1133,7 +1143,7 @@ bool PGMonitor::prepare_command(MMonCommand *m)
     }
     ss << "pg " << m->cmd[2] << " now creating, ok";
     getline(ss, rs);
-    paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+    paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, get_version()));
     return true;
   }
   else if (m->cmd.size() > 1 && m->cmd[1] == "set_full_ratio") {
@@ -1149,7 +1159,7 @@ bool PGMonitor::prepare_command(MMonCommand *m)
       goto out;
     }
     pending_inc.full_ratio = n;
-    paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+    paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, get_version()));
     return true;
   }
   else if (m->cmd.size() > 1 && m->cmd[1] == "set_nearfull_ratio") {
@@ -1165,7 +1175,7 @@ bool PGMonitor::prepare_command(MMonCommand *m)
       goto out;
     }
     pending_inc.nearfull_ratio = n;
-    paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, paxos->get_version()));
+    paxos->wait_for_commit(new Monitor::C_Command(mon, m, 0, rs, get_version()));
     return true;
   }
 
@@ -1173,7 +1183,7 @@ bool PGMonitor::prepare_command(MMonCommand *m)
   getline(ss, rs);
   if (r < 0 && rs.length() == 0)
     rs = cpp_strerror(r);
-  mon->reply_command(m, r, rs, paxos->get_version());
+  mon->reply_command(m, r, rs, get_version());
   return false;
 }
 
index c150e157b9dd57e1107f80338786374185a2e4ad..626a590be7cbc0d0941fd5499579dcc5dac296d6 100644 (file)
@@ -30,6 +30,7 @@ using namespace std;
 #include "include/utime.h"
 #include "msg/Messenger.h"
 #include "common/config.h"
+#include "mon/MonitorDBStore.h"
 
 #include "messages/MPGStats.h"
 #include "messages/MPGStatsAck.h"
@@ -52,8 +53,8 @@ private:
   void update_from_paxos();
   void handle_osd_timeouts();
   void create_pending();  // prepare a new pending
-  void encode_pending(bufferlist &bl);  // propose pending update to peers
-
+  // propose pending update to peers
+  void encode_pending(MonitorDBStore::Transaction *t);
   void update_logger();
 
   bool preprocess_query(PaxosServiceMessage *m);  // true if processed.
@@ -125,8 +126,9 @@ private:
                          vector<const char*>& args) const;
 
 public:
-  PGMonitor(Monitor *mn, Paxos *p);
-  virtual ~PGMonitor();
+  PGMonitor(Monitor *mn, Paxos *p, const string& service_name)
+  : PaxosService(mn, p, service_name), need_check_down_pgs(false) { }
+  ~PGMonitor() { }
 
   virtual void on_restart();
 
index 20f3089b49716056f7ccd265a6f4d2151064e7f9..1658a8da1b72e4499a78c45f54c0767f1597087c 100644 (file)
  * 
  */
 
+#include <sstream>
 #include "Paxos.h"
 #include "Monitor.h"
-#include "MonitorStore.h"
+#include "MonitorDBStore.h"
 
 #include "messages/MMonPaxos.h"
 
 #include "common/config.h"
 #include "include/assert.h"
+#include "common/Formatter.h"
 
 #define dout_subsys ceph_subsys_paxos
 #undef dout_prefix
-#define dout_prefix _prefix(_dout, mon, mon->name, mon->rank, machine_name, state, first_committed, last_committed)
+#define dout_prefix _prefix(_dout, mon, mon->name, mon->rank, paxos_name, state, first_committed, last_committed)
 static ostream& _prefix(std::ostream *_dout, Monitor *mon, const string& name,
-                       int rank, const char *machine_name, int state,
+                       int rank, const string& paxos_name, int state,
                        version_t first_committed, version_t last_committed)
 {
   return *_dout << "mon." << name << "@" << rank
                << "(" << mon->get_state_name() << ")"
-               << ".paxos(" << machine_name << " " << Paxos::get_statename(state)
+               << ".paxos(" << paxos_name << " " << Paxos::get_statename(state)
                << " c " << first_committed << ".." << last_committed
                << ") ";
 }
 
+void Paxos::prepare_bootstrap()
+{
+  dout(0) << __func__ << dendl;
+
+  going_to_bootstrap = true;
+}
 
+MonitorDBStore *Paxos::get_store()
+{
+  return mon->store;
+}
 
 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");
-  first_committed = mon->store->get_int(machine_name, "first_committed");
-  bufferlist temp;
-  latest_stashed = get_stashed(temp);
-  slurping = mon->store->get_int(machine_name, "slurping");
+  last_pn = get_store()->get(get_name(), "last_pn");
+  accepted_pn = get_store()->get(get_name(), "accepted_pn");
+  last_committed = get_store()->get(get_name(), "last_committed");
+  first_committed = get_store()->get(get_name(), "first_committed");
+
+  dout(10) << __func__ << " last_pn: " << last_pn << " accepted_pn: "
+          << accepted_pn << " last_committed: " << last_committed
+          << " first_committed: " << first_committed << dendl;
+
+  //slurping = get_store()->get(get_name(), "slurping");
 
   dout(10) << "init" << dendl;
 }
@@ -70,10 +85,10 @@ void Paxos::collect(version_t oldpn)
   peer_last_committed.clear();
 
   // look for uncommitted value
-  if (mon->store->exists_bl_sn(machine_name, last_committed+1)) {
+  if (get_store()->exists(get_name(), last_committed+1)) {
     uncommitted_v = last_committed+1;
     uncommitted_pn = accepted_pn;
-    mon->store->get_bl_sn_safe(uncommitted_value, machine_name, last_committed+1);
+    get_store()->get(get_name(), last_committed+1, uncommitted_value);
     assert(uncommitted_value.length());
     dout(10) << "learned uncommitted " << (last_committed+1)
             << " (" << uncommitted_value.length() << " bytes) from myself" 
@@ -93,7 +108,7 @@ void Paxos::collect(version_t oldpn)
     if (*p == mon->rank) continue;
     
     MMonPaxos *collect = new MMonPaxos(mon->get_epoch(), MMonPaxos::OP_COLLECT,
-                                      machine_id, ceph_clock_now(g_ceph_context));
+                                      ceph_clock_now(g_ceph_context));
     collect->last_committed = last_committed;
     collect->first_committed = first_committed;
     collect->pn = accepted_pn;
@@ -117,7 +132,7 @@ void Paxos::handle_collect(MMonPaxos *collect)
   state = STATE_RECOVERING;
 
   // reply
-  MMonPaxos *last = new MMonPaxos(mon->get_epoch(), MMonPaxos::OP_LAST, machine_id,
+  MMonPaxos *last = new MMonPaxos(mon->get_epoch(), MMonPaxos::OP_LAST,
                                  ceph_clock_now(g_ceph_context));
   last->last_committed = last_committed;
   last->first_committed = first_committed;
@@ -129,7 +144,15 @@ void Paxos::handle_collect(MMonPaxos *collect)
     accepted_pn_from = collect->pn_from;
     dout(10) << "accepting pn " << accepted_pn << " from " 
             << accepted_pn_from << dendl;
-    mon->store->put_int(accepted_pn, machine_name, "accepted_pn");
+  
+    MonitorDBStore::Transaction t;
+    t.put(get_name(), "accepted_pn", accepted_pn);
+    JSONFormatter f(true);
+    t.dump(&f);
+    dout(30) << __func__ << " transaction dump:\n";
+    f.flush(*_dout);
+    *_dout << dendl;
+    get_store()->apply_transaction(t);
   } else {
     // don't accept!
     dout(10) << "NOT accepting pn " << collect->pn << " from " << collect->pn_from
@@ -146,8 +169,8 @@ void Paxos::handle_collect(MMonPaxos *collect)
   // do we have an accepted but uncommitted value?
   //  (it'll be at last_committed+1)
   bufferlist bl;
-  if (mon->store->exists_bl_sn(machine_name, last_committed+1)) {
-    mon->store->get_bl_sn_safe(bl, machine_name, last_committed+1);
+  if (get_store()->exists(get_name(), last_committed+1)) {
+    get_store()->get(get_name(), last_committed+1, bl);
     assert(bl.length() > 0);
     dout(10) << " sharing our accepted but uncommitted value for " 
             << last_committed+1 << " (" << bl.length() << " bytes)" << dendl;
@@ -160,6 +183,18 @@ void Paxos::handle_collect(MMonPaxos *collect)
   collect->put();
 }
 
+/**
+ * @note This is Okay. We share our versions between peer_last_committed and
+ *      our last_committed (inclusive), and add their bufferlists to the
+ *      message. It will be the peer's job to apply them to his store, as
+ *      these bufferlists will contain raw transactions.
+ *      This function is called by both the Peon and the Leader. The Peon will
+ *      share the state with the Leader during handle_collect(), sharing any
+ *      values the leader may be missing (i.e., the leader's last_committed is
+ *      lower than the peon's last_committed). The Leader will share the state
+ *      with the Peon during handle_last(), if the peon's last_committed is
+ *      lower than the leader's last_committed.
+ */
 void Paxos::share_state(MMonPaxos *m, version_t peer_first_committed,
                        version_t peer_last_committed)
 {
@@ -169,54 +204,36 @@ void Paxos::share_state(MMonPaxos *m, version_t peer_first_committed,
           << " lc " << peer_last_committed << dendl;
   version_t v = peer_last_committed + 1;
 
-  // start with a stashed full copy?
-  if (peer_last_committed + 1 < first_committed) {
-    bufferlist bl;
-    version_t l = get_stashed(bl);
-    assert(l <= last_committed);
-    dout(10) << "share_state starting with latest " << l 
-            << " (" << bl.length() << " bytes)" << dendl;
-    m->latest_value.claim(bl);
-    m->latest_version = l;
-    v = l;
-  }
-
   // include incrementals
   for ( ; v <= last_committed; v++) {
-    if (mon->store->exists_bl_sn(machine_name, v)) {
-      mon->store->get_bl_sn_safe(m->values[v], machine_name, v);
+    if (get_store()->exists(get_name(), v)) {
+      get_store()->get(get_name(), v, m->values[v]);
       assert(m->values[v].length());
-      dout(10) << " sharing " << v << " (" 
+      dout(10) << " sharing " << v << " ("
               << m->values[v].length() << " bytes)" << dendl;
-   }
+    }
   }
 
   m->last_committed = last_committed;
 }
 
+/**
+ * Store on disk a state that was shared with us
+ *
+ * Basically, we received a set of version. Or just one. It doesn't matter.
+ * What matters is that we have to stash it in the store. So, we will simply
+ * write every single bufferlist into their own versions on our side (i.e.,
+ * onto paxos-related keys), and then we will decode those same bufferlists
+ * we just wrote and apply the transactions they hold. We will also update
+ * our first and last committed values to point to the new values, if need
+ * be. All all this is done tightly wrapped in a transaction to ensure we
+ * enjoy the atomicity guarantees given by our awesome k/v store.
+ */
 void Paxos::store_state(MMonPaxos *m)
 {
+  MonitorDBStore::Transaction t;
   map<version_t,bufferlist>::iterator start = m->values.begin();
 
-  // stash?
-  if (m->latest_version && m->latest_version > last_committed) {
-    dout(10) << "store_state got stash version " 
-            << m->latest_version << ", zapping old states" << dendl;
-
-    assert(start != m->values.end() && start->first == m->latest_version);
-
-    // wipe out everything we had previously
-    trim_to(last_committed + 1);
-
-    stash_latest(m->latest_version, m->latest_value);
-
-    first_committed = m->latest_version;
-    last_committed = m->latest_version;
-    mon->store->put_bl_sn(start->second, machine_name, m->latest_version);
-    mon->store->put_int(first_committed, machine_name, "first_committed");
-    mon->store->put_int(last_committed, machine_name, "last_committed");
-  }
-
   // build map of values to store
   // we want to write the range [last_committed, m->last_committed] only.
   if (start != m->values.end() &&
@@ -227,10 +244,14 @@ void Paxos::store_state(MMonPaxos *m)
     start = m->values.end();
   }
 
+  // push forward the start position on the message's values iterator, up until
+  // we run out of positions or we find a position matching 'last_committed'.
   while (start != m->values.end() && start->first <= last_committed) {
     ++start;
   }
 
+  // make sure we get the right interval of values to apply by pushing forward
+  // the 'end' iterator until it matches the message's 'last_committed'.
   map<version_t,bufferlist>::iterator end = start;
   while (end != m->values.end() && end->first <= m->last_committed) {
     last_committed = end->first;
@@ -244,14 +265,37 @@ void Paxos::store_state(MMonPaxos *m)
   } else {
     dout(10) << "store_state [" << start->first << ".." 
             << last_committed << "]" << dendl;
-
-    mon->store->put_bl_sn_map(machine_name, start, end);
-    mon->store->put_int(last_committed, machine_name, "last_committed");
-    mon->store->put_int(first_committed, machine_name, "first_committed");
+    t.put(get_name(), "last_committed", last_committed);
+    // we write our first_committed version before we append the message's
+    // transaction because this transaction may be a trim; if so, it will
+    // update the first_committed value, and we must let it clobber our own
+    // in order to obtain an updated state.
+    t.put(get_name(), "first_committed", first_committed);
+    // we should apply the state here -- decode every single bufferlist in the
+    // map and append the transactions to 't'.
+    map<version_t,bufferlist>::iterator it;
+    for (it = start; it != end; ++it) {
+      // write the bufferlist as the version's value
+      t.put(get_name(), it->first, it->second);
+      // decode the bufferlist and append it to the transaction we will shortly
+      // apply.
+      decode_append_transaction(t, it->second);
+    }
+  }
+  if (!t.empty()) {
+    JSONFormatter f(true);
+    t.dump(&f);
+    dout(30) << __func__ << " transaction dump:\n";
+    f.flush(*_dout);
+    *_dout << dendl;
+    get_store()->apply_transaction(t);
+
+    // update the first and last committed in-memory values.
+    first_committed = get_store()->get(get_name(), "first_committed");
+    last_committed = get_store()->get(get_name(), "last_committed");
   }
 }
 
-
 // leader
 void Paxos::handle_last(MMonPaxos *last)
 {
@@ -267,7 +311,7 @@ void Paxos::handle_last(MMonPaxos *last)
   // push it to them.
   peer_last_committed[last->get_source().num()] = last->last_committed;
 
-  // did we receive a committed value?
+  // store any committed values if any are specified in the message
   store_state(last);
       
   // do they accept your pn?
@@ -312,7 +356,7 @@ void Paxos::handle_last(MMonPaxos *last)
          // share committed values
          dout(10) << " sending commit to mon." << p->first << dendl;
          MMonPaxos *commit = new MMonPaxos(mon->get_epoch(),
-                                           MMonPaxos::OP_COMMIT, machine_id,
+                                           MMonPaxos::OP_COMMIT,
                                            ceph_clock_now(g_ceph_context));
          share_state(commit, peer_first_committed[p->first], p->second);
          mon->messenger->send_message(commit, mon->monmap->get_inst(p->first));
@@ -322,15 +366,16 @@ void Paxos::handle_last(MMonPaxos *last)
       peer_last_committed.clear();
 
       // almost...
-      state = STATE_ACTIVE;
 
       // did we learn an old value?
       if (uncommitted_v == last_committed+1 &&
          uncommitted_value.length()) {
        dout(10) << "that's everyone.  begin on old learned value" << dendl;
+       state = STATE_PREPARING | STATE_LOCKED;
        begin(uncommitted_value);
       } else {
        // active!
+       state = STATE_ACTIVE;
        dout(10) << "that's everyone.  active!" << dendl;
        extend_lease();
 
@@ -364,8 +409,9 @@ void Paxos::begin(bufferlist& v)
           << dendl;
 
   assert(mon->is_leader());
-  assert(is_active());
-  state = STATE_UPDATING;
+  assert(is_preparing());
+  state &= ~STATE_PREPARING;
+  state |= STATE_UPDATING;
 
   // we must already have a majority for this to work.
   assert(mon->get_quorum().size() == 1 ||
@@ -378,13 +424,30 @@ void Paxos::begin(bufferlist& v)
   accepted.clear();
   accepted.insert(mon->rank);
   new_value = v;
-  mon->store->put_bl_sn(new_value, machine_name, last_committed+1);
+  // store the proposed value in the store. IF it is accepted, we will then
+  // have to decode it into a transaction and apply it.
+  MonitorDBStore::Transaction t;
+  t.put(get_name(), last_committed+1, new_value);
+
+  JSONFormatter f(true);
+  t.dump(&f);
+  dout(30) << __func__ << " transaction dump:\n";
+  f.flush(*_dout);
+  MonitorDBStore::Transaction debug_tx;
+  bufferlist::iterator new_value_it = new_value.begin();
+  debug_tx.decode(new_value_it);
+  debug_tx.dump(&f);
+  *_dout << "\nbl dump:\n";
+  f.flush(*_dout);
+  *_dout << dendl;
+
+  get_store()->apply_transaction(t);
 
   if (mon->get_quorum().size() == 1) {
     // we're alone, take it easy
     commit();
     state = STATE_ACTIVE;
-
+    finish_proposal();
     finish_contexts(g_ceph_context, waiting_for_active);
     finish_contexts(g_ceph_context, waiting_for_commit);
     finish_contexts(g_ceph_context, waiting_for_readable);
@@ -401,7 +464,7 @@ void Paxos::begin(bufferlist& v)
     
     dout(10) << " sending begin to mon." << *p << dendl;
     MMonPaxos *begin = new MMonPaxos(mon->get_epoch(), MMonPaxos::OP_BEGIN,
-                                    machine_id, ceph_clock_now(g_ceph_context));
+                                    ceph_clock_now(g_ceph_context));
     begin->values[last_committed+1] = new_value;
     begin->last_committed = last_committed;
     begin->pn = accepted_pn;
@@ -435,11 +498,22 @@ void Paxos::handle_begin(MMonPaxos *begin)
   // yes.
   version_t v = last_committed+1;
   dout(10) << "accepting value for " << v << " pn " << accepted_pn << dendl;
-  mon->store->put_bl_sn(begin->values[v], machine_name, v);
+  // store the accepted value onto our store. We will have to decode it and
+  // apply its transaction once we receive permission to commit.
+  MonitorDBStore::Transaction t;
+  t.put(get_name(), v, begin->values[v]);
+
+  JSONFormatter f(true);
+  t.dump(&f);
+  dout(30) << __func__ << " transaction dump:\n";
+  f.flush(*_dout);
+  *_dout << dendl;
+
+  get_store()->apply_transaction(t);
 
   // reply
   MMonPaxos *accept = new MMonPaxos(mon->get_epoch(), MMonPaxos::OP_ACCEPT,
-                                   machine_id, ceph_clock_now(g_ceph_context));
+                                   ceph_clock_now(g_ceph_context));
   accept->pn = accepted_pn;
   accept->last_committed = last_committed;
   mon->messenger->send_message(accept, begin->get_source_inst());
@@ -468,7 +542,7 @@ void Paxos::handle_accept(MMonPaxos *accept)
   assert(accept->last_committed == last_committed ||   // not committed
         accept->last_committed == last_committed-1);  // committed
 
-  assert(state == STATE_UPDATING);
+  assert(is_updating());
   assert(accepted.count(from) == 0);
   accepted.insert(from);
   dout(10) << " now " << accepted << " have accepted" << dendl;
@@ -491,6 +565,8 @@ void Paxos::handle_accept(MMonPaxos *accept)
     // yay!
     state = STATE_ACTIVE;
     extend_lease();
+  
+    finish_proposal();
 
     // wake people up
     finish_contexts(g_ceph_context, waiting_for_active);
@@ -519,15 +595,29 @@ void Paxos::commit()
   //   leader still got a majority and committed with out us.)
   lease_expire = utime_t();  // cancel lease
 
+  MonitorDBStore::Transaction t;
+
   // commit locally
   last_committed++;
   last_commit_time = ceph_clock_now(g_ceph_context);
-  mon->store->put_int(last_committed, machine_name, "last_committed");
+  t.put(get_name(), "last_committed", last_committed);
   if (!first_committed) {
     first_committed = last_committed;
-    mon->store->put_int(last_committed, machine_name, "first_committed");
+    t.put(get_name(), "first_committed", last_committed);
   }
 
+  // decode the value and apply its transaction to the store.
+  // this value can now be read from last_committed.
+  decode_append_transaction(t, new_value);
+
+  JSONFormatter f(true);
+  t.dump(&f);
+  dout(30) << __func__ << " transaction dump:\n";
+  f.flush(*_dout);
+  *_dout << dendl;
+
+  get_store()->apply_transaction(t);
+
   // tell everyone
   for (set<int>::const_iterator p = mon->get_quorum().begin();
        p != mon->get_quorum().end();
@@ -536,7 +626,7 @@ void Paxos::commit()
 
     dout(10) << " sending commit to mon." << *p << dendl;
     MMonPaxos *commit = new MMonPaxos(mon->get_epoch(), MMonPaxos::OP_COMMIT,
-                                     machine_id, ceph_clock_now(g_ceph_context));
+                                     ceph_clock_now(g_ceph_context));
     commit->values[last_committed] = new_value;
     commit->pn = accepted_pn;
     commit->last_committed = last_committed;
@@ -586,7 +676,7 @@ void Paxos::extend_lease()
 
     if (*p == mon->rank) continue;
     MMonPaxos *lease = new MMonPaxos(mon->get_epoch(), MMonPaxos::OP_LEASE,
-                                    machine_id, ceph_clock_now(g_ceph_context));
+                                    ceph_clock_now(g_ceph_context));
     lease->last_committed = last_committed;
     lease->lease_timestamp = lease_expire;
     lease->first_committed = first_committed;
@@ -628,6 +718,61 @@ void Paxos::warn_on_future_time(utime_t t, entity_name_t from)
 
 }
 
+void Paxos::finish_proposal()
+{
+  /* There is a lot of debug still going around. We will get rid of it later
+   * on, as soon as everything "just works (tm)"
+   */
+  assert(mon->is_leader());
+
+  dout(10) << __func__ << " finishing proposal" << dendl;
+  C_Proposal *proposal = (C_Proposal*) proposals.front();
+  dout(10) << __func__ << " finish it (proposal = "
+          << proposal << ")" << dendl;;
+
+  if (!proposal) // this is okay. It happens when we propose an old value.
+    return;
+
+  if (!proposal->proposed) {
+    dout(10) << __func__ << " we must have received a stay message and we're "
+             << "trying to finish before time. "
+            << "Instead, propose it (if we are active)!" << dendl;
+  } else {
+    dout(10) << __func__ << " proposal took "
+            << (ceph_clock_now(NULL) - proposal->proposal_time)
+            << " to finish" << dendl;
+
+    proposals.pop_front();
+    proposal->finish(0);
+  }
+
+  dout(10) << __func__ << " state " << state
+          << " proposals left " << proposals.size() << dendl;
+
+  /* Update the internal Paxos state.
+   *
+   * Since we moved to a single-paxos instance across all monitor services, we
+   * can no longer rely on each individual service to update paxos state.
+   * Therefore, once we conclude a proposal, we must update our internal
+   * state (say, such variables as 'first_committed'), because no one else
+   * will take care of that for us -- and we rely on these variables for
+   * several other mechanisms; trimming comes to mind.
+   */
+  first_committed = get_store()->get(get_name(), "first_committed");
+  last_committed = get_store()->get(get_name(), "last_committed");
+
+  if ((proposals.size() == 0) && going_to_bootstrap) {
+    dout(0) << __func__ << " no more proposals; bootstraping." << dendl;
+    mon->bootstrap();
+    return;
+  }
+
+
+  if (is_active() && (proposals.size() > 0)) {
+    propose_queued();
+  }
+}
+
 // peon
 void Paxos::handle_lease(MMonPaxos *lease)
 {
@@ -661,7 +806,7 @@ void Paxos::handle_lease(MMonPaxos *lease)
 
   // ack
   MMonPaxos *ack = new MMonPaxos(mon->get_epoch(), MMonPaxos::OP_LEASE_ACK,
-                                machine_id, ceph_clock_now(g_ceph_context));
+                                ceph_clock_now(g_ceph_context));
   ack->last_committed = last_committed;
   ack->first_committed = first_committed;
   ack->lease_timestamp = ceph_clock_now(g_ceph_context);
@@ -747,26 +892,36 @@ void Paxos::lease_renew_timeout()
  * trim old states
  */
 
-void Paxos::trim_to(version_t first, bool force)
+void Paxos::trim_to(MonitorDBStore::Transaction *t, version_t first, bool force)
 {
   dout(10) << "trim_to " << first << " (was " << first_committed << ")"
-          << ", latest_stashed " << latest_stashed
           << dendl;
 
   if (first_committed >= first)
     return;
 
-  while (first_committed < first &&
-        (force || first_committed < latest_stashed)) {
+  while (first_committed < first) {
     dout(10) << "trim " << first_committed << dendl;
-    mon->store->erase_sn(machine_name, first_committed);
-    for (list<string>::iterator p = extra_state_dirs.begin();
-        p != extra_state_dirs.end();
-        ++p)
-      mon->store->erase_sn(p->c_str(), first_committed);
+    t->erase(get_name(), first_committed);
     first_committed++;
   }
-  mon->store->put_int(first_committed, machine_name, "first_committed");
+  t->put(get_name(), "first_committed", first_committed);
+}
+
+void Paxos::trim_to(version_t first, bool force)
+{
+  MonitorDBStore::Transaction t;
+  
+  trim_to(&t, first, force);
+
+  if (!t.empty()) {
+    JSONFormatter f(true);
+    t.dump(&f);
+    dout(30) << __func__ << " transaction dump:\n";
+    f.flush(*_dout);
+    *_dout << dendl;
+    get_store()->apply_transaction(t);
+  }
 }
 
 /*
@@ -784,7 +939,14 @@ version_t Paxos::get_new_proposal_number(version_t gt)
   last_pn += (version_t)mon->rank;
   
   // write
-  mon->store->put_int(last_pn, machine_name, "last_pn");
+  MonitorDBStore::Transaction t;
+  t.put(get_name(), "last_pn", last_pn);
+  JSONFormatter f(true);
+  t.dump(&f);
+  dout(30) << __func__ << " transaction dump:\n";
+  f.flush(*_dout);
+  *_dout << dendl;
+  get_store()->apply_transaction(t);
 
   dout(10) << "get_new_proposal_number = " << last_pn << dendl;
   return last_pn;
@@ -827,6 +989,10 @@ void Paxos::leader_init()
 {
   cancel_events();
   new_value.clear();
+  if (proposals.size() > 0)
+    proposals.clear();
+
+  going_to_bootstrap = false;
 
   if (mon->get_quorum().size() == 1) {
     state = STATE_ACTIVE;
@@ -858,6 +1024,11 @@ void Paxos::restart()
   dout(10) << "restart -- canceling timeouts" << dendl;
   cancel_events();
   new_value.clear();
+  dout(10) << __func__ << " -- clearing queued proposals" << dendl;
+  if (proposals.size() > 0)
+    proposals.clear();
+
+  going_to_bootstrap = false;
 
   finish_contexts(g_ceph_context, waiting_for_commit, -EAGAIN);
   finish_contexts(g_ceph_context, waiting_for_active, -EAGAIN);
@@ -935,12 +1106,12 @@ bool Paxos::is_readable(version_t v)
     (is_active() || is_updating()) &&
     last_committed > 0 &&           // must have a value
     (mon->get_quorum().size() == 1 ||  // alone, or
-     ceph_clock_now(g_ceph_context) < lease_expire);    // have lease
+     is_lease_valid()); // have lease
 }
 
 bool Paxos::read(version_t v, bufferlist &bl)
 {
-  if (mon->store->get_bl_sn(bl, machine_name, v) <= 0)
+  if (!get_store()->get(get_name(), v, bl))
     return false;
   return true;
 }
@@ -953,96 +1124,91 @@ version_t Paxos::read_current(bufferlist &bl)
 }
 
 
-
+bool Paxos::is_lease_valid()
+{
+  return ((mon->get_quorum().size() == 1)
+      || (ceph_clock_now(g_ceph_context) < lease_expire));
+}
 
 // -- WRITE --
 
 bool Paxos::is_writeable()
 {
-  if (mon->get_quorum().size() == 1) return true;
   return
     mon->is_leader() &&
     is_active() &&
-    ceph_clock_now(g_ceph_context) < lease_expire;
+    is_lease_valid();
 }
 
-bool Paxos::propose_new_value(bufferlist& bl, Context *oncommit)
+void Paxos::list_proposals(ostream& out)
 {
-  /*
-  // writeable?
-  if (!is_writeable()) {
-    dout(5) << "propose_new_value " << last_committed+1 << " " << bl.length() << " bytes"
-           << " -- not writeable" << dendl;
-    if (oncommit) {
-      oncommit->finish(-1);
-      delete oncommit;
-    }
-    return false;
+  out << __func__ << " " << proposals.size() << " in queue:\n";
+  list<Context*>::iterator p_it = proposals.begin();
+  for (int i = 0; p_it != proposals.end(); ++p_it, ++i) {
+    C_Proposal *p = (C_Proposal*) *p_it;
+    out << "-- entry #" << i << "\n";
+    out << *p << "\n";
   }
-  */
-  
-  assert(mon->is_leader() && is_active());
+}
+
+void Paxos::propose_queued()
+{
+  assert(is_active());
+  assert(proposals.size() > 0);
+
+  state = STATE_PREPARING;
+
+  C_Proposal *proposal = (C_Proposal*) proposals.front();
+  assert(!proposal->proposed);
 
-  // cancel lease renewal and timeout events.
   cancel_events();
+  dout(5) << __func__ << " " << (last_committed + 1)
+         << " " << proposal->bl.length() << " bytes" << dendl;
+  proposal->proposed = true;
 
-  // ok!
-  dout(5) << "propose_new_value " << last_committed+1 << " " << bl.length() << " bytes" << dendl;
-  if (oncommit)
-    waiting_for_commit.push_back(oncommit);
-  begin(bl);
-  
-  return true;
+  dout(10) << __func__ << " ";
+  list_proposals(*_dout);
+  *_dout << dendl;
+
+  begin(proposal->bl);
 }
 
-void Paxos::stash_latest(version_t v, bufferlist& bl)
+void Paxos::queue_proposal(bufferlist& bl, Context *onfinished)
 {
-  if (v == latest_stashed) {
-    dout(10) << "stash_latest v" << v << " already stashed" << dendl;
-    return;  // already stashed.
-  }
-
-  bufferlist final;
-  ::encode(v, final);
-  ::encode(bl, final);
-  
-  dout(10) << "stash_latest v" << v << " len " << bl.length() << dendl;
-  mon->store->put_bl_ss(final, machine_name, "latest");
+  dout(5) << __func__ << " bl " << bl.length() << " bytes;"
+         << " ctx = " << onfinished << dendl;
 
-  latest_stashed = v;
+  proposals.push_back(new C_Proposal(onfinished, bl));
 }
 
-version_t Paxos::get_stashed(bufferlist& bl)
+bool Paxos::propose_new_value(bufferlist& bl, Context *onfinished)
 {
-  bufferlist full;
-  mon->store->get_bl_ss_safe(full, machine_name, "latest");
-  if (!full.length()) {
-    dout(10) << "get_stashed not found" << dendl;
-    return 0;
+  assert(mon->is_leader());
+
+  queue_proposal(bl, onfinished);
+
+  if (!is_active()) {
+    dout(5) << __func__ << " not active; proposal queued" << dendl; 
+    return true;
   }
-  bufferlist::iterator p = full.begin();
-  version_t v;
-  ::decode(v, p);
-  ::decode(bl, p);
-
-  latest_stashed = v;
-  dout(10) << "get_stashed v" << latest_stashed << " len " << bl.length() << dendl;
-  return latest_stashed;  
-}
 
+  propose_queued();
+  
+  return true;
+}
 
 bool Paxos::is_consistent()
 {
   bool consistent = true;
-  if (first_committed > 1 && first_committed > latest_stashed)
-    consistent = false;
   if (first_committed > last_committed)
     consistent = false;
-  if (latest_stashed > last_committed)
-    consistent = false;
   if (slurping != 0)
     consistent = false;
 
+  if (!(consistent || (slurping == 1))) {
+    dout(5) << "consistent: " << consistent 
+           << " slurping: " << slurping << dendl;
+  }
   assert(consistent || (slurping == 1));
   return consistent;
 }
@@ -1051,7 +1217,7 @@ void Paxos::start_slurping()
 {
   if (slurping != 1) {
     slurping = 1;
-    mon->store->put_int(1, machine_name, "slurping");
+//    get_store()->put(get_name(), "slurping", 1);
   }
 }
 
@@ -1059,7 +1225,8 @@ void Paxos::end_slurping()
 {
   if (slurping == 1) {
     slurping = 0;
-    mon->store->put_int(slurping, machine_name, "slurping");
+  //  get_store()->put(get_name(), "slurping", slurping);
   }
   assert(is_consistent());
 }
+
index d148855d02747ef868f84762d8b1513bb4f00acc..f9bf09798f58b3415f0df3b03bbe389ce51139d1 100644 (file)
@@ -31,10 +31,81 @@ b 12v
 c 14v
 d
 e 12v
-
-
 */
 
+/**
+ * Paxos storage layout and behavior
+ *
+ * Currently, we use a key/value store to hold all the Paxos-related data, but
+ * it can logically be depicted as this:
+ *
+ *  paxos:
+ *    first_committed -> 1
+ *     last_committed -> 4
+ *                 1 -> value_1
+ *                 2 -> value_2
+ *                 3 -> value_3
+ *                 4 -> value_4
+ *
+ * Since we are relying on a k/v store supporting atomic transactions, we can
+ * guarantee that if 'last_committed' has a value of '4', then we have up to
+ * version 4 on the store, and no more than that; the same applies to
+ * 'first_committed', which holding '1' will strictly meaning that our lowest
+ * version is 1.
+ *
+ * Each version's value (value_1, value_2, ..., value_n) is a blob of data,
+ * incomprehensible to the Paxos. These values are proposed to the Paxos on
+ * propose_new_value() and each one is a transaction encoded in a bufferlist.
+ *
+ * The Paxos will write the value to disk, associating it with its version,
+ * but will take a step further: the value shall be decoded, and the operations
+ * on that transaction shall be applied during the same transaction that will
+ * write the value's encoded bufferlist to disk. This behavior ensures that
+ * whatever is being proposed will only be available on the store when it is
+ * applied by Paxos, which will then be aware of such new values, guaranteeing
+ * the store state is always consistent without requiring shady workarounds.
+ *
+ * So, let's say that FooMonitor proposes the following transaction, neatly
+ * encoded on a bufferlist of course:
+ *
+ *  Tx_Foo
+ *    put(foo, last_committed, 3)
+ *    put(foo, 3, foo_value_3)
+ *    erase(foo, 2)
+ *    erase(foo, 1)
+ *    put(foo, first_committed, 3)
+ *
+ * And knowing that the Paxos is proposed Tx_Foo as a bufferlist, once it is
+ * ready to commit, and assuming we are now committing version 5 of the Paxos,
+ * we will do something along the lines of:
+ *
+ *  Tx proposed_tx;
+ *  proposed_tx.decode(Tx_foo_bufferlist);
+ *
+ *  Tx our_tx;
+ *  our_tx.put(paxos, last_committed, 5);
+ *  our_tx.put(paxos, 5, Tx_foo_bufferlist);
+ *  our_tx.append(proposed_tx);
+ *
+ *  store_apply(our_tx);
+ *
+ * And the store should look like this after we apply 'our_tx':
+ *
+ *  paxos:
+ *    first_committed -> 1
+ *     last_committed -> 5
+ *                 1 -> value_1
+ *                 2 -> value_2
+ *                 3 -> value_3
+ *                 4 -> value_4
+ *                 5 -> Tx_foo_bufferlist
+ *  foo:
+ *    first_committed -> 3
+ *     last_committed -> 3
+ *                 3 -> foo_value_3
+ *
+ */
+
 #ifndef CEPH_MON_PAXOS_H
 #define CEPH_MON_PAXOS_H
 
@@ -49,11 +120,12 @@ e 12v
 #include "common/Timer.h"
 #include <errno.h>
 
+#include "MonitorDBStore.h"
+
 class Monitor;
 class MMonPaxos;
 class Paxos;
 
-
 // i am one state machine.
 /**
  * This libary is based on the Paxos algorithm, but varies in a few key ways:
@@ -76,8 +148,7 @@ class Paxos {
   Monitor *mon;
 
   // my state machine info
-  int machine_id;
-  const char *machine_name;
+  const string paxos_name;
 
   friend class Monitor;
   friend class PaxosService;
@@ -95,16 +166,22 @@ public:
   /**
    * Leader/Peon is in Paxos' Recovery state
    */
-  const static int STATE_RECOVERING = 1;
+  const static int STATE_RECOVERING = 0x01;
   /**
    * Leader/Peon is idle, and the Peon may or may not have a valid lease.
    */
-  const static int STATE_ACTIVE     = 2;
+  const static int STATE_ACTIVE     = 0x02;
   /**
    * Leader/Peon is updating to a new value.
    */
-  const static int STATE_UPDATING   = 3;
+  const static int STATE_UPDATING   = 0x04;
+  /**
+   * Leader is about to propose a new value, but hasn't gotten to do it yet.
+   */
+  const static int STATE_PREPARING  = 0x08;
+
+  const static int STATE_LOCKED     = 0x10;
+
   /**
    * Obtain state name from constant value.
    *
@@ -115,12 +192,26 @@ public:
    * @return The state's name.
    */
   static 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;
+    stringstream ss;
+    if (s & STATE_RECOVERING) {
+      ss << "recovering";
+      assert(!(s & ~(STATE_RECOVERING|STATE_LOCKED)));
+    } else if (s & STATE_ACTIVE) {
+      ss << "active";
+      assert(s == STATE_ACTIVE);
+    } else if (s & STATE_UPDATING) {
+      ss << "updating";
+      assert(!(s & ~(STATE_UPDATING|STATE_LOCKED)));
+    } else if (s & STATE_PREPARING) {
+      ss << "preparing update";
+      assert(!(s & ~(STATE_PREPARING|STATE_LOCKED)));
+    } else {
+      assert(0 == "We shouldn't have gotten here!");
     }
+
+    if (s & STATE_LOCKED)
+      ss << " (locked)";
+    return ss.str().c_str();
   }
 
 private:
@@ -138,7 +229,7 @@ public:
    *
    * @return 'true' if we are on the Recovering state; 'false' otherwise.
    */
-  bool is_recovering() const { return state == STATE_RECOVERING; }
+  bool is_recovering() const { return (state & STATE_RECOVERING); }
   /**
    * Check if we are active.
    *
@@ -150,7 +241,10 @@ public:
    *
    * @return 'true' if we are on the Updating state; 'false' otherwise.
    */
-  bool is_updating() const { return state == STATE_UPDATING; }
+  bool is_updating() const { return (state & STATE_UPDATING); }
+
+  bool is_preparing() const { return (state & STATE_PREPARING); }
+  bool is_locked() const { return (state & STATE_LOCKED); }
 
 private:
   /**
@@ -260,11 +354,6 @@ private:
    * uncommitted values --, or if we're on a quorum of one.
    */
   list<Context*> waiting_for_readable;
-
-  /**
-   * Latest version written to the store after the latest commit.
-   */
-  version_t latest_stashed;
   /**
    * @}
    */
@@ -423,6 +512,10 @@ private:
    *         fully committed.
    */
   list<Context*> waiting_for_commit;
+  /**
+   *
+   */
+  list<Context*> proposals;
   /**
    * @}
    */
@@ -438,6 +531,7 @@ private:
    * @}
    */
 
+  bool going_to_bootstrap;
 
   /**
    * @defgroup Paxos_h_callbacks Callback classes.
@@ -512,10 +606,35 @@ private:
       paxos->lease_renew_timeout();
     }
   };
+
   /**
-   * @}
+   *
    */
+public:
+  class C_Proposal : public Context {
+    Context *proposer_context;
+  public:
+    bufferlist bl;
+    // for debug purposes. Will go away. Soon.
+    bool proposed;
+    utime_t proposal_time;
+
+    C_Proposal(Context *c, bufferlist& proposal_bl) :
+       proposer_context(c),
+       bl(proposal_bl),
+        proposed(false),
+       proposal_time(ceph_clock_now(NULL))
+      { }
 
+    void finish(int r) {
+      if (proposer_context)
+        proposer_context->finish(r);
+    }
+  };
+  /**
+   * @}
+   */
+private:
   /**
    * @defgroup Paxos_h_election_triggered Steps triggered by an election.
    *
@@ -841,15 +960,28 @@ private:
    */
   void warn_on_future_time(utime_t t, entity_name_t from);
 
+  /**
+   * Queue a new proposal by pushing it at the back of the queue; do not
+   * propose it.
+   *
+   * @param bl The bufferlist to be proposed
+   * @param onfinished The callback to be called once the proposal finishes
+   */
+  void queue_proposal(bufferlist& bl, Context *onfinished);
+  /**
+   * Begin proposing the Proposal at the front of the proposals queue.
+   */
+  void propose_queued();
+  void finish_proposal();
+
 public:
   /**
    * @param m A monitor
    * @param mid A machine id
    */
-  Paxos(Monitor *m,
-       int mid) : mon(m),
-                  machine_id(mid), 
-                  machine_name(get_paxos_name(mid)),
+  Paxos(Monitor *m, const string name) 
+                : mon(m),
+                  paxos_name(name),
                   state(STATE_RECOVERING),
                   first_committed(0),
                   last_pn(0),
@@ -857,7 +989,6 @@ public:
                   accepted_pn(0),
                   accepted_pn_from(0),
                   slurping(0),
-                  latest_stashed(0),
                   num_last(0),
                   uncommitted_v(0), uncommitted_pn(0),
                   collect_timeout_event(0),
@@ -865,12 +996,16 @@ public:
                   lease_ack_timeout_event(0),
                   lease_timeout_event(0),
                   accept_timeout_event(0),
-                  clock_drift_warned(0) { }
+                  clock_drift_warned(0),
+                  going_to_bootstrap(false) { }
 
-  const char *get_machine_name() const {
-    return machine_name;
+  const string get_name() const {
+    return paxos_name;
   }
 
+  bool is_bootstrapping() { return going_to_bootstrap; }
+  void prepare_bootstrap();
+
   void dispatch(PaxosServiceMessage *m);
 
   void init();
@@ -920,11 +1055,38 @@ public:
   void share_state(MMonPaxos *m, version_t peer_first_committed,
                   version_t peer_last_committed);
   /**
-   * Store the state held on the message m into local, stable storage.
+   * Store on disk a state that was shared with us
+   *
+   * Basically, we received a set of version. Or just one. It doesn't matter.
+   * What matters is that we have to stash it in the store. So, we will simply
+   * write every single bufferlist into their own versions on our side (i.e.,
+   * onto paxos-related keys), and then we will decode those same bufferlists
+   * we just wrote and apply the transactions they hold. We will also update
+   * our first and last committed values to point to the new values, if need
+   * be. All all this is done tightly wrapped in a transaction to ensure we
+   * enjoy the atomicity guarantees given by our awesome k/v store.
    *
    * @param m A message
    */
   void store_state(MMonPaxos *m);
+  /**
+   * Helper function to decode a bufferlist into a transaction and append it
+   * to another transaction.
+   *
+   * This function is used during the Leader's commit and during the
+   * Paxos::store_state in order to apply the bufferlist's transaction onto
+   * the store.
+   *
+   * @param t The transaction to which we will append the operations
+   * @param bl A bufferlist containing an encoded transaction
+   */
+  void decode_append_transaction(MonitorDBStore::Transaction& t,
+                                bufferlist& bl) {
+    MonitorDBStore::Transaction vt;
+    bufferlist::iterator it = bl.begin();
+    vt.decode(it);
+    t.append(vt);
+  }
 
   /**
    * @todo This appears to be used only by the OSDMonitor, and I would say
@@ -952,7 +1114,16 @@ public:
    * @param force If specified, we may even erase the latest stashed version
    *             iif @p first is higher than that version.
    */
-  void trim_to(version_t first, bool force=false);
+  void trim_to(version_t first, bool force = false);
+  /**
+   * Erase old states from stable storage.
+   *
+   * @param t A transaction
+   * @param first The version we are trimming to
+   * @param force If specified, we may even erase the latest stashed version
+   *             iif @p first is higher than that version.
+   */
+  void trim_to(MonitorDBStore::Transaction *t, version_t first, bool force=false);
  
   /**
    * @defgroup Paxos_h_slurping_funcs Slurping-related functions
@@ -1018,7 +1189,7 @@ public:
    * @param onreadable A callback
    */
   void wait_for_readable(Context *onreadable) {
-    //assert(!is_readable());
+    assert(!is_readable());
     waiting_for_readable.push_back(onreadable);
   }
   /**
@@ -1026,10 +1197,11 @@ public:
    */
 
   /**
-   * @warning This declaration is not implemented anywhere and appears to be
-   *         just some lingering code.
+   * Check if we have a valid lease.
+   *
+   * @returns true if the lease is still valid; false otherwise.
    */
-  bool is_leader();
+  bool is_lease_valid();
   // write
   /**
    * @defgroup Paxos_h_write_funcs Write-related functions
@@ -1057,6 +1229,13 @@ public:
     waiting_for_writeable.push_back(c);
   }
 
+  /**
+   * List all queued proposals
+   *
+   * @param out[out] Output Stream onto which we will output the list
+   *                of queued proposals.
+   */
+  void list_proposals(ostream& out);
   /**
    * Propose a new value to the Leader.
    *
@@ -1064,9 +1243,9 @@ public:
    * will trigger a new proposal.
    *
    * @param bl A bufferlist holding the value to be proposed
-   * @param oncommit A callback to be fired up once we finish committing bl
+   * @param onfinish A callback to be fired up once we finish the proposal
    */
-  bool propose_new_value(bufferlist& bl, Context *oncommit=0);
+  bool propose_new_value(bufferlist& bl, Context *onfinished=0);
   /**
    * Add oncommit to the back of the list of callbacks waiting for us to
    * finish committing.
@@ -1089,47 +1268,27 @@ public:
    * @}
    */
 
-  /**
-   * @defgroup Paxos_h_stash_funcs State values stashing-related functions
-   *
-   * If the state values are incrementals, it is useful to keep the latest
-   * copy of the complete structure.
-   *
-   * @{
-   */
-  /**
-   * Get the latest version onto stable storage.
-   *
-   * Keeping the latest version on a predefined location makes it easier to
-   * access, since we know we always have the latest version on the same
-   * place.
-   *
-   * @param v the latest version
-   * @param bl the latest version's value
-   */
-  void stash_latest(version_t v, bufferlist& bl);
-  /**
-   * Get the latest stashed version's value
-   *
-   * @param[out] bl the latest stashed version's value
-   * @return the latest stashed version
-   */
-  version_t get_stashed(bufferlist& bl);
-  /**
-   * Get the latest stashed version
-   *
-   * @return the latest stashed version
-   */
-  version_t get_stashed_version() { return latest_stashed; }
-  /**
-   * @}
-   */
-
   /**
    * @}
    */
+ protected:
+  MonitorDBStore *get_store();
 };
 
+inline ostream& operator<<(ostream& out, Paxos::C_Proposal& p)
+{
+  string proposed = (p.proposed ? "proposed" : "unproposed");
+  out << " " << proposed
+      << " queued " << (ceph_clock_now(NULL) - p.proposal_time)
+      << " tx dump:\n";
+  MonitorDBStore::Transaction t;
+  bufferlist::iterator p_it = p.bl.begin();
+  t.decode(p_it);
+  JSONFormatter f(true);
+  t.dump(&f);
+  f.flush(out);
+  return out;
+}
 
 #endif
 
index ef59327012d2985fd26e8f8f9048b6f26b20c3c6..e5c4f2129c48dc846841247f684222034687d565 100644 (file)
 #include "PaxosService.h"
 #include "common/Clock.h"
 #include "Monitor.h"
+#include "MonitorDBStore.h"
+
 
 #include "common/config.h"
 #include "include/assert.h"
+#include "common/Formatter.h"
 
 #define dout_subsys ceph_subsys_paxos
 #undef dout_prefix
-#define dout_prefix _prefix(_dout, mon, paxos, paxos->machine_id)
-static ostream& _prefix(std::ostream *_dout, Monitor *mon, Paxos *paxos, int machine_id) {
+#define dout_prefix _prefix(_dout, mon, paxos, service_name)
+static ostream& _prefix(std::ostream *_dout, Monitor *mon, Paxos *paxos, string service_name) {
   return *_dout << "mon." << mon->name << "@" << mon->rank
                << "(" << mon->get_state_name()
-               << ").paxosservice(" << get_paxos_name(machine_id) << ") ";
+               << ").paxosservice(" << service_name << ") ";
 }
 
-const char *PaxosService::get_machine_name()
-{
-  return paxos->get_machine_name();
-}
-
-
 bool PaxosService::dispatch(PaxosServiceMessage *m)
 {
   dout(10) << "dispatch " << *m << " from " << m->get_orig_source_inst() << dendl;
   // make sure our map is readable and up to date
-  if (!paxos->is_readable(m->version)) {
+  if (!is_readable(m->version)) {
     dout(10) << " waiting for paxos -> readable (v" << m->version << ")" << dendl;
-    paxos->wait_for_readable(new C_RetryMessage(this, m));
+    wait_for_readable(new C_RetryMessage(this, m), m->version);
     return true;
   }
 
@@ -58,9 +55,9 @@ bool PaxosService::dispatch(PaxosServiceMessage *m)
   }
   
   // writeable?
-  if (!paxos->is_writeable()) {
+  if (!is_writeable()) {
     dout(10) << " waiting for paxos -> writeable" << dendl;
-    paxos->wait_for_writeable(new C_RetryMessage(this, m));
+    wait_for_writeable(new C_RetryMessage(this, m));
     return true;
   }
 
@@ -90,7 +87,7 @@ bool PaxosService::dispatch(PaxosServiceMessage *m)
 bool PaxosService::should_propose(double& delay)
 {
   // simple default policy: quick startup, then some damping.
-  if (paxos->last_committed <= 1)
+  if (get_last_committed() <= 1)
     delay = 0.0;
   else {
     utime_t now = ceph_clock_now(g_ceph_context);
@@ -108,7 +105,10 @@ void PaxosService::propose_pending()
 {
   dout(10) << "propose_pending" << dendl;
   assert(have_pending);
-  assert(mon->is_leader() && paxos->is_active());
+  assert(mon->is_leader());
+  assert(is_active());
+  if (!is_active())
+    return;
 
   if (proposal_timer) {
     mon->timer.cancel_event(proposal_timer);
@@ -123,13 +123,23 @@ void PaxosService::propose_pending()
    *      to encode whatever is pending on the implementation class into a
    *      bufferlist, so we can then propose that as a value through Paxos.
    */
+  MonitorDBStore::Transaction t;
   bufferlist bl;
-  encode_pending(bl);
+  encode_pending(&t);
   have_pending = false;
 
+  JSONFormatter f(true);
+  t.dump(&f);
+  dout(30) << __func__ << " transaction dump:\n";
+  f.flush(*_dout);
+  *_dout << dendl;
+  
+
+  t.encode(bl);
+
   // apply to paxos
-  paxos->wait_for_commit_front(new C_Active(this));
-  paxos->propose_new_value(bl);
+  proposing.set(1);
+  paxos->propose_new_value(bl, new C_Committed(this));
 }
 
 
@@ -141,6 +151,8 @@ void PaxosService::restart()
     mon->timer.cancel_event(proposal_timer);
     proposal_timer = 0;
   }
+  // ignore any callbacks waiting for us to finish our proposal
+  waiting_for_finished_proposal.clear();
 
   on_restart();
 }
@@ -158,19 +170,23 @@ void PaxosService::election_finished()
     discard_pending();
     have_pending = false;
   }
+  proposing.set(0);
+
+  // ignore any callbacks waiting for us to finish our proposal
+  waiting_for_finished_proposal.clear();
 
   // make sure we update our state
-  if (paxos->is_active())
+  if (is_active())
     _active();
   else
-    paxos->wait_for_active(new C_Active(this));
+    wait_for_active(new C_Active(this));
 }
 
 void PaxosService::_active()
 {
-  if (!paxos->is_active()) {
+  if (!is_active()) {
     dout(10) << "_active - not active" << dendl;
-    paxos->wait_for_active(new C_Active(this));
+    wait_for_active(new C_Active(this));
     return;
   }
   dout(10) << "_active" << dendl;
@@ -179,35 +195,103 @@ void PaxosService::_active()
   update_from_paxos();
 
   // create pending state?
-  if (mon->is_leader() && paxos->is_active()) {
+  if (mon->is_leader() && is_active()) {
     dout(7) << "_active creating new pending" << dendl;
     if (!have_pending) {
       create_pending();
       have_pending = true;
     }
 
-    if (paxos->get_version() == 0) {
+    if (get_version() == 0) {
       // create initial state
       create_initial();
       propose_pending();
       return;
     }
+  } else {
+    if (!mon->is_leader()) {
+      dout(7) << __func__ << " we are not the leader, hence we propose nothing!" << dendl;
+    } else if (!is_active()) {
+      dout(7) << __func__ << " we are not active, hence we propose nothing!" << dendl;
+    }
   }
 
+  /* wake people up before calling on_active(). We don't know how long we'll be
+   * on the service's on_active(), and we really should wake people up!
+   */
+  wakeup_proposing_waiters();
   // NOTE: it's possible that this will get called twice if we commit
   // an old paxos value.  Implementations should be mindful of that.
-  if (paxos->is_active())
+  if (is_active())
     on_active();
 }
 
 
 void PaxosService::shutdown()
 {
-  paxos->cancel_events();
-  paxos->shutdown();
+  cancel_events();
 
   if (proposal_timer) {
     mon->timer.cancel_event(proposal_timer);
     proposal_timer = 0;
   }
+  // ignore any callbacks waiting for us to finish our proposal
+  waiting_for_finished_proposal.clear();
+}
+
+void PaxosService::put_version(MonitorDBStore::Transaction *t,
+                              const string& prefix, version_t ver,
+                              bufferlist& bl)
+{
+  ostringstream os;
+  os << ver;
+  string key = mon->store->combine_strings(prefix, os.str());
+  t->put(get_service_name(), key, bl);
+}
+
+int PaxosService::get_version(const string& prefix, version_t ver,
+                             bufferlist& bl)
+{
+  ostringstream os;
+  os << ver;
+  string key = mon->store->combine_strings(prefix, os.str());
+  return mon->store->get(get_service_name(), key, bl);
+}
+
+void PaxosService::wakeup_proposing_waiters()
+{
+  finish_contexts(g_ceph_context, waiting_for_finished_proposal);
+}
+
+void PaxosService::trim_to(version_t first, bool force)
+{
+  version_t first_committed = get_first_committed();
+  version_t latest_full = get_version("full", "latest");
+
+  string latest_key = mon->store->combine_strings("full", latest_full);
+  bool has_full = mon->store->exists(get_service_name(), latest_key);
+
+  dout(10) << __func__ << " " << first << " (was " << first_committed << ")"
+          << ", latest full " << latest_full << dendl;
+
+  if (first_committed >= first)
+    return;
+
+  MonitorDBStore::Transaction t;
+  while ((first_committed < first)
+      && (force || (first_committed < latest_full))) {
+    dout(20) << __func__ << first_committed << dendl;
+    t.erase(get_service_name(), first_committed);
+
+    if (has_full) {
+      latest_key = mon->store->combine_strings("full", first_committed);
+      if (mon->store->exists(get_service_name(), latest_key))
+       t.erase(get_service_name(), latest_key);
+    }
+
+    first_committed++;
+  }
+  put_first_committed(&t, first_committed);
+  mon->store->apply_transaction(t);
 }
+
index 78bcc8d367de6017e8d1fdd51905ebb2bc3b5ea3..4a787b539b9c13d91f069c2ed95ade6a086acf6d 100644 (file)
@@ -1,4 +1,4 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- 
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
 // vim: ts=8 sw=2 smarttab
 /*
  * Ceph - scalable distributed file system
@@ -18,6 +18,9 @@
 #include "messages/PaxosServiceMessage.h"
 #include "include/Context.h"
 #include <errno.h>
+#include "Paxos.h"
+#include "Monitor.h"
+#include "MonitorDBStore.h"
 
 class Monitor;
 class Paxos;
@@ -32,7 +35,7 @@ class PaxosService {
    * @defgroup PaxosService_h_class Paxos Service
    * @{
    */
-public:
+ public:
   /**
    * The Monitor to which this class is associated with
    */
@@ -41,7 +44,39 @@ public:
    * The Paxos instance to which this class is associated with
    */
   Paxos *paxos;
-  
+  /**
+   * Our name. This will be associated with the class implementing us, and will
+   * be used mainly for store-related operations.
+   */
+  string service_name;
+  /**
+   * If we are or have queued anything for proposal, this variable will be true
+   * until our proposal has been finished.
+   */
+  atomic_t proposing;
+
+ protected:
+  /**
+   * Services implementing us used to depend on the Paxos version, back when
+   * each service would have a Paxos instance for itself. However, now we only
+   * have a single Paxos instance, shared by all the services. Each service now
+   * must keep its own version, if so they wish. This variable should be used
+   * for that purpose.
+   */
+  version_t service_version;
+
+ private:
+  /**
+   * Event callback responsible for proposing our pending value once a timer 
+   * runs out and fires.
+   */
+  Context *proposal_timer;
+  /**
+   * If the implementation class has anything pending to be proposed to Paxos,
+   * then have_pending should be true; otherwise, false.
+   */
+  bool have_pending; 
+
 protected:
   /**
    * @defgroup PaxosService_h_callbacks Callback classes
@@ -85,7 +120,7 @@ protected:
   public:
     C_Active(PaxosService *s) : svc(s) {}
     void finish(int r) {
-      if (r >= 0) 
+      if (r >= 0)
        svc->_active();
     }
   };
@@ -105,40 +140,58 @@ protected:
       ps->propose_pending(); 
     }
   };
-  /**
-   * @}
-   */
-  friend class C_Propose;
-  
 
-private:
   /**
-   * Event callback responsible for proposing our pending value once a timer 
-   * runs out and fires.
+   * Callback class used to mark us as active once a proposal finishes going
+   * through Paxos.
+   *
+   * We should wake people up *only* *after* we inform the service we
+   * just went active. And we should wake people up only once we finish
+   * going active. This is why we first go active, avoiding to wake up the
+   * wrong people at the wrong time, such as waking up a C_RetryMessage
+   * before waking up a C_Active, thus ending up without a pending value.
    */
-  Context *proposal_timer;
+  class C_Committed : public Context {
+    PaxosService *ps;
+  public:
+    C_Committed(PaxosService *p) : ps(p) { }
+    void finish(int r) {
+      ps->proposing.set(0);
+      ps->_active();
+    }
+  };
   /**
-   * If the implementation class has anything pending to be proposed to Paxos,
-   * then have_pending should be true; otherwise, false.
+   * @}
    */
-  bool have_pending;
+  friend class C_Propose;
+  
 
 public:
   /**
    * @param mn A Monitor instance
    * @param p A Paxos instance
-   */
-  PaxosService(Monitor *mn, Paxos *p) : mon(mn), paxos(p),
-                                       proposal_timer(0),
-                                       have_pending(false) { }
+   * @parem name Our service's name.
+   */
+  PaxosService(Monitor *mn, Paxos *p, string name) 
+    : mon(mn), paxos(p), service_name(name),
+      service_version(0), proposal_timer(0), have_pending(false),
+      last_committed_name("last_committed"),
+      first_committed_name("first_committed"),
+      last_accepted_name("last_accepted"),
+      mkfs_name("mkfs"),
+      full_version_name("full"), full_latest_name("latest")
+  {
+    proposing.set(0);
+  }
+
   virtual ~PaxosService() {}
 
   /**
-   * Get the machine name.
+   * Get the service's name.
    *
-   * @returns The machine name.
+   * @returns The service's name.
    */
-  const char *get_machine_name();
+  string get_service_name() { return service_name; }
   
   // i implement and you ignore
   /**
@@ -248,9 +301,9 @@ public:
    *
    * @invariant This function is only called on a Leader.
    *
-   * @param[out] bl A bufferlist containing the encoded pending state
+   * @param t The transaction to hold all changes.
    */
-  virtual void encode_pending(bufferlist& bl) = 0;
+  virtual void encode_pending(MonitorDBStore::Transaction *t) = 0;
 
   /**
    * Discard the pending state
@@ -340,6 +393,443 @@ public:
   virtual void get_health(list<pair<health_status_t,string> >& summary,
                          list<pair<health_status_t,string> > *detail) const { }
 
+ private:
+  /**
+   * @defgroup PaxosService_h_store_keys Set of keys that are usually used on
+   *                                    all the services implementing this
+   *                                    class, and, being almost the only keys
+   *                                    used, should be standardized to avoid
+   *                                    mistakes.
+   * @{
+   */
+  const string last_committed_name;
+  const string first_committed_name;
+  const string last_accepted_name;
+  const string mkfs_name;
+  const string full_version_name;
+  const string full_latest_name;
+  /**
+   * @}
+   */
+
+  /**
+   * Callback list to be used whenever we are running a proposal through
+   * Paxos. These callbacks will be awaken whenever the said proposal
+   * finishes.
+   */
+  list<Context*> waiting_for_finished_proposal;
+
+ public:
+
+  /**
+   * Check if we are proposing a value through Paxos
+   *
+   * @returns true if we are proposing; false otherwise.
+   */
+  bool is_proposing() {
+    return ((int) proposing.read() == 1);
+  }
+
+  /**
+   * Check if we are in the Paxos ACTIVE state.
+   *
+   * @note This function is a wrapper for Paxos::is_active
+   *
+   * @returns true if in state ACTIVE; false otherwise.
+   */
+  bool is_active() {
+    return (!is_proposing() && !paxos->is_recovering()
+        && !paxos->is_locked()
+       && !paxos->is_bootstrapping());
+  }
+
+  /**
+   * Check if we are readable.
+   *
+   * We consider that a given version @p ver is readable if:
+   *
+   *  - it exists (i.e., is lower than the last committed version);
+   *  - we have at least one committed version (i.e., last committed version
+   *    is greater than zero);
+   *  - our monitor is a member of the cluster (either a peon or the leader);
+   *  - we are not proposing a new version;
+   *  - the Paxos is not recovering;
+   *  - we either belong to a quorum and have a valid lease, or we belong to
+   *    a quorum of one.
+   *
+   * @param ver The version we want to check if is readable
+   * @returns true if it is readable; false otherwise
+   */
+  bool is_readable(version_t ver = 0) {
+    if ((ver > get_last_committed())
+       || ((!mon->is_peon() && !mon->is_leader()))
+       || (is_proposing() || paxos->is_recovering() || paxos->is_locked())
+       || (get_last_committed() <= 0)
+       || ((mon->get_quorum().size() != 1) && !paxos->is_lease_valid())) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Check if we are writeable.
+   *
+   * We consider to be writeable iff:
+   *
+   *  - we are not proposing a new version;
+   *  - our monitor is the leader;
+   *  - we have a valid lease;
+   *  - Paxos is not boostrapping.
+   *
+   * @returns true if writeable; false otherwise
+   */
+  bool is_writeable() {
+    return (!is_proposing() && mon->is_leader()
+        && !paxos->is_locked()
+       && paxos->is_lease_valid() && !paxos->is_bootstrapping());
+  }
+
+  /**
+   * Wait for a proposal to finish.
+   *
+   * Add a callback to be awaken whenever our current proposal finishes being
+   * proposed through Paxos.
+   *
+   * @param c The callback to be awaken once the proposal is finished.
+   */
+  void wait_for_finished_proposal(Context *c) {
+    waiting_for_finished_proposal.push_back(c);
+  }
+
+  /**
+   * Wait for us to become active
+   *
+   * @param c The callback to be awaken once we become active.
+   */
+  void wait_for_active(Context *c) {
+    if (paxos->is_bootstrapping() || !is_proposing()) {
+      paxos->wait_for_active(c);
+      return;
+    }
+    wait_for_finished_proposal(c);
+  }
+
+  /**
+   * Wait for us to become readable
+   *
+   * @param c The callback to be awaken once we become active.
+   * @param ver The version we want to wait on.
+   */
+  void wait_for_readable(Context *c, version_t ver = 0) {
+    /* This is somewhat of a hack. We only do check if a version is readable on
+     * PaxosService::dispatch(), but, nonetheless, we must make sure that if that
+     * is why we are not readable, then we must wait on PaxosService and not on
+     * Paxos; otherwise, we may assert on Paxos::wait_for_readable() if it
+     * happens to be readable at that specific point in time.
+     */
+    if (is_proposing() || (ver > get_last_committed())
+       || (get_last_committed() <= 0))
+      wait_for_finished_proposal(c);
+    else
+      paxos->wait_for_readable(c);
+  }
+
+  /**
+   * Wait for us to become writeable
+   *
+   * @param c The callback to be awaken once we become writeable.
+   */
+  void wait_for_writeable(Context *c) {
+    if (paxos->is_bootstrapping() || !is_proposing()) {
+      paxos->wait_for_writeable(c);
+      return;
+    }
+
+    wait_for_finished_proposal(c);
+  }
+
+  /**
+   * Wakeup all the callbacks waiting for the proposal to be finished
+   */
+  void wakeup_proposing_waiters();
+
+  /**
+   * Trim our log. This implies getting rid of versions on the k/v store.
+   * Services implementing us don't have to implement this function if they
+   * don't want to, but we won't implement it for them either.
+   *
+   * This function had to be inheritted from the Paxos, since the existing
+   * services made use of it. This function should be tuned for each service's
+   * needs. We have it in this interface to make sure its usage and purpose is
+   * well understood by the underlying services.
+   *
+   * @param first The version that should become the first one in the log.
+   * @param force Optional. Each service may use it as it sees fit, but the
+   *             expected behavior is that, when 'true', we will remove all
+   *             the log versions even if we don't have a full map in store.
+   */
+  void trim_to(version_t first, bool force = false);
+
+
+  /**
+   * Cancel events.
+   *
+   * @note This function is a wrapper for Paxos::cancel_events
+   */
+  void cancel_events() {
+    paxos->cancel_events();
+  }
+
+  /**
+   * @defgroup PaxosService_h_store_funcs Back storage interface functions
+   * @{
+   */
+  /**
+   * @defgroup PaxosService_h_store_modify Wrapper function interface to access
+   *                                      the back store for modification
+   *                                      purposes
+   * @{
+   */
+  void put_first_committed(MonitorDBStore::Transaction *t, version_t ver) {
+    t->put(get_service_name(), first_committed_name, ver);
+  }
+  /**
+   * Set the last committed version to @p ver
+   *
+   * @param t A transaction to which we add this put operation
+   * @param ver The last committed version number being put
+   */
+  void put_last_committed(MonitorDBStore::Transaction *t, version_t ver) {
+    t->put(get_service_name(), last_committed_name, ver);
+
+    /* We only need to do this once, and that is when we are about to make our
+     * first proposal. There are some services that rely on first_committed
+     * being set -- and it should! -- so we need to guarantee that it is,
+     * specially because the services itself do not do it themselves. They do
+     * rely on it, but they expect us to deal with it, and so we shall.
+     */
+    if (!get_first_committed())
+      put_first_committed(t, ver);
+  }
+  /**
+   * Put the contents of @p bl into version @p ver
+   *
+   * @param t A transaction to which we will add this put operation
+   * @param ver The version to which we will add the value
+   * @param bl A bufferlist containing the version's value
+   */
+  void put_version(MonitorDBStore::Transaction *t, version_t ver,
+                  bufferlist& bl) {
+    t->put(get_service_name(), ver, bl);
+  }
+  /**
+   * Put the contents of @p bl into version @p ver (prefixed with @p prefix)
+   *
+   * @param t A transaction to which we will add this put operation
+   * @param prefix The version's prefix
+   * @param ver The version to which we will add the value
+   * @param bl A bufferlist containing the version's value
+   */
+  void put_version(MonitorDBStore::Transaction *t, 
+                  const string& prefix, version_t ver, bufferlist& bl);
+  /**
+   * Put a version number into a key composed by @p prefix and @p name
+   * combined.
+   *
+   * @param t The transaction to which we will add this put operation
+   * @param prefix The key's prefix
+   * @param name The key's suffix
+   * @param ver A version number
+   */
+  void put_version(MonitorDBStore::Transaction *t,
+                  const string& prefix, const string& name, version_t ver) {
+    string key = mon->store->combine_strings(prefix, name);
+    t->put(get_service_name(), key, ver);
+  }
+  /**
+   * Put the contents of @p bl into a full version key for this service, that
+   * will be created with @p ver in mind.
+   *
+   * @param t The transaction to which we will add this put operation
+   * @param ver A version number
+   * @param bl A bufferlist containing the version's value
+   */
+  void put_version_full(MonitorDBStore::Transaction *t,
+                       version_t ver, bufferlist& bl) {
+    string key = mon->store->combine_strings(full_version_name, ver);
+    t->put(get_service_name(), key, bl);
+  }
+  /**
+   * Put the version number in @p ver into the key pointing to the latest full
+   * version of this service.
+   *
+   * @param t The transaction to which we will add this put operation
+   * @param ver A version number
+   */
+  void put_version_latest_full(MonitorDBStore::Transaction *t, version_t ver) {
+    string key =
+      mon->store->combine_strings(full_version_name, full_latest_name);
+    t->put(get_service_name(), key, ver);
+  }
+  /**
+   * Put the contents of @p bl into the key @p key.
+   *
+   * @param t A transaction to which we will add this put operation
+   * @param key The key to which we will add the value
+   * @param bl A bufferlist containing the value
+   */
+  void put_value(MonitorDBStore::Transaction *t,
+                const string& key, bufferlist& bl) {
+    t->put(get_service_name(), key, bl);
+  }
+  /**
+   * Put the contents of @p bl into a key composed of @p prefix and @p name
+   * concatenated.
+   *
+   * @param t A transaction to which we will add this put operation
+   * @param prefix The key's prefix
+   * @param name The key's suffix
+   * @param bl A bufferlist containing the value
+   */
+  void put_value(MonitorDBStore::Transaction *t,
+                const string& prefix, const string& name, bufferlist& bl) {
+    string key = mon->store->combine_strings(prefix, name);
+    t->put(get_service_name(), key, bl);
+  }
+  /**
+   * Remove our mkfs entry from the store
+   *
+   * @param t A transaction to which we will add this erase operation
+   */
+  void erase_mkfs(MonitorDBStore::Transaction *t) {
+    t->erase(mkfs_name, get_service_name());
+  }
+  /**
+   * @}
+   */
+
+  /**
+   * @defgroup PaxosService_h_store_get Wrapper function interface to access
+   *                                   the back store for reading purposes
+   * @{
+   */
+  /**
+   * Get the first committed version
+   *
+   * @returns Our first committed version (that is available)
+   */
+  version_t get_first_committed() {
+    return mon->store->get(get_service_name(), first_committed_name);
+  }
+  /**
+   * Get the last committed version
+   *
+   * @returns Our last committed version
+   */
+  version_t get_last_committed() {
+    return mon->store->get(get_service_name(), last_committed_name);
+  }
+  /**
+   * Get our current version
+   *
+   * @returns Our current version
+   */
+  version_t get_version() {
+    return get_last_committed();
+  }
+  /**
+   * Get the contents of a given version @p ver
+   *
+   * @param ver The version being obtained
+   * @param bl The bufferlist to be populated
+   * @return 0 on success; <0 otherwise
+   */
+  int get_version(version_t ver, bufferlist& bl) {
+    return mon->store->get(get_service_name(), ver, bl);
+  }
+  /**
+   * Get the contents of a given version @p ver with a given prefix @p prefix
+   *
+   * @param prefix The intended prefix
+   * @param ver The version being obtained
+   * @param bl The bufferlist to be populated
+   * @return 0 on success; <0 otherwise
+   */
+  int get_version(const string& prefix, version_t ver, bufferlist& bl);
+  /**
+   * Get a version number from a given key, whose name is composed by
+   * @p prefix and @p name combined.
+   *
+   * @param prefix Key's prefix
+   * @param name Key's suffix
+   * @returns A version number
+   */
+  version_t get_version(const string& prefix, const string& name) {
+    string key = mon->store->combine_strings(prefix, name);
+    return mon->store->get(get_service_name(), key);
+  }
+  /**
+   * Get the contents of a given full version of this service.
+   *
+   * @param ver A version number
+   * @param bl The bufferlist to be populated
+   * @returns 0 on success; <0 otherwise
+   */
+  int get_version_full(version_t ver, bufferlist& bl) {
+    string key = mon->store->combine_strings(full_version_name, ver);
+    return mon->store->get(get_service_name(), key, bl);
+  }
+  /**
+   * Get the latest full version number
+   *
+   * @returns A version number
+   */
+  version_t get_version_latest_full() {
+    return get_version(full_version_name, full_latest_name);
+  }
+  /**
+   * Get a value from a given key, composed by @p prefix and @p name combined.
+   *
+   * @param[in] prefix Key's prefix
+   * @param[in] name Key's suffix
+   * @param[out] bl The bufferlist to be populated with the value
+   */
+  int get_value(const string& prefix, const string& name, bufferlist& bl) {
+    string key = mon->store->combine_strings(prefix, name);
+    return mon->store->get(get_service_name(), key, bl);
+  }
+  /**
+   * Get a value from a given key.
+   *
+   * @param[in] key The key
+   * @param[out] bl The bufferlist to be populated with the value
+   */
+  int get_value(const string& key, bufferlist& bl) {
+    return mon->store->get(get_service_name(), key, bl);
+  }
+  /**
+   * Get the contents of our mkfs entry
+   *
+   * @param bl A bufferlist to populate with the contents of the entry
+   * @return 0 on success; <0 otherwise
+   */
+  int get_mkfs(bufferlist& bl) {
+    return mon->store->get(mkfs_name, get_service_name(), bl);
+  }
+  /**
+   * Checks if a given key composed by @p prefix and @p name exists.
+   *
+   * @param prefix Key's prefix
+   * @param name Key's suffix
+   * @returns true if it exists; false otherwise.
+   */
+  bool exists_key(const string& prefix, const string& name) {
+    string key = mon->store->combine_strings(prefix, name);
+    return mon->store->exists(get_service_name(), key);
+  }
+  /**
+   * @}
+   */
   /**
    * @}
    */